-
Notifications
You must be signed in to change notification settings - Fork 1.2k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
perf: Improve materialisation performance of SortPreservingMergeExec #691
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -95,3 +95,7 @@ harness = false | |
[[bench]] | ||
name = "scalar" | ||
harness = false | ||
|
||
[[bench]] | ||
name = "physical_plan" | ||
harness = false |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,176 @@ | ||
// 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. | ||
|
||
#[macro_use] | ||
extern crate criterion; | ||
use criterion::{BatchSize, Criterion}; | ||
extern crate arrow; | ||
extern crate datafusion; | ||
|
||
use std::{iter::FromIterator, sync::Arc}; | ||
|
||
use arrow::{ | ||
array::{ArrayRef, Int64Array, StringArray}, | ||
record_batch::RecordBatch, | ||
}; | ||
use tokio::runtime::Runtime; | ||
|
||
use datafusion::physical_plan::{ | ||
collect, | ||
expressions::{col, PhysicalSortExpr}, | ||
memory::MemoryExec, | ||
sort_preserving_merge::SortPreservingMergeExec, | ||
}; | ||
|
||
// Initialise the operator using the provided record batches and the sort key | ||
// as inputs. All record batches must have the same schema. | ||
fn sort_preserving_merge_operator(batches: Vec<RecordBatch>, sort: &[&str]) { | ||
let schema = batches[0].schema(); | ||
|
||
let sort = sort | ||
.iter() | ||
.map(|name| PhysicalSortExpr { | ||
expr: col(name, &schema).unwrap(), | ||
options: Default::default(), | ||
}) | ||
.collect::<Vec<_>>(); | ||
|
||
let exec = MemoryExec::try_new( | ||
&batches.into_iter().map(|rb| vec![rb]).collect::<Vec<_>>(), | ||
schema, | ||
None, | ||
) | ||
.unwrap(); | ||
let merge = Arc::new(SortPreservingMergeExec::new(sort, Arc::new(exec), 8192)); | ||
|
||
let rt = Runtime::new().unwrap(); | ||
rt.block_on(collect(merge)).unwrap(); | ||
} | ||
|
||
// Produces `n` record batches of row size `m`. Each record batch will have | ||
// identical contents except for if the `batch_offset` is set. In that case the | ||
// values for column "d" in each subsequent record batch will be offset in | ||
// value. | ||
// | ||
// The `rows_per_key` value controls how many rows are generated per "key", | ||
// which is defined as columns a, b and c. | ||
fn batches( | ||
n: usize, | ||
m: usize, | ||
rows_per_sort_key: usize, | ||
batch_offset: usize, | ||
) -> Vec<RecordBatch> { | ||
let mut rbs = Vec::with_capacity(n); | ||
let mut curr_batch_offset = 0; | ||
|
||
for _ in 0..n { | ||
let mut col_a = Vec::with_capacity(m); | ||
let mut col_b = Vec::with_capacity(m); | ||
let mut col_c = Vec::with_capacity(m); | ||
let mut col_d = Vec::with_capacity(m); | ||
|
||
let mut j = 0; | ||
let mut current_rows_per_sort_key = 0; | ||
|
||
for i in 0..m { | ||
if current_rows_per_sort_key == rows_per_sort_key { | ||
current_rows_per_sort_key = 0; | ||
j = i; | ||
} | ||
|
||
col_a.push(Some(format!("a-{:?}", j))); | ||
col_b.push(Some(format!("b-{:?}", j))); | ||
col_c.push(Some(format!("c-{:?}", j))); | ||
col_d.push(Some((i + curr_batch_offset) as i64)); | ||
|
||
current_rows_per_sort_key += 1; | ||
} | ||
|
||
col_a.sort(); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This data is only sorted on There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I sorted the columns independently because it was a bit simpler than invoking a lex sort on them and technically the rows are still sorted by For example to create two inputs where the sort keys don't overlap the test data might look like: Input A:
Then using a
In this case the operator would take the first five rows from input A then have to alternate taking the next five rows from input A and input B (for rows where Then it would move back to input A for the row containing The idea here is that in |
||
col_b.sort(); | ||
col_c.sort(); | ||
|
||
let col_a: ArrayRef = Arc::new(StringArray::from_iter(col_a)); | ||
let col_b: ArrayRef = Arc::new(StringArray::from_iter(col_b)); | ||
let col_c: ArrayRef = Arc::new(StringArray::from_iter(col_c)); | ||
let col_d: ArrayRef = Arc::new(Int64Array::from(col_d)); | ||
|
||
let rb = RecordBatch::try_from_iter(vec![ | ||
("a", col_a), | ||
("b", col_b), | ||
("c", col_c), | ||
("d", col_d), | ||
]) | ||
.unwrap(); | ||
rbs.push(rb); | ||
|
||
curr_batch_offset += batch_offset; | ||
} | ||
|
||
rbs | ||
} | ||
|
||
fn criterion_benchmark(c: &mut Criterion) { | ||
let small_batch = batches(1, 100, 10, 0).remove(0); | ||
let large_batch = batches(1, 1000, 1, 0).remove(0); | ||
|
||
let benches = vec![ | ||
// Two batches with identical rows. They will need to be merged together | ||
// with one row from each batch being taken until both batches are | ||
// drained. | ||
("interleave_batches", batches(2, 1000, 10, 1)), | ||
// Two batches with a small overlapping region of rows for each unique | ||
// sort key. | ||
("merge_batches_some_overlap_small", batches(2, 1000, 10, 5)), | ||
// Two batches with a large overlapping region of rows for each unique | ||
// sort key. | ||
( | ||
"merge_batches_some_overlap_large", | ||
batches(2, 1000, 250, 125), | ||
), | ||
// Two batches with no overlapping region of rows for each unique | ||
// sort key. For a given unique sort key all rows are drained from one | ||
// batch, then all the rows for the same key from the second batch. | ||
// This repeats until all rows are drained. There are a small number of | ||
// rows (10) for each unique sort key. | ||
("merge_batches_no_overlap_small", batches(2, 1000, 10, 12)), | ||
// As above but this time there are a larger number of rows (250) for | ||
// each unique sort key - still no overlaps. | ||
("merge_batches_no_overlap_large", batches(2, 1000, 250, 252)), | ||
// Merges two batches where one batch is significantly larger than the | ||
// other. | ||
( | ||
"merge_batches_small_into_large", | ||
vec![large_batch, small_batch], | ||
), | ||
]; | ||
|
||
for (name, input) in benches { | ||
c.bench_function(name, move |b| { | ||
b.iter_batched( | ||
|| input.clone(), | ||
|input| { | ||
sort_preserving_merge_operator(input, &["a", "b", "c", "d"]); | ||
}, | ||
BatchSize::LargeInput, | ||
) | ||
}); | ||
} | ||
} | ||
|
||
criterion_group!(benches, criterion_benchmark); | ||
criterion_main!(benches); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Should this have the same amount of partitions as batches here?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
For a more realistic scenario, we should have some 8-100 partitions with many batches each
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Do you want me to add more benchmarks in this PR? Or alternatively I could extend the benchmarks if I find any other performance improvements in a future one.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The point I tried to make is that a test with as many partitions as batches is not a normal workload. So I would change the benchmark here to e.g. test something like 8 partitions with 100 batches each (or something like this).
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
But currently I am seeing only one or two batches in the benchmark? So for now it's not a big deal, this could be covered in a future PR.