Skip to content

Commit 99f475e

Browse files
jayzhan211bgjackma
authored andcommitted
Avoid RowConverter for multi column grouping (10% faster clickbench queries) (apache#12269)
* row like group values to avoid rowconverter Signed-off-by: jayzhan211 <[email protected]> * comment out unused Signed-off-by: jayzhan211 <[email protected]> * implement to Arrow's builder Signed-off-by: jayzhan211 <[email protected]> * cleanup Signed-off-by: jayzhan211 <[email protected]> * switch back to vector Signed-off-by: jayzhan211 <[email protected]> * clippy Signed-off-by: jayzhan211 <[email protected]> * optimize for non-null Signed-off-by: jayzhan211 <[email protected]> * use truncate Signed-off-by: jayzhan211 <[email protected]> * cleanup Signed-off-by: jayzhan211 <[email protected]> * cleanup Signed-off-by: jayzhan211 <[email protected]> * fix first N bug Signed-off-by: jayzhan211 <[email protected]> * fix null check Signed-off-by: jayzhan211 <[email protected]> * fast path null Signed-off-by: jayzhan211 <[email protected]> * fix bug Signed-off-by: jayzhan211 <[email protected]> * fmt Signed-off-by: jayzhan211 <[email protected]> * fix error Signed-off-by: jayzhan211 <[email protected]> * clippy Signed-off-by: jayzhan211 <[email protected]> * adjust spill mode max mem Signed-off-by: jayzhan211 <[email protected]> * revert test_create_external_table_with_terminator_with_newlines_in_values Signed-off-by: jayzhan211 <[email protected]> * fix null handle bug Signed-off-by: jayzhan211 <[email protected]> * cleanup Signed-off-by: jayzhan211 <[email protected]> * support binary Signed-off-by: jayzhan211 <[email protected]> * add binary test Signed-off-by: jayzhan211 <[email protected]> * use Vec<T> instead of Option<Vec<T>> Signed-off-by: jayzhan211 <[email protected]> * add test and doc Signed-off-by: jayzhan211 <[email protected]> * debug assert Signed-off-by: jayzhan211 <[email protected]> * mv & rename Signed-off-by: jayzhan211 <[email protected]> * fix take_n logic Signed-off-by: jayzhan211 <[email protected]> * address comment Signed-off-by: jayzhan211 <[email protected]> * cleanup Signed-off-by: jayzhan211 <[email protected]> --------- Signed-off-by: jayzhan211 <[email protected]>
1 parent 6f6fb17 commit 99f475e

File tree

7 files changed

+891
-3
lines changed

7 files changed

+891
-3
lines changed

datafusion/physical-expr-common/src/binary_map.rs

+1-1
Original file line numberDiff line numberDiff line change
@@ -237,7 +237,7 @@ where
237237
/// The size, in number of entries, of the initial hash table
238238
const INITIAL_MAP_CAPACITY: usize = 128;
239239
/// The initial size, in bytes, of the string data
240-
const INITIAL_BUFFER_CAPACITY: usize = 8 * 1024;
240+
pub const INITIAL_BUFFER_CAPACITY: usize = 8 * 1024;
241241
impl<O: OffsetSizeTrait, V> ArrowBytesMap<O, V>
242242
where
243243
V: Debug + PartialEq + Eq + Clone + Copy + Default,
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,314 @@
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+
use crate::aggregates::group_values::group_value_row::{
19+
ArrayRowEq, ByteGroupValueBuilder, PrimitiveGroupValueBuilder,
20+
};
21+
use crate::aggregates::group_values::GroupValues;
22+
use ahash::RandomState;
23+
use arrow::compute::cast;
24+
use arrow::datatypes::{
25+
Date32Type, Date64Type, Float32Type, Float64Type, Int16Type, Int32Type, Int64Type,
26+
Int8Type, UInt16Type, UInt32Type, UInt64Type, UInt8Type,
27+
};
28+
use arrow::record_batch::RecordBatch;
29+
use arrow_array::{Array, ArrayRef};
30+
use arrow_schema::{DataType, SchemaRef};
31+
use datafusion_common::hash_utils::create_hashes;
32+
use datafusion_common::{DataFusionError, Result};
33+
use datafusion_execution::memory_pool::proxy::{RawTableAllocExt, VecAllocExt};
34+
use datafusion_expr::EmitTo;
35+
use datafusion_physical_expr::binary_map::OutputType;
36+
37+
use hashbrown::raw::RawTable;
38+
39+
/// Compare GroupValue Rows column by column
40+
pub struct GroupValuesColumn {
41+
/// The output schema
42+
schema: SchemaRef,
43+
44+
/// Logically maps group values to a group_index in
45+
/// [`Self::group_values`] and in each accumulator
46+
///
47+
/// Uses the raw API of hashbrown to avoid actually storing the
48+
/// keys (group values) in the table
49+
///
50+
/// keys: u64 hashes of the GroupValue
51+
/// values: (hash, group_index)
52+
map: RawTable<(u64, usize)>,
53+
54+
/// The size of `map` in bytes
55+
map_size: usize,
56+
57+
/// The actual group by values, stored column-wise. Compare from
58+
/// the left to right, each column is stored as `ArrayRowEq`.
59+
/// This is shown faster than the row format
60+
group_values: Vec<Box<dyn ArrayRowEq>>,
61+
62+
/// reused buffer to store hashes
63+
hashes_buffer: Vec<u64>,
64+
65+
/// Random state for creating hashes
66+
random_state: RandomState,
67+
}
68+
69+
impl GroupValuesColumn {
70+
pub fn try_new(schema: SchemaRef) -> Result<Self> {
71+
let map = RawTable::with_capacity(0);
72+
Ok(Self {
73+
schema,
74+
map,
75+
map_size: 0,
76+
group_values: vec![],
77+
hashes_buffer: Default::default(),
78+
random_state: Default::default(),
79+
})
80+
}
81+
}
82+
83+
impl GroupValues for GroupValuesColumn {
84+
fn intern(&mut self, cols: &[ArrayRef], groups: &mut Vec<usize>) -> Result<()> {
85+
let n_rows = cols[0].len();
86+
87+
if self.group_values.is_empty() {
88+
let mut v = Vec::with_capacity(cols.len());
89+
90+
for f in self.schema.fields().iter() {
91+
let nullable = f.is_nullable();
92+
match f.data_type() {
93+
&DataType::Int8 => {
94+
let b = PrimitiveGroupValueBuilder::<Int8Type>::new(nullable);
95+
v.push(Box::new(b) as _)
96+
}
97+
&DataType::Int16 => {
98+
let b = PrimitiveGroupValueBuilder::<Int16Type>::new(nullable);
99+
v.push(Box::new(b) as _)
100+
}
101+
&DataType::Int32 => {
102+
let b = PrimitiveGroupValueBuilder::<Int32Type>::new(nullable);
103+
v.push(Box::new(b) as _)
104+
}
105+
&DataType::Int64 => {
106+
let b = PrimitiveGroupValueBuilder::<Int64Type>::new(nullable);
107+
v.push(Box::new(b) as _)
108+
}
109+
&DataType::UInt8 => {
110+
let b = PrimitiveGroupValueBuilder::<UInt8Type>::new(nullable);
111+
v.push(Box::new(b) as _)
112+
}
113+
&DataType::UInt16 => {
114+
let b = PrimitiveGroupValueBuilder::<UInt16Type>::new(nullable);
115+
v.push(Box::new(b) as _)
116+
}
117+
&DataType::UInt32 => {
118+
let b = PrimitiveGroupValueBuilder::<UInt32Type>::new(nullable);
119+
v.push(Box::new(b) as _)
120+
}
121+
&DataType::UInt64 => {
122+
let b = PrimitiveGroupValueBuilder::<UInt64Type>::new(nullable);
123+
v.push(Box::new(b) as _)
124+
}
125+
&DataType::Float32 => {
126+
let b = PrimitiveGroupValueBuilder::<Float32Type>::new(nullable);
127+
v.push(Box::new(b) as _)
128+
}
129+
&DataType::Float64 => {
130+
let b = PrimitiveGroupValueBuilder::<Float64Type>::new(nullable);
131+
v.push(Box::new(b) as _)
132+
}
133+
&DataType::Date32 => {
134+
let b = PrimitiveGroupValueBuilder::<Date32Type>::new(nullable);
135+
v.push(Box::new(b) as _)
136+
}
137+
&DataType::Date64 => {
138+
let b = PrimitiveGroupValueBuilder::<Date64Type>::new(nullable);
139+
v.push(Box::new(b) as _)
140+
}
141+
&DataType::Utf8 => {
142+
let b = ByteGroupValueBuilder::<i32>::new(OutputType::Utf8);
143+
v.push(Box::new(b) as _)
144+
}
145+
&DataType::LargeUtf8 => {
146+
let b = ByteGroupValueBuilder::<i64>::new(OutputType::Utf8);
147+
v.push(Box::new(b) as _)
148+
}
149+
&DataType::Binary => {
150+
let b = ByteGroupValueBuilder::<i32>::new(OutputType::Binary);
151+
v.push(Box::new(b) as _)
152+
}
153+
&DataType::LargeBinary => {
154+
let b = ByteGroupValueBuilder::<i64>::new(OutputType::Binary);
155+
v.push(Box::new(b) as _)
156+
}
157+
dt => todo!("{dt} not impl"),
158+
}
159+
}
160+
self.group_values = v;
161+
}
162+
163+
// tracks to which group each of the input rows belongs
164+
groups.clear();
165+
166+
// 1.1 Calculate the group keys for the group values
167+
let batch_hashes = &mut self.hashes_buffer;
168+
batch_hashes.clear();
169+
batch_hashes.resize(n_rows, 0);
170+
create_hashes(cols, &self.random_state, batch_hashes)?;
171+
172+
for (row, &target_hash) in batch_hashes.iter().enumerate() {
173+
let entry = self.map.get_mut(target_hash, |(exist_hash, group_idx)| {
174+
// Somewhat surprisingly, this closure can be called even if the
175+
// hash doesn't match, so check the hash first with an integer
176+
// comparison first avoid the more expensive comparison with
177+
// group value. https://github.com/apache/datafusion/pull/11718
178+
if target_hash != *exist_hash {
179+
return false;
180+
}
181+
182+
fn check_row_equal(
183+
array_row: &dyn ArrayRowEq,
184+
lhs_row: usize,
185+
array: &ArrayRef,
186+
rhs_row: usize,
187+
) -> bool {
188+
array_row.equal_to(lhs_row, array, rhs_row)
189+
}
190+
191+
for (i, group_val) in self.group_values.iter().enumerate() {
192+
if !check_row_equal(group_val.as_ref(), *group_idx, &cols[i], row) {
193+
return false;
194+
}
195+
}
196+
197+
true
198+
});
199+
200+
let group_idx = match entry {
201+
// Existing group_index for this group value
202+
Some((_hash, group_idx)) => *group_idx,
203+
// 1.2 Need to create new entry for the group
204+
None => {
205+
// Add new entry to aggr_state and save newly created index
206+
// let group_idx = group_values.num_rows();
207+
// group_values.push(group_rows.row(row));
208+
209+
let mut checklen = 0;
210+
let group_idx = self.group_values[0].len();
211+
for (i, group_value) in self.group_values.iter_mut().enumerate() {
212+
group_value.append_val(&cols[i], row);
213+
let len = group_value.len();
214+
if i == 0 {
215+
checklen = len;
216+
} else {
217+
debug_assert_eq!(checklen, len);
218+
}
219+
}
220+
221+
// for hasher function, use precomputed hash value
222+
self.map.insert_accounted(
223+
(target_hash, group_idx),
224+
|(hash, _group_index)| *hash,
225+
&mut self.map_size,
226+
);
227+
group_idx
228+
}
229+
};
230+
groups.push(group_idx);
231+
}
232+
233+
Ok(())
234+
}
235+
236+
fn size(&self) -> usize {
237+
let group_values_size: usize = self.group_values.iter().map(|v| v.size()).sum();
238+
group_values_size + self.map_size + self.hashes_buffer.allocated_size()
239+
}
240+
241+
fn is_empty(&self) -> bool {
242+
self.len() == 0
243+
}
244+
245+
fn len(&self) -> usize {
246+
if self.group_values.is_empty() {
247+
return 0;
248+
}
249+
250+
self.group_values[0].len()
251+
}
252+
253+
fn emit(&mut self, emit_to: EmitTo) -> Result<Vec<ArrayRef>> {
254+
let mut output = match emit_to {
255+
EmitTo::All => {
256+
let group_values = std::mem::take(&mut self.group_values);
257+
debug_assert!(self.group_values.is_empty());
258+
259+
group_values
260+
.into_iter()
261+
.map(|v| v.build())
262+
.collect::<Vec<_>>()
263+
}
264+
EmitTo::First(n) => {
265+
let output = self
266+
.group_values
267+
.iter_mut()
268+
.map(|v| v.take_n(n))
269+
.collect::<Vec<_>>();
270+
271+
// SAFETY: self.map outlives iterator and is not modified concurrently
272+
unsafe {
273+
for bucket in self.map.iter() {
274+
// Decrement group index by n
275+
match bucket.as_ref().1.checked_sub(n) {
276+
// Group index was >= n, shift value down
277+
Some(sub) => bucket.as_mut().1 = sub,
278+
// Group index was < n, so remove from table
279+
None => self.map.erase(bucket),
280+
}
281+
}
282+
}
283+
284+
output
285+
}
286+
};
287+
288+
// TODO: Materialize dictionaries in group keys (#7647)
289+
for (field, array) in self.schema.fields.iter().zip(&mut output) {
290+
let expected = field.data_type();
291+
if let DataType::Dictionary(_, v) = expected {
292+
let actual = array.data_type();
293+
if v.as_ref() != actual {
294+
return Err(DataFusionError::Internal(format!(
295+
"Converted group rows expected dictionary of {v} got {actual}"
296+
)));
297+
}
298+
*array = cast(array.as_ref(), expected)?;
299+
}
300+
}
301+
302+
Ok(output)
303+
}
304+
305+
fn clear_shrink(&mut self, batch: &RecordBatch) {
306+
let count = batch.num_rows();
307+
self.group_values.clear();
308+
self.map.clear();
309+
self.map.shrink_to(count, |_| 0); // hasher does not matter since the map is cleared
310+
self.map_size = self.map.capacity() * std::mem::size_of::<(u64, usize)>();
311+
self.hashes_buffer.clear();
312+
self.hashes_buffer.shrink_to(count);
313+
}
314+
}

0 commit comments

Comments
 (0)