From ec449104d7ca6fe9b843fdc03e8338ecb768af3c Mon Sep 17 00:00:00 2001 From: Jay Miller <3744812+jaylmiller@users.noreply.github.com> Date: Fri, 10 Feb 2023 11:56:26 -0500 Subject: [PATCH 01/29] modify sort_batch to use arrow row format for multi-column sorts --- .../core/src/physical_plan/sorts/sort.rs | 27 +++++++++++++++++-- 1 file changed, 25 insertions(+), 2 deletions(-) diff --git a/datafusion/core/src/physical_plan/sorts/sort.rs b/datafusion/core/src/physical_plan/sorts/sort.rs index ad08504c3b86..286bfab02d12 100644 --- a/datafusion/core/src/physical_plan/sorts/sort.rs +++ b/datafusion/core/src/physical_plan/sorts/sort.rs @@ -38,13 +38,14 @@ use crate::physical_plan::{ RecordBatchStream, SendableRecordBatchStream, Statistics, }; use crate::prelude::SessionConfig; -use arrow::array::{make_array, Array, ArrayRef, MutableArrayData}; +use arrow::array::{make_array, Array, ArrayRef, MutableArrayData, UInt32Array}; pub use arrow::compute::SortOptions; use arrow::compute::{concat, lexsort_to_indices, take, SortColumn, TakeOptions}; use arrow::datatypes::SchemaRef; use arrow::error::ArrowError; use arrow::ipc::reader::FileReader; use arrow::record_batch::RecordBatch; +use arrow::row::{Row, RowConverter, SortField}; use datafusion_physical_expr::EquivalenceProperties; use futures::{Stream, StreamExt, TryStreamExt}; use log::{debug, error}; @@ -820,7 +821,29 @@ fn sort_batch( .map(|e| e.evaluate_to_sort_column(&batch)) .collect::>>()?; - let indices = lexsort_to_indices(&sort_columns, fetch)?; + let indices = if sort_columns.len() == 1 { + lexsort_to_indices(&sort_columns, fetch)? + } else { + let sort_fields = sort_columns + .iter() + .map(|c| { + let datatype = c.values.data_type().to_owned(); + SortField::new_with_options(datatype, c.options.unwrap_or_default()) + }) + .collect::>(); + let arrays: Vec = + sort_columns.iter().map(|c| c.values.clone()).collect(); + let mut row_converter = RowConverter::new(sort_fields)?; + let rows = row_converter.convert_columns(&arrays)?; + + let mut to_sort: Vec<(usize, Row)> = rows.into_iter().enumerate().collect(); + to_sort.sort_unstable_by(|(_, row_a), (_, row_b)| row_a.cmp(row_b)); + let limit = match fetch { + Some(lim) => lim.min(to_sort.len()), + None => to_sort.len(), + }; + UInt32Array::from_iter(to_sort.into_iter().take(limit).map(|(idx, _)| idx as u32)) + }; // reorder all rows based on sorted indices let sorted_batch = RecordBatch::try_new( From c7c43e44008cc81d312d3b1668a1d6b46ffb8b5f Mon Sep 17 00:00:00 2001 From: Jay Miller <3744812+jaylmiller@users.noreply.github.com> Date: Sat, 11 Feb 2023 11:33:47 -0500 Subject: [PATCH 02/29] use row encoding for in memory partial sorting within SortExec --- .../core/src/physical_plan/sorts/sort.rs | 126 ++++++++++++------ 1 file changed, 87 insertions(+), 39 deletions(-) diff --git a/datafusion/core/src/physical_plan/sorts/sort.rs b/datafusion/core/src/physical_plan/sorts/sort.rs index 286bfab02d12..0762d1ca9082 100644 --- a/datafusion/core/src/physical_plan/sorts/sort.rs +++ b/datafusion/core/src/physical_plan/sorts/sort.rs @@ -45,7 +45,7 @@ use arrow::datatypes::SchemaRef; use arrow::error::ArrowError; use arrow::ipc::reader::FileReader; use arrow::record_batch::RecordBatch; -use arrow::row::{Row, RowConverter, SortField}; +use arrow::row::{OwnedRow, Row, RowConverter, SortField}; use datafusion_physical_expr::EquivalenceProperties; use futures::{Stream, StreamExt, TryStreamExt}; use log::{debug, error}; @@ -293,24 +293,22 @@ fn in_mem_partial_sort( tracking_metrics, ))) } else { - let (sorted_arrays, batches): (Vec>, Vec) = - buffered_batches - .drain(..) - .into_iter() - .map(|b| { - let BatchWithSortArray { - sort_arrays, - sorted_batch: batch, - } = b; - (sort_arrays, batch) - }) - .unzip(); - + let (batches, sort_data): (Vec<_>, Vec<_>) = buffered_batches + .drain(..) + .into_iter() + .map(|b| { + let BatchWithSortArray { + sort_data, + sorted_batch, + } = b; + (sorted_batch, sort_data) + }) + .unzip(); let sorted_iter = { // NB timer records time taken on drop, so there are no // calls to `timer.done()` below. let _timer = tracking_metrics.elapsed_compute().timer(); - get_sorted_iter(&sorted_arrays, expressions, batch_size, fetch)? + get_sorted_iter(&sort_data, expressions, batch_size, fetch)? }; Ok(Box::pin(SortedSizedRecordBatchStream::new( schema, @@ -327,18 +325,18 @@ struct CompositeIndex { row_idx: u32, } -/// Get sorted iterator by sort concatenated `SortColumn`s +/// Get sorted iterator using each sorted batches `SortData` fn get_sorted_iter( - sort_arrays: &[Vec], + sort_data: &[SortData], expr: &[PhysicalSortExpr], batch_size: usize, fetch: Option, ) -> Result { - let row_indices = sort_arrays + let row_indices = sort_data .iter() .enumerate() - .flat_map(|(i, arrays)| { - (0..arrays[0].len()).map(move |r| CompositeIndex { + .flat_map(|(i, data)| { + (0..data.arrays[0].len()).map(move |r| CompositeIndex { // since we original use UInt32Array to index the combined mono batch, // component record batches won't overflow as well, // use u32 here for space efficiency. @@ -347,22 +345,54 @@ fn get_sorted_iter( }) }) .collect::>(); - - let sort_columns = expr + let batch_rows: Option>> = sort_data .iter() - .enumerate() - .map(|(i, expr)| { - let columns_i = sort_arrays - .iter() - .map(|cs| cs[i].as_ref()) - .collect::>(); - Ok(SortColumn { - values: concat(columns_i.as_slice())?, - options: Some(expr.options), - }) + .map(|data| { + data.rows + .as_ref() + .map(|rows| rows.iter().map(|r| r.row()).collect()) }) - .collect::>>()?; - let indices = lexsort_to_indices(&sort_columns, fetch)?; + .collect(); + let (indices, _rows) = match batch_rows { + Some(rows) => { + let mut to_sort: Vec<(usize, Row)> = + rows.into_iter().flatten().enumerate().collect(); + assert_eq!( + to_sort.len(), + row_indices.len(), + "one or more batches unexectedly did not use row encoding" + ); + to_sort.sort_unstable_by(|(_, row_a), (_, row_b)| row_a.cmp(row_b)); + let limit = match fetch { + Some(lim) => lim.min(to_sort.len()), + None => to_sort.len(), + }; + let (indices, new_rows): (Vec, Vec) = + to_sort.into_iter().take(limit).unzip(); + let indices = UInt32Array::from_iter(indices.into_iter().map(|i| i as u32)); + (indices, Some(new_rows)) + } + None => { + let sort_columns = expr + .iter() + .enumerate() + .map(|(i, expr)| { + let columns_i = sort_data + .iter() + .map(|d| { + let cs = &d.arrays; + cs[i].as_ref() + }) + .collect::>(); + Ok(SortColumn { + values: concat(columns_i.as_slice())?, + options: Some(expr.options), + }) + }) + .collect::>>()?; + (lexsort_to_indices(&sort_columns, fetch)?, None) + } + }; // Calculate composite index based on sorted indices let row_indices = indices @@ -804,9 +834,15 @@ impl ExecutionPlan for SortExec { self.input.statistics() } } +/// preserved data used for sorting a single batch +struct SortData { + arrays: Vec, + /// None if row encoding was not used to sort batch + rows: Option>, +} struct BatchWithSortArray { - sort_arrays: Vec, + sort_data: SortData, sorted_batch: RecordBatch, } @@ -821,8 +857,8 @@ fn sort_batch( .map(|e| e.evaluate_to_sort_column(&batch)) .collect::>>()?; - let indices = if sort_columns.len() == 1 { - lexsort_to_indices(&sort_columns, fetch)? + let (indices, sorted_rows) = if sort_columns.len() == 1 { + (lexsort_to_indices(&sort_columns, fetch)?, None) } else { let sort_fields = sort_columns .iter() @@ -842,7 +878,16 @@ fn sort_batch( Some(lim) => lim.min(to_sort.len()), None => to_sort.len(), }; - UInt32Array::from_iter(to_sort.into_iter().take(limit).map(|(idx, _)| idx as u32)) + + let indices = UInt32Array::from_iter( + to_sort.iter().take(limit).map(|(idx, _)| *idx as u32), + ); + let rows = to_sort + .iter() + .take(limit) + .map(|(_, row)| row.owned()) + .collect::>(); + (indices, Some(rows)) }; // reorder all rows based on sorted indices @@ -879,7 +924,10 @@ fn sort_batch( .collect::>>()?; Ok(BatchWithSortArray { - sort_arrays, + sort_data: SortData { + rows: sorted_rows, + arrays: sort_arrays, + }, sorted_batch, }) } From 2e1c1438f10d872f4ba7669dc3789ea3877ea3c7 Mon Sep 17 00:00:00 2001 From: Jay Miller <3744812+jaylmiller@users.noreply.github.com> Date: Sun, 12 Feb 2023 09:25:21 -0500 Subject: [PATCH 03/29] Revert preserving row encoding changes --- .../core/src/physical_plan/sorts/sort.rs | 126 ++++++------------ 1 file changed, 39 insertions(+), 87 deletions(-) diff --git a/datafusion/core/src/physical_plan/sorts/sort.rs b/datafusion/core/src/physical_plan/sorts/sort.rs index 0762d1ca9082..286bfab02d12 100644 --- a/datafusion/core/src/physical_plan/sorts/sort.rs +++ b/datafusion/core/src/physical_plan/sorts/sort.rs @@ -45,7 +45,7 @@ use arrow::datatypes::SchemaRef; use arrow::error::ArrowError; use arrow::ipc::reader::FileReader; use arrow::record_batch::RecordBatch; -use arrow::row::{OwnedRow, Row, RowConverter, SortField}; +use arrow::row::{Row, RowConverter, SortField}; use datafusion_physical_expr::EquivalenceProperties; use futures::{Stream, StreamExt, TryStreamExt}; use log::{debug, error}; @@ -293,22 +293,24 @@ fn in_mem_partial_sort( tracking_metrics, ))) } else { - let (batches, sort_data): (Vec<_>, Vec<_>) = buffered_batches - .drain(..) - .into_iter() - .map(|b| { - let BatchWithSortArray { - sort_data, - sorted_batch, - } = b; - (sorted_batch, sort_data) - }) - .unzip(); + let (sorted_arrays, batches): (Vec>, Vec) = + buffered_batches + .drain(..) + .into_iter() + .map(|b| { + let BatchWithSortArray { + sort_arrays, + sorted_batch: batch, + } = b; + (sort_arrays, batch) + }) + .unzip(); + let sorted_iter = { // NB timer records time taken on drop, so there are no // calls to `timer.done()` below. let _timer = tracking_metrics.elapsed_compute().timer(); - get_sorted_iter(&sort_data, expressions, batch_size, fetch)? + get_sorted_iter(&sorted_arrays, expressions, batch_size, fetch)? }; Ok(Box::pin(SortedSizedRecordBatchStream::new( schema, @@ -325,18 +327,18 @@ struct CompositeIndex { row_idx: u32, } -/// Get sorted iterator using each sorted batches `SortData` +/// Get sorted iterator by sort concatenated `SortColumn`s fn get_sorted_iter( - sort_data: &[SortData], + sort_arrays: &[Vec], expr: &[PhysicalSortExpr], batch_size: usize, fetch: Option, ) -> Result { - let row_indices = sort_data + let row_indices = sort_arrays .iter() .enumerate() - .flat_map(|(i, data)| { - (0..data.arrays[0].len()).map(move |r| CompositeIndex { + .flat_map(|(i, arrays)| { + (0..arrays[0].len()).map(move |r| CompositeIndex { // since we original use UInt32Array to index the combined mono batch, // component record batches won't overflow as well, // use u32 here for space efficiency. @@ -345,54 +347,22 @@ fn get_sorted_iter( }) }) .collect::>(); - let batch_rows: Option>> = sort_data + + let sort_columns = expr .iter() - .map(|data| { - data.rows - .as_ref() - .map(|rows| rows.iter().map(|r| r.row()).collect()) - }) - .collect(); - let (indices, _rows) = match batch_rows { - Some(rows) => { - let mut to_sort: Vec<(usize, Row)> = - rows.into_iter().flatten().enumerate().collect(); - assert_eq!( - to_sort.len(), - row_indices.len(), - "one or more batches unexectedly did not use row encoding" - ); - to_sort.sort_unstable_by(|(_, row_a), (_, row_b)| row_a.cmp(row_b)); - let limit = match fetch { - Some(lim) => lim.min(to_sort.len()), - None => to_sort.len(), - }; - let (indices, new_rows): (Vec, Vec) = - to_sort.into_iter().take(limit).unzip(); - let indices = UInt32Array::from_iter(indices.into_iter().map(|i| i as u32)); - (indices, Some(new_rows)) - } - None => { - let sort_columns = expr + .enumerate() + .map(|(i, expr)| { + let columns_i = sort_arrays .iter() - .enumerate() - .map(|(i, expr)| { - let columns_i = sort_data - .iter() - .map(|d| { - let cs = &d.arrays; - cs[i].as_ref() - }) - .collect::>(); - Ok(SortColumn { - values: concat(columns_i.as_slice())?, - options: Some(expr.options), - }) - }) - .collect::>>()?; - (lexsort_to_indices(&sort_columns, fetch)?, None) - } - }; + .map(|cs| cs[i].as_ref()) + .collect::>(); + Ok(SortColumn { + values: concat(columns_i.as_slice())?, + options: Some(expr.options), + }) + }) + .collect::>>()?; + let indices = lexsort_to_indices(&sort_columns, fetch)?; // Calculate composite index based on sorted indices let row_indices = indices @@ -834,15 +804,9 @@ impl ExecutionPlan for SortExec { self.input.statistics() } } -/// preserved data used for sorting a single batch -struct SortData { - arrays: Vec, - /// None if row encoding was not used to sort batch - rows: Option>, -} struct BatchWithSortArray { - sort_data: SortData, + sort_arrays: Vec, sorted_batch: RecordBatch, } @@ -857,8 +821,8 @@ fn sort_batch( .map(|e| e.evaluate_to_sort_column(&batch)) .collect::>>()?; - let (indices, sorted_rows) = if sort_columns.len() == 1 { - (lexsort_to_indices(&sort_columns, fetch)?, None) + let indices = if sort_columns.len() == 1 { + lexsort_to_indices(&sort_columns, fetch)? } else { let sort_fields = sort_columns .iter() @@ -878,16 +842,7 @@ fn sort_batch( Some(lim) => lim.min(to_sort.len()), None => to_sort.len(), }; - - let indices = UInt32Array::from_iter( - to_sort.iter().take(limit).map(|(idx, _)| *idx as u32), - ); - let rows = to_sort - .iter() - .take(limit) - .map(|(_, row)| row.owned()) - .collect::>(); - (indices, Some(rows)) + UInt32Array::from_iter(to_sort.into_iter().take(limit).map(|(idx, _)| idx as u32)) }; // reorder all rows based on sorted indices @@ -924,10 +879,7 @@ fn sort_batch( .collect::>>()?; Ok(BatchWithSortArray { - sort_data: SortData { - rows: sorted_rows, - arrays: sort_arrays, - }, + sort_arrays, sorted_batch, }) } From 2ab03eeeeca6b2a7feb613747d62ddb1deff8143 Mon Sep 17 00:00:00 2001 From: Jay Miller <3744812+jaylmiller@users.noreply.github.com> Date: Sun, 12 Feb 2023 11:08:50 -0500 Subject: [PATCH 04/29] add bench for SortExec --- datafusion/core/Cargo.toml | 4 + datafusion/core/benches/sort.rs | 595 ++++++++++++++++++++++++++++++++ 2 files changed, 599 insertions(+) create mode 100644 datafusion/core/benches/sort.rs diff --git a/datafusion/core/Cargo.toml b/datafusion/core/Cargo.toml index 6cde4aa8e4fe..c76dedb41fac 100644 --- a/datafusion/core/Cargo.toml +++ b/datafusion/core/Cargo.toml @@ -170,6 +170,10 @@ required-features = ["jit"] harness = false name = "merge" +[[bench]] +harness = false +name = "sort" + [[test]] harness = false name = "sqllogictests" diff --git a/datafusion/core/benches/sort.rs b/datafusion/core/benches/sort.rs new file mode 100644 index 000000000000..801b0959e6a6 --- /dev/null +++ b/datafusion/core/benches/sort.rs @@ -0,0 +1,595 @@ +// 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. + +//! Adapted from merge benchmark. Primary difference is that the input data is not ordered. +use std::sync::Arc; + +use arrow::array::DictionaryArray; +use arrow::datatypes::Int32Type; +use arrow::{ + array::{Float64Array, Int64Array, StringArray, UInt64Array}, + compute::{self, SortOptions, TakeOptions}, + datatypes::Schema, + record_batch::RecordBatch, +}; + +/// Benchmarks for SortExec +use criterion::{criterion_group, criterion_main, Criterion}; +use datafusion::{ + execution::context::TaskContext, + physical_plan::{memory::MemoryExec, sorts::sort::SortExec, ExecutionPlan}, + prelude::SessionContext, +}; +use datafusion_physical_expr::{expressions::col, PhysicalSortExpr}; +use futures::StreamExt; +use rand::rngs::StdRng; +use rand::{Rng, SeedableRng}; +use tokio::runtime::Runtime; + +use lazy_static::lazy_static; + +/// Total number of streams to divide each input into +/// models 8 partition plan (should it be 16??) +const NUM_STREAMS: u64 = 8; + +/// Total number of input rows to generate +const INPUT_SIZE: u64 = 100000; +// cases: + +// * physical sort expr (X, Y Z, NULLS FIRST, ASC) (not parameterized) +// +// streams of distinct values +// streams with 10% duplicated values (within each stream, and across streams) +// These cases are intended to model important usecases in TPCH +// parameters: +// +// Input schemas +lazy_static! { + static ref I64_STREAMS: Vec> = i64_streams(); + static ref F64_STREAMS: Vec> = f64_streams(); + + static ref UTF8_LOW_CARDINALITY_STREAMS: Vec> = utf8_low_cardinality_streams(); + static ref UTF8_HIGH_CARDINALITY_STREAMS: Vec> = utf8_high_cardinality_streams(); + + static ref DICTIONARY_STREAMS: Vec> = dictionary_streams(); + static ref DICTIONARY_TUPLE_STREAMS: Vec> = dictionary_tuple_streams(); + static ref MIXED_DICTIONARY_TUPLE_STREAMS: Vec> = mixed_dictionary_tuple_streams(); + // * (string(low), string(low), string(high)) -- tpch q1 + iox + static ref UTF8_TUPLE_STREAMS: Vec> = utf8_tuple_streams(); + // * (f64, string, string, int) -- tpch q2 + static ref MIXED_TUPLE_STREAMS: Vec> = mixed_tuple_streams(); + +} + +fn criterion_benchmark(c: &mut Criterion) { + c.bench_function("sort i64", |b| { + let case = SortBenchCase::new(&I64_STREAMS); + + b.iter(move || case.run()) + }); + c.bench_function("sort i64 preserve partitioning", |b| { + let case = SortBenchCasePreservePartitioning::new(&I64_STREAMS); + + b.iter(move || case.run()) + }); + + c.bench_function("sort f64", |b| { + let case = SortBenchCase::new(&F64_STREAMS); + + b.iter(move || case.run()) + }); + c.bench_function("sort f64 preserve partitioning", |b| { + let case = SortBenchCasePreservePartitioning::new(&F64_STREAMS); + + b.iter(move || case.run()) + }); + + c.bench_function("sort utf8 low cardinality", |b| { + let case = SortBenchCase::new(&UTF8_LOW_CARDINALITY_STREAMS); + + b.iter(move || case.run()) + }); + c.bench_function("sort utf8 low cardinality preserve partitioning", |b| { + let case = SortBenchCase::new(&UTF8_LOW_CARDINALITY_STREAMS); + + b.iter(move || case.run()) + }); + + c.bench_function("sort utf8 high cardinality", |b| { + let case = SortBenchCase::new(&UTF8_HIGH_CARDINALITY_STREAMS); + + b.iter(move || case.run()) + }); + c.bench_function("sort utf8 high cardinality preserve partitioning", |b| { + let case = SortBenchCasePreservePartitioning::new(&UTF8_HIGH_CARDINALITY_STREAMS); + + b.iter(move || case.run()) + }); + + c.bench_function("sort utf8 tuple", |b| { + let case = SortBenchCase::new(&UTF8_TUPLE_STREAMS); + + b.iter(move || case.run()) + }); + c.bench_function("sort utf8 tuple preserve partitioning", |b| { + let case = SortBenchCasePreservePartitioning::new(&UTF8_TUPLE_STREAMS); + + b.iter(move || case.run()) + }); + + c.bench_function("sort utf8 dictionary", |b| { + let case = SortBenchCase::new(&DICTIONARY_STREAMS); + + b.iter(move || case.run()) + }); + c.bench_function("sort utf8 dictionary preserve partitioning", |b| { + let case = SortBenchCasePreservePartitioning::new(&DICTIONARY_STREAMS); + + b.iter(move || case.run()) + }); + + c.bench_function("sort utf8 dictionary tuple", |b| { + let case = SortBenchCase::new(&DICTIONARY_TUPLE_STREAMS); + b.iter(move || case.run()) + }); + c.bench_function("sort utf8 dictionary tuple preserve partitioning", |b| { + let case = SortBenchCasePreservePartitioning::new(&DICTIONARY_TUPLE_STREAMS); + b.iter(move || case.run()) + }); + + c.bench_function("sort mixed utf8 dictionary tuple", |b| { + let case = SortBenchCase::new(&MIXED_DICTIONARY_TUPLE_STREAMS); + b.iter(move || case.run()) + }); + + c.bench_function( + "sort mixed utf8 dictionary tuple preserve partitioning", + |b| { + let case = + SortBenchCasePreservePartitioning::new(&MIXED_DICTIONARY_TUPLE_STREAMS); + b.iter(move || case.run()) + }, + ); + + c.bench_function("sort mixed tuple", |b| { + let case = SortBenchCase::new(&MIXED_TUPLE_STREAMS); + + b.iter(move || case.run()) + }); + c.bench_function("sort mixed tuple preserve partitioning", |b| { + let case = SortBenchCasePreservePartitioning::new(&MIXED_TUPLE_STREAMS); + + b.iter(move || case.run()) + }); +} + +/// Encapsulates running a test case where input partitioning is not preserved. +struct SortBenchCase { + runtime: Runtime, + task_ctx: Arc, + + // The plan to run + plan: Arc, +} + +impl SortBenchCase { + /// Prepare to run a benchmark that merges the specified + /// partitions (streams) together using all keyes + fn new(partitions: &[Vec]) -> Self { + let runtime = tokio::runtime::Builder::new_multi_thread().build().unwrap(); + let session_ctx = SessionContext::new(); + let task_ctx = session_ctx.task_ctx(); + + let schema = partitions[0][0].schema(); + let sort = make_sort_exprs(schema.as_ref()); + + let projection = None; + let exec = MemoryExec::try_new(partitions, schema, projection).unwrap(); + let plan = Arc::new(SortExec::try_new(sort, Arc::new(exec), None).unwrap()); + + Self { + runtime, + task_ctx, + plan, + } + } + + /// runs the specified plan to completion, draining all input and + /// panic'ing on error + fn run(&self) { + let plan = Arc::clone(&self.plan); + let task_ctx = Arc::clone(&self.task_ctx); + + assert_eq!(plan.output_partitioning().partition_count(), 1); + + self.runtime.block_on(async move { + let mut stream = plan.execute(0, task_ctx).unwrap(); + while let Some(b) = stream.next().await { + b.expect("unexpected execution error"); + } + }) + } +} +/// Encapsulates running a test case where input partitioning is not preserved. +struct SortBenchCasePreservePartitioning { + runtime: Runtime, + task_ctx: Arc, + + // The plan to run + plan: Arc, + partition_count: usize, +} + +impl SortBenchCasePreservePartitioning { + /// Prepare to run a benchmark that merges the specified + /// partitions (streams) together using all keyes + fn new(partitions: &[Vec]) -> Self { + let partition_count = partitions.len(); + let runtime = tokio::runtime::Builder::new_multi_thread().build().unwrap(); + let session_ctx = SessionContext::new(); + let task_ctx = session_ctx.task_ctx(); + + let schema = partitions[0][0].schema(); + let sort = make_sort_exprs(schema.as_ref()); + + let projection = None; + let exec = MemoryExec::try_new(partitions, schema, projection).unwrap(); + let plan = Arc::new(SortExec::new_with_partitioning( + sort, + Arc::new(exec), + true, + None, + )); + + Self { + runtime, + task_ctx, + plan, + partition_count, + } + } + + /// runs the specified plan to completion, draining all input and + /// panic'ing on error + fn run(&self) { + let plan = Arc::clone(&self.plan); + let task_ctx = Arc::clone(&self.task_ctx); + + assert_eq!( + plan.output_partitioning().partition_count(), + self.partition_count + ); + + self.runtime.block_on(async move { + let mut stream = plan.execute(0, task_ctx).unwrap(); + while let Some(b) = stream.next().await { + b.expect("unexpected execution error"); + } + }) + } +} + +/// Make sort exprs for each column in `schema` +fn make_sort_exprs(schema: &Schema) -> Vec { + schema + .fields() + .iter() + .map(|f| PhysicalSortExpr { + expr: col(f.name(), schema).unwrap(), + options: SortOptions::default(), + }) + .collect() +} + +/// Create streams of int64 (where approximately 1/3 values is repeated) +fn i64_streams() -> Vec> { + let array: Int64Array = DataGenerator::new().i64_values().into_iter().collect(); + + let batch = RecordBatch::try_from_iter(vec![("i64", Arc::new(array) as _)]).unwrap(); + + split_batch(batch) +} + +/// Create streams of f64 (where approximately 1/3 values are repeated) +/// with the same distribution as i64_streams +fn f64_streams() -> Vec> { + let array: Float64Array = DataGenerator::new().f64_values().into_iter().collect(); + let batch = RecordBatch::try_from_iter(vec![("f64", Arc::new(array) as _)]).unwrap(); + + split_batch(batch) +} + +/// Create streams of random low cardinality utf8 values +fn utf8_low_cardinality_streams() -> Vec> { + let array: StringArray = DataGenerator::new() + .utf8_low_cardinality_values() + .into_iter() + .collect(); + + let batch = + RecordBatch::try_from_iter(vec![("utf_low", Arc::new(array) as _)]).unwrap(); + + split_batch(batch) +} + +/// Create streams of high cardinality (~ no duplicates) utf8 values +fn utf8_high_cardinality_streams() -> Vec> { + let array: StringArray = DataGenerator::new() + .utf8_high_cardinality_values() + .into_iter() + .collect(); + + let batch = + RecordBatch::try_from_iter(vec![("utf_high", Arc::new(array) as _)]).unwrap(); + + split_batch(batch) +} + +/// Create a batch of (utf8_low, utf8_low, utf8_high) +fn utf8_tuple_streams() -> Vec> { + let mut gen = DataGenerator::new(); + + // need to sort by the combined key, so combine them together + let mut tuples: Vec<_> = gen + .utf8_low_cardinality_values() + .into_iter() + .zip(gen.utf8_low_cardinality_values().into_iter()) + .zip(gen.utf8_high_cardinality_values().into_iter()) + .collect(); + + tuples.sort_unstable(); + + let (tuples, utf8_high): (Vec<_>, Vec<_>) = tuples.into_iter().unzip(); + let (utf8_low1, utf8_low2): (Vec<_>, Vec<_>) = tuples.into_iter().unzip(); + + let utf8_high: StringArray = utf8_high.into_iter().collect(); + let utf8_low1: StringArray = utf8_low1.into_iter().collect(); + let utf8_low2: StringArray = utf8_low2.into_iter().collect(); + + let batch = RecordBatch::try_from_iter(vec![ + ("utf_low1", Arc::new(utf8_low1) as _), + ("utf_low2", Arc::new(utf8_low2) as _), + ("utf_high", Arc::new(utf8_high) as _), + ]) + .unwrap(); + + split_batch(batch) +} + +/// Create a batch of (f64, utf8_low, utf8_low, i64) +fn mixed_tuple_streams() -> Vec> { + let mut gen = DataGenerator::new(); + + // need to sort by the combined key, so combine them together + let mut tuples: Vec<_> = gen + .i64_values() + .into_iter() + .zip(gen.utf8_low_cardinality_values().into_iter()) + .zip(gen.utf8_low_cardinality_values().into_iter()) + .zip(gen.i64_values().into_iter()) + .collect(); + tuples.sort_unstable(); + + let (tuples, i64_values): (Vec<_>, Vec<_>) = tuples.into_iter().unzip(); + let (tuples, utf8_low2): (Vec<_>, Vec<_>) = tuples.into_iter().unzip(); + let (f64_values, utf8_low1): (Vec<_>, Vec<_>) = tuples.into_iter().unzip(); + + let f64_values: Float64Array = f64_values.into_iter().map(|v| v as f64).collect(); + let utf8_low1: StringArray = utf8_low1.into_iter().collect(); + let utf8_low2: StringArray = utf8_low2.into_iter().collect(); + let i64_values: Int64Array = i64_values.into_iter().collect(); + + let batch = RecordBatch::try_from_iter(vec![ + ("f64", Arc::new(f64_values) as _), + ("utf_low1", Arc::new(utf8_low1) as _), + ("utf_low2", Arc::new(utf8_low2) as _), + ("i64", Arc::new(i64_values) as _), + ]) + .unwrap(); + + split_batch(batch) +} + +/// Create a batch of (utf8_dict) +fn dictionary_streams() -> Vec> { + let mut gen = DataGenerator::new(); + let values = gen.utf8_low_cardinality_values(); + let dictionary: DictionaryArray = + values.iter().map(Option::as_deref).collect(); + + let batch = + RecordBatch::try_from_iter(vec![("dict", Arc::new(dictionary) as _)]).unwrap(); + + split_batch(batch) +} + +/// Create a batch of (utf8_dict, utf8_dict, utf8_dict) +fn dictionary_tuple_streams() -> Vec> { + let mut gen = DataGenerator::new(); + let mut tuples: Vec<_> = gen + .utf8_low_cardinality_values() + .into_iter() + .zip(gen.utf8_low_cardinality_values()) + .zip(gen.utf8_low_cardinality_values()) + .collect(); + tuples.sort_unstable(); + + let (tuples, c): (Vec<_>, Vec<_>) = tuples.into_iter().unzip(); + let (a, b): (Vec<_>, Vec<_>) = tuples.into_iter().unzip(); + + let a: DictionaryArray = a.iter().map(Option::as_deref).collect(); + let b: DictionaryArray = b.iter().map(Option::as_deref).collect(); + let c: DictionaryArray = c.iter().map(Option::as_deref).collect(); + + let batch = RecordBatch::try_from_iter(vec![ + ("a", Arc::new(a) as _), + ("b", Arc::new(b) as _), + ("c", Arc::new(c) as _), + ]) + .unwrap(); + + split_batch(batch) +} + +/// Create a batch of (utf8_dict, utf8_dict, utf8_dict, i64) +fn mixed_dictionary_tuple_streams() -> Vec> { + let mut gen = DataGenerator::new(); + let mut tuples: Vec<_> = gen + .utf8_low_cardinality_values() + .into_iter() + .zip(gen.utf8_low_cardinality_values()) + .zip(gen.utf8_low_cardinality_values()) + .zip(gen.i64_values()) + .collect(); + tuples.sort_unstable(); + + let (tuples, d): (Vec<_>, Vec<_>) = tuples.into_iter().unzip(); + let (tuples, c): (Vec<_>, Vec<_>) = tuples.into_iter().unzip(); + let (a, b): (Vec<_>, Vec<_>) = tuples.into_iter().unzip(); + + let a: DictionaryArray = a.iter().map(Option::as_deref).collect(); + let b: DictionaryArray = b.iter().map(Option::as_deref).collect(); + let c: DictionaryArray = c.iter().map(Option::as_deref).collect(); + let d: Int64Array = d.into_iter().collect(); + + let batch = RecordBatch::try_from_iter(vec![ + ("a", Arc::new(a) as _), + ("b", Arc::new(b) as _), + ("c", Arc::new(c) as _), + ("d", Arc::new(d) as _), + ]) + .unwrap(); + + split_batch(batch) +} + +/// Encapsulates creating data for this test +struct DataGenerator { + rng: StdRng, +} + +impl DataGenerator { + fn new() -> Self { + Self { + rng: StdRng::seed_from_u64(42), + } + } + + /// Create an array of i64 unsorted values (where approximately 1/3 values is repeated) + fn i64_values(&mut self) -> Vec { + let vec: Vec<_> = (0..INPUT_SIZE) + .map(|_| self.rng.gen_range(0..INPUT_SIZE as i64)) + .collect(); + + vec + } + + /// Create an array of f64 sorted values (with same distribution of `i64_values`) + fn f64_values(&mut self) -> Vec { + self.i64_values().into_iter().map(|v| v as f64).collect() + } + + /// array of low cardinality (100 distinct) values + fn utf8_low_cardinality_values(&mut self) -> Vec>> { + let strings = (0..100) + .map(|s| format!("value{s}").into()) + .collect::>(); + + // pick from the 100 strings randomly + let input = (0..INPUT_SIZE) + .map(|_| { + let idx = self.rng.gen_range(0..strings.len()); + let s = Arc::clone(&strings[idx]); + Some(s) + }) + .collect::>(); + input + } + + /// Create values of high cardinality (~ no duplicates) utf8 values + fn utf8_high_cardinality_values(&mut self) -> Vec> { + // make random strings + let input = (0..INPUT_SIZE) + .map(|_| Some(self.random_string())) + .collect::>(); + input + } + + fn random_string(&mut self) -> String { + let rng = &mut self.rng; + rng.sample_iter(rand::distributions::Alphanumeric) + .filter(|c| c.is_ascii_alphabetic()) + .take(20) + .map(char::from) + .collect::() + } +} + +/// Splits the `input_batch` randomly into `NUM_STREAMS` approximately evenly sorted streams +fn split_batch(input_batch: RecordBatch) -> Vec> { + // figure out which inputs go where + let mut rng = StdRng::seed_from_u64(1337); + + // randomly assign rows to streams + let stream_assignments = (0..input_batch.num_rows()) + .map(|_| rng.gen_range(0..NUM_STREAMS)) + .collect(); + + // split the inputs into streams + (0..NUM_STREAMS) + .map(|stream| { + // make a "stream" of 1 record batch + vec![take_columns(&input_batch, &stream_assignments, stream)] + }) + .collect::>() +} + +/// returns a record batch that contains all there values where +/// stream_assignment[i] = stream (aka this is the equivalent of +/// calling take(indicies) where indicies[i] == stream_index) +fn take_columns( + input_batch: &RecordBatch, + stream_assignments: &UInt64Array, + stream: u64, +) -> RecordBatch { + // find just the indicies needed from record batches to extract + let stream_indices: UInt64Array = stream_assignments + .iter() + .enumerate() + .filter_map(|(idx, stream_idx)| { + if stream_idx.unwrap() == stream { + Some(idx as u64) + } else { + None + } + }) + .collect(); + + let options = Some(TakeOptions { check_bounds: true }); + + // now, get the columns from each array + let new_columns = input_batch + .columns() + .iter() + .map(|array| compute::take(array, &stream_indices, options.clone()).unwrap()) + .collect(); + + RecordBatch::try_new(input_batch.schema(), new_columns).unwrap() +} + +criterion_group!(benches, criterion_benchmark); +criterion_main!(benches); From 11be061ddf0cd4879f5662c4f43dba0c85d2bd0c Mon Sep 17 00:00:00 2001 From: Jay Miller <3744812+jaylmiller@users.noreply.github.com> Date: Mon, 13 Feb 2023 13:53:09 -0500 Subject: [PATCH 05/29] fix preserve partitioning case to run every partition instead of just first --- datafusion/core/benches/sort.rs | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/datafusion/core/benches/sort.rs b/datafusion/core/benches/sort.rs index 801b0959e6a6..e127c08ccd8a 100644 --- a/datafusion/core/benches/sort.rs +++ b/datafusion/core/benches/sort.rs @@ -275,9 +275,11 @@ impl SortBenchCasePreservePartitioning { ); self.runtime.block_on(async move { - let mut stream = plan.execute(0, task_ctx).unwrap(); - while let Some(b) = stream.next().await { - b.expect("unexpected execution error"); + for i in 0..self.partition_count { + let mut stream = plan.execute(i, task_ctx.clone()).unwrap(); + while let Some(b) = stream.next().await { + b.expect("unexpected execution error"); + } } }) } From 730a89c816b7cdc65f9518d96f8afdc1115cafcf Mon Sep 17 00:00:00 2001 From: Jay Miller <3744812+jaylmiller@users.noreply.github.com> Date: Sun, 12 Feb 2023 10:42:53 -0500 Subject: [PATCH 06/29] rough draft: sorting works. still need to update metrics tracking and spilling row format to disk --- .../core/src/physical_plan/sorts/cursor.rs | 8 +- .../core/src/physical_plan/sorts/mod.rs | 242 +++++++++++- .../core/src/physical_plan/sorts/sort.rs | 358 +++++++++++++----- .../sorts/sort_preserving_merge.rs | 310 +++++++++------ datafusion/core/tests/sort_key_cursor.rs | 2 +- 5 files changed, 702 insertions(+), 218 deletions(-) diff --git a/datafusion/core/src/physical_plan/sorts/cursor.rs b/datafusion/core/src/physical_plan/sorts/cursor.rs index 53df698c33ac..cb3dbf2d337b 100644 --- a/datafusion/core/src/physical_plan/sorts/cursor.rs +++ b/datafusion/core/src/physical_plan/sorts/cursor.rs @@ -15,9 +15,11 @@ // specific language governing permissions and limitations // under the License. -use arrow::row::{Row, Rows}; +use arrow::row::Row; use std::cmp::Ordering; +use super::RowBatch; + /// A `SortKeyCursor` is created from a `RecordBatch`, and a set of /// `PhysicalExpr` that when evaluated on the `RecordBatch` yield the sort keys. /// @@ -35,7 +37,7 @@ pub struct SortKeyCursor { // An id uniquely identifying the record batch scanned by this cursor. batch_id: usize, - rows: Rows, + rows: RowBatch, } impl std::fmt::Debug for SortKeyCursor { @@ -50,7 +52,7 @@ impl std::fmt::Debug for SortKeyCursor { impl SortKeyCursor { /// Create a new SortKeyCursor - pub fn new(stream_idx: usize, batch_id: usize, rows: Rows) -> Self { + pub fn new(stream_idx: usize, batch_id: usize, rows: RowBatch) -> Self { Self { stream_idx, cur_row: 0, diff --git a/datafusion/core/src/physical_plan/sorts/mod.rs b/datafusion/core/src/physical_plan/sorts/mod.rs index db6ab5c604e2..e8816e83b2d7 100644 --- a/datafusion/core/src/physical_plan/sorts/mod.rs +++ b/datafusion/core/src/physical_plan/sorts/mod.rs @@ -17,30 +17,258 @@ //! Sort functionalities -use crate::physical_plan::SendableRecordBatchStream; -use std::fmt::{Debug, Formatter}; +use crate::{error::Result, physical_plan::SendableRecordBatchStream}; +use std::{ + fmt::{Debug, Formatter}, + pin::Pin, + sync::Arc, +}; mod cursor; mod index; pub mod sort; pub mod sort_preserving_merge; +use arrow::{ + record_batch::RecordBatch, + row::{Row, Rows}, +}; pub use cursor::SortKeyCursor; +use futures::{stream, Stream, StreamExt}; pub use index::RowIndex; +pub(crate) type RowStream = Pin> + Send>>; +pub(crate) type SortStreamItem = Result<(RecordBatch, Option)>; +pub(crate) type SendableSortStream = Pin + Send>>; pub(crate) struct SortedStream { - stream: SendableRecordBatchStream, + stream: SendableSortStream, mem_used: usize, } - +impl SortedStream { + pub(crate) fn new( + stream: SendableRecordBatchStream, + mem_used: usize, + row_stream: RowStream, + ) -> Self { + let stream = Box::pin(stream.zip(row_stream).map(|item| { + let batch: Result = item.0; + let rows: Option = item.1; + match batch { + Ok(batch) => Ok((batch, rows)), + Err(err) => Err(err), + } + })); + Self { stream, mem_used } + } + /// create stream where the row encoding for each batch is always None + pub(crate) fn new_no_row_encoding( + stream: SendableRecordBatchStream, + mem_used: usize, + ) -> Self { + Self::new( + stream, + mem_used, + // stream will end as soon as the record batch stream ends + Box::pin(stream::repeat(None)), + ) + } +} impl Debug for SortedStream { fn fmt(&self, f: &mut Formatter) -> std::fmt::Result { write!(f, "InMemSorterStream") } } -impl SortedStream { - pub(crate) fn new(stream: SendableRecordBatchStream, mem_used: usize) -> Self { - Self { stream, mem_used } +/// A batch of rows taken from multiple [RowSelection]s +#[derive(Debug, Clone)] +pub struct RowBatch { + // refs to the rows referenced by `indices` + rows: Vec>, + // first item = index of the ref in `rows`, second item=index within that row + // TODO: make this field optional for case where the RowBatch is just one `Rows` + // and it is in the exact same order to save memory. + indices: Vec<(usize, usize)>, +} + +impl RowBatch { + /// Create new batch of rows selected from `rows`. + /// + /// `indices` defines where each row comes from: first element of the tuple is the index + /// of the ref in `rows`, second is the index within that `RowSelection`. + pub fn new(rows: Vec>, indices: Vec<(usize, usize)>) -> Self { + Self { rows, indices } + } + + /// Returns the nth row in the batch. + pub fn row(&self, n: usize) -> Row { + let (rows_ref_idx, row_idx) = self.indices[n]; + self.rows[rows_ref_idx].row(row_idx) + } + + /// Number of rows selected + pub fn num_rows(&self) -> usize { + self.indices.len() + } + /// Iterate over rows in their selected order + pub fn iter(&self) -> RowBatchIter { + RowBatchIter { + row_selection: &self, + cur_idx: 0, + } + } + /// Amount of bytes + pub fn memory_size(&self) -> usize { + let indices_size = self.indices.len() * 2 * std::mem::size_of::(); + let rows_size = self.rows.iter().map(|r| r.size()).sum::(); + rows_size + indices_size + std::mem::size_of::() + } +} +impl From for RowBatch { + fn from(value: RowSelection) -> Self { + Self { + indices: (0..value.indices.len()).map(|i| (0, i)).collect(), + rows: vec![Arc::new(value)], + } + } +} +impl From for RowBatch { + fn from(value: Rows) -> Self { + Into::::into(value).into() + } +} + +/// Iterate over each row in a [`RowBatch`] +pub struct RowBatchIter<'a> { + row_selection: &'a RowBatch, + cur_idx: usize, +} +impl<'a> Iterator for RowBatchIter<'a> { + type Item = Row<'a>; + + fn next(&mut self) -> Option { + if self.cur_idx < self.row_selection.num_rows() { + let row = self.row_selection.row(self.cur_idx); + self.cur_idx += 1; + Some(row) + } else { + None + } + } +} + +/// A sorted selection of rows from the same [`Rows`]. +#[derive(Debug)] +pub struct RowSelection { + rows: Rows, + // todo: make None in case where RowSelection is equivalent to Rows + // to save memory + indices: Vec, +} +impl RowSelection { + /// New + pub fn new(rows: Rows, indices: Vec) -> Self { + Self { rows, indices } + } + /// Get the nth row of the selection. + pub fn row(&self, n: usize) -> Row { + let idx = self.indices[n]; + self.rows.row(idx) + } + /// Iterate over the rows in the selected order. + pub fn iter(&self) -> RowSelectionIter { + RowSelectionIter { + row_selection: &self, + cur_n: 0, + } + } + /// Number of bytes held in rows and indices. + pub fn size(&self) -> usize { + self.rows.size() + + self.indices.len() * std::mem::size_of::() + + std::mem::size_of::() + } +} +impl From for RowSelection { + fn from(value: Rows) -> Self { + Self { + indices: (0..value.num_rows()).collect(), + rows: value, + } + } +} +/// Iterator for [`SortedRows`] +pub struct RowSelectionIter<'a> { + row_selection: &'a RowSelection, + cur_n: usize, +} +impl<'a> Iterator for RowSelectionIter<'a> { + type Item = Row<'a>; + + fn next(&mut self) -> Option { + if self.cur_n < self.row_selection.indices.len() { + let row = self.row_selection.row(self.cur_n); + self.cur_n += 1; + Some(row) + } else { + None + } + } +} + +#[cfg(test)] +mod tests { + use arrow::{ + array::Int64Array, + datatypes::DataType, + record_batch::RecordBatch, + row::{RowConverter, SortField}, + }; + + use crate::assert_batches_eq; + + use super::*; + + fn int64_rows( + conv: &mut RowConverter, + values: impl IntoIterator, + ) -> Rows { + let array: Int64Array = values.into_iter().map(Some).collect(); + let batch = + RecordBatch::try_from_iter(vec![("c1", Arc::new(array) as _)]).unwrap(); + conv.convert_columns(batch.columns()).unwrap() + } + + #[test] + fn test_row_batch_and_sorted_rows() { + let mut conv = RowConverter::new(vec![SortField::new(DataType::Int64)]).unwrap(); + let s1 = RowSelection::new(int64_rows(&mut conv, 0..3), vec![2, 2, 1]); + let s2 = RowSelection::new(int64_rows(&mut conv, 5..8), vec![1, 2, 0]); + let s3: RowSelection = int64_rows(&mut conv, 2..4).into(); + let selection = RowBatch::new( + vec![s1, s2, s3].into_iter().map(Arc::new).collect(), + vec![ + (2, 0), // 2 + (0, 2), // 1 + (0, 0), // 2 + (1, 1), // 7 + ], + ); + let rows: Vec = selection.iter().collect(); + assert_eq!(rows.len(), 4); + let parsed = conv.convert_rows(rows).unwrap(); + let batch = + RecordBatch::try_from_iter(vec![("c1", parsed.get(0).unwrap().clone())]) + .unwrap(); + let expected = vec![ + "+----+", // + "| c1 |", // + "+----+", // + "| 2 |", // + "| 1 |", // + "| 2 |", // + "| 7 |", // + "+----+", + ]; + assert_batches_eq!(expected, &[batch]); } } diff --git a/datafusion/core/src/physical_plan/sorts/sort.rs b/datafusion/core/src/physical_plan/sorts/sort.rs index 286bfab02d12..45480a953d93 100644 --- a/datafusion/core/src/physical_plan/sorts/sort.rs +++ b/datafusion/core/src/physical_plan/sorts/sort.rs @@ -19,6 +19,8 @@ //! It will do in-memory sorting if it has enough memory budget //! but spills to disk if needed. +use super::{RowBatch, RowSelection, RowStream}; +use super::{SendableSortStream, SortStreamItem}; use crate::error::{DataFusionError, Result}; use crate::execution::context::TaskContext; use crate::execution::memory_pool::{ @@ -34,7 +36,7 @@ use crate::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeStrea use crate::physical_plan::sorts::SortedStream; use crate::physical_plan::stream::{RecordBatchReceiverStream, RecordBatchStreamAdapter}; use crate::physical_plan::{ - DisplayFormatType, Distribution, EmptyRecordBatchStream, ExecutionPlan, Partitioning, + displayable, DisplayFormatType, Distribution, ExecutionPlan, Partitioning, RecordBatchStream, SendableRecordBatchStream, Statistics, }; use crate::prelude::SessionConfig; @@ -59,8 +61,9 @@ use std::path::{Path, PathBuf}; use std::sync::Arc; use std::task::{Context, Poll}; use tempfile::NamedTempFile; -use tokio::sync::mpsc::{Receiver, Sender}; +use tokio::sync::mpsc::{self, Receiver, Sender}; use tokio::task; +use tokio_stream::wrappers::UnboundedReceiverStream; /// Sort arbitrary size of data to get a total order (may spill several times during sorting based on free memory available). /// @@ -101,7 +104,6 @@ impl ExternalSorter { let reservation = MemoryConsumer::new(format!("ExternalSorter[{partition_id}]")) .with_can_spill(true) .register(&runtime.memory_pool); - Self { schema, in_mem_batches: vec![], @@ -166,7 +168,9 @@ impl ExternalSorter { } /// MergeSort in mem batches as well as spills into total order with `SortPreservingMergeStream`. - fn sort(&mut self) -> Result { + /// + /// todo: add flag to specify that the row encoding should not be preserved (to save memory) + fn sort(&mut self) -> Result { let batch_size = self.session_config.batch_size(); if self.spilled_before() { @@ -175,7 +179,7 @@ impl ExternalSorter { .new_intermediate_tracking(self.partition_id, &self.runtime.memory_pool); let mut streams: Vec = vec![]; if !self.in_mem_batches.is_empty() { - let in_mem_stream = in_mem_partial_sort( + let (in_mem_stream, in_mem_rows) = in_mem_partial_sort( &mut self.in_mem_batches, self.schema.clone(), &self.expr, @@ -184,12 +188,17 @@ impl ExternalSorter { self.fetch, )?; let prev_used = self.reservation.free(); - streams.push(SortedStream::new(in_mem_stream, prev_used)); + streams.push(SortedStream::new(in_mem_stream, prev_used, in_mem_rows)); } for spill in self.spills.drain(..) { - let stream = read_spill_as_stream(spill, self.schema.clone())?; - streams.push(SortedStream::new(stream, 0)); + let (tx, rx) = mpsc::unbounded_channel(); + let stream = read_spill_as_stream(spill, self.schema.clone(), tx)?; + streams.push(SortedStream::new( + stream, + 0, + UnboundedReceiverStream::new(rx).boxed(), + )); } let tracking_metrics = self .metrics_set @@ -202,22 +211,29 @@ impl ExternalSorter { self.session_config.batch_size(), )?)) } else if !self.in_mem_batches.is_empty() { + // sort in mem doesnt require SortPreservingMergeStream let tracking_metrics = self .metrics_set .new_final_tracking(self.partition_id, &self.runtime.memory_pool); - let result = in_mem_partial_sort( + let (stream, row_stream) = in_mem_partial_sort( &mut self.in_mem_batches, self.schema.clone(), &self.expr, batch_size, tracking_metrics, self.fetch, - ); + )?; // Report to the memory manager we are no longer using memory self.reservation.free(); - result + let output_stream = stream.zip(row_stream).map(|i| { + let rec_batch_result: Result = i.0; + let maybe_rows = i.1; + rec_batch_result.map(|batch| (batch, maybe_rows)) + }); + Ok(Box::pin(output_stream)) } else { - Ok(Box::pin(EmptyRecordBatchStream::new(self.schema.clone()))) + todo!() + // Ok(Box::pin(EmptyRecordBatchStream::new(self.schema.clone()))) } } @@ -238,24 +254,22 @@ impl ExternalSorter { if self.in_mem_batches.is_empty() { return Ok(0); } - debug!("Spilling sort data of ExternalSorter to disk whilst inserting"); let tracking_metrics = self .metrics_set .new_intermediate_tracking(self.partition_id, &self.runtime.memory_pool); - let spillfile = self.runtime.disk_manager.create_tmp_file("Sorting")?; - let stream = in_mem_partial_sort( + // TODO: spill row data + let (mut stream, _row_stream) = in_mem_partial_sort( &mut self.in_mem_batches, self.schema.clone(), &self.expr, self.session_config.batch_size(), tracking_metrics, self.fetch, - ); - - spill_partial_sorted_stream(&mut stream?, spillfile.path(), self.schema.clone()) + )?; + spill_partial_sorted_stream(&mut stream, spillfile.path(), self.schema.clone()) .await?; self.reservation.free(); let used = self.metrics.mem_used().set(0); @@ -283,41 +297,57 @@ fn in_mem_partial_sort( batch_size: usize, tracking_metrics: MemTrackingMetrics, fetch: Option, -) -> Result { +) -> Result<(SendableRecordBatchStream, RowStream)> { + let (row_tx, row_rx) = mpsc::unbounded_channel(); assert_ne!(buffered_batches.len(), 0); if buffered_batches.len() == 1 { - let result = buffered_batches.pop(); - Ok(Box::pin(SizedRecordBatchStream::new( - schema, - vec![Arc::new(result.unwrap().sorted_batch)], - tracking_metrics, - ))) + let result = buffered_batches.pop().unwrap(); + let BatchWithSortArray { + sort_data, + sorted_batch, + } = result; + let rowbatch: Option = sort_data.rows.map(Into::into); + Ok(( + Box::pin(SizedRecordBatchStream::new( + schema, + vec![Arc::new(sorted_batch)], + tracking_metrics, + )), + Box::pin(futures::stream::once(futures::future::ready(rowbatch))), + )) } else { - let (sorted_arrays, batches): (Vec>, Vec) = - buffered_batches - .drain(..) - .into_iter() - .map(|b| { - let BatchWithSortArray { - sort_arrays, - sorted_batch: batch, - } = b; - (sort_arrays, batch) - }) - .unzip(); + let (sort_data, batches): (Vec, Vec) = buffered_batches + .drain(..) + .into_iter() + .map(|b| { + let BatchWithSortArray { + sort_data, + sorted_batch: batch, + } = b; + (sort_data, batch) + }) + .unzip(); let sorted_iter = { // NB timer records time taken on drop, so there are no // calls to `timer.done()` below. let _timer = tracking_metrics.elapsed_compute().timer(); - get_sorted_iter(&sorted_arrays, expressions, batch_size, fetch)? + get_sorted_iter(&sort_data, expressions, batch_size, fetch)? }; - Ok(Box::pin(SortedSizedRecordBatchStream::new( + let rows = sort_data + .into_iter() + .map(|d| d.rows) + .collect::>>(); + let batch_stream = Box::pin(SortedSizedRecordBatchStream::new( schema, batches, sorted_iter, tracking_metrics, - ))) + rows.map(|rs| rs.into_iter().map(Arc::new).collect()), + Some(row_tx), + )); + let row_stream = UnboundedReceiverStream::new(row_rx).boxed(); + Ok((batch_stream, row_stream)) } } @@ -329,16 +359,16 @@ struct CompositeIndex { /// Get sorted iterator by sort concatenated `SortColumn`s fn get_sorted_iter( - sort_arrays: &[Vec], + sort_data: &[SortData], expr: &[PhysicalSortExpr], batch_size: usize, fetch: Option, ) -> Result { - let row_indices = sort_arrays + let row_indices = sort_data .iter() .enumerate() - .flat_map(|(i, arrays)| { - (0..arrays[0].len()).map(move |r| CompositeIndex { + .flat_map(|(i, d)| { + (0..d.arrays[0].len()).map(move |r| CompositeIndex { // since we original use UInt32Array to index the combined mono batch, // component record batches won't overflow as well, // use u32 here for space efficiency. @@ -347,22 +377,41 @@ fn get_sorted_iter( }) }) .collect::>(); - - let sort_columns = expr - .iter() - .enumerate() - .map(|(i, expr)| { - let columns_i = sort_arrays + let rows_per_batch: Option> = + sort_data.iter().map(|d| d.rows.as_ref()).collect(); + let indices = match rows_per_batch { + Some(rows_per_batch) => { + // concat rows in their selection order and then sort + let mut to_sort = rows_per_batch .iter() - .map(|cs| cs[i].as_ref()) - .collect::>(); - Ok(SortColumn { - values: concat(columns_i.as_slice())?, - options: Some(expr.options), - }) - }) - .collect::>>()?; - let indices = lexsort_to_indices(&sort_columns, fetch)?; + .flat_map(|r| r.iter()) + .enumerate() + .collect::>(); + to_sort.sort_unstable_by(|(_, row_a), (_, row_b)| row_a.cmp(row_b)); + let limit = match fetch { + Some(lim) => lim.min(to_sort.len()), + None => to_sort.len(), + }; + UInt32Array::from_iter(to_sort.iter().take(limit).map(|(idx, _)| *idx as u32)) + } + None => { + let sort_columns = expr + .iter() + .enumerate() + .map(|(i, expr)| { + let columns_i = sort_data + .iter() + .map(|data| data.arrays[i].as_ref()) + .collect::>(); + Ok(SortColumn { + values: concat(columns_i.as_slice())?, + options: Some(expr.options), + }) + }) + .collect::>>()?; + lexsort_to_indices(&sort_columns, fetch)? + } + }; // Calculate composite index based on sorted indices let row_indices = indices @@ -479,6 +528,8 @@ struct SortedSizedRecordBatchStream { sorted_iter: SortedIterator, num_cols: usize, metrics: MemTrackingMetrics, + rows: Option>>, + rows_tx: Option>>, } impl SortedSizedRecordBatchStream { @@ -488,17 +539,26 @@ impl SortedSizedRecordBatchStream { batches: Vec, sorted_iter: SortedIterator, mut metrics: MemTrackingMetrics, + rows: Option>>, + rows_tx: Option>>, ) -> Self { let size = batches.iter().map(batch_byte_size).sum::() - + sorted_iter.memory_size(); + + sorted_iter.memory_size() + // include rows if non-None + + rows + .as_ref() + .map(|r| r.iter().map(|r| r.size()).sum()) + .unwrap_or(0); metrics.init_mem_used(size); let num_cols = batches[0].num_columns(); SortedSizedRecordBatchStream { schema, batches, sorted_iter, + rows, num_cols, metrics, + rows_tx, } } } @@ -514,6 +574,7 @@ impl Stream for SortedSizedRecordBatchStream { None => Poll::Ready(None), Some(slices) => { let num_rows = slices.iter().map(|s| s.len).sum(); + // create columns for record batch let output = (0..self.num_cols) .map(|i| { let arrays = self @@ -534,8 +595,37 @@ impl Stream for SortedSizedRecordBatchStream { .collect::>(); let batch = RecordBatch::try_new(self.schema.clone(), output).map_err(Into::into); - let poll = Poll::Ready(Some(batch)); - self.metrics.record_poll(poll) + match batch { + Ok(batch) => { + // construct `RowBatch` batch if sorted row encodings were preserved + let row_batch = self.rows.as_ref().map(|rows| { + let row_refs = + rows.iter().map(Arc::clone).collect::>(); + let indices = slices + .iter() + .flat_map(|s| { + (0..s.len).map(|i| { + ( + s.batch_idx as usize, + s.start_row_idx as usize + i, + ) + }) + }) + .collect::>(); + RowBatch::new(row_refs, indices) + }); + + if let Some(ref tx) = self.rows_tx { + tx.send(row_batch).unwrap(); + } + let poll = Poll::Ready(Some(Ok(batch))); + self.metrics.record_poll(poll) + } + Err(err) => { + let poll = Poll::Ready(Some(Err(err))); + self.metrics.record_poll(poll) + } + } } } } @@ -558,7 +648,7 @@ async fn spill_partial_sorted_stream( path: &Path, schema: SchemaRef, ) -> Result<()> { - let (sender, receiver) = tokio::sync::mpsc::channel(2); + let (sender, receiver) = mpsc::channel(2); let path: PathBuf = path.into(); let handle = task::spawn_blocking(move || write_sorted(receiver, path, schema)); while let Some(item) = in_mem_stream.next().await { @@ -576,11 +666,12 @@ async fn spill_partial_sorted_stream( fn read_spill_as_stream( path: NamedTempFile, schema: SchemaRef, + tx: mpsc::UnboundedSender>, ) -> Result { let (sender, receiver): (Sender>, Receiver>) = - tokio::sync::mpsc::channel(2); + mpsc::channel(2); let join_handle = task::spawn_blocking(move || { - if let Err(e) = read_spill(sender, path.path()) { + if let Err(e) = read_spill(sender, path.path(), tx) { error!("Failure while reading spill file: {:?}. Error: {}", path, e); } }); @@ -610,10 +701,16 @@ fn write_sorted( Ok(()) } -fn read_spill(sender: Sender>, path: &Path) -> Result<()> { +fn read_spill( + sender: Sender>, + path: &Path, + tx: mpsc::UnboundedSender>, +) -> Result<()> { let file = BufReader::new(File::open(path)?); let reader = FileReader::try_new(file, None)?; for batch in reader { + // TODO: read spilled row data + tx.send(None).unwrap(); sender .blocking_send(batch.map_err(Into::into)) .map_err(|e| DataFusionError::Execution(format!("{e}")))?; @@ -667,7 +764,6 @@ impl SortExec { fetch, } } - /// Input schema pub fn input(&self) -> &Arc { &self.input @@ -682,6 +778,80 @@ impl SortExec { pub fn fetch(&self) -> Option { self.fetch } + /// to be used by parent nodes to run execute that incldues the row + /// encodings in the result stream + pub(crate) fn execute_save_row_encoding( + &self, + partition: usize, + context: Arc, + ) -> Result { + debug!("Start SortExec::execute for partition {} of context session_id {} and task_id {:?}", partition, context.session_id(), context.task_id()); + + debug!( + "Start invoking SortExec's input.execute for partition: {}", + partition + ); + + let input = self.input.execute(partition, context.clone())?; + + debug!("End SortExec's input.execute for partition: {}", partition); + Ok(Box::pin( + futures::stream::once(do_sort( + input, + partition, + self.expr.clone(), + self.metrics_set.clone(), + context, + self.fetch(), + )) + .try_flatten(), + )) + } + /// to be used by parent nodes to spawn execution into tokio threadpool + /// and write results to `tx` + pub(crate) fn execution_spawn_task( + &self, + partition: usize, + context: Arc, + tx: mpsc::Sender, + ) -> tokio::task::JoinHandle<()> { + // create owned vars for task + let input = self.input.clone(); + let expr = self.expr.clone(); + let metrics = self.metrics_set.clone(); + let fetch = self.fetch(); + let disp = displayable(input.as_ref()).one_line().to_string(); + tokio::spawn(async move { + debug!("Start SortExec::execute for partition {} of context session_id {} and task_id {:?}", partition, context.session_id(), context.task_id()); + + debug!( + "Start invoking SortExec's input.execute for partition: {}", + partition + ); + let input = match input.execute(partition, context.clone()) { + Err(e) => { + tx.send(Err(e)).await.ok(); + return; + } + Ok(stream) => stream, + }; + debug!("End SortExec's input.execute for partition: {}", partition); + let mut sort_item_stream = + match do_sort(input, partition, expr, metrics, context, fetch).await { + Ok(stream) => stream, + Err(err) => { + tx.send(Err(err)).await.ok(); + return; + } + }; + while let Some(item) = sort_item_stream.next().await { + if tx.send(item).await.is_err() { + debug!("Stopping execution: output is gone, plan cancelling: {disp}"); + return; + } + } + }) + } } impl ExecutionPlan for SortExec { @@ -758,28 +928,11 @@ impl ExecutionPlan for SortExec { partition: usize, context: Arc, ) -> Result { - debug!("Start SortExec::execute for partition {} of context session_id {} and task_id {:?}", partition, context.session_id(), context.task_id()); - - debug!( - "Start invoking SortExec's input.execute for partition: {}", - partition - ); - - let input = self.input.execute(partition, context.clone())?; - - debug!("End SortExec's input.execute for partition: {}", partition); - Ok(Box::pin(RecordBatchStreamAdapter::new( self.schema(), - futures::stream::once(do_sort( - input, - partition, - self.expr.clone(), - self.metrics_set.clone(), - context, - self.fetch(), - )) - .try_flatten(), + self.execute_save_row_encoding(partition, context)? + // take the record batch and ignore the rows + .map_ok(|(record_batch, _rows)| record_batch), ))) } @@ -805,8 +958,12 @@ impl ExecutionPlan for SortExec { } } +struct SortData { + arrays: Vec, + rows: Option, +} struct BatchWithSortArray { - sort_arrays: Vec, + sort_data: SortData, sorted_batch: RecordBatch, } @@ -821,8 +978,8 @@ fn sort_batch( .map(|e| e.evaluate_to_sort_column(&batch)) .collect::>>()?; - let indices = if sort_columns.len() == 1 { - lexsort_to_indices(&sort_columns, fetch)? + let (indices, rows) = if sort_columns.len() == 1 { + (lexsort_to_indices(&sort_columns, fetch)?, None) } else { let sort_fields = sort_columns .iter() @@ -842,7 +999,15 @@ fn sort_batch( Some(lim) => lim.min(to_sort.len()), None => to_sort.len(), }; - UInt32Array::from_iter(to_sort.into_iter().take(limit).map(|(idx, _)| idx as u32)) + let sorted_indices = to_sort + .iter() + .take(limit) + .map(|(idx, _)| *idx) + .collect::>(); + ( + UInt32Array::from_iter(sorted_indices.iter().map(|i| *i as u32)), + Some(RowSelection::new(rows, sorted_indices)), + ) }; // reorder all rows based on sorted indices @@ -879,11 +1044,15 @@ fn sort_batch( .collect::>>()?; Ok(BatchWithSortArray { - sort_arrays, + sort_data: SortData { + arrays: sort_arrays, + rows, + }, sorted_batch, }) } - +// todo: add option to always emit None for row encoding to save memory in cases +// where the parent node does not care about the row encoding. async fn do_sort( mut input: SendableRecordBatchStream, partition_id: usize, @@ -891,7 +1060,7 @@ async fn do_sort( metrics_set: CompositeMetricsSet, context: Arc, fetch: Option, -) -> Result { +) -> Result { debug!( "Start do_sort for partition {} of context session_id {} and task_id {:?}", partition_id, @@ -915,6 +1084,7 @@ async fn do_sort( sorter.insert_batch(batch, &tracking_metrics).await?; } let result = sorter.sort(); + debug!( "End do_sort for partition {} of context session_id {} and task_id {:?}", partition_id, diff --git a/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs b/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs index a324bcc486fd..8e68dd3e78df 100644 --- a/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs +++ b/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs @@ -30,24 +30,30 @@ use arrow::{ record_batch::RecordBatch, }; use futures::stream::{Fuse, FusedStream}; -use futures::{ready, Stream, StreamExt}; +use futures::{ready, Stream, StreamExt, TryStreamExt}; use log::debug; use tokio::sync::mpsc; +use tokio::task::JoinHandle; +use tokio_stream::wrappers::ReceiverStream; use crate::error::{DataFusionError, Result}; use crate::execution::context::TaskContext; +use crate::physical_plan::common::AbortOnDropSingle; use crate::physical_plan::metrics::{ ExecutionPlanMetricsSet, MemTrackingMetrics, MetricsSet, }; +use crate::physical_plan::sorts::sort::SortExec; use crate::physical_plan::sorts::{RowIndex, SortKeyCursor, SortedStream}; -use crate::physical_plan::stream::RecordBatchReceiverStream; +use crate::physical_plan::stream::{RecordBatchReceiverStream, RecordBatchStreamAdapter}; use crate::physical_plan::{ common::spawn_execution, expressions::PhysicalSortExpr, DisplayFormatType, - Distribution, ExecutionPlan, Partitioning, PhysicalExpr, RecordBatchStream, - SendableRecordBatchStream, Statistics, + Distribution, ExecutionPlan, Partitioning, PhysicalExpr, SendableRecordBatchStream, + Statistics, }; use datafusion_physical_expr::EquivalenceProperties; +use super::{SendableSortStream, SortStreamItem}; + /// Sort preserving merge execution plan /// /// This takes an input execution plan and a list of sort expressions, and @@ -191,48 +197,74 @@ impl ExecutionPlan for SortPreservingMergeExec { // Use tokio only if running from a tokio context (#2201) let receivers = match tokio::runtime::Handle::try_current() { Ok(_) => (0..input_partitions) - .into_iter() .map(|part_i| { - let (sender, receiver) = mpsc::channel(1); - let join_handle = spawn_execution( - self.input.clone(), - sender, - part_i, - context.clone(), - ); - - SortedStream::new( - RecordBatchReceiverStream::create( - &schema, - receiver, - join_handle, - ), - 0, - ) + if let Some(sort_plan) = + self.input.as_any().downcast_ref::() + { + let (tx, rx) = mpsc::channel(1); + let join_handle = sort_plan.execution_spawn_task( + part_i, + context.clone(), + tx, + ); + let stream = + Box::pin(SortReceiverStream::new(rx, join_handle)); + SortedStream { + stream, + mem_used: 0, + } + } else { + let (sender, receiver) = mpsc::channel(1); + let join_handle = spawn_execution( + self.input.clone(), + sender, + part_i, + context.clone(), + ); + SortedStream::new_no_row_encoding( + RecordBatchReceiverStream::create( + &schema, + receiver, + join_handle, + ), + 0, + ) + } }) .collect(), Err(_) => (0..input_partitions) .map(|partition| { - let stream = - self.input.execute(partition, context.clone())?; - Ok(SortedStream::new(stream, 0)) + if let Some(sort_plan) = + self.input.as_any().downcast_ref::() + { + let stream = sort_plan.execute_save_row_encoding( + partition, + context.clone(), + )?; + Ok(SortedStream { + stream, + mem_used: 0, + }) + } else { + let stream = + self.input.execute(partition, context.clone())?; + Ok(SortedStream::new_no_row_encoding(stream, 0)) + } }) .collect::>()?, }; - debug!("Done setting up sender-receiver for SortPreservingMergeExec::execute"); - let result = Box::pin(SortPreservingMergeStream::new_from_streams( + let result = SortPreservingMergeStream::new_from_streams( receivers, schema, &self.expr, tracking_metrics, context.session_config().batch_size(), - )?); + )?; debug!("Got stream result from SortPreservingMergeStream::new_from_receivers"); - - Ok(result) + Ok(result.into()) } } } @@ -261,7 +293,7 @@ impl ExecutionPlan for SortPreservingMergeExec { struct MergingStreams { /// The sorted input streams to merge together - streams: Vec>, + streams: Vec>, /// number of streams num_streams: usize, } @@ -275,7 +307,7 @@ impl std::fmt::Debug for MergingStreams { } impl MergingStreams { - fn new(input_streams: Vec>) -> Self { + fn new(input_streams: Vec>) -> Self { Self { num_streams: input_streams.len(), streams: input_streams, @@ -360,7 +392,6 @@ impl SortPreservingMergeStream { .map(|_| VecDeque::new()) .collect(); tracking_metrics.init_mem_used(streams.iter().map(|s| s.mem_used).sum()); - let wrappers = streams.into_iter().map(|s| s.stream.fuse()).collect(); let sort_fields = expressions .iter() @@ -374,7 +405,9 @@ impl SortPreservingMergeStream { Ok(Self { schema, batches, - streams: MergingStreams::new(wrappers), + streams: MergingStreams::new( + streams.into_iter().map(|s| s.stream.fuse()).collect(), + ), column_expressions: expressions.iter().map(|x| x.expr.clone()).collect(), tracking_metrics, aborted: false, @@ -417,7 +450,7 @@ impl SortPreservingMergeStream { Some(Err(e)) => { return Poll::Ready(Err(e)); } - Some(Ok(batch)) => { + Some(Ok((batch, preserved_rows))) => { if batch.num_rows() > 0 { let cols = self .column_expressions @@ -426,18 +459,29 @@ impl SortPreservingMergeStream { Ok(expr.evaluate(&batch)?.into_array(batch.num_rows())) }) .collect::>>()?; - - let rows = match self.row_converter.convert_columns(&cols) { - Ok(rows) => rows, - Err(e) => { - return Poll::Ready(Err(DataFusionError::ArrowError(e))); + // use preserved row encoding if it existed, otherwise create now + // + // (could be because: either not used at all (single col) + // or currently rows are not spilled to disk.) + let rows = match preserved_rows { + Some(rows) => { + println!("got preserved row encoding..."); + rows } + None => match self.row_converter.convert_columns(&cols) { + Ok(rows) => rows.into(), + Err(e) => { + return Poll::Ready(Err( + DataFusionError::ArrowError(e), + )); + } + }, }; self.cursors[idx] = Some(SortKeyCursor::new( idx, self.next_batch_id, // assign this batch an ID - rows, + rows.into(), )); self.next_batch_id += 1; self.batches[idx].push_back(batch) @@ -458,7 +502,7 @@ impl SortPreservingMergeStream { /// Drains the in_progress row indexes, and builds a new RecordBatch from them /// /// Will then drop any batches for which all rows have been yielded to the output - fn build_record_batch(&mut self) -> Result { + fn build_record_batch(&mut self) -> SortStreamItem { // Mapping from stream index to the index of the first buffer from that stream let mut buffer_idx = 0; let mut stream_to_buffer_idx = Vec::with_capacity(self.batches.len()); @@ -540,19 +584,33 @@ impl SortPreservingMergeStream { } } - RecordBatch::try_new(self.schema.clone(), columns).map_err(Into::into) + RecordBatch::try_new(self.schema.clone(), columns) + .map(|batch| (batch, None)) + .map_err(Into::into) } } impl Stream for SortPreservingMergeStream { - type Item = Result; + type Item = SortStreamItem; fn poll_next( mut self: Pin<&mut Self>, cx: &mut Context<'_>, ) -> Poll> { let poll = self.poll_next_inner(cx); - self.tracking_metrics.record_poll(poll) + // cant use `tracking_metrics.record_poll` since Self::Item is wrong type + // this should do the same thing as `tracking_metrics.record_poll` + if let Poll::Ready(maybe_sort_item) = &poll { + match maybe_sort_item { + Some(Ok((batch, _rows))) => { + self.tracking_metrics.record_output(batch.num_rows()) + } + Some(Err(_)) | None => { + self.tracking_metrics.done(); + } + } + } + poll } } @@ -561,7 +619,7 @@ impl SortPreservingMergeStream { fn poll_next_inner( self: &mut Pin<&mut Self>, cx: &mut Context<'_>, - ) -> Poll>> { + ) -> Poll> { if self.aborted { return Poll::Ready(None); } @@ -703,10 +761,36 @@ impl SortPreservingMergeStream { Poll::Ready(Ok(())) } } +impl Into for SortPreservingMergeStream { + fn into(self) -> SendableRecordBatchStream { + Box::pin(RecordBatchStreamAdapter::new( + self.schema.clone(), + self.map_ok(|(rb, _rows)| rb), + )) + } +} +struct SortReceiverStream { + inner: ReceiverStream, + #[allow(dead_code)] + drop_helper: AbortOnDropSingle<()>, +} +impl SortReceiverStream { + fn new(rx: mpsc::Receiver, handle: JoinHandle<()>) -> Self { + let stream = ReceiverStream::new(rx); + Self { + inner: stream, + drop_helper: AbortOnDropSingle::new(handle), + } + } +} +impl Stream for SortReceiverStream { + type Item = SortStreamItem; -impl RecordBatchStream for SortPreservingMergeStream { - fn schema(&self) -> SchemaRef { - self.schema.clone() + fn poll_next( + mut self: std::pin::Pin<&mut Self>, + cx: &mut std::task::Context<'_>, + ) -> std::task::Poll> { + self.inner.poll_next_unpin(cx) } } @@ -718,7 +802,6 @@ mod tests { use arrow::compute::SortOptions; use arrow::datatypes::{DataType, Field, Schema}; use futures::FutureExt; - use tokio_stream::StreamExt; use crate::arrow::array::{Int32Array, StringArray, TimestampNanosecondArray}; use crate::from_slice::FromSlice; @@ -1265,70 +1348,71 @@ mod tests { ); } - #[tokio::test] - async fn test_async() { - let session_ctx = SessionContext::new(); - let task_ctx = session_ctx.task_ctx(); - let schema = test_util::aggr_test_schema(); - let sort = vec![PhysicalSortExpr { - expr: col("c12", &schema).unwrap(), - options: SortOptions::default(), - }]; - - let batches = - sorted_partitioned_input(sort.clone(), &[5, 7, 3], task_ctx.clone()).await; - - let partition_count = batches.output_partitioning().partition_count(); - let mut streams = Vec::with_capacity(partition_count); - - for partition in 0..partition_count { - let (sender, receiver) = mpsc::channel(1); - let mut stream = batches.execute(partition, task_ctx.clone()).unwrap(); - let join_handle = tokio::spawn(async move { - while let Some(batch) = stream.next().await { - sender.send(batch).await.unwrap(); - // This causes the MergeStream to wait for more input - tokio::time::sleep(tokio::time::Duration::from_millis(10)).await; - } - }); - - streams.push(SortedStream::new( - RecordBatchReceiverStream::create(&schema, receiver, join_handle), - 0, - )); - } - - let metrics = ExecutionPlanMetricsSet::new(); - let tracking_metrics = - MemTrackingMetrics::new(&metrics, task_ctx.memory_pool(), 0); - - let merge_stream = SortPreservingMergeStream::new_from_streams( - streams, - batches.schema(), - sort.as_slice(), - tracking_metrics, - task_ctx.session_config().batch_size(), - ) - .unwrap(); - - let mut merged = common::collect(Box::pin(merge_stream)).await.unwrap(); - - assert_eq!(merged.len(), 1); - let merged = merged.remove(0); - let basic = basic_sort(batches, sort.clone(), task_ctx.clone()).await; - - let basic = arrow::util::pretty::pretty_format_batches(&[basic]) - .unwrap() - .to_string(); - let partition = arrow::util::pretty::pretty_format_batches(&[merged]) - .unwrap() - .to_string(); - - assert_eq!( - basic, partition, - "basic:\n\n{basic}\n\npartition:\n\n{partition}\n\n" - ); - } + // #[tokio::test] + // async fn test_async() { + // let session_ctx = SessionContext::new(); + // let task_ctx = session_ctx.task_ctx(); + // let schema = test_util::aggr_test_schema(); + // let sort = vec![PhysicalSortExpr { + // expr: col("c12", &schema).unwrap(), + // options: SortOptions::default(), + // }]; + + // let batches = + // sorted_partitioned_input(sort.clone(), &[5, 7, 3], task_ctx.clone()).await; + + // let partition_count = batches.output_partitioning().partition_count(); + // let mut streams = Vec::with_capacity(partition_count); + + // for partition in 0..partition_count { + // let (sender, receiver) = mpsc::channel(1); + // let mut stream = batches.execute(partition, task_ctx.clone()).unwrap(); + // let join_handle = tokio::spawn(async move { + // while let Some(batch) = stream.next().await { + // sender.send(batch).await.unwrap(); + // // This causes the MergeStream to wait for more input + // tokio::time::sleep(tokio::time::Duration::from_millis(10)).await; + // } + // }); + + // streams.push(SortedStream::new( + // RecordBatchReceiverStream::create(&schema, receiver, join_handle), + // 0, + // futures::stream::empty().boxed(), + // )); + // } + + // let metrics = ExecutionPlanMetricsSet::new(); + // let tracking_metrics = + // MemTrackingMetrics::new(&metrics, task_ctx.memory_pool(), 0); + + // let merge_stream = SortPreservingMergeStream::new_from_streams( + // streams, + // batches.schema(), + // sort.as_slice(), + // tracking_metrics, + // task_ctx.session_config().batch_size(), + // ) + // .unwrap(); + + // let mut merged = common::collect(Box::pin(merge_stream)).await.unwrap(); + + // assert_eq!(merged.len(), 1); + // let merged = merged.remove(0); + // let basic = basic_sort(batches, sort.clone(), task_ctx.clone()).await; + + // let basic = arrow::util::pretty::pretty_format_batches(&[basic]) + // .unwrap() + // .to_string(); + // let partition = arrow::util::pretty::pretty_format_batches(&[merged]) + // .unwrap() + // .to_string(); + + // assert_eq!( + // basic, partition, + // "basic:\n\n{basic}\n\npartition:\n\n{partition}\n\n" + // ); + // } #[tokio::test] async fn test_merge_metrics() { diff --git a/datafusion/core/tests/sort_key_cursor.rs b/datafusion/core/tests/sort_key_cursor.rs index 7d03ffc87bf5..0cea5caa978d 100644 --- a/datafusion/core/tests/sort_key_cursor.rs +++ b/datafusion/core/tests/sort_key_cursor.rs @@ -186,7 +186,7 @@ impl CursorBuilder { SortKeyCursor::new( stream_idx.expect("stream idx not set"), batch_id.expect("batch id not set"), - rows, + rows.into(), ) } } From 7aaddb5163eb9c404fe7aaac8eda6db60bfaae30 Mon Sep 17 00:00:00 2001 From: Jay Miller <3744812+jaylmiller@users.noreply.github.com> Date: Tue, 14 Feb 2023 14:17:58 -0500 Subject: [PATCH 07/29] remove some todos --- .../core/src/physical_plan/sorts/mod.rs | 45 ++++++++++++------- .../core/src/physical_plan/sorts/sort.rs | 3 +- 2 files changed, 31 insertions(+), 17 deletions(-) diff --git a/datafusion/core/src/physical_plan/sorts/mod.rs b/datafusion/core/src/physical_plan/sorts/mod.rs index e8816e83b2d7..4a42d97f2fab 100644 --- a/datafusion/core/src/physical_plan/sorts/mod.rs +++ b/datafusion/core/src/physical_plan/sorts/mod.rs @@ -85,8 +85,6 @@ pub struct RowBatch { // refs to the rows referenced by `indices` rows: Vec>, // first item = index of the ref in `rows`, second item=index within that row - // TODO: make this field optional for case where the RowBatch is just one `Rows` - // and it is in the exact same order to save memory. indices: Vec<(usize, usize)>, } @@ -126,7 +124,7 @@ impl RowBatch { impl From for RowBatch { fn from(value: RowSelection) -> Self { Self { - indices: (0..value.indices.len()).map(|i| (0, i)).collect(), + indices: (0..value.num_rows()).map(|i| (0, i)).collect(), rows: vec![Arc::new(value)], } } @@ -160,19 +158,25 @@ impl<'a> Iterator for RowBatchIter<'a> { #[derive(Debug)] pub struct RowSelection { rows: Rows, - // todo: make None in case where RowSelection is equivalent to Rows - // to save memory - indices: Vec, + // None when this `RowSelection` is equivalent to its `Rows` + indices: Option>, } impl RowSelection { /// New pub fn new(rows: Rows, indices: Vec) -> Self { - Self { rows, indices } + Self { + rows, + indices: Some(indices), + } } /// Get the nth row of the selection. pub fn row(&self, n: usize) -> Row { - let idx = self.indices[n]; - self.rows.row(idx) + if let Some(ref indices) = self.indices { + let idx = indices[n]; + self.rows.row(idx) + } else { + self.rows.row(n) + } } /// Iterate over the rows in the selected order. pub fn iter(&self) -> RowSelectionIter { @@ -183,15 +187,26 @@ impl RowSelection { } /// Number of bytes held in rows and indices. pub fn size(&self) -> usize { - self.rows.size() - + self.indices.len() * std::mem::size_of::() - + std::mem::size_of::() + let indices_size = self + .indices + .as_ref() + .map(|i| i.len() * std::mem::size_of::()) + .unwrap_or(0); + self.rows.size() + indices_size + std::mem::size_of::() + } + + fn num_rows(&self) -> usize { + if let Some(ref indices) = self.indices { + indices.len() + } else { + self.rows.num_rows() + } } } impl From for RowSelection { fn from(value: Rows) -> Self { Self { - indices: (0..value.num_rows()).collect(), + indices: None, rows: value, } } @@ -205,7 +220,7 @@ impl<'a> Iterator for RowSelectionIter<'a> { type Item = Row<'a>; fn next(&mut self) -> Option { - if self.cur_n < self.row_selection.indices.len() { + if self.cur_n < self.row_selection.num_rows() { let row = self.row_selection.row(self.cur_n); self.cur_n += 1; Some(row) @@ -243,7 +258,7 @@ mod tests { let mut conv = RowConverter::new(vec![SortField::new(DataType::Int64)]).unwrap(); let s1 = RowSelection::new(int64_rows(&mut conv, 0..3), vec![2, 2, 1]); let s2 = RowSelection::new(int64_rows(&mut conv, 5..8), vec![1, 2, 0]); - let s3: RowSelection = int64_rows(&mut conv, 2..4).into(); + let s3: RowSelection = int64_rows(&mut conv, 2..4).into(); // null indices case let selection = RowBatch::new( vec![s1, s2, s3].into_iter().map(Arc::new).collect(), vec![ diff --git a/datafusion/core/src/physical_plan/sorts/sort.rs b/datafusion/core/src/physical_plan/sorts/sort.rs index e5770380ca07..b465238a8daa 100644 --- a/datafusion/core/src/physical_plan/sorts/sort.rs +++ b/datafusion/core/src/physical_plan/sorts/sort.rs @@ -232,8 +232,7 @@ impl ExternalSorter { }); Ok(Box::pin(output_stream)) } else { - todo!() - // Ok(Box::pin(EmptyRecordBatchStream::new(self.schema.clone()))) + Ok(Box::pin(futures::stream::empty())) } } From 96a2e15d7f6d774fff7447464d391dc416a30865 Mon Sep 17 00:00:00 2001 From: Jay Miller <3744812+jaylmiller@users.noreply.github.com> Date: Tue, 14 Feb 2023 15:02:19 -0500 Subject: [PATCH 08/29] fix clippy warnings --- .../core/src/physical_plan/sorts/mod.rs | 4 +- .../core/src/physical_plan/sorts/sort.rs | 1 - .../sorts/sort_preserving_merge.rs | 139 +++++++++--------- 3 files changed, 70 insertions(+), 74 deletions(-) diff --git a/datafusion/core/src/physical_plan/sorts/mod.rs b/datafusion/core/src/physical_plan/sorts/mod.rs index 4a42d97f2fab..f8262c5e9415 100644 --- a/datafusion/core/src/physical_plan/sorts/mod.rs +++ b/datafusion/core/src/physical_plan/sorts/mod.rs @@ -110,7 +110,7 @@ impl RowBatch { /// Iterate over rows in their selected order pub fn iter(&self) -> RowBatchIter { RowBatchIter { - row_selection: &self, + row_selection: self, cur_idx: 0, } } @@ -181,7 +181,7 @@ impl RowSelection { /// Iterate over the rows in the selected order. pub fn iter(&self) -> RowSelectionIter { RowSelectionIter { - row_selection: &self, + row_selection: self, cur_n: 0, } } diff --git a/datafusion/core/src/physical_plan/sorts/sort.rs b/datafusion/core/src/physical_plan/sorts/sort.rs index b465238a8daa..e26dd2533af5 100644 --- a/datafusion/core/src/physical_plan/sorts/sort.rs +++ b/datafusion/core/src/physical_plan/sorts/sort.rs @@ -317,7 +317,6 @@ fn in_mem_partial_sort( } else { let (sort_data, batches): (Vec, Vec) = buffered_batches .drain(..) - .into_iter() .map(|b| { let BatchWithSortArray { sort_data, diff --git a/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs b/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs index cb3b50aa8adc..31f24229a246 100644 --- a/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs +++ b/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs @@ -478,7 +478,7 @@ impl SortPreservingMergeStream { self.cursors[idx] = Some(SortKeyCursor::new( idx, self.next_batch_id, // assign this batch an ID - rows.into(), + rows, )); self.next_batch_id += 1; self.batches[idx].push_back(batch) @@ -758,14 +758,15 @@ impl SortPreservingMergeStream { Poll::Ready(Ok(())) } } -impl Into for SortPreservingMergeStream { - fn into(self) -> SendableRecordBatchStream { +impl From for SendableRecordBatchStream { + fn from(value: SortPreservingMergeStream) -> Self { Box::pin(RecordBatchStreamAdapter::new( - self.schema.clone(), - self.map_ok(|(rb, _rows)| rb), + value.schema.clone(), + value.into_stream().map_ok(|(rb, _rows)| rb), )) } } + struct SortReceiverStream { inner: ReceiverStream, #[allow(dead_code)] @@ -790,7 +791,6 @@ impl Stream for SortReceiverStream { self.inner.poll_next_unpin(cx) } } - #[cfg(test)] mod tests { use std::iter::FromIterator; @@ -1344,71 +1344,68 @@ mod tests { ); } - // #[tokio::test] - // async fn test_async() { - // let session_ctx = SessionContext::new(); - // let task_ctx = session_ctx.task_ctx(); - // let schema = test_util::aggr_test_schema(); - // let sort = vec![PhysicalSortExpr { - // expr: col("c12", &schema).unwrap(), - // options: SortOptions::default(), - // }]; - - // let batches = - // sorted_partitioned_input(sort.clone(), &[5, 7, 3], task_ctx.clone()).await; - - // let partition_count = batches.output_partitioning().partition_count(); - // let mut streams = Vec::with_capacity(partition_count); - - // for partition in 0..partition_count { - // let (sender, receiver) = mpsc::channel(1); - // let mut stream = batches.execute(partition, task_ctx.clone()).unwrap(); - // let join_handle = tokio::spawn(async move { - // while let Some(batch) = stream.next().await { - // sender.send(batch).await.unwrap(); - // // This causes the MergeStream to wait for more input - // tokio::time::sleep(tokio::time::Duration::from_millis(10)).await; - // } - // }); - - // streams.push(SortedStream::new( - // RecordBatchReceiverStream::create(&schema, receiver, join_handle), - // 0, - // futures::stream::empty().boxed(), - // )); - // } - - // let metrics = ExecutionPlanMetricsSet::new(); - // let tracking_metrics = - // MemTrackingMetrics::new(&metrics, task_ctx.memory_pool(), 0); - - // let merge_stream = SortPreservingMergeStream::new_from_streams( - // streams, - // batches.schema(), - // sort.as_slice(), - // tracking_metrics, - // task_ctx.session_config().batch_size(), - // ) - // .unwrap(); - - // let mut merged = common::collect(Box::pin(merge_stream)).await.unwrap(); - - // assert_eq!(merged.len(), 1); - // let merged = merged.remove(0); - // let basic = basic_sort(batches, sort.clone(), task_ctx.clone()).await; - - // let basic = arrow::util::pretty::pretty_format_batches(&[basic]) - // .unwrap() - // .to_string(); - // let partition = arrow::util::pretty::pretty_format_batches(&[merged]) - // .unwrap() - // .to_string(); - - // assert_eq!( - // basic, partition, - // "basic:\n\n{basic}\n\npartition:\n\n{partition}\n\n" - // ); - // } + #[tokio::test] + async fn test_async() { + let session_ctx = SessionContext::new(); + let task_ctx = session_ctx.task_ctx(); + let schema = test_util::aggr_test_schema(); + let sort = vec![PhysicalSortExpr { + expr: col("c12", &schema).unwrap(), + options: SortOptions::default(), + }]; + + let batches = + sorted_partitioned_input(sort.clone(), &[5, 7, 3], task_ctx.clone()).await; + + let partition_count = batches.output_partitioning().partition_count(); + let mut streams = Vec::with_capacity(partition_count); + + for partition in 0..partition_count { + let (sender, receiver) = mpsc::channel(1); + let mut stream = batches.execute(partition, task_ctx.clone()).unwrap(); + let join_handle = tokio::spawn(async move { + while let Some(batch) = stream.next().await { + sender.send(batch).await.unwrap(); + // This causes the MergeStream to wait for more input + tokio::time::sleep(tokio::time::Duration::from_millis(10)).await; + } + }); + + streams.push(SortedStream::new_no_row_encoding( + RecordBatchReceiverStream::create(&schema, receiver, join_handle), + 0, + )); + } + + let metrics = ExecutionPlanMetricsSet::new(); + let tracking_metrics = + MemTrackingMetrics::new(&metrics, task_ctx.memory_pool(), 0); + let merge_stream = SortPreservingMergeStream::new_from_streams( + streams, + batches.schema(), + sort.as_slice(), + tracking_metrics, + task_ctx.session_config().batch_size(), + ) + .unwrap(); + let mut merged = common::collect(merge_stream.into()).await.unwrap(); + + assert_eq!(merged.len(), 1); + let merged = merged.remove(0); + let basic = basic_sort(batches, sort.clone(), task_ctx.clone()).await; + + let basic = arrow::util::pretty::pretty_format_batches(&[basic]) + .unwrap() + .to_string(); + let partition = arrow::util::pretty::pretty_format_batches(&[merged]) + .unwrap() + .to_string(); + + assert_eq!( + basic, partition, + "basic:\n\n{basic}\n\npartition:\n\n{partition}\n\n" + ); + } #[tokio::test] async fn test_merge_metrics() { From a3c632c8ffeb118032a3df5269b4313281a19c4e Mon Sep 17 00:00:00 2001 From: Jay Miller <3744812+jaylmiller@users.noreply.github.com> Date: Tue, 14 Feb 2023 16:37:16 -0500 Subject: [PATCH 09/29] checkpointing --- datafusion/core/src/physical_plan/sorts/mod.rs | 3 ++- datafusion/core/src/physical_plan/sorts/sort.rs | 15 ++++++++++----- .../physical_plan/sorts/sort_preserving_merge.rs | 7 +++++++ 3 files changed, 19 insertions(+), 6 deletions(-) diff --git a/datafusion/core/src/physical_plan/sorts/mod.rs b/datafusion/core/src/physical_plan/sorts/mod.rs index f8262c5e9415..56c558677e2f 100644 --- a/datafusion/core/src/physical_plan/sorts/mod.rs +++ b/datafusion/core/src/physical_plan/sorts/mod.rs @@ -117,7 +117,8 @@ impl RowBatch { /// Amount of bytes pub fn memory_size(&self) -> usize { let indices_size = self.indices.len() * 2 * std::mem::size_of::(); - let rows_size = self.rows.iter().map(|r| r.size()).sum::(); + // rows are refs so dont count the size inside the refs, just the refs itself? + let rows_size = 0; rows_size + indices_size + std::mem::size_of::() } } diff --git a/datafusion/core/src/physical_plan/sorts/sort.rs b/datafusion/core/src/physical_plan/sorts/sort.rs index e26dd2533af5..b9234be53f6c 100644 --- a/datafusion/core/src/physical_plan/sorts/sort.rs +++ b/datafusion/core/src/physical_plan/sorts/sort.rs @@ -140,7 +140,14 @@ impl ExternalSorter { // The resulting batch might be smaller (or larger, see #3747) than the input // batch due to either a propagated limit or the re-construction of arrays. So // for being reliable, we need to reflect the memory usage of the partial batch. - let new_size = batch_byte_size(&partial.sorted_batch); + // + // In addition, if it's row encoding was preserved, that would also change the size. + let new_size = batch_byte_size(&partial.sorted_batch) + + partial + .sort_data + .rows + .as_ref() + .map_or(0, |rows| rows.size()); match new_size.cmp(&size) { Ordering::Greater => { // We don't have to call try_grow here, since we have already used the @@ -209,6 +216,7 @@ impl ExternalSorter { &self.expr, tracking_metrics, self.session_config.batch_size(), + true, )?)) } else if !self.in_mem_batches.is_empty() { // sort in mem doesnt require SortPreservingMergeStream @@ -545,8 +553,7 @@ impl SortedSizedRecordBatchStream { // include rows if non-None + rows .as_ref() - .map(|r| r.iter().map(|r| r.size()).sum()) - .unwrap_or(0); + .map_or(0, |r| r.iter().map(|r| r.size()).sum()); metrics.init_mem_used(size); let num_cols = batches[0].num_columns(); SortedSizedRecordBatchStream { @@ -1054,8 +1061,6 @@ fn sort_batch( sorted_batch, }) } -// todo: add option to always emit None for row encoding to save memory in cases -// where the parent node does not care about the row encoding. async fn do_sort( mut input: SendableRecordBatchStream, partition_id: usize, diff --git a/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs b/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs index 31f24229a246..81317168989e 100644 --- a/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs +++ b/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs @@ -261,6 +261,7 @@ impl ExecutionPlan for SortPreservingMergeExec { &self.expr, tracking_metrics, context.session_config().batch_size(), + false, )?; debug!("Got stream result from SortPreservingMergeStream::new_from_receivers"); @@ -376,6 +377,8 @@ pub(crate) struct SortPreservingMergeStream { /// row converter row_converter: RowConverter, + /// if this is false it will always yield None for the row encoding + preserve_row_encoding: bool, } impl SortPreservingMergeStream { @@ -385,6 +388,8 @@ impl SortPreservingMergeStream { expressions: &[PhysicalSortExpr], mut tracking_metrics: MemTrackingMetrics, batch_size: usize, + // when used from within SortExec this should be true + preserve_row_encoding: bool, ) -> Result { let stream_count = streams.len(); let batches = (0..stream_count).map(|_| VecDeque::new()).collect(); @@ -415,6 +420,7 @@ impl SortPreservingMergeStream { loser_tree_adjusted: false, batch_size, row_converter, + preserve_row_encoding, }) } @@ -1386,6 +1392,7 @@ mod tests { sort.as_slice(), tracking_metrics, task_ctx.session_config().batch_size(), + false, ) .unwrap(); let mut merged = common::collect(merge_stream.into()).await.unwrap(); From c51b23c4c662aeecaecc4390f8fa5f46d72f91ba Mon Sep 17 00:00:00 2001 From: Jay Miller <3744812+jaylmiller@users.noreply.github.com> Date: Tue, 14 Feb 2023 17:47:15 -0500 Subject: [PATCH 10/29] SortPreservingMergeStream emits row encodings when used from SortExec --- .../core/src/physical_plan/sorts/mod.rs | 11 ++- .../core/src/physical_plan/sorts/sort.rs | 13 ++-- .../sorts/sort_preserving_merge.rs | 78 +++++++++++++++++-- 3 files changed, 85 insertions(+), 17 deletions(-) diff --git a/datafusion/core/src/physical_plan/sorts/mod.rs b/datafusion/core/src/physical_plan/sorts/mod.rs index 56c558677e2f..0d45c0b15371 100644 --- a/datafusion/core/src/physical_plan/sorts/mod.rs +++ b/datafusion/core/src/physical_plan/sorts/mod.rs @@ -79,13 +79,13 @@ impl Debug for SortedStream { } } -/// A batch of rows taken from multiple [RowSelection]s +/// Cloneable batch of rows taken from multiple [RowSelection]s #[derive(Debug, Clone)] pub struct RowBatch { // refs to the rows referenced by `indices` rows: Vec>, // first item = index of the ref in `rows`, second item=index within that row - indices: Vec<(usize, usize)>, + indices: Arc>, } impl RowBatch { @@ -94,7 +94,10 @@ impl RowBatch { /// `indices` defines where each row comes from: first element of the tuple is the index /// of the ref in `rows`, second is the index within that `RowSelection`. pub fn new(rows: Vec>, indices: Vec<(usize, usize)>) -> Self { - Self { rows, indices } + Self { + rows, + indices: Arc::new(indices), + } } /// Returns the nth row in the batch. @@ -125,7 +128,7 @@ impl RowBatch { impl From for RowBatch { fn from(value: RowSelection) -> Self { Self { - indices: (0..value.num_rows()).map(|i| (0, i)).collect(), + indices: Arc::new((0..value.num_rows()).map(|i| (0, i)).collect()), rows: vec![Arc::new(value)], } } diff --git a/datafusion/core/src/physical_plan/sorts/sort.rs b/datafusion/core/src/physical_plan/sorts/sort.rs index b9234be53f6c..9f0fde6d8ebc 100644 --- a/datafusion/core/src/physical_plan/sorts/sort.rs +++ b/datafusion/core/src/physical_plan/sorts/sort.rs @@ -142,12 +142,12 @@ impl ExternalSorter { // for being reliable, we need to reflect the memory usage of the partial batch. // // In addition, if it's row encoding was preserved, that would also change the size. - let new_size = batch_byte_size(&partial.sorted_batch) - + partial - .sort_data - .rows - .as_ref() - .map_or(0, |rows| rows.size()); + let new_size = batch_byte_size(&partial.sorted_batch); + // + partial + // .sort_data + // .rows + // .as_ref() + // .map_or(0, |rows| rows.size()); match new_size.cmp(&size) { Ordering::Greater => { // We don't have to call try_grow here, since we have already used the @@ -210,6 +210,7 @@ impl ExternalSorter { let tracking_metrics = self .metrics_set .new_final_tracking(self.partition_id, &self.runtime.memory_pool); + println!("using SortPreservingMergeStream to emit row encodings"); Ok(Box::pin(SortPreservingMergeStream::new_from_streams( streams, self.schema.clone(), diff --git a/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs b/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs index 81317168989e..f6878da28ed1 100644 --- a/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs +++ b/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs @@ -52,7 +52,7 @@ use crate::physical_plan::{ }; use datafusion_physical_expr::EquivalenceProperties; -use super::{SendableSortStream, SortStreamItem}; +use super::{RowBatch, RowSelection, SendableSortStream, SortStreamItem}; /// Sort preserving merge execution plan /// @@ -261,6 +261,7 @@ impl ExecutionPlan for SortPreservingMergeExec { &self.expr, tracking_metrics, context.session_config().batch_size(), + // dont emit row encodings for this plan false, )?; @@ -332,8 +333,7 @@ pub(crate) struct SortPreservingMergeStream { /// /// Exhausted batches will be popped off the front once all /// their rows have been yielded to the output - batches: Vec>, - + batches: Vec)>>, /// The accumulated row indexes for the next record batch in_progress: Vec, @@ -378,6 +378,8 @@ pub(crate) struct SortPreservingMergeStream { /// row converter row_converter: RowConverter, /// if this is false it will always yield None for the row encoding + /// this is true when `SortPreservingMergeStream` is used within `SortExec` + /// but not when its used in `SortPreservingMergeStream` preserve_row_encoding: bool, } @@ -481,13 +483,17 @@ impl SortPreservingMergeStream { }, }; + if self.preserve_row_encoding { + self.batches[idx].push_back((batch, Some(rows.clone()))) + } else { + self.batches[idx].push_back((batch, None)) + } self.cursors[idx] = Some(SortKeyCursor::new( idx, self.next_batch_id, // assign this batch an ID rows, )); self.next_batch_id += 1; - self.batches[idx].push_back(batch) } else { empty_batch = true; } @@ -525,7 +531,9 @@ impl SortPreservingMergeStream { .batches .iter() .flat_map(|batch| { - batch.iter().map(|batch| batch.column(column_idx).data()) + batch + .iter() + .map(|(batch, _rows)| batch.column(column_idx).data()) }) .collect(); @@ -569,6 +577,63 @@ impl SortPreservingMergeStream { make_arrow_array(array_data.freeze()) }) .collect(); + dbg!(self.preserve_row_encoding); + let rows = if self.preserve_row_encoding { + if self.in_progress.is_empty() { + Some(RowBatch::new(vec![], vec![])) + } else { + let rows = self + .batches + .iter() + .flat_map(|batch| { + batch.iter().map(|(_, rows)| { + rows.as_ref().expect( + "if preserve_row_encoding was true \ + then row data should've been saved in batch", + ) + }) + }) + .collect::>(); + let mut new_indices: Vec<(usize, usize)> = + Vec::with_capacity(self.in_progress.len()); + let mut new_rows: Vec> = vec![]; + let first = &self.in_progress[0]; + let mut buffer_idx = + stream_to_buffer_idx[first.stream_idx] + first.batch_idx; + let mut start_row_idx = first.row_idx; + let mut end_row_idx = start_row_idx + 1; + for row_index in self.in_progress.iter().skip(1) { + let next_buffer_idx = + stream_to_buffer_idx[row_index.stream_idx] + row_index.batch_idx; + + if next_buffer_idx == buffer_idx && row_index.row_idx == end_row_idx { + // subsequent row in same batch + end_row_idx += 1; + continue; + } + let row_batch = rows[buffer_idx]; + let row_indices = &row_batch.indices[start_row_idx..end_row_idx]; + new_indices.extend( + row_indices.iter().map(|(x, y)| (*x + new_rows.len(), *y)), + ); + new_rows.extend(row_batch.rows.iter().map(Arc::clone)); + // start new batch of rows + buffer_idx = next_buffer_idx; + start_row_idx = row_index.row_idx; + end_row_idx = start_row_idx + 1; + } + // emit final batch of rows + let row_batch = rows[buffer_idx]; + let row_indices = &row_batch.indices[start_row_idx..end_row_idx]; + new_indices + .extend(row_indices.iter().map(|(x, y)| (*x + new_rows.len(), *y))); + new_rows.extend(row_batch.rows.iter().map(Arc::clone)); + assert_eq!(new_indices.len(), self.in_progress.len()); + Some(RowBatch::new(new_rows, new_indices)) + } + } else { + None as Option + }; self.in_progress.clear(); @@ -588,7 +653,7 @@ impl SortPreservingMergeStream { } RecordBatch::try_new(self.schema.clone(), columns) - .map(|batch| (batch, None)) + .map(|batch| (batch, rows)) .map_err(Into::into) } } @@ -647,7 +712,6 @@ impl SortPreservingMergeStream { .as_mut() .filter(|cursor| !cursor.is_finished()) .map(|cursor| (cursor.stream_idx(), cursor.advance())); - if let Some((stream_idx, row_idx)) = next { self.loser_tree_adjusted = false; let batch_idx = self.batches[stream_idx].len() - 1; From 0f7bfc3830d4ba5fbc91f7551fde893d6c08a28f Mon Sep 17 00:00:00 2001 From: Jay Miller <3744812+jaylmiller@users.noreply.github.com> Date: Wed, 15 Feb 2023 09:10:14 -0500 Subject: [PATCH 11/29] spill logic working (w/ a temporary serialization format solution) --- .../core/src/physical_plan/sorts/mod.rs | 115 ++++++- .../core/src/physical_plan/sorts/sort.rs | 300 ++++++++++++++---- .../sorts/sort_preserving_merge.rs | 43 +-- 3 files changed, 347 insertions(+), 111 deletions(-) diff --git a/datafusion/core/src/physical_plan/sorts/mod.rs b/datafusion/core/src/physical_plan/sorts/mod.rs index 0d45c0b15371..66497c16d211 100644 --- a/datafusion/core/src/physical_plan/sorts/mod.rs +++ b/datafusion/core/src/physical_plan/sorts/mod.rs @@ -31,11 +31,15 @@ pub mod sort_preserving_merge; use arrow::{ record_batch::RecordBatch, - row::{Row, Rows}, + row::{Row, RowParser, Rows}, }; pub use cursor::SortKeyCursor; -use futures::{stream, Stream, StreamExt}; +use futures::{Stream, StreamExt, TryStreamExt}; pub use index::RowIndex; +use tokio::{sync::mpsc, task::JoinHandle}; +use tokio_stream::wrappers::ReceiverStream; + +use super::common::AbortOnDropSingle; pub(crate) type RowStream = Pin> + Send>>; pub(crate) type SortStreamItem = Result<(RecordBatch, Option)>; @@ -43,9 +47,19 @@ pub(crate) type SendableSortStream = Pin + pub(crate) struct SortedStream { stream: SendableSortStream, mem_used: usize, + // flag is only true if this was intialized wiith `new_no_row_encoding` + row_encoding_ignored: bool, } impl SortedStream { - pub(crate) fn new( + pub(crate) fn new(stream: SendableSortStream, mem_used: usize) -> Self { + Self { + stream, + mem_used, + row_encoding_ignored: false, + } + } + + pub(crate) fn new_from_streams( stream: SendableRecordBatchStream, mem_used: usize, row_stream: RowStream, @@ -58,19 +72,27 @@ impl SortedStream { Err(err) => Err(err), } })); - Self { stream, mem_used } + Self { + stream, + mem_used, + row_encoding_ignored: false, + } + // if let Some(row_stream) = row_stream { + // } else { + // Self::new_no_row_encoding(stream, mem_used) + // } } /// create stream where the row encoding for each batch is always None pub(crate) fn new_no_row_encoding( stream: SendableRecordBatchStream, mem_used: usize, ) -> Self { - Self::new( + let stream = Box::pin(stream.map_ok(|batch| (batch, None))); + Self { stream, mem_used, - // stream will end as soon as the record batch stream ends - Box::pin(stream::repeat(None)), - ) + row_encoding_ignored: true, + } } } impl Debug for SortedStream { @@ -84,7 +106,7 @@ impl Debug for SortedStream { pub struct RowBatch { // refs to the rows referenced by `indices` rows: Vec>, - // first item = index of the ref in `rows`, second item=index within that row + // first item = index of the ref in `rows`, second item=index within that `RowSelection` indices: Arc>, } @@ -158,21 +180,55 @@ impl<'a> Iterator for RowBatchIter<'a> { } } -/// A sorted selection of rows from the same [`Rows`]. +/// A selection of rows from the same [`RowData`]. #[derive(Debug)] pub struct RowSelection { - rows: Rows, + rows: RowData, // None when this `RowSelection` is equivalent to its `Rows` indices: Option>, } +#[derive(Debug)] +enum RowData { + Rows(Rows), + Spilled { + parser: RowParser, + bytes: Vec, + }, +} +impl RowData { + fn row(&self, n: usize) -> Row { + match self { + RowData::Rows(rows) => rows.row(n), + RowData::Spilled { parser, bytes } => parser.parse(&bytes[n]), + } + } + fn size(&self) -> usize { + match self { + RowData::Rows(rows) => rows.size(), + RowData::Spilled { bytes, .. } => bytes.len() + std::mem::size_of::(), + } + } + fn num_rows(&self) -> usize { + match self { + RowData::Rows(rows) => rows.num_rows(), + RowData::Spilled { bytes, .. } => bytes.len(), + } + } +} impl RowSelection { /// New pub fn new(rows: Rows, indices: Vec) -> Self { Self { - rows, + rows: RowData::Rows(rows), indices: Some(indices), } } + fn from_spilled(parser: RowParser, bytes: Vec) -> Self { + Self { + rows: RowData::Spilled { parser, bytes }, + indices: None, + } + } /// Get the nth row of the selection. pub fn row(&self, n: usize) -> Row { if let Some(ref indices) = self.indices { @@ -182,6 +238,7 @@ impl RowSelection { self.rows.row(n) } } + /// Iterate over the rows in the selected order. pub fn iter(&self) -> RowSelectionIter { RowSelectionIter { @@ -209,13 +266,21 @@ impl RowSelection { } impl From for RowSelection { fn from(value: Rows) -> Self { + Self { + indices: None, + rows: RowData::Rows(value), + } + } +} +impl From for RowSelection { + fn from(value: RowData) -> Self { Self { indices: None, rows: value, } } } -/// Iterator for [`SortedRows`] +/// Iterator for [`RowSelection`] pub struct RowSelectionIter<'a> { row_selection: &'a RowSelection, cur_n: usize, @@ -233,6 +298,30 @@ impl<'a> Iterator for RowSelectionIter<'a> { } } } +pub(crate) struct SortReceiverStream { + inner: ReceiverStream, + #[allow(dead_code)] + drop_helper: AbortOnDropSingle<()>, +} +impl SortReceiverStream { + fn new(rx: mpsc::Receiver, handle: JoinHandle<()>) -> Self { + let stream = ReceiverStream::new(rx); + Self { + inner: stream, + drop_helper: AbortOnDropSingle::new(handle), + } + } +} +impl Stream for SortReceiverStream { + type Item = SortStreamItem; + + fn poll_next( + mut self: std::pin::Pin<&mut Self>, + cx: &mut std::task::Context<'_>, + ) -> std::task::Poll> { + self.inner.poll_next_unpin(cx) + } +} #[cfg(test)] mod tests { diff --git a/datafusion/core/src/physical_plan/sorts/sort.rs b/datafusion/core/src/physical_plan/sorts/sort.rs index 9f0fde6d8ebc..a7d5296cbca4 100644 --- a/datafusion/core/src/physical_plan/sorts/sort.rs +++ b/datafusion/core/src/physical_plan/sorts/sort.rs @@ -19,7 +19,7 @@ //! It will do in-memory sorting if it has enough memory budget //! but spills to disk if needed. -use super::{RowBatch, RowSelection, RowStream}; +use super::{RowBatch, RowSelection, SortReceiverStream}; use super::{SendableSortStream, SortStreamItem}; use crate::error::{DataFusionError, Result}; use crate::execution::context::TaskContext; @@ -34,7 +34,7 @@ use crate::physical_plan::metrics::{ }; use crate::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeStream; use crate::physical_plan::sorts::SortedStream; -use crate::physical_plan::stream::{RecordBatchReceiverStream, RecordBatchStreamAdapter}; +use crate::physical_plan::stream::RecordBatchStreamAdapter; use crate::physical_plan::{ displayable, DisplayFormatType, Distribution, ExecutionPlan, Partitioning, RecordBatchStream, SendableRecordBatchStream, Statistics, @@ -77,7 +77,7 @@ use tokio_stream::wrappers::UnboundedReceiverStream; struct ExternalSorter { schema: SchemaRef, in_mem_batches: Vec, - spills: Vec, + spills: Vec, /// Sort expressions expr: Vec, session_config: Arc, @@ -88,6 +88,11 @@ struct ExternalSorter { reservation: MemoryReservation, partition_id: usize, } +struct Spill { + record_batch_file: NamedTempFile, + // `None` when row encoding not used + rows_file: Option, +} impl ExternalSorter { pub fn new( @@ -143,6 +148,7 @@ impl ExternalSorter { // // In addition, if it's row encoding was preserved, that would also change the size. let new_size = batch_byte_size(&partial.sorted_batch); + // TODO // + partial // .sort_data // .rows @@ -186,7 +192,7 @@ impl ExternalSorter { .new_intermediate_tracking(self.partition_id, &self.runtime.memory_pool); let mut streams: Vec = vec![]; if !self.in_mem_batches.is_empty() { - let (in_mem_stream, in_mem_rows) = in_mem_partial_sort( + let mut stream = in_mem_partial_sort( &mut self.in_mem_batches, self.schema.clone(), &self.expr, @@ -195,17 +201,22 @@ impl ExternalSorter { self.fetch, )?; let prev_used = self.reservation.free(); - streams.push(SortedStream::new(in_mem_stream, prev_used, in_mem_rows)); + stream.mem_used = prev_used; + streams.push(stream); } - + let sort_fields = self + .expr + .iter() + .map(|e| { + Ok(SortField::new_with_options( + e.expr.data_type(&self.schema)?, + e.options, + )) + }) + .collect::>>()?; for spill in self.spills.drain(..) { - let (tx, rx) = mpsc::unbounded_channel(); - let stream = read_spill_as_stream(spill, self.schema.clone(), tx)?; - streams.push(SortedStream::new( - stream, - 0, - UnboundedReceiverStream::new(rx).boxed(), - )); + let stream = read_spill_as_stream(spill, sort_fields.to_owned())?; + streams.push(SortedStream::new(stream, 0)); } let tracking_metrics = self .metrics_set @@ -224,7 +235,7 @@ impl ExternalSorter { let tracking_metrics = self .metrics_set .new_final_tracking(self.partition_id, &self.runtime.memory_pool); - let (stream, row_stream) = in_mem_partial_sort( + let stream = in_mem_partial_sort( &mut self.in_mem_batches, self.schema.clone(), &self.expr, @@ -234,12 +245,7 @@ impl ExternalSorter { )?; // Report to the memory manager we are no longer using memory self.reservation.free(); - let output_stream = stream.zip(row_stream).map(|i| { - let rec_batch_result: Result = i.0; - let maybe_rows = i.1; - rec_batch_result.map(|batch| (batch, maybe_rows)) - }); - Ok(Box::pin(output_stream)) + Ok(stream.stream) } else { Ok(Box::pin(futures::stream::empty())) } @@ -268,8 +274,7 @@ impl ExternalSorter { .metrics_set .new_intermediate_tracking(self.partition_id, &self.runtime.memory_pool); let spillfile = self.runtime.disk_manager.create_tmp_file("Sorting")?; - // TODO: spill row data - let (mut stream, _row_stream) = in_mem_partial_sort( + let mut stream = in_mem_partial_sort( &mut self.in_mem_batches, self.schema.clone(), &self.expr, @@ -277,12 +282,29 @@ impl ExternalSorter { tracking_metrics, self.fetch, )?; - spill_partial_sorted_stream(&mut stream, spillfile.path(), self.schema.clone()) - .await?; + let rows_file = if stream.row_encoding_ignored { + None + } else { + Some( + self.runtime + .disk_manager + .create_tmp_file("Sorting row encodings")?, + ) + }; + spill_partial_sorted_stream( + &mut stream.stream, + spillfile.path(), + rows_file.as_ref().map(|f| f.path()), + self.schema.clone(), + ) + .await?; self.reservation.free(); let used = self.metrics.mem_used().set(0); self.metrics.record_spill(used); - self.spills.push(spillfile); + self.spills.push(Spill { + record_batch_file: spillfile, + rows_file, + }); Ok(used) } } @@ -305,7 +327,7 @@ fn in_mem_partial_sort( batch_size: usize, tracking_metrics: MemTrackingMetrics, fetch: Option, -) -> Result<(SendableRecordBatchStream, RowStream)> { +) -> Result { let (row_tx, row_rx) = mpsc::unbounded_channel(); assert_ne!(buffered_batches.len(), 0); if buffered_batches.len() == 1 { @@ -315,14 +337,22 @@ fn in_mem_partial_sort( sorted_batch, } = result; let rowbatch: Option = sort_data.rows.map(Into::into); - Ok(( - Box::pin(SizedRecordBatchStream::new( - schema, - vec![Arc::new(sorted_batch)], - tracking_metrics, - )), - Box::pin(futures::stream::once(futures::future::ready(rowbatch))), - )) + let stream = Box::pin(SizedRecordBatchStream::new( + schema, + vec![Arc::new(sorted_batch)], + tracking_metrics, + )); + if let Some(rowbatch) = rowbatch { + Ok(SortedStream::new_from_streams( + stream, + 0, + Box::pin(futures::stream::once(futures::future::ready(Some( + rowbatch, + )))), + )) + } else { + Ok(SortedStream::new_no_row_encoding(stream, 0)) + } } else { let (sort_data, batches): (Vec, Vec) = buffered_batches .drain(..) @@ -345,6 +375,7 @@ fn in_mem_partial_sort( .into_iter() .map(|d| d.rows) .collect::>>(); + let used_rows = rows.is_some(); let batch_stream = Box::pin(SortedSizedRecordBatchStream::new( schema, batches, @@ -353,8 +384,12 @@ fn in_mem_partial_sort( rows.map(|rs| rs.into_iter().map(Arc::new).collect()), Some(row_tx), )); - let row_stream = UnboundedReceiverStream::new(row_rx).boxed(); - Ok((batch_stream, row_stream)) + if used_rows { + let row_stream = UnboundedReceiverStream::new(row_rx).boxed(); + Ok(SortedStream::new_from_streams(batch_stream, 0, row_stream)) + } else { + Ok(SortedStream::new_no_row_encoding(batch_stream, 0)) + } } } @@ -650,13 +685,16 @@ impl RecordBatchStream for SortedSizedRecordBatchStream { } async fn spill_partial_sorted_stream( - in_mem_stream: &mut SendableRecordBatchStream, + in_mem_stream: &mut SendableSortStream, path: &Path, + row_path: Option<&Path>, schema: SchemaRef, ) -> Result<()> { let (sender, receiver) = mpsc::channel(2); let path: PathBuf = path.into(); - let handle = task::spawn_blocking(move || write_sorted(receiver, path, schema)); + let row_path = row_path.map(|p| p.to_path_buf()); + let handle = + task::spawn_blocking(move || write_sorted(receiver, path, row_path, schema)); while let Some(item) = in_mem_stream.next().await { sender.send(item).await.ok(); } @@ -670,55 +708,62 @@ async fn spill_partial_sorted_stream( } fn read_spill_as_stream( - path: NamedTempFile, - schema: SchemaRef, - tx: mpsc::UnboundedSender>, -) -> Result { - let (sender, receiver): (Sender>, Receiver>) = - mpsc::channel(2); + spill: Spill, + sort_fields: Vec, +) -> Result { + let (sender, receiver) = mpsc::channel::(2); let join_handle = task::spawn_blocking(move || { - if let Err(e) = read_spill(sender, path.path(), tx) { - error!("Failure while reading spill file: {:?}. Error: {}", path, e); + if let Err(e) = read_spill(sender, &spill, sort_fields) { + error!( + "Failure while reading spill file: ({:?}, {:?}). Error: {}", + spill.record_batch_file, spill.rows_file, e + ); } }); - Ok(RecordBatchReceiverStream::create( - &schema, - receiver, - join_handle, - )) + Ok(Box::pin(SortReceiverStream::new(receiver, join_handle))) } fn write_sorted( - mut receiver: Receiver>, + mut receiver: Receiver, path: PathBuf, + row_path: Option, schema: SchemaRef, ) -> Result<()> { let mut writer = IPCWriter::new(path.as_ref(), schema.as_ref())?; + let mut row_writer = RowWriter::try_new(row_path.as_ref())?; while let Some(batch) = receiver.blocking_recv() { - writer.write(&batch?)?; + let (recbatch, rows) = batch?; + writer.write(&recbatch)?; + row_writer.write(rows)?; } writer.finish()?; + row_writer.finish()?; debug!( "Spilled {} batches of total {} rows to disk, memory released {}", writer.num_batches, writer.num_rows, - human_readable_size(writer.num_bytes as usize), + human_readable_size(writer.num_bytes as usize + row_writer.num_bytes), ); Ok(()) } fn read_spill( - sender: Sender>, - path: &Path, - tx: mpsc::UnboundedSender>, + sender: Sender, + spill: &Spill, + sort_fields: Vec, ) -> Result<()> { - let file = BufReader::new(File::open(path)?); + let file = BufReader::new(File::open(&spill.record_batch_file)?); let reader = FileReader::try_new(file, None)?; - for batch in reader { + let row_reader = RowReader::try_new(spill.rows_file.as_ref(), sort_fields)?; + for zipped in reader.zip(row_reader) { + let item = match zipped { + (Ok(batch), Ok(rows)) => Ok((batch, rows)), + (Err(err), Ok(_)) | (Err(err), Err(_)) => Err(err.into()), + (Ok(_), Err(err)) => Err(err), + }; // TODO: read spilled row data - tx.send(None).unwrap(); sender - .blocking_send(batch.map_err(Into::into)) + .blocking_send(item) .map_err(|e| DataFusionError::Execution(format!("{e}")))?; } Ok(()) @@ -1103,6 +1148,139 @@ async fn do_sort( result } +/// manages writing potential rows to and from disk +struct RowWriter { + // serializing w/ arrow ipc format for maximum code simplicity... probably sub-optimal + ipc_writer: Option, + schema: Option, + num_bytes: usize, +} +impl RowWriter { + fn try_new(path: Option>) -> Result { + use arrow::datatypes::{DataType, Field, Schema}; + match path { + Some(p) => { + let schema = Arc::new(Schema::new(vec![Field::new( + "bytes", + DataType::Binary, + false, + )])); + Ok(Self { + ipc_writer: Some(IPCWriter::new(p.as_ref(), schema.as_ref())?), + schema: Some(schema), + num_bytes: 0, + }) + } + None => Ok(Self { + ipc_writer: None, + schema: None, + num_bytes: 0, + }), + } + } + fn write(&mut self, rows: Option) -> Result<()> { + use arrow::array::BinaryBuilder; + if let (Some(writer), Some(schema), Some(rows)) = + (self.ipc_writer.as_mut(), self.schema.as_ref(), rows) + { + let numbytes = rows + .iter() + .map(|v| { + let bytes: &[u8] = v.as_ref(); + bytes.len() + }) + .sum::(); + println!("writing {} to spill", rows.num_rows()); + let mut builder = BinaryBuilder::with_capacity(rows.num_rows(), numbytes); + for r in rows.iter() { + let bytes: &[u8] = r.as_ref(); + builder.append_value(bytes); + } + let arr = builder.finish(); + let batch = RecordBatch::try_new(schema.clone(), vec![Arc::new(arr)])?; + writer.write(&batch)?; + self.num_bytes += numbytes; + Ok(()) + } else { + Ok(()) + } + } + fn finish(&mut self) -> Result<()> { + self.ipc_writer + .as_mut() + .map(|v| v.finish()) + .unwrap_or(Ok(())) + } +} + +/// manages reading potential rows to and from disk. +struct RowReader { + /// temporary file format solution is storing it w/ arrow IPC + reader: Option>>, + row_conv: RowConverter, +} +impl RowReader { + fn try_new( + path: Option>, + sort_fields: Vec, + ) -> Result { + let row_conv = RowConverter::new(sort_fields)?; + if let Some(path) = path { + let file = BufReader::new(File::open(path)?); + let reader = FileReader::try_new(file, None)?; + + Ok(Self { + reader: Some(reader), + row_conv, + }) + } else { + Ok(Self { + reader: None, + row_conv, + }) + } + } + fn parse_batch_helper(&self, batch: RecordBatch) -> Result> { + use arrow::array::BinaryArray; + let col = batch.column(0); + let bincol = col.as_any().downcast_ref::().ok_or_else(|| { + DataFusionError::Internal( + "unexepected error while parsing spilled row data".to_string(), + ) + })?; + let bytes = bincol + .into_iter() + .map(|v| { + v.map(bytes::Bytes::copy_from_slice).ok_or_else(|| { + DataFusionError::Internal( + "unexepected error while parsing spilled row data".to_string(), + ) + }) + }) + .collect::>>()?; + Ok(Some( + RowSelection::from_spilled(self.row_conv.parser(), bytes).into(), + )) + } +} +impl Iterator for RowReader { + type Item = Result>; + + fn next(&mut self) -> Option { + if let Some(reader) = self.reader.as_mut() { + match reader.next() { + Some(Ok(batch)) => Some(self.parse_batch_helper(batch)), + Some(Err(err)) => Some(Err(err.into())), + None => None, + } + } else { + // will be zipped with the main record batch reader so + // just yield None forever + Some(Ok(None)) + } + } +} + #[cfg(test)] mod tests { use super::*; diff --git a/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs b/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs index f6878da28ed1..681b63d7ac00 100644 --- a/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs +++ b/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs @@ -33,12 +33,9 @@ use futures::stream::{Fuse, FusedStream}; use futures::{ready, Stream, StreamExt, TryStreamExt}; use log::debug; use tokio::sync::mpsc; -use tokio::task::JoinHandle; -use tokio_stream::wrappers::ReceiverStream; use crate::error::{DataFusionError, Result}; use crate::execution::context::TaskContext; -use crate::physical_plan::common::AbortOnDropSingle; use crate::physical_plan::metrics::{ ExecutionPlanMetricsSet, MemTrackingMetrics, MetricsSet, }; @@ -207,12 +204,11 @@ impl ExecutionPlan for SortPreservingMergeExec { context.clone(), tx, ); - let stream = - Box::pin(SortReceiverStream::new(rx, join_handle)); - SortedStream { - stream, - mem_used: 0, - } + let stream = Box::pin(super::SortReceiverStream::new( + rx, + join_handle, + )); + SortedStream::new(stream, 0) } else { let (sender, receiver) = mpsc::channel(1); let join_handle = spawn_execution( @@ -241,10 +237,7 @@ impl ExecutionPlan for SortPreservingMergeExec { partition, context.clone(), )?; - Ok(SortedStream { - stream, - mem_used: 0, - }) + Ok(SortedStream::new(stream, 0)) } else { let stream = self.input.execute(partition, context.clone())?; @@ -837,30 +830,6 @@ impl From for SendableRecordBatchStream { } } -struct SortReceiverStream { - inner: ReceiverStream, - #[allow(dead_code)] - drop_helper: AbortOnDropSingle<()>, -} -impl SortReceiverStream { - fn new(rx: mpsc::Receiver, handle: JoinHandle<()>) -> Self { - let stream = ReceiverStream::new(rx); - Self { - inner: stream, - drop_helper: AbortOnDropSingle::new(handle), - } - } -} -impl Stream for SortReceiverStream { - type Item = SortStreamItem; - - fn poll_next( - mut self: std::pin::Pin<&mut Self>, - cx: &mut std::task::Context<'_>, - ) -> std::task::Poll> { - self.inner.poll_next_unpin(cx) - } -} #[cfg(test)] mod tests { use std::iter::FromIterator; From e313134295dfd1c7e8a189d05a7838aa08185127 Mon Sep 17 00:00:00 2001 From: Jay Miller <3744812+jaylmiller@users.noreply.github.com> Date: Wed, 15 Feb 2023 15:59:17 -0500 Subject: [PATCH 12/29] add row encoding sizes to spill calculations. update memory sizes in spill related unit tests to account for rows limit sorts dont use row encoding. --- .../core/src/physical_plan/sorts/sort.rs | 84 ++++++++----------- .../sorts/sort_preserving_merge.rs | 13 +-- 2 files changed, 41 insertions(+), 56 deletions(-) diff --git a/datafusion/core/src/physical_plan/sorts/sort.rs b/datafusion/core/src/physical_plan/sorts/sort.rs index a12626260300..8d0873294b56 100644 --- a/datafusion/core/src/physical_plan/sorts/sort.rs +++ b/datafusion/core/src/physical_plan/sorts/sort.rs @@ -147,13 +147,12 @@ impl ExternalSorter { // for being reliable, we need to reflect the memory usage of the partial batch. // // In addition, if it's row encoding was preserved, that would also change the size. - let new_size = batch_byte_size(&partial.sorted_batch); - // TODO - // + partial - // .sort_data - // .rows - // .as_ref() - // .map_or(0, |rows| rows.size()); + let new_size = batch_byte_size(&partial.sorted_batch) + + partial + .sort_data + .rows + .as_ref() + .map_or(0, |rows| rows.size()); match new_size.cmp(&size) { Ordering::Greater => { // We don't have to call try_grow here, since we have already used the @@ -181,8 +180,6 @@ impl ExternalSorter { } /// MergeSort in mem batches as well as spills into total order with `SortPreservingMergeStream`. - /// - /// todo: add flag to specify that the row encoding should not be preserved (to save memory) fn sort(&mut self) -> Result { let batch_size = self.session_config.batch_size(); @@ -221,7 +218,6 @@ impl ExternalSorter { let tracking_metrics = self .metrics_set .new_final_tracking(self.partition_id, &self.runtime.memory_pool); - println!("using SortPreservingMergeStream to emit row encodings"); Ok(Box::pin(SortPreservingMergeStream::new_from_streams( streams, self.schema.clone(), @@ -231,7 +227,6 @@ impl ExternalSorter { true, )?)) } else if !self.in_mem_batches.is_empty() { - // sort in mem doesnt require SortPreservingMergeStream let tracking_metrics = self .metrics_set .new_final_tracking(self.partition_id, &self.runtime.memory_pool); @@ -761,7 +756,6 @@ fn read_spill( (Err(err), Ok(_)) | (Err(err), Err(_)) => Err(err.into()), (Ok(_), Err(err)) => Err(err), }; - // TODO: read spilled row data sender .blocking_send(item) .map_err(|e| DataFusionError::Execution(format!("{e}")))?; @@ -866,7 +860,6 @@ impl SortExec { context: Arc, tx: mpsc::Sender, ) -> tokio::task::JoinHandle<()> { - // create owned vars for task let input = self.input.clone(); let expr = self.expr.clone(); let metrics = self.metrics_set.clone(); @@ -1033,37 +1026,30 @@ fn sort_batch( .iter() .map(|e| e.evaluate_to_sort_column(&batch)) .collect::>>()?; + let (indices, rows) = match (sort_columns.len(), fetch) { + // if single column or there's a limit, fallback to regular sort + (1, None) | (_, Some(_)) => (lexsort_to_indices(&sort_columns, fetch)?, None), + _ => { + let sort_fields = sort_columns + .iter() + .map(|c| { + let datatype = c.values.data_type().to_owned(); + SortField::new_with_options(datatype, c.options.unwrap_or_default()) + }) + .collect::>(); + let arrays: Vec = + sort_columns.iter().map(|c| c.values.clone()).collect(); + let mut row_converter = RowConverter::new(sort_fields)?; + let rows = row_converter.convert_columns(&arrays)?; - let (indices, rows) = if sort_columns.len() == 1 { - (lexsort_to_indices(&sort_columns, fetch)?, None) - } else { - let sort_fields = sort_columns - .iter() - .map(|c| { - let datatype = c.values.data_type().to_owned(); - SortField::new_with_options(datatype, c.options.unwrap_or_default()) - }) - .collect::>(); - let arrays: Vec = - sort_columns.iter().map(|c| c.values.clone()).collect(); - let mut row_converter = RowConverter::new(sort_fields)?; - let rows = row_converter.convert_columns(&arrays)?; - - let mut to_sort: Vec<(usize, Row)> = rows.into_iter().enumerate().collect(); - to_sort.sort_unstable_by(|(_, row_a), (_, row_b)| row_a.cmp(row_b)); - let limit = match fetch { - Some(lim) => lim.min(to_sort.len()), - None => to_sort.len(), - }; - let sorted_indices = to_sort - .iter() - .take(limit) - .map(|(idx, _)| *idx) - .collect::>(); - ( - UInt32Array::from_iter(sorted_indices.iter().map(|i| *i as u32)), - Some(RowSelection::new(rows, sorted_indices)), - ) + let mut to_sort: Vec<(usize, Row)> = rows.into_iter().enumerate().collect(); + to_sort.sort_unstable_by(|(_, row_a), (_, row_b)| row_a.cmp(row_b)); + let sorted_indices = to_sort.iter().map(|(idx, _)| *idx).collect::>(); + ( + UInt32Array::from_iter(sorted_indices.iter().map(|i| *i as u32)), + Some(RowSelection::new(rows, sorted_indices)), + ) + } }; // reorder all rows based on sorted indices @@ -1190,7 +1176,6 @@ impl RowWriter { bytes.len() }) .sum::(); - println!("writing {} to spill", rows.num_rows()); let mut builder = BinaryBuilder::with_capacity(rows.num_rows(), numbytes); for r in rows.iter() { let bytes: &[u8] = r.as_ref(); @@ -1361,7 +1346,9 @@ mod tests { #[tokio::test] async fn test_sort_spill() -> Result<()> { // trigger spill there will be 4 batches with 5.5KB for each - let config = RuntimeConfig::new().with_memory_limit(12288, 1.0); + // plus 1289 bytes of row data for each batch + let row_size = 1289; + let config = RuntimeConfig::new().with_memory_limit(12288 + (row_size * 4), 1.0); let runtime = Arc::new(RuntimeEnv::new(config)?); let session_ctx = SessionContext::with_config_rt(SessionConfig::new(), runtime); @@ -1432,6 +1419,7 @@ mod tests { // This test mirrors down the size from the example above. let avg_batch_size = 5336; let partitions = 4; + let added_row_size = 1289 * partitions; // A tuple of (fetch, expect_spillage) let test_options = vec![ @@ -1444,8 +1432,10 @@ mod tests { ]; for (fetch, expect_spillage) in test_options { - let config = RuntimeConfig::new() - .with_memory_limit(avg_batch_size * (partitions - 1), 1.0); + let config = RuntimeConfig::new().with_memory_limit( + avg_batch_size * (partitions - 1) + added_row_size, + 1.0, + ); let runtime = Arc::new(RuntimeEnv::new(config)?); let session_ctx = SessionContext::with_config_rt(SessionConfig::new(), runtime); diff --git a/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs b/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs index 681b63d7ac00..ff1dad34fe70 100644 --- a/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs +++ b/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs @@ -458,14 +458,8 @@ impl SortPreservingMergeStream { }) .collect::>>()?; // use preserved row encoding if it existed, otherwise create now - // - // (could be because: either not used at all (single col) - // or currently rows are not spilled to disk.) let rows = match preserved_rows { - Some(rows) => { - println!("got preserved row encoding..."); - rows - } + Some(rows) => rows, None => match self.row_converter.convert_columns(&cols) { Ok(rows) => rows.into(), Err(e) => { @@ -475,7 +469,9 @@ impl SortPreservingMergeStream { } }, }; - + // if this stream should emit the row encoding, save it in + // batches so that the sorted rows can be constructed + // when the sroted record batches are if self.preserve_row_encoding { self.batches[idx].push_back((batch, Some(rows.clone()))) } else { @@ -570,7 +566,6 @@ impl SortPreservingMergeStream { make_arrow_array(array_data.freeze()) }) .collect(); - dbg!(self.preserve_row_encoding); let rows = if self.preserve_row_encoding { if self.in_progress.is_empty() { Some(RowBatch::new(vec![], vec![])) From 8b2450ba607542c903d3db0d1a7c54e1399acd1d Mon Sep 17 00:00:00 2001 From: Jay Miller <3744812+jaylmiller@users.noreply.github.com> Date: Wed, 15 Feb 2023 16:14:21 -0500 Subject: [PATCH 13/29] clean comments and small todos --- datafusion/core/src/physical_plan/sorts/mod.rs | 9 +++------ datafusion/core/src/physical_plan/sorts/sort.rs | 3 +-- 2 files changed, 4 insertions(+), 8 deletions(-) diff --git a/datafusion/core/src/physical_plan/sorts/mod.rs b/datafusion/core/src/physical_plan/sorts/mod.rs index 66497c16d211..ea5497b2330a 100644 --- a/datafusion/core/src/physical_plan/sorts/mod.rs +++ b/datafusion/core/src/physical_plan/sorts/mod.rs @@ -77,10 +77,6 @@ impl SortedStream { mem_used, row_encoding_ignored: false, } - // if let Some(row_stream) = row_stream { - // } else { - // Self::new_no_row_encoding(stream, mem_used) - // } } /// create stream where the row encoding for each batch is always None pub(crate) fn new_no_row_encoding( @@ -142,8 +138,7 @@ impl RowBatch { /// Amount of bytes pub fn memory_size(&self) -> usize { let indices_size = self.indices.len() * 2 * std::mem::size_of::(); - // rows are refs so dont count the size inside the refs, just the refs itself? - let rows_size = 0; + let rows_size = self.rows.iter().map(|r| r.size()).sum::(); rows_size + indices_size + std::mem::size_of::() } } @@ -189,7 +184,9 @@ pub struct RowSelection { } #[derive(Debug)] enum RowData { + /// Rows that have always been in memory Rows(Rows), + /// Rows that were spilled to disk and then later read back into mem Spilled { parser: RowParser, bytes: Vec, diff --git a/datafusion/core/src/physical_plan/sorts/sort.rs b/datafusion/core/src/physical_plan/sorts/sort.rs index 8d0873294b56..7f50535d2f25 100644 --- a/datafusion/core/src/physical_plan/sorts/sort.rs +++ b/datafusion/core/src/physical_plan/sorts/sort.rs @@ -90,7 +90,7 @@ struct ExternalSorter { } struct Spill { record_batch_file: NamedTempFile, - // `None` when row encoding not used + // `None` when row encoding not preserved rows_file: Option, } @@ -418,7 +418,6 @@ fn get_sorted_iter( sort_data.iter().map(|d| d.rows.as_ref()).collect(); let indices = match rows_per_batch { Some(rows_per_batch) => { - // concat rows in their selection order and then sort let mut to_sort = rows_per_batch .iter() .flat_map(|r| r.iter()) From 085a87102492a855fdaaa0b62fcc546f971ec52e Mon Sep 17 00:00:00 2001 From: Jay Miller <3744812+jaylmiller@users.noreply.github.com> Date: Thu, 16 Feb 2023 14:35:58 -0500 Subject: [PATCH 14/29] cleanup SortedStream types - use SortedStream instead of dynamic dispatch everywhere, gets rid of all the Box::pin being when passing streams between funcs --- .../core/src/physical_plan/sorts/mod.rs | 209 ++++++++++++++---- .../core/src/physical_plan/sorts/sort.rs | 52 +++-- .../sorts/sort_preserving_merge.rs | 41 ++-- 3 files changed, 208 insertions(+), 94 deletions(-) diff --git a/datafusion/core/src/physical_plan/sorts/mod.rs b/datafusion/core/src/physical_plan/sorts/mod.rs index ea5497b2330a..1381a42c71d6 100644 --- a/datafusion/core/src/physical_plan/sorts/mod.rs +++ b/datafusion/core/src/physical_plan/sorts/mod.rs @@ -22,8 +22,8 @@ use std::{ fmt::{Debug, Formatter}, pin::Pin, sync::Arc, + task::Poll, }; - mod cursor; mod index; pub mod sort; @@ -34,48 +34,85 @@ use arrow::{ row::{Row, RowParser, Rows}, }; pub use cursor::SortKeyCursor; -use futures::{Stream, StreamExt, TryStreamExt}; +use futures::{stream::Fuse, Stream, StreamExt}; pub use index::RowIndex; -use tokio::{sync::mpsc, task::JoinHandle}; -use tokio_stream::wrappers::ReceiverStream; +use pin_project_lite::pin_project; +use tokio::task::JoinHandle; -use super::common::AbortOnDropSingle; +use super::{common::AbortOnDropSingle, metrics::MemTrackingMetrics}; -pub(crate) type RowStream = Pin> + Send>>; +pub(crate) type SendableRowStream = Pin> + Send>>; pub(crate) type SortStreamItem = Result<(RecordBatch, Option)>; pub(crate) type SendableSortStream = Pin + Send>>; -pub(crate) struct SortedStream { - stream: SendableSortStream, - mem_used: usize, - // flag is only true if this was intialized wiith `new_no_row_encoding` - row_encoding_ignored: bool, + +pin_project! { + pub(crate) struct SortedStream { + #[pin] + batches: Option>, + #[pin] + rows: Option>, + #[pin] + pairs_stream: Option, + pairs_rx: Option>, + last_batch: Option>, + last_row: Option>, + mem_used: usize, + // flag is only true if this was intialized wiith `new_no_row_encoding` + row_encoding_ignored: bool, + rx_drop_helper: Option>, + is_empty: bool + } } + impl SortedStream { pub(crate) fn new(stream: SendableSortStream, mem_used: usize) -> Self { Self { - stream, + batches: None, + rows: None, + pairs_rx: None, + pairs_stream: Some(stream), + rx_drop_helper: None, mem_used, row_encoding_ignored: false, + last_batch: None, + last_row: None, + is_empty: false, + } + } + pub(crate) fn new_from_rx( + rx: tokio::sync::mpsc::Receiver, + handle: JoinHandle<()>, + mem_used: usize, + ) -> Self { + Self { + batches: None, + rows: None, + pairs_rx: Some(rx), + pairs_stream: None, + rx_drop_helper: Some(AbortOnDropSingle::new(handle)), + mem_used, + row_encoding_ignored: false, + last_batch: None, + last_row: None, + is_empty: false, } } - pub(crate) fn new_from_streams( stream: SendableRecordBatchStream, mem_used: usize, - row_stream: RowStream, + row_stream: SendableRowStream, ) -> Self { - let stream = Box::pin(stream.zip(row_stream).map(|item| { - let batch: Result = item.0; - let rows: Option = item.1; - match batch { - Ok(batch) => Ok((batch, rows)), - Err(err) => Err(err), - } - })); Self { - stream, + batches: Some(stream.fuse()), + rows: Some(row_stream.fuse()), + pairs_rx: None, + pairs_stream: None, mem_used, row_encoding_ignored: false, + last_batch: None, + last_row: None, + rx_drop_helper: None, + is_empty: false, } } /// create stream where the row encoding for each batch is always None @@ -83,11 +120,32 @@ impl SortedStream { stream: SendableRecordBatchStream, mem_used: usize, ) -> Self { - let stream = Box::pin(stream.map_ok(|batch| (batch, None))); Self { - stream, + batches: Some(stream.fuse()), + rows: None, mem_used, + pairs_rx: None, + pairs_stream: None, row_encoding_ignored: true, + last_batch: None, + last_row: None, + rx_drop_helper: None, + is_empty: false, + } + } + pub(crate) fn empty() -> Self { + Self { + is_empty: true, + + batches: None, + rows: None, + mem_used: 0, + pairs_rx: None, + pairs_stream: None, + row_encoding_ignored: true, + last_batch: None, + last_row: None, + rx_drop_helper: None, } } } @@ -96,6 +154,83 @@ impl Debug for SortedStream { write!(f, "InMemSorterStream") } } +impl Stream for SortedStream { + type Item = SortStreamItem; + + fn poll_next( + self: Pin<&mut Self>, + cx: &mut std::task::Context<'_>, + ) -> Poll> { + let this = self.project(); + if *this.is_empty { + return Poll::Ready(None); + } + if this.pairs_rx.is_some() { + return this.pairs_rx.as_mut().unwrap().poll_recv(cx); + } + if this.pairs_stream.is_some() { + return this.pairs_stream.as_pin_mut().unwrap().poll_next(cx); + } + if this.rows.is_none() { + // even if no rows stream there has to be a batch stream + return match this.batches.as_pin_mut().unwrap().poll_next(cx) { + Poll::Ready(Some(Ok(batch))) => Poll::Ready(Some(Ok((batch, None)))), + Poll::Ready(Some(Err(err))) => Poll::Ready(Some(Err(err))), + Poll::Ready(None) => Poll::Ready(None), + Poll::Pending => Poll::Pending, + }; + } + // otherwise both batches and rows exist + let mut batches = this.batches.as_pin_mut().unwrap(); + let mut rows = this.rows.as_pin_mut().unwrap(); + if this.last_batch.is_none() { + match batches.as_mut().poll_next(cx) { + Poll::Ready(Some(res)) => *this.last_batch = Some(res), + Poll::Ready(None) | Poll::Pending => {} + } + } + if this.last_row.is_none() { + match rows.as_mut().poll_next(cx) { + Poll::Ready(Some(maybe_rows)) => *this.last_row = Some(maybe_rows), + Poll::Ready(None) | Poll::Pending => {} + } + } + if this.last_batch.is_some() && this.last_row.is_some() { + let result = this.last_batch.take().unwrap(); + let maybe_row = this.last_row.take().unwrap(); + Poll::Ready(Some(result.map(|batch| (batch, maybe_row)))) + } else if rows.is_done() || batches.is_done() { + Poll::Ready(None) + } else { + Poll::Pending + } + } +} +// helper logic used a few times. version of metrics.record_poll with different inner type +pub(crate) fn record_poll_sort_item( + metrics: &MemTrackingMetrics, + poll: std::task::Poll>, +) -> std::task::Poll> { + if let std::task::Poll::Ready(maybe_sort_item) = &poll { + match maybe_sort_item { + Some(Ok((batch, _rows))) => metrics.record_output(batch.num_rows()), + Some(Err(_)) | None => { + metrics.done(); + } + } + } + poll +} + +// pub(crate) struct SortedReceiverStream { +// batch_rx: ReceiverStream>, +// row_rx: Option>>>, +// #[allow(dead_code)] +// drop_helper: AbortOnDropSingle<()>, +// } +// impl Stream for SortedReceiverStream { + +// } /// Cloneable batch of rows taken from multiple [RowSelection]s #[derive(Debug, Clone)] @@ -295,30 +430,6 @@ impl<'a> Iterator for RowSelectionIter<'a> { } } } -pub(crate) struct SortReceiverStream { - inner: ReceiverStream, - #[allow(dead_code)] - drop_helper: AbortOnDropSingle<()>, -} -impl SortReceiverStream { - fn new(rx: mpsc::Receiver, handle: JoinHandle<()>) -> Self { - let stream = ReceiverStream::new(rx); - Self { - inner: stream, - drop_helper: AbortOnDropSingle::new(handle), - } - } -} -impl Stream for SortReceiverStream { - type Item = SortStreamItem; - - fn poll_next( - mut self: std::pin::Pin<&mut Self>, - cx: &mut std::task::Context<'_>, - ) -> std::task::Poll> { - self.inner.poll_next_unpin(cx) - } -} #[cfg(test)] mod tests { diff --git a/datafusion/core/src/physical_plan/sorts/sort.rs b/datafusion/core/src/physical_plan/sorts/sort.rs index 7f50535d2f25..4c342df25810 100644 --- a/datafusion/core/src/physical_plan/sorts/sort.rs +++ b/datafusion/core/src/physical_plan/sorts/sort.rs @@ -19,7 +19,7 @@ //! It will do in-memory sorting if it has enough memory budget //! but spills to disk if needed. -use super::{RowBatch, RowSelection, SortReceiverStream}; +use super::{RowBatch, RowSelection}; use super::{SendableSortStream, SortStreamItem}; use crate::error::{DataFusionError, Result}; use crate::execution::context::TaskContext; @@ -62,7 +62,7 @@ use std::sync::Arc; use std::task::{Context, Poll}; use tempfile::NamedTempFile; use tokio::sync::mpsc::{self, Receiver, Sender}; -use tokio::task; +use tokio::task::{self, JoinHandle}; use tokio_stream::wrappers::UnboundedReceiverStream; /// Sort arbitrary size of data to get a total order (may spill several times during sorting based on free memory available). @@ -180,7 +180,7 @@ impl ExternalSorter { } /// MergeSort in mem batches as well as spills into total order with `SortPreservingMergeStream`. - fn sort(&mut self) -> Result { + fn sort(&mut self) -> Result { let batch_size = self.session_config.batch_size(); if self.spilled_before() { @@ -212,20 +212,21 @@ impl ExternalSorter { }) .collect::>>()?; for spill in self.spills.drain(..) { - let stream = read_spill_as_stream(spill, sort_fields.to_owned())?; - streams.push(SortedStream::new(stream, 0)); + let (rx, handle) = read_spill_as_stream(spill, sort_fields.to_owned())?; + streams.push(SortedStream::new_from_rx(rx, handle, 0)); } let tracking_metrics = self .metrics_set .new_final_tracking(self.partition_id, &self.runtime.memory_pool); - Ok(Box::pin(SortPreservingMergeStream::new_from_streams( + let sort_stream = SortPreservingMergeStream::new_from_streams( streams, self.schema.clone(), &self.expr, tracking_metrics, self.session_config.batch_size(), true, - )?)) + )?; + Ok(SortedStream::new(Box::pin(sort_stream), 0)) } else if !self.in_mem_batches.is_empty() { let tracking_metrics = self .metrics_set @@ -240,9 +241,9 @@ impl ExternalSorter { )?; // Report to the memory manager we are no longer using memory self.reservation.free(); - Ok(stream.stream) + Ok(stream) } else { - Ok(Box::pin(futures::stream::empty())) + Ok(SortedStream::empty()) } } @@ -287,7 +288,7 @@ impl ExternalSorter { ) }; spill_partial_sorted_stream( - &mut stream.stream, + &mut stream, spillfile.path(), rows_file.as_ref().map(|f| f.path()), self.schema.clone(), @@ -679,7 +680,7 @@ impl RecordBatchStream for SortedSizedRecordBatchStream { } async fn spill_partial_sorted_stream( - in_mem_stream: &mut SendableSortStream, + in_mem_stream: &mut SortedStream, path: &Path, row_path: Option<&Path>, schema: SchemaRef, @@ -704,7 +705,7 @@ async fn spill_partial_sorted_stream( fn read_spill_as_stream( spill: Spill, sort_fields: Vec, -) -> Result { +) -> Result<(mpsc::Receiver, JoinHandle<()>)> { let (sender, receiver) = mpsc::channel::(2); let join_handle = task::spawn_blocking(move || { if let Err(e) = read_spill(sender, &spill, sort_fields) { @@ -714,7 +715,7 @@ fn read_spill_as_stream( ); } }); - Ok(Box::pin(SortReceiverStream::new(receiver, join_handle))) + Ok((receiver, join_handle)) } fn write_sorted( @@ -971,11 +972,28 @@ impl ExecutionPlan for SortExec { partition: usize, context: Arc, ) -> Result { + debug!("Start SortExec::execute for partition {} of context session_id {} and task_id {:?}", partition, context.session_id(), context.task_id()); + + debug!( + "Start invoking SortExec's input.execute for partition: {}", + partition + ); + + let input = self.input.execute(partition, context.clone())?; + + debug!("End SortExec's input.execute for partition: {}", partition); Ok(Box::pin(RecordBatchStreamAdapter::new( self.schema(), - self.execute_save_row_encoding(partition, context)? - // take the record batch and ignore the rows - .map_ok(|(record_batch, _rows)| record_batch), + futures::stream::once(do_sort( + input, + partition, + self.expr.clone(), + self.metrics_set.clone(), + context, + self.fetch(), + )) + .try_flatten() + .map_ok(|(record_batch, _rows)| record_batch), ))) } @@ -1099,7 +1117,7 @@ async fn do_sort( metrics_set: CompositeMetricsSet, context: Arc, fetch: Option, -) -> Result { +) -> Result { debug!( "Start do_sort for partition {} of context session_id {} and task_id {:?}", partition_id, diff --git a/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs b/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs index ff1dad34fe70..c68a2bfdd715 100644 --- a/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs +++ b/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs @@ -34,6 +34,7 @@ use futures::{ready, Stream, StreamExt, TryStreamExt}; use log::debug; use tokio::sync::mpsc; +use super::{record_poll_sort_item, RowBatch, RowSelection, SortStreamItem}; use crate::error::{DataFusionError, Result}; use crate::execution::context::TaskContext; use crate::physical_plan::metrics::{ @@ -49,8 +50,6 @@ use crate::physical_plan::{ }; use datafusion_physical_expr::EquivalenceProperties; -use super::{RowBatch, RowSelection, SendableSortStream, SortStreamItem}; - /// Sort preserving merge execution plan /// /// This takes an input execution plan and a list of sort expressions, and @@ -204,11 +203,7 @@ impl ExecutionPlan for SortPreservingMergeExec { context.clone(), tx, ); - let stream = Box::pin(super::SortReceiverStream::new( - rx, - join_handle, - )); - SortedStream::new(stream, 0) + SortedStream::new_from_rx(rx, join_handle, 0) } else { let (sender, receiver) = mpsc::channel(1); let join_handle = spawn_execution( @@ -233,11 +228,11 @@ impl ExecutionPlan for SortPreservingMergeExec { if let Some(sort_plan) = self.input.as_any().downcast_ref::() { - let stream = sort_plan.execute_save_row_encoding( + let sortstream = sort_plan.execute_save_row_encoding( partition, context.clone(), )?; - Ok(SortedStream::new(stream, 0)) + Ok(SortedStream::new(sortstream, 0)) } else { let stream = self.input.execute(partition, context.clone())?; @@ -288,7 +283,7 @@ impl ExecutionPlan for SortPreservingMergeExec { struct MergingStreams { /// The sorted input streams to merge together - streams: Vec>, + streams: Vec>, /// number of streams num_streams: usize, } @@ -302,7 +297,7 @@ impl std::fmt::Debug for MergingStreams { } impl MergingStreams { - fn new(input_streams: Vec>) -> Self { + fn new(input_streams: Vec>) -> Self { Self { num_streams: input_streams.len(), streams: input_streams, @@ -402,9 +397,7 @@ impl SortPreservingMergeStream { Ok(Self { schema, batches, - streams: MergingStreams::new( - streams.into_iter().map(|s| s.stream.fuse()).collect(), - ), + streams: MergingStreams::new(streams.into_iter().map(|s| s.fuse()).collect()), column_expressions: expressions.iter().map(|x| x.expr.clone()).collect(), tracking_metrics, aborted: false, @@ -459,7 +452,10 @@ impl SortPreservingMergeStream { .collect::>>()?; // use preserved row encoding if it existed, otherwise create now let rows = match preserved_rows { - Some(rows) => rows, + Some(rows) => { + // dbg!(&rows); + rows + } None => match self.row_converter.convert_columns(&cols) { Ok(rows) => rows.into(), Err(e) => { @@ -570,6 +566,7 @@ impl SortPreservingMergeStream { if self.in_progress.is_empty() { Some(RowBatch::new(vec![], vec![])) } else { + println!("building rowbatch"); let rows = self .batches .iter() @@ -654,19 +651,7 @@ impl Stream for SortPreservingMergeStream { cx: &mut Context<'_>, ) -> Poll> { let poll = self.poll_next_inner(cx); - // cant use `tracking_metrics.record_poll` since Self::Item is wrong type - // this should do the same thing as `tracking_metrics.record_poll` - if let Poll::Ready(maybe_sort_item) = &poll { - match maybe_sort_item { - Some(Ok((batch, _rows))) => { - self.tracking_metrics.record_output(batch.num_rows()) - } - Some(Err(_)) | None => { - self.tracking_metrics.done(); - } - } - } - poll + record_poll_sort_item(&self.tracking_metrics, poll) } } From 4196a25f224092bd92245e762707c39348d072c5 Mon Sep 17 00:00:00 2001 From: Jay Miller <3744812+jaylmiller@users.noreply.github.com> Date: Thu, 16 Feb 2023 12:05:59 -0500 Subject: [PATCH 15/29] add SortExec input case to each merge bench case fix lil typo error in sort bench --- datafusion/core/benches/merge.rs | 72 ++++++++++++++++++++++++++++++++ datafusion/core/benches/sort.rs | 2 +- 2 files changed, 73 insertions(+), 1 deletion(-) diff --git a/datafusion/core/benches/merge.rs b/datafusion/core/benches/merge.rs index a7ac6cd41dac..f1c4736039f9 100644 --- a/datafusion/core/benches/merge.rs +++ b/datafusion/core/benches/merge.rs @@ -80,6 +80,7 @@ use arrow::{ /// Benchmarks for SortPreservingMerge stream use criterion::{criterion_group, criterion_main, Criterion}; +use datafusion::physical_plan::sorts::sort::SortExec; use datafusion::{ execution::context::TaskContext, physical_plan::{ @@ -136,11 +137,22 @@ fn criterion_benchmark(c: &mut Criterion) { b.iter(move || case.run()) }); + c.bench_function("merge i64 SortExec input", |b| { + let case = MergeBenchCase::new_with_sort_input(&I64_STREAMS); + + b.iter(move || case.run()) + }); + c.bench_function("merge f64", |b| { let case = MergeBenchCase::new(&F64_STREAMS); b.iter(move || case.run()) }); + c.bench_function("merge f64 SortExec input", |b| { + let case = MergeBenchCase::new_with_sort_input(&F64_STREAMS); + + b.iter(move || case.run()) + }); c.bench_function("merge utf8 low cardinality", |b| { let case = MergeBenchCase::new(&UTF8_LOW_CARDINALITY_STREAMS); @@ -148,39 +160,79 @@ fn criterion_benchmark(c: &mut Criterion) { b.iter(move || case.run()) }); + c.bench_function("merge utf8 low cardinality SortExec", |b| { + let case = MergeBenchCase::new_with_sort_input(&UTF8_LOW_CARDINALITY_STREAMS); + + b.iter(move || case.run()) + }); + c.bench_function("merge utf8 high cardinality", |b| { let case = MergeBenchCase::new(&UTF8_HIGH_CARDINALITY_STREAMS); b.iter(move || case.run()) }); + c.bench_function("merge utf8 high cardinality SortExec input", |b| { + let case = MergeBenchCase::new_with_sort_input(&UTF8_HIGH_CARDINALITY_STREAMS); + + b.iter(move || case.run()) + }); + c.bench_function("merge utf8 tuple", |b| { let case = MergeBenchCase::new(&UTF8_TUPLE_STREAMS); b.iter(move || case.run()) }); + c.bench_function("merge utf8 tuple SortExec input", |b| { + let case = MergeBenchCase::new_with_sort_input(&UTF8_TUPLE_STREAMS); + + b.iter(move || case.run()) + }); + c.bench_function("merge utf8 dictionary", |b| { let case = MergeBenchCase::new(&DICTIONARY_STREAMS); b.iter(move || case.run()) }); + c.bench_function("merge utf8 dictionary SortExec input", |b| { + let case = MergeBenchCase::new_with_sort_input(&DICTIONARY_STREAMS); + + b.iter(move || case.run()) + }); + c.bench_function("merge utf8 dictionary tuple", |b| { let case = MergeBenchCase::new(&DICTIONARY_TUPLE_STREAMS); b.iter(move || case.run()) }); + c.bench_function("merge utf8 dictionary tuple SortExec input", |b| { + let case = MergeBenchCase::new_with_sort_input(&DICTIONARY_TUPLE_STREAMS); + b.iter(move || case.run()) + }); + c.bench_function("merge mixed utf8 dictionary tuple", |b| { let case = MergeBenchCase::new(&MIXED_DICTIONARY_TUPLE_STREAMS); b.iter(move || case.run()) }); + c.bench_function("merge mixed utf8 dictionary tuple SortExec input", |b| { + let case = MergeBenchCase::new_with_sort_input(&MIXED_DICTIONARY_TUPLE_STREAMS); + b.iter(move || case.run()) + }); + c.bench_function("merge mixed tuple", |b| { let case = MergeBenchCase::new(&MIXED_TUPLE_STREAMS); b.iter(move || case.run()) }); + + c.bench_function("merge mixed tuple SortExec input", |b| { + let case = MergeBenchCase::new_with_sort_input(&MIXED_TUPLE_STREAMS); + + b.iter(move || case.run()) + }); } /// Encapsulates running each test case @@ -214,6 +266,26 @@ impl MergeBenchCase { } } + fn new_with_sort_input(partitions: &[Vec]) -> Self { + let runtime = tokio::runtime::Builder::new_multi_thread().build().unwrap(); + let session_ctx = SessionContext::new(); + let task_ctx = session_ctx.task_ctx(); + + let schema = partitions[0][0].schema(); + let sort = make_sort_exprs(schema.as_ref()); + + let projection = None; + let exec = Arc::new(MemoryExec::try_new(partitions, schema, projection).unwrap()); + let sort_exec = SortExec::try_new(sort.to_owned(), exec, None).unwrap(); + let plan = Arc::new(SortPreservingMergeExec::new(sort, Arc::new(sort_exec))); + + Self { + runtime, + task_ctx, + plan, + } + } + /// runs the specified plan to completion, draining all input and /// panic'ing on error fn run(&self) { diff --git a/datafusion/core/benches/sort.rs b/datafusion/core/benches/sort.rs index 2d9417d8bd3b..5d9905d9d989 100644 --- a/datafusion/core/benches/sort.rs +++ b/datafusion/core/benches/sort.rs @@ -104,7 +104,7 @@ fn criterion_benchmark(c: &mut Criterion) { b.iter(move || case.run()) }); c.bench_function("sort utf8 low cardinality preserve partitioning", |b| { - let case = SortBenchCase::new(&UTF8_LOW_CARDINALITY_STREAMS); + let case = SortBenchCasePreservePartitioning::new(&UTF8_LOW_CARDINALITY_STREAMS); b.iter(move || case.run()) }); From d4f5c104ce8847058b012b0b5378fe5909e6bcb8 Mon Sep 17 00:00:00 2001 From: Jay Miller <3744812+jaylmiller@users.noreply.github.com> Date: Thu, 16 Feb 2023 17:52:21 -0500 Subject: [PATCH 16/29] row serialization format --- .../core/src/physical_plan/sorts/mod.rs | 4 +- .../core/src/physical_plan/sorts/sort.rs | 221 ++++++++++++------ .../sorts/sort_preserving_merge.rs | 1 - 3 files changed, 146 insertions(+), 80 deletions(-) diff --git a/datafusion/core/src/physical_plan/sorts/mod.rs b/datafusion/core/src/physical_plan/sorts/mod.rs index 1381a42c71d6..71289aadb3e7 100644 --- a/datafusion/core/src/physical_plan/sorts/mod.rs +++ b/datafusion/core/src/physical_plan/sorts/mod.rs @@ -324,7 +324,7 @@ enum RowData { /// Rows that were spilled to disk and then later read back into mem Spilled { parser: RowParser, - bytes: Vec, + bytes: Vec>, }, } impl RowData { @@ -355,7 +355,7 @@ impl RowSelection { indices: Some(indices), } } - fn from_spilled(parser: RowParser, bytes: Vec) -> Self { + fn from_spilled(parser: RowParser, bytes: Vec>) -> Self { Self { rows: RowData::Spilled { parser, bytes }, indices: None, diff --git a/datafusion/core/src/physical_plan/sorts/sort.rs b/datafusion/core/src/physical_plan/sorts/sort.rs index 4c342df25810..02329ac8c31e 100644 --- a/datafusion/core/src/physical_plan/sorts/sort.rs +++ b/datafusion/core/src/physical_plan/sorts/sort.rs @@ -57,6 +57,7 @@ use std::fmt; use std::fmt::{Debug, Formatter}; use std::fs::File; use std::io::BufReader; +use std::io::{Read, Write}; use std::path::{Path, PathBuf}; use std::sync::Arc; use std::task::{Context, Poll}; @@ -737,7 +738,9 @@ fn write_sorted( "Spilled {} batches of total {} rows to disk, memory released {}", writer.num_batches, writer.num_rows, - human_readable_size(writer.num_bytes as usize + row_writer.num_bytes), + human_readable_size( + writer.num_bytes as usize + row_writer.num_row_bytes as usize + ), ); Ok(()) } @@ -1154,72 +1157,60 @@ async fn do_sort( /// manages writing potential rows to and from disk struct RowWriter { // serializing w/ arrow ipc format for maximum code simplicity... probably sub-optimal - ipc_writer: Option, - schema: Option, - num_bytes: usize, + file: Option, + num_row_bytes: u32, } +const MAGIC_BYTES: &[u8] = b"AROW"; impl RowWriter { fn try_new(path: Option>) -> Result { - use arrow::datatypes::{DataType, Field, Schema}; match path { Some(p) => { - let schema = Arc::new(Schema::new(vec![Field::new( - "bytes", - DataType::Binary, - false, - )])); + let mut file = File::create(p)?; + file.write_all(MAGIC_BYTES)?; Ok(Self { - ipc_writer: Some(IPCWriter::new(p.as_ref(), schema.as_ref())?), - schema: Some(schema), - num_bytes: 0, + file: Some(file), + num_row_bytes: 0, }) } None => Ok(Self { - ipc_writer: None, - schema: None, - num_bytes: 0, + file: None, + num_row_bytes: 0, }), } } fn write(&mut self, rows: Option) -> Result<()> { - use arrow::array::BinaryBuilder; - if let (Some(writer), Some(schema), Some(rows)) = - (self.ipc_writer.as_mut(), self.schema.as_ref(), rows) - { - let numbytes = rows - .iter() - .map(|v| { - let bytes: &[u8] = v.as_ref(); - bytes.len() - }) - .sum::(); - let mut builder = BinaryBuilder::with_capacity(rows.num_rows(), numbytes); - for r in rows.iter() { - let bytes: &[u8] = r.as_ref(); - builder.append_value(bytes); + match (rows, self.file.as_mut()) { + (Some(rows), Some(file)) => { + file.write_all(&(rows.num_rows() as u32).to_le_bytes())?; + for row in rows.iter() { + let bytes: &[u8] = row.as_ref(); + let num_bytes = bytes.len() as u32; + self.num_row_bytes += num_bytes; + file.write_all(&num_bytes.to_le_bytes())?; + file.write_all(bytes)?; + } + Ok(()) } - let arr = builder.finish(); - let batch = RecordBatch::try_new(schema.clone(), vec![Arc::new(arr)])?; - writer.write(&batch)?; - self.num_bytes += numbytes; + // no-op + _ => Ok(()), + } + } + fn finish(&mut self) -> Result<()> { + if let Some(file) = self.file.as_mut() { + file.flush()?; Ok(()) } else { Ok(()) } } - fn finish(&mut self) -> Result<()> { - self.ipc_writer - .as_mut() - .map(|v| v.finish()) - .unwrap_or(Ok(())) - } } /// manages reading potential rows to and from disk. struct RowReader { /// temporary file format solution is storing it w/ arrow IPC - reader: Option>>, + file: Option, row_conv: RowConverter, + stopped: bool, } impl RowReader { fn try_new( @@ -1227,39 +1218,52 @@ impl RowReader { sort_fields: Vec, ) -> Result { let row_conv = RowConverter::new(sort_fields)?; - if let Some(path) = path { - let file = BufReader::new(File::open(path)?); - let reader = FileReader::try_new(file, None)?; - - Ok(Self { - reader: Some(reader), - row_conv, - }) - } else { - Ok(Self { - reader: None, + match path { + Some(p) => { + let mut file = File::open(p)?; + let mut buf = [0_u8; 4]; + file.read_exact(&mut buf)?; + if buf != MAGIC_BYTES { + return Err(DataFusionError::Internal( + "unexpected magic bytes in serialized rows file".to_owned(), + )); + } + Ok(Self { + file: Some(file), + row_conv, + stopped: false, + }) + } + None => Ok(Self { + file: None, row_conv, - }) + stopped: false, + }), } } - fn parse_batch_helper(&self, batch: RecordBatch) -> Result> { - use arrow::array::BinaryArray; - let col = batch.column(0); - let bincol = col.as_any().downcast_ref::().ok_or_else(|| { - DataFusionError::Internal( - "unexepected error while parsing spilled row data".to_string(), - ) - })?; - let bytes = bincol - .into_iter() - .map(|v| { - v.map(bytes::Bytes::copy_from_slice).ok_or_else(|| { - DataFusionError::Internal( - "unexepected error while parsing spilled row data".to_string(), - ) - }) - }) - .collect::>>()?; + + fn read_batch(&mut self) -> Result> { + let file = self.file.as_mut().unwrap(); + let mut buf = [0_u8; 4]; + match file.read_exact(&mut buf) { + Ok(_) => {} + Err(io_err) => { + if io_err.kind() == std::io::ErrorKind::UnexpectedEof { + return Ok(None); + } + return Err(io_err.into()); + } + } + let num_rows = u32::from_le_bytes(buf); + let mut bytes: Vec> = Vec::with_capacity(num_rows as usize); + for _ in 0..num_rows { + let mut buf = [0_u8; 4]; + file.read_exact(&mut buf)?; + let n = u32::from_le_bytes(buf); + let mut buf = vec![0_u8; n as usize]; + file.read_exact(&mut buf)?; + bytes.push(buf); + } Ok(Some( RowSelection::from_spilled(self.row_conv.parser(), bytes).into(), )) @@ -1269,11 +1273,18 @@ impl Iterator for RowReader { type Item = Result>; fn next(&mut self) -> Option { - if let Some(reader) = self.reader.as_mut() { - match reader.next() { - Some(Ok(batch)) => Some(self.parse_batch_helper(batch)), - Some(Err(err)) => Some(Err(err.into())), - None => None, + if self.stopped { + return None; + } + if self.file.is_some() { + let res = self.read_batch(); + match res { + Ok(Some(batch)) => Some(Ok(Some(batch))), + Ok(None) => None, + Err(err) => { + self.stopped = true; + Some(Err(err)) + } } } else { // will be zipped with the main record batch reader so @@ -1303,6 +1314,62 @@ mod tests { use futures::FutureExt; use std::collections::HashMap; + #[test] + fn test_row_writer_reader() { + use crate::prelude::SessionContext; + use arrow::array::{Int64Array, StringArray}; + use arrow::datatypes::DataType; + let sort_fields = vec![ + SortField::new(DataType::Int64), + SortField::new(DataType::Utf8), + ]; + let mut conv = RowConverter::new(sort_fields.to_owned()).unwrap(); + + fn makebatch(n: i64) -> RecordBatch { + let ints: Int64Array = (0..n).map(Some).collect(); + let varlengths: StringArray = + StringArray::from_iter((0..n).map(|i| i + 100).map(|i| { + if i % 3 == 0 { + None + } else { + Some((i.pow(2)).to_string()) + } + })); + RecordBatch::try_from_iter(vec![ + ("c1", Arc::new(ints) as _), + ("c2", Arc::new(varlengths) as _), + ]) + .unwrap() + } + let row_lens = vec![10, 0, 0, 1, 50]; + let batches = row_lens.iter().map(|i| makebatch(*i)).collect::>(); + let rows = batches + .iter() + .map(|b| conv.convert_columns(b.columns()).unwrap()) + .collect::>(); + + let ctx = SessionContext::new(); + let runtime = ctx.runtime_env(); + let tempfile = runtime.disk_manager.create_tmp_file("Sorting").unwrap(); + let mut wr = RowWriter::try_new(Some(tempfile.path())).unwrap(); + for r in rows { + wr.write(Some(r.into())).unwrap(); + } + wr.finish().unwrap(); + + let rdr = RowReader::try_new(Some(tempfile.path()), sort_fields).unwrap(); + let batches = rdr.collect::>(); + assert_eq!(batches.len(), row_lens.len()); + let read_lens = batches + .iter() + .map(|b| { + let rowbatch = b.as_ref().unwrap().as_ref().unwrap(); + rowbatch.num_rows() as i64 + }) + .collect::>(); + assert_eq!(row_lens, read_lens); + } + #[tokio::test] async fn test_in_mem_sort() -> Result<()> { let session_ctx = SessionContext::new(); diff --git a/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs b/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs index c68a2bfdd715..013ec14c4df3 100644 --- a/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs +++ b/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs @@ -566,7 +566,6 @@ impl SortPreservingMergeStream { if self.in_progress.is_empty() { Some(RowBatch::new(vec![], vec![])) } else { - println!("building rowbatch"); let rows = self .batches .iter() From 33c611c435d1b45d8490858d75c804022f19303c Mon Sep 17 00:00:00 2001 From: Jay Miller <3744812+jaylmiller@users.noreply.github.com> Date: Thu, 16 Feb 2023 20:10:32 -0500 Subject: [PATCH 17/29] RowBatch construction re-use row ref offsets instead of just appending --- .../sorts/sort_preserving_merge.rs | 20 +++++++++++++++---- 1 file changed, 16 insertions(+), 4 deletions(-) diff --git a/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs b/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs index 013ec14c4df3..153e85619bf1 100644 --- a/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs +++ b/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs @@ -578,9 +578,13 @@ impl SortPreservingMergeStream { }) }) .collect::>(); + // let stream_idx_count = self.in_progress.iter().map(|v| (v.batch_idx, v.stream_idx)).unique().count(); let mut new_indices: Vec<(usize, usize)> = Vec::with_capacity(self.in_progress.len()); - let mut new_rows: Vec> = vec![]; + let mut new_rows: Vec> = + Vec::with_capacity(rows.iter().map(|r| r.num_rows()).sum()); + // map index of `rows` to the location in `new_rows` + let mut offsets: Vec> = vec![None; rows.len()]; let first = &self.in_progress[0]; let mut buffer_idx = stream_to_buffer_idx[first.stream_idx] + first.batch_idx; @@ -596,10 +600,18 @@ impl SortPreservingMergeStream { continue; } let row_batch = rows[buffer_idx]; + let offset = match offsets[buffer_idx] { + Some(offset) => offset, + None => { + let offset = new_rows.len(); + new_rows.extend(row_batch.rows.iter().map(Arc::clone)); + offsets[buffer_idx] = Some(offset); + offset + } + }; let row_indices = &row_batch.indices[start_row_idx..end_row_idx]; - new_indices.extend( - row_indices.iter().map(|(x, y)| (*x + new_rows.len(), *y)), - ); + new_indices + .extend(row_indices.iter().map(|(x, y)| (*x + offset, *y))); new_rows.extend(row_batch.rows.iter().map(Arc::clone)); // start new batch of rows buffer_idx = next_buffer_idx; From 1513f9aa4c43d85060742531ee1a349ed839bba4 Mon Sep 17 00:00:00 2001 From: Jay Miller <3744812+jaylmiller@users.noreply.github.com> Date: Sat, 18 Feb 2023 13:00:10 -0500 Subject: [PATCH 18/29] dont need to keep array refs if we use rows --- .../core/src/physical_plan/sorts/sort.rs | 90 ++++++++++++------- 1 file changed, 59 insertions(+), 31 deletions(-) diff --git a/datafusion/core/src/physical_plan/sorts/sort.rs b/datafusion/core/src/physical_plan/sorts/sort.rs index 02329ac8c31e..8e6f98e61e3c 100644 --- a/datafusion/core/src/physical_plan/sorts/sort.rs +++ b/datafusion/core/src/physical_plan/sorts/sort.rs @@ -149,11 +149,10 @@ impl ExternalSorter { // // In addition, if it's row encoding was preserved, that would also change the size. let new_size = batch_byte_size(&partial.sorted_batch) - + partial - .sort_data - .rows - .as_ref() - .map_or(0, |rows| rows.size()); + + match partial.sort_data { + SortData::Rows(ref rows) => rows.size(), + SortData::Arrays(_) => 0, + }; match new_size.cmp(&size) { Ordering::Greater => { // We don't have to call try_grow here, since we have already used the @@ -333,7 +332,10 @@ fn in_mem_partial_sort( sort_data, sorted_batch, } = result; - let rowbatch: Option = sort_data.rows.map(Into::into); + let rowbatch: Option = match sort_data { + SortData::Rows(rows) => Some(rows.into()), + SortData::Arrays(_) => None, + }; let stream = Box::pin(SizedRecordBatchStream::new( schema, vec![Arc::new(sorted_batch)], @@ -370,7 +372,10 @@ fn in_mem_partial_sort( }; let rows = sort_data .into_iter() - .map(|d| d.rows) + .map(|d| match d { + SortData::Rows(rows) => Some(rows), + SortData::Arrays(_) => None, + }) .collect::>>(); let used_rows = rows.is_some(); let batch_stream = Box::pin(SortedSizedRecordBatchStream::new( @@ -407,7 +412,7 @@ fn get_sorted_iter( .iter() .enumerate() .flat_map(|(i, d)| { - (0..d.arrays[0].len()).map(move |r| CompositeIndex { + (0..d.num_rows()).map(move |r| CompositeIndex { // since we original use UInt32Array to index the combined mono batch, // component record batches won't overflow as well, // use u32 here for space efficiency. @@ -416,8 +421,13 @@ fn get_sorted_iter( }) }) .collect::>(); - let rows_per_batch: Option> = - sort_data.iter().map(|d| d.rows.as_ref()).collect(); + let rows_per_batch: Option> = sort_data + .iter() + .map(|d| match d { + SortData::Rows(ref rows) => Some(rows), + SortData::Arrays(_) => None, + }) + .collect(); let indices = match rows_per_batch { Some(rows_per_batch) => { let mut to_sort = rows_per_batch @@ -439,7 +449,11 @@ fn get_sorted_iter( .map(|(i, expr)| { let columns_i = sort_data .iter() - .map(|data| data.arrays[i].as_ref()) + .map(|data| match data { + // todo fix + SortData::Rows(_) => unreachable!(), + SortData::Arrays(arrays) => arrays[i].as_ref(), + }) .collect::>(); Ok(SortColumn { values: concat(columns_i.as_slice())?, @@ -1027,9 +1041,20 @@ impl ExecutionPlan for SortExec { } } -struct SortData { - arrays: Vec, - rows: Option, +enum SortData { + Rows(RowSelection), + Arrays(Vec), +} +impl SortData { + fn num_rows(&self) -> usize { + match self { + SortData::Rows(r) => r.num_rows(), + SortData::Arrays(a) => { + let first_col = &a[0]; + first_col.len() + } + } + } } struct BatchWithSortArray { sort_data: SortData, @@ -1091,25 +1116,28 @@ fn sort_batch( }) .collect::, ArrowError>>()?, )?; - - let sort_arrays = sort_columns - .into_iter() - .map(|sc| { - Ok(take( - sc.values.as_ref(), - &indices, - Some(TakeOptions { - check_bounds: false, - }), - )?) - }) - .collect::>>()?; + let sort_data = match rows { + Some(rows) => SortData::Rows(rows), + None => { + // only need sort_arrays when we dont have rows. + let sort_arrays = sort_columns + .into_iter() + .map(|sc| { + Ok(take( + sc.values.as_ref(), + &indices, + Some(TakeOptions { + check_bounds: false, + }), + )?) + }) + .collect::>>()?; + SortData::Arrays(sort_arrays) + } + }; Ok(BatchWithSortArray { - sort_data: SortData { - arrays: sort_arrays, - rows, - }, + sort_data, sorted_batch, }) } From 2ebcbc7996ad34954b249ed37e9c7cf954498807 Mon Sep 17 00:00:00 2001 From: Jay Miller <3744812+jaylmiller@users.noreply.github.com> Date: Sat, 18 Feb 2023 17:18:24 -0500 Subject: [PATCH 19/29] dont use channel for SortedSizedStream in sort (emit tuple) --- .../core/src/physical_plan/sorts/mod.rs | 10 ---- .../core/src/physical_plan/sorts/sort.rs | 50 +++++++------------ 2 files changed, 17 insertions(+), 43 deletions(-) diff --git a/datafusion/core/src/physical_plan/sorts/mod.rs b/datafusion/core/src/physical_plan/sorts/mod.rs index 71289aadb3e7..d90efa63cbdd 100644 --- a/datafusion/core/src/physical_plan/sorts/mod.rs +++ b/datafusion/core/src/physical_plan/sorts/mod.rs @@ -222,16 +222,6 @@ pub(crate) fn record_poll_sort_item( poll } -// pub(crate) struct SortedReceiverStream { -// batch_rx: ReceiverStream>, -// row_rx: Option>>>, -// #[allow(dead_code)] -// drop_helper: AbortOnDropSingle<()>, -// } -// impl Stream for SortedReceiverStream { - -// } - /// Cloneable batch of rows taken from multiple [RowSelection]s #[derive(Debug, Clone)] pub struct RowBatch { diff --git a/datafusion/core/src/physical_plan/sorts/sort.rs b/datafusion/core/src/physical_plan/sorts/sort.rs index 8e6f98e61e3c..6bdd27da3657 100644 --- a/datafusion/core/src/physical_plan/sorts/sort.rs +++ b/datafusion/core/src/physical_plan/sorts/sort.rs @@ -19,7 +19,7 @@ //! It will do in-memory sorting if it has enough memory budget //! but spills to disk if needed. -use super::{RowBatch, RowSelection}; +use super::{record_poll_sort_item, RowBatch, RowSelection}; use super::{SendableSortStream, SortStreamItem}; use crate::error::{DataFusionError, Result}; use crate::execution::context::TaskContext; @@ -37,7 +37,7 @@ use crate::physical_plan::sorts::SortedStream; use crate::physical_plan::stream::RecordBatchStreamAdapter; use crate::physical_plan::{ displayable, DisplayFormatType, Distribution, ExecutionPlan, Partitioning, - RecordBatchStream, SendableRecordBatchStream, Statistics, + SendableRecordBatchStream, Statistics, }; use crate::prelude::SessionConfig; use arrow::array::{make_array, Array, ArrayRef, MutableArrayData, UInt32Array}; @@ -64,7 +64,6 @@ use std::task::{Context, Poll}; use tempfile::NamedTempFile; use tokio::sync::mpsc::{self, Receiver, Sender}; use tokio::task::{self, JoinHandle}; -use tokio_stream::wrappers::UnboundedReceiverStream; /// Sort arbitrary size of data to get a total order (may spill several times during sorting based on free memory available). /// @@ -324,7 +323,6 @@ fn in_mem_partial_sort( tracking_metrics: MemTrackingMetrics, fetch: Option, ) -> Result { - let (row_tx, row_rx) = mpsc::unbounded_channel(); assert_ne!(buffered_batches.len(), 0); if buffered_batches.len() == 1 { let result = buffered_batches.pop().unwrap(); @@ -378,20 +376,19 @@ fn in_mem_partial_sort( }) .collect::>>(); let used_rows = rows.is_some(); - let batch_stream = Box::pin(SortedSizedRecordBatchStream::new( + let batch_stream = SortedSizedStream::new( schema, batches, sorted_iter, tracking_metrics, rows.map(|rs| rs.into_iter().map(Arc::new).collect()), - Some(row_tx), - )); - if used_rows { - let row_stream = UnboundedReceiverStream::new(row_rx).boxed(); - Ok(SortedStream::new_from_streams(batch_stream, 0, row_stream)) - } else { - Ok(SortedStream::new_no_row_encoding(batch_stream, 0)) + ) + .boxed(); + let mut stream = SortedStream::new(batch_stream, 0); + if !used_rows { + stream.row_encoding_ignored = true; } + Ok(stream) } } @@ -574,17 +571,16 @@ fn group_indices( } /// Stream of sorted record batches -struct SortedSizedRecordBatchStream { +struct SortedSizedStream { schema: SchemaRef, batches: Vec, sorted_iter: SortedIterator, num_cols: usize, metrics: MemTrackingMetrics, rows: Option>>, - rows_tx: Option>>, } -impl SortedSizedRecordBatchStream { +impl SortedSizedStream { /// new pub fn new( schema: SchemaRef, @@ -592,7 +588,6 @@ impl SortedSizedRecordBatchStream { sorted_iter: SortedIterator, mut metrics: MemTrackingMetrics, rows: Option>>, - rows_tx: Option>>, ) -> Self { let size = batches.iter().map(batch_byte_size).sum::() + sorted_iter.memory_size() @@ -602,20 +597,19 @@ impl SortedSizedRecordBatchStream { .map_or(0, |r| r.iter().map(|r| r.size()).sum()); metrics.init_mem_used(size); let num_cols = batches[0].num_columns(); - SortedSizedRecordBatchStream { + SortedSizedStream { schema, batches, sorted_iter, rows, num_cols, metrics, - rows_tx, } } } -impl Stream for SortedSizedRecordBatchStream { - type Item = Result; +impl Stream for SortedSizedStream { + type Item = SortStreamItem; fn poll_next( mut self: std::pin::Pin<&mut Self>, @@ -665,16 +659,12 @@ impl Stream for SortedSizedRecordBatchStream { .collect::>(); RowBatch::new(row_refs, indices) }); - - if let Some(ref tx) = self.rows_tx { - tx.send(row_batch).ok(); - } - let poll = Poll::Ready(Some(Ok(batch))); - self.metrics.record_poll(poll) + let poll = Poll::Ready(Some(Ok((batch, row_batch)))); + record_poll_sort_item(&self.metrics, poll) } Err(err) => { let poll = Poll::Ready(Some(Err(err))); - self.metrics.record_poll(poll) + record_poll_sort_item(&self.metrics, poll) } } } @@ -688,12 +678,6 @@ struct CompositeSlice { len: usize, } -impl RecordBatchStream for SortedSizedRecordBatchStream { - fn schema(&self) -> SchemaRef { - self.schema.clone() - } -} - async fn spill_partial_sorted_stream( in_mem_stream: &mut SortedStream, path: &Path, From 73549523edd9643c5cc061d918cbaa2c9d2609d5 Mon Sep 17 00:00:00 2001 From: Jay Miller <3744812+jaylmiller@users.noreply.github.com> Date: Sat, 18 Feb 2023 17:36:08 -0500 Subject: [PATCH 20/29] add unit test for edge case where we skip spilling the row data --- .../core/src/physical_plan/sorts/sort.rs | 34 +++++++++++++++++++ 1 file changed, 34 insertions(+) diff --git a/datafusion/core/src/physical_plan/sorts/sort.rs b/datafusion/core/src/physical_plan/sorts/sort.rs index 6bdd27da3657..8978e355c276 100644 --- a/datafusion/core/src/physical_plan/sorts/sort.rs +++ b/datafusion/core/src/physical_plan/sorts/sort.rs @@ -1510,6 +1510,40 @@ mod tests { Ok(()) } + #[tokio::test] + async fn test_spill_no_row_encoding_edge_case() -> Result<()> { + // trigger spill there will be 4 batches with 5.5KB for each + let config = RuntimeConfig::new().with_memory_limit(12288, 1.0); + let runtime = Arc::new(RuntimeEnv::new(config)?); + let session_ctx = SessionContext::with_config_rt(SessionConfig::new(), runtime); + + let partitions = 4; + let csv = test::scan_partitioned_csv(partitions)?; + let schema = csv.schema(); + + let sort_exec = Arc::new(SortExec::try_new( + vec![ + // c2 uin32 column + PhysicalSortExpr { + expr: col("c2", &schema)?, + options: SortOptions::default(), + }, + ], + Arc::new(CoalescePartitionsExec::new(csv)), + None, + )?); + let task_ctx = session_ctx.task_ctx(); + let result = collect(sort_exec.clone(), task_ctx).await?; + assert_eq!(result.len(), 1); + assert_eq!( + session_ctx.runtime_env().memory_pool.reserved(), + 0, + "The sort should have returned all memory used back to the memory manager" + ); + + Ok(()) + } + #[tokio::test] async fn test_sort_fetch_memory_calculation() -> Result<()> { // This test mirrors down the size from the example above. From e6fe175bc850a69a64140543b9144d753c0e746d Mon Sep 17 00:00:00 2001 From: Jay Miller <3744812+jaylmiller@users.noreply.github.com> Date: Sat, 18 Feb 2023 17:58:54 -0500 Subject: [PATCH 21/29] fix sort bench to actually use full data set in non-preserve partition case --- datafusion/core/benches/sort.rs | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/datafusion/core/benches/sort.rs b/datafusion/core/benches/sort.rs index 5d9905d9d989..0507a9308a28 100644 --- a/datafusion/core/benches/sort.rs +++ b/datafusion/core/benches/sort.rs @@ -29,6 +29,7 @@ use arrow::{ /// Benchmarks for SortExec use criterion::{criterion_group, criterion_main, Criterion}; +use datafusion::physical_plan::coalesce_partitions::CoalescePartitionsExec; use datafusion::{ execution::context::TaskContext, physical_plan::{memory::MemoryExec, sorts::sort::SortExec, ExecutionPlan}, @@ -199,7 +200,8 @@ impl SortBenchCase { let projection = None; let exec = MemoryExec::try_new(partitions, schema, projection).unwrap(); - let plan = Arc::new(SortExec::try_new(sort, Arc::new(exec), None).unwrap()); + let exec = Arc::new(CoalescePartitionsExec::new(Arc::new(exec))); + let plan = Arc::new(SortExec::try_new(sort, exec, None).unwrap()); Self { runtime, From ec4949214e63d24b3a1b16e4cbde4e4df85e267f Mon Sep 17 00:00:00 2001 From: Jay Miller <3744812+jaylmiller@users.noreply.github.com> Date: Mon, 20 Feb 2023 12:24:38 -0500 Subject: [PATCH 22/29] clippy --- datafusion/core/src/physical_plan/sorts/sort.rs | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/datafusion/core/src/physical_plan/sorts/sort.rs b/datafusion/core/src/physical_plan/sorts/sort.rs index a75f2c09708e..d3aef0e5521d 100644 --- a/datafusion/core/src/physical_plan/sorts/sort.rs +++ b/datafusion/core/src/physical_plan/sorts/sort.rs @@ -106,7 +106,6 @@ impl ExternalSorter { session_config: Arc, runtime: Arc, fetch: Option, - preserve_output_rows: bool, ) -> Self { let metrics = metrics_set.new_intermediate_baseline(partition_id); @@ -125,10 +124,14 @@ impl ExternalSorter { fetch, reservation, partition_id, - preserve_output_rows, + preserve_output_rows: false, } } + pub fn set_preserve_output_rows(&mut self, val: bool) { + self.preserve_output_rows = val; + } + async fn insert_batch( &mut self, input: RecordBatch, @@ -1162,8 +1165,10 @@ async fn do_sort( Arc::new(context.session_config().clone()), context.runtime_env(), fetch, - preserve_rows, ); + if preserve_rows { + sorter.set_preserve_output_rows(true); + } while let Some(batch) = input.next().await { let batch = batch?; sorter.insert_batch(batch, &tracking_metrics).await?; From e4304706092025153f457d4a61acbd65b8412399 Mon Sep 17 00:00:00 2001 From: Jay Miller <3744812+jaylmiller@users.noreply.github.com> Date: Tue, 21 Feb 2023 08:32:47 -0500 Subject: [PATCH 23/29] add data skewed to first partition case for the tuple sorts in bench --- datafusion/core/benches/sort.rs | 55 +++++++++++++++++++++++++++++++++ 1 file changed, 55 insertions(+) diff --git a/datafusion/core/benches/sort.rs b/datafusion/core/benches/sort.rs index 0507a9308a28..39e3bd44669d 100644 --- a/datafusion/core/benches/sort.rs +++ b/datafusion/core/benches/sort.rs @@ -132,6 +132,17 @@ fn criterion_benchmark(c: &mut Criterion) { b.iter(move || case.run()) }); + c.bench_function( + "sort utf8 tuple preserve partitioning data skewed to first", + |b| { + let case = SortBenchCasePreservePartitioning::new_all_data_in_first_partition( + &UTF8_TUPLE_STREAMS, + ); + + b.iter(move || case.run()) + }, + ); + c.bench_function("sort utf8 dictionary", |b| { let case = SortBenchCase::new(&DICTIONARY_STREAMS); @@ -151,6 +162,15 @@ fn criterion_benchmark(c: &mut Criterion) { let case = SortBenchCasePreservePartitioning::new(&DICTIONARY_TUPLE_STREAMS); b.iter(move || case.run()) }); + c.bench_function( + "sort utf8 dictionary tuple preserve partitioning data skewed to first", + |b| { + let case = SortBenchCasePreservePartitioning::new_all_data_in_first_partition( + &DICTIONARY_TUPLE_STREAMS, + ); + b.iter(move || case.run()) + }, + ); c.bench_function("sort mixed utf8 dictionary tuple", |b| { let case = SortBenchCase::new(&MIXED_DICTIONARY_TUPLE_STREAMS); @@ -165,6 +185,15 @@ fn criterion_benchmark(c: &mut Criterion) { b.iter(move || case.run()) }, ); + c.bench_function( + "sort mixed utf8 dictionary tuple preserve partitioning data skewed to first", + |b| { + let case = SortBenchCasePreservePartitioning::new_all_data_in_first_partition( + &MIXED_DICTIONARY_TUPLE_STREAMS, + ); + b.iter(move || case.run()) + }, + ); c.bench_function("sort mixed tuple", |b| { let case = SortBenchCase::new(&MIXED_TUPLE_STREAMS); @@ -176,6 +205,16 @@ fn criterion_benchmark(c: &mut Criterion) { b.iter(move || case.run()) }); + c.bench_function( + "sort mixed tuple preserve partitioning data skewed to first", + |b| { + let case = SortBenchCasePreservePartitioning::new_all_data_in_first_partition( + &MIXED_TUPLE_STREAMS, + ); + + b.iter(move || case.run()) + }, + ); } /// Encapsulates running a test case where input partitioning is not preserved. @@ -264,6 +303,22 @@ impl SortBenchCasePreservePartitioning { partition_count, } } + /// This case preserves partitioning but all the data is in the first partition. + fn new_all_data_in_first_partition(partitions: &[Vec]) -> Self { + // modify partitions to move all the data into the first partition. + // still keeping same number of total partitions. + let partitions = (0..partitions.len()) + .map(|i| { + if i == 0 { + partitions.iter().cloned().flatten().collect::>() + } else { + vec![] + } + }) + .collect::>(); + // rest of bench case is the exact same + Self::new(&partitions) + } /// runs the specified plan to completion, draining all input and /// panic'ing on error From 279c6f5c3b739e47838a6809c3f5054f9fde4a50 Mon Sep 17 00:00:00 2001 From: Jay Miller <3744812+jaylmiller@users.noreply.github.com> Date: Tue, 21 Feb 2023 09:34:48 -0500 Subject: [PATCH 24/29] clippy err --- datafusion/core/benches/sort.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/core/benches/sort.rs b/datafusion/core/benches/sort.rs index 39e3bd44669d..5cb057430f56 100644 --- a/datafusion/core/benches/sort.rs +++ b/datafusion/core/benches/sort.rs @@ -310,7 +310,7 @@ impl SortBenchCasePreservePartitioning { let partitions = (0..partitions.len()) .map(|i| { if i == 0 { - partitions.iter().cloned().flatten().collect::>() + partitions.iter().flatten().cloned().collect::>() } else { vec![] } From b82545e9ab430710ba90743d29247fbcddab2bda Mon Sep 17 00:00:00 2001 From: Jay Miller <3744812+jaylmiller@users.noreply.github.com> Date: Sat, 4 Mar 2023 17:35:43 -0500 Subject: [PATCH 25/29] use mergesort in the merge step of sort exec --- datafusion/core/src/physical_plan/sorts/sort.rs | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/datafusion/core/src/physical_plan/sorts/sort.rs b/datafusion/core/src/physical_plan/sorts/sort.rs index e99b90ea617c..f5a8f374e029 100644 --- a/datafusion/core/src/physical_plan/sorts/sort.rs +++ b/datafusion/core/src/physical_plan/sorts/sort.rs @@ -440,7 +440,10 @@ fn get_sorted_iter( .flat_map(|r| r.iter()) .enumerate() .collect::>(); - to_sort.sort_unstable_by(|(_, row_a), (_, row_b)| row_a.cmp(row_b)); + // NB: according to the rust docs, `sort` is a mergesort (while + // `sort_unstable` is quicksort.) so right here, `sort` should be faster + // since we are sorting a bunch of concatenated sorted sequences. + to_sort.sort_by(|(_, row_a), (_, row_b)| row_a.cmp(row_b)); let limit = match fetch { Some(lim) => lim.min(to_sort.len()), None => to_sort.len(), From c685a0d9d6d058fa297e151bbc6a70778be63e20 Mon Sep 17 00:00:00 2001 From: Jay Miller <3744812+jaylmiller@users.noreply.github.com> Date: Sat, 4 Mar 2023 17:39:46 -0500 Subject: [PATCH 26/29] remove experimental bench cases --- datafusion/core/benches/sort.rs | 55 --------------------------------- 1 file changed, 55 deletions(-) diff --git a/datafusion/core/benches/sort.rs b/datafusion/core/benches/sort.rs index 5cb057430f56..0507a9308a28 100644 --- a/datafusion/core/benches/sort.rs +++ b/datafusion/core/benches/sort.rs @@ -132,17 +132,6 @@ fn criterion_benchmark(c: &mut Criterion) { b.iter(move || case.run()) }); - c.bench_function( - "sort utf8 tuple preserve partitioning data skewed to first", - |b| { - let case = SortBenchCasePreservePartitioning::new_all_data_in_first_partition( - &UTF8_TUPLE_STREAMS, - ); - - b.iter(move || case.run()) - }, - ); - c.bench_function("sort utf8 dictionary", |b| { let case = SortBenchCase::new(&DICTIONARY_STREAMS); @@ -162,15 +151,6 @@ fn criterion_benchmark(c: &mut Criterion) { let case = SortBenchCasePreservePartitioning::new(&DICTIONARY_TUPLE_STREAMS); b.iter(move || case.run()) }); - c.bench_function( - "sort utf8 dictionary tuple preserve partitioning data skewed to first", - |b| { - let case = SortBenchCasePreservePartitioning::new_all_data_in_first_partition( - &DICTIONARY_TUPLE_STREAMS, - ); - b.iter(move || case.run()) - }, - ); c.bench_function("sort mixed utf8 dictionary tuple", |b| { let case = SortBenchCase::new(&MIXED_DICTIONARY_TUPLE_STREAMS); @@ -185,15 +165,6 @@ fn criterion_benchmark(c: &mut Criterion) { b.iter(move || case.run()) }, ); - c.bench_function( - "sort mixed utf8 dictionary tuple preserve partitioning data skewed to first", - |b| { - let case = SortBenchCasePreservePartitioning::new_all_data_in_first_partition( - &MIXED_DICTIONARY_TUPLE_STREAMS, - ); - b.iter(move || case.run()) - }, - ); c.bench_function("sort mixed tuple", |b| { let case = SortBenchCase::new(&MIXED_TUPLE_STREAMS); @@ -205,16 +176,6 @@ fn criterion_benchmark(c: &mut Criterion) { b.iter(move || case.run()) }); - c.bench_function( - "sort mixed tuple preserve partitioning data skewed to first", - |b| { - let case = SortBenchCasePreservePartitioning::new_all_data_in_first_partition( - &MIXED_TUPLE_STREAMS, - ); - - b.iter(move || case.run()) - }, - ); } /// Encapsulates running a test case where input partitioning is not preserved. @@ -303,22 +264,6 @@ impl SortBenchCasePreservePartitioning { partition_count, } } - /// This case preserves partitioning but all the data is in the first partition. - fn new_all_data_in_first_partition(partitions: &[Vec]) -> Self { - // modify partitions to move all the data into the first partition. - // still keeping same number of total partitions. - let partitions = (0..partitions.len()) - .map(|i| { - if i == 0 { - partitions.iter().flatten().cloned().collect::>() - } else { - vec![] - } - }) - .collect::>(); - // rest of bench case is the exact same - Self::new(&partitions) - } /// runs the specified plan to completion, draining all input and /// panic'ing on error From 08b3fe58c03f60af107351062c9b0b8e4ad3cf00 Mon Sep 17 00:00:00 2001 From: Jay Miller <3744812+jaylmiller@users.noreply.github.com> Date: Sat, 4 Mar 2023 17:52:31 -0500 Subject: [PATCH 27/29] move gating logic outside of sort_batch --- .../core/src/physical_plan/sorts/sort.rs | 18 +++++++++++++++--- 1 file changed, 15 insertions(+), 3 deletions(-) diff --git a/datafusion/core/src/physical_plan/sorts/sort.rs b/datafusion/core/src/physical_plan/sorts/sort.rs index f5a8f374e029..2eaefa696b53 100644 --- a/datafusion/core/src/physical_plan/sorts/sort.rs +++ b/datafusion/core/src/physical_plan/sorts/sort.rs @@ -87,6 +87,7 @@ struct ExternalSorter { fetch: Option, reservation: MemoryReservation, partition_id: usize, + use_row_encoding: bool, // if this flag is true, the output of the sort will // have non-None `RowBatch` preserve_output_rows: bool, @@ -125,12 +126,16 @@ impl ExternalSorter { reservation, partition_id, preserve_output_rows: false, + use_row_encoding: false, } } pub fn set_preserve_output_rows(&mut self, val: bool) { self.preserve_output_rows = val; } + fn set_use_row_encoding(&mut self, val: bool) { + self.use_row_encoding = val; + } async fn insert_batch( &mut self, @@ -148,7 +153,13 @@ impl ExternalSorter { // NB timer records time taken on drop, so there are no // calls to `timer.done()` below. let _timer = tracking_metrics.elapsed_compute().timer(); - let partial = sort_batch(input, self.schema.clone(), &self.expr, self.fetch)?; + let partial = sort_batch( + input, + self.schema.clone(), + &self.expr, + self.fetch, + self.use_row_encoding, + )?; // The resulting batch might be smaller (or larger, see #3747) than the input // batch due to either a propagated limit or the re-construction of arrays. So @@ -1067,14 +1078,15 @@ fn sort_batch( schema: SchemaRef, expr: &[PhysicalSortExpr], fetch: Option, + use_row_encoding: bool, ) -> Result { let sort_columns = expr .iter() .map(|e| e.evaluate_to_sort_column(&batch)) .collect::>>()?; - let (indices, rows) = match (sort_columns.len(), fetch) { + let (indices, rows) = match use_row_encoding { // if single column or there's a limit, fallback to regular sort - (1, None) | (_, Some(_)) => (lexsort_to_indices(&sort_columns, fetch)?, None), + false => (lexsort_to_indices(&sort_columns, fetch)?, None), _ => { let sort_fields = sort_columns .iter() From 790546f8e25eced69e81e7186d7655cd5a46e27a Mon Sep 17 00:00:00 2001 From: Jay Miller <3744812+jaylmiller@users.noreply.github.com> Date: Sat, 4 Mar 2023 18:27:24 -0500 Subject: [PATCH 28/29] dont use row encoding on single batch code path --- .../core/src/physical_plan/sorts/sort.rs | 22 +++++++++++++++++++ 1 file changed, 22 insertions(+) diff --git a/datafusion/core/src/physical_plan/sorts/sort.rs b/datafusion/core/src/physical_plan/sorts/sort.rs index 2eaefa696b53..9668abd2627a 100644 --- a/datafusion/core/src/physical_plan/sorts/sort.rs +++ b/datafusion/core/src/physical_plan/sorts/sort.rs @@ -1169,6 +1169,7 @@ async fn do_sort( context.session_id(), context.task_id() ); + let n_sort_cols = expr.len(); let schema = input.schema(); let tracking_metrics = metrics_set.new_intermediate_tracking(partition_id, context.memory_pool()); @@ -1184,8 +1185,29 @@ async fn do_sort( if preserve_rows { sorter.set_preserve_output_rows(true); } + sorter.set_use_row_encoding(match (n_sort_cols, fetch) { + // if single column or there's a limit, fallback to regular sort + (1, None) | (_, Some(_)) => false, + _ => true, + }); + // single batch case has a different code path (no merging required) and + // row encoding seems to hurt performance on that path. + // dont immediately insert first batch + // if its the only batch seen, then we dont use row encoding + let mut first_batch = None as Option; while let Some(batch) = input.next().await { let batch = batch?; + // sorter.insert_batch(batch, &tracking_metrics).await?; + if first_batch.is_none() { + first_batch = Some(batch); + } else { + sorter.insert_batch(batch, &tracking_metrics).await?; + } + } + if let Some(batch) = first_batch { + if sorter.in_mem_batches.is_empty() && !sorter.spilled_before() { + sorter.set_use_row_encoding(false); + } sorter.insert_batch(batch, &tracking_metrics).await?; } let result = sorter.sort(); From d13912bc6876eeea2106be18655024e771c12596 Mon Sep 17 00:00:00 2001 From: Jay Miller <3744812+jaylmiller@users.noreply.github.com> Date: Mon, 6 Mar 2023 09:54:15 -0500 Subject: [PATCH 29/29] batch insertion order fix --- .../core/src/physical_plan/sorts/sort.rs | 54 +++++++++++++------ 1 file changed, 38 insertions(+), 16 deletions(-) diff --git a/datafusion/core/src/physical_plan/sorts/sort.rs b/datafusion/core/src/physical_plan/sorts/sort.rs index 9668abd2627a..05f6331187e3 100644 --- a/datafusion/core/src/physical_plan/sorts/sort.rs +++ b/datafusion/core/src/physical_plan/sorts/sort.rs @@ -1190,25 +1190,47 @@ async fn do_sort( (1, None) | (_, Some(_)) => false, _ => true, }); - // single batch case has a different code path (no merging required) and - // row encoding seems to hurt performance on that path. - // dont immediately insert first batch - // if its the only batch seen, then we dont use row encoding - let mut first_batch = None as Option; - while let Some(batch) = input.next().await { - let batch = batch?; - // sorter.insert_batch(batch, &tracking_metrics).await?; - if first_batch.is_none() { - first_batch = Some(batch); - } else { - sorter.insert_batch(batch, &tracking_metrics).await?; + if sorter.use_row_encoding { + // wait til more than 1 batch is seen before inserting the first batch + // (still maintains the order, just inserts first 2 batches together). + // then if theres only a single batch to sort, we dont use row encoding + let mut first_batch = Vec::with_capacity(1); + let mut inserted_first = false; + while let Some(batch) = input.next().await { + let batch = batch?; + match (inserted_first, first_batch.is_empty()) { + (false, true) => { + first_batch.push(batch); + } + (false, false) => { + // maintain batch insertion order + sorter + .insert_batch(first_batch.pop().unwrap(), &tracking_metrics) + .await?; + sorter.insert_batch(batch, &tracking_metrics).await?; + inserted_first = true; + } + (true, true) => { + sorter.insert_batch(batch, &tracking_metrics).await?; + } + (true, false) => { + unreachable!() + } + } } - } - if let Some(batch) = first_batch { - if sorter.in_mem_batches.is_empty() && !sorter.spilled_before() { + if !inserted_first && !first_batch.is_empty() { + // only one batch was inserted, dont use row encoding sorter.set_use_row_encoding(false); + assert!(!sorter.spilled_before()); + assert!(sorter.in_mem_batches.is_empty()); + let batch = first_batch.pop().unwrap(); + sorter.insert_batch(batch, &tracking_metrics).await?; + } + } else { + while let Some(batch) = input.next().await { + let batch = batch?; + sorter.insert_batch(batch, &tracking_metrics).await?; } - sorter.insert_batch(batch, &tracking_metrics).await?; } let result = sorter.sort();