From 4c4b24a780444dac5ef5aa5e3e5a149540407735 Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Mon, 27 Oct 2025 09:44:28 -0500 Subject: [PATCH 01/11] add BloomFilterExpr --- Cargo.lock | 1 + datafusion/physical-expr/Cargo.toml | 1 + datafusion/physical-expr/src/bloom_filter.rs | 342 ++++++++++ .../src/expressions/bloom_filter_expr.rs | 619 ++++++++++++++++++ .../physical-expr/src/expressions/mod.rs | 2 + datafusion/physical-expr/src/lib.rs | 1 + 6 files changed, 966 insertions(+) create mode 100644 datafusion/physical-expr/src/bloom_filter.rs create mode 100644 datafusion/physical-expr/src/expressions/bloom_filter_expr.rs diff --git a/Cargo.lock b/Cargo.lock index e368dcf9a91e..f68fcb1a71b7 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2419,6 +2419,7 @@ dependencies = [ "petgraph 0.8.3", "rand 0.9.2", "rstest", + "twox-hash", ] [[package]] diff --git a/datafusion/physical-expr/Cargo.toml b/datafusion/physical-expr/Cargo.toml index b7654a0f6f60..a5c2a1b08a32 100644 --- a/datafusion/physical-expr/Cargo.toml +++ b/datafusion/physical-expr/Cargo.toml @@ -52,6 +52,7 @@ itertools = { workspace = true, features = ["use_std"] } parking_lot = { workspace = true } paste = "^1.0" petgraph = "0.8.3" +twox-hash = { version = "2.0", default-features = false, features = ["xxhash64"] } [dev-dependencies] arrow = { workspace = true, features = ["test_utils"] } diff --git a/datafusion/physical-expr/src/bloom_filter.rs b/datafusion/physical-expr/src/bloom_filter.rs new file mode 100644 index 000000000000..613959cde6eb --- /dev/null +++ b/datafusion/physical-expr/src/bloom_filter.rs @@ -0,0 +1,342 @@ +// 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. + +//! Bloom filter implementation for physical expressions +//! +//! This module contains a vendored copy of the Split Block Bloom Filter (SBBF) +//! implementation from the parquet crate. This avoids circular dependencies +//! while allowing physical expressions to use bloom filters for runtime pruning. +//! +//! TODO: If this bloom filter approach is successful, extract this into a shared +//! crate (e.g., `datafusion-bloom-filter`) that both parquet and physical-expr +//! can depend on. +//! +//! The implementation below is adapted from: +//! arrow-rs/parquet/src/bloom_filter/mod.rs + +use datafusion_common::{internal_err, Result}; +use std::mem::size_of; +use twox_hash::XxHash64; + +/// Salt values as defined in the Parquet specification +/// Although we don't *need* to follow the Parquet spec here, using the same +/// constants allows us to be compatible with Parquet bloom filters in the future +/// e.g. to do binary intersection of bloom filters. +const SALT: [u32; 8] = [ + 0x47b6137b_u32, + 0x44974d91_u32, + 0x8824ad5b_u32, + 0xa2b7289d_u32, + 0x705495c7_u32, + 0x2df1424b_u32, + 0x9efc4947_u32, + 0x5c6bfb31_u32, +]; + +/// Minimum bitset length in bytes +const BITSET_MIN_LENGTH: usize = 32; +/// Maximum bitset length in bytes +const BITSET_MAX_LENGTH: usize = 128 * 1024 * 1024; + +/// Hash seed for xxHash +const SEED: u64 = 0; + +/// Each block is 256 bits, broken up into eight contiguous "words", each consisting of 32 bits. +/// Each word is thought of as an array of bits; each bit is either "set" or "not set". +#[derive(Debug, Copy, Clone)] +#[repr(transparent)] +struct Block([u32; 8]); + +impl Block { + const ZERO: Block = Block([0; 8]); + + /// Takes as its argument a single unsigned 32-bit integer and returns a block in which each + /// word has exactly one bit set. + fn mask(x: u32) -> Self { + let mut result = [0_u32; 8]; + for i in 0..8 { + // wrapping instead of checking for overflow + let y = x.wrapping_mul(SALT[i]); + let y = y >> 27; + result[i] = 1 << y; + } + Self(result) + } + + /// Setting every bit in the block that was also set in the result from mask + fn insert(&mut self, hash: u32) { + let mask = Self::mask(hash); + for i in 0..8 { + self[i] |= mask[i]; + } + } + + /// Returns true when every bit that is set in the result of mask is also set in the block. + fn check(&self, hash: u32) -> bool { + let mask = Self::mask(hash); + for i in 0..8 { + if self[i] & mask[i] == 0 { + return false; + } + } + true + } +} + +impl std::ops::Index for Block { + type Output = u32; + + #[inline] + fn index(&self, index: usize) -> &Self::Output { + self.0.index(index) + } +} + +impl std::ops::IndexMut for Block { + #[inline] + fn index_mut(&mut self, index: usize) -> &mut Self::Output { + self.0.index_mut(index) + } +} + +/// A Split Block Bloom Filter (SBBF) +/// +/// This is a space-efficient probabilistic data structure used to test whether +/// an element is a member of a set. False positive matches are possible, but +/// false negatives are not. +#[derive(Debug, Clone)] +pub struct Sbbf(Vec); + +impl Sbbf { + /// Create a new Sbbf with given number of distinct values and false positive probability. + /// Will return an error if `fpp` is greater than or equal to 1.0 or less than 0.0. + pub fn new_with_ndv_fpp(ndv: u64, fpp: f64) -> Result { + if !(0.0..1.0).contains(&fpp) { + return internal_err!( + "False positive probability must be between 0.0 and 1.0, got {fpp}" + ); + } + let num_bits = num_of_bits_from_ndv_fpp(ndv, fpp); + Ok(Self::new_with_num_of_bytes(num_bits / 8)) + } + + /// Create a new Sbbf with given number of bytes, the exact number of bytes will be adjusted + /// to the next power of two bounded by BITSET_MIN_LENGTH and BITSET_MAX_LENGTH. + fn new_with_num_of_bytes(num_bytes: usize) -> Self { + let num_bytes = optimal_num_of_bytes(num_bytes); + assert_eq!(num_bytes % size_of::(), 0); + let num_blocks = num_bytes / size_of::(); + let bitset = vec![Block::ZERO; num_blocks]; + Self(bitset) + } + + #[inline] + fn hash_to_block_index(&self, hash: u64) -> usize { + // unchecked_mul is unstable, but in reality this is safe, we'd just use saturating mul + // but it will not saturate + (((hash >> 32).saturating_mul(self.0.len() as u64)) >> 32) as usize + } + + /// Insert a value into the filter + pub fn insert(&mut self, value: &T) { + self.insert_hash(hash_as_bytes(value)); + } + + /// Insert a hash into the filter + fn insert_hash(&mut self, hash: u64) { + let block_index = self.hash_to_block_index(hash); + self.0[block_index].insert(hash as u32) + } + + /// Check if a value is probably present or definitely absent in the filter + pub fn check(&self, value: &T) -> bool { + self.check_hash(hash_as_bytes(value)) + } + + /// Check if a hash is in the filter. May return + /// true for values that were never inserted ("false positive") + /// but will always return false if a hash has not been inserted. + fn check_hash(&self, hash: u64) -> bool { + let block_index = self.hash_to_block_index(hash); + self.0[block_index].check(hash as u32) + } +} + +/// Trait for types that can be converted to bytes for hashing +pub trait AsBytes { + /// Return a byte slice representation of this value + fn as_bytes(&self) -> &[u8]; +} + +impl AsBytes for str { + fn as_bytes(&self) -> &[u8] { + str::as_bytes(self) + } +} + +impl AsBytes for [u8] { + fn as_bytes(&self) -> &[u8] { + self + } +} + +impl AsBytes for bool { + fn as_bytes(&self) -> &[u8] { + if *self { + &[1u8] + } else { + &[0u8] + } + } +} + +impl AsBytes for i32 { + fn as_bytes(&self) -> &[u8] { + unsafe { + std::slice::from_raw_parts(self as *const i32 as *const u8, size_of::()) + } + } +} + +impl AsBytes for i64 { + fn as_bytes(&self) -> &[u8] { + unsafe { + std::slice::from_raw_parts(self as *const i64 as *const u8, size_of::()) + } + } +} + +impl AsBytes for u32 { + fn as_bytes(&self) -> &[u8] { + unsafe { + std::slice::from_raw_parts(self as *const u32 as *const u8, size_of::()) + } + } +} + +impl AsBytes for u64 { + fn as_bytes(&self) -> &[u8] { + unsafe { + std::slice::from_raw_parts(self as *const u64 as *const u8, size_of::()) + } + } +} + +impl AsBytes for f32 { + fn as_bytes(&self) -> &[u8] { + unsafe { + std::slice::from_raw_parts(self as *const f32 as *const u8, size_of::()) + } + } +} + +impl AsBytes for f64 { + fn as_bytes(&self) -> &[u8] { + unsafe { + std::slice::from_raw_parts(self as *const f64 as *const u8, size_of::()) + } + } +} + +/// Hash a value using xxHash64 with seed 0 +#[inline] +fn hash_as_bytes(value: &A) -> u64 { + XxHash64::oneshot(SEED, value.as_bytes()) +} + +/// Calculate optimal number of bytes, bounded by min/max and rounded to power of 2 +#[inline] +fn optimal_num_of_bytes(num_bytes: usize) -> usize { + let num_bytes = num_bytes.min(BITSET_MAX_LENGTH); + let num_bytes = num_bytes.max(BITSET_MIN_LENGTH); + num_bytes.next_power_of_two() +} + +/// Calculate number of bits needed given NDV and FPP +/// Formula: m = -k * n / ln(1 - f^(1/k)) +/// where k=8 (number of hash functions), n=ndv, f=fpp, m=num_bits +#[inline] +fn num_of_bits_from_ndv_fpp(ndv: u64, fpp: f64) -> usize { + let num_bits = -8.0 * ndv as f64 / (1.0 - fpp.powf(1.0 / 8.0)).ln(); + num_bits as usize +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_hash_bytes() { + assert_eq!(hash_as_bytes(""), 17241709254077376921); + } + + #[test] + fn test_mask_set_quick_check() { + for i in 0..1_000 { + let result = Block::mask(i); + assert!(result.0.iter().all(|&x| x.is_power_of_two())); + } + } + + #[test] + fn test_block_insert_and_check() { + for i in 0..1_000 { + let mut block = Block::ZERO; + block.insert(i); + assert!(block.check(i)); + } + } + + #[test] + fn test_sbbf_insert_and_check() { + let mut sbbf = Sbbf(vec![Block::ZERO; 1_000]); + for i in 0..10_000 { + sbbf.insert(&i); + assert!(sbbf.check(&i)); + } + } + + #[test] + fn test_optimal_num_of_bytes() { + for (input, expected) in &[ + (0, 32), + (9, 32), + (31, 32), + (32, 32), + (33, 64), + (99, 128), + (1024, 1024), + (999_000_000, 128 * 1024 * 1024), + ] { + assert_eq!(*expected, optimal_num_of_bytes(*input)); + } + } + + #[test] + fn test_num_of_bits_from_ndv_fpp() { + for (fpp, ndv, num_bits) in &[ + (0.1, 10, 57), + (0.01, 10, 96), + (0.001, 10, 146), + (0.1, 100, 577), + (0.01, 100, 968), + (0.001, 100, 1460), + ] { + assert_eq!(*num_bits, num_of_bits_from_ndv_fpp(*ndv, *fpp) as u64); + } + } +} diff --git a/datafusion/physical-expr/src/expressions/bloom_filter_expr.rs b/datafusion/physical-expr/src/expressions/bloom_filter_expr.rs new file mode 100644 index 000000000000..233b6c225d24 --- /dev/null +++ b/datafusion/physical-expr/src/expressions/bloom_filter_expr.rs @@ -0,0 +1,619 @@ +// 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. + +//! Bloom filter physical expression + +use crate::bloom_filter::Sbbf; +use crate::PhysicalExpr; +use arrow::array::{ArrayRef, BooleanArray}; +use arrow::datatypes::{DataType, Schema}; +use arrow::record_batch::RecordBatch; +use datafusion_common::{exec_err, internal_err, Result, ScalarValue}; +use datafusion_expr_common::columnar_value::ColumnarValue; +use std::any::Any; +use std::fmt; +use std::hash::{Hash, Hasher}; +use std::sync::Arc; + +/// A progressive builder for creating bloom filters +/// +/// This builder allows incremental insertion of values from record batches +/// and produces a static `BloomFilterExpr` when finished. +/// +/// # Example +/// ```ignore +/// let mut builder = BloomFilterBuilder::new(1000, 0.01)?; +/// builder.insert_scalar(&ScalarValue::Int32(Some(42)))?; +/// builder.insert_array(&int_array)?; +/// let expr = builder.finish(col_expr)?; +/// ``` +pub struct BloomFilterBuilder { + /// The underlying bloom filter + sbbf: Sbbf, +} + +impl BloomFilterBuilder { + /// Create a new bloom filter builder + /// + /// # Arguments + /// * `ndv` - Expected number of distinct values + /// * `fpp` - Desired false positive probability (0.0 to 1.0) + pub fn new(ndv: u64, fpp: f64) -> Result { + let sbbf = Sbbf::new_with_ndv_fpp(ndv, fpp)?; + Ok(Self { sbbf }) + } + + /// Insert a single scalar value into the bloom filter + pub fn insert_scalar(&mut self, value: &ScalarValue) -> Result<()> { + if value.is_null() { + return Ok(()); + } + + match value { + ScalarValue::Boolean(Some(v)) => self.sbbf.insert(v), + ScalarValue::Int32(Some(v)) => self.sbbf.insert(v), + ScalarValue::Int64(Some(v)) => self.sbbf.insert(v), + ScalarValue::UInt32(Some(v)) => self.sbbf.insert(v), + ScalarValue::UInt64(Some(v)) => self.sbbf.insert(v), + ScalarValue::Float32(Some(v)) => self.sbbf.insert(v), + ScalarValue::Float64(Some(v)) => self.sbbf.insert(v), + ScalarValue::Utf8(Some(v)) | ScalarValue::LargeUtf8(Some(v)) => { + self.sbbf.insert(v.as_str()) + } + ScalarValue::Utf8View(Some(v)) => self.sbbf.insert(v.as_str()), + ScalarValue::Binary(Some(v)) + | ScalarValue::LargeBinary(Some(v)) + | ScalarValue::FixedSizeBinary(_, Some(v)) => self.sbbf.insert(v.as_slice()), + ScalarValue::BinaryView(Some(v)) => self.sbbf.insert(v.as_slice()), + _ => { + return exec_err!( + "Unsupported data type for bloom filter: {}", + value.data_type() + ) + } + } + Ok(()) + } + + /// Insert all non-null values from an array into the bloom filter + pub fn insert_array(&mut self, array: &ArrayRef) -> Result<()> { + use arrow::array::*; + use arrow::datatypes::DataType; + + match array.data_type() { + DataType::Boolean => { + let array = array.as_any().downcast_ref::().unwrap(); + for i in 0..array.len() { + if !array.is_null(i) { + self.sbbf.insert(&array.value(i)); + } + } + } + DataType::Int32 => { + let array = array.as_any().downcast_ref::().unwrap(); + for i in 0..array.len() { + if !array.is_null(i) { + self.sbbf.insert(&array.value(i)); + } + } + } + DataType::Int64 => { + let array = array.as_any().downcast_ref::().unwrap(); + for i in 0..array.len() { + if !array.is_null(i) { + self.sbbf.insert(&array.value(i)); + } + } + } + DataType::UInt32 => { + let array = array.as_any().downcast_ref::().unwrap(); + for i in 0..array.len() { + if !array.is_null(i) { + self.sbbf.insert(&array.value(i)); + } + } + } + DataType::UInt64 => { + let array = array.as_any().downcast_ref::().unwrap(); + for i in 0..array.len() { + if !array.is_null(i) { + self.sbbf.insert(&array.value(i)); + } + } + } + DataType::Float32 => { + let array = array.as_any().downcast_ref::().unwrap(); + for i in 0..array.len() { + if !array.is_null(i) { + self.sbbf.insert(&array.value(i)); + } + } + } + DataType::Float64 => { + let array = array.as_any().downcast_ref::().unwrap(); + for i in 0..array.len() { + if !array.is_null(i) { + self.sbbf.insert(&array.value(i)); + } + } + } + DataType::Utf8 => { + let array = array.as_any().downcast_ref::().unwrap(); + for i in 0..array.len() { + if !array.is_null(i) { + self.sbbf.insert(array.value(i)); + } + } + } + DataType::LargeUtf8 => { + let array = array.as_any().downcast_ref::().unwrap(); + for i in 0..array.len() { + if !array.is_null(i) { + self.sbbf.insert(array.value(i)); + } + } + } + DataType::Utf8View => { + let array = array.as_any().downcast_ref::().unwrap(); + for i in 0..array.len() { + if !array.is_null(i) { + self.sbbf.insert(array.value(i)); + } + } + } + DataType::Binary => { + let array = array.as_any().downcast_ref::().unwrap(); + for i in 0..array.len() { + if !array.is_null(i) { + self.sbbf.insert(array.value(i)); + } + } + } + DataType::LargeBinary => { + let array = array.as_any().downcast_ref::().unwrap(); + for i in 0..array.len() { + if !array.is_null(i) { + self.sbbf.insert(array.value(i)); + } + } + } + DataType::BinaryView => { + let array = array.as_any().downcast_ref::().unwrap(); + for i in 0..array.len() { + if !array.is_null(i) { + self.sbbf.insert(array.value(i)); + } + } + } + DataType::FixedSizeBinary(_) => { + let array = array + .as_any() + .downcast_ref::() + .unwrap(); + for i in 0..array.len() { + if !array.is_null(i) { + self.sbbf.insert(array.value(i)); + } + } + } + _ => { + return exec_err!( + "Unsupported data type for bloom filter: {}", + array.data_type() + ) + } + } + Ok(()) + } + + /// Finish building and create a `BloomFilterExpr` + /// + /// This consumes the builder and returns a static PhysicalExpr + /// that checks values against the constructed bloom filter. + /// + /// # Arguments + /// * `expr` - The expression to evaluate and check against the bloom filter + pub fn finish(self, expr: Arc) -> BloomFilterExpr { + BloomFilterExpr::new(expr, self.sbbf) + } +} + +/// Physical expression that checks values against a bloom filter +/// +/// This is a static expression (similar to `InListExpr`) that evaluates +/// a child expression and checks each value against a pre-built bloom filter. +/// Returns a boolean array indicating whether each value might be present +/// (true) or is definitely absent (false). +/// +/// Note: Bloom filters can produce false positives but never false negatives. +#[derive(Debug, Clone)] +pub struct BloomFilterExpr { + /// The expression to evaluate + expr: Arc, + /// The bloom filter to check against + bloom_filter: Arc, +} + +impl BloomFilterExpr { + /// Create a new bloom filter expression + pub fn new(expr: Arc, bloom_filter: Sbbf) -> Self { + Self { + expr, + bloom_filter: Arc::new(bloom_filter), + } + } + + /// Check a scalar value against the bloom filter + fn check_scalar(&self, value: &ScalarValue) -> bool { + if value.is_null() { + return false; + } + + match value { + ScalarValue::Boolean(Some(v)) => self.bloom_filter.check(v), + ScalarValue::Int32(Some(v)) => self.bloom_filter.check(v), + ScalarValue::Int64(Some(v)) => self.bloom_filter.check(v), + ScalarValue::UInt32(Some(v)) => self.bloom_filter.check(v), + ScalarValue::UInt64(Some(v)) => self.bloom_filter.check(v), + ScalarValue::Float32(Some(v)) => self.bloom_filter.check(v), + ScalarValue::Float64(Some(v)) => self.bloom_filter.check(v), + ScalarValue::Utf8(Some(v)) | ScalarValue::LargeUtf8(Some(v)) => { + self.bloom_filter.check(v.as_str()) + } + ScalarValue::Utf8View(Some(v)) => self.bloom_filter.check(v.as_str()), + ScalarValue::Binary(Some(v)) + | ScalarValue::LargeBinary(Some(v)) + | ScalarValue::FixedSizeBinary(_, Some(v)) => { + self.bloom_filter.check(v.as_slice()) + } + ScalarValue::BinaryView(Some(v)) => self.bloom_filter.check(v.as_slice()), + _ => true, // Unsupported types default to "might be present" + } + } + + /// Check an array against the bloom filter + fn check_array(&self, array: &ArrayRef) -> Result { + use arrow::array::*; + use arrow::datatypes::DataType; + + let len = array.len(); + let mut builder = BooleanArray::builder(len); + + match array.data_type() { + DataType::Boolean => { + let array = array.as_any().downcast_ref::().unwrap(); + for i in 0..len { + if array.is_null(i) { + builder.append_value(false); + } else { + builder.append_value(self.bloom_filter.check(&array.value(i))); + } + } + } + DataType::Int32 => { + let array = array.as_any().downcast_ref::().unwrap(); + for i in 0..len { + if array.is_null(i) { + builder.append_value(false); + } else { + builder.append_value(self.bloom_filter.check(&array.value(i))); + } + } + } + DataType::Int64 => { + let array = array.as_any().downcast_ref::().unwrap(); + for i in 0..len { + if array.is_null(i) { + builder.append_value(false); + } else { + builder.append_value(self.bloom_filter.check(&array.value(i))); + } + } + } + DataType::UInt32 => { + let array = array.as_any().downcast_ref::().unwrap(); + for i in 0..len { + if array.is_null(i) { + builder.append_value(false); + } else { + builder.append_value(self.bloom_filter.check(&array.value(i))); + } + } + } + DataType::UInt64 => { + let array = array.as_any().downcast_ref::().unwrap(); + for i in 0..len { + if array.is_null(i) { + builder.append_value(false); + } else { + builder.append_value(self.bloom_filter.check(&array.value(i))); + } + } + } + DataType::Float32 => { + let array = array.as_any().downcast_ref::().unwrap(); + for i in 0..len { + if array.is_null(i) { + builder.append_value(false); + } else { + builder.append_value(self.bloom_filter.check(&array.value(i))); + } + } + } + DataType::Float64 => { + let array = array.as_any().downcast_ref::().unwrap(); + for i in 0..len { + if array.is_null(i) { + builder.append_value(false); + } else { + builder.append_value(self.bloom_filter.check(&array.value(i))); + } + } + } + DataType::Utf8 => { + let array = array.as_any().downcast_ref::().unwrap(); + for i in 0..len { + if array.is_null(i) { + builder.append_value(false); + } else { + builder.append_value(self.bloom_filter.check(array.value(i))); + } + } + } + DataType::LargeUtf8 => { + let array = array.as_any().downcast_ref::().unwrap(); + for i in 0..len { + if array.is_null(i) { + builder.append_value(false); + } else { + builder.append_value(self.bloom_filter.check(array.value(i))); + } + } + } + DataType::Utf8View => { + let array = array.as_any().downcast_ref::().unwrap(); + for i in 0..len { + if array.is_null(i) { + builder.append_value(false); + } else { + builder.append_value(self.bloom_filter.check(array.value(i))); + } + } + } + DataType::Binary => { + let array = array.as_any().downcast_ref::().unwrap(); + for i in 0..len { + if array.is_null(i) { + builder.append_value(false); + } else { + builder.append_value(self.bloom_filter.check(array.value(i))); + } + } + } + DataType::LargeBinary => { + let array = array.as_any().downcast_ref::().unwrap(); + for i in 0..len { + if array.is_null(i) { + builder.append_value(false); + } else { + builder.append_value(self.bloom_filter.check(array.value(i))); + } + } + } + DataType::BinaryView => { + let array = array.as_any().downcast_ref::().unwrap(); + for i in 0..len { + if array.is_null(i) { + builder.append_value(false); + } else { + builder.append_value(self.bloom_filter.check(array.value(i))); + } + } + } + DataType::FixedSizeBinary(_) => { + let array = array + .as_any() + .downcast_ref::() + .unwrap(); + for i in 0..len { + if array.is_null(i) { + builder.append_value(false); + } else { + builder.append_value(self.bloom_filter.check(array.value(i))); + } + } + } + _ => { + return internal_err!( + "Unsupported data type for bloom filter check: {}", + array.data_type() + ) + } + } + + Ok(builder.finish()) + } +} + +impl fmt::Display for BloomFilterExpr { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + write!(f, "{} IN BLOOM_FILTER", self.expr) + } +} + +impl PartialEq for BloomFilterExpr { + fn eq(&self, other: &Self) -> bool { + // Two bloom filter expressions are equal if they have the same child expression + // We can't compare bloom filters directly, so we use pointer equality + self.expr.eq(&other.expr) && Arc::ptr_eq(&self.bloom_filter, &other.bloom_filter) + } +} + +impl Eq for BloomFilterExpr {} + +impl Hash for BloomFilterExpr { + fn hash(&self, state: &mut H) { + self.expr.hash(state); + // Hash the pointer to the bloom filter + Arc::as_ptr(&self.bloom_filter).hash(state); + } +} + +impl PhysicalExpr for BloomFilterExpr { + fn as_any(&self) -> &dyn Any { + self + } + + fn data_type(&self, _input_schema: &Schema) -> Result { + Ok(DataType::Boolean) + } + + fn nullable(&self, _input_schema: &Schema) -> Result { + Ok(false) + } + + fn evaluate(&self, batch: &RecordBatch) -> Result { + let value = self.expr.evaluate(batch)?; + match value { + ColumnarValue::Array(array) => { + let result = self.check_array(&array)?; + Ok(ColumnarValue::Array(Arc::new(result))) + } + ColumnarValue::Scalar(scalar) => { + let result = self.check_scalar(&scalar); + Ok(ColumnarValue::Scalar(ScalarValue::Boolean(Some(result)))) + } + } + } + + fn children(&self) -> Vec<&Arc> { + vec![&self.expr] + } + + fn with_new_children( + self: Arc, + children: Vec>, + ) -> Result> { + if children.len() != 1 { + return internal_err!("BloomFilterExpr should have exactly 1 child"); + } + Ok(Arc::new(BloomFilterExpr { + expr: Arc::clone(&children[0]), + bloom_filter: Arc::clone(&self.bloom_filter), + })) + } + + fn fmt_sql(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + write!(f, "{} IN BLOOM_FILTER", self.expr) + } +} + +#[cfg(test)] +mod tests { + use super::*; + use crate::expressions::col; + use arrow::datatypes::{Field, Schema}; + + #[test] + fn test_bloom_filter_builder() -> Result<()> { + let mut builder = BloomFilterBuilder::new(100, 0.01)?; + + // Insert some values + builder.insert_scalar(&ScalarValue::Int32(Some(1)))?; + builder.insert_scalar(&ScalarValue::Int32(Some(2)))?; + builder.insert_scalar(&ScalarValue::Int32(Some(3)))?; + + let schema = Schema::new(vec![Field::new("a", DataType::Int32, false)]); + let expr = col("a", &schema)?; + let bloom_expr = builder.finish(expr); + + // Check that inserted values are found + assert!(bloom_expr.check_scalar(&ScalarValue::Int32(Some(1)))); + assert!(bloom_expr.check_scalar(&ScalarValue::Int32(Some(2)))); + assert!(bloom_expr.check_scalar(&ScalarValue::Int32(Some(3)))); + + // A value that wasn't inserted might not be found + // (but could be a false positive, so we can't assert false) + + Ok(()) + } + + #[test] + fn test_bloom_filter_expr_evaluation() -> Result<()> { + use arrow::array::Int32Array; + + // Build a bloom filter with values 1, 2, 3 + let mut builder = BloomFilterBuilder::new(100, 0.01)?; + let training_array = Arc::new(Int32Array::from(vec![1, 2, 3])) as ArrayRef; + builder.insert_array(&training_array)?; + + // Create the expression + let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int32, false)])); + let expr = col("a", &schema)?; + let bloom_expr = Arc::new(builder.finish(expr)); + + // Create a test batch with values [1, 2, 4, 5] + let test_array = Arc::new(Int32Array::from(vec![1, 2, 4, 5])) as ArrayRef; + let batch = RecordBatch::try_new(Arc::clone(&schema), vec![test_array])?; + + // Evaluate the expression + let result = bloom_expr.evaluate(&batch)?; + let result_array = result.into_array(4)?; + let result_bool = result_array + .as_any() + .downcast_ref::() + .unwrap(); + + // Values 1 and 2 should definitely be found + assert!(result_bool.value(0)); // 1 is in the filter + assert!(result_bool.value(1)); // 2 is in the filter + + // Values 4 and 5 were not inserted, but might be false positives + // We can't assert they're false without making the test flaky + + Ok(()) + } + + #[test] + fn test_bloom_filter_with_strings() -> Result<()> { + use arrow::array::StringArray; + + let mut builder = BloomFilterBuilder::new(100, 0.01)?; + builder.insert_scalar(&ScalarValue::Utf8(Some("hello".to_string())))?; + builder.insert_scalar(&ScalarValue::Utf8(Some("world".to_string())))?; + + let schema = Arc::new(Schema::new(vec![Field::new("s", DataType::Utf8, false)])); + let expr = col("s", &schema)?; + let bloom_expr = Arc::new(builder.finish(expr)); + + let test_array = + Arc::new(StringArray::from(vec!["hello", "world", "foo"])) as ArrayRef; + let batch = RecordBatch::try_new(Arc::clone(&schema), vec![test_array])?; + + let result = bloom_expr.evaluate(&batch)?; + let result_array = result.into_array(3)?; + let result_bool = result_array + .as_any() + .downcast_ref::() + .unwrap(); + + assert!(result_bool.value(0)); // "hello" is in the filter + assert!(result_bool.value(1)); // "world" is in the filter + + Ok(()) + } +} diff --git a/datafusion/physical-expr/src/expressions/mod.rs b/datafusion/physical-expr/src/expressions/mod.rs index 59d675753d98..846ccfccb8d7 100644 --- a/datafusion/physical-expr/src/expressions/mod.rs +++ b/datafusion/physical-expr/src/expressions/mod.rs @@ -19,6 +19,7 @@ #[macro_use] mod binary; +mod bloom_filter_expr; mod case; mod cast; mod cast_column; @@ -40,6 +41,7 @@ pub use crate::aggregate::stats::StatsType; pub use crate::PhysicalSortExpr; pub use binary::{binary, similar_to, BinaryExpr}; +pub use bloom_filter_expr::{BloomFilterBuilder, BloomFilterExpr}; pub use case::{case, CaseExpr}; pub use cast::{cast, CastExpr}; pub use cast_column::CastColumnExpr; diff --git a/datafusion/physical-expr/src/lib.rs b/datafusion/physical-expr/src/lib.rs index aa8c9e50fd71..b67b1db0e92b 100644 --- a/datafusion/physical-expr/src/lib.rs +++ b/datafusion/physical-expr/src/lib.rs @@ -31,6 +31,7 @@ pub mod binary_map { pub use datafusion_physical_expr_common::binary_map::{ArrowBytesSet, OutputType}; } pub mod async_scalar_function; +pub mod bloom_filter; pub mod equivalence; pub mod expressions; pub mod intervals; From 323521d027200c2e921dcf7c01111edc5247c56f Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Mon, 27 Oct 2025 10:47:47 -0500 Subject: [PATCH 02/11] use bloom filters to push down hash table lookups in HashJoinExec --- .../physical_optimizer/filter_pushdown/mod.rs | 12 +- datafusion/physical-expr/src/bloom_filter.rs | 16 +++ .../src/expressions/bloom_filter_expr.rs | 112 ++++++++++++++++++ .../physical-plan/src/joins/hash_join/exec.rs | 63 ++++++---- .../src/joins/hash_join/shared_bounds.rs | 74 +++++++++--- .../src/joins/hash_join/stream.rs | 7 +- 6 files changed, 240 insertions(+), 44 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs b/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs index b91c1732260c..3e204cb68ad3 100644 --- a/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs +++ b/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs @@ -278,7 +278,7 @@ async fn test_dynamic_filter_pushdown_through_hash_join_with_topk() { - SortExec: TopK(fetch=2), expr=[e@4 ASC], preserve_partitioning=[false], filter=[e@4 IS NULL OR e@4 < bb] - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, d@0)] - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[d, e, f], file_type=test, pushdown_supported=true, predicate=DynamicFilter [ d@0 >= aa AND d@0 <= ab ] AND DynamicFilter [ e@1 IS NULL OR e@1 < bb ] + - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[d, e, f], file_type=test, pushdown_supported=true, predicate=DynamicFilter [ d@0 >= aa AND d@0 <= ab AND d@0 IN BLOOM_FILTER ] AND DynamicFilter [ e@1 IS NULL OR e@1 < bb ] " ); } @@ -1078,7 +1078,7 @@ async fn test_hashjoin_dynamic_filter_pushdown() { @r" - HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@0, a@0), (b@1, b@1)] - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate=DynamicFilter [ a@0 >= aa AND a@0 <= ab AND b@1 >= ba AND b@1 <= bb ] + - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate=DynamicFilter [ a@0 >= aa AND a@0 <= ab AND a@0 IN BLOOM_FILTER AND b@1 >= ba AND b@1 <= bb AND b@1 IN BLOOM_FILTER ] " ); } @@ -1309,7 +1309,7 @@ async fn test_hashjoin_dynamic_filter_pushdown_partitioned() { - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([a@0, b@1], 12), input_partitions=1 - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate=DynamicFilter [ a@0 >= ab AND a@0 <= ab AND b@1 >= bb AND b@1 <= bb OR a@0 >= aa AND a@0 <= aa AND b@1 >= ba AND b@1 <= ba ] + - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate=DynamicFilter [ a@0 >= ab AND a@0 <= ab AND a@0 IN BLOOM_FILTER AND b@1 >= bb AND b@1 <= bb AND b@1 IN BLOOM_FILTER OR a@0 >= aa AND a@0 <= aa AND a@0 IN BLOOM_FILTER AND b@1 >= ba AND b@1 <= ba AND b@1 IN BLOOM_FILTER ] " ); @@ -1503,7 +1503,7 @@ async fn test_hashjoin_dynamic_filter_pushdown_collect_left() { - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([a@0, b@1], 12), input_partitions=1 - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate=DynamicFilter [ a@0 >= aa AND a@0 <= ab AND b@1 >= ba AND b@1 <= bb ] + - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate=DynamicFilter [ a@0 >= aa AND a@0 <= ab AND a@0 IN BLOOM_FILTER AND b@1 >= ba AND b@1 <= bb AND b@1 IN BLOOM_FILTER ] " ); @@ -1671,8 +1671,8 @@ async fn test_nested_hashjoin_dynamic_filter_pushdown() { - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, b@0)] - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, x], file_type=test, pushdown_supported=true - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@1, d@0)] - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[b, c, y], file_type=test, pushdown_supported=true, predicate=DynamicFilter [ b@0 >= aa AND b@0 <= ab ] - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[d, z], file_type=test, pushdown_supported=true, predicate=DynamicFilter [ d@0 >= ca AND d@0 <= cb ] + - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[b, c, y], file_type=test, pushdown_supported=true, predicate=DynamicFilter [ b@0 >= aa AND b@0 <= ab AND b@0 IN BLOOM_FILTER ] + - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[d, z], file_type=test, pushdown_supported=true, predicate=DynamicFilter [ d@0 >= ca AND d@0 <= cb AND d@0 IN BLOOM_FILTER ] " ); } diff --git a/datafusion/physical-expr/src/bloom_filter.rs b/datafusion/physical-expr/src/bloom_filter.rs index 613959cde6eb..b98199aa014f 100644 --- a/datafusion/physical-expr/src/bloom_filter.rs +++ b/datafusion/physical-expr/src/bloom_filter.rs @@ -252,6 +252,22 @@ impl AsBytes for f64 { } } +impl AsBytes for i128 { + fn as_bytes(&self) -> &[u8] { + // Use big-endian for i128 to match Parquet's FIXED_LEN_BYTE_ARRAY representation + // This allows compatibility with Parquet bloom filters + unsafe { + std::slice::from_raw_parts(self as *const i128 as *const u8, size_of::()) + } + } +} + +impl AsBytes for [u8; 32] { + fn as_bytes(&self) -> &[u8] { + self + } +} + /// Hash a value using xxHash64 with seed 0 #[inline] fn hash_as_bytes(value: &A) -> u64 { diff --git a/datafusion/physical-expr/src/expressions/bloom_filter_expr.rs b/datafusion/physical-expr/src/expressions/bloom_filter_expr.rs index 233b6c225d24..4084c5bc4ef2 100644 --- a/datafusion/physical-expr/src/expressions/bloom_filter_expr.rs +++ b/datafusion/physical-expr/src/expressions/bloom_filter_expr.rs @@ -79,6 +79,14 @@ impl BloomFilterBuilder { | ScalarValue::LargeBinary(Some(v)) | ScalarValue::FixedSizeBinary(_, Some(v)) => self.sbbf.insert(v.as_slice()), ScalarValue::BinaryView(Some(v)) => self.sbbf.insert(v.as_slice()), + ScalarValue::Decimal32(Some(v), _, _) => self.sbbf.insert(v), + ScalarValue::Decimal64(Some(v), _, _) => self.sbbf.insert(v), + ScalarValue::Decimal128(Some(v), _, _) => self.sbbf.insert(v), + ScalarValue::Decimal256(Some(v), _, _) => { + // Convert i256 to bytes + let bytes = v.to_be_bytes(); + self.sbbf.insert(&bytes) + } _ => { return exec_err!( "Unsupported data type for bloom filter: {}", @@ -210,6 +218,29 @@ impl BloomFilterBuilder { } } } + DataType::Decimal128(_, _) => { + let array = array + .as_any() + .downcast_ref::() + .unwrap(); + for i in 0..array.len() { + if !array.is_null(i) { + self.sbbf.insert(&array.value(i)); + } + } + } + DataType::Decimal256(_, _) => { + let array = array + .as_any() + .downcast_ref::() + .unwrap(); + for i in 0..array.len() { + if !array.is_null(i) { + let bytes = array.value(i).to_be_bytes(); + self.sbbf.insert(&bytes); + } + } + } _ => { return exec_err!( "Unsupported data type for bloom filter: {}", @@ -257,6 +288,11 @@ impl BloomFilterExpr { } } + /// Get a reference to the underlying bloom filter + pub fn bloom_filter(&self) -> &Sbbf { + &self.bloom_filter + } + /// Check a scalar value against the bloom filter fn check_scalar(&self, value: &ScalarValue) -> bool { if value.is_null() { @@ -281,6 +317,13 @@ impl BloomFilterExpr { self.bloom_filter.check(v.as_slice()) } ScalarValue::BinaryView(Some(v)) => self.bloom_filter.check(v.as_slice()), + ScalarValue::Decimal32(Some(v), _, _) => self.bloom_filter.check(v), + ScalarValue::Decimal64(Some(v), _, _) => self.bloom_filter.check(v), + ScalarValue::Decimal128(Some(v), _, _) => self.bloom_filter.check(v), + ScalarValue::Decimal256(Some(v), _, _) => { + let bytes = v.to_be_bytes(); + self.bloom_filter.check(&bytes) + } _ => true, // Unsupported types default to "might be present" } } @@ -437,6 +480,33 @@ impl BloomFilterExpr { } } } + DataType::Decimal128(_, _) => { + let array = array + .as_any() + .downcast_ref::() + .unwrap(); + for i in 0..len { + if array.is_null(i) { + builder.append_value(false); + } else { + builder.append_value(self.bloom_filter.check(&array.value(i))); + } + } + } + DataType::Decimal256(_, _) => { + let array = array + .as_any() + .downcast_ref::() + .unwrap(); + for i in 0..len { + if array.is_null(i) { + builder.append_value(false); + } else { + let bytes = array.value(i).to_be_bytes(); + builder.append_value(self.bloom_filter.check(&bytes)); + } + } + } _ => { return internal_err!( "Unsupported data type for bloom filter check: {}", @@ -616,4 +686,46 @@ mod tests { Ok(()) } + + #[test] + fn test_bloom_filter_with_decimals() -> Result<()> { + use arrow::array::Decimal128Array; + + // Build a bloom filter with decimal values + let mut builder = BloomFilterBuilder::new(100, 0.01)?; + builder.insert_scalar(&ScalarValue::Decimal128(Some(12345), 10, 2))?; + builder.insert_scalar(&ScalarValue::Decimal128(Some(67890), 10, 2))?; + + let schema = Arc::new(Schema::new(vec![Field::new( + "d", + DataType::Decimal128(10, 2), + false, + )])); + let expr = col("d", &schema)?; + let bloom_expr = Arc::new(builder.finish(expr)); + + // Create test array with decimal values + let test_array = Arc::new( + Decimal128Array::from(vec![12345, 67890, 11111]) + .with_precision_and_scale(10, 2)?, + ) as ArrayRef; + let batch = RecordBatch::try_new(Arc::clone(&schema), vec![test_array])?; + + // Evaluate the expression + let result = bloom_expr.evaluate(&batch)?; + let result_array = result.into_array(3)?; + let result_bool = result_array + .as_any() + .downcast_ref::() + .unwrap(); + + // Values that were inserted should be found + assert!(result_bool.value(0)); // 12345 is in the filter + assert!(result_bool.value(1)); // 67890 is in the filter + + // Value 11111 was not inserted, but might be a false positive + // We can't assert it's false without making the test flaky + + Ok(()) + } } diff --git a/datafusion/physical-plan/src/joins/hash_join/exec.rs b/datafusion/physical-plan/src/joins/hash_join/exec.rs index b5fe5ee5cda1..224358722a42 100644 --- a/datafusion/physical-plan/src/joins/hash_join/exec.rs +++ b/datafusion/physical-plan/src/joins/hash_join/exec.rs @@ -53,6 +53,7 @@ use crate::{ DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, Partitioning, PlanProperties, SendableRecordBatchStream, Statistics, }; +use datafusion_physical_expr::bloom_filter::Sbbf; use arrow::array::{ArrayRef, BooleanBufferBuilder}; use arrow::compute::concat_batches; @@ -72,7 +73,9 @@ use datafusion_functions_aggregate_common::min_max::{MaxAccumulator, MinAccumula use datafusion_physical_expr::equivalence::{ join_equivalence_properties, ProjectionMapping, }; -use datafusion_physical_expr::expressions::{lit, DynamicFilterPhysicalExpr}; +use datafusion_physical_expr::expressions::{ + lit, BloomFilterBuilder, DynamicFilterPhysicalExpr, +}; use datafusion_physical_expr::{PhysicalExpr, PhysicalExprRef}; use ahash::RandomState; @@ -104,10 +107,13 @@ pub(super) struct JoinLeftData { _reservation: MemoryReservation, /// Bounds computed from the build side for dynamic filter pushdown pub(super) bounds: Option>, + /// Bloom filters computed from the build side for dynamic filter pushdown + pub(super) bloom_filters: Option>, } impl JoinLeftData { /// Create a new `JoinLeftData` from its parts + #[allow(clippy::too_many_arguments)] pub(super) fn new( hash_map: Box, batch: RecordBatch, @@ -116,6 +122,7 @@ impl JoinLeftData { probe_threads_counter: AtomicUsize, reservation: MemoryReservation, bounds: Option>, + bloom_filters: Option>, ) -> Self { Self { hash_map, @@ -125,6 +132,7 @@ impl JoinLeftData { probe_threads_counter, _reservation: reservation, bounds, + bloom_filters, } } @@ -1207,14 +1215,14 @@ impl ExecutionPlan for HashJoinExec { } } -/// Accumulator for collecting min/max bounds from build-side data during hash join. +/// Accumulator for collecting min/max bounds and bloom filters from build-side data during hash join. /// /// This struct encapsulates the logic for progressively computing column bounds -/// (minimum and maximum values) for a specific join key expression as batches +/// (minimum and maximum values) and bloom filters for a specific join key expression as batches /// are processed during the build phase of a hash join. /// -/// The bounds are used for dynamic filter pushdown optimization, where filters -/// based on the actual data ranges can be pushed down to the probe side to +/// The bounds and bloom filters are used for dynamic filter pushdown optimization, where filters +/// based on the actual data ranges and membership can be pushed down to the probe side to /// eliminate unnecessary data early. struct CollectLeftAccumulator { /// The physical expression to evaluate for each batch @@ -1223,6 +1231,8 @@ struct CollectLeftAccumulator { min: MinAccumulator, /// Accumulator for tracking the maximum value across all batches max: MaxAccumulator, + /// Bloom filter builder for membership testing + bloom_filter: BloomFilterBuilder, } impl CollectLeftAccumulator { @@ -1249,17 +1259,23 @@ impl CollectLeftAccumulator { .data_type(schema) // Min/Max can operate on dictionary data but expect to be initialized with the underlying value type .map(|dt| dictionary_value_type(&dt))?; + + // Create bloom filter with default parameters + // NDV (number of distinct values) = 10000, FPP (false positive probability) = 0.01 (1%) + let bloom_filter = BloomFilterBuilder::new(10000, 0.01)?; + Ok(Self { expr, min: MinAccumulator::try_new(&data_type)?, max: MaxAccumulator::try_new(&data_type)?, + bloom_filter, }) } /// Updates the accumulators with values from a new batch. /// - /// Evaluates the expression on the batch and updates both min and max - /// accumulators with the resulting values. + /// Evaluates the expression on the batch and updates min, max, and bloom filter + /// with the resulting values. /// /// # Arguments /// * `batch` - The record batch to process @@ -1270,20 +1286,24 @@ impl CollectLeftAccumulator { let array = self.expr.evaluate(batch)?.into_array(batch.num_rows())?; self.min.update_batch(std::slice::from_ref(&array))?; self.max.update_batch(std::slice::from_ref(&array))?; + // Insert values into bloom filter + self.bloom_filter.insert_array(&array)?; Ok(()) } - /// Finalizes the accumulation and returns the computed bounds. + /// Finalizes the accumulation and returns the computed bounds and bloom filter. /// - /// Consumes self to extract the final min and max values from the accumulators. + /// Consumes self to extract the final min and max values from the accumulators + /// and the built bloom filter. /// /// # Returns - /// The `ColumnBounds` containing the minimum and maximum values observed - fn evaluate(mut self) -> Result { - Ok(ColumnBounds::new( - self.min.evaluate()?, - self.max.evaluate()?, - )) + /// A tuple of (`ColumnBounds`, `Sbbf`) containing the minimum/maximum values and bloom filter + fn evaluate(mut self) -> Result<(ColumnBounds, Sbbf)> { + let bounds = ColumnBounds::new(self.min.evaluate()?, self.max.evaluate()?); + let bloom_filter_expr = self.bloom_filter.finish(Arc::clone(&self.expr)); + // Extract the Sbbf from the BloomFilterExpr + let bloom_filter = bloom_filter_expr.bloom_filter().clone(); + Ok((bounds, bloom_filter)) } } @@ -1475,16 +1495,18 @@ async fn collect_left_input( }) .collect::>>()?; - // Compute bounds for dynamic filter if enabled - let bounds = match bounds_accumulators { + // Compute bounds and bloom filters for dynamic filter if enabled + let (bounds, bloom_filters) = match bounds_accumulators { Some(accumulators) if num_rows > 0 => { - let bounds = accumulators + let results: Vec<_> = accumulators .into_iter() .map(CollectLeftAccumulator::evaluate) .collect::>>()?; - Some(bounds) + // Separate bounds and bloom filters + let (bounds, bloom_filters): (Vec<_>, Vec<_>) = results.into_iter().unzip(); + (Some(bounds), Some(bloom_filters)) } - _ => None, + _ => (None, None), }; let data = JoinLeftData::new( @@ -1495,6 +1517,7 @@ async fn collect_left_input( AtomicUsize::new(probe_threads_count), reservation, bounds, + bloom_filters, ); Ok(data) diff --git a/datafusion/physical-plan/src/joins/hash_join/shared_bounds.rs b/datafusion/physical-plan/src/joins/hash_join/shared_bounds.rs index 25f7a0de31ac..69af975ee34e 100644 --- a/datafusion/physical-plan/src/joins/hash_join/shared_bounds.rs +++ b/datafusion/physical-plan/src/joins/hash_join/shared_bounds.rs @@ -27,7 +27,10 @@ use crate::ExecutionPlanProperties; use datafusion_common::{Result, ScalarValue}; use datafusion_expr::Operator; -use datafusion_physical_expr::expressions::{lit, BinaryExpr, DynamicFilterPhysicalExpr}; +use datafusion_physical_expr::bloom_filter::Sbbf; +use datafusion_physical_expr::expressions::{ + lit, BinaryExpr, BloomFilterExpr, DynamicFilterPhysicalExpr, +}; use datafusion_physical_expr::{PhysicalExpr, PhysicalExprRef}; use itertools::Itertools; @@ -51,7 +54,7 @@ impl ColumnBounds { } /// Represents the bounds for all join key columns from a single partition. -/// This contains the min/max values computed from one partition's build-side data. +/// This contains the min/max values and bloom filters computed from one partition's build-side data. #[derive(Debug, Clone)] pub(crate) struct PartitionBounds { /// Partition identifier for debugging and determinism (not strictly necessary) @@ -59,13 +62,21 @@ pub(crate) struct PartitionBounds { /// Min/max bounds for each join key column in this partition. /// Index corresponds to the join key expression index. column_bounds: Vec, + /// Bloom filters for each join key column in this partition. + /// Index corresponds to the join key expression index. + bloom_filters: Vec, } impl PartitionBounds { - pub(crate) fn new(partition: usize, column_bounds: Vec) -> Self { + pub(crate) fn new( + partition: usize, + column_bounds: Vec, + bloom_filters: Vec, + ) -> Self { Self { partition, column_bounds, + bloom_filters, } } @@ -76,6 +87,10 @@ impl PartitionBounds { pub(crate) fn get_column_bounds(&self, index: usize) -> Option<&ColumnBounds> { self.column_bounds.get(index) } + + pub(crate) fn get_bloom_filter(&self, index: usize) -> Option<&Sbbf> { + self.bloom_filters.get(index) + } } /// Coordinates dynamic filter bounds collection across multiple partitions @@ -175,15 +190,15 @@ impl SharedBoundsAccumulator { } } - /// Create a filter expression from individual partition bounds using OR logic. + /// Create a filter expression from individual partition bounds and bloom filters using OR logic. /// - /// This creates a filter where each partition's bounds form a conjunction (AND) - /// of column range predicates, and all partitions are combined with OR. + /// This creates a filter where each partition's bounds and bloom filters form a conjunction (AND) + /// of column range predicates and bloom filter checks, and all partitions are combined with OR. /// /// For example, with 2 partitions and 2 columns: - /// ((col0 >= p0_min0 AND col0 <= p0_max0 AND col1 >= p0_min1 AND col1 <= p0_max1) + /// ((col0 >= p0_min0 AND col0 <= p0_max0 AND col0 IN BLOOM_FILTER_0 AND col1 >= p0_min1 AND col1 <= p0_max1 AND col1 IN BLOOM_FILTER_1) /// OR - /// (col0 >= p1_min0 AND col0 <= p1_max0 AND col1 >= p1_min1 AND col1 <= p1_max1)) + /// (col0 >= p1_min0 AND col0 <= p1_max0 AND col0 IN BLOOM_FILTER_0 AND col1 >= p1_min1 AND col1 <= p1_max1 AND col1 IN BLOOM_FILTER_1)) pub(crate) fn create_filter_from_partition_bounds( &self, bounds: &[PartitionBounds], @@ -196,7 +211,7 @@ impl SharedBoundsAccumulator { let mut partition_predicates = Vec::with_capacity(bounds.len()); for partition_bounds in bounds.iter().sorted_by_key(|b| b.partition) { - // Create range predicates for each join key in this partition + // Create range predicates and bloom filter checks for each join key in this partition let mut column_predicates = Vec::with_capacity(partition_bounds.len()); for (col_idx, right_expr) in self.on_right.iter().enumerate() { @@ -215,7 +230,28 @@ impl SharedBoundsAccumulator { let range_expr = Arc::new(BinaryExpr::new(min_expr, Operator::And, max_expr)) as Arc; - column_predicates.push(range_expr); + + // Create bloom filter check: col IN BLOOM_FILTER + if let Some(bloom_filter) = partition_bounds.get_bloom_filter(col_idx) + { + let bloom_expr = Arc::new(BloomFilterExpr::new( + Arc::clone(right_expr), + bloom_filter.clone(), + )) + as Arc; + + // Combine range and bloom filter: (range_expr AND bloom_expr) + let combined_expr = Arc::new(BinaryExpr::new( + range_expr, + Operator::And, + bloom_expr, + )) + as Arc; + column_predicates.push(combined_expr); + } else { + // If no bloom filter, just use range expression + column_predicates.push(range_expr); + } } } @@ -247,8 +283,8 @@ impl SharedBoundsAccumulator { /// Report bounds from a completed partition and update dynamic filter if all partitions are done /// /// This method coordinates the dynamic filter updates across all partitions. It stores the - /// bounds from the current partition, increments the completion counter, and when all - /// partitions have reported, creates an OR'd filter from individual partition bounds. + /// bounds and bloom filters from the current partition, increments the completion counter, and when all + /// partitions have reported, creates an OR'd filter from individual partition bounds and bloom filters. /// /// This method is async and uses a [`tokio::sync::Barrier`] to wait for all partitions /// to report their bounds. Once that occurs, the method will resolve for all callers and the @@ -264,6 +300,7 @@ impl SharedBoundsAccumulator { /// # Arguments /// * `left_side_partition_id` - The identifier for the **left-side** partition reporting its bounds /// * `partition_bounds` - The bounds computed by this partition (if any) + /// * `bloom_filters` - The bloom filters computed by this partition (if any) /// /// # Returns /// * `Result<()>` - Ok if successful, Err if filter update failed @@ -271,9 +308,10 @@ impl SharedBoundsAccumulator { &self, left_side_partition_id: usize, partition_bounds: Option>, + bloom_filters: Option>, ) -> Result<()> { - // Store bounds in the accumulator - this runs once per partition - if let Some(bounds) = partition_bounds { + // Store bounds and bloom filters in the accumulator - this runs once per partition + if let (Some(bounds), Some(filters)) = (partition_bounds, bloom_filters) { let mut guard = self.inner.lock(); let should_push = if let Some(last_bound) = guard.bounds.last() { @@ -286,9 +324,11 @@ impl SharedBoundsAccumulator { }; if should_push { - guard - .bounds - .push(PartitionBounds::new(left_side_partition_id, bounds)); + guard.bounds.push(PartitionBounds::new( + left_side_partition_id, + bounds, + filters, + )); } } diff --git a/datafusion/physical-plan/src/joins/hash_join/stream.rs b/datafusion/physical-plan/src/joins/hash_join/stream.rs index 88c50c2eb2ce..afd985a86e48 100644 --- a/datafusion/physical-plan/src/joins/hash_join/stream.rs +++ b/datafusion/physical-plan/src/joins/hash_join/stream.rs @@ -420,9 +420,14 @@ impl HashJoinStream { }; let left_data_bounds = left_data.bounds.clone(); + let left_data_bloom_filters = left_data.bloom_filters.clone(); self.bounds_waiter = Some(OnceFut::new(async move { bounds_accumulator - .report_partition_bounds(left_side_partition_id, left_data_bounds) + .report_partition_bounds( + left_side_partition_id, + left_data_bounds, + left_data_bloom_filters, + ) .await })); self.state = HashJoinStreamState::WaitPartitionBoundsReport; From 2ea6709b0c06b37d60f54fd6caf120cb9dabbb1f Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Mon, 27 Oct 2025 12:33:16 -0500 Subject: [PATCH 03/11] use builder, clean up unnamed tuples --- datafusion/physical-expr/src/bloom_filter.rs | 2 +- .../src/expressions/bloom_filter_expr.rs | 33 +++--- .../physical-plan/src/joins/hash_join/exec.rs | 46 +++----- .../src/joins/hash_join/shared_bounds.rs | 105 ++++++++---------- .../src/joins/hash_join/stream.rs | 9 +- 5 files changed, 85 insertions(+), 110 deletions(-) diff --git a/datafusion/physical-expr/src/bloom_filter.rs b/datafusion/physical-expr/src/bloom_filter.rs index b98199aa014f..a15aadcf692f 100644 --- a/datafusion/physical-expr/src/bloom_filter.rs +++ b/datafusion/physical-expr/src/bloom_filter.rs @@ -119,7 +119,7 @@ impl std::ops::IndexMut for Block { /// an element is a member of a set. False positive matches are possible, but /// false negatives are not. #[derive(Debug, Clone)] -pub struct Sbbf(Vec); +pub(crate) struct Sbbf(Vec); impl Sbbf { /// Create a new Sbbf with given number of distinct values and false positive probability. diff --git a/datafusion/physical-expr/src/expressions/bloom_filter_expr.rs b/datafusion/physical-expr/src/expressions/bloom_filter_expr.rs index 4084c5bc4ef2..479f2be8761c 100644 --- a/datafusion/physical-expr/src/expressions/bloom_filter_expr.rs +++ b/datafusion/physical-expr/src/expressions/bloom_filter_expr.rs @@ -39,8 +39,9 @@ use std::sync::Arc; /// let mut builder = BloomFilterBuilder::new(1000, 0.01)?; /// builder.insert_scalar(&ScalarValue::Int32(Some(42)))?; /// builder.insert_array(&int_array)?; -/// let expr = builder.finish(col_expr)?; +/// let expr = builder.build(&col_expr); /// ``` +#[derive(Clone, Debug)] pub struct BloomFilterBuilder { /// The underlying bloom filter sbbf: Sbbf, @@ -251,15 +252,16 @@ impl BloomFilterBuilder { Ok(()) } - /// Finish building and create a `BloomFilterExpr` + /// Build a `BloomFilterExpr` from this builder /// - /// This consumes the builder and returns a static PhysicalExpr - /// that checks values against the constructed bloom filter. + /// This method does not consume the builder, allowing multiple + /// `BloomFilterExpr` instances to be created with different expressions + /// but sharing the same bloom filter data. /// /// # Arguments /// * `expr` - The expression to evaluate and check against the bloom filter - pub fn finish(self, expr: Arc) -> BloomFilterExpr { - BloomFilterExpr::new(expr, self.sbbf) + pub fn build(&self, expr: Arc) -> BloomFilterExpr { + BloomFilterExpr::new(expr, self.sbbf.clone()) } } @@ -280,19 +282,16 @@ pub struct BloomFilterExpr { } impl BloomFilterExpr { - /// Create a new bloom filter expression - pub fn new(expr: Arc, bloom_filter: Sbbf) -> Self { + /// Create a new bloom filter expression (internal use only) + /// + /// Users should create bloom filter expressions through `BloomFilterBuilder::build()` + pub(crate) fn new(expr: Arc, bloom_filter: Sbbf) -> Self { Self { expr, bloom_filter: Arc::new(bloom_filter), } } - /// Get a reference to the underlying bloom filter - pub fn bloom_filter(&self) -> &Sbbf { - &self.bloom_filter - } - /// Check a scalar value against the bloom filter fn check_scalar(&self, value: &ScalarValue) -> bool { if value.is_null() { @@ -609,7 +608,7 @@ mod tests { let schema = Schema::new(vec![Field::new("a", DataType::Int32, false)]); let expr = col("a", &schema)?; - let bloom_expr = builder.finish(expr); + let bloom_expr = builder.build(expr); // Check that inserted values are found assert!(bloom_expr.check_scalar(&ScalarValue::Int32(Some(1)))); @@ -634,7 +633,7 @@ mod tests { // Create the expression let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int32, false)])); let expr = col("a", &schema)?; - let bloom_expr = Arc::new(builder.finish(expr)); + let bloom_expr = Arc::new(builder.build(expr)); // Create a test batch with values [1, 2, 4, 5] let test_array = Arc::new(Int32Array::from(vec![1, 2, 4, 5])) as ArrayRef; @@ -668,7 +667,7 @@ mod tests { let schema = Arc::new(Schema::new(vec![Field::new("s", DataType::Utf8, false)])); let expr = col("s", &schema)?; - let bloom_expr = Arc::new(builder.finish(expr)); + let bloom_expr = Arc::new(builder.build(expr)); let test_array = Arc::new(StringArray::from(vec!["hello", "world", "foo"])) as ArrayRef; @@ -702,7 +701,7 @@ mod tests { false, )])); let expr = col("d", &schema)?; - let bloom_expr = Arc::new(builder.finish(expr)); + let bloom_expr = Arc::new(builder.build(expr)); // Create test array with decimal values let test_array = Arc::new( diff --git a/datafusion/physical-plan/src/joins/hash_join/exec.rs b/datafusion/physical-plan/src/joins/hash_join/exec.rs index 224358722a42..46794fd989af 100644 --- a/datafusion/physical-plan/src/joins/hash_join/exec.rs +++ b/datafusion/physical-plan/src/joins/hash_join/exec.rs @@ -26,7 +26,9 @@ use crate::filter_pushdown::{ ChildPushdownResult, FilterDescription, FilterPushdownPhase, FilterPushdownPropagation, }; -use crate::joins::hash_join::shared_bounds::{ColumnBounds, SharedBoundsAccumulator}; +use crate::joins::hash_join::shared_bounds::{ + ColumnBounds, ColumnFilterData, SharedBoundsAccumulator, +}; use crate::joins::hash_join::stream::{ BuildSide, BuildSideInitialState, HashJoinStream, HashJoinStreamState, }; @@ -53,7 +55,6 @@ use crate::{ DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, Partitioning, PlanProperties, SendableRecordBatchStream, Statistics, }; -use datafusion_physical_expr::bloom_filter::Sbbf; use arrow::array::{ArrayRef, BooleanBufferBuilder}; use arrow::compute::concat_batches; @@ -105,15 +106,12 @@ pub(super) struct JoinLeftData { /// This could hide potential out-of-memory issues, especially when upstream operators increase their memory consumption. /// The MemoryReservation ensures proper tracking of memory resources throughout the join operation's lifecycle. _reservation: MemoryReservation, - /// Bounds computed from the build side for dynamic filter pushdown - pub(super) bounds: Option>, - /// Bloom filters computed from the build side for dynamic filter pushdown - pub(super) bloom_filters: Option>, + /// Filter data (bounds + bloom filters) computed from the build side for dynamic filter pushdown + pub(super) column_filters: Option>, } impl JoinLeftData { /// Create a new `JoinLeftData` from its parts - #[allow(clippy::too_many_arguments)] pub(super) fn new( hash_map: Box, batch: RecordBatch, @@ -121,8 +119,7 @@ impl JoinLeftData { visited_indices_bitmap: SharedBitmapBuilder, probe_threads_counter: AtomicUsize, reservation: MemoryReservation, - bounds: Option>, - bloom_filters: Option>, + column_filters: Option>, ) -> Self { Self { hash_map, @@ -131,8 +128,7 @@ impl JoinLeftData { visited_indices_bitmap, probe_threads_counter, _reservation: reservation, - bounds, - bloom_filters, + column_filters, } } @@ -1291,19 +1287,16 @@ impl CollectLeftAccumulator { Ok(()) } - /// Finalizes the accumulation and returns the computed bounds and bloom filter. + /// Finalizes the accumulation and returns the computed filter data. /// /// Consumes self to extract the final min and max values from the accumulators - /// and the built bloom filter. + /// and the bloom filter builder. /// /// # Returns - /// A tuple of (`ColumnBounds`, `Sbbf`) containing the minimum/maximum values and bloom filter - fn evaluate(mut self) -> Result<(ColumnBounds, Sbbf)> { + /// `ColumnFilterData` containing the bounds and bloom filter builder + fn evaluate(mut self) -> Result { let bounds = ColumnBounds::new(self.min.evaluate()?, self.max.evaluate()?); - let bloom_filter_expr = self.bloom_filter.finish(Arc::clone(&self.expr)); - // Extract the Sbbf from the BloomFilterExpr - let bloom_filter = bloom_filter_expr.bloom_filter().clone(); - Ok((bounds, bloom_filter)) + Ok(ColumnFilterData::new(bounds, self.bloom_filter)) } } @@ -1495,18 +1488,16 @@ async fn collect_left_input( }) .collect::>>()?; - // Compute bounds and bloom filters for dynamic filter if enabled - let (bounds, bloom_filters) = match bounds_accumulators { + // Compute filter data (bounds + bloom filters) for dynamic filter if enabled + let column_filters = match bounds_accumulators { Some(accumulators) if num_rows > 0 => { - let results: Vec<_> = accumulators + let column_filters: Vec<_> = accumulators .into_iter() .map(CollectLeftAccumulator::evaluate) .collect::>>()?; - // Separate bounds and bloom filters - let (bounds, bloom_filters): (Vec<_>, Vec<_>) = results.into_iter().unzip(); - (Some(bounds), Some(bloom_filters)) + Some(column_filters) } - _ => (None, None), + _ => None, }; let data = JoinLeftData::new( @@ -1516,8 +1507,7 @@ async fn collect_left_input( Mutex::new(visited_indices_bitmap), AtomicUsize::new(probe_threads_count), reservation, - bounds, - bloom_filters, + column_filters, ); Ok(data) diff --git a/datafusion/physical-plan/src/joins/hash_join/shared_bounds.rs b/datafusion/physical-plan/src/joins/hash_join/shared_bounds.rs index 69af975ee34e..d97c4c8b4534 100644 --- a/datafusion/physical-plan/src/joins/hash_join/shared_bounds.rs +++ b/datafusion/physical-plan/src/joins/hash_join/shared_bounds.rs @@ -27,9 +27,8 @@ use crate::ExecutionPlanProperties; use datafusion_common::{Result, ScalarValue}; use datafusion_expr::Operator; -use datafusion_physical_expr::bloom_filter::Sbbf; use datafusion_physical_expr::expressions::{ - lit, BinaryExpr, BloomFilterExpr, DynamicFilterPhysicalExpr, + lit, BinaryExpr, BloomFilterBuilder, DynamicFilterPhysicalExpr, }; use datafusion_physical_expr::{PhysicalExpr, PhysicalExprRef}; @@ -43,7 +42,7 @@ use tokio::sync::Barrier; pub(crate) struct ColumnBounds { /// The minimum value observed for this column min: ScalarValue, - /// The maximum value observed for this column + /// The maximum value observed for this column max: ScalarValue, } @@ -53,43 +52,50 @@ impl ColumnBounds { } } +/// Filter data for a single join key column, combining bounds and bloom filter. +/// Used in dynamic filter pushdown for comprehensive filtering. +#[derive(Debug, Clone)] +pub(crate) struct ColumnFilterData { + /// Min/max bounds for range filtering + pub(crate) bounds: ColumnBounds, + /// Bloom filter builder for membership testing + pub(crate) bloom_filter: BloomFilterBuilder, +} + +impl ColumnFilterData { + pub(crate) fn new(bounds: ColumnBounds, bloom_filter: BloomFilterBuilder) -> Self { + Self { + bounds, + bloom_filter, + } + } +} + /// Represents the bounds for all join key columns from a single partition. -/// This contains the min/max values and bloom filters computed from one partition's build-side data. +/// This contains the filter data (min/max values and bloom filters) computed from one partition's build-side data. #[derive(Debug, Clone)] pub(crate) struct PartitionBounds { /// Partition identifier for debugging and determinism (not strictly necessary) partition: usize, - /// Min/max bounds for each join key column in this partition. - /// Index corresponds to the join key expression index. - column_bounds: Vec, - /// Bloom filters for each join key column in this partition. + /// Filter data (bounds + bloom filter) for each join key column in this partition. /// Index corresponds to the join key expression index. - bloom_filters: Vec, + column_filters: Vec, } impl PartitionBounds { - pub(crate) fn new( - partition: usize, - column_bounds: Vec, - bloom_filters: Vec, - ) -> Self { + pub(crate) fn new(partition: usize, column_filters: Vec) -> Self { Self { partition, - column_bounds, - bloom_filters, + column_filters, } } pub(crate) fn len(&self) -> usize { - self.column_bounds.len() + self.column_filters.len() } - pub(crate) fn get_column_bounds(&self, index: usize) -> Option<&ColumnBounds> { - self.column_bounds.get(index) - } - - pub(crate) fn get_bloom_filter(&self, index: usize) -> Option<&Sbbf> { - self.bloom_filters.get(index) + pub(crate) fn get_column_filter(&self, index: usize) -> Option<&ColumnFilterData> { + self.column_filters.get(index) } } @@ -215,43 +221,32 @@ impl SharedBoundsAccumulator { let mut column_predicates = Vec::with_capacity(partition_bounds.len()); for (col_idx, right_expr) in self.on_right.iter().enumerate() { - if let Some(column_bounds) = partition_bounds.get_column_bounds(col_idx) { + if let Some(filter_data) = partition_bounds.get_column_filter(col_idx) { // Create predicate: col >= min AND col <= max let min_expr = Arc::new(BinaryExpr::new( Arc::clone(right_expr), Operator::GtEq, - lit(column_bounds.min.clone()), + lit(filter_data.bounds.min.clone()), )) as Arc; let max_expr = Arc::new(BinaryExpr::new( Arc::clone(right_expr), Operator::LtEq, - lit(column_bounds.max.clone()), + lit(filter_data.bounds.max.clone()), )) as Arc; let range_expr = Arc::new(BinaryExpr::new(min_expr, Operator::And, max_expr)) as Arc; // Create bloom filter check: col IN BLOOM_FILTER - if let Some(bloom_filter) = partition_bounds.get_bloom_filter(col_idx) - { - let bloom_expr = Arc::new(BloomFilterExpr::new( - Arc::clone(right_expr), - bloom_filter.clone(), - )) - as Arc; + let bloom_expr = Arc::new(filter_data.bloom_filter.build(Arc::clone( + right_expr, + ))) as Arc; - // Combine range and bloom filter: (range_expr AND bloom_expr) - let combined_expr = Arc::new(BinaryExpr::new( - range_expr, - Operator::And, - bloom_expr, - )) + // Combine range and bloom filter: (range_expr AND bloom_expr) + let combined_expr = + Arc::new(BinaryExpr::new(range_expr, Operator::And, bloom_expr)) as Arc; - column_predicates.push(combined_expr); - } else { - // If no bloom filter, just use range expression - column_predicates.push(range_expr); - } + column_predicates.push(combined_expr); } } @@ -283,8 +278,8 @@ impl SharedBoundsAccumulator { /// Report bounds from a completed partition and update dynamic filter if all partitions are done /// /// This method coordinates the dynamic filter updates across all partitions. It stores the - /// bounds and bloom filters from the current partition, increments the completion counter, and when all - /// partitions have reported, creates an OR'd filter from individual partition bounds and bloom filters. + /// filter data from the current partition, increments the completion counter, and when all + /// partitions have reported, creates an OR'd filter from individual partition filter data. /// /// This method is async and uses a [`tokio::sync::Barrier`] to wait for all partitions /// to report their bounds. Once that occurs, the method will resolve for all callers and the @@ -299,19 +294,17 @@ impl SharedBoundsAccumulator { /// /// # Arguments /// * `left_side_partition_id` - The identifier for the **left-side** partition reporting its bounds - /// * `partition_bounds` - The bounds computed by this partition (if any) - /// * `bloom_filters` - The bloom filters computed by this partition (if any) + /// * `column_filters` - The filter data computed by this partition (if any) /// /// # Returns /// * `Result<()>` - Ok if successful, Err if filter update failed pub(crate) async fn report_partition_bounds( &self, left_side_partition_id: usize, - partition_bounds: Option>, - bloom_filters: Option>, + column_filters: Option>, ) -> Result<()> { - // Store bounds and bloom filters in the accumulator - this runs once per partition - if let (Some(bounds), Some(filters)) = (partition_bounds, bloom_filters) { + // Store filter data in the accumulator - this runs once per partition + if let Some(filters) = column_filters { let mut guard = self.inner.lock(); let should_push = if let Some(last_bound) = guard.bounds.last() { @@ -324,11 +317,9 @@ impl SharedBoundsAccumulator { }; if should_push { - guard.bounds.push(PartitionBounds::new( - left_side_partition_id, - bounds, - filters, - )); + guard + .bounds + .push(PartitionBounds::new(left_side_partition_id, filters)); } } diff --git a/datafusion/physical-plan/src/joins/hash_join/stream.rs b/datafusion/physical-plan/src/joins/hash_join/stream.rs index afd985a86e48..5ce24e3addeb 100644 --- a/datafusion/physical-plan/src/joins/hash_join/stream.rs +++ b/datafusion/physical-plan/src/joins/hash_join/stream.rs @@ -419,15 +419,10 @@ impl HashJoinStream { PartitionMode::Auto => unreachable!("PartitionMode::Auto should not be present at execution time. This is a bug in DataFusion, please report it!"), }; - let left_data_bounds = left_data.bounds.clone(); - let left_data_bloom_filters = left_data.bloom_filters.clone(); + let column_filters = left_data.column_filters.clone(); self.bounds_waiter = Some(OnceFut::new(async move { bounds_accumulator - .report_partition_bounds( - left_side_partition_id, - left_data_bounds, - left_data_bloom_filters, - ) + .report_partition_bounds(left_side_partition_id, column_filters) .await })); self.state = HashJoinStreamState::WaitPartitionBoundsReport; From 7b020d86f065dcd876d38fbd968bea242aaded0a Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Mon, 27 Oct 2025 13:26:54 -0500 Subject: [PATCH 04/11] avoid cloning bloom filter --- datafusion/physical-expr/src/bloom_filter.rs | 5 +- .../src/expressions/bloom_filter_expr.rs | 127 ++++++++++++++---- .../physical-plan/src/joins/hash_join/exec.rs | 9 +- .../src/joins/hash_join/shared_bounds.rs | 118 ++++++++-------- .../src/joins/hash_join/stream.rs | 3 +- 5 files changed, 178 insertions(+), 84 deletions(-) diff --git a/datafusion/physical-expr/src/bloom_filter.rs b/datafusion/physical-expr/src/bloom_filter.rs index a15aadcf692f..b880f7db3b04 100644 --- a/datafusion/physical-expr/src/bloom_filter.rs +++ b/datafusion/physical-expr/src/bloom_filter.rs @@ -257,7 +257,10 @@ impl AsBytes for i128 { // Use big-endian for i128 to match Parquet's FIXED_LEN_BYTE_ARRAY representation // This allows compatibility with Parquet bloom filters unsafe { - std::slice::from_raw_parts(self as *const i128 as *const u8, size_of::()) + std::slice::from_raw_parts( + self as *const i128 as *const u8, + size_of::(), + ) } } } diff --git a/datafusion/physical-expr/src/expressions/bloom_filter_expr.rs b/datafusion/physical-expr/src/expressions/bloom_filter_expr.rs index 479f2be8761c..dac6fc775e1e 100644 --- a/datafusion/physical-expr/src/expressions/bloom_filter_expr.rs +++ b/datafusion/physical-expr/src/expressions/bloom_filter_expr.rs @@ -39,9 +39,9 @@ use std::sync::Arc; /// let mut builder = BloomFilterBuilder::new(1000, 0.01)?; /// builder.insert_scalar(&ScalarValue::Int32(Some(42)))?; /// builder.insert_array(&int_array)?; -/// let expr = builder.build(&col_expr); +/// let expr = builder.build(col_expr); // Consumes builder /// ``` -#[derive(Clone, Debug)] +#[derive(Debug)] pub struct BloomFilterBuilder { /// The underlying bloom filter sbbf: Sbbf, @@ -220,10 +220,7 @@ impl BloomFilterBuilder { } } DataType::Decimal128(_, _) => { - let array = array - .as_any() - .downcast_ref::() - .unwrap(); + let array = array.as_any().downcast_ref::().unwrap(); for i in 0..array.len() { if !array.is_null(i) { self.sbbf.insert(&array.value(i)); @@ -231,10 +228,7 @@ impl BloomFilterBuilder { } } DataType::Decimal256(_, _) => { - let array = array - .as_any() - .downcast_ref::() - .unwrap(); + let array = array.as_any().downcast_ref::().unwrap(); for i in 0..array.len() { if !array.is_null(i) { let bytes = array.value(i).to_be_bytes(); @@ -252,16 +246,15 @@ impl BloomFilterBuilder { Ok(()) } - /// Build a `BloomFilterExpr` from this builder + /// Build a `BloomFilterExpr` from this builder, consuming the builder. /// - /// This method does not consume the builder, allowing multiple - /// `BloomFilterExpr` instances to be created with different expressions - /// but sharing the same bloom filter data. + /// This consumes the builder and moves the bloom filter data into the expression, + /// avoiding any clones of the (potentially large) bloom filter. /// /// # Arguments /// * `expr` - The expression to evaluate and check against the bloom filter - pub fn build(&self, expr: Arc) -> BloomFilterExpr { - BloomFilterExpr::new(expr, self.sbbf.clone()) + pub fn build(self, expr: Arc) -> BloomFilterExpr { + BloomFilterExpr::new(expr, self.sbbf) } } @@ -480,10 +473,7 @@ impl BloomFilterExpr { } } DataType::Decimal128(_, _) => { - let array = array - .as_any() - .downcast_ref::() - .unwrap(); + let array = array.as_any().downcast_ref::().unwrap(); for i in 0..len { if array.is_null(i) { builder.append_value(false); @@ -493,10 +483,7 @@ impl BloomFilterExpr { } } DataType::Decimal256(_, _) => { - let array = array - .as_any() - .downcast_ref::() - .unwrap(); + let array = array.as_any().downcast_ref::().unwrap(); for i in 0..len { if array.is_null(i) { builder.append_value(false); @@ -727,4 +714,96 @@ mod tests { Ok(()) } + + #[test] + fn test_bloom_filter_negative_lookups() -> Result<()> { + use arrow::array::{Float64Array, Int32Array, StringArray}; + + // Test Int32: Use extremely low FPP (0.00001) to make false positives negligible + let mut builder = BloomFilterBuilder::new(10, 0.00001)?; + for i in 1..=10 { + builder.insert_scalar(&ScalarValue::Int32(Some(i)))?; + } + let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int32, false)])); + let expr = col("a", &schema)?; + let bloom_expr = Arc::new(builder.build(expr)); + + // Test values far outside the inserted range (1000-1099) + let test_values: Vec = (1000..1100).collect(); + let test_array = Arc::new(Int32Array::from(test_values)) as ArrayRef; + let batch = RecordBatch::try_new(Arc::clone(&schema), vec![test_array])?; + let result = bloom_expr.evaluate(&batch)?; + let result_array = result.into_array(100)?; + let result_bool = result_array + .as_any() + .downcast_ref::() + .unwrap(); + for i in 0..100 { + assert!( + !result_bool.value(i), + "Int32 value {} should not match", + i + 1000 + ); + } + + // Test Float64 + let mut builder = BloomFilterBuilder::new(10, 0.00001)?; + for i in 0..10 { + builder.insert_scalar(&ScalarValue::Float64(Some(i as f64 * 0.5)))?; + } + let schema = + Arc::new(Schema::new(vec![Field::new("f", DataType::Float64, false)])); + let expr = col("f", &schema)?; + let bloom_expr = Arc::new(builder.build(expr)); + + let test_values: Vec = (100..200).map(|i| i as f64 * 10.0).collect(); + let test_array = Arc::new(Float64Array::from(test_values)) as ArrayRef; + let batch = RecordBatch::try_new(Arc::clone(&schema), vec![test_array])?; + let result = bloom_expr.evaluate(&batch)?; + let result_array = result.into_array(100)?; + let result_bool = result_array + .as_any() + .downcast_ref::() + .unwrap(); + for i in 0..100 { + assert!( + !result_bool.value(i), + "Float64 value {} should not match", + (i + 100) as f64 * 10.0 + ); + } + + // Test Strings + let mut builder = BloomFilterBuilder::new(5, 0.00001)?; + builder.insert_scalar(&ScalarValue::Utf8(Some("apple".to_string())))?; + builder.insert_scalar(&ScalarValue::Utf8(Some("banana".to_string())))?; + builder.insert_scalar(&ScalarValue::Utf8(Some("cherry".to_string())))?; + let schema = Arc::new(Schema::new(vec![Field::new("s", DataType::Utf8, false)])); + let expr = col("s", &schema)?; + let bloom_expr = Arc::new(builder.build(expr)); + + let test_strings = vec![ + "zebra", + "yak", + "xylophone", + "walrus", + "vulture", + "umbrella", + "tiger", + "snake", + ]; + let test_array = Arc::new(StringArray::from(test_strings.clone())) as ArrayRef; + let batch = RecordBatch::try_new(Arc::clone(&schema), vec![test_array])?; + let result = bloom_expr.evaluate(&batch)?; + let result_array = result.into_array(8)?; + let result_bool = result_array + .as_any() + .downcast_ref::() + .unwrap(); + for (i, s) in test_strings.iter().enumerate() { + assert!(!result_bool.value(i), "String '{}' should not match", s); + } + + Ok(()) + } } diff --git a/datafusion/physical-plan/src/joins/hash_join/exec.rs b/datafusion/physical-plan/src/joins/hash_join/exec.rs index 46794fd989af..b77cc52bb1a3 100644 --- a/datafusion/physical-plan/src/joins/hash_join/exec.rs +++ b/datafusion/physical-plan/src/joins/hash_join/exec.rs @@ -55,6 +55,7 @@ use crate::{ DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, Partitioning, PlanProperties, SendableRecordBatchStream, Statistics, }; +use parking_lot::Mutex; use arrow::array::{ArrayRef, BooleanBufferBuilder}; use arrow::compute::concat_batches; @@ -82,7 +83,6 @@ use datafusion_physical_expr::{PhysicalExpr, PhysicalExprRef}; use ahash::RandomState; use datafusion_physical_expr_common::physical_expr::fmt_sql; 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 = @@ -106,8 +106,9 @@ pub(super) struct JoinLeftData { /// This could hide potential out-of-memory issues, especially when upstream operators increase their memory consumption. /// The MemoryReservation ensures proper tracking of memory resources throughout the join operation's lifecycle. _reservation: MemoryReservation, - /// Filter data (bounds + bloom filters) computed from the build side for dynamic filter pushdown - pub(super) column_filters: Option>, + /// Filter data (bounds + bloom filters) computed from the build side for dynamic filter pushdown. + /// Wrapped in Mutex> to allow taking ownership when reporting to the accumulator. + pub(super) column_filters: Mutex>>, } impl JoinLeftData { @@ -128,7 +129,7 @@ impl JoinLeftData { visited_indices_bitmap, probe_threads_counter, _reservation: reservation, - column_filters, + column_filters: Mutex::new(column_filters), } } diff --git a/datafusion/physical-plan/src/joins/hash_join/shared_bounds.rs b/datafusion/physical-plan/src/joins/hash_join/shared_bounds.rs index d97c4c8b4534..6fc5cff63310 100644 --- a/datafusion/physical-plan/src/joins/hash_join/shared_bounds.rs +++ b/datafusion/physical-plan/src/joins/hash_join/shared_bounds.rs @@ -32,7 +32,6 @@ use datafusion_physical_expr::expressions::{ }; use datafusion_physical_expr::{PhysicalExpr, PhysicalExprRef}; -use itertools::Itertools; use parking_lot::Mutex; use tokio::sync::Barrier; @@ -54,7 +53,7 @@ impl ColumnBounds { /// Filter data for a single join key column, combining bounds and bloom filter. /// Used in dynamic filter pushdown for comprehensive filtering. -#[derive(Debug, Clone)] +#[derive(Debug)] pub(crate) struct ColumnFilterData { /// Min/max bounds for range filtering pub(crate) bounds: ColumnBounds, @@ -73,13 +72,13 @@ impl ColumnFilterData { /// Represents the bounds for all join key columns from a single partition. /// This contains the filter data (min/max values and bloom filters) computed from one partition's build-side data. -#[derive(Debug, Clone)] +#[derive(Debug)] pub(crate) struct PartitionBounds { /// Partition identifier for debugging and determinism (not strictly necessary) - partition: usize, + pub(crate) partition: usize, /// Filter data (bounds + bloom filter) for each join key column in this partition. /// Index corresponds to the join key expression index. - column_filters: Vec, + pub(crate) column_filters: Vec, } impl PartitionBounds { @@ -89,14 +88,6 @@ impl PartitionBounds { column_filters, } } - - pub(crate) fn len(&self) -> usize { - self.column_filters.len() - } - - pub(crate) fn get_column_filter(&self, index: usize) -> Option<&ColumnFilterData> { - self.column_filters.get(index) - } } /// Coordinates dynamic filter bounds collection across multiple partitions @@ -123,8 +114,9 @@ impl PartitionBounds { /// All fields use a single mutex to ensure correct coordination between concurrent /// partition executions. pub(crate) struct SharedBoundsAccumulator { - /// Shared state protected by a single mutex to avoid ordering concerns - inner: Mutex, + /// Shared state protected by a single mutex to avoid ordering concerns. + /// After filter creation, this becomes None as the state is consumed. + inner: Mutex>, barrier: Barrier, /// Dynamic filter for pushdown to probe side dynamic_filter: Arc, @@ -187,9 +179,9 @@ impl SharedBoundsAccumulator { PartitionMode::Auto => unreachable!("PartitionMode::Auto should not be present at execution time. This is a bug in DataFusion, please report it!"), }; Self { - inner: Mutex::new(SharedBoundsState { + inner: Mutex::new(Some(SharedBoundsState { bounds: Vec::with_capacity(expected_calls), - }), + })), barrier: Barrier::new(expected_calls), dynamic_filter, on_right, @@ -205,49 +197,58 @@ impl SharedBoundsAccumulator { /// ((col0 >= p0_min0 AND col0 <= p0_max0 AND col0 IN BLOOM_FILTER_0 AND col1 >= p0_min1 AND col1 <= p0_max1 AND col1 IN BLOOM_FILTER_1) /// OR /// (col0 >= p1_min0 AND col0 <= p1_max0 AND col0 IN BLOOM_FILTER_0 AND col1 >= p1_min1 AND col1 <= p1_max1 AND col1 IN BLOOM_FILTER_1)) + /// + /// This method consumes the bounds to allow moving bloom filter builders. pub(crate) fn create_filter_from_partition_bounds( &self, - bounds: &[PartitionBounds], + mut bounds: Vec, ) -> Result> { if bounds.is_empty() { return Ok(lit(true)); } + // Sort by partition for determinism + bounds.sort_by_key(|b| b.partition); + // Create a predicate for each partition let mut partition_predicates = Vec::with_capacity(bounds.len()); - for partition_bounds in bounds.iter().sorted_by_key(|b| b.partition) { + for partition_bounds in bounds.into_iter() { // Create range predicates and bloom filter checks for each join key in this partition - let mut column_predicates = Vec::with_capacity(partition_bounds.len()); - - for (col_idx, right_expr) in self.on_right.iter().enumerate() { - if let Some(filter_data) = partition_bounds.get_column_filter(col_idx) { - // Create predicate: col >= min AND col <= max - let min_expr = Arc::new(BinaryExpr::new( - Arc::clone(right_expr), - Operator::GtEq, - lit(filter_data.bounds.min.clone()), - )) as Arc; - let max_expr = Arc::new(BinaryExpr::new( - Arc::clone(right_expr), - Operator::LtEq, - lit(filter_data.bounds.max.clone()), - )) as Arc; - let range_expr = - Arc::new(BinaryExpr::new(min_expr, Operator::And, max_expr)) - as Arc; - - // Create bloom filter check: col IN BLOOM_FILTER - let bloom_expr = Arc::new(filter_data.bloom_filter.build(Arc::clone( - right_expr, - ))) as Arc; - - // Combine range and bloom filter: (range_expr AND bloom_expr) - let combined_expr = - Arc::new(BinaryExpr::new(range_expr, Operator::And, bloom_expr)) - as Arc; - column_predicates.push(combined_expr); - } + let mut column_predicates = + Vec::with_capacity(partition_bounds.column_filters.len()); + + // Consume column_filters by taking ownership + for (col_idx, filter_data) in + partition_bounds.column_filters.into_iter().enumerate() + { + let right_expr = &self.on_right[col_idx]; + + // Create predicate: col >= min AND col <= max + let min_expr = Arc::new(BinaryExpr::new( + Arc::clone(right_expr), + Operator::GtEq, + lit(filter_data.bounds.min.clone()), + )) as Arc; + let max_expr = Arc::new(BinaryExpr::new( + Arc::clone(right_expr), + Operator::LtEq, + lit(filter_data.bounds.max.clone()), + )) as Arc; + let range_expr = + Arc::new(BinaryExpr::new(min_expr, Operator::And, max_expr)) + as Arc; + + // Create bloom filter check by consuming the builder + let bloom_expr = + Arc::new(filter_data.bloom_filter.build(Arc::clone(right_expr))) + as Arc; + + // Combine range and bloom filter: (range_expr AND bloom_expr) + let combined_expr = + Arc::new(BinaryExpr::new(range_expr, Operator::And, bloom_expr)) + as Arc; + column_predicates.push(combined_expr); } // Combine all column predicates for this partition with AND @@ -306,8 +307,11 @@ impl SharedBoundsAccumulator { // Store filter data in the accumulator - this runs once per partition if let Some(filters) = column_filters { let mut guard = self.inner.lock(); + let state = guard + .as_mut() + .expect("SharedBoundsState should exist during partition reporting"); - let should_push = if let Some(last_bound) = guard.bounds.last() { + let should_push = if let Some(last_bound) = state.bounds.last() { // In `PartitionMode::CollectLeft`, all streams on the left side share the same partition id (0). // Since this function can be called multiple times for that same partition, we must deduplicate // by checking against the last recorded bound. @@ -317,7 +321,7 @@ impl SharedBoundsAccumulator { }; if should_push { - guard + state .bounds .push(PartitionBounds::new(left_side_partition_id, filters)); } @@ -325,10 +329,16 @@ impl SharedBoundsAccumulator { if self.barrier.wait().await.is_leader() { // All partitions have reported, so we can update the filter - let inner = self.inner.lock(); - if !inner.bounds.is_empty() { + // Take ownership of the state (consuming it) + let state = self + .inner + .lock() + .take() + .expect("SharedBoundsState should exist when creating filter"); + + if !state.bounds.is_empty() { let filter_expr = - self.create_filter_from_partition_bounds(&inner.bounds)?; + self.create_filter_from_partition_bounds(state.bounds)?; self.dynamic_filter.update(filter_expr)?; } } diff --git a/datafusion/physical-plan/src/joins/hash_join/stream.rs b/datafusion/physical-plan/src/joins/hash_join/stream.rs index 5ce24e3addeb..8fed06d0ffd2 100644 --- a/datafusion/physical-plan/src/joins/hash_join/stream.rs +++ b/datafusion/physical-plan/src/joins/hash_join/stream.rs @@ -419,7 +419,8 @@ impl HashJoinStream { PartitionMode::Auto => unreachable!("PartitionMode::Auto should not be present at execution time. This is a bug in DataFusion, please report it!"), }; - let column_filters = left_data.column_filters.clone(); + // Take ownership of column_filters to avoid cloning + let column_filters = left_data.column_filters.lock().take(); self.bounds_waiter = Some(OnceFut::new(async move { bounds_accumulator .report_partition_bounds(left_side_partition_id, column_filters) From 7ce0f5bfe47b28b3c3c959c21659efb691997bf9 Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Mon, 27 Oct 2025 14:06:11 -0500 Subject: [PATCH 05/11] tweak sizes --- datafusion/physical-plan/src/joins/hash_join/exec.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion/physical-plan/src/joins/hash_join/exec.rs b/datafusion/physical-plan/src/joins/hash_join/exec.rs index b77cc52bb1a3..edee3b741a7c 100644 --- a/datafusion/physical-plan/src/joins/hash_join/exec.rs +++ b/datafusion/physical-plan/src/joins/hash_join/exec.rs @@ -1258,8 +1258,8 @@ impl CollectLeftAccumulator { .map(|dt| dictionary_value_type(&dt))?; // Create bloom filter with default parameters - // NDV (number of distinct values) = 10000, FPP (false positive probability) = 0.01 (1%) - let bloom_filter = BloomFilterBuilder::new(10000, 0.01)?; + // NDV (number of distinct values) = 1000, FPP (false positive probability) = 0.05 (5%) + let bloom_filter = BloomFilterBuilder::new(1000, 0.05)?; Ok(Self { expr, From 7ad4b5abe33213f813e083ef4a072b4bd486fc3d Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Mon, 27 Oct 2025 14:44:55 -0500 Subject: [PATCH 06/11] support more data types in bloom filter, fix snaps --- .../physical_optimizer/filter_pushdown/mod.rs | 2 +- datafusion/physical-expr/src/bloom_filter.rs | 30 +++ .../src/expressions/bloom_filter_expr.rs | 220 +++++++++++++++++- 3 files changed, 250 insertions(+), 2 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs b/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs index 3e204cb68ad3..3ec878882b02 100644 --- a/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs +++ b/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs @@ -1326,7 +1326,7 @@ async fn test_hashjoin_dynamic_filter_pushdown_partitioned() { - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([a@0, b@1], 12), input_partitions=1 - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate=DynamicFilter [ a@0 >= aa AND a@0 <= ab AND b@1 >= ba AND b@1 <= bb ] + - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate=DynamicFilter [ a@0 >= aa AND a@0 <= ab AND a@0 IN BLOOM_FILTER AND b@1 >= ba AND b@1 <= bb AND b@1 IN BLOOM_FILTER ] " ); diff --git a/datafusion/physical-expr/src/bloom_filter.rs b/datafusion/physical-expr/src/bloom_filter.rs index b880f7db3b04..3512f242a72b 100644 --- a/datafusion/physical-expr/src/bloom_filter.rs +++ b/datafusion/physical-expr/src/bloom_filter.rs @@ -204,6 +204,22 @@ impl AsBytes for bool { } } +impl AsBytes for i8 { + fn as_bytes(&self) -> &[u8] { + unsafe { + std::slice::from_raw_parts(self as *const i8 as *const u8, size_of::()) + } + } +} + +impl AsBytes for i16 { + fn as_bytes(&self) -> &[u8] { + unsafe { + std::slice::from_raw_parts(self as *const i16 as *const u8, size_of::()) + } + } +} + impl AsBytes for i32 { fn as_bytes(&self) -> &[u8] { unsafe { @@ -220,6 +236,20 @@ impl AsBytes for i64 { } } +impl AsBytes for u8 { + fn as_bytes(&self) -> &[u8] { + unsafe { std::slice::from_raw_parts(self as *const u8, size_of::()) } + } +} + +impl AsBytes for u16 { + fn as_bytes(&self) -> &[u8] { + unsafe { + std::slice::from_raw_parts(self as *const u16 as *const u8, size_of::()) + } + } +} + impl AsBytes for u32 { fn as_bytes(&self) -> &[u8] { unsafe { diff --git a/datafusion/physical-expr/src/expressions/bloom_filter_expr.rs b/datafusion/physical-expr/src/expressions/bloom_filter_expr.rs index dac6fc775e1e..984d2d6bacc0 100644 --- a/datafusion/physical-expr/src/expressions/bloom_filter_expr.rs +++ b/datafusion/physical-expr/src/expressions/bloom_filter_expr.rs @@ -66,8 +66,12 @@ impl BloomFilterBuilder { match value { ScalarValue::Boolean(Some(v)) => self.sbbf.insert(v), + ScalarValue::Int8(Some(v)) => self.sbbf.insert(v), + ScalarValue::Int16(Some(v)) => self.sbbf.insert(v), ScalarValue::Int32(Some(v)) => self.sbbf.insert(v), ScalarValue::Int64(Some(v)) => self.sbbf.insert(v), + ScalarValue::UInt8(Some(v)) => self.sbbf.insert(v), + ScalarValue::UInt16(Some(v)) => self.sbbf.insert(v), ScalarValue::UInt32(Some(v)) => self.sbbf.insert(v), ScalarValue::UInt64(Some(v)) => self.sbbf.insert(v), ScalarValue::Float32(Some(v)) => self.sbbf.insert(v), @@ -88,6 +92,12 @@ impl BloomFilterBuilder { let bytes = v.to_be_bytes(); self.sbbf.insert(&bytes) } + ScalarValue::Date32(Some(v)) => self.sbbf.insert(v), + ScalarValue::Date64(Some(v)) => self.sbbf.insert(v), + ScalarValue::TimestampSecond(Some(v), _) + | ScalarValue::TimestampMillisecond(Some(v), _) + | ScalarValue::TimestampMicrosecond(Some(v), _) + | ScalarValue::TimestampNanosecond(Some(v), _) => self.sbbf.insert(v), _ => { return exec_err!( "Unsupported data type for bloom filter: {}", @@ -112,6 +122,22 @@ impl BloomFilterBuilder { } } } + DataType::Int8 => { + let array = array.as_any().downcast_ref::().unwrap(); + for i in 0..array.len() { + if !array.is_null(i) { + self.sbbf.insert(&array.value(i)); + } + } + } + DataType::Int16 => { + let array = array.as_any().downcast_ref::().unwrap(); + for i in 0..array.len() { + if !array.is_null(i) { + self.sbbf.insert(&array.value(i)); + } + } + } DataType::Int32 => { let array = array.as_any().downcast_ref::().unwrap(); for i in 0..array.len() { @@ -128,6 +154,22 @@ impl BloomFilterBuilder { } } } + DataType::UInt8 => { + let array = array.as_any().downcast_ref::().unwrap(); + for i in 0..array.len() { + if !array.is_null(i) { + self.sbbf.insert(&array.value(i)); + } + } + } + DataType::UInt16 => { + let array = array.as_any().downcast_ref::().unwrap(); + for i in 0..array.len() { + if !array.is_null(i) { + self.sbbf.insert(&array.value(i)); + } + } + } DataType::UInt32 => { let array = array.as_any().downcast_ref::().unwrap(); for i in 0..array.len() { @@ -236,6 +278,59 @@ impl BloomFilterBuilder { } } } + DataType::Date32 => { + let array = array.as_any().downcast_ref::().unwrap(); + for i in 0..array.len() { + if !array.is_null(i) { + self.sbbf.insert(&array.value(i)); + } + } + } + DataType::Date64 => { + let array = array.as_any().downcast_ref::().unwrap(); + for i in 0..array.len() { + if !array.is_null(i) { + self.sbbf.insert(&array.value(i)); + } + } + } + DataType::Timestamp(_, _) => { + // All timestamp types store i64 values internally + // Try each timestamp array type + if let Some(ts_array) = + array.as_any().downcast_ref::() + { + for i in 0..ts_array.len() { + if !ts_array.is_null(i) { + self.sbbf.insert(&ts_array.value(i)); + } + } + } else if let Some(ts_array) = + array.as_any().downcast_ref::() + { + for i in 0..ts_array.len() { + if !ts_array.is_null(i) { + self.sbbf.insert(&ts_array.value(i)); + } + } + } else if let Some(ts_array) = + array.as_any().downcast_ref::() + { + for i in 0..ts_array.len() { + if !ts_array.is_null(i) { + self.sbbf.insert(&ts_array.value(i)); + } + } + } else if let Some(ts_array) = + array.as_any().downcast_ref::() + { + for i in 0..ts_array.len() { + if !ts_array.is_null(i) { + self.sbbf.insert(&ts_array.value(i)); + } + } + } + } _ => { return exec_err!( "Unsupported data type for bloom filter: {}", @@ -293,8 +388,12 @@ impl BloomFilterExpr { match value { ScalarValue::Boolean(Some(v)) => self.bloom_filter.check(v), + ScalarValue::Int8(Some(v)) => self.bloom_filter.check(v), + ScalarValue::Int16(Some(v)) => self.bloom_filter.check(v), ScalarValue::Int32(Some(v)) => self.bloom_filter.check(v), ScalarValue::Int64(Some(v)) => self.bloom_filter.check(v), + ScalarValue::UInt8(Some(v)) => self.bloom_filter.check(v), + ScalarValue::UInt16(Some(v)) => self.bloom_filter.check(v), ScalarValue::UInt32(Some(v)) => self.bloom_filter.check(v), ScalarValue::UInt64(Some(v)) => self.bloom_filter.check(v), ScalarValue::Float32(Some(v)) => self.bloom_filter.check(v), @@ -316,6 +415,12 @@ impl BloomFilterExpr { let bytes = v.to_be_bytes(); self.bloom_filter.check(&bytes) } + ScalarValue::Date32(Some(v)) => self.bloom_filter.check(v), + ScalarValue::Date64(Some(v)) => self.bloom_filter.check(v), + ScalarValue::TimestampSecond(Some(v), _) + | ScalarValue::TimestampMillisecond(Some(v), _) + | ScalarValue::TimestampMicrosecond(Some(v), _) + | ScalarValue::TimestampNanosecond(Some(v), _) => self.bloom_filter.check(v), _ => true, // Unsupported types default to "might be present" } } @@ -339,6 +444,26 @@ impl BloomFilterExpr { } } } + DataType::Int8 => { + let array = array.as_any().downcast_ref::().unwrap(); + for i in 0..len { + if array.is_null(i) { + builder.append_value(false); + } else { + builder.append_value(self.bloom_filter.check(&array.value(i))); + } + } + } + DataType::Int16 => { + let array = array.as_any().downcast_ref::().unwrap(); + for i in 0..len { + if array.is_null(i) { + builder.append_value(false); + } else { + builder.append_value(self.bloom_filter.check(&array.value(i))); + } + } + } DataType::Int32 => { let array = array.as_any().downcast_ref::().unwrap(); for i in 0..len { @@ -359,6 +484,26 @@ impl BloomFilterExpr { } } } + DataType::UInt8 => { + let array = array.as_any().downcast_ref::().unwrap(); + for i in 0..len { + if array.is_null(i) { + builder.append_value(false); + } else { + builder.append_value(self.bloom_filter.check(&array.value(i))); + } + } + } + DataType::UInt16 => { + let array = array.as_any().downcast_ref::().unwrap(); + for i in 0..len { + if array.is_null(i) { + builder.append_value(false); + } else { + builder.append_value(self.bloom_filter.check(&array.value(i))); + } + } + } DataType::UInt32 => { let array = array.as_any().downcast_ref::().unwrap(); for i in 0..len { @@ -493,6 +638,79 @@ impl BloomFilterExpr { } } } + DataType::Date32 => { + let array = array.as_any().downcast_ref::().unwrap(); + for i in 0..len { + if array.is_null(i) { + builder.append_value(false); + } else { + builder.append_value(self.bloom_filter.check(&array.value(i))); + } + } + } + DataType::Date64 => { + let array = array.as_any().downcast_ref::().unwrap(); + for i in 0..len { + if array.is_null(i) { + builder.append_value(false); + } else { + builder.append_value(self.bloom_filter.check(&array.value(i))); + } + } + } + DataType::Timestamp(_, _) => { + // All timestamp types store i64 values internally + // Try each timestamp array type + if let Some(ts_array) = + array.as_any().downcast_ref::() + { + for i in 0..len { + if ts_array.is_null(i) { + builder.append_value(false); + } else { + builder.append_value( + self.bloom_filter.check(&ts_array.value(i)), + ); + } + } + } else if let Some(ts_array) = + array.as_any().downcast_ref::() + { + for i in 0..len { + if ts_array.is_null(i) { + builder.append_value(false); + } else { + builder.append_value( + self.bloom_filter.check(&ts_array.value(i)), + ); + } + } + } else if let Some(ts_array) = + array.as_any().downcast_ref::() + { + for i in 0..len { + if ts_array.is_null(i) { + builder.append_value(false); + } else { + builder.append_value( + self.bloom_filter.check(&ts_array.value(i)), + ); + } + } + } else if let Some(ts_array) = + array.as_any().downcast_ref::() + { + for i in 0..len { + if ts_array.is_null(i) { + builder.append_value(false); + } else { + builder.append_value( + self.bloom_filter.check(&ts_array.value(i)), + ); + } + } + } + } _ => { return internal_err!( "Unsupported data type for bloom filter check: {}", @@ -801,7 +1019,7 @@ mod tests { .downcast_ref::() .unwrap(); for (i, s) in test_strings.iter().enumerate() { - assert!(!result_bool.value(i), "String '{}' should not match", s); + assert!(!result_bool.value(i), "String '{s}' should not match"); } Ok(()) From b7fb68e859424ea5f958f967e3ee7fbf881be5ac Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Mon, 27 Oct 2025 15:05:28 -0500 Subject: [PATCH 07/11] fix ci --- datafusion/physical-expr/src/bloom_filter.rs | 9 +- .../src/expressions/bloom_filter_expr.rs | 607 ++---------------- 2 files changed, 40 insertions(+), 576 deletions(-) diff --git a/datafusion/physical-expr/src/bloom_filter.rs b/datafusion/physical-expr/src/bloom_filter.rs index 3512f242a72b..0f833b0f6b2d 100644 --- a/datafusion/physical-expr/src/bloom_filter.rs +++ b/datafusion/physical-expr/src/bloom_filter.rs @@ -151,13 +151,8 @@ impl Sbbf { (((hash >> 32).saturating_mul(self.0.len() as u64)) >> 32) as usize } - /// Insert a value into the filter - pub fn insert(&mut self, value: &T) { - self.insert_hash(hash_as_bytes(value)); - } - /// Insert a hash into the filter - fn insert_hash(&mut self, hash: u64) { + pub(crate) fn insert_hash(&mut self, hash: u64) { let block_index = self.hash_to_block_index(hash); self.0[block_index].insert(hash as u32) } @@ -170,7 +165,7 @@ impl Sbbf { /// Check if a hash is in the filter. May return /// true for values that were never inserted ("false positive") /// but will always return false if a hash has not been inserted. - fn check_hash(&self, hash: u64) -> bool { + pub(crate) fn check_hash(&self, hash: u64) -> bool { let block_index = self.hash_to_block_index(hash); self.0[block_index].check(hash as u32) } diff --git a/datafusion/physical-expr/src/expressions/bloom_filter_expr.rs b/datafusion/physical-expr/src/expressions/bloom_filter_expr.rs index 984d2d6bacc0..9603403906bc 100644 --- a/datafusion/physical-expr/src/expressions/bloom_filter_expr.rs +++ b/datafusion/physical-expr/src/expressions/bloom_filter_expr.rs @@ -19,10 +19,12 @@ use crate::bloom_filter::Sbbf; use crate::PhysicalExpr; +use ahash::RandomState; use arrow::array::{ArrayRef, BooleanArray}; use arrow::datatypes::{DataType, Schema}; use arrow::record_batch::RecordBatch; -use datafusion_common::{exec_err, internal_err, Result, ScalarValue}; +use datafusion_common::hash_utils::create_hashes; +use datafusion_common::{internal_err, Result, ScalarValue}; use datafusion_expr_common::columnar_value::ColumnarValue; use std::any::Any; use std::fmt; @@ -58,286 +60,19 @@ impl BloomFilterBuilder { Ok(Self { sbbf }) } - /// Insert a single scalar value into the bloom filter - pub fn insert_scalar(&mut self, value: &ScalarValue) -> Result<()> { - if value.is_null() { - return Ok(()); - } - - match value { - ScalarValue::Boolean(Some(v)) => self.sbbf.insert(v), - ScalarValue::Int8(Some(v)) => self.sbbf.insert(v), - ScalarValue::Int16(Some(v)) => self.sbbf.insert(v), - ScalarValue::Int32(Some(v)) => self.sbbf.insert(v), - ScalarValue::Int64(Some(v)) => self.sbbf.insert(v), - ScalarValue::UInt8(Some(v)) => self.sbbf.insert(v), - ScalarValue::UInt16(Some(v)) => self.sbbf.insert(v), - ScalarValue::UInt32(Some(v)) => self.sbbf.insert(v), - ScalarValue::UInt64(Some(v)) => self.sbbf.insert(v), - ScalarValue::Float32(Some(v)) => self.sbbf.insert(v), - ScalarValue::Float64(Some(v)) => self.sbbf.insert(v), - ScalarValue::Utf8(Some(v)) | ScalarValue::LargeUtf8(Some(v)) => { - self.sbbf.insert(v.as_str()) - } - ScalarValue::Utf8View(Some(v)) => self.sbbf.insert(v.as_str()), - ScalarValue::Binary(Some(v)) - | ScalarValue::LargeBinary(Some(v)) - | ScalarValue::FixedSizeBinary(_, Some(v)) => self.sbbf.insert(v.as_slice()), - ScalarValue::BinaryView(Some(v)) => self.sbbf.insert(v.as_slice()), - ScalarValue::Decimal32(Some(v), _, _) => self.sbbf.insert(v), - ScalarValue::Decimal64(Some(v), _, _) => self.sbbf.insert(v), - ScalarValue::Decimal128(Some(v), _, _) => self.sbbf.insert(v), - ScalarValue::Decimal256(Some(v), _, _) => { - // Convert i256 to bytes - let bytes = v.to_be_bytes(); - self.sbbf.insert(&bytes) - } - ScalarValue::Date32(Some(v)) => self.sbbf.insert(v), - ScalarValue::Date64(Some(v)) => self.sbbf.insert(v), - ScalarValue::TimestampSecond(Some(v), _) - | ScalarValue::TimestampMillisecond(Some(v), _) - | ScalarValue::TimestampMicrosecond(Some(v), _) - | ScalarValue::TimestampNanosecond(Some(v), _) => self.sbbf.insert(v), - _ => { - return exec_err!( - "Unsupported data type for bloom filter: {}", - value.data_type() - ) - } - } - Ok(()) - } - - /// Insert all non-null values from an array into the bloom filter + /// Insert all values from an array into the bloom filter pub fn insert_array(&mut self, array: &ArrayRef) -> Result<()> { - use arrow::array::*; - use arrow::datatypes::DataType; - - match array.data_type() { - DataType::Boolean => { - let array = array.as_any().downcast_ref::().unwrap(); - for i in 0..array.len() { - if !array.is_null(i) { - self.sbbf.insert(&array.value(i)); - } - } - } - DataType::Int8 => { - let array = array.as_any().downcast_ref::().unwrap(); - for i in 0..array.len() { - if !array.is_null(i) { - self.sbbf.insert(&array.value(i)); - } - } - } - DataType::Int16 => { - let array = array.as_any().downcast_ref::().unwrap(); - for i in 0..array.len() { - if !array.is_null(i) { - self.sbbf.insert(&array.value(i)); - } - } - } - DataType::Int32 => { - let array = array.as_any().downcast_ref::().unwrap(); - for i in 0..array.len() { - if !array.is_null(i) { - self.sbbf.insert(&array.value(i)); - } - } - } - DataType::Int64 => { - let array = array.as_any().downcast_ref::().unwrap(); - for i in 0..array.len() { - if !array.is_null(i) { - self.sbbf.insert(&array.value(i)); - } - } - } - DataType::UInt8 => { - let array = array.as_any().downcast_ref::().unwrap(); - for i in 0..array.len() { - if !array.is_null(i) { - self.sbbf.insert(&array.value(i)); - } - } - } - DataType::UInt16 => { - let array = array.as_any().downcast_ref::().unwrap(); - for i in 0..array.len() { - if !array.is_null(i) { - self.sbbf.insert(&array.value(i)); - } - } - } - DataType::UInt32 => { - let array = array.as_any().downcast_ref::().unwrap(); - for i in 0..array.len() { - if !array.is_null(i) { - self.sbbf.insert(&array.value(i)); - } - } - } - DataType::UInt64 => { - let array = array.as_any().downcast_ref::().unwrap(); - for i in 0..array.len() { - if !array.is_null(i) { - self.sbbf.insert(&array.value(i)); - } - } - } - DataType::Float32 => { - let array = array.as_any().downcast_ref::().unwrap(); - for i in 0..array.len() { - if !array.is_null(i) { - self.sbbf.insert(&array.value(i)); - } - } - } - DataType::Float64 => { - let array = array.as_any().downcast_ref::().unwrap(); - for i in 0..array.len() { - if !array.is_null(i) { - self.sbbf.insert(&array.value(i)); - } - } - } - DataType::Utf8 => { - let array = array.as_any().downcast_ref::().unwrap(); - for i in 0..array.len() { - if !array.is_null(i) { - self.sbbf.insert(array.value(i)); - } - } - } - DataType::LargeUtf8 => { - let array = array.as_any().downcast_ref::().unwrap(); - for i in 0..array.len() { - if !array.is_null(i) { - self.sbbf.insert(array.value(i)); - } - } - } - DataType::Utf8View => { - let array = array.as_any().downcast_ref::().unwrap(); - for i in 0..array.len() { - if !array.is_null(i) { - self.sbbf.insert(array.value(i)); - } - } - } - DataType::Binary => { - let array = array.as_any().downcast_ref::().unwrap(); - for i in 0..array.len() { - if !array.is_null(i) { - self.sbbf.insert(array.value(i)); - } - } - } - DataType::LargeBinary => { - let array = array.as_any().downcast_ref::().unwrap(); - for i in 0..array.len() { - if !array.is_null(i) { - self.sbbf.insert(array.value(i)); - } - } - } - DataType::BinaryView => { - let array = array.as_any().downcast_ref::().unwrap(); - for i in 0..array.len() { - if !array.is_null(i) { - self.sbbf.insert(array.value(i)); - } - } - } - DataType::FixedSizeBinary(_) => { - let array = array - .as_any() - .downcast_ref::() - .unwrap(); - for i in 0..array.len() { - if !array.is_null(i) { - self.sbbf.insert(array.value(i)); - } - } - } - DataType::Decimal128(_, _) => { - let array = array.as_any().downcast_ref::().unwrap(); - for i in 0..array.len() { - if !array.is_null(i) { - self.sbbf.insert(&array.value(i)); - } - } - } - DataType::Decimal256(_, _) => { - let array = array.as_any().downcast_ref::().unwrap(); - for i in 0..array.len() { - if !array.is_null(i) { - let bytes = array.value(i).to_be_bytes(); - self.sbbf.insert(&bytes); - } - } - } - DataType::Date32 => { - let array = array.as_any().downcast_ref::().unwrap(); - for i in 0..array.len() { - if !array.is_null(i) { - self.sbbf.insert(&array.value(i)); - } - } - } - DataType::Date64 => { - let array = array.as_any().downcast_ref::().unwrap(); - for i in 0..array.len() { - if !array.is_null(i) { - self.sbbf.insert(&array.value(i)); - } - } - } - DataType::Timestamp(_, _) => { - // All timestamp types store i64 values internally - // Try each timestamp array type - if let Some(ts_array) = - array.as_any().downcast_ref::() - { - for i in 0..ts_array.len() { - if !ts_array.is_null(i) { - self.sbbf.insert(&ts_array.value(i)); - } - } - } else if let Some(ts_array) = - array.as_any().downcast_ref::() - { - for i in 0..ts_array.len() { - if !ts_array.is_null(i) { - self.sbbf.insert(&ts_array.value(i)); - } - } - } else if let Some(ts_array) = - array.as_any().downcast_ref::() - { - for i in 0..ts_array.len() { - if !ts_array.is_null(i) { - self.sbbf.insert(&ts_array.value(i)); - } - } - } else if let Some(ts_array) = - array.as_any().downcast_ref::() - { - for i in 0..ts_array.len() { - if !ts_array.is_null(i) { - self.sbbf.insert(&ts_array.value(i)); - } - } - } - } - _ => { - return exec_err!( - "Unsupported data type for bloom filter: {}", - array.data_type() - ) - } + // Use create_hashes to compute hash values for all array types + // This handles Dictionary, Struct, Null, and all other types uniformly + let mut hashes = vec![0u64; array.len()]; + let random_state = RandomState::with_seeds(0, 0, 0, 0); + create_hashes(&[Arc::clone(array)], &random_state, &mut hashes)?; + + // Insert each hash into the bloom filter + for hash in hashes { + self.sbbf.insert_hash(hash); } + Ok(()) } @@ -427,296 +162,16 @@ impl BloomFilterExpr { /// Check an array against the bloom filter fn check_array(&self, array: &ArrayRef) -> Result { - use arrow::array::*; - use arrow::datatypes::DataType; - - let len = array.len(); - let mut builder = BooleanArray::builder(len); - - match array.data_type() { - DataType::Boolean => { - let array = array.as_any().downcast_ref::().unwrap(); - for i in 0..len { - if array.is_null(i) { - builder.append_value(false); - } else { - builder.append_value(self.bloom_filter.check(&array.value(i))); - } - } - } - DataType::Int8 => { - let array = array.as_any().downcast_ref::().unwrap(); - for i in 0..len { - if array.is_null(i) { - builder.append_value(false); - } else { - builder.append_value(self.bloom_filter.check(&array.value(i))); - } - } - } - DataType::Int16 => { - let array = array.as_any().downcast_ref::().unwrap(); - for i in 0..len { - if array.is_null(i) { - builder.append_value(false); - } else { - builder.append_value(self.bloom_filter.check(&array.value(i))); - } - } - } - DataType::Int32 => { - let array = array.as_any().downcast_ref::().unwrap(); - for i in 0..len { - if array.is_null(i) { - builder.append_value(false); - } else { - builder.append_value(self.bloom_filter.check(&array.value(i))); - } - } - } - DataType::Int64 => { - let array = array.as_any().downcast_ref::().unwrap(); - for i in 0..len { - if array.is_null(i) { - builder.append_value(false); - } else { - builder.append_value(self.bloom_filter.check(&array.value(i))); - } - } - } - DataType::UInt8 => { - let array = array.as_any().downcast_ref::().unwrap(); - for i in 0..len { - if array.is_null(i) { - builder.append_value(false); - } else { - builder.append_value(self.bloom_filter.check(&array.value(i))); - } - } - } - DataType::UInt16 => { - let array = array.as_any().downcast_ref::().unwrap(); - for i in 0..len { - if array.is_null(i) { - builder.append_value(false); - } else { - builder.append_value(self.bloom_filter.check(&array.value(i))); - } - } - } - DataType::UInt32 => { - let array = array.as_any().downcast_ref::().unwrap(); - for i in 0..len { - if array.is_null(i) { - builder.append_value(false); - } else { - builder.append_value(self.bloom_filter.check(&array.value(i))); - } - } - } - DataType::UInt64 => { - let array = array.as_any().downcast_ref::().unwrap(); - for i in 0..len { - if array.is_null(i) { - builder.append_value(false); - } else { - builder.append_value(self.bloom_filter.check(&array.value(i))); - } - } - } - DataType::Float32 => { - let array = array.as_any().downcast_ref::().unwrap(); - for i in 0..len { - if array.is_null(i) { - builder.append_value(false); - } else { - builder.append_value(self.bloom_filter.check(&array.value(i))); - } - } - } - DataType::Float64 => { - let array = array.as_any().downcast_ref::().unwrap(); - for i in 0..len { - if array.is_null(i) { - builder.append_value(false); - } else { - builder.append_value(self.bloom_filter.check(&array.value(i))); - } - } - } - DataType::Utf8 => { - let array = array.as_any().downcast_ref::().unwrap(); - for i in 0..len { - if array.is_null(i) { - builder.append_value(false); - } else { - builder.append_value(self.bloom_filter.check(array.value(i))); - } - } - } - DataType::LargeUtf8 => { - let array = array.as_any().downcast_ref::().unwrap(); - for i in 0..len { - if array.is_null(i) { - builder.append_value(false); - } else { - builder.append_value(self.bloom_filter.check(array.value(i))); - } - } - } - DataType::Utf8View => { - let array = array.as_any().downcast_ref::().unwrap(); - for i in 0..len { - if array.is_null(i) { - builder.append_value(false); - } else { - builder.append_value(self.bloom_filter.check(array.value(i))); - } - } - } - DataType::Binary => { - let array = array.as_any().downcast_ref::().unwrap(); - for i in 0..len { - if array.is_null(i) { - builder.append_value(false); - } else { - builder.append_value(self.bloom_filter.check(array.value(i))); - } - } - } - DataType::LargeBinary => { - let array = array.as_any().downcast_ref::().unwrap(); - for i in 0..len { - if array.is_null(i) { - builder.append_value(false); - } else { - builder.append_value(self.bloom_filter.check(array.value(i))); - } - } - } - DataType::BinaryView => { - let array = array.as_any().downcast_ref::().unwrap(); - for i in 0..len { - if array.is_null(i) { - builder.append_value(false); - } else { - builder.append_value(self.bloom_filter.check(array.value(i))); - } - } - } - DataType::FixedSizeBinary(_) => { - let array = array - .as_any() - .downcast_ref::() - .unwrap(); - for i in 0..len { - if array.is_null(i) { - builder.append_value(false); - } else { - builder.append_value(self.bloom_filter.check(array.value(i))); - } - } - } - DataType::Decimal128(_, _) => { - let array = array.as_any().downcast_ref::().unwrap(); - for i in 0..len { - if array.is_null(i) { - builder.append_value(false); - } else { - builder.append_value(self.bloom_filter.check(&array.value(i))); - } - } - } - DataType::Decimal256(_, _) => { - let array = array.as_any().downcast_ref::().unwrap(); - for i in 0..len { - if array.is_null(i) { - builder.append_value(false); - } else { - let bytes = array.value(i).to_be_bytes(); - builder.append_value(self.bloom_filter.check(&bytes)); - } - } - } - DataType::Date32 => { - let array = array.as_any().downcast_ref::().unwrap(); - for i in 0..len { - if array.is_null(i) { - builder.append_value(false); - } else { - builder.append_value(self.bloom_filter.check(&array.value(i))); - } - } - } - DataType::Date64 => { - let array = array.as_any().downcast_ref::().unwrap(); - for i in 0..len { - if array.is_null(i) { - builder.append_value(false); - } else { - builder.append_value(self.bloom_filter.check(&array.value(i))); - } - } - } - DataType::Timestamp(_, _) => { - // All timestamp types store i64 values internally - // Try each timestamp array type - if let Some(ts_array) = - array.as_any().downcast_ref::() - { - for i in 0..len { - if ts_array.is_null(i) { - builder.append_value(false); - } else { - builder.append_value( - self.bloom_filter.check(&ts_array.value(i)), - ); - } - } - } else if let Some(ts_array) = - array.as_any().downcast_ref::() - { - for i in 0..len { - if ts_array.is_null(i) { - builder.append_value(false); - } else { - builder.append_value( - self.bloom_filter.check(&ts_array.value(i)), - ); - } - } - } else if let Some(ts_array) = - array.as_any().downcast_ref::() - { - for i in 0..len { - if ts_array.is_null(i) { - builder.append_value(false); - } else { - builder.append_value( - self.bloom_filter.check(&ts_array.value(i)), - ); - } - } - } else if let Some(ts_array) = - array.as_any().downcast_ref::() - { - for i in 0..len { - if ts_array.is_null(i) { - builder.append_value(false); - } else { - builder.append_value( - self.bloom_filter.check(&ts_array.value(i)), - ); - } - } - } - } - _ => { - return internal_err!( - "Unsupported data type for bloom filter check: {}", - array.data_type() - ) - } + // Use create_hashes to compute hash values for all array types + // This handles Dictionary, Struct, Null, and all other types uniformly + let mut hashes = vec![0u64; array.len()]; + let random_state = RandomState::with_seeds(0, 0, 0, 0); + create_hashes(&[Arc::clone(array)], &random_state, &mut hashes)?; + + // Check each hash against the bloom filter + let mut builder = BooleanArray::builder(array.len()); + for hash in hashes { + builder.append_value(self.bloom_filter.check_hash(hash)); } Ok(builder.finish()) @@ -802,6 +257,20 @@ mod tests { use crate::expressions::col; use arrow::datatypes::{Field, Schema}; + // Helper trait to add insert_scalar for tests + trait BloomFilterBuilderTestExt { + fn insert_scalar(&mut self, value: &ScalarValue) -> Result<()>; + } + + impl BloomFilterBuilderTestExt for BloomFilterBuilder { + /// Insert a single scalar value by converting to array and using insert_array + /// This is less efficient but sufficient for tests + fn insert_scalar(&mut self, value: &ScalarValue) -> Result<()> { + let array = value.to_array()?; + self.insert_array(&array) + } + } + #[test] fn test_bloom_filter_builder() -> Result<()> { let mut builder = BloomFilterBuilder::new(100, 0.01)?; From a768385a8e9ef5f2e174a22795ec816a7ea102de Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Mon, 27 Oct 2025 15:13:20 -0500 Subject: [PATCH 08/11] cleanup --- datafusion/physical-expr/src/bloom_filter.rs | 33 ++--------- .../src/expressions/bloom_filter_expr.rs | 57 ++++--------------- 2 files changed, 16 insertions(+), 74 deletions(-) diff --git a/datafusion/physical-expr/src/bloom_filter.rs b/datafusion/physical-expr/src/bloom_filter.rs index 0f833b0f6b2d..034a234f192c 100644 --- a/datafusion/physical-expr/src/bloom_filter.rs +++ b/datafusion/physical-expr/src/bloom_filter.rs @@ -27,10 +27,13 @@ //! //! The implementation below is adapted from: //! arrow-rs/parquet/src/bloom_filter/mod.rs +//! +//! One thing to consider is if we can make this implementaion compatible with Parquet's +//! byte for byte (it currently is not) so that we can do binary intersection of bloom filters +//! between DataFusion and Parquet. use datafusion_common::{internal_err, Result}; use std::mem::size_of; -use twox_hash::XxHash64; /// Salt values as defined in the Parquet specification /// Although we don't *need* to follow the Parquet spec here, using the same @@ -52,9 +55,6 @@ const BITSET_MIN_LENGTH: usize = 32; /// Maximum bitset length in bytes const BITSET_MAX_LENGTH: usize = 128 * 1024 * 1024; -/// Hash seed for xxHash -const SEED: u64 = 0; - /// Each block is 256 bits, broken up into eight contiguous "words", each consisting of 32 bits. /// Each word is thought of as an array of bits; each bit is either "set" or "not set". #[derive(Debug, Copy, Clone)] @@ -157,11 +157,6 @@ impl Sbbf { self.0[block_index].insert(hash as u32) } - /// Check if a value is probably present or definitely absent in the filter - pub fn check(&self, value: &T) -> bool { - self.check_hash(hash_as_bytes(value)) - } - /// Check if a hash is in the filter. May return /// true for values that were never inserted ("false positive") /// but will always return false if a hash has not been inserted. @@ -296,12 +291,6 @@ impl AsBytes for [u8; 32] { } } -/// Hash a value using xxHash64 with seed 0 -#[inline] -fn hash_as_bytes(value: &A) -> u64 { - XxHash64::oneshot(SEED, value.as_bytes()) -} - /// Calculate optimal number of bytes, bounded by min/max and rounded to power of 2 #[inline] fn optimal_num_of_bytes(num_bytes: usize) -> usize { @@ -323,11 +312,6 @@ fn num_of_bits_from_ndv_fpp(ndv: u64, fpp: f64) -> usize { mod tests { use super::*; - #[test] - fn test_hash_bytes() { - assert_eq!(hash_as_bytes(""), 17241709254077376921); - } - #[test] fn test_mask_set_quick_check() { for i in 0..1_000 { @@ -345,15 +329,6 @@ mod tests { } } - #[test] - fn test_sbbf_insert_and_check() { - let mut sbbf = Sbbf(vec![Block::ZERO; 1_000]); - for i in 0..10_000 { - sbbf.insert(&i); - assert!(sbbf.check(&i)); - } - } - #[test] fn test_optimal_num_of_bytes() { for (input, expected) in &[ diff --git a/datafusion/physical-expr/src/expressions/bloom_filter_expr.rs b/datafusion/physical-expr/src/expressions/bloom_filter_expr.rs index 9603403906bc..a67d05bce65f 100644 --- a/datafusion/physical-expr/src/expressions/bloom_filter_expr.rs +++ b/datafusion/physical-expr/src/expressions/bloom_filter_expr.rs @@ -116,48 +116,15 @@ impl BloomFilterExpr { } /// Check a scalar value against the bloom filter - fn check_scalar(&self, value: &ScalarValue) -> bool { - if value.is_null() { - return false; - } - - match value { - ScalarValue::Boolean(Some(v)) => self.bloom_filter.check(v), - ScalarValue::Int8(Some(v)) => self.bloom_filter.check(v), - ScalarValue::Int16(Some(v)) => self.bloom_filter.check(v), - ScalarValue::Int32(Some(v)) => self.bloom_filter.check(v), - ScalarValue::Int64(Some(v)) => self.bloom_filter.check(v), - ScalarValue::UInt8(Some(v)) => self.bloom_filter.check(v), - ScalarValue::UInt16(Some(v)) => self.bloom_filter.check(v), - ScalarValue::UInt32(Some(v)) => self.bloom_filter.check(v), - ScalarValue::UInt64(Some(v)) => self.bloom_filter.check(v), - ScalarValue::Float32(Some(v)) => self.bloom_filter.check(v), - ScalarValue::Float64(Some(v)) => self.bloom_filter.check(v), - ScalarValue::Utf8(Some(v)) | ScalarValue::LargeUtf8(Some(v)) => { - self.bloom_filter.check(v.as_str()) - } - ScalarValue::Utf8View(Some(v)) => self.bloom_filter.check(v.as_str()), - ScalarValue::Binary(Some(v)) - | ScalarValue::LargeBinary(Some(v)) - | ScalarValue::FixedSizeBinary(_, Some(v)) => { - self.bloom_filter.check(v.as_slice()) - } - ScalarValue::BinaryView(Some(v)) => self.bloom_filter.check(v.as_slice()), - ScalarValue::Decimal32(Some(v), _, _) => self.bloom_filter.check(v), - ScalarValue::Decimal64(Some(v), _, _) => self.bloom_filter.check(v), - ScalarValue::Decimal128(Some(v), _, _) => self.bloom_filter.check(v), - ScalarValue::Decimal256(Some(v), _, _) => { - let bytes = v.to_be_bytes(); - self.bloom_filter.check(&bytes) - } - ScalarValue::Date32(Some(v)) => self.bloom_filter.check(v), - ScalarValue::Date64(Some(v)) => self.bloom_filter.check(v), - ScalarValue::TimestampSecond(Some(v), _) - | ScalarValue::TimestampMillisecond(Some(v), _) - | ScalarValue::TimestampMicrosecond(Some(v), _) - | ScalarValue::TimestampNanosecond(Some(v), _) => self.bloom_filter.check(v), - _ => true, // Unsupported types default to "might be present" + fn check_scalar(&self, value: &ScalarValue) -> Result { + let array = value.to_array()?; + let result = self.check_array(&array)?; + // Since the array has length 1, return the first value + #[cfg(debug_assertions)] + { + assert_eq!(result.len(), 1); } + Ok(result.value(0)) } /// Check an array against the bloom filter @@ -223,7 +190,7 @@ impl PhysicalExpr for BloomFilterExpr { Ok(ColumnarValue::Array(Arc::new(result))) } ColumnarValue::Scalar(scalar) => { - let result = self.check_scalar(&scalar); + let result = self.check_scalar(&scalar)?; Ok(ColumnarValue::Scalar(ScalarValue::Boolean(Some(result)))) } } @@ -285,9 +252,9 @@ mod tests { let bloom_expr = builder.build(expr); // Check that inserted values are found - assert!(bloom_expr.check_scalar(&ScalarValue::Int32(Some(1)))); - assert!(bloom_expr.check_scalar(&ScalarValue::Int32(Some(2)))); - assert!(bloom_expr.check_scalar(&ScalarValue::Int32(Some(3)))); + assert!(bloom_expr.check_scalar(&ScalarValue::Int32(Some(1)))?); + assert!(bloom_expr.check_scalar(&ScalarValue::Int32(Some(2)))?); + assert!(bloom_expr.check_scalar(&ScalarValue::Int32(Some(3)))?); // A value that wasn't inserted might not be found // (but could be a false positive, so we can't assert false) From f53018f1c5fb8111e8b6fb798855439e35495059 Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Mon, 27 Oct 2025 15:59:02 -0500 Subject: [PATCH 09/11] try to re-use hashes --- datafusion/physical-expr/src/bloom_filter.rs | 2 +- .../src/expressions/bloom_filter_expr.rs | 84 ++++++++++++------- .../physical-plan/src/joins/hash_join/exec.rs | 34 ++++++-- 3 files changed, 80 insertions(+), 40 deletions(-) diff --git a/datafusion/physical-expr/src/bloom_filter.rs b/datafusion/physical-expr/src/bloom_filter.rs index 034a234f192c..9a644260be92 100644 --- a/datafusion/physical-expr/src/bloom_filter.rs +++ b/datafusion/physical-expr/src/bloom_filter.rs @@ -27,7 +27,7 @@ //! //! The implementation below is adapted from: //! arrow-rs/parquet/src/bloom_filter/mod.rs -//! +//! //! One thing to consider is if we can make this implementaion compatible with Parquet's //! byte for byte (it currently is not) so that we can do binary intersection of bloom filters //! between DataFusion and Parquet. diff --git a/datafusion/physical-expr/src/expressions/bloom_filter_expr.rs b/datafusion/physical-expr/src/expressions/bloom_filter_expr.rs index a67d05bce65f..80368cdb7bf8 100644 --- a/datafusion/physical-expr/src/expressions/bloom_filter_expr.rs +++ b/datafusion/physical-expr/src/expressions/bloom_filter_expr.rs @@ -38,15 +38,17 @@ use std::sync::Arc; /// /// # Example /// ```ignore -/// let mut builder = BloomFilterBuilder::new(1000, 0.01)?; -/// builder.insert_scalar(&ScalarValue::Int32(Some(42)))?; -/// builder.insert_array(&int_array)?; +/// let random_state = RandomState::with_seeds(0, 0, 0, 0); +/// let mut builder = BloomFilterBuilder::new(1000, 0.01, random_state)?; +/// builder.insert_hashes(&hashes)?; /// let expr = builder.build(col_expr); // Consumes builder /// ``` #[derive(Debug)] pub struct BloomFilterBuilder { /// The underlying bloom filter sbbf: Sbbf, + /// Random state for consistent hashing + random_state: RandomState, } impl BloomFilterBuilder { @@ -55,25 +57,24 @@ impl BloomFilterBuilder { /// # Arguments /// * `ndv` - Expected number of distinct values /// * `fpp` - Desired false positive probability (0.0 to 1.0) - pub fn new(ndv: u64, fpp: f64) -> Result { + /// * `random_state` - Random state for consistent hashing across build and probe phases + pub fn new(ndv: u64, fpp: f64, random_state: RandomState) -> Result { let sbbf = Sbbf::new_with_ndv_fpp(ndv, fpp)?; - Ok(Self { sbbf }) + Ok(Self { sbbf, random_state }) } - /// Insert all values from an array into the bloom filter - pub fn insert_array(&mut self, array: &ArrayRef) -> Result<()> { - // Use create_hashes to compute hash values for all array types - // This handles Dictionary, Struct, Null, and all other types uniformly - let mut hashes = vec![0u64; array.len()]; - let random_state = RandomState::with_seeds(0, 0, 0, 0); - create_hashes(&[Arc::clone(array)], &random_state, &mut hashes)?; - - // Insert each hash into the bloom filter - for hash in hashes { + /// Insert pre-computed hash values into the bloom filter + /// + /// This method allows reusing hash values that were already computed + /// for other purposes (e.g., hash table insertion), avoiding redundant + /// hash computation. + /// + /// # Arguments + /// * `hashes` - Pre-computed hash values to insert + pub fn insert_hashes(&mut self, hashes: &[u64]) { + for &hash in hashes { self.sbbf.insert_hash(hash); } - - Ok(()) } /// Build a `BloomFilterExpr` from this builder, consuming the builder. @@ -84,7 +85,7 @@ impl BloomFilterBuilder { /// # Arguments /// * `expr` - The expression to evaluate and check against the bloom filter pub fn build(self, expr: Arc) -> BloomFilterExpr { - BloomFilterExpr::new(expr, self.sbbf) + BloomFilterExpr::new(expr, self.sbbf, self.random_state) } } @@ -102,16 +103,23 @@ pub struct BloomFilterExpr { expr: Arc, /// The bloom filter to check against bloom_filter: Arc, + /// Random state for consistent hashing + random_state: RandomState, } impl BloomFilterExpr { /// Create a new bloom filter expression (internal use only) /// /// Users should create bloom filter expressions through `BloomFilterBuilder::build()` - pub(crate) fn new(expr: Arc, bloom_filter: Sbbf) -> Self { + pub(crate) fn new( + expr: Arc, + bloom_filter: Sbbf, + random_state: RandomState, + ) -> Self { Self { expr, bloom_filter: Arc::new(bloom_filter), + random_state, } } @@ -132,8 +140,7 @@ impl BloomFilterExpr { // Use create_hashes to compute hash values for all array types // This handles Dictionary, Struct, Null, and all other types uniformly let mut hashes = vec![0u64; array.len()]; - let random_state = RandomState::with_seeds(0, 0, 0, 0); - create_hashes(&[Arc::clone(array)], &random_state, &mut hashes)?; + create_hashes(&[Arc::clone(array)], &self.random_state, &mut hashes)?; // Check each hash against the bloom filter let mut builder = BooleanArray::builder(array.len()); @@ -210,6 +217,7 @@ impl PhysicalExpr for BloomFilterExpr { Ok(Arc::new(BloomFilterExpr { expr: Arc::clone(&children[0]), bloom_filter: Arc::clone(&self.bloom_filter), + random_state: self.random_state.clone(), })) } @@ -230,17 +238,21 @@ mod tests { } impl BloomFilterBuilderTestExt for BloomFilterBuilder { - /// Insert a single scalar value by converting to array and using insert_array + /// Insert a single scalar value by converting to array and computing hashes /// This is less efficient but sufficient for tests fn insert_scalar(&mut self, value: &ScalarValue) -> Result<()> { let array = value.to_array()?; - self.insert_array(&array) + let mut hashes = vec![0u64; array.len()]; + create_hashes(&[array], &self.random_state, &mut hashes)?; + self.insert_hashes(&hashes); + Ok(()) } } #[test] fn test_bloom_filter_builder() -> Result<()> { - let mut builder = BloomFilterBuilder::new(100, 0.01)?; + let random_state = RandomState::with_seeds(0, 0, 0, 0); + let mut builder = BloomFilterBuilder::new(100, 0.01, random_state)?; // Insert some values builder.insert_scalar(&ScalarValue::Int32(Some(1)))?; @@ -267,9 +279,16 @@ mod tests { use arrow::array::Int32Array; // Build a bloom filter with values 1, 2, 3 - let mut builder = BloomFilterBuilder::new(100, 0.01)?; + let random_state = RandomState::with_seeds(0, 0, 0, 0); + let mut builder = BloomFilterBuilder::new(100, 0.01, random_state)?; let training_array = Arc::new(Int32Array::from(vec![1, 2, 3])) as ArrayRef; - builder.insert_array(&training_array)?; + let mut hashes = vec![0u64; training_array.len()]; + create_hashes( + &[Arc::clone(&training_array)], + &builder.random_state, + &mut hashes, + )?; + builder.insert_hashes(&hashes); // Create the expression let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int32, false)])); @@ -302,7 +321,8 @@ mod tests { fn test_bloom_filter_with_strings() -> Result<()> { use arrow::array::StringArray; - let mut builder = BloomFilterBuilder::new(100, 0.01)?; + let random_state = RandomState::with_seeds(0, 0, 0, 0); + let mut builder = BloomFilterBuilder::new(100, 0.01, random_state)?; builder.insert_scalar(&ScalarValue::Utf8(Some("hello".to_string())))?; builder.insert_scalar(&ScalarValue::Utf8(Some("world".to_string())))?; @@ -332,7 +352,8 @@ mod tests { use arrow::array::Decimal128Array; // Build a bloom filter with decimal values - let mut builder = BloomFilterBuilder::new(100, 0.01)?; + let random_state = RandomState::with_seeds(0, 0, 0, 0); + let mut builder = BloomFilterBuilder::new(100, 0.01, random_state)?; builder.insert_scalar(&ScalarValue::Decimal128(Some(12345), 10, 2))?; builder.insert_scalar(&ScalarValue::Decimal128(Some(67890), 10, 2))?; @@ -374,7 +395,8 @@ mod tests { use arrow::array::{Float64Array, Int32Array, StringArray}; // Test Int32: Use extremely low FPP (0.00001) to make false positives negligible - let mut builder = BloomFilterBuilder::new(10, 0.00001)?; + let random_state = RandomState::with_seeds(0, 0, 0, 0); + let mut builder = BloomFilterBuilder::new(10, 0.00001, random_state.clone())?; for i in 1..=10 { builder.insert_scalar(&ScalarValue::Int32(Some(i)))?; } @@ -401,7 +423,7 @@ mod tests { } // Test Float64 - let mut builder = BloomFilterBuilder::new(10, 0.00001)?; + let mut builder = BloomFilterBuilder::new(10, 0.00001, random_state.clone())?; for i in 0..10 { builder.insert_scalar(&ScalarValue::Float64(Some(i as f64 * 0.5)))?; } @@ -428,7 +450,7 @@ mod tests { } // Test Strings - let mut builder = BloomFilterBuilder::new(5, 0.00001)?; + let mut builder = BloomFilterBuilder::new(5, 0.00001, random_state)?; builder.insert_scalar(&ScalarValue::Utf8(Some("apple".to_string())))?; builder.insert_scalar(&ScalarValue::Utf8(Some("banana".to_string())))?; builder.insert_scalar(&ScalarValue::Utf8(Some("cherry".to_string())))?; diff --git a/datafusion/physical-plan/src/joins/hash_join/exec.rs b/datafusion/physical-plan/src/joins/hash_join/exec.rs index edee3b741a7c..59f59447ad65 100644 --- a/datafusion/physical-plan/src/joins/hash_join/exec.rs +++ b/datafusion/physical-plan/src/joins/hash_join/exec.rs @@ -1238,10 +1238,15 @@ impl CollectLeftAccumulator { /// # Arguments /// * `expr` - The physical expression to track bounds for /// * `schema` - The schema of the input data + /// * `random_state` - Random state for consistent hashing /// /// # Returns /// A new `CollectLeftAccumulator` instance configured for the expression's data type - fn try_new(expr: Arc, schema: &SchemaRef) -> Result { + fn try_new( + expr: Arc, + schema: &SchemaRef, + random_state: RandomState, + ) -> Result { /// Recursively unwraps dictionary types to get the underlying value type. fn dictionary_value_type(data_type: &DataType) -> DataType { match data_type { @@ -1259,7 +1264,7 @@ impl CollectLeftAccumulator { // Create bloom filter with default parameters // NDV (number of distinct values) = 1000, FPP (false positive probability) = 0.05 (5%) - let bloom_filter = BloomFilterBuilder::new(1000, 0.05)?; + let bloom_filter = BloomFilterBuilder::new(1000, 0.05, random_state)?; Ok(Self { expr, @@ -1271,8 +1276,8 @@ impl CollectLeftAccumulator { /// Updates the accumulators with values from a new batch. /// - /// Evaluates the expression on the batch and updates min, max, and bloom filter - /// with the resulting values. + /// Evaluates the expression on the batch and updates min and max bounds. + /// Bloom filter population is deferred to Pass 2 to reuse hash computations. /// /// # Arguments /// * `batch` - The record batch to process @@ -1283,8 +1288,7 @@ impl CollectLeftAccumulator { let array = self.expr.evaluate(batch)?.into_array(batch.num_rows())?; self.min.update_batch(std::slice::from_ref(&array))?; self.max.update_batch(std::slice::from_ref(&array))?; - // Insert values into bloom filter - self.bloom_filter.insert_array(&array)?; + // Bloom filter population deferred to Pass 2 to reuse hash table hashes Ok(()) } @@ -1318,6 +1322,7 @@ impl BuildSideState { on_left: Vec>, schema: &SchemaRef, should_compute_bounds: bool, + random_state: &RandomState, ) -> Result { Ok(Self { batches: Vec::new(), @@ -1329,7 +1334,11 @@ impl BuildSideState { on_left .iter() .map(|expr| { - CollectLeftAccumulator::try_new(Arc::clone(expr), schema) + CollectLeftAccumulator::try_new( + Arc::clone(expr), + schema, + random_state.clone(), + ) }) .collect::>>() }) @@ -1388,6 +1397,7 @@ async fn collect_left_input( on_left.clone(), &schema, should_compute_bounds, + &random_state, )?; let state = left_stream @@ -1421,7 +1431,7 @@ async fn collect_left_input( num_rows, metrics, mut reservation, - bounds_accumulators, + mut bounds_accumulators, } = state; // Estimation of memory size, required for hashtable, prior to allocation. @@ -1463,6 +1473,14 @@ async fn collect_left_input( 0, true, )?; + + // Populate bloom filters with computed hashes to avoid redundant hash computation + if let Some(ref mut accumulators) = bounds_accumulators { + for accumulator in accumulators.iter_mut() { + accumulator.bloom_filter.insert_hashes(&hashes_buffer); + } + } + offset += batch.num_rows(); } // Merge all batches into a single batch, so we can directly index into the arrays From 980bcd72c7b020faff5da248b79e9e95209da308 Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Mon, 27 Oct 2025 16:12:49 -0500 Subject: [PATCH 10/11] fix ci --- Cargo.lock | 1 - datafusion/physical-expr/Cargo.toml | 1 - datafusion/physical-expr/src/bloom_filter.rs | 2 +- .../src/expressions/bloom_filter_expr.rs | 93 ------------------- 4 files changed, 1 insertion(+), 96 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index f68fcb1a71b7..e368dcf9a91e 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2419,7 +2419,6 @@ dependencies = [ "petgraph 0.8.3", "rand 0.9.2", "rstest", - "twox-hash", ] [[package]] diff --git a/datafusion/physical-expr/Cargo.toml b/datafusion/physical-expr/Cargo.toml index a5c2a1b08a32..b7654a0f6f60 100644 --- a/datafusion/physical-expr/Cargo.toml +++ b/datafusion/physical-expr/Cargo.toml @@ -52,7 +52,6 @@ itertools = { workspace = true, features = ["use_std"] } parking_lot = { workspace = true } paste = "^1.0" petgraph = "0.8.3" -twox-hash = { version = "2.0", default-features = false, features = ["xxhash64"] } [dev-dependencies] arrow = { workspace = true, features = ["test_utils"] } diff --git a/datafusion/physical-expr/src/bloom_filter.rs b/datafusion/physical-expr/src/bloom_filter.rs index 9a644260be92..15a51fc0efa0 100644 --- a/datafusion/physical-expr/src/bloom_filter.rs +++ b/datafusion/physical-expr/src/bloom_filter.rs @@ -28,7 +28,7 @@ //! The implementation below is adapted from: //! arrow-rs/parquet/src/bloom_filter/mod.rs //! -//! One thing to consider is if we can make this implementaion compatible with Parquet's +//! One thing to consider is if we can make this implementation compatible with Parquet's //! byte for byte (it currently is not) so that we can do binary intersection of bloom filters //! between DataFusion and Parquet. diff --git a/datafusion/physical-expr/src/expressions/bloom_filter_expr.rs b/datafusion/physical-expr/src/expressions/bloom_filter_expr.rs index 80368cdb7bf8..ef21917fa9b1 100644 --- a/datafusion/physical-expr/src/expressions/bloom_filter_expr.rs +++ b/datafusion/physical-expr/src/expressions/bloom_filter_expr.rs @@ -389,97 +389,4 @@ mod tests { Ok(()) } - - #[test] - fn test_bloom_filter_negative_lookups() -> Result<()> { - use arrow::array::{Float64Array, Int32Array, StringArray}; - - // Test Int32: Use extremely low FPP (0.00001) to make false positives negligible - let random_state = RandomState::with_seeds(0, 0, 0, 0); - let mut builder = BloomFilterBuilder::new(10, 0.00001, random_state.clone())?; - for i in 1..=10 { - builder.insert_scalar(&ScalarValue::Int32(Some(i)))?; - } - let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int32, false)])); - let expr = col("a", &schema)?; - let bloom_expr = Arc::new(builder.build(expr)); - - // Test values far outside the inserted range (1000-1099) - let test_values: Vec = (1000..1100).collect(); - let test_array = Arc::new(Int32Array::from(test_values)) as ArrayRef; - let batch = RecordBatch::try_new(Arc::clone(&schema), vec![test_array])?; - let result = bloom_expr.evaluate(&batch)?; - let result_array = result.into_array(100)?; - let result_bool = result_array - .as_any() - .downcast_ref::() - .unwrap(); - for i in 0..100 { - assert!( - !result_bool.value(i), - "Int32 value {} should not match", - i + 1000 - ); - } - - // Test Float64 - let mut builder = BloomFilterBuilder::new(10, 0.00001, random_state.clone())?; - for i in 0..10 { - builder.insert_scalar(&ScalarValue::Float64(Some(i as f64 * 0.5)))?; - } - let schema = - Arc::new(Schema::new(vec![Field::new("f", DataType::Float64, false)])); - let expr = col("f", &schema)?; - let bloom_expr = Arc::new(builder.build(expr)); - - let test_values: Vec = (100..200).map(|i| i as f64 * 10.0).collect(); - let test_array = Arc::new(Float64Array::from(test_values)) as ArrayRef; - let batch = RecordBatch::try_new(Arc::clone(&schema), vec![test_array])?; - let result = bloom_expr.evaluate(&batch)?; - let result_array = result.into_array(100)?; - let result_bool = result_array - .as_any() - .downcast_ref::() - .unwrap(); - for i in 0..100 { - assert!( - !result_bool.value(i), - "Float64 value {} should not match", - (i + 100) as f64 * 10.0 - ); - } - - // Test Strings - let mut builder = BloomFilterBuilder::new(5, 0.00001, random_state)?; - builder.insert_scalar(&ScalarValue::Utf8(Some("apple".to_string())))?; - builder.insert_scalar(&ScalarValue::Utf8(Some("banana".to_string())))?; - builder.insert_scalar(&ScalarValue::Utf8(Some("cherry".to_string())))?; - let schema = Arc::new(Schema::new(vec![Field::new("s", DataType::Utf8, false)])); - let expr = col("s", &schema)?; - let bloom_expr = Arc::new(builder.build(expr)); - - let test_strings = vec![ - "zebra", - "yak", - "xylophone", - "walrus", - "vulture", - "umbrella", - "tiger", - "snake", - ]; - let test_array = Arc::new(StringArray::from(test_strings.clone())) as ArrayRef; - let batch = RecordBatch::try_new(Arc::clone(&schema), vec![test_array])?; - let result = bloom_expr.evaluate(&batch)?; - let result_array = result.into_array(8)?; - let result_bool = result_array - .as_any() - .downcast_ref::() - .unwrap(); - for (i, s) in test_strings.iter().enumerate() { - assert!(!result_bool.value(i), "String '{s}' should not match"); - } - - Ok(()) - } } From 9164c17183d6ed70a784bb2bb8c89be04b7286ed Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Mon, 27 Oct 2025 22:20:25 -0500 Subject: [PATCH 11/11] fix --- .../physical_optimizer/filter_pushdown/mod.rs | 8 +- .../src/expressions/bloom_filter_expr.rs | 166 ++++++++++++------ .../src/joins/hash_join/shared_bounds.rs | 39 ++-- 3 files changed, 137 insertions(+), 76 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs b/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs index 3ec878882b02..090839e46982 100644 --- a/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs +++ b/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs @@ -1078,7 +1078,7 @@ async fn test_hashjoin_dynamic_filter_pushdown() { @r" - HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@0, a@0), (b@1, b@1)] - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate=DynamicFilter [ a@0 >= aa AND a@0 <= ab AND a@0 IN BLOOM_FILTER AND b@1 >= ba AND b@1 <= bb AND b@1 IN BLOOM_FILTER ] + - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate=DynamicFilter [ a@0 >= aa AND a@0 <= ab AND b@1 >= ba AND b@1 <= bb AND (a@0, b@1) IN BLOOM_FILTER ] " ); } @@ -1309,7 +1309,7 @@ async fn test_hashjoin_dynamic_filter_pushdown_partitioned() { - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([a@0, b@1], 12), input_partitions=1 - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate=DynamicFilter [ a@0 >= ab AND a@0 <= ab AND a@0 IN BLOOM_FILTER AND b@1 >= bb AND b@1 <= bb AND b@1 IN BLOOM_FILTER OR a@0 >= aa AND a@0 <= aa AND a@0 IN BLOOM_FILTER AND b@1 >= ba AND b@1 <= ba AND b@1 IN BLOOM_FILTER ] + - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate=DynamicFilter [ a@0 >= ab AND a@0 <= ab AND b@1 >= bb AND b@1 <= bb AND (a@0, b@1) IN BLOOM_FILTER OR a@0 >= aa AND a@0 <= aa AND b@1 >= ba AND b@1 <= ba AND (a@0, b@1) IN BLOOM_FILTER ] " ); @@ -1326,7 +1326,7 @@ async fn test_hashjoin_dynamic_filter_pushdown_partitioned() { - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([a@0, b@1], 12), input_partitions=1 - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate=DynamicFilter [ a@0 >= aa AND a@0 <= ab AND a@0 IN BLOOM_FILTER AND b@1 >= ba AND b@1 <= bb AND b@1 IN BLOOM_FILTER ] + - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate=DynamicFilter [ a@0 >= aa AND a@0 <= ab AND b@1 >= ba AND b@1 <= bb AND (a@0, b@1) IN BLOOM_FILTER ] " ); @@ -1503,7 +1503,7 @@ async fn test_hashjoin_dynamic_filter_pushdown_collect_left() { - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([a@0, b@1], 12), input_partitions=1 - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate=DynamicFilter [ a@0 >= aa AND a@0 <= ab AND a@0 IN BLOOM_FILTER AND b@1 >= ba AND b@1 <= bb AND b@1 IN BLOOM_FILTER ] + - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate=DynamicFilter [ a@0 >= aa AND a@0 <= ab AND b@1 >= ba AND b@1 <= bb AND (a@0, b@1) IN BLOOM_FILTER ] " ); diff --git a/datafusion/physical-expr/src/expressions/bloom_filter_expr.rs b/datafusion/physical-expr/src/expressions/bloom_filter_expr.rs index ef21917fa9b1..404f7b277c3c 100644 --- a/datafusion/physical-expr/src/expressions/bloom_filter_expr.rs +++ b/datafusion/physical-expr/src/expressions/bloom_filter_expr.rs @@ -24,7 +24,7 @@ use arrow::array::{ArrayRef, BooleanArray}; use arrow::datatypes::{DataType, Schema}; use arrow::record_batch::RecordBatch; use datafusion_common::hash_utils::create_hashes; -use datafusion_common::{internal_err, Result, ScalarValue}; +use datafusion_common::{Result, ScalarValue}; use datafusion_expr_common::columnar_value::ColumnarValue; use std::any::Any; use std::fmt; @@ -83,24 +83,25 @@ impl BloomFilterBuilder { /// avoiding any clones of the (potentially large) bloom filter. /// /// # Arguments - /// * `expr` - The expression to evaluate and check against the bloom filter - pub fn build(self, expr: Arc) -> BloomFilterExpr { - BloomFilterExpr::new(expr, self.sbbf, self.random_state) + /// * `exprs` - The expressions to evaluate and check against the bloom filter + pub fn build(self, exprs: Vec>) -> BloomFilterExpr { + BloomFilterExpr::new(exprs, self.sbbf, self.random_state) } } /// Physical expression that checks values against a bloom filter /// /// This is a static expression (similar to `InListExpr`) that evaluates -/// a child expression and checks each value against a pre-built bloom filter. +/// one or more child expressions and checks each value against a pre-built bloom filter. +/// When multiple expressions are provided, they are combined via hashing (similar to join key hashing). /// Returns a boolean array indicating whether each value might be present /// (true) or is definitely absent (false). /// /// Note: Bloom filters can produce false positives but never false negatives. #[derive(Debug, Clone)] pub struct BloomFilterExpr { - /// The expression to evaluate - expr: Arc, + /// The expressions to evaluate (one or more columns) + exprs: Vec>, /// The bloom filter to check against bloom_filter: Arc, /// Random state for consistent hashing @@ -112,38 +113,55 @@ impl BloomFilterExpr { /// /// Users should create bloom filter expressions through `BloomFilterBuilder::build()` pub(crate) fn new( - expr: Arc, + exprs: Vec>, bloom_filter: Sbbf, random_state: RandomState, ) -> Self { Self { - expr, + exprs, bloom_filter: Arc::new(bloom_filter), random_state, } } - /// Check a scalar value against the bloom filter - fn check_scalar(&self, value: &ScalarValue) -> Result { - let array = value.to_array()?; - let result = self.check_array(&array)?; - // Since the array has length 1, return the first value - #[cfg(debug_assertions)] - { - assert_eq!(result.len(), 1); - } - Ok(result.value(0)) + /// Check scalar expressions against the bloom filter + fn check_scalar_batch(&self, batch: &RecordBatch) -> Result { + // Evaluate all expressions to get their scalar values + let arrays: Vec = self + .exprs + .iter() + .map(|expr| { + let value = expr.evaluate(batch)?; + match value { + ColumnarValue::Scalar(s) => s.to_array(), + ColumnarValue::Array(a) => Ok(a), + } + }) + .collect::>>()?; + + // Compute combined hash + let mut hashes = vec![0u64; 1]; + create_hashes(&arrays, &self.random_state, &mut hashes)?; + + Ok(self.bloom_filter.check_hash(hashes[0])) } - /// Check an array against the bloom filter - fn check_array(&self, array: &ArrayRef) -> Result { - // Use create_hashes to compute hash values for all array types - // This handles Dictionary, Struct, Null, and all other types uniformly - let mut hashes = vec![0u64; array.len()]; - create_hashes(&[Arc::clone(array)], &self.random_state, &mut hashes)?; + /// Check arrays against the bloom filter + fn check_arrays(&self, batch: &RecordBatch) -> Result { + // Evaluate all expressions to get their arrays + let arrays: Vec = self + .exprs + .iter() + .map(|expr| expr.evaluate(batch)?.into_array(batch.num_rows())) + .collect::>>()?; + + // Use create_hashes to compute combined hash values for all columns + // This matches how the build side computes hashes (combining all join columns) + let mut hashes = vec![0u64; batch.num_rows()]; + create_hashes(&arrays, &self.random_state, &mut hashes)?; // Check each hash against the bloom filter - let mut builder = BooleanArray::builder(array.len()); + let mut builder = BooleanArray::builder(batch.num_rows()); for hash in hashes { builder.append_value(self.bloom_filter.check_hash(hash)); } @@ -154,15 +172,27 @@ impl BloomFilterExpr { impl fmt::Display for BloomFilterExpr { fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { - write!(f, "{} IN BLOOM_FILTER", self.expr) + if self.exprs.len() == 1 { + write!(f, "{} IN BLOOM_FILTER", self.exprs[0]) + } else { + write!(f, "(")?; + for (i, expr) in self.exprs.iter().enumerate() { + if i > 0 { + write!(f, ", ")?; + } + write!(f, "{}", expr)?; + } + write!(f, ") IN BLOOM_FILTER") + } } } impl PartialEq for BloomFilterExpr { fn eq(&self, other: &Self) -> bool { - // Two bloom filter expressions are equal if they have the same child expression + // Two bloom filter expressions are equal if they have the same child expressions // We can't compare bloom filters directly, so we use pointer equality - self.expr.eq(&other.expr) && Arc::ptr_eq(&self.bloom_filter, &other.bloom_filter) + self.exprs.eq(&other.exprs) + && Arc::ptr_eq(&self.bloom_filter, &other.bloom_filter) } } @@ -170,7 +200,9 @@ impl Eq for BloomFilterExpr {} impl Hash for BloomFilterExpr { fn hash(&self, state: &mut H) { - self.expr.hash(state); + for expr in &self.exprs { + expr.hash(state); + } // Hash the pointer to the bloom filter Arc::as_ptr(&self.bloom_filter).hash(state); } @@ -190,39 +222,54 @@ impl PhysicalExpr for BloomFilterExpr { } fn evaluate(&self, batch: &RecordBatch) -> Result { - let value = self.expr.evaluate(batch)?; - match value { - ColumnarValue::Array(array) => { - let result = self.check_array(&array)?; - Ok(ColumnarValue::Array(Arc::new(result))) - } - ColumnarValue::Scalar(scalar) => { - let result = self.check_scalar(&scalar)?; - Ok(ColumnarValue::Scalar(ScalarValue::Boolean(Some(result)))) - } + // Check if all expressions return scalars + let all_scalars = self + .exprs + .iter() + .map(|expr| expr.evaluate(batch)) + .collect::>>()? + .iter() + .all(|v| matches!(v, ColumnarValue::Scalar(_))); + + if all_scalars { + // If all are scalars, check them and return a scalar + let result = self.check_scalar_batch(batch)?; + Ok(ColumnarValue::Scalar(ScalarValue::Boolean(Some(result)))) + } else { + // Otherwise, check arrays + let result = self.check_arrays(batch)?; + Ok(ColumnarValue::Array(Arc::new(result))) } } fn children(&self) -> Vec<&Arc> { - vec![&self.expr] + self.exprs.iter().collect() } fn with_new_children( self: Arc, children: Vec>, ) -> Result> { - if children.len() != 1 { - return internal_err!("BloomFilterExpr should have exactly 1 child"); - } Ok(Arc::new(BloomFilterExpr { - expr: Arc::clone(&children[0]), + exprs: children, bloom_filter: Arc::clone(&self.bloom_filter), random_state: self.random_state.clone(), })) } fn fmt_sql(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - write!(f, "{} IN BLOOM_FILTER", self.expr) + if self.exprs.len() == 1 { + write!(f, "{} IN BLOOM_FILTER", self.exprs[0]) + } else { + write!(f, "(")?; + for (i, expr) in self.exprs.iter().enumerate() { + if i > 0 { + write!(f, ", ")?; + } + write!(f, "{}", expr)?; + } + write!(f, ") IN BLOOM_FILTER") + } } } @@ -259,17 +306,20 @@ mod tests { builder.insert_scalar(&ScalarValue::Int32(Some(2)))?; builder.insert_scalar(&ScalarValue::Int32(Some(3)))?; - let schema = Schema::new(vec![Field::new("a", DataType::Int32, false)]); + let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int32, false)])); let expr = col("a", &schema)?; - let bloom_expr = builder.build(expr); - - // Check that inserted values are found - assert!(bloom_expr.check_scalar(&ScalarValue::Int32(Some(1)))?); - assert!(bloom_expr.check_scalar(&ScalarValue::Int32(Some(2)))?); - assert!(bloom_expr.check_scalar(&ScalarValue::Int32(Some(3)))?); + let bloom_expr = Arc::new(builder.build(vec![expr])); - // A value that wasn't inserted might not be found - // (but could be a false positive, so we can't assert false) + // Check that inserted values are found by creating test batches + let test_array = Arc::new(arrow::array::Int32Array::from(vec![1])) as ArrayRef; + let batch = RecordBatch::try_new(Arc::clone(&schema), vec![test_array])?; + let result = bloom_expr.evaluate(&batch)?; + assert!(result + .into_array(1)? + .as_any() + .downcast_ref::() + .unwrap() + .value(0)); Ok(()) } @@ -293,7 +343,7 @@ mod tests { // Create the expression let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int32, false)])); let expr = col("a", &schema)?; - let bloom_expr = Arc::new(builder.build(expr)); + let bloom_expr = Arc::new(builder.build(vec![expr])); // Create a test batch with values [1, 2, 4, 5] let test_array = Arc::new(Int32Array::from(vec![1, 2, 4, 5])) as ArrayRef; @@ -328,7 +378,7 @@ mod tests { let schema = Arc::new(Schema::new(vec![Field::new("s", DataType::Utf8, false)])); let expr = col("s", &schema)?; - let bloom_expr = Arc::new(builder.build(expr)); + let bloom_expr = Arc::new(builder.build(vec![expr])); let test_array = Arc::new(StringArray::from(vec!["hello", "world", "foo"])) as ArrayRef; @@ -363,7 +413,7 @@ mod tests { false, )])); let expr = col("d", &schema)?; - let bloom_expr = Arc::new(builder.build(expr)); + let bloom_expr = Arc::new(builder.build(vec![expr])); // Create test array with decimal values let test_array = Arc::new( diff --git a/datafusion/physical-plan/src/joins/hash_join/shared_bounds.rs b/datafusion/physical-plan/src/joins/hash_join/shared_bounds.rs index 6fc5cff63310..d8a590f16a59 100644 --- a/datafusion/physical-plan/src/joins/hash_join/shared_bounds.rs +++ b/datafusion/physical-plan/src/joins/hash_join/shared_bounds.rs @@ -214,10 +214,13 @@ impl SharedBoundsAccumulator { let mut partition_predicates = Vec::with_capacity(bounds.len()); for partition_bounds in bounds.into_iter() { - // Create range predicates and bloom filter checks for each join key in this partition - let mut column_predicates = + // Create range predicates for each join key in this partition + let mut range_predicates = Vec::with_capacity(partition_bounds.column_filters.len()); + // Get the first bloom filter (they all have the same data - combined hash of all columns) + let mut bloom_filter_builder = None; + // Consume column_filters by taking ownership for (col_idx, filter_data) in partition_bounds.column_filters.into_iter().enumerate() @@ -239,27 +242,35 @@ impl SharedBoundsAccumulator { Arc::new(BinaryExpr::new(min_expr, Operator::And, max_expr)) as Arc; - // Create bloom filter check by consuming the builder - let bloom_expr = - Arc::new(filter_data.bloom_filter.build(Arc::clone(right_expr))) - as Arc; + range_predicates.push(range_expr); - // Combine range and bloom filter: (range_expr AND bloom_expr) - let combined_expr = - Arc::new(BinaryExpr::new(range_expr, Operator::And, bloom_expr)) - as Arc; - column_predicates.push(combined_expr); + // Save the first bloom filter (all bloom filters have identical data) + if bloom_filter_builder.is_none() { + bloom_filter_builder = Some(filter_data.bloom_filter); + } } - // Combine all column predicates for this partition with AND - if !column_predicates.is_empty() { - let partition_predicate = column_predicates + // Create a single bloom filter check for all columns combined + if let Some(builder) = bloom_filter_builder { + // Build bloom filter with all on_right expressions (matching how build side populated it) + let bloom_expr = Arc::new(builder.build(self.on_right.clone())) + as Arc; + + // Combine all range predicates with bloom filter + // First combine all range predicates with AND + let combined_ranges = range_predicates .into_iter() .reduce(|acc, pred| { Arc::new(BinaryExpr::new(acc, Operator::And, pred)) as Arc }) .unwrap(); + + // Then AND with the bloom filter + let partition_predicate = + Arc::new(BinaryExpr::new(combined_ranges, Operator::And, bloom_expr)) + as Arc; + partition_predicates.push(partition_predicate); } }