From 49267992431b4383abdbc9c72a72b8a8f675979b Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Thu, 17 Oct 2024 20:51:50 +0800 Subject: [PATCH 01/47] first draft Signed-off-by: jayzhan211 --- benchmarks/bench.sh | 10 + benchmarks/queries/spm/queries.sql | 1 + benchmarks/src/bin/dfbench.rs | 4 +- benchmarks/src/lib.rs | 1 + benchmarks/src/spm.rs | 165 +++++++ datafusion/execution/src/memory_pool/mod.rs | 2 + datafusion/physical-plan/src/sorts/cursor.rs | 54 +++ datafusion/physical-plan/src/sorts/merge.rs | 49 +- .../sort_preserving_merge_repartition.slt | 233 ++++++++++ .../sort_preserving_merge_union.slt | 435 ++++++++++++++++++ 10 files changed, 951 insertions(+), 3 deletions(-) create mode 100644 benchmarks/queries/spm/queries.sql create mode 100644 benchmarks/src/spm.rs create mode 100644 datafusion/sqllogictest/test_files/sort_preserving_merge_repartition.slt create mode 100644 datafusion/sqllogictest/test_files/sort_preserving_merge_union.slt diff --git a/benchmarks/bench.sh b/benchmarks/bench.sh index 70faa9ef2b73..e0135ec9bce7 100755 --- a/benchmarks/bench.sh +++ b/benchmarks/bench.sh @@ -231,6 +231,9 @@ main() { sort) run_sort ;; + spm) + run_spm + ;; clickbench_1) run_clickbench_1 ;; @@ -422,6 +425,13 @@ run_clickbench_1() { $CARGO_COMMAND --bin dfbench -- clickbench --iterations 5 --path "${DATA_DIR}/hits.parquet" --queries-path "${SCRIPT_DIR}/queries/clickbench/queries.sql" -o "${RESULTS_FILE}" } +run_spm() { + RESULTS_FILE="${RESULTS_DIR}/spm.json" + echo "RESULTS_FILE: ${RESULTS_FILE}" + echo "Running spm benchmark..." + $CARGO_COMMAND --bin dfbench -- spm --iterations 5 --path "${DATA_DIR}/low_card_spm_small_batch.csv" --queries-path "${SCRIPT_DIR}/queries/spm/queries.sql" -o "${RESULTS_FILE}" +} + # Runs the clickbench benchmark with the partitioned parquet files run_clickbench_partitioned() { RESULTS_FILE="${RESULTS_DIR}/clickbench_partitioned.json" diff --git a/benchmarks/queries/spm/queries.sql b/benchmarks/queries/spm/queries.sql new file mode 100644 index 000000000000..c689a66bb441 --- /dev/null +++ b/benchmarks/queries/spm/queries.sql @@ -0,0 +1 @@ +SELECT * FROM spm_table WHERE a=0 ORDER BY b, c; \ No newline at end of file diff --git a/benchmarks/src/bin/dfbench.rs b/benchmarks/src/bin/dfbench.rs index f7b84116e793..0fd749b45d89 100644 --- a/benchmarks/src/bin/dfbench.rs +++ b/benchmarks/src/bin/dfbench.rs @@ -33,7 +33,7 @@ static ALLOC: snmalloc_rs::SnMalloc = snmalloc_rs::SnMalloc; #[global_allocator] static ALLOC: mimalloc::MiMalloc = mimalloc::MiMalloc; -use datafusion_benchmarks::{clickbench, imdb, parquet_filter, sort, tpch}; +use datafusion_benchmarks::{clickbench, imdb, parquet_filter, sort, spm, tpch}; #[derive(Debug, StructOpt)] #[structopt(about = "benchmark command")] @@ -41,6 +41,7 @@ enum Options { Tpch(tpch::RunOpt), TpchConvert(tpch::ConvertOpt), Clickbench(clickbench::RunOpt), + Spm(spm::RunOpt), ParquetFilter(parquet_filter::RunOpt), Sort(sort::RunOpt), Imdb(imdb::RunOpt), @@ -55,6 +56,7 @@ pub async fn main() -> Result<()> { Options::Tpch(opt) => opt.run().await, Options::TpchConvert(opt) => opt.run().await, Options::Clickbench(opt) => opt.run().await, + Options::Spm(opt) => opt.run().await, Options::ParquetFilter(opt) => opt.run().await, Options::Sort(opt) => opt.run().await, Options::Imdb(opt) => opt.run().await, diff --git a/benchmarks/src/lib.rs b/benchmarks/src/lib.rs index 52d81ca91816..2123187bb2b1 100644 --- a/benchmarks/src/lib.rs +++ b/benchmarks/src/lib.rs @@ -20,6 +20,7 @@ pub mod clickbench; pub mod imdb; pub mod parquet_filter; pub mod sort; +pub mod spm; pub mod tpch; mod util; pub use util::*; diff --git a/benchmarks/src/spm.rs b/benchmarks/src/spm.rs new file mode 100644 index 000000000000..3ec0ff460d01 --- /dev/null +++ b/benchmarks/src/spm.rs @@ -0,0 +1,165 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use std::path::Path; +use std::path::PathBuf; + +use datafusion::{ + error::{DataFusionError, Result}, + prelude::SessionContext, +}; +use datafusion_common::exec_datafusion_err; +use datafusion_common::instant::Instant; +use structopt::StructOpt; + +use crate::{BenchmarkRun, CommonOpt}; + +/// Run the clickbench benchmark +/// +/// The ClickBench[1] benchmarks are widely cited in the industry and +/// focus on grouping / aggregation / filtering. This runner uses the +/// scripts and queries from [2]. +/// +/// [1]: https://github.com/ClickHouse/ClickBench +/// [2]: https://github.com/ClickHouse/ClickBench/tree/main/datafusion +#[derive(Debug, StructOpt, Clone)] +#[structopt(verbatim_doc_comment)] +pub struct RunOpt { + /// Query number (between 0 and 42). If not specified, runs all queries + #[structopt(short, long)] + query: Option, + + /// Common options + #[structopt(flatten)] + common: CommonOpt, + + /// Path to hits.parquet (single file) or `hits_partitioned` + /// (partitioned, 100 files) + #[structopt( + parse(from_os_str), + short = "p", + long = "path", + default_value = "benchmarks/data/low_card_spm.csv" + )] + path: PathBuf, + + /// Path to queries.sql (single file) + #[structopt( + parse(from_os_str), + short = "r", + long = "queries-path", + default_value = "benchmarks/queries/clickbench/queries.sql" + )] + queries_path: PathBuf, + + /// If present, write results json here + #[structopt(parse(from_os_str), short = "o", long = "output")] + output_path: Option, +} + +struct AllQueries { + queries: Vec, +} + +impl AllQueries { + fn try_new(path: &Path) -> Result { + // ClickBench has all queries in a single file identified by line number + let all_queries = std::fs::read_to_string(path) + .map_err(|e| exec_datafusion_err!("Could not open {path:?}: {e}"))?; + Ok(Self { + queries: all_queries.lines().map(|s| s.to_string()).collect(), + }) + } + + /// Returns the text of query `query_id` + fn get_query(&self, query_id: usize) -> Result<&str> { + self.queries + .get(query_id) + .ok_or_else(|| { + let min_id = self.min_query_id(); + let max_id = self.max_query_id(); + exec_datafusion_err!( + "Invalid query id {query_id}. Must be between {min_id} and {max_id}" + ) + }) + .map(|s| s.as_str()) + } + + fn min_query_id(&self) -> usize { + 0 + } + + fn max_query_id(&self) -> usize { + self.queries.len() - 1 + } +} +impl RunOpt { + pub async fn run(self) -> Result<()> { + println!("Running benchmarks with the following options: {self:?}"); + let queries = AllQueries::try_new(self.queries_path.as_path())?; + let query_range = match self.query { + Some(query_id) => query_id..=query_id, + None => queries.min_query_id()..=queries.max_query_id(), + }; + + let mut config = self.common.config(); + config + .options_mut() + .execution + .parquet + .schema_force_view_types = self.common.force_view_types; + + let ctx = SessionContext::new_with_config(config); + self.register_spm(&ctx).await?; + + let iterations = self.common.iterations; + let mut benchmark_run = BenchmarkRun::new(); + for query_id in query_range { + benchmark_run.start_new_case(&format!("Query {query_id}")); + let sql = queries.get_query(query_id)?; + println!("Q{query_id}: {sql}"); + + for i in 0..iterations { + let start = Instant::now(); + let results = ctx.sql(sql).await?.collect().await?; + let elapsed = start.elapsed(); + let ms = elapsed.as_secs_f64() * 1000.0; + let row_count: usize = results.iter().map(|b| b.num_rows()).sum(); + println!( + "Query {query_id} iteration {i} took {ms:.1} ms and returned {row_count} rows" + ); + benchmark_run.write_iter(elapsed, row_count); + } + } + benchmark_run.maybe_write_json(self.output_path.as_ref())?; + Ok(()) + } + + /// Registrs the `hits.parquet` as a table named `hits` + async fn register_spm(&self, ctx: &SessionContext) -> Result<()> { + let options = Default::default(); + let path = self.path.as_os_str().to_str().unwrap(); + ctx.register_csv("spm_table", path, options) + .await + .map_err(|e| { + DataFusionError::Context( + format!("Registering 'spm' as {path}"), + Box::new(e), + ) + }) + } +} diff --git a/datafusion/execution/src/memory_pool/mod.rs b/datafusion/execution/src/memory_pool/mod.rs index d87ce1ebfed7..6b51343f76b9 100644 --- a/datafusion/execution/src/memory_pool/mod.rs +++ b/datafusion/execution/src/memory_pool/mod.rs @@ -292,6 +292,8 @@ impl MemoryReservation { pub fn try_grow(&mut self, capacity: usize) -> Result<()> { self.registration.pool.try_grow(self, capacity)?; self.size += capacity; + // TODO: Remove. For measuring the memory between the change + // println!("self.size: {:?}", self.size); Ok(()) } diff --git a/datafusion/physical-plan/src/sorts/cursor.rs b/datafusion/physical-plan/src/sorts/cursor.rs index df90c97faf68..24b6dc92cbb7 100644 --- a/datafusion/physical-plan/src/sorts/cursor.rs +++ b/datafusion/physical-plan/src/sorts/cursor.rs @@ -38,6 +38,10 @@ pub trait CursorValues { /// Returns true if `l[l_idx] == r[r_idx]` fn eq(l: &Self, l_idx: usize, r: &Self, r_idx: usize) -> bool; + /// Returns true if `row[idx] == row[idx - 1]` + /// idx should be greater than 0 + fn eq_to_previous(cursor: &Self, idx: usize) -> bool; + /// Returns comparison of `l[l_idx]` and `r[r_idx]` fn compare(l: &Self, l_idx: usize, r: &Self, r_idx: usize) -> Ordering; } @@ -95,6 +99,16 @@ impl Cursor { self.offset += 1; t } + + pub fn is_eq_to_prev_one(&self, prev_cursor: Option<&Cursor>) -> bool { + if self.offset > 0 { + self.is_eq_to_prev_row() + } else if let Some(prev_cursor) = prev_cursor { + self.is_eq_to_prev_row_in_prev_batch(prev_cursor) + } else { + false + } + } } impl PartialEq for Cursor { @@ -103,6 +117,22 @@ impl PartialEq for Cursor { } } +impl Cursor { + fn is_eq_to_prev_row(&self) -> bool { + T::eq_to_previous(&self.values, self.offset) + } + + fn is_eq_to_prev_row_in_prev_batch(&self, other: &Self) -> bool { + assert_eq!(self.offset, 0); + T::eq( + &self.values, + self.offset, + &other.values, + other.values.len() - 1, + ) + } +} + impl Eq for Cursor {} impl PartialOrd for Cursor { @@ -156,6 +186,11 @@ impl CursorValues for RowValues { l.rows.row(l_idx) == r.rows.row(r_idx) } + fn eq_to_previous(cursor: &Self, idx: usize) -> bool { + assert!(idx > 0); + cursor.rows.row(idx) == cursor.rows.row(idx - 1) + } + fn compare(l: &Self, l_idx: usize, r: &Self, r_idx: usize) -> Ordering { l.rows.row(l_idx).cmp(&r.rows.row(r_idx)) } @@ -188,6 +223,11 @@ impl CursorValues for PrimitiveValues { l.0[l_idx].is_eq(r.0[r_idx]) } + fn eq_to_previous(cursor: &Self, idx: usize) -> bool { + assert!(idx > 0); + cursor.0[idx].is_eq(cursor.0[idx - 1]) + } + fn compare(l: &Self, l_idx: usize, r: &Self, r_idx: usize) -> Ordering { l.0[l_idx].compare(r.0[r_idx]) } @@ -219,6 +259,11 @@ impl CursorValues for ByteArrayValues { l.value(l_idx) == r.value(r_idx) } + fn eq_to_previous(cursor: &Self, idx: usize) -> bool { + assert!(idx > 0); + cursor.value(idx) == cursor.value(idx - 1) + } + fn compare(l: &Self, l_idx: usize, r: &Self, r_idx: usize) -> Ordering { l.value(l_idx).cmp(r.value(r_idx)) } @@ -284,6 +329,15 @@ impl CursorValues for ArrayValues { } } + fn eq_to_previous(cursor: &Self, idx: usize) -> bool { + assert!(idx > 0); + match (cursor.is_null(idx), cursor.is_null(idx - 1)) { + (true, true) => true, + (false, false) => T::eq(&cursor.values, idx, &cursor.values, idx - 1), + _ => false, + } + } + fn compare(l: &Self, l_idx: usize, r: &Self, r_idx: usize) -> Ordering { match (l.is_null(l_idx), r.is_null(r_idx)) { (true, true) => Ordering::Equal, diff --git a/datafusion/physical-plan/src/sorts/merge.rs b/datafusion/physical-plan/src/sorts/merge.rs index e0644e3d99e5..f048b2bf71e2 100644 --- a/datafusion/physical-plan/src/sorts/merge.rs +++ b/datafusion/physical-plan/src/sorts/merge.rs @@ -96,6 +96,14 @@ pub(crate) struct SortPreservingMergeStream { /// Cursors for each input partition. `None` means the input is exhausted cursors: Vec>>, + /// Calculate the number of polled with the same value. + /// For Round robin if the values are tied. + num_of_polled_with_same_value: Vec, + /// TODO: Remove it for upstream merge, for demo only + /// partition index, partition polls count + records: (usize, usize), + + prev_cursors: Vec>>, /// Optional number of rows to fetch fetch: Option, @@ -127,6 +135,9 @@ impl SortPreservingMergeStream { metrics, aborted: false, cursors: (0..stream_count).map(|_| None).collect(), + prev_cursors: (0..stream_count).map(|_| None).collect(), + records: (0, 0), + num_of_polled_with_same_value: vec![0; stream_count], loser_tree: vec![], loser_tree_adjusted: false, batch_size, @@ -214,11 +225,21 @@ impl SortPreservingMergeStream { self.aborted = true; return Poll::Ready(Some(Err(e))); } + + // adjust poll count + self.adjuct_poll_count(winner); + self.update_loser_tree(); } let stream_idx = self.loser_tree[0]; if self.advance(stream_idx) { + if stream_idx == self.records.0 { + self.records.1 += 1 + } else { + self.records.1 = 0; + } + println!("records: {:?}", self.records); self.loser_tree_adjusted = false; self.in_progress.push_row(stream_idx); @@ -236,6 +257,21 @@ impl SortPreservingMergeStream { } } + fn adjuct_poll_count(&mut self, stream_idx: usize) { + let slot = &mut self.cursors[stream_idx]; + + if let Some(c) = slot.as_mut() { + // To compare with the last row in the previous batch + let prev_cursor = &self.prev_cursors[stream_idx]; + let is_eq = c.is_eq_to_prev_one(prev_cursor.as_ref()); + if is_eq { + self.num_of_polled_with_same_value[stream_idx] += 1; + } else { + self.num_of_polled_with_same_value[stream_idx] = 0; + } + } + } + fn fetch_reached(&mut self) -> bool { self.fetch .map(|fetch| self.produced + self.in_progress.len() >= fetch) @@ -246,8 +282,9 @@ impl SortPreservingMergeStream { let slot = &mut self.cursors[stream_idx]; match slot.as_mut() { Some(c) => { - c.advance(); + let _ = c.advance(); if c.is_finished() { + std::mem::swap(slot, &mut self.prev_cursors[stream_idx]); *slot = None; } true @@ -262,7 +299,15 @@ impl SortPreservingMergeStream { match (&self.cursors[a], &self.cursors[b]) { (None, _) => true, (_, None) => false, - (Some(ac), Some(bc)) => ac.cmp(bc).then_with(|| a.cmp(&b)).is_gt(), + (Some(ac), Some(bc)) => { + let number_of_polled_a = self.num_of_polled_with_same_value[a]; + let number_of_polled_b = self.num_of_polled_with_same_value[b]; + ac.cmp(bc) + .then_with(|| number_of_polled_a.cmp(&number_of_polled_b)) + .then_with(|| a.cmp(&b)) + .is_gt() + } // TODO: remove. Switch to the old strategy to see the difference + // (Some(ac), Some(bc)) => ac.cmp(bc).then_with(|| a.cmp(&b)).is_gt(), } } diff --git a/datafusion/sqllogictest/test_files/sort_preserving_merge_repartition.slt b/datafusion/sqllogictest/test_files/sort_preserving_merge_repartition.slt new file mode 100644 index 000000000000..91cb4df91945 --- /dev/null +++ b/datafusion/sqllogictest/test_files/sort_preserving_merge_repartition.slt @@ -0,0 +1,233 @@ +# Test smaller batches + +statement ok +set datafusion.execution.target_partitions = 2; + +statement ok +set datafusion.execution.batch_size = 4; + + +statement ok +CREATE EXTERNAL TABLE annotated_data_finite2 ( + a0 INT, + a INT, + b INT, + c INT, + d INT +) +STORED AS CSV +WITH ORDER (a ASC, b ASC, c ASC) +LOCATION '../core/tests/data/low_card_spm_small_batch.csv' +OPTIONS ('format.has_header' 'true'); + + +query TT +EXPLAIN SELECT * +FROM annotated_data_finite2 +WHERE a=0 +ORDER BY b, c; +---- +logical_plan +01)Sort: annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST +02)--Filter: annotated_data_finite2.a = Int32(0) +03)----TableScan: annotated_data_finite2 projection=[a0, a, b, c, d], partial_filters=[annotated_data_finite2.a = Int32(0)] +physical_plan +01)SortPreservingMergeExec: [b@2 ASC NULLS LAST,c@3 ASC NULLS LAST] +02)--CoalesceBatchesExec: target_batch_size=4 +03)----FilterExec: a@1 = 0 +04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/low_card_spm_small_batch.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true + +query IIIII +SELECT * +FROM annotated_data_finite2 +WHERE a=0 +ORDER BY b, c; +---- +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 +0 0 0 1 1 + +statement ok +drop table annotated_data_finite2; diff --git a/datafusion/sqllogictest/test_files/sort_preserving_merge_union.slt b/datafusion/sqllogictest/test_files/sort_preserving_merge_union.slt new file mode 100644 index 000000000000..dd74841d9aab --- /dev/null +++ b/datafusion/sqllogictest/test_files/sort_preserving_merge_union.slt @@ -0,0 +1,435 @@ +statement ok +CREATE EXTERNAL TABLE ordered_table ( + a0 INT, + a INT, + b INT, + c INT, + d INT +) +STORED AS CSV +WITH ORDER (c ASC) +LOCATION '../core/tests/data/low_card_spm.csv' +OPTIONS ('format.has_header' 'true'); + +# Test for low cardinality case + +query TT +explain select * from (select b, c, a, a0 from ordered_table) t1 +union all +select * from (select b, c, a, a0 from ordered_table) t2 +order by d, c, a, a0, b +; +---- +logical_plan +01)Projection: t1.b, t1.c, t1.a, t1.a0 +02)--Sort: t1.d ASC NULLS LAST, t1.c ASC NULLS LAST, t1.a ASC NULLS LAST, t1.a0 ASC NULLS LAST, t1.b ASC NULLS LAST +03)----Union +04)------SubqueryAlias: t1 +05)--------Projection: ordered_table.b, ordered_table.c, ordered_table.a, ordered_table.a0, ordered_table.d +06)----------TableScan: ordered_table projection=[a0, a, b, c, d] +07)------SubqueryAlias: t2 +08)--------Projection: ordered_table.b, ordered_table.c, ordered_table.a, ordered_table.a0, ordered_table.d +09)----------TableScan: ordered_table projection=[a0, a, b, c, d] +physical_plan +01)ProjectionExec: expr=[b@0 as b, c@1 as c, a@2 as a, a0@3 as a0] +02)--SortPreservingMergeExec: [d@4 ASC NULLS LAST,c@1 ASC NULLS LAST,a@2 ASC NULLS LAST,a0@3 ASC NULLS LAST,b@0 ASC NULLS LAST] +03)----UnionExec +04)------SortExec: expr=[d@4 ASC NULLS LAST,c@1 ASC NULLS LAST,a@2 ASC NULLS LAST,a0@3 ASC NULLS LAST,b@0 ASC NULLS LAST], preserve_partitioning=[false] +05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/low_card_spm.csv]]}, projection=[b, c, a, a0, d], output_ordering=[c@1 ASC NULLS LAST], has_header=true +06)------SortExec: expr=[d@4 ASC NULLS LAST,c@1 ASC NULLS LAST,a@2 ASC NULLS LAST,a0@3 ASC NULLS LAST,b@0 ASC NULLS LAST], preserve_partitioning=[false] +07)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/low_card_spm.csv]]}, projection=[b, c, a, a0, d], output_ordering=[c@1 ASC NULLS LAST], has_header=true + +query IIII +select * from (select b, c, a, a0 from ordered_table) t1 +union all +select * from (select b, c, a, a0 from ordered_table) t2 +order by d, c, a, a0, b +; +---- +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 + +statement ok +drop table ordered_table; From 666f9fe07af10880c02237ebebd08b112c058ee6 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Thu, 17 Oct 2024 21:04:58 +0800 Subject: [PATCH 02/47] add data Signed-off-by: jayzhan211 --- benchmarks/bench.sh | 4 +- benchmarks/spm_data/low_card_spm.csv | 193 ++++++++++++++++++ .../spm_data/low_card_spm_small_batch.csv | 185 +++++++++++++++++ datafusion/core/tests/data/low_card_spm.csv | 193 ++++++++++++++++++ .../tests/data/low_card_spm_small_batch.csv | 185 +++++++++++++++++ datafusion/physical-plan/src/sorts/merge.rs | 12 +- 6 files changed, 765 insertions(+), 7 deletions(-) create mode 100644 benchmarks/spm_data/low_card_spm.csv create mode 100644 benchmarks/spm_data/low_card_spm_small_batch.csv create mode 100644 datafusion/core/tests/data/low_card_spm.csv create mode 100644 datafusion/core/tests/data/low_card_spm_small_batch.csv diff --git a/benchmarks/bench.sh b/benchmarks/bench.sh index e0135ec9bce7..4ceac6a1fdbd 100755 --- a/benchmarks/bench.sh +++ b/benchmarks/bench.sh @@ -34,6 +34,7 @@ COMMAND= BENCHMARK=all DATAFUSION_DIR=${DATAFUSION_DIR:-$SCRIPT_DIR/..} DATA_DIR=${DATA_DIR:-$SCRIPT_DIR/data} +DATA_SPM_DIR=${DATA_DIR:-$SCRIPT_DIR/spm_data} CARGO_COMMAND=${CARGO_COMMAND:-"cargo run --release"} PREFER_HASH_JOIN=${PREFER_HASH_JOIN:-true} VIRTUAL_ENV=${VIRTUAL_ENV:-$SCRIPT_DIR/venv} @@ -429,7 +430,8 @@ run_spm() { RESULTS_FILE="${RESULTS_DIR}/spm.json" echo "RESULTS_FILE: ${RESULTS_FILE}" echo "Running spm benchmark..." - $CARGO_COMMAND --bin dfbench -- spm --iterations 5 --path "${DATA_DIR}/low_card_spm_small_batch.csv" --queries-path "${SCRIPT_DIR}/queries/spm/queries.sql" -o "${RESULTS_FILE}" + $CARGO_COMMAND --bin dfbench -- spm --iterations 5 --path "${DATA_SPM_DIR}/low_card_spm_small_batch.csv" --queries-path "${SCRIPT_DIR}/queries/spm/queries.sql" -o "${RESULTS_FILE}" + $CARGO_COMMAND --bin dfbench -- spm --iterations 5 --path "${DATA_SPM_DIR}/low_card_spm.csv" --queries-path "${SCRIPT_DIR}/queries/spm/queries.sql" -o "${RESULTS_FILE}" } # Runs the clickbench benchmark with the partitioned parquet files diff --git a/benchmarks/spm_data/low_card_spm.csv b/benchmarks/spm_data/low_card_spm.csv new file mode 100644 index 000000000000..7f5f8ce38159 --- /dev/null +++ b/benchmarks/spm_data/low_card_spm.csv @@ -0,0 +1,193 @@ +a0,a,b,c,d +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 \ No newline at end of file diff --git a/benchmarks/spm_data/low_card_spm_small_batch.csv b/benchmarks/spm_data/low_card_spm_small_batch.csv new file mode 100644 index 000000000000..fbb90d291210 --- /dev/null +++ b/benchmarks/spm_data/low_card_spm_small_batch.csv @@ -0,0 +1,185 @@ +a0,a,b,c,d +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 \ No newline at end of file diff --git a/datafusion/core/tests/data/low_card_spm.csv b/datafusion/core/tests/data/low_card_spm.csv new file mode 100644 index 000000000000..7f5f8ce38159 --- /dev/null +++ b/datafusion/core/tests/data/low_card_spm.csv @@ -0,0 +1,193 @@ +a0,a,b,c,d +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 \ No newline at end of file diff --git a/datafusion/core/tests/data/low_card_spm_small_batch.csv b/datafusion/core/tests/data/low_card_spm_small_batch.csv new file mode 100644 index 000000000000..fbb90d291210 --- /dev/null +++ b/datafusion/core/tests/data/low_card_spm_small_batch.csv @@ -0,0 +1,185 @@ +a0,a,b,c,d +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,0,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 +0,0,0,1,1 \ No newline at end of file diff --git a/datafusion/physical-plan/src/sorts/merge.rs b/datafusion/physical-plan/src/sorts/merge.rs index f048b2bf71e2..01f0eaf0947b 100644 --- a/datafusion/physical-plan/src/sorts/merge.rs +++ b/datafusion/physical-plan/src/sorts/merge.rs @@ -234,12 +234,12 @@ impl SortPreservingMergeStream { let stream_idx = self.loser_tree[0]; if self.advance(stream_idx) { - if stream_idx == self.records.0 { - self.records.1 += 1 - } else { - self.records.1 = 0; - } - println!("records: {:?}", self.records); + // if stream_idx == self.records.0 { + // self.records.1 += 1 + // } else { + // self.records.1 = 0; + // } + // println!("records: {:?}", self.records); self.loser_tree_adjusted = false; self.in_progress.push_row(stream_idx); From 449c93049ee11faab8d89e25fbe51d6da69c31db Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Thu, 17 Oct 2024 21:20:30 +0800 Subject: [PATCH 03/47] fix benchmark Signed-off-by: jayzhan211 --- benchmarks/bench.sh | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/benchmarks/bench.sh b/benchmarks/bench.sh index 4ceac6a1fdbd..e135c5b647a7 100755 --- a/benchmarks/bench.sh +++ b/benchmarks/bench.sh @@ -34,7 +34,7 @@ COMMAND= BENCHMARK=all DATAFUSION_DIR=${DATAFUSION_DIR:-$SCRIPT_DIR/..} DATA_DIR=${DATA_DIR:-$SCRIPT_DIR/data} -DATA_SPM_DIR=${DATA_DIR:-$SCRIPT_DIR/spm_data} +DATA_SPM_DIR=${DATA_SPM_DIR:-$SCRIPT_DIR/spm_data} CARGO_COMMAND=${CARGO_COMMAND:-"cargo run --release"} PREFER_HASH_JOIN=${PREFER_HASH_JOIN:-true} VIRTUAL_ENV=${VIRTUAL_ENV:-$SCRIPT_DIR/venv} @@ -430,6 +430,7 @@ run_spm() { RESULTS_FILE="${RESULTS_DIR}/spm.json" echo "RESULTS_FILE: ${RESULTS_FILE}" echo "Running spm benchmark..." + echo "Path: ${DATA_SPM_DIR}" $CARGO_COMMAND --bin dfbench -- spm --iterations 5 --path "${DATA_SPM_DIR}/low_card_spm_small_batch.csv" --queries-path "${SCRIPT_DIR}/queries/spm/queries.sql" -o "${RESULTS_FILE}" $CARGO_COMMAND --bin dfbench -- spm --iterations 5 --path "${DATA_SPM_DIR}/low_card_spm.csv" --queries-path "${SCRIPT_DIR}/queries/spm/queries.sql" -o "${RESULTS_FILE}" } From 2566bab8f28cdf12640ead5909d708b47fb8a9a7 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Fri, 18 Oct 2024 08:04:33 +0800 Subject: [PATCH 04/47] add more bencmark data Signed-off-by: jayzhan211 --- benchmarks/bench.sh | 5 +- benchmarks/spm_data/high_card_spm.csv | 101 ++++++++++++ benchmarks/src/spm.rs | 19 +++ datafusion/core/tests/data/high_card_spm.csv | 101 ++++++++++++ ...preserving_merge_repartition_hard_card.slt | 149 ++++++++++++++++++ 5 files changed, 374 insertions(+), 1 deletion(-) create mode 100644 benchmarks/spm_data/high_card_spm.csv create mode 100644 datafusion/core/tests/data/high_card_spm.csv create mode 100644 datafusion/sqllogictest/test_files/sort_preserving_merge_repartition_hard_card.slt diff --git a/benchmarks/bench.sh b/benchmarks/bench.sh index e135c5b647a7..961bd580fe84 100755 --- a/benchmarks/bench.sh +++ b/benchmarks/bench.sh @@ -431,8 +431,11 @@ run_spm() { echo "RESULTS_FILE: ${RESULTS_FILE}" echo "Running spm benchmark..." echo "Path: ${DATA_SPM_DIR}" - $CARGO_COMMAND --bin dfbench -- spm --iterations 5 --path "${DATA_SPM_DIR}/low_card_spm_small_batch.csv" --queries-path "${SCRIPT_DIR}/queries/spm/queries.sql" -o "${RESULTS_FILE}" + $CARGO_COMMAND --bin dfbench -- spm --iterations 5 --batch_size 4 --path "${DATA_SPM_DIR}/low_card_spm_small_batch.csv" --queries-path "${SCRIPT_DIR}/queries/spm/queries.sql" -o "${RESULTS_FILE}" + $CARGO_COMMAND --bin dfbench -- spm --iterations 5 --batch_size 4 --partition 2 --path "${DATA_SPM_DIR}/low_card_spm_small_batch.csv" --queries-path "${SCRIPT_DIR}/queries/spm/queries.sql" -o "${RESULTS_FILE}" $CARGO_COMMAND --bin dfbench -- spm --iterations 5 --path "${DATA_SPM_DIR}/low_card_spm.csv" --queries-path "${SCRIPT_DIR}/queries/spm/queries.sql" -o "${RESULTS_FILE}" + $CARGO_COMMAND --bin dfbench -- spm --iterations 5 --partition 2 --path "${DATA_SPM_DIR}/low_card_spm.csv" --queries-path "${SCRIPT_DIR}/queries/spm/queries.sql" -o "${RESULTS_FILE}" + $CARGO_COMMAND --bin dfbench -- spm --iterations 5 --path "${DATA_SPM_DIR}/high_card_spm.csv" --queries-path "${SCRIPT_DIR}/queries/spm/queries.sql" -o "${RESULTS_FILE}" } # Runs the clickbench benchmark with the partitioned parquet files diff --git a/benchmarks/spm_data/high_card_spm.csv b/benchmarks/spm_data/high_card_spm.csv new file mode 100644 index 000000000000..db796cda5b0e --- /dev/null +++ b/benchmarks/spm_data/high_card_spm.csv @@ -0,0 +1,101 @@ +a0,a,b,c,d +0,0,0,1,0 +0,0,0,2,0 +0,0,0,3,0 +0,0,0,4,0 +0,0,0,5,0 +0,0,0,6,0 +0,0,0,7,0 +0,0,0,8,0 +0,0,0,9,0 +0,0,0,10,0 +0,0,0,11,0 +0,0,0,12,0 +0,0,0,13,0 +0,0,0,14,0 +0,0,0,15,0 +0,0,0,16,0 +0,0,0,17,0 +0,0,0,18,0 +0,0,0,19,0 +0,0,0,20,0 +0,0,0,21,0 +0,0,0,22,0 +0,0,0,23,0 +0,0,0,24,0 +0,0,0,25,0 +0,0,0,26,0 +0,0,0,27,0 +0,0,0,28,0 +0,0,0,29,0 +0,0,0,30,0 +0,0,0,31,0 +0,0,0,32,0 +0,0,0,33,0 +0,0,0,34,0 +0,0,0,35,0 +0,0,0,36,0 +0,0,0,37,0 +0,0,0,38,0 +0,0,0,39,0 +0,0,0,40,0 +0,0,0,41,0 +0,0,0,42,0 +0,0,0,43,0 +0,0,0,44,0 +0,0,0,45,0 +0,0,0,46,0 +0,0,0,47,0 +0,0,0,48,0 +0,0,0,49,0 +0,0,0,50,0 +0,0,0,51,0 +0,0,0,52,0 +0,0,0,53,0 +0,0,0,54,0 +0,0,0,55,0 +0,0,0,56,0 +0,0,0,57,0 +0,0,0,58,0 +0,0,0,59,0 +0,0,0,60,0 +0,0,0,61,0 +0,0,0,62,0 +0,0,0,63,0 +0,0,0,64,0 +0,0,0,65,0 +0,0,0,66,0 +0,0,0,67,0 +0,0,0,68,0 +0,0,0,69,0 +0,0,0,70,0 +0,0,0,71,0 +0,0,0,72,0 +0,0,0,73,0 +0,0,0,74,0 +0,0,0,75,0 +0,0,0,76,0 +0,0,0,77,0 +0,0,0,78,0 +0,0,0,79,0 +0,0,0,80,0 +0,0,0,81,0 +0,0,0,82,0 +0,0,0,83,0 +0,0,0,84,0 +0,0,0,85,0 +0,0,0,86,0 +0,0,0,87,0 +0,0,0,88,0 +0,0,0,89,0 +0,0,0,90,0 +0,0,0,91,0 +0,0,0,92,0 +0,0,0,93,0 +0,0,0,94,0 +0,0,0,95,0 +0,0,0,96,0 +0,0,0,97,0 +0,0,0,98,0 +0,0,0,99,0 +0,0,0,100,0 diff --git a/benchmarks/src/spm.rs b/benchmarks/src/spm.rs index 3ec0ff460d01..8dfac7d15599 100644 --- a/benchmarks/src/spm.rs +++ b/benchmarks/src/spm.rs @@ -69,6 +69,20 @@ pub struct RunOpt { /// If present, write results json here #[structopt(parse(from_os_str), short = "o", long = "output")] output_path: Option, + + #[structopt( + short = "b", + long = "batch_size", + default_value = "8192" + )] + batch_size: usize, + + #[structopt( + short = "p", + long = "partition", + default_value = "0" + )] + partition: usize, } struct AllQueries { @@ -117,6 +131,11 @@ impl RunOpt { }; let mut config = self.common.config(); + config = config.with_batch_size(self.batch_size); + if self.partition > 0 { + config = config.with_target_partitions(self.partition); + } + config .options_mut() .execution diff --git a/datafusion/core/tests/data/high_card_spm.csv b/datafusion/core/tests/data/high_card_spm.csv new file mode 100644 index 000000000000..db796cda5b0e --- /dev/null +++ b/datafusion/core/tests/data/high_card_spm.csv @@ -0,0 +1,101 @@ +a0,a,b,c,d +0,0,0,1,0 +0,0,0,2,0 +0,0,0,3,0 +0,0,0,4,0 +0,0,0,5,0 +0,0,0,6,0 +0,0,0,7,0 +0,0,0,8,0 +0,0,0,9,0 +0,0,0,10,0 +0,0,0,11,0 +0,0,0,12,0 +0,0,0,13,0 +0,0,0,14,0 +0,0,0,15,0 +0,0,0,16,0 +0,0,0,17,0 +0,0,0,18,0 +0,0,0,19,0 +0,0,0,20,0 +0,0,0,21,0 +0,0,0,22,0 +0,0,0,23,0 +0,0,0,24,0 +0,0,0,25,0 +0,0,0,26,0 +0,0,0,27,0 +0,0,0,28,0 +0,0,0,29,0 +0,0,0,30,0 +0,0,0,31,0 +0,0,0,32,0 +0,0,0,33,0 +0,0,0,34,0 +0,0,0,35,0 +0,0,0,36,0 +0,0,0,37,0 +0,0,0,38,0 +0,0,0,39,0 +0,0,0,40,0 +0,0,0,41,0 +0,0,0,42,0 +0,0,0,43,0 +0,0,0,44,0 +0,0,0,45,0 +0,0,0,46,0 +0,0,0,47,0 +0,0,0,48,0 +0,0,0,49,0 +0,0,0,50,0 +0,0,0,51,0 +0,0,0,52,0 +0,0,0,53,0 +0,0,0,54,0 +0,0,0,55,0 +0,0,0,56,0 +0,0,0,57,0 +0,0,0,58,0 +0,0,0,59,0 +0,0,0,60,0 +0,0,0,61,0 +0,0,0,62,0 +0,0,0,63,0 +0,0,0,64,0 +0,0,0,65,0 +0,0,0,66,0 +0,0,0,67,0 +0,0,0,68,0 +0,0,0,69,0 +0,0,0,70,0 +0,0,0,71,0 +0,0,0,72,0 +0,0,0,73,0 +0,0,0,74,0 +0,0,0,75,0 +0,0,0,76,0 +0,0,0,77,0 +0,0,0,78,0 +0,0,0,79,0 +0,0,0,80,0 +0,0,0,81,0 +0,0,0,82,0 +0,0,0,83,0 +0,0,0,84,0 +0,0,0,85,0 +0,0,0,86,0 +0,0,0,87,0 +0,0,0,88,0 +0,0,0,89,0 +0,0,0,90,0 +0,0,0,91,0 +0,0,0,92,0 +0,0,0,93,0 +0,0,0,94,0 +0,0,0,95,0 +0,0,0,96,0 +0,0,0,97,0 +0,0,0,98,0 +0,0,0,99,0 +0,0,0,100,0 diff --git a/datafusion/sqllogictest/test_files/sort_preserving_merge_repartition_hard_card.slt b/datafusion/sqllogictest/test_files/sort_preserving_merge_repartition_hard_card.slt new file mode 100644 index 000000000000..92161f10509c --- /dev/null +++ b/datafusion/sqllogictest/test_files/sort_preserving_merge_repartition_hard_card.slt @@ -0,0 +1,149 @@ +# Test smaller batches + +statement ok +set datafusion.execution.target_partitions = 2; + +statement ok +set datafusion.execution.batch_size = 4; + + +statement ok +CREATE EXTERNAL TABLE annotated_data_finite2 ( + a0 INT, + a INT, + b INT, + c INT, + d INT +) +STORED AS CSV +WITH ORDER (a ASC, b ASC, c ASC) +LOCATION '../core/tests/data/high_card_spm.csv' +OPTIONS ('format.has_header' 'true'); + + +query TT +EXPLAIN SELECT * +FROM annotated_data_finite2 +WHERE a=0 +ORDER BY b, c; +---- +logical_plan +01)Sort: annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST +02)--Filter: annotated_data_finite2.a = Int32(0) +03)----TableScan: annotated_data_finite2 projection=[a0, a, b, c, d], partial_filters=[annotated_data_finite2.a = Int32(0)] +physical_plan +01)SortPreservingMergeExec: [b@2 ASC NULLS LAST,c@3 ASC NULLS LAST] +02)--CoalesceBatchesExec: target_batch_size=4 +03)----FilterExec: a@1 = 0 +04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/high_card_spm.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true + +query IIIII +SELECT * +FROM annotated_data_finite2 +WHERE a=0 +ORDER BY b, c; +---- +0 0 0 1 0 +0 0 0 2 0 +0 0 0 3 0 +0 0 0 4 0 +0 0 0 5 0 +0 0 0 6 0 +0 0 0 7 0 +0 0 0 8 0 +0 0 0 9 0 +0 0 0 10 0 +0 0 0 11 0 +0 0 0 12 0 +0 0 0 13 0 +0 0 0 14 0 +0 0 0 15 0 +0 0 0 16 0 +0 0 0 17 0 +0 0 0 18 0 +0 0 0 19 0 +0 0 0 20 0 +0 0 0 21 0 +0 0 0 22 0 +0 0 0 23 0 +0 0 0 24 0 +0 0 0 25 0 +0 0 0 26 0 +0 0 0 27 0 +0 0 0 28 0 +0 0 0 29 0 +0 0 0 30 0 +0 0 0 31 0 +0 0 0 32 0 +0 0 0 33 0 +0 0 0 34 0 +0 0 0 35 0 +0 0 0 36 0 +0 0 0 37 0 +0 0 0 38 0 +0 0 0 39 0 +0 0 0 40 0 +0 0 0 41 0 +0 0 0 42 0 +0 0 0 43 0 +0 0 0 44 0 +0 0 0 45 0 +0 0 0 46 0 +0 0 0 47 0 +0 0 0 48 0 +0 0 0 49 0 +0 0 0 50 0 +0 0 0 51 0 +0 0 0 52 0 +0 0 0 53 0 +0 0 0 54 0 +0 0 0 55 0 +0 0 0 56 0 +0 0 0 57 0 +0 0 0 58 0 +0 0 0 59 0 +0 0 0 60 0 +0 0 0 61 0 +0 0 0 62 0 +0 0 0 63 0 +0 0 0 64 0 +0 0 0 65 0 +0 0 0 66 0 +0 0 0 67 0 +0 0 0 68 0 +0 0 0 69 0 +0 0 0 70 0 +0 0 0 71 0 +0 0 0 72 0 +0 0 0 73 0 +0 0 0 74 0 +0 0 0 75 0 +0 0 0 76 0 +0 0 0 77 0 +0 0 0 78 0 +0 0 0 79 0 +0 0 0 80 0 +0 0 0 81 0 +0 0 0 82 0 +0 0 0 83 0 +0 0 0 84 0 +0 0 0 85 0 +0 0 0 86 0 +0 0 0 87 0 +0 0 0 88 0 +0 0 0 89 0 +0 0 0 90 0 +0 0 0 91 0 +0 0 0 92 0 +0 0 0 93 0 +0 0 0 94 0 +0 0 0 95 0 +0 0 0 96 0 +0 0 0 97 0 +0 0 0 98 0 +0 0 0 99 0 +0 0 0 100 0 + +statement ok +drop table annotated_data_finite2; From 6000d8745e572358f2377fa8687325c2c7ccbf98 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Fri, 18 Oct 2024 10:26:51 +0800 Subject: [PATCH 05/47] fix benchmark Signed-off-by: jayzhan211 --- benchmarks/bench.sh | 16 ++++++++++------ 1 file changed, 10 insertions(+), 6 deletions(-) diff --git a/benchmarks/bench.sh b/benchmarks/bench.sh index 961bd580fe84..45f6b6347410 100755 --- a/benchmarks/bench.sh +++ b/benchmarks/bench.sh @@ -427,15 +427,19 @@ run_clickbench_1() { } run_spm() { - RESULTS_FILE="${RESULTS_DIR}/spm.json" + RESULTS_FILE_1="${RESULTS_DIR}/spm_low_card_b4.json" + RESULTS_FILE_2="${RESULTS_DIR}/spm_low_card_b4_p2.json" + RESULTS_FILE_3="${RESULTS_DIR}/spm_low_card.json" + RESULTS_FILE_4="${RESULTS_DIR}/spm_low_card_p2.json" + RESULTS_FILE_5="${RESULTS_DIR}/spm_high_card.json" echo "RESULTS_FILE: ${RESULTS_FILE}" echo "Running spm benchmark..." echo "Path: ${DATA_SPM_DIR}" - $CARGO_COMMAND --bin dfbench -- spm --iterations 5 --batch_size 4 --path "${DATA_SPM_DIR}/low_card_spm_small_batch.csv" --queries-path "${SCRIPT_DIR}/queries/spm/queries.sql" -o "${RESULTS_FILE}" - $CARGO_COMMAND --bin dfbench -- spm --iterations 5 --batch_size 4 --partition 2 --path "${DATA_SPM_DIR}/low_card_spm_small_batch.csv" --queries-path "${SCRIPT_DIR}/queries/spm/queries.sql" -o "${RESULTS_FILE}" - $CARGO_COMMAND --bin dfbench -- spm --iterations 5 --path "${DATA_SPM_DIR}/low_card_spm.csv" --queries-path "${SCRIPT_DIR}/queries/spm/queries.sql" -o "${RESULTS_FILE}" - $CARGO_COMMAND --bin dfbench -- spm --iterations 5 --partition 2 --path "${DATA_SPM_DIR}/low_card_spm.csv" --queries-path "${SCRIPT_DIR}/queries/spm/queries.sql" -o "${RESULTS_FILE}" - $CARGO_COMMAND --bin dfbench -- spm --iterations 5 --path "${DATA_SPM_DIR}/high_card_spm.csv" --queries-path "${SCRIPT_DIR}/queries/spm/queries.sql" -o "${RESULTS_FILE}" + $CARGO_COMMAND --bin dfbench -- spm --iterations 5 --batch_size 4 --path "${DATA_SPM_DIR}/low_card_spm_small_batch.csv" --queries-path "${SCRIPT_DIR}/queries/spm/queries.sql" -o "${RESULTS_FILE_1}" + $CARGO_COMMAND --bin dfbench -- spm --iterations 5 --batch_size 4 --partition 2 --path "${DATA_SPM_DIR}/low_card_spm_small_batch.csv" --queries-path "${SCRIPT_DIR}/queries/spm/queries.sql" -o "${RESULTS_FILE_2}" + $CARGO_COMMAND --bin dfbench -- spm --iterations 5 --path "${DATA_SPM_DIR}/low_card_spm.csv" --queries-path "${SCRIPT_DIR}/queries/spm/queries.sql" -o "${RESULTS_FILE_3}" + $CARGO_COMMAND --bin dfbench -- spm --iterations 5 --partition 2 --path "${DATA_SPM_DIR}/low_card_spm.csv" --queries-path "${SCRIPT_DIR}/queries/spm/queries.sql" -o "${RESULTS_FILE_4}" + $CARGO_COMMAND --bin dfbench -- spm --iterations 5 --path "${DATA_SPM_DIR}/high_card_spm.csv" --queries-path "${SCRIPT_DIR}/queries/spm/queries.sql" -o "${RESULTS_FILE_5}" } # Runs the clickbench benchmark with the partitioned parquet files From 5cd87330123b8ad4b63d67a05b458d18c9ccae86 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Fri, 18 Oct 2024 10:32:22 +0800 Subject: [PATCH 06/47] fmt Signed-off-by: jayzhan211 --- benchmarks/src/spm.rs | 12 ++---------- datafusion/physical-plan/src/sorts/merge.rs | 5 ++--- 2 files changed, 4 insertions(+), 13 deletions(-) diff --git a/benchmarks/src/spm.rs b/benchmarks/src/spm.rs index 8dfac7d15599..700df11959d6 100644 --- a/benchmarks/src/spm.rs +++ b/benchmarks/src/spm.rs @@ -70,18 +70,10 @@ pub struct RunOpt { #[structopt(parse(from_os_str), short = "o", long = "output")] output_path: Option, - #[structopt( - short = "b", - long = "batch_size", - default_value = "8192" - )] + #[structopt(short = "b", long = "batch_size", default_value = "8192")] batch_size: usize, - #[structopt( - short = "p", - long = "partition", - default_value = "0" - )] + #[structopt(short = "p", long = "partition", default_value = "0")] partition: usize, } diff --git a/datafusion/physical-plan/src/sorts/merge.rs b/datafusion/physical-plan/src/sorts/merge.rs index 01f0eaf0947b..6117e4d3e59e 100644 --- a/datafusion/physical-plan/src/sorts/merge.rs +++ b/datafusion/physical-plan/src/sorts/merge.rs @@ -101,8 +101,7 @@ pub(crate) struct SortPreservingMergeStream { num_of_polled_with_same_value: Vec, /// TODO: Remove it for upstream merge, for demo only /// partition index, partition polls count - records: (usize, usize), - + // records: (usize, usize), prev_cursors: Vec>>, /// Optional number of rows to fetch @@ -136,7 +135,7 @@ impl SortPreservingMergeStream { aborted: false, cursors: (0..stream_count).map(|_| None).collect(), prev_cursors: (0..stream_count).map(|_| None).collect(), - records: (0, 0), + // records: (0, 0), num_of_polled_with_same_value: vec![0; stream_count], loser_tree: vec![], loser_tree_adjusted: false, From 74d3d9bd7bc18c6a37714e95daaa626cf9d284bb Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Fri, 18 Oct 2024 10:58:57 +0800 Subject: [PATCH 07/47] get max size Signed-off-by: jayzhan211 --- datafusion/execution/src/memory_pool/mod.rs | 9 ++++++++- datafusion/physical-plan/src/sorts/merge.rs | 2 +- ...t => sort_preserving_merge_high_card_repartition.slt} | 0 3 files changed, 9 insertions(+), 2 deletions(-) rename datafusion/sqllogictest/test_files/{sort_preserving_merge_repartition_hard_card.slt => sort_preserving_merge_high_card_repartition.slt} (100%) diff --git a/datafusion/execution/src/memory_pool/mod.rs b/datafusion/execution/src/memory_pool/mod.rs index 6b51343f76b9..5ee86fea47d2 100644 --- a/datafusion/execution/src/memory_pool/mod.rs +++ b/datafusion/execution/src/memory_pool/mod.rs @@ -181,6 +181,7 @@ impl MemoryConsumer { consumer: self, }), size: 0, + max_size: 0, } } } @@ -210,6 +211,7 @@ impl Drop for SharedRegistration { pub struct MemoryReservation { registration: Arc, size: usize, + max_size: usize, } impl MemoryReservation { @@ -293,7 +295,10 @@ impl MemoryReservation { self.registration.pool.try_grow(self, capacity)?; self.size += capacity; // TODO: Remove. For measuring the memory between the change - // println!("self.size: {:?}", self.size); + // if self.size > self.max_size { + // self.max_size = self.size; + // println!("max size: {}", self.max_size) + // } Ok(()) } @@ -312,6 +317,7 @@ impl MemoryReservation { Self { size: capacity, registration: Arc::clone(&self.registration), + max_size: self.max_size, } } @@ -320,6 +326,7 @@ impl MemoryReservation { Self { size: 0, registration: Arc::clone(&self.registration), + max_size: self.max_size, } } diff --git a/datafusion/physical-plan/src/sorts/merge.rs b/datafusion/physical-plan/src/sorts/merge.rs index 6117e4d3e59e..854514ccf9da 100644 --- a/datafusion/physical-plan/src/sorts/merge.rs +++ b/datafusion/physical-plan/src/sorts/merge.rs @@ -306,7 +306,7 @@ impl SortPreservingMergeStream { .then_with(|| a.cmp(&b)) .is_gt() } // TODO: remove. Switch to the old strategy to see the difference - // (Some(ac), Some(bc)) => ac.cmp(bc).then_with(|| a.cmp(&b)).is_gt(), + // (Some(ac), Some(bc)) => ac.cmp(bc).then_with(|| a.cmp(&b)).is_gt(), } } diff --git a/datafusion/sqllogictest/test_files/sort_preserving_merge_repartition_hard_card.slt b/datafusion/sqllogictest/test_files/sort_preserving_merge_high_card_repartition.slt similarity index 100% rename from datafusion/sqllogictest/test_files/sort_preserving_merge_repartition_hard_card.slt rename to datafusion/sqllogictest/test_files/sort_preserving_merge_high_card_repartition.slt From e6287647adb38d56dcb1354a3aa30d4a650a0e51 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Fri, 18 Oct 2024 11:08:56 +0800 Subject: [PATCH 08/47] add license Signed-off-by: jayzhan211 --- ...t_preserving_merge_high_card_repartition.slt | 17 +++++++++++++++++ .../sort_preserving_merge_repartition.slt | 17 +++++++++++++++++ .../test_files/sort_preserving_merge_union.slt | 17 +++++++++++++++++ 3 files changed, 51 insertions(+) diff --git a/datafusion/sqllogictest/test_files/sort_preserving_merge_high_card_repartition.slt b/datafusion/sqllogictest/test_files/sort_preserving_merge_high_card_repartition.slt index 92161f10509c..f756516451e0 100644 --- a/datafusion/sqllogictest/test_files/sort_preserving_merge_high_card_repartition.slt +++ b/datafusion/sqllogictest/test_files/sort_preserving_merge_high_card_repartition.slt @@ -1,3 +1,20 @@ +# 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. + # Test smaller batches statement ok diff --git a/datafusion/sqllogictest/test_files/sort_preserving_merge_repartition.slt b/datafusion/sqllogictest/test_files/sort_preserving_merge_repartition.slt index 91cb4df91945..dd2f805486b4 100644 --- a/datafusion/sqllogictest/test_files/sort_preserving_merge_repartition.slt +++ b/datafusion/sqllogictest/test_files/sort_preserving_merge_repartition.slt @@ -1,3 +1,20 @@ +# 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. + # Test smaller batches statement ok diff --git a/datafusion/sqllogictest/test_files/sort_preserving_merge_union.slt b/datafusion/sqllogictest/test_files/sort_preserving_merge_union.slt index dd74841d9aab..50f7487cb355 100644 --- a/datafusion/sqllogictest/test_files/sort_preserving_merge_union.slt +++ b/datafusion/sqllogictest/test_files/sort_preserving_merge_union.slt @@ -1,3 +1,20 @@ +# 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. + statement ok CREATE EXTERNAL TABLE ordered_table ( a0 INT, From 4277eecab685b11e59d5815aee00a64a48dd66c5 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Fri, 18 Oct 2024 13:14:16 +0800 Subject: [PATCH 09/47] rm code for merge Signed-off-by: jayzhan211 --- benchmarks/bench.sh | 20 - benchmarks/spm_data/high_card_spm.csv | 101 ---- benchmarks/spm_data/low_card_spm.csv | 193 -------- .../spm_data/low_card_spm_small_batch.csv | 185 ------- benchmarks/src/bin/dfbench.rs | 4 +- benchmarks/src/lib.rs | 1 - benchmarks/src/spm.rs | 176 ------- datafusion/execution/src/memory_pool/mod.rs | 9 - datafusion/physical-plan/src/sorts/merge.rs | 16 +- .../test_files/sort_preserving_merge.slt | 173 +++++++ ...preserving_merge_high_card_repartition.slt | 166 ------- .../sort_preserving_merge_repartition.slt | 250 ---------- .../sort_preserving_merge_union.slt | 452 ------------------ 13 files changed, 182 insertions(+), 1564 deletions(-) delete mode 100644 benchmarks/spm_data/high_card_spm.csv delete mode 100644 benchmarks/spm_data/low_card_spm.csv delete mode 100644 benchmarks/spm_data/low_card_spm_small_batch.csv delete mode 100644 benchmarks/src/spm.rs create mode 100644 datafusion/sqllogictest/test_files/sort_preserving_merge.slt delete mode 100644 datafusion/sqllogictest/test_files/sort_preserving_merge_high_card_repartition.slt delete mode 100644 datafusion/sqllogictest/test_files/sort_preserving_merge_repartition.slt delete mode 100644 datafusion/sqllogictest/test_files/sort_preserving_merge_union.slt diff --git a/benchmarks/bench.sh b/benchmarks/bench.sh index 45f6b6347410..70faa9ef2b73 100755 --- a/benchmarks/bench.sh +++ b/benchmarks/bench.sh @@ -34,7 +34,6 @@ COMMAND= BENCHMARK=all DATAFUSION_DIR=${DATAFUSION_DIR:-$SCRIPT_DIR/..} DATA_DIR=${DATA_DIR:-$SCRIPT_DIR/data} -DATA_SPM_DIR=${DATA_SPM_DIR:-$SCRIPT_DIR/spm_data} CARGO_COMMAND=${CARGO_COMMAND:-"cargo run --release"} PREFER_HASH_JOIN=${PREFER_HASH_JOIN:-true} VIRTUAL_ENV=${VIRTUAL_ENV:-$SCRIPT_DIR/venv} @@ -232,9 +231,6 @@ main() { sort) run_sort ;; - spm) - run_spm - ;; clickbench_1) run_clickbench_1 ;; @@ -426,22 +422,6 @@ run_clickbench_1() { $CARGO_COMMAND --bin dfbench -- clickbench --iterations 5 --path "${DATA_DIR}/hits.parquet" --queries-path "${SCRIPT_DIR}/queries/clickbench/queries.sql" -o "${RESULTS_FILE}" } -run_spm() { - RESULTS_FILE_1="${RESULTS_DIR}/spm_low_card_b4.json" - RESULTS_FILE_2="${RESULTS_DIR}/spm_low_card_b4_p2.json" - RESULTS_FILE_3="${RESULTS_DIR}/spm_low_card.json" - RESULTS_FILE_4="${RESULTS_DIR}/spm_low_card_p2.json" - RESULTS_FILE_5="${RESULTS_DIR}/spm_high_card.json" - echo "RESULTS_FILE: ${RESULTS_FILE}" - echo "Running spm benchmark..." - echo "Path: ${DATA_SPM_DIR}" - $CARGO_COMMAND --bin dfbench -- spm --iterations 5 --batch_size 4 --path "${DATA_SPM_DIR}/low_card_spm_small_batch.csv" --queries-path "${SCRIPT_DIR}/queries/spm/queries.sql" -o "${RESULTS_FILE_1}" - $CARGO_COMMAND --bin dfbench -- spm --iterations 5 --batch_size 4 --partition 2 --path "${DATA_SPM_DIR}/low_card_spm_small_batch.csv" --queries-path "${SCRIPT_DIR}/queries/spm/queries.sql" -o "${RESULTS_FILE_2}" - $CARGO_COMMAND --bin dfbench -- spm --iterations 5 --path "${DATA_SPM_DIR}/low_card_spm.csv" --queries-path "${SCRIPT_DIR}/queries/spm/queries.sql" -o "${RESULTS_FILE_3}" - $CARGO_COMMAND --bin dfbench -- spm --iterations 5 --partition 2 --path "${DATA_SPM_DIR}/low_card_spm.csv" --queries-path "${SCRIPT_DIR}/queries/spm/queries.sql" -o "${RESULTS_FILE_4}" - $CARGO_COMMAND --bin dfbench -- spm --iterations 5 --path "${DATA_SPM_DIR}/high_card_spm.csv" --queries-path "${SCRIPT_DIR}/queries/spm/queries.sql" -o "${RESULTS_FILE_5}" -} - # Runs the clickbench benchmark with the partitioned parquet files run_clickbench_partitioned() { RESULTS_FILE="${RESULTS_DIR}/clickbench_partitioned.json" diff --git a/benchmarks/spm_data/high_card_spm.csv b/benchmarks/spm_data/high_card_spm.csv deleted file mode 100644 index db796cda5b0e..000000000000 --- a/benchmarks/spm_data/high_card_spm.csv +++ /dev/null @@ -1,101 +0,0 @@ -a0,a,b,c,d -0,0,0,1,0 -0,0,0,2,0 -0,0,0,3,0 -0,0,0,4,0 -0,0,0,5,0 -0,0,0,6,0 -0,0,0,7,0 -0,0,0,8,0 -0,0,0,9,0 -0,0,0,10,0 -0,0,0,11,0 -0,0,0,12,0 -0,0,0,13,0 -0,0,0,14,0 -0,0,0,15,0 -0,0,0,16,0 -0,0,0,17,0 -0,0,0,18,0 -0,0,0,19,0 -0,0,0,20,0 -0,0,0,21,0 -0,0,0,22,0 -0,0,0,23,0 -0,0,0,24,0 -0,0,0,25,0 -0,0,0,26,0 -0,0,0,27,0 -0,0,0,28,0 -0,0,0,29,0 -0,0,0,30,0 -0,0,0,31,0 -0,0,0,32,0 -0,0,0,33,0 -0,0,0,34,0 -0,0,0,35,0 -0,0,0,36,0 -0,0,0,37,0 -0,0,0,38,0 -0,0,0,39,0 -0,0,0,40,0 -0,0,0,41,0 -0,0,0,42,0 -0,0,0,43,0 -0,0,0,44,0 -0,0,0,45,0 -0,0,0,46,0 -0,0,0,47,0 -0,0,0,48,0 -0,0,0,49,0 -0,0,0,50,0 -0,0,0,51,0 -0,0,0,52,0 -0,0,0,53,0 -0,0,0,54,0 -0,0,0,55,0 -0,0,0,56,0 -0,0,0,57,0 -0,0,0,58,0 -0,0,0,59,0 -0,0,0,60,0 -0,0,0,61,0 -0,0,0,62,0 -0,0,0,63,0 -0,0,0,64,0 -0,0,0,65,0 -0,0,0,66,0 -0,0,0,67,0 -0,0,0,68,0 -0,0,0,69,0 -0,0,0,70,0 -0,0,0,71,0 -0,0,0,72,0 -0,0,0,73,0 -0,0,0,74,0 -0,0,0,75,0 -0,0,0,76,0 -0,0,0,77,0 -0,0,0,78,0 -0,0,0,79,0 -0,0,0,80,0 -0,0,0,81,0 -0,0,0,82,0 -0,0,0,83,0 -0,0,0,84,0 -0,0,0,85,0 -0,0,0,86,0 -0,0,0,87,0 -0,0,0,88,0 -0,0,0,89,0 -0,0,0,90,0 -0,0,0,91,0 -0,0,0,92,0 -0,0,0,93,0 -0,0,0,94,0 -0,0,0,95,0 -0,0,0,96,0 -0,0,0,97,0 -0,0,0,98,0 -0,0,0,99,0 -0,0,0,100,0 diff --git a/benchmarks/spm_data/low_card_spm.csv b/benchmarks/spm_data/low_card_spm.csv deleted file mode 100644 index 7f5f8ce38159..000000000000 --- a/benchmarks/spm_data/low_card_spm.csv +++ /dev/null @@ -1,193 +0,0 @@ -a0,a,b,c,d -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 \ No newline at end of file diff --git a/benchmarks/spm_data/low_card_spm_small_batch.csv b/benchmarks/spm_data/low_card_spm_small_batch.csv deleted file mode 100644 index fbb90d291210..000000000000 --- a/benchmarks/spm_data/low_card_spm_small_batch.csv +++ /dev/null @@ -1,185 +0,0 @@ -a0,a,b,c,d -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 \ No newline at end of file diff --git a/benchmarks/src/bin/dfbench.rs b/benchmarks/src/bin/dfbench.rs index 0fd749b45d89..f7b84116e793 100644 --- a/benchmarks/src/bin/dfbench.rs +++ b/benchmarks/src/bin/dfbench.rs @@ -33,7 +33,7 @@ static ALLOC: snmalloc_rs::SnMalloc = snmalloc_rs::SnMalloc; #[global_allocator] static ALLOC: mimalloc::MiMalloc = mimalloc::MiMalloc; -use datafusion_benchmarks::{clickbench, imdb, parquet_filter, sort, spm, tpch}; +use datafusion_benchmarks::{clickbench, imdb, parquet_filter, sort, tpch}; #[derive(Debug, StructOpt)] #[structopt(about = "benchmark command")] @@ -41,7 +41,6 @@ enum Options { Tpch(tpch::RunOpt), TpchConvert(tpch::ConvertOpt), Clickbench(clickbench::RunOpt), - Spm(spm::RunOpt), ParquetFilter(parquet_filter::RunOpt), Sort(sort::RunOpt), Imdb(imdb::RunOpt), @@ -56,7 +55,6 @@ pub async fn main() -> Result<()> { Options::Tpch(opt) => opt.run().await, Options::TpchConvert(opt) => opt.run().await, Options::Clickbench(opt) => opt.run().await, - Options::Spm(opt) => opt.run().await, Options::ParquetFilter(opt) => opt.run().await, Options::Sort(opt) => opt.run().await, Options::Imdb(opt) => opt.run().await, diff --git a/benchmarks/src/lib.rs b/benchmarks/src/lib.rs index 2123187bb2b1..52d81ca91816 100644 --- a/benchmarks/src/lib.rs +++ b/benchmarks/src/lib.rs @@ -20,7 +20,6 @@ pub mod clickbench; pub mod imdb; pub mod parquet_filter; pub mod sort; -pub mod spm; pub mod tpch; mod util; pub use util::*; diff --git a/benchmarks/src/spm.rs b/benchmarks/src/spm.rs deleted file mode 100644 index 700df11959d6..000000000000 --- a/benchmarks/src/spm.rs +++ /dev/null @@ -1,176 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -use std::path::Path; -use std::path::PathBuf; - -use datafusion::{ - error::{DataFusionError, Result}, - prelude::SessionContext, -}; -use datafusion_common::exec_datafusion_err; -use datafusion_common::instant::Instant; -use structopt::StructOpt; - -use crate::{BenchmarkRun, CommonOpt}; - -/// Run the clickbench benchmark -/// -/// The ClickBench[1] benchmarks are widely cited in the industry and -/// focus on grouping / aggregation / filtering. This runner uses the -/// scripts and queries from [2]. -/// -/// [1]: https://github.com/ClickHouse/ClickBench -/// [2]: https://github.com/ClickHouse/ClickBench/tree/main/datafusion -#[derive(Debug, StructOpt, Clone)] -#[structopt(verbatim_doc_comment)] -pub struct RunOpt { - /// Query number (between 0 and 42). If not specified, runs all queries - #[structopt(short, long)] - query: Option, - - /// Common options - #[structopt(flatten)] - common: CommonOpt, - - /// Path to hits.parquet (single file) or `hits_partitioned` - /// (partitioned, 100 files) - #[structopt( - parse(from_os_str), - short = "p", - long = "path", - default_value = "benchmarks/data/low_card_spm.csv" - )] - path: PathBuf, - - /// Path to queries.sql (single file) - #[structopt( - parse(from_os_str), - short = "r", - long = "queries-path", - default_value = "benchmarks/queries/clickbench/queries.sql" - )] - queries_path: PathBuf, - - /// If present, write results json here - #[structopt(parse(from_os_str), short = "o", long = "output")] - output_path: Option, - - #[structopt(short = "b", long = "batch_size", default_value = "8192")] - batch_size: usize, - - #[structopt(short = "p", long = "partition", default_value = "0")] - partition: usize, -} - -struct AllQueries { - queries: Vec, -} - -impl AllQueries { - fn try_new(path: &Path) -> Result { - // ClickBench has all queries in a single file identified by line number - let all_queries = std::fs::read_to_string(path) - .map_err(|e| exec_datafusion_err!("Could not open {path:?}: {e}"))?; - Ok(Self { - queries: all_queries.lines().map(|s| s.to_string()).collect(), - }) - } - - /// Returns the text of query `query_id` - fn get_query(&self, query_id: usize) -> Result<&str> { - self.queries - .get(query_id) - .ok_or_else(|| { - let min_id = self.min_query_id(); - let max_id = self.max_query_id(); - exec_datafusion_err!( - "Invalid query id {query_id}. Must be between {min_id} and {max_id}" - ) - }) - .map(|s| s.as_str()) - } - - fn min_query_id(&self) -> usize { - 0 - } - - fn max_query_id(&self) -> usize { - self.queries.len() - 1 - } -} -impl RunOpt { - pub async fn run(self) -> Result<()> { - println!("Running benchmarks with the following options: {self:?}"); - let queries = AllQueries::try_new(self.queries_path.as_path())?; - let query_range = match self.query { - Some(query_id) => query_id..=query_id, - None => queries.min_query_id()..=queries.max_query_id(), - }; - - let mut config = self.common.config(); - config = config.with_batch_size(self.batch_size); - if self.partition > 0 { - config = config.with_target_partitions(self.partition); - } - - config - .options_mut() - .execution - .parquet - .schema_force_view_types = self.common.force_view_types; - - let ctx = SessionContext::new_with_config(config); - self.register_spm(&ctx).await?; - - let iterations = self.common.iterations; - let mut benchmark_run = BenchmarkRun::new(); - for query_id in query_range { - benchmark_run.start_new_case(&format!("Query {query_id}")); - let sql = queries.get_query(query_id)?; - println!("Q{query_id}: {sql}"); - - for i in 0..iterations { - let start = Instant::now(); - let results = ctx.sql(sql).await?.collect().await?; - let elapsed = start.elapsed(); - let ms = elapsed.as_secs_f64() * 1000.0; - let row_count: usize = results.iter().map(|b| b.num_rows()).sum(); - println!( - "Query {query_id} iteration {i} took {ms:.1} ms and returned {row_count} rows" - ); - benchmark_run.write_iter(elapsed, row_count); - } - } - benchmark_run.maybe_write_json(self.output_path.as_ref())?; - Ok(()) - } - - /// Registrs the `hits.parquet` as a table named `hits` - async fn register_spm(&self, ctx: &SessionContext) -> Result<()> { - let options = Default::default(); - let path = self.path.as_os_str().to_str().unwrap(); - ctx.register_csv("spm_table", path, options) - .await - .map_err(|e| { - DataFusionError::Context( - format!("Registering 'spm' as {path}"), - Box::new(e), - ) - }) - } -} diff --git a/datafusion/execution/src/memory_pool/mod.rs b/datafusion/execution/src/memory_pool/mod.rs index 5ee86fea47d2..d87ce1ebfed7 100644 --- a/datafusion/execution/src/memory_pool/mod.rs +++ b/datafusion/execution/src/memory_pool/mod.rs @@ -181,7 +181,6 @@ impl MemoryConsumer { consumer: self, }), size: 0, - max_size: 0, } } } @@ -211,7 +210,6 @@ impl Drop for SharedRegistration { pub struct MemoryReservation { registration: Arc, size: usize, - max_size: usize, } impl MemoryReservation { @@ -294,11 +292,6 @@ impl MemoryReservation { pub fn try_grow(&mut self, capacity: usize) -> Result<()> { self.registration.pool.try_grow(self, capacity)?; self.size += capacity; - // TODO: Remove. For measuring the memory between the change - // if self.size > self.max_size { - // self.max_size = self.size; - // println!("max size: {}", self.max_size) - // } Ok(()) } @@ -317,7 +310,6 @@ impl MemoryReservation { Self { size: capacity, registration: Arc::clone(&self.registration), - max_size: self.max_size, } } @@ -326,7 +318,6 @@ impl MemoryReservation { Self { size: 0, registration: Arc::clone(&self.registration), - max_size: self.max_size, } } diff --git a/datafusion/physical-plan/src/sorts/merge.rs b/datafusion/physical-plan/src/sorts/merge.rs index 854514ccf9da..5da6f710c393 100644 --- a/datafusion/physical-plan/src/sorts/merge.rs +++ b/datafusion/physical-plan/src/sorts/merge.rs @@ -101,7 +101,7 @@ pub(crate) struct SortPreservingMergeStream { num_of_polled_with_same_value: Vec, /// TODO: Remove it for upstream merge, for demo only /// partition index, partition polls count - // records: (usize, usize), + records: (usize, usize), prev_cursors: Vec>>, /// Optional number of rows to fetch @@ -135,7 +135,7 @@ impl SortPreservingMergeStream { aborted: false, cursors: (0..stream_count).map(|_| None).collect(), prev_cursors: (0..stream_count).map(|_| None).collect(), - // records: (0, 0), + records: (0, 0), num_of_polled_with_same_value: vec![0; stream_count], loser_tree: vec![], loser_tree_adjusted: false, @@ -233,12 +233,12 @@ impl SortPreservingMergeStream { let stream_idx = self.loser_tree[0]; if self.advance(stream_idx) { - // if stream_idx == self.records.0 { - // self.records.1 += 1 - // } else { - // self.records.1 = 0; - // } - // println!("records: {:?}", self.records); + if stream_idx == self.records.0 { + self.records.1 += 1 + } else { + self.records.1 = 0; + } + println!("records: {:?}", self.records); self.loser_tree_adjusted = false; self.in_progress.push_row(stream_idx); diff --git a/datafusion/sqllogictest/test_files/sort_preserving_merge.slt b/datafusion/sqllogictest/test_files/sort_preserving_merge.slt new file mode 100644 index 000000000000..d841dcfae857 --- /dev/null +++ b/datafusion/sqllogictest/test_files/sort_preserving_merge.slt @@ -0,0 +1,173 @@ +# 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. + +statement ok +CREATE EXTERNAL TABLE ordered_table ( + a0 INT, + a INT, + b INT, + c INT, + d INT +) +STORED AS CSV +WITH ORDER (c ASC) +LOCATION '../core/tests/data/low_card_spm.csv' +OPTIONS ('format.has_header' 'true'); + +# Test for low cardinality case + +query TT +explain select * from (select b, c, a, a0 from ordered_table) t1 +union all +select * from (select b, c, a, a0 from ordered_table) t2 +order by d, c, a, a0, b +limit 4; +---- +logical_plan +01)Projection: t1.b, t1.c, t1.a, t1.a0 +02)--Sort: t1.d ASC NULLS LAST, t1.c ASC NULLS LAST, t1.a ASC NULLS LAST, t1.a0 ASC NULLS LAST, t1.b ASC NULLS LAST, fetch=4 +03)----Union +04)------SubqueryAlias: t1 +05)--------Projection: ordered_table.b, ordered_table.c, ordered_table.a, ordered_table.a0, ordered_table.d +06)----------TableScan: ordered_table projection=[a0, a, b, c, d] +07)------SubqueryAlias: t2 +08)--------Projection: ordered_table.b, ordered_table.c, ordered_table.a, ordered_table.a0, ordered_table.d +09)----------TableScan: ordered_table projection=[a0, a, b, c, d] +physical_plan +01)ProjectionExec: expr=[b@0 as b, c@1 as c, a@2 as a, a0@3 as a0] +02)--SortPreservingMergeExec: [d@4 ASC NULLS LAST,c@1 ASC NULLS LAST,a@2 ASC NULLS LAST,a0@3 ASC NULLS LAST,b@0 ASC NULLS LAST], fetch=4 +03)----UnionExec +04)------SortExec: TopK(fetch=4), expr=[d@4 ASC NULLS LAST,c@1 ASC NULLS LAST,a@2 ASC NULLS LAST,a0@3 ASC NULLS LAST,b@0 ASC NULLS LAST], preserve_partitioning=[false] +05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/low_card_spm.csv]]}, projection=[b, c, a, a0, d], output_ordering=[c@1 ASC NULLS LAST], has_header=true +06)------SortExec: TopK(fetch=4), expr=[d@4 ASC NULLS LAST,c@1 ASC NULLS LAST,a@2 ASC NULLS LAST,a0@3 ASC NULLS LAST,b@0 ASC NULLS LAST], preserve_partitioning=[false] +07)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/low_card_spm.csv]]}, projection=[b, c, a, a0, d], output_ordering=[c@1 ASC NULLS LAST], has_header=true + +query IIII +select * from (select b, c, a, a0 from ordered_table) t1 +union all +select * from (select b, c, a, a0 from ordered_table) t2 +order by d, c, a, a0, b +limit 4; +---- +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 + +statement ok +drop table ordered_table; + +# Test smaller batches + +statement ok +set datafusion.execution.target_partitions = 2; + +statement ok +set datafusion.execution.batch_size = 4; + +statement ok +CREATE EXTERNAL TABLE annotated_data_finite2 ( + a0 INT, + a INT, + b INT, + c INT, + d INT +) +STORED AS CSV +WITH ORDER (a ASC, b ASC, c ASC) +LOCATION '../core/tests/data/low_card_spm_small_batch.csv' +OPTIONS ('format.has_header' 'true'); + + +query TT +EXPLAIN SELECT * +FROM annotated_data_finite2 +WHERE a=0 +ORDER BY b, c +limit 4; +---- +logical_plan +01)Sort: annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST, fetch=4 +02)--Filter: annotated_data_finite2.a = Int32(0) +03)----TableScan: annotated_data_finite2 projection=[a0, a, b, c, d], partial_filters=[annotated_data_finite2.a = Int32(0)] +physical_plan +01)SortPreservingMergeExec: [b@2 ASC NULLS LAST,c@3 ASC NULLS LAST], fetch=4 +02)--CoalesceBatchesExec: target_batch_size=4, fetch=4 +03)----FilterExec: a@1 = 0 +04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/low_card_spm_small_batch.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true + +query IIIII +SELECT * +FROM annotated_data_finite2 +WHERE a=0 +ORDER BY b, c +limit 4; +---- +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 +0 0 0 0 1 + +statement ok +drop table annotated_data_finite2; + +statement ok +CREATE EXTERNAL TABLE annotated_data_finite2 ( + a0 INT, + a INT, + b INT, + c INT, + d INT +) +STORED AS CSV +WITH ORDER (a ASC, b ASC, c ASC) +LOCATION '../core/tests/data/high_card_spm.csv' +OPTIONS ('format.has_header' 'true'); + +query TT +EXPLAIN SELECT * +FROM annotated_data_finite2 +WHERE a=0 +ORDER BY b, c +limit 4; +---- +logical_plan +01)Sort: annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST, fetch=4 +02)--Filter: annotated_data_finite2.a = Int32(0) +03)----TableScan: annotated_data_finite2 projection=[a0, a, b, c, d], partial_filters=[annotated_data_finite2.a = Int32(0)] +physical_plan +01)SortPreservingMergeExec: [b@2 ASC NULLS LAST,c@3 ASC NULLS LAST], fetch=4 +02)--CoalesceBatchesExec: target_batch_size=4, fetch=4 +03)----FilterExec: a@1 = 0 +04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/high_card_spm.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true + +query IIIII +SELECT * +FROM annotated_data_finite2 +WHERE a=0 +ORDER BY b, c +limit 4; +---- +0 0 0 1 0 +0 0 0 2 0 +0 0 0 3 0 +0 0 0 4 0 + +statement ok +drop table annotated_data_finite2; \ No newline at end of file diff --git a/datafusion/sqllogictest/test_files/sort_preserving_merge_high_card_repartition.slt b/datafusion/sqllogictest/test_files/sort_preserving_merge_high_card_repartition.slt deleted file mode 100644 index f756516451e0..000000000000 --- a/datafusion/sqllogictest/test_files/sort_preserving_merge_high_card_repartition.slt +++ /dev/null @@ -1,166 +0,0 @@ -# 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. - -# Test smaller batches - -statement ok -set datafusion.execution.target_partitions = 2; - -statement ok -set datafusion.execution.batch_size = 4; - - -statement ok -CREATE EXTERNAL TABLE annotated_data_finite2 ( - a0 INT, - a INT, - b INT, - c INT, - d INT -) -STORED AS CSV -WITH ORDER (a ASC, b ASC, c ASC) -LOCATION '../core/tests/data/high_card_spm.csv' -OPTIONS ('format.has_header' 'true'); - - -query TT -EXPLAIN SELECT * -FROM annotated_data_finite2 -WHERE a=0 -ORDER BY b, c; ----- -logical_plan -01)Sort: annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST -02)--Filter: annotated_data_finite2.a = Int32(0) -03)----TableScan: annotated_data_finite2 projection=[a0, a, b, c, d], partial_filters=[annotated_data_finite2.a = Int32(0)] -physical_plan -01)SortPreservingMergeExec: [b@2 ASC NULLS LAST,c@3 ASC NULLS LAST] -02)--CoalesceBatchesExec: target_batch_size=4 -03)----FilterExec: a@1 = 0 -04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/high_card_spm.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true - -query IIIII -SELECT * -FROM annotated_data_finite2 -WHERE a=0 -ORDER BY b, c; ----- -0 0 0 1 0 -0 0 0 2 0 -0 0 0 3 0 -0 0 0 4 0 -0 0 0 5 0 -0 0 0 6 0 -0 0 0 7 0 -0 0 0 8 0 -0 0 0 9 0 -0 0 0 10 0 -0 0 0 11 0 -0 0 0 12 0 -0 0 0 13 0 -0 0 0 14 0 -0 0 0 15 0 -0 0 0 16 0 -0 0 0 17 0 -0 0 0 18 0 -0 0 0 19 0 -0 0 0 20 0 -0 0 0 21 0 -0 0 0 22 0 -0 0 0 23 0 -0 0 0 24 0 -0 0 0 25 0 -0 0 0 26 0 -0 0 0 27 0 -0 0 0 28 0 -0 0 0 29 0 -0 0 0 30 0 -0 0 0 31 0 -0 0 0 32 0 -0 0 0 33 0 -0 0 0 34 0 -0 0 0 35 0 -0 0 0 36 0 -0 0 0 37 0 -0 0 0 38 0 -0 0 0 39 0 -0 0 0 40 0 -0 0 0 41 0 -0 0 0 42 0 -0 0 0 43 0 -0 0 0 44 0 -0 0 0 45 0 -0 0 0 46 0 -0 0 0 47 0 -0 0 0 48 0 -0 0 0 49 0 -0 0 0 50 0 -0 0 0 51 0 -0 0 0 52 0 -0 0 0 53 0 -0 0 0 54 0 -0 0 0 55 0 -0 0 0 56 0 -0 0 0 57 0 -0 0 0 58 0 -0 0 0 59 0 -0 0 0 60 0 -0 0 0 61 0 -0 0 0 62 0 -0 0 0 63 0 -0 0 0 64 0 -0 0 0 65 0 -0 0 0 66 0 -0 0 0 67 0 -0 0 0 68 0 -0 0 0 69 0 -0 0 0 70 0 -0 0 0 71 0 -0 0 0 72 0 -0 0 0 73 0 -0 0 0 74 0 -0 0 0 75 0 -0 0 0 76 0 -0 0 0 77 0 -0 0 0 78 0 -0 0 0 79 0 -0 0 0 80 0 -0 0 0 81 0 -0 0 0 82 0 -0 0 0 83 0 -0 0 0 84 0 -0 0 0 85 0 -0 0 0 86 0 -0 0 0 87 0 -0 0 0 88 0 -0 0 0 89 0 -0 0 0 90 0 -0 0 0 91 0 -0 0 0 92 0 -0 0 0 93 0 -0 0 0 94 0 -0 0 0 95 0 -0 0 0 96 0 -0 0 0 97 0 -0 0 0 98 0 -0 0 0 99 0 -0 0 0 100 0 - -statement ok -drop table annotated_data_finite2; diff --git a/datafusion/sqllogictest/test_files/sort_preserving_merge_repartition.slt b/datafusion/sqllogictest/test_files/sort_preserving_merge_repartition.slt deleted file mode 100644 index dd2f805486b4..000000000000 --- a/datafusion/sqllogictest/test_files/sort_preserving_merge_repartition.slt +++ /dev/null @@ -1,250 +0,0 @@ -# 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. - -# Test smaller batches - -statement ok -set datafusion.execution.target_partitions = 2; - -statement ok -set datafusion.execution.batch_size = 4; - - -statement ok -CREATE EXTERNAL TABLE annotated_data_finite2 ( - a0 INT, - a INT, - b INT, - c INT, - d INT -) -STORED AS CSV -WITH ORDER (a ASC, b ASC, c ASC) -LOCATION '../core/tests/data/low_card_spm_small_batch.csv' -OPTIONS ('format.has_header' 'true'); - - -query TT -EXPLAIN SELECT * -FROM annotated_data_finite2 -WHERE a=0 -ORDER BY b, c; ----- -logical_plan -01)Sort: annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST -02)--Filter: annotated_data_finite2.a = Int32(0) -03)----TableScan: annotated_data_finite2 projection=[a0, a, b, c, d], partial_filters=[annotated_data_finite2.a = Int32(0)] -physical_plan -01)SortPreservingMergeExec: [b@2 ASC NULLS LAST,c@3 ASC NULLS LAST] -02)--CoalesceBatchesExec: target_batch_size=4 -03)----FilterExec: a@1 = 0 -04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/low_card_spm_small_batch.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true - -query IIIII -SELECT * -FROM annotated_data_finite2 -WHERE a=0 -ORDER BY b, c; ----- -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 -0 0 0 1 1 - -statement ok -drop table annotated_data_finite2; diff --git a/datafusion/sqllogictest/test_files/sort_preserving_merge_union.slt b/datafusion/sqllogictest/test_files/sort_preserving_merge_union.slt deleted file mode 100644 index 50f7487cb355..000000000000 --- a/datafusion/sqllogictest/test_files/sort_preserving_merge_union.slt +++ /dev/null @@ -1,452 +0,0 @@ -# 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. - -statement ok -CREATE EXTERNAL TABLE ordered_table ( - a0 INT, - a INT, - b INT, - c INT, - d INT -) -STORED AS CSV -WITH ORDER (c ASC) -LOCATION '../core/tests/data/low_card_spm.csv' -OPTIONS ('format.has_header' 'true'); - -# Test for low cardinality case - -query TT -explain select * from (select b, c, a, a0 from ordered_table) t1 -union all -select * from (select b, c, a, a0 from ordered_table) t2 -order by d, c, a, a0, b -; ----- -logical_plan -01)Projection: t1.b, t1.c, t1.a, t1.a0 -02)--Sort: t1.d ASC NULLS LAST, t1.c ASC NULLS LAST, t1.a ASC NULLS LAST, t1.a0 ASC NULLS LAST, t1.b ASC NULLS LAST -03)----Union -04)------SubqueryAlias: t1 -05)--------Projection: ordered_table.b, ordered_table.c, ordered_table.a, ordered_table.a0, ordered_table.d -06)----------TableScan: ordered_table projection=[a0, a, b, c, d] -07)------SubqueryAlias: t2 -08)--------Projection: ordered_table.b, ordered_table.c, ordered_table.a, ordered_table.a0, ordered_table.d -09)----------TableScan: ordered_table projection=[a0, a, b, c, d] -physical_plan -01)ProjectionExec: expr=[b@0 as b, c@1 as c, a@2 as a, a0@3 as a0] -02)--SortPreservingMergeExec: [d@4 ASC NULLS LAST,c@1 ASC NULLS LAST,a@2 ASC NULLS LAST,a0@3 ASC NULLS LAST,b@0 ASC NULLS LAST] -03)----UnionExec -04)------SortExec: expr=[d@4 ASC NULLS LAST,c@1 ASC NULLS LAST,a@2 ASC NULLS LAST,a0@3 ASC NULLS LAST,b@0 ASC NULLS LAST], preserve_partitioning=[false] -05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/low_card_spm.csv]]}, projection=[b, c, a, a0, d], output_ordering=[c@1 ASC NULLS LAST], has_header=true -06)------SortExec: expr=[d@4 ASC NULLS LAST,c@1 ASC NULLS LAST,a@2 ASC NULLS LAST,a0@3 ASC NULLS LAST,b@0 ASC NULLS LAST], preserve_partitioning=[false] -07)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/low_card_spm.csv]]}, projection=[b, c, a, a0, d], output_ordering=[c@1 ASC NULLS LAST], has_header=true - -query IIII -select * from (select b, c, a, a0 from ordered_table) t1 -union all -select * from (select b, c, a, a0 from ordered_table) t2 -order by d, c, a, a0, b -; ----- -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 - -statement ok -drop table ordered_table; From 625b925839e3c64327202f8914f4ce9198287dc9 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Fri, 18 Oct 2024 13:18:39 +0800 Subject: [PATCH 10/47] cleanup Signed-off-by: jayzhan211 --- benchmarks/queries/spm/queries.sql | 1 - datafusion/physical-plan/src/sorts/merge.rs | 13 ++----------- 2 files changed, 2 insertions(+), 12 deletions(-) delete mode 100644 benchmarks/queries/spm/queries.sql diff --git a/benchmarks/queries/spm/queries.sql b/benchmarks/queries/spm/queries.sql deleted file mode 100644 index c689a66bb441..000000000000 --- a/benchmarks/queries/spm/queries.sql +++ /dev/null @@ -1 +0,0 @@ -SELECT * FROM spm_table WHERE a=0 ORDER BY b, c; \ No newline at end of file diff --git a/datafusion/physical-plan/src/sorts/merge.rs b/datafusion/physical-plan/src/sorts/merge.rs index 5da6f710c393..1bdb9729a46e 100644 --- a/datafusion/physical-plan/src/sorts/merge.rs +++ b/datafusion/physical-plan/src/sorts/merge.rs @@ -97,11 +97,9 @@ pub(crate) struct SortPreservingMergeStream { /// Cursors for each input partition. `None` means the input is exhausted cursors: Vec>>, /// Calculate the number of polled with the same value. - /// For Round robin if the values are tied. + /// We select the one that has less poll counts for tie-breaker in loser tree. num_of_polled_with_same_value: Vec, - /// TODO: Remove it for upstream merge, for demo only - /// partition index, partition polls count - records: (usize, usize), + /// Store previous batch for tracking the poll counts on the same value prev_cursors: Vec>>, /// Optional number of rows to fetch @@ -135,7 +133,6 @@ impl SortPreservingMergeStream { aborted: false, cursors: (0..stream_count).map(|_| None).collect(), prev_cursors: (0..stream_count).map(|_| None).collect(), - records: (0, 0), num_of_polled_with_same_value: vec![0; stream_count], loser_tree: vec![], loser_tree_adjusted: false, @@ -233,12 +230,6 @@ impl SortPreservingMergeStream { let stream_idx = self.loser_tree[0]; if self.advance(stream_idx) { - if stream_idx == self.records.0 { - self.records.1 += 1 - } else { - self.records.1 = 0; - } - println!("records: {:?}", self.records); self.loser_tree_adjusted = false; self.in_progress.push_row(stream_idx); From e4f9bfeadd0b148d37203105f5cc13ae249638ff Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Fri, 18 Oct 2024 13:19:47 +0800 Subject: [PATCH 11/47] cleanup Signed-off-by: jayzhan211 --- datafusion/physical-plan/src/sorts/merge.rs | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/datafusion/physical-plan/src/sorts/merge.rs b/datafusion/physical-plan/src/sorts/merge.rs index 1bdb9729a46e..67e27804c652 100644 --- a/datafusion/physical-plan/src/sorts/merge.rs +++ b/datafusion/physical-plan/src/sorts/merge.rs @@ -296,8 +296,7 @@ impl SortPreservingMergeStream { .then_with(|| number_of_polled_a.cmp(&number_of_polled_b)) .then_with(|| a.cmp(&b)) .is_gt() - } // TODO: remove. Switch to the old strategy to see the difference - // (Some(ac), Some(bc)) => ac.cmp(bc).then_with(|| a.cmp(&b)).is_gt(), + } } } From 3fa2e3264229251216e2773b64f24f93819347ff Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Mon, 21 Oct 2024 11:19:01 +0800 Subject: [PATCH 12/47] update poll count only we have tie Signed-off-by: jayzhan211 --- datafusion/physical-plan/src/sorts/merge.rs | 35 ++++++++++++++++++--- 1 file changed, 31 insertions(+), 4 deletions(-) diff --git a/datafusion/physical-plan/src/sorts/merge.rs b/datafusion/physical-plan/src/sorts/merge.rs index 854514ccf9da..b5fee179e950 100644 --- a/datafusion/physical-plan/src/sorts/merge.rs +++ b/datafusion/physical-plan/src/sorts/merge.rs @@ -103,6 +103,7 @@ pub(crate) struct SortPreservingMergeStream { /// partition index, partition polls count // records: (usize, usize), prev_cursors: Vec>>, + round_robin_tie_breaker_mode: bool, /// Optional number of rows to fetch fetch: Option, @@ -135,6 +136,7 @@ impl SortPreservingMergeStream { aborted: false, cursors: (0..stream_count).map(|_| None).collect(), prev_cursors: (0..stream_count).map(|_| None).collect(), + round_robin_tie_breaker_mode: false, // records: (0, 0), num_of_polled_with_same_value: vec![0; stream_count], loser_tree: vec![], @@ -225,9 +227,6 @@ impl SortPreservingMergeStream { return Poll::Ready(Some(Err(e))); } - // adjust poll count - self.adjuct_poll_count(winner); - self.update_loser_tree(); } @@ -256,7 +255,7 @@ impl SortPreservingMergeStream { } } - fn adjuct_poll_count(&mut self, stream_idx: usize) { + fn adjust_poll_count(&mut self, stream_idx: usize) { let slot = &mut self.cursors[stream_idx]; if let Some(c) = slot.as_mut() { @@ -310,6 +309,15 @@ impl SortPreservingMergeStream { } } + #[inline] + fn is_eq(&self, a: usize, b: usize) -> bool { + match (&self.cursors[a], &self.cursors[b]) { + (None, _) => false, + (_, None) => false, + (Some(ac), Some(bc)) => ac.cmp(bc).is_eq(), + } + } + /// Find the leaf node index in the loser tree for the given cursor index /// /// Note that this is not necessarily a leaf node in the tree, but it can @@ -378,6 +386,25 @@ impl SortPreservingMergeStream { let mut cmp_node = self.lt_leaf_node_index(winner); while cmp_node != 0 { let challenger = self.loser_tree[cmp_node]; + // final match + if cmp_node == 1 { + // Start round robin fasion tie breaker + if self.is_eq(winner, challenger) { + if !self.round_robin_tie_breaker_mode { + self.round_robin_tie_breaker_mode = true; + // Use round_robin_tie_breaker_mode flag to know when to cleanup the polls count in previous tie breaker. + self.num_of_polled_with_same_value.fill(0); + } + // Update poll count only if the winner survive at the final match, otherwise + // it means it has new value and no longer need to compete in the tie breaker. + if winner == self.loser_tree[0] { + self.adjust_poll_count(winner); + } + } else { + self.round_robin_tie_breaker_mode = false; + } + } + if self.is_gt(winner, challenger) { self.loser_tree[cmp_node] = winner; winner = challenger; From e8da793fdaf3bdb1f5f71cda5ccb82628a147e88 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Mon, 21 Oct 2024 13:13:19 +0800 Subject: [PATCH 13/47] upd comment Signed-off-by: jayzhan211 --- datafusion/physical-plan/src/sorts/merge.rs | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/datafusion/physical-plan/src/sorts/merge.rs b/datafusion/physical-plan/src/sorts/merge.rs index b5fee179e950..64858b5b8ae0 100644 --- a/datafusion/physical-plan/src/sorts/merge.rs +++ b/datafusion/physical-plan/src/sorts/merge.rs @@ -388,11 +388,11 @@ impl SortPreservingMergeStream { let challenger = self.loser_tree[cmp_node]; // final match if cmp_node == 1 { - // Start round robin fasion tie breaker + // Value is the same, we need round robin fasion tie breaker based on poll count if self.is_eq(winner, challenger) { if !self.round_robin_tie_breaker_mode { self.round_robin_tie_breaker_mode = true; - // Use round_robin_tie_breaker_mode flag to know when to cleanup the polls count in previous tie breaker. + // Use round_robin_tie_breaker_mode flag to know when to cleanup the poll count in previous tie breaker. self.num_of_polled_with_same_value.fill(0); } // Update poll count only if the winner survive at the final match, otherwise @@ -401,6 +401,7 @@ impl SortPreservingMergeStream { self.adjust_poll_count(winner); } } else { + // Ends of tie breaker self.round_robin_tie_breaker_mode = false; } } From d22ba2508527356c4b683a2ce33d47160e709319 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Mon, 21 Oct 2024 20:22:31 +0800 Subject: [PATCH 14/47] fix logic Signed-off-by: jayzhan211 --- datafusion/physical-plan/src/sorts/merge.rs | 49 +++++++++++++-------- 1 file changed, 30 insertions(+), 19 deletions(-) diff --git a/datafusion/physical-plan/src/sorts/merge.rs b/datafusion/physical-plan/src/sorts/merge.rs index 64858b5b8ae0..f1057e572a27 100644 --- a/datafusion/physical-plan/src/sorts/merge.rs +++ b/datafusion/physical-plan/src/sorts/merge.rs @@ -227,6 +227,7 @@ impl SortPreservingMergeStream { return Poll::Ready(Some(Err(e))); } + // self.adjust_poll_count(winner); self.update_loser_tree(); } @@ -297,27 +298,31 @@ impl SortPreservingMergeStream { match (&self.cursors[a], &self.cursors[b]) { (None, _) => true, (_, None) => false, - (Some(ac), Some(bc)) => { - let number_of_polled_a = self.num_of_polled_with_same_value[a]; - let number_of_polled_b = self.num_of_polled_with_same_value[b]; - ac.cmp(bc) - .then_with(|| number_of_polled_a.cmp(&number_of_polled_b)) - .then_with(|| a.cmp(&b)) - .is_gt() - } // TODO: remove. Switch to the old strategy to see the difference - // (Some(ac), Some(bc)) => ac.cmp(bc).then_with(|| a.cmp(&b)).is_gt(), + (Some(ac), Some(bc)) => ac.cmp(bc).then_with(|| a.cmp(&b)).is_gt(), } } #[inline] fn is_eq(&self, a: usize, b: usize) -> bool { match (&self.cursors[a], &self.cursors[b]) { - (None, _) => false, - (_, None) => false, (Some(ac), Some(bc)) => ac.cmp(bc).is_eq(), + _ => false, } } + #[inline] + fn is_poll_count_gt(&self, a: usize, b: usize) -> bool { + let poll_a = self.num_of_polled_with_same_value[a]; + let poll_b = self.num_of_polled_with_same_value[b]; + poll_a.cmp(&poll_b).then_with(|| a.cmp(&b)).is_gt() + } + + #[inline] + fn update_winner(&mut self, cmp_node: usize, winner: &mut usize, challenger: usize) { + self.loser_tree[cmp_node] = *winner; + *winner = challenger; + } + /// Find the leaf node index in the loser tree for the given cursor index /// /// Note that this is not necessarily a leaf node in the tree, but it can @@ -386,30 +391,36 @@ impl SortPreservingMergeStream { let mut cmp_node = self.lt_leaf_node_index(winner); while cmp_node != 0 { let challenger = self.loser_tree[cmp_node]; - // final match + // Handle final match (cmp_node == 1) and round-robin tie-breaker logic if cmp_node == 1 { - // Value is the same, we need round robin fasion tie breaker based on poll count if self.is_eq(winner, challenger) { if !self.round_robin_tie_breaker_mode { self.round_robin_tie_breaker_mode = true; // Use round_robin_tie_breaker_mode flag to know when to cleanup the poll count in previous tie breaker. self.num_of_polled_with_same_value.fill(0); } - // Update poll count only if the winner survive at the final match, otherwise - // it means it has new value and no longer need to compete in the tie breaker. + // Update poll count if the winner survives in the final match if winner == self.loser_tree[0] { self.adjust_poll_count(winner); + if self.is_poll_count_gt(winner, challenger) { + self.update_winner(cmp_node, &mut winner, challenger); + } + } else if challenger < winner { + // If poll count is not updated, prefer the smaller index + self.update_winner(cmp_node, &mut winner, challenger); } } else { // Ends of tie breaker self.round_robin_tie_breaker_mode = false; + + if self.is_gt(winner, challenger) { + self.update_winner(cmp_node, &mut winner, challenger); + } } + } else if self.is_gt(winner, challenger) { + self.update_winner(cmp_node, &mut winner, challenger); } - if self.is_gt(winner, challenger) { - self.loser_tree[cmp_node] = winner; - winner = challenger; - } cmp_node = self.lt_parent_node_index(cmp_node); } self.loser_tree[0] = winner; From 4b2a4acae9a5163b1be9fae22bcfa616b7fc5c58 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Mon, 21 Oct 2024 20:51:19 +0800 Subject: [PATCH 15/47] configurable Signed-off-by: jayzhan211 --- datafusion/physical-plan/src/sorts/merge.rs | 56 ++++++++++++------- .../src/sorts/streaming_merge.rs | 25 ++++++--- 2 files changed, 53 insertions(+), 28 deletions(-) diff --git a/datafusion/physical-plan/src/sorts/merge.rs b/datafusion/physical-plan/src/sorts/merge.rs index 44f74541df39..a26d9232ac1c 100644 --- a/datafusion/physical-plan/src/sorts/merge.rs +++ b/datafusion/physical-plan/src/sorts/merge.rs @@ -96,11 +96,14 @@ pub(crate) struct SortPreservingMergeStream { /// Cursors for each input partition. `None` means the input is exhausted cursors: Vec>>, + + enable_round_robin_tie_breaker: bool, /// Calculate the number of polled with the same value. /// We select the one that has less poll counts for tie-breaker in loser tree. num_of_polled_with_same_value: Vec, /// Store previous batch for tracking the poll counts on the same value prev_cursors: Vec>>, + /// Flag to indicates whether we are in tie breaker in loser tree round_robin_tie_breaker_mode: bool, /// Optional number of rows to fetch @@ -124,6 +127,7 @@ impl SortPreservingMergeStream { batch_size: usize, fetch: Option, reservation: MemoryReservation, + enable_round_robin_tie_breaker: bool, ) -> Self { let stream_count = streams.partitions(); @@ -135,7 +139,6 @@ impl SortPreservingMergeStream { cursors: (0..stream_count).map(|_| None).collect(), prev_cursors: (0..stream_count).map(|_| None).collect(), round_robin_tie_breaker_mode: false, - // records: (0, 0), num_of_polled_with_same_value: vec![0; stream_count], loser_tree: vec![], loser_tree_adjusted: false, @@ -143,6 +146,7 @@ impl SortPreservingMergeStream { fetch, produced: 0, uninitiated_partitions: (0..stream_count).collect(), + enable_round_robin_tie_breaker, } } @@ -225,7 +229,6 @@ impl SortPreservingMergeStream { return Poll::Ready(Some(Err(e))); } - // self.adjust_poll_count(winner); self.update_loser_tree(); } @@ -376,6 +379,30 @@ impl SortPreservingMergeStream { self.loser_tree_adjusted = true; } + #[inline] + fn handle_tie_breaker( + &mut self, + cmp_node: usize, + winner: &mut usize, + challenger: usize, + ) { + if !self.round_robin_tie_breaker_mode { + self.round_robin_tie_breaker_mode = true; + self.num_of_polled_with_same_value.fill(0); // Reset poll count for tie-breaker + } + + // Update poll count if the winner survives in the final match + if *winner == self.loser_tree[0] { + self.adjust_poll_count(*winner); + if self.is_poll_count_gt(*winner, challenger) { + self.update_winner(cmp_node, winner, challenger); + } + } else if challenger < *winner { + // If poll count is not updated, prefer the smaller index + self.update_winner(cmp_node, winner, challenger); + } + } + /// Attempts to update the loser tree, following winner replacement, if possible fn update_loser_tree(&mut self) { let mut winner = self.loser_tree[0]; @@ -383,28 +410,14 @@ impl SortPreservingMergeStream { let mut cmp_node = self.lt_leaf_node_index(winner); while cmp_node != 0 { let challenger = self.loser_tree[cmp_node]; - // Handle final match (cmp_node == 1) and round-robin tie-breaker logic - if cmp_node == 1 { + + // If round-robin tie breaker is enabled and we're at the final match (cmp_node == 1) + if self.enable_round_robin_tie_breaker && cmp_node == 1 { if self.is_eq(winner, challenger) { - if !self.round_robin_tie_breaker_mode { - self.round_robin_tie_breaker_mode = true; - // Use round_robin_tie_breaker_mode flag to know when to cleanup the poll count in previous tie breaker. - self.num_of_polled_with_same_value.fill(0); - } - // Update poll count if the winner survives in the final match - if winner == self.loser_tree[0] { - self.adjust_poll_count(winner); - if self.is_poll_count_gt(winner, challenger) { - self.update_winner(cmp_node, &mut winner, challenger); - } - } else if challenger < winner { - // If poll count is not updated, prefer the smaller index - self.update_winner(cmp_node, &mut winner, challenger); - } + self.handle_tie_breaker(cmp_node, &mut winner, challenger); } else { - // Ends of tie breaker + // End of tie breaker self.round_robin_tie_breaker_mode = false; - if self.is_gt(winner, challenger) { self.update_winner(cmp_node, &mut winner, challenger); } @@ -415,6 +428,7 @@ impl SortPreservingMergeStream { cmp_node = self.lt_parent_node_index(cmp_node); } + self.loser_tree[0] = winner; self.loser_tree_adjusted = true; } diff --git a/datafusion/physical-plan/src/sorts/streaming_merge.rs b/datafusion/physical-plan/src/sorts/streaming_merge.rs index ad640d8e8470..fd8af835d5de 100644 --- a/datafusion/physical-plan/src/sorts/streaming_merge.rs +++ b/datafusion/physical-plan/src/sorts/streaming_merge.rs @@ -36,7 +36,7 @@ macro_rules! primitive_merge_helper { } macro_rules! merge_helper { - ($t:ty, $sort:ident, $streams:ident, $schema:ident, $tracking_metrics:ident, $batch_size:ident, $fetch:ident, $reservation:ident) => {{ + ($t:ty, $sort:ident, $streams:ident, $schema:ident, $tracking_metrics:ident, $batch_size:ident, $fetch:ident, $reservation:ident, $enable_round_robin_tie_breaker:ident) => {{ let streams = FieldCursorStream::<$t>::new($sort, $streams); return Ok(Box::pin(SortPreservingMergeStream::new( Box::new(streams), @@ -45,6 +45,7 @@ macro_rules! merge_helper { $batch_size, $fetch, $reservation, + $enable_round_robin_tie_breaker, ))); }}; } @@ -58,11 +59,14 @@ pub struct StreamingMergeBuilder<'a> { batch_size: Option, fetch: Option, reservation: Option, + enable_round_robin_tie_breaker: bool, } impl<'a> StreamingMergeBuilder<'a> { pub fn new() -> Self { - Self::default() + let mut builder = Self::default(); + builder.enable_round_robin_tie_breaker = true; + builder } pub fn with_streams(mut self, streams: Vec) -> Self { @@ -100,6 +104,11 @@ impl<'a> StreamingMergeBuilder<'a> { self } + pub fn with_round_robin_tie_breaker(mut self, enable_round_robin_tie_breaker: bool) -> Self { + self.enable_round_robin_tie_breaker = enable_round_robin_tie_breaker; + self + } + pub fn build(self) -> Result { let Self { streams, @@ -109,6 +118,7 @@ impl<'a> StreamingMergeBuilder<'a> { reservation, fetch, expressions, + enable_round_robin_tie_breaker, } = self; // Early return if streams or expressions are empty @@ -141,11 +151,11 @@ impl<'a> StreamingMergeBuilder<'a> { let sort = expressions[0].clone(); let data_type = sort.expr.data_type(schema.as_ref())?; downcast_primitive! { - data_type => (primitive_merge_helper, sort, streams, schema, metrics, batch_size, fetch, reservation), - DataType::Utf8 => merge_helper!(StringArray, sort, streams, schema, metrics, batch_size, fetch, reservation) - DataType::LargeUtf8 => merge_helper!(LargeStringArray, sort, streams, schema, metrics, batch_size, fetch, reservation) - DataType::Binary => merge_helper!(BinaryArray, sort, streams, schema, metrics, batch_size, fetch, reservation) - DataType::LargeBinary => merge_helper!(LargeBinaryArray, sort, streams, schema, metrics, batch_size, fetch, reservation) + data_type => (primitive_merge_helper, sort, streams, schema, metrics, batch_size, fetch, reservation, enable_round_robin_tie_breaker), + DataType::Utf8 => merge_helper!(StringArray, sort, streams, schema, metrics, batch_size, fetch, reservation, enable_round_robin_tie_breaker) + DataType::LargeUtf8 => merge_helper!(LargeStringArray, sort, streams, schema, metrics, batch_size, fetch, reservation, enable_round_robin_tie_breaker) + DataType::Binary => merge_helper!(BinaryArray, sort, streams, schema, metrics, batch_size, fetch, reservation, enable_round_robin_tie_breaker) + DataType::LargeBinary => merge_helper!(LargeBinaryArray, sort, streams, schema, metrics, batch_size, fetch, reservation, enable_round_robin_tie_breaker) _ => {} } } @@ -163,6 +173,7 @@ impl<'a> StreamingMergeBuilder<'a> { batch_size, fetch, reservation, + enable_round_robin_tie_breaker, ))) } } From 920fe6a065918fd4b10ee93af221d6d82c7be910 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Mon, 21 Oct 2024 22:23:08 +0800 Subject: [PATCH 16/47] fmt Signed-off-by: jayzhan211 --- .../physical-plan/src/sorts/streaming_merge.rs | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/datafusion/physical-plan/src/sorts/streaming_merge.rs b/datafusion/physical-plan/src/sorts/streaming_merge.rs index fd8af835d5de..e8330a7cabc0 100644 --- a/datafusion/physical-plan/src/sorts/streaming_merge.rs +++ b/datafusion/physical-plan/src/sorts/streaming_merge.rs @@ -64,9 +64,10 @@ pub struct StreamingMergeBuilder<'a> { impl<'a> StreamingMergeBuilder<'a> { pub fn new() -> Self { - let mut builder = Self::default(); - builder.enable_round_robin_tie_breaker = true; - builder + Self { + enable_round_robin_tie_breaker: true, + ..Default::default() + } } pub fn with_streams(mut self, streams: Vec) -> Self { @@ -104,7 +105,10 @@ impl<'a> StreamingMergeBuilder<'a> { self } - pub fn with_round_robin_tie_breaker(mut self, enable_round_robin_tie_breaker: bool) -> Self { + pub fn with_round_robin_tie_breaker( + mut self, + enable_round_robin_tie_breaker: bool, + ) -> Self { self.enable_round_robin_tie_breaker = enable_round_robin_tie_breaker; self } From d80286cd31cae4d1c436e0ca6615f2ffdff0aa9d Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Tue, 22 Oct 2024 13:56:42 +0800 Subject: [PATCH 17/47] add mem limit test Signed-off-by: jayzhan211 --- datafusion/execution/src/memory_pool/pool.rs | 71 +++++++++++++++- datafusion/execution/src/runtime_env.rs | 18 ++++ .../src/sorts/sort_preserving_merge.rs | 82 +++++++++++++++++++ 3 files changed, 168 insertions(+), 3 deletions(-) diff --git a/datafusion/execution/src/memory_pool/pool.rs b/datafusion/execution/src/memory_pool/pool.rs index e169c1f319cc..78a5096d626b 100644 --- a/datafusion/execution/src/memory_pool/pool.rs +++ b/datafusion/execution/src/memory_pool/pool.rs @@ -22,7 +22,10 @@ use log::debug; use parking_lot::Mutex; use std::{ num::NonZeroUsize, - sync::atomic::{AtomicU64, AtomicUsize, Ordering}, + sync::{ + atomic::{AtomicU64, AtomicUsize, Ordering}, + RwLock, + }, }; /// A [`MemoryPool`] that enforces no limit @@ -58,7 +61,9 @@ impl MemoryPool for UnboundedMemoryPool { #[derive(Debug)] pub struct GreedyMemoryPool { pool_size: usize, + pool_size_per_consumer: HashMap, used: AtomicUsize, + used_per_consumer: RwLock>, } impl GreedyMemoryPool { @@ -67,21 +72,81 @@ impl GreedyMemoryPool { debug!("Created new GreedyMemoryPool(pool_size={pool_size})"); Self { pool_size, + pool_size_per_consumer: Default::default(), used: AtomicUsize::new(0), + used_per_consumer: RwLock::new(HashMap::new()), } } + + pub fn with_pool_size_per_consumer( + mut self, + pool_size_per_consumer: HashMap, + ) -> Self { + self.pool_size_per_consumer = pool_size_per_consumer; + self + } +} + +fn get_prefix(name: &str) -> &str { + name.split('[').next().unwrap_or(name) } impl MemoryPool for GreedyMemoryPool { - fn grow(&self, _reservation: &MemoryReservation, additional: usize) { + fn grow(&self, reservation: &MemoryReservation, additional: usize) { self.used.fetch_add(additional, Ordering::Relaxed); + + let name = reservation.consumer().name(); + let name = get_prefix(name); + let mut used_per_consumer = self.used_per_consumer.write().unwrap(); + let consumer_usage = used_per_consumer + .entry(name.to_string()) + .or_insert_with(|| AtomicUsize::new(0)); + consumer_usage.fetch_add(additional, Ordering::Relaxed); } - fn shrink(&self, _reservation: &MemoryReservation, shrink: usize) { + fn shrink(&self, reservation: &MemoryReservation, shrink: usize) { self.used.fetch_sub(shrink, Ordering::Relaxed); + + let name = reservation.consumer().name(); + let name = get_prefix(name); + let mut used_per_consumer = self.used_per_consumer.write().unwrap(); + let consumer_usage = used_per_consumer + .entry(name.to_string()) + .or_insert_with(|| AtomicUsize::new(0)); + consumer_usage.fetch_sub(shrink, Ordering::Relaxed); } fn try_grow(&self, reservation: &MemoryReservation, additional: usize) -> Result<()> { + let name = reservation.consumer().name(); + let name = get_prefix(name); + + let mut used_per_consumer = self.used_per_consumer.write().unwrap(); + let consumer_usage = used_per_consumer + .entry(name.to_string()) + .or_insert_with(|| AtomicUsize::new(0)); + consumer_usage + .fetch_update(Ordering::Relaxed, Ordering::Relaxed, |used| { + let new_used = used + additional; + let pool_size_per_consumer = self + .pool_size_per_consumer + .get(name) + .cloned() + .unwrap_or(self.pool_size); + + (new_used <= pool_size_per_consumer).then_some(new_used) + }) + .map_err(|used| { + insufficient_capacity_err( + reservation, + additional, + self.pool_size_per_consumer + .get(name) + .cloned() + .unwrap_or(self.pool_size) + .saturating_sub(used), + ) + })?; + self.used .fetch_update(Ordering::Relaxed, Ordering::Relaxed, |used| { let new_used = used + additional; diff --git a/datafusion/execution/src/runtime_env.rs b/datafusion/execution/src/runtime_env.rs index 4022eb07de0c..d7eddaf60ab1 100644 --- a/datafusion/execution/src/runtime_env.rs +++ b/datafusion/execution/src/runtime_env.rs @@ -28,6 +28,7 @@ use crate::{ use crate::cache::cache_manager::{CacheManager, CacheManagerConfig}; use datafusion_common::{DataFusionError, Result}; +use hashbrown::HashMap; use object_store::ObjectStore; use std::path::PathBuf; use std::sync::Arc; @@ -232,6 +233,23 @@ impl RuntimeEnvBuilder { ))) } + /// Set memory limit per consumer, if not set, by default is the same as the total pool size + /// For example, if pool size is 4000, repartition is 3000. Total pool size: 4000, + /// RepartitionExec pool size: 3000, SortPreservingMergeExec pool size: 4000 + pub fn with_memory_limit_per_consumer( + self, + max_memory: usize, + memory_fraction: f64, + pool_size_per_consumer: HashMap, + ) -> Self { + let pool_size = (max_memory as f64 * memory_fraction) as usize; + self.with_memory_pool(Arc::new(TrackConsumersPool::new( + GreedyMemoryPool::new(pool_size) + .with_pool_size_per_consumer(pool_size_per_consumer), + NonZeroUsize::new(5).unwrap(), + ))) + } + /// Use the specified path to create any needed temporary files pub fn with_temp_file_path(self, path: impl Into) -> Self { self.with_disk_manager(DiskManagerConfig::new_specified(vec![path.into()])) diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index 3d3f9dcb98ee..3258e7ad480b 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -82,6 +82,7 @@ pub struct SortPreservingMergeExec { fetch: Option, /// Cache holding plan properties like equivalences, output partitioning etc. cache: PlanProperties, + enable_round_robin_repartition: bool, } impl SortPreservingMergeExec { @@ -94,6 +95,7 @@ impl SortPreservingMergeExec { metrics: ExecutionPlanMetricsSet::new(), fetch: None, cache, + enable_round_robin_repartition: true, } } /// Sets the number of rows to fetch @@ -102,6 +104,14 @@ impl SortPreservingMergeExec { self } + pub fn with_round_robin_repartition( + mut self, + enable_round_robin_repartition: bool, + ) -> Self { + self.enable_round_robin_repartition = enable_round_robin_repartition; + self + } + /// Input schema pub fn input(&self) -> &Arc { &self.input @@ -182,6 +192,7 @@ impl ExecutionPlan for SortPreservingMergeExec { metrics: self.metrics.clone(), fetch: limit, cache: self.cache.clone(), + enable_round_robin_repartition: true, })) } @@ -281,6 +292,7 @@ impl ExecutionPlan for SortPreservingMergeExec { .with_batch_size(context.session_config().batch_size()) .with_fetch(self.fetch) .with_reservation(reservation) + .with_round_robin_tie_breaker(self.enable_round_robin_repartition) .build()?; debug!("Got stream result from SortPreservingMergeStream::new_from_receivers"); @@ -312,10 +324,12 @@ mod tests { use std::time::Duration; use super::*; + use crate::coalesce_batches::CoalesceBatchesExec; use crate::coalesce_partitions::CoalescePartitionsExec; use crate::expressions::col; use crate::memory::MemoryExec; use crate::metrics::{MetricValue, Timestamp}; + use crate::repartition::RepartitionExec; use crate::sorts::sort::SortExec; use crate::stream::RecordBatchReceiverStream; use crate::test::exec::{assert_strong_count_converges_to_zero, BlockingExec}; @@ -326,18 +340,86 @@ mod tests { use arrow::compute::SortOptions; use arrow::datatypes::{DataType, Field, Schema}; use arrow::record_batch::RecordBatch; + use arrow_array::Int64Array; use arrow_schema::SchemaRef; use datafusion_common::{assert_batches_eq, assert_contains, DataFusionError}; use datafusion_common_runtime::SpawnedTask; use datafusion_execution::config::SessionConfig; + use datafusion_execution::runtime_env::RuntimeEnvBuilder; use datafusion_execution::RecordBatchStream; use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::EquivalenceProperties; use datafusion_physical_expr_common::physical_expr::PhysicalExpr; use futures::{FutureExt, Stream, StreamExt}; + use hashbrown::HashMap; use tokio::time::timeout; + fn generate_task_ctx_for_round_robin_tie_breaker() -> Result> { + let mut pool_per_consumer = HashMap::new(); + pool_per_consumer.insert("RepartitionExec".to_string(), 8000); + + let runtime = RuntimeEnvBuilder::new() + .with_memory_limit_per_consumer(40000, 1.0, pool_per_consumer) + .build_arc()?; + let config = SessionConfig::new(); + let task_ctx = TaskContext::default() + .with_runtime(runtime) + .with_session_config(config); + Ok(Arc::new(task_ctx)) + } + fn generate_spm_for_round_robin_tie_breaker( + enable_round_robin_repartition: bool, + ) -> Result> { + let target_batch_size = 4; + let row_size = 32; + let a: ArrayRef = Arc::new(Int32Array::from(vec![1; row_size])); + let b: ArrayRef = Arc::new(StringArray::from_iter(vec![Some("a"); row_size])); + let c: ArrayRef = Arc::new(Int64Array::from_iter(vec![0; row_size])); + let rb = RecordBatch::try_from_iter(vec![("a", a), ("b", b), ("c", c)]).unwrap(); + + let rbs = (0..16).map(|_| rb.clone()).collect::>(); + + let schema = rb.schema(); + let sort = vec![ + PhysicalSortExpr { + expr: col("b", &schema).unwrap(), + options: Default::default(), + }, + PhysicalSortExpr { + expr: col("c", &schema).unwrap(), + options: Default::default(), + }, + ]; + + let exec = MemoryExec::try_new(&[rbs], schema, None).unwrap(); + let repartition_exec = + RepartitionExec::try_new(Arc::new(exec), Partitioning::RoundRobinBatch(2))?; + let coalesce_batches_exec = + CoalesceBatchesExec::new(Arc::new(repartition_exec), target_batch_size); + let spm = SortPreservingMergeExec::new(sort, Arc::new(coalesce_batches_exec)) + .with_round_robin_repartition(enable_round_robin_repartition); + Ok(Arc::new(spm)) + } + + #[tokio::test(flavor = "multi_thread")] + async fn test_round_robin_tie_breaker_success() -> Result<()> { + let task_ctx = generate_task_ctx_for_round_robin_tie_breaker()?; + let spm = generate_spm_for_round_robin_tie_breaker(true)?; + let _collected = collect(spm, task_ctx).await.unwrap(); + + Ok(()) + } + + #[tokio::test(flavor = "multi_thread")] + async fn test_round_robin_tie_breaker_fail() -> Result<()> { + let task_ctx = generate_task_ctx_for_round_robin_tie_breaker()?; + let spm = generate_spm_for_round_robin_tie_breaker(false)?; + let _err = collect(spm, task_ctx).await.unwrap_err(); + + Ok(()) + } + #[tokio::test] async fn test_merge_interleave() { let task_ctx = Arc::new(TaskContext::default()); From 750261ad75d9b9d7ff0277de3f435be197817ad8 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Tue, 22 Oct 2024 15:14:13 +0800 Subject: [PATCH 18/47] rm test Signed-off-by: jayzhan211 --- datafusion/core/tests/data/high_card_spm.csv | 101 --------- datafusion/core/tests/data/low_card_spm.csv | 193 ------------------ .../tests/data/low_card_spm_small_batch.csv | 185 ----------------- datafusion/execution/src/memory_pool/pool.rs | 15 +- .../test_files/sort_preserving_merge.slt | 173 ---------------- 5 files changed, 7 insertions(+), 660 deletions(-) delete mode 100644 datafusion/core/tests/data/high_card_spm.csv delete mode 100644 datafusion/core/tests/data/low_card_spm.csv delete mode 100644 datafusion/core/tests/data/low_card_spm_small_batch.csv delete mode 100644 datafusion/sqllogictest/test_files/sort_preserving_merge.slt diff --git a/datafusion/core/tests/data/high_card_spm.csv b/datafusion/core/tests/data/high_card_spm.csv deleted file mode 100644 index db796cda5b0e..000000000000 --- a/datafusion/core/tests/data/high_card_spm.csv +++ /dev/null @@ -1,101 +0,0 @@ -a0,a,b,c,d -0,0,0,1,0 -0,0,0,2,0 -0,0,0,3,0 -0,0,0,4,0 -0,0,0,5,0 -0,0,0,6,0 -0,0,0,7,0 -0,0,0,8,0 -0,0,0,9,0 -0,0,0,10,0 -0,0,0,11,0 -0,0,0,12,0 -0,0,0,13,0 -0,0,0,14,0 -0,0,0,15,0 -0,0,0,16,0 -0,0,0,17,0 -0,0,0,18,0 -0,0,0,19,0 -0,0,0,20,0 -0,0,0,21,0 -0,0,0,22,0 -0,0,0,23,0 -0,0,0,24,0 -0,0,0,25,0 -0,0,0,26,0 -0,0,0,27,0 -0,0,0,28,0 -0,0,0,29,0 -0,0,0,30,0 -0,0,0,31,0 -0,0,0,32,0 -0,0,0,33,0 -0,0,0,34,0 -0,0,0,35,0 -0,0,0,36,0 -0,0,0,37,0 -0,0,0,38,0 -0,0,0,39,0 -0,0,0,40,0 -0,0,0,41,0 -0,0,0,42,0 -0,0,0,43,0 -0,0,0,44,0 -0,0,0,45,0 -0,0,0,46,0 -0,0,0,47,0 -0,0,0,48,0 -0,0,0,49,0 -0,0,0,50,0 -0,0,0,51,0 -0,0,0,52,0 -0,0,0,53,0 -0,0,0,54,0 -0,0,0,55,0 -0,0,0,56,0 -0,0,0,57,0 -0,0,0,58,0 -0,0,0,59,0 -0,0,0,60,0 -0,0,0,61,0 -0,0,0,62,0 -0,0,0,63,0 -0,0,0,64,0 -0,0,0,65,0 -0,0,0,66,0 -0,0,0,67,0 -0,0,0,68,0 -0,0,0,69,0 -0,0,0,70,0 -0,0,0,71,0 -0,0,0,72,0 -0,0,0,73,0 -0,0,0,74,0 -0,0,0,75,0 -0,0,0,76,0 -0,0,0,77,0 -0,0,0,78,0 -0,0,0,79,0 -0,0,0,80,0 -0,0,0,81,0 -0,0,0,82,0 -0,0,0,83,0 -0,0,0,84,0 -0,0,0,85,0 -0,0,0,86,0 -0,0,0,87,0 -0,0,0,88,0 -0,0,0,89,0 -0,0,0,90,0 -0,0,0,91,0 -0,0,0,92,0 -0,0,0,93,0 -0,0,0,94,0 -0,0,0,95,0 -0,0,0,96,0 -0,0,0,97,0 -0,0,0,98,0 -0,0,0,99,0 -0,0,0,100,0 diff --git a/datafusion/core/tests/data/low_card_spm.csv b/datafusion/core/tests/data/low_card_spm.csv deleted file mode 100644 index 7f5f8ce38159..000000000000 --- a/datafusion/core/tests/data/low_card_spm.csv +++ /dev/null @@ -1,193 +0,0 @@ -a0,a,b,c,d -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 \ No newline at end of file diff --git a/datafusion/core/tests/data/low_card_spm_small_batch.csv b/datafusion/core/tests/data/low_card_spm_small_batch.csv deleted file mode 100644 index fbb90d291210..000000000000 --- a/datafusion/core/tests/data/low_card_spm_small_batch.csv +++ /dev/null @@ -1,185 +0,0 @@ -a0,a,b,c,d -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,0,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 -0,0,0,1,1 \ No newline at end of file diff --git a/datafusion/execution/src/memory_pool/pool.rs b/datafusion/execution/src/memory_pool/pool.rs index 78a5096d626b..f9ae005fb569 100644 --- a/datafusion/execution/src/memory_pool/pool.rs +++ b/datafusion/execution/src/memory_pool/pool.rs @@ -87,7 +87,10 @@ impl GreedyMemoryPool { } } -fn get_prefix(name: &str) -> &str { +/// Consumer name is like RepartitionExec[0] or SortPreservingMergeExec[0], +/// but we don't need partitioned index +fn consumer_name_without_index(reservation: &MemoryReservation) -> &str { + let name = reservation.consumer().name(); name.split('[').next().unwrap_or(name) } @@ -95,8 +98,7 @@ impl MemoryPool for GreedyMemoryPool { fn grow(&self, reservation: &MemoryReservation, additional: usize) { self.used.fetch_add(additional, Ordering::Relaxed); - let name = reservation.consumer().name(); - let name = get_prefix(name); + let name = consumer_name_without_index(reservation); let mut used_per_consumer = self.used_per_consumer.write().unwrap(); let consumer_usage = used_per_consumer .entry(name.to_string()) @@ -107,8 +109,7 @@ impl MemoryPool for GreedyMemoryPool { fn shrink(&self, reservation: &MemoryReservation, shrink: usize) { self.used.fetch_sub(shrink, Ordering::Relaxed); - let name = reservation.consumer().name(); - let name = get_prefix(name); + let name = consumer_name_without_index(reservation); let mut used_per_consumer = self.used_per_consumer.write().unwrap(); let consumer_usage = used_per_consumer .entry(name.to_string()) @@ -117,9 +118,7 @@ impl MemoryPool for GreedyMemoryPool { } fn try_grow(&self, reservation: &MemoryReservation, additional: usize) -> Result<()> { - let name = reservation.consumer().name(); - let name = get_prefix(name); - + let name = consumer_name_without_index(reservation); let mut used_per_consumer = self.used_per_consumer.write().unwrap(); let consumer_usage = used_per_consumer .entry(name.to_string()) diff --git a/datafusion/sqllogictest/test_files/sort_preserving_merge.slt b/datafusion/sqllogictest/test_files/sort_preserving_merge.slt deleted file mode 100644 index d841dcfae857..000000000000 --- a/datafusion/sqllogictest/test_files/sort_preserving_merge.slt +++ /dev/null @@ -1,173 +0,0 @@ -# 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. - -statement ok -CREATE EXTERNAL TABLE ordered_table ( - a0 INT, - a INT, - b INT, - c INT, - d INT -) -STORED AS CSV -WITH ORDER (c ASC) -LOCATION '../core/tests/data/low_card_spm.csv' -OPTIONS ('format.has_header' 'true'); - -# Test for low cardinality case - -query TT -explain select * from (select b, c, a, a0 from ordered_table) t1 -union all -select * from (select b, c, a, a0 from ordered_table) t2 -order by d, c, a, a0, b -limit 4; ----- -logical_plan -01)Projection: t1.b, t1.c, t1.a, t1.a0 -02)--Sort: t1.d ASC NULLS LAST, t1.c ASC NULLS LAST, t1.a ASC NULLS LAST, t1.a0 ASC NULLS LAST, t1.b ASC NULLS LAST, fetch=4 -03)----Union -04)------SubqueryAlias: t1 -05)--------Projection: ordered_table.b, ordered_table.c, ordered_table.a, ordered_table.a0, ordered_table.d -06)----------TableScan: ordered_table projection=[a0, a, b, c, d] -07)------SubqueryAlias: t2 -08)--------Projection: ordered_table.b, ordered_table.c, ordered_table.a, ordered_table.a0, ordered_table.d -09)----------TableScan: ordered_table projection=[a0, a, b, c, d] -physical_plan -01)ProjectionExec: expr=[b@0 as b, c@1 as c, a@2 as a, a0@3 as a0] -02)--SortPreservingMergeExec: [d@4 ASC NULLS LAST,c@1 ASC NULLS LAST,a@2 ASC NULLS LAST,a0@3 ASC NULLS LAST,b@0 ASC NULLS LAST], fetch=4 -03)----UnionExec -04)------SortExec: TopK(fetch=4), expr=[d@4 ASC NULLS LAST,c@1 ASC NULLS LAST,a@2 ASC NULLS LAST,a0@3 ASC NULLS LAST,b@0 ASC NULLS LAST], preserve_partitioning=[false] -05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/low_card_spm.csv]]}, projection=[b, c, a, a0, d], output_ordering=[c@1 ASC NULLS LAST], has_header=true -06)------SortExec: TopK(fetch=4), expr=[d@4 ASC NULLS LAST,c@1 ASC NULLS LAST,a@2 ASC NULLS LAST,a0@3 ASC NULLS LAST,b@0 ASC NULLS LAST], preserve_partitioning=[false] -07)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/low_card_spm.csv]]}, projection=[b, c, a, a0, d], output_ordering=[c@1 ASC NULLS LAST], has_header=true - -query IIII -select * from (select b, c, a, a0 from ordered_table) t1 -union all -select * from (select b, c, a, a0 from ordered_table) t2 -order by d, c, a, a0, b -limit 4; ----- -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 - -statement ok -drop table ordered_table; - -# Test smaller batches - -statement ok -set datafusion.execution.target_partitions = 2; - -statement ok -set datafusion.execution.batch_size = 4; - -statement ok -CREATE EXTERNAL TABLE annotated_data_finite2 ( - a0 INT, - a INT, - b INT, - c INT, - d INT -) -STORED AS CSV -WITH ORDER (a ASC, b ASC, c ASC) -LOCATION '../core/tests/data/low_card_spm_small_batch.csv' -OPTIONS ('format.has_header' 'true'); - - -query TT -EXPLAIN SELECT * -FROM annotated_data_finite2 -WHERE a=0 -ORDER BY b, c -limit 4; ----- -logical_plan -01)Sort: annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST, fetch=4 -02)--Filter: annotated_data_finite2.a = Int32(0) -03)----TableScan: annotated_data_finite2 projection=[a0, a, b, c, d], partial_filters=[annotated_data_finite2.a = Int32(0)] -physical_plan -01)SortPreservingMergeExec: [b@2 ASC NULLS LAST,c@3 ASC NULLS LAST], fetch=4 -02)--CoalesceBatchesExec: target_batch_size=4, fetch=4 -03)----FilterExec: a@1 = 0 -04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/low_card_spm_small_batch.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true - -query IIIII -SELECT * -FROM annotated_data_finite2 -WHERE a=0 -ORDER BY b, c -limit 4; ----- -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 -0 0 0 0 1 - -statement ok -drop table annotated_data_finite2; - -statement ok -CREATE EXTERNAL TABLE annotated_data_finite2 ( - a0 INT, - a INT, - b INT, - c INT, - d INT -) -STORED AS CSV -WITH ORDER (a ASC, b ASC, c ASC) -LOCATION '../core/tests/data/high_card_spm.csv' -OPTIONS ('format.has_header' 'true'); - -query TT -EXPLAIN SELECT * -FROM annotated_data_finite2 -WHERE a=0 -ORDER BY b, c -limit 4; ----- -logical_plan -01)Sort: annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST, fetch=4 -02)--Filter: annotated_data_finite2.a = Int32(0) -03)----TableScan: annotated_data_finite2 projection=[a0, a, b, c, d], partial_filters=[annotated_data_finite2.a = Int32(0)] -physical_plan -01)SortPreservingMergeExec: [b@2 ASC NULLS LAST,c@3 ASC NULLS LAST], fetch=4 -02)--CoalesceBatchesExec: target_batch_size=4, fetch=4 -03)----FilterExec: a@1 = 0 -04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/high_card_spm.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true - -query IIIII -SELECT * -FROM annotated_data_finite2 -WHERE a=0 -ORDER BY b, c -limit 4; ----- -0 0 0 1 0 -0 0 0 2 0 -0 0 0 3 0 -0 0 0 4 0 - -statement ok -drop table annotated_data_finite2; \ No newline at end of file From 79a6df0388a5aad60bbda508ff601b8c5bd04674 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Tue, 22 Oct 2024 15:19:03 +0800 Subject: [PATCH 19/47] escape bracket Signed-off-by: jayzhan211 --- datafusion/execution/src/memory_pool/pool.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/execution/src/memory_pool/pool.rs b/datafusion/execution/src/memory_pool/pool.rs index f9ae005fb569..d285d7213c94 100644 --- a/datafusion/execution/src/memory_pool/pool.rs +++ b/datafusion/execution/src/memory_pool/pool.rs @@ -87,7 +87,7 @@ impl GreedyMemoryPool { } } -/// Consumer name is like RepartitionExec[0] or SortPreservingMergeExec[0], +/// Consumer name is like RepartitionExec\[0\] or SortPreservingMergeExec\[0\], /// but we don't need partitioned index fn consumer_name_without_index(reservation: &MemoryReservation) -> &str { let name = reservation.consumer().name(); From afdd9814b26dda6e8170d6a6752b1f8fb88293f2 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Tue, 22 Oct 2024 20:09:26 +0800 Subject: [PATCH 20/47] add test Signed-off-by: jayzhan211 --- datafusion/execution/src/memory_pool/pool.rs | 19 +++++++- .../src/sorts/sort_preserving_merge.rs | 46 +++++++++++++++++++ 2 files changed, 63 insertions(+), 2 deletions(-) diff --git a/datafusion/execution/src/memory_pool/pool.rs b/datafusion/execution/src/memory_pool/pool.rs index d285d7213c94..c12a63d63191 100644 --- a/datafusion/execution/src/memory_pool/pool.rs +++ b/datafusion/execution/src/memory_pool/pool.rs @@ -97,8 +97,15 @@ fn consumer_name_without_index(reservation: &MemoryReservation) -> &str { impl MemoryPool for GreedyMemoryPool { fn grow(&self, reservation: &MemoryReservation, additional: usize) { self.used.fetch_add(additional, Ordering::Relaxed); - + let s = self.used.load(Ordering::Relaxed); let name = consumer_name_without_index(reservation); + println!( + "name: {} grow to size: {} orin: {}", + name, + s, + reservation.consumer().name() + ); + let mut used_per_consumer = self.used_per_consumer.write().unwrap(); let consumer_usage = used_per_consumer .entry(name.to_string()) @@ -108,8 +115,10 @@ impl MemoryPool for GreedyMemoryPool { fn shrink(&self, reservation: &MemoryReservation, shrink: usize) { self.used.fetch_sub(shrink, Ordering::Relaxed); - + let s = self.used.load(Ordering::Relaxed); let name = consumer_name_without_index(reservation); + println!("name: {} shrink to size: {}", name, s); + let mut used_per_consumer = self.used_per_consumer.write().unwrap(); let consumer_usage = used_per_consumer .entry(name.to_string()) @@ -149,6 +158,12 @@ impl MemoryPool for GreedyMemoryPool { self.used .fetch_update(Ordering::Relaxed, Ordering::Relaxed, |used| { let new_used = used + additional; + println!( + "name: {} new_used: {} origin: {}", + name, + new_used, + reservation.consumer().name() + ); (new_used <= self.pool_size).then_some(new_used) }) .map_err(|used| { diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index 3258e7ad480b..59bed8c514aa 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -420,6 +420,52 @@ mod tests { Ok(()) } + #[tokio::test(flavor = "multi_thread")] + async fn test_round_robin_tie_breaker_v2() -> Result<()> { + let runtime = RuntimeEnvBuilder::new() + .with_memory_limit(135000000, 1.0) + .build_arc()?; + let config = SessionConfig::new(); + let task_ctx = TaskContext::default() + .with_runtime(runtime) + .with_session_config(config); + let task_ctx = Arc::new(task_ctx); + + let target_batch_size = 8192; + let row_size = 8192; + let a: ArrayRef = Arc::new(Int32Array::from(vec![1; row_size])); + let b: ArrayRef = Arc::new(StringArray::from_iter(vec![Some("a"); row_size])); + let c: ArrayRef = Arc::new(Int64Array::from_iter(vec![0; row_size])); + let rb = RecordBatch::try_from_iter(vec![("a", a), ("b", b), ("c", c)]).unwrap(); + + let rbs = (0..1024).map(|_| rb.clone()).collect::>(); + + let schema = rb.schema(); + let sort = vec![ + PhysicalSortExpr { + expr: col("b", &schema).unwrap(), + options: Default::default(), + }, + PhysicalSortExpr { + expr: col("c", &schema).unwrap(), + options: Default::default(), + }, + ]; + + let exec = MemoryExec::try_new(&[rbs], schema, None).unwrap(); + let repartition_exec = + RepartitionExec::try_new(Arc::new(exec), Partitioning::RoundRobinBatch(128))?; + let coalesce_batches_exec = + CoalesceBatchesExec::new(Arc::new(repartition_exec), target_batch_size); + let spm = SortPreservingMergeExec::new(sort, Arc::new(coalesce_batches_exec)) + .with_round_robin_repartition(false); + + let spm = Arc::new(spm); + let _collected = collect(spm, task_ctx).await.unwrap(); + + Ok(()) + } + #[tokio::test] async fn test_merge_interleave() { let task_ctx = Arc::new(TaskContext::default()); From 635884375a3e780a83f61461b69d511f4fb0dab0 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Tue, 22 Oct 2024 20:29:42 +0800 Subject: [PATCH 21/47] rm per consumer record Signed-off-by: jayzhan211 --- datafusion/execution/src/memory_pool/pool.rs | 85 +------------------ datafusion/execution/src/runtime_env.rs | 18 ---- .../src/sorts/sort_preserving_merge.rs | 60 ++----------- 3 files changed, 8 insertions(+), 155 deletions(-) diff --git a/datafusion/execution/src/memory_pool/pool.rs b/datafusion/execution/src/memory_pool/pool.rs index c12a63d63191..e169c1f319cc 100644 --- a/datafusion/execution/src/memory_pool/pool.rs +++ b/datafusion/execution/src/memory_pool/pool.rs @@ -22,10 +22,7 @@ use log::debug; use parking_lot::Mutex; use std::{ num::NonZeroUsize, - sync::{ - atomic::{AtomicU64, AtomicUsize, Ordering}, - RwLock, - }, + sync::atomic::{AtomicU64, AtomicUsize, Ordering}, }; /// A [`MemoryPool`] that enforces no limit @@ -61,9 +58,7 @@ impl MemoryPool for UnboundedMemoryPool { #[derive(Debug)] pub struct GreedyMemoryPool { pool_size: usize, - pool_size_per_consumer: HashMap, used: AtomicUsize, - used_per_consumer: RwLock>, } impl GreedyMemoryPool { @@ -72,98 +67,24 @@ impl GreedyMemoryPool { debug!("Created new GreedyMemoryPool(pool_size={pool_size})"); Self { pool_size, - pool_size_per_consumer: Default::default(), used: AtomicUsize::new(0), - used_per_consumer: RwLock::new(HashMap::new()), } } - - pub fn with_pool_size_per_consumer( - mut self, - pool_size_per_consumer: HashMap, - ) -> Self { - self.pool_size_per_consumer = pool_size_per_consumer; - self - } -} - -/// Consumer name is like RepartitionExec\[0\] or SortPreservingMergeExec\[0\], -/// but we don't need partitioned index -fn consumer_name_without_index(reservation: &MemoryReservation) -> &str { - let name = reservation.consumer().name(); - name.split('[').next().unwrap_or(name) } impl MemoryPool for GreedyMemoryPool { - fn grow(&self, reservation: &MemoryReservation, additional: usize) { + fn grow(&self, _reservation: &MemoryReservation, additional: usize) { self.used.fetch_add(additional, Ordering::Relaxed); - let s = self.used.load(Ordering::Relaxed); - let name = consumer_name_without_index(reservation); - println!( - "name: {} grow to size: {} orin: {}", - name, - s, - reservation.consumer().name() - ); - - let mut used_per_consumer = self.used_per_consumer.write().unwrap(); - let consumer_usage = used_per_consumer - .entry(name.to_string()) - .or_insert_with(|| AtomicUsize::new(0)); - consumer_usage.fetch_add(additional, Ordering::Relaxed); } - fn shrink(&self, reservation: &MemoryReservation, shrink: usize) { + fn shrink(&self, _reservation: &MemoryReservation, shrink: usize) { self.used.fetch_sub(shrink, Ordering::Relaxed); - let s = self.used.load(Ordering::Relaxed); - let name = consumer_name_without_index(reservation); - println!("name: {} shrink to size: {}", name, s); - - let mut used_per_consumer = self.used_per_consumer.write().unwrap(); - let consumer_usage = used_per_consumer - .entry(name.to_string()) - .or_insert_with(|| AtomicUsize::new(0)); - consumer_usage.fetch_sub(shrink, Ordering::Relaxed); } fn try_grow(&self, reservation: &MemoryReservation, additional: usize) -> Result<()> { - let name = consumer_name_without_index(reservation); - let mut used_per_consumer = self.used_per_consumer.write().unwrap(); - let consumer_usage = used_per_consumer - .entry(name.to_string()) - .or_insert_with(|| AtomicUsize::new(0)); - consumer_usage - .fetch_update(Ordering::Relaxed, Ordering::Relaxed, |used| { - let new_used = used + additional; - let pool_size_per_consumer = self - .pool_size_per_consumer - .get(name) - .cloned() - .unwrap_or(self.pool_size); - - (new_used <= pool_size_per_consumer).then_some(new_used) - }) - .map_err(|used| { - insufficient_capacity_err( - reservation, - additional, - self.pool_size_per_consumer - .get(name) - .cloned() - .unwrap_or(self.pool_size) - .saturating_sub(used), - ) - })?; - self.used .fetch_update(Ordering::Relaxed, Ordering::Relaxed, |used| { let new_used = used + additional; - println!( - "name: {} new_used: {} origin: {}", - name, - new_used, - reservation.consumer().name() - ); (new_used <= self.pool_size).then_some(new_used) }) .map_err(|used| { diff --git a/datafusion/execution/src/runtime_env.rs b/datafusion/execution/src/runtime_env.rs index d7eddaf60ab1..4022eb07de0c 100644 --- a/datafusion/execution/src/runtime_env.rs +++ b/datafusion/execution/src/runtime_env.rs @@ -28,7 +28,6 @@ use crate::{ use crate::cache::cache_manager::{CacheManager, CacheManagerConfig}; use datafusion_common::{DataFusionError, Result}; -use hashbrown::HashMap; use object_store::ObjectStore; use std::path::PathBuf; use std::sync::Arc; @@ -233,23 +232,6 @@ impl RuntimeEnvBuilder { ))) } - /// Set memory limit per consumer, if not set, by default is the same as the total pool size - /// For example, if pool size is 4000, repartition is 3000. Total pool size: 4000, - /// RepartitionExec pool size: 3000, SortPreservingMergeExec pool size: 4000 - pub fn with_memory_limit_per_consumer( - self, - max_memory: usize, - memory_fraction: f64, - pool_size_per_consumer: HashMap, - ) -> Self { - let pool_size = (max_memory as f64 * memory_fraction) as usize; - self.with_memory_pool(Arc::new(TrackConsumersPool::new( - GreedyMemoryPool::new(pool_size) - .with_pool_size_per_consumer(pool_size_per_consumer), - NonZeroUsize::new(5).unwrap(), - ))) - } - /// Use the specified path to create any needed temporary files pub fn with_temp_file_path(self, path: impl Into) -> Self { self.with_disk_manager(DiskManagerConfig::new_specified(vec![path.into()])) diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index 59bed8c514aa..2dcd16bf0a6b 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -352,15 +352,11 @@ mod tests { use datafusion_physical_expr_common::physical_expr::PhysicalExpr; use futures::{FutureExt, Stream, StreamExt}; - use hashbrown::HashMap; use tokio::time::timeout; fn generate_task_ctx_for_round_robin_tie_breaker() -> Result> { - let mut pool_per_consumer = HashMap::new(); - pool_per_consumer.insert("RepartitionExec".to_string(), 8000); - let runtime = RuntimeEnvBuilder::new() - .with_memory_limit_per_consumer(40000, 1.0, pool_per_consumer) + .with_memory_limit(135000000, 1.0) .build_arc()?; let config = SessionConfig::new(); let task_ctx = TaskContext::default() @@ -371,14 +367,14 @@ mod tests { fn generate_spm_for_round_robin_tie_breaker( enable_round_robin_repartition: bool, ) -> Result> { - let target_batch_size = 4; - let row_size = 32; + let target_batch_size = 8192; + let row_size = 8192; let a: ArrayRef = Arc::new(Int32Array::from(vec![1; row_size])); let b: ArrayRef = Arc::new(StringArray::from_iter(vec![Some("a"); row_size])); let c: ArrayRef = Arc::new(Int64Array::from_iter(vec![0; row_size])); let rb = RecordBatch::try_from_iter(vec![("a", a), ("b", b), ("c", c)]).unwrap(); - let rbs = (0..16).map(|_| rb.clone()).collect::>(); + let rbs = (0..1024).map(|_| rb.clone()).collect::>(); let schema = rb.schema(); let sort = vec![ @@ -394,7 +390,7 @@ mod tests { let exec = MemoryExec::try_new(&[rbs], schema, None).unwrap(); let repartition_exec = - RepartitionExec::try_new(Arc::new(exec), Partitioning::RoundRobinBatch(2))?; + RepartitionExec::try_new(Arc::new(exec), Partitioning::RoundRobinBatch(128))?; let coalesce_batches_exec = CoalesceBatchesExec::new(Arc::new(repartition_exec), target_batch_size); let spm = SortPreservingMergeExec::new(sort, Arc::new(coalesce_batches_exec)) @@ -420,52 +416,6 @@ mod tests { Ok(()) } - #[tokio::test(flavor = "multi_thread")] - async fn test_round_robin_tie_breaker_v2() -> Result<()> { - let runtime = RuntimeEnvBuilder::new() - .with_memory_limit(135000000, 1.0) - .build_arc()?; - let config = SessionConfig::new(); - let task_ctx = TaskContext::default() - .with_runtime(runtime) - .with_session_config(config); - let task_ctx = Arc::new(task_ctx); - - let target_batch_size = 8192; - let row_size = 8192; - let a: ArrayRef = Arc::new(Int32Array::from(vec![1; row_size])); - let b: ArrayRef = Arc::new(StringArray::from_iter(vec![Some("a"); row_size])); - let c: ArrayRef = Arc::new(Int64Array::from_iter(vec![0; row_size])); - let rb = RecordBatch::try_from_iter(vec![("a", a), ("b", b), ("c", c)]).unwrap(); - - let rbs = (0..1024).map(|_| rb.clone()).collect::>(); - - let schema = rb.schema(); - let sort = vec![ - PhysicalSortExpr { - expr: col("b", &schema).unwrap(), - options: Default::default(), - }, - PhysicalSortExpr { - expr: col("c", &schema).unwrap(), - options: Default::default(), - }, - ]; - - let exec = MemoryExec::try_new(&[rbs], schema, None).unwrap(); - let repartition_exec = - RepartitionExec::try_new(Arc::new(exec), Partitioning::RoundRobinBatch(128))?; - let coalesce_batches_exec = - CoalesceBatchesExec::new(Arc::new(repartition_exec), target_batch_size); - let spm = SortPreservingMergeExec::new(sort, Arc::new(coalesce_batches_exec)) - .with_round_robin_repartition(false); - - let spm = Arc::new(spm); - let _collected = collect(spm, task_ctx).await.unwrap(); - - Ok(()) - } - #[tokio::test] async fn test_merge_interleave() { let task_ctx = Arc::new(TaskContext::default()); From c9337c67cc8d6643145affdef4ab22ad04865742 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Wed, 23 Oct 2024 21:05:12 +0800 Subject: [PATCH 22/47] repartition limit Signed-off-by: jayzhan211 --- datafusion/execution/src/memory_pool/pool.rs | 63 ++++++++++++++++++- datafusion/execution/src/runtime_env.rs | 18 ++++++ .../src/sorts/sort_preserving_merge.rs | 15 +++-- 3 files changed, 89 insertions(+), 7 deletions(-) diff --git a/datafusion/execution/src/memory_pool/pool.rs b/datafusion/execution/src/memory_pool/pool.rs index e169c1f319cc..f6205d4661fd 100644 --- a/datafusion/execution/src/memory_pool/pool.rs +++ b/datafusion/execution/src/memory_pool/pool.rs @@ -19,7 +19,7 @@ use crate::memory_pool::{MemoryConsumer, MemoryPool, MemoryReservation}; use datafusion_common::{resources_datafusion_err, DataFusionError, Result}; use hashbrown::HashMap; use log::debug; -use parking_lot::Mutex; +use parking_lot::{Mutex, RwLock}; use std::{ num::NonZeroUsize, sync::atomic::{AtomicU64, AtomicUsize, Ordering}, @@ -58,7 +58,9 @@ impl MemoryPool for UnboundedMemoryPool { #[derive(Debug)] pub struct GreedyMemoryPool { pool_size: usize, + pool_size_per_consumer: HashMap, used: AtomicUsize, + used_per_consumer: RwLock>, } impl GreedyMemoryPool { @@ -67,21 +69,76 @@ impl GreedyMemoryPool { debug!("Created new GreedyMemoryPool(pool_size={pool_size})"); Self { pool_size, + pool_size_per_consumer: Default::default(), used: AtomicUsize::new(0), + used_per_consumer: RwLock::new(HashMap::new()), } } + + pub fn with_pool_size_per_consumer( + mut self, + pool_size_per_consumer: HashMap, + ) -> Self { + self.pool_size_per_consumer = pool_size_per_consumer; + self + } } impl MemoryPool for GreedyMemoryPool { - fn grow(&self, _reservation: &MemoryReservation, additional: usize) { + fn grow(&self, reservation: &MemoryReservation, additional: usize) { + let consumer_name = reservation.consumer().name(); self.used.fetch_add(additional, Ordering::Relaxed); + + let mut used_per_consumer = self.used_per_consumer.write(); + let consumer_usage = used_per_consumer + .entry(consumer_name.to_string()) + .or_insert_with(|| AtomicUsize::new(0)); + consumer_usage.fetch_add(additional, Ordering::Relaxed); } - fn shrink(&self, _reservation: &MemoryReservation, shrink: usize) { + fn shrink(&self, reservation: &MemoryReservation, shrink: usize) { + let consumer_name = reservation.consumer().name(); + self.used.fetch_sub(shrink, Ordering::Relaxed); + + let mut used_per_consumer = self.used_per_consumer.write(); + let consumer_usage = used_per_consumer + .entry(consumer_name.to_string()) + .or_insert_with(|| AtomicUsize::new(0)); + consumer_usage.fetch_sub(shrink, Ordering::Relaxed); } fn try_grow(&self, reservation: &MemoryReservation, additional: usize) -> Result<()> { + let consumer_name = reservation.consumer().name(); + + let mut used_per_consumer = self.used_per_consumer.write(); + let consumer_usage = used_per_consumer + .entry(consumer_name.to_string()) + .or_insert_with(|| AtomicUsize::new(0)); + consumer_usage + .fetch_update(Ordering::Relaxed, Ordering::Relaxed, |used| { + let new_used = used + additional; + + let pool_size_per_consumer = self + .pool_size_per_consumer + .get(consumer_name) + .cloned() + .unwrap_or(self.pool_size); + + (new_used <= pool_size_per_consumer).then_some(new_used) + }) + .map_err(|used| { + insufficient_capacity_err( + reservation, + additional, + self.pool_size_per_consumer + .get(consumer_name) + .cloned() + .unwrap_or(self.pool_size) + .saturating_sub(used), + ) + })?; + self.used .fetch_update(Ordering::Relaxed, Ordering::Relaxed, |used| { let new_used = used + additional; diff --git a/datafusion/execution/src/runtime_env.rs b/datafusion/execution/src/runtime_env.rs index 4022eb07de0c..d7eddaf60ab1 100644 --- a/datafusion/execution/src/runtime_env.rs +++ b/datafusion/execution/src/runtime_env.rs @@ -28,6 +28,7 @@ use crate::{ use crate::cache::cache_manager::{CacheManager, CacheManagerConfig}; use datafusion_common::{DataFusionError, Result}; +use hashbrown::HashMap; use object_store::ObjectStore; use std::path::PathBuf; use std::sync::Arc; @@ -232,6 +233,23 @@ impl RuntimeEnvBuilder { ))) } + /// Set memory limit per consumer, if not set, by default is the same as the total pool size + /// For example, if pool size is 4000, repartition is 3000. Total pool size: 4000, + /// RepartitionExec pool size: 3000, SortPreservingMergeExec pool size: 4000 + pub fn with_memory_limit_per_consumer( + self, + max_memory: usize, + memory_fraction: f64, + pool_size_per_consumer: HashMap, + ) -> Self { + let pool_size = (max_memory as f64 * memory_fraction) as usize; + self.with_memory_pool(Arc::new(TrackConsumersPool::new( + GreedyMemoryPool::new(pool_size) + .with_pool_size_per_consumer(pool_size_per_consumer), + NonZeroUsize::new(5).unwrap(), + ))) + } + /// Use the specified path to create any needed temporary files pub fn with_temp_file_path(self, path: impl Into) -> Self { self.with_disk_manager(DiskManagerConfig::new_specified(vec![path.into()])) diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index 2dcd16bf0a6b..f7cd67a09b88 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -352,11 +352,18 @@ mod tests { use datafusion_physical_expr_common::physical_expr::PhysicalExpr; use futures::{FutureExt, Stream, StreamExt}; + use hashbrown::HashMap; use tokio::time::timeout; fn generate_task_ctx_for_round_robin_tie_breaker() -> Result> { + let mut pool_per_consumer = HashMap::new(); + // Number from 660000 to 30000000 (or even more) are all valid limit + pool_per_consumer.insert("RepartitionExec[0]".to_string(), 10000000); + pool_per_consumer.insert("RepartitionExec[1]".to_string(), 10000000); + let runtime = RuntimeEnvBuilder::new() - .with_memory_limit(135000000, 1.0) + // random large number for total mem limit, we care about RepartitionExec only + .with_memory_limit_per_consumer(2000000000, 1.0, pool_per_consumer) .build_arc()?; let config = SessionConfig::new(); let task_ctx = TaskContext::default() @@ -367,8 +374,8 @@ mod tests { fn generate_spm_for_round_robin_tie_breaker( enable_round_robin_repartition: bool, ) -> Result> { - let target_batch_size = 8192; - let row_size = 8192; + let target_batch_size = 12500; + let row_size = 12500; let a: ArrayRef = Arc::new(Int32Array::from(vec![1; row_size])); let b: ArrayRef = Arc::new(StringArray::from_iter(vec![Some("a"); row_size])); let c: ArrayRef = Arc::new(Int64Array::from_iter(vec![0; row_size])); @@ -390,7 +397,7 @@ mod tests { let exec = MemoryExec::try_new(&[rbs], schema, None).unwrap(); let repartition_exec = - RepartitionExec::try_new(Arc::new(exec), Partitioning::RoundRobinBatch(128))?; + RepartitionExec::try_new(Arc::new(exec), Partitioning::RoundRobinBatch(2))?; let coalesce_batches_exec = CoalesceBatchesExec::new(Arc::new(repartition_exec), target_batch_size); let spm = SortPreservingMergeExec::new(sort, Arc::new(coalesce_batches_exec)) From bc816813d202779b865cd76d31783c94a43b709d Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Thu, 24 Oct 2024 21:44:10 +0800 Subject: [PATCH 23/47] add benchmark Signed-off-by: jayzhan211 --- datafusion/physical-plan/Cargo.toml | 5 + datafusion/physical-plan/benches/spm.rs | 132 ++++++++++++++++++++++++ 2 files changed, 137 insertions(+) create mode 100644 datafusion/physical-plan/benches/spm.rs diff --git a/datafusion/physical-plan/Cargo.toml b/datafusion/physical-plan/Cargo.toml index 7fcd719539ec..d703f54986cd 100644 --- a/datafusion/physical-plan/Cargo.toml +++ b/datafusion/physical-plan/Cargo.toml @@ -68,6 +68,7 @@ rand = { workspace = true } tokio = { workspace = true } [dev-dependencies] +criterion = { version = "0.5", features = ["async_futures"] } datafusion-functions-aggregate = { workspace = true } rstest = { workspace = true } rstest_reuse = "0.7.0" @@ -76,3 +77,7 @@ tokio = { workspace = true, features = [ "fs", "parking_lot", ] } + +[[bench]] +harness = false +name = "spm" \ No newline at end of file diff --git a/datafusion/physical-plan/benches/spm.rs b/datafusion/physical-plan/benches/spm.rs new file mode 100644 index 000000000000..5206bf3fa2be --- /dev/null +++ b/datafusion/physical-plan/benches/spm.rs @@ -0,0 +1,132 @@ +// 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 arrow::record_batch::RecordBatch; +use arrow_array::{ArrayRef, Int32Array, Int64Array, StringArray}; +use criterion::async_executor::FuturesExecutor; +use criterion::{black_box, criterion_group, criterion_main, Criterion}; +use datafusion_execution::TaskContext; +use datafusion_physical_expr::expressions::col; +use datafusion_physical_expr::PhysicalSortExpr; +use datafusion_physical_plan::memory::MemoryExec; +use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; +use datafusion_physical_plan::{collect, ExecutionPlan}; +use std::sync::Arc; + +fn generate_spm_for_round_robin_tie_breaker( + has_same_value: bool, + enable_round_robin_repartition: bool, +) -> SortPreservingMergeExec { + let row_size = 256; + let rb = if has_same_value { + let a: ArrayRef = Arc::new(Int32Array::from(vec![1; row_size])); + let b: ArrayRef = Arc::new(StringArray::from_iter(vec![Some("a"); row_size])); + let c: ArrayRef = Arc::new(Int64Array::from_iter(vec![0; row_size])); + RecordBatch::try_from_iter(vec![("a", a), ("b", b), ("c", c)]).unwrap() + } else { + let v = (0i32..row_size as i32).collect::>(); + let a: ArrayRef = Arc::new(Int32Array::from(v)); + + // Use alphanumeric characters + let charset: Vec = + "abcdefghijklmnopqrstuvwxyzABCDEFGHIJKLMNOPQRSTUVWXYZ0123456789" + .chars() + .collect(); + + let mut strings = Vec::new(); + // Create 256 unique strings + for i in 0..256 { + let mut s = String::new(); + s.push(charset[i % charset.len()]); // Use modulo to wrap around + s.push(charset[(i / charset.len()) % charset.len()]); + strings.push(Some(s)); + } + + let b: ArrayRef = Arc::new(StringArray::from_iter(strings)); + + let v = (0i64..row_size as i64).collect::>(); + let c: ArrayRef = Arc::new(Int64Array::from_iter(v)); + RecordBatch::try_from_iter(vec![("a", a), ("b", b), ("c", c)]).unwrap() + }; + + let rbs = (0..16).map(|_| rb.clone()).collect::>(); + // Ensure this to be larger than 2 so it goes to the SPM code path + let partitiones = vec![rbs.clone(); 2]; + + let schema = rb.schema(); + let sort = vec![ + PhysicalSortExpr { + expr: col("b", &schema).unwrap(), + options: Default::default(), + }, + PhysicalSortExpr { + expr: col("c", &schema).unwrap(), + options: Default::default(), + }, + ]; + + let exec = MemoryExec::try_new(&partitiones, schema, None).unwrap(); + SortPreservingMergeExec::new(sort, Arc::new(exec)) + .with_round_robin_repartition(enable_round_robin_repartition) +} + +fn criterion_benchmark(c: &mut Criterion) { + // create input data + let task_ctx = TaskContext::default(); + let task_ctx = Arc::new(task_ctx); + + let spm_wo_tb = Arc::new(generate_spm_for_round_robin_tie_breaker(true, false)) + as Arc; + c.bench_function("spm without tie breaker low card", |b| { + b.to_async(FuturesExecutor).iter(|| { + // Ensure the data is deep cloned to avoid mixing them + black_box(collect(Arc::clone(&spm_wo_tb), Arc::clone(&task_ctx))) + }) + }); + + let spm_w_tb = Arc::new(generate_spm_for_round_robin_tie_breaker(true, true)) + as Arc; + c.bench_function("spm with tie breaker low card", |b| { + b.to_async(FuturesExecutor).iter(|| { + // Ensure the data is deep cloned to avoid mixing them + // let spm_w_tb = generate_spm_for_round_robin_tie_breaker(true); + black_box(collect(Arc::clone(&spm_w_tb), Arc::clone(&task_ctx))) + }) + }); + + let spm_wo_tb = Arc::new(generate_spm_for_round_robin_tie_breaker(false, false)) + as Arc; + c.bench_function("spm without tie breaker high card", |b| { + b.to_async(FuturesExecutor).iter(|| { + // Ensure the data is deep cloned to avoid mixing them + black_box(collect(Arc::clone(&spm_wo_tb), Arc::clone(&task_ctx))) + }) + }); + + let spm_w_tb = Arc::new(generate_spm_for_round_robin_tie_breaker(false, true)) + as Arc; + c.bench_function("spm with tie breaker high card", |b| { + b.to_async(FuturesExecutor).iter(|| { + // Ensure the data is deep cloned to avoid mixing them + // let spm_w_tb = generate_spm_for_round_robin_tie_breaker(true); + black_box(collect(Arc::clone(&spm_w_tb), Arc::clone(&task_ctx))) + }) + }); +} + +criterion_group!(benches, criterion_benchmark); +criterion_main!(benches); From e4970f5f531c88d23a0033a86762eff396637617 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Thu, 24 Oct 2024 21:47:05 +0800 Subject: [PATCH 24/47] cleanup Signed-off-by: jayzhan211 --- datafusion/physical-plan/benches/spm.rs | 8 +------- 1 file changed, 1 insertion(+), 7 deletions(-) diff --git a/datafusion/physical-plan/benches/spm.rs b/datafusion/physical-plan/benches/spm.rs index 5206bf3fa2be..6684d3aafcbc 100644 --- a/datafusion/physical-plan/benches/spm.rs +++ b/datafusion/physical-plan/benches/spm.rs @@ -51,7 +51,7 @@ fn generate_spm_for_round_robin_tie_breaker( // Create 256 unique strings for i in 0..256 { let mut s = String::new(); - s.push(charset[i % charset.len()]); // Use modulo to wrap around + s.push(charset[i % charset.len()]); s.push(charset[(i / charset.len()) % charset.len()]); strings.push(Some(s)); } @@ -93,7 +93,6 @@ fn criterion_benchmark(c: &mut Criterion) { as Arc; c.bench_function("spm without tie breaker low card", |b| { b.to_async(FuturesExecutor).iter(|| { - // Ensure the data is deep cloned to avoid mixing them black_box(collect(Arc::clone(&spm_wo_tb), Arc::clone(&task_ctx))) }) }); @@ -102,8 +101,6 @@ fn criterion_benchmark(c: &mut Criterion) { as Arc; c.bench_function("spm with tie breaker low card", |b| { b.to_async(FuturesExecutor).iter(|| { - // Ensure the data is deep cloned to avoid mixing them - // let spm_w_tb = generate_spm_for_round_robin_tie_breaker(true); black_box(collect(Arc::clone(&spm_w_tb), Arc::clone(&task_ctx))) }) }); @@ -112,7 +109,6 @@ fn criterion_benchmark(c: &mut Criterion) { as Arc; c.bench_function("spm without tie breaker high card", |b| { b.to_async(FuturesExecutor).iter(|| { - // Ensure the data is deep cloned to avoid mixing them black_box(collect(Arc::clone(&spm_wo_tb), Arc::clone(&task_ctx))) }) }); @@ -121,8 +117,6 @@ fn criterion_benchmark(c: &mut Criterion) { as Arc; c.bench_function("spm with tie breaker high card", |b| { b.to_async(FuturesExecutor).iter(|| { - // Ensure the data is deep cloned to avoid mixing them - // let spm_w_tb = generate_spm_for_round_robin_tie_breaker(true); black_box(collect(Arc::clone(&spm_w_tb), Arc::clone(&task_ctx))) }) }); From e7abf680691ac05cbeaa28fab32628632e9fed8f Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Fri, 25 Oct 2024 06:51:28 +0800 Subject: [PATCH 25/47] benchmark with parameter Signed-off-by: jayzhan211 --- datafusion/physical-plan/benches/spm.rs | 93 +++++++++++++++---------- 1 file changed, 56 insertions(+), 37 deletions(-) diff --git a/datafusion/physical-plan/benches/spm.rs b/datafusion/physical-plan/benches/spm.rs index 6684d3aafcbc..624141234247 100644 --- a/datafusion/physical-plan/benches/spm.rs +++ b/datafusion/physical-plan/benches/spm.rs @@ -15,21 +15,25 @@ // specific language governing permissions and limitations // under the License. +use std::sync::Arc; + use arrow::record_batch::RecordBatch; use arrow_array::{ArrayRef, Int32Array, Int64Array, StringArray}; -use criterion::async_executor::FuturesExecutor; -use criterion::{black_box, criterion_group, criterion_main, Criterion}; use datafusion_execution::TaskContext; use datafusion_physical_expr::expressions::col; use datafusion_physical_expr::PhysicalSortExpr; use datafusion_physical_plan::memory::MemoryExec; use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; use datafusion_physical_plan::{collect, ExecutionPlan}; -use std::sync::Arc; + +use criterion::async_executor::FuturesExecutor; +use criterion::{black_box, criterion_group, criterion_main, Criterion}; fn generate_spm_for_round_robin_tie_breaker( has_same_value: bool, enable_round_robin_repartition: bool, + batch_count: usize, + partition_count: usize, ) -> SortPreservingMergeExec { let row_size = 256; let rb = if has_same_value { @@ -48,7 +52,6 @@ fn generate_spm_for_round_robin_tie_breaker( .collect(); let mut strings = Vec::new(); - // Create 256 unique strings for i in 0..256 { let mut s = String::new(); s.push(charset[i % charset.len()]); @@ -63,9 +66,8 @@ fn generate_spm_for_round_robin_tie_breaker( RecordBatch::try_from_iter(vec![("a", a), ("b", b), ("c", c)]).unwrap() }; - let rbs = (0..16).map(|_| rb.clone()).collect::>(); - // Ensure this to be larger than 2 so it goes to the SPM code path - let partitiones = vec![rbs.clone(); 2]; + let rbs = (0..batch_count).map(|_| rb.clone()).collect::>(); + let partitiones = vec![rbs.clone(); partition_count]; let schema = rb.schema(); let sort = vec![ @@ -84,43 +86,60 @@ fn generate_spm_for_round_robin_tie_breaker( .with_round_robin_repartition(enable_round_robin_repartition) } -fn criterion_benchmark(c: &mut Criterion) { - // create input data +fn run_bench( + c: &mut Criterion, + has_same_value: bool, + enable_round_robin_repartition: bool, + batch_count: usize, + partition_count: usize, + description: &str, +) { let task_ctx = TaskContext::default(); let task_ctx = Arc::new(task_ctx); - let spm_wo_tb = Arc::new(generate_spm_for_round_robin_tie_breaker(true, false)) - as Arc; - c.bench_function("spm without tie breaker low card", |b| { - b.to_async(FuturesExecutor).iter(|| { - black_box(collect(Arc::clone(&spm_wo_tb), Arc::clone(&task_ctx))) - }) - }); + let spm = Arc::new(generate_spm_for_round_robin_tie_breaker( + has_same_value, + enable_round_robin_repartition, + batch_count, + partition_count, + )) as Arc; - let spm_w_tb = Arc::new(generate_spm_for_round_robin_tie_breaker(true, true)) - as Arc; - c.bench_function("spm with tie breaker low card", |b| { - b.to_async(FuturesExecutor).iter(|| { - black_box(collect(Arc::clone(&spm_w_tb), Arc::clone(&task_ctx))) - }) + c.bench_function(description, |b| { + b.to_async(FuturesExecutor) + .iter(|| black_box(collect(Arc::clone(&spm), Arc::clone(&task_ctx)))) }); +} - let spm_wo_tb = Arc::new(generate_spm_for_round_robin_tie_breaker(false, false)) - as Arc; - c.bench_function("spm without tie breaker high card", |b| { - b.to_async(FuturesExecutor).iter(|| { - black_box(collect(Arc::clone(&spm_wo_tb), Arc::clone(&task_ctx))) - }) - }); +fn criterion_benchmark(c: &mut Criterion) { + let params = [ + (true, false, "low_card_without_tiebreaker"), // low cardinality, no tie breaker + (true, true, "low_card_with_tiebreaker"), // low cardinality, with tie breaker + (false, false, "high_card_without_tiebreaker"), // high cardinality, no tie breaker + (false, true, "high_card_with_tiebreaker"), // high cardinality, with tie breaker + ]; - let spm_w_tb = Arc::new(generate_spm_for_round_robin_tie_breaker(false, true)) - as Arc; - c.bench_function("spm with tie breaker high card", |b| { - b.to_async(FuturesExecutor).iter(|| { - black_box(collect(Arc::clone(&spm_w_tb), Arc::clone(&task_ctx))) - }) - }); + let batch_counts = [1, 25, 625]; + let partition_counts = [2, 8, 32]; + + for &(has_same_value, enable_round_robin_repartition, cardinality_label) in ¶ms { + for &batch_count in &batch_counts { + for &partition_count in &partition_counts { + let description = format!( + "{}_batch_count_{}_partition_count_{}", + cardinality_label, batch_count, partition_count + ); + run_bench( + c, + has_same_value, + enable_round_robin_repartition, + batch_count, + partition_count, + &description, + ); + } + } + } } criterion_group!(benches, criterion_benchmark); -criterion_main!(benches); +criterion_main!(benches); \ No newline at end of file From 7cb119898e1ba3924d00ac526c1f4dc11677a911 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Fri, 25 Oct 2024 07:16:40 +0800 Subject: [PATCH 26/47] only calculate consumer pool if the limit is set Signed-off-by: jayzhan211 --- datafusion/execution/src/memory_pool/pool.rs | 45 ++++++++------------ datafusion/physical-plan/Cargo.toml | 2 +- datafusion/physical-plan/benches/spm.rs | 2 +- 3 files changed, 20 insertions(+), 29 deletions(-) diff --git a/datafusion/execution/src/memory_pool/pool.rs b/datafusion/execution/src/memory_pool/pool.rs index f6205d4661fd..27d9357fca6a 100644 --- a/datafusion/execution/src/memory_pool/pool.rs +++ b/datafusion/execution/src/memory_pool/pool.rs @@ -111,33 +111,24 @@ impl MemoryPool for GreedyMemoryPool { fn try_grow(&self, reservation: &MemoryReservation, additional: usize) -> Result<()> { let consumer_name = reservation.consumer().name(); - let mut used_per_consumer = self.used_per_consumer.write(); - let consumer_usage = used_per_consumer - .entry(consumer_name.to_string()) - .or_insert_with(|| AtomicUsize::new(0)); - consumer_usage - .fetch_update(Ordering::Relaxed, Ordering::Relaxed, |used| { - let new_used = used + additional; - - let pool_size_per_consumer = self - .pool_size_per_consumer - .get(consumer_name) - .cloned() - .unwrap_or(self.pool_size); - - (new_used <= pool_size_per_consumer).then_some(new_used) - }) - .map_err(|used| { - insufficient_capacity_err( - reservation, - additional, - self.pool_size_per_consumer - .get(consumer_name) - .cloned() - .unwrap_or(self.pool_size) - .saturating_sub(used), - ) - })?; + if let Some(pool_size) = self.pool_size_per_consumer.get(consumer_name) { + let mut used_per_consumer = self.used_per_consumer.write(); + let consumer_usage = used_per_consumer + .entry(consumer_name.to_string()) + .or_insert_with(|| AtomicUsize::new(0)); + consumer_usage + .fetch_update(Ordering::Relaxed, Ordering::Relaxed, |used| { + let new_used = used + additional; + (new_used <= *pool_size).then_some(new_used) + }) + .map_err(|used| { + insufficient_capacity_err( + reservation, + additional, + pool_size.saturating_sub(used), + ) + })?; + } self.used .fetch_update(Ordering::Relaxed, Ordering::Relaxed, |used| { diff --git a/datafusion/physical-plan/Cargo.toml b/datafusion/physical-plan/Cargo.toml index d703f54986cd..a9f9b22fafda 100644 --- a/datafusion/physical-plan/Cargo.toml +++ b/datafusion/physical-plan/Cargo.toml @@ -80,4 +80,4 @@ tokio = { workspace = true, features = [ [[bench]] harness = false -name = "spm" \ No newline at end of file +name = "spm" diff --git a/datafusion/physical-plan/benches/spm.rs b/datafusion/physical-plan/benches/spm.rs index 624141234247..9cc703f5f726 100644 --- a/datafusion/physical-plan/benches/spm.rs +++ b/datafusion/physical-plan/benches/spm.rs @@ -142,4 +142,4 @@ fn criterion_benchmark(c: &mut Criterion) { } criterion_group!(benches, criterion_benchmark); -criterion_main!(benches); \ No newline at end of file +criterion_main!(benches); From 5f4c83e959926de7b7e022ef0d04e69361b01c54 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Fri, 25 Oct 2024 08:45:59 +0800 Subject: [PATCH 27/47] combine eq and gt Signed-off-by: jayzhan211 --- datafusion/physical-plan/src/sorts/merge.rs | 19 ++++++++++++++----- 1 file changed, 14 insertions(+), 5 deletions(-) diff --git a/datafusion/physical-plan/src/sorts/merge.rs b/datafusion/physical-plan/src/sorts/merge.rs index a26d9232ac1c..7fd3a50e7bf4 100644 --- a/datafusion/physical-plan/src/sorts/merge.rs +++ b/datafusion/physical-plan/src/sorts/merge.rs @@ -298,10 +298,18 @@ impl SortPreservingMergeStream { } #[inline] - fn is_eq(&self, a: usize, b: usize) -> bool { + fn is_eq_and_gt(&self, a: usize, b: usize) -> (bool, bool) { match (&self.cursors[a], &self.cursors[b]) { - (Some(ac), Some(bc)) => ac.cmp(bc).is_eq(), - _ => false, + (Some(ac), Some(bc)) => { + let ord = ac.cmp(bc); + if ord.is_eq() { + (true, true) + } else { + (false, ord.then_with(|| a.cmp(&b)).is_gt()) + } + } + (None, _) => (false, true), + (_, None) => (false, false), } } @@ -413,12 +421,13 @@ impl SortPreservingMergeStream { // If round-robin tie breaker is enabled and we're at the final match (cmp_node == 1) if self.enable_round_robin_tie_breaker && cmp_node == 1 { - if self.is_eq(winner, challenger) { + let (is_eq, is_gt) = self.is_eq_and_gt(winner, challenger); + if is_eq { self.handle_tie_breaker(cmp_node, &mut winner, challenger); } else { // End of tie breaker self.round_robin_tie_breaker_mode = false; - if self.is_gt(winner, challenger) { + if is_gt { self.update_winner(cmp_node, &mut winner, challenger); } } From e418d6cc2c594c45d432d645d206ca804a4bcabb Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Sat, 26 Oct 2024 21:06:26 +0300 Subject: [PATCH 28/47] review part 1 --- datafusion/physical-plan/src/sorts/cursor.rs | 2 +- datafusion/physical-plan/src/sorts/merge.rs | 118 +++++++++++------- .../src/sorts/sort_preserving_merge.rs | 15 +-- 3 files changed, 82 insertions(+), 53 deletions(-) diff --git a/datafusion/physical-plan/src/sorts/cursor.rs b/datafusion/physical-plan/src/sorts/cursor.rs index 24b6dc92cbb7..133d736c1467 100644 --- a/datafusion/physical-plan/src/sorts/cursor.rs +++ b/datafusion/physical-plan/src/sorts/cursor.rs @@ -39,7 +39,7 @@ pub trait CursorValues { fn eq(l: &Self, l_idx: usize, r: &Self, r_idx: usize) -> bool; /// Returns true if `row[idx] == row[idx - 1]` - /// idx should be greater than 0 + /// Given `idx` should be greater than 0 fn eq_to_previous(cursor: &Self, idx: usize) -> bool; /// Returns comparison of `l[l_idx]` and `r[r_idx]` diff --git a/datafusion/physical-plan/src/sorts/merge.rs b/datafusion/physical-plan/src/sorts/merge.rs index 7fd3a50e7bf4..8240f2b9fb8a 100644 --- a/datafusion/physical-plan/src/sorts/merge.rs +++ b/datafusion/physical-plan/src/sorts/merge.rs @@ -97,14 +97,19 @@ pub(crate) struct SortPreservingMergeStream { /// Cursors for each input partition. `None` means the input is exhausted cursors: Vec>>, + /// Configuration parameter to enable round-robin selection of tied winners of loser tree. enable_round_robin_tie_breaker: bool, - /// Calculate the number of polled with the same value. + + /// Flag indicating whether we are in the mode of round-robin + /// tie breaker for the loser tree winners. + round_robin_tie_breaker_mode: bool, + + /// Total number of polls returning the same value, as per partition. /// We select the one that has less poll counts for tie-breaker in loser tree. num_of_polled_with_same_value: Vec, - /// Store previous batch for tracking the poll counts on the same value + + /// Stores the previous value of each partitions for tracking the poll counts on the same value. prev_cursors: Vec>>, - /// Flag to indicates whether we are in tie breaker in loser tree - round_robin_tie_breaker_mode: bool, /// Optional number of rows to fetch fetch: Option, @@ -228,12 +233,11 @@ impl SortPreservingMergeStream { self.aborted = true; return Poll::Ready(Some(Err(e))); } - self.update_loser_tree(); } let stream_idx = self.loser_tree[0]; - if self.advance(stream_idx) { + if self.advance_cursors(stream_idx) { self.loser_tree_adjusted = false; self.in_progress.push_row(stream_idx); @@ -251,17 +255,18 @@ impl SortPreservingMergeStream { } } - fn adjust_poll_count(&mut self, stream_idx: usize) { - let slot = &mut self.cursors[stream_idx]; + /// For the given partition, updates the poll count. If the current value is the same + /// of the previous value, it increases the count by 1; otherwise, it is reset as 0. + fn update_poll_count_on_the_same_value(&mut self, partition_idx: usize) { + let cursor = &mut self.cursors[partition_idx]; - if let Some(c) = slot.as_mut() { - // To compare with the last row in the previous batch - let prev_cursor = &self.prev_cursors[stream_idx]; - let is_eq = c.is_eq_to_prev_one(prev_cursor.as_ref()); - if is_eq { - self.num_of_polled_with_same_value[stream_idx] += 1; + if let Some(c) = cursor.as_mut() { + // Compare with the last row in the previous batch + let prev_cursor = &self.prev_cursors[partition_idx]; + if c.is_eq_to_prev_one(prev_cursor.as_ref()) { + self.num_of_polled_with_same_value[partition_idx] += 1; } else { - self.num_of_polled_with_same_value[stream_idx] = 0; + self.num_of_polled_with_same_value[partition_idx] = 0; } } } @@ -272,22 +277,25 @@ impl SortPreservingMergeStream { .unwrap_or(false) } - fn advance(&mut self, stream_idx: usize) -> bool { - let slot = &mut self.cursors[stream_idx]; - match slot.as_mut() { - Some(c) => { - let _ = c.advance(); - if c.is_finished() { - std::mem::swap(slot, &mut self.prev_cursors[stream_idx]); - *slot = None; - } - true + /// Advances the actual cursor. If it reaches its end, update the + /// previous cursor with it. + /// + /// If the given partition is not exhausted, the function returns `true`. + fn advance_cursors(&mut self, stream_idx: usize) -> bool { + if let Some(cursor) = &mut self.cursors[stream_idx] { + let _ = cursor.advance(); + if cursor.is_finished() { + // Take the current cursor, leaving `None` in its place + self.prev_cursors[stream_idx] = self.cursors[stream_idx].take(); } - None => false, + true + } else { + false } } - /// Returns `true` if the cursor at index `a` is greater than at index `b` + /// Returns `true` if the cursor at index `a` is greater than at index `b`. + /// In an equality case, it compares the partition indices given. #[inline] fn is_gt(&self, a: usize, b: usize) -> bool { match (&self.cursors[a], &self.cursors[b]) { @@ -303,7 +311,7 @@ impl SortPreservingMergeStream { (Some(ac), Some(bc)) => { let ord = ac.cmp(bc); if ord.is_eq() { - (true, true) + (true, false) } else { (false, ord.then_with(|| a.cmp(&b)).is_gt()) } @@ -387,43 +395,65 @@ impl SortPreservingMergeStream { self.loser_tree_adjusted = true; } + /// Handles tie-breaking logic during the adjustment of the loser tree. + /// + /// When comparing elements from multiple partitions in the `update_loser_tree` process, a tie can occur + /// between the current winner and a challenger. This function is invoked when such a tie needs to be + /// resolved according to the round-robin tie-breaker mode. + /// + /// If round-robin tie-breaking is not active, it is enabled, and the poll counts for all elements are reset. + /// The function then compares the poll counts of the current winner and the challenger: + /// - If the winner remains at the top after the final comparison, it increments the winner's poll count. + /// - If the challenger has a higher poll count than the winner, the challenger is declared the new winner. + /// - If the poll counts are equal but the challenger's index is smaller, the challenger is preferred. + /// + /// # Parameters + /// - `cmp_node`: The index of the comparison node in the loser tree where the tie-breaking is happening. + /// - `winner`: A mutable reference to the current winner, which may be updated based on the tie-breaking result. + /// - `challenger`: The index of the challenger being compared against the winner. + /// + /// This function ensures fair selection among elements with equal values when tie-breaking mode is enabled, + /// aiming to balance the polling across different partitions. #[inline] - fn handle_tie_breaker( - &mut self, - cmp_node: usize, - winner: &mut usize, - challenger: usize, - ) { + fn handle_tie(&mut self, cmp_node: usize, winner: &mut usize, challenger: usize) { if !self.round_robin_tie_breaker_mode { self.round_robin_tie_breaker_mode = true; - self.num_of_polled_with_same_value.fill(0); // Reset poll count for tie-breaker + // Reset poll count for tie-breaker + self.num_of_polled_with_same_value.fill(0); } - // Update poll count if the winner survives in the final match if *winner == self.loser_tree[0] { - self.adjust_poll_count(*winner); + self.update_poll_count_on_the_same_value(*winner); if self.is_poll_count_gt(*winner, challenger) { self.update_winner(cmp_node, winner, challenger); } } else if challenger < *winner { - // If poll count is not updated, prefer the smaller index + // The winner loses, assign the new winner self.update_winner(cmp_node, winner, challenger); } } - /// Attempts to update the loser tree, following winner replacement, if possible + /// Updates the loser tree to reflect the new winner after the previous winner is consumed. + /// This function adjusts the tree by comparing the current winner with challengers from + /// other partitions. + /// + /// If `enable_round_robin_tie_breaker` is true and a tie occurs at the final level, the + /// tie-breaker logic will be applied to ensure fair selection among equal elements. fn update_loser_tree(&mut self) { + // Start with the current winner let mut winner = self.loser_tree[0]; - // Replace overall winner by walking tree of losers + + // Find the leaf node index of the winner in the loser tree. let mut cmp_node = self.lt_leaf_node_index(winner); + + // Traverse up the tree to adjust comparisons until reaching the root. while cmp_node != 0 { let challenger = self.loser_tree[cmp_node]; - - // If round-robin tie breaker is enabled and we're at the final match (cmp_node == 1) + // If round-robin tie-breaker is enabled and we're at the final comparison (cmp_node == 1) if self.enable_round_robin_tie_breaker && cmp_node == 1 { let (is_eq, is_gt) = self.is_eq_and_gt(winner, challenger); if is_eq { - self.handle_tie_breaker(cmp_node, &mut winner, challenger); + self.handle_tie(cmp_node, &mut winner, challenger); } else { // End of tie breaker self.round_robin_tie_breaker_mode = false; @@ -434,10 +464,8 @@ impl SortPreservingMergeStream { } else if self.is_gt(winner, challenger) { self.update_winner(cmp_node, &mut winner, challenger); } - cmp_node = self.lt_parent_node_index(cmp_node); } - self.loser_tree[0] = winner; self.loser_tree_adjusted = true; } diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index f7cd67a09b88..37c3b5e175c5 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -82,6 +82,7 @@ pub struct SortPreservingMergeExec { fetch: Option, /// Cache holding plan properties like equivalences, output partitioning etc. cache: PlanProperties, + /// Configuration parameter to enable round-robin selection of tied winners of loser tree. enable_round_robin_repartition: bool, } @@ -98,12 +99,14 @@ impl SortPreservingMergeExec { enable_round_robin_repartition: true, } } + /// Sets the number of rows to fetch pub fn with_fetch(mut self, fetch: Option) -> Self { self.fetch = fetch; self } + /// Sets the selection strategy of tied winners of the loser tree algorithm pub fn with_round_robin_repartition( mut self, enable_round_robin_repartition: bool, @@ -357,13 +360,13 @@ mod tests { fn generate_task_ctx_for_round_robin_tie_breaker() -> Result> { let mut pool_per_consumer = HashMap::new(); - // Number from 660000 to 30000000 (or even more) are all valid limit - pool_per_consumer.insert("RepartitionExec[0]".to_string(), 10000000); - pool_per_consumer.insert("RepartitionExec[1]".to_string(), 10000000); + // Bytes from 660_000 to 30_000_000 (or even more) are all valid limits + pool_per_consumer.insert("RepartitionExec[0]".to_string(), 10_000_000); + pool_per_consumer.insert("RepartitionExec[1]".to_string(), 10_000_000); let runtime = RuntimeEnvBuilder::new() - // random large number for total mem limit, we care about RepartitionExec only - .with_memory_limit_per_consumer(2000000000, 1.0, pool_per_consumer) + // Random large number for total mem limit, we only care about RepartitionExec only + .with_memory_limit_per_consumer(2_000_000_000, 1.0, pool_per_consumer) .build_arc()?; let config = SessionConfig::new(); let task_ctx = TaskContext::default() @@ -410,7 +413,6 @@ mod tests { let task_ctx = generate_task_ctx_for_round_robin_tie_breaker()?; let spm = generate_spm_for_round_robin_tie_breaker(true)?; let _collected = collect(spm, task_ctx).await.unwrap(); - Ok(()) } @@ -419,7 +421,6 @@ mod tests { let task_ctx = generate_task_ctx_for_round_robin_tie_breaker()?; let spm = generate_spm_for_round_robin_tie_breaker(false)?; let _err = collect(spm, task_ctx).await.unwrap_err(); - Ok(()) } From 288e2fe312b9288fb3df0262769fe70c2c41cf81 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Sat, 26 Oct 2024 22:22:37 +0300 Subject: [PATCH 29/47] Update merge.rs --- datafusion/physical-plan/src/sorts/merge.rs | 21 ++++++++++++++++++++- 1 file changed, 20 insertions(+), 1 deletion(-) diff --git a/datafusion/physical-plan/src/sorts/merge.rs b/datafusion/physical-plan/src/sorts/merge.rs index 8240f2b9fb8a..d684bfabb4b6 100644 --- a/datafusion/physical-plan/src/sorts/merge.rs +++ b/datafusion/physical-plan/src/sorts/merge.rs @@ -108,6 +108,12 @@ pub(crate) struct SortPreservingMergeStream { /// We select the one that has less poll counts for tie-breaker in loser tree. num_of_polled_with_same_value: Vec, + /// To keep track of reset counts + poll_reset_epochs: Vec, + + /// Current reset count + current_reset_epoch: usize, + /// Stores the previous value of each partitions for tracking the poll counts on the same value. prev_cursors: Vec>>, @@ -145,6 +151,8 @@ impl SortPreservingMergeStream { prev_cursors: (0..stream_count).map(|_| None).collect(), round_robin_tie_breaker_mode: false, num_of_polled_with_same_value: vec![0; stream_count], + current_reset_epoch: 0, + poll_reset_epochs: vec![0; stream_count], loser_tree: vec![], loser_tree_adjusted: false, batch_size, @@ -260,6 +268,12 @@ impl SortPreservingMergeStream { fn update_poll_count_on_the_same_value(&mut self, partition_idx: usize) { let cursor = &mut self.cursors[partition_idx]; + // Check if the current partition's poll count is logically "reset" + if self.poll_reset_epochs[partition_idx] != self.current_reset_epoch { + self.poll_reset_epochs[partition_idx] = self.current_reset_epoch; + self.num_of_polled_with_same_value[partition_idx] = 0; + } + if let Some(c) = cursor.as_mut() { // Compare with the last row in the previous batch let prev_cursor = &self.prev_cursors[partition_idx]; @@ -395,6 +409,11 @@ impl SortPreservingMergeStream { self.loser_tree_adjusted = true; } + /// Resets the poll count by incrementing the reset epoch. + fn reset_poll_counts(&mut self) { + self.current_reset_epoch += 1; + } + /// Handles tie-breaking logic during the adjustment of the loser tree. /// /// When comparing elements from multiple partitions in the `update_loser_tree` process, a tie can occur @@ -419,7 +438,7 @@ impl SortPreservingMergeStream { if !self.round_robin_tie_breaker_mode { self.round_robin_tie_breaker_mode = true; // Reset poll count for tie-breaker - self.num_of_polled_with_same_value.fill(0); + self.reset_poll_counts(); } // Update poll count if the winner survives in the final match if *winner == self.loser_tree[0] { From 23d5fc7a1b2c32a34f1cb7841bb04580b20102b3 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Sun, 27 Oct 2024 19:42:00 +0800 Subject: [PATCH 30/47] upd doc Signed-off-by: jayzhan211 --- datafusion/physical-plan/src/sorts/merge.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/physical-plan/src/sorts/merge.rs b/datafusion/physical-plan/src/sorts/merge.rs index d684bfabb4b6..1a49cc949462 100644 --- a/datafusion/physical-plan/src/sorts/merge.rs +++ b/datafusion/physical-plan/src/sorts/merge.rs @@ -423,7 +423,7 @@ impl SortPreservingMergeStream { /// If round-robin tie-breaking is not active, it is enabled, and the poll counts for all elements are reset. /// The function then compares the poll counts of the current winner and the challenger: /// - If the winner remains at the top after the final comparison, it increments the winner's poll count. - /// - If the challenger has a higher poll count than the winner, the challenger is declared the new winner. + /// - If the challenger has a lower poll count than the current winner, the challenger becomes the new winner. /// - If the poll counts are equal but the challenger's index is smaller, the challenger is preferred. /// /// # Parameters From 2d02aec0dadff65f360a66ba770f967cfaa77292 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Mon, 28 Oct 2024 16:39:52 +0800 Subject: [PATCH 31/47] no need index comparison Signed-off-by: jayzhan211 --- datafusion/physical-plan/src/sorts/merge.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/physical-plan/src/sorts/merge.rs b/datafusion/physical-plan/src/sorts/merge.rs index 1a49cc949462..d943749f5ec3 100644 --- a/datafusion/physical-plan/src/sorts/merge.rs +++ b/datafusion/physical-plan/src/sorts/merge.rs @@ -327,7 +327,7 @@ impl SortPreservingMergeStream { if ord.is_eq() { (true, false) } else { - (false, ord.then_with(|| a.cmp(&b)).is_gt()) + (false, ord.is_gt()) } } (None, _) => (false, true), From 010f8690d10a81a0f5053ee8eef9aa8180d60fb2 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Mon, 28 Oct 2024 16:47:20 +0800 Subject: [PATCH 32/47] combine handle tie and eq check Signed-off-by: jayzhan211 --- datafusion/physical-plan/src/sorts/merge.rs | 45 ++++++++++----------- 1 file changed, 22 insertions(+), 23 deletions(-) diff --git a/datafusion/physical-plan/src/sorts/merge.rs b/datafusion/physical-plan/src/sorts/merge.rs index d943749f5ec3..a227ccfcb00e 100644 --- a/datafusion/physical-plan/src/sorts/merge.rs +++ b/datafusion/physical-plan/src/sorts/merge.rs @@ -319,22 +319,6 @@ impl SortPreservingMergeStream { } } - #[inline] - fn is_eq_and_gt(&self, a: usize, b: usize) -> (bool, bool) { - match (&self.cursors[a], &self.cursors[b]) { - (Some(ac), Some(bc)) => { - let ord = ac.cmp(bc); - if ord.is_eq() { - (true, false) - } else { - (false, ord.is_gt()) - } - } - (None, _) => (false, true), - (_, None) => (false, false), - } - } - #[inline] fn is_poll_count_gt(&self, a: usize, b: usize) -> bool { let poll_a = self.num_of_polled_with_same_value[a]; @@ -470,15 +454,30 @@ impl SortPreservingMergeStream { let challenger = self.loser_tree[cmp_node]; // If round-robin tie-breaker is enabled and we're at the final comparison (cmp_node == 1) if self.enable_round_robin_tie_breaker && cmp_node == 1 { - let (is_eq, is_gt) = self.is_eq_and_gt(winner, challenger); - if is_eq { - self.handle_tie(cmp_node, &mut winner, challenger); - } else { - // End of tie breaker - self.round_robin_tie_breaker_mode = false; - if is_gt { + match (&self.cursors[winner], &self.cursors[challenger]) { + (Some(ac), Some(bc)) => { + let ord = ac.cmp(bc); + if ord.is_eq() { + self.handle_tie(cmp_node, &mut winner, challenger); + } else { + // Ends of tie breaker + self.round_robin_tie_breaker_mode = false; + if ord.is_gt() { + self.update_winner(cmp_node, &mut winner, challenger); + } + } + } + (None, _) => { + // Challenger wins, update winner + // Ends of tie breaker + self.round_robin_tie_breaker_mode = false; self.update_winner(cmp_node, &mut winner, challenger); } + (_, None) => { + // Winner wins again + // Ends of tie breaker + self.round_robin_tie_breaker_mode = false; + } } } else if self.is_gt(winner, challenger) { self.update_winner(cmp_node, &mut winner, challenger); From 3605f850a1af6c9c2144f76d06c33459129ec99c Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Mon, 28 Oct 2024 16:49:16 +0800 Subject: [PATCH 33/47] upd doc Signed-off-by: jayzhan211 --- datafusion/physical-plan/src/sorts/merge.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/physical-plan/src/sorts/merge.rs b/datafusion/physical-plan/src/sorts/merge.rs index a227ccfcb00e..93c1d7d08aa9 100644 --- a/datafusion/physical-plan/src/sorts/merge.rs +++ b/datafusion/physical-plan/src/sorts/merge.rs @@ -430,7 +430,7 @@ impl SortPreservingMergeStream { if self.is_poll_count_gt(*winner, challenger) { self.update_winner(cmp_node, winner, challenger); } - } else if challenger < *winner { + } else if challenger < *winner { // Value and polls count are all the same, we choose the smaller index // The winner loses, assign the new winner self.update_winner(cmp_node, winner, challenger); } From 18f86e87a8f5f79f74a764eceebd954edeaf337c Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Mon, 28 Oct 2024 16:49:22 +0800 Subject: [PATCH 34/47] fmt Signed-off-by: jayzhan211 --- datafusion/physical-plan/src/sorts/merge.rs | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/datafusion/physical-plan/src/sorts/merge.rs b/datafusion/physical-plan/src/sorts/merge.rs index 93c1d7d08aa9..e56e8e6be95c 100644 --- a/datafusion/physical-plan/src/sorts/merge.rs +++ b/datafusion/physical-plan/src/sorts/merge.rs @@ -430,7 +430,8 @@ impl SortPreservingMergeStream { if self.is_poll_count_gt(*winner, challenger) { self.update_winner(cmp_node, winner, challenger); } - } else if challenger < *winner { // Value and polls count are all the same, we choose the smaller index + } else if challenger < *winner { + // Value and polls count are all the same, we choose the smaller index // The winner loses, assign the new winner self.update_winner(cmp_node, winner, challenger); } From 8adf14e7edbe5461af19b109aecf2cb3bc7c3c26 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Mon, 28 Oct 2024 17:41:36 +0800 Subject: [PATCH 35/47] add more comment Signed-off-by: jayzhan211 --- datafusion/physical-plan/src/sorts/merge.rs | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/datafusion/physical-plan/src/sorts/merge.rs b/datafusion/physical-plan/src/sorts/merge.rs index e56e8e6be95c..3f0947d7f588 100644 --- a/datafusion/physical-plan/src/sorts/merge.rs +++ b/datafusion/physical-plan/src/sorts/merge.rs @@ -431,8 +431,9 @@ impl SortPreservingMergeStream { self.update_winner(cmp_node, winner, challenger); } } else if challenger < *winner { - // Value and polls count are all the same, we choose the smaller index - // The winner loses, assign the new winner + // If the winner doesn't surivie in the final match, it means the value has changed, + // The polls count are outdated (because the value advanced) but not yet cleanup at this point. + // Given the value is equal, we choose the smaller index if the value is the same. self.update_winner(cmp_node, winner, challenger); } } From 8d6c0a6de540ed60c387c0d7abf124309922e159 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Tue, 29 Oct 2024 07:28:33 +0800 Subject: [PATCH 36/47] remove flag Signed-off-by: jayzhan211 --- datafusion/physical-plan/benches/spm.rs | 13 ++----- datafusion/physical-plan/src/sorts/merge.rs | 9 ++--- .../src/sorts/sort_preserving_merge.rs | 34 +++---------------- .../src/sorts/streaming_merge.rs | 29 ++++------------ 4 files changed, 17 insertions(+), 68 deletions(-) diff --git a/datafusion/physical-plan/benches/spm.rs b/datafusion/physical-plan/benches/spm.rs index 9cc703f5f726..48d1a83ffd72 100644 --- a/datafusion/physical-plan/benches/spm.rs +++ b/datafusion/physical-plan/benches/spm.rs @@ -31,7 +31,6 @@ use criterion::{black_box, criterion_group, criterion_main, Criterion}; fn generate_spm_for_round_robin_tie_breaker( has_same_value: bool, - enable_round_robin_repartition: bool, batch_count: usize, partition_count: usize, ) -> SortPreservingMergeExec { @@ -83,13 +82,11 @@ fn generate_spm_for_round_robin_tie_breaker( let exec = MemoryExec::try_new(&partitiones, schema, None).unwrap(); SortPreservingMergeExec::new(sort, Arc::new(exec)) - .with_round_robin_repartition(enable_round_robin_repartition) } fn run_bench( c: &mut Criterion, has_same_value: bool, - enable_round_robin_repartition: bool, batch_count: usize, partition_count: usize, description: &str, @@ -99,7 +96,6 @@ fn run_bench( let spm = Arc::new(generate_spm_for_round_robin_tie_breaker( has_same_value, - enable_round_robin_repartition, batch_count, partition_count, )) as Arc; @@ -112,16 +108,14 @@ fn run_bench( fn criterion_benchmark(c: &mut Criterion) { let params = [ - (true, false, "low_card_without_tiebreaker"), // low cardinality, no tie breaker - (true, true, "low_card_with_tiebreaker"), // low cardinality, with tie breaker - (false, false, "high_card_without_tiebreaker"), // high cardinality, no tie breaker - (false, true, "high_card_with_tiebreaker"), // high cardinality, with tie breaker + (true, "low_card"), // low cardinality, with tie breaker + (false, "high_card"), // high cardinality, with tie breaker ]; let batch_counts = [1, 25, 625]; let partition_counts = [2, 8, 32]; - for &(has_same_value, enable_round_robin_repartition, cardinality_label) in ¶ms { + for &(has_same_value, cardinality_label) in ¶ms { for &batch_count in &batch_counts { for &partition_count in &partition_counts { let description = format!( @@ -131,7 +125,6 @@ fn criterion_benchmark(c: &mut Criterion) { run_bench( c, has_same_value, - enable_round_robin_repartition, batch_count, partition_count, &description, diff --git a/datafusion/physical-plan/src/sorts/merge.rs b/datafusion/physical-plan/src/sorts/merge.rs index 3f0947d7f588..9d456f3a75ab 100644 --- a/datafusion/physical-plan/src/sorts/merge.rs +++ b/datafusion/physical-plan/src/sorts/merge.rs @@ -97,9 +97,6 @@ pub(crate) struct SortPreservingMergeStream { /// Cursors for each input partition. `None` means the input is exhausted cursors: Vec>>, - /// Configuration parameter to enable round-robin selection of tied winners of loser tree. - enable_round_robin_tie_breaker: bool, - /// Flag indicating whether we are in the mode of round-robin /// tie breaker for the loser tree winners. round_robin_tie_breaker_mode: bool, @@ -138,7 +135,6 @@ impl SortPreservingMergeStream { batch_size: usize, fetch: Option, reservation: MemoryReservation, - enable_round_robin_tie_breaker: bool, ) -> Self { let stream_count = streams.partitions(); @@ -159,7 +155,6 @@ impl SortPreservingMergeStream { fetch, produced: 0, uninitiated_partitions: (0..stream_count).collect(), - enable_round_robin_tie_breaker, } } @@ -442,7 +437,7 @@ impl SortPreservingMergeStream { /// This function adjusts the tree by comparing the current winner with challengers from /// other partitions. /// - /// If `enable_round_robin_tie_breaker` is true and a tie occurs at the final level, the + /// If a tie occurs at the final level, the /// tie-breaker logic will be applied to ensure fair selection among equal elements. fn update_loser_tree(&mut self) { // Start with the current winner @@ -455,7 +450,7 @@ impl SortPreservingMergeStream { while cmp_node != 0 { let challenger = self.loser_tree[cmp_node]; // If round-robin tie-breaker is enabled and we're at the final comparison (cmp_node == 1) - if self.enable_round_robin_tie_breaker && cmp_node == 1 { + if cmp_node == 1 { match (&self.cursors[winner], &self.cursors[challenger]) { (Some(ac), Some(bc)) => { let ord = ac.cmp(bc); diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index 37c3b5e175c5..d1173b778585 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -82,8 +82,6 @@ pub struct SortPreservingMergeExec { fetch: Option, /// Cache holding plan properties like equivalences, output partitioning etc. cache: PlanProperties, - /// Configuration parameter to enable round-robin selection of tied winners of loser tree. - enable_round_robin_repartition: bool, } impl SortPreservingMergeExec { @@ -96,7 +94,6 @@ impl SortPreservingMergeExec { metrics: ExecutionPlanMetricsSet::new(), fetch: None, cache, - enable_round_robin_repartition: true, } } @@ -106,15 +103,6 @@ impl SortPreservingMergeExec { self } - /// Sets the selection strategy of tied winners of the loser tree algorithm - pub fn with_round_robin_repartition( - mut self, - enable_round_robin_repartition: bool, - ) -> Self { - self.enable_round_robin_repartition = enable_round_robin_repartition; - self - } - /// Input schema pub fn input(&self) -> &Arc { &self.input @@ -195,7 +183,6 @@ impl ExecutionPlan for SortPreservingMergeExec { metrics: self.metrics.clone(), fetch: limit, cache: self.cache.clone(), - enable_round_robin_repartition: true, })) } @@ -295,7 +282,6 @@ impl ExecutionPlan for SortPreservingMergeExec { .with_batch_size(context.session_config().batch_size()) .with_fetch(self.fetch) .with_reservation(reservation) - .with_round_robin_tie_breaker(self.enable_round_robin_repartition) .build()?; debug!("Got stream result from SortPreservingMergeStream::new_from_receivers"); @@ -374,9 +360,8 @@ mod tests { .with_session_config(config); Ok(Arc::new(task_ctx)) } - fn generate_spm_for_round_robin_tie_breaker( - enable_round_robin_repartition: bool, - ) -> Result> { + fn generate_spm_for_round_robin_tie_breaker() -> Result> + { let target_batch_size = 12500; let row_size = 12500; let a: ArrayRef = Arc::new(Int32Array::from(vec![1; row_size])); @@ -403,27 +388,18 @@ mod tests { RepartitionExec::try_new(Arc::new(exec), Partitioning::RoundRobinBatch(2))?; let coalesce_batches_exec = CoalesceBatchesExec::new(Arc::new(repartition_exec), target_batch_size); - let spm = SortPreservingMergeExec::new(sort, Arc::new(coalesce_batches_exec)) - .with_round_robin_repartition(enable_round_robin_repartition); + let spm = SortPreservingMergeExec::new(sort, Arc::new(coalesce_batches_exec)); Ok(Arc::new(spm)) } #[tokio::test(flavor = "multi_thread")] - async fn test_round_robin_tie_breaker_success() -> Result<()> { + async fn test_round_robin_tie_breaker() -> Result<()> { let task_ctx = generate_task_ctx_for_round_robin_tie_breaker()?; - let spm = generate_spm_for_round_robin_tie_breaker(true)?; + let spm = generate_spm_for_round_robin_tie_breaker()?; let _collected = collect(spm, task_ctx).await.unwrap(); Ok(()) } - #[tokio::test(flavor = "multi_thread")] - async fn test_round_robin_tie_breaker_fail() -> Result<()> { - let task_ctx = generate_task_ctx_for_round_robin_tie_breaker()?; - let spm = generate_spm_for_round_robin_tie_breaker(false)?; - let _err = collect(spm, task_ctx).await.unwrap_err(); - Ok(()) - } - #[tokio::test] async fn test_merge_interleave() { let task_ctx = Arc::new(TaskContext::default()); diff --git a/datafusion/physical-plan/src/sorts/streaming_merge.rs b/datafusion/physical-plan/src/sorts/streaming_merge.rs index e8330a7cabc0..ad640d8e8470 100644 --- a/datafusion/physical-plan/src/sorts/streaming_merge.rs +++ b/datafusion/physical-plan/src/sorts/streaming_merge.rs @@ -36,7 +36,7 @@ macro_rules! primitive_merge_helper { } macro_rules! merge_helper { - ($t:ty, $sort:ident, $streams:ident, $schema:ident, $tracking_metrics:ident, $batch_size:ident, $fetch:ident, $reservation:ident, $enable_round_robin_tie_breaker:ident) => {{ + ($t:ty, $sort:ident, $streams:ident, $schema:ident, $tracking_metrics:ident, $batch_size:ident, $fetch:ident, $reservation:ident) => {{ let streams = FieldCursorStream::<$t>::new($sort, $streams); return Ok(Box::pin(SortPreservingMergeStream::new( Box::new(streams), @@ -45,7 +45,6 @@ macro_rules! merge_helper { $batch_size, $fetch, $reservation, - $enable_round_robin_tie_breaker, ))); }}; } @@ -59,15 +58,11 @@ pub struct StreamingMergeBuilder<'a> { batch_size: Option, fetch: Option, reservation: Option, - enable_round_robin_tie_breaker: bool, } impl<'a> StreamingMergeBuilder<'a> { pub fn new() -> Self { - Self { - enable_round_robin_tie_breaker: true, - ..Default::default() - } + Self::default() } pub fn with_streams(mut self, streams: Vec) -> Self { @@ -105,14 +100,6 @@ impl<'a> StreamingMergeBuilder<'a> { self } - pub fn with_round_robin_tie_breaker( - mut self, - enable_round_robin_tie_breaker: bool, - ) -> Self { - self.enable_round_robin_tie_breaker = enable_round_robin_tie_breaker; - self - } - pub fn build(self) -> Result { let Self { streams, @@ -122,7 +109,6 @@ impl<'a> StreamingMergeBuilder<'a> { reservation, fetch, expressions, - enable_round_robin_tie_breaker, } = self; // Early return if streams or expressions are empty @@ -155,11 +141,11 @@ impl<'a> StreamingMergeBuilder<'a> { let sort = expressions[0].clone(); let data_type = sort.expr.data_type(schema.as_ref())?; downcast_primitive! { - data_type => (primitive_merge_helper, sort, streams, schema, metrics, batch_size, fetch, reservation, enable_round_robin_tie_breaker), - DataType::Utf8 => merge_helper!(StringArray, sort, streams, schema, metrics, batch_size, fetch, reservation, enable_round_robin_tie_breaker) - DataType::LargeUtf8 => merge_helper!(LargeStringArray, sort, streams, schema, metrics, batch_size, fetch, reservation, enable_round_robin_tie_breaker) - DataType::Binary => merge_helper!(BinaryArray, sort, streams, schema, metrics, batch_size, fetch, reservation, enable_round_robin_tie_breaker) - DataType::LargeBinary => merge_helper!(LargeBinaryArray, sort, streams, schema, metrics, batch_size, fetch, reservation, enable_round_robin_tie_breaker) + data_type => (primitive_merge_helper, sort, streams, schema, metrics, batch_size, fetch, reservation), + DataType::Utf8 => merge_helper!(StringArray, sort, streams, schema, metrics, batch_size, fetch, reservation) + DataType::LargeUtf8 => merge_helper!(LargeStringArray, sort, streams, schema, metrics, batch_size, fetch, reservation) + DataType::Binary => merge_helper!(BinaryArray, sort, streams, schema, metrics, batch_size, fetch, reservation) + DataType::LargeBinary => merge_helper!(LargeBinaryArray, sort, streams, schema, metrics, batch_size, fetch, reservation) _ => {} } } @@ -177,7 +163,6 @@ impl<'a> StreamingMergeBuilder<'a> { batch_size, fetch, reservation, - enable_round_robin_tie_breaker, ))) } } From a18cba8a61f089313d96ed84940ca17d9c7a7b2d Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Tue, 29 Oct 2024 16:53:36 +0800 Subject: [PATCH 37/47] upd comment Signed-off-by: jayzhan211 --- datafusion/physical-plan/src/sorts/merge.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/datafusion/physical-plan/src/sorts/merge.rs b/datafusion/physical-plan/src/sorts/merge.rs index 9d456f3a75ab..e4082ad53c3b 100644 --- a/datafusion/physical-plan/src/sorts/merge.rs +++ b/datafusion/physical-plan/src/sorts/merge.rs @@ -399,7 +399,7 @@ impl SortPreservingMergeStream { /// between the current winner and a challenger. This function is invoked when such a tie needs to be /// resolved according to the round-robin tie-breaker mode. /// - /// If round-robin tie-breaking is not active, it is enabled, and the poll counts for all elements are reset. + /// If round-robin tie-breaking is not active and the poll counts for all elements are reset. /// The function then compares the poll counts of the current winner and the challenger: /// - If the winner remains at the top after the final comparison, it increments the winner's poll count. /// - If the challenger has a lower poll count than the current winner, the challenger becomes the new winner. @@ -410,7 +410,7 @@ impl SortPreservingMergeStream { /// - `winner`: A mutable reference to the current winner, which may be updated based on the tie-breaking result. /// - `challenger`: The index of the challenger being compared against the winner. /// - /// This function ensures fair selection among elements with equal values when tie-breaking mode is enabled, + /// This function ensures fair selection among elements with equal values /// aiming to balance the polling across different partitions. #[inline] fn handle_tie(&mut self, cmp_node: usize, winner: &mut usize, challenger: usize) { @@ -449,7 +449,7 @@ impl SortPreservingMergeStream { // Traverse up the tree to adjust comparisons until reaching the root. while cmp_node != 0 { let challenger = self.loser_tree[cmp_node]; - // If round-robin tie-breaker is enabled and we're at the final comparison (cmp_node == 1) + // If we're at the final comparison (cmp_node == 1) if cmp_node == 1 { match (&self.cursors[winner], &self.cursors[challenger]) { (Some(ac), Some(bc)) => { From d2f3a8404c827e32cd9d8086646661c8b6594f43 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Wed, 30 Oct 2024 07:19:58 +0800 Subject: [PATCH 38/47] Revert "remove flag" This reverts commit 8d6c0a6de540ed60c387c0d7abf124309922e159. --- datafusion/physical-plan/benches/spm.rs | 13 +++++-- datafusion/physical-plan/src/sorts/merge.rs | 11 ++++-- .../src/sorts/sort_preserving_merge.rs | 34 ++++++++++++++++--- .../src/sorts/streaming_merge.rs | 29 ++++++++++++---- 4 files changed, 69 insertions(+), 18 deletions(-) diff --git a/datafusion/physical-plan/benches/spm.rs b/datafusion/physical-plan/benches/spm.rs index 48d1a83ffd72..9cc703f5f726 100644 --- a/datafusion/physical-plan/benches/spm.rs +++ b/datafusion/physical-plan/benches/spm.rs @@ -31,6 +31,7 @@ use criterion::{black_box, criterion_group, criterion_main, Criterion}; fn generate_spm_for_round_robin_tie_breaker( has_same_value: bool, + enable_round_robin_repartition: bool, batch_count: usize, partition_count: usize, ) -> SortPreservingMergeExec { @@ -82,11 +83,13 @@ fn generate_spm_for_round_robin_tie_breaker( let exec = MemoryExec::try_new(&partitiones, schema, None).unwrap(); SortPreservingMergeExec::new(sort, Arc::new(exec)) + .with_round_robin_repartition(enable_round_robin_repartition) } fn run_bench( c: &mut Criterion, has_same_value: bool, + enable_round_robin_repartition: bool, batch_count: usize, partition_count: usize, description: &str, @@ -96,6 +99,7 @@ fn run_bench( let spm = Arc::new(generate_spm_for_round_robin_tie_breaker( has_same_value, + enable_round_robin_repartition, batch_count, partition_count, )) as Arc; @@ -108,14 +112,16 @@ fn run_bench( fn criterion_benchmark(c: &mut Criterion) { let params = [ - (true, "low_card"), // low cardinality, with tie breaker - (false, "high_card"), // high cardinality, with tie breaker + (true, false, "low_card_without_tiebreaker"), // low cardinality, no tie breaker + (true, true, "low_card_with_tiebreaker"), // low cardinality, with tie breaker + (false, false, "high_card_without_tiebreaker"), // high cardinality, no tie breaker + (false, true, "high_card_with_tiebreaker"), // high cardinality, with tie breaker ]; let batch_counts = [1, 25, 625]; let partition_counts = [2, 8, 32]; - for &(has_same_value, cardinality_label) in ¶ms { + for &(has_same_value, enable_round_robin_repartition, cardinality_label) in ¶ms { for &batch_count in &batch_counts { for &partition_count in &partition_counts { let description = format!( @@ -125,6 +131,7 @@ fn criterion_benchmark(c: &mut Criterion) { run_bench( c, has_same_value, + enable_round_robin_repartition, batch_count, partition_count, &description, diff --git a/datafusion/physical-plan/src/sorts/merge.rs b/datafusion/physical-plan/src/sorts/merge.rs index e4082ad53c3b..4f5a1427924d 100644 --- a/datafusion/physical-plan/src/sorts/merge.rs +++ b/datafusion/physical-plan/src/sorts/merge.rs @@ -97,6 +97,9 @@ pub(crate) struct SortPreservingMergeStream { /// Cursors for each input partition. `None` means the input is exhausted cursors: Vec>>, + /// Configuration parameter to enable round-robin selection of tied winners of loser tree. + enable_round_robin_tie_breaker: bool, + /// Flag indicating whether we are in the mode of round-robin /// tie breaker for the loser tree winners. round_robin_tie_breaker_mode: bool, @@ -135,6 +138,7 @@ impl SortPreservingMergeStream { batch_size: usize, fetch: Option, reservation: MemoryReservation, + enable_round_robin_tie_breaker: bool, ) -> Self { let stream_count = streams.partitions(); @@ -155,6 +159,7 @@ impl SortPreservingMergeStream { fetch, produced: 0, uninitiated_partitions: (0..stream_count).collect(), + enable_round_robin_tie_breaker, } } @@ -437,7 +442,7 @@ impl SortPreservingMergeStream { /// This function adjusts the tree by comparing the current winner with challengers from /// other partitions. /// - /// If a tie occurs at the final level, the + /// If `enable_round_robin_tie_breaker` is true and a tie occurs at the final level, the /// tie-breaker logic will be applied to ensure fair selection among equal elements. fn update_loser_tree(&mut self) { // Start with the current winner @@ -449,8 +454,8 @@ impl SortPreservingMergeStream { // Traverse up the tree to adjust comparisons until reaching the root. while cmp_node != 0 { let challenger = self.loser_tree[cmp_node]; - // If we're at the final comparison (cmp_node == 1) - if cmp_node == 1 { + // If round-robin tie-breaker is enabled and we're at the final comparison (cmp_node == 1) + if self.enable_round_robin_tie_breaker && cmp_node == 1 { match (&self.cursors[winner], &self.cursors[challenger]) { (Some(ac), Some(bc)) => { let ord = ac.cmp(bc); diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index d1173b778585..37c3b5e175c5 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -82,6 +82,8 @@ pub struct SortPreservingMergeExec { fetch: Option, /// Cache holding plan properties like equivalences, output partitioning etc. cache: PlanProperties, + /// Configuration parameter to enable round-robin selection of tied winners of loser tree. + enable_round_robin_repartition: bool, } impl SortPreservingMergeExec { @@ -94,6 +96,7 @@ impl SortPreservingMergeExec { metrics: ExecutionPlanMetricsSet::new(), fetch: None, cache, + enable_round_robin_repartition: true, } } @@ -103,6 +106,15 @@ impl SortPreservingMergeExec { self } + /// Sets the selection strategy of tied winners of the loser tree algorithm + pub fn with_round_robin_repartition( + mut self, + enable_round_robin_repartition: bool, + ) -> Self { + self.enable_round_robin_repartition = enable_round_robin_repartition; + self + } + /// Input schema pub fn input(&self) -> &Arc { &self.input @@ -183,6 +195,7 @@ impl ExecutionPlan for SortPreservingMergeExec { metrics: self.metrics.clone(), fetch: limit, cache: self.cache.clone(), + enable_round_robin_repartition: true, })) } @@ -282,6 +295,7 @@ impl ExecutionPlan for SortPreservingMergeExec { .with_batch_size(context.session_config().batch_size()) .with_fetch(self.fetch) .with_reservation(reservation) + .with_round_robin_tie_breaker(self.enable_round_robin_repartition) .build()?; debug!("Got stream result from SortPreservingMergeStream::new_from_receivers"); @@ -360,8 +374,9 @@ mod tests { .with_session_config(config); Ok(Arc::new(task_ctx)) } - fn generate_spm_for_round_robin_tie_breaker() -> Result> - { + fn generate_spm_for_round_robin_tie_breaker( + enable_round_robin_repartition: bool, + ) -> Result> { let target_batch_size = 12500; let row_size = 12500; let a: ArrayRef = Arc::new(Int32Array::from(vec![1; row_size])); @@ -388,18 +403,27 @@ mod tests { RepartitionExec::try_new(Arc::new(exec), Partitioning::RoundRobinBatch(2))?; let coalesce_batches_exec = CoalesceBatchesExec::new(Arc::new(repartition_exec), target_batch_size); - let spm = SortPreservingMergeExec::new(sort, Arc::new(coalesce_batches_exec)); + let spm = SortPreservingMergeExec::new(sort, Arc::new(coalesce_batches_exec)) + .with_round_robin_repartition(enable_round_robin_repartition); Ok(Arc::new(spm)) } #[tokio::test(flavor = "multi_thread")] - async fn test_round_robin_tie_breaker() -> Result<()> { + async fn test_round_robin_tie_breaker_success() -> Result<()> { let task_ctx = generate_task_ctx_for_round_robin_tie_breaker()?; - let spm = generate_spm_for_round_robin_tie_breaker()?; + let spm = generate_spm_for_round_robin_tie_breaker(true)?; let _collected = collect(spm, task_ctx).await.unwrap(); Ok(()) } + #[tokio::test(flavor = "multi_thread")] + async fn test_round_robin_tie_breaker_fail() -> Result<()> { + let task_ctx = generate_task_ctx_for_round_robin_tie_breaker()?; + let spm = generate_spm_for_round_robin_tie_breaker(false)?; + let _err = collect(spm, task_ctx).await.unwrap_err(); + Ok(()) + } + #[tokio::test] async fn test_merge_interleave() { let task_ctx = Arc::new(TaskContext::default()); diff --git a/datafusion/physical-plan/src/sorts/streaming_merge.rs b/datafusion/physical-plan/src/sorts/streaming_merge.rs index ad640d8e8470..e8330a7cabc0 100644 --- a/datafusion/physical-plan/src/sorts/streaming_merge.rs +++ b/datafusion/physical-plan/src/sorts/streaming_merge.rs @@ -36,7 +36,7 @@ macro_rules! primitive_merge_helper { } macro_rules! merge_helper { - ($t:ty, $sort:ident, $streams:ident, $schema:ident, $tracking_metrics:ident, $batch_size:ident, $fetch:ident, $reservation:ident) => {{ + ($t:ty, $sort:ident, $streams:ident, $schema:ident, $tracking_metrics:ident, $batch_size:ident, $fetch:ident, $reservation:ident, $enable_round_robin_tie_breaker:ident) => {{ let streams = FieldCursorStream::<$t>::new($sort, $streams); return Ok(Box::pin(SortPreservingMergeStream::new( Box::new(streams), @@ -45,6 +45,7 @@ macro_rules! merge_helper { $batch_size, $fetch, $reservation, + $enable_round_robin_tie_breaker, ))); }}; } @@ -58,11 +59,15 @@ pub struct StreamingMergeBuilder<'a> { batch_size: Option, fetch: Option, reservation: Option, + enable_round_robin_tie_breaker: bool, } impl<'a> StreamingMergeBuilder<'a> { pub fn new() -> Self { - Self::default() + Self { + enable_round_robin_tie_breaker: true, + ..Default::default() + } } pub fn with_streams(mut self, streams: Vec) -> Self { @@ -100,6 +105,14 @@ impl<'a> StreamingMergeBuilder<'a> { self } + pub fn with_round_robin_tie_breaker( + mut self, + enable_round_robin_tie_breaker: bool, + ) -> Self { + self.enable_round_robin_tie_breaker = enable_round_robin_tie_breaker; + self + } + pub fn build(self) -> Result { let Self { streams, @@ -109,6 +122,7 @@ impl<'a> StreamingMergeBuilder<'a> { reservation, fetch, expressions, + enable_round_robin_tie_breaker, } = self; // Early return if streams or expressions are empty @@ -141,11 +155,11 @@ impl<'a> StreamingMergeBuilder<'a> { let sort = expressions[0].clone(); let data_type = sort.expr.data_type(schema.as_ref())?; downcast_primitive! { - data_type => (primitive_merge_helper, sort, streams, schema, metrics, batch_size, fetch, reservation), - DataType::Utf8 => merge_helper!(StringArray, sort, streams, schema, metrics, batch_size, fetch, reservation) - DataType::LargeUtf8 => merge_helper!(LargeStringArray, sort, streams, schema, metrics, batch_size, fetch, reservation) - DataType::Binary => merge_helper!(BinaryArray, sort, streams, schema, metrics, batch_size, fetch, reservation) - DataType::LargeBinary => merge_helper!(LargeBinaryArray, sort, streams, schema, metrics, batch_size, fetch, reservation) + data_type => (primitive_merge_helper, sort, streams, schema, metrics, batch_size, fetch, reservation, enable_round_robin_tie_breaker), + DataType::Utf8 => merge_helper!(StringArray, sort, streams, schema, metrics, batch_size, fetch, reservation, enable_round_robin_tie_breaker) + DataType::LargeUtf8 => merge_helper!(LargeStringArray, sort, streams, schema, metrics, batch_size, fetch, reservation, enable_round_robin_tie_breaker) + DataType::Binary => merge_helper!(BinaryArray, sort, streams, schema, metrics, batch_size, fetch, reservation, enable_round_robin_tie_breaker) + DataType::LargeBinary => merge_helper!(LargeBinaryArray, sort, streams, schema, metrics, batch_size, fetch, reservation, enable_round_robin_tie_breaker) _ => {} } } @@ -163,6 +177,7 @@ impl<'a> StreamingMergeBuilder<'a> { batch_size, fetch, reservation, + enable_round_robin_tie_breaker, ))) } } From 905eea7eb1730ca461ba5ee5996ac1580de3b397 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Wed, 30 Oct 2024 07:20:22 +0800 Subject: [PATCH 39/47] Revert "upd comment" This reverts commit a18cba8a61f089313d96ed84940ca17d9c7a7b2d. --- datafusion/physical-plan/src/sorts/merge.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion/physical-plan/src/sorts/merge.rs b/datafusion/physical-plan/src/sorts/merge.rs index 4f5a1427924d..3f0947d7f588 100644 --- a/datafusion/physical-plan/src/sorts/merge.rs +++ b/datafusion/physical-plan/src/sorts/merge.rs @@ -404,7 +404,7 @@ impl SortPreservingMergeStream { /// between the current winner and a challenger. This function is invoked when such a tie needs to be /// resolved according to the round-robin tie-breaker mode. /// - /// If round-robin tie-breaking is not active and the poll counts for all elements are reset. + /// If round-robin tie-breaking is not active, it is enabled, and the poll counts for all elements are reset. /// The function then compares the poll counts of the current winner and the challenger: /// - If the winner remains at the top after the final comparison, it increments the winner's poll count. /// - If the challenger has a lower poll count than the current winner, the challenger becomes the new winner. @@ -415,7 +415,7 @@ impl SortPreservingMergeStream { /// - `winner`: A mutable reference to the current winner, which may be updated based on the tie-breaking result. /// - `challenger`: The index of the challenger being compared against the winner. /// - /// This function ensures fair selection among elements with equal values + /// This function ensures fair selection among elements with equal values when tie-breaking mode is enabled, /// aiming to balance the polling across different partitions. #[inline] fn handle_tie(&mut self, cmp_node: usize, winner: &mut usize, challenger: usize) { From 98135eeca57437e136852e3a218680a5a1661091 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Wed, 30 Oct 2024 07:30:36 +0800 Subject: [PATCH 40/47] add more comment Signed-off-by: jayzhan211 --- datafusion/execution/src/memory_pool/pool.rs | 2 ++ .../physical-plan/src/sorts/sort_preserving_merge.rs | 10 ++++++++++ 2 files changed, 12 insertions(+) diff --git a/datafusion/execution/src/memory_pool/pool.rs b/datafusion/execution/src/memory_pool/pool.rs index 27d9357fca6a..55f91576e1c5 100644 --- a/datafusion/execution/src/memory_pool/pool.rs +++ b/datafusion/execution/src/memory_pool/pool.rs @@ -58,8 +58,10 @@ impl MemoryPool for UnboundedMemoryPool { #[derive(Debug)] pub struct GreedyMemoryPool { pool_size: usize, + // Pool size limit for each consumer, if one of the consumer exceeds the limit, error is returned pool_size_per_consumer: HashMap, used: AtomicUsize, + // Memory usage for each consumer, used to check aginst `pool_size_per_consumer` used_per_consumer: RwLock>, } diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index 37c3b5e175c5..8f3e91e7046b 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -409,6 +409,11 @@ mod tests { } #[tokio::test(flavor = "multi_thread")] + /// This test verifies that memory usage stays within limits when the tie breaker is enabled. + /// Any errors here could indicate unintended changes in tie breaker logic. + /// + /// Note: If you adjust constants in this test, ensure that memory usage differs + /// based on whether the tie breaker is enabled or disabled. async fn test_round_robin_tie_breaker_success() -> Result<()> { let task_ctx = generate_task_ctx_for_round_robin_tie_breaker()?; let spm = generate_spm_for_round_robin_tie_breaker(true)?; @@ -416,6 +421,11 @@ mod tests { Ok(()) } + /// This test verifies that memory usage stays within limits when the tie breaker is enabled. + /// Any errors here could indicate unintended changes in tie breaker logic. + /// + /// Note: If you adjust constants in this test, ensure that memory usage differs + /// based on whether the tie breaker is enabled or disabled. #[tokio::test(flavor = "multi_thread")] async fn test_round_robin_tie_breaker_fail() -> Result<()> { let task_ctx = generate_task_ctx_for_round_robin_tie_breaker()?; From 1b418c4174c8649f7a1d01cb53059c98a8ea16e8 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Wed, 30 Oct 2024 07:53:20 +0800 Subject: [PATCH 41/47] add more comment Signed-off-by: jayzhan211 --- datafusion/physical-plan/src/sorts/merge.rs | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/datafusion/physical-plan/src/sorts/merge.rs b/datafusion/physical-plan/src/sorts/merge.rs index 3f0947d7f588..08085e6fc1f8 100644 --- a/datafusion/physical-plan/src/sorts/merge.rs +++ b/datafusion/physical-plan/src/sorts/merge.rs @@ -98,6 +98,20 @@ pub(crate) struct SortPreservingMergeStream { cursors: Vec>>, /// Configuration parameter to enable round-robin selection of tied winners of loser tree. + /// + /// To address the issue of unbalanced polling between partitions due to tie-breakers being based + /// on partition index, especially in cases of low cardinality, we are making changes to the winner + /// selection mechanism. Previously, partitions with smaller indices were consistently chosen as the winners, + /// leading to an uneven distribution of polling. This caused upstream operator buffers for the other partitions + /// to grow excessively, as they continued receiving data without consuming it. + /// + /// For example, an upstream operator like a repartition execution would keep sending data to certain partitions, + /// but those partitions wouldn't consume the data if they weren't selected as winners. This resulted in inefficient buffer usage. + /// + /// To resolve this, we are modifying the tie-breaking logic. Instead of always choosing the partition with the smallest index, + /// we now select the partition that has the fewest poll counts for the same value. + /// This ensures that multiple partitions with the same value are chosen equally, distributing the polling load in a round-robin fashion. + /// This approach balances the workload more effectively across partitions and avoids excessive buffer growth.Round robin tie breaker enable_round_robin_tie_breaker: bool, /// Flag indicating whether we are in the mode of round-robin From 8297f58e7c1f9b0114f168a15397baf819a2bbd3 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Wed, 30 Oct 2024 07:55:11 +0800 Subject: [PATCH 42/47] fmt Signed-off-by: jayzhan211 --- datafusion/physical-plan/src/sorts/merge.rs | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/datafusion/physical-plan/src/sorts/merge.rs b/datafusion/physical-plan/src/sorts/merge.rs index 08085e6fc1f8..061d28fd208a 100644 --- a/datafusion/physical-plan/src/sorts/merge.rs +++ b/datafusion/physical-plan/src/sorts/merge.rs @@ -98,19 +98,19 @@ pub(crate) struct SortPreservingMergeStream { cursors: Vec>>, /// Configuration parameter to enable round-robin selection of tied winners of loser tree. - /// - /// To address the issue of unbalanced polling between partitions due to tie-breakers being based - /// on partition index, especially in cases of low cardinality, we are making changes to the winner + /// + /// To address the issue of unbalanced polling between partitions due to tie-breakers being based + /// on partition index, especially in cases of low cardinality, we are making changes to the winner /// selection mechanism. Previously, partitions with smaller indices were consistently chosen as the winners, /// leading to an uneven distribution of polling. This caused upstream operator buffers for the other partitions /// to grow excessively, as they continued receiving data without consuming it. - /// - /// For example, an upstream operator like a repartition execution would keep sending data to certain partitions, + /// + /// For example, an upstream operator like a repartition execution would keep sending data to certain partitions, /// but those partitions wouldn't consume the data if they weren't selected as winners. This resulted in inefficient buffer usage. - /// - /// To resolve this, we are modifying the tie-breaking logic. Instead of always choosing the partition with the smallest index, - /// we now select the partition that has the fewest poll counts for the same value. - /// This ensures that multiple partitions with the same value are chosen equally, distributing the polling load in a round-robin fashion. + /// + /// To resolve this, we are modifying the tie-breaking logic. Instead of always choosing the partition with the smallest index, + /// we now select the partition that has the fewest poll counts for the same value. + /// This ensures that multiple partitions with the same value are chosen equally, distributing the polling load in a round-robin fashion. /// This approach balances the workload more effectively across partitions and avoids excessive buffer growth.Round robin tie breaker enable_round_robin_tie_breaker: bool, From b652802d429d41976b7b4fd2a93683e217edb733 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Wed, 30 Oct 2024 15:31:39 +0800 Subject: [PATCH 43/47] simpliy mem pool Signed-off-by: jayzhan211 --- datafusion/execution/src/memory_pool/pool.rs | 54 +------------------ datafusion/execution/src/runtime_env.rs | 17 ------ .../src/sorts/sort_preserving_merge.rs | 9 +--- 3 files changed, 3 insertions(+), 77 deletions(-) diff --git a/datafusion/execution/src/memory_pool/pool.rs b/datafusion/execution/src/memory_pool/pool.rs index 55f91576e1c5..014e2ee3d089 100644 --- a/datafusion/execution/src/memory_pool/pool.rs +++ b/datafusion/execution/src/memory_pool/pool.rs @@ -58,11 +58,7 @@ impl MemoryPool for UnboundedMemoryPool { #[derive(Debug)] pub struct GreedyMemoryPool { pool_size: usize, - // Pool size limit for each consumer, if one of the consumer exceeds the limit, error is returned - pool_size_per_consumer: HashMap, used: AtomicUsize, - // Memory usage for each consumer, used to check aginst `pool_size_per_consumer` - used_per_consumer: RwLock>, } impl GreedyMemoryPool { @@ -71,67 +67,21 @@ impl GreedyMemoryPool { debug!("Created new GreedyMemoryPool(pool_size={pool_size})"); Self { pool_size, - pool_size_per_consumer: Default::default(), used: AtomicUsize::new(0), - used_per_consumer: RwLock::new(HashMap::new()), } } - - pub fn with_pool_size_per_consumer( - mut self, - pool_size_per_consumer: HashMap, - ) -> Self { - self.pool_size_per_consumer = pool_size_per_consumer; - self - } } impl MemoryPool for GreedyMemoryPool { - fn grow(&self, reservation: &MemoryReservation, additional: usize) { - let consumer_name = reservation.consumer().name(); + fn grow(&self, _reservation: &MemoryReservation, additional: usize) { self.used.fetch_add(additional, Ordering::Relaxed); - - let mut used_per_consumer = self.used_per_consumer.write(); - let consumer_usage = used_per_consumer - .entry(consumer_name.to_string()) - .or_insert_with(|| AtomicUsize::new(0)); - consumer_usage.fetch_add(additional, Ordering::Relaxed); } - fn shrink(&self, reservation: &MemoryReservation, shrink: usize) { - let consumer_name = reservation.consumer().name(); - + fn shrink(&self, _reservation: &MemoryReservation, shrink: usize) { self.used.fetch_sub(shrink, Ordering::Relaxed); - - let mut used_per_consumer = self.used_per_consumer.write(); - let consumer_usage = used_per_consumer - .entry(consumer_name.to_string()) - .or_insert_with(|| AtomicUsize::new(0)); - consumer_usage.fetch_sub(shrink, Ordering::Relaxed); } fn try_grow(&self, reservation: &MemoryReservation, additional: usize) -> Result<()> { - let consumer_name = reservation.consumer().name(); - - if let Some(pool_size) = self.pool_size_per_consumer.get(consumer_name) { - let mut used_per_consumer = self.used_per_consumer.write(); - let consumer_usage = used_per_consumer - .entry(consumer_name.to_string()) - .or_insert_with(|| AtomicUsize::new(0)); - consumer_usage - .fetch_update(Ordering::Relaxed, Ordering::Relaxed, |used| { - let new_used = used + additional; - (new_used <= *pool_size).then_some(new_used) - }) - .map_err(|used| { - insufficient_capacity_err( - reservation, - additional, - pool_size.saturating_sub(used), - ) - })?; - } - self.used .fetch_update(Ordering::Relaxed, Ordering::Relaxed, |used| { let new_used = used + additional; diff --git a/datafusion/execution/src/runtime_env.rs b/datafusion/execution/src/runtime_env.rs index d7eddaf60ab1..3c7ea5a67a3a 100644 --- a/datafusion/execution/src/runtime_env.rs +++ b/datafusion/execution/src/runtime_env.rs @@ -233,23 +233,6 @@ impl RuntimeEnvBuilder { ))) } - /// Set memory limit per consumer, if not set, by default is the same as the total pool size - /// For example, if pool size is 4000, repartition is 3000. Total pool size: 4000, - /// RepartitionExec pool size: 3000, SortPreservingMergeExec pool size: 4000 - pub fn with_memory_limit_per_consumer( - self, - max_memory: usize, - memory_fraction: f64, - pool_size_per_consumer: HashMap, - ) -> Self { - let pool_size = (max_memory as f64 * memory_fraction) as usize; - self.with_memory_pool(Arc::new(TrackConsumersPool::new( - GreedyMemoryPool::new(pool_size) - .with_pool_size_per_consumer(pool_size_per_consumer), - NonZeroUsize::new(5).unwrap(), - ))) - } - /// Use the specified path to create any needed temporary files pub fn with_temp_file_path(self, path: impl Into) -> Self { self.with_disk_manager(DiskManagerConfig::new_specified(vec![path.into()])) diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index 8f3e91e7046b..22a28669bf02 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -355,18 +355,11 @@ mod tests { use datafusion_physical_expr_common::physical_expr::PhysicalExpr; use futures::{FutureExt, Stream, StreamExt}; - use hashbrown::HashMap; use tokio::time::timeout; fn generate_task_ctx_for_round_robin_tie_breaker() -> Result> { - let mut pool_per_consumer = HashMap::new(); - // Bytes from 660_000 to 30_000_000 (or even more) are all valid limits - pool_per_consumer.insert("RepartitionExec[0]".to_string(), 10_000_000); - pool_per_consumer.insert("RepartitionExec[1]".to_string(), 10_000_000); - let runtime = RuntimeEnvBuilder::new() - // Random large number for total mem limit, we only care about RepartitionExec only - .with_memory_limit_per_consumer(2_000_000_000, 1.0, pool_per_consumer) + .with_memory_limit(20_000_000, 1.0) .build_arc()?; let config = SessionConfig::new(); let task_ctx = TaskContext::default() From f68fa9bb0c93ab9b6fb4d2f9af726aa4c1c71cdd Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Wed, 30 Oct 2024 15:34:27 +0800 Subject: [PATCH 44/47] clippy Signed-off-by: jayzhan211 --- datafusion/execution/src/memory_pool/pool.rs | 2 +- datafusion/execution/src/runtime_env.rs | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/datafusion/execution/src/memory_pool/pool.rs b/datafusion/execution/src/memory_pool/pool.rs index 014e2ee3d089..e169c1f319cc 100644 --- a/datafusion/execution/src/memory_pool/pool.rs +++ b/datafusion/execution/src/memory_pool/pool.rs @@ -19,7 +19,7 @@ use crate::memory_pool::{MemoryConsumer, MemoryPool, MemoryReservation}; use datafusion_common::{resources_datafusion_err, DataFusionError, Result}; use hashbrown::HashMap; use log::debug; -use parking_lot::{Mutex, RwLock}; +use parking_lot::Mutex; use std::{ num::NonZeroUsize, sync::atomic::{AtomicU64, AtomicUsize, Ordering}, diff --git a/datafusion/execution/src/runtime_env.rs b/datafusion/execution/src/runtime_env.rs index 3c7ea5a67a3a..4022eb07de0c 100644 --- a/datafusion/execution/src/runtime_env.rs +++ b/datafusion/execution/src/runtime_env.rs @@ -28,7 +28,6 @@ use crate::{ use crate::cache::cache_manager::{CacheManager, CacheManagerConfig}; use datafusion_common::{DataFusionError, Result}; -use hashbrown::HashMap; use object_store::ObjectStore; use std::path::PathBuf; use std::sync::Arc; From b63a6313884fb47423b11fd91514e1f397416ff0 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Wed, 30 Oct 2024 16:21:27 +0300 Subject: [PATCH 45/47] Update merge.rs --- datafusion/physical-plan/src/sorts/merge.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion/physical-plan/src/sorts/merge.rs b/datafusion/physical-plan/src/sorts/merge.rs index 061d28fd208a..54ed1242010d 100644 --- a/datafusion/physical-plan/src/sorts/merge.rs +++ b/datafusion/physical-plan/src/sorts/merge.rs @@ -445,8 +445,8 @@ impl SortPreservingMergeStream { self.update_winner(cmp_node, winner, challenger); } } else if challenger < *winner { - // If the winner doesn't surivie in the final match, it means the value has changed, - // The polls count are outdated (because the value advanced) but not yet cleanup at this point. + // If the winner doesn't survive in the final match, it means the value has changed. + // The polls count are outdated (because the value advanced) but not yet cleaned-up at this point. // Given the value is equal, we choose the smaller index if the value is the same. self.update_winner(cmp_node, winner, challenger); } From 666b3feac97aded62160f7f3e10809a88a82a481 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Wed, 30 Oct 2024 16:30:07 +0300 Subject: [PATCH 46/47] minor --- datafusion/physical-plan/src/sorts/merge.rs | 2 +- datafusion/physical-plan/src/sorts/sort_preserving_merge.rs | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion/physical-plan/src/sorts/merge.rs b/datafusion/physical-plan/src/sorts/merge.rs index 54ed1242010d..d2b79d57d1be 100644 --- a/datafusion/physical-plan/src/sorts/merge.rs +++ b/datafusion/physical-plan/src/sorts/merge.rs @@ -111,7 +111,7 @@ pub(crate) struct SortPreservingMergeStream { /// To resolve this, we are modifying the tie-breaking logic. Instead of always choosing the partition with the smallest index, /// we now select the partition that has the fewest poll counts for the same value. /// This ensures that multiple partitions with the same value are chosen equally, distributing the polling load in a round-robin fashion. - /// This approach balances the workload more effectively across partitions and avoids excessive buffer growth.Round robin tie breaker + /// This approach balances the workload more effectively across partitions and avoids excessive buffer growth. enable_round_robin_tie_breaker: bool, /// Flag indicating whether we are in the mode of round-robin diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index 22a28669bf02..019f3b2d59f2 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -401,12 +401,12 @@ mod tests { Ok(Arc::new(spm)) } - #[tokio::test(flavor = "multi_thread")] /// This test verifies that memory usage stays within limits when the tie breaker is enabled. /// Any errors here could indicate unintended changes in tie breaker logic. /// /// Note: If you adjust constants in this test, ensure that memory usage differs /// based on whether the tie breaker is enabled or disabled. + #[tokio::test(flavor = "multi_thread")] async fn test_round_robin_tie_breaker_success() -> Result<()> { let task_ctx = generate_task_ctx_for_round_robin_tie_breaker()?; let spm = generate_spm_for_round_robin_tie_breaker(true)?; From 003fce3db7a80263b341bc6d7ffc6b72456b263a Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Wed, 30 Oct 2024 22:46:43 +0800 Subject: [PATCH 47/47] add comment Signed-off-by: jayzhan211 --- datafusion/physical-plan/src/sorts/merge.rs | 11 ++++++++--- .../physical-plan/src/sorts/sort_preserving_merge.rs | 4 ++++ 2 files changed, 12 insertions(+), 3 deletions(-) diff --git a/datafusion/physical-plan/src/sorts/merge.rs b/datafusion/physical-plan/src/sorts/merge.rs index d2b79d57d1be..458c1c29c0cf 100644 --- a/datafusion/physical-plan/src/sorts/merge.rs +++ b/datafusion/physical-plan/src/sorts/merge.rs @@ -445,9 +445,14 @@ impl SortPreservingMergeStream { self.update_winner(cmp_node, winner, challenger); } } else if challenger < *winner { - // If the winner doesn't survive in the final match, it means the value has changed. - // The polls count are outdated (because the value advanced) but not yet cleaned-up at this point. - // Given the value is equal, we choose the smaller index if the value is the same. + // If the winner doesn’t survive in the final match, it indicates that the original winner + // has moved up in value, so the challenger now becomes the new winner. + // This also means that we’re in a new round of the tie breaker, + // and the polls count is outdated (though not yet cleaned up). + // + // By the time we reach this code, both the new winner and the current challenger + // have the same value, and neither has an updated polls count. + // Therefore, we simply select the one with the smaller index. self.update_winner(cmp_node, winner, challenger); } } diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index 019f3b2d59f2..caae5c5598b6 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -357,6 +357,8 @@ mod tests { use futures::{FutureExt, Stream, StreamExt}; use tokio::time::timeout; + // The number in the function is highly related to the memory limit we are testing + // any change of the constant should be aware of fn generate_task_ctx_for_round_robin_tie_breaker() -> Result> { let runtime = RuntimeEnvBuilder::new() .with_memory_limit(20_000_000, 1.0) @@ -367,6 +369,8 @@ mod tests { .with_session_config(config); Ok(Arc::new(task_ctx)) } + // The number in the function is highly related to the memory limit we are testing, + // any change of the constant should be aware of fn generate_spm_for_round_robin_tie_breaker( enable_round_robin_repartition: bool, ) -> Result> {