From 37d1382b52a3c7b212e2adcd8914988950ba147c Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Sat, 31 Aug 2024 11:56:18 +0800 Subject: [PATCH 01/30] row like group values to avoid rowconverter Signed-off-by: jayzhan211 --- benchmarks/queries/clickbench/queries.sql | 7 +- .../physical-expr-common/src/binary_map.rs | 6 +- .../src/group_value_row.rs | 644 ++++++++++++++++++ datafusion/physical-expr-common/src/lib.rs | 1 + .../src/aggregates/group_values/mod.rs | 29 +- .../src/aggregates/group_values/row_like.rs | 398 +++++++++++ .../physical-plan/src/aggregates/row_hash.rs | 1 + datafusion/sqllogictest/test_files/test1.slt | 33 + datafusion/sqllogictest/test_files/test2.slt | 9 + 9 files changed, 1118 insertions(+), 10 deletions(-) create mode 100644 datafusion/physical-expr-common/src/group_value_row.rs create mode 100644 datafusion/physical-plan/src/aggregates/group_values/row_like.rs create mode 100644 datafusion/sqllogictest/test_files/test1.slt create mode 100644 datafusion/sqllogictest/test_files/test2.slt diff --git a/benchmarks/queries/clickbench/queries.sql b/benchmarks/queries/clickbench/queries.sql index 52e72e02e1e0..7ae48828de14 100644 --- a/benchmarks/queries/clickbench/queries.sql +++ b/benchmarks/queries/clickbench/queries.sql @@ -35,9 +35,4 @@ SELECT "URL", COUNT(*) AS c FROM hits GROUP BY "URL" ORDER BY c DESC LIMIT 10; SELECT 1, "URL", COUNT(*) AS c FROM hits GROUP BY 1, "URL" ORDER BY c DESC LIMIT 10; SELECT "ClientIP", "ClientIP" - 1, "ClientIP" - 2, "ClientIP" - 3, COUNT(*) AS c FROM hits GROUP BY "ClientIP", "ClientIP" - 1, "ClientIP" - 2, "ClientIP" - 3 ORDER BY c DESC LIMIT 10; SELECT "URL", COUNT(*) AS PageViews FROM hits WHERE "CounterID" = 62 AND "EventDate"::INT::DATE >= '2013-07-01' AND "EventDate"::INT::DATE <= '2013-07-31' AND "DontCountHits" = 0 AND "IsRefresh" = 0 AND "URL" <> '' GROUP BY "URL" ORDER BY PageViews DESC LIMIT 10; -SELECT "Title", COUNT(*) AS PageViews FROM hits WHERE "CounterID" = 62 AND "EventDate"::INT::DATE >= '2013-07-01' AND "EventDate"::INT::DATE <= '2013-07-31' AND "DontCountHits" = 0 AND "IsRefresh" = 0 AND "Title" <> '' GROUP BY "Title" ORDER BY PageViews DESC LIMIT 10; -SELECT "URL", COUNT(*) AS PageViews FROM hits WHERE "CounterID" = 62 AND "EventDate"::INT::DATE >= '2013-07-01' AND "EventDate"::INT::DATE <= '2013-07-31' AND "IsRefresh" = 0 AND "IsLink" <> 0 AND "IsDownload" = 0 GROUP BY "URL" ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; -SELECT "TraficSourceID", "SearchEngineID", "AdvEngineID", CASE WHEN ("SearchEngineID" = 0 AND "AdvEngineID" = 0) THEN "Referer" ELSE '' END AS Src, "URL" AS Dst, COUNT(*) AS PageViews FROM hits WHERE "CounterID" = 62 AND "EventDate"::INT::DATE >= '2013-07-01' AND "EventDate"::INT::DATE <= '2013-07-31' AND "IsRefresh" = 0 GROUP BY "TraficSourceID", "SearchEngineID", "AdvEngineID", Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; -SELECT "URLHash", "EventDate"::INT::DATE, COUNT(*) AS PageViews FROM hits WHERE "CounterID" = 62 AND "EventDate"::INT::DATE >= '2013-07-01' AND "EventDate"::INT::DATE <= '2013-07-31' AND "IsRefresh" = 0 AND "TraficSourceID" IN (-1, 6) AND "RefererHash" = 3594120000172545465 GROUP BY "URLHash", "EventDate"::INT::DATE ORDER BY PageViews DESC LIMIT 10 OFFSET 100; -SELECT "WindowClientWidth", "WindowClientHeight", COUNT(*) AS PageViews FROM hits WHERE "CounterID" = 62 AND "EventDate"::INT::DATE >= '2013-07-01' AND "EventDate"::INT::DATE <= '2013-07-31' AND "IsRefresh" = 0 AND "DontCountHits" = 0 AND "URLHash" = 2868770270353813622 GROUP BY "WindowClientWidth", "WindowClientHeight" ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; -SELECT DATE_TRUNC('minute', to_timestamp_seconds("EventTime")) AS M, COUNT(*) AS PageViews FROM hits WHERE "CounterID" = 62 AND "EventDate"::INT::DATE >= '2013-07-14' AND "EventDate"::INT::DATE <= '2013-07-15' AND "IsRefresh" = 0 AND "DontCountHits" = 0 GROUP BY DATE_TRUNC('minute', to_timestamp_seconds("EventTime")) ORDER BY DATE_TRUNC('minute', M) LIMIT 10 OFFSET 1000; +SELECT "Title", COUNT(*) AS PageViews FROM hits WHERE "CounterID" = 62 AND "EventDate"::INT::DATE >= '2013-07-01' AND "EventDate"::INT::DATE <= '2013-07-31' AND "DontCountHits" = 0 AND "IsRefresh" = 0 AND "Title" <> '' GROUP BY "Title" ORDER BY PageViews DESC LIMIT 10; \ No newline at end of file diff --git a/datafusion/physical-expr-common/src/binary_map.rs b/datafusion/physical-expr-common/src/binary_map.rs index d21bdb3434c4..651937b4f06b 100644 --- a/datafusion/physical-expr-common/src/binary_map.rs +++ b/datafusion/physical-expr-common/src/binary_map.rs @@ -235,9 +235,9 @@ where } /// The size, in number of entries, of the initial hash table -const INITIAL_MAP_CAPACITY: usize = 128; +pub(crate) const INITIAL_MAP_CAPACITY: usize = 128; /// The initial size, in bytes, of the string data -const INITIAL_BUFFER_CAPACITY: usize = 8 * 1024; +pub(crate) const INITIAL_BUFFER_CAPACITY: usize = 8 * 1024; impl ArrowBytesMap where V: Debug + PartialEq + Eq + Clone + Copy + Default, @@ -574,7 +574,7 @@ where } /// Maximum size of a value that can be inlined in the hash table -const SHORT_VALUE_LEN: usize = mem::size_of::(); +pub(crate) const SHORT_VALUE_LEN: usize = mem::size_of::(); /// Entry in the hash table -- see [`ArrowBytesMap`] for more details #[derive(Debug, PartialEq, Eq, Hash, Clone, Copy)] diff --git a/datafusion/physical-expr-common/src/group_value_row.rs b/datafusion/physical-expr-common/src/group_value_row.rs new file mode 100644 index 000000000000..1d5aaa81e62d --- /dev/null +++ b/datafusion/physical-expr-common/src/group_value_row.rs @@ -0,0 +1,644 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use ahash::RandomState; +use arrow::array::BooleanBufferBuilder; +use arrow::array::BufferBuilder; +use arrow::array::GenericBinaryArray; +use arrow::array::GenericStringArray; +use arrow::array::OffsetSizeTrait; +use arrow::array::StringArray; +use arrow::buffer::NullBuffer; +use arrow::buffer::OffsetBuffer; +use arrow::buffer::ScalarBuffer; +use arrow::datatypes::ArrowNativeType; +use arrow::datatypes::DataType; +use arrow::datatypes::GenericBinaryType; +use arrow::datatypes::GenericStringType; +use arrow::{ + array::{Array, ArrayRef, ArrowPrimitiveType, AsArray, PrimitiveArray}, + datatypes::ByteArrayType, +}; +use datafusion_common::hash_utils::create_hashes; +use datafusion_common::utils::proxy::RawTableAllocExt; + +use crate::binary_map::OutputType; +use crate::binary_map::{INITIAL_BUFFER_CAPACITY, INITIAL_MAP_CAPACITY, SHORT_VALUE_LEN}; + +use std::{fmt::Debug, ops::Range, sync::Arc}; + +pub trait ArrayEq: Send + Sync { + fn equal_to(&self, lhs_row: usize, array: &ArrayRef, rhs_row: usize) -> bool; + fn append_val(&mut self, array: &ArrayRef, row: usize); + fn len(&self) -> usize; + fn build(self: Box) -> ArrayRef; +} + +pub struct PrimitiveGroupValueBuilder(Vec>); + +impl PrimitiveGroupValueBuilder { + pub fn new() -> Self { + Self(vec![]) + } +} + +impl ArrayEq for PrimitiveGroupValueBuilder { + fn equal_to(&self, lhs_row: usize, array: &ArrayRef, rhs_row: usize) -> bool { + let elem = self.0[lhs_row]; + let arr = array.as_primitive::(); + let is_rhs_null = arr.is_null(rhs_row); + if elem.is_none() && is_rhs_null { + true + } else if elem.is_some() && !is_rhs_null { + elem.unwrap() == arr.value(rhs_row) + } else { + false + } + } + + fn append_val(&mut self, array: &ArrayRef, row: usize) { + let arr = array.as_primitive::(); + if arr.is_null(row) { + self.0.push(None) + } else { + let elem = arr.value(row); + self.0.push(Some(elem)) + } + } + + fn len(&self) -> usize { + self.0.len() + } + + fn build(self: Box) -> ArrayRef { + Arc::new(PrimitiveArray::::from_iter(self.0)) + } +} + +pub struct StringGroupValueBuilder(Vec>); + +impl StringGroupValueBuilder { + pub fn new() -> Self { + Self(vec![]) + } +} + +impl ArrayEq for StringGroupValueBuilder { + fn equal_to(&self, lhs_row: usize, array: &ArrayRef, rhs_row: usize) -> bool { + let elem = &self.0[lhs_row]; + let arr = array.as_string::(); + let is_rhs_null = arr.is_null(rhs_row); + if elem.is_none() && is_rhs_null { + true + } else if elem.is_some() && !is_rhs_null { + let e = elem.as_ref().unwrap(); + e.as_str() == arr.value(rhs_row) + } else { + false + } + } + + fn append_val(&mut self, array: &ArrayRef, row: usize) { + let arr = array.as_string::(); + if arr.is_null(row) { + self.0.push(None) + } else { + let elem = arr.value(row); + self.0.push(Some(elem.to_string())) + } + } + + fn len(&self) -> usize { + self.0.len() + } + + fn build(self: Box) -> ArrayRef { + Arc::new(StringArray::from_iter(self.0)) + } +} + +pub struct ByteGroupValueBuilderNaive +where + O: OffsetSizeTrait, +{ + output_type: OutputType, + buffer: BufferBuilder, + /// Offsets into `buffer` for each distinct value. These offsets as used + /// directly to create the final `GenericBinaryArray`. The `i`th string is + /// stored in the range `offsets[i]..offsets[i+1]` in `buffer`. Null values + /// are stored as a zero length string. + offsets: Vec, + /// Null indexes in offsets + nulls: Vec, +} + +impl ByteGroupValueBuilderNaive +where + O: OffsetSizeTrait, +{ + pub fn new(_array: &ArrayRef, output_type: OutputType) -> Self { + Self { + output_type, + buffer: BufferBuilder::new(INITIAL_BUFFER_CAPACITY), + offsets: vec![O::default()], + nulls: vec![], + } + } + + fn append_val_inner(&mut self, array: &ArrayRef, row: usize) + where + B: ByteArrayType, + { + let arr = array.as_bytes::(); + if arr.is_null(row) { + self.nulls.push(self.len()); + // nulls need a zero length in the offset buffer + let offset = self.buffer.len(); + self.offsets.push(O::usize_as(offset)); + return; + } + + let value: &[u8] = arr.value(row).as_ref(); + self.buffer.append_slice(value); + self.offsets.push(O::usize_as(self.buffer.len())); + } + + fn equal_to_inner(&self, lhs_row: usize, array: &ArrayRef, rhs_row: usize) -> bool + where + B: ByteArrayType, + { + // Handle nulls + let is_lhs_null = self.nulls.iter().any(|null_idx| *null_idx == lhs_row); + let arr = array.as_bytes::(); + if is_lhs_null { + return arr.is_null(rhs_row); + } else if arr.is_null(rhs_row) { + return false; + } + + let arr = array.as_bytes::(); + let rhs_elem: &[u8] = arr.value(rhs_row).as_ref(); + let rhs_elem_len = arr.value_length(rhs_row).as_usize(); + assert_eq!(rhs_elem_len, rhs_elem.len()); + let l = self.offsets[lhs_row].as_usize(); + let r = self.offsets[lhs_row + 1].as_usize(); + let existing_elem = unsafe { self.buffer.as_slice().get_unchecked(l..r) }; + existing_elem.len() == rhs_elem.len() && rhs_elem == existing_elem + // if rhs_elem.len() <= SHORT_VALUE_LEN { + // let inline = rhs_elem + // .iter() + // .fold(0usize, |acc, &x| acc << 8 | x as usize); + + // let existing_elem = unsafe { self.buffer.as_slice().get_unchecked(l..r) }; + // let existing_inline = existing_elem.iter().fold(0usize, |acc, &x| acc << 8 | x as usize); + // existing_inline == inline + // } else { + // let existing_elem = unsafe { self.buffer.as_slice().get_unchecked(l..r) }; + // existing_elem.len() == rhs_elem.len() && rhs_elem == existing_elem + // } + } +} + +impl ArrayEq for ByteGroupValueBuilderNaive +where + O: OffsetSizeTrait, +{ + fn equal_to(&self, lhs_row: usize, column: &ArrayRef, rhs_row: usize) -> bool { + // Sanity array type + match self.output_type { + OutputType::Binary => { + assert!(matches!( + column.data_type(), + DataType::Binary | DataType::LargeBinary + )); + self.equal_to_inner::>(lhs_row, column, rhs_row) + } + OutputType::Utf8 => { + assert!(matches!( + column.data_type(), + DataType::Utf8 | DataType::LargeUtf8 + )); + self.equal_to_inner::>(lhs_row, column, rhs_row) + } + _ => unreachable!("View types should use `ArrowBytesViewMap`"), + } + } + + fn append_val(&mut self, column: &ArrayRef, row: usize) { + // Sanity array type + match self.output_type { + OutputType::Binary => { + assert!(matches!( + column.data_type(), + DataType::Binary | DataType::LargeBinary + )); + self.append_val_inner::>(column, row) + } + OutputType::Utf8 => { + assert!(matches!( + column.data_type(), + DataType::Utf8 | DataType::LargeUtf8 + )); + self.append_val_inner::>(column, row) + } + _ => unreachable!("View types should use `ArrowBytesViewMap`"), + }; + } + + fn len(&self) -> usize { + self.offsets.len() - 1 + } + + fn build(self: Box) -> ArrayRef { + let Self { + output_type, + mut buffer, + offsets, + nulls, + } = *self; + + let null_buffer = if nulls.is_empty() { + None + } else { + // Only make a `NullBuffer` if there was a null value + let num_values = offsets.len() - 1; + let mut bool_builder = BooleanBufferBuilder::new(num_values); + bool_builder.append_n(num_values, true); + nulls.into_iter().for_each(|null_index| { + bool_builder.set_bit(null_index, false); + }); + Some(NullBuffer::from(bool_builder.finish())) + }; + + // let nulls = null.map(|null_index| { + // let num_values = offsets.len() - 1; + // single_null_buffer(num_values, null_index) + // }); + // SAFETY: the offsets were constructed correctly in `insert_if_new` -- + // monotonically increasing, overflows were checked. + let offsets = unsafe { OffsetBuffer::new_unchecked(ScalarBuffer::from(offsets)) }; + let values = buffer.finish(); + + match output_type { + OutputType::Binary => { + // SAFETY: the offsets were constructed correctly + Arc::new(unsafe { + GenericBinaryArray::new_unchecked(offsets, values, null_buffer) + }) + } + OutputType::Utf8 => { + // SAFETY: + // 1. the offsets were constructed safely + // + // 2. we asserted the input arrays were all the correct type and + // thus since all the values that went in were valid (e.g. utf8) + // so are all the values that come out + let res = Arc::new(unsafe { + GenericStringArray::new_unchecked(offsets, values, null_buffer) + }); + res + } + _ => unreachable!("View types should use `ArrowBytesViewMap`"), + } + } +} + +pub struct ByteGroupValueBuilder +where + O: OffsetSizeTrait, +{ + output_type: OutputType, + /// Underlying hash set for each distinct value + map: hashbrown::raw::RawTable>, + /// Total size of the map in bytes + map_size: usize, + buffer: BufferBuilder, + /// Offsets into `buffer` for each distinct value. These offsets as used + /// directly to create the final `GenericBinaryArray`. The `i`th string is + /// stored in the range `offsets[i]..offsets[i+1]` in `buffer`. Null values + /// are stored as a zero length string. + offsets: Vec, + /// buffer that stores hash values (reused across batches to save allocations) + hashes_buffer: Vec, + /// Null indexes in offsets + nulls: Vec, + // Store the offset + len for group values + group_values_offset: Vec>, +} + +impl ByteGroupValueBuilder +where + O: OffsetSizeTrait, +{ + pub fn new(array: &ArrayRef, output_type: OutputType) -> Self { + let n_rows = array.len(); + let random_state = RandomState::new(); + let mut hashes_buffer = vec![]; + let batch_hashes = &mut hashes_buffer; + batch_hashes.clear(); + batch_hashes.resize(n_rows, 0); + create_hashes(&[array.to_owned()], &random_state, batch_hashes) + // hash is supported for all types and create_hashes only + // returns errors for unsupported types + .unwrap(); + + Self { + output_type, + map: hashbrown::raw::RawTable::with_capacity(INITIAL_MAP_CAPACITY), + map_size: 0, + buffer: BufferBuilder::new(INITIAL_BUFFER_CAPACITY), + offsets: vec![O::default()], + hashes_buffer, + nulls: vec![], + group_values_offset: vec![], + } + } + + fn append_val_inner(&mut self, array: &ArrayRef, row: usize) + where + B: ByteArrayType, + { + let arr = array.as_bytes::(); + if arr.is_null(row) { + self.nulls.push(self.offsets.len() - 1); + // nulls need a zero length in the offset buffer + let offset = self.buffer.len(); + self.offsets.push(O::usize_as(offset)); + return; + } + + let hash = self.hashes_buffer[row]; + let value: &[u8] = arr.value(row).as_ref(); + let value_len = O::usize_as(value.len()); + + if value.len() <= SHORT_VALUE_LEN { + let inline = value.iter().fold(0usize, |acc, &x| acc << 8 | x as usize); + // is value is already present in the set? + let entry = self.map.get(hash, |header| { + // compare value if hashes match + if header.len != value_len { + return false; + } + // value is stored inline so no need to consult buffer + // (this is the "small string optimization") + inline == header.offset_or_inline + }); + + // Put the small values into buffer and offsets so it appears + // the output array, but store the actual bytes inline for + // comparison + self.buffer.append_slice(value); + self.offsets.push(O::usize_as(self.buffer.len())); + if let Some(entry) = entry { + } + // if no existing entry, make a new one + else { + // let payload = make_payload_fn(Some(value)); + let new_header = EntryWithPayload { + hash, + len: value_len, + offset_or_inline: inline, + }; + self.map.insert_accounted( + new_header, + |header| header.hash, + &mut self.map_size, + ); + } + } else { + // Check if the value is already present in the set + let entry = self.map.get_mut(hash, |header| { + // compare value if hashes match + if header.len != value_len { + return false; + } + // Need to compare the bytes in the buffer + // SAFETY: buffer is only appended to, and we correctly inserted values and offsets + let existing_value = + unsafe { self.buffer.as_slice().get_unchecked(header.range()) }; + value == existing_value + }); + + let offset = self.buffer.len(); // offset of start for data + self.buffer.append_slice(value); + self.offsets.push(O::usize_as(self.buffer.len())); + + if let Some(entry) = entry { + } + // if no existing entry, make a new header in map for equality check + else { + let new_header = EntryWithPayload { + hash, + len: value_len, + offset_or_inline: offset, + }; + self.map.insert_accounted( + new_header, + |header| header.hash, + &mut self.map_size, + ); + } + }; + } + + fn equal_to_inner(&self, lhs_row: usize, array: &ArrayRef, rhs_row: usize) -> bool + where + B: ByteArrayType, + { + // Handle nulls + let is_lhs_null = self.nulls.iter().any(|null_idx| *null_idx == lhs_row); + let arr = array.as_bytes::(); + if is_lhs_null { + return arr.is_null(rhs_row); + } else if arr.is_null(rhs_row) { + return false; + } + + let hash = self.hashes_buffer[rhs_row]; + let arr = array.as_bytes::(); + let rhs_elem: &[u8] = arr.value(rhs_row).as_ref(); + let rhs_elem_len = O::usize_as(rhs_elem.len()); + if rhs_elem.len() <= SHORT_VALUE_LEN { + let inline = rhs_elem + .iter() + .fold(0usize, |acc, &x| acc << 8 | x as usize); + // is value is already present in the set? + let entry = self.map.get(hash, |header| { + // compare value if hashes match + if header.len != rhs_elem_len { + return false; + } + // value is stored inline so no need to consult buffer + // (this is the "small string optimization") + inline == header.offset_or_inline + }); + entry.is_some() + } else { + // Check if the value is already present in the set + let entry = self.map.get(hash, |header| { + // if header.hash != hash { + // return false; + // } + + // compare value if hashes match + if header.len != rhs_elem_len { + return false; + } + // Need to compare the bytes in the buffer + // SAFETY: buffer is only appended to, and we correctly inserted values and offsets + let existing_elem = + unsafe { self.buffer.as_slice().get_unchecked(header.range()) }; + rhs_elem == existing_elem + }); + entry.is_some() + } + } +} + +impl ArrayEq for ByteGroupValueBuilder +where + O: OffsetSizeTrait, +{ + fn equal_to(&self, lhs_row: usize, column: &ArrayRef, rhs_row: usize) -> bool { + // Sanity array type + match self.output_type { + OutputType::Binary => { + assert!(matches!( + column.data_type(), + DataType::Binary | DataType::LargeBinary + )); + self.equal_to_inner::>(lhs_row, column, rhs_row) + } + OutputType::Utf8 => { + assert!(matches!( + column.data_type(), + DataType::Utf8 | DataType::LargeUtf8 + )); + self.equal_to_inner::>(lhs_row, column, rhs_row) + } + _ => unreachable!("View types should use `ArrowBytesViewMap`"), + } + } + + fn append_val(&mut self, column: &ArrayRef, row: usize) { + // Sanity array type + match self.output_type { + OutputType::Binary => { + assert!(matches!( + column.data_type(), + DataType::Binary | DataType::LargeBinary + )); + self.append_val_inner::>(column, row) + } + OutputType::Utf8 => { + assert!(matches!( + column.data_type(), + DataType::Utf8 | DataType::LargeUtf8 + )); + self.append_val_inner::>(column, row) + } + _ => unreachable!("View types should use `ArrowBytesViewMap`"), + }; + } + + fn len(&self) -> usize { + self.offsets.len() - 1 + } + + fn build(self: Box) -> ArrayRef { + let Self { + map: _, + map_size: _, + mut buffer, + offsets, + hashes_buffer: _, + nulls, + output_type, + group_values_offset, + } = *self; + + let null_buffer = if nulls.is_empty() { + None + } else { + // Only make a `NullBuffer` if there was a null value + let num_values = offsets.len() - 1; + let mut bool_builder = BooleanBufferBuilder::new(num_values); + bool_builder.append_n(num_values, true); + nulls.into_iter().for_each(|null_index| { + bool_builder.set_bit(null_index, false); + }); + Some(NullBuffer::from(bool_builder.finish())) + }; + + // let nulls = null.map(|null_index| { + // let num_values = offsets.len() - 1; + // single_null_buffer(num_values, null_index) + // }); + // SAFETY: the offsets were constructed correctly in `insert_if_new` -- + // monotonically increasing, overflows were checked. + let offsets = unsafe { OffsetBuffer::new_unchecked(ScalarBuffer::from(offsets)) }; + let values = buffer.finish(); + + match output_type { + OutputType::Binary => { + // SAFETY: the offsets were constructed correctly + Arc::new(unsafe { + GenericBinaryArray::new_unchecked(offsets, values, null_buffer) + }) + } + OutputType::Utf8 => { + // SAFETY: + // 1. the offsets were constructed safely + // + // 2. we asserted the input arrays were all the correct type and + // thus since all the values that went in were valid (e.g. utf8) + // so are all the values that come out + let res = Arc::new(unsafe { + GenericStringArray::new_unchecked(offsets, values, null_buffer) + }); + res + } + _ => unreachable!("View types should use `ArrowBytesViewMap`"), + } + } +} + +#[derive(Debug, PartialEq, Eq, Hash, Clone, Copy)] +struct EntryWithPayload +where + O: OffsetSizeTrait, +{ + /// hash of the value (stored to avoid recomputing it in hash table check) + hash: u64, + /// if len =< [`SHORT_VALUE_LEN`]: the data inlined + /// if len > [`SHORT_VALUE_LEN`], the offset of where the data starts + offset_or_inline: usize, + /// length of the value, in bytes (use O here so we use only i32 for + /// strings, rather 64 bit usize) + len: O, +} + +impl EntryWithPayload +where + O: OffsetSizeTrait, +{ + /// returns self.offset..self.offset + self.len + #[inline(always)] + fn range(&self) -> Range { + self.offset_or_inline..self.offset_or_inline + self.len.as_usize() + } +} diff --git a/datafusion/physical-expr-common/src/lib.rs b/datafusion/physical-expr-common/src/lib.rs index 7e2ea0c49397..bd2d7c93606b 100644 --- a/datafusion/physical-expr-common/src/lib.rs +++ b/datafusion/physical-expr-common/src/lib.rs @@ -23,6 +23,7 @@ pub mod binary_map; pub mod binary_view_map; pub mod datum; +pub mod group_value_row; pub mod physical_expr; pub mod sort_expr; pub mod tree_node; diff --git a/datafusion/physical-plan/src/aggregates/group_values/mod.rs b/datafusion/physical-plan/src/aggregates/group_values/mod.rs index be7ac934d7bc..d78706926aa1 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/mod.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/mod.rs @@ -26,7 +26,9 @@ use datafusion_expr::EmitTo; use primitive::GroupValuesPrimitive; mod row; +mod row_like; use row::GroupValuesRows; +use row_like::GroupValuesRowLike; mod bytes; mod bytes_view; @@ -92,5 +94,30 @@ pub fn new_group_values(schema: SchemaRef) -> Result> { } } - Ok(Box::new(GroupValuesRows::try_new(schema)?)) + if schema + .fields() + .iter() + .map(|f| f.data_type()) + .all(has_row_like_feature) + { + Ok(Box::new(GroupValuesRowLike::try_new(schema)?)) + } else { + Ok(Box::new(GroupValuesRows::try_new(schema)?)) + } +} + +fn has_row_like_feature(data_type: &DataType) -> bool { + match *data_type { + DataType::Int8 + | DataType::Int16 + | DataType::Int32 + | DataType::Int64 + | DataType::UInt8 + | DataType::UInt16 + | DataType::UInt32 + | DataType::UInt64 + | DataType::Utf8 + | DataType::LargeUtf8 => true, + _ => false, + } } diff --git a/datafusion/physical-plan/src/aggregates/group_values/row_like.rs b/datafusion/physical-plan/src/aggregates/group_values/row_like.rs new file mode 100644 index 000000000000..0d6137741e1a --- /dev/null +++ b/datafusion/physical-plan/src/aggregates/group_values/row_like.rs @@ -0,0 +1,398 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use crate::aggregates::group_values::GroupValues; +use ahash::RandomState; +use arrow::compute::cast; +use arrow::datatypes::{ + Float32Type, Float64Type, Int16Type, Int32Type, Int64Type, Int8Type, UInt16Type, + UInt32Type, UInt64Type, UInt8Type, +}; +use arrow::record_batch::RecordBatch; +use arrow::row::{RowConverter, Rows, SortField}; +use arrow_array::{Array, ArrayRef}; +use arrow_schema::{DataType, SchemaRef}; +use datafusion_common::hash_utils::create_hashes; +use datafusion_common::{internal_err, DataFusionError, Result}; +use datafusion_execution::memory_pool::proxy::{RawTableAllocExt, VecAllocExt}; +use datafusion_expr::EmitTo; +use datafusion_physical_expr::binary_map::OutputType; +use datafusion_physical_expr_common::group_value_row::{ + ArrayEq, ByteGroupValueBuilder, ByteGroupValueBuilderNaive, + PrimitiveGroupValueBuilder, +}; +use hashbrown::raw::RawTable; + +/// A [`GroupValues`] making use of [`Rows`] +pub struct GroupValuesRowLike { + /// The output schema + schema: SchemaRef, + + /// Converter for the group values + row_converter: RowConverter, + + /// Logically maps group values to a group_index in + /// [`Self::group_values`] and in each accumulator + /// + /// Uses the raw API of hashbrown to avoid actually storing the + /// keys (group values) in the table + /// + /// keys: u64 hashes of the GroupValue + /// values: (hash, group_index) + map: RawTable<(u64, usize)>, + + /// The size of `map` in bytes + map_size: usize, + + /// The actual group by values, stored in arrow [`Row`] format. + /// `group_values[i]` holds the group value for group_index `i`. + /// + /// The row format is used to compare group keys quickly and store + /// them efficiently in memory. Quick comparison is especially + /// important for multi-column group keys. + /// + /// [`Row`]: arrow::row::Row + // group_values: Option, + + /// reused buffer to store hashes + hashes_buffer: Vec, + + /// reused buffer to store rows + rows_buffer: Rows, + + /// Random state for creating hashes + random_state: RandomState, + group_values_v2: Option>>, +} + +impl GroupValuesRowLike { + pub fn try_new(schema: SchemaRef) -> Result { + let row_converter = RowConverter::new( + schema + .fields() + .iter() + .map(|f| SortField::new(f.data_type().clone())) + .collect(), + )?; + + let map = RawTable::with_capacity(0); + + let starting_rows_capacity = 1000; + let starting_data_capacity = 64 * starting_rows_capacity; + let rows_buffer = + row_converter.empty_rows(starting_rows_capacity, starting_data_capacity); + Ok(Self { + schema, + row_converter, + map, + map_size: 0, + // group_values: None, + group_values_v2: None, + hashes_buffer: Default::default(), + rows_buffer, + random_state: Default::default(), + }) + } +} + +impl GroupValues for GroupValuesRowLike { + fn intern(&mut self, cols: &[ArrayRef], groups: &mut Vec) -> Result<()> { + // Convert the group keys into the row format + // let group_rows = &mut self.rows_buffer; + // group_rows.clear(); + // self.row_converter.append(group_rows, cols)?; + // let n_rows = group_rows.num_rows(); + + // let mut group_values = match self.group_values.take() { + // Some(group_values) => group_values, + // None => self.row_converter.empty_rows(0, 0), + // }; + + let n_rows = cols[0].len(); + let mut group_values_v2 = match self.group_values_v2.take() { + Some(group_values) => group_values, + None => { + let len = cols.len(); + let mut v = Vec::with_capacity(len); + // Move to `try_new` + for (i, f) in self.schema.fields().iter().enumerate() { + match f.data_type() { + &DataType::Int8 => { + let b = PrimitiveGroupValueBuilder::::new(); + v.push(Box::new(b) as _) + } + &DataType::Int16 => { + let b = PrimitiveGroupValueBuilder::::new(); + v.push(Box::new(b) as _) + } + &DataType::Int32 => { + let b = PrimitiveGroupValueBuilder::::new(); + v.push(Box::new(b) as _) + } + &DataType::Int64 => { + let b = PrimitiveGroupValueBuilder::::new(); + v.push(Box::new(b) as _) + } + &DataType::UInt8 => { + let b = PrimitiveGroupValueBuilder::::new(); + v.push(Box::new(b) as _) + } + &DataType::UInt16 => { + let b = PrimitiveGroupValueBuilder::::new(); + v.push(Box::new(b) as _) + } + &DataType::UInt32 => { + let b = PrimitiveGroupValueBuilder::::new(); + v.push(Box::new(b) as _) + } + &DataType::UInt64 => { + let b = PrimitiveGroupValueBuilder::::new(); + v.push(Box::new(b) as _) + } + &DataType::Float32 => { + let b = PrimitiveGroupValueBuilder::::new(); + v.push(Box::new(b) as _) + } + &DataType::Float64 => { + let b = PrimitiveGroupValueBuilder::::new(); + v.push(Box::new(b) as _) + } + &DataType::Utf8 => { + // let b = StringGroupValueBuilder::new(); + let b = ByteGroupValueBuilderNaive::::new( + &cols[i], + OutputType::Utf8, + ); + v.push(Box::new(b) as _) + } + &DataType::LargeUtf8 => { + let b = ByteGroupValueBuilder::::new( + &cols[i], + OutputType::Utf8, + ); + v.push(Box::new(b) as _) + } + dt => todo!("{dt} not impl"), + } + } + v + } + }; + + // tracks to which group each of the input rows belongs + groups.clear(); + + // 1.1 Calculate the group keys for the group values + let batch_hashes = &mut self.hashes_buffer; + batch_hashes.clear(); + batch_hashes.resize(n_rows, 0); + create_hashes(cols, &self.random_state, batch_hashes)?; + + for (row, &target_hash) in batch_hashes.iter().enumerate() { + let entry = self.map.get_mut(target_hash, |(exist_hash, group_idx)| { + // Somewhat surprisingly, this closure can be called even if the + // hash doesn't match, so check the hash first with an integer + // comparison first avoid the more expensive comparison with + // group value. https://github.com/apache/datafusion/pull/11718 + if target_hash != *exist_hash { + return false; + } + // verify that the group that we are inserting with hash is + // actually the same key value as the group in + // existing_idx (aka group_values @ row) + // && group_rows.row(row) == group_values.row(*group_idx) + + fn compare_equal( + arry_eq: &dyn ArrayEq, + lhs_row: usize, + array: &ArrayRef, + rhs_row: usize, + ) -> bool { + arry_eq.equal_to(lhs_row, array, rhs_row) + } + + for (i, group_val) in group_values_v2.iter().enumerate() { + if !compare_equal(group_val.as_ref(), *group_idx, &cols[i], row) { + return false; + } + } + + true + }); + + let group_idx = match entry { + // Existing group_index for this group value + Some((_hash, group_idx)) => *group_idx, + // 1.2 Need to create new entry for the group + None => { + // Add new entry to aggr_state and save newly created index + // let group_idx = group_values.num_rows(); + // group_values.push(group_rows.row(row)); + + let mut checklen = 0; + let group_idx = group_values_v2[0].len(); + for (i, group_value) in group_values_v2.iter_mut().enumerate() { + group_value.append_val(&cols[i], row); + let len = group_value.len(); + if i == 0 { + checklen = len; + } else { + assert_eq!(checklen, len); + } + } + + // for hasher function, use precomputed hash value + self.map.insert_accounted( + (target_hash, group_idx), + |(hash, _group_index)| *hash, + &mut self.map_size, + ); + group_idx + } + }; + groups.push(group_idx); + } + + // self.group_values = Some(group_values); + self.group_values_v2 = Some(group_values_v2); + + Ok(()) + } + + fn size(&self) -> usize { + // TODO: get real size + let group_values_size = + self.group_values_v2.as_ref().map(|v| v.len()).unwrap_or(0); + // let group_values_size = self.group_values.as_ref().map(|v| v.size()).unwrap_or(0); + self.row_converter.size() + + group_values_size + + self.map_size + + self.rows_buffer.size() + + self.hashes_buffer.allocated_size() + } + + fn is_empty(&self) -> bool { + self.len() == 0 + } + + fn len(&self) -> usize { + // self.group_values + // .as_ref() + // .map(|group_values| group_values.num_rows()) + // .unwrap_or(0) + + self.group_values_v2 + .as_ref() + .map(|v| v[0].len()) + .unwrap_or(0) + } + + fn emit(&mut self, emit_to: EmitTo) -> Result> { + // println!("emit"); + // let mut group_values = self + // .group_values + // .take() + // .expect("Can not emit from empty rows"); + + let mut group_values_v2 = self + .group_values_v2 + .take() + .expect("Can not emit from empty rows"); + + let mut output = match emit_to { + EmitTo::All => { + let output = group_values_v2 + .into_iter() + .map(|v| v.build()) + .collect::>(); + // let output = self.row_converter.convert_rows(&group_values)?; + // group_values.clear(); + // println!("output: {:?}", output); + self.group_values_v2 = None; + output + } + EmitTo::First(n) => { + return internal_err!("there is error"); + + // println!("to first n"); + let len = group_values_v2.len(); + let first_n: Vec> = group_values_v2.drain(..n).collect(); + let output = first_n.into_iter().map(|v| v.build()).collect::>(); + assert_eq!(len, group_values_v2.len() + n); + self.group_values_v2 = Some(group_values_v2); + + // let groups_rows = group_values.iter().take(n); + // let output = self.row_converter.convert_rows(groups_rows)?; + // // Clear out first n group keys by copying them to a new Rows. + // // TODO file some ticket in arrow-rs to make this more efficient? + // let mut new_group_values = self.row_converter.empty_rows(0, 0); + // for row in group_values.iter().skip(n) { + // new_group_values.push(row); + // } + // std::mem::swap(&mut new_group_values, &mut group_values); + + // SAFETY: self.map outlives iterator and is not modified concurrently + unsafe { + for bucket in self.map.iter() { + // Decrement group index by n + match bucket.as_ref().1.checked_sub(n) { + // Group index was >= n, shift value down + Some(sub) => bucket.as_mut().1 = sub, + // Group index was < n, so remove from table + None => self.map.erase(bucket), + } + } + } + output + } + }; + + // TODO: Materialize dictionaries in group keys (#7647) + for (field, array) in self.schema.fields.iter().zip(&mut output) { + let expected = field.data_type(); + if let DataType::Dictionary(_, v) = expected { + let actual = array.data_type(); + if v.as_ref() != actual { + return Err(DataFusionError::Internal(format!( + "Converted group rows expected dictionary of {v} got {actual}" + ))); + } + *array = cast(array.as_ref(), expected)?; + } + } + + // self.group_values = Some(group_values); + Ok(output) + } + + fn clear_shrink(&mut self, batch: &RecordBatch) { + let count = batch.num_rows(); + // self.group_values = self.group_values.take().map(|mut rows| { + // rows.clear(); + // rows + // }); + self.group_values_v2 = self.group_values_v2.take().map(|mut rows| { + rows.clear(); + rows + }); + self.map.clear(); + self.map.shrink_to(count, |_| 0); // hasher does not matter since the map is cleared + self.map_size = self.map.capacity() * std::mem::size_of::<(u64, usize)>(); + self.hashes_buffer.clear(); + self.hashes_buffer.shrink_to(count); + } +} diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index 0332131d4b57..d90a3242aa50 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -618,6 +618,7 @@ impl Stream for GroupedHashAggregateStream { match ready!(self.input.poll_next_unpin(cx)) { // new batch to aggregate Some(Ok(batch)) => { + // println!("stat"); let timer = elapsed_compute.timer(); let input_rows = batch.num_rows(); diff --git a/datafusion/sqllogictest/test_files/test1.slt b/datafusion/sqllogictest/test_files/test1.slt new file mode 100644 index 000000000000..f977395f75c4 --- /dev/null +++ b/datafusion/sqllogictest/test_files/test1.slt @@ -0,0 +1,33 @@ + +statement ok +CREATE EXTERNAL TABLE hits +STORED AS PARQUET +LOCATION '../../benchmarks/data/hits.parquet'; + +# query ITI +# SELECT "UserID", "SearchPhrase", COUNT(*) FROM hits GROUP BY "UserID", "SearchPhrase" ORDER BY COUNT(*) DESC LIMIT 10; +# ---- +# 1313338681122956954 (empty) 29097 +# 1907779576417363396 (empty) 25333 +# 2305303682471783379 (empty) 10597 +# 7982623143712728547 (empty) 6669 +# 7280399273658728997 (empty) 6408 +# 1090981537032625727 (empty) 6196 +# 5730251990344211405 (empty) 6019 +# 6018350421959114808 (empty) 5990 +# 835157184735512989 (empty) 5209 +# 770542365400669095 (empty) 4906 + +query IIIII +SELECT "ClientIP", "ClientIP" - 1, "ClientIP" - 2, "ClientIP" - 3, COUNT(*) AS c FROM hits GROUP BY "ClientIP", "ClientIP" - 1, "ClientIP" - 2, "ClientIP" - 3 ORDER BY c DESC LIMIT 10; +---- +-39921974 -39921975 -39921976 -39921977 47008 +-1698104457 -1698104458 -1698104459 -1698104460 29121 +-1175819552 -1175819553 -1175819554 -1175819555 25333 +1696638182 1696638181 1696638180 1696638179 20220 +1138507705 1138507704 1138507703 1138507702 15778 +-927025522 -927025523 -927025524 -927025525 12768 +-1262139876 -1262139877 -1262139878 -1262139879 11348 +1740861572 1740861571 1740861570 1740861569 11314 +-807147100 -807147101 -807147102 -807147103 9880 +-631062503 -631062504 -631062505 -631062506 9718 diff --git a/datafusion/sqllogictest/test_files/test2.slt b/datafusion/sqllogictest/test_files/test2.slt new file mode 100644 index 000000000000..e801264a0a77 --- /dev/null +++ b/datafusion/sqllogictest/test_files/test2.slt @@ -0,0 +1,9 @@ +statement ok +create table t (a int, b string, c int) as values (1, 'a', 3), (1, 'a', 3), (1, 'a', 3), (1, 'a', 3), (2, 'a', 1), (2, 'a', 1), (2, 'a', 1), (3, null, 2); + +query ITI +select a, b, count(*) from t group by a, b order by count(*) desc; +---- +1 a 4 +2 a 3 +3 NULL 1 From f40a16439a2d8113e96234fe0af3c13a9a03b257 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Sat, 31 Aug 2024 12:07:35 +0800 Subject: [PATCH 02/30] comment out unused Signed-off-by: jayzhan211 --- .../physical-expr-common/src/binary_map.rs | 6 +- .../src/group_value_row.rs | 758 +++++++++--------- .../src/aggregates/group_values/row_like.rs | 16 +- 3 files changed, 377 insertions(+), 403 deletions(-) diff --git a/datafusion/physical-expr-common/src/binary_map.rs b/datafusion/physical-expr-common/src/binary_map.rs index 651937b4f06b..6a56f66a6ffb 100644 --- a/datafusion/physical-expr-common/src/binary_map.rs +++ b/datafusion/physical-expr-common/src/binary_map.rs @@ -235,9 +235,9 @@ where } /// The size, in number of entries, of the initial hash table -pub(crate) const INITIAL_MAP_CAPACITY: usize = 128; +pub(super) const INITIAL_MAP_CAPACITY: usize = 128; /// The initial size, in bytes, of the string data -pub(crate) const INITIAL_BUFFER_CAPACITY: usize = 8 * 1024; +pub(super) const INITIAL_BUFFER_CAPACITY: usize = 8 * 1024; impl ArrowBytesMap where V: Debug + PartialEq + Eq + Clone + Copy + Default, @@ -574,7 +574,7 @@ where } /// Maximum size of a value that can be inlined in the hash table -pub(crate) const SHORT_VALUE_LEN: usize = mem::size_of::(); +const SHORT_VALUE_LEN: usize = mem::size_of::(); /// Entry in the hash table -- see [`ArrowBytesMap`] for more details #[derive(Debug, PartialEq, Eq, Hash, Clone, Copy)] diff --git a/datafusion/physical-expr-common/src/group_value_row.rs b/datafusion/physical-expr-common/src/group_value_row.rs index 1d5aaa81e62d..399323006950 100644 --- a/datafusion/physical-expr-common/src/group_value_row.rs +++ b/datafusion/physical-expr-common/src/group_value_row.rs @@ -15,13 +15,11 @@ // specific language governing permissions and limitations // under the License. -use ahash::RandomState; use arrow::array::BooleanBufferBuilder; use arrow::array::BufferBuilder; use arrow::array::GenericBinaryArray; use arrow::array::GenericStringArray; use arrow::array::OffsetSizeTrait; -use arrow::array::StringArray; use arrow::buffer::NullBuffer; use arrow::buffer::OffsetBuffer; use arrow::buffer::ScalarBuffer; @@ -33,13 +31,11 @@ use arrow::{ array::{Array, ArrayRef, ArrowPrimitiveType, AsArray, PrimitiveArray}, datatypes::ByteArrayType, }; -use datafusion_common::hash_utils::create_hashes; -use datafusion_common::utils::proxy::RawTableAllocExt; use crate::binary_map::OutputType; -use crate::binary_map::{INITIAL_BUFFER_CAPACITY, INITIAL_MAP_CAPACITY, SHORT_VALUE_LEN}; +use crate::binary_map::INITIAL_BUFFER_CAPACITY; -use std::{fmt::Debug, ops::Range, sync::Arc}; +use std::sync::Arc; pub trait ArrayEq: Send + Sync { fn equal_to(&self, lhs_row: usize, array: &ArrayRef, rhs_row: usize) -> bool; @@ -89,47 +85,47 @@ impl ArrayEq for PrimitiveGroupValueBuilder { } } -pub struct StringGroupValueBuilder(Vec>); - -impl StringGroupValueBuilder { - pub fn new() -> Self { - Self(vec![]) - } -} - -impl ArrayEq for StringGroupValueBuilder { - fn equal_to(&self, lhs_row: usize, array: &ArrayRef, rhs_row: usize) -> bool { - let elem = &self.0[lhs_row]; - let arr = array.as_string::(); - let is_rhs_null = arr.is_null(rhs_row); - if elem.is_none() && is_rhs_null { - true - } else if elem.is_some() && !is_rhs_null { - let e = elem.as_ref().unwrap(); - e.as_str() == arr.value(rhs_row) - } else { - false - } - } - - fn append_val(&mut self, array: &ArrayRef, row: usize) { - let arr = array.as_string::(); - if arr.is_null(row) { - self.0.push(None) - } else { - let elem = arr.value(row); - self.0.push(Some(elem.to_string())) - } - } - - fn len(&self) -> usize { - self.0.len() - } - - fn build(self: Box) -> ArrayRef { - Arc::new(StringArray::from_iter(self.0)) - } -} +// pub struct StringGroupValueBuilder(Vec>); + +// impl StringGroupValueBuilder { +// pub fn new() -> Self { +// Self(vec![]) +// } +// } + +// impl ArrayEq for StringGroupValueBuilder { +// fn equal_to(&self, lhs_row: usize, array: &ArrayRef, rhs_row: usize) -> bool { +// let elem = &self.0[lhs_row]; +// let arr = array.as_string::(); +// let is_rhs_null = arr.is_null(rhs_row); +// if elem.is_none() && is_rhs_null { +// true +// } else if elem.is_some() && !is_rhs_null { +// let e = elem.as_ref().unwrap(); +// e.as_str() == arr.value(rhs_row) +// } else { +// false +// } +// } + +// fn append_val(&mut self, array: &ArrayRef, row: usize) { +// let arr = array.as_string::(); +// if arr.is_null(row) { +// self.0.push(None) +// } else { +// let elem = arr.value(row); +// self.0.push(Some(elem.to_string())) +// } +// } + +// fn len(&self) -> usize { +// self.0.len() +// } + +// fn build(self: Box) -> ArrayRef { +// Arc::new(StringArray::from_iter(self.0)) +// } +// } pub struct ByteGroupValueBuilderNaive where @@ -150,7 +146,7 @@ impl ByteGroupValueBuilderNaive where O: OffsetSizeTrait, { - pub fn new(_array: &ArrayRef, output_type: OutputType) -> Self { + pub fn new(output_type: OutputType) -> Self { Self { output_type, buffer: BufferBuilder::new(INITIAL_BUFFER_CAPACITY), @@ -198,18 +194,6 @@ where let r = self.offsets[lhs_row + 1].as_usize(); let existing_elem = unsafe { self.buffer.as_slice().get_unchecked(l..r) }; existing_elem.len() == rhs_elem.len() && rhs_elem == existing_elem - // if rhs_elem.len() <= SHORT_VALUE_LEN { - // let inline = rhs_elem - // .iter() - // .fold(0usize, |acc, &x| acc << 8 | x as usize); - - // let existing_elem = unsafe { self.buffer.as_slice().get_unchecked(l..r) }; - // let existing_inline = existing_elem.iter().fold(0usize, |acc, &x| acc << 8 | x as usize); - // existing_inline == inline - // } else { - // let existing_elem = unsafe { self.buffer.as_slice().get_unchecked(l..r) }; - // existing_elem.len() == rhs_elem.len() && rhs_elem == existing_elem - // } } } @@ -284,310 +268,6 @@ where Some(NullBuffer::from(bool_builder.finish())) }; - // let nulls = null.map(|null_index| { - // let num_values = offsets.len() - 1; - // single_null_buffer(num_values, null_index) - // }); - // SAFETY: the offsets were constructed correctly in `insert_if_new` -- - // monotonically increasing, overflows were checked. - let offsets = unsafe { OffsetBuffer::new_unchecked(ScalarBuffer::from(offsets)) }; - let values = buffer.finish(); - - match output_type { - OutputType::Binary => { - // SAFETY: the offsets were constructed correctly - Arc::new(unsafe { - GenericBinaryArray::new_unchecked(offsets, values, null_buffer) - }) - } - OutputType::Utf8 => { - // SAFETY: - // 1. the offsets were constructed safely - // - // 2. we asserted the input arrays were all the correct type and - // thus since all the values that went in were valid (e.g. utf8) - // so are all the values that come out - let res = Arc::new(unsafe { - GenericStringArray::new_unchecked(offsets, values, null_buffer) - }); - res - } - _ => unreachable!("View types should use `ArrowBytesViewMap`"), - } - } -} - -pub struct ByteGroupValueBuilder -where - O: OffsetSizeTrait, -{ - output_type: OutputType, - /// Underlying hash set for each distinct value - map: hashbrown::raw::RawTable>, - /// Total size of the map in bytes - map_size: usize, - buffer: BufferBuilder, - /// Offsets into `buffer` for each distinct value. These offsets as used - /// directly to create the final `GenericBinaryArray`. The `i`th string is - /// stored in the range `offsets[i]..offsets[i+1]` in `buffer`. Null values - /// are stored as a zero length string. - offsets: Vec, - /// buffer that stores hash values (reused across batches to save allocations) - hashes_buffer: Vec, - /// Null indexes in offsets - nulls: Vec, - // Store the offset + len for group values - group_values_offset: Vec>, -} - -impl ByteGroupValueBuilder -where - O: OffsetSizeTrait, -{ - pub fn new(array: &ArrayRef, output_type: OutputType) -> Self { - let n_rows = array.len(); - let random_state = RandomState::new(); - let mut hashes_buffer = vec![]; - let batch_hashes = &mut hashes_buffer; - batch_hashes.clear(); - batch_hashes.resize(n_rows, 0); - create_hashes(&[array.to_owned()], &random_state, batch_hashes) - // hash is supported for all types and create_hashes only - // returns errors for unsupported types - .unwrap(); - - Self { - output_type, - map: hashbrown::raw::RawTable::with_capacity(INITIAL_MAP_CAPACITY), - map_size: 0, - buffer: BufferBuilder::new(INITIAL_BUFFER_CAPACITY), - offsets: vec![O::default()], - hashes_buffer, - nulls: vec![], - group_values_offset: vec![], - } - } - - fn append_val_inner(&mut self, array: &ArrayRef, row: usize) - where - B: ByteArrayType, - { - let arr = array.as_bytes::(); - if arr.is_null(row) { - self.nulls.push(self.offsets.len() - 1); - // nulls need a zero length in the offset buffer - let offset = self.buffer.len(); - self.offsets.push(O::usize_as(offset)); - return; - } - - let hash = self.hashes_buffer[row]; - let value: &[u8] = arr.value(row).as_ref(); - let value_len = O::usize_as(value.len()); - - if value.len() <= SHORT_VALUE_LEN { - let inline = value.iter().fold(0usize, |acc, &x| acc << 8 | x as usize); - // is value is already present in the set? - let entry = self.map.get(hash, |header| { - // compare value if hashes match - if header.len != value_len { - return false; - } - // value is stored inline so no need to consult buffer - // (this is the "small string optimization") - inline == header.offset_or_inline - }); - - // Put the small values into buffer and offsets so it appears - // the output array, but store the actual bytes inline for - // comparison - self.buffer.append_slice(value); - self.offsets.push(O::usize_as(self.buffer.len())); - if let Some(entry) = entry { - } - // if no existing entry, make a new one - else { - // let payload = make_payload_fn(Some(value)); - let new_header = EntryWithPayload { - hash, - len: value_len, - offset_or_inline: inline, - }; - self.map.insert_accounted( - new_header, - |header| header.hash, - &mut self.map_size, - ); - } - } else { - // Check if the value is already present in the set - let entry = self.map.get_mut(hash, |header| { - // compare value if hashes match - if header.len != value_len { - return false; - } - // Need to compare the bytes in the buffer - // SAFETY: buffer is only appended to, and we correctly inserted values and offsets - let existing_value = - unsafe { self.buffer.as_slice().get_unchecked(header.range()) }; - value == existing_value - }); - - let offset = self.buffer.len(); // offset of start for data - self.buffer.append_slice(value); - self.offsets.push(O::usize_as(self.buffer.len())); - - if let Some(entry) = entry { - } - // if no existing entry, make a new header in map for equality check - else { - let new_header = EntryWithPayload { - hash, - len: value_len, - offset_or_inline: offset, - }; - self.map.insert_accounted( - new_header, - |header| header.hash, - &mut self.map_size, - ); - } - }; - } - - fn equal_to_inner(&self, lhs_row: usize, array: &ArrayRef, rhs_row: usize) -> bool - where - B: ByteArrayType, - { - // Handle nulls - let is_lhs_null = self.nulls.iter().any(|null_idx| *null_idx == lhs_row); - let arr = array.as_bytes::(); - if is_lhs_null { - return arr.is_null(rhs_row); - } else if arr.is_null(rhs_row) { - return false; - } - - let hash = self.hashes_buffer[rhs_row]; - let arr = array.as_bytes::(); - let rhs_elem: &[u8] = arr.value(rhs_row).as_ref(); - let rhs_elem_len = O::usize_as(rhs_elem.len()); - if rhs_elem.len() <= SHORT_VALUE_LEN { - let inline = rhs_elem - .iter() - .fold(0usize, |acc, &x| acc << 8 | x as usize); - // is value is already present in the set? - let entry = self.map.get(hash, |header| { - // compare value if hashes match - if header.len != rhs_elem_len { - return false; - } - // value is stored inline so no need to consult buffer - // (this is the "small string optimization") - inline == header.offset_or_inline - }); - entry.is_some() - } else { - // Check if the value is already present in the set - let entry = self.map.get(hash, |header| { - // if header.hash != hash { - // return false; - // } - - // compare value if hashes match - if header.len != rhs_elem_len { - return false; - } - // Need to compare the bytes in the buffer - // SAFETY: buffer is only appended to, and we correctly inserted values and offsets - let existing_elem = - unsafe { self.buffer.as_slice().get_unchecked(header.range()) }; - rhs_elem == existing_elem - }); - entry.is_some() - } - } -} - -impl ArrayEq for ByteGroupValueBuilder -where - O: OffsetSizeTrait, -{ - fn equal_to(&self, lhs_row: usize, column: &ArrayRef, rhs_row: usize) -> bool { - // Sanity array type - match self.output_type { - OutputType::Binary => { - assert!(matches!( - column.data_type(), - DataType::Binary | DataType::LargeBinary - )); - self.equal_to_inner::>(lhs_row, column, rhs_row) - } - OutputType::Utf8 => { - assert!(matches!( - column.data_type(), - DataType::Utf8 | DataType::LargeUtf8 - )); - self.equal_to_inner::>(lhs_row, column, rhs_row) - } - _ => unreachable!("View types should use `ArrowBytesViewMap`"), - } - } - - fn append_val(&mut self, column: &ArrayRef, row: usize) { - // Sanity array type - match self.output_type { - OutputType::Binary => { - assert!(matches!( - column.data_type(), - DataType::Binary | DataType::LargeBinary - )); - self.append_val_inner::>(column, row) - } - OutputType::Utf8 => { - assert!(matches!( - column.data_type(), - DataType::Utf8 | DataType::LargeUtf8 - )); - self.append_val_inner::>(column, row) - } - _ => unreachable!("View types should use `ArrowBytesViewMap`"), - }; - } - - fn len(&self) -> usize { - self.offsets.len() - 1 - } - - fn build(self: Box) -> ArrayRef { - let Self { - map: _, - map_size: _, - mut buffer, - offsets, - hashes_buffer: _, - nulls, - output_type, - group_values_offset, - } = *self; - - let null_buffer = if nulls.is_empty() { - None - } else { - // Only make a `NullBuffer` if there was a null value - let num_values = offsets.len() - 1; - let mut bool_builder = BooleanBufferBuilder::new(num_values); - bool_builder.append_n(num_values, true); - nulls.into_iter().for_each(|null_index| { - bool_builder.set_bit(null_index, false); - }); - Some(NullBuffer::from(bool_builder.finish())) - }; - - // let nulls = null.map(|null_index| { - // let num_values = offsets.len() - 1; - // single_null_buffer(num_values, null_index) - // }); // SAFETY: the offsets were constructed correctly in `insert_if_new` -- // monotonically increasing, overflows were checked. let offsets = unsafe { OffsetBuffer::new_unchecked(ScalarBuffer::from(offsets)) }; @@ -617,28 +297,328 @@ where } } -#[derive(Debug, PartialEq, Eq, Hash, Clone, Copy)] -struct EntryWithPayload -where - O: OffsetSizeTrait, -{ - /// hash of the value (stored to avoid recomputing it in hash table check) - hash: u64, - /// if len =< [`SHORT_VALUE_LEN`]: the data inlined - /// if len > [`SHORT_VALUE_LEN`], the offset of where the data starts - offset_or_inline: usize, - /// length of the value, in bytes (use O here so we use only i32 for - /// strings, rather 64 bit usize) - len: O, -} - -impl EntryWithPayload -where - O: OffsetSizeTrait, -{ - /// returns self.offset..self.offset + self.len - #[inline(always)] - fn range(&self) -> Range { - self.offset_or_inline..self.offset_or_inline + self.len.as_usize() - } -} +// pub struct ByteGroupValueBuilder +// where +// O: OffsetSizeTrait, +// { +// output_type: OutputType, +// /// Underlying hash set for each distinct value +// map: hashbrown::raw::RawTable>, +// /// Total size of the map in bytes +// map_size: usize, +// buffer: BufferBuilder, +// /// Offsets into `buffer` for each distinct value. These offsets as used +// /// directly to create the final `GenericBinaryArray`. The `i`th string is +// /// stored in the range `offsets[i]..offsets[i+1]` in `buffer`. Null values +// /// are stored as a zero length string. +// offsets: Vec, +// /// buffer that stores hash values (reused across batches to save allocations) +// hashes_buffer: Vec, +// /// Null indexes in offsets +// nulls: Vec, +// // Store the offset + len for group values +// group_values_offset: Vec>, +// } + +// impl ByteGroupValueBuilder +// where +// O: OffsetSizeTrait, +// { +// pub fn new(array: &ArrayRef, output_type: OutputType) -> Self { +// let n_rows = array.len(); +// let random_state = RandomState::new(); +// let mut hashes_buffer = vec![]; +// let batch_hashes = &mut hashes_buffer; +// batch_hashes.clear(); +// batch_hashes.resize(n_rows, 0); +// create_hashes(&[array.to_owned()], &random_state, batch_hashes) +// // hash is supported for all types and create_hashes only +// // returns errors for unsupported types +// .unwrap(); + +// Self { +// output_type, +// map: hashbrown::raw::RawTable::with_capacity(INITIAL_MAP_CAPACITY), +// map_size: 0, +// buffer: BufferBuilder::new(INITIAL_BUFFER_CAPACITY), +// offsets: vec![O::default()], +// hashes_buffer, +// nulls: vec![], +// group_values_offset: vec![], +// } +// } + +// fn append_val_inner(&mut self, array: &ArrayRef, row: usize) +// where +// B: ByteArrayType, +// { +// let arr = array.as_bytes::(); +// if arr.is_null(row) { +// self.nulls.push(self.offsets.len() - 1); +// // nulls need a zero length in the offset buffer +// let offset = self.buffer.len(); +// self.offsets.push(O::usize_as(offset)); +// return; +// } + +// let hash = self.hashes_buffer[row]; +// let value: &[u8] = arr.value(row).as_ref(); +// let value_len = O::usize_as(value.len()); + +// if value.len() <= SHORT_VALUE_LEN { +// let inline = value.iter().fold(0usize, |acc, &x| acc << 8 | x as usize); +// // is value is already present in the set? +// let entry = self.map.get(hash, |header| { +// // compare value if hashes match +// if header.len != value_len { +// return false; +// } +// // value is stored inline so no need to consult buffer +// // (this is the "small string optimization") +// inline == header.offset_or_inline +// }); + +// // Put the small values into buffer and offsets so it appears +// // the output array, but store the actual bytes inline for +// // comparison +// self.buffer.append_slice(value); +// self.offsets.push(O::usize_as(self.buffer.len())); +// if let Some(entry) = entry { +// } +// // if no existing entry, make a new one +// else { +// // let payload = make_payload_fn(Some(value)); +// let new_header = EntryWithPayload { +// hash, +// len: value_len, +// offset_or_inline: inline, +// }; +// self.map.insert_accounted( +// new_header, +// |header| header.hash, +// &mut self.map_size, +// ); +// } +// } else { +// // Check if the value is already present in the set +// let entry = self.map.get_mut(hash, |header| { +// // compare value if hashes match +// if header.len != value_len { +// return false; +// } +// // Need to compare the bytes in the buffer +// // SAFETY: buffer is only appended to, and we correctly inserted values and offsets +// let existing_value = +// unsafe { self.buffer.as_slice().get_unchecked(header.range()) }; +// value == existing_value +// }); + +// let offset = self.buffer.len(); // offset of start for data +// self.buffer.append_slice(value); +// self.offsets.push(O::usize_as(self.buffer.len())); + +// if let Some(entry) = entry { +// } +// // if no existing entry, make a new header in map for equality check +// else { +// let new_header = EntryWithPayload { +// hash, +// len: value_len, +// offset_or_inline: offset, +// }; +// self.map.insert_accounted( +// new_header, +// |header| header.hash, +// &mut self.map_size, +// ); +// } +// }; +// } + +// fn equal_to_inner(&self, lhs_row: usize, array: &ArrayRef, rhs_row: usize) -> bool +// where +// B: ByteArrayType, +// { +// // Handle nulls +// let is_lhs_null = self.nulls.iter().any(|null_idx| *null_idx == lhs_row); +// let arr = array.as_bytes::(); +// if is_lhs_null { +// return arr.is_null(rhs_row); +// } else if arr.is_null(rhs_row) { +// return false; +// } + +// let hash = self.hashes_buffer[rhs_row]; +// let arr = array.as_bytes::(); +// let rhs_elem: &[u8] = arr.value(rhs_row).as_ref(); +// let rhs_elem_len = O::usize_as(rhs_elem.len()); +// if rhs_elem.len() <= SHORT_VALUE_LEN { +// let inline = rhs_elem +// .iter() +// .fold(0usize, |acc, &x| acc << 8 | x as usize); +// // is value is already present in the set? +// let entry = self.map.get(hash, |header| { +// // compare value if hashes match +// if header.len != rhs_elem_len { +// return false; +// } +// // value is stored inline so no need to consult buffer +// // (this is the "small string optimization") +// inline == header.offset_or_inline +// }); +// entry.is_some() +// } else { +// // Check if the value is already present in the set +// let entry = self.map.get(hash, |header| { +// // if header.hash != hash { +// // return false; +// // } + +// // compare value if hashes match +// if header.len != rhs_elem_len { +// return false; +// } +// // Need to compare the bytes in the buffer +// // SAFETY: buffer is only appended to, and we correctly inserted values and offsets +// let existing_elem = +// unsafe { self.buffer.as_slice().get_unchecked(header.range()) }; +// rhs_elem == existing_elem +// }); +// entry.is_some() +// } +// } +// } + +// impl ArrayEq for ByteGroupValueBuilder +// where +// O: OffsetSizeTrait, +// { +// fn equal_to(&self, lhs_row: usize, column: &ArrayRef, rhs_row: usize) -> bool { +// // Sanity array type +// match self.output_type { +// OutputType::Binary => { +// assert!(matches!( +// column.data_type(), +// DataType::Binary | DataType::LargeBinary +// )); +// self.equal_to_inner::>(lhs_row, column, rhs_row) +// } +// OutputType::Utf8 => { +// assert!(matches!( +// column.data_type(), +// DataType::Utf8 | DataType::LargeUtf8 +// )); +// self.equal_to_inner::>(lhs_row, column, rhs_row) +// } +// _ => unreachable!("View types should use `ArrowBytesViewMap`"), +// } +// } + +// fn append_val(&mut self, column: &ArrayRef, row: usize) { +// // Sanity array type +// match self.output_type { +// OutputType::Binary => { +// assert!(matches!( +// column.data_type(), +// DataType::Binary | DataType::LargeBinary +// )); +// self.append_val_inner::>(column, row) +// } +// OutputType::Utf8 => { +// assert!(matches!( +// column.data_type(), +// DataType::Utf8 | DataType::LargeUtf8 +// )); +// self.append_val_inner::>(column, row) +// } +// _ => unreachable!("View types should use `ArrowBytesViewMap`"), +// }; +// } + +// fn len(&self) -> usize { +// self.offsets.len() - 1 +// } + +// fn build(self: Box) -> ArrayRef { +// let Self { +// map: _, +// map_size: _, +// mut buffer, +// offsets, +// hashes_buffer: _, +// nulls, +// output_type, +// group_values_offset, +// } = *self; + +// let null_buffer = if nulls.is_empty() { +// None +// } else { +// // Only make a `NullBuffer` if there was a null value +// let num_values = offsets.len() - 1; +// let mut bool_builder = BooleanBufferBuilder::new(num_values); +// bool_builder.append_n(num_values, true); +// nulls.into_iter().for_each(|null_index| { +// bool_builder.set_bit(null_index, false); +// }); +// Some(NullBuffer::from(bool_builder.finish())) +// }; + +// // let nulls = null.map(|null_index| { +// // let num_values = offsets.len() - 1; +// // single_null_buffer(num_values, null_index) +// // }); +// // SAFETY: the offsets were constructed correctly in `insert_if_new` -- +// // monotonically increasing, overflows were checked. +// let offsets = unsafe { OffsetBuffer::new_unchecked(ScalarBuffer::from(offsets)) }; +// let values = buffer.finish(); + +// match output_type { +// OutputType::Binary => { +// // SAFETY: the offsets were constructed correctly +// Arc::new(unsafe { +// GenericBinaryArray::new_unchecked(offsets, values, null_buffer) +// }) +// } +// OutputType::Utf8 => { +// // SAFETY: +// // 1. the offsets were constructed safely +// // +// // 2. we asserted the input arrays were all the correct type and +// // thus since all the values that went in were valid (e.g. utf8) +// // so are all the values that come out +// let res = Arc::new(unsafe { +// GenericStringArray::new_unchecked(offsets, values, null_buffer) +// }); +// res +// } +// _ => unreachable!("View types should use `ArrowBytesViewMap`"), +// } +// } +// } + +// #[derive(Debug, PartialEq, Eq, Hash, Clone, Copy)] +// struct EntryWithPayload +// where +// O: OffsetSizeTrait, +// { +// /// hash of the value (stored to avoid recomputing it in hash table check) +// hash: u64, +// /// if len =< [`SHORT_VALUE_LEN`]: the data inlined +// /// if len > [`SHORT_VALUE_LEN`], the offset of where the data starts +// offset_or_inline: usize, +// /// length of the value, in bytes (use O here so we use only i32 for +// /// strings, rather 64 bit usize) +// len: O, +// } + +// impl EntryWithPayload +// where +// O: OffsetSizeTrait, +// { +// /// returns self.offset..self.offset + self.len +// #[inline(always)] +// fn range(&self) -> Range { +// self.offset_or_inline..self.offset_or_inline + self.len.as_usize() +// } +// } diff --git a/datafusion/physical-plan/src/aggregates/group_values/row_like.rs b/datafusion/physical-plan/src/aggregates/group_values/row_like.rs index 0d6137741e1a..6111b7749106 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/row_like.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/row_like.rs @@ -32,8 +32,7 @@ use datafusion_execution::memory_pool::proxy::{RawTableAllocExt, VecAllocExt}; use datafusion_expr::EmitTo; use datafusion_physical_expr::binary_map::OutputType; use datafusion_physical_expr_common::group_value_row::{ - ArrayEq, ByteGroupValueBuilder, ByteGroupValueBuilderNaive, - PrimitiveGroupValueBuilder, + ArrayEq, ByteGroupValueBuilderNaive, PrimitiveGroupValueBuilder, }; use hashbrown::raw::RawTable; @@ -172,18 +171,13 @@ impl GroupValues for GroupValuesRowLike { v.push(Box::new(b) as _) } &DataType::Utf8 => { - // let b = StringGroupValueBuilder::new(); - let b = ByteGroupValueBuilderNaive::::new( - &cols[i], - OutputType::Utf8, - ); + let b = + ByteGroupValueBuilderNaive::::new(OutputType::Utf8); v.push(Box::new(b) as _) } &DataType::LargeUtf8 => { - let b = ByteGroupValueBuilder::::new( - &cols[i], - OutputType::Utf8, - ); + let b = + ByteGroupValueBuilderNaive::::new(OutputType::Utf8); v.push(Box::new(b) as _) } dt => todo!("{dt} not impl"), From 9c61a8be9b1fe2fb5ce0bb908700ec5e7ef93953 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Sat, 14 Sep 2024 09:31:51 +0800 Subject: [PATCH 03/30] implement to Arrow's builder Signed-off-by: jayzhan211 --- benchmarks/queries/clickbench/queries.sql | 7 +- .../src/group_value_row.rs | 167 ++++++++++++++++++ .../src/aggregates/group_values/mod.rs | 4 +- .../src/aggregates/group_values/row_like.rs | 74 +++++--- datafusion/sqllogictest/test_files/test1.slt | 26 +-- datafusion/sqllogictest/test_files/test2.slt | 31 +++- 6 files changed, 263 insertions(+), 46 deletions(-) diff --git a/benchmarks/queries/clickbench/queries.sql b/benchmarks/queries/clickbench/queries.sql index 7ae48828de14..b3279f0e672b 100644 --- a/benchmarks/queries/clickbench/queries.sql +++ b/benchmarks/queries/clickbench/queries.sql @@ -35,4 +35,9 @@ SELECT "URL", COUNT(*) AS c FROM hits GROUP BY "URL" ORDER BY c DESC LIMIT 10; SELECT 1, "URL", COUNT(*) AS c FROM hits GROUP BY 1, "URL" ORDER BY c DESC LIMIT 10; SELECT "ClientIP", "ClientIP" - 1, "ClientIP" - 2, "ClientIP" - 3, COUNT(*) AS c FROM hits GROUP BY "ClientIP", "ClientIP" - 1, "ClientIP" - 2, "ClientIP" - 3 ORDER BY c DESC LIMIT 10; SELECT "URL", COUNT(*) AS PageViews FROM hits WHERE "CounterID" = 62 AND "EventDate"::INT::DATE >= '2013-07-01' AND "EventDate"::INT::DATE <= '2013-07-31' AND "DontCountHits" = 0 AND "IsRefresh" = 0 AND "URL" <> '' GROUP BY "URL" ORDER BY PageViews DESC LIMIT 10; -SELECT "Title", COUNT(*) AS PageViews FROM hits WHERE "CounterID" = 62 AND "EventDate"::INT::DATE >= '2013-07-01' AND "EventDate"::INT::DATE <= '2013-07-31' AND "DontCountHits" = 0 AND "IsRefresh" = 0 AND "Title" <> '' GROUP BY "Title" ORDER BY PageViews DESC LIMIT 10; \ No newline at end of file +SELECT "Title", COUNT(*) AS PageViews FROM hits WHERE "CounterID" = 62 AND "EventDate"::INT::DATE >= '2013-07-01' AND "EventDate"::INT::DATE <= '2013-07-31' AND "DontCountHits" = 0 AND "IsRefresh" = 0 AND "Title" <> '' GROUP BY "Title" ORDER BY PageViews DESC LIMIT 10; +SELECT "URL", COUNT(*) AS PageViews FROM hits WHERE "CounterID" = 62 AND "EventDate"::INT::DATE >= '2013-07-01' AND "EventDate"::INT::DATE <= '2013-07-31' AND "IsRefresh" = 0 AND "IsLink" <> 0 AND "IsDownload" = 0 GROUP BY "URL" ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; +SELECT "TraficSourceID", "SearchEngineID", "AdvEngineID", CASE WHEN ("SearchEngineID" = 0 AND "AdvEngineID" = 0) THEN "Referer" ELSE '' END AS Src, "URL" AS Dst, COUNT(*) AS PageViews FROM hits WHERE "CounterID" = 62 AND "EventDate"::INT::DATE >= '2013-07-01' AND "EventDate"::INT::DATE <= '2013-07-31' AND "IsRefresh" = 0 GROUP BY "TraficSourceID", "SearchEngineID", "AdvEngineID", Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; +SELECT "URLHash", "EventDate"::INT::DATE, COUNT(*) AS PageViews FROM hits WHERE "CounterID" = 62 AND "EventDate"::INT::DATE >= '2013-07-01' AND "EventDate"::INT::DATE <= '2013-07-31' AND "IsRefresh" = 0 AND "TraficSourceID" IN (-1, 6) AND "RefererHash" = 3594120000172545465 GROUP BY "URLHash", "EventDate"::INT::DATE ORDER BY PageViews DESC LIMIT 10 OFFSET 100; +SELECT "WindowClientWidth", "WindowClientHeight", COUNT(*) AS PageViews FROM hits WHERE "CounterID" = 62 AND "EventDate"::INT::DATE >= '2013-07-01' AND "EventDate"::INT::DATE <= '2013-07-31' AND "IsRefresh" = 0 AND "DontCountHits" = 0 AND "URLHash" = 2868770270353813622 GROUP BY "WindowClientWidth", "WindowClientHeight" ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; +SELECT DATE_TRUNC('minute', to_timestamp_seconds("EventTime")) AS M, COUNT(*) AS PageViews FROM hits WHERE "CounterID" = 62 AND "EventDate"::INT::DATE >= '2013-07-14' AND "EventDate"::INT::DATE <= '2013-07-15' AND "IsRefresh" = 0 AND "DontCountHits" = 0 GROUP BY DATE_TRUNC('minute', to_timestamp_seconds("EventTime")) ORDER BY DATE_TRUNC('minute', M) LIMIT 10 OFFSET 1000; \ No newline at end of file diff --git a/datafusion/physical-expr-common/src/group_value_row.rs b/datafusion/physical-expr-common/src/group_value_row.rs index 399323006950..db45648bc6cb 100644 --- a/datafusion/physical-expr-common/src/group_value_row.rs +++ b/datafusion/physical-expr-common/src/group_value_row.rs @@ -18,8 +18,11 @@ use arrow::array::BooleanBufferBuilder; use arrow::array::BufferBuilder; use arrow::array::GenericBinaryArray; +use arrow::array::GenericBinaryBuilder; use arrow::array::GenericStringArray; +use arrow::array::GenericStringBuilder; use arrow::array::OffsetSizeTrait; +use arrow::array::PrimitiveBuilder; use arrow::buffer::NullBuffer; use arrow::buffer::OffsetBuffer; use arrow::buffer::ScalarBuffer; @@ -37,6 +40,13 @@ use crate::binary_map::INITIAL_BUFFER_CAPACITY; use std::sync::Arc; +pub trait ArrayEqV2: Send + Sync { + fn equal_to(&self, lhs_row: usize, array: &ArrayRef, rhs_row: usize) -> bool; + fn append_val(&mut self, array: &ArrayRef, row: usize); + fn len(&self) -> usize; + fn build(&mut self) -> ArrayRef; +} + pub trait ArrayEq: Send + Sync { fn equal_to(&self, lhs_row: usize, array: &ArrayRef, rhs_row: usize) -> bool; fn append_val(&mut self, array: &ArrayRef, row: usize); @@ -44,6 +54,47 @@ pub trait ArrayEq: Send + Sync { fn build(self: Box) -> ArrayRef; } +impl ArrayEqV2 for PrimitiveBuilder +where + T: ArrowPrimitiveType, +{ + fn equal_to(&self, lhs_row: usize, array: &ArrayRef, rhs_row: usize) -> bool { + let arr = array.as_primitive::(); + + if let Some(nulls) = self.validity_slice() { + let null_slice_index = lhs_row / 8; + let null_bit_map_index = lhs_row % 8; + let is_elem_null = ((nulls[null_slice_index] >> null_bit_map_index) & 1) == 1; + if is_elem_null { + return arr.is_null(rhs_row); + } else if arr.is_null(rhs_row) { + return false; + } + } + + let elem = self.values_slice()[lhs_row]; + elem == arr.value(rhs_row) + } + + fn append_val(&mut self, array: &ArrayRef, row: usize) { + let arr = array.as_primitive::(); + if arr.is_null(row) { + self.append_null(); + } else { + let elem = arr.value(row); + self.append_value(elem); + } + } + + fn len(&self) -> usize { + self.values_slice().len() + } + + fn build(&mut self) -> ArrayRef { + Arc::new(self.finish()) + } +} + pub struct PrimitiveGroupValueBuilder(Vec>); impl PrimitiveGroupValueBuilder { @@ -127,6 +178,99 @@ impl ArrayEq for PrimitiveGroupValueBuilder { // } // } +impl ArrayEqV2 for GenericStringBuilder +where + O: OffsetSizeTrait, +{ + fn equal_to(&self, lhs_row: usize, array: &ArrayRef, rhs_row: usize) -> bool { + let arr = array.as_bytes::>(); + if let Some(nulls) = self.validity_slice() { + let null_slice_index = lhs_row / 8; + let null_bit_map_index = lhs_row % 8; + + let is_lhs_null = ((nulls[null_slice_index] >> null_bit_map_index) & 1) == 1; + if is_lhs_null { + return arr.is_null(rhs_row); + } else if arr.is_null(rhs_row) { + return false; + } + } + + let rhs_elem: &[u8] = arr.value(rhs_row).as_ref(); + let rhs_elem_len = arr.value_length(rhs_row).as_usize(); + assert_eq!(rhs_elem_len, rhs_elem.len()); + let l = O::as_usize(self.offsets_slice()[lhs_row]); + let r = O::as_usize(self.offsets_slice()[lhs_row + 1]); + let existing_elem = &self.values_slice()[l..r]; + existing_elem.len() == rhs_elem.len() && rhs_elem == existing_elem + } + + fn append_val(&mut self, array: &ArrayRef, row: usize) { + let arr = array.as_string::(); + if arr.is_null(row) { + self.append_null(); + return; + } + + let value = arr.value(row); + self.append_value(value); + } + + fn len(&self) -> usize { + self.offsets_slice().len() - 1 + } + + fn build(&mut self) -> ArrayRef { + Arc::new(self.finish()) + } +} + +impl ArrayEqV2 for GenericBinaryBuilder +where + O: OffsetSizeTrait, +{ + fn equal_to(&self, lhs_row: usize, array: &ArrayRef, rhs_row: usize) -> bool { + let arr = array.as_bytes::>(); + if let Some(nulls) = self.validity_slice() { + let null_slice_index = lhs_row / 8; + let null_bit_map_index = lhs_row % 8; + let is_lhs_null = ((nulls[null_slice_index] >> null_bit_map_index) & 1) == 1; + if is_lhs_null { + return arr.is_null(rhs_row); + } else if arr.is_null(rhs_row) { + return false; + } + } + + let rhs_elem: &[u8] = arr.value(rhs_row).as_ref(); + let rhs_elem_len = arr.value_length(rhs_row).as_usize(); + assert_eq!(rhs_elem_len, rhs_elem.len()); + let l = O::as_usize(self.offsets_slice()[lhs_row]); + let r = O::as_usize(self.offsets_slice()[lhs_row + 1]); + let existing_elem = &self.values_slice()[l..r]; + existing_elem.len() == rhs_elem.len() && rhs_elem == existing_elem + } + + fn append_val(&mut self, array: &ArrayRef, row: usize) { + let arr = array.as_binary::(); + if arr.is_null(row) { + self.append_null(); + return; + } + + let value: &[u8] = arr.value(row).as_ref(); + self.append_value(value); + } + + fn len(&self) -> usize { + self.values_slice().len() + } + + fn build(&mut self) -> ArrayRef { + Arc::new(self.finish()) + } +} + pub struct ByteGroupValueBuilderNaive where O: OffsetSizeTrait, @@ -622,3 +766,26 @@ where // self.offset_or_inline..self.offset_or_inline + self.len.as_usize() // } // } + +#[cfg(test)] +mod tests { + use arrow::{array::GenericByteBuilder, datatypes::GenericStringType}; + + #[test] + fn test123() { + let mut a = GenericByteBuilder::>::new(); + a.append_null(); + a.append_value("a"); + a.append_null(); + a.append_value("bc"); + a.append_value("def"); + a.append_null(); + + let s = a.validity_slice(); + println!("s: {:?}", s); + let v = a.values_slice(); + let o = a.offsets_slice(); + println!("v: {:?}", v); + println!("o: {:?}", o); + } +} diff --git a/datafusion/physical-plan/src/aggregates/group_values/mod.rs b/datafusion/physical-plan/src/aggregates/group_values/mod.rs index d78706926aa1..434a5c25370d 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/mod.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/mod.rs @@ -117,7 +117,9 @@ fn has_row_like_feature(data_type: &DataType) -> bool { | DataType::UInt32 | DataType::UInt64 | DataType::Utf8 - | DataType::LargeUtf8 => true, + | DataType::LargeUtf8 + | DataType::Date32 + | DataType::Date64 => true, _ => false, } } diff --git a/datafusion/physical-plan/src/aggregates/group_values/row_like.rs b/datafusion/physical-plan/src/aggregates/group_values/row_like.rs index 6111b7749106..96234e28e4a6 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/row_like.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/row_like.rs @@ -15,12 +15,15 @@ // specific language governing permissions and limitations // under the License. +use std::ops::DerefMut; + use crate::aggregates::group_values::GroupValues; use ahash::RandomState; +use arrow::array::{GenericStringBuilder, PrimitiveBuilder}; use arrow::compute::cast; use arrow::datatypes::{ - Float32Type, Float64Type, Int16Type, Int32Type, Int64Type, Int8Type, UInt16Type, - UInt32Type, UInt64Type, UInt8Type, + Date32Type, Date64Type, Float32Type, Float64Type, Int16Type, Int32Type, Int64Type, + Int8Type, UInt16Type, UInt32Type, UInt64Type, UInt8Type, }; use arrow::record_batch::RecordBatch; use arrow::row::{RowConverter, Rows, SortField}; @@ -30,12 +33,11 @@ use datafusion_common::hash_utils::create_hashes; use datafusion_common::{internal_err, DataFusionError, Result}; use datafusion_execution::memory_pool::proxy::{RawTableAllocExt, VecAllocExt}; use datafusion_expr::EmitTo; -use datafusion_physical_expr::binary_map::OutputType; -use datafusion_physical_expr_common::group_value_row::{ - ArrayEq, ByteGroupValueBuilderNaive, PrimitiveGroupValueBuilder, -}; +use datafusion_physical_expr_common::group_value_row::ArrayEqV2; use hashbrown::raw::RawTable; +pub(super) const INITIAL_CAPACITY: usize = 8 * 1024; + /// A [`GroupValues`] making use of [`Rows`] pub struct GroupValuesRowLike { /// The output schema @@ -75,7 +77,7 @@ pub struct GroupValuesRowLike { /// Random state for creating hashes random_state: RandomState, - group_values_v2: Option>>, + group_values_v2: Option>>, } impl GroupValuesRowLike { @@ -131,53 +133,65 @@ impl GroupValues for GroupValuesRowLike { for (i, f) in self.schema.fields().iter().enumerate() { match f.data_type() { &DataType::Int8 => { - let b = PrimitiveGroupValueBuilder::::new(); + let b = PrimitiveBuilder::::new(); v.push(Box::new(b) as _) } &DataType::Int16 => { - let b = PrimitiveGroupValueBuilder::::new(); + let b = PrimitiveBuilder::::new(); v.push(Box::new(b) as _) } &DataType::Int32 => { - let b = PrimitiveGroupValueBuilder::::new(); + let b = PrimitiveBuilder::::new(); v.push(Box::new(b) as _) } &DataType::Int64 => { - let b = PrimitiveGroupValueBuilder::::new(); + let b = PrimitiveBuilder::::new(); v.push(Box::new(b) as _) } &DataType::UInt8 => { - let b = PrimitiveGroupValueBuilder::::new(); + let b = PrimitiveBuilder::::new(); v.push(Box::new(b) as _) } &DataType::UInt16 => { - let b = PrimitiveGroupValueBuilder::::new(); + let b = PrimitiveBuilder::::new(); v.push(Box::new(b) as _) } &DataType::UInt32 => { - let b = PrimitiveGroupValueBuilder::::new(); + let b = PrimitiveBuilder::::new(); v.push(Box::new(b) as _) } &DataType::UInt64 => { - let b = PrimitiveGroupValueBuilder::::new(); + let b = PrimitiveBuilder::::new(); v.push(Box::new(b) as _) } &DataType::Float32 => { - let b = PrimitiveGroupValueBuilder::::new(); + let b = PrimitiveBuilder::::new(); v.push(Box::new(b) as _) } &DataType::Float64 => { - let b = PrimitiveGroupValueBuilder::::new(); + let b = PrimitiveBuilder::::new(); + v.push(Box::new(b) as _) + } + &DataType::Date32 => { + let b = PrimitiveBuilder::::new(); + v.push(Box::new(b) as _) + } + &DataType::Date64 => { + let b = PrimitiveBuilder::::new(); v.push(Box::new(b) as _) } &DataType::Utf8 => { - let b = - ByteGroupValueBuilderNaive::::new(OutputType::Utf8); + let b = GenericStringBuilder::::with_capacity( + INITIAL_CAPACITY, + INITIAL_CAPACITY, + ); v.push(Box::new(b) as _) } &DataType::LargeUtf8 => { - let b = - ByteGroupValueBuilderNaive::::new(OutputType::Utf8); + let b = GenericStringBuilder::::with_capacity( + INITIAL_CAPACITY, + INITIAL_CAPACITY, + ); v.push(Box::new(b) as _) } dt => todo!("{dt} not impl"), @@ -211,7 +225,7 @@ impl GroupValues for GroupValuesRowLike { // && group_rows.row(row) == group_values.row(*group_idx) fn compare_equal( - arry_eq: &dyn ArrayEq, + arry_eq: &dyn ArrayEqV2, lhs_row: usize, array: &ArrayRef, rhs_row: usize, @@ -311,7 +325,10 @@ impl GroupValues for GroupValuesRowLike { EmitTo::All => { let output = group_values_v2 .into_iter() - .map(|v| v.build()) + .map(|mut v| { + let p = v.deref_mut().build(); + p + }) .collect::>(); // let output = self.row_converter.convert_rows(&group_values)?; // group_values.clear(); @@ -324,8 +341,15 @@ impl GroupValues for GroupValuesRowLike { // println!("to first n"); let len = group_values_v2.len(); - let first_n: Vec> = group_values_v2.drain(..n).collect(); - let output = first_n.into_iter().map(|v| v.build()).collect::>(); + let first_n: Vec> = + group_values_v2.drain(..n).collect(); + let output = first_n + .into_iter() + .map(|mut v| { + let p = v.deref_mut().build(); + p + }) + .collect::>(); assert_eq!(len, group_values_v2.len() + n); self.group_values_v2 = Some(group_values_v2); diff --git a/datafusion/sqllogictest/test_files/test1.slt b/datafusion/sqllogictest/test_files/test1.slt index f977395f75c4..10a668ef08a8 100644 --- a/datafusion/sqllogictest/test_files/test1.slt +++ b/datafusion/sqllogictest/test_files/test1.slt @@ -4,19 +4,19 @@ CREATE EXTERNAL TABLE hits STORED AS PARQUET LOCATION '../../benchmarks/data/hits.parquet'; -# query ITI -# SELECT "UserID", "SearchPhrase", COUNT(*) FROM hits GROUP BY "UserID", "SearchPhrase" ORDER BY COUNT(*) DESC LIMIT 10; -# ---- -# 1313338681122956954 (empty) 29097 -# 1907779576417363396 (empty) 25333 -# 2305303682471783379 (empty) 10597 -# 7982623143712728547 (empty) 6669 -# 7280399273658728997 (empty) 6408 -# 1090981537032625727 (empty) 6196 -# 5730251990344211405 (empty) 6019 -# 6018350421959114808 (empty) 5990 -# 835157184735512989 (empty) 5209 -# 770542365400669095 (empty) 4906 +query ITI +SELECT "UserID", "SearchPhrase", COUNT(*) FROM hits GROUP BY "UserID", "SearchPhrase" ORDER BY COUNT(*) DESC LIMIT 10; +---- +1313338681122956954 (empty) 29097 +1907779576417363396 (empty) 25333 +2305303682471783379 (empty) 10597 +7982623143712728547 (empty) 6669 +7280399273658728997 (empty) 6408 +1090981537032625727 (empty) 6196 +5730251990344211405 (empty) 6019 +6018350421959114808 (empty) 5990 +835157184735512989 (empty) 5209 +770542365400669095 (empty) 4906 query IIIII SELECT "ClientIP", "ClientIP" - 1, "ClientIP" - 2, "ClientIP" - 3, COUNT(*) AS c FROM hits GROUP BY "ClientIP", "ClientIP" - 1, "ClientIP" - 2, "ClientIP" - 3 ORDER BY c DESC LIMIT 10; diff --git a/datafusion/sqllogictest/test_files/test2.slt b/datafusion/sqllogictest/test_files/test2.slt index e801264a0a77..dcad020d5bb0 100644 --- a/datafusion/sqllogictest/test_files/test2.slt +++ b/datafusion/sqllogictest/test_files/test2.slt @@ -1,9 +1,28 @@ statement ok -create table t (a int, b string, c int) as values (1, 'a', 3), (1, 'a', 3), (1, 'a', 3), (1, 'a', 3), (2, 'a', 1), (2, 'a', 1), (2, 'a', 1), (3, null, 2); +CREATE TABLE test( + i_item_desc VARCHAR, + d1_date DATE, + d2_date DATE, + d3_date DATE +) as VALUES + ('a','2022-12-12','2022-12-12','2022-12-12'), + ('b','2022-12-12','2022-12-11','2022-12-12'), + ('c','2022-12-12','2022-12-10','2022-12-12'), + ('d','2022-12-12','2022-12-9','2022-12-12'), + ('e','2022-12-12','2022-12-8','2022-12-12'), + ('f','2022-12-12','2022-12-7','2022-12-12'), + ('g','2022-12-12','2022-12-6','2022-12-12'), + ('h','2022-12-12','2022-12-5','2022-12-12') +; -query ITI -select a, b, count(*) from t group by a, b order by count(*) desc; +query DDI +select d1_date, d2_date, count(*) as c from test group by d2_date, d1_date order by c desc; ---- -1 a 4 -2 a 3 -3 NULL 1 +2022-12-12 2022-12-09 1 +2022-12-12 2022-12-06 1 +2022-12-12 2022-12-05 1 +2022-12-12 2022-12-12 1 +2022-12-12 2022-12-11 1 +2022-12-12 2022-12-07 1 +2022-12-12 2022-12-10 1 +2022-12-12 2022-12-08 1 From f20780e88187c7a911e52cf5a7c3668f36a37134 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Sat, 14 Sep 2024 11:18:58 +0800 Subject: [PATCH 04/30] cleanup Signed-off-by: jayzhan211 --- .../src/group_value_row.rs | 716 +++--------------- .../src/aggregates/group_values/row_like.rs | 37 +- datafusion/sqllogictest/test_files/test2.slt | 43 +- 3 files changed, 119 insertions(+), 677 deletions(-) diff --git a/datafusion/physical-expr-common/src/group_value_row.rs b/datafusion/physical-expr-common/src/group_value_row.rs index db45648bc6cb..30ad14dad5fe 100644 --- a/datafusion/physical-expr-common/src/group_value_row.rs +++ b/datafusion/physical-expr-common/src/group_value_row.rs @@ -15,57 +15,37 @@ // specific language governing permissions and limitations // under the License. -use arrow::array::BooleanBufferBuilder; -use arrow::array::BufferBuilder; -use arrow::array::GenericBinaryArray; use arrow::array::GenericBinaryBuilder; -use arrow::array::GenericStringArray; use arrow::array::GenericStringBuilder; use arrow::array::OffsetSizeTrait; use arrow::array::PrimitiveBuilder; -use arrow::buffer::NullBuffer; -use arrow::buffer::OffsetBuffer; -use arrow::buffer::ScalarBuffer; -use arrow::datatypes::ArrowNativeType; -use arrow::datatypes::DataType; +use arrow::array::{Array, ArrayRef, ArrowPrimitiveType, AsArray}; use arrow::datatypes::GenericBinaryType; use arrow::datatypes::GenericStringType; -use arrow::{ - array::{Array, ArrayRef, ArrowPrimitiveType, AsArray, PrimitiveArray}, - datatypes::ByteArrayType, -}; - -use crate::binary_map::OutputType; -use crate::binary_map::INITIAL_BUFFER_CAPACITY; use std::sync::Arc; -pub trait ArrayEqV2: Send + Sync { +pub trait ArrayRowEq: Send + Sync { fn equal_to(&self, lhs_row: usize, array: &ArrayRef, rhs_row: usize) -> bool; fn append_val(&mut self, array: &ArrayRef, row: usize); fn len(&self) -> usize; + // take n elements as ArrayRef and adjusted the underlying buffer + fn take_n(&mut self, n: usize) -> ArrayRef; fn build(&mut self) -> ArrayRef; } -pub trait ArrayEq: Send + Sync { - fn equal_to(&self, lhs_row: usize, array: &ArrayRef, rhs_row: usize) -> bool; - fn append_val(&mut self, array: &ArrayRef, row: usize); - fn len(&self) -> usize; - fn build(self: Box) -> ArrayRef; -} - -impl ArrayEqV2 for PrimitiveBuilder +impl ArrayRowEq for PrimitiveBuilder where T: ArrowPrimitiveType, { fn equal_to(&self, lhs_row: usize, array: &ArrayRef, rhs_row: usize) -> bool { let arr = array.as_primitive::(); - if let Some(nulls) = self.validity_slice() { - let null_slice_index = lhs_row / 8; - let null_bit_map_index = lhs_row % 8; - let is_elem_null = ((nulls[null_slice_index] >> null_bit_map_index) & 1) == 1; - if is_elem_null { + if let Some(validity_slice) = self.validity_slice() { + let validity_slice_index = lhs_row / 8; + let bit_map_index = lhs_row % 8; + let is_lhs_null = ((validity_slice[validity_slice_index] >> bit_map_index) & 1) == 0; + if is_lhs_null { return arr.is_null(rhs_row); } else if arr.is_null(rhs_row) { return false; @@ -86,109 +66,60 @@ where } } - fn len(&self) -> usize { - self.values_slice().len() - } + fn take_n(&mut self, n: usize) -> ArrayRef { + todo!("") - fn build(&mut self) -> ArrayRef { - Arc::new(self.finish()) - } -} + // let num_remaining = self.values_slice().len() - n; + // assert!(num_remaining >= 0); -pub struct PrimitiveGroupValueBuilder(Vec>); + // let mut builder = PrimitiveBuilder::::new(); + // let vs = self.values_slice(); + // builder.append_slice(vs); -impl PrimitiveGroupValueBuilder { - pub fn new() -> Self { - Self(vec![]) - } -} + // let mut values_left = vec![T::default_value(); num_remaining]; + // let mut null_buffer_left = NullBuffer::new_null(num_remaining); -impl ArrayEq for PrimitiveGroupValueBuilder { - fn equal_to(&self, lhs_row: usize, array: &ArrayRef, rhs_row: usize) -> bool { - let elem = self.0[lhs_row]; - let arr = array.as_primitive::(); - let is_rhs_null = arr.is_null(rhs_row); - if elem.is_none() && is_rhs_null { - true - } else if elem.is_some() && !is_rhs_null { - elem.unwrap() == arr.value(rhs_row) - } else { - false - } - } + // let null_buffer_left = self.validity_slice(); + + // let len = self.len(); + // let nulls = self.null_buffer_builder.finish(); + // let builder = ArrayData::builder(self.data_type.clone()) + // .len(len) + // .add_buffer(self.values_builder.finish()) + // .nulls(nulls); + + // let array_data = unsafe { builder.build_unchecked() }; + // PrimitiveArray::::from(array_data) + + // let output = self.finish(); + + // let mut values_buffer = MutableBuffer::new(num_remaining); + // values_buffer.extend_from_slice(&values_left); + // let mut null_buffer = MutableBuffer::new_null(num_remaining); + // null_buffer.extend_from_slice(items) + - fn append_val(&mut self, array: &ArrayRef, row: usize) { - let arr = array.as_primitive::(); - if arr.is_null(row) { - self.0.push(None) - } else { - let elem = arr.value(row); - self.0.push(Some(elem)) - } } fn len(&self) -> usize { - self.0.len() + self.values_slice().len() } - fn build(self: Box) -> ArrayRef { - Arc::new(PrimitiveArray::::from_iter(self.0)) + fn build(&mut self) -> ArrayRef { + Arc::new(self.finish()) } } -// pub struct StringGroupValueBuilder(Vec>); - -// impl StringGroupValueBuilder { -// pub fn new() -> Self { -// Self(vec![]) -// } -// } - -// impl ArrayEq for StringGroupValueBuilder { -// fn equal_to(&self, lhs_row: usize, array: &ArrayRef, rhs_row: usize) -> bool { -// let elem = &self.0[lhs_row]; -// let arr = array.as_string::(); -// let is_rhs_null = arr.is_null(rhs_row); -// if elem.is_none() && is_rhs_null { -// true -// } else if elem.is_some() && !is_rhs_null { -// let e = elem.as_ref().unwrap(); -// e.as_str() == arr.value(rhs_row) -// } else { -// false -// } -// } - -// fn append_val(&mut self, array: &ArrayRef, row: usize) { -// let arr = array.as_string::(); -// if arr.is_null(row) { -// self.0.push(None) -// } else { -// let elem = arr.value(row); -// self.0.push(Some(elem.to_string())) -// } -// } - -// fn len(&self) -> usize { -// self.0.len() -// } - -// fn build(self: Box) -> ArrayRef { -// Arc::new(StringArray::from_iter(self.0)) -// } -// } - -impl ArrayEqV2 for GenericStringBuilder +impl ArrayRowEq for GenericStringBuilder where O: OffsetSizeTrait, { fn equal_to(&self, lhs_row: usize, array: &ArrayRef, rhs_row: usize) -> bool { let arr = array.as_bytes::>(); - if let Some(nulls) = self.validity_slice() { - let null_slice_index = lhs_row / 8; - let null_bit_map_index = lhs_row % 8; - - let is_lhs_null = ((nulls[null_slice_index] >> null_bit_map_index) & 1) == 1; + if let Some(validity_slice) = self.validity_slice() { + let validity_slice_index = lhs_row / 8; + let bit_map_index = lhs_row % 8; + let is_lhs_null = ((validity_slice[validity_slice_index] >> bit_map_index) & 1) == 0; if is_lhs_null { return arr.is_null(rhs_row); } else if arr.is_null(rhs_row) { @@ -216,6 +147,10 @@ where self.append_value(value); } + fn take_n(&mut self, n: usize) -> ArrayRef { + todo!("") + } + fn len(&self) -> usize { self.offsets_slice().len() - 1 } @@ -225,16 +160,16 @@ where } } -impl ArrayEqV2 for GenericBinaryBuilder +impl ArrayRowEq for GenericBinaryBuilder where O: OffsetSizeTrait, { fn equal_to(&self, lhs_row: usize, array: &ArrayRef, rhs_row: usize) -> bool { let arr = array.as_bytes::>(); - if let Some(nulls) = self.validity_slice() { - let null_slice_index = lhs_row / 8; - let null_bit_map_index = lhs_row % 8; - let is_lhs_null = ((nulls[null_slice_index] >> null_bit_map_index) & 1) == 1; + if let Some(validity_slice) = self.validity_slice() { + let validity_slice_index = lhs_row / 8; + let bit_map_index = lhs_row % 8; + let is_lhs_null = ((validity_slice[validity_slice_index] >> bit_map_index) & 1) == 0; if is_lhs_null { return arr.is_null(rhs_row); } else if arr.is_null(rhs_row) { @@ -262,6 +197,10 @@ where self.append_value(value); } + fn take_n(&mut self, n: usize) -> ArrayRef { + todo!("") + } + fn len(&self) -> usize { self.values_slice().len() } @@ -271,521 +210,48 @@ where } } -pub struct ByteGroupValueBuilderNaive -where - O: OffsetSizeTrait, -{ - output_type: OutputType, - buffer: BufferBuilder, - /// Offsets into `buffer` for each distinct value. These offsets as used - /// directly to create the final `GenericBinaryArray`. The `i`th string is - /// stored in the range `offsets[i]..offsets[i+1]` in `buffer`. Null values - /// are stored as a zero length string. - offsets: Vec, - /// Null indexes in offsets - nulls: Vec, -} - -impl ByteGroupValueBuilderNaive -where - O: OffsetSizeTrait, -{ - pub fn new(output_type: OutputType) -> Self { - Self { - output_type, - buffer: BufferBuilder::new(INITIAL_BUFFER_CAPACITY), - offsets: vec![O::default()], - nulls: vec![], - } - } - - fn append_val_inner(&mut self, array: &ArrayRef, row: usize) - where - B: ByteArrayType, - { - let arr = array.as_bytes::(); - if arr.is_null(row) { - self.nulls.push(self.len()); - // nulls need a zero length in the offset buffer - let offset = self.buffer.len(); - self.offsets.push(O::usize_as(offset)); - return; - } - - let value: &[u8] = arr.value(row).as_ref(); - self.buffer.append_slice(value); - self.offsets.push(O::usize_as(self.buffer.len())); - } - - fn equal_to_inner(&self, lhs_row: usize, array: &ArrayRef, rhs_row: usize) -> bool - where - B: ByteArrayType, - { - // Handle nulls - let is_lhs_null = self.nulls.iter().any(|null_idx| *null_idx == lhs_row); - let arr = array.as_bytes::(); - if is_lhs_null { - return arr.is_null(rhs_row); - } else if arr.is_null(rhs_row) { - return false; - } - - let arr = array.as_bytes::(); - let rhs_elem: &[u8] = arr.value(rhs_row).as_ref(); - let rhs_elem_len = arr.value_length(rhs_row).as_usize(); - assert_eq!(rhs_elem_len, rhs_elem.len()); - let l = self.offsets[lhs_row].as_usize(); - let r = self.offsets[lhs_row + 1].as_usize(); - let existing_elem = unsafe { self.buffer.as_slice().get_unchecked(l..r) }; - existing_elem.len() == rhs_elem.len() && rhs_elem == existing_elem - } -} - -impl ArrayEq for ByteGroupValueBuilderNaive -where - O: OffsetSizeTrait, -{ - fn equal_to(&self, lhs_row: usize, column: &ArrayRef, rhs_row: usize) -> bool { - // Sanity array type - match self.output_type { - OutputType::Binary => { - assert!(matches!( - column.data_type(), - DataType::Binary | DataType::LargeBinary - )); - self.equal_to_inner::>(lhs_row, column, rhs_row) - } - OutputType::Utf8 => { - assert!(matches!( - column.data_type(), - DataType::Utf8 | DataType::LargeUtf8 - )); - self.equal_to_inner::>(lhs_row, column, rhs_row) - } - _ => unreachable!("View types should use `ArrowBytesViewMap`"), - } - } - - fn append_val(&mut self, column: &ArrayRef, row: usize) { - // Sanity array type - match self.output_type { - OutputType::Binary => { - assert!(matches!( - column.data_type(), - DataType::Binary | DataType::LargeBinary - )); - self.append_val_inner::>(column, row) - } - OutputType::Utf8 => { - assert!(matches!( - column.data_type(), - DataType::Utf8 | DataType::LargeUtf8 - )); - self.append_val_inner::>(column, row) - } - _ => unreachable!("View types should use `ArrowBytesViewMap`"), - }; - } +#[cfg(test)] +mod tests { + use arrow::{array::{GenericStringBuilder, PrimitiveBuilder}, datatypes::Int32Type}; - fn len(&self) -> usize { - self.offsets.len() - 1 - } + #[test] + fn te1() { + let mut a = PrimitiveBuilder::::new(); + a.append_value(1); + a.append_value(2); + a.append_null(); + a.append_null(); + a.append_value(2); - fn build(self: Box) -> ArrayRef { - let Self { - output_type, - mut buffer, - offsets, - nulls, - } = *self; + let s = a.values_slice(); + let v = a.validity_slice(); + println!("s: {:?}", s); + println!("v: {:?}", v); - let null_buffer = if nulls.is_empty() { - None - } else { - // Only make a `NullBuffer` if there was a null value - let num_values = offsets.len() - 1; - let mut bool_builder = BooleanBufferBuilder::new(num_values); - bool_builder.append_n(num_values, true); - nulls.into_iter().for_each(|null_index| { - bool_builder.set_bit(null_index, false); - }); - Some(NullBuffer::from(bool_builder.finish())) - }; - - // SAFETY: the offsets were constructed correctly in `insert_if_new` -- - // monotonically increasing, overflows were checked. - let offsets = unsafe { OffsetBuffer::new_unchecked(ScalarBuffer::from(offsets)) }; - let values = buffer.finish(); - - match output_type { - OutputType::Binary => { - // SAFETY: the offsets were constructed correctly - Arc::new(unsafe { - GenericBinaryArray::new_unchecked(offsets, values, null_buffer) - }) - } - OutputType::Utf8 => { - // SAFETY: - // 1. the offsets were constructed safely - // - // 2. we asserted the input arrays were all the correct type and - // thus since all the values that went in were valid (e.g. utf8) - // so are all the values that come out - let res = Arc::new(unsafe { - GenericStringArray::new_unchecked(offsets, values, null_buffer) - }); - res - } - _ => unreachable!("View types should use `ArrowBytesViewMap`"), - } } -} - -// pub struct ByteGroupValueBuilder -// where -// O: OffsetSizeTrait, -// { -// output_type: OutputType, -// /// Underlying hash set for each distinct value -// map: hashbrown::raw::RawTable>, -// /// Total size of the map in bytes -// map_size: usize, -// buffer: BufferBuilder, -// /// Offsets into `buffer` for each distinct value. These offsets as used -// /// directly to create the final `GenericBinaryArray`. The `i`th string is -// /// stored in the range `offsets[i]..offsets[i+1]` in `buffer`. Null values -// /// are stored as a zero length string. -// offsets: Vec, -// /// buffer that stores hash values (reused across batches to save allocations) -// hashes_buffer: Vec, -// /// Null indexes in offsets -// nulls: Vec, -// // Store the offset + len for group values -// group_values_offset: Vec>, -// } - -// impl ByteGroupValueBuilder -// where -// O: OffsetSizeTrait, -// { -// pub fn new(array: &ArrayRef, output_type: OutputType) -> Self { -// let n_rows = array.len(); -// let random_state = RandomState::new(); -// let mut hashes_buffer = vec![]; -// let batch_hashes = &mut hashes_buffer; -// batch_hashes.clear(); -// batch_hashes.resize(n_rows, 0); -// create_hashes(&[array.to_owned()], &random_state, batch_hashes) -// // hash is supported for all types and create_hashes only -// // returns errors for unsupported types -// .unwrap(); - -// Self { -// output_type, -// map: hashbrown::raw::RawTable::with_capacity(INITIAL_MAP_CAPACITY), -// map_size: 0, -// buffer: BufferBuilder::new(INITIAL_BUFFER_CAPACITY), -// offsets: vec![O::default()], -// hashes_buffer, -// nulls: vec![], -// group_values_offset: vec![], -// } -// } - -// fn append_val_inner(&mut self, array: &ArrayRef, row: usize) -// where -// B: ByteArrayType, -// { -// let arr = array.as_bytes::(); -// if arr.is_null(row) { -// self.nulls.push(self.offsets.len() - 1); -// // nulls need a zero length in the offset buffer -// let offset = self.buffer.len(); -// self.offsets.push(O::usize_as(offset)); -// return; -// } - -// let hash = self.hashes_buffer[row]; -// let value: &[u8] = arr.value(row).as_ref(); -// let value_len = O::usize_as(value.len()); - -// if value.len() <= SHORT_VALUE_LEN { -// let inline = value.iter().fold(0usize, |acc, &x| acc << 8 | x as usize); -// // is value is already present in the set? -// let entry = self.map.get(hash, |header| { -// // compare value if hashes match -// if header.len != value_len { -// return false; -// } -// // value is stored inline so no need to consult buffer -// // (this is the "small string optimization") -// inline == header.offset_or_inline -// }); - -// // Put the small values into buffer and offsets so it appears -// // the output array, but store the actual bytes inline for -// // comparison -// self.buffer.append_slice(value); -// self.offsets.push(O::usize_as(self.buffer.len())); -// if let Some(entry) = entry { -// } -// // if no existing entry, make a new one -// else { -// // let payload = make_payload_fn(Some(value)); -// let new_header = EntryWithPayload { -// hash, -// len: value_len, -// offset_or_inline: inline, -// }; -// self.map.insert_accounted( -// new_header, -// |header| header.hash, -// &mut self.map_size, -// ); -// } -// } else { -// // Check if the value is already present in the set -// let entry = self.map.get_mut(hash, |header| { -// // compare value if hashes match -// if header.len != value_len { -// return false; -// } -// // Need to compare the bytes in the buffer -// // SAFETY: buffer is only appended to, and we correctly inserted values and offsets -// let existing_value = -// unsafe { self.buffer.as_slice().get_unchecked(header.range()) }; -// value == existing_value -// }); - -// let offset = self.buffer.len(); // offset of start for data -// self.buffer.append_slice(value); -// self.offsets.push(O::usize_as(self.buffer.len())); - -// if let Some(entry) = entry { -// } -// // if no existing entry, make a new header in map for equality check -// else { -// let new_header = EntryWithPayload { -// hash, -// len: value_len, -// offset_or_inline: offset, -// }; -// self.map.insert_accounted( -// new_header, -// |header| header.hash, -// &mut self.map_size, -// ); -// } -// }; -// } - -// fn equal_to_inner(&self, lhs_row: usize, array: &ArrayRef, rhs_row: usize) -> bool -// where -// B: ByteArrayType, -// { -// // Handle nulls -// let is_lhs_null = self.nulls.iter().any(|null_idx| *null_idx == lhs_row); -// let arr = array.as_bytes::(); -// if is_lhs_null { -// return arr.is_null(rhs_row); -// } else if arr.is_null(rhs_row) { -// return false; -// } - -// let hash = self.hashes_buffer[rhs_row]; -// let arr = array.as_bytes::(); -// let rhs_elem: &[u8] = arr.value(rhs_row).as_ref(); -// let rhs_elem_len = O::usize_as(rhs_elem.len()); -// if rhs_elem.len() <= SHORT_VALUE_LEN { -// let inline = rhs_elem -// .iter() -// .fold(0usize, |acc, &x| acc << 8 | x as usize); -// // is value is already present in the set? -// let entry = self.map.get(hash, |header| { -// // compare value if hashes match -// if header.len != rhs_elem_len { -// return false; -// } -// // value is stored inline so no need to consult buffer -// // (this is the "small string optimization") -// inline == header.offset_or_inline -// }); -// entry.is_some() -// } else { -// // Check if the value is already present in the set -// let entry = self.map.get(hash, |header| { -// // if header.hash != hash { -// // return false; -// // } - -// // compare value if hashes match -// if header.len != rhs_elem_len { -// return false; -// } -// // Need to compare the bytes in the buffer -// // SAFETY: buffer is only appended to, and we correctly inserted values and offsets -// let existing_elem = -// unsafe { self.buffer.as_slice().get_unchecked(header.range()) }; -// rhs_elem == existing_elem -// }); -// entry.is_some() -// } -// } -// } - -// impl ArrayEq for ByteGroupValueBuilder -// where -// O: OffsetSizeTrait, -// { -// fn equal_to(&self, lhs_row: usize, column: &ArrayRef, rhs_row: usize) -> bool { -// // Sanity array type -// match self.output_type { -// OutputType::Binary => { -// assert!(matches!( -// column.data_type(), -// DataType::Binary | DataType::LargeBinary -// )); -// self.equal_to_inner::>(lhs_row, column, rhs_row) -// } -// OutputType::Utf8 => { -// assert!(matches!( -// column.data_type(), -// DataType::Utf8 | DataType::LargeUtf8 -// )); -// self.equal_to_inner::>(lhs_row, column, rhs_row) -// } -// _ => unreachable!("View types should use `ArrowBytesViewMap`"), -// } -// } - -// fn append_val(&mut self, column: &ArrayRef, row: usize) { -// // Sanity array type -// match self.output_type { -// OutputType::Binary => { -// assert!(matches!( -// column.data_type(), -// DataType::Binary | DataType::LargeBinary -// )); -// self.append_val_inner::>(column, row) -// } -// OutputType::Utf8 => { -// assert!(matches!( -// column.data_type(), -// DataType::Utf8 | DataType::LargeUtf8 -// )); -// self.append_val_inner::>(column, row) -// } -// _ => unreachable!("View types should use `ArrowBytesViewMap`"), -// }; -// } - -// fn len(&self) -> usize { -// self.offsets.len() - 1 -// } - -// fn build(self: Box) -> ArrayRef { -// let Self { -// map: _, -// map_size: _, -// mut buffer, -// offsets, -// hashes_buffer: _, -// nulls, -// output_type, -// group_values_offset, -// } = *self; - -// let null_buffer = if nulls.is_empty() { -// None -// } else { -// // Only make a `NullBuffer` if there was a null value -// let num_values = offsets.len() - 1; -// let mut bool_builder = BooleanBufferBuilder::new(num_values); -// bool_builder.append_n(num_values, true); -// nulls.into_iter().for_each(|null_index| { -// bool_builder.set_bit(null_index, false); -// }); -// Some(NullBuffer::from(bool_builder.finish())) -// }; - -// // let nulls = null.map(|null_index| { -// // let num_values = offsets.len() - 1; -// // single_null_buffer(num_values, null_index) -// // }); -// // SAFETY: the offsets were constructed correctly in `insert_if_new` -- -// // monotonically increasing, overflows were checked. -// let offsets = unsafe { OffsetBuffer::new_unchecked(ScalarBuffer::from(offsets)) }; -// let values = buffer.finish(); - -// match output_type { -// OutputType::Binary => { -// // SAFETY: the offsets were constructed correctly -// Arc::new(unsafe { -// GenericBinaryArray::new_unchecked(offsets, values, null_buffer) -// }) -// } -// OutputType::Utf8 => { -// // SAFETY: -// // 1. the offsets were constructed safely -// // -// // 2. we asserted the input arrays were all the correct type and -// // thus since all the values that went in were valid (e.g. utf8) -// // so are all the values that come out -// let res = Arc::new(unsafe { -// GenericStringArray::new_unchecked(offsets, values, null_buffer) -// }); -// res -// } -// _ => unreachable!("View types should use `ArrowBytesViewMap`"), -// } -// } -// } - -// #[derive(Debug, PartialEq, Eq, Hash, Clone, Copy)] -// struct EntryWithPayload -// where -// O: OffsetSizeTrait, -// { -// /// hash of the value (stored to avoid recomputing it in hash table check) -// hash: u64, -// /// if len =< [`SHORT_VALUE_LEN`]: the data inlined -// /// if len > [`SHORT_VALUE_LEN`], the offset of where the data starts -// offset_or_inline: usize, -// /// length of the value, in bytes (use O here so we use only i32 for -// /// strings, rather 64 bit usize) -// len: O, -// } - -// impl EntryWithPayload -// where -// O: OffsetSizeTrait, -// { -// /// returns self.offset..self.offset + self.len -// #[inline(always)] -// fn range(&self) -> Range { -// self.offset_or_inline..self.offset_or_inline + self.len.as_usize() -// } -// } - -#[cfg(test)] -mod tests { - use arrow::{array::GenericByteBuilder, datatypes::GenericStringType}; #[test] fn test123() { - let mut a = GenericByteBuilder::>::new(); + let mut a = GenericStringBuilder::::new(); a.append_null(); - a.append_value("a"); + let p = a.offsets_slice(); + println!("p: {:?}", p); + let s = a.validity_slice(); + println!("s: {:?}", s); a.append_null(); - a.append_value("bc"); - a.append_value("def"); + let p = a.offsets_slice(); + println!("p: {:?}", p); + let s = a.validity_slice(); + println!("s: {:?}", s); + a.append_value("12"); + let p = a.offsets_slice(); + println!("p: {:?}", p); + let s = a.validity_slice(); + println!("s: {:?}", s); a.append_null(); - + let p = a.offsets_slice(); + println!("p: {:?}", p); let s = a.validity_slice(); println!("s: {:?}", s); - let v = a.values_slice(); - let o = a.offsets_slice(); - println!("v: {:?}", v); - println!("o: {:?}", o); } -} +} \ No newline at end of file diff --git a/datafusion/physical-plan/src/aggregates/group_values/row_like.rs b/datafusion/physical-plan/src/aggregates/group_values/row_like.rs index 96234e28e4a6..3e270f3f0d3b 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/row_like.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/row_like.rs @@ -33,7 +33,7 @@ use datafusion_common::hash_utils::create_hashes; use datafusion_common::{internal_err, DataFusionError, Result}; use datafusion_execution::memory_pool::proxy::{RawTableAllocExt, VecAllocExt}; use datafusion_expr::EmitTo; -use datafusion_physical_expr_common::group_value_row::ArrayEqV2; +use datafusion_physical_expr_common::group_value_row::ArrayRowEq; use hashbrown::raw::RawTable; pub(super) const INITIAL_CAPACITY: usize = 8 * 1024; @@ -77,7 +77,7 @@ pub struct GroupValuesRowLike { /// Random state for creating hashes random_state: RandomState, - group_values_v2: Option>>, + group_values_v2: Option>>, } impl GroupValuesRowLike { @@ -112,17 +112,6 @@ impl GroupValuesRowLike { impl GroupValues for GroupValuesRowLike { fn intern(&mut self, cols: &[ArrayRef], groups: &mut Vec) -> Result<()> { - // Convert the group keys into the row format - // let group_rows = &mut self.rows_buffer; - // group_rows.clear(); - // self.row_converter.append(group_rows, cols)?; - // let n_rows = group_rows.num_rows(); - - // let mut group_values = match self.group_values.take() { - // Some(group_values) => group_values, - // None => self.row_converter.empty_rows(0, 0), - // }; - let n_rows = cols[0].len(); let mut group_values_v2 = match self.group_values_v2.take() { Some(group_values) => group_values, @@ -130,7 +119,7 @@ impl GroupValues for GroupValuesRowLike { let len = cols.len(); let mut v = Vec::with_capacity(len); // Move to `try_new` - for (i, f) in self.schema.fields().iter().enumerate() { + for f in self.schema.fields().iter() { match f.data_type() { &DataType::Int8 => { let b = PrimitiveBuilder::::new(); @@ -225,7 +214,7 @@ impl GroupValues for GroupValuesRowLike { // && group_rows.row(row) == group_values.row(*group_idx) fn compare_equal( - arry_eq: &dyn ArrayEqV2, + arry_eq: &dyn ArrayRowEq, lhs_row: usize, array: &ArrayRef, rhs_row: usize, @@ -310,24 +299,19 @@ impl GroupValues for GroupValuesRowLike { } fn emit(&mut self, emit_to: EmitTo) -> Result> { - // println!("emit"); - // let mut group_values = self - // .group_values - // .take() - // .expect("Can not emit from empty rows"); - let mut group_values_v2 = self .group_values_v2 .take() .expect("Can not emit from empty rows"); + // println!("emit_to: {:?}", emit_to); + let mut output = match emit_to { EmitTo::All => { let output = group_values_v2 - .into_iter() - .map(|mut v| { - let p = v.deref_mut().build(); - p + .iter_mut() + .map(|v| { + v.deref_mut().build() }) .collect::>(); // let output = self.row_converter.convert_rows(&group_values)?; @@ -341,7 +325,7 @@ impl GroupValues for GroupValuesRowLike { // println!("to first n"); let len = group_values_v2.len(); - let first_n: Vec> = + let first_n: Vec> = group_values_v2.drain(..n).collect(); let output = first_n .into_iter() @@ -394,6 +378,7 @@ impl GroupValues for GroupValuesRowLike { } // self.group_values = Some(group_values); + // println!("output: {:?}", output); Ok(output) } diff --git a/datafusion/sqllogictest/test_files/test2.slt b/datafusion/sqllogictest/test_files/test2.slt index dcad020d5bb0..572cf25b539a 100644 --- a/datafusion/sqllogictest/test_files/test2.slt +++ b/datafusion/sqllogictest/test_files/test2.slt @@ -1,28 +1,19 @@ -statement ok -CREATE TABLE test( - i_item_desc VARCHAR, - d1_date DATE, - d2_date DATE, - d3_date DATE -) as VALUES - ('a','2022-12-12','2022-12-12','2022-12-12'), - ('b','2022-12-12','2022-12-11','2022-12-12'), - ('c','2022-12-12','2022-12-10','2022-12-12'), - ('d','2022-12-12','2022-12-9','2022-12-12'), - ('e','2022-12-12','2022-12-8','2022-12-12'), - ('f','2022-12-12','2022-12-7','2022-12-12'), - ('g','2022-12-12','2022-12-6','2022-12-12'), - ('h','2022-12-12','2022-12-5','2022-12-12') -; +query ok +create table multi_null_data(c1 int, c2 string) as values + (0, NULL), + (0, NULL), + (3, 'foo'), + (NULL, NULL), + (NULL, 'bar'), + (3, 'foo'), + (0, NULL), + (NULL, 'bar'), + (3, 'foo'); -query DDI -select d1_date, d2_date, count(*) as c from test group by d2_date, d1_date order by c desc; +query IIT rowsort +SELECT COUNT(*), c1, c2 FROM multi_null_data GROUP BY c1, c2 ---- -2022-12-12 2022-12-09 1 -2022-12-12 2022-12-06 1 -2022-12-12 2022-12-05 1 -2022-12-12 2022-12-12 1 -2022-12-12 2022-12-11 1 -2022-12-12 2022-12-07 1 -2022-12-12 2022-12-10 1 -2022-12-12 2022-12-08 1 +1 NULL NULL +2 NULL bar +3 0 NULL +3 3 foo \ No newline at end of file From d3f54edde087d66b700db1b98469b95aa624df9d Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Sun, 15 Sep 2024 22:08:19 +0800 Subject: [PATCH 05/30] switch back to vector Signed-off-by: jayzhan211 --- .../src/group_value_row.rs | 294 ++++++++++++++++++ .../src/aggregates/group_values/row_like.rs | 70 ++--- 2 files changed, 318 insertions(+), 46 deletions(-) diff --git a/datafusion/physical-expr-common/src/group_value_row.rs b/datafusion/physical-expr-common/src/group_value_row.rs index 30ad14dad5fe..e5c567833b50 100644 --- a/datafusion/physical-expr-common/src/group_value_row.rs +++ b/datafusion/physical-expr-common/src/group_value_row.rs @@ -15,16 +15,310 @@ // specific language governing permissions and limitations // under the License. +use arrow::array::BooleanBufferBuilder; +use arrow::array::GenericBinaryArray; use arrow::array::GenericBinaryBuilder; +use arrow::array::GenericStringArray; use arrow::array::GenericStringBuilder; use arrow::array::OffsetSizeTrait; +use arrow::array::PrimitiveArray; use arrow::array::PrimitiveBuilder; use arrow::array::{Array, ArrayRef, ArrowPrimitiveType, AsArray}; +use arrow::buffer::Buffer; +use arrow::buffer::NullBuffer; +use arrow::buffer::OffsetBuffer; +use arrow::buffer::ScalarBuffer; +use arrow::datatypes::ArrowNativeType; +use arrow::datatypes::ByteArrayType; +use arrow::datatypes::DataType; use arrow::datatypes::GenericBinaryType; use arrow::datatypes::GenericStringType; use std::sync::Arc; +use crate::binary_map::OutputType; +use crate::binary_map::INITIAL_BUFFER_CAPACITY; + +pub trait ArrayEq: Send + Sync { + fn equal_to(&self, lhs_row: usize, array: &ArrayRef, rhs_row: usize) -> bool; + fn append_val(&mut self, array: &ArrayRef, row: usize); + fn len(&self) -> usize; + fn build(self: Box) -> ArrayRef; + fn take_n(&mut self, n: usize) -> ArrayRef; +} + +pub struct PrimitiveGroupValueBuilder(Vec>); +impl PrimitiveGroupValueBuilder { + pub fn new() -> Self { + Self(vec![]) + } +} + +impl ArrayEq for PrimitiveGroupValueBuilder { + fn equal_to(&self, lhs_row: usize, array: &ArrayRef, rhs_row: usize) -> bool { + let elem = self.0[lhs_row]; + let arr = array.as_primitive::(); + let is_rhs_null = arr.is_null(rhs_row); + if elem.is_none() && is_rhs_null { + true + } else if elem.is_some() && !is_rhs_null { + elem.unwrap() == arr.value(rhs_row) + } else { + false + } + } + + fn append_val(&mut self, array: &ArrayRef, row: usize) { + let arr = array.as_primitive::(); + if arr.is_null(row) { + self.0.push(None) + } else { + let elem = arr.value(row); + self.0.push(Some(elem)) + } + } + + fn len(&self) -> usize { + self.0.len() + } + + fn build(self: Box) -> ArrayRef { + Arc::new(PrimitiveArray::::from_iter(self.0)) + } + + fn take_n(&mut self, n: usize) -> ArrayRef { + let first_n = self.0.drain(0..n).collect::>(); + Arc::new(PrimitiveArray::::from_iter(first_n)) + } +} + +pub struct ByteGroupValueBuilderNaive +where + O: OffsetSizeTrait, +{ + output_type: OutputType, + // buffer: BufferBuilder, + buffer: Vec, + /// Offsets into `buffer` for each distinct value. These offsets as used + /// directly to create the final `GenericBinaryArray`. The `i`th string is + /// stored in the range `offsets[i]..offsets[i+1]` in `buffer`. Null values + /// are stored as a zero length string. + offsets: Vec, + /// Null indexes in offsets + nulls: Vec, +} + +impl ByteGroupValueBuilderNaive +where + O: OffsetSizeTrait, +{ + pub fn new(output_type: OutputType) -> Self { + Self { + output_type, + buffer: Vec::with_capacity(INITIAL_BUFFER_CAPACITY), + offsets: vec![O::default()], + nulls: vec![], + } + } + + fn append_val_inner(&mut self, array: &ArrayRef, row: usize) + where + B: ByteArrayType, + { + let arr = array.as_bytes::(); + if arr.is_null(row) { + self.nulls.push(self.len()); + // nulls need a zero length in the offset buffer + let offset = self.buffer.len(); + + self.offsets.push(O::usize_as(offset)); + return; + } + + let value: &[u8] = arr.value(row).as_ref(); + self.buffer.extend_from_slice(value); + self.offsets.push(O::usize_as(self.buffer.len())); + } + + fn equal_to_inner(&self, lhs_row: usize, array: &ArrayRef, rhs_row: usize) -> bool + where + B: ByteArrayType, + { + // Handle nulls + let is_lhs_null = self.nulls.iter().any(|null_idx| *null_idx == lhs_row); + let arr = array.as_bytes::(); + if is_lhs_null { + return arr.is_null(rhs_row); + } else if arr.is_null(rhs_row) { + return false; + } + + let arr = array.as_bytes::(); + let rhs_elem: &[u8] = arr.value(rhs_row).as_ref(); + let rhs_elem_len = arr.value_length(rhs_row).as_usize(); + assert_eq!(rhs_elem_len, rhs_elem.len()); + let l = self.offsets[lhs_row].as_usize(); + let r = self.offsets[lhs_row + 1].as_usize(); + let existing_elem = unsafe { self.buffer.as_slice().get_unchecked(l..r) }; + existing_elem.len() == rhs_elem.len() && rhs_elem == existing_elem + } +} + +impl ArrayEq for ByteGroupValueBuilderNaive +where + O: OffsetSizeTrait, +{ + fn equal_to(&self, lhs_row: usize, column: &ArrayRef, rhs_row: usize) -> bool { + // Sanity array type + match self.output_type { + OutputType::Binary => { + assert!(matches!( + column.data_type(), + DataType::Binary | DataType::LargeBinary + )); + self.equal_to_inner::>(lhs_row, column, rhs_row) + } + OutputType::Utf8 => { + assert!(matches!( + column.data_type(), + DataType::Utf8 | DataType::LargeUtf8 + )); + self.equal_to_inner::>(lhs_row, column, rhs_row) + } + _ => unreachable!("View types should use `ArrowBytesViewMap`"), + } + } + + fn append_val(&mut self, column: &ArrayRef, row: usize) { + // Sanity array type + match self.output_type { + OutputType::Binary => { + assert!(matches!( + column.data_type(), + DataType::Binary | DataType::LargeBinary + )); + self.append_val_inner::>(column, row) + } + OutputType::Utf8 => { + assert!(matches!( + column.data_type(), + DataType::Utf8 | DataType::LargeUtf8 + )); + self.append_val_inner::>(column, row) + } + _ => unreachable!("View types should use `ArrowBytesViewMap`"), + }; + } + + fn len(&self) -> usize { + self.offsets.len() - 1 + } + + fn build(self: Box) -> ArrayRef { + let Self { + output_type, + buffer, + offsets, + nulls, + } = *self; + + let null_buffer = if nulls.is_empty() { + None + } else { + // Only make a `NullBuffer` if there was a null value + let num_values = offsets.len() - 1; + let mut bool_builder = BooleanBufferBuilder::new(num_values); + bool_builder.append_n(num_values, true); + nulls.into_iter().for_each(|null_index| { + bool_builder.set_bit(null_index, false); + }); + Some(NullBuffer::from(bool_builder.finish())) + }; + + // SAFETY: the offsets were constructed correctly in `insert_if_new` -- + // monotonically increasing, overflows were checked. + let offsets = unsafe { OffsetBuffer::new_unchecked(ScalarBuffer::from(offsets)) }; + let values = Buffer::from_vec(buffer); + + match output_type { + OutputType::Binary => { + // SAFETY: the offsets were constructed correctly + Arc::new(unsafe { + GenericBinaryArray::new_unchecked(offsets, values, null_buffer) + }) + } + OutputType::Utf8 => { + // SAFETY: + // 1. the offsets were constructed safely + // + // 2. we asserted the input arrays were all the correct type and + // thus since all the values that went in were valid (e.g. utf8) + // so are all the values that come out + let res = Arc::new(unsafe { + GenericStringArray::new_unchecked(offsets, values, null_buffer) + }); + res + } + _ => unreachable!("View types should use `ArrowBytesViewMap`"), + } + } + + fn take_n(&mut self, n: usize) -> ArrayRef { + let null_buffer = if self.nulls.is_empty() { + None + } else { + // Only make a `NullBuffer` if there was a null value + let num_values = self.offsets.len() - 1; + let mut bool_builder = BooleanBufferBuilder::new(num_values); + bool_builder.append_n(num_values, true); + self.nulls.iter().for_each(|null_index| { + bool_builder.set_bit(*null_index, false); + }); + Some(NullBuffer::from(bool_builder.finish())) + }; + + // Given offests like [0, 2, 4, 5] and n = 1, we expect to get + // offsets [0, 2, 3]. We first create two offsets for first_n as [0, 2] and the remaining as [2, 4, 5]. + // And we shift the offset starting from 0 for the remaining one, [2, 4, 5] -> [0, 2, 3]. + let mut first_n_offsets = self.offsets.drain(0..n).collect::>(); + let offset_n = self.offsets.first().unwrap().clone(); + for offset in self.offsets.iter_mut() { + *offset = offset.sub(offset_n); + } + first_n_offsets.push(offset_n); + + // SAFETY: the offsets were constructed correctly in `insert_if_new` -- + // monotonically increasing, overflows were checked. + let offsets = unsafe { OffsetBuffer::new_unchecked(ScalarBuffer::from(first_n_offsets)) }; + + // Consume first (n - nulls count) of elements since we don't push any value for null case. + let first_n_valid_buffer = self.buffer.drain(0..(n - self.nulls.len())).collect(); + let values = Buffer::from_vec(first_n_valid_buffer); + + match self.output_type { + OutputType::Binary => { + // SAFETY: the offsets were constructed correctly + Arc::new(unsafe { + GenericBinaryArray::new_unchecked(offsets, values, null_buffer) + }) + } + OutputType::Utf8 => { + // SAFETY: + // 1. the offsets were constructed safely + // + // 2. we asserted the input arrays were all the correct type and + // thus since all the values that went in were valid (e.g. utf8) + // so are all the values that come out + let res = Arc::new(unsafe { + GenericStringArray::new_unchecked(offsets, values, null_buffer) + }); + res + } + _ => unreachable!("View types should use `ArrowBytesViewMap`"), + } + } +} + pub trait ArrayRowEq: Send + Sync { fn equal_to(&self, lhs_row: usize, array: &ArrayRef, rhs_row: usize) -> bool; fn append_val(&mut self, array: &ArrayRef, row: usize); diff --git a/datafusion/physical-plan/src/aggregates/group_values/row_like.rs b/datafusion/physical-plan/src/aggregates/group_values/row_like.rs index 3e270f3f0d3b..133fa0d9bd93 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/row_like.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/row_like.rs @@ -15,11 +15,8 @@ // specific language governing permissions and limitations // under the License. -use std::ops::DerefMut; - use crate::aggregates::group_values::GroupValues; use ahash::RandomState; -use arrow::array::{GenericStringBuilder, PrimitiveBuilder}; use arrow::compute::cast; use arrow::datatypes::{ Date32Type, Date64Type, Float32Type, Float64Type, Int16Type, Int32Type, Int64Type, @@ -30,14 +27,13 @@ use arrow::row::{RowConverter, Rows, SortField}; use arrow_array::{Array, ArrayRef}; use arrow_schema::{DataType, SchemaRef}; use datafusion_common::hash_utils::create_hashes; -use datafusion_common::{internal_err, DataFusionError, Result}; +use datafusion_common::{DataFusionError, Result}; use datafusion_execution::memory_pool::proxy::{RawTableAllocExt, VecAllocExt}; use datafusion_expr::EmitTo; -use datafusion_physical_expr_common::group_value_row::ArrayRowEq; +use datafusion_physical_expr::binary_map::OutputType; +use datafusion_physical_expr_common::group_value_row::{ArrayEq, ByteGroupValueBuilderNaive, PrimitiveGroupValueBuilder}; use hashbrown::raw::RawTable; -pub(super) const INITIAL_CAPACITY: usize = 8 * 1024; - /// A [`GroupValues`] making use of [`Rows`] pub struct GroupValuesRowLike { /// The output schema @@ -77,7 +73,7 @@ pub struct GroupValuesRowLike { /// Random state for creating hashes random_state: RandomState, - group_values_v2: Option>>, + group_values_v2: Option>>, } impl GroupValuesRowLike { @@ -122,65 +118,59 @@ impl GroupValues for GroupValuesRowLike { for f in self.schema.fields().iter() { match f.data_type() { &DataType::Int8 => { - let b = PrimitiveBuilder::::new(); + let b = PrimitiveGroupValueBuilder::::new(); v.push(Box::new(b) as _) } &DataType::Int16 => { - let b = PrimitiveBuilder::::new(); + let b = PrimitiveGroupValueBuilder::::new(); v.push(Box::new(b) as _) } &DataType::Int32 => { - let b = PrimitiveBuilder::::new(); + let b = PrimitiveGroupValueBuilder::::new(); v.push(Box::new(b) as _) } &DataType::Int64 => { - let b = PrimitiveBuilder::::new(); + let b = PrimitiveGroupValueBuilder::::new(); v.push(Box::new(b) as _) } &DataType::UInt8 => { - let b = PrimitiveBuilder::::new(); + let b = PrimitiveGroupValueBuilder::::new(); v.push(Box::new(b) as _) } &DataType::UInt16 => { - let b = PrimitiveBuilder::::new(); + let b = PrimitiveGroupValueBuilder::::new(); v.push(Box::new(b) as _) } &DataType::UInt32 => { - let b = PrimitiveBuilder::::new(); + let b = PrimitiveGroupValueBuilder::::new(); v.push(Box::new(b) as _) } &DataType::UInt64 => { - let b = PrimitiveBuilder::::new(); + let b = PrimitiveGroupValueBuilder::::new(); v.push(Box::new(b) as _) } &DataType::Float32 => { - let b = PrimitiveBuilder::::new(); + let b = PrimitiveGroupValueBuilder::::new(); v.push(Box::new(b) as _) } &DataType::Float64 => { - let b = PrimitiveBuilder::::new(); + let b = PrimitiveGroupValueBuilder::::new(); v.push(Box::new(b) as _) } &DataType::Date32 => { - let b = PrimitiveBuilder::::new(); + let b = PrimitiveGroupValueBuilder::::new(); v.push(Box::new(b) as _) } &DataType::Date64 => { - let b = PrimitiveBuilder::::new(); + let b = PrimitiveGroupValueBuilder::::new(); v.push(Box::new(b) as _) } &DataType::Utf8 => { - let b = GenericStringBuilder::::with_capacity( - INITIAL_CAPACITY, - INITIAL_CAPACITY, - ); + let b = ByteGroupValueBuilderNaive::::new(OutputType::Utf8); v.push(Box::new(b) as _) } &DataType::LargeUtf8 => { - let b = GenericStringBuilder::::with_capacity( - INITIAL_CAPACITY, - INITIAL_CAPACITY, - ); + let b = ByteGroupValueBuilderNaive::::new(OutputType::Utf8); v.push(Box::new(b) as _) } dt => todo!("{dt} not impl"), @@ -214,7 +204,7 @@ impl GroupValues for GroupValuesRowLike { // && group_rows.row(row) == group_values.row(*group_idx) fn compare_equal( - arry_eq: &dyn ArrayRowEq, + arry_eq: &dyn ArrayEq, lhs_row: usize, array: &ArrayRef, rhs_row: usize, @@ -309,9 +299,9 @@ impl GroupValues for GroupValuesRowLike { let mut output = match emit_to { EmitTo::All => { let output = group_values_v2 - .iter_mut() + .into_iter() .map(|v| { - v.deref_mut().build() + v.build() }) .collect::>(); // let output = self.row_converter.convert_rows(&group_values)?; @@ -321,21 +311,8 @@ impl GroupValues for GroupValuesRowLike { output } EmitTo::First(n) => { - return internal_err!("there is error"); - - // println!("to first n"); - let len = group_values_v2.len(); - let first_n: Vec> = - group_values_v2.drain(..n).collect(); - let output = first_n - .into_iter() - .map(|mut v| { - let p = v.deref_mut().build(); - p - }) - .collect::>(); - assert_eq!(len, group_values_v2.len() + n); - self.group_values_v2 = Some(group_values_v2); + // return internal_err!("there is error"); + let output = group_values_v2.iter_mut().map(|v|v.take_n(n)).collect::>(); // let groups_rows = group_values.iter().take(n); // let output = self.row_converter.convert_rows(groups_rows)?; @@ -359,6 +336,7 @@ impl GroupValues for GroupValuesRowLike { } } } + self.group_values_v2 = Some(group_values_v2); output } }; From 6e0b179007ecad44287753c9752071c93cd2774f Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Sun, 15 Sep 2024 23:05:35 +0800 Subject: [PATCH 06/30] clippy Signed-off-by: jayzhan211 --- .../src/group_value_row.rs | 313 +++--------------- .../src/aggregates/group_values/mod.rs | 28 +- .../src/aggregates/group_values/row_like.rs | 136 +++----- 3 files changed, 101 insertions(+), 376 deletions(-) diff --git a/datafusion/physical-expr-common/src/group_value_row.rs b/datafusion/physical-expr-common/src/group_value_row.rs index e5c567833b50..955836ea515b 100644 --- a/datafusion/physical-expr-common/src/group_value_row.rs +++ b/datafusion/physical-expr-common/src/group_value_row.rs @@ -16,15 +16,12 @@ // under the License. use arrow::array::BooleanBufferBuilder; +use arrow::array::BufferBuilder; use arrow::array::GenericBinaryArray; -use arrow::array::GenericBinaryBuilder; use arrow::array::GenericStringArray; -use arrow::array::GenericStringBuilder; use arrow::array::OffsetSizeTrait; use arrow::array::PrimitiveArray; -use arrow::array::PrimitiveBuilder; use arrow::array::{Array, ArrayRef, ArrowPrimitiveType, AsArray}; -use arrow::buffer::Buffer; use arrow::buffer::NullBuffer; use arrow::buffer::OffsetBuffer; use arrow::buffer::ScalarBuffer; @@ -39,22 +36,23 @@ use std::sync::Arc; use crate::binary_map::OutputType; use crate::binary_map::INITIAL_BUFFER_CAPACITY; -pub trait ArrayEq: Send + Sync { +pub trait ArrayRowEq: Send + Sync { fn equal_to(&self, lhs_row: usize, array: &ArrayRef, rhs_row: usize) -> bool; fn append_val(&mut self, array: &ArrayRef, row: usize); fn len(&self) -> usize; + fn is_empty(&self) -> bool; fn build(self: Box) -> ArrayRef; fn take_n(&mut self, n: usize) -> ArrayRef; } pub struct PrimitiveGroupValueBuilder(Vec>); -impl PrimitiveGroupValueBuilder { - pub fn new() -> Self { +impl Default for PrimitiveGroupValueBuilder { + fn default() -> Self { Self(vec![]) } } -impl ArrayEq for PrimitiveGroupValueBuilder { +impl ArrayRowEq for PrimitiveGroupValueBuilder { fn equal_to(&self, lhs_row: usize, array: &ArrayRef, rhs_row: usize) -> bool { let elem = self.0[lhs_row]; let arr = array.as_primitive::(); @@ -82,6 +80,10 @@ impl ArrayEq for PrimitiveGroupValueBuilder { self.0.len() } + fn is_empty(&self) -> bool { + self.len() == 0 + } + fn build(self: Box) -> ArrayRef { Arc::new(PrimitiveArray::::from_iter(self.0)) } @@ -92,13 +94,13 @@ impl ArrayEq for PrimitiveGroupValueBuilder { } } -pub struct ByteGroupValueBuilderNaive +pub struct ByteGroupValueBuilder where O: OffsetSizeTrait, { output_type: OutputType, - // buffer: BufferBuilder, - buffer: Vec, + buffer_v2: BufferBuilder, + // buffer: Vec, /// Offsets into `buffer` for each distinct value. These offsets as used /// directly to create the final `GenericBinaryArray`. The `i`th string is /// stored in the range `offsets[i]..offsets[i+1]` in `buffer`. Null values @@ -108,14 +110,15 @@ where nulls: Vec, } -impl ByteGroupValueBuilderNaive +impl ByteGroupValueBuilder where O: OffsetSizeTrait, { pub fn new(output_type: OutputType) -> Self { Self { output_type, - buffer: Vec::with_capacity(INITIAL_BUFFER_CAPACITY), + buffer_v2: BufferBuilder::new(INITIAL_BUFFER_CAPACITY), + // buffer: Vec::with_capacity(INITIAL_BUFFER_CAPACITY), offsets: vec![O::default()], nulls: vec![], } @@ -129,15 +132,16 @@ where if arr.is_null(row) { self.nulls.push(self.len()); // nulls need a zero length in the offset buffer - let offset = self.buffer.len(); + let offset = self.buffer_v2.len(); self.offsets.push(O::usize_as(offset)); return; } let value: &[u8] = arr.value(row).as_ref(); - self.buffer.extend_from_slice(value); - self.offsets.push(O::usize_as(self.buffer.len())); + // self.buffer.extend_from_slice(value); + self.buffer_v2.append_slice(value); + self.offsets.push(O::usize_as(self.buffer_v2.len())); } fn equal_to_inner(&self, lhs_row: usize, array: &ArrayRef, rhs_row: usize) -> bool @@ -159,12 +163,12 @@ where assert_eq!(rhs_elem_len, rhs_elem.len()); let l = self.offsets[lhs_row].as_usize(); let r = self.offsets[lhs_row + 1].as_usize(); - let existing_elem = unsafe { self.buffer.as_slice().get_unchecked(l..r) }; + let existing_elem = unsafe { self.buffer_v2.as_slice().get_unchecked(l..r) }; existing_elem.len() == rhs_elem.len() && rhs_elem == existing_elem } } -impl ArrayEq for ByteGroupValueBuilderNaive +impl ArrayRowEq for ByteGroupValueBuilder where O: OffsetSizeTrait, { @@ -214,10 +218,15 @@ where self.offsets.len() - 1 } + fn is_empty(&self) -> bool { + self.len() == 0 + } + fn build(self: Box) -> ArrayRef { let Self { output_type, - buffer, + mut buffer_v2, + // buffer, offsets, nulls, } = *self; @@ -238,7 +247,8 @@ where // SAFETY: the offsets were constructed correctly in `insert_if_new` -- // monotonically increasing, overflows were checked. let offsets = unsafe { OffsetBuffer::new_unchecked(ScalarBuffer::from(offsets)) }; - let values = Buffer::from_vec(buffer); + // let values = Buffer::from_vec(buffer); + let values = buffer_v2.finish(); match output_type { OutputType::Binary => { @@ -254,10 +264,9 @@ where // 2. we asserted the input arrays were all the correct type and // thus since all the values that went in were valid (e.g. utf8) // so are all the values that come out - let res = Arc::new(unsafe { + Arc::new(unsafe { GenericStringArray::new_unchecked(offsets, values, null_buffer) - }); - res + }) } _ => unreachable!("View types should use `ArrowBytesViewMap`"), } @@ -281,19 +290,27 @@ where // offsets [0, 2, 3]. We first create two offsets for first_n as [0, 2] and the remaining as [2, 4, 5]. // And we shift the offset starting from 0 for the remaining one, [2, 4, 5] -> [0, 2, 3]. let mut first_n_offsets = self.offsets.drain(0..n).collect::>(); - let offset_n = self.offsets.first().unwrap().clone(); - for offset in self.offsets.iter_mut() { - *offset = offset.sub(offset_n); - } + let offset_n = *self.offsets.first().unwrap(); + self.offsets + .iter_mut() + .for_each(|offset| *offset = offset.sub(offset_n)); first_n_offsets.push(offset_n); // SAFETY: the offsets were constructed correctly in `insert_if_new` -- // monotonically increasing, overflows were checked. - let offsets = unsafe { OffsetBuffer::new_unchecked(ScalarBuffer::from(first_n_offsets)) }; + let offsets = + unsafe { OffsetBuffer::new_unchecked(ScalarBuffer::from(first_n_offsets)) }; // Consume first (n - nulls count) of elements since we don't push any value for null case. - let first_n_valid_buffer = self.buffer.drain(0..(n - self.nulls.len())).collect(); - let values = Buffer::from_vec(first_n_valid_buffer); + let r = n - self.nulls.len(); + + let mut remaining_buffer = BufferBuilder::new(self.buffer_v2.len() - r); + remaining_buffer.append_slice(&self.buffer_v2.as_slice()[r..]); + self.buffer_v2.truncate(r); + let values = self.buffer_v2.finish(); + + // let first_n_valid_buffer = self.buffer.drain(0..(n - self.nulls.len())).collect(); + // let values = Buffer::from_vec(first_n_valid_buffer); match self.output_type { OutputType::Binary => { @@ -309,243 +326,11 @@ where // 2. we asserted the input arrays were all the correct type and // thus since all the values that went in were valid (e.g. utf8) // so are all the values that come out - let res = Arc::new(unsafe { + Arc::new(unsafe { GenericStringArray::new_unchecked(offsets, values, null_buffer) - }); - res + }) } _ => unreachable!("View types should use `ArrowBytesViewMap`"), } } } - -pub trait ArrayRowEq: Send + Sync { - fn equal_to(&self, lhs_row: usize, array: &ArrayRef, rhs_row: usize) -> bool; - fn append_val(&mut self, array: &ArrayRef, row: usize); - fn len(&self) -> usize; - // take n elements as ArrayRef and adjusted the underlying buffer - fn take_n(&mut self, n: usize) -> ArrayRef; - fn build(&mut self) -> ArrayRef; -} - -impl ArrayRowEq for PrimitiveBuilder -where - T: ArrowPrimitiveType, -{ - fn equal_to(&self, lhs_row: usize, array: &ArrayRef, rhs_row: usize) -> bool { - let arr = array.as_primitive::(); - - if let Some(validity_slice) = self.validity_slice() { - let validity_slice_index = lhs_row / 8; - let bit_map_index = lhs_row % 8; - let is_lhs_null = ((validity_slice[validity_slice_index] >> bit_map_index) & 1) == 0; - if is_lhs_null { - return arr.is_null(rhs_row); - } else if arr.is_null(rhs_row) { - return false; - } - } - - let elem = self.values_slice()[lhs_row]; - elem == arr.value(rhs_row) - } - - fn append_val(&mut self, array: &ArrayRef, row: usize) { - let arr = array.as_primitive::(); - if arr.is_null(row) { - self.append_null(); - } else { - let elem = arr.value(row); - self.append_value(elem); - } - } - - fn take_n(&mut self, n: usize) -> ArrayRef { - todo!("") - - // let num_remaining = self.values_slice().len() - n; - // assert!(num_remaining >= 0); - - // let mut builder = PrimitiveBuilder::::new(); - // let vs = self.values_slice(); - // builder.append_slice(vs); - - // let mut values_left = vec![T::default_value(); num_remaining]; - // let mut null_buffer_left = NullBuffer::new_null(num_remaining); - - // let null_buffer_left = self.validity_slice(); - - // let len = self.len(); - // let nulls = self.null_buffer_builder.finish(); - // let builder = ArrayData::builder(self.data_type.clone()) - // .len(len) - // .add_buffer(self.values_builder.finish()) - // .nulls(nulls); - - // let array_data = unsafe { builder.build_unchecked() }; - // PrimitiveArray::::from(array_data) - - // let output = self.finish(); - - // let mut values_buffer = MutableBuffer::new(num_remaining); - // values_buffer.extend_from_slice(&values_left); - // let mut null_buffer = MutableBuffer::new_null(num_remaining); - // null_buffer.extend_from_slice(items) - - - } - - fn len(&self) -> usize { - self.values_slice().len() - } - - fn build(&mut self) -> ArrayRef { - Arc::new(self.finish()) - } -} - -impl ArrayRowEq for GenericStringBuilder -where - O: OffsetSizeTrait, -{ - fn equal_to(&self, lhs_row: usize, array: &ArrayRef, rhs_row: usize) -> bool { - let arr = array.as_bytes::>(); - if let Some(validity_slice) = self.validity_slice() { - let validity_slice_index = lhs_row / 8; - let bit_map_index = lhs_row % 8; - let is_lhs_null = ((validity_slice[validity_slice_index] >> bit_map_index) & 1) == 0; - if is_lhs_null { - return arr.is_null(rhs_row); - } else if arr.is_null(rhs_row) { - return false; - } - } - - let rhs_elem: &[u8] = arr.value(rhs_row).as_ref(); - let rhs_elem_len = arr.value_length(rhs_row).as_usize(); - assert_eq!(rhs_elem_len, rhs_elem.len()); - let l = O::as_usize(self.offsets_slice()[lhs_row]); - let r = O::as_usize(self.offsets_slice()[lhs_row + 1]); - let existing_elem = &self.values_slice()[l..r]; - existing_elem.len() == rhs_elem.len() && rhs_elem == existing_elem - } - - fn append_val(&mut self, array: &ArrayRef, row: usize) { - let arr = array.as_string::(); - if arr.is_null(row) { - self.append_null(); - return; - } - - let value = arr.value(row); - self.append_value(value); - } - - fn take_n(&mut self, n: usize) -> ArrayRef { - todo!("") - } - - fn len(&self) -> usize { - self.offsets_slice().len() - 1 - } - - fn build(&mut self) -> ArrayRef { - Arc::new(self.finish()) - } -} - -impl ArrayRowEq for GenericBinaryBuilder -where - O: OffsetSizeTrait, -{ - fn equal_to(&self, lhs_row: usize, array: &ArrayRef, rhs_row: usize) -> bool { - let arr = array.as_bytes::>(); - if let Some(validity_slice) = self.validity_slice() { - let validity_slice_index = lhs_row / 8; - let bit_map_index = lhs_row % 8; - let is_lhs_null = ((validity_slice[validity_slice_index] >> bit_map_index) & 1) == 0; - if is_lhs_null { - return arr.is_null(rhs_row); - } else if arr.is_null(rhs_row) { - return false; - } - } - - let rhs_elem: &[u8] = arr.value(rhs_row).as_ref(); - let rhs_elem_len = arr.value_length(rhs_row).as_usize(); - assert_eq!(rhs_elem_len, rhs_elem.len()); - let l = O::as_usize(self.offsets_slice()[lhs_row]); - let r = O::as_usize(self.offsets_slice()[lhs_row + 1]); - let existing_elem = &self.values_slice()[l..r]; - existing_elem.len() == rhs_elem.len() && rhs_elem == existing_elem - } - - fn append_val(&mut self, array: &ArrayRef, row: usize) { - let arr = array.as_binary::(); - if arr.is_null(row) { - self.append_null(); - return; - } - - let value: &[u8] = arr.value(row).as_ref(); - self.append_value(value); - } - - fn take_n(&mut self, n: usize) -> ArrayRef { - todo!("") - } - - fn len(&self) -> usize { - self.values_slice().len() - } - - fn build(&mut self) -> ArrayRef { - Arc::new(self.finish()) - } -} - -#[cfg(test)] -mod tests { - use arrow::{array::{GenericStringBuilder, PrimitiveBuilder}, datatypes::Int32Type}; - - #[test] - fn te1() { - let mut a = PrimitiveBuilder::::new(); - a.append_value(1); - a.append_value(2); - a.append_null(); - a.append_null(); - a.append_value(2); - - let s = a.values_slice(); - let v = a.validity_slice(); - println!("s: {:?}", s); - println!("v: {:?}", v); - - } - - #[test] - fn test123() { - let mut a = GenericStringBuilder::::new(); - a.append_null(); - let p = a.offsets_slice(); - println!("p: {:?}", p); - let s = a.validity_slice(); - println!("s: {:?}", s); - a.append_null(); - let p = a.offsets_slice(); - println!("p: {:?}", p); - let s = a.validity_slice(); - println!("s: {:?}", s); - a.append_value("12"); - let p = a.offsets_slice(); - println!("p: {:?}", p); - let s = a.validity_slice(); - println!("s: {:?}", s); - a.append_null(); - let p = a.offsets_slice(); - println!("p: {:?}", p); - let s = a.validity_slice(); - println!("s: {:?}", s); - } -} \ No newline at end of file diff --git a/datafusion/physical-plan/src/aggregates/group_values/mod.rs b/datafusion/physical-plan/src/aggregates/group_values/mod.rs index 434a5c25370d..84bbf221f8fd 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/mod.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/mod.rs @@ -107,19 +107,19 @@ pub fn new_group_values(schema: SchemaRef) -> Result> { } fn has_row_like_feature(data_type: &DataType) -> bool { - match *data_type { + matches!( + *data_type, DataType::Int8 - | DataType::Int16 - | DataType::Int32 - | DataType::Int64 - | DataType::UInt8 - | DataType::UInt16 - | DataType::UInt32 - | DataType::UInt64 - | DataType::Utf8 - | DataType::LargeUtf8 - | DataType::Date32 - | DataType::Date64 => true, - _ => false, - } + | DataType::Int16 + | DataType::Int32 + | DataType::Int64 + | DataType::UInt8 + | DataType::UInt16 + | DataType::UInt32 + | DataType::UInt64 + | DataType::Utf8 + | DataType::LargeUtf8 + | DataType::Date32 + | DataType::Date64 + ) } diff --git a/datafusion/physical-plan/src/aggregates/group_values/row_like.rs b/datafusion/physical-plan/src/aggregates/group_values/row_like.rs index 133fa0d9bd93..52878c870ee4 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/row_like.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/row_like.rs @@ -23,7 +23,6 @@ use arrow::datatypes::{ Int8Type, UInt16Type, UInt32Type, UInt64Type, UInt8Type, }; use arrow::record_batch::RecordBatch; -use arrow::row::{RowConverter, Rows, SortField}; use arrow_array::{Array, ArrayRef}; use arrow_schema::{DataType, SchemaRef}; use datafusion_common::hash_utils::create_hashes; @@ -31,17 +30,16 @@ use datafusion_common::{DataFusionError, Result}; use datafusion_execution::memory_pool::proxy::{RawTableAllocExt, VecAllocExt}; use datafusion_expr::EmitTo; use datafusion_physical_expr::binary_map::OutputType; -use datafusion_physical_expr_common::group_value_row::{ArrayEq, ByteGroupValueBuilderNaive, PrimitiveGroupValueBuilder}; +use datafusion_physical_expr_common::group_value_row::{ + ArrayRowEq, ByteGroupValueBuilder, PrimitiveGroupValueBuilder, +}; use hashbrown::raw::RawTable; -/// A [`GroupValues`] making use of [`Rows`] +/// Compare GroupValue Rows column by column pub struct GroupValuesRowLike { /// The output schema schema: SchemaRef, - /// Converter for the group values - row_converter: RowConverter, - /// Logically maps group values to a group_index in /// [`Self::group_values`] and in each accumulator /// @@ -68,39 +66,20 @@ pub struct GroupValuesRowLike { /// reused buffer to store hashes hashes_buffer: Vec, - /// reused buffer to store rows - rows_buffer: Rows, - /// Random state for creating hashes random_state: RandomState, - group_values_v2: Option>>, + group_values: Option>>, } impl GroupValuesRowLike { pub fn try_new(schema: SchemaRef) -> Result { - let row_converter = RowConverter::new( - schema - .fields() - .iter() - .map(|f| SortField::new(f.data_type().clone())) - .collect(), - )?; - let map = RawTable::with_capacity(0); - - let starting_rows_capacity = 1000; - let starting_data_capacity = 64 * starting_rows_capacity; - let rows_buffer = - row_converter.empty_rows(starting_rows_capacity, starting_data_capacity); Ok(Self { schema, - row_converter, map, map_size: 0, - // group_values: None, - group_values_v2: None, + group_values: None, hashes_buffer: Default::default(), - rows_buffer, random_state: Default::default(), }) } @@ -109,68 +88,67 @@ impl GroupValuesRowLike { impl GroupValues for GroupValuesRowLike { fn intern(&mut self, cols: &[ArrayRef], groups: &mut Vec) -> Result<()> { let n_rows = cols[0].len(); - let mut group_values_v2 = match self.group_values_v2.take() { + let mut group_values = match self.group_values.take() { Some(group_values) => group_values, None => { let len = cols.len(); let mut v = Vec::with_capacity(len); - // Move to `try_new` for f in self.schema.fields().iter() { match f.data_type() { &DataType::Int8 => { - let b = PrimitiveGroupValueBuilder::::new(); + let b = PrimitiveGroupValueBuilder::::default(); v.push(Box::new(b) as _) } &DataType::Int16 => { - let b = PrimitiveGroupValueBuilder::::new(); + let b = PrimitiveGroupValueBuilder::::default(); v.push(Box::new(b) as _) } &DataType::Int32 => { - let b = PrimitiveGroupValueBuilder::::new(); + let b = PrimitiveGroupValueBuilder::::default(); v.push(Box::new(b) as _) } &DataType::Int64 => { - let b = PrimitiveGroupValueBuilder::::new(); + let b = PrimitiveGroupValueBuilder::::default(); v.push(Box::new(b) as _) } &DataType::UInt8 => { - let b = PrimitiveGroupValueBuilder::::new(); + let b = PrimitiveGroupValueBuilder::::default(); v.push(Box::new(b) as _) } &DataType::UInt16 => { - let b = PrimitiveGroupValueBuilder::::new(); + let b = PrimitiveGroupValueBuilder::::default(); v.push(Box::new(b) as _) } &DataType::UInt32 => { - let b = PrimitiveGroupValueBuilder::::new(); + let b = PrimitiveGroupValueBuilder::::default(); v.push(Box::new(b) as _) } &DataType::UInt64 => { - let b = PrimitiveGroupValueBuilder::::new(); + let b = PrimitiveGroupValueBuilder::::default(); v.push(Box::new(b) as _) } &DataType::Float32 => { - let b = PrimitiveGroupValueBuilder::::new(); + let b = PrimitiveGroupValueBuilder::::default(); v.push(Box::new(b) as _) } &DataType::Float64 => { - let b = PrimitiveGroupValueBuilder::::new(); + let b = PrimitiveGroupValueBuilder::::default(); v.push(Box::new(b) as _) } &DataType::Date32 => { - let b = PrimitiveGroupValueBuilder::::new(); + let b = PrimitiveGroupValueBuilder::::default(); v.push(Box::new(b) as _) } &DataType::Date64 => { - let b = PrimitiveGroupValueBuilder::::new(); + let b = PrimitiveGroupValueBuilder::::default(); v.push(Box::new(b) as _) } &DataType::Utf8 => { - let b = ByteGroupValueBuilderNaive::::new(OutputType::Utf8); + let b = ByteGroupValueBuilder::::new(OutputType::Utf8); v.push(Box::new(b) as _) } &DataType::LargeUtf8 => { - let b = ByteGroupValueBuilderNaive::::new(OutputType::Utf8); + let b = ByteGroupValueBuilder::::new(OutputType::Utf8); v.push(Box::new(b) as _) } dt => todo!("{dt} not impl"), @@ -204,7 +182,7 @@ impl GroupValues for GroupValuesRowLike { // && group_rows.row(row) == group_values.row(*group_idx) fn compare_equal( - arry_eq: &dyn ArrayEq, + arry_eq: &dyn ArrayRowEq, lhs_row: usize, array: &ArrayRef, rhs_row: usize, @@ -212,7 +190,7 @@ impl GroupValues for GroupValuesRowLike { arry_eq.equal_to(lhs_row, array, rhs_row) } - for (i, group_val) in group_values_v2.iter().enumerate() { + for (i, group_val) in group_values.iter().enumerate() { if !compare_equal(group_val.as_ref(), *group_idx, &cols[i], row) { return false; } @@ -231,8 +209,8 @@ impl GroupValues for GroupValuesRowLike { // group_values.push(group_rows.row(row)); let mut checklen = 0; - let group_idx = group_values_v2[0].len(); - for (i, group_value) in group_values_v2.iter_mut().enumerate() { + let group_idx = group_values[0].len(); + for (i, group_value) in group_values.iter_mut().enumerate() { group_value.append_val(&cols[i], row); let len = group_value.len(); if i == 0 { @@ -254,22 +232,13 @@ impl GroupValues for GroupValuesRowLike { groups.push(group_idx); } - // self.group_values = Some(group_values); - self.group_values_v2 = Some(group_values_v2); - + self.group_values = Some(group_values); Ok(()) } fn size(&self) -> usize { - // TODO: get real size - let group_values_size = - self.group_values_v2.as_ref().map(|v| v.len()).unwrap_or(0); - // let group_values_size = self.group_values.as_ref().map(|v| v.size()).unwrap_or(0); - self.row_converter.size() - + group_values_size - + self.map_size - + self.rows_buffer.size() - + self.hashes_buffer.allocated_size() + let group_values_size = self.group_values.as_ref().map(|v| v.len()).unwrap_or(0); + group_values_size + self.map_size + self.hashes_buffer.allocated_size() } fn is_empty(&self) -> bool { @@ -277,52 +246,29 @@ impl GroupValues for GroupValuesRowLike { } fn len(&self) -> usize { - // self.group_values - // .as_ref() - // .map(|group_values| group_values.num_rows()) - // .unwrap_or(0) - - self.group_values_v2 - .as_ref() - .map(|v| v[0].len()) - .unwrap_or(0) + self.group_values.as_ref().map(|v| v[0].len()).unwrap_or(0) } fn emit(&mut self, emit_to: EmitTo) -> Result> { let mut group_values_v2 = self - .group_values_v2 + .group_values .take() .expect("Can not emit from empty rows"); - // println!("emit_to: {:?}", emit_to); - let mut output = match emit_to { EmitTo::All => { let output = group_values_v2 .into_iter() - .map(|v| { - v.build() - }) + .map(|v| v.build()) .collect::>(); - // let output = self.row_converter.convert_rows(&group_values)?; - // group_values.clear(); - // println!("output: {:?}", output); - self.group_values_v2 = None; + self.group_values = None; output } EmitTo::First(n) => { - // return internal_err!("there is error"); - let output = group_values_v2.iter_mut().map(|v|v.take_n(n)).collect::>(); - - // let groups_rows = group_values.iter().take(n); - // let output = self.row_converter.convert_rows(groups_rows)?; - // // Clear out first n group keys by copying them to a new Rows. - // // TODO file some ticket in arrow-rs to make this more efficient? - // let mut new_group_values = self.row_converter.empty_rows(0, 0); - // for row in group_values.iter().skip(n) { - // new_group_values.push(row); - // } - // std::mem::swap(&mut new_group_values, &mut group_values); + let output = group_values_v2 + .iter_mut() + .map(|v| v.take_n(n)) + .collect::>(); // SAFETY: self.map outlives iterator and is not modified concurrently unsafe { @@ -336,7 +282,7 @@ impl GroupValues for GroupValuesRowLike { } } } - self.group_values_v2 = Some(group_values_v2); + self.group_values = Some(group_values_v2); output } }; @@ -355,18 +301,12 @@ impl GroupValues for GroupValuesRowLike { } } - // self.group_values = Some(group_values); - // println!("output: {:?}", output); Ok(output) } fn clear_shrink(&mut self, batch: &RecordBatch) { let count = batch.num_rows(); - // self.group_values = self.group_values.take().map(|mut rows| { - // rows.clear(); - // rows - // }); - self.group_values_v2 = self.group_values_v2.take().map(|mut rows| { + self.group_values = self.group_values.take().map(|mut rows| { rows.clear(); rows }); From 10d3d18c1f27ff8b3f4a355dd8a8db2a5c334824 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Mon, 16 Sep 2024 09:09:03 +0800 Subject: [PATCH 07/30] optimize for non-null Signed-off-by: jayzhan211 --- .../src/group_value_row.rs | 50 ++++++++++++------- .../src/aggregates/group_values/row_like.rs | 1 + datafusion/sqllogictest/test_files/test1.slt | 14 ++++++ 3 files changed, 46 insertions(+), 19 deletions(-) diff --git a/datafusion/physical-expr-common/src/group_value_row.rs b/datafusion/physical-expr-common/src/group_value_row.rs index 955836ea515b..e8fe5ffc53d2 100644 --- a/datafusion/physical-expr-common/src/group_value_row.rs +++ b/datafusion/physical-expr-common/src/group_value_row.rs @@ -45,34 +45,35 @@ pub trait ArrayRowEq: Send + Sync { fn take_n(&mut self, n: usize) -> ArrayRef; } -pub struct PrimitiveGroupValueBuilder(Vec>); +pub struct PrimitiveGroupValueBuilder(Vec, Vec, bool); impl Default for PrimitiveGroupValueBuilder { fn default() -> Self { - Self(vec![]) + Self(vec![], vec![], false) } } impl ArrayRowEq for PrimitiveGroupValueBuilder { fn equal_to(&self, lhs_row: usize, array: &ArrayRef, rhs_row: usize) -> bool { - let elem = self.0[lhs_row]; - let arr = array.as_primitive::(); - let is_rhs_null = arr.is_null(rhs_row); - if elem.is_none() && is_rhs_null { - true - } else if elem.is_some() && !is_rhs_null { - elem.unwrap() == arr.value(rhs_row) - } else { - false + if self.1[lhs_row] { + if array.is_null(rhs_row) { + return false + } + + return self.0[lhs_row] == array.as_primitive::().value(rhs_row); } + + array.is_null(rhs_row) } fn append_val(&mut self, array: &ArrayRef, row: usize) { - let arr = array.as_primitive::(); - if arr.is_null(row) { - self.0.push(None) + if array.is_null(row) { + self.0.push(T::default_value()); + self.1.push(false); + self.2 = true; } else { - let elem = arr.value(row); - self.0.push(Some(elem)) + let elem = array.as_primitive::().value(row); + self.0.push(elem); + self.1.push(true); } } @@ -85,12 +86,23 @@ impl ArrayRowEq for PrimitiveGroupValueBuilder { } fn build(self: Box) -> ArrayRef { - Arc::new(PrimitiveArray::::from_iter(self.0)) + if self.2 { + Arc::new(PrimitiveArray::::new(ScalarBuffer::from(self.0), Some(NullBuffer::from(self.1)))) + } else { + Arc::new(PrimitiveArray::::new(ScalarBuffer::from(self.0), None)) + } } fn take_n(&mut self, n: usize) -> ArrayRef { - let first_n = self.0.drain(0..n).collect::>(); - Arc::new(PrimitiveArray::::from_iter(first_n)) + if self.2 { + let first_n = self.0.drain(0..n).collect::>(); + let first_n_nulls = self.1.drain(0..n).collect::>(); + Arc::new(PrimitiveArray::::new(ScalarBuffer::from(first_n), Some(NullBuffer::from(first_n_nulls)))) + } else { + let first_n = self.0.drain(0..n).collect::>(); + self.1 = vec![true; self.1.len() - n]; + Arc::new(PrimitiveArray::::new(ScalarBuffer::from(first_n), None)) + } } } diff --git a/datafusion/physical-plan/src/aggregates/group_values/row_like.rs b/datafusion/physical-plan/src/aggregates/group_values/row_like.rs index 52878c870ee4..6440c53f1e01 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/row_like.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/row_like.rs @@ -93,6 +93,7 @@ impl GroupValues for GroupValuesRowLike { None => { let len = cols.len(); let mut v = Vec::with_capacity(len); + for f in self.schema.fields().iter() { match f.data_type() { &DataType::Int8 => { diff --git a/datafusion/sqllogictest/test_files/test1.slt b/datafusion/sqllogictest/test_files/test1.slt index 10a668ef08a8..21f17d038800 100644 --- a/datafusion/sqllogictest/test_files/test1.slt +++ b/datafusion/sqllogictest/test_files/test1.slt @@ -4,6 +4,20 @@ CREATE EXTERNAL TABLE hits STORED AS PARQUET LOCATION '../../benchmarks/data/hits.parquet'; +query IIIII +SELECT "ClientIP", "ClientIP" - 1, "ClientIP" - 2, "ClientIP" - 3, COUNT(*) AS c FROM hits GROUP BY "ClientIP", "ClientIP" - 1, "ClientIP" - 2, "ClientIP" - 3 ORDER BY c DESC LIMIT 10; +---- +-39921974 -39921975 -39921976 -39921977 47008 +-1698104457 -1698104458 -1698104459 -1698104460 29121 +-1175819552 -1175819553 -1175819554 -1175819555 25333 +1696638182 1696638181 1696638180 1696638179 20220 +1138507705 1138507704 1138507703 1138507702 15778 +-927025522 -927025523 -927025524 -927025525 12768 +-1262139876 -1262139877 -1262139878 -1262139879 11348 +1740861572 1740861571 1740861570 1740861569 11314 +-807147100 -807147101 -807147102 -807147103 9880 +-631062503 -631062504 -631062505 -631062506 9718 + query ITI SELECT "UserID", "SearchPhrase", COUNT(*) FROM hits GROUP BY "UserID", "SearchPhrase" ORDER BY COUNT(*) DESC LIMIT 10; ---- From 7602a184e3df24dc6ca5142a40f6208d25388ac4 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Mon, 16 Sep 2024 09:16:35 +0800 Subject: [PATCH 08/30] use truncate Signed-off-by: jayzhan211 --- datafusion/physical-expr-common/src/group_value_row.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/physical-expr-common/src/group_value_row.rs b/datafusion/physical-expr-common/src/group_value_row.rs index e8fe5ffc53d2..0872892e95af 100644 --- a/datafusion/physical-expr-common/src/group_value_row.rs +++ b/datafusion/physical-expr-common/src/group_value_row.rs @@ -100,7 +100,7 @@ impl ArrayRowEq for PrimitiveGroupValueBuilder { Arc::new(PrimitiveArray::::new(ScalarBuffer::from(first_n), Some(NullBuffer::from(first_n_nulls)))) } else { let first_n = self.0.drain(0..n).collect::>(); - self.1 = vec![true; self.1.len() - n]; + self.1.truncate(self.1.len() - n); Arc::new(PrimitiveArray::::new(ScalarBuffer::from(first_n), None)) } } From 7a4dbd59ec646bc6ac56a1576cf582154ad65848 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Mon, 16 Sep 2024 09:20:39 +0800 Subject: [PATCH 09/30] cleanup Signed-off-by: jayzhan211 --- .../src/group_value_row.rs | 52 ++++++++++--------- 1 file changed, 27 insertions(+), 25 deletions(-) diff --git a/datafusion/physical-expr-common/src/group_value_row.rs b/datafusion/physical-expr-common/src/group_value_row.rs index 0872892e95af..6904d77f43fa 100644 --- a/datafusion/physical-expr-common/src/group_value_row.rs +++ b/datafusion/physical-expr-common/src/group_value_row.rs @@ -45,7 +45,11 @@ pub trait ArrayRowEq: Send + Sync { fn take_n(&mut self, n: usize) -> ArrayRef; } -pub struct PrimitiveGroupValueBuilder(Vec, Vec, bool); +pub struct PrimitiveGroupValueBuilder( + Vec, + Vec, + bool, +); impl Default for PrimitiveGroupValueBuilder { fn default() -> Self { Self(vec![], vec![], false) @@ -56,7 +60,7 @@ impl ArrayRowEq for PrimitiveGroupValueBuilder { fn equal_to(&self, lhs_row: usize, array: &ArrayRef, rhs_row: usize) -> bool { if self.1[lhs_row] { if array.is_null(rhs_row) { - return false + return false; } return self.0[lhs_row] == array.as_primitive::().value(rhs_row); @@ -87,7 +91,10 @@ impl ArrayRowEq for PrimitiveGroupValueBuilder { fn build(self: Box) -> ArrayRef { if self.2 { - Arc::new(PrimitiveArray::::new(ScalarBuffer::from(self.0), Some(NullBuffer::from(self.1)))) + Arc::new(PrimitiveArray::::new( + ScalarBuffer::from(self.0), + Some(NullBuffer::from(self.1)), + )) } else { Arc::new(PrimitiveArray::::new(ScalarBuffer::from(self.0), None)) } @@ -97,7 +104,10 @@ impl ArrayRowEq for PrimitiveGroupValueBuilder { if self.2 { let first_n = self.0.drain(0..n).collect::>(); let first_n_nulls = self.1.drain(0..n).collect::>(); - Arc::new(PrimitiveArray::::new(ScalarBuffer::from(first_n), Some(NullBuffer::from(first_n_nulls)))) + Arc::new(PrimitiveArray::::new( + ScalarBuffer::from(first_n), + Some(NullBuffer::from(first_n_nulls)), + )) } else { let first_n = self.0.drain(0..n).collect::>(); self.1.truncate(self.1.len() - n); @@ -111,8 +121,7 @@ where O: OffsetSizeTrait, { output_type: OutputType, - buffer_v2: BufferBuilder, - // buffer: Vec, + buffer: BufferBuilder, /// Offsets into `buffer` for each distinct value. These offsets as used /// directly to create the final `GenericBinaryArray`. The `i`th string is /// stored in the range `offsets[i]..offsets[i+1]` in `buffer`. Null values @@ -129,8 +138,7 @@ where pub fn new(output_type: OutputType) -> Self { Self { output_type, - buffer_v2: BufferBuilder::new(INITIAL_BUFFER_CAPACITY), - // buffer: Vec::with_capacity(INITIAL_BUFFER_CAPACITY), + buffer: BufferBuilder::new(INITIAL_BUFFER_CAPACITY), offsets: vec![O::default()], nulls: vec![], } @@ -144,16 +152,15 @@ where if arr.is_null(row) { self.nulls.push(self.len()); // nulls need a zero length in the offset buffer - let offset = self.buffer_v2.len(); + let offset = self.buffer.len(); self.offsets.push(O::usize_as(offset)); return; } let value: &[u8] = arr.value(row).as_ref(); - // self.buffer.extend_from_slice(value); - self.buffer_v2.append_slice(value); - self.offsets.push(O::usize_as(self.buffer_v2.len())); + self.buffer.append_slice(value); + self.offsets.push(O::usize_as(self.buffer.len())); } fn equal_to_inner(&self, lhs_row: usize, array: &ArrayRef, rhs_row: usize) -> bool @@ -175,7 +182,7 @@ where assert_eq!(rhs_elem_len, rhs_elem.len()); let l = self.offsets[lhs_row].as_usize(); let r = self.offsets[lhs_row + 1].as_usize(); - let existing_elem = unsafe { self.buffer_v2.as_slice().get_unchecked(l..r) }; + let existing_elem = unsafe { self.buffer.as_slice().get_unchecked(l..r) }; existing_elem.len() == rhs_elem.len() && rhs_elem == existing_elem } } @@ -237,8 +244,7 @@ where fn build(self: Box) -> ArrayRef { let Self { output_type, - mut buffer_v2, - // buffer, + mut buffer, offsets, nulls, } = *self; @@ -259,8 +265,7 @@ where // SAFETY: the offsets were constructed correctly in `insert_if_new` -- // monotonically increasing, overflows were checked. let offsets = unsafe { OffsetBuffer::new_unchecked(ScalarBuffer::from(offsets)) }; - // let values = Buffer::from_vec(buffer); - let values = buffer_v2.finish(); + let values = buffer.finish(); match output_type { OutputType::Binary => { @@ -315,14 +320,11 @@ where // Consume first (n - nulls count) of elements since we don't push any value for null case. let r = n - self.nulls.len(); - - let mut remaining_buffer = BufferBuilder::new(self.buffer_v2.len() - r); - remaining_buffer.append_slice(&self.buffer_v2.as_slice()[r..]); - self.buffer_v2.truncate(r); - let values = self.buffer_v2.finish(); - - // let first_n_valid_buffer = self.buffer.drain(0..(n - self.nulls.len())).collect(); - // let values = Buffer::from_vec(first_n_valid_buffer); + + let mut remaining_buffer = BufferBuilder::new(self.buffer.len() - r); + remaining_buffer.append_slice(&self.buffer.as_slice()[r..]); + self.buffer.truncate(r); + let values = self.buffer.finish(); match self.output_type { OutputType::Binary => { From 05fb4663484bbd7282b7d96babb46a58a3184ad5 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Mon, 16 Sep 2024 10:51:25 +0800 Subject: [PATCH 10/30] cleanup Signed-off-by: jayzhan211 --- .../src/group_value_row.rs | 8 ++-- .../src/aggregates/group_values/row_like.rs | 26 ++++------ .../physical-plan/src/aggregates/row_hash.rs | 1 - datafusion/sqllogictest/test_files/test1.slt | 47 ------------------- datafusion/sqllogictest/test_files/test2.slt | 19 -------- 5 files changed, 13 insertions(+), 88 deletions(-) delete mode 100644 datafusion/sqllogictest/test_files/test1.slt delete mode 100644 datafusion/sqllogictest/test_files/test2.slt diff --git a/datafusion/physical-expr-common/src/group_value_row.rs b/datafusion/physical-expr-common/src/group_value_row.rs index 6904d77f43fa..27d607b86ed3 100644 --- a/datafusion/physical-expr-common/src/group_value_row.rs +++ b/datafusion/physical-expr-common/src/group_value_row.rs @@ -36,6 +36,7 @@ use std::sync::Arc; use crate::binary_map::OutputType; use crate::binary_map::INITIAL_BUFFER_CAPACITY; +/// Trait for group values column-wise row comparison pub trait ArrayRowEq: Send + Sync { fn equal_to(&self, lhs_row: usize, array: &ArrayRef, rhs_row: usize) -> bool; fn append_val(&mut self, array: &ArrayRef, row: usize); @@ -46,10 +47,11 @@ pub trait ArrayRowEq: Send + Sync { } pub struct PrimitiveGroupValueBuilder( - Vec, - Vec, - bool, + Vec, // group value + Vec, // null value + bool, // whether the array contains at least one null, for fast non-null path ); + impl Default for PrimitiveGroupValueBuilder { fn default() -> Self { Self(vec![], vec![], false) diff --git a/datafusion/physical-plan/src/aggregates/group_values/row_like.rs b/datafusion/physical-plan/src/aggregates/group_values/row_like.rs index 6440c53f1e01..e8a443a4ab61 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/row_like.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/row_like.rs @@ -53,22 +53,16 @@ pub struct GroupValuesRowLike { /// The size of `map` in bytes map_size: usize, - /// The actual group by values, stored in arrow [`Row`] format. - /// `group_values[i]` holds the group value for group_index `i`. - /// - /// The row format is used to compare group keys quickly and store - /// them efficiently in memory. Quick comparison is especially - /// important for multi-column group keys. - /// - /// [`Row`]: arrow::row::Row - // group_values: Option, + /// The actual group by values, stored column-wise. Compare from + /// the left to right, each column is stored as `ArrayRowEq`. + /// This is shown faster than the row format + group_values: Option>>, /// reused buffer to store hashes hashes_buffer: Vec, /// Random state for creating hashes random_state: RandomState, - group_values: Option>>, } impl GroupValuesRowLike { @@ -177,22 +171,18 @@ impl GroupValues for GroupValuesRowLike { if target_hash != *exist_hash { return false; } - // verify that the group that we are inserting with hash is - // actually the same key value as the group in - // existing_idx (aka group_values @ row) - // && group_rows.row(row) == group_values.row(*group_idx) - fn compare_equal( - arry_eq: &dyn ArrayRowEq, + fn check_row_equal( + array_row: &dyn ArrayRowEq, lhs_row: usize, array: &ArrayRef, rhs_row: usize, ) -> bool { - arry_eq.equal_to(lhs_row, array, rhs_row) + array_row.equal_to(lhs_row, array, rhs_row) } for (i, group_val) in group_values.iter().enumerate() { - if !compare_equal(group_val.as_ref(), *group_idx, &cols[i], row) { + if !check_row_equal(group_val.as_ref(), *group_idx, &cols[i], row) { return false; } } diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index d90a3242aa50..0332131d4b57 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -618,7 +618,6 @@ impl Stream for GroupedHashAggregateStream { match ready!(self.input.poll_next_unpin(cx)) { // new batch to aggregate Some(Ok(batch)) => { - // println!("stat"); let timer = elapsed_compute.timer(); let input_rows = batch.num_rows(); diff --git a/datafusion/sqllogictest/test_files/test1.slt b/datafusion/sqllogictest/test_files/test1.slt deleted file mode 100644 index 21f17d038800..000000000000 --- a/datafusion/sqllogictest/test_files/test1.slt +++ /dev/null @@ -1,47 +0,0 @@ - -statement ok -CREATE EXTERNAL TABLE hits -STORED AS PARQUET -LOCATION '../../benchmarks/data/hits.parquet'; - -query IIIII -SELECT "ClientIP", "ClientIP" - 1, "ClientIP" - 2, "ClientIP" - 3, COUNT(*) AS c FROM hits GROUP BY "ClientIP", "ClientIP" - 1, "ClientIP" - 2, "ClientIP" - 3 ORDER BY c DESC LIMIT 10; ----- --39921974 -39921975 -39921976 -39921977 47008 --1698104457 -1698104458 -1698104459 -1698104460 29121 --1175819552 -1175819553 -1175819554 -1175819555 25333 -1696638182 1696638181 1696638180 1696638179 20220 -1138507705 1138507704 1138507703 1138507702 15778 --927025522 -927025523 -927025524 -927025525 12768 --1262139876 -1262139877 -1262139878 -1262139879 11348 -1740861572 1740861571 1740861570 1740861569 11314 --807147100 -807147101 -807147102 -807147103 9880 --631062503 -631062504 -631062505 -631062506 9718 - -query ITI -SELECT "UserID", "SearchPhrase", COUNT(*) FROM hits GROUP BY "UserID", "SearchPhrase" ORDER BY COUNT(*) DESC LIMIT 10; ----- -1313338681122956954 (empty) 29097 -1907779576417363396 (empty) 25333 -2305303682471783379 (empty) 10597 -7982623143712728547 (empty) 6669 -7280399273658728997 (empty) 6408 -1090981537032625727 (empty) 6196 -5730251990344211405 (empty) 6019 -6018350421959114808 (empty) 5990 -835157184735512989 (empty) 5209 -770542365400669095 (empty) 4906 - -query IIIII -SELECT "ClientIP", "ClientIP" - 1, "ClientIP" - 2, "ClientIP" - 3, COUNT(*) AS c FROM hits GROUP BY "ClientIP", "ClientIP" - 1, "ClientIP" - 2, "ClientIP" - 3 ORDER BY c DESC LIMIT 10; ----- --39921974 -39921975 -39921976 -39921977 47008 --1698104457 -1698104458 -1698104459 -1698104460 29121 --1175819552 -1175819553 -1175819554 -1175819555 25333 -1696638182 1696638181 1696638180 1696638179 20220 -1138507705 1138507704 1138507703 1138507702 15778 --927025522 -927025523 -927025524 -927025525 12768 --1262139876 -1262139877 -1262139878 -1262139879 11348 -1740861572 1740861571 1740861570 1740861569 11314 --807147100 -807147101 -807147102 -807147103 9880 --631062503 -631062504 -631062505 -631062506 9718 diff --git a/datafusion/sqllogictest/test_files/test2.slt b/datafusion/sqllogictest/test_files/test2.slt deleted file mode 100644 index 572cf25b539a..000000000000 --- a/datafusion/sqllogictest/test_files/test2.slt +++ /dev/null @@ -1,19 +0,0 @@ -query ok -create table multi_null_data(c1 int, c2 string) as values - (0, NULL), - (0, NULL), - (3, 'foo'), - (NULL, NULL), - (NULL, 'bar'), - (3, 'foo'), - (0, NULL), - (NULL, 'bar'), - (3, 'foo'); - -query IIT rowsort -SELECT COUNT(*), c1, c2 FROM multi_null_data GROUP BY c1, c2 ----- -1 NULL NULL -2 NULL bar -3 0 NULL -3 3 foo \ No newline at end of file From 8efcf07ddce47744465a56112be2699f90c51bb8 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Mon, 16 Sep 2024 17:59:42 +0800 Subject: [PATCH 11/30] fix first N bug Signed-off-by: jayzhan211 --- .../src/group_value_row.rs | 19 ++++++++++++--- datafusion/sqllogictest/test_files/window.slt | 23 ++++++++++++++++++- 2 files changed, 38 insertions(+), 4 deletions(-) diff --git a/datafusion/physical-expr-common/src/group_value_row.rs b/datafusion/physical-expr-common/src/group_value_row.rs index 27d607b86ed3..79ec25dc6d5f 100644 --- a/datafusion/physical-expr-common/src/group_value_row.rs +++ b/datafusion/physical-expr-common/src/group_value_row.rs @@ -268,7 +268,6 @@ where // monotonically increasing, overflows were checked. let offsets = unsafe { OffsetBuffer::new_unchecked(ScalarBuffer::from(offsets)) }; let values = buffer.finish(); - match output_type { OutputType::Binary => { // SAFETY: the offsets were constructed correctly @@ -292,6 +291,9 @@ where } fn take_n(&mut self, n: usize) -> ArrayRef { + assert!(self.len() >= n); + + let mut nulls_count = 0; let null_buffer = if self.nulls.is_empty() { None } else { @@ -299,9 +301,19 @@ where let num_values = self.offsets.len() - 1; let mut bool_builder = BooleanBufferBuilder::new(num_values); bool_builder.append_n(num_values, true); + + let last_offset = O::as_usize(self.offsets[n]); + let mut new_nulls = vec![]; self.nulls.iter().for_each(|null_index| { - bool_builder.set_bit(*null_index, false); + if *null_index <= last_offset { + nulls_count += 1; + bool_builder.set_bit(*null_index, false); + } else { + new_nulls.push(null_index - last_offset); + } }); + + self.nulls = new_nulls; Some(NullBuffer::from(bool_builder.finish())) }; @@ -321,12 +333,13 @@ where unsafe { OffsetBuffer::new_unchecked(ScalarBuffer::from(first_n_offsets)) }; // Consume first (n - nulls count) of elements since we don't push any value for null case. - let r = n - self.nulls.len(); + let r = n - nulls_count; let mut remaining_buffer = BufferBuilder::new(self.buffer.len() - r); remaining_buffer.append_slice(&self.buffer.as_slice()[r..]); self.buffer.truncate(r); let values = self.buffer.finish(); + self.buffer = remaining_buffer; match self.output_type { OutputType::Binary => { diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 505c66aef058..046a2eaa18b2 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -4872,4 +4872,25 @@ query error DataFusion error: Execution error: Expected a signed integer literal SELECT NTH_VALUE('+Inf'::Double, v1) OVER (PARTITION BY v1) FROM t1; statement ok -DROP TABLE t1; \ No newline at end of file +DROP TABLE t1; + +statement ok +create table t(c1 int, c2 varchar) as values (1, 'a'), (2, 'b'), (1, 'a'), (3, null), (null, 'a4'), (null, 'de'); + +# test multi group FirstN mode with nulls +query ITI +SELECT * +FROM (SELECT c1, c2, ROW_NUMBER() OVER() as rn + FROM t + LIMIT 5) +GROUP BY rn +ORDER BY rn; +---- +1 a 1 +2 b 2 +1 a 3 +3 (empty) 4 +NULL a4 5 + +statement ok +drop table t From bc16d55287fc770abb6580907959ea40aa426654 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Tue, 17 Sep 2024 08:28:48 +0800 Subject: [PATCH 12/30] fix null check Signed-off-by: jayzhan211 --- datafusion/physical-expr-common/src/group_value_row.rs | 10 +++++++--- datafusion/sqllogictest/test_files/window.slt | 2 +- 2 files changed, 8 insertions(+), 4 deletions(-) diff --git a/datafusion/physical-expr-common/src/group_value_row.rs b/datafusion/physical-expr-common/src/group_value_row.rs index 79ec25dc6d5f..4bc16b5eccd7 100644 --- a/datafusion/physical-expr-common/src/group_value_row.rs +++ b/datafusion/physical-expr-common/src/group_value_row.rs @@ -302,14 +302,18 @@ where let mut bool_builder = BooleanBufferBuilder::new(num_values); bool_builder.append_n(num_values, true); - let last_offset = O::as_usize(self.offsets[n]); + let nth_offset = O::as_usize(self.offsets[n]); + // Given offsets [0, 1, 2, 2], we could know that the 3rd index is null since the offset diff is 0 + let is_nth_offset_null = O::as_usize(self.offsets[n - 1]) == nth_offset; let mut new_nulls = vec![]; self.nulls.iter().for_each(|null_index| { - if *null_index <= last_offset { + if *null_index < nth_offset + || (*null_index == nth_offset && is_nth_offset_null) + { nulls_count += 1; bool_builder.set_bit(*null_index, false); } else { - new_nulls.push(null_index - last_offset); + new_nulls.push(null_index - nth_offset); } }); diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 046a2eaa18b2..7fee84f9bcd9 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -4889,7 +4889,7 @@ ORDER BY rn; 1 a 1 2 b 2 1 a 3 -3 (empty) 4 +3 NULL 4 NULL a4 5 statement ok From fb944859db5ad9409513bf83e23e4c8e4c8b1029 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Tue, 17 Sep 2024 16:01:59 +0800 Subject: [PATCH 13/30] fast path null Signed-off-by: jayzhan211 --- .../src/group_value_row.rs | 80 ++++++++++++------- .../src/aggregates/group_values/mod.rs | 4 + .../src/aggregates/group_values/row_like.rs | 36 ++++++--- 3 files changed, 80 insertions(+), 40 deletions(-) diff --git a/datafusion/physical-expr-common/src/group_value_row.rs b/datafusion/physical-expr-common/src/group_value_row.rs index 4bc16b5eccd7..f5cfe504970f 100644 --- a/datafusion/physical-expr-common/src/group_value_row.rs +++ b/datafusion/physical-expr-common/src/group_value_row.rs @@ -32,6 +32,7 @@ use arrow::datatypes::GenericBinaryType; use arrow::datatypes::GenericStringType; use std::sync::Arc; +use std::vec; use crate::binary_map::OutputType; use crate::binary_map::INITIAL_BUFFER_CAPACITY; @@ -46,45 +47,65 @@ pub trait ArrayRowEq: Send + Sync { fn take_n(&mut self, n: usize) -> ArrayRef; } -pub struct PrimitiveGroupValueBuilder( - Vec, // group value - Vec, // null value - bool, // whether the array contains at least one null, for fast non-null path -); +pub struct PrimitiveGroupValueBuilder { + group_values: Vec, + nulls: Vec, + // whether the array contains at least one null, for fast non-null path + has_null: bool, + nullable: bool, +} -impl Default for PrimitiveGroupValueBuilder { - fn default() -> Self { - Self(vec![], vec![], false) +impl PrimitiveGroupValueBuilder +where + T: ArrowPrimitiveType, +{ + pub fn new(nullable: bool) -> Self { + Self { + group_values: vec![], + nulls: vec![], + has_null: false, + nullable, + } } } impl ArrayRowEq for PrimitiveGroupValueBuilder { fn equal_to(&self, lhs_row: usize, array: &ArrayRef, rhs_row: usize) -> bool { - if self.1[lhs_row] { + // non-null fast path + if !self.nullable || !self.has_null { + return self.group_values[lhs_row] + == array.as_primitive::().value(rhs_row); + } + + if self.nulls[lhs_row] { if array.is_null(rhs_row) { return false; } - return self.0[lhs_row] == array.as_primitive::().value(rhs_row); + return self.group_values[lhs_row] + == array.as_primitive::().value(rhs_row); } array.is_null(rhs_row) } fn append_val(&mut self, array: &ArrayRef, row: usize) { - if array.is_null(row) { - self.0.push(T::default_value()); - self.1.push(false); - self.2 = true; - } else { - let elem = array.as_primitive::().value(row); - self.0.push(elem); - self.1.push(true); + // non-null fast path + if !self.nullable || !self.has_null { + } else if array.is_null(row) { + self.group_values.push(T::default_value()); + self.nulls.push(false); + self.has_null = true; + return; } + + let elem = array.as_primitive::().value(row); + self.group_values.push(elem); + self.nulls.push(true); } fn len(&self) -> usize { - self.0.len() + self.group_values.len() } fn is_empty(&self) -> bool { @@ -92,27 +113,30 @@ impl ArrayRowEq for PrimitiveGroupValueBuilder { } fn build(self: Box) -> ArrayRef { - if self.2 { + if self.nullable && self.has_null { Arc::new(PrimitiveArray::::new( - ScalarBuffer::from(self.0), - Some(NullBuffer::from(self.1)), + ScalarBuffer::from(self.group_values), + Some(NullBuffer::from(self.nulls)), )) } else { - Arc::new(PrimitiveArray::::new(ScalarBuffer::from(self.0), None)) + Arc::new(PrimitiveArray::::new( + ScalarBuffer::from(self.group_values), + None, + )) } } fn take_n(&mut self, n: usize) -> ArrayRef { - if self.2 { - let first_n = self.0.drain(0..n).collect::>(); - let first_n_nulls = self.1.drain(0..n).collect::>(); + if self.nullable && self.has_null { + let first_n = self.group_values.drain(0..n).collect::>(); + let first_n_nulls = self.nulls.drain(0..n).collect::>(); Arc::new(PrimitiveArray::::new( ScalarBuffer::from(first_n), Some(NullBuffer::from(first_n_nulls)), )) } else { - let first_n = self.0.drain(0..n).collect::>(); - self.1.truncate(self.1.len() - n); + let first_n = self.group_values.drain(0..n).collect::>(); + self.nulls.truncate(self.nulls.len() - n); Arc::new(PrimitiveArray::::new(ScalarBuffer::from(first_n), None)) } } diff --git a/datafusion/physical-plan/src/aggregates/group_values/mod.rs b/datafusion/physical-plan/src/aggregates/group_values/mod.rs index 84bbf221f8fd..f3300b56e109 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/mod.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/mod.rs @@ -117,8 +117,12 @@ fn has_row_like_feature(data_type: &DataType) -> bool { | DataType::UInt16 | DataType::UInt32 | DataType::UInt64 + | DataType::Float32 + | DataType::Float64 | DataType::Utf8 | DataType::LargeUtf8 + | DataType::Binary + | DataType::LargeBinary | DataType::Date32 | DataType::Date64 ) diff --git a/datafusion/physical-plan/src/aggregates/group_values/row_like.rs b/datafusion/physical-plan/src/aggregates/group_values/row_like.rs index e8a443a4ab61..233627cb5398 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/row_like.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/row_like.rs @@ -89,53 +89,65 @@ impl GroupValues for GroupValuesRowLike { let mut v = Vec::with_capacity(len); for f in self.schema.fields().iter() { + let nullable = f.is_nullable(); match f.data_type() { &DataType::Int8 => { - let b = PrimitiveGroupValueBuilder::::default(); + let b = PrimitiveGroupValueBuilder::::new(nullable); v.push(Box::new(b) as _) } &DataType::Int16 => { - let b = PrimitiveGroupValueBuilder::::default(); + let b = + PrimitiveGroupValueBuilder::::new(nullable); v.push(Box::new(b) as _) } &DataType::Int32 => { - let b = PrimitiveGroupValueBuilder::::default(); + let b = + PrimitiveGroupValueBuilder::::new(nullable); v.push(Box::new(b) as _) } &DataType::Int64 => { - let b = PrimitiveGroupValueBuilder::::default(); + let b = + PrimitiveGroupValueBuilder::::new(nullable); v.push(Box::new(b) as _) } &DataType::UInt8 => { - let b = PrimitiveGroupValueBuilder::::default(); + let b = + PrimitiveGroupValueBuilder::::new(nullable); v.push(Box::new(b) as _) } &DataType::UInt16 => { - let b = PrimitiveGroupValueBuilder::::default(); + let b = + PrimitiveGroupValueBuilder::::new(nullable); v.push(Box::new(b) as _) } &DataType::UInt32 => { - let b = PrimitiveGroupValueBuilder::::default(); + let b = + PrimitiveGroupValueBuilder::::new(nullable); v.push(Box::new(b) as _) } &DataType::UInt64 => { - let b = PrimitiveGroupValueBuilder::::default(); + let b = + PrimitiveGroupValueBuilder::::new(nullable); v.push(Box::new(b) as _) } &DataType::Float32 => { - let b = PrimitiveGroupValueBuilder::::default(); + let b = + PrimitiveGroupValueBuilder::::new(nullable); v.push(Box::new(b) as _) } &DataType::Float64 => { - let b = PrimitiveGroupValueBuilder::::default(); + let b = + PrimitiveGroupValueBuilder::::new(nullable); v.push(Box::new(b) as _) } &DataType::Date32 => { - let b = PrimitiveGroupValueBuilder::::default(); + let b = + PrimitiveGroupValueBuilder::::new(nullable); v.push(Box::new(b) as _) } &DataType::Date64 => { - let b = PrimitiveGroupValueBuilder::::default(); + let b = + PrimitiveGroupValueBuilder::::new(nullable); v.push(Box::new(b) as _) } &DataType::Utf8 => { From 07ed9660ce0968997e397acfa753d314a0cf3d2e Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Tue, 17 Sep 2024 16:37:03 +0800 Subject: [PATCH 14/30] fix bug Signed-off-by: jayzhan211 --- datafusion/core/tests/dataframe/mod.rs | 2 +- .../physical-expr-common/src/group_value_row.rs | 16 +++++++--------- 2 files changed, 8 insertions(+), 10 deletions(-) diff --git a/datafusion/core/tests/dataframe/mod.rs b/datafusion/core/tests/dataframe/mod.rs index 171ef9561e55..6e250293716d 100644 --- a/datafusion/core/tests/dataframe/mod.rs +++ b/datafusion/core/tests/dataframe/mod.rs @@ -676,7 +676,7 @@ async fn test_grouping_set_array_agg_with_overflow() -> Result<()> { let grouping_set_expr = Expr::GroupingSet(GroupingSet::GroupingSets(vec![ vec![col("c1")], - vec![col("c2")], + vec![col("c2")] , vec![col("c1"), col("c2")], ])); diff --git a/datafusion/physical-expr-common/src/group_value_row.rs b/datafusion/physical-expr-common/src/group_value_row.rs index f5cfe504970f..85503c2632d7 100644 --- a/datafusion/physical-expr-common/src/group_value_row.rs +++ b/datafusion/physical-expr-common/src/group_value_row.rs @@ -91,17 +91,15 @@ impl ArrayRowEq for PrimitiveGroupValueBuilder { fn append_val(&mut self, array: &ArrayRef, row: usize) { // non-null fast path - if !self.nullable || !self.has_null { - } else if array.is_null(row) { + if !self.nullable || !array.is_null(row) { + let elem = array.as_primitive::().value(row); + self.group_values.push(elem); + self.nulls.push(true); + } else { self.group_values.push(T::default_value()); self.nulls.push(false); self.has_null = true; - return; } - - let elem = array.as_primitive::().value(row); - self.group_values.push(elem); - self.nulls.push(true); } fn len(&self) -> usize { @@ -113,7 +111,7 @@ impl ArrayRowEq for PrimitiveGroupValueBuilder { } fn build(self: Box) -> ArrayRef { - if self.nullable && self.has_null { + if self.has_null { Arc::new(PrimitiveArray::::new( ScalarBuffer::from(self.group_values), Some(NullBuffer::from(self.nulls)), @@ -127,7 +125,7 @@ impl ArrayRowEq for PrimitiveGroupValueBuilder { } fn take_n(&mut self, n: usize) -> ArrayRef { - if self.nullable && self.has_null { + if self.has_null { let first_n = self.group_values.drain(0..n).collect::>(); let first_n_nulls = self.nulls.drain(0..n).collect::>(); Arc::new(PrimitiveArray::::new( From 925166d0e62dcdc518ad8306d506c6d90e168c68 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Tue, 17 Sep 2024 16:37:22 +0800 Subject: [PATCH 15/30] fmt Signed-off-by: jayzhan211 --- datafusion/core/tests/dataframe/mod.rs | 2 +- datafusion/physical-expr-common/src/group_value_row.rs | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion/core/tests/dataframe/mod.rs b/datafusion/core/tests/dataframe/mod.rs index 6e250293716d..171ef9561e55 100644 --- a/datafusion/core/tests/dataframe/mod.rs +++ b/datafusion/core/tests/dataframe/mod.rs @@ -676,7 +676,7 @@ async fn test_grouping_set_array_agg_with_overflow() -> Result<()> { let grouping_set_expr = Expr::GroupingSet(GroupingSet::GroupingSets(vec![ vec![col("c1")], - vec![col("c2")] , + vec![col("c2")], vec![col("c1"), col("c2")], ])); diff --git a/datafusion/physical-expr-common/src/group_value_row.rs b/datafusion/physical-expr-common/src/group_value_row.rs index 85503c2632d7..e4748ff8fb59 100644 --- a/datafusion/physical-expr-common/src/group_value_row.rs +++ b/datafusion/physical-expr-common/src/group_value_row.rs @@ -91,7 +91,7 @@ impl ArrayRowEq for PrimitiveGroupValueBuilder { fn append_val(&mut self, array: &ArrayRef, row: usize) { // non-null fast path - if !self.nullable || !array.is_null(row) { + if !self.nullable || !array.is_null(row) { let elem = array.as_primitive::().value(row); self.group_values.push(elem); self.nulls.push(true); From f59d11e3453b5435603a2a789f6c31bba71f815a Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Tue, 17 Sep 2024 17:21:32 +0800 Subject: [PATCH 16/30] fix error Signed-off-by: jayzhan211 --- .../physical-plan/src/aggregates/mod.rs | 24 +++++++------------ 1 file changed, 8 insertions(+), 16 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index c3bc7b042e65..eabaa93d997f 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -1380,24 +1380,16 @@ mod tests { "+---+-----+-----------------+", "| a | b | COUNT(1)[count] |", "+---+-----+-----------------+", - "| | 1.0 | 1 |", - "| | 1.0 | 1 |", - "| | 2.0 | 1 |", - "| | 2.0 | 1 |", - "| | 3.0 | 1 |", - "| | 3.0 | 1 |", - "| | 4.0 | 1 |", - "| | 4.0 | 1 |", - "| 2 | | 1 |", - "| 2 | | 1 |", - "| 2 | 1.0 | 1 |", - "| 2 | 1.0 | 1 |", - "| 3 | | 1 |", - "| 3 | | 2 |", + "| | 1.0 | 2 |", + "| | 2.0 | 2 |", + "| | 3.0 | 2 |", + "| | 4.0 | 2 |", + "| 2 | | 2 |", + "| 2 | 1.0 | 2 |", + "| 3 | | 3 |", "| 3 | 2.0 | 2 |", "| 3 | 3.0 | 1 |", - "| 4 | | 1 |", - "| 4 | | 2 |", + "| 4 | | 3 |", "| 4 | 3.0 | 1 |", "| 4 | 4.0 | 2 |", "+---+-----+-----------------+", From e9f9abc9f0448d1b5d676f018563a43186eb7eeb Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Tue, 17 Sep 2024 17:41:37 +0800 Subject: [PATCH 17/30] clippy Signed-off-by: jayzhan211 --- .../physical-plan/src/aggregates/mod.rs | 57 ++++++------------- 1 file changed, 18 insertions(+), 39 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index eabaa93d997f..97fb930c8786 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -1375,45 +1375,24 @@ mod tests { let result = common::collect(partial_aggregate.execute(0, Arc::clone(&task_ctx))?).await?; - let expected = if spill { - vec![ - "+---+-----+-----------------+", - "| a | b | COUNT(1)[count] |", - "+---+-----+-----------------+", - "| | 1.0 | 2 |", - "| | 2.0 | 2 |", - "| | 3.0 | 2 |", - "| | 4.0 | 2 |", - "| 2 | | 2 |", - "| 2 | 1.0 | 2 |", - "| 3 | | 3 |", - "| 3 | 2.0 | 2 |", - "| 3 | 3.0 | 1 |", - "| 4 | | 3 |", - "| 4 | 3.0 | 1 |", - "| 4 | 4.0 | 2 |", - "+---+-----+-----------------+", - ] - } else { - vec![ - "+---+-----+-----------------+", - "| a | b | COUNT(1)[count] |", - "+---+-----+-----------------+", - "| | 1.0 | 2 |", - "| | 2.0 | 2 |", - "| | 3.0 | 2 |", - "| | 4.0 | 2 |", - "| 2 | | 2 |", - "| 2 | 1.0 | 2 |", - "| 3 | | 3 |", - "| 3 | 2.0 | 2 |", - "| 3 | 3.0 | 1 |", - "| 4 | | 3 |", - "| 4 | 3.0 | 1 |", - "| 4 | 4.0 | 2 |", - "+---+-----+-----------------+", - ] - }; + let expected = vec![ + "+---+-----+-----------------+", + "| a | b | COUNT(1)[count] |", + "+---+-----+-----------------+", + "| | 1.0 | 2 |", + "| | 2.0 | 2 |", + "| | 3.0 | 2 |", + "| | 4.0 | 2 |", + "| 2 | | 2 |", + "| 2 | 1.0 | 2 |", + "| 3 | | 3 |", + "| 3 | 2.0 | 2 |", + "| 3 | 3.0 | 1 |", + "| 4 | | 3 |", + "| 4 | 3.0 | 1 |", + "| 4 | 4.0 | 2 |", + "+---+-----+-----------------+", + ]; assert_batches_sorted_eq!(expected, &result); let groups = partial_aggregate.group_expr().expr().to_vec(); From 720c3430771f73de6fefbdd9d1b84427282e73a3 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Wed, 18 Sep 2024 08:26:17 +0800 Subject: [PATCH 18/30] adjust spill mode max mem Signed-off-by: jayzhan211 --- .../core/src/datasource/physical_plan/csv.rs | 16 ++--- .../physical-plan/src/aggregates/mod.rs | 70 ++++++++++++++----- 2 files changed, 59 insertions(+), 27 deletions(-) diff --git a/datafusion/core/src/datasource/physical_plan/csv.rs b/datafusion/core/src/datasource/physical_plan/csv.rs index 6cd1864deb1d..0e51effdf41a 100644 --- a/datafusion/core/src/datasource/physical_plan/csv.rs +++ b/datafusion/core/src/datasource/physical_plan/csv.rs @@ -1329,15 +1329,15 @@ mod tests { let df = ctx.sql(r#"select * from t1"#).await?.collect().await?; let expected = [ - "+-------+-----------------------------+", - "| col1 | col2 |", - "+-------+-----------------------------+", - "| 1 | hello\rworld |", - "| 2 | something\relse |", + "+-------+------------------------+", + "| col1 | col2 |", + "+-------+------------------------+", + "| 1 | hello\rworld |", + "| 2 | something\relse |", "| 3 | \rmany\rlines\rmake\rgood test\r |", - "| 4 | unquoted |", - "| value | end |", - "+-------+-----------------------------+", + "| 4 | unquoted |", + "| value | end |", + "+-------+------------------------+", ]; crate::assert_batches_eq!(expected, &df); Ok(()) diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 97fb930c8786..29127a8d40e3 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -1358,7 +1358,8 @@ mod tests { .build()?]; let task_ctx = if spill { - new_spill_ctx(4, 1000) + // adjust the max memory size to have the partial aggregate result for spill mode. + new_spill_ctx(4, 500) } else { Arc::new(TaskContext::default()) }; @@ -1375,24 +1376,55 @@ mod tests { let result = common::collect(partial_aggregate.execute(0, Arc::clone(&task_ctx))?).await?; - let expected = vec![ - "+---+-----+-----------------+", - "| a | b | COUNT(1)[count] |", - "+---+-----+-----------------+", - "| | 1.0 | 2 |", - "| | 2.0 | 2 |", - "| | 3.0 | 2 |", - "| | 4.0 | 2 |", - "| 2 | | 2 |", - "| 2 | 1.0 | 2 |", - "| 3 | | 3 |", - "| 3 | 2.0 | 2 |", - "| 3 | 3.0 | 1 |", - "| 4 | | 3 |", - "| 4 | 3.0 | 1 |", - "| 4 | 4.0 | 2 |", - "+---+-----+-----------------+", - ]; + let expected = if spill { + // In spill mode, we test with the limited memory, if the mem usage exceeds, + // we trigger the early emit rule, which turns out the partial aggregate result. + vec![ + "+---+-----+-----------------+", + "| a | b | COUNT(1)[count] |", + "+---+-----+-----------------+", + "| | 1.0 | 1 |", + "| | 1.0 | 1 |", + "| | 2.0 | 1 |", + "| | 2.0 | 1 |", + "| | 3.0 | 1 |", + "| | 3.0 | 1 |", + "| | 4.0 | 1 |", + "| | 4.0 | 1 |", + "| 2 | | 1 |", + "| 2 | | 1 |", + "| 2 | 1.0 | 1 |", + "| 2 | 1.0 | 1 |", + "| 3 | | 1 |", + "| 3 | | 2 |", + "| 3 | 2.0 | 2 |", + "| 3 | 3.0 | 1 |", + "| 4 | | 1 |", + "| 4 | | 2 |", + "| 4 | 3.0 | 1 |", + "| 4 | 4.0 | 2 |", + "+---+-----+-----------------+", + ] + } else { + vec![ + "+---+-----+-----------------+", + "| a | b | COUNT(1)[count] |", + "+---+-----+-----------------+", + "| | 1.0 | 2 |", + "| | 2.0 | 2 |", + "| | 3.0 | 2 |", + "| | 4.0 | 2 |", + "| 2 | | 2 |", + "| 2 | 1.0 | 2 |", + "| 3 | | 3 |", + "| 3 | 2.0 | 2 |", + "| 3 | 3.0 | 1 |", + "| 4 | | 3 |", + "| 4 | 3.0 | 1 |", + "| 4 | 4.0 | 2 |", + "+---+-----+-----------------+", + ] + }; assert_batches_sorted_eq!(expected, &result); let groups = partial_aggregate.group_expr().expr().to_vec(); From 5b8aceb5fca4073bf71449df2c804b8cb1b6e866 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Wed, 18 Sep 2024 13:51:40 +0800 Subject: [PATCH 19/30] revert test_create_external_table_with_terminator_with_newlines_in_values Signed-off-by: jayzhan211 --- .../core/src/datasource/physical_plan/csv.rs | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/datafusion/core/src/datasource/physical_plan/csv.rs b/datafusion/core/src/datasource/physical_plan/csv.rs index 0e51effdf41a..6cd1864deb1d 100644 --- a/datafusion/core/src/datasource/physical_plan/csv.rs +++ b/datafusion/core/src/datasource/physical_plan/csv.rs @@ -1329,15 +1329,15 @@ mod tests { let df = ctx.sql(r#"select * from t1"#).await?.collect().await?; let expected = [ - "+-------+------------------------+", - "| col1 | col2 |", - "+-------+------------------------+", - "| 1 | hello\rworld |", - "| 2 | something\relse |", + "+-------+-----------------------------+", + "| col1 | col2 |", + "+-------+-----------------------------+", + "| 1 | hello\rworld |", + "| 2 | something\relse |", "| 3 | \rmany\rlines\rmake\rgood test\r |", - "| 4 | unquoted |", - "| value | end |", - "+-------+------------------------+", + "| 4 | unquoted |", + "| value | end |", + "+-------+-----------------------------+", ]; crate::assert_batches_eq!(expected, &df); Ok(()) From ed9b78ecdf18ef4c8f336a48d5ea3c7354d57272 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Wed, 18 Sep 2024 15:17:56 +0800 Subject: [PATCH 20/30] fix null handle bug Signed-off-by: jayzhan211 --- datafusion/physical-expr-common/src/group_value_row.rs | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/datafusion/physical-expr-common/src/group_value_row.rs b/datafusion/physical-expr-common/src/group_value_row.rs index e4748ff8fb59..441a31cb690c 100644 --- a/datafusion/physical-expr-common/src/group_value_row.rs +++ b/datafusion/physical-expr-common/src/group_value_row.rs @@ -72,12 +72,12 @@ where impl ArrayRowEq for PrimitiveGroupValueBuilder { fn equal_to(&self, lhs_row: usize, array: &ArrayRef, rhs_row: usize) -> bool { // non-null fast path - if !self.nullable || !self.has_null { + if !self.nullable { return self.group_values[lhs_row] == array.as_primitive::().value(rhs_row); } - if self.nulls[lhs_row] { + if !self.has_null || self.nulls[lhs_row] { if array.is_null(rhs_row) { return false; } @@ -125,6 +125,7 @@ impl ArrayRowEq for PrimitiveGroupValueBuilder { } fn take_n(&mut self, n: usize) -> ArrayRef { + println!("go"); if self.has_null { let first_n = self.group_values.drain(0..n).collect::>(); let first_n_nulls = self.nulls.drain(0..n).collect::>(); From fb1b745051d342bd5f6a990e2097b5d447540ffe Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Wed, 18 Sep 2024 15:18:34 +0800 Subject: [PATCH 21/30] cleanup Signed-off-by: jayzhan211 --- datafusion/physical-expr-common/src/group_value_row.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/datafusion/physical-expr-common/src/group_value_row.rs b/datafusion/physical-expr-common/src/group_value_row.rs index 441a31cb690c..b9d826d6f421 100644 --- a/datafusion/physical-expr-common/src/group_value_row.rs +++ b/datafusion/physical-expr-common/src/group_value_row.rs @@ -125,7 +125,6 @@ impl ArrayRowEq for PrimitiveGroupValueBuilder { } fn take_n(&mut self, n: usize) -> ArrayRef { - println!("go"); if self.has_null { let first_n = self.group_values.drain(0..n).collect::>(); let first_n_nulls = self.nulls.drain(0..n).collect::>(); From 2cab4c23b0e2bea6ae3740399f018c5b56dbb046 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Thu, 19 Sep 2024 07:40:10 +0800 Subject: [PATCH 22/30] support binary Signed-off-by: jayzhan211 --- datafusion/physical-expr-common/src/group_value_row.rs | 4 +++- .../physical-plan/src/aggregates/group_values/row_like.rs | 8 ++++++++ 2 files changed, 11 insertions(+), 1 deletion(-) diff --git a/datafusion/physical-expr-common/src/group_value_row.rs b/datafusion/physical-expr-common/src/group_value_row.rs index b9d826d6f421..b5c20bba7eab 100644 --- a/datafusion/physical-expr-common/src/group_value_row.rs +++ b/datafusion/physical-expr-common/src/group_value_row.rs @@ -72,12 +72,14 @@ where impl ArrayRowEq for PrimitiveGroupValueBuilder { fn equal_to(&self, lhs_row: usize, array: &ArrayRef, rhs_row: usize) -> bool { // non-null fast path + // both non-null if !self.nullable { return self.group_values[lhs_row] == array.as_primitive::().value(rhs_row); } - if !self.has_null || self.nulls[lhs_row] { + // lhs is non-null + if self.nulls[lhs_row] { if array.is_null(rhs_row) { return false; } diff --git a/datafusion/physical-plan/src/aggregates/group_values/row_like.rs b/datafusion/physical-plan/src/aggregates/group_values/row_like.rs index 233627cb5398..24f646e209c0 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/row_like.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/row_like.rs @@ -158,6 +158,14 @@ impl GroupValues for GroupValuesRowLike { let b = ByteGroupValueBuilder::::new(OutputType::Utf8); v.push(Box::new(b) as _) } + &DataType::Binary => { + let b = ByteGroupValueBuilder::::new(OutputType::Binary); + v.push(Box::new(b) as _) + } + &DataType::LargeBinary => { + let b = ByteGroupValueBuilder::::new(OutputType::Binary); + v.push(Box::new(b) as _) + } dt => todo!("{dt} not impl"), } } From 0090008d8cf015eb27fef9f0cd7a7a185f1fc3dc Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Thu, 19 Sep 2024 07:45:27 +0800 Subject: [PATCH 23/30] add binary test Signed-off-by: jayzhan211 --- .../sqllogictest/test_files/group_by.slt | 20 +++++++++++++++++++ 1 file changed, 20 insertions(+) diff --git a/datafusion/sqllogictest/test_files/group_by.slt b/datafusion/sqllogictest/test_files/group_by.slt index 73bfd9844609..cd23d76a1b5c 100644 --- a/datafusion/sqllogictest/test_files/group_by.slt +++ b/datafusion/sqllogictest/test_files/group_by.slt @@ -5148,3 +5148,23 @@ NULL statement ok drop table test_case_expr + +statement ok +drop table t; + +# test multi group by for binary type +statement ok +create table t(a int, b bytea) as values (1, 0xa), (1, 0xa), (2, null), (null, 0xb), (null, 0xb); + +query I?I +select a, b, count(*) from t group by grouping sets ((a, b), (a), (b)); +---- +1 0a 2 +2 NULL 2 +NULL 0b 4 +1 NULL 2 +NULL NULL 3 +NULL 0a 2 + +statement ok +drop table t; From e6d1890b595140b402575ed86731e3fa5fcf5250 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Sat, 21 Sep 2024 11:20:53 +0800 Subject: [PATCH 24/30] use Vec instead of Option> Signed-off-by: jayzhan211 --- .../src/group_value_row.rs | 10 +- .../src/aggregates/group_values/row_like.rs | 204 ++++++++---------- 2 files changed, 99 insertions(+), 115 deletions(-) diff --git a/datafusion/physical-expr-common/src/group_value_row.rs b/datafusion/physical-expr-common/src/group_value_row.rs index b5c20bba7eab..f48651e93244 100644 --- a/datafusion/physical-expr-common/src/group_value_row.rs +++ b/datafusion/physical-expr-common/src/group_value_row.rs @@ -205,7 +205,7 @@ where let arr = array.as_bytes::(); let rhs_elem: &[u8] = arr.value(rhs_row).as_ref(); let rhs_elem_len = arr.value_length(rhs_row).as_usize(); - assert_eq!(rhs_elem_len, rhs_elem.len()); + debug_assert_eq!(rhs_elem_len, rhs_elem.len()); let l = self.offsets[lhs_row].as_usize(); let r = self.offsets[lhs_row + 1].as_usize(); let existing_elem = unsafe { self.buffer.as_slice().get_unchecked(l..r) }; @@ -221,14 +221,14 @@ where // Sanity array type match self.output_type { OutputType::Binary => { - assert!(matches!( + debug_assert!(matches!( column.data_type(), DataType::Binary | DataType::LargeBinary )); self.equal_to_inner::>(lhs_row, column, rhs_row) } OutputType::Utf8 => { - assert!(matches!( + debug_assert!(matches!( column.data_type(), DataType::Utf8 | DataType::LargeUtf8 )); @@ -242,14 +242,14 @@ where // Sanity array type match self.output_type { OutputType::Binary => { - assert!(matches!( + debug_assert!(matches!( column.data_type(), DataType::Binary | DataType::LargeBinary )); self.append_val_inner::>(column, row) } OutputType::Utf8 => { - assert!(matches!( + debug_assert!(matches!( column.data_type(), DataType::Utf8 | DataType::LargeUtf8 )); diff --git a/datafusion/physical-plan/src/aggregates/group_values/row_like.rs b/datafusion/physical-plan/src/aggregates/group_values/row_like.rs index 24f646e209c0..8003c3baa69b 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/row_like.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/row_like.rs @@ -56,7 +56,7 @@ pub struct GroupValuesRowLike { /// The actual group by values, stored column-wise. Compare from /// the left to right, each column is stored as `ArrayRowEq`. /// This is shown faster than the row format - group_values: Option>>, + group_values: Vec>, /// reused buffer to store hashes hashes_buffer: Vec, @@ -72,7 +72,7 @@ impl GroupValuesRowLike { schema, map, map_size: 0, - group_values: None, + group_values: vec![], hashes_buffer: Default::default(), random_state: Default::default(), }) @@ -82,96 +82,83 @@ impl GroupValuesRowLike { impl GroupValues for GroupValuesRowLike { fn intern(&mut self, cols: &[ArrayRef], groups: &mut Vec) -> Result<()> { let n_rows = cols[0].len(); - let mut group_values = match self.group_values.take() { - Some(group_values) => group_values, - None => { - let len = cols.len(); - let mut v = Vec::with_capacity(len); - - for f in self.schema.fields().iter() { - let nullable = f.is_nullable(); - match f.data_type() { - &DataType::Int8 => { - let b = PrimitiveGroupValueBuilder::::new(nullable); - v.push(Box::new(b) as _) - } - &DataType::Int16 => { - let b = - PrimitiveGroupValueBuilder::::new(nullable); - v.push(Box::new(b) as _) - } - &DataType::Int32 => { - let b = - PrimitiveGroupValueBuilder::::new(nullable); - v.push(Box::new(b) as _) - } - &DataType::Int64 => { - let b = - PrimitiveGroupValueBuilder::::new(nullable); - v.push(Box::new(b) as _) - } - &DataType::UInt8 => { - let b = - PrimitiveGroupValueBuilder::::new(nullable); - v.push(Box::new(b) as _) - } - &DataType::UInt16 => { - let b = - PrimitiveGroupValueBuilder::::new(nullable); - v.push(Box::new(b) as _) - } - &DataType::UInt32 => { - let b = - PrimitiveGroupValueBuilder::::new(nullable); - v.push(Box::new(b) as _) - } - &DataType::UInt64 => { - let b = - PrimitiveGroupValueBuilder::::new(nullable); - v.push(Box::new(b) as _) - } - &DataType::Float32 => { - let b = - PrimitiveGroupValueBuilder::::new(nullable); - v.push(Box::new(b) as _) - } - &DataType::Float64 => { - let b = - PrimitiveGroupValueBuilder::::new(nullable); - v.push(Box::new(b) as _) - } - &DataType::Date32 => { - let b = - PrimitiveGroupValueBuilder::::new(nullable); - v.push(Box::new(b) as _) - } - &DataType::Date64 => { - let b = - PrimitiveGroupValueBuilder::::new(nullable); - v.push(Box::new(b) as _) - } - &DataType::Utf8 => { - let b = ByteGroupValueBuilder::::new(OutputType::Utf8); - v.push(Box::new(b) as _) - } - &DataType::LargeUtf8 => { - let b = ByteGroupValueBuilder::::new(OutputType::Utf8); - v.push(Box::new(b) as _) - } - &DataType::Binary => { - let b = ByteGroupValueBuilder::::new(OutputType::Binary); - v.push(Box::new(b) as _) - } - &DataType::LargeBinary => { - let b = ByteGroupValueBuilder::::new(OutputType::Binary); - v.push(Box::new(b) as _) - } - dt => todo!("{dt} not impl"), + + if self.group_values.is_empty() { + let len = cols.len(); + let mut v = Vec::with_capacity(len); + + for f in self.schema.fields().iter() { + let nullable = f.is_nullable(); + match f.data_type() { + &DataType::Int8 => { + let b = PrimitiveGroupValueBuilder::::new(nullable); + v.push(Box::new(b) as _) + } + &DataType::Int16 => { + let b = PrimitiveGroupValueBuilder::::new(nullable); + v.push(Box::new(b) as _) + } + &DataType::Int32 => { + let b = PrimitiveGroupValueBuilder::::new(nullable); + v.push(Box::new(b) as _) + } + &DataType::Int64 => { + let b = PrimitiveGroupValueBuilder::::new(nullable); + v.push(Box::new(b) as _) } + &DataType::UInt8 => { + let b = PrimitiveGroupValueBuilder::::new(nullable); + v.push(Box::new(b) as _) + } + &DataType::UInt16 => { + let b = PrimitiveGroupValueBuilder::::new(nullable); + v.push(Box::new(b) as _) + } + &DataType::UInt32 => { + let b = PrimitiveGroupValueBuilder::::new(nullable); + v.push(Box::new(b) as _) + } + &DataType::UInt64 => { + let b = PrimitiveGroupValueBuilder::::new(nullable); + v.push(Box::new(b) as _) + } + &DataType::Float32 => { + let b = PrimitiveGroupValueBuilder::::new(nullable); + v.push(Box::new(b) as _) + } + &DataType::Float64 => { + let b = PrimitiveGroupValueBuilder::::new(nullable); + v.push(Box::new(b) as _) + } + &DataType::Date32 => { + let b = PrimitiveGroupValueBuilder::::new(nullable); + v.push(Box::new(b) as _) + } + &DataType::Date64 => { + let b = PrimitiveGroupValueBuilder::::new(nullable); + v.push(Box::new(b) as _) + } + &DataType::Utf8 => { + let b = ByteGroupValueBuilder::::new(OutputType::Utf8); + v.push(Box::new(b) as _) + } + &DataType::LargeUtf8 => { + let b = ByteGroupValueBuilder::::new(OutputType::Utf8); + v.push(Box::new(b) as _) + } + &DataType::Binary => { + let b = ByteGroupValueBuilder::::new(OutputType::Binary); + v.push(Box::new(b) as _) + } + &DataType::LargeBinary => { + let b = ByteGroupValueBuilder::::new(OutputType::Binary); + v.push(Box::new(b) as _) + } + dt => todo!("{dt} not impl"), } - v } - }; + self.group_values = v; + } // tracks to which group each of the input rows belongs groups.clear(); @@ -201,7 +188,7 @@ impl GroupValues for GroupValuesRowLike { array_row.equal_to(lhs_row, array, rhs_row) } - for (i, group_val) in group_values.iter().enumerate() { + for (i, group_val) in self.group_values.iter().enumerate() { if !check_row_equal(group_val.as_ref(), *group_idx, &cols[i], row) { return false; } @@ -220,8 +207,8 @@ impl GroupValues for GroupValuesRowLike { // group_values.push(group_rows.row(row)); let mut checklen = 0; - let group_idx = group_values[0].len(); - for (i, group_value) in group_values.iter_mut().enumerate() { + let group_idx = self.group_values[0].len(); + for (i, group_value) in self.group_values.iter_mut().enumerate() { group_value.append_val(&cols[i], row); let len = group_value.len(); if i == 0 { @@ -243,12 +230,11 @@ impl GroupValues for GroupValuesRowLike { groups.push(group_idx); } - self.group_values = Some(group_values); Ok(()) } fn size(&self) -> usize { - let group_values_size = self.group_values.as_ref().map(|v| v.len()).unwrap_or(0); + let group_values_size = self.group_values.len(); group_values_size + self.map_size + self.hashes_buffer.allocated_size() } @@ -257,26 +243,27 @@ impl GroupValues for GroupValuesRowLike { } fn len(&self) -> usize { - self.group_values.as_ref().map(|v| v[0].len()).unwrap_or(0) + if self.group_values.is_empty() { + return 0; + } + + self.group_values[0].len() } fn emit(&mut self, emit_to: EmitTo) -> Result> { - let mut group_values_v2 = self - .group_values - .take() - .expect("Can not emit from empty rows"); - let mut output = match emit_to { EmitTo::All => { - let output = group_values_v2 + let group_values = std::mem::take(&mut self.group_values); + assert!(self.group_values.is_empty()); + + group_values .into_iter() .map(|v| v.build()) - .collect::>(); - self.group_values = None; - output + .collect::>() } EmitTo::First(n) => { - let output = group_values_v2 + let output = self + .group_values .iter_mut() .map(|v| v.take_n(n)) .collect::>(); @@ -293,7 +280,7 @@ impl GroupValues for GroupValuesRowLike { } } } - self.group_values = Some(group_values_v2); + output } }; @@ -317,10 +304,7 @@ impl GroupValues for GroupValuesRowLike { fn clear_shrink(&mut self, batch: &RecordBatch) { let count = batch.num_rows(); - self.group_values = self.group_values.take().map(|mut rows| { - rows.clear(); - rows - }); + self.group_values.clear(); self.map.clear(); self.map.shrink_to(count, |_| 0); // hasher does not matter since the map is cleared self.map_size = self.map.capacity() * std::mem::size_of::<(u64, usize)>(); From 77efb1a06cf932ecf92d4d27ec77afa68149e6b6 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Sat, 21 Sep 2024 11:28:25 +0800 Subject: [PATCH 25/30] add test and doc Signed-off-by: jayzhan211 --- .../src/group_value_row.rs | 13 ++++++++++++ .../sqllogictest/test_files/group_by.slt | 21 ++++++++++++++++++- 2 files changed, 33 insertions(+), 1 deletion(-) diff --git a/datafusion/physical-expr-common/src/group_value_row.rs b/datafusion/physical-expr-common/src/group_value_row.rs index f48651e93244..79d43715fd16 100644 --- a/datafusion/physical-expr-common/src/group_value_row.rs +++ b/datafusion/physical-expr-common/src/group_value_row.rs @@ -38,12 +38,25 @@ use crate::binary_map::OutputType; use crate::binary_map::INITIAL_BUFFER_CAPACITY; /// Trait for group values column-wise row comparison +/// +/// Implementations of this trait store a in-progress collection of group values +/// (similar to various builders in Arrow-rs) that allow for quick comparison to +/// incoming rows. +/// pub trait ArrayRowEq: Send + Sync { + /// Returns equal if the row stored in this builder at `lhs_row` is equal to + /// the row in `array` at `rhs_row` fn equal_to(&self, lhs_row: usize, array: &ArrayRef, rhs_row: usize) -> bool; + /// Appends the row at `row` in `array` to this builder fn append_val(&mut self, array: &ArrayRef, row: usize); + /// Returns the number of rows stored in this builder fn len(&self) -> usize; + /// Returns true if this builder is empty fn is_empty(&self) -> bool; + /// Builds a new array from all of the stored rows fn build(self: Box) -> ArrayRef; + /// Builds a new array from the first `n` stored rows, shifting the + /// remaining rows to the start of the builder fn take_n(&mut self, n: usize) -> ArrayRef; } diff --git a/datafusion/sqllogictest/test_files/group_by.slt b/datafusion/sqllogictest/test_files/group_by.slt index cd23d76a1b5c..94e811383c62 100644 --- a/datafusion/sqllogictest/test_files/group_by.slt +++ b/datafusion/sqllogictest/test_files/group_by.slt @@ -5152,7 +5152,7 @@ drop table test_case_expr statement ok drop table t; -# test multi group by for binary type +# test multi group by for binary type with nulls statement ok create table t(a int, b bytea) as values (1, 0xa), (1, 0xa), (2, null), (null, 0xb), (null, 0xb); @@ -5168,3 +5168,22 @@ NULL 0a 2 statement ok drop table t; + +# test multi group by for binary type without nulls +statement ok +create table t(a int, b bytea) as values (1, 0xa), (1, 0xa), (2, 0xb), (3, 0xb), (3, 0xb); + +query I?I +select a, b, count(*) from t group by grouping sets ((a, b), (a), (b)); +---- +1 0a 2 +2 0b 1 +3 0b 2 +1 NULL 2 +2 NULL 1 +3 NULL 2 +NULL 0a 2 +NULL 0b 3 + +statement ok +drop table t; From ef274a419b7bf8cb899f3347290591b1b60987fd Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Sat, 21 Sep 2024 11:36:19 +0800 Subject: [PATCH 26/30] debug assert Signed-off-by: jayzhan211 --- .../physical-plan/src/aggregates/group_values/row_like.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/group_values/row_like.rs b/datafusion/physical-plan/src/aggregates/group_values/row_like.rs index 8003c3baa69b..80ececbd98ce 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/row_like.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/row_like.rs @@ -214,7 +214,7 @@ impl GroupValues for GroupValuesRowLike { if i == 0 { checklen = len; } else { - assert_eq!(checklen, len); + debug_assert_eq!(checklen, len); } } @@ -254,7 +254,7 @@ impl GroupValues for GroupValuesRowLike { let mut output = match emit_to { EmitTo::All => { let group_values = std::mem::take(&mut self.group_values); - assert!(self.group_values.is_empty()); + debug_assert!(self.group_values.is_empty()); group_values .into_iter() From 864bdfaa19fa31c8ab5133cf8011afa5054f2530 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Sat, 21 Sep 2024 11:53:59 +0800 Subject: [PATCH 27/30] mv & rename Signed-off-by: jayzhan211 --- datafusion/physical-expr-common/src/binary_map.rs | 2 +- datafusion/physical-expr-common/src/lib.rs | 1 - .../group_values/{row_like.rs => column_wise.rs} | 13 +++++++------ .../aggregates/group_values}/group_value_row.rs | 15 ++------------- .../src/aggregates/group_values/mod.rs | 8 +++++--- 5 files changed, 15 insertions(+), 24 deletions(-) rename datafusion/physical-plan/src/aggregates/group_values/{row_like.rs => column_wise.rs} (98%) rename datafusion/{physical-expr-common/src => physical-plan/src/aggregates/group_values}/group_value_row.rs (97%) diff --git a/datafusion/physical-expr-common/src/binary_map.rs b/datafusion/physical-expr-common/src/binary_map.rs index 6a56f66a6ffb..1119809e466e 100644 --- a/datafusion/physical-expr-common/src/binary_map.rs +++ b/datafusion/physical-expr-common/src/binary_map.rs @@ -237,7 +237,7 @@ where /// The size, in number of entries, of the initial hash table pub(super) const INITIAL_MAP_CAPACITY: usize = 128; /// The initial size, in bytes, of the string data -pub(super) const INITIAL_BUFFER_CAPACITY: usize = 8 * 1024; +pub const INITIAL_BUFFER_CAPACITY: usize = 8 * 1024; impl ArrowBytesMap where V: Debug + PartialEq + Eq + Clone + Copy + Default, diff --git a/datafusion/physical-expr-common/src/lib.rs b/datafusion/physical-expr-common/src/lib.rs index bd2d7c93606b..7e2ea0c49397 100644 --- a/datafusion/physical-expr-common/src/lib.rs +++ b/datafusion/physical-expr-common/src/lib.rs @@ -23,7 +23,6 @@ pub mod binary_map; pub mod binary_view_map; pub mod datum; -pub mod group_value_row; pub mod physical_expr; pub mod sort_expr; pub mod tree_node; diff --git a/datafusion/physical-plan/src/aggregates/group_values/row_like.rs b/datafusion/physical-plan/src/aggregates/group_values/column_wise.rs similarity index 98% rename from datafusion/physical-plan/src/aggregates/group_values/row_like.rs rename to datafusion/physical-plan/src/aggregates/group_values/column_wise.rs index 80ececbd98ce..f6d89e8f1bd0 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/row_like.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/column_wise.rs @@ -15,6 +15,9 @@ // specific language governing permissions and limitations // under the License. +use crate::aggregates::group_values::group_value_row::{ + ArrayRowEq, ByteGroupValueBuilder, PrimitiveGroupValueBuilder, +}; use crate::aggregates::group_values::GroupValues; use ahash::RandomState; use arrow::compute::cast; @@ -30,13 +33,11 @@ use datafusion_common::{DataFusionError, Result}; use datafusion_execution::memory_pool::proxy::{RawTableAllocExt, VecAllocExt}; use datafusion_expr::EmitTo; use datafusion_physical_expr::binary_map::OutputType; -use datafusion_physical_expr_common::group_value_row::{ - ArrayRowEq, ByteGroupValueBuilder, PrimitiveGroupValueBuilder, -}; + use hashbrown::raw::RawTable; /// Compare GroupValue Rows column by column -pub struct GroupValuesRowLike { +pub struct GroupValuesColumn { /// The output schema schema: SchemaRef, @@ -65,7 +66,7 @@ pub struct GroupValuesRowLike { random_state: RandomState, } -impl GroupValuesRowLike { +impl GroupValuesColumn { pub fn try_new(schema: SchemaRef) -> Result { let map = RawTable::with_capacity(0); Ok(Self { @@ -79,7 +80,7 @@ impl GroupValuesRowLike { } } -impl GroupValues for GroupValuesRowLike { +impl GroupValues for GroupValuesColumn { fn intern(&mut self, cols: &[ArrayRef], groups: &mut Vec) -> Result<()> { let n_rows = cols[0].len(); diff --git a/datafusion/physical-expr-common/src/group_value_row.rs b/datafusion/physical-plan/src/aggregates/group_values/group_value_row.rs similarity index 97% rename from datafusion/physical-expr-common/src/group_value_row.rs rename to datafusion/physical-plan/src/aggregates/group_values/group_value_row.rs index 79d43715fd16..e78588fe8324 100644 --- a/datafusion/physical-expr-common/src/group_value_row.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/group_value_row.rs @@ -34,8 +34,7 @@ use arrow::datatypes::GenericStringType; use std::sync::Arc; use std::vec; -use crate::binary_map::OutputType; -use crate::binary_map::INITIAL_BUFFER_CAPACITY; +use datafusion_physical_expr_common::binary_map::{OutputType, INITIAL_BUFFER_CAPACITY}; /// Trait for group values column-wise row comparison /// @@ -51,8 +50,6 @@ pub trait ArrayRowEq: Send + Sync { fn append_val(&mut self, array: &ArrayRef, row: usize); /// Returns the number of rows stored in this builder fn len(&self) -> usize; - /// Returns true if this builder is empty - fn is_empty(&self) -> bool; /// Builds a new array from all of the stored rows fn build(self: Box) -> ArrayRef; /// Builds a new array from the first `n` stored rows, shifting the @@ -121,10 +118,6 @@ impl ArrayRowEq for PrimitiveGroupValueBuilder { self.group_values.len() } - fn is_empty(&self) -> bool { - self.len() == 0 - } - fn build(self: Box) -> ArrayRef { if self.has_null { Arc::new(PrimitiveArray::::new( @@ -276,10 +269,6 @@ where self.offsets.len() - 1 } - fn is_empty(&self) -> bool { - self.len() == 0 - } - fn build(self: Box) -> ArrayRef { let Self { output_type, @@ -328,7 +317,7 @@ where } fn take_n(&mut self, n: usize) -> ArrayRef { - assert!(self.len() >= n); + debug_assert!(self.len() >= n); let mut nulls_count = 0; let null_buffer = if self.nulls.is_empty() { diff --git a/datafusion/physical-plan/src/aggregates/group_values/mod.rs b/datafusion/physical-plan/src/aggregates/group_values/mod.rs index f3300b56e109..275cc7fcbf4e 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/mod.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/mod.rs @@ -25,16 +25,18 @@ pub(crate) mod primitive; use datafusion_expr::EmitTo; use primitive::GroupValuesPrimitive; +mod column_wise; mod row; -mod row_like; +use column_wise::GroupValuesColumn; use row::GroupValuesRows; -use row_like::GroupValuesRowLike; mod bytes; mod bytes_view; use bytes::GroupValuesByes; use datafusion_physical_expr::binary_map::OutputType; +mod group_value_row; + /// An interning store for group keys pub trait GroupValues: Send { /// Calculates the `groups` for each input row of `cols` @@ -100,7 +102,7 @@ pub fn new_group_values(schema: SchemaRef) -> Result> { .map(|f| f.data_type()) .all(has_row_like_feature) { - Ok(Box::new(GroupValuesRowLike::try_new(schema)?)) + Ok(Box::new(GroupValuesColumn::try_new(schema)?)) } else { Ok(Box::new(GroupValuesRows::try_new(schema)?)) } From 103c45d1e85dd74cf37e6f9e0d71ad2cb468efd5 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Sun, 22 Sep 2024 08:43:22 +0800 Subject: [PATCH 28/30] fix take_n logic Signed-off-by: jayzhan211 --- .../group_values/group_value_row.rs | 85 +++++++++++++++---- 1 file changed, 68 insertions(+), 17 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/group_values/group_value_row.rs b/datafusion/physical-plan/src/aggregates/group_values/group_value_row.rs index e78588fe8324..a66d8859ae11 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/group_value_row.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/group_value_row.rs @@ -159,7 +159,7 @@ where /// stored in the range `offsets[i]..offsets[i+1]` in `buffer`. Null values /// are stored as a zero length string. offsets: Vec, - /// Null indexes in offsets + /// Null indexes in offsets, if `i` is in nulls, `offsets[i]` should be equals to `offsets[i+1]` nulls: Vec, } @@ -324,22 +324,16 @@ where None } else { // Only make a `NullBuffer` if there was a null value - let num_values = self.offsets.len() - 1; - let mut bool_builder = BooleanBufferBuilder::new(num_values); - bool_builder.append_n(num_values, true); + let mut bool_builder = BooleanBufferBuilder::new(n); + bool_builder.append_n(n, true); - let nth_offset = O::as_usize(self.offsets[n]); - // Given offsets [0, 1, 2, 2], we could know that the 3rd index is null since the offset diff is 0 - let is_nth_offset_null = O::as_usize(self.offsets[n - 1]) == nth_offset; let mut new_nulls = vec![]; self.nulls.iter().for_each(|null_index| { - if *null_index < nth_offset - || (*null_index == nth_offset && is_nth_offset_null) - { + if *null_index < n { nulls_count += 1; bool_builder.set_bit(*null_index, false); } else { - new_nulls.push(null_index - nth_offset); + new_nulls.push(null_index - n); } }); @@ -347,6 +341,8 @@ where Some(NullBuffer::from(bool_builder.finish())) }; + let first_remaining_offset = O::as_usize(self.offsets[n]); + // Given offests like [0, 2, 4, 5] and n = 1, we expect to get // offsets [0, 2, 3]. We first create two offsets for first_n as [0, 2] and the remaining as [2, 4, 5]. // And we shift the offset starting from 0 for the remaining one, [2, 4, 5] -> [0, 2, 3]. @@ -362,12 +358,12 @@ where let offsets = unsafe { OffsetBuffer::new_unchecked(ScalarBuffer::from(first_n_offsets)) }; - // Consume first (n - nulls count) of elements since we don't push any value for null case. - let r = n - nulls_count; - - let mut remaining_buffer = BufferBuilder::new(self.buffer.len() - r); - remaining_buffer.append_slice(&self.buffer.as_slice()[r..]); - self.buffer.truncate(r); + let mut remaining_buffer = + BufferBuilder::new(self.buffer.len() - first_remaining_offset); + // TODO: Current approach copy the remaining and truncate the original one + // Find out a way to avoid copying buffer but split the original one into two. + remaining_buffer.append_slice(&self.buffer.as_slice()[first_remaining_offset..]); + self.buffer.truncate(first_remaining_offset); let values = self.buffer.finish(); self.buffer = remaining_buffer; @@ -393,3 +389,58 @@ where } } } + +#[cfg(test)] +mod tests { + use std::sync::Arc; + + use arrow_array::{ArrayRef, StringArray}; + use datafusion_physical_expr::binary_map::OutputType; + + use super::{ArrayRowEq, ByteGroupValueBuilder}; + + #[test] + fn test_take_n() { + let mut builder = ByteGroupValueBuilder::::new(OutputType::Utf8); + let array = Arc::new(StringArray::from(vec![Some("a"), None])) as ArrayRef; + // a, null, null + builder.append_val(&array, 0); + builder.append_val(&array, 1); + builder.append_val(&array, 1); + + // (a, null) remaining: null + let output = builder.take_n(2); + assert_eq!(&output, &array); + + // null, a, null, a + builder.append_val(&array, 0); + builder.append_val(&array, 1); + builder.append_val(&array, 0); + + // (null, a) remaining: (null, a) + let output = builder.take_n(2); + let array = Arc::new(StringArray::from(vec![None, Some("a")])) as ArrayRef; + assert_eq!(&output, &array); + + let array = Arc::new(StringArray::from(vec![ + Some("a"), + None, + Some("longstringfortest"), + ])) as ArrayRef; + + // null, a, longstringfortest, null, null + builder.append_val(&array, 2); + builder.append_val(&array, 1); + builder.append_val(&array, 1); + + // (null, a, longstringfortest, null) remaining: (null) + let output = builder.take_n(4); + let array = Arc::new(StringArray::from(vec![ + None, + Some("a"), + Some("longstringfortest"), + None, + ])) as ArrayRef; + assert_eq!(&output, &array); + } +} From 0bd5251e0440127e61132816a7a7f7bc6f719653 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Mon, 23 Sep 2024 22:14:21 +0800 Subject: [PATCH 29/30] address comment Signed-off-by: jayzhan211 --- .../aggregates/group_values/column_wise.rs | 5 ++-- .../group_values/group_value_row.rs | 28 +++++++++++++------ .../sqllogictest/test_files/group_by.slt | 20 +++++++++++++ 3 files changed, 41 insertions(+), 12 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/group_values/column_wise.rs b/datafusion/physical-plan/src/aggregates/group_values/column_wise.rs index f6d89e8f1bd0..b35d58701b5c 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/column_wise.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/column_wise.rs @@ -85,8 +85,7 @@ impl GroupValues for GroupValuesColumn { let n_rows = cols[0].len(); if self.group_values.is_empty() { - let len = cols.len(); - let mut v = Vec::with_capacity(len); + let mut v = Vec::with_capacity(cols.len()); for f in self.schema.fields().iter() { let nullable = f.is_nullable(); @@ -235,7 +234,7 @@ impl GroupValues for GroupValuesColumn { } fn size(&self) -> usize { - let group_values_size = self.group_values.len(); + let group_values_size: usize = self.group_values.iter().map(|v| v.size()).sum(); group_values_size + self.map_size + self.hashes_buffer.allocated_size() } diff --git a/datafusion/physical-plan/src/aggregates/group_values/group_value_row.rs b/datafusion/physical-plan/src/aggregates/group_values/group_value_row.rs index a66d8859ae11..ad8da37e7ca0 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/group_value_row.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/group_value_row.rs @@ -30,6 +30,7 @@ use arrow::datatypes::ByteArrayType; use arrow::datatypes::DataType; use arrow::datatypes::GenericBinaryType; use arrow::datatypes::GenericStringType; +use datafusion_common::utils::proxy::VecAllocExt; use std::sync::Arc; use std::vec; @@ -50,6 +51,8 @@ pub trait ArrayRowEq: Send + Sync { fn append_val(&mut self, array: &ArrayRef, row: usize); /// Returns the number of rows stored in this builder fn len(&self) -> usize; + /// Returns the number of bytes used by this [`ArrayRowEq`] + fn size(&self) -> usize; /// Builds a new array from all of the stored rows fn build(self: Box) -> ArrayRef; /// Builds a new array from the first `n` stored rows, shifting the @@ -102,15 +105,14 @@ impl ArrayRowEq for PrimitiveGroupValueBuilder { } fn append_val(&mut self, array: &ArrayRef, row: usize) { - // non-null fast path - if !self.nullable || !array.is_null(row) { - let elem = array.as_primitive::().value(row); - self.group_values.push(elem); - self.nulls.push(true); - } else { + if self.nullable && array.is_null(row) { self.group_values.push(T::default_value()); self.nulls.push(false); self.has_null = true; + } else { + let elem = array.as_primitive::().value(row); + self.group_values.push(elem); + self.nulls.push(true); } } @@ -118,6 +120,10 @@ impl ArrayRowEq for PrimitiveGroupValueBuilder { self.group_values.len() } + fn size(&self) -> usize { + self.group_values.allocated_size() + self.nulls.allocated_size() + } + fn build(self: Box) -> ArrayRef { if self.has_null { Arc::new(PrimitiveArray::::new( @@ -215,7 +221,7 @@ where let l = self.offsets[lhs_row].as_usize(); let r = self.offsets[lhs_row + 1].as_usize(); let existing_elem = unsafe { self.buffer.as_slice().get_unchecked(l..r) }; - existing_elem.len() == rhs_elem.len() && rhs_elem == existing_elem + rhs_elem == existing_elem } } @@ -269,6 +275,12 @@ where self.offsets.len() - 1 } + fn size(&self) -> usize { + self.buffer.capacity() * std::mem::size_of::() + + self.offsets.allocated_size() + + self.nulls.allocated_size() + } + fn build(self: Box) -> ArrayRef { let Self { output_type, @@ -319,7 +331,6 @@ where fn take_n(&mut self, n: usize) -> ArrayRef { debug_assert!(self.len() >= n); - let mut nulls_count = 0; let null_buffer = if self.nulls.is_empty() { None } else { @@ -330,7 +341,6 @@ where let mut new_nulls = vec![]; self.nulls.iter().for_each(|null_index| { if *null_index < n { - nulls_count += 1; bool_builder.set_bit(*null_index, false); } else { new_nulls.push(null_index - n); diff --git a/datafusion/sqllogictest/test_files/group_by.slt b/datafusion/sqllogictest/test_files/group_by.slt index 94e811383c62..86651f6ce43c 100644 --- a/datafusion/sqllogictest/test_files/group_by.slt +++ b/datafusion/sqllogictest/test_files/group_by.slt @@ -5152,6 +5152,8 @@ drop table test_case_expr statement ok drop table t; +# TODO: Current grouping set result is not align with Postgres and DuckDB, we might want to change the result +# See https://github.com/apache/datafusion/issues/12570 # test multi group by for binary type with nulls statement ok create table t(a int, b bytea) as values (1, 0xa), (1, 0xa), (2, null), (null, 0xb), (null, 0xb); @@ -5187,3 +5189,21 @@ NULL 0b 3 statement ok drop table t; + +# test multi group by int + utf8 +statement ok +create table t(a int, b varchar) as values (1, 'a'), (1, 'a'), (2, 'ab'), (3, 'abc'), (3, 'cba'), (null, null), (null, 'a'), (null, null), (null, 'a'), (1, 'null'); + +query ITI rowsort +select a, b, count(*) from t group by a, b; +---- +1 a 2 +1 null 1 +2 ab 1 +3 abc 1 +3 cba 1 +NULL NULL 2 +NULL a 2 + +statement ok +drop table t; From bf5dcc45258660893d78cd8552255662cb586bdc Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Tue, 24 Sep 2024 07:25:40 +0800 Subject: [PATCH 30/30] cleanup Signed-off-by: jayzhan211 --- benchmarks/queries/clickbench/queries.sql | 2 +- datafusion/physical-expr-common/src/binary_map.rs | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/benchmarks/queries/clickbench/queries.sql b/benchmarks/queries/clickbench/queries.sql index b3279f0e672b..52e72e02e1e0 100644 --- a/benchmarks/queries/clickbench/queries.sql +++ b/benchmarks/queries/clickbench/queries.sql @@ -40,4 +40,4 @@ SELECT "URL", COUNT(*) AS PageViews FROM hits WHERE "CounterID" = 62 AND "EventD SELECT "TraficSourceID", "SearchEngineID", "AdvEngineID", CASE WHEN ("SearchEngineID" = 0 AND "AdvEngineID" = 0) THEN "Referer" ELSE '' END AS Src, "URL" AS Dst, COUNT(*) AS PageViews FROM hits WHERE "CounterID" = 62 AND "EventDate"::INT::DATE >= '2013-07-01' AND "EventDate"::INT::DATE <= '2013-07-31' AND "IsRefresh" = 0 GROUP BY "TraficSourceID", "SearchEngineID", "AdvEngineID", Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; SELECT "URLHash", "EventDate"::INT::DATE, COUNT(*) AS PageViews FROM hits WHERE "CounterID" = 62 AND "EventDate"::INT::DATE >= '2013-07-01' AND "EventDate"::INT::DATE <= '2013-07-31' AND "IsRefresh" = 0 AND "TraficSourceID" IN (-1, 6) AND "RefererHash" = 3594120000172545465 GROUP BY "URLHash", "EventDate"::INT::DATE ORDER BY PageViews DESC LIMIT 10 OFFSET 100; SELECT "WindowClientWidth", "WindowClientHeight", COUNT(*) AS PageViews FROM hits WHERE "CounterID" = 62 AND "EventDate"::INT::DATE >= '2013-07-01' AND "EventDate"::INT::DATE <= '2013-07-31' AND "IsRefresh" = 0 AND "DontCountHits" = 0 AND "URLHash" = 2868770270353813622 GROUP BY "WindowClientWidth", "WindowClientHeight" ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; -SELECT DATE_TRUNC('minute', to_timestamp_seconds("EventTime")) AS M, COUNT(*) AS PageViews FROM hits WHERE "CounterID" = 62 AND "EventDate"::INT::DATE >= '2013-07-14' AND "EventDate"::INT::DATE <= '2013-07-15' AND "IsRefresh" = 0 AND "DontCountHits" = 0 GROUP BY DATE_TRUNC('minute', to_timestamp_seconds("EventTime")) ORDER BY DATE_TRUNC('minute', M) LIMIT 10 OFFSET 1000; \ No newline at end of file +SELECT DATE_TRUNC('minute', to_timestamp_seconds("EventTime")) AS M, COUNT(*) AS PageViews FROM hits WHERE "CounterID" = 62 AND "EventDate"::INT::DATE >= '2013-07-14' AND "EventDate"::INT::DATE <= '2013-07-15' AND "IsRefresh" = 0 AND "DontCountHits" = 0 GROUP BY DATE_TRUNC('minute', to_timestamp_seconds("EventTime")) ORDER BY DATE_TRUNC('minute', M) LIMIT 10 OFFSET 1000; diff --git a/datafusion/physical-expr-common/src/binary_map.rs b/datafusion/physical-expr-common/src/binary_map.rs index 1119809e466e..f320ebcc06b5 100644 --- a/datafusion/physical-expr-common/src/binary_map.rs +++ b/datafusion/physical-expr-common/src/binary_map.rs @@ -235,7 +235,7 @@ where } /// The size, in number of entries, of the initial hash table -pub(super) const INITIAL_MAP_CAPACITY: usize = 128; +const INITIAL_MAP_CAPACITY: usize = 128; /// The initial size, in bytes, of the string data pub const INITIAL_BUFFER_CAPACITY: usize = 8 * 1024; impl ArrowBytesMap