From cd6433c54c08dc1930991726b75210c4da1ec08c Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Fri, 21 Nov 2025 17:14:35 -0500 Subject: [PATCH 01/23] Use BatchCoaleser in sort merge join instead of calling coalesce_batches on vector of RecordBatches. Add benchmarks, update tests. --- benchmarks/README.md | 15 +- benchmarks/bench.sh | 17 + benchmarks/src/bin/dfbench.rs | 4 +- benchmarks/src/lib.rs | 1 + benchmarks/src/smj.rs | 524 ++++++++++++++++++ .../src/joins/sort_merge_join/stream.rs | 267 ++++++--- .../src/joins/sort_merge_join/tests.rs | 55 +- 7 files changed, 787 insertions(+), 96 deletions(-) create mode 100644 benchmarks/src/smj.rs diff --git a/benchmarks/README.md b/benchmarks/README.md index 8fed85fa02b8..142762e86561 100644 --- a/benchmarks/README.md +++ b/benchmarks/README.md @@ -762,7 +762,7 @@ Different queries are included to test nested loop joins under various workloads ## Hash Join -This benchmark focuses on the performance of queries with nested hash joins, minimizing other overheads such as scanning data sources or evaluating predicates. +This benchmark focuses on the performance of queries with hash joins, minimizing other overheads such as scanning data sources or evaluating predicates. Several queries are included to test hash joins under various workloads. @@ -774,6 +774,19 @@ Several queries are included to test hash joins under various workloads. ./bench.sh run hj ``` +## Sort Merge Join + +This benchmark focuses on the performance of queries with sort merge joins joins, minimizing other overheads such as scanning data sources or evaluating predicates. + +Several queries are included to test sort merge joins under various workloads. + +### Example Run + +```bash +# No need to generate data: this benchmark uses table function `range()` as the data source + +./bench.sh run smj +``` ## Cancellation Test performance of cancelling queries. diff --git a/benchmarks/bench.sh b/benchmarks/bench.sh index dbfd319dd9ad..948f75635311 100755 --- a/benchmarks/bench.sh +++ b/benchmarks/bench.sh @@ -126,6 +126,7 @@ imdb: Join Order Benchmark (JOB) using the IMDB dataset conver cancellation: How long cancelling a query takes nlj: Benchmark for simple nested loop joins, testing various join scenarios hj: Benchmark for simple hash joins, testing various join scenarios +smj: Benchmark for simple sort merge joins, testing various join scenarios compile_profile: Compile and execute TPC-H across selected Cargo profiles, reporting timing and binary size @@ -311,6 +312,10 @@ main() { # hj uses range() function, no data generation needed echo "HJ benchmark does not require data generation" ;; + smj) + # smj uses range() function, no data generation needed + echo "SMJ benchmark does not require data generation" + ;; compile_profile) data_tpch "1" ;; @@ -384,6 +389,7 @@ main() { run_external_aggr run_nlj run_hj + run_smj ;; tpch) run_tpch "1" "parquet" @@ -494,6 +500,9 @@ main() { hj) run_hj ;; + smj) + run_smj + ;; compile_profile) run_compile_profile "${PROFILE_ARGS[@]}" ;; @@ -1154,6 +1163,14 @@ run_hj() { debug_run $CARGO_COMMAND --bin dfbench -- hj --iterations 5 -o "${RESULTS_FILE}" ${QUERY_ARG} } +# Runs the smj benchmark +run_smj() { + RESULTS_FILE="${RESULTS_DIR}/smj.json" + echo "RESULTS_FILE: ${RESULTS_FILE}" + echo "Running smj benchmark..." + debug_run $CARGO_COMMAND --bin dfbench -- smj --iterations 5 -o "${RESULTS_FILE}" ${QUERY_ARG} +} + compare_benchmarks() { BASE_RESULTS_DIR="${SCRIPT_DIR}/results" diff --git a/benchmarks/src/bin/dfbench.rs b/benchmarks/src/bin/dfbench.rs index 816cae0e3855..2fbc7bef3dca 100644 --- a/benchmarks/src/bin/dfbench.rs +++ b/benchmarks/src/bin/dfbench.rs @@ -34,7 +34,7 @@ static ALLOC: snmalloc_rs::SnMalloc = snmalloc_rs::SnMalloc; static ALLOC: mimalloc::MiMalloc = mimalloc::MiMalloc; use datafusion_benchmarks::{ - cancellation, clickbench, h2o, hj, imdb, nlj, sort_tpch, tpch, + cancellation, clickbench, h2o, hj, imdb, nlj, smj, sort_tpch, tpch, }; #[derive(Debug, StructOpt)] @@ -46,6 +46,7 @@ enum Options { HJ(hj::RunOpt), Imdb(imdb::RunOpt), Nlj(nlj::RunOpt), + Smj(smj::RunOpt), SortTpch(sort_tpch::RunOpt), Tpch(tpch::RunOpt), TpchConvert(tpch::ConvertOpt), @@ -63,6 +64,7 @@ pub async fn main() -> Result<()> { Options::HJ(opt) => opt.run().await, Options::Imdb(opt) => Box::pin(opt.run()).await, Options::Nlj(opt) => opt.run().await, + Options::Smj(opt) => opt.run().await, Options::SortTpch(opt) => opt.run().await, Options::Tpch(opt) => Box::pin(opt.run()).await, Options::TpchConvert(opt) => opt.run().await, diff --git a/benchmarks/src/lib.rs b/benchmarks/src/lib.rs index 07cffa5ae468..5c8c2dcff568 100644 --- a/benchmarks/src/lib.rs +++ b/benchmarks/src/lib.rs @@ -22,6 +22,7 @@ pub mod h2o; pub mod hj; pub mod imdb; pub mod nlj; +pub mod smj; pub mod sort_tpch; pub mod tpch; pub mod util; diff --git a/benchmarks/src/smj.rs b/benchmarks/src/smj.rs new file mode 100644 index 000000000000..32a620a12d4f --- /dev/null +++ b/benchmarks/src/smj.rs @@ -0,0 +1,524 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use crate::util::{BenchmarkRun, CommonOpt, QueryResult}; +use datafusion::physical_plan::execute_stream; +use datafusion::{error::Result, prelude::SessionContext}; +use datafusion_common::instant::Instant; +use datafusion_common::{exec_datafusion_err, exec_err, DataFusionError}; +use structopt::StructOpt; + +use futures::StreamExt; + +/// Run the Sort Merge Join (SMJ) benchmark +/// +/// This micro-benchmark focuses on the performance characteristics of SMJs. +/// +/// It uses equality join predicates (to ensure SMJ is selected) and varies: +/// - Join type: Inner/Left/Right/Full/LeftSemi/LeftAnti/RightSemi/RightAnti +/// - Key cardinality: 1:1, 1:N, N:M relationships +/// - Filter selectivity: Low (1%), Medium (10%), High (50%) +/// - Input sizes: Small to large, balanced and skewed +/// +/// All inputs are pre-sorted in CTEs before the join to isolate join +/// performance from sort overhead. +#[derive(Debug, StructOpt, Clone)] +#[structopt(verbatim_doc_comment)] +pub struct RunOpt { + /// Query number (between 1 and 20). If not specified, runs all queries + #[structopt(short, long)] + query: Option, + + /// Common options + #[structopt(flatten)] + common: CommonOpt, + + /// If present, write results json here + #[structopt(parse(from_os_str), short = "o", long = "output")] + output_path: Option, +} + +/// Inline SQL queries for SMJ benchmarks +/// +/// Each query's comment includes: +/// - Join type +/// - Left row count × Right row count +/// - Key cardinality (rows per key) +/// - Filter selectivity (if applicable) +const SMJ_QUERIES: &[&str] = &[ + // Q1: INNER 100K x 100K | 1:1 + r#" + WITH t1_sorted AS ( + SELECT value as key FROM range(100000) ORDER BY value + ), + t2_sorted AS ( + SELECT value as key FROM range(100000) ORDER BY value + ) + SELECT t1_sorted.key as k1, t2_sorted.key as k2 + FROM t1_sorted JOIN t2_sorted ON t1_sorted.key = t2_sorted.key + "#, + // Q2: INNER 100K x 1M | 1:10 + r#" + WITH t1_sorted AS ( + SELECT value % 10000 as key, value as data + FROM range(100000) + ORDER BY key, data + ), + t2_sorted AS ( + SELECT value % 10000 as key, value as data + FROM range(1000000) + ORDER BY key, data + ) + SELECT t1_sorted.key, t1_sorted.data as d1, t2_sorted.data as d2 + FROM t1_sorted JOIN t2_sorted ON t1_sorted.key = t2_sorted.key + "#, + // Q3: INNER 1M x 1M | 1:100 + r#" + WITH t1_sorted AS ( + SELECT value % 10000 as key, value as data + FROM range(1000000) + ORDER BY key, data + ), + t2_sorted AS ( + SELECT value % 10000 as key, value as data + FROM range(1000000) + ORDER BY key, data + ) + SELECT t1_sorted.key, t1_sorted.data as d1, t2_sorted.data as d2 + FROM t1_sorted JOIN t2_sorted ON t1_sorted.key = t2_sorted.key + "#, + // Q4: INNER 100K x 1M | 1:10 | 1% + r#" + WITH t1_sorted AS ( + SELECT value % 10000 as key, value as data + FROM range(100000) + ORDER BY key, data + ), + t2_sorted AS ( + SELECT value % 10000 as key, value as data + FROM range(1000000) + ORDER BY key, data + ) + SELECT t1_sorted.key, t1_sorted.data as d1, t2_sorted.data as d2 + FROM t1_sorted JOIN t2_sorted ON t1_sorted.key = t2_sorted.key + WHERE t2_sorted.data % 100 = 0 + "#, + // Q5: INNER 1M x 1M | 1:100 | 10% + r#" + WITH t1_sorted AS ( + SELECT value % 10000 as key, value as data + FROM range(1000000) + ORDER BY key, data + ), + t2_sorted AS ( + SELECT value % 10000 as key, value as data + FROM range(1000000) + ORDER BY key, data + ) + SELECT t1_sorted.key, t1_sorted.data as d1, t2_sorted.data as d2 + FROM t1_sorted JOIN t2_sorted ON t1_sorted.key = t2_sorted.key + WHERE t1_sorted.data <> t2_sorted.data AND t2_sorted.data % 10 = 0 + "#, + // Q6: LEFT 100K x 1M | 1:10 + r#" + WITH t1_sorted AS ( + SELECT value % 10500 as key, value as data + FROM range(100000) + ORDER BY key, data + ), + t2_sorted AS ( + SELECT value % 10000 as key, value as data + FROM range(1000000) + ORDER BY key, data + ) + SELECT t1_sorted.key, t1_sorted.data as d1, t2_sorted.data as d2 + FROM t1_sorted LEFT JOIN t2_sorted ON t1_sorted.key = t2_sorted.key + "#, + // Q7: LEFT 100K x 1M | 1:10 | 50% + r#" + WITH t1_sorted AS ( + SELECT value % 10000 as key, value as data + FROM range(100000) + ORDER BY key, data + ), + t2_sorted AS ( + SELECT value % 10000 as key, value as data + FROM range(1000000) + ORDER BY key, data + ) + SELECT t1_sorted.key, t1_sorted.data as d1, t2_sorted.data as d2 + FROM t1_sorted LEFT JOIN t2_sorted ON t1_sorted.key = t2_sorted.key + WHERE t2_sorted.data IS NULL OR t2_sorted.data % 2 = 0 + "#, + // Q8: FULL 100K x 100K | 1:10 + r#" + WITH t1_sorted AS ( + SELECT value % 10000 as key, value as data + FROM range(100000) + ORDER BY key, data + ), + t2_sorted AS ( + SELECT value % 12500 as key, value as data + FROM range(100000) + ORDER BY key, data + ) + SELECT t1_sorted.key as k1, t1_sorted.data as d1, + t2_sorted.key as k2, t2_sorted.data as d2 + FROM t1_sorted FULL JOIN t2_sorted ON t1_sorted.key = t2_sorted.key + "#, + // Q9: FULL 100K x 1M | 1:10 | 10% + r#" + WITH t1_sorted AS ( + SELECT value % 10000 as key, value as data + FROM range(100000) + ORDER BY key, data + ), + t2_sorted AS ( + SELECT value % 10000 as key, value as data + FROM range(1000000) + ORDER BY key, data + ) + SELECT t1_sorted.key as k1, t1_sorted.data as d1, + t2_sorted.key as k2, t2_sorted.data as d2 + FROM t1_sorted FULL JOIN t2_sorted ON t1_sorted.key = t2_sorted.key + WHERE (t1_sorted.data IS NULL OR t2_sorted.data IS NULL + OR t1_sorted.data <> t2_sorted.data) + AND (t1_sorted.data IS NULL OR t1_sorted.data % 10 = 0) + "#, + // Q10: LEFT SEMI 100K x 1M | 1:10 + r#" + WITH t1_sorted AS ( + SELECT value % 10000 as key, value as data + FROM range(100000) + ORDER BY key, data + ), + t2_sorted AS ( + SELECT value % 10000 as key + FROM range(1000000) + ORDER BY key + ) + SELECT t1_sorted.key, t1_sorted.data + FROM t1_sorted + WHERE EXISTS ( + SELECT 1 FROM t2_sorted + WHERE t2_sorted.key = t1_sorted.key + ) + "#, + // Q11: LEFT SEMI 100K x 1M | 1:10 | 1% + r#" + WITH t1_sorted AS ( + SELECT value % 10000 as key, value as data + FROM range(100000) + ORDER BY key, data + ), + t2_sorted AS ( + SELECT value % 10000 as key, value as data + FROM range(1000000) + ORDER BY key, data + ) + SELECT t1_sorted.key, t1_sorted.data + FROM t1_sorted + WHERE EXISTS ( + SELECT 1 FROM t2_sorted + WHERE t2_sorted.key = t1_sorted.key + AND t2_sorted.data <> t1_sorted.data + AND t2_sorted.data % 100 = 0 + ) + "#, + // Q12: LEFT SEMI 100K x 1M | 1:10 | 50% + r#" + WITH t1_sorted AS ( + SELECT value % 10000 as key, value as data + FROM range(100000) + ORDER BY key, data + ), + t2_sorted AS ( + SELECT value % 10000 as key, value as data + FROM range(1000000) + ORDER BY key, data + ) + SELECT t1_sorted.key, t1_sorted.data + FROM t1_sorted + WHERE EXISTS ( + SELECT 1 FROM t2_sorted + WHERE t2_sorted.key = t1_sorted.key + AND t2_sorted.data <> t1_sorted.data + AND t2_sorted.data % 2 = 0 + ) + "#, + // Q13: LEFT SEMI 100K x 1M | 1:10 | 90% + r#" + WITH t1_sorted AS ( + SELECT value % 10000 as key, value as data + FROM range(100000) + ORDER BY key, data + ), + t2_sorted AS ( + SELECT value % 10000 as key, value as data + FROM range(1000000) + ORDER BY key, data + ) + SELECT t1_sorted.key, t1_sorted.data + FROM t1_sorted + WHERE EXISTS ( + SELECT 1 FROM t2_sorted + WHERE t2_sorted.key = t1_sorted.key + AND t2_sorted.data % 10 <> 0 + ) + "#, + // Q14: LEFT ANTI 100K x 1M | 1:10 + r#" + WITH t1_sorted AS ( + SELECT value % 10500 as key, value as data + FROM range(100000) + ORDER BY key, data + ), + t2_sorted AS ( + SELECT value % 10000 as key + FROM range(1000000) + ORDER BY key + ) + SELECT t1_sorted.key, t1_sorted.data + FROM t1_sorted + WHERE NOT EXISTS ( + SELECT 1 FROM t2_sorted + WHERE t2_sorted.key = t1_sorted.key + ) + "#, + // Q15: LEFT ANTI 100K x 1M | 1:10 | partial match + r#" + WITH t1_sorted AS ( + SELECT value % 12000 as key, value as data + FROM range(100000) + ORDER BY key, data + ), + t2_sorted AS ( + SELECT value % 10000 as key + FROM range(1000000) + ORDER BY key + ) + SELECT t1_sorted.key, t1_sorted.data + FROM t1_sorted + WHERE NOT EXISTS ( + SELECT 1 FROM t2_sorted + WHERE t2_sorted.key = t1_sorted.key + ) + "#, + // Q16: LEFT ANTI 100K x 100K | 1:1 | stress + r#" + WITH t1_sorted AS ( + SELECT value % 11000 as key, value as data + FROM range(100000) + ORDER BY key, data + ), + t2_sorted AS ( + SELECT value % 10000 as key + FROM range(100000) + ORDER BY key + ) + SELECT t1_sorted.key, t1_sorted.data + FROM t1_sorted + WHERE NOT EXISTS ( + SELECT 1 FROM t2_sorted + WHERE t2_sorted.key = t1_sorted.key + ) + "#, + // Q17: INNER 100K x 5M | 1:50 | 5% + r#" + WITH t1_sorted AS ( + SELECT value % 10000 as key, value as data + FROM range(100000) + ORDER BY key, data + ), + t2_sorted AS ( + SELECT value % 10000 as key, value as data + FROM range(5000000) + ORDER BY key, data + ) + SELECT t1_sorted.key, t1_sorted.data as d1, t2_sorted.data as d2 + FROM t1_sorted JOIN t2_sorted ON t1_sorted.key = t2_sorted.key + WHERE t2_sorted.data <> t1_sorted.data AND t2_sorted.data % 20 = 0 + "#, + // Q18: LEFT SEMI 100K x 5M | 1:50 | 2% + r#" + WITH t1_sorted AS ( + SELECT value % 10000 as key, value as data + FROM range(100000) + ORDER BY key, data + ), + t2_sorted AS ( + SELECT value % 10000 as key, value as data + FROM range(5000000) + ORDER BY key, data + ) + SELECT t1_sorted.key, t1_sorted.data + FROM t1_sorted + WHERE EXISTS ( + SELECT 1 FROM t2_sorted + WHERE t2_sorted.key = t1_sorted.key + AND t2_sorted.data <> t1_sorted.data + AND t2_sorted.data % 50 = 0 + ) + "#, + // Q19: LEFT ANTI 100K x 5M | 1:50 | partial match + r#" + WITH t1_sorted AS ( + SELECT value % 15000 as key, value as data + FROM range(100000) + ORDER BY key, data + ), + t2_sorted AS ( + SELECT value % 10000 as key + FROM range(5000000) + ORDER BY key + ) + SELECT t1_sorted.key, t1_sorted.data + FROM t1_sorted + WHERE NOT EXISTS ( + SELECT 1 FROM t2_sorted + WHERE t2_sorted.key = t1_sorted.key + ) + "#, + // Q20: INNER 1M x 10M | 1:100 + GROUP BY + r#" + WITH t1_sorted AS ( + SELECT value % 10000 as key, value as data + FROM range(1000000) + ORDER BY key, data + ), + t2_sorted AS ( + SELECT value % 10000 as key, value as data + FROM range(10000000) + ORDER BY key, data + ) + SELECT t1_sorted.key, count(*) as cnt + FROM t1_sorted JOIN t2_sorted ON t1_sorted.key = t2_sorted.key + GROUP BY t1_sorted.key + "#, +]; + +impl RunOpt { + pub async fn run(self) -> Result<()> { + println!("Running SMJ benchmarks with the following options: {self:#?}\n"); + + // Define query range + let query_range = match self.query { + Some(query_id) => { + if query_id >= 1 && query_id <= SMJ_QUERIES.len() { + query_id..=query_id + } else { + return exec_err!( + "Query {query_id} not found. Available queries: 1 to {}", + SMJ_QUERIES.len() + ); + } + } + None => 1..=SMJ_QUERIES.len(), + }; + + let mut config = self.common.config()?; + // Disable hash joins to force SMJ + config = config.set_bool("datafusion.optimizer.prefer_hash_join", false); + let rt_builder = self.common.runtime_env_builder()?; + let ctx = SessionContext::new_with_config_rt(config, rt_builder.build_arc()?); + + let mut benchmark_run = BenchmarkRun::new(); + for query_id in query_range { + let query_index = query_id - 1; // Convert 1-based to 0-based index + + let sql = SMJ_QUERIES[query_index]; + benchmark_run.start_new_case(&format!("Query {query_id}")); + let query_run = self.benchmark_query(sql, &query_id.to_string(), &ctx).await; + match query_run { + Ok(query_results) => { + for iter in query_results { + benchmark_run.write_iter(iter.elapsed, iter.row_count); + } + } + Err(e) => { + return Err(DataFusionError::Context( + format!("SMJ benchmark Q{query_id} failed with error:"), + Box::new(e), + )); + } + } + } + + benchmark_run.maybe_write_json(self.output_path.as_ref())?; + Ok(()) + } + + async fn benchmark_query( + &self, + sql: &str, + query_name: &str, + ctx: &SessionContext, + ) -> Result> { + let mut query_results = vec![]; + + // Validate that the query plan includes a Sort Merge Join + let df = ctx.sql(sql).await?; + let physical_plan = df.create_physical_plan().await?; + let plan_string = format!("{physical_plan:#?}"); + + if !plan_string.contains("SortMergeJoinExec") { + return Err(exec_datafusion_err!( + "Query {query_name} does not use Sort Merge Join. Physical plan: {plan_string}" + )); + } + + for i in 0..self.common.iterations { + let start = Instant::now(); + + let row_count = Self::execute_sql_without_result_buffering(sql, ctx).await?; + + let elapsed = start.elapsed(); + + println!( + "Query {query_name} iteration {i} returned {row_count} rows in {elapsed:?}" + ); + + query_results.push(QueryResult { elapsed, row_count }); + } + + Ok(query_results) + } + + /// Executes the SQL query and drops each result batch after evaluation, to + /// minimizes memory usage by not buffering results. + /// + /// Returns the total result row count + async fn execute_sql_without_result_buffering( + sql: &str, + ctx: &SessionContext, + ) -> Result { + let mut row_count = 0; + + let df = ctx.sql(sql).await?; + let physical_plan = df.create_physical_plan().await?; + let mut stream = execute_stream(physical_plan, ctx.task_ctx())?; + + while let Some(batch) = stream.next().await { + row_count += batch?.num_rows(); + + // Evaluate the result and do nothing, the result will be dropped + // to reduce memory pressure + } + + Ok(row_count) + } +} diff --git a/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs b/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs index 0325e37d42e7..dd86628b2be2 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs @@ -41,7 +41,8 @@ use crate::{PhysicalExpr, RecordBatchStream, SendableRecordBatchStream}; use arrow::array::{types::UInt64Type, *}; use arrow::compute::{ - self, concat_batches, filter_record_batch, is_not_null, take, SortOptions, + self, concat_batches, filter_record_batch, is_not_null, take, BatchCoalescer, + SortOptions, }; use arrow::datatypes::{DataType, SchemaRef, TimeUnit}; use arrow::error::ArrowError; @@ -323,11 +324,15 @@ pub(super) struct SortMergeJoinStream { pub staging_output_record_batches: JoinedRecordBatches, /// Output buffer. Currently used by filtering as it requires double buffering /// to avoid small/empty batches. Non-filtered join outputs directly from `staging_output_record_batches.batches` - pub output: RecordBatch, + /// Uses BatchCoalescer to accumulate small batches efficiently without repeated concatenation. + pub output_buffer: BatchCoalescer, /// Staging output size, including output batches and staging joined results. /// Increased when we put rows into buffer and decreased after we actually output batches. /// Used to trigger output when sufficient rows are ready pub output_size: usize, + /// Flag indicating that staging_output_record_batches coalescer has reached target + /// and should be processed + pub staging_ready: bool, /// The comparison result of current streamed row and buffered batches pub current_ordering: Ordering, /// Manages the process of spilling and reading back intermediate data @@ -350,7 +355,7 @@ pub(super) struct SortMergeJoinStream { /// Joined batches with attached join filter information pub(super) struct JoinedRecordBatches { /// Joined batches. Each batch is already joined columns from left and right sources - pub batches: Vec, + pub coalescer: BatchCoalescer, /// Filter match mask for each row(matched/non-matched) pub filter_mask: BooleanBuilder, /// Left row indices to glue together rows in `batches` and `filter_mask` @@ -363,7 +368,7 @@ pub(super) struct JoinedRecordBatches { impl JoinedRecordBatches { fn clear(&mut self) { - self.batches.clear(); + // Note: BatchCoalescer clears itself in finish_buffered_batch() self.batch_ids.clear(); self.filter_mask = BooleanBuilder::new(); self.row_indices = UInt64Builder::new(); @@ -592,29 +597,42 @@ impl Stream for SortMergeJoinStream { self.freeze_all()?; // If join is filtered and there is joined tuples waiting - // to be filtered - if !self - .staging_output_record_batches - .batches - .is_empty() - { + // to be filtered. Process when coalescer has reached target size. + if self.staging_ready { + // Track buffered row count before draining the coalescer + let pre_filter_row_count = self + .staging_output_record_batches + .coalescer + .get_buffered_rows(); + // Apply filter on joined tuples and get filtered batch let out_filtered_batch = self.filter_joined_batch()?; + // Decrement output_size by the number of unfiltered rows processed. + // output_size tracks unfiltered pairs, but we just processed + // pre_filter_row_count rows from the coalescer. + if pre_filter_row_count > self.output_size { + self.output_size = 0; + } else { + self.output_size -= pre_filter_row_count; + } + + // Reset the flag after processing + self.staging_ready = false; + // Append filtered batch to the output buffer - self.output = concat_batches( - &self.schema(), - [&self.output, &out_filtered_batch], - )?; - - // Send to output if the output buffer surpassed the `batch_size` - if self.output.num_rows() >= self.batch_size { - let record_batch = std::mem::replace( - &mut self.output, - RecordBatch::new_empty( - out_filtered_batch.schema(), - ), + self.output_buffer + .push_batch(out_filtered_batch)?; + if self.output_buffer.has_completed_batch() { + self.output_buffer + .finish_buffered_batch()?; + let record_batch = self + .output_buffer + .next_completed_batch() + .unwrap(); + (&record_batch).record_output( + &self.join_metrics.baseline_metrics(), ); return Poll::Ready(Some(Ok( record_batch, @@ -676,13 +694,11 @@ impl Stream for SortMergeJoinStream { } } else { self.freeze_all()?; - if !self.staging_output_record_batches.batches.is_empty() { - let record_batch = self.output_record_batch_and_reset()?; - // For non-filtered join output whenever the target output batch size - // is hit. For filtered join its needed to output on later phase - // because target output batch size can be hit in the middle of - // filtering causing the filtering to be incomplete and causing - // correctness issues + // Only process if coalescer has reached target + if self.staging_ready { + // For filtered joins, batches accumulate across multiple freeze_all() calls + // and are processed at safe transition points (between streamed batches or + // at Exhausted state). Don't output here in the tight JoinOutput loop. if self.filter.is_some() && matches!( self.join_type, @@ -697,19 +713,31 @@ impl Stream for SortMergeJoinStream { | JoinType::Full ) { + // Keep staging_ready set to let it propagate to Init state + // where it will be processed. Transition to Init state to continue. + self.buffered_data.scanning_reset(); + self.state = SortMergeJoinState::Init; continue; + } else { + // Non-filtered joins output immediately + let record_batch = + self.output_record_batch_and_reset()?; + self.staging_ready = false; + (&record_batch) + .record_output(&self.join_metrics.baseline_metrics()); + return Poll::Ready(Some(Ok(record_batch))); } - - return Poll::Ready(Some(Ok(record_batch))); } - return Poll::Pending; + // Reset scanning and transition to Init to continue processing + self.buffered_data.scanning_reset(); + self.state = SortMergeJoinState::Init; } } SortMergeJoinState::Exhausted => { self.freeze_all()?; - // if there is still something not processed - if !self.staging_output_record_batches.batches.is_empty() { + // if there is still something not processed in coalescer + if !self.staging_output_record_batches.coalescer.is_empty() { if self.filter.is_some() && matches!( self.join_type, @@ -725,18 +753,24 @@ impl Stream for SortMergeJoinStream { ) { let record_batch = self.filter_joined_batch()?; + (&record_batch) + .record_output(&self.join_metrics.baseline_metrics()); return Poll::Ready(Some(Ok(record_batch))); } else { let record_batch = self.output_record_batch_and_reset()?; + (&record_batch) + .record_output(&self.join_metrics.baseline_metrics()); return Poll::Ready(Some(Ok(record_batch))); } - } else if self.output.num_rows() > 0 { + } else if !self.output_buffer.is_empty() { // if processed but still not outputted because it didn't hit batch size before - let schema = self.output.schema(); - let record_batch = std::mem::replace( - &mut self.output, - RecordBatch::new_empty(schema), - ); + self.output_buffer.finish_buffered_batch()?; + let record_batch = + self.output_buffer.next_completed_batch().unwrap_or_else( + || RecordBatch::new_empty(Arc::clone(&self.schema)), + ); + (&record_batch) + .record_output(&self.join_metrics.baseline_metrics()); return Poll::Ready(Some(Ok(record_batch))); } else { return Poll::Ready(None); @@ -794,13 +828,16 @@ impl SortMergeJoinStream { on_buffered, filter, staging_output_record_batches: JoinedRecordBatches { - batches: vec![], + coalescer: BatchCoalescer::new(Arc::clone(&schema), batch_size) + .with_biggest_coalesce_batch_size(Some(batch_size / 2)), filter_mask: BooleanBuilder::new(), row_indices: UInt64Builder::new(), batch_ids: vec![], }, - output: RecordBatch::new_empty(schema), + output_buffer: BatchCoalescer::new(Arc::clone(&schema), batch_size) + .with_biggest_coalesce_batch_size(Some(batch_size / 2)), output_size: 0, + staging_ready: false, batch_size, join_type, join_metrics, @@ -1202,8 +1239,15 @@ impl SortMergeJoinStream { ); self.staging_output_record_batches - .batches - .push(record_batch); + .coalescer + .push_batch(record_batch)?; + if self + .staging_output_record_batches + .coalescer + .has_completed_batch() + { + self.staging_ready = true; + } } buffered_batch.null_joined.clear(); } @@ -1248,8 +1292,15 @@ impl SortMergeJoinStream { 0, ); self.staging_output_record_batches - .batches - .push(record_batch); + .coalescer + .push_batch(record_batch)?; + if self + .staging_output_record_batches + .coalescer + .has_completed_batch() + { + self.staging_ready = true; + } } buffered_batch.join_filter_not_matched_map.clear(); @@ -1259,6 +1310,7 @@ impl SortMergeJoinStream { // Produces and stages record batch for all output indices found // for current streamed batch and clears staged output indices. fn freeze_streamed(&mut self) -> Result<()> { + let mut rows_processed = 0; for chunk in self.streamed_batch.output_indices.iter_mut() { // The row indices of joined streamed batch let left_indices = chunk.streamed_indices.finish(); @@ -1267,6 +1319,8 @@ impl SortMergeJoinStream { continue; } + rows_processed += left_indices.len(); + let mut left_columns = self .streamed_batch .batch @@ -1391,13 +1445,26 @@ impl SortMergeJoinStream { | JoinType::Full ) { self.staging_output_record_batches - .batches - .push(output_batch); + .coalescer + .push_batch(output_batch)?; + if self + .staging_output_record_batches + .coalescer + .has_completed_batch() + { + self.staging_ready = true; + } } else { - let filtered_batch = filter_record_batch(&output_batch, &mask)?; self.staging_output_record_batches - .batches - .push(filtered_batch); + .coalescer + .push_batch_with_filter(output_batch, &mask)?; + if self + .staging_output_record_batches + .coalescer + .has_completed_batch() + { + self.staging_ready = true; + } } if !matches!(self.join_type, JoinType::Full) { @@ -1445,25 +1512,71 @@ impl SortMergeJoinStream { } } else { self.staging_output_record_batches - .batches - .push(output_batch); + .coalescer + .push_batch(output_batch)?; + if self + .staging_output_record_batches + .coalescer + .has_completed_batch() + { + self.staging_ready = true; + } } } else { self.staging_output_record_batches - .batches - .push(output_batch); + .coalescer + .push_batch(output_batch)?; + if self + .staging_output_record_batches + .coalescer + .has_completed_batch() + { + self.staging_ready = true; + } } } self.streamed_batch.output_indices.clear(); + // Decrement output_size by the number of rows we just processed and added to the coalescer + if rows_processed > self.output_size { + self.output_size = 0; + } else { + self.output_size -= rows_processed; + } + + // After clearing output_indices, if the coalescer has buffered data but hasn't + // reached the target yet, we may need to force a flush to prevent deadlock. + // This is only necessary for filtered joins where partial batches can accumulate + // without reaching the target batch size. + if !self.staging_output_record_batches.coalescer.is_empty() + && !self.staging_ready + && self.filter.is_some() + { + self.staging_output_record_batches + .coalescer + .finish_buffered_batch()?; + if self + .staging_output_record_batches + .coalescer + .has_completed_batch() + { + self.staging_ready = true; + } + } + Ok(()) } fn output_record_batch_and_reset(&mut self) -> Result { - let record_batch = - concat_batches(&self.schema, &self.staging_output_record_batches.batches)?; - (&record_batch).record_output(&self.join_metrics.baseline_metrics()); + self.staging_output_record_batches + .coalescer + .finish_buffered_batch()?; + let record_batch = self + .staging_output_record_batches + .coalescer + .next_completed_batch() + .unwrap_or_else(|| RecordBatch::new_empty(Arc::clone(&self.schema))); // If join filter exists, `self.output_size` is not accurate as we don't know the exact // number of rows in the output record batch. If streamed row joined with buffered rows, // once join filter is applied, the number of output rows may be more than 1. @@ -1475,29 +1588,25 @@ impl SortMergeJoinStream { self.output_size -= record_batch.num_rows(); } - if !(self.filter.is_some() - && matches!( - self.join_type, - JoinType::Left - | JoinType::LeftSemi - | JoinType::Right - | JoinType::RightSemi - | JoinType::LeftAnti - | JoinType::RightAnti - | JoinType::LeftMark - | JoinType::RightMark - | JoinType::Full - )) - { - self.staging_output_record_batches.batches.clear(); - } + // Note: coalescer is already cleared by finish_buffered_batch() above + // The metadata MUST also be cleared since the batches they refer to are gone. + // For non-filtered joins, clear everything immediately. + // For filtered joins, this path shouldn't be hit (they use filter_joined_batch), + // but if it is, we still need to clear to avoid desync. + self.staging_output_record_batches.clear(); Ok(record_batch) } fn filter_joined_batch(&mut self) -> Result { - let record_batch = - concat_batches(&self.schema, &self.staging_output_record_batches.batches)?; + self.staging_output_record_batches + .coalescer + .finish_buffered_batch()?; + let record_batch = self + .staging_output_record_batches + .coalescer + .next_completed_batch() + .unwrap_or_else(|| RecordBatch::new_empty(Arc::clone(&self.schema))); let mut out_indices = self.staging_output_record_batches.row_indices.finish(); let mut out_mask = self.staging_output_record_batches.filter_mask.finish(); let mut batch_ids = &self.staging_output_record_batches.batch_ids; @@ -1515,7 +1624,11 @@ impl SortMergeJoinStream { } if out_mask.is_empty() { - self.staging_output_record_batches.batches.clear(); + // Coalescer already cleared by finish_buffered_batch() above + // Clear metadata only + self.staging_output_record_batches.batch_ids.clear(); + self.staging_output_record_batches.filter_mask = BooleanBuilder::new(); + self.staging_output_record_batches.row_indices = UInt64Builder::new(); return Ok(record_batch); } diff --git a/datafusion/physical-plan/src/joins/sort_merge_join/tests.rs b/datafusion/physical-plan/src/joins/sort_merge_join/tests.rs index f91bffbed78f..842480b4268b 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join/tests.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join/tests.rs @@ -2319,14 +2319,14 @@ fn build_joined_record_batches() -> Result { ])); let mut batches = JoinedRecordBatches { - batches: vec![], + coalescer: arrow::compute::BatchCoalescer::new(Arc::clone(&schema), 8192), filter_mask: BooleanBuilder::new(), row_indices: UInt64Builder::new(), batch_ids: vec![], }; // Insert already prejoined non-filtered rows - batches.batches.push(RecordBatch::try_new( + batches.coalescer.push_batch(RecordBatch::try_new( Arc::clone(&schema), vec![ Arc::new(Int32Array::from(vec![1, 1])), @@ -2334,9 +2334,9 @@ fn build_joined_record_batches() -> Result { Arc::new(Int32Array::from(vec![1, 1])), Arc::new(Int32Array::from(vec![11, 9])), ], - )?); + )?)?; - batches.batches.push(RecordBatch::try_new( + batches.coalescer.push_batch(RecordBatch::try_new( Arc::clone(&schema), vec![ Arc::new(Int32Array::from(vec![1])), @@ -2344,9 +2344,9 @@ fn build_joined_record_batches() -> Result { Arc::new(Int32Array::from(vec![1])), Arc::new(Int32Array::from(vec![12])), ], - )?); + )?)?; - batches.batches.push(RecordBatch::try_new( + batches.coalescer.push_batch(RecordBatch::try_new( Arc::clone(&schema), vec![ Arc::new(Int32Array::from(vec![1, 1])), @@ -2354,9 +2354,9 @@ fn build_joined_record_batches() -> Result { Arc::new(Int32Array::from(vec![1, 1])), Arc::new(Int32Array::from(vec![11, 13])), ], - )?); + )?)?; - batches.batches.push(RecordBatch::try_new( + batches.coalescer.push_batch(RecordBatch::try_new( Arc::clone(&schema), vec![ Arc::new(Int32Array::from(vec![1])), @@ -2364,9 +2364,9 @@ fn build_joined_record_batches() -> Result { Arc::new(Int32Array::from(vec![1])), Arc::new(Int32Array::from(vec![12])), ], - )?); + )?)?; - batches.batches.push(RecordBatch::try_new( + batches.coalescer.push_batch(RecordBatch::try_new( Arc::clone(&schema), vec![ Arc::new(Int32Array::from(vec![1, 1])), @@ -2374,7 +2374,7 @@ fn build_joined_record_batches() -> Result { Arc::new(Int32Array::from(vec![1, 1])), Arc::new(Int32Array::from(vec![12, 11])), ], - )?); + )?)?; let streamed_indices = vec![0, 0]; batches.batch_ids.extend(vec![0; streamed_indices.len()]); @@ -2424,9 +2424,16 @@ fn build_joined_record_batches() -> Result { #[tokio::test] async fn test_left_outer_join_filtered_mask() -> Result<()> { let mut joined_batches = build_joined_record_batches()?; - let schema = joined_batches.batches.first().unwrap().schema(); - let output = concat_batches(&schema, &joined_batches.batches)?; + // Extract the batches from the coalescer + joined_batches.coalescer.finish_buffered_batch()?; + let mut batches_vec = vec![]; + while let Some(batch) = joined_batches.coalescer.next_completed_batch() { + batches_vec.push(batch); + } + let schema = batches_vec.first().unwrap().schema(); + + let output = concat_batches(&schema, &batches_vec)?; let out_mask = joined_batches.filter_mask.finish(); let out_indices = joined_batches.row_indices.finish(); @@ -2631,9 +2638,16 @@ async fn test_left_outer_join_filtered_mask() -> Result<()> { async fn test_semi_join_filtered_mask() -> Result<()> { for join_type in [LeftSemi, RightSemi] { let mut joined_batches = build_joined_record_batches()?; - let schema = joined_batches.batches.first().unwrap().schema(); - let output = concat_batches(&schema, &joined_batches.batches)?; + // Extract the batches from the coalescer + joined_batches.coalescer.finish_buffered_batch()?; + let mut batches_vec = vec![]; + while let Some(batch) = joined_batches.coalescer.next_completed_batch() { + batches_vec.push(batch); + } + let schema = batches_vec.first().unwrap().schema(); + + let output = concat_batches(&schema, &batches_vec)?; let out_mask = joined_batches.filter_mask.finish(); let out_indices = joined_batches.row_indices.finish(); @@ -2806,9 +2820,16 @@ async fn test_semi_join_filtered_mask() -> Result<()> { async fn test_anti_join_filtered_mask() -> Result<()> { for join_type in [LeftAnti, RightAnti] { let mut joined_batches = build_joined_record_batches()?; - let schema = joined_batches.batches.first().unwrap().schema(); - let output = concat_batches(&schema, &joined_batches.batches)?; + // Extract the batches from the coalescer + joined_batches.coalescer.finish_buffered_batch()?; + let mut batches_vec = vec![]; + while let Some(batch) = joined_batches.coalescer.next_completed_batch() { + batches_vec.push(batch); + } + let schema = batches_vec.first().unwrap().schema(); + + let output = concat_batches(&schema, &batches_vec)?; let out_mask = joined_batches.filter_mask.finish(); let out_indices = joined_batches.row_indices.finish(); From a655212c2fc38f044835778b16f4ec5bc42cea2b Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 24 Nov 2025 20:05:52 -0500 Subject: [PATCH 02/23] stash --- Cargo.lock | 842 +++++++++--------- .../src/joins/sort_merge_join/stream.rs | 261 ++---- .../src/joins/sort_merge_join/tests.rs | 55 +- 3 files changed, 510 insertions(+), 648 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 4a896ff25d2e..ad9dba4eb5da 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -83,9 +83,9 @@ dependencies = [ [[package]] name = "aho-corasick" -version = "1.1.3" +version = "1.1.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8e60d3430d3a69478ad0993f19238d2df97c507009a52b3c10addcd7f6bcb916" +checksum = "ddd31a130427c27518df266943a5308ed92d4b226cc639f5a8f1002816174301" dependencies = [ "memchr", ] @@ -128,9 +128,9 @@ checksum = "4b46cbb362ab8752921c97e041f5e366ee6297bd428a31275b9fcf1e380f7299" [[package]] name = "anstream" -version = "0.6.20" +version = "0.6.21" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3ae563653d1938f79b1ab1b5e668c87c76a9930414574a6583a7b7e11a8e6192" +checksum = "43d5b281e737544384e969a5ccad3f1cdd24b48086a0fc1b2a5262a26b8f4f4a" dependencies = [ "anstyle", "anstyle-parse", @@ -143,9 +143,9 @@ dependencies = [ [[package]] name = "anstyle" -version = "1.0.11" +version = "1.0.13" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "862ed96ca487e809f1c8e5a8447f6ee2cf102f846893800b20cebdf541fc6bbd" +checksum = "5192cca8006f1fd4f7237516f40fa183bb07f8fbdfedaa0036de5ea9b0b45e78" [[package]] name = "anstyle-parse" @@ -158,22 +158,22 @@ dependencies = [ [[package]] name = "anstyle-query" -version = "1.1.4" +version = "1.1.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9e231f6134f61b71076a3eab506c379d4f36122f2af15a9ff04415ea4c3339e2" +checksum = "40c48f72fd53cd289104fc64099abca73db4166ad86ea0b4341abe65af83dadc" dependencies = [ - "windows-sys 0.60.2", + "windows-sys 0.61.2", ] [[package]] name = "anstyle-wincon" -version = "3.0.10" +version = "3.0.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3e0633414522a32ffaac8ac6cc8f748e090c5717661fddeea04219e2344f5f2a" +checksum = "291e6a250ff86cd4a820112fb8898808a366d8f9f58ce16d1f538353ad55747d" dependencies = [ "anstyle", "once_cell_polyfill", - "windows-sys 0.60.2", + "windows-sys 0.61.2", ] [[package]] @@ -204,13 +204,22 @@ dependencies = [ "serde_bytes", "serde_json", "snap", - "strum 0.27.2", - "strum_macros 0.27.2", + "strum", + "strum_macros", "thiserror", "uuid", "zstd", ] +[[package]] +name = "ar_archive_writer" +version = "0.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f0c269894b6fe5e9d7ada0cf69b5bf847ff35bc25fc271f08e1d080fce80339a" +dependencies = [ + "object", +] + [[package]] name = "arrayref" version = "0.3.9" @@ -440,7 +449,7 @@ version = "57.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9cfa7a03d1eee2a4d061476e1840ad5c9867a544ca6c4c59256496af5d0a8be5" dependencies = [ - "bitflags 2.9.4", + "bitflags 2.10.0", "serde", "serde_core", "serde_json", @@ -507,9 +516,9 @@ dependencies = [ [[package]] name = "async-compression" -version = "0.4.34" +version = "0.4.33" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0e86f6d3dc9dc4352edeea6b8e499e13e3f5dc3b964d7ca5fd411415a3498473" +checksum = "93c1f86859c1af3d514fa19e8323147ff10ea98684e6c7b307912509f50e67b2" dependencies = [ "compression-codecs", "compression-core", @@ -535,7 +544,7 @@ checksum = "3b43422f69d8ff38f95f1b2bb76517c91589a924d1559a0e935d7c8ce0274c11" dependencies = [ "proc-macro2", "quote", - "syn 2.0.108", + "syn 2.0.110", ] [[package]] @@ -557,7 +566,7 @@ checksum = "c7c24de15d275a1ecfd47a380fb4d5ec9bfe0933f309ed5e705b775596a3574d" dependencies = [ "proc-macro2", "quote", - "syn 2.0.108", + "syn 2.0.110", ] [[package]] @@ -568,7 +577,7 @@ checksum = "9035ad2d096bed7955a320ee7e2230574d28fd3c3a0f186cbea1ff3c7eed5dbb" dependencies = [ "proc-macro2", "quote", - "syn 2.0.108", + "syn 2.0.110", ] [[package]] @@ -594,9 +603,9 @@ checksum = "c08606f8c3cbf4ce6ec8e28fb0014a2c086708fe954eaa885384a6165172e7e8" [[package]] name = "aws-config" -version = "1.8.7" +version = "1.8.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "04b37ddf8d2e9744a0b9c19ce0b78efe4795339a90b66b7bae77987092cd2e69" +checksum = "1856b1b48b65f71a4dd940b1c0931f9a7b646d4a924b9828ffefc1454714668a" dependencies = [ "aws-credential-types", "aws-runtime", @@ -624,9 +633,9 @@ dependencies = [ [[package]] name = "aws-credential-types" -version = "1.2.7" +version = "1.2.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "799a1290207254984cb7c05245111bc77958b92a3c9bb449598044b36341cce6" +checksum = "86590e57ea40121d47d3f2e131bfd873dea15d78dc2f4604f4734537ad9e56c4" dependencies = [ "aws-smithy-async", "aws-smithy-runtime-api", @@ -636,9 +645,9 @@ dependencies = [ [[package]] name = "aws-lc-rs" -version = "1.14.0" +version = "1.15.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "94b8ff6c09cd57b16da53641caa860168b88c172a5ee163b0288d3d6eea12786" +checksum = "5932a7d9d28b0d2ea34c6b3779d35e3dd6f6345317c34e73438c4f1f29144151" dependencies = [ "aws-lc-sys", "zeroize", @@ -646,9 +655,9 @@ dependencies = [ [[package]] name = "aws-lc-sys" -version = "0.31.0" +version = "0.33.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0e44d16778acaf6a9ec9899b92cebd65580b83f685446bf2e1f5d3d732f99dcd" +checksum = "1826f2e4cfc2cd19ee53c42fbf68e2f81ec21108e0b7ecf6a71cf062137360fc" dependencies = [ "bindgen", "cc", @@ -659,9 +668,9 @@ dependencies = [ [[package]] name = "aws-runtime" -version = "1.5.11" +version = "1.5.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2e1ed337dabcf765ad5f2fb426f13af22d576328aaf09eac8f70953530798ec0" +checksum = "8fe0fd441565b0b318c76e7206c8d1d0b0166b3e986cf30e890b61feb6192045" dependencies = [ "aws-credential-types", "aws-sigv4", @@ -683,9 +692,9 @@ dependencies = [ [[package]] name = "aws-sdk-sso" -version = "1.85.0" +version = "1.89.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2f2c741e2e439f07b5d1b33155e246742353d82167c785a2ff547275b7e32483" +checksum = "a9c1b1af02288f729e95b72bd17988c009aa72e26dcb59b3200f86d7aea726c9" dependencies = [ "aws-credential-types", "aws-runtime", @@ -705,9 +714,9 @@ dependencies = [ [[package]] name = "aws-sdk-ssooidc" -version = "1.87.0" +version = "1.91.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6428ae5686b18c0ee99f6f3c39d94ae3f8b42894cdc35c35d8fb2470e9db2d4c" +checksum = "4e8122301558dc7c6c68e878af918880b82ff41897a60c8c4e18e4dc4d93e9f1" dependencies = [ "aws-credential-types", "aws-runtime", @@ -727,9 +736,9 @@ dependencies = [ [[package]] name = "aws-sdk-sts" -version = "1.87.0" +version = "1.92.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5871bec9a79a3e8d928c7788d654f135dde0e71d2dd98089388bab36b37ef607" +checksum = "a0c7808adcff8333eaa76a849e6de926c6ac1a1268b9fd6afe32de9c29ef29d2" dependencies = [ "aws-credential-types", "aws-runtime", @@ -750,9 +759,9 @@ dependencies = [ [[package]] name = "aws-sigv4" -version = "1.3.4" +version = "1.3.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "084c34162187d39e3740cb635acd73c4e3a551a36146ad6fe8883c929c9f876c" +checksum = "c35452ec3f001e1f2f6db107b6373f1f48f05ec63ba2c5c9fa91f07dad32af11" dependencies = [ "aws-credential-types", "aws-smithy-http", @@ -772,9 +781,9 @@ dependencies = [ [[package]] name = "aws-smithy-async" -version = "1.2.5" +version = "1.2.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1e190749ea56f8c42bf15dd76c65e14f8f765233e6df9b0506d9d934ebef867c" +checksum = "127fcfad33b7dfc531141fda7e1c402ac65f88aca5511a4d31e2e3d2cd01ce9c" dependencies = [ "futures-util", "pin-project-lite", @@ -783,15 +792,16 @@ dependencies = [ [[package]] name = "aws-smithy-http" -version = "0.62.3" +version = "0.62.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7c4dacf2d38996cf729f55e7a762b30918229917eca115de45dfa8dfb97796c9" +checksum = "445d5d720c99eed0b4aa674ed00d835d9b1427dd73e04adaf2f94c6b2d6f9fca" dependencies = [ "aws-smithy-runtime-api", "aws-smithy-types", "bytes", "bytes-utils", "futures-core", + "futures-util", "http 0.2.12", "http 1.3.1", "http-body 0.4.6", @@ -803,9 +813,9 @@ dependencies = [ [[package]] name = "aws-smithy-http-client" -version = "1.1.1" +version = "1.1.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "147e8eea63a40315d704b97bf9bc9b8c1402ae94f89d5ad6f7550d963309da1b" +checksum = "623254723e8dfd535f566ee7b2381645f8981da086b5c4aa26c0c41582bb1d2c" dependencies = [ "aws-smithy-async", "aws-smithy-runtime-api", @@ -827,27 +837,27 @@ dependencies = [ [[package]] name = "aws-smithy-json" -version = "0.61.5" +version = "0.61.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "eaa31b350998e703e9826b2104dd6f63be0508666e1aba88137af060e8944047" +checksum = "2db31f727935fc63c6eeae8b37b438847639ec330a9161ece694efba257e0c54" dependencies = [ "aws-smithy-types", ] [[package]] name = "aws-smithy-observability" -version = "0.1.3" +version = "0.1.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9364d5989ac4dd918e5cc4c4bdcc61c9be17dcd2586ea7f69e348fc7c6cab393" +checksum = "2d1881b1ea6d313f9890710d65c158bdab6fb08c91ea825f74c1c8c357baf4cc" dependencies = [ "aws-smithy-runtime-api", ] [[package]] name = "aws-smithy-query" -version = "0.60.7" +version = "0.60.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f2fbd61ceb3fe8a1cb7352e42689cec5335833cd9f94103a61e98f9bb61c64bb" +checksum = "d28a63441360c477465f80c7abac3b9c4d075ca638f982e605b7dc2a2c7156c9" dependencies = [ "aws-smithy-types", "urlencoding", @@ -855,9 +865,9 @@ dependencies = [ [[package]] name = "aws-smithy-runtime" -version = "1.9.2" +version = "1.9.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4fa63ad37685ceb7762fa4d73d06f1d5493feb88e3f27259b9ed277f4c01b185" +checksum = "0bbe9d018d646b96c7be063dd07987849862b0e6d07c778aad7d93d1be6c1ef0" dependencies = [ "aws-smithy-async", "aws-smithy-http", @@ -879,9 +889,9 @@ dependencies = [ [[package]] name = "aws-smithy-runtime-api" -version = "1.9.0" +version = "1.9.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "07f5e0fc8a6b3f2303f331b94504bbf754d85488f402d6f1dd7a6080f99afe56" +checksum = "ec7204f9fd94749a7c53b26da1b961b4ac36bf070ef1e0b94bb09f79d4f6c193" dependencies = [ "aws-smithy-async", "aws-smithy-types", @@ -896,9 +906,9 @@ dependencies = [ [[package]] name = "aws-smithy-types" -version = "1.3.2" +version = "1.3.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d498595448e43de7f4296b7b7a18a8a02c61ec9349128c80a368f7c3b4ab11a8" +checksum = "25f535879a207fce0db74b679cfc3e91a3159c8144d717d55f5832aea9eef46e" dependencies = [ "base64-simd", "bytes", @@ -919,18 +929,18 @@ dependencies = [ [[package]] name = "aws-smithy-xml" -version = "0.60.10" +version = "0.60.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3db87b96cb1b16c024980f133968d52882ca0daaee3a086c6decc500f6c99728" +checksum = "eab77cdd036b11056d2a30a7af7b775789fb024bf216acc13884c6c97752ae56" dependencies = [ "xmlparser", ] [[package]] name = "aws-types" -version = "1.3.8" +version = "1.3.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b069d19bf01e46298eaedd7c6f283fe565a59263e53eebec945f3e6398f42390" +checksum = "d79fb68e3d7fe5d4833ea34dc87d2e97d26d3086cb3da660bb6b1f76d98680b6" dependencies = [ "aws-credential-types", "aws-smithy-async", @@ -942,9 +952,9 @@ dependencies = [ [[package]] name = "axum" -version = "0.8.4" +version = "0.8.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "021e862c184ae977658b36c4500f7feac3221ca5da43e3f25bd04ab6c79a29b5" +checksum = "8a18ed336352031311f4e0b4dd2ff392d4fbb370777c9d18d7fc9d7359f73871" dependencies = [ "axum-core", "bytes", @@ -958,8 +968,7 @@ dependencies = [ "mime", "percent-encoding", "pin-project-lite", - "rustversion", - "serde", + "serde_core", "sync_wrapper", "tower", "tower-layer", @@ -968,9 +977,9 @@ dependencies = [ [[package]] name = "axum-core" -version = "0.5.2" +version = "0.5.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "68464cd0412f486726fb3373129ef5d2993f90c34bc2bc1c1e9943b2f4fc7ca6" +checksum = "59446ce19cd142f8833f856eb31f3eb097812d1479ab224f54d72428ca21ea22" dependencies = [ "bytes", "futures-core", @@ -979,7 +988,6 @@ dependencies = [ "http-body-util", "mime", "pin-project-lite", - "rustversion", "sync_wrapper", "tower-layer", "tower-service", @@ -1027,7 +1035,7 @@ version = "0.72.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "993776b509cfb49c750f11b8f07a46fa23e0a1386ffc01fb1e7d343efc387895" dependencies = [ - "bitflags 2.9.4", + "bitflags 2.10.0", "cexpr", "clang-sys", "itertools 0.13.0", @@ -1038,7 +1046,7 @@ dependencies = [ "regex", "rustc-hash", "shlex", - "syn 2.0.108", + "syn 2.0.110", ] [[package]] @@ -1049,9 +1057,9 @@ checksum = "bef38d45163c2f1dde094a7dfd33ccf595c92905c8f8f4fdc18d06fb1037718a" [[package]] name = "bitflags" -version = "2.9.4" +version = "2.10.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2261d10cca569e4643e526d8dc2e62e433cc8aba21ab764233731f8d369bf394" +checksum = "812e12b5285cc515a9c72a5c1d3b6d46a19dac5acfef5265968c166106e31dd3" [[package]] name = "bitvec" @@ -1098,13 +1106,13 @@ dependencies = [ [[package]] name = "bollard" -version = "0.19.3" +version = "0.19.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ec7646ee90964aa59e9f832a67182791396a19a5b1d76eb17599a8310a7e2e09" +checksum = "87a52479c9237eb04047ddb94788c41ca0d26eaff8b697ecfbb4c32f7fdc3b1b" dependencies = [ "async-stream", "base64 0.22.1", - "bitflags 2.9.4", + "bitflags 2.10.0", "bollard-buildkit-proto", "bollard-stubs", "bytes", @@ -1195,7 +1203,7 @@ dependencies = [ "proc-macro2", "quote", "rustversion", - "syn 2.0.108", + "syn 2.0.110", ] [[package]] @@ -1218,7 +1226,7 @@ dependencies = [ "proc-macro-crate", "proc-macro2", "quote", - "syn 2.0.108", + "syn 2.0.110", ] [[package]] @@ -1244,9 +1252,9 @@ dependencies = [ [[package]] name = "bstr" -version = "1.12.0" +version = "1.12.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "234113d19d0d7d613b40e86fb654acf958910802bcceab913a4f9e7cda03b1a4" +checksum = "63044e1ae8e69f3b5a92c736ca6269b8d12fa7efe39bf34ddb06d102cf0e2cab" dependencies = [ "memchr", "serde", @@ -1319,9 +1327,9 @@ checksum = "37b2a672a2cb129a2e41c10b1224bb368f9f37a2b16b612598138befd7b37eb5" [[package]] name = "cc" -version = "1.2.38" +version = "1.2.46" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "80f41ae168f955c12fb8960b057d70d0ca153fb83182b57d86380443527be7e9" +checksum = "b97463e1064cb1b1c1384ad0a0b9c8abd0988e2a91f52606c80ef14aadb63e36" dependencies = [ "find-msvc-tools", "jobserver", @@ -1340,9 +1348,9 @@ dependencies = [ [[package]] name = "cfg-if" -version = "1.0.3" +version = "1.0.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2fd1289c04a9ea8cb22300a459a72a385d7c73d3259e2ed7dcb2af674838cfa9" +checksum = "9330f8b2ff13f34540b44e946ef35111825727b38d33286ef986142615121801" [[package]] name = "cfg_aliases" @@ -1361,7 +1369,7 @@ dependencies = [ "num-traits", "serde", "wasm-bindgen", - "windows-link 0.2.0", + "windows-link 0.2.1", ] [[package]] @@ -1425,9 +1433,9 @@ dependencies = [ [[package]] name = "clap" -version = "4.5.50" +version = "4.5.51" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0c2cfd7bf8a6017ddaa4e32ffe7403d547790db06bd171c1c53926faab501623" +checksum = "4c26d721170e0295f191a69bd9a1f93efcdb0aff38684b61ab5750468972e5f5" dependencies = [ "clap_builder", "clap_derive", @@ -1435,9 +1443,9 @@ dependencies = [ [[package]] name = "clap_builder" -version = "4.5.50" +version = "4.5.51" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0a4c05b9e80c5ccd3a7ef080ad7b6ba7d6fc00a985b8b157197075677c82c7a0" +checksum = "75835f0c7bf681bfd05abe44e965760fea999a5286c6eb2d59883634fd02011a" dependencies = [ "anstream", "anstyle", @@ -1454,14 +1462,14 @@ dependencies = [ "heck 0.5.0", "proc-macro2", "quote", - "syn 2.0.108", + "syn 2.0.110", ] [[package]] name = "clap_lex" -version = "0.7.5" +version = "0.7.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b94f61472cee1439c0b966b47e3aca9ae07e45d070759512cd390ea2bebc6675" +checksum = "a1d728cc89cf3aee9ff92b05e62b19ee65a02b5702cff7d5a377e32c6ae29d8d" [[package]] name = "clipboard-win" @@ -1489,20 +1497,19 @@ checksum = "b05b61dc5112cbb17e4b6cd61790d9845d13888356391624cbe7e41efeac1e75" [[package]] name = "comfy-table" -version = "7.1.2" +version = "7.2.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e0d05af1e006a2407bedef5af410552494ce5be9090444dbbcb57258c1af3d56" +checksum = "b03b7db8e0b4b2fdad6c551e634134e99ec000e5c8c3b6856c65e8bbaded7a3b" dependencies = [ - "strum 0.26.3", - "strum_macros 0.26.4", - "unicode-width 0.2.1", + "unicode-segmentation", + "unicode-width 0.2.2", ] [[package]] name = "compression-codecs" -version = "0.4.33" +version = "0.4.32" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "302266479cb963552d11bd042013a58ef1adc56768016c8b82b4199488f2d4ad" +checksum = "680dc087785c5230f8e8843e2e57ac7c1c90488b6a91b88caa265410568f441b" dependencies = [ "bzip2", "compression-core", @@ -1515,9 +1522,9 @@ dependencies = [ [[package]] name = "compression-core" -version = "0.4.31" +version = "0.4.30" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "75984efb6ed102a0d42db99afb6c1948f0380d1d91808d5529916e6c08b49d8d" +checksum = "3a9b614a5787ef0c8802a55766480563cb3a93b435898c422ed2a359cf811582" [[package]] name = "console" @@ -1540,8 +1547,8 @@ dependencies = [ "encode_unicode", "libc", "once_cell", - "unicode-width 0.2.1", - "windows-sys 0.61.0", + "unicode-width 0.2.2", + "windows-sys 0.61.2", ] [[package]] @@ -1647,7 +1654,7 @@ dependencies = [ "anes", "cast", "ciborium", - "clap 4.5.50", + "clap 4.5.51", "criterion-plot", "futures", "itertools 0.13.0", @@ -1715,9 +1722,9 @@ checksum = "460fbee9c2c2f33933d720630a6a0bac33ba7053db5344fac858d4b8952d77d5" [[package]] name = "crypto-common" -version = "0.1.6" +version = "0.1.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1bfb12502f3fc46cca1bb51ac28df9d618d813cdc3d2f25b9fe775a34af26bb3" +checksum = "78c8292055d1c1df0cce5d180393dc8cce0abec0a7102adb6c7b1eef6016d60a" dependencies = [ "generic-array", "typenum", @@ -1725,21 +1732,21 @@ dependencies = [ [[package]] name = "csv" -version = "1.3.1" +version = "1.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "acdc4883a9c96732e4733212c01447ebd805833b7275a73ca3ee080fd77afdaf" +checksum = "52cd9d68cf7efc6ddfaaee42e7288d3a99d613d4b50f76ce9827ae0c6e14f938" dependencies = [ "csv-core", "itoa", "ryu", - "serde", + "serde_core", ] [[package]] name = "csv-core" -version = "0.1.12" +version = "0.1.13" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7d02f3b0da4c6504f86e9cd789d8dbafab48c2321be74e9987593de5a894d93d" +checksum = "704a3c26996a80471189265814dbc2c257598b96b8a7feae2d31ace646bb9782" dependencies = [ "memchr", ] @@ -1787,7 +1794,7 @@ dependencies = [ "proc-macro2", "quote", "strsim", - "syn 2.0.108", + "syn 2.0.110", ] [[package]] @@ -1798,7 +1805,7 @@ checksum = "d38308df82d1080de0afee5d069fa14b0326a88c14f15c5ccda35b4a6c414c81" dependencies = [ "darling_core", "quote", - "syn 2.0.108", + "syn 2.0.110", ] [[package]] @@ -1966,7 +1973,7 @@ dependencies = [ "aws-config", "aws-credential-types", "chrono", - "clap 4.5.50", + "clap 4.5.51", "ctor", "datafusion", "datafusion-common", @@ -2419,7 +2426,7 @@ version = "51.0.0" dependencies = [ "datafusion-doc", "quote", - "syn 2.0.108", + "syn 2.0.110", ] [[package]] @@ -2689,7 +2696,7 @@ dependencies = [ "bigdecimal", "bytes", "chrono", - "clap 4.5.50", + "clap 4.5.51", "datafusion", "datafusion-spark", "datafusion-substrait", @@ -2759,12 +2766,12 @@ dependencies = [ [[package]] name = "deranged" -version = "0.5.3" +version = "0.5.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d630bccd429a5bb5a64b5e94f693bfc48c9f8566418fda4c494cc94f911f87cc" +checksum = "ececcb659e7ba858fb4f10388c250a7252eb0a27373f1a72b8748afdd248e587" dependencies = [ "powerfmt", - "serde", + "serde_core", ] [[package]] @@ -2802,7 +2809,7 @@ dependencies = [ "libc", "option-ext", "redox_users", - "windows-sys 0.61.0", + "windows-sys 0.61.2", ] [[package]] @@ -2813,14 +2820,14 @@ checksum = "97369cbbc041bc366949bc74d34658d6cda5621039731c6310521892a3a20ae0" dependencies = [ "proc-macro2", "quote", - "syn 2.0.108", + "syn 2.0.110", ] [[package]] name = "doc-comment" -version = "0.3.3" +version = "0.3.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fea41bba32d969b513997752735605054bc0dfa92b4c56bf1189f2e174be7a10" +checksum = "780955b8b195a21ab8e4ac6b60dd1dbdcec1dc6c51c0617964b08c81785e12c9" [[package]] name = "docker_credential" @@ -2869,7 +2876,7 @@ dependencies = [ "enum-ordinalize", "proc-macro2", "quote", - "syn 2.0.108", + "syn 2.0.110", ] [[package]] @@ -2892,29 +2899,29 @@ checksum = "c34f04666d835ff5d62e058c3995147c06f42fe86ff053337632bca83e42702d" [[package]] name = "enum-ordinalize" -version = "4.3.0" +version = "4.3.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fea0dcfa4e54eeb516fe454635a95753ddd39acda650ce703031c6973e315dd5" +checksum = "4a1091a7bb1f8f2c4b28f1fe2cef4980ca2d410a3d727d67ecc3178c9b0800f0" dependencies = [ "enum-ordinalize-derive", ] [[package]] name = "enum-ordinalize-derive" -version = "4.3.1" +version = "4.3.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0d28318a75d4aead5c4db25382e8ef717932d0346600cacae6357eb5941bc5ff" +checksum = "8ca9601fb2d62598ee17836250842873a413586e5d7ed88b356e38ddbb0ec631" dependencies = [ "proc-macro2", "quote", - "syn 2.0.108", + "syn 2.0.110", ] [[package]] name = "env_filter" -version = "0.1.3" +version = "0.1.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "186e05a59d4c50738528153b83b0b0194d3a29507dfec16eccd4b342903397d0" +checksum = "1bf3c259d255ca70051b30e2e95b5446cdb8949ac4cd22c0d7fd634d89f568e2" dependencies = [ "log", "regex", @@ -2946,7 +2953,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "39cab71617ae0d63f51a36d69f866391735b51691dbda63cf6f96d042b63efeb" dependencies = [ "libc", - "windows-sys 0.61.0", + "windows-sys 0.61.2", ] [[package]] @@ -3038,9 +3045,9 @@ dependencies = [ [[package]] name = "find-msvc-tools" -version = "0.1.2" +version = "0.1.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1ced73b1dacfc750a6db6c0a0c3a3853c8b41997e2e2c563dc90804ae6867959" +checksum = "3a3076410a55c90011c298b04d0cfa770b00fa04e1e3c97d3f6c9de105a03844" [[package]] name = "fixedbitset" @@ -3050,19 +3057,19 @@ checksum = "1d674e81391d1e1ab681a28d99df07927c6d4aa5b027d7da16ba32d1d21ecd99" [[package]] name = "flatbuffers" -version = "25.2.10" +version = "25.9.23" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1045398c1bfd89168b5fd3f1fc11f6e70b34f6f66300c87d44d3de849463abf1" +checksum = "09b6620799e7340ebd9968d2e0708eb82cf1971e9a16821e2091b6d6e475eed5" dependencies = [ - "bitflags 2.9.4", + "bitflags 2.10.0", "rustc_version", ] [[package]] name = "flate2" -version = "1.1.4" +version = "1.1.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dc5a4e564e38c699f2880d3fda590bedc2e69f3f84cd48b457bd892ce61d0aa9" +checksum = "bfe33edd8e85a12a67454e37f8c75e730830d83e313556ab9ebf9ee7fbeb3bfb" dependencies = [ "crc32fast", "libz-rs-sys", @@ -3081,6 +3088,12 @@ version = "0.1.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d9c4f5dac5e15c24eb999c26181a6ca40b39fe946cbe4c263c7209467bc83af2" +[[package]] +name = "foldhash" +version = "0.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "77ce24cb58228fbb8aa041425bb1050850ac19177686ea6e0f41a70416f56fdb" + [[package]] name = "form_urlencoded" version = "1.2.2" @@ -3092,9 +3105,9 @@ dependencies = [ [[package]] name = "fs-err" -version = "3.1.2" +version = "3.1.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "44f150ffc8782f35521cec2b23727707cb4045706ba3c854e86bef66b3a8cdbd" +checksum = "6ad492b2cf1d89d568a43508ab24f98501fe03f2f31c01e1d0fe7366a71745d2" dependencies = [ "autocfg", ] @@ -3167,7 +3180,7 @@ checksum = "162ee34ebcb7c64a8abebc059ce0fee27c2262618d7b60ed8faf72fef13c3650" dependencies = [ "proc-macro2", "quote", - "syn 2.0.108", + "syn 2.0.110", ] [[package]] @@ -3276,9 +3289,9 @@ checksum = "0cc23270f6e1808e30a928bdc84dea0b9b4136a8bc82338574f23baf47bbd280" [[package]] name = "globset" -version = "0.4.16" +version = "0.4.18" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "54a1028dfc5f5df5da8a56a73e6c153c9a9708ec57232470703592a3f18e49f5" +checksum = "52dfc19153a48bde0cbd630453615c8151bce3a5adfac7a0aebfbf0a1e1f57e3" dependencies = [ "aho-corasick", "bstr", @@ -3343,9 +3356,7 @@ version = "0.15.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9229cfe53dfd69f0609a49f65461bd93001ea1ef889cd5529dd176593f5338a1" dependencies = [ - "allocator-api2", - "equivalent", - "foldhash", + "foldhash 0.1.5", ] [[package]] @@ -3353,6 +3364,11 @@ name = "hashbrown" version = "0.16.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "5419bdc4f6a9207fbeba6d11b604d481addf78ecd10c11ad51e76c2f6482748d" +dependencies = [ + "allocator-api2", + "equivalent", + "foldhash 0.2.0", +] [[package]] name = "heck" @@ -3386,11 +3402,11 @@ dependencies = [ [[package]] name = "home" -version = "0.5.11" +version = "0.5.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "589533453244b0995c858700322199b2becb13b627df2851f64a2775d024abcf" +checksum = "cc627f471c528ff0c4a49e1d5e60450c8f6461dd6d10ba9dcd3a61d3dff7728d" dependencies = [ - "windows-sys 0.59.0", + "windows-sys 0.61.2", ] [[package]] @@ -3469,9 +3485,9 @@ checksum = "135b12329e5e3ce057a9f972339ea52bc954fe1e9358ef27f95e89716fbc5424" [[package]] name = "hyper" -version = "1.7.0" +version = "1.8.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "eb3aa54a13a0dfe7fbe3a59e0c76093041720fdc77b110cc0fc260fafb4dc51e" +checksum = "2ab2d4f250c3d7b1c9fcdff1cece94ea4e2dfbec68614f7b87cb205f24ca9d11" dependencies = [ "atomic-waker", "bytes", @@ -3537,9 +3553,9 @@ dependencies = [ [[package]] name = "hyper-util" -version = "0.1.17" +version = "0.1.18" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3c6995591a8f1380fcb4ba966a252a4b29188d51d2b89e3a252f5305be65aea8" +checksum = "52e9a2a24dc5c6821e71a7030e1e14b7b632acac55c40e9d2e082c621261bb56" dependencies = [ "base64 0.22.1", "bytes", @@ -3586,7 +3602,7 @@ dependencies = [ "js-sys", "log", "wasm-bindgen", - "windows-core 0.62.0", + "windows-core 0.62.2", ] [[package]] @@ -3600,9 +3616,9 @@ dependencies = [ [[package]] name = "icu_collections" -version = "2.0.0" +version = "2.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "200072f5d0e3614556f94a9930d5dc3e0662a652823904c3a75dc3b0af7fee47" +checksum = "4c6b649701667bbe825c3b7e6388cb521c23d88644678e83c0c4d0a621a34b43" dependencies = [ "displaydoc", "potential_utf", @@ -3613,9 +3629,9 @@ dependencies = [ [[package]] name = "icu_locale_core" -version = "2.0.0" +version = "2.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0cde2700ccaed3872079a65fb1a78f6c0a36c91570f28755dda67bc8f7d9f00a" +checksum = "edba7861004dd3714265b4db54a3c390e880ab658fec5f7db895fae2046b5bb6" dependencies = [ "displaydoc", "litemap", @@ -3626,11 +3642,10 @@ dependencies = [ [[package]] name = "icu_normalizer" -version = "2.0.0" +version = "2.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "436880e8e18df4d7bbc06d58432329d6458cc84531f7ac5f024e93deadb37979" +checksum = "5f6c8828b67bf8908d82127b2054ea1b4427ff0230ee9141c54251934ab1b599" dependencies = [ - "displaydoc", "icu_collections", "icu_normalizer_data", "icu_properties", @@ -3641,42 +3656,38 @@ dependencies = [ [[package]] name = "icu_normalizer_data" -version = "2.0.0" +version = "2.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "00210d6893afc98edb752b664b8890f0ef174c8adbb8d0be9710fa66fbbf72d3" +checksum = "7aedcccd01fc5fe81e6b489c15b247b8b0690feb23304303a9e560f37efc560a" [[package]] name = "icu_properties" -version = "2.0.1" +version = "2.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "016c619c1eeb94efb86809b015c58f479963de65bdb6253345c1a1276f22e32b" +checksum = "e93fcd3157766c0c8da2f8cff6ce651a31f0810eaa1c51ec363ef790bbb5fb99" dependencies = [ - "displaydoc", "icu_collections", "icu_locale_core", "icu_properties_data", "icu_provider", - "potential_utf", "zerotrie", "zerovec", ] [[package]] name = "icu_properties_data" -version = "2.0.1" +version = "2.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "298459143998310acd25ffe6810ed544932242d3f07083eee1084d83a71bd632" +checksum = "02845b3647bb045f1100ecd6480ff52f34c35f82d9880e029d329c21d1054899" [[package]] name = "icu_provider" -version = "2.0.0" +version = "2.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "03c80da27b5f4187909049ee2d72f276f0d9f99a42c306bd0131ecfe04d8e5af" +checksum = "85962cf0ce02e1e0a629cc34e7ca3e373ce20dda4c4d7294bbd0bf1fdb59e614" dependencies = [ "displaydoc", "icu_locale_core", - "stable_deref_trait", - "tinystr", "writeable", "yoke", "zerofrom", @@ -3742,7 +3753,7 @@ checksum = "9375e112e4b463ec1b1c6c011953545c65a30164fbab5b581df32b3abf0dcb88" dependencies = [ "console 0.16.1", "portable-atomic", - "unicode-width 0.2.1", + "unicode-width 0.2.2", "unit-prefix", "web-time", ] @@ -3787,9 +3798,9 @@ checksum = "469fb0b9cefa57e3ef31275ee7cacb78f2fdca44e4765491884a2b119d4eb130" [[package]] name = "iri-string" -version = "0.7.8" +version = "0.7.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dbc5ebe9c3a1a7a5127f920a418f7585e9e758e911d0466ed004f393b0e380b2" +checksum = "4f867b9d1d896b67beb18518eda36fdb77a32ea590de864f1325b294a6d14397" dependencies = [ "memchr", "serde", @@ -3797,9 +3808,9 @@ dependencies = [ [[package]] name = "is_terminal_polyfill" -version = "1.70.1" +version = "1.70.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7943c866cc5cd64cbc25b2e01621d07fa8eb2a1a23160ee81ce38704e97b8ecf" +checksum = "a6cb138bb79a146c1bd460005623e142ef0181e3d0219cb493e02f7d08a35695" [[package]] name = "itertools" @@ -3827,26 +3838,26 @@ checksum = "4a5f13b858c8d314ee3e8f639011f7ccefe71f97f96e50151fb991f267928e2c" [[package]] name = "jiff" -version = "0.2.15" +version = "0.2.16" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "be1f93b8b1eb69c77f24bbb0afdf66f54b632ee39af40ca21c4365a1d7347e49" +checksum = "49cce2b81f2098e7e3efc35bc2e0a6b7abec9d34128283d7a26fa8f32a6dbb35" dependencies = [ "jiff-static", "log", "portable-atomic", "portable-atomic-util", - "serde", + "serde_core", ] [[package]] name = "jiff-static" -version = "0.2.15" +version = "0.2.16" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "03343451ff899767262ec32146f6d559dd759fdadf42ff0e227c7c48f72594b4" +checksum = "980af8b43c3ad5d8d349ace167ec8170839f753a42d233ba19e08afe1850fa69" dependencies = [ "proc-macro2", "quote", - "syn 2.0.108", + "syn 2.0.110", ] [[package]] @@ -3961,7 +3972,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d7c4b02199fee7c5d21a5ae7d8cfa79a6ef5bb2fc834d6e9058e89c825efdc55" dependencies = [ "cfg-if", - "windows-link 0.2.0", + "windows-link 0.2.1", ] [[package]] @@ -4007,7 +4018,7 @@ version = "0.1.10" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "416f7e718bdb06000964960ffa43b4335ad4012ae8b99060261aa4a8088d5ccb" dependencies = [ - "bitflags 2.9.4", + "bitflags 2.10.0", "libc", "redox_syscall", ] @@ -4020,7 +4031,7 @@ checksum = "5297962ef19edda4ce33aaa484386e0a5b3d7f2f4e037cbeee00503ef6b29d33" dependencies = [ "anstream", "anstyle", - "clap 4.5.50", + "clap 4.5.51", "escape8259", ] @@ -4041,17 +4052,16 @@ checksum = "df1d3c3b53da64cf5760482273a98e575c651a67eec7f77df96b5b642de8f039" [[package]] name = "litemap" -version = "0.8.0" +version = "0.8.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "241eaef5fd12c88705a01fc1066c48c4b36e0dd4377dcdc7ec3942cea7a69956" +checksum = "6373607a59f0be73a39b6fe456b8192fcc3585f602af20751600e974dd455e77" [[package]] name = "lock_api" -version = "0.4.13" +version = "0.4.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "96936507f153605bddfcda068dd804796c84324ed2510809e5b2a624c81da765" +checksum = "224399e74b87b5f3557511d98dff8b14089b3dadafcab6bb93eab67d3aace965" dependencies = [ - "autocfg", "scopeguard", ] @@ -4094,9 +4104,9 @@ dependencies = [ [[package]] name = "memchr" -version = "2.7.5" +version = "2.7.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "32a282da65faaf38286cf3be983213fcf1d2e2a58700e808f83f4ea9a4804bc0" +checksum = "f52b00d39961fc5b2736ea853c9cc86238e165017a493d1d5c8eac6bdc4cc273" [[package]] name = "mimalloc" @@ -4141,13 +4151,13 @@ dependencies = [ [[package]] name = "mio" -version = "1.0.4" +version = "1.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "78bed444cc8a2160f01cbcf811ef18cac863ad68ae8ca62092e8db51d51c761c" +checksum = "69d83b0086dc8ecf3ce9ae2874b2d1290252e2a30720bea58a5c6639b0092873" dependencies = [ "libc", "wasi", - "windows-sys 0.59.0", + "windows-sys 0.61.2", ] [[package]] @@ -4171,7 +4181,7 @@ version = "0.30.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "74523f3a35e05aba87a1d978330aef40f67b0304ac79c1c00b294c9830543db6" dependencies = [ - "bitflags 2.9.4", + "bitflags 2.10.0", "cfg-if", "cfg_aliases", "libc", @@ -4198,11 +4208,11 @@ dependencies = [ [[package]] name = "nu-ansi-term" -version = "0.50.1" +version = "0.50.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d4a28e057d01f97e61255210fcff094d74ed0466038633e95017f5beb68e4399" +checksum = "7957b9740744892f114936ab4a57b3f487491bbeafaf8083688b16841a4240e5" dependencies = [ - "windows-sys 0.52.0", + "windows-sys 0.61.2", ] [[package]] @@ -4288,23 +4298,32 @@ dependencies = [ [[package]] name = "objc2-core-foundation" -version = "0.3.1" +version = "0.3.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1c10c2894a6fed806ade6027bcd50662746363a9589d3ec9d9bef30a4e4bc166" +checksum = "2a180dd8642fa45cdb7dd721cd4c11b1cadd4929ce112ebd8b9f5803cc79d536" dependencies = [ - "bitflags 2.9.4", + "bitflags 2.10.0", ] [[package]] name = "objc2-io-kit" -version = "0.3.1" +version = "0.3.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "71c1c64d6120e51cd86033f67176b1cb66780c2efe34dec55176f77befd93c0a" +checksum = "33fafba39597d6dc1fb709123dfa8289d39406734be322956a69f0931c73bb15" dependencies = [ "libc", "objc2-core-foundation", ] +[[package]] +name = "object" +version = "0.32.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a6a622008b6e321afc04970976f62ee297fdbaa6f95318ca343e3eebb9648441" +dependencies = [ + "memchr", +] + [[package]] name = "object_store" version = "0.12.4" @@ -4350,9 +4369,9 @@ checksum = "42f5e15c9953c5e4ccceeb2e7382a716482c34515315f7b03532b8b4e8393d2d" [[package]] name = "once_cell_polyfill" -version = "1.70.1" +version = "1.70.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a4895175b425cb1f87721b59f0f286c2092bd4af812243672510e1ac53e2e0ad" +checksum = "384b8ab6d37215f3c5301a95a4accb5d64aa607f1fcb26a11b5303878451b4fe" [[package]] name = "oorandom" @@ -4389,15 +4408,15 @@ checksum = "1a80800c0488c3a21695ea981a54918fbb37abf04f4d0720c453632255e2ff0e" [[package]] name = "owo-colors" -version = "4.2.2" +version = "4.2.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "48dd4f4a2c8405440fd0462561f0e5806bd0f77e86f51c761481bdd4018b545e" +checksum = "9c6901729fa79e91a0913333229e9ca5dc725089d1c363b2f4b4760709dc4a52" [[package]] name = "parking_lot" -version = "0.12.4" +version = "0.12.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "70d58bf43669b5795d1576d0641cfb6fbb2057bf629506267a92807158584a13" +checksum = "93857453250e3077bd71ff98b6a65ea6621a19bb0f559a85248955ac12c45a1a" dependencies = [ "lock_api", "parking_lot_core", @@ -4405,15 +4424,15 @@ dependencies = [ [[package]] name = "parking_lot_core" -version = "0.9.11" +version = "0.9.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bc838d2a56b5b1a6c25f55575dfc605fabb63bb2365f6c2353ef9159aa69e4a5" +checksum = "2621685985a2ebf1c516881c026032ac7deafcda1a2c9b7850dc81e3dfcb64c1" dependencies = [ "cfg-if", "libc", "redox_syscall", "smallvec", - "windows-targets 0.52.6", + "windows-link 0.2.1", ] [[package]] @@ -4476,7 +4495,7 @@ dependencies = [ "regex", "regex-syntax", "structmeta", - "syn 2.0.108", + "syn 2.0.110", ] [[package]] @@ -4604,7 +4623,7 @@ checksum = "6e918e4ff8c4549eb882f14b3a4bc8c8bc93de829416eacf579f1207a8fbf861" dependencies = [ "proc-macro2", "quote", - "syn 2.0.108", + "syn 2.0.110", ] [[package]] @@ -4677,7 +4696,7 @@ dependencies = [ "heck 0.5.0", "proc-macro2", "quote", - "syn 2.0.108", + "syn 2.0.110", ] [[package]] @@ -4713,9 +4732,9 @@ dependencies = [ [[package]] name = "potential_utf" -version = "0.1.3" +version = "0.1.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "84df19adbe5b5a0782edcab45899906947ab039ccf4573713735ee7de1e6b08a" +checksum = "b73949432f5e2a09657003c25bca5e19a0e9c84f8058ca374f49e0ebe605af77" dependencies = [ "zerovec", ] @@ -4752,7 +4771,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "479ca8adacdd7ce8f1fb39ce9ecccbfe93a3f1344b3d0d97f20bc0196208f62b" dependencies = [ "proc-macro2", - "syn 2.0.108", + "syn 2.0.110", ] [[package]] @@ -4790,9 +4809,9 @@ dependencies = [ [[package]] name = "proc-macro2" -version = "1.0.101" +version = "1.0.103" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "89ae43fd86e4158d6db51ad8e2b80f313af9cc74f5c0e03ccb87de09998732de" +checksum = "5ee95bc4ef87b8d5ba32e8b7714ccc834865276eab0aed5c9958d00ec45f49e8" dependencies = [ "unicode-ident", ] @@ -4823,7 +4842,7 @@ dependencies = [ "prost", "prost-types", "regex", - "syn 2.0.108", + "syn 2.0.110", "tempfile", ] @@ -4837,7 +4856,7 @@ dependencies = [ "itertools 0.14.0", "proc-macro2", "quote", - "syn 2.0.108", + "syn 2.0.110", ] [[package]] @@ -4860,10 +4879,11 @@ dependencies = [ [[package]] name = "psm" -version = "0.1.26" +version = "0.1.28" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6e944464ec8536cd1beb0bbfd96987eb5e3b72f2ecdafdc5c769a37f1fa2ae1f" +checksum = "d11f2fedc3b7dafdc2851bc52f277377c5473d378859be234bc7ebb593144d01" dependencies = [ + "ar_archive_writer", "cc", ] @@ -4895,9 +4915,9 @@ checksum = "5a651516ddc9168ebd67b24afd085a718be02f8858fe406591b013d101ce2f40" [[package]] name = "quick-xml" -version = "0.38.3" +version = "0.38.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "42a232e7487fc2ef313d96dde7948e7a3c05101870d8985e4fd8d26aedd27b89" +checksum = "b66c2058c55a409d601666cffe35f04333cf1013010882cec174a7467cd4e21c" dependencies = [ "memchr", "serde", @@ -4960,9 +4980,9 @@ dependencies = [ [[package]] name = "quote" -version = "1.0.41" +version = "1.0.42" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ce25767e7b499d1b604768e7cde645d14cc8584231ea6b295e9c9eb22c02e1d1" +checksum = "a338cc41d27e6cc6dce6cefc13a0729dfbb81c262b1f519331575dd80ef3067f" dependencies = [ "proc-macro2", ] @@ -5095,16 +5115,16 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "76009fbe0614077fc1a2ce255e3a1881a2e3a3527097d5dc6d8212c585e7e38b" dependencies = [ "quote", - "syn 2.0.108", + "syn 2.0.110", ] [[package]] name = "redox_syscall" -version = "0.5.17" +version = "0.5.18" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5407465600fb0548f1442edf71dd20683c6ed326200ace4b1ef0763521bb3b77" +checksum = "ed2bf2547551a7053d6fdfafda3f938979645c44812fbfcda098faae3f1a362d" dependencies = [ - "bitflags 2.9.4", + "bitflags 2.10.0", ] [[package]] @@ -5120,22 +5140,22 @@ dependencies = [ [[package]] name = "ref-cast" -version = "1.0.24" +version = "1.0.25" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4a0ae411dbe946a674d89546582cea4ba2bb8defac896622d6496f14c23ba5cf" +checksum = "f354300ae66f76f1c85c5f84693f0ce81d747e2c3f21a45fef496d89c960bf7d" dependencies = [ "ref-cast-impl", ] [[package]] name = "ref-cast-impl" -version = "1.0.24" +version = "1.0.25" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1165225c21bff1f3bbce98f5a1f889949bc902d3575308cc7b0de30b4f6d27c7" +checksum = "b7186006dcb21920990093f30e3dea63b7d6e977bf1256be20c3563a5db070da" dependencies = [ "proc-macro2", "quote", - "syn 2.0.108", + "syn 2.0.110", ] [[package]] @@ -5169,17 +5189,17 @@ checksum = "8d942b98df5e658f56f20d592c7f868833fe38115e65c33003d8cd224b0155da" [[package]] name = "regex-syntax" -version = "0.8.6" +version = "0.8.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "caf4aa5b0f434c91fe5c7f1ecb6a5ece2130b02ad2a590589dda5146df959001" +checksum = "7a2d987857b319362043e95f5353c0535c1f58eec5336fdfcf626430af7def58" [[package]] name = "regress" -version = "0.10.4" +version = "0.10.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "145bb27393fe455dd64d6cbc8d059adfa392590a45eadf079c01b11857e7b010" +checksum = "2057b2325e68a893284d1538021ab90279adac1139957ca2a74426c6f118fb48" dependencies = [ - "hashbrown 0.15.5", + "hashbrown 0.16.0", "memchr", ] @@ -5209,9 +5229,9 @@ dependencies = [ [[package]] name = "reqwest" -version = "0.12.23" +version = "0.12.24" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d429f34c8092b2d42c7c93cec323bb4adeb7c67698f70839adec842ec10c7ceb" +checksum = "9d0946410b9f7b082a427e4ef5c8ff541a88b357bc6c637c40db3a68ac70a36f" dependencies = [ "base64 0.22.1", "bytes", @@ -5317,7 +5337,7 @@ dependencies = [ "regex", "relative-path", "rustc_version", - "syn 2.0.108", + "syn 2.0.110", "unicode-ident", ] @@ -5329,14 +5349,14 @@ checksum = "b3a8fb4672e840a587a66fc577a5491375df51ddb88f2a2c2a792598c326fe14" dependencies = [ "quote", "rand 0.8.5", - "syn 2.0.108", + "syn 2.0.110", ] [[package]] name = "rust_decimal" -version = "1.38.0" +version = "1.39.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c8975fc98059f365204d635119cf9c5a60ae67b841ed49b5422a9a7e56cdfac0" +checksum = "35affe401787a9bd846712274d97654355d21b2a2c092a3139aabe31e9022282" dependencies = [ "arrayvec", "borsh", @@ -5370,18 +5390,18 @@ version = "1.1.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "cd15f8a2c5551a84d56efdc1cd049089e409ac19a3072d5037a17fd70719ff3e" dependencies = [ - "bitflags 2.9.4", + "bitflags 2.10.0", "errno", "libc", "linux-raw-sys", - "windows-sys 0.61.0", + "windows-sys 0.61.2", ] [[package]] name = "rustls" -version = "0.23.32" +version = "0.23.35" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cd3c25631629d034ce7cd9940adc9d45762d46de2b0f57193c4443b92c6d4d40" +checksum = "533f54bc6a7d4f647e46ad909549eda97bf5afc1585190ef692b4286b198bd8f" dependencies = [ "aws-lc-rs", "log", @@ -5395,9 +5415,9 @@ dependencies = [ [[package]] name = "rustls-native-certs" -version = "0.8.1" +version = "0.8.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7fcff2dd52b58a8d98a70243663a0d234c4e2b79235637849d15913394a247d3" +checksum = "9980d917ebb0c0536119ba501e90834767bffc3d60641457fd84a1f3fd337923" dependencies = [ "openssl-probe", "rustls-pki-types", @@ -5416,9 +5436,9 @@ dependencies = [ [[package]] name = "rustls-pki-types" -version = "1.12.0" +version = "1.13.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "229a4a4c221013e7e1f1a043678c5cc39fe5171437c88fb47151a21e6f5b5c79" +checksum = "94182ad936a0c91c324cd46c6511b9510ed16af436d7b5bab34beab0afd55f7a" dependencies = [ "web-time", "zeroize", @@ -5426,9 +5446,9 @@ dependencies = [ [[package]] name = "rustls-webpki" -version = "0.103.6" +version = "0.103.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8572f3c2cb9934231157b45499fc41e1f58c589fdfb81a844ba873265e80f8eb" +checksum = "2ffdfa2f5286e2247234e03f680868ac2815974dc39e00ea15adc445d0aafe52" dependencies = [ "aws-lc-rs", "ring", @@ -5448,7 +5468,7 @@ version = "17.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e902948a25149d50edc1a8e0141aad50f54e22ba83ff988cf8f7c9ef07f50564" dependencies = [ - "bitflags 2.9.4", + "bitflags 2.10.0", "cfg-if", "clipboard-win", "fd-lock", @@ -5459,7 +5479,7 @@ dependencies = [ "nix", "radix_trie", "unicode-segmentation", - "unicode-width 0.2.1", + "unicode-width 0.2.2", "utf8parse", "windows-sys 0.60.2", ] @@ -5485,7 +5505,7 @@ version = "0.1.28" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "891d81b926048e76efe18581bf793546b4c0eaf8448d72be8de2bbee5fd166e1" dependencies = [ - "windows-sys 0.61.0", + "windows-sys 0.61.2", ] [[package]] @@ -5514,9 +5534,9 @@ dependencies = [ [[package]] name = "schemars" -version = "1.0.4" +version = "1.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "82d20c4491bc164fa2f6c5d44565947a52ad80b9505d8e36f8d54c27c739fcd0" +checksum = "9558e172d4e8533736ba97870c4b2cd63f84b382a3d6eb063da41b91cce17289" dependencies = [ "dyn-clone", "ref-cast", @@ -5533,7 +5553,7 @@ dependencies = [ "proc-macro2", "quote", "serde_derive_internals", - "syn 2.0.108", + "syn 2.0.110", ] [[package]] @@ -5550,11 +5570,11 @@ checksum = "1c107b6f4780854c8b126e228ea8869f4d7b71260f962fefb57b996b8959ba6b" [[package]] name = "security-framework" -version = "3.5.0" +version = "3.5.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cc198e42d9b7510827939c9a15f5062a0c913f3371d765977e586d2fe6c16f4a" +checksum = "b3297343eaf830f66ede390ea39da1d462b6b0c1b000f420d0a83f898bbbe6ef" dependencies = [ - "bitflags 2.9.4", + "bitflags 2.10.0", "core-foundation", "core-foundation-sys", "libc", @@ -5624,7 +5644,7 @@ checksum = "d540f220d3187173da220f885ab66608367b6574e925011a9353e4badda91d79" dependencies = [ "proc-macro2", "quote", - "syn 2.0.108", + "syn 2.0.110", ] [[package]] @@ -5635,7 +5655,7 @@ checksum = "18d26a20a969b9e3fdf2fc2d9f21eda6c40e2de84c9408bb5d3b05d499aae711" dependencies = [ "proc-macro2", "quote", - "syn 2.0.108", + "syn 2.0.110", ] [[package]] @@ -5659,7 +5679,7 @@ checksum = "175ee3e80ae9982737ca543e96133087cbd9a485eecc3bc4de9c1a37b47ea59c" dependencies = [ "proc-macro2", "quote", - "syn 2.0.108", + "syn 2.0.110", ] [[package]] @@ -5671,7 +5691,7 @@ dependencies = [ "proc-macro2", "quote", "serde", - "syn 2.0.108", + "syn 2.0.110", ] [[package]] @@ -5688,9 +5708,9 @@ dependencies = [ [[package]] name = "serde_with" -version = "3.14.1" +version = "3.15.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c522100790450cf78eeac1507263d0a350d4d5b30df0c8e1fe051a10c22b376e" +checksum = "aa66c845eee442168b2c8134fec70ac50dc20e760769c8ba0ad1319ca1959b04" dependencies = [ "base64 0.22.1", "chrono", @@ -5698,9 +5718,8 @@ dependencies = [ "indexmap 1.9.3", "indexmap 2.12.0", "schemars 0.9.0", - "schemars 1.0.4", - "serde", - "serde_derive", + "schemars 1.1.0", + "serde_core", "serde_json", "serde_with_macros", "time", @@ -5708,14 +5727,14 @@ dependencies = [ [[package]] name = "serde_with_macros" -version = "3.14.1" +version = "3.15.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "327ada00f7d64abaac1e55a6911e90cf665aa051b9a561c7006c157f4633135e" +checksum = "b91a903660542fced4e99881aa481bdbaec1634568ee02e0b8bd57c64cb38955" dependencies = [ "darling", "proc-macro2", "quote", - "syn 2.0.108", + "syn 2.0.110", ] [[package]] @@ -5839,12 +5858,12 @@ dependencies = [ [[package]] name = "socket2" -version = "0.6.0" +version = "0.6.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "233504af464074f9d066d7b5416c5f9b894a5862a6506e306f7b816cdd6f1807" +checksum = "17129e116933cf371d018bb80ae557e889637989d8638274fb25622827b03881" dependencies = [ "libc", - "windows-sys 0.59.0", + "windows-sys 0.60.2", ] [[package]] @@ -5891,20 +5910,20 @@ checksum = "da5fc6819faabb412da764b99d3b713bb55083c11e7e0c00144d386cd6a1939c" dependencies = [ "proc-macro2", "quote", - "syn 2.0.108", + "syn 2.0.110", ] [[package]] name = "stable_deref_trait" -version = "1.2.0" +version = "1.2.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a8f112729512f8e442d81f95a8a7ddf2b7c6b8a1a6f509a95864142b30cab2d3" +checksum = "6ce2be8dc25455e1f91df71bfa12ad37d7af1092ae736f3a6cd0e37bc7810596" [[package]] name = "stacker" -version = "0.1.21" +version = "0.1.22" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cddb07e32ddb770749da91081d8d0ac3a16f1a569a18b20348cd371f5dead06b" +checksum = "e1f8b29fb42aafcea4edeeb6b2f2d7ecd0d969c48b4cf0d2e64aafc471dd6e59" dependencies = [ "cc", "cfg-if", @@ -5939,7 +5958,7 @@ dependencies = [ "proc-macro2", "quote", "structmeta-derive", - "syn 2.0.108", + "syn 2.0.110", ] [[package]] @@ -5950,7 +5969,7 @@ checksum = "152a0b65a590ff6c3da95cabe2353ee04e6167c896b28e3b14478c2636c922fc" dependencies = [ "proc-macro2", "quote", - "syn 2.0.108", + "syn 2.0.110", ] [[package]] @@ -5977,31 +5996,12 @@ dependencies = [ "syn 1.0.109", ] -[[package]] -name = "strum" -version = "0.26.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8fec0f0aef304996cf250b31b5a10dee7980c85da9d759361292b8bca5a18f06" - [[package]] name = "strum" version = "0.27.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "af23d6f6c1a224baef9d3f61e287d2761385a5b88fdab4eb4c6f11aeb54c4bcf" -[[package]] -name = "strum_macros" -version = "0.26.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4c6bee85a5a24955dc440386795aa378cd9cf82acd5f764469152d2270e581be" -dependencies = [ - "heck 0.5.0", - "proc-macro2", - "quote", - "rustversion", - "syn 2.0.108", -] - [[package]] name = "strum_macros" version = "0.27.2" @@ -6011,7 +6011,7 @@ dependencies = [ "heck 0.5.0", "proc-macro2", "quote", - "syn 2.0.108", + "syn 2.0.110", ] [[package]] @@ -6045,7 +6045,7 @@ dependencies = [ "serde", "serde_json", "serde_yaml", - "syn 2.0.108", + "syn 2.0.110", "typify", "walkdir", ] @@ -6069,9 +6069,9 @@ dependencies = [ [[package]] name = "syn" -version = "2.0.108" +version = "2.0.110" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "da58917d35242480a05c2897064da0a80589a2a0476c9a3f2fdc83b53502e917" +checksum = "a99801b5bd34ede4cf3fc688c5919368fea4e4814a4664359503e6015b280aea" dependencies = [ "proc-macro2", "quote", @@ -6095,7 +6095,7 @@ checksum = "728a70f3dbaf5bab7f0c4b1ac8d7ae5ea60a4b5549c8a5914361c99147a709d2" dependencies = [ "proc-macro2", "quote", - "syn 2.0.108", + "syn 2.0.110", ] [[package]] @@ -6128,7 +6128,7 @@ dependencies = [ "getrandom 0.3.4", "once_cell", "rustix", - "windows-sys 0.61.0", + "windows-sys 0.61.2", ] [[package]] @@ -6206,7 +6206,7 @@ checksum = "3ff15c8ecd7de3849db632e14d18d2571fa09dfc5ed93479bc4485c7a517c913" dependencies = [ "proc-macro2", "quote", - "syn 2.0.108", + "syn 2.0.110", ] [[package]] @@ -6271,9 +6271,9 @@ dependencies = [ [[package]] name = "tinystr" -version = "0.8.1" +version = "0.8.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5d4f6d1145dcb577acf783d4e601bc1d76a13337bb54e6233add580b07344c8b" +checksum = "42d3e9c45c09de15d06dd8acf5f4e0e399e85927b7f00711024eb7ae10fa4869" dependencies = [ "displaydoc", "zerovec", @@ -6318,7 +6318,7 @@ dependencies = [ "signal-hook-registry", "socket2", "tokio-macros", - "windows-sys 0.61.0", + "windows-sys 0.61.2", ] [[package]] @@ -6329,14 +6329,14 @@ checksum = "af407857209536a95c8e56f8231ef2c2e2aff839b22e07a1ffcbc617e9db9fa5" dependencies = [ "proc-macro2", "quote", - "syn 2.0.108", + "syn 2.0.110", ] [[package]] name = "tokio-postgres" -version = "0.7.14" +version = "0.7.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a156efe7fff213168257853e1dfde202eed5f487522cbbbf7d219941d753d853" +checksum = "2b40d66d9b2cfe04b628173409368e58247e8eddbbd3b0e6c6ba1d09f20f6c9e" dependencies = [ "async-trait", "byteorder", @@ -6360,9 +6360,9 @@ dependencies = [ [[package]] name = "tokio-rustls" -version = "0.26.3" +version = "0.26.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "05f63835928ca123f1bef57abbcd23bb2ba0ac9ae1235f1e65bda0d06e7786bd" +checksum = "1729aa945f29d91ba541258c8df89027d5792d85a8841fb65e8bf0f4ede4ef61" dependencies = [ "rustls", "tokio", @@ -6394,18 +6394,18 @@ dependencies = [ [[package]] name = "toml_datetime" -version = "0.7.2" +version = "0.7.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "32f1085dec27c2b6632b04c80b3bb1b4300d6495d1e129693bdda7d91e72eec1" +checksum = "f2cdb639ebbc97961c51720f858597f7f24c4fc295327923af55b74c3c724533" dependencies = [ "serde_core", ] [[package]] name = "toml_edit" -version = "0.23.6" +version = "0.23.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f3effe7c0e86fdff4f69cdd2ccc1b96f933e24811c5441d44904e8683e27184b" +checksum = "6485ef6d0d9b5d0ec17244ff7eb05310113c3f316f2d14200d4de56b3cb98f8d" dependencies = [ "indexmap 2.12.0", "toml_datetime", @@ -6415,9 +6415,9 @@ dependencies = [ [[package]] name = "toml_parser" -version = "1.0.3" +version = "1.0.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4cf893c33be71572e0e9aa6dd15e6677937abd686b066eac3f8cd3531688a627" +checksum = "c0cbe268d35bdb4bb5a56a2de88d0ad0eb70af5384a99d648cd4b3d04039800e" dependencies = [ "winnow", ] @@ -6487,7 +6487,7 @@ version = "0.6.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "adc82fd73de2a9722ac5da747f12383d2bfdb93591ee6c58486e0097890f05f2" dependencies = [ - "bitflags 2.9.4", + "bitflags 2.10.0", "bytes", "futures-util", "http 1.3.1", @@ -6530,7 +6530,7 @@ checksum = "81383ab64e72a7a8b8e13130c49e3dab29def6d0c7d76a03087b3cf71c5c6903" dependencies = [ "proc-macro2", "quote", - "syn 2.0.108", + "syn 2.0.110", ] [[package]] @@ -6603,9 +6603,9 @@ checksum = "6af6ae20167a9ece4bcb41af5b80f8a1f1df981f6391189ce00fd257af04126a" [[package]] name = "typenum" -version = "1.18.0" +version = "1.19.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1dccffe3ce07af9386bfd29e80c0ab1a8205a2fc34e4bcd40364df902cfa8f3f" +checksum = "562d481066bde0658276a35467c4af00bdc6ee726305698a55b86e61d7ad82bb" [[package]] name = "typewit" @@ -6638,7 +6638,7 @@ dependencies = [ "semver", "serde", "serde_json", - "syn 2.0.108", + "syn 2.0.110", "thiserror", "unicode-ident", ] @@ -6656,7 +6656,7 @@ dependencies = [ "serde", "serde_json", "serde_tokenstream", - "syn 2.0.108", + "syn 2.0.110", "typify-impl", ] @@ -6678,24 +6678,24 @@ checksum = "5c1cb5db39152898a79168971543b1cb5020dff7fe43c8dc468b0885f5e29df5" [[package]] name = "unicode-ident" -version = "1.0.19" +version = "1.0.22" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f63a545481291138910575129486daeaf8ac54aee4387fe7906919f7830c7d9d" +checksum = "9312f7c4f6ff9069b165498234ce8be658059c6728633667c526e27dc2cf1df5" [[package]] name = "unicode-normalization" -version = "0.1.24" +version = "0.1.25" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5033c97c4262335cded6d6fc3e5c18ab755e1a3dc96376350f3d8e9f009ad956" +checksum = "5fd4f6878c9cb28d874b009da9e8d183b5abc80117c40bbd187a1fde336be6e8" dependencies = [ "tinyvec", ] [[package]] name = "unicode-properties" -version = "0.1.3" +version = "0.1.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e70f2a8b45122e719eb623c01822704c4e0907e7e426a05927e1a1cfff5b75d0" +checksum = "7df058c713841ad818f1dc5d3fd88063241cc61f49f5fbea4b951e8cf5a8d71d" [[package]] name = "unicode-segmentation" @@ -6711,9 +6711,9 @@ checksum = "7dd6e30e90baa6f72411720665d41d89b9a3d039dc45b8faea1ddd07f617f6af" [[package]] name = "unicode-width" -version = "0.2.1" +version = "0.2.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4a1a07cc7db3810833284e8d372ccdc6da29741639ecc70c9ec107df0fa6154c" +checksum = "b4ac048d71ede7ee76d585517add45da530660ef4390e49b098733c6e897f254" [[package]] name = "unit-prefix" @@ -6735,15 +6735,14 @@ checksum = "8ecb6da28b8a351d773b68d5825ac39017e680750f980f3a1a85cd8dd28a47c1" [[package]] name = "ureq" -version = "3.1.2" +version = "3.1.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "99ba1025f18a4a3fc3e9b48c868e9beb4f24f4b4b1a325bada26bd4119f46537" +checksum = "d39cb1dbab692d82a977c0392ffac19e188bd9186a9f32806f0aaa859d75585a" dependencies = [ "base64 0.22.1", "log", "percent-encoding", "rustls", - "rustls-pemfile", "rustls-pki-types", "ureq-proto", "utf-8", @@ -6913,7 +6912,7 @@ dependencies = [ "bumpalo", "proc-macro2", "quote", - "syn 2.0.108", + "syn 2.0.110", "wasm-bindgen-shared", ] @@ -6947,7 +6946,7 @@ checksum = "085b2df989e1e6f9620c1311df6c996e83fe16f57792b272ce1e024ac16a90f1" dependencies = [ "proc-macro2", "quote", - "syn 2.0.108", + "syn 2.0.110", ] [[package]] @@ -6985,9 +6984,9 @@ dependencies = [ [[package]] name = "webpki-roots" -version = "1.0.3" +version = "1.0.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "32b130c0d2d49f8b6889abc456e795e82525204f27c42cf767cf0d7734e089b8" +checksum = "b2878ef029c47c6e8cf779119f20fcf52bde7ad42a731b2a304bc221df17571e" dependencies = [ "rustls-pki-types", ] @@ -7025,7 +7024,7 @@ version = "0.1.11" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c2a7b1c03c876122aa43f3020e6c3c3ee5c05081c9a00739faf7503aeba10d22" dependencies = [ - "windows-sys 0.61.0", + "windows-sys 0.61.2", ] [[package]] @@ -7071,15 +7070,15 @@ dependencies = [ [[package]] name = "windows-core" -version = "0.62.0" +version = "0.62.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "57fe7168f7de578d2d8a05b07fd61870d2e73b4020e9f49aa00da8471723497c" +checksum = "b8e83a14d34d0623b51dce9581199302a221863196a1dde71a7663a4c2be9deb" dependencies = [ "windows-implement", "windows-interface", - "windows-link 0.2.0", - "windows-result 0.4.0", - "windows-strings 0.5.0", + "windows-link 0.2.1", + "windows-result 0.4.1", + "windows-strings 0.5.1", ] [[package]] @@ -7095,24 +7094,24 @@ dependencies = [ [[package]] name = "windows-implement" -version = "0.60.0" +version = "0.60.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a47fddd13af08290e67f4acabf4b459f647552718f683a7b415d290ac744a836" +checksum = "053e2e040ab57b9dc951b72c264860db7eb3b0200ba345b4e4c3b14f67855ddf" dependencies = [ "proc-macro2", "quote", - "syn 2.0.108", + "syn 2.0.110", ] [[package]] name = "windows-interface" -version = "0.59.1" +version = "0.59.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bd9211b69f8dcdfa817bfd14bf1c97c9188afa36f4750130fcdf3f400eca9fa8" +checksum = "3f316c4a2570ba26bbec722032c4099d8c8bc095efccdc15688708623367e358" dependencies = [ "proc-macro2", "quote", - "syn 2.0.108", + "syn 2.0.110", ] [[package]] @@ -7123,9 +7122,9 @@ checksum = "5e6ad25900d524eaabdbbb96d20b4311e1e7ae1699af4fb28c17ae66c80d798a" [[package]] name = "windows-link" -version = "0.2.0" +version = "0.2.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "45e46c0661abb7180e7b9c281db115305d49ca1709ab8242adf09666d2173c65" +checksum = "f0805222e57f7521d6a62e36fa9163bc891acd422f971defe97d64e70d0a4fe5" [[package]] name = "windows-numerics" @@ -7148,11 +7147,11 @@ dependencies = [ [[package]] name = "windows-result" -version = "0.4.0" +version = "0.4.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7084dcc306f89883455a206237404d3eaf961e5bd7e0f312f7c91f57eb44167f" +checksum = "7781fa89eaf60850ac3d2da7af8e5242a5ea78d1a11c49bf2910bb5a73853eb5" dependencies = [ - "windows-link 0.2.0", + "windows-link 0.2.1", ] [[package]] @@ -7166,11 +7165,11 @@ dependencies = [ [[package]] name = "windows-strings" -version = "0.5.0" +version = "0.5.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7218c655a553b0bed4426cf54b20d7ba363ef543b52d515b3e48d7fd55318dda" +checksum = "7837d08f69c77cf6b07689544538e017c1bfcf57e34b4c0ff58e6c2cd3b37091" dependencies = [ - "windows-link 0.2.0", + "windows-link 0.2.1", ] [[package]] @@ -7197,16 +7196,16 @@ version = "0.60.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f2f500e4d28234f72040990ec9d39e3a6b950f9f22d3dba18416c35882612bcb" dependencies = [ - "windows-targets 0.53.3", + "windows-targets 0.53.5", ] [[package]] name = "windows-sys" -version = "0.61.0" +version = "0.61.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e201184e40b2ede64bc2ea34968b28e33622acdbbf37104f0e4a33f7abe657aa" +checksum = "ae137229bcbd6cdf0f7b80a31df61766145077ddf49416a728b02cb3921ff3fc" dependencies = [ - "windows-link 0.2.0", + "windows-link 0.2.1", ] [[package]] @@ -7227,19 +7226,19 @@ dependencies = [ [[package]] name = "windows-targets" -version = "0.53.3" +version = "0.53.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d5fe6031c4041849d7c496a8ded650796e7b6ecc19df1a431c1a363342e5dc91" +checksum = "4945f9f551b88e0d65f3db0bc25c33b8acea4d9e41163edf90dcd0b19f9069f3" dependencies = [ - "windows-link 0.1.3", - "windows_aarch64_gnullvm 0.53.0", - "windows_aarch64_msvc 0.53.0", - "windows_i686_gnu 0.53.0", - "windows_i686_gnullvm 0.53.0", - "windows_i686_msvc 0.53.0", - "windows_x86_64_gnu 0.53.0", - "windows_x86_64_gnullvm 0.53.0", - "windows_x86_64_msvc 0.53.0", + "windows-link 0.2.1", + "windows_aarch64_gnullvm 0.53.1", + "windows_aarch64_msvc 0.53.1", + "windows_i686_gnu 0.53.1", + "windows_i686_gnullvm 0.53.1", + "windows_i686_msvc 0.53.1", + "windows_x86_64_gnu 0.53.1", + "windows_x86_64_gnullvm 0.53.1", + "windows_x86_64_msvc 0.53.1", ] [[package]] @@ -7259,9 +7258,9 @@ checksum = "32a4622180e7a0ec044bb555404c800bc9fd9ec262ec147edd5989ccd0c02cd3" [[package]] name = "windows_aarch64_gnullvm" -version = "0.53.0" +version = "0.53.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "86b8d5f90ddd19cb4a147a5fa63ca848db3df085e25fee3cc10b39b6eebae764" +checksum = "a9d8416fa8b42f5c947f8482c43e7d89e73a173cead56d044f6a56104a6d1b53" [[package]] name = "windows_aarch64_msvc" @@ -7271,9 +7270,9 @@ checksum = "09ec2a7bb152e2252b53fa7803150007879548bc709c039df7627cabbd05d469" [[package]] name = "windows_aarch64_msvc" -version = "0.53.0" +version = "0.53.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c7651a1f62a11b8cbd5e0d42526e55f2c99886c77e007179efff86c2b137e66c" +checksum = "b9d782e804c2f632e395708e99a94275910eb9100b2114651e04744e9b125006" [[package]] name = "windows_i686_gnu" @@ -7283,9 +7282,9 @@ checksum = "8e9b5ad5ab802e97eb8e295ac6720e509ee4c243f69d781394014ebfe8bbfa0b" [[package]] name = "windows_i686_gnu" -version = "0.53.0" +version = "0.53.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c1dc67659d35f387f5f6c479dc4e28f1d4bb90ddd1a5d3da2e5d97b42d6272c3" +checksum = "960e6da069d81e09becb0ca57a65220ddff016ff2d6af6a223cf372a506593a3" [[package]] name = "windows_i686_gnullvm" @@ -7295,9 +7294,9 @@ checksum = "0eee52d38c090b3caa76c563b86c3a4bd71ef1a819287c19d586d7334ae8ed66" [[package]] name = "windows_i686_gnullvm" -version = "0.53.0" +version = "0.53.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9ce6ccbdedbf6d6354471319e781c0dfef054c81fbc7cf83f338a4296c0cae11" +checksum = "fa7359d10048f68ab8b09fa71c3daccfb0e9b559aed648a8f95469c27057180c" [[package]] name = "windows_i686_msvc" @@ -7307,9 +7306,9 @@ checksum = "240948bc05c5e7c6dabba28bf89d89ffce3e303022809e73deaefe4f6ec56c66" [[package]] name = "windows_i686_msvc" -version = "0.53.0" +version = "0.53.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "581fee95406bb13382d2f65cd4a908ca7b1e4c2f1917f143ba16efe98a589b5d" +checksum = "1e7ac75179f18232fe9c285163565a57ef8d3c89254a30685b57d83a38d326c2" [[package]] name = "windows_x86_64_gnu" @@ -7319,9 +7318,9 @@ checksum = "147a5c80aabfbf0c7d901cb5895d1de30ef2907eb21fbbab29ca94c5b08b1a78" [[package]] name = "windows_x86_64_gnu" -version = "0.53.0" +version = "0.53.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2e55b5ac9ea33f2fc1716d1742db15574fd6fc8dadc51caab1c16a3d3b4190ba" +checksum = "9c3842cdd74a865a8066ab39c8a7a473c0778a3f29370b5fd6b4b9aa7df4a499" [[package]] name = "windows_x86_64_gnullvm" @@ -7331,9 +7330,9 @@ checksum = "24d5b23dc417412679681396f2b49f3de8c1473deb516bd34410872eff51ed0d" [[package]] name = "windows_x86_64_gnullvm" -version = "0.53.0" +version = "0.53.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0a6e035dd0599267ce1ee132e51c27dd29437f63325753051e71dd9e42406c57" +checksum = "0ffa179e2d07eee8ad8f57493436566c7cc30ac536a3379fdf008f47f6bb7ae1" [[package]] name = "windows_x86_64_msvc" @@ -7343,9 +7342,9 @@ checksum = "589f6da84c646204747d1270a2a5661ea66ed1cced2631d546fdfb155959f9ec" [[package]] name = "windows_x86_64_msvc" -version = "0.53.0" +version = "0.53.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "271414315aff87387382ec3d271b52d7ae78726f5d44ac98b4f4030c91880486" +checksum = "d6bbff5f0aada427a1e5a6da5f1f98158182f26556f345ac9e04d36d0ebed650" [[package]] name = "winnow" @@ -7364,9 +7363,9 @@ checksum = "f17a85883d4e6d00e8a97c586de764dabcc06133f7f1d55dce5cdc070ad7fe59" [[package]] name = "writeable" -version = "0.6.1" +version = "0.6.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ea2f10b9bb0928dfb1b42b65e1f9e36f7f54dbdf08457afefb38afcdec4fa2bb" +checksum = "9edde0db4769d2dc68579893f2306b26c6ecfbe0ef499b013d731b7b9247e0b9" [[package]] name = "wyz" @@ -7401,11 +7400,10 @@ checksum = "cfe53a6657fd280eaa890a3bc59152892ffa3e30101319d168b781ed6529b049" [[package]] name = "yoke" -version = "0.8.0" +version = "0.8.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5f41bb01b8226ef4bfd589436a297c53d118f65921786300e427be8d487695cc" +checksum = "72d6e5c6afb84d73944e5cedb052c4680d5657337201555f9f2a16b7406d4954" dependencies = [ - "serde", "stable_deref_trait", "yoke-derive", "zerofrom", @@ -7413,13 +7411,13 @@ dependencies = [ [[package]] name = "yoke-derive" -version = "0.8.0" +version = "0.8.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "38da3c9736e16c5d3c8c597a9aaa5d1fa565d0532ae05e27c24aa62fb32c0ab6" +checksum = "b659052874eb698efe5b9e8cf382204678a0086ebf46982b79d6ca3182927e5d" dependencies = [ "proc-macro2", "quote", - "syn 2.0.108", + "syn 2.0.110", "synstructure", ] @@ -7440,7 +7438,7 @@ checksum = "88d2b8d9c68ad2b9e4340d7832716a4d21a22a1154777ad56ea55c51a9cf3831" dependencies = [ "proc-macro2", "quote", - "syn 2.0.108", + "syn 2.0.110", ] [[package]] @@ -7460,21 +7458,21 @@ checksum = "d71e5d6e06ab090c67b5e44993ec16b72dcbaabc526db883a360057678b48502" dependencies = [ "proc-macro2", "quote", - "syn 2.0.108", + "syn 2.0.110", "synstructure", ] [[package]] name = "zeroize" -version = "1.8.1" +version = "1.8.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ced3678a2879b30306d323f4542626697a464a97c0a07c9aebf7ebca65cd4dde" +checksum = "b97154e67e32c85465826e8bcc1c59429aaaf107c1e4a9e53c8d8ccd5eff88d0" [[package]] name = "zerotrie" -version = "0.2.2" +version = "0.2.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "36f0bbd478583f79edad978b407914f61b2972f5af6fa089686016be8f9af595" +checksum = "2a59c17a5562d507e4b54960e8569ebee33bee890c70aa3fe7b97e85a9fd7851" dependencies = [ "displaydoc", "yoke", @@ -7483,9 +7481,9 @@ dependencies = [ [[package]] name = "zerovec" -version = "0.11.4" +version = "0.11.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e7aa2bd55086f1ab526693ecbe444205da57e25f4489879da80635a46d90e73b" +checksum = "6c28719294829477f525be0186d13efa9a3c602f7ec202ca9e353d310fb9a002" dependencies = [ "yoke", "zerofrom", @@ -7494,13 +7492,13 @@ dependencies = [ [[package]] name = "zerovec-derive" -version = "0.11.1" +version = "0.11.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5b96237efa0c878c64bd89c436f661be4e46b2f3eff1ebb976f7ef2321d2f58f" +checksum = "eadce39539ca5cb3985590102671f2567e659fca9666581ad3411d59207951f3" dependencies = [ "proc-macro2", "quote", - "syn 2.0.108", + "syn 2.0.110", ] [[package]] diff --git a/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs b/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs index 0fe86acb3604..7f37d5e508d1 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs @@ -324,15 +324,11 @@ pub(super) struct SortMergeJoinStream { pub staging_output_record_batches: JoinedRecordBatches, /// Output buffer. Currently used by filtering as it requires double buffering /// to avoid small/empty batches. Non-filtered join outputs directly from `staging_output_record_batches.batches` - /// Uses BatchCoalescer to accumulate small batches efficiently without repeated concatenation. - pub output_buffer: BatchCoalescer, + pub output: BatchCoalescer, /// Staging output size, including output batches and staging joined results. /// Increased when we put rows into buffer and decreased after we actually output batches. /// Used to trigger output when sufficient rows are ready pub output_size: usize, - /// Flag indicating that staging_output_record_batches coalescer has reached target - /// and should be processed - pub staging_ready: bool, /// The comparison result of current streamed row and buffered batches pub current_ordering: Ordering, /// Manages the process of spilling and reading back intermediate data @@ -355,7 +351,7 @@ pub(super) struct SortMergeJoinStream { /// Joined batches with attached join filter information pub(super) struct JoinedRecordBatches { /// Joined batches. Each batch is already joined columns from left and right sources - pub coalescer: BatchCoalescer, + pub batches: Vec, /// Filter match mask for each row(matched/non-matched) pub filter_mask: BooleanBuilder, /// Left row indices to glue together rows in `batches` and `filter_mask` @@ -368,7 +364,7 @@ pub(super) struct JoinedRecordBatches { impl JoinedRecordBatches { fn clear(&mut self) { - // Note: BatchCoalescer clears itself in finish_buffered_batch() + self.batches.clear(); self.batch_ids.clear(); self.filter_mask = BooleanBuilder::new(); self.row_indices = UInt64Builder::new(); @@ -597,43 +593,26 @@ impl Stream for SortMergeJoinStream { self.freeze_all()?; // If join is filtered and there is joined tuples waiting - // to be filtered. Process when coalescer has reached target size. - if self.staging_ready { - // Track buffered row count before draining the coalescer - let pre_filter_row_count = self - .staging_output_record_batches - .coalescer - .get_buffered_rows(); - + // to be filtered + if !self + .staging_output_record_batches + .batches + .is_empty() + { // Apply filter on joined tuples and get filtered batch let out_filtered_batch = self.filter_joined_batch()?; - // Decrement output_size by the number of unfiltered rows processed. - // output_size tracks unfiltered pairs, but we just processed - // pre_filter_row_count rows from the coalescer. - if pre_filter_row_count > self.output_size { - self.output_size = 0; - } else { - self.output_size -= pre_filter_row_count; - } - - // Reset the flag after processing - self.staging_ready = false; - // Append filtered batch to the output buffer - self.output_buffer - .push_batch(out_filtered_batch)?; - if self.output_buffer.has_completed_batch() { - self.output_buffer - .finish_buffered_batch()?; + self.output + .push_batch(out_filtered_batch) + .expect("Failed to push output batch"); + + if self.output.has_completed_batch() { let record_batch = self - .output_buffer + .output .next_completed_batch() - .unwrap(); - (&record_batch).record_output( - &self.join_metrics.baseline_metrics(), - ); + .expect("Failed to get output batch"); return Poll::Ready(Some(Ok( record_batch, ))); @@ -694,11 +673,13 @@ impl Stream for SortMergeJoinStream { } } else { self.freeze_all()?; - // Only process if coalescer has reached target - if self.staging_ready { - // For filtered joins, batches accumulate across multiple freeze_all() calls - // and are processed at safe transition points (between streamed batches or - // at Exhausted state). Don't output here in the tight JoinOutput loop. + if !self.staging_output_record_batches.batches.is_empty() { + let record_batch = self.output_record_batch_and_reset()?; + // For non-filtered join output whenever the target output batch size + // is hit. For filtered join its needed to output on later phase + // because target output batch size can be hit in the middle of + // filtering causing the filtering to be incomplete and causing + // correctness issues if self.filter.is_some() && matches!( self.join_type, @@ -713,31 +694,19 @@ impl Stream for SortMergeJoinStream { | JoinType::Full ) { - // Keep staging_ready set to let it propagate to Init state - // where it will be processed. Transition to Init state to continue. - self.buffered_data.scanning_reset(); - self.state = SortMergeJoinState::Init; continue; - } else { - // Non-filtered joins output immediately - let record_batch = - self.output_record_batch_and_reset()?; - self.staging_ready = false; - (&record_batch) - .record_output(&self.join_metrics.baseline_metrics()); - return Poll::Ready(Some(Ok(record_batch))); } + + return Poll::Ready(Some(Ok(record_batch))); } - // Reset scanning and transition to Init to continue processing - self.buffered_data.scanning_reset(); - self.state = SortMergeJoinState::Init; + return Poll::Pending; } } SortMergeJoinState::Exhausted => { self.freeze_all()?; - // if there is still something not processed in coalescer - if !self.staging_output_record_batches.coalescer.is_empty() { + // if there is still something not processed + if !self.staging_output_record_batches.batches.is_empty() { if self.filter.is_some() && matches!( self.join_type, @@ -753,24 +722,19 @@ impl Stream for SortMergeJoinStream { ) { let record_batch = self.filter_joined_batch()?; - (&record_batch) - .record_output(&self.join_metrics.baseline_metrics()); return Poll::Ready(Some(Ok(record_batch))); } else { let record_batch = self.output_record_batch_and_reset()?; - (&record_batch) - .record_output(&self.join_metrics.baseline_metrics()); return Poll::Ready(Some(Ok(record_batch))); } - } else if !self.output_buffer.is_empty() { - // if processed but still not outputted because it didn't hit batch size before - self.output_buffer.finish_buffered_batch()?; - let record_batch = - self.output_buffer.next_completed_batch().unwrap_or_else( - || RecordBatch::new_empty(Arc::clone(&self.schema)), - ); - (&record_batch) - .record_output(&self.join_metrics.baseline_metrics()); + } else if !self.output.is_empty() { + self.output + .finish_buffered_batch() + .expect("Failed to finish last batch"); + let record_batch = self + .output + .next_completed_batch() + .expect("Failed to get last batch"); return Poll::Ready(Some(Ok(record_batch))); } else { return Poll::Ready(None); @@ -828,16 +792,13 @@ impl SortMergeJoinStream { on_buffered, filter, staging_output_record_batches: JoinedRecordBatches { - coalescer: BatchCoalescer::new(Arc::clone(&schema), batch_size) - .with_biggest_coalesce_batch_size(Some(batch_size / 2)), + batches: vec![], filter_mask: BooleanBuilder::new(), row_indices: UInt64Builder::new(), batch_ids: vec![], }, - output_buffer: BatchCoalescer::new(Arc::clone(&schema), batch_size) - .with_biggest_coalesce_batch_size(Some(batch_size / 2)), + output: BatchCoalescer::new(schema, batch_size), output_size: 0, - staging_ready: false, batch_size, join_type, join_metrics, @@ -1239,15 +1200,8 @@ impl SortMergeJoinStream { ); self.staging_output_record_batches - .coalescer - .push_batch(record_batch)?; - if self - .staging_output_record_batches - .coalescer - .has_completed_batch() - { - self.staging_ready = true; - } + .batches + .push(record_batch); } buffered_batch.null_joined.clear(); } @@ -1292,15 +1246,8 @@ impl SortMergeJoinStream { 0, ); self.staging_output_record_batches - .coalescer - .push_batch(record_batch)?; - if self - .staging_output_record_batches - .coalescer - .has_completed_batch() - { - self.staging_ready = true; - } + .batches + .push(record_batch); } buffered_batch.join_filter_not_matched_map.clear(); @@ -1310,7 +1257,6 @@ impl SortMergeJoinStream { // Produces and stages record batch for all output indices found // for current streamed batch and clears staged output indices. fn freeze_streamed(&mut self) -> Result<()> { - let mut rows_processed = 0; for chunk in self.streamed_batch.output_indices.iter_mut() { // The row indices of joined streamed batch let left_indices = chunk.streamed_indices.finish(); @@ -1319,8 +1265,6 @@ impl SortMergeJoinStream { continue; } - rows_processed += left_indices.len(); - let mut left_columns = self .streamed_batch .batch @@ -1445,26 +1389,13 @@ impl SortMergeJoinStream { | JoinType::Full ) { self.staging_output_record_batches - .coalescer - .push_batch(output_batch)?; - if self - .staging_output_record_batches - .coalescer - .has_completed_batch() - { - self.staging_ready = true; - } + .batches + .push(output_batch); } else { + let filtered_batch = filter_record_batch(&output_batch, &mask)?; self.staging_output_record_batches - .coalescer - .push_batch_with_filter(output_batch, &mask)?; - if self - .staging_output_record_batches - .coalescer - .has_completed_batch() - { - self.staging_ready = true; - } + .batches + .push(filtered_batch); } if !matches!(self.join_type, JoinType::Full) { @@ -1512,71 +1443,25 @@ impl SortMergeJoinStream { } } else { self.staging_output_record_batches - .coalescer - .push_batch(output_batch)?; - if self - .staging_output_record_batches - .coalescer - .has_completed_batch() - { - self.staging_ready = true; - } + .batches + .push(output_batch); } } else { self.staging_output_record_batches - .coalescer - .push_batch(output_batch)?; - if self - .staging_output_record_batches - .coalescer - .has_completed_batch() - { - self.staging_ready = true; - } + .batches + .push(output_batch); } } self.streamed_batch.output_indices.clear(); - // Decrement output_size by the number of rows we just processed and added to the coalescer - if rows_processed > self.output_size { - self.output_size = 0; - } else { - self.output_size -= rows_processed; - } - - // After clearing output_indices, if the coalescer has buffered data but hasn't - // reached the target yet, we may need to force a flush to prevent deadlock. - // This is only necessary for filtered joins where partial batches can accumulate - // without reaching the target batch size. - if !self.staging_output_record_batches.coalescer.is_empty() - && !self.staging_ready - && self.filter.is_some() - { - self.staging_output_record_batches - .coalescer - .finish_buffered_batch()?; - if self - .staging_output_record_batches - .coalescer - .has_completed_batch() - { - self.staging_ready = true; - } - } - Ok(()) } fn output_record_batch_and_reset(&mut self) -> Result { - self.staging_output_record_batches - .coalescer - .finish_buffered_batch()?; - let record_batch = self - .staging_output_record_batches - .coalescer - .next_completed_batch() - .unwrap_or_else(|| RecordBatch::new_empty(Arc::clone(&self.schema))); + let record_batch = + concat_batches(&self.schema, &self.staging_output_record_batches.batches)?; + (&record_batch).record_output(&self.join_metrics.baseline_metrics()); // If join filter exists, `self.output_size` is not accurate as we don't know the exact // number of rows in the output record batch. If streamed row joined with buffered rows, // once join filter is applied, the number of output rows may be more than 1. @@ -1588,25 +1473,29 @@ impl SortMergeJoinStream { self.output_size -= record_batch.num_rows(); } - // Note: coalescer is already cleared by finish_buffered_batch() above - // The metadata MUST also be cleared since the batches they refer to are gone. - // For non-filtered joins, clear everything immediately. - // For filtered joins, this path shouldn't be hit (they use filter_joined_batch), - // but if it is, we still need to clear to avoid desync. - self.staging_output_record_batches.clear(); + if !(self.filter.is_some() + && matches!( + self.join_type, + JoinType::Left + | JoinType::LeftSemi + | JoinType::Right + | JoinType::RightSemi + | JoinType::LeftAnti + | JoinType::RightAnti + | JoinType::LeftMark + | JoinType::RightMark + | JoinType::Full + )) + { + self.staging_output_record_batches.batches.clear(); + } Ok(record_batch) } fn filter_joined_batch(&mut self) -> Result { - self.staging_output_record_batches - .coalescer - .finish_buffered_batch()?; - let record_batch = self - .staging_output_record_batches - .coalescer - .next_completed_batch() - .unwrap_or_else(|| RecordBatch::new_empty(Arc::clone(&self.schema))); + let record_batch = + concat_batches(&self.schema, &self.staging_output_record_batches.batches)?; let mut out_indices = self.staging_output_record_batches.row_indices.finish(); let mut out_mask = self.staging_output_record_batches.filter_mask.finish(); let mut batch_ids = &self.staging_output_record_batches.batch_ids; @@ -1624,11 +1513,7 @@ impl SortMergeJoinStream { } if out_mask.is_empty() { - // Coalescer already cleared by finish_buffered_batch() above - // Clear metadata only - self.staging_output_record_batches.batch_ids.clear(); - self.staging_output_record_batches.filter_mask = BooleanBuilder::new(); - self.staging_output_record_batches.row_indices = UInt64Builder::new(); + self.staging_output_record_batches.batches.clear(); return Ok(record_batch); } diff --git a/datafusion/physical-plan/src/joins/sort_merge_join/tests.rs b/datafusion/physical-plan/src/joins/sort_merge_join/tests.rs index 842480b4268b..f91bffbed78f 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join/tests.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join/tests.rs @@ -2319,14 +2319,14 @@ fn build_joined_record_batches() -> Result { ])); let mut batches = JoinedRecordBatches { - coalescer: arrow::compute::BatchCoalescer::new(Arc::clone(&schema), 8192), + batches: vec![], filter_mask: BooleanBuilder::new(), row_indices: UInt64Builder::new(), batch_ids: vec![], }; // Insert already prejoined non-filtered rows - batches.coalescer.push_batch(RecordBatch::try_new( + batches.batches.push(RecordBatch::try_new( Arc::clone(&schema), vec![ Arc::new(Int32Array::from(vec![1, 1])), @@ -2334,9 +2334,9 @@ fn build_joined_record_batches() -> Result { Arc::new(Int32Array::from(vec![1, 1])), Arc::new(Int32Array::from(vec![11, 9])), ], - )?)?; + )?); - batches.coalescer.push_batch(RecordBatch::try_new( + batches.batches.push(RecordBatch::try_new( Arc::clone(&schema), vec![ Arc::new(Int32Array::from(vec![1])), @@ -2344,9 +2344,9 @@ fn build_joined_record_batches() -> Result { Arc::new(Int32Array::from(vec![1])), Arc::new(Int32Array::from(vec![12])), ], - )?)?; + )?); - batches.coalescer.push_batch(RecordBatch::try_new( + batches.batches.push(RecordBatch::try_new( Arc::clone(&schema), vec![ Arc::new(Int32Array::from(vec![1, 1])), @@ -2354,9 +2354,9 @@ fn build_joined_record_batches() -> Result { Arc::new(Int32Array::from(vec![1, 1])), Arc::new(Int32Array::from(vec![11, 13])), ], - )?)?; + )?); - batches.coalescer.push_batch(RecordBatch::try_new( + batches.batches.push(RecordBatch::try_new( Arc::clone(&schema), vec![ Arc::new(Int32Array::from(vec![1])), @@ -2364,9 +2364,9 @@ fn build_joined_record_batches() -> Result { Arc::new(Int32Array::from(vec![1])), Arc::new(Int32Array::from(vec![12])), ], - )?)?; + )?); - batches.coalescer.push_batch(RecordBatch::try_new( + batches.batches.push(RecordBatch::try_new( Arc::clone(&schema), vec![ Arc::new(Int32Array::from(vec![1, 1])), @@ -2374,7 +2374,7 @@ fn build_joined_record_batches() -> Result { Arc::new(Int32Array::from(vec![1, 1])), Arc::new(Int32Array::from(vec![12, 11])), ], - )?)?; + )?); let streamed_indices = vec![0, 0]; batches.batch_ids.extend(vec![0; streamed_indices.len()]); @@ -2424,16 +2424,9 @@ fn build_joined_record_batches() -> Result { #[tokio::test] async fn test_left_outer_join_filtered_mask() -> Result<()> { let mut joined_batches = build_joined_record_batches()?; + let schema = joined_batches.batches.first().unwrap().schema(); - // Extract the batches from the coalescer - joined_batches.coalescer.finish_buffered_batch()?; - let mut batches_vec = vec![]; - while let Some(batch) = joined_batches.coalescer.next_completed_batch() { - batches_vec.push(batch); - } - let schema = batches_vec.first().unwrap().schema(); - - let output = concat_batches(&schema, &batches_vec)?; + let output = concat_batches(&schema, &joined_batches.batches)?; let out_mask = joined_batches.filter_mask.finish(); let out_indices = joined_batches.row_indices.finish(); @@ -2638,16 +2631,9 @@ async fn test_left_outer_join_filtered_mask() -> Result<()> { async fn test_semi_join_filtered_mask() -> Result<()> { for join_type in [LeftSemi, RightSemi] { let mut joined_batches = build_joined_record_batches()?; + let schema = joined_batches.batches.first().unwrap().schema(); - // Extract the batches from the coalescer - joined_batches.coalescer.finish_buffered_batch()?; - let mut batches_vec = vec![]; - while let Some(batch) = joined_batches.coalescer.next_completed_batch() { - batches_vec.push(batch); - } - let schema = batches_vec.first().unwrap().schema(); - - let output = concat_batches(&schema, &batches_vec)?; + let output = concat_batches(&schema, &joined_batches.batches)?; let out_mask = joined_batches.filter_mask.finish(); let out_indices = joined_batches.row_indices.finish(); @@ -2820,16 +2806,9 @@ async fn test_semi_join_filtered_mask() -> Result<()> { async fn test_anti_join_filtered_mask() -> Result<()> { for join_type in [LeftAnti, RightAnti] { let mut joined_batches = build_joined_record_batches()?; + let schema = joined_batches.batches.first().unwrap().schema(); - // Extract the batches from the coalescer - joined_batches.coalescer.finish_buffered_batch()?; - let mut batches_vec = vec![]; - while let Some(batch) = joined_batches.coalescer.next_completed_batch() { - batches_vec.push(batch); - } - let schema = batches_vec.first().unwrap().schema(); - - let output = concat_batches(&schema, &batches_vec)?; + let output = concat_batches(&schema, &joined_batches.batches)?; let out_mask = joined_batches.filter_mask.finish(); let out_indices = joined_batches.row_indices.finish(); From 4ed5cd471a9cb72a2df1315209a5a3c91464c74d Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 25 Nov 2025 09:57:48 -0500 Subject: [PATCH 03/23] Stash with assertions. --- .../src/joins/sort_merge_join/stream.rs | 179 ++++++++++++++++++ .../src/joins/sort_merge_join/tests.rs | 3 +- 2 files changed, 181 insertions(+), 1 deletion(-) diff --git a/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs b/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs index 7f37d5e508d1..9799fd8e1b44 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs @@ -352,6 +352,7 @@ pub(super) struct SortMergeJoinStream { pub(super) struct JoinedRecordBatches { /// Joined batches. Each batch is already joined columns from left and right sources pub batches: Vec, + pub joined_batches: BatchCoalescer, /// Filter match mask for each row(matched/non-matched) pub filter_mask: BooleanBuilder, /// Left row indices to glue together rows in `batches` and `filter_mask` @@ -363,11 +364,80 @@ pub(super) struct JoinedRecordBatches { } impl JoinedRecordBatches { + /// Asserts that internal metadata arrays are consistent with each other + /// Only checks if metadata is actually being used (i.e., not all empty) + #[inline] + fn debug_assert_metadata_aligned(&self) { + // Metadata arrays should be aligned IF they're being used + // (For non-filtered joins, they may all be empty) + if self.filter_mask.len() > 0 + || self.row_indices.len() > 0 + || !self.batch_ids.is_empty() + { + debug_assert_eq!( + self.filter_mask.len(), + self.row_indices.len(), + "filter_mask and row_indices must have same length when metadata is used" + ); + debug_assert_eq!( + self.filter_mask.len(), + self.batch_ids.len(), + "filter_mask and batch_ids must have same length when metadata is used" + ); + } + } + + /// Asserts that if batches is empty, metadata is also empty + #[inline] + fn debug_assert_empty_consistency(&self) { + if self.batches.is_empty() { + debug_assert_eq!( + self.filter_mask.len(), + 0, + "filter_mask should be empty when batches is empty" + ); + debug_assert_eq!( + self.row_indices.len(), + 0, + "row_indices should be empty when batches is empty" + ); + debug_assert_eq!( + self.batch_ids.len(), + 0, + "batch_ids should be empty when batches is empty" + ); + } + } + fn clear(&mut self) { self.batches.clear(); self.batch_ids.clear(); self.filter_mask = BooleanBuilder::new(); self.row_indices = UInt64Builder::new(); + + // After clear, everything should be empty + self.debug_assert_empty_consistency(); + } + + /// Drains all data from the BatchCoalescer into a single RecordBatch + /// + /// This finalizes any buffered data and concatenates all completed batches. + fn drain_to_batch(&mut self) -> Result { + // Finalize any partial buffered data + self.joined_batches.finish_buffered_batch()?; + + // Collect all completed batches + let mut batches = Vec::new(); + while let Some(batch) = self.joined_batches.next_completed_batch() { + batches.push(batch); + } + + // Return appropriate result based on what we collected + match batches.len() { + 0 => Ok(RecordBatch::new_empty(self.joined_batches.schema())), + 1 => Ok(batches.into_iter().next().unwrap()), + _ => Ok(concat_batches(&self.joined_batches.schema(), &batches)?), + } } } impl RecordBatchStream for SortMergeJoinStream { @@ -387,6 +457,23 @@ fn last_index_for_row( batch_ids: &[usize], indices_len: usize, ) -> bool { + debug_assert_eq!( + indices.len(), + indices_len, + "indices.len() should match indices_len parameter" + ); + debug_assert_eq!( + batch_ids.len(), + indices_len, + "batch_ids.len() should match indices_len" + ); + debug_assert!( + row_index < indices_len, + "row_index {} should be < indices_len {}", + row_index, + indices_len + ); + row_index == indices_len - 1 || batch_ids[row_index] != batch_ids[row_index + 1] || indices.value(row_index) != indices.value(row_index + 1) @@ -592,6 +679,10 @@ impl Stream for SortMergeJoinStream { { self.freeze_all()?; + // Verify metadata alignment before checking if we have batches to filter + self.staging_output_record_batches + .debug_assert_metadata_aligned(); + // If join is filtered and there is joined tuples waiting // to be filtered if !self @@ -673,6 +764,11 @@ impl Stream for SortMergeJoinStream { } } else { self.freeze_all()?; + + // Verify metadata alignment before checking if we have batches to output + self.staging_output_record_batches + .debug_assert_metadata_aligned(); + if !self.staging_output_record_batches.batches.is_empty() { let record_batch = self.output_record_batch_and_reset()?; // For non-filtered join output whenever the target output batch size @@ -705,6 +801,10 @@ impl Stream for SortMergeJoinStream { SortMergeJoinState::Exhausted => { self.freeze_all()?; + // Verify metadata alignment before final output + self.staging_output_record_batches + .debug_assert_metadata_aligned(); + // if there is still something not processed if !self.staging_output_record_batches.batches.is_empty() { if self.filter.is_some() @@ -793,6 +893,7 @@ impl SortMergeJoinStream { filter, staging_output_record_batches: JoinedRecordBatches { batches: vec![], + joined_batches: BatchCoalescer::new(Arc::clone(&schema), batch_size), filter_mask: BooleanBuilder::new(), row_indices: UInt64Builder::new(), batch_ids: vec![], @@ -1154,6 +1255,11 @@ impl SortMergeJoinStream { fn freeze_all(&mut self) -> Result<()> { self.freeze_buffered(self.buffered_data.batches.len())?; self.freeze_streamed()?; + + // After freezing, metadata should be aligned + self.staging_output_record_batches + .debug_assert_metadata_aligned(); + Ok(()) } @@ -1165,6 +1271,11 @@ impl SortMergeJoinStream { self.freeze_streamed()?; // Only freeze and produce the first batch in buffered_data as the batch is fully processed self.freeze_buffered(1)?; + + // After freezing, metadata should be aligned + self.staging_output_record_batches + .debug_assert_metadata_aligned(); + Ok(()) } @@ -1199,6 +1310,10 @@ impl SortMergeJoinStream { 0, ); + // Verify metadata arrays stayed aligned after extending + self.staging_output_record_batches + .debug_assert_metadata_aligned(); + self.staging_output_record_batches .batches .push(record_batch); @@ -1245,6 +1360,11 @@ impl SortMergeJoinStream { self.staging_output_record_batches.batch_ids.len() + num_rows, 0, ); + + // Verify metadata arrays stayed aligned after extending + self.staging_output_record_batches + .debug_assert_metadata_aligned(); + self.staging_output_record_batches .batches .push(record_batch); @@ -1414,6 +1534,10 @@ impl SortMergeJoinStream { self.streamed_batch_counter.load(Relaxed), ); + // Verify metadata arrays stayed aligned after extending + self.staging_output_record_batches + .debug_assert_metadata_aligned(); + // For outer joins, we need to push the null joined rows to the output if // all joined rows are failed on the join filter. // I.e., if all rows joined from a streamed row are failed with the join filter, @@ -1459,6 +1583,10 @@ impl SortMergeJoinStream { } fn output_record_batch_and_reset(&mut self) -> Result { + // Metadata should be aligned before we concat and output + self.staging_output_record_batches + .debug_assert_metadata_aligned(); + let record_batch = concat_batches(&self.schema, &self.staging_output_record_batches.batches)?; (&record_batch).record_output(&self.join_metrics.baseline_metrics()); @@ -1487,13 +1615,26 @@ impl SortMergeJoinStream { | JoinType::Full )) { + // For non-filtered outer joins, we clear batches immediately after concat + // since we don't need them for deferred filter processing self.staging_output_record_batches.batches.clear(); + } else { + // For filtered outer joins, we keep the batches for later filter processing + // in filter_joined_batch(). Batches should still contain data. + debug_assert!( + !self.staging_output_record_batches.batches.is_empty(), + "For filtered outer joins, batches should not be empty after concat since they're needed for filter processing" + ); } Ok(record_batch) } fn filter_joined_batch(&mut self) -> Result { + // Metadata should be aligned before processing + self.staging_output_record_batches + .debug_assert_metadata_aligned(); + let record_batch = concat_batches(&self.schema, &self.staging_output_record_batches.batches)?; let mut out_indices = self.staging_output_record_batches.row_indices.finish(); @@ -1512,11 +1653,40 @@ impl SortMergeJoinStream { batch_ids = &default_batch_ids; } + // After potential reconstruction, metadata should align with batch row count + debug_assert_eq!( + out_indices.len(), + record_batch.num_rows(), + "out_indices length should match record_batch row count" + ); + debug_assert_eq!( + out_mask.len(), + record_batch.num_rows(), + "out_mask length should match record_batch row count (unless empty)" + ); + debug_assert_eq!( + batch_ids.len(), + record_batch.num_rows(), + "batch_ids length should match record_batch row count" + ); + if out_mask.is_empty() { self.staging_output_record_batches.batches.clear(); return Ok(record_batch); } + // Validate inputs to get_corrected_filter_mask + debug_assert_eq!( + out_indices.len(), + out_mask.len(), + "out_indices and out_mask must have same length for get_corrected_filter_mask" + ); + debug_assert_eq!( + batch_ids.len(), + out_mask.len(), + "batch_ids and out_mask must have same length for get_corrected_filter_mask" + ); + let maybe_corrected_mask = get_corrected_filter_mask( self.join_type, &out_indices, @@ -1539,6 +1709,15 @@ impl SortMergeJoinStream { record_batch: &RecordBatch, corrected_mask: &BooleanArray, ) -> Result { + // Corrected mask should have length matching or exceeding record_batch rows + // (for outer joins it may be longer to include null-joined rows) + debug_assert!( + corrected_mask.len() >= record_batch.num_rows(), + "corrected_mask length ({}) should be >= record_batch rows ({})", + corrected_mask.len(), + record_batch.num_rows() + ); + let mut filtered_record_batch = filter_record_batch(record_batch, corrected_mask)?; let left_columns_length = self.streamed_schema.fields.len(); diff --git a/datafusion/physical-plan/src/joins/sort_merge_join/tests.rs b/datafusion/physical-plan/src/joins/sort_merge_join/tests.rs index f91bffbed78f..f58266bdd3ae 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join/tests.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join/tests.rs @@ -31,7 +31,7 @@ use arrow::array::{ BinaryArray, BooleanArray, Date32Array, Date64Array, FixedSizeBinaryArray, Int32Array, RecordBatch, UInt64Array, }; -use arrow::compute::{concat_batches, filter_record_batch, SortOptions}; +use arrow::compute::{concat_batches, filter_record_batch, BatchCoalescer, SortOptions}; use arrow::datatypes::{DataType, Field, Schema}; use datafusion_common::JoinType::*; @@ -2320,6 +2320,7 @@ fn build_joined_record_batches() -> Result { let mut batches = JoinedRecordBatches { batches: vec![], + joined_batches: BatchCoalescer::new(Arc::clone(&schema), 8192), filter_mask: BooleanBuilder::new(), row_indices: UInt64Builder::new(), batch_ids: vec![], From 43646568b9a8e2ae33f4882f970bb70d132d4adc Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 25 Nov 2025 10:03:03 -0500 Subject: [PATCH 04/23] Stash with assertions. --- datafusion/physical-plan/src/joins/sort_merge_join/stream.rs | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs b/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs index 9799fd8e1b44..7dcf87edea74 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs @@ -410,6 +410,11 @@ impl JoinedRecordBatches { } fn clear(&mut self) { + // Note: clear() can be called when batches still contains data! + // This happens in filter_joined_batch() after concat_batches() has read + // the batches but before they're removed. The batches have been processed + // into output, so clearing them here is the final cleanup step. + self.batches.clear(); self.batch_ids.clear(); self.filter_mask = BooleanBuilder::new(); From 7a41fe6d4ae29c73135c6a47ed854b43db6c53eb Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 25 Nov 2025 10:50:07 -0500 Subject: [PATCH 05/23] encapsulate --- .../src/joins/sort_merge_join/stream.rs | 180 +++++++++++------- 1 file changed, 113 insertions(+), 67 deletions(-) diff --git a/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs b/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs index 7dcf87edea74..72b4871724c2 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs @@ -409,6 +409,93 @@ impl JoinedRecordBatches { } } + /// Pushes a batch with null metadata (used for Full join null-joined rows) + /// + /// This is used when outputting rows that didn't match any rows from the other side. + /// The metadata is set to nulls because these rows don't correspond to any input row index. + fn push_batch_with_null_metadata(&mut self, batch: RecordBatch, join_type: JoinType) { + debug_assert!( + matches!(join_type, JoinType::Full), + "push_batch_with_null_metadata should only be called for Full joins" + ); + + let num_rows = batch.num_rows(); + + self.filter_mask.append_nulls(num_rows); + self.row_indices.append_nulls(num_rows); + self.batch_ids.resize( + self.batch_ids.len() + num_rows, + 0, // batch_id = 0 for null-joined rows + ); + + self.debug_assert_metadata_aligned(); + self.batches.push(batch); + } + + /// Pushes a batch with filter metadata (used for filtered outer/semi/anti/mark joins) + /// + /// This is the primary method for adding batches in filtered joins where we need to track: + /// - Which rows passed the filter (filter_mask) + /// - Which input row each output row came from (row_indices) + /// - Which input batch each output row came from (batch_ids) + /// + /// The metadata is essential for get_corrected_filter_mask() to implement outer join semantics + /// (ensuring at least one output row per input row, filling with nulls when needed). + fn push_batch_with_filter_metadata( + &mut self, + batch: RecordBatch, + row_indices: &UInt64Array, + filter_mask: &BooleanArray, + streamed_batch_id: usize, + join_type: JoinType, + ) { + debug_assert!( + matches!( + join_type, + JoinType::Left + | JoinType::LeftSemi + | JoinType::LeftMark + | JoinType::Right + | JoinType::RightSemi + | JoinType::RightMark + | JoinType::LeftAnti + | JoinType::RightAnti + | JoinType::Full + ), + "push_batch_with_filter_metadata should only be called for outer/semi/anti/mark joins that need deferred filtering" + ); + + debug_assert_eq!( + row_indices.len(), + filter_mask.len(), + "row_indices and filter_mask must have same length" + ); + + // For Full joins, we keep the pre_mask (with nulls), for others we keep the cleaned mask + self.filter_mask.extend(filter_mask); + self.row_indices.extend(row_indices); + self.batch_ids.resize( + self.batch_ids.len() + row_indices.len(), + streamed_batch_id, + ); + + self.debug_assert_metadata_aligned(); + self.batches.push(batch); + } + + /// Pushes a batch without metadata (used for non-filtered joins) + /// + /// For non-filtered joins, we don't need to track row-level metadata because + /// the output is produced directly without deferred filter processing. + /// + /// Note: For Full joins without filters, metadata may exist from null-joined rows + /// that were produced earlier, but we don't add metadata for the regular joined rows. + fn push_batch_without_metadata(&mut self, batch: RecordBatch, _join_type: JoinType) { + // No preconditions to check - batches can be pushed regardless of metadata state + // because this is used in non-filtered paths where metadata isn't needed + self.batches.push(batch); + } + fn clear(&mut self) { // Note: clear() can be called when batches still contains data! // This happens in filter_joined_batch() after concat_batches() has read @@ -1303,25 +1390,8 @@ impl SortMergeJoinStream { &buffered_indices, buffered_batch, )? { - let num_rows = record_batch.num_rows(); - self.staging_output_record_batches - .filter_mask - .append_nulls(num_rows); - self.staging_output_record_batches - .row_indices - .append_nulls(num_rows); - self.staging_output_record_batches.batch_ids.resize( - self.staging_output_record_batches.batch_ids.len() + num_rows, - 0, - ); - - // Verify metadata arrays stayed aligned after extending - self.staging_output_record_batches - .debug_assert_metadata_aligned(); - self.staging_output_record_batches - .batches - .push(record_batch); + .push_batch_with_null_metadata(record_batch, self.join_type); } buffered_batch.null_joined.clear(); } @@ -1353,26 +1423,8 @@ impl SortMergeJoinStream { &buffered_indices, buffered_batch, )? { - let num_rows = record_batch.num_rows(); - - self.staging_output_record_batches - .filter_mask - .append_nulls(num_rows); - self.staging_output_record_batches - .row_indices - .append_nulls(num_rows); - self.staging_output_record_batches.batch_ids.resize( - self.staging_output_record_batches.batch_ids.len() + num_rows, - 0, - ); - - // Verify metadata arrays stayed aligned after extending - self.staging_output_record_batches - .debug_assert_metadata_aligned(); - self.staging_output_record_batches - .batches - .push(record_batch); + .push_batch_with_null_metadata(record_batch, self.join_type); } buffered_batch.join_filter_not_matched_map.clear(); @@ -1501,7 +1553,9 @@ impl SortMergeJoinStream { }; // Push the filtered batch which contains rows passing join filter to the output - if matches!( + // For outer/semi/anti/mark joins with deferred filtering, push the unfiltered batch with metadata + // For INNER joins, filter immediately and push without metadata + let needs_deferred_filtering = matches!( self.join_type, JoinType::Left | JoinType::LeftSemi @@ -1512,37 +1566,31 @@ impl SortMergeJoinStream { | JoinType::LeftMark | JoinType::RightMark | JoinType::Full - ) { + ); + + if needs_deferred_filtering { + // Outer/semi/anti/mark joins: push unfiltered batch with metadata for deferred filtering + let mask_to_use = if !matches!(self.join_type, JoinType::Full) { + &mask + } else { + pre_mask + }; + self.staging_output_record_batches - .batches - .push(output_batch); + .push_batch_with_filter_metadata( + output_batch, + &left_indices, + mask_to_use, + self.streamed_batch_counter.load(Relaxed), + self.join_type, + ); } else { + // INNER joins: filter immediately and push without metadata let filtered_batch = filter_record_batch(&output_batch, &mask)?; self.staging_output_record_batches - .batches - .push(filtered_batch); + .push_batch_without_metadata(filtered_batch, self.join_type); } - if !matches!(self.join_type, JoinType::Full) { - self.staging_output_record_batches.filter_mask.extend(&mask); - } else { - self.staging_output_record_batches - .filter_mask - .extend(pre_mask); - } - self.staging_output_record_batches - .row_indices - .extend(&left_indices); - self.staging_output_record_batches.batch_ids.resize( - self.staging_output_record_batches.batch_ids.len() - + left_indices.len(), - self.streamed_batch_counter.load(Relaxed), - ); - - // Verify metadata arrays stayed aligned after extending - self.staging_output_record_batches - .debug_assert_metadata_aligned(); - // For outer joins, we need to push the null joined rows to the output if // all joined rows are failed on the join filter. // I.e., if all rows joined from a streamed row are failed with the join filter, @@ -1572,13 +1620,11 @@ impl SortMergeJoinStream { } } else { self.staging_output_record_batches - .batches - .push(output_batch); + .push_batch_without_metadata(output_batch, self.join_type); } } else { self.staging_output_record_batches - .batches - .push(output_batch); + .push_batch_without_metadata(output_batch, self.join_type); } } From b986fd7cf0334f524793ab3a4871f0ac74ee63f2 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 25 Nov 2025 12:55:31 -0500 Subject: [PATCH 06/23] encapsulate --- .../src/joins/sort_merge_join/stream.rs | 70 ++++++++++++------- 1 file changed, 46 insertions(+), 24 deletions(-) diff --git a/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs b/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs index 72b4871724c2..86b47e3880e8 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs @@ -351,19 +351,45 @@ pub(super) struct SortMergeJoinStream { /// Joined batches with attached join filter information pub(super) struct JoinedRecordBatches { /// Joined batches. Each batch is already joined columns from left and right sources - pub batches: Vec, - pub joined_batches: BatchCoalescer, + pub(super) batches: Vec, + pub(super) joined_batches: BatchCoalescer, /// Filter match mask for each row(matched/non-matched) - pub filter_mask: BooleanBuilder, + pub(super) filter_mask: BooleanBuilder, /// Left row indices to glue together rows in `batches` and `filter_mask` - pub row_indices: UInt64Builder, + pub(super) row_indices: UInt64Builder, /// Which unique batch id the row belongs to /// It is necessary to differentiate rows that are distributed the way when they point to the same /// row index but in not the same batches - pub batch_ids: Vec, + pub(super) batch_ids: Vec, } impl JoinedRecordBatches { + /// Returns true if there are no batches accumulated + #[inline] + fn is_empty(&self) -> bool { + self.batches.is_empty() + } + + /// Concatenates all accumulated batches into a single RecordBatch + fn concat_batches(&self, schema: &SchemaRef) -> Result { + Ok(concat_batches(schema, &self.batches)?) + } + + /// Finishes and returns the metadata arrays, clearing the builders + /// + /// Returns (row_indices, filter_mask, batch_ids_ref) + /// Note: batch_ids is returned as a reference since it's still needed in the struct + fn finish_metadata(&mut self) -> (UInt64Array, BooleanArray, &[usize]) { + let row_indices = self.row_indices.finish(); + let filter_mask = self.filter_mask.finish(); + (row_indices, filter_mask, &self.batch_ids) + } + + /// Clears only the batches vector (used in non-filtered path) + fn clear_batches(&mut self) { + self.batches.clear(); + } + /// Asserts that internal metadata arrays are consistent with each other /// Only checks if metadata is actually being used (i.e., not all empty) #[inline] @@ -474,10 +500,8 @@ impl JoinedRecordBatches { // For Full joins, we keep the pre_mask (with nulls), for others we keep the cleaned mask self.filter_mask.extend(filter_mask); self.row_indices.extend(row_indices); - self.batch_ids.resize( - self.batch_ids.len() + row_indices.len(), - streamed_batch_id, - ); + self.batch_ids + .resize(self.batch_ids.len() + row_indices.len(), streamed_batch_id); self.debug_assert_metadata_aligned(); self.batches.push(batch); @@ -777,10 +801,7 @@ impl Stream for SortMergeJoinStream { // If join is filtered and there is joined tuples waiting // to be filtered - if !self - .staging_output_record_batches - .batches - .is_empty() + if !self.staging_output_record_batches.is_empty() { // Apply filter on joined tuples and get filtered batch let out_filtered_batch = @@ -898,7 +919,7 @@ impl Stream for SortMergeJoinStream { .debug_assert_metadata_aligned(); // if there is still something not processed - if !self.staging_output_record_batches.batches.is_empty() { + if !self.staging_output_record_batches.is_empty() { if self.filter.is_some() && matches!( self.join_type, @@ -1638,8 +1659,9 @@ impl SortMergeJoinStream { self.staging_output_record_batches .debug_assert_metadata_aligned(); - let record_batch = - concat_batches(&self.schema, &self.staging_output_record_batches.batches)?; + let record_batch = self + .staging_output_record_batches + .concat_batches(&self.schema)?; (&record_batch).record_output(&self.join_metrics.baseline_metrics()); // If join filter exists, `self.output_size` is not accurate as we don't know the exact // number of rows in the output record batch. If streamed row joined with buffered rows, @@ -1668,12 +1690,12 @@ impl SortMergeJoinStream { { // For non-filtered outer joins, we clear batches immediately after concat // since we don't need them for deferred filter processing - self.staging_output_record_batches.batches.clear(); + self.staging_output_record_batches.clear_batches(); } else { // For filtered outer joins, we keep the batches for later filter processing // in filter_joined_batch(). Batches should still contain data. debug_assert!( - !self.staging_output_record_batches.batches.is_empty(), + !self.staging_output_record_batches.is_empty(), "For filtered outer joins, batches should not be empty after concat since they're needed for filter processing" ); } @@ -1686,11 +1708,11 @@ impl SortMergeJoinStream { self.staging_output_record_batches .debug_assert_metadata_aligned(); - let record_batch = - concat_batches(&self.schema, &self.staging_output_record_batches.batches)?; - let mut out_indices = self.staging_output_record_batches.row_indices.finish(); - let mut out_mask = self.staging_output_record_batches.filter_mask.finish(); - let mut batch_ids = &self.staging_output_record_batches.batch_ids; + let record_batch = self + .staging_output_record_batches + .concat_batches(&self.schema)?; + let (mut out_indices, mut out_mask, mut batch_ids) = + self.staging_output_record_batches.finish_metadata(); let default_batch_ids = vec![0; record_batch.num_rows()]; // If only nulls come in and indices sizes doesn't match with expected record batch count @@ -1722,7 +1744,7 @@ impl SortMergeJoinStream { ); if out_mask.is_empty() { - self.staging_output_record_batches.batches.clear(); + self.staging_output_record_batches.clear_batches(); return Ok(record_batch); } From 387c882aff9b01c7592cd76b901cb88ee4086ebc Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 1 Dec 2025 11:11:31 -0500 Subject: [PATCH 07/23] encapsulate --- datafusion/physical-plan/src/joins/sort_merge_join/stream.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs b/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs index 86b47e3880e8..a549fbe5e335 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs @@ -882,7 +882,7 @@ impl Stream for SortMergeJoinStream { self.staging_output_record_batches .debug_assert_metadata_aligned(); - if !self.staging_output_record_batches.batches.is_empty() { + if !self.staging_output_record_batches.is_empty() { let record_batch = self.output_record_batch_and_reset()?; // For non-filtered join output whenever the target output batch size // is hit. For filtered join its needed to output on later phase From efa299688ca3945e99f71ccc244a0e2e8fcbf07f Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 1 Dec 2025 15:52:22 -0500 Subject: [PATCH 08/23] pre-refactor --- .../src/joins/sort_merge_join/stream.rs | 88 ++++++++++++++----- 1 file changed, 64 insertions(+), 24 deletions(-) diff --git a/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs b/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs index a549fbe5e335..cd62c904db32 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs @@ -348,29 +348,71 @@ pub(super) struct SortMergeJoinStream { pub streamed_batch_counter: AtomicUsize, } -/// Joined batches with attached join filter information +/// Staging area for joined data before output +/// +/// Accumulates joined rows until either: +/// - Target batch size reached (for efficiency) +/// - Stream exhausted (flush remaining data) +/// +/// # Metadata Tracking (Critical for Filtered Joins) +/// +/// Outer joins must emit at least one row per input row. When a join filter rejects +/// all matches for an input row, we must emit a null-joined row. This requires grouping +/// output rows by input row to check if ANY match passed the filter. +/// +/// Metadata fields (one entry per output row): +/// - `row_indices[i]`: Which input row within its batch (for grouping output rows) +/// - `filter_mask[i]`: Did this output row pass the join filter? (detect if input row matched) +/// - `batch_ids[i]`: Which input batch (disambiguates row_indices across batches) +/// +/// Invariant: `metadata.len() == sum(batch.num_rows())` for all batches +/// +/// # Usage Pattern: `is_empty()` Always Means "Any Data At All" +/// +/// Called in three contexts, all checking for ANY remaining data (not optimal batch): +/// 1. Init state: Filter accumulated data before next streamed row +/// 2. JoinOutput state: Output accumulated data after hitting batch_size +/// 3. Exhausted state: Flush any remaining data +/// +/// # Future Refactor: Using BatchCoalescer +/// +/// To replace `batches: Vec` with `BatchCoalescer`: +/// - `is_empty()` → Must check BOTH buffered data AND completed batches +/// - `concat_batches()` → Drain with `finish_buffered_batch()` + `next_completed_batch()` +/// - Previous refactor failed: only checked `has_completed_batch()`, missing buffered data +/// - BatchCoalescer preserves row order, so metadata alignment is maintained pub(super) struct JoinedRecordBatches { /// Joined batches. Each batch is already joined columns from left and right sources pub(super) batches: Vec, + /// UNUSED: Placeholder for future refactor. Requires proper is_empty() semantics. pub(super) joined_batches: BatchCoalescer, - /// Filter match mask for each row(matched/non-matched) + /// Did each output row pass the join filter? (detect if input row found any match) pub(super) filter_mask: BooleanBuilder, - /// Left row indices to glue together rows in `batches` and `filter_mask` + /// Which input row (within batch) produced each output row? (for grouping by input row) pub(super) row_indices: UInt64Builder, - /// Which unique batch id the row belongs to - /// It is necessary to differentiate rows that are distributed the way when they point to the same - /// row index but in not the same batches + /// Which input batch did each output row come from? (disambiguates row_indices) pub(super) batch_ids: Vec, } impl JoinedRecordBatches { /// Returns true if there are no batches accumulated + /// + /// Used in three contexts, all checking for ANY data (not optimal batch size): + /// 1. Init state: Filter accumulated data before next streamed row + /// 2. JoinOutput state: Output data after hitting batch_size threshold + /// 3. Exhausted state: Flush remaining data + /// + /// For future BatchCoalescer refactor, use `joined_batches.is_empty()`. #[inline] fn is_empty(&self) -> bool { self.batches.is_empty() } /// Concatenates all accumulated batches into a single RecordBatch + /// + /// For future BatchCoalescer refactor: + /// - Call `finish_buffered_batch()` to finalize partial data + /// - Drain with `next_completed_batch()` in a loop fn concat_batches(&self, schema: &SchemaRef) -> Result { Ok(concat_batches(schema, &self.batches)?) } @@ -435,10 +477,12 @@ impl JoinedRecordBatches { } } - /// Pushes a batch with null metadata (used for Full join null-joined rows) + /// Pushes a batch with null metadata (Full join null-joined rows only) + /// + /// These buffered rows had NO matching streamed rows. Since we can't group + /// by input row (no input row exists), we use null metadata as a sentinel. /// - /// This is used when outputting rows that didn't match any rows from the other side. - /// The metadata is set to nulls because these rows don't correspond to any input row index. + /// Maintains invariant: N rows → N metadata entries (nulls) fn push_batch_with_null_metadata(&mut self, batch: RecordBatch, join_type: JoinType) { debug_assert!( matches!(join_type, JoinType::Full), @@ -458,15 +502,15 @@ impl JoinedRecordBatches { self.batches.push(batch); } - /// Pushes a batch with filter metadata (used for filtered outer/semi/anti/mark joins) + /// Pushes a batch with filter metadata (filtered outer/semi/anti/mark joins) /// - /// This is the primary method for adding batches in filtered joins where we need to track: - /// - Which rows passed the filter (filter_mask) - /// - Which input row each output row came from (row_indices) - /// - Which input batch each output row came from (batch_ids) + /// Deferred filtering: An input row may join with multiple buffered rows, but we + /// don't know yet if ALL matches failed the filter. We track metadata so + /// `get_corrected_filter_mask()` can later group by input row and decide: + /// - If ANY match passed: emit passing rows + /// - If ALL matches failed: emit null-joined row /// - /// The metadata is essential for get_corrected_filter_mask() to implement outer join semantics - /// (ensuring at least one output row per input row, filling with nulls when needed). + /// Maintains invariant: N rows → N metadata entries fn push_batch_with_filter_metadata( &mut self, batch: RecordBatch, @@ -507,16 +551,12 @@ impl JoinedRecordBatches { self.batches.push(batch); } - /// Pushes a batch without metadata (used for non-filtered joins) - /// - /// For non-filtered joins, we don't need to track row-level metadata because - /// the output is produced directly without deferred filter processing. + /// Pushes a batch without metadata (non-filtered joins) /// - /// Note: For Full joins without filters, metadata may exist from null-joined rows - /// that were produced earlier, but we don't add metadata for the regular joined rows. + /// No deferred filtering needed. Either every join match is output (Inner), + /// or null-joined rows are handled separately. No need to track which input + /// row produced which output row. fn push_batch_without_metadata(&mut self, batch: RecordBatch, _join_type: JoinType) { - // No preconditions to check - batches can be pushed regardless of metadata state - // because this is used in non-filtered paths where metadata isn't needed self.batches.push(batch); } From a5c926f20dd9ef31109a7d844d81fa6ff74ce250 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 1 Dec 2025 17:28:50 -0500 Subject: [PATCH 09/23] get rid of confusing output_size --- .../src/joins/sort_merge_join/stream.rs | 57 ++++++------------- 1 file changed, 16 insertions(+), 41 deletions(-) diff --git a/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs b/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs index cd62c904db32..9d64b0e96fef 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs @@ -156,6 +156,14 @@ impl StreamedBatch { } } + /// Number of unfrozen output pairs in this streamed batch + fn num_output_rows(&self) -> usize { + self.output_indices + .iter() + .map(|chunk| chunk.streamed_indices.len()) + .sum() + } + /// Appends new pair consisting of current streamed index and `buffered_idx` /// index of buffered batch with `buffered_batch_idx` index. fn append_output_pair( @@ -325,10 +333,6 @@ pub(super) struct SortMergeJoinStream { /// Output buffer. Currently used by filtering as it requires double buffering /// to avoid small/empty batches. Non-filtered join outputs directly from `staging_output_record_batches.batches` pub output: BatchCoalescer, - /// Staging output size, including output batches and staging joined results. - /// Increased when we put rows into buffer and decreased after we actually output batches. - /// Used to trigger output when sufficient rows are ready - pub output_size: usize, /// The comparison result of current streamed row and buffered batches pub current_ordering: Ordering, /// Manages the process of spilling and reading back intermediate data @@ -390,7 +394,7 @@ pub(super) struct JoinedRecordBatches { pub(super) filter_mask: BooleanBuilder, /// Which input row (within batch) produced each output row? (for grouping by input row) pub(super) row_indices: UInt64Builder, - /// Which input batch did each output row come from? (disambiguates row_indices) + /// Which input batch did each output row come from? (disambiguate row_indices) pub(super) batch_ids: Vec, } @@ -574,27 +578,6 @@ impl JoinedRecordBatches { // After clear, everything should be empty self.debug_assert_empty_consistency(); } - - /// Drains all data from the BatchCoalescer into a single RecordBatch - /// - /// This finalizes any buffered data and concatenates all completed batches. - fn drain_to_batch(&mut self) -> Result { - // Finalize any partial buffered data - self.joined_batches.finish_buffered_batch()?; - - // Collect all completed batches - let mut batches = Vec::new(); - while let Some(batch) = self.joined_batches.next_completed_batch() { - batches.push(batch); - } - - // Return appropriate result based on what we collected - match batches.len() { - 0 => Ok(RecordBatch::new_empty(self.joined_batches.schema())), - 1 => Ok(batches.into_iter().next().unwrap()), - _ => Ok(concat_batches(&self.joined_batches.schema(), &batches)?), - } - } } impl RecordBatchStream for SortMergeJoinStream { fn schema(&self) -> SchemaRef { @@ -910,7 +893,7 @@ impl Stream for SortMergeJoinStream { SortMergeJoinState::JoinOutput => { self.join_partial()?; - if self.output_size < self.batch_size { + if self.num_unfrozen_pairs() < self.batch_size { if self.buffered_data.scanning_finished() { self.buffered_data.scanning_reset(); self.state = SortMergeJoinState::Init; @@ -1052,7 +1035,6 @@ impl SortMergeJoinStream { batch_ids: vec![], }, output: BatchCoalescer::new(schema, batch_size), - output_size: 0, batch_size, join_type, join_metrics, @@ -1063,6 +1045,11 @@ impl SortMergeJoinStream { }) } + /// Number of unfrozen output pairs (used to decide when to freeze + output) + fn num_unfrozen_pairs(&self) -> usize { + self.streamed_batch.num_output_rows() + } + /// Poll next streamed row fn poll_streamed_row(&mut self, cx: &mut Context) -> Poll>> { loop { @@ -1362,7 +1349,7 @@ impl SortMergeJoinStream { if join_buffered { // joining streamed/nulls and buffered while !self.buffered_data.scanning_finished() - && self.output_size < self.batch_size + && self.num_unfrozen_pairs() < self.batch_size { let scanning_idx = self.buffered_data.scanning_idx(); if join_streamed { @@ -1378,7 +1365,6 @@ impl SortMergeJoinStream { .null_joined .push(scanning_idx); } - self.output_size += 1; self.buffered_data.scanning_advance(); if self.buffered_data.scanning_finished() { @@ -1398,7 +1384,6 @@ impl SortMergeJoinStream { self.streamed_batch .append_output_pair(scanning_batch_idx, scanning_idx); - self.output_size += 1; self.buffered_data.scanning_finish(); self.streamed_joined = true; } @@ -1703,16 +1688,6 @@ impl SortMergeJoinStream { .staging_output_record_batches .concat_batches(&self.schema)?; (&record_batch).record_output(&self.join_metrics.baseline_metrics()); - // If join filter exists, `self.output_size` is not accurate as we don't know the exact - // number of rows in the output record batch. If streamed row joined with buffered rows, - // once join filter is applied, the number of output rows may be more than 1. - // If `record_batch` is empty, we should reset `self.output_size` to 0. It could be happened - // when the join filter is applied and all rows are filtered out. - if record_batch.num_rows() == 0 || record_batch.num_rows() > self.output_size { - self.output_size = 0; - } else { - self.output_size -= record_batch.num_rows(); - } if !(self.filter.is_some() && matches!( From f72530832583113d459f8bd2d7f11010cf8dbcbd Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 1 Dec 2025 17:48:44 -0500 Subject: [PATCH 10/23] refactor --- .../src/joins/sort_merge_join/stream.rs | 144 ++++++++---------- 1 file changed, 61 insertions(+), 83 deletions(-) diff --git a/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs b/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs index 9d64b0e96fef..de9b5144d81f 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs @@ -802,48 +802,12 @@ impl Stream for SortMergeJoinStream { match self.current_ordering { Ordering::Less | Ordering::Equal => { if !streamed_exhausted { - if self.filter.is_some() - && matches!( - self.join_type, - JoinType::Left - | JoinType::LeftSemi - | JoinType::LeftMark - | JoinType::Right - | JoinType::RightSemi - | JoinType::RightMark - | JoinType::LeftAnti - | JoinType::RightAnti - | JoinType::Full - ) - { - self.freeze_all()?; - - // Verify metadata alignment before checking if we have batches to filter - self.staging_output_record_batches - .debug_assert_metadata_aligned(); - - // If join is filtered and there is joined tuples waiting - // to be filtered - if !self.staging_output_record_batches.is_empty() - { - // Apply filter on joined tuples and get filtered batch - let out_filtered_batch = - self.filter_joined_batch()?; - - // Append filtered batch to the output buffer - self.output - .push_batch(out_filtered_batch) - .expect("Failed to push output batch"); - - if self.output.has_completed_batch() { - let record_batch = self - .output - .next_completed_batch() - .expect("Failed to get output batch"); - return Poll::Ready(Some(Ok( - record_batch, - ))); + if self.needs_deferred_filtering() { + match self.process_filtered_batches()? { + Poll::Ready(Some(batch)) => { + return Poll::Ready(Some(Ok(batch))); } + Poll::Ready(None) | Poll::Pending => {} } } @@ -912,20 +876,7 @@ impl Stream for SortMergeJoinStream { // because target output batch size can be hit in the middle of // filtering causing the filtering to be incomplete and causing // correctness issues - if self.filter.is_some() - && matches!( - self.join_type, - JoinType::Left - | JoinType::LeftSemi - | JoinType::Right - | JoinType::RightSemi - | JoinType::LeftAnti - | JoinType::RightAnti - | JoinType::LeftMark - | JoinType::RightMark - | JoinType::Full - ) - { + if self.needs_deferred_filtering() { continue; } @@ -943,20 +894,7 @@ impl Stream for SortMergeJoinStream { // if there is still something not processed if !self.staging_output_record_batches.is_empty() { - if self.filter.is_some() - && matches!( - self.join_type, - JoinType::Left - | JoinType::LeftSemi - | JoinType::Right - | JoinType::RightSemi - | JoinType::LeftAnti - | JoinType::RightAnti - | JoinType::Full - | JoinType::LeftMark - | JoinType::RightMark - ) - { + if self.needs_deferred_filtering() { let record_batch = self.filter_joined_batch()?; return Poll::Ready(Some(Ok(record_batch))); } else { @@ -1050,6 +988,59 @@ impl SortMergeJoinStream { self.streamed_batch.num_output_rows() } + /// Returns true if this join needs deferred filtering + /// + /// Deferred filtering is needed when a filter exists and the join type requires + /// ensuring each input row produces at least one output row (or exactly one for semi). + fn needs_deferred_filtering(&self) -> bool { + self.filter.is_some() + && matches!( + self.join_type, + JoinType::Left + | JoinType::LeftSemi + | JoinType::LeftMark + | JoinType::Right + | JoinType::RightSemi + | JoinType::RightMark + | JoinType::LeftAnti + | JoinType::RightAnti + | JoinType::Full + ) + } + + /// Process accumulated batches for filtered joins + /// + /// Freezes unfrozen pairs, applies deferred filtering, and outputs if ready. + /// Returns Poll::Ready with a batch if one is available, otherwise Poll::Pending. + fn process_filtered_batches(&mut self) -> Poll>> { + self.freeze_all()?; + + // Verify metadata alignment before checking if we have batches to filter + self.staging_output_record_batches + .debug_assert_metadata_aligned(); + + // If join is filtered and there is joined tuples waiting to be filtered + if !self.staging_output_record_batches.is_empty() { + // Apply filter on joined tuples and get filtered batch + let out_filtered_batch = self.filter_joined_batch()?; + + // Append filtered batch to the output buffer + self.output + .push_batch(out_filtered_batch) + .expect("Failed to push output batch"); + + if self.output.has_completed_batch() { + let record_batch = self + .output + .next_completed_batch() + .expect("Failed to get output batch"); + return Poll::Ready(Some(Ok(record_batch))); + } + } + + Poll::Pending + } + /// Poll next streamed row fn poll_streamed_row(&mut self, cx: &mut Context) -> Poll>> { loop { @@ -1689,20 +1680,7 @@ impl SortMergeJoinStream { .concat_batches(&self.schema)?; (&record_batch).record_output(&self.join_metrics.baseline_metrics()); - if !(self.filter.is_some() - && matches!( - self.join_type, - JoinType::Left - | JoinType::LeftSemi - | JoinType::Right - | JoinType::RightSemi - | JoinType::LeftAnti - | JoinType::RightAnti - | JoinType::LeftMark - | JoinType::RightMark - | JoinType::Full - )) - { + if !self.needs_deferred_filtering() { // For non-filtered outer joins, we clear batches immediately after concat // since we don't need them for deferred filter processing self.staging_output_record_batches.clear_batches(); From 4cc21e84ff4cb85f7df4afdb47f4039ad40e79b0 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 1 Dec 2025 17:51:06 -0500 Subject: [PATCH 11/23] refactor --- .../physical-plan/src/joins/sort_merge_join/stream.rs | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs b/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs index de9b5144d81f..a89086b33f1e 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs @@ -893,13 +893,13 @@ impl Stream for SortMergeJoinStream { .debug_assert_metadata_aligned(); // if there is still something not processed - if !self.staging_output_record_batches.is_empty() { + return if !self.staging_output_record_batches.is_empty() { if self.needs_deferred_filtering() { let record_batch = self.filter_joined_batch()?; - return Poll::Ready(Some(Ok(record_batch))); + Poll::Ready(Some(Ok(record_batch))) } else { let record_batch = self.output_record_batch_and_reset()?; - return Poll::Ready(Some(Ok(record_batch))); + Poll::Ready(Some(Ok(record_batch))) } } else if !self.output.is_empty() { self.output @@ -909,9 +909,9 @@ impl Stream for SortMergeJoinStream { .output .next_completed_batch() .expect("Failed to get last batch"); - return Poll::Ready(Some(Ok(record_batch))); + Poll::Ready(Some(Ok(record_batch))) } else { - return Poll::Ready(None); + Poll::Ready(None) } } } From f6430db217a531bcefba952082af91df108d4b85 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 1 Dec 2025 18:14:43 -0500 Subject: [PATCH 12/23] fix double concat for filtered joins --- .../physical-plan/src/joins/sort_merge_join/stream.rs | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) diff --git a/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs b/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs index a89086b33f1e..dbaafa40da83 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs @@ -870,16 +870,13 @@ impl Stream for SortMergeJoinStream { .debug_assert_metadata_aligned(); if !self.staging_output_record_batches.is_empty() { - let record_batch = self.output_record_batch_and_reset()?; - // For non-filtered join output whenever the target output batch size - // is hit. For filtered join its needed to output on later phase - // because target output batch size can be hit in the middle of - // filtering causing the filtering to be incomplete and causing - // correctness issues + // For filtered joins, skip concat here and let Init state handle it + // to avoid double-concatenation if self.needs_deferred_filtering() { continue; } + let record_batch = self.output_record_batch_and_reset()?; return Poll::Ready(Some(Ok(record_batch))); } return Poll::Pending; From 32021cbab7c3f5f9a0a163d237d7a3eeafd03900 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 1 Dec 2025 21:23:32 -0500 Subject: [PATCH 13/23] more elided concats --- .../src/joins/sort_merge_join/stream.rs | 91 +++++++------------ 1 file changed, 35 insertions(+), 56 deletions(-) diff --git a/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs b/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs index dbaafa40da83..c443f246e3ee 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs @@ -329,7 +329,7 @@ pub(super) struct SortMergeJoinStream { /// Current state of the stream pub state: SortMergeJoinState, /// Staging output array builders - pub staging_output_record_batches: JoinedRecordBatches, + pub joined_record_batches: JoinedRecordBatches, /// Output buffer. Currently used by filtering as it requires double buffering /// to avoid small/empty batches. Non-filtered join outputs directly from `staging_output_record_batches.batches` pub output: BatchCoalescer, @@ -866,10 +866,9 @@ impl Stream for SortMergeJoinStream { self.freeze_all()?; // Verify metadata alignment before checking if we have batches to output - self.staging_output_record_batches - .debug_assert_metadata_aligned(); + self.joined_record_batches.debug_assert_metadata_aligned(); - if !self.staging_output_record_batches.is_empty() { + if !self.joined_record_batches.is_empty() { // For filtered joins, skip concat here and let Init state handle it // to avoid double-concatenation if self.needs_deferred_filtering() { @@ -886,11 +885,10 @@ impl Stream for SortMergeJoinStream { self.freeze_all()?; // Verify metadata alignment before final output - self.staging_output_record_batches - .debug_assert_metadata_aligned(); + self.joined_record_batches.debug_assert_metadata_aligned(); // if there is still something not processed - return if !self.staging_output_record_batches.is_empty() { + return if !self.joined_record_batches.is_empty() { if self.needs_deferred_filtering() { let record_batch = self.filter_joined_batch()?; Poll::Ready(Some(Ok(record_batch))) @@ -909,7 +907,7 @@ impl Stream for SortMergeJoinStream { Poll::Ready(Some(Ok(record_batch))) } else { Poll::Ready(None) - } + }; } } } @@ -962,7 +960,7 @@ impl SortMergeJoinStream { on_streamed, on_buffered, filter, - staging_output_record_batches: JoinedRecordBatches { + joined_record_batches: JoinedRecordBatches { batches: vec![], joined_batches: BatchCoalescer::new(Arc::clone(&schema), batch_size), filter_mask: BooleanBuilder::new(), @@ -1013,11 +1011,10 @@ impl SortMergeJoinStream { self.freeze_all()?; // Verify metadata alignment before checking if we have batches to filter - self.staging_output_record_batches - .debug_assert_metadata_aligned(); + self.joined_record_batches.debug_assert_metadata_aligned(); // If join is filtered and there is joined tuples waiting to be filtered - if !self.staging_output_record_batches.is_empty() { + if !self.joined_record_batches.is_empty() { // Apply filter on joined tuples and get filtered batch let out_filtered_batch = self.filter_joined_batch()?; @@ -1383,8 +1380,7 @@ impl SortMergeJoinStream { self.freeze_streamed()?; // After freezing, metadata should be aligned - self.staging_output_record_batches - .debug_assert_metadata_aligned(); + self.joined_record_batches.debug_assert_metadata_aligned(); Ok(()) } @@ -1399,8 +1395,7 @@ impl SortMergeJoinStream { self.freeze_buffered(1)?; // After freezing, metadata should be aligned - self.staging_output_record_batches - .debug_assert_metadata_aligned(); + self.joined_record_batches.debug_assert_metadata_aligned(); Ok(()) } @@ -1424,7 +1419,7 @@ impl SortMergeJoinStream { &buffered_indices, buffered_batch, )? { - self.staging_output_record_batches + self.joined_record_batches .push_batch_with_null_metadata(record_batch, self.join_type); } buffered_batch.null_joined.clear(); @@ -1457,7 +1452,7 @@ impl SortMergeJoinStream { &buffered_indices, buffered_batch, )? { - self.staging_output_record_batches + self.joined_record_batches .push_batch_with_null_metadata(record_batch, self.join_type); } buffered_batch.join_filter_not_matched_map.clear(); @@ -1610,18 +1605,17 @@ impl SortMergeJoinStream { pre_mask }; - self.staging_output_record_batches - .push_batch_with_filter_metadata( - output_batch, - &left_indices, - mask_to_use, - self.streamed_batch_counter.load(Relaxed), - self.join_type, - ); + self.joined_record_batches.push_batch_with_filter_metadata( + output_batch, + &left_indices, + mask_to_use, + self.streamed_batch_counter.load(Relaxed), + self.join_type, + ); } else { // INNER joins: filter immediately and push without metadata let filtered_batch = filter_record_batch(&output_batch, &mask)?; - self.staging_output_record_batches + self.joined_record_batches .push_batch_without_metadata(filtered_batch, self.join_type); } @@ -1653,11 +1647,11 @@ impl SortMergeJoinStream { } } } else { - self.staging_output_record_batches + self.joined_record_batches .push_batch_without_metadata(output_batch, self.join_type); } } else { - self.staging_output_record_batches + self.joined_record_batches .push_batch_without_metadata(output_batch, self.join_type); } } @@ -1667,42 +1661,27 @@ impl SortMergeJoinStream { Ok(()) } + /// Output accumulated batches and reset staging (non-filtered joins only) + /// + /// Only called when !needs_deferred_filtering(), since filtered joins handle + /// output through filter_joined_batch() instead. fn output_record_batch_and_reset(&mut self) -> Result { - // Metadata should be aligned before we concat and output - self.staging_output_record_batches - .debug_assert_metadata_aligned(); + self.joined_record_batches.debug_assert_metadata_aligned(); - let record_batch = self - .staging_output_record_batches - .concat_batches(&self.schema)?; + let record_batch = self.joined_record_batches.concat_batches(&self.schema)?; (&record_batch).record_output(&self.join_metrics.baseline_metrics()); - - if !self.needs_deferred_filtering() { - // For non-filtered outer joins, we clear batches immediately after concat - // since we don't need them for deferred filter processing - self.staging_output_record_batches.clear_batches(); - } else { - // For filtered outer joins, we keep the batches for later filter processing - // in filter_joined_batch(). Batches should still contain data. - debug_assert!( - !self.staging_output_record_batches.is_empty(), - "For filtered outer joins, batches should not be empty after concat since they're needed for filter processing" - ); - } + self.joined_record_batches.clear_batches(); Ok(record_batch) } fn filter_joined_batch(&mut self) -> Result { // Metadata should be aligned before processing - self.staging_output_record_batches - .debug_assert_metadata_aligned(); + self.joined_record_batches.debug_assert_metadata_aligned(); - let record_batch = self - .staging_output_record_batches - .concat_batches(&self.schema)?; + let record_batch = self.joined_record_batches.concat_batches(&self.schema)?; let (mut out_indices, mut out_mask, mut batch_ids) = - self.staging_output_record_batches.finish_metadata(); + self.joined_record_batches.finish_metadata(); let default_batch_ids = vec![0; record_batch.num_rows()]; // If only nulls come in and indices sizes doesn't match with expected record batch count @@ -1734,7 +1713,7 @@ impl SortMergeJoinStream { ); if out_mask.is_empty() { - self.staging_output_record_batches.clear_batches(); + self.joined_record_batches.clear_batches(); return Ok(record_batch); } @@ -1904,7 +1883,7 @@ impl SortMergeJoinStream { )?; } - self.staging_output_record_batches.clear(); + self.joined_record_batches.clear(); Ok(filtered_record_batch) } From 2e0f211f6f02d65296488cef46b8ef5667ec8f47 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 1 Dec 2025 22:13:01 -0500 Subject: [PATCH 14/23] remove dead code --- .../physical-plan/src/joins/sort_merge_join/stream.rs | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs b/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs index c443f246e3ee..d85f4eadd9e5 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs @@ -402,8 +402,8 @@ impl JoinedRecordBatches { /// Returns true if there are no batches accumulated /// /// Used in three contexts, all checking for ANY data (not optimal batch size): - /// 1. Init state: Filter accumulated data before next streamed row - /// 2. JoinOutput state: Output data after hitting batch_size threshold + /// 1. Init state: Check if filtered data needs processing before next streamed row + /// 2. JoinOutput state: Check if freeze produced any batches to output /// 3. Exhausted state: Flush remaining data /// /// For future BatchCoalescer refactor, use `joined_batches.is_empty()`. @@ -1646,9 +1646,6 @@ impl SortMergeJoinStream { ); } } - } else { - self.joined_record_batches - .push_batch_without_metadata(output_batch, self.join_type); } } else { self.joined_record_batches From 37bb875e5abb2673099036b39fc8483c97ba299c Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 2 Dec 2025 08:44:29 -0500 Subject: [PATCH 15/23] passes --- .../src/joins/sort_merge_join/stream.rs | 153 ++++++++++-------- .../src/joins/sort_merge_join/tests.rs | 33 ++-- 2 files changed, 101 insertions(+), 85 deletions(-) diff --git a/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs b/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs index d85f4eadd9e5..306d16be4619 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs @@ -387,8 +387,6 @@ pub(super) struct SortMergeJoinStream { /// - BatchCoalescer preserves row order, so metadata alignment is maintained pub(super) struct JoinedRecordBatches { /// Joined batches. Each batch is already joined columns from left and right sources - pub(super) batches: Vec, - /// UNUSED: Placeholder for future refactor. Requires proper is_empty() semantics. pub(super) joined_batches: BatchCoalescer, /// Did each output row pass the join filter? (detect if input row found any match) pub(super) filter_mask: BooleanBuilder, @@ -399,26 +397,21 @@ pub(super) struct JoinedRecordBatches { } impl JoinedRecordBatches { - /// Returns true if there are no batches accumulated - /// - /// Used in three contexts, all checking for ANY data (not optimal batch size): - /// 1. Init state: Check if filtered data needs processing before next streamed row - /// 2. JoinOutput state: Check if freeze produced any batches to output - /// 3. Exhausted state: Flush remaining data - /// - /// For future BatchCoalescer refactor, use `joined_batches.is_empty()`. - #[inline] - fn is_empty(&self) -> bool { - self.batches.is_empty() - } - /// Concatenates all accumulated batches into a single RecordBatch /// - /// For future BatchCoalescer refactor: - /// - Call `finish_buffered_batch()` to finalize partial data - /// - Drain with `next_completed_batch()` in a loop - fn concat_batches(&self, schema: &SchemaRef) -> Result { - Ok(concat_batches(schema, &self.batches)?) + /// Must drain ALL batches from BatchCoalescer for filtered joins to ensure + /// metadata alignment when applying get_corrected_filter_mask(). + pub(super) fn concat_batches(&mut self, schema: &SchemaRef) -> Result { + // Finish any buffered data to create final completed batch + self.joined_batches.finish_buffered_batch()?; + + // Collect all completed batches + let mut all_batches = vec![]; + while let Some(batch) = self.joined_batches.next_completed_batch() { + all_batches.push(batch); + } + + Ok(concat_batches(schema, &all_batches)?) } /// Finishes and returns the metadata arrays, clearing the builders @@ -431,9 +424,10 @@ impl JoinedRecordBatches { (row_indices, filter_mask, &self.batch_ids) } - /// Clears only the batches vector (used in non-filtered path) - fn clear_batches(&mut self) { - self.batches.clear(); + /// Clears only the batches (used in non-filtered path) + fn clear_batches(&mut self, schema: &SchemaRef, batch_size: usize) { + // Replace with a new empty BatchCoalescer + self.joined_batches = BatchCoalescer::new(Arc::clone(schema), batch_size); } /// Asserts that internal metadata arrays are consistent with each other @@ -462,7 +456,7 @@ impl JoinedRecordBatches { /// Asserts that if batches is empty, metadata is also empty #[inline] fn debug_assert_empty_consistency(&self) { - if self.batches.is_empty() { + if self.joined_batches.is_empty() { debug_assert_eq!( self.filter_mask.len(), 0, @@ -503,7 +497,9 @@ impl JoinedRecordBatches { ); self.debug_assert_metadata_aligned(); - self.batches.push(batch); + self.joined_batches + .push_batch(batch) + .expect("Failed to push batch to BatchCoalescer"); } /// Pushes a batch with filter metadata (filtered outer/semi/anti/mark joins) @@ -552,7 +548,9 @@ impl JoinedRecordBatches { .resize(self.batch_ids.len() + row_indices.len(), streamed_batch_id); self.debug_assert_metadata_aligned(); - self.batches.push(batch); + self.joined_batches + .push_batch(batch) + .expect("Failed to push batch to BatchCoalescer"); } /// Pushes a batch without metadata (non-filtered joins) @@ -561,16 +559,20 @@ impl JoinedRecordBatches { /// or null-joined rows are handled separately. No need to track which input /// row produced which output row. fn push_batch_without_metadata(&mut self, batch: RecordBatch, _join_type: JoinType) { - self.batches.push(batch); + self.joined_batches + .push_batch(batch) + .expect("Failed to push batch to BatchCoalescer"); } - fn clear(&mut self) { + fn clear(&mut self, schema: &SchemaRef, batch_size: usize) { // Note: clear() can be called when batches still contains data! // This happens in filter_joined_batch() after concat_batches() has read // the batches but before they're removed. The batches have been processed // into output, so clearing them here is the final cleanup step. - self.batches.clear(); + // Replace with a new empty BatchCoalescer + self.joined_batches = BatchCoalescer::new(Arc::clone(schema), batch_size); + self.batch_ids.clear(); self.filter_mask = BooleanBuilder::new(); self.row_indices = UInt64Builder::new(); @@ -868,17 +870,27 @@ impl Stream for SortMergeJoinStream { // Verify metadata alignment before checking if we have batches to output self.joined_record_batches.debug_assert_metadata_aligned(); - if !self.joined_record_batches.is_empty() { - // For filtered joins, skip concat here and let Init state handle it - // to avoid double-concatenation - if self.needs_deferred_filtering() { - continue; - } + // For filtered joins, skip output and let Init state handle it + if self.needs_deferred_filtering() { + continue; + } - let record_batch = self.output_record_batch_and_reset()?; + // For non-filtered joins, only output if we have a completed batch + // (opportunistic output when target batch size is reached) + if self + .joined_record_batches + .joined_batches + .has_completed_batch() + { + let record_batch = self + .joined_record_batches + .joined_batches + .next_completed_batch() + .expect("has_completed_batch was true"); + (&record_batch).record_output(&self.join_metrics.baseline_metrics()); return Poll::Ready(Some(Ok(record_batch))); } - return Poll::Pending; + // Otherwise keep buffering (don't output yet) } } SortMergeJoinState::Exhausted => { @@ -887,19 +899,37 @@ impl Stream for SortMergeJoinStream { // Verify metadata alignment before final output self.joined_record_batches.debug_assert_metadata_aligned(); - // if there is still something not processed - return if !self.joined_record_batches.is_empty() { - if self.needs_deferred_filtering() { - let record_batch = self.filter_joined_batch()?; - Poll::Ready(Some(Ok(record_batch))) - } else { - let record_batch = self.output_record_batch_and_reset()?; - Poll::Ready(Some(Ok(record_batch))) - } - } else if !self.output.is_empty() { - self.output - .finish_buffered_batch() - .expect("Failed to finish last batch"); + // For filtered joins, must concat and filter ALL data at once + if self.needs_deferred_filtering() + && !self.joined_record_batches.joined_batches.is_empty() + { + let record_batch = self.filter_joined_batch()?; + return Poll::Ready(Some(Ok(record_batch))); + } + + // For non-filtered joins, finish buffered data first + if !self.joined_record_batches.joined_batches.is_empty() { + self.joined_record_batches.joined_batches.finish_buffered_batch()?; + } + + // Output one completed batch at a time (stay in Exhausted until empty) + if self + .joined_record_batches + .joined_batches + .has_completed_batch() + { + let record_batch = self + .joined_record_batches + .joined_batches + .next_completed_batch() + .expect("has_completed_batch was true"); + (&record_batch).record_output(&self.join_metrics.baseline_metrics()); + return Poll::Ready(Some(Ok(record_batch))); + } + + // Finally check self.output BatchCoalescer (used by filtered joins) + return if !self.output.is_empty() { + self.output.finish_buffered_batch()?; let record_batch = self .output .next_completed_batch() @@ -961,7 +991,6 @@ impl SortMergeJoinStream { on_buffered, filter, joined_record_batches: JoinedRecordBatches { - batches: vec![], joined_batches: BatchCoalescer::new(Arc::clone(&schema), batch_size), filter_mask: BooleanBuilder::new(), row_indices: UInt64Builder::new(), @@ -1014,7 +1043,7 @@ impl SortMergeJoinStream { self.joined_record_batches.debug_assert_metadata_aligned(); // If join is filtered and there is joined tuples waiting to be filtered - if !self.joined_record_batches.is_empty() { + if !self.joined_record_batches.joined_batches.is_empty() { // Apply filter on joined tuples and get filtered batch let out_filtered_batch = self.filter_joined_batch()?; @@ -1658,20 +1687,6 @@ impl SortMergeJoinStream { Ok(()) } - /// Output accumulated batches and reset staging (non-filtered joins only) - /// - /// Only called when !needs_deferred_filtering(), since filtered joins handle - /// output through filter_joined_batch() instead. - fn output_record_batch_and_reset(&mut self) -> Result { - self.joined_record_batches.debug_assert_metadata_aligned(); - - let record_batch = self.joined_record_batches.concat_batches(&self.schema)?; - (&record_batch).record_output(&self.join_metrics.baseline_metrics()); - self.joined_record_batches.clear_batches(); - - Ok(record_batch) - } - fn filter_joined_batch(&mut self) -> Result { // Metadata should be aligned before processing self.joined_record_batches.debug_assert_metadata_aligned(); @@ -1710,7 +1725,8 @@ impl SortMergeJoinStream { ); if out_mask.is_empty() { - self.joined_record_batches.clear_batches(); + self.joined_record_batches + .clear_batches(&self.schema, self.batch_size); return Ok(record_batch); } @@ -1880,7 +1896,8 @@ impl SortMergeJoinStream { )?; } - self.joined_record_batches.clear(); + self.joined_record_batches + .clear(&self.schema, self.batch_size); Ok(filtered_record_batch) } diff --git a/datafusion/physical-plan/src/joins/sort_merge_join/tests.rs b/datafusion/physical-plan/src/joins/sort_merge_join/tests.rs index f58266bdd3ae..f13ac932e810 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join/tests.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join/tests.rs @@ -2319,7 +2319,6 @@ fn build_joined_record_batches() -> Result { ])); let mut batches = JoinedRecordBatches { - batches: vec![], joined_batches: BatchCoalescer::new(Arc::clone(&schema), 8192), filter_mask: BooleanBuilder::new(), row_indices: UInt64Builder::new(), @@ -2327,7 +2326,7 @@ fn build_joined_record_batches() -> Result { }; // Insert already prejoined non-filtered rows - batches.batches.push(RecordBatch::try_new( + batches.joined_batches.push_batch(RecordBatch::try_new( Arc::clone(&schema), vec![ Arc::new(Int32Array::from(vec![1, 1])), @@ -2335,9 +2334,9 @@ fn build_joined_record_batches() -> Result { Arc::new(Int32Array::from(vec![1, 1])), Arc::new(Int32Array::from(vec![11, 9])), ], - )?); + )?)?; - batches.batches.push(RecordBatch::try_new( + batches.joined_batches.push_batch(RecordBatch::try_new( Arc::clone(&schema), vec![ Arc::new(Int32Array::from(vec![1])), @@ -2345,9 +2344,9 @@ fn build_joined_record_batches() -> Result { Arc::new(Int32Array::from(vec![1])), Arc::new(Int32Array::from(vec![12])), ], - )?); + )?)?; - batches.batches.push(RecordBatch::try_new( + batches.joined_batches.push_batch(RecordBatch::try_new( Arc::clone(&schema), vec![ Arc::new(Int32Array::from(vec![1, 1])), @@ -2355,9 +2354,9 @@ fn build_joined_record_batches() -> Result { Arc::new(Int32Array::from(vec![1, 1])), Arc::new(Int32Array::from(vec![11, 13])), ], - )?); + )?)?; - batches.batches.push(RecordBatch::try_new( + batches.joined_batches.push_batch(RecordBatch::try_new( Arc::clone(&schema), vec![ Arc::new(Int32Array::from(vec![1])), @@ -2365,9 +2364,9 @@ fn build_joined_record_batches() -> Result { Arc::new(Int32Array::from(vec![1])), Arc::new(Int32Array::from(vec![12])), ], - )?); + )?)?; - batches.batches.push(RecordBatch::try_new( + batches.joined_batches.push_batch(RecordBatch::try_new( Arc::clone(&schema), vec![ Arc::new(Int32Array::from(vec![1, 1])), @@ -2375,7 +2374,7 @@ fn build_joined_record_batches() -> Result { Arc::new(Int32Array::from(vec![1, 1])), Arc::new(Int32Array::from(vec![12, 11])), ], - )?); + )?)?; let streamed_indices = vec![0, 0]; batches.batch_ids.extend(vec![0; streamed_indices.len()]); @@ -2425,9 +2424,9 @@ fn build_joined_record_batches() -> Result { #[tokio::test] async fn test_left_outer_join_filtered_mask() -> Result<()> { let mut joined_batches = build_joined_record_batches()?; - let schema = joined_batches.batches.first().unwrap().schema(); + let schema = joined_batches.joined_batches.schema(); - let output = concat_batches(&schema, &joined_batches.batches)?; + let output = joined_batches.concat_batches(&schema)?; let out_mask = joined_batches.filter_mask.finish(); let out_indices = joined_batches.row_indices.finish(); @@ -2632,9 +2631,9 @@ async fn test_left_outer_join_filtered_mask() -> Result<()> { async fn test_semi_join_filtered_mask() -> Result<()> { for join_type in [LeftSemi, RightSemi] { let mut joined_batches = build_joined_record_batches()?; - let schema = joined_batches.batches.first().unwrap().schema(); + let schema = joined_batches.joined_batches.schema(); - let output = concat_batches(&schema, &joined_batches.batches)?; + let output = joined_batches.concat_batches(&schema)?; let out_mask = joined_batches.filter_mask.finish(); let out_indices = joined_batches.row_indices.finish(); @@ -2807,9 +2806,9 @@ async fn test_semi_join_filtered_mask() -> Result<()> { async fn test_anti_join_filtered_mask() -> Result<()> { for join_type in [LeftAnti, RightAnti] { let mut joined_batches = build_joined_record_batches()?; - let schema = joined_batches.batches.first().unwrap().schema(); + let schema = joined_batches.joined_batches.schema(); - let output = concat_batches(&schema, &joined_batches.batches)?; + let output = joined_batches.concat_batches(&schema)?; let out_mask = joined_batches.filter_mask.finish(); let out_indices = joined_batches.row_indices.finish(); From 8c6905640adeba9112a3ccbc8bc872e222a6f74d Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 2 Dec 2025 08:57:14 -0500 Subject: [PATCH 16/23] comments --- .../src/joins/sort_merge_join/stream.rs | 51 ++++--------------- 1 file changed, 10 insertions(+), 41 deletions(-) diff --git a/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs b/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs index b551f681db4f..d62ed3affc34 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs @@ -357,34 +357,6 @@ pub(super) struct SortMergeJoinStream { /// Accumulates joined rows until either: /// - Target batch size reached (for efficiency) /// - Stream exhausted (flush remaining data) -/// -/// # Metadata Tracking (Critical for Filtered Joins) -/// -/// Outer joins must emit at least one row per input row. When a join filter rejects -/// all matches for an input row, we must emit a null-joined row. This requires grouping -/// output rows by input row to check if ANY match passed the filter. -/// -/// Metadata fields (one entry per output row): -/// - `row_indices[i]`: Which input row within its batch (for grouping output rows) -/// - `filter_mask[i]`: Did this output row pass the join filter? (detect if input row matched) -/// - `batch_ids[i]`: Which input batch (disambiguates row_indices across batches) -/// -/// Invariant: `metadata.len() == sum(batch.num_rows())` for all batches -/// -/// # Usage Pattern: `is_empty()` Always Means "Any Data At All" -/// -/// Called in three contexts, all checking for ANY remaining data (not optimal batch): -/// 1. Init state: Filter accumulated data before next streamed row -/// 2. JoinOutput state: Output accumulated data after hitting batch_size -/// 3. Exhausted state: Flush any remaining data -/// -/// # Future Refactor: Using BatchCoalescer -/// -/// To replace `batches: Vec` with `BatchCoalescer`: -/// - `is_empty()` → Must check BOTH buffered data AND completed batches -/// - `concat_batches()` → Drain with `finish_buffered_batch()` + `next_completed_batch()` -/// - Previous refactor failed: only checked `has_completed_batch()`, missing buffered data -/// - BatchCoalescer preserves row order, so metadata alignment is maintained pub(super) struct JoinedRecordBatches { /// Joined batches. Each batch is already joined columns from left and right sources pub(super) joined_batches: BatchCoalescer, @@ -402,10 +374,8 @@ impl JoinedRecordBatches { /// Must drain ALL batches from BatchCoalescer for filtered joins to ensure /// metadata alignment when applying get_corrected_filter_mask(). pub(super) fn concat_batches(&mut self, schema: &SchemaRef) -> Result { - // Finish any buffered data to create final completed batch self.joined_batches.finish_buffered_batch()?; - // Collect all completed batches let mut all_batches = vec![]; while let Some(batch) = self.joined_batches.next_completed_batch() { all_batches.push(batch); @@ -505,10 +475,10 @@ impl JoinedRecordBatches { /// Pushes a batch with filter metadata (filtered outer/semi/anti/mark joins) /// /// Deferred filtering: An input row may join with multiple buffered rows, but we - /// don't know yet if ALL matches failed the filter. We track metadata so + /// don't know yet if all matches failed the filter. We track metadata so /// `get_corrected_filter_mask()` can later group by input row and decide: - /// - If ANY match passed: emit passing rows - /// - If ALL matches failed: emit null-joined row + /// - If any match passed: emit passing rows + /// - If all matches failed: emit null-joined row /// /// Maintains invariant: N rows → N metadata entries fn push_batch_with_filter_metadata( @@ -565,11 +535,6 @@ impl JoinedRecordBatches { } fn clear(&mut self, schema: &SchemaRef, batch_size: usize) { - // Note: clear() can be called when batches still contains data! - // This happens in filter_joined_batch() after concat_batches() has read - // the batches but before they're removed. The batches have been processed - // into output, so clearing them here is the final cleanup step. - // Replace with a new empty BatchCoalescer self.joined_batches = BatchCoalescer::new(Arc::clone(schema), batch_size); @@ -887,7 +852,8 @@ impl Stream for SortMergeJoinStream { .joined_batches .next_completed_batch() .expect("has_completed_batch was true"); - (&record_batch).record_output(&self.join_metrics.baseline_metrics()); + (&record_batch) + .record_output(&self.join_metrics.baseline_metrics()); return Poll::Ready(Some(Ok(record_batch))); } // Otherwise keep buffering (don't output yet) @@ -909,7 +875,9 @@ impl Stream for SortMergeJoinStream { // For non-filtered joins, finish buffered data first if !self.joined_record_batches.joined_batches.is_empty() { - self.joined_record_batches.joined_batches.finish_buffered_batch()?; + self.joined_record_batches + .joined_batches + .finish_buffered_batch()?; } // Output one completed batch at a time (stay in Exhausted until empty) @@ -923,7 +891,8 @@ impl Stream for SortMergeJoinStream { .joined_batches .next_completed_batch() .expect("has_completed_batch was true"); - (&record_batch).record_output(&self.join_metrics.baseline_metrics()); + (&record_batch) + .record_output(&self.join_metrics.baseline_metrics()); return Poll::Ready(Some(Ok(record_batch))); } From 67877e6fd11465b74cba48cdac19c5f7225718af Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 2 Dec 2025 09:04:24 -0500 Subject: [PATCH 17/23] clippy, comments --- .../src/joins/sort_merge_join/stream.rs | 16 ++-------------- 1 file changed, 2 insertions(+), 14 deletions(-) diff --git a/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs b/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs index d62ed3affc34..9d1dbd796d82 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs @@ -394,9 +394,8 @@ impl JoinedRecordBatches { (row_indices, filter_mask, &self.batch_ids) } - /// Clears only the batches (used in non-filtered path) + /// Clears batches without touching metadata (for early return when no filtering needed) fn clear_batches(&mut self, schema: &SchemaRef, batch_size: usize) { - // Replace with a new empty BatchCoalescer self.joined_batches = BatchCoalescer::new(Arc::clone(schema), batch_size); } @@ -535,14 +534,10 @@ impl JoinedRecordBatches { } fn clear(&mut self, schema: &SchemaRef, batch_size: usize) { - // Replace with a new empty BatchCoalescer self.joined_batches = BatchCoalescer::new(Arc::clone(schema), batch_size); - self.batch_ids.clear(); self.filter_mask = BooleanBuilder::new(); self.row_indices = UInt64Builder::new(); - - // After clear, everything should be empty self.debug_assert_empty_consistency(); } } @@ -575,9 +570,7 @@ fn last_index_for_row( ); debug_assert!( row_index < indices_len, - "row_index {} should be < indices_len {}", - row_index, - indices_len + "row_index {row_index} should be < indices_len {indices_len}", ); row_index == indices_len - 1 @@ -1008,15 +1001,10 @@ impl SortMergeJoinStream { fn process_filtered_batches(&mut self) -> Poll>> { self.freeze_all()?; - // Verify metadata alignment before checking if we have batches to filter self.joined_record_batches.debug_assert_metadata_aligned(); - // If join is filtered and there is joined tuples waiting to be filtered if !self.joined_record_batches.joined_batches.is_empty() { - // Apply filter on joined tuples and get filtered batch let out_filtered_batch = self.filter_joined_batch()?; - - // Append filtered batch to the output buffer self.output .push_batch(out_filtered_batch) .expect("Failed to push output batch"); From e7b94e56095cf216d43fe21284ea70000eae8a7c Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 2 Dec 2025 09:15:56 -0500 Subject: [PATCH 18/23] Remove unused import --- datafusion/physical-plan/src/joins/sort_merge_join/tests.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/physical-plan/src/joins/sort_merge_join/tests.rs b/datafusion/physical-plan/src/joins/sort_merge_join/tests.rs index f13ac932e810..c3e9a1d16cb1 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join/tests.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join/tests.rs @@ -31,7 +31,7 @@ use arrow::array::{ BinaryArray, BooleanArray, Date32Array, Date64Array, FixedSizeBinaryArray, Int32Array, RecordBatch, UInt64Array, }; -use arrow::compute::{concat_batches, filter_record_batch, BatchCoalescer, SortOptions}; +use arrow::compute::{filter_record_batch, BatchCoalescer, SortOptions}; use arrow::datatypes::{DataType, Field, Schema}; use datafusion_common::JoinType::*; From 7c55ad9842d71c1adb9543e8ee885cce54f0e384 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 2 Dec 2025 10:18:25 -0500 Subject: [PATCH 19/23] optimize concat_batches call --- .../physical-plan/src/joins/sort_merge_join/stream.rs | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs b/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs index 9d1dbd796d82..7f45b3f964d8 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs @@ -381,7 +381,11 @@ impl JoinedRecordBatches { all_batches.push(batch); } - Ok(concat_batches(schema, &all_batches)?) + match all_batches.len() { + 0 => unreachable!("concat_batches called with empty BatchCoalescer"), + 1 => Ok(all_batches.pop().unwrap()), + _ => Ok(concat_batches(schema, &all_batches)?), + } } /// Finishes and returns the metadata arrays, clearing the builders From 43a945ffabdeba66ce81151c1b5cfef056f514b5 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 2 Dec 2025 11:51:15 -0500 Subject: [PATCH 20/23] fix metrics collection filtered joins --- datafusion/physical-plan/src/joins/sort_merge_join/stream.rs | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs b/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs index 7f45b3f964d8..431e2bc68035 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs @@ -867,6 +867,8 @@ impl Stream for SortMergeJoinStream { && !self.joined_record_batches.joined_batches.is_empty() { let record_batch = self.filter_joined_batch()?; + (&record_batch) + .record_output(&self.join_metrics.baseline_metrics()); return Poll::Ready(Some(Ok(record_batch))); } @@ -900,6 +902,8 @@ impl Stream for SortMergeJoinStream { .output .next_completed_batch() .expect("Failed to get last batch"); + (&record_batch) + .record_output(&self.join_metrics.baseline_metrics()); Poll::Ready(Some(Ok(record_batch))) } else { Poll::Ready(None) @@ -1018,6 +1022,7 @@ impl SortMergeJoinStream { .output .next_completed_batch() .expect("Failed to get output batch"); + (&record_batch).record_output(&self.join_metrics.baseline_metrics()); return Poll::Ready(Some(Ok(record_batch))); } } From 6a4e6646d15245d2d6f24d6c67df3e8158124037 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 2 Dec 2025 19:32:32 -0500 Subject: [PATCH 21/23] pass through batches that are batch_size / 2 similar to LimitedBatchCoalescer --- .../src/joins/sort_merge_join/stream.rs | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs b/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs index 431e2bc68035..d4ce25c86f72 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs @@ -400,7 +400,8 @@ impl JoinedRecordBatches { /// Clears batches without touching metadata (for early return when no filtering needed) fn clear_batches(&mut self, schema: &SchemaRef, batch_size: usize) { - self.joined_batches = BatchCoalescer::new(Arc::clone(schema), batch_size); + self.joined_batches = BatchCoalescer::new(Arc::clone(schema), batch_size) + .with_biggest_coalesce_batch_size(Option::from(batch_size / 2)); } /// Asserts that internal metadata arrays are consistent with each other @@ -538,7 +539,8 @@ impl JoinedRecordBatches { } fn clear(&mut self, schema: &SchemaRef, batch_size: usize) { - self.joined_batches = BatchCoalescer::new(Arc::clone(schema), batch_size); + self.joined_batches = BatchCoalescer::new(Arc::clone(schema), batch_size) + .with_biggest_coalesce_batch_size(Option::from(batch_size / 2)); self.batch_ids.clear(); self.filter_mask = BooleanBuilder::new(); self.row_indices = UInt64Builder::new(); @@ -961,12 +963,14 @@ impl SortMergeJoinStream { on_buffered, filter, joined_record_batches: JoinedRecordBatches { - joined_batches: BatchCoalescer::new(Arc::clone(&schema), batch_size), + joined_batches: BatchCoalescer::new(Arc::clone(&schema), batch_size) + .with_biggest_coalesce_batch_size(Option::from(batch_size / 2)), filter_mask: BooleanBuilder::new(), row_indices: UInt64Builder::new(), batch_ids: vec![], }, - output: BatchCoalescer::new(schema, batch_size), + output: BatchCoalescer::new(schema, batch_size) + .with_biggest_coalesce_batch_size(Option::from(batch_size / 2)), batch_size, join_type, join_metrics, From 66ea02717b004f9184cc2ec25f85f2247f36e7d4 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Thu, 4 Dec 2025 10:20:59 -0500 Subject: [PATCH 22/23] Address PR feedback. --- .../physical-plan/src/joins/sort_merge_join/stream.rs | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs b/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs index a71bb5535931..3cc8985ed0bf 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs @@ -40,6 +40,7 @@ use crate::spill::spill_manager::SpillManager; use crate::{PhysicalExpr, RecordBatchStream, SendableRecordBatchStream}; use arrow::array::{types::UInt64Type, *}; +use arrow::compute::kernels::numeric::mul; use arrow::compute::{ self, concat_batches, filter_record_batch, is_not_null, take, BatchCoalescer, SortOptions, @@ -381,10 +382,10 @@ impl JoinedRecordBatches { all_batches.push(batch); } - match all_batches.len() { - 0 => unreachable!("concat_batches called with empty BatchCoalescer"), - 1 => Ok(all_batches.pop().unwrap()), - _ => Ok(concat_batches(schema, &all_batches)?), + match all_batches.as_slice() { + [] => unreachable!("concat_batches called with empty BatchCoalescer"), + [single_batch] => Ok(single_batch.clone()), + multiple_batches => Ok(concat_batches(schema, multiple_batches)?), } } From 86cbc5ca1f611895d17d8abfa98297f458c66fee Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Thu, 4 Dec 2025 10:34:45 -0500 Subject: [PATCH 23/23] Remove stray import. --- datafusion/physical-plan/src/joins/sort_merge_join/stream.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs b/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs index 3cc8985ed0bf..b94dfd6243a3 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join/stream.rs @@ -40,7 +40,6 @@ use crate::spill::spill_manager::SpillManager; use crate::{PhysicalExpr, RecordBatchStream, SendableRecordBatchStream}; use arrow::array::{types::UInt64Type, *}; -use arrow::compute::kernels::numeric::mul; use arrow::compute::{ self, concat_batches, filter_record_batch, is_not_null, take, BatchCoalescer, SortOptions,