Skip to content

Commit 024bd89

Browse files
authored
perf: Improve materialisation performance of SortPreservingMergeExec (#691)
* test: add benchmarks for SortPreservingMergeExec * perf: minimise array data extend calls The `SortPreservingMergeStream` operator merges two streams together by creating an output record batch that is build from the contents of the input. Previously each row of input would be pushed into the output sink even if though the API supports pushing batches of rows. This commit implements the logic to push batches of rows from inputs where possible. Performance benchmarks show an improvement of between 3-12%. ``` group master pr ----- ------ -- interleave_batches 1.04 637.5±51.84µs ? ?/sec 1.00 615.5±12.13µs ? ?/sec merge_batches_no_overlap_large 1.12 454.9±2.90µs ? ?/sec 1.00 404.9±10.94µs ? ?/sec merge_batches_no_overlap_small 1.14 485.1±6.67µs ? ?/sec 1.00 425.7±9.33µs ? ?/sec merge_batches_small_into_large 1.14 263.0±8.85µs ? ?/sec 1.00 229.7±5.23µs ? ?/sec merge_batches_some_overlap_large 1.05 532.5±8.33µs ? ?/sec 1.00 508.3±14.24µs ? ?/sec merge_batches_some_overlap_small 1.06 546.9±12.82µs ? ?/sec 1.00 516.9±13.20µs ? ?/sec ``` * test: more test coverage * refactor: update batch size
1 parent f94f639 commit 024bd89

File tree

3 files changed

+341
-41
lines changed

3 files changed

+341
-41
lines changed

datafusion/Cargo.toml

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -95,3 +95,7 @@ harness = false
9595
[[bench]]
9696
name = "scalar"
9797
harness = false
98+
99+
[[bench]]
100+
name = "physical_plan"
101+
harness = false
Lines changed: 176 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,176 @@
1+
// Licensed to the Apache Software Foundation (ASF) under one
2+
// or more contributor license agreements. See the NOTICE file
3+
// distributed with this work for additional information
4+
// regarding copyright ownership. The ASF licenses this file
5+
// to you under the Apache License, Version 2.0 (the
6+
// "License"); you may not use this file except in compliance
7+
// with the License. You may obtain a copy of the License at
8+
//
9+
// http://www.apache.org/licenses/LICENSE-2.0
10+
//
11+
// Unless required by applicable law or agreed to in writing,
12+
// software distributed under the License is distributed on an
13+
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
14+
// KIND, either express or implied. See the License for the
15+
// specific language governing permissions and limitations
16+
// under the License.
17+
18+
#[macro_use]
19+
extern crate criterion;
20+
use criterion::{BatchSize, Criterion};
21+
extern crate arrow;
22+
extern crate datafusion;
23+
24+
use std::{iter::FromIterator, sync::Arc};
25+
26+
use arrow::{
27+
array::{ArrayRef, Int64Array, StringArray},
28+
record_batch::RecordBatch,
29+
};
30+
use tokio::runtime::Runtime;
31+
32+
use datafusion::physical_plan::{
33+
collect,
34+
expressions::{col, PhysicalSortExpr},
35+
memory::MemoryExec,
36+
sort_preserving_merge::SortPreservingMergeExec,
37+
};
38+
39+
// Initialise the operator using the provided record batches and the sort key
40+
// as inputs. All record batches must have the same schema.
41+
fn sort_preserving_merge_operator(batches: Vec<RecordBatch>, sort: &[&str]) {
42+
let schema = batches[0].schema();
43+
44+
let sort = sort
45+
.iter()
46+
.map(|name| PhysicalSortExpr {
47+
expr: col(name, &schema).unwrap(),
48+
options: Default::default(),
49+
})
50+
.collect::<Vec<_>>();
51+
52+
let exec = MemoryExec::try_new(
53+
&batches.into_iter().map(|rb| vec![rb]).collect::<Vec<_>>(),
54+
schema,
55+
None,
56+
)
57+
.unwrap();
58+
let merge = Arc::new(SortPreservingMergeExec::new(sort, Arc::new(exec), 8192));
59+
60+
let rt = Runtime::new().unwrap();
61+
rt.block_on(collect(merge)).unwrap();
62+
}
63+
64+
// Produces `n` record batches of row size `m`. Each record batch will have
65+
// identical contents except for if the `batch_offset` is set. In that case the
66+
// values for column "d" in each subsequent record batch will be offset in
67+
// value.
68+
//
69+
// The `rows_per_key` value controls how many rows are generated per "key",
70+
// which is defined as columns a, b and c.
71+
fn batches(
72+
n: usize,
73+
m: usize,
74+
rows_per_sort_key: usize,
75+
batch_offset: usize,
76+
) -> Vec<RecordBatch> {
77+
let mut rbs = Vec::with_capacity(n);
78+
let mut curr_batch_offset = 0;
79+
80+
for _ in 0..n {
81+
let mut col_a = Vec::with_capacity(m);
82+
let mut col_b = Vec::with_capacity(m);
83+
let mut col_c = Vec::with_capacity(m);
84+
let mut col_d = Vec::with_capacity(m);
85+
86+
let mut j = 0;
87+
let mut current_rows_per_sort_key = 0;
88+
89+
for i in 0..m {
90+
if current_rows_per_sort_key == rows_per_sort_key {
91+
current_rows_per_sort_key = 0;
92+
j = i;
93+
}
94+
95+
col_a.push(Some(format!("a-{:?}", j)));
96+
col_b.push(Some(format!("b-{:?}", j)));
97+
col_c.push(Some(format!("c-{:?}", j)));
98+
col_d.push(Some((i + curr_batch_offset) as i64));
99+
100+
current_rows_per_sort_key += 1;
101+
}
102+
103+
col_a.sort();
104+
col_b.sort();
105+
col_c.sort();
106+
107+
let col_a: ArrayRef = Arc::new(StringArray::from_iter(col_a));
108+
let col_b: ArrayRef = Arc::new(StringArray::from_iter(col_b));
109+
let col_c: ArrayRef = Arc::new(StringArray::from_iter(col_c));
110+
let col_d: ArrayRef = Arc::new(Int64Array::from(col_d));
111+
112+
let rb = RecordBatch::try_from_iter(vec![
113+
("a", col_a),
114+
("b", col_b),
115+
("c", col_c),
116+
("d", col_d),
117+
])
118+
.unwrap();
119+
rbs.push(rb);
120+
121+
curr_batch_offset += batch_offset;
122+
}
123+
124+
rbs
125+
}
126+
127+
fn criterion_benchmark(c: &mut Criterion) {
128+
let small_batch = batches(1, 100, 10, 0).remove(0);
129+
let large_batch = batches(1, 1000, 1, 0).remove(0);
130+
131+
let benches = vec![
132+
// Two batches with identical rows. They will need to be merged together
133+
// with one row from each batch being taken until both batches are
134+
// drained.
135+
("interleave_batches", batches(2, 1000, 10, 1)),
136+
// Two batches with a small overlapping region of rows for each unique
137+
// sort key.
138+
("merge_batches_some_overlap_small", batches(2, 1000, 10, 5)),
139+
// Two batches with a large overlapping region of rows for each unique
140+
// sort key.
141+
(
142+
"merge_batches_some_overlap_large",
143+
batches(2, 1000, 250, 125),
144+
),
145+
// Two batches with no overlapping region of rows for each unique
146+
// sort key. For a given unique sort key all rows are drained from one
147+
// batch, then all the rows for the same key from the second batch.
148+
// This repeats until all rows are drained. There are a small number of
149+
// rows (10) for each unique sort key.
150+
("merge_batches_no_overlap_small", batches(2, 1000, 10, 12)),
151+
// As above but this time there are a larger number of rows (250) for
152+
// each unique sort key - still no overlaps.
153+
("merge_batches_no_overlap_large", batches(2, 1000, 250, 252)),
154+
// Merges two batches where one batch is significantly larger than the
155+
// other.
156+
(
157+
"merge_batches_small_into_large",
158+
vec![large_batch, small_batch],
159+
),
160+
];
161+
162+
for (name, input) in benches {
163+
c.bench_function(name, move |b| {
164+
b.iter_batched(
165+
|| input.clone(),
166+
|input| {
167+
sort_preserving_merge_operator(input, &["a", "b", "c", "d"]);
168+
},
169+
BatchSize::LargeInput,
170+
)
171+
});
172+
}
173+
}
174+
175+
criterion_group!(benches, criterion_benchmark);
176+
criterion_main!(benches);

0 commit comments

Comments
 (0)