From b80e702110153a2e9ea3c56f5ab288baaaaf7111 Mon Sep 17 00:00:00 2001 From: Jayjeet Chakraborty Date: Thu, 24 Aug 2023 16:44:25 -0700 Subject: [PATCH 01/55] Use CardinalityAwareRow converter --- .../sorts/sort_preserving_merge.rs | 123 +++++++++++++++++- .../core/src/physical_plan/sorts/stream.rs | 6 +- 2 files changed, 123 insertions(+), 6 deletions(-) diff --git a/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs b/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs index 27c1f79db5bc..838dae75b453 100644 --- a/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs +++ b/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs @@ -267,15 +267,19 @@ impl ExecutionPlan for SortPreservingMergeExec { #[cfg(test)] mod tests { use std::iter::FromIterator; + use arrow_array::UInt32Array; + use rand::Rng; + use uuid::Uuid; use arrow::array::ArrayRef; use arrow::compute::SortOptions; - use arrow::datatypes::{DataType, Field, Schema}; + use arrow::datatypes::{DataType, Field, Schema, Int32Type}; use arrow::record_batch::RecordBatch; use datafusion_execution::config::SessionConfig; - use futures::{FutureExt, StreamExt}; use tempfile::TempDir; + use futures::{FutureExt, StreamExt, stream::BoxStream}; + use crate::execution::context::SessionContext; use crate::physical_plan::coalesce_partitions::CoalescePartitionsExec; use crate::physical_plan::expressions::col; use crate::physical_plan::memory::MemoryExec; @@ -286,10 +290,123 @@ mod tests { use crate::test::exec::{assert_strong_count_converges_to_zero, BlockingExec}; use crate::test::{self, assert_is_pending}; use crate::{assert_batches_eq, test_util}; - use arrow::array::{Int32Array, StringArray, TimestampNanosecondArray}; + use arrow::array::{Int32Array, StringArray, TimestampNanosecondArray, DictionaryArray}; + + use crate::physical_plan::streaming::PartitionStream; + use crate::physical_plan::stream::RecordBatchStreamAdapter; + use crate::datasource::{streaming::StreamingTable, TableProvider}; use super::*; + fn make_infinite_sorted_stream(col_b_init: &u32) -> BoxStream<'static, RecordBatch> { + let col_b_init_clone = col_b_init.clone(); + futures::stream::unfold((0, col_b_init_clone), move |(mut counter, mut col_b_ascii)| async move { + // stop the stream at 20 batch now. + // Need to figure out how all the columns in the batches are sorted. + if counter >= 12000 { + return None; + } + + if counter % 5 == 0 { + col_b_ascii = col_b_ascii + 2; + } + + counter = counter + 1; + + // building col `a` + let mut values_vector: Vec = Vec::new(); + for _i in 1..=8192 { + // values_vector.push(rand::thread_rng().gen_range(1..=1000)); + values_vector.push(String::from(Uuid::new_v4().to_string())); + } + let values = StringArray::from(values_vector); + + let mut keys_vector: Vec = Vec::new(); + for _i in 1..=8192 { + keys_vector.push(rand::thread_rng().gen_range(0..8192)); + } + let keys = Int32Array::from(keys_vector); + let col_a: ArrayRef = Arc::new(DictionaryArray::::try_new(keys, Arc::new(values)).unwrap()); + + // building col `b` + let mut values: Vec = Vec::new(); + for _i in 1..=8192 { + // let ascii_value = rand::thread_rng().gen_range(97..=110); + // values.push(String::from(from_u32(col_b_ascii).unwrap())); + values.push(col_b_ascii); + // values.sort(); + } + let col_b: ArrayRef = Arc::new(UInt32Array::from(values)); + + // build a record batch out of col `a` and col `b` + let batch: RecordBatch = RecordBatch::try_from_iter(vec![("a", col_a), ("b", col_b)]).unwrap(); + Some((batch, (counter, col_b_ascii))) + }).boxed() + } + + struct InfiniteStream { + schema: SchemaRef, + col_b_init: u32 + } + + impl PartitionStream for InfiniteStream { + fn schema(&self) -> &SchemaRef { + &self.schema + } + + fn execute(&self, _ctx: Arc) -> SendableRecordBatchStream { + // We create an iterator from the record batches and map them into Ok values, + // converting the iterator into a futures::stream::Stream + Box::pin(RecordBatchStreamAdapter::new( + self.schema.clone(), + make_infinite_sorted_stream(&self.col_b_init).map(Ok) + )) + } + } + + #[tokio::test] + async fn test_dict_merge_infinite() { + let session_ctx = SessionContext::new(); + let task_ctx: Arc = session_ctx.task_ctx(); + + let schema = SchemaRef::new(Schema::new(vec![ + Field::new("a", DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)), false), + Field::new("b", DataType::UInt32, false), + ])); + + let stream_1 = Arc::new(InfiniteStream { + schema: schema.clone(), col_b_init: 1 + }); + + let stream_2 = Arc::new(InfiniteStream { + schema: schema.clone(), col_b_init: 2 + }); + + println!("SortPreservingMergeExec result: "); + + let sort = vec![ + PhysicalSortExpr { + expr: col("b", &schema).unwrap(), + options: Default::default(), + }, + PhysicalSortExpr { + expr: col("a", &schema).unwrap(), + options: Default::default(), + }, + ]; + + let provider = StreamingTable::try_new(schema, vec![stream_1, stream_2]).unwrap(); + + let plan = provider.scan(&session_ctx.state(), None, &[], None).await.unwrap(); + let exec = Arc::new(SortPreservingMergeExec::new(sort, plan)); + let mut stream = exec.execute(0, task_ctx).unwrap(); + while let Some(batch) = stream.next().await { + println!("{}", arrow::util::pretty::pretty_format_batches(&[batch.unwrap().clone()]) + .unwrap() + .to_string()); + } + } + #[tokio::test] async fn test_merge_interleave() { let task_ctx = Arc::new(TaskContext::default()); diff --git a/datafusion/core/src/physical_plan/sorts/stream.rs b/datafusion/core/src/physical_plan/sorts/stream.rs index 9ef13b7eb25e..621ba5e6f47f 100644 --- a/datafusion/core/src/physical_plan/sorts/stream.rs +++ b/datafusion/core/src/physical_plan/sorts/stream.rs @@ -21,7 +21,7 @@ use crate::physical_plan::{PhysicalExpr, PhysicalSortExpr}; use arrow::array::Array; use arrow::datatypes::Schema; use arrow::record_batch::RecordBatch; -use arrow::row::{RowConverter, SortField}; +use arrow::row::{RowConverter, SortField, CardinalityAwareRowConverter}; use datafusion_common::Result; use datafusion_execution::memory_pool::MemoryReservation; use futures::stream::{Fuse, StreamExt}; @@ -80,7 +80,7 @@ impl FusedStreams { #[derive(Debug)] pub struct RowCursorStream { /// Converter to convert output of physical expressions - converter: RowConverter, + converter: CardinalityAwareRowConverter, /// The physical expressions to sort by column_expressions: Vec>, /// Input streams @@ -105,7 +105,7 @@ impl RowCursorStream { .collect::>>()?; let streams = streams.into_iter().map(|s| s.fuse()).collect(); - let converter = RowConverter::new(sort_fields)?; + let converter = CardinalityAwareRowConverter::new(sort_fields)?; Ok(Self { converter, reservation, From d83fae2c37add2594b675c910cbc668f4a250e7a Mon Sep 17 00:00:00 2001 From: Jayjeet Chakraborty Date: Tue, 29 Aug 2023 10:57:17 -0700 Subject: [PATCH 02/55] Move CardinalityAwareRowConverter in df * Try to add cardinality aware row converter in df * Move CardinalityAwareRowConverter in df --- datafusion/core/src/physical_plan/mod.rs | 1 + .../core/src/physical_plan/sorts/stream.rs | 3 +- datafusion/core/src/physical_plan/wrapper.rs | 91 +++++++++++++++++++ 3 files changed, 94 insertions(+), 1 deletion(-) create mode 100644 datafusion/core/src/physical_plan/wrapper.rs diff --git a/datafusion/core/src/physical_plan/mod.rs b/datafusion/core/src/physical_plan/mod.rs index 43ce14a1e836..245228cfa70b 100644 --- a/datafusion/core/src/physical_plan/mod.rs +++ b/datafusion/core/src/physical_plan/mod.rs @@ -402,6 +402,7 @@ pub mod union; pub mod unnest; pub mod values; pub mod windows; +pub mod wrapper; use crate::physical_plan::repartition::RepartitionExec; use crate::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; diff --git a/datafusion/core/src/physical_plan/sorts/stream.rs b/datafusion/core/src/physical_plan/sorts/stream.rs index 621ba5e6f47f..94ea627e7cfd 100644 --- a/datafusion/core/src/physical_plan/sorts/stream.rs +++ b/datafusion/core/src/physical_plan/sorts/stream.rs @@ -18,10 +18,11 @@ use crate::physical_plan::sorts::cursor::{FieldArray, FieldCursor, RowCursor}; use crate::physical_plan::SendableRecordBatchStream; use crate::physical_plan::{PhysicalExpr, PhysicalSortExpr}; +use crate::physical_plan::wrapper::CardinalityAwareRowConverter; use arrow::array::Array; use arrow::datatypes::Schema; use arrow::record_batch::RecordBatch; -use arrow::row::{RowConverter, SortField, CardinalityAwareRowConverter}; +use arrow::row::SortField; use datafusion_common::Result; use datafusion_execution::memory_pool::MemoryReservation; use futures::stream::{Fuse, StreamExt}; diff --git a/datafusion/core/src/physical_plan/wrapper.rs b/datafusion/core/src/physical_plan/wrapper.rs new file mode 100644 index 000000000000..f0ac381a2d69 --- /dev/null +++ b/datafusion/core/src/physical_plan/wrapper.rs @@ -0,0 +1,91 @@ +// 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 arrow::row::{SortField, Rows}; +use arrow::datatypes::DataType; +use arrow::error::ArrowError; +use arrow_array::*; +use arrow_array::types::*; +use arrow::row::RowConverter; + +const LOW_CARDINALITY_THRESHOLD: usize = 10; + +macro_rules! downcast_dict { + ($array:ident, $key:ident) => {{ + $array + .as_any() + .downcast_ref::>() + .unwrap() + }}; +} + +#[derive(Debug)] +pub struct CardinalityAwareRowConverter { + fields: Vec, + inner: Option, + done: bool, +} + +impl CardinalityAwareRowConverter { + pub fn new(fields: Vec) -> Result { + Ok(Self { + fields: fields.clone(), + inner: None, + done: false, + }) + } + + pub fn size(&self) -> usize { + match &self.inner { + Some(inner) => inner.size(), + None => 0, + } + } + + pub fn convert_rows(&self, rows: &Rows) -> Result, ArrowError> { + self.inner.as_ref().unwrap().convert_rows(rows) + } + + pub fn convert_columns( + &mut self, + columns: &[ArrayRef]) -> Result { + + if !self.done { + for (i, col) in columns.iter().enumerate() { + if let DataType::Dictionary(k, _) = col.data_type() { + let cardinality = match k.as_ref() { + DataType::Int8 => downcast_dict!(col, Int32Type).values().len(), + DataType::Int16 => downcast_dict!(col, Int32Type).values().len(), + DataType::Int32 => downcast_dict!(col, Int32Type).values().len(), + DataType::Int64 => downcast_dict!(col, Int64Type).values().len(), + DataType::UInt16 => downcast_dict!(col, UInt16Type).values().len(), + DataType::UInt32 => downcast_dict!(col, UInt32Type).values().len(), + DataType::UInt64 => downcast_dict!(col, UInt64Type).values().len(), + _ => unreachable!(), + }; + + if cardinality >= LOW_CARDINALITY_THRESHOLD { + self.fields[i] = self.fields[i].clone().preserve_dictionaries(false); + } + } + } + self.inner = Some(RowConverter::new(self.fields.clone())?); + self.done = true; + } + println!("convert_columns"); + self.inner.as_mut().unwrap().convert_columns(columns) + } +} From 546ccffa92bedb5e77130b7a39b6c116dd4957b6 Mon Sep 17 00:00:00 2001 From: Jayjeet Chakraborty Date: Tue, 29 Aug 2023 13:06:02 -0700 Subject: [PATCH 03/55] Move CardinalityAwareRowConverter in df --- datafusion/core/src/physical_plan/wrapper.rs | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) diff --git a/datafusion/core/src/physical_plan/wrapper.rs b/datafusion/core/src/physical_plan/wrapper.rs index f0ac381a2d69..7548148b5391 100644 --- a/datafusion/core/src/physical_plan/wrapper.rs +++ b/datafusion/core/src/physical_plan/wrapper.rs @@ -49,10 +49,7 @@ impl CardinalityAwareRowConverter { } pub fn size(&self) -> usize { - match &self.inner { - Some(inner) => inner.size(), - None => 0, - } + return self.inner.as_ref().unwrap().size(); } pub fn convert_rows(&self, rows: &Rows) -> Result, ArrowError> { @@ -82,10 +79,10 @@ impl CardinalityAwareRowConverter { } } } - self.inner = Some(RowConverter::new(self.fields.clone())?); self.done = true; } - println!("convert_columns"); + + self.inner = Some(RowConverter::new(self.fields.clone())?); self.inner.as_mut().unwrap().convert_columns(columns) } } From e9fb8af1cb500470b04e08da4acabf218c9a9004 Mon Sep 17 00:00:00 2001 From: Jayjeet Chakraborty Date: Wed, 30 Aug 2023 11:10:12 -0700 Subject: [PATCH 04/55] Remove unnecessary clone and make wrapper mod private --- datafusion/core/src/physical_plan/mod.rs | 2 +- datafusion/core/src/physical_plan/wrapper.rs | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion/core/src/physical_plan/mod.rs b/datafusion/core/src/physical_plan/mod.rs index 245228cfa70b..d10888b64543 100644 --- a/datafusion/core/src/physical_plan/mod.rs +++ b/datafusion/core/src/physical_plan/mod.rs @@ -402,7 +402,7 @@ pub mod union; pub mod unnest; pub mod values; pub mod windows; -pub mod wrapper; +mod wrapper; use crate::physical_plan::repartition::RepartitionExec; use crate::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; diff --git a/datafusion/core/src/physical_plan/wrapper.rs b/datafusion/core/src/physical_plan/wrapper.rs index 7548148b5391..98648639e3de 100644 --- a/datafusion/core/src/physical_plan/wrapper.rs +++ b/datafusion/core/src/physical_plan/wrapper.rs @@ -42,7 +42,7 @@ pub struct CardinalityAwareRowConverter { impl CardinalityAwareRowConverter { pub fn new(fields: Vec) -> Result { Ok(Self { - fields: fields.clone(), + fields, inner: None, done: false, }) From c5c707a94e3211eaae5a0981d1248c2b7264a5cb Mon Sep 17 00:00:00 2001 From: Jayjeet Chakraborty Date: Wed, 30 Aug 2023 11:23:12 -0700 Subject: [PATCH 05/55] Use as_any_dictionary_opt --- datafusion/core/src/physical_plan/wrapper.rs | 29 ++------------------ 1 file changed, 3 insertions(+), 26 deletions(-) diff --git a/datafusion/core/src/physical_plan/wrapper.rs b/datafusion/core/src/physical_plan/wrapper.rs index 98648639e3de..715497a1733b 100644 --- a/datafusion/core/src/physical_plan/wrapper.rs +++ b/datafusion/core/src/physical_plan/wrapper.rs @@ -18,20 +18,11 @@ use arrow::row::{SortField, Rows}; use arrow::datatypes::DataType; use arrow::error::ArrowError; use arrow_array::*; -use arrow_array::types::*; +use arrow_array::cast::AsArray; use arrow::row::RowConverter; const LOW_CARDINALITY_THRESHOLD: usize = 10; -macro_rules! downcast_dict { - ($array:ident, $key:ident) => {{ - $array - .as_any() - .downcast_ref::>() - .unwrap() - }}; -} - #[derive(Debug)] pub struct CardinalityAwareRowConverter { fields: Vec, @@ -52,28 +43,14 @@ impl CardinalityAwareRowConverter { return self.inner.as_ref().unwrap().size(); } - pub fn convert_rows(&self, rows: &Rows) -> Result, ArrowError> { - self.inner.as_ref().unwrap().convert_rows(rows) - } - pub fn convert_columns( &mut self, columns: &[ArrayRef]) -> Result { if !self.done { for (i, col) in columns.iter().enumerate() { - if let DataType::Dictionary(k, _) = col.data_type() { - let cardinality = match k.as_ref() { - DataType::Int8 => downcast_dict!(col, Int32Type).values().len(), - DataType::Int16 => downcast_dict!(col, Int32Type).values().len(), - DataType::Int32 => downcast_dict!(col, Int32Type).values().len(), - DataType::Int64 => downcast_dict!(col, Int64Type).values().len(), - DataType::UInt16 => downcast_dict!(col, UInt16Type).values().len(), - DataType::UInt32 => downcast_dict!(col, UInt32Type).values().len(), - DataType::UInt64 => downcast_dict!(col, UInt64Type).values().len(), - _ => unreachable!(), - }; - + if let DataType::Dictionary(_, _) = col.data_type() { + let cardinality = col.as_any_dictionary_opt().unwrap().values().len(); if cardinality >= LOW_CARDINALITY_THRESHOLD { self.fields[i] = self.fields[i].clone().preserve_dictionaries(false); } From 22fb15993ff48cd8e8735322338a0f2a40e5ab62 Mon Sep 17 00:00:00 2001 From: Jayjeet Chakraborty Date: Wed, 30 Aug 2023 11:38:58 -0700 Subject: [PATCH 06/55] Remove unnecessary comments --- .../core/src/physical_plan/sorts/sort_preserving_merge.rs | 8 +------- 1 file changed, 1 insertion(+), 7 deletions(-) diff --git a/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs b/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs index 838dae75b453..f5ee7ab4cbf5 100644 --- a/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs +++ b/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs @@ -301,8 +301,6 @@ mod tests { fn make_infinite_sorted_stream(col_b_init: &u32) -> BoxStream<'static, RecordBatch> { let col_b_init_clone = col_b_init.clone(); futures::stream::unfold((0, col_b_init_clone), move |(mut counter, mut col_b_ascii)| async move { - // stop the stream at 20 batch now. - // Need to figure out how all the columns in the batches are sorted. if counter >= 12000 { return None; } @@ -316,7 +314,6 @@ mod tests { // building col `a` let mut values_vector: Vec = Vec::new(); for _i in 1..=8192 { - // values_vector.push(rand::thread_rng().gen_range(1..=1000)); values_vector.push(String::from(Uuid::new_v4().to_string())); } let values = StringArray::from(values_vector); @@ -331,10 +328,7 @@ mod tests { // building col `b` let mut values: Vec = Vec::new(); for _i in 1..=8192 { - // let ascii_value = rand::thread_rng().gen_range(97..=110); - // values.push(String::from(from_u32(col_b_ascii).unwrap())); values.push(col_b_ascii); - // values.sort(); } let col_b: ArrayRef = Arc::new(UInt32Array::from(values)); @@ -343,7 +337,7 @@ mod tests { Some((batch, (counter, col_b_ascii))) }).boxed() } - + struct InfiniteStream { schema: SchemaRef, col_b_init: u32 From 65a02097666281488d618ee8ff3f324b16d5a5d1 Mon Sep 17 00:00:00 2001 From: Jayjeet Chakraborty Date: Wed, 30 Aug 2023 13:41:43 -0700 Subject: [PATCH 07/55] Remove done --- datafusion/core/src/physical_plan/wrapper.rs | 16 ++++++---------- 1 file changed, 6 insertions(+), 10 deletions(-) diff --git a/datafusion/core/src/physical_plan/wrapper.rs b/datafusion/core/src/physical_plan/wrapper.rs index 715497a1733b..1a4ea1acdacb 100644 --- a/datafusion/core/src/physical_plan/wrapper.rs +++ b/datafusion/core/src/physical_plan/wrapper.rs @@ -25,17 +25,15 @@ const LOW_CARDINALITY_THRESHOLD: usize = 10; #[derive(Debug)] pub struct CardinalityAwareRowConverter { - fields: Vec, + fields: Option>, inner: Option, - done: bool, } impl CardinalityAwareRowConverter { pub fn new(fields: Vec) -> Result { Ok(Self { - fields, + fields: Some(fields), inner: None, - done: false, }) } @@ -46,20 +44,18 @@ impl CardinalityAwareRowConverter { pub fn convert_columns( &mut self, columns: &[ArrayRef]) -> Result { - - if !self.done { + if self.fields != None { + let mut updated_fields = self.fields.take(); for (i, col) in columns.iter().enumerate() { if let DataType::Dictionary(_, _) = col.data_type() { let cardinality = col.as_any_dictionary_opt().unwrap().values().len(); if cardinality >= LOW_CARDINALITY_THRESHOLD { - self.fields[i] = self.fields[i].clone().preserve_dictionaries(false); + updated_fields.as_mut().unwrap()[i] = updated_fields.as_ref().unwrap()[i].clone().preserve_dictionaries(false); } } } - self.done = true; + self.inner = Some(RowConverter::new(updated_fields.unwrap())?); } - - self.inner = Some(RowConverter::new(self.fields.clone())?); self.inner.as_mut().unwrap().convert_columns(columns) } } From bd5faf75faf049351b1f7a76af51013c537aa055 Mon Sep 17 00:00:00 2001 From: Jayjeet Chakraborty Date: Thu, 31 Aug 2023 13:04:06 -0700 Subject: [PATCH 08/55] Add test for cardinality aware row converter on high card dict --- datafusion/core/src/physical_plan/wrapper.rs | 72 ++++++++++++++++++++ 1 file changed, 72 insertions(+) diff --git a/datafusion/core/src/physical_plan/wrapper.rs b/datafusion/core/src/physical_plan/wrapper.rs index 1a4ea1acdacb..0017de171256 100644 --- a/datafusion/core/src/physical_plan/wrapper.rs +++ b/datafusion/core/src/physical_plan/wrapper.rs @@ -1,3 +1,5 @@ +use std::sync::Arc; + // 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 @@ -40,6 +42,10 @@ impl CardinalityAwareRowConverter { pub fn size(&self) -> usize { return self.inner.as_ref().unwrap().size(); } + + pub fn convert_rows(&self, rows: &Rows) -> Result, ArrowError> { + return self.inner.as_ref().unwrap().convert_rows(rows); + } pub fn convert_columns( &mut self, @@ -59,3 +65,69 @@ impl CardinalityAwareRowConverter { self.inner.as_mut().unwrap().convert_columns(columns) } } + +mod tests { + use std::sync::Arc; + + use uuid::Uuid; + use rand::Rng; + + use arrow::error::ArrowError; + use arrow::datatypes::{DataType, Field}; + use arrow_schema::{Schema, SchemaRef, SortOptions}; + use arrow_array::{StringArray, DictionaryArray, Int32Array, types::Int32Type, RecordBatch, ArrayRef}; + + use super::*; + + // Generate a record batch with a high cardinality dictionary field + fn generate_batch_with_high_card_dict_field() -> Result { + let schema = SchemaRef::new(Schema::new(vec![ + Field::new("a_dict", DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)), false), + Field::new("b_prim", DataType::Int32, false), + ])); + + // building column `a_dict` + let mut values_vector: Vec = Vec::new(); + for _i in 1..=20 { + values_vector.push(String::from(Uuid::new_v4().to_string())); + } + let values = StringArray::from(values_vector); + + let mut keys_vector: Vec = Vec::new(); + for _i in 1..=20 { + keys_vector.push(rand::thread_rng().gen_range(0..20)); + } + let keys = Int32Array::from(keys_vector); + let col_a: ArrayRef = Arc::new(DictionaryArray::::try_new(keys, Arc::new(values)).unwrap()); + + // building column `b_prim` + let mut values: Vec = Vec::new(); + for _i in 1..=20 { + values.push(rand::thread_rng().gen_range(0..20)); + } + let col_b: ArrayRef = Arc::new(Int32Array::from(values)); + + // building the record batch + RecordBatch::try_from_iter(vec![("a_dict", col_a), ("b_prim", col_b)]) + } + + // fn generate_batch_with_low_card_dict_field() {} + + #[tokio::test] + async fn test_cardinality_decision() { + let batch = generate_batch_with_high_card_dict_field().unwrap(); + let sort_fields = vec![ + arrow::row::SortField::new_with_options(DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)), SortOptions::default()), + arrow::row::SortField::new_with_options(DataType::Int32, SortOptions::default()) + ]; + let mut converter = CardinalityAwareRowConverter::new(sort_fields.clone()).unwrap(); + let rows = converter.convert_columns(&batch.columns()).unwrap(); + let converted_batch = converter.convert_rows(&rows).unwrap(); + assert_eq!(converted_batch[0].data_type(), &DataType::Utf8); + + let mut converter = RowConverter::new(sort_fields.clone()).unwrap(); + let rows = converter.convert_columns(&batch.columns()).unwrap(); + let converted_batch = converter.convert_rows(&rows).unwrap(); + assert_eq!(converted_batch[0].data_type(), &DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8))); + } +} From 80d3bf2cec5ecde63f58d4754c1b9cff984e9e98 Mon Sep 17 00:00:00 2001 From: Jayjeet Chakraborty Date: Thu, 31 Aug 2023 13:37:42 -0700 Subject: [PATCH 09/55] Add test for cardinality aware row converter on low card dict --- datafusion/core/src/physical_plan/wrapper.rs | 63 ++++++++++++++------ 1 file changed, 46 insertions(+), 17 deletions(-) diff --git a/datafusion/core/src/physical_plan/wrapper.rs b/datafusion/core/src/physical_plan/wrapper.rs index 0017de171256..42604f1388dc 100644 --- a/datafusion/core/src/physical_plan/wrapper.rs +++ b/datafusion/core/src/physical_plan/wrapper.rs @@ -80,25 +80,38 @@ mod tests { use super::*; // Generate a record batch with a high cardinality dictionary field - fn generate_batch_with_high_card_dict_field() -> Result { + fn generate_batch_with_cardinality(card: String) -> Result { let schema = SchemaRef::new(Schema::new(vec![ Field::new("a_dict", DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)), false), Field::new("b_prim", DataType::Int32, false), ])); + + let col_a: ArrayRef; + if card == "high" { + // building column `a_dict` + let mut values_vector: Vec = Vec::new(); + for _i in 1..=20 { + values_vector.push(String::from(Uuid::new_v4().to_string())); + } + let values = StringArray::from(values_vector); - // building column `a_dict` - let mut values_vector: Vec = Vec::new(); - for _i in 1..=20 { - values_vector.push(String::from(Uuid::new_v4().to_string())); - } - let values = StringArray::from(values_vector); - - let mut keys_vector: Vec = Vec::new(); - for _i in 1..=20 { - keys_vector.push(rand::thread_rng().gen_range(0..20)); + let mut keys_vector: Vec = Vec::new(); + for _i in 1..=20 { + keys_vector.push(rand::thread_rng().gen_range(0..20)); + } + let keys = Int32Array::from(keys_vector); + col_a = Arc::new(DictionaryArray::::try_new(keys, Arc::new(values)).unwrap()); + } else { + let values_vector = vec!["a", "b", "c"]; + let values = StringArray::from(values_vector); + + let mut keys_vector: Vec = Vec::new(); + for _i in 1..=20 { + keys_vector.push(rand::thread_rng().gen_range(0..2)); + } + let keys = Int32Array::from(keys_vector); + col_a = Arc::new(DictionaryArray::::try_new(keys, Arc::new(values)).unwrap()); } - let keys = Int32Array::from(keys_vector); - let col_a: ArrayRef = Arc::new(DictionaryArray::::try_new(keys, Arc::new(values)).unwrap()); // building column `b_prim` let mut values: Vec = Vec::new(); @@ -110,12 +123,10 @@ mod tests { // building the record batch RecordBatch::try_from_iter(vec![("a_dict", col_a), ("b_prim", col_b)]) } - - // fn generate_batch_with_low_card_dict_field() {} #[tokio::test] - async fn test_cardinality_decision() { - let batch = generate_batch_with_high_card_dict_field().unwrap(); + async fn test_with_high_card() { + let batch = generate_batch_with_cardinality(String::from("high")).unwrap(); let sort_fields = vec![ arrow::row::SortField::new_with_options(DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)), SortOptions::default()), arrow::row::SortField::new_with_options(DataType::Int32, SortOptions::default()) @@ -130,4 +141,22 @@ mod tests { let converted_batch = converter.convert_rows(&rows).unwrap(); assert_eq!(converted_batch[0].data_type(), &DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8))); } + + #[tokio::test] + async fn test_with_low_card() { + let batch = generate_batch_with_cardinality(String::from("low")).unwrap(); + let sort_fields = vec![ + arrow::row::SortField::new_with_options(DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)), SortOptions::default()), + arrow::row::SortField::new_with_options(DataType::Int32, SortOptions::default()) + ]; + let mut converter = CardinalityAwareRowConverter::new(sort_fields.clone()).unwrap(); + let rows = converter.convert_columns(&batch.columns()).unwrap(); + let converted_batch = converter.convert_rows(&rows).unwrap(); + assert_eq!(converted_batch[0].data_type(), &DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8))); + + let mut converter = RowConverter::new(sort_fields.clone()).unwrap(); + let rows = converter.convert_columns(&batch.columns()).unwrap(); + let converted_batch = converter.convert_rows(&rows).unwrap(); + assert_eq!(converted_batch[0].data_type(), &DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8))); + } } From 5eff541c870fbd0c18d61f73e0bbb680f2affb83 Mon Sep 17 00:00:00 2001 From: Jayjeet Chakraborty Date: Thu, 31 Aug 2023 14:04:14 -0700 Subject: [PATCH 10/55] Ignore the test_dict_merge_infinite test --- datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs b/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs index f5ee7ab4cbf5..c5b85b68eedd 100644 --- a/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs +++ b/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs @@ -359,6 +359,7 @@ mod tests { } #[tokio::test] + #[ignore] async fn test_dict_merge_infinite() { let session_ctx = SessionContext::new(); let task_ctx: Arc = session_ctx.task_ctx(); From 2651532c0332b983635528924168dba46b96fb6f Mon Sep 17 00:00:00 2001 From: Jayjeet Chakraborty Date: Thu, 31 Aug 2023 19:59:11 -0700 Subject: [PATCH 11/55] Remove phantom Arc import --- datafusion/core/src/physical_plan/wrapper.rs | 2 -- 1 file changed, 2 deletions(-) diff --git a/datafusion/core/src/physical_plan/wrapper.rs b/datafusion/core/src/physical_plan/wrapper.rs index 42604f1388dc..ac76df0ed8bf 100644 --- a/datafusion/core/src/physical_plan/wrapper.rs +++ b/datafusion/core/src/physical_plan/wrapper.rs @@ -1,5 +1,3 @@ -use std::sync::Arc; - // 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 From 7da368147546d7ec1ffbcbd17fcb06f6d2c9ae18 Mon Sep 17 00:00:00 2001 From: Jayjeet Chakraborty Date: Fri, 1 Sep 2023 13:29:04 -0700 Subject: [PATCH 12/55] Remove the infinite stream test --- .../sorts/sort_preserving_merge.rs | 118 +----------------- 1 file changed, 3 insertions(+), 115 deletions(-) diff --git a/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs b/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs index c5b85b68eedd..27c1f79db5bc 100644 --- a/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs +++ b/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs @@ -267,19 +267,15 @@ impl ExecutionPlan for SortPreservingMergeExec { #[cfg(test)] mod tests { use std::iter::FromIterator; - use arrow_array::UInt32Array; - use rand::Rng; - use uuid::Uuid; use arrow::array::ArrayRef; use arrow::compute::SortOptions; - use arrow::datatypes::{DataType, Field, Schema, Int32Type}; + use arrow::datatypes::{DataType, Field, Schema}; use arrow::record_batch::RecordBatch; use datafusion_execution::config::SessionConfig; + use futures::{FutureExt, StreamExt}; use tempfile::TempDir; - use futures::{FutureExt, StreamExt, stream::BoxStream}; - use crate::execution::context::SessionContext; use crate::physical_plan::coalesce_partitions::CoalescePartitionsExec; use crate::physical_plan::expressions::col; use crate::physical_plan::memory::MemoryExec; @@ -290,118 +286,10 @@ mod tests { use crate::test::exec::{assert_strong_count_converges_to_zero, BlockingExec}; use crate::test::{self, assert_is_pending}; use crate::{assert_batches_eq, test_util}; - use arrow::array::{Int32Array, StringArray, TimestampNanosecondArray, DictionaryArray}; - - use crate::physical_plan::streaming::PartitionStream; - use crate::physical_plan::stream::RecordBatchStreamAdapter; - use crate::datasource::{streaming::StreamingTable, TableProvider}; + use arrow::array::{Int32Array, StringArray, TimestampNanosecondArray}; use super::*; - fn make_infinite_sorted_stream(col_b_init: &u32) -> BoxStream<'static, RecordBatch> { - let col_b_init_clone = col_b_init.clone(); - futures::stream::unfold((0, col_b_init_clone), move |(mut counter, mut col_b_ascii)| async move { - if counter >= 12000 { - return None; - } - - if counter % 5 == 0 { - col_b_ascii = col_b_ascii + 2; - } - - counter = counter + 1; - - // building col `a` - let mut values_vector: Vec = Vec::new(); - for _i in 1..=8192 { - values_vector.push(String::from(Uuid::new_v4().to_string())); - } - let values = StringArray::from(values_vector); - - let mut keys_vector: Vec = Vec::new(); - for _i in 1..=8192 { - keys_vector.push(rand::thread_rng().gen_range(0..8192)); - } - let keys = Int32Array::from(keys_vector); - let col_a: ArrayRef = Arc::new(DictionaryArray::::try_new(keys, Arc::new(values)).unwrap()); - - // building col `b` - let mut values: Vec = Vec::new(); - for _i in 1..=8192 { - values.push(col_b_ascii); - } - let col_b: ArrayRef = Arc::new(UInt32Array::from(values)); - - // build a record batch out of col `a` and col `b` - let batch: RecordBatch = RecordBatch::try_from_iter(vec![("a", col_a), ("b", col_b)]).unwrap(); - Some((batch, (counter, col_b_ascii))) - }).boxed() - } - - struct InfiniteStream { - schema: SchemaRef, - col_b_init: u32 - } - - impl PartitionStream for InfiniteStream { - fn schema(&self) -> &SchemaRef { - &self.schema - } - - fn execute(&self, _ctx: Arc) -> SendableRecordBatchStream { - // We create an iterator from the record batches and map them into Ok values, - // converting the iterator into a futures::stream::Stream - Box::pin(RecordBatchStreamAdapter::new( - self.schema.clone(), - make_infinite_sorted_stream(&self.col_b_init).map(Ok) - )) - } - } - - #[tokio::test] - #[ignore] - async fn test_dict_merge_infinite() { - let session_ctx = SessionContext::new(); - let task_ctx: Arc = session_ctx.task_ctx(); - - let schema = SchemaRef::new(Schema::new(vec![ - Field::new("a", DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)), false), - Field::new("b", DataType::UInt32, false), - ])); - - let stream_1 = Arc::new(InfiniteStream { - schema: schema.clone(), col_b_init: 1 - }); - - let stream_2 = Arc::new(InfiniteStream { - schema: schema.clone(), col_b_init: 2 - }); - - println!("SortPreservingMergeExec result: "); - - let sort = vec![ - PhysicalSortExpr { - expr: col("b", &schema).unwrap(), - options: Default::default(), - }, - PhysicalSortExpr { - expr: col("a", &schema).unwrap(), - options: Default::default(), - }, - ]; - - let provider = StreamingTable::try_new(schema, vec![stream_1, stream_2]).unwrap(); - - let plan = provider.scan(&session_ctx.state(), None, &[], None).await.unwrap(); - let exec = Arc::new(SortPreservingMergeExec::new(sort, plan)); - let mut stream = exec.execute(0, task_ctx).unwrap(); - while let Some(batch) = stream.next().await { - println!("{}", arrow::util::pretty::pretty_format_batches(&[batch.unwrap().clone()]) - .unwrap() - .to_string()); - } - } - #[tokio::test] async fn test_merge_interleave() { let task_ctx = Arc::new(TaskContext::default()); From 6cc64680e8afcb9953728f7a17585b8981f6e308 Mon Sep 17 00:00:00 2001 From: Jayjeet Chakraborty Date: Thu, 7 Sep 2023 00:23:45 -0700 Subject: [PATCH 13/55] Update datafusion/core/src/physical_plan/wrapper.rs Co-authored-by: Andrew Lamb --- datafusion/core/src/physical_plan/wrapper.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/datafusion/core/src/physical_plan/wrapper.rs b/datafusion/core/src/physical_plan/wrapper.rs index ac76df0ed8bf..6b36d426c292 100644 --- a/datafusion/core/src/physical_plan/wrapper.rs +++ b/datafusion/core/src/physical_plan/wrapper.rs @@ -64,6 +64,7 @@ impl CardinalityAwareRowConverter { } } +#[cfg(test)] mod tests { use std::sync::Arc; From 74aaa59a2d6ae1a61455990de624fdd7822e33f1 Mon Sep 17 00:00:00 2001 From: Jayjeet Chakraborty Date: Fri, 8 Sep 2023 08:08:59 -0700 Subject: [PATCH 14/55] Update convert_rows signature and add empty_rows --- datafusion/core/src/physical_plan/wrapper.rs | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/datafusion/core/src/physical_plan/wrapper.rs b/datafusion/core/src/physical_plan/wrapper.rs index 6b36d426c292..f366609173fa 100644 --- a/datafusion/core/src/physical_plan/wrapper.rs +++ b/datafusion/core/src/physical_plan/wrapper.rs @@ -19,7 +19,7 @@ use arrow::datatypes::DataType; use arrow::error::ArrowError; use arrow_array::*; use arrow_array::cast::AsArray; -use arrow::row::RowConverter; +use arrow::row::{RowConverter, Row}; const LOW_CARDINALITY_THRESHOLD: usize = 10; @@ -41,7 +41,14 @@ impl CardinalityAwareRowConverter { return self.inner.as_ref().unwrap().size(); } - pub fn convert_rows(&self, rows: &Rows) -> Result, ArrowError> { + pub fn empty_rows(&self, row_capacity: usize, data_capacity: usize) -> Rows { + return self.inner.as_ref().unwrap().empty_rows(row_capacity, data_capacity); + } + + pub fn convert_rows<'a, I>(&self, rows: I) -> Result, ArrowError> + where + I: IntoIterator>, + { return self.inner.as_ref().unwrap().convert_rows(rows); } From 0d305ee3b574f41c2fdad21c50a9aacf114ac841 Mon Sep 17 00:00:00 2001 From: Jayjeet Chakraborty Date: Fri, 8 Sep 2023 08:18:09 -0700 Subject: [PATCH 15/55] Add comments to the test --- datafusion/core/src/physical_plan/wrapper.rs | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/datafusion/core/src/physical_plan/wrapper.rs b/datafusion/core/src/physical_plan/wrapper.rs index f366609173fa..871a862fd37d 100644 --- a/datafusion/core/src/physical_plan/wrapper.rs +++ b/datafusion/core/src/physical_plan/wrapper.rs @@ -137,6 +137,9 @@ mod tests { arrow::row::SortField::new_with_options(DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)), SortOptions::default()), arrow::row::SortField::new_with_options(DataType::Int32, SortOptions::default()) ]; + + // With the `CardinalityAwareRowConverter`, when the high cardinality dictionary-encoded sort field is + // converted to the `Row` format, the dictionary encoding is not preserved and we switch to Utf8 encoding. let mut converter = CardinalityAwareRowConverter::new(sort_fields.clone()).unwrap(); let rows = converter.convert_columns(&batch.columns()).unwrap(); let converted_batch = converter.convert_rows(&rows).unwrap(); @@ -144,7 +147,8 @@ mod tests { let mut converter = RowConverter::new(sort_fields.clone()).unwrap(); let rows = converter.convert_columns(&batch.columns()).unwrap(); - let converted_batch = converter.convert_rows(&rows).unwrap(); + let converted_batch: Vec> = converter.convert_rows(&rows).unwrap(); + // With the `RowConverter`, the dictionary encoding is preserved. assert_eq!(converted_batch[0].data_type(), &DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8))); } @@ -155,6 +159,8 @@ mod tests { arrow::row::SortField::new_with_options(DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)), SortOptions::default()), arrow::row::SortField::new_with_options(DataType::Int32, SortOptions::default()) ]; + // With low cardinality dictionary-encoded sort fields, both `CardinalityAwareRowConverter` and `RowConverter` + // preserves the dictionary encoding. let mut converter = CardinalityAwareRowConverter::new(sort_fields.clone()).unwrap(); let rows = converter.convert_columns(&batch.columns()).unwrap(); let converted_batch = converter.convert_rows(&rows).unwrap(); From 2391f4373ae3cce692d990085e075ea11f559d2b Mon Sep 17 00:00:00 2001 From: Jayjeet Chakraborty Date: Fri, 8 Sep 2023 08:25:18 -0700 Subject: [PATCH 16/55] Use Some and take() semantics --- datafusion/core/src/physical_plan/wrapper.rs | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/datafusion/core/src/physical_plan/wrapper.rs b/datafusion/core/src/physical_plan/wrapper.rs index 871a862fd37d..211aa5578e72 100644 --- a/datafusion/core/src/physical_plan/wrapper.rs +++ b/datafusion/core/src/physical_plan/wrapper.rs @@ -55,17 +55,16 @@ impl CardinalityAwareRowConverter { pub fn convert_columns( &mut self, columns: &[ArrayRef]) -> Result { - if self.fields != None { - let mut updated_fields = self.fields.take(); + if let Some(mut updated_fields) = self.fields.take() { for (i, col) in columns.iter().enumerate() { if let DataType::Dictionary(_, _) = col.data_type() { let cardinality = col.as_any_dictionary_opt().unwrap().values().len(); if cardinality >= LOW_CARDINALITY_THRESHOLD { - updated_fields.as_mut().unwrap()[i] = updated_fields.as_ref().unwrap()[i].clone().preserve_dictionaries(false); + updated_fields[i] = updated_fields[i].clone().preserve_dictionaries(false); } } } - self.inner = Some(RowConverter::new(updated_fields.unwrap())?); + self.inner = Some(RowConverter::new(updated_fields)?); } self.inner.as_mut().unwrap().convert_columns(columns) } From 05685a13faacd7ee88c890b6878a4262b88159bb Mon Sep 17 00:00:00 2001 From: Jayjeet Chakraborty Date: Fri, 8 Sep 2023 08:57:04 -0700 Subject: [PATCH 17/55] Init with a row converter instance instead of none --- datafusion/core/src/physical_plan/wrapper.rs | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/datafusion/core/src/physical_plan/wrapper.rs b/datafusion/core/src/physical_plan/wrapper.rs index 211aa5578e72..4ed0faeb577c 100644 --- a/datafusion/core/src/physical_plan/wrapper.rs +++ b/datafusion/core/src/physical_plan/wrapper.rs @@ -31,9 +31,10 @@ pub struct CardinalityAwareRowConverter { impl CardinalityAwareRowConverter { pub fn new(fields: Vec) -> Result { + let converter = RowConverter::new(fields.clone())?; Ok(Self { fields: Some(fields), - inner: None, + inner: Some(converter), }) } From f107f271102346f0517768b5ac3208d5b062b21b Mon Sep 17 00:00:00 2001 From: Jayjeet Chakraborty Date: Fri, 8 Sep 2023 11:52:43 -0700 Subject: [PATCH 18/55] Remove unused variable --- datafusion/core/src/physical_plan/wrapper.rs | 5 ----- 1 file changed, 5 deletions(-) diff --git a/datafusion/core/src/physical_plan/wrapper.rs b/datafusion/core/src/physical_plan/wrapper.rs index 4ed0faeb577c..4fd6603a600f 100644 --- a/datafusion/core/src/physical_plan/wrapper.rs +++ b/datafusion/core/src/physical_plan/wrapper.rs @@ -87,11 +87,6 @@ mod tests { // Generate a record batch with a high cardinality dictionary field fn generate_batch_with_cardinality(card: String) -> Result { - let schema = SchemaRef::new(Schema::new(vec![ - Field::new("a_dict", DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)), false), - Field::new("b_prim", DataType::Int32, false), - ])); - let col_a: ArrayRef; if card == "high" { // building column `a_dict` From e44e600aad8cd120069f2d7c8605230a78cf305b Mon Sep 17 00:00:00 2001 From: Jayjeet Chakraborty Date: Fri, 8 Sep 2023 11:53:08 -0700 Subject: [PATCH 19/55] Remove unused imports --- datafusion/core/src/physical_plan/wrapper.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/core/src/physical_plan/wrapper.rs b/datafusion/core/src/physical_plan/wrapper.rs index 4fd6603a600f..6a2f15a5e261 100644 --- a/datafusion/core/src/physical_plan/wrapper.rs +++ b/datafusion/core/src/physical_plan/wrapper.rs @@ -80,7 +80,7 @@ mod tests { use arrow::error::ArrowError; use arrow::datatypes::{DataType, Field}; - use arrow_schema::{Schema, SchemaRef, SortOptions}; + use arrow_schema::SortOptions; use arrow_array::{StringArray, DictionaryArray, Int32Array, types::Int32Type, RecordBatch, ArrayRef}; use super::*; From 068695efb3c94d9798c9b60f499c15e31b912462 Mon Sep 17 00:00:00 2001 From: Jayjeet Chakraborty Date: Fri, 8 Sep 2023 11:53:20 -0700 Subject: [PATCH 20/55] Remove unused imports --- datafusion/core/src/physical_plan/wrapper.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/core/src/physical_plan/wrapper.rs b/datafusion/core/src/physical_plan/wrapper.rs index 6a2f15a5e261..5d76a9e1fe69 100644 --- a/datafusion/core/src/physical_plan/wrapper.rs +++ b/datafusion/core/src/physical_plan/wrapper.rs @@ -79,7 +79,7 @@ mod tests { use rand::Rng; use arrow::error::ArrowError; - use arrow::datatypes::{DataType, Field}; + use arrow::datatypes::DataType; use arrow_schema::SortOptions; use arrow_array::{StringArray, DictionaryArray, Int32Array, types::Int32Type, RecordBatch, ArrayRef}; From c7fa02050287f25b35fd1337c90059195b959bfc Mon Sep 17 00:00:00 2001 From: Jayjeet Chakraborty Date: Fri, 8 Sep 2023 12:47:40 -0700 Subject: [PATCH 21/55] Change GroupValues --- .../aggregates/group_values/row.rs | 27 ++++++++++--------- 1 file changed, 14 insertions(+), 13 deletions(-) diff --git a/datafusion/core/src/physical_plan/aggregates/group_values/row.rs b/datafusion/core/src/physical_plan/aggregates/group_values/row.rs index 4eb660d52590..6ceff9de785b 100644 --- a/datafusion/core/src/physical_plan/aggregates/group_values/row.rs +++ b/datafusion/core/src/physical_plan/aggregates/group_values/row.rs @@ -52,7 +52,7 @@ pub struct GroupValuesRows { /// important for multi-column group keys. /// /// [`Row`]: arrow::row::Row - group_values: Rows, + group_values: Option, // buffer to be reused to store hashes hashes_buffer: Vec, @@ -71,14 +71,13 @@ impl GroupValuesRows { .collect(), )?; - let map = RawTable::with_capacity(0); - let group_values = row_converter.empty_rows(0, 0); + let map: RawTable<(u64, usize)> = RawTable::with_capacity(0); Ok(Self { row_converter, map, map_size: 0, - group_values, + group_values: None, hashes_buffer: Default::default(), random_state: Default::default(), }) @@ -90,6 +89,8 @@ impl GroupValues for GroupValuesRows { // Convert the group keys into the row format // Avoid reallocation when https://github.com/apache/arrow-rs/issues/4479 is available let group_rows = self.row_converter.convert_columns(cols)?; + self.group_values = Some(self.row_converter.empty_rows(0, 0)); + let n_rows = group_rows.num_rows(); // tracks to which group each of the input rows belongs @@ -106,7 +107,7 @@ impl GroupValues for GroupValuesRows { // verify that a 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) == self.group_values.row(*group_idx) + group_rows.row(row) == self.group_values.as_mut().unwrap().row(*group_idx) }); let group_idx = match entry { @@ -115,8 +116,8 @@ impl GroupValues for GroupValuesRows { // 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 = self.group_values.num_rows(); - self.group_values.push(group_rows.row(row)); + let group_idx = self.group_values.as_ref().unwrap().num_rows(); + self.group_values.as_mut().unwrap().push(group_rows.row(row)); // for hasher function, use precomputed hash value self.map.insert_accounted( @@ -135,7 +136,7 @@ impl GroupValues for GroupValuesRows { fn size(&self) -> usize { self.row_converter.size() - + self.group_values.size() + + self.group_values.as_ref().unwrap().size() + self.map_size + self.hashes_buffer.allocated_size() } @@ -145,25 +146,25 @@ impl GroupValues for GroupValuesRows { } fn len(&self) -> usize { - self.group_values.num_rows() + self.group_values.as_ref().unwrap().num_rows() } fn emit(&mut self, emit_to: EmitTo) -> Result> { Ok(match emit_to { EmitTo::All => { // Eventually we may also want to clear the hash table here - self.row_converter.convert_rows(&self.group_values)? + self.row_converter.convert_rows(self.group_values.as_ref().unwrap())? } EmitTo::First(n) => { - let groups_rows = self.group_values.iter().take(n); + let groups_rows = self.group_values.as_ref().unwrap().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 efficent? let mut new_group_values = self.row_converter.empty_rows(0, 0); - for row in self.group_values.iter().skip(n) { + for row in self.group_values.as_ref().unwrap().iter().skip(n) { new_group_values.push(row); } - std::mem::swap(&mut new_group_values, &mut self.group_values); + std::mem::swap(&mut new_group_values, &mut self.group_values.as_mut().unwrap()); // SAFETY: self.map outlives iterator and is not modified concurrently unsafe { From 4c054f61cea7942fa3b7add84fed42f7fefc7ac3 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Fri, 8 Sep 2023 15:58:56 -0400 Subject: [PATCH 22/55] Add comments, run fmt --- .../aggregates/group_values/row.rs | 13 +- .../core/src/physical_plan/sorts/stream.rs | 2 +- datafusion/core/src/physical_plan/wrapper.rs | 137 +++++++++++++----- 3 files changed, 115 insertions(+), 37 deletions(-) diff --git a/datafusion/core/src/physical_plan/aggregates/group_values/row.rs b/datafusion/core/src/physical_plan/aggregates/group_values/row.rs index 6ceff9de785b..43cca4971c31 100644 --- a/datafusion/core/src/physical_plan/aggregates/group_values/row.rs +++ b/datafusion/core/src/physical_plan/aggregates/group_values/row.rs @@ -117,7 +117,10 @@ impl GroupValues for GroupValuesRows { None => { // Add new entry to aggr_state and save newly created index let group_idx = self.group_values.as_ref().unwrap().num_rows(); - self.group_values.as_mut().unwrap().push(group_rows.row(row)); + self.group_values + .as_mut() + .unwrap() + .push(group_rows.row(row)); // for hasher function, use precomputed hash value self.map.insert_accounted( @@ -153,7 +156,8 @@ impl GroupValues for GroupValuesRows { Ok(match emit_to { EmitTo::All => { // Eventually we may also want to clear the hash table here - self.row_converter.convert_rows(self.group_values.as_ref().unwrap())? + self.row_converter + .convert_rows(self.group_values.as_ref().unwrap())? } EmitTo::First(n) => { let groups_rows = self.group_values.as_ref().unwrap().iter().take(n); @@ -164,7 +168,10 @@ impl GroupValues for GroupValuesRows { for row in self.group_values.as_ref().unwrap().iter().skip(n) { new_group_values.push(row); } - std::mem::swap(&mut new_group_values, &mut self.group_values.as_mut().unwrap()); + std::mem::swap( + &mut new_group_values, + &mut self.group_values.as_mut().unwrap(), + ); // SAFETY: self.map outlives iterator and is not modified concurrently unsafe { diff --git a/datafusion/core/src/physical_plan/sorts/stream.rs b/datafusion/core/src/physical_plan/sorts/stream.rs index 94ea627e7cfd..aba3835bf013 100644 --- a/datafusion/core/src/physical_plan/sorts/stream.rs +++ b/datafusion/core/src/physical_plan/sorts/stream.rs @@ -16,9 +16,9 @@ // under the License. use crate::physical_plan::sorts::cursor::{FieldArray, FieldCursor, RowCursor}; +use crate::physical_plan::wrapper::CardinalityAwareRowConverter; use crate::physical_plan::SendableRecordBatchStream; use crate::physical_plan::{PhysicalExpr, PhysicalSortExpr}; -use crate::physical_plan::wrapper::CardinalityAwareRowConverter; use arrow::array::Array; use arrow::datatypes::Schema; use arrow::record_batch::RecordBatch; diff --git a/datafusion/core/src/physical_plan/wrapper.rs b/datafusion/core/src/physical_plan/wrapper.rs index 5d76a9e1fe69..cdf76156c6b5 100644 --- a/datafusion/core/src/physical_plan/wrapper.rs +++ b/datafusion/core/src/physical_plan/wrapper.rs @@ -14,15 +14,52 @@ // KIND, either express or implied. See the License for the // specific language governing permissions and limitations // under the License. -use arrow::row::{SortField, Rows}; + +//! [`CardinalityAwareRowConverter`] for converting data to +//! [`arrow_row`] format. + use arrow::datatypes::DataType; use arrow::error::ArrowError; -use arrow_array::*; +use arrow::row::{Row, RowConverter}; +use arrow::row::{Rows, SortField}; use arrow_array::cast::AsArray; -use arrow::row::{RowConverter, Row}; +use arrow_array::*; +/// The threshold of the number of values at which to consider a +/// [`DictionaryArray`] "high" cardinality. +/// +/// Since [`RowConverter`] blindly generates a mapping for all values, +/// regardless of if they appear in the keys, this value is compared +/// to the length of values. +/// +/// The assumption is that the number of potential distinct key values +/// (aka the length of the values array) is a more robust predictor of +/// being "high" cardinality than the actual number of keys used. The +/// intuition for this is that other values in the dictionary could be +/// used in subsequent batches. +/// +/// While an argument can made for doing something more sophisticated, +/// this would likely only really make sense if the dictionary +/// interner itself followed a similar approach, which it did not at +/// the time of this writing. const LOW_CARDINALITY_THRESHOLD: usize = 10; +/// Wrapper around an [`arrow_row`] [`RowConverter` that disables +/// dictionary preservation for high cardinality columns, based on the +/// observed cardinalities in the first columns converted. +/// +/// ## Background +/// +/// By default, the [`RowConverter`] interns (and keeps a copy of) all +/// values from [`DictionaryArray`]s. For low cardinality columns +/// (with few distinct values) this approach is both faster and more +/// memory efficient. However for high cardinality coumns it is slower +/// and requires more memory. In certain degenerate cases, such as +/// columns of nearly unique values, the `RowConverter` will keep a +/// copy of the entire column. +/// +/// See for +/// more details #[derive(Debug)] pub struct CardinalityAwareRowConverter { fields: Option>, @@ -37,13 +74,17 @@ impl CardinalityAwareRowConverter { inner: Some(converter), }) } - + pub fn size(&self) -> usize { return self.inner.as_ref().unwrap().size(); } - + pub fn empty_rows(&self, row_capacity: usize, data_capacity: usize) -> Rows { - return self.inner.as_ref().unwrap().empty_rows(row_capacity, data_capacity); + return self + .inner + .as_ref() + .unwrap() + .empty_rows(row_capacity, data_capacity); } pub fn convert_rows<'a, I>(&self, rows: I) -> Result, ArrowError> @@ -53,15 +94,16 @@ impl CardinalityAwareRowConverter { return self.inner.as_ref().unwrap().convert_rows(rows); } - pub fn convert_columns( - &mut self, - columns: &[ArrayRef]) -> Result { + pub fn convert_columns(&mut self, columns: &[ArrayRef]) -> Result { if let Some(mut updated_fields) = self.fields.take() { for (i, col) in columns.iter().enumerate() { if let DataType::Dictionary(_, _) = col.data_type() { + // see comments on LOW_CARDINALITY_THRESHOLD for + // the rationale of this calculation let cardinality = col.as_any_dictionary_opt().unwrap().values().len(); if cardinality >= LOW_CARDINALITY_THRESHOLD { - updated_fields[i] = updated_fields[i].clone().preserve_dictionaries(false); + updated_fields[i] = + updated_fields[i].clone().preserve_dictionaries(false); } } } @@ -75,13 +117,15 @@ impl CardinalityAwareRowConverter { mod tests { use std::sync::Arc; - use uuid::Uuid; use rand::Rng; + use uuid::Uuid; - use arrow::error::ArrowError; use arrow::datatypes::DataType; + use arrow::error::ArrowError; + use arrow_array::{ + types::Int32Type, ArrayRef, DictionaryArray, Int32Array, RecordBatch, StringArray, + }; use arrow_schema::SortOptions; - use arrow_array::{StringArray, DictionaryArray, Int32Array, types::Int32Type, RecordBatch, ArrayRef}; use super::*; @@ -101,24 +145,28 @@ mod tests { keys_vector.push(rand::thread_rng().gen_range(0..20)); } let keys = Int32Array::from(keys_vector); - col_a = Arc::new(DictionaryArray::::try_new(keys, Arc::new(values)).unwrap()); + col_a = Arc::new( + DictionaryArray::::try_new(keys, Arc::new(values)).unwrap(), + ); } else { let values_vector = vec!["a", "b", "c"]; let values = StringArray::from(values_vector); - + let mut keys_vector: Vec = Vec::new(); for _i in 1..=20 { keys_vector.push(rand::thread_rng().gen_range(0..2)); } let keys = Int32Array::from(keys_vector); - col_a = Arc::new(DictionaryArray::::try_new(keys, Arc::new(values)).unwrap()); + col_a = Arc::new( + DictionaryArray::::try_new(keys, Arc::new(values)).unwrap(), + ); } // building column `b_prim` let mut values: Vec = Vec::new(); - for _i in 1..=20 { - values.push(rand::thread_rng().gen_range(0..20)); - } + for _i in 1..=20 { + values.push(rand::thread_rng().gen_range(0..20)); + } let col_b: ArrayRef = Arc::new(Int32Array::from(values)); // building the record batch @@ -129,41 +177,64 @@ mod tests { async fn test_with_high_card() { let batch = generate_batch_with_cardinality(String::from("high")).unwrap(); let sort_fields = vec![ - arrow::row::SortField::new_with_options(DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)), SortOptions::default()), - arrow::row::SortField::new_with_options(DataType::Int32, SortOptions::default()) + arrow::row::SortField::new_with_options( + DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)), + SortOptions::default(), + ), + arrow::row::SortField::new_with_options( + DataType::Int32, + SortOptions::default(), + ), ]; - // With the `CardinalityAwareRowConverter`, when the high cardinality dictionary-encoded sort field is + // With the `CardinalityAwareRowConverter`, when the high cardinality dictionary-encoded sort field is // converted to the `Row` format, the dictionary encoding is not preserved and we switch to Utf8 encoding. - let mut converter = CardinalityAwareRowConverter::new(sort_fields.clone()).unwrap(); - let rows = converter.convert_columns(&batch.columns()).unwrap(); + let mut converter = + CardinalityAwareRowConverter::new(sort_fields.clone()).unwrap(); + let rows = converter.convert_columns(&batch.columns()).unwrap(); let converted_batch = converter.convert_rows(&rows).unwrap(); assert_eq!(converted_batch[0].data_type(), &DataType::Utf8); let mut converter = RowConverter::new(sort_fields.clone()).unwrap(); - let rows = converter.convert_columns(&batch.columns()).unwrap(); + let rows = converter.convert_columns(&batch.columns()).unwrap(); let converted_batch: Vec> = converter.convert_rows(&rows).unwrap(); // With the `RowConverter`, the dictionary encoding is preserved. - assert_eq!(converted_batch[0].data_type(), &DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8))); + assert_eq!( + converted_batch[0].data_type(), + &DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)) + ); } #[tokio::test] async fn test_with_low_card() { let batch = generate_batch_with_cardinality(String::from("low")).unwrap(); let sort_fields = vec![ - arrow::row::SortField::new_with_options(DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)), SortOptions::default()), - arrow::row::SortField::new_with_options(DataType::Int32, SortOptions::default()) + arrow::row::SortField::new_with_options( + DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)), + SortOptions::default(), + ), + arrow::row::SortField::new_with_options( + DataType::Int32, + SortOptions::default(), + ), ]; // With low cardinality dictionary-encoded sort fields, both `CardinalityAwareRowConverter` and `RowConverter` // preserves the dictionary encoding. - let mut converter = CardinalityAwareRowConverter::new(sort_fields.clone()).unwrap(); - let rows = converter.convert_columns(&batch.columns()).unwrap(); + let mut converter = + CardinalityAwareRowConverter::new(sort_fields.clone()).unwrap(); + let rows = converter.convert_columns(&batch.columns()).unwrap(); let converted_batch = converter.convert_rows(&rows).unwrap(); - assert_eq!(converted_batch[0].data_type(), &DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8))); + assert_eq!( + converted_batch[0].data_type(), + &DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)) + ); let mut converter = RowConverter::new(sort_fields.clone()).unwrap(); - let rows = converter.convert_columns(&batch.columns()).unwrap(); + let rows = converter.convert_columns(&batch.columns()).unwrap(); let converted_batch = converter.convert_rows(&rows).unwrap(); - assert_eq!(converted_batch[0].data_type(), &DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8))); + assert_eq!( + converted_batch[0].data_type(), + &DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)) + ); } } From c58539bbab5bee5d4efe9e228f8795579f8149b5 Mon Sep 17 00:00:00 2001 From: Jayjeet Chakraborty Date: Fri, 8 Sep 2023 15:10:16 -0700 Subject: [PATCH 23/55] Init with a empty row converter --- .../core/src/physical_plan/aggregates/group_values/row.rs | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/datafusion/core/src/physical_plan/aggregates/group_values/row.rs b/datafusion/core/src/physical_plan/aggregates/group_values/row.rs index 43cca4971c31..eebb487c69d0 100644 --- a/datafusion/core/src/physical_plan/aggregates/group_values/row.rs +++ b/datafusion/core/src/physical_plan/aggregates/group_values/row.rs @@ -71,13 +71,14 @@ impl GroupValuesRows { .collect(), )?; - let map: RawTable<(u64, usize)> = RawTable::with_capacity(0); + let map = RawTable::with_capacity(0); + let group_values = row_converter.empty_rows(0, 0); Ok(Self { row_converter, map, map_size: 0, - group_values: None, + group_values: Some(group_values), hashes_buffer: Default::default(), random_state: Default::default(), }) From 74533c6cfb9687da304e6ccdc7ed7a396c1256f4 Mon Sep 17 00:00:00 2001 From: Jayjeet Chakraborty Date: Fri, 8 Sep 2023 15:11:48 -0700 Subject: [PATCH 24/55] Use the cardinality aware row converter --- .../core/src/physical_plan/aggregates/group_values/row.rs | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/datafusion/core/src/physical_plan/aggregates/group_values/row.rs b/datafusion/core/src/physical_plan/aggregates/group_values/row.rs index eebb487c69d0..c1964fc5bda0 100644 --- a/datafusion/core/src/physical_plan/aggregates/group_values/row.rs +++ b/datafusion/core/src/physical_plan/aggregates/group_values/row.rs @@ -16,6 +16,7 @@ // under the License. use crate::physical_plan::aggregates::group_values::GroupValues; +use crate::physical_plan::wrapper::CardinalityAwareRowConverter; use ahash::RandomState; use arrow::row::{RowConverter, Rows, SortField}; use arrow_array::ArrayRef; @@ -29,7 +30,7 @@ use hashbrown::raw::RawTable; /// A [`GroupValues`] making use of [`Rows`] pub struct GroupValuesRows { /// Converter for the group values - row_converter: RowConverter, + row_converter: CardinalityAwareRowConverter, /// Logically maps group values to a group_index in /// [`Self::group_values`] and in each accumulator @@ -63,7 +64,7 @@ pub struct GroupValuesRows { impl GroupValuesRows { pub fn try_new(schema: SchemaRef) -> Result { - let row_converter = RowConverter::new( + let row_converter = CardinalityAwareRowConverter::new( schema .fields() .iter() From b90f40daae34183e9d8a196fe6c70e4d63054191 Mon Sep 17 00:00:00 2001 From: Jayjeet Chakraborty Date: Fri, 8 Sep 2023 22:40:44 -0700 Subject: [PATCH 25/55] Reconvert the group values --- .../core/src/physical_plan/aggregates/group_values/row.rs | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/datafusion/core/src/physical_plan/aggregates/group_values/row.rs b/datafusion/core/src/physical_plan/aggregates/group_values/row.rs index c1964fc5bda0..dd3e25bbbb49 100644 --- a/datafusion/core/src/physical_plan/aggregates/group_values/row.rs +++ b/datafusion/core/src/physical_plan/aggregates/group_values/row.rs @@ -90,8 +90,10 @@ impl GroupValues for GroupValuesRows { fn intern(&mut self, cols: &[ArrayRef], groups: &mut Vec) -> Result<()> { // Convert the group keys into the row format // Avoid reallocation when https://github.com/apache/arrow-rs/issues/4479 is available + + let group_values_cols = self.row_converter.convert_rows(self.group_values.as_ref().unwrap())?; let group_rows = self.row_converter.convert_columns(cols)?; - self.group_values = Some(self.row_converter.empty_rows(0, 0)); + self.group_values = Some(self.row_converter.convert_columns(&group_values_cols)?); let n_rows = group_rows.num_rows(); From 4cf0218a765c95bd09326ad7040ab5bebf9fb06b Mon Sep 17 00:00:00 2001 From: Jayjeet Chakraborty Date: Fri, 8 Sep 2023 22:56:24 -0700 Subject: [PATCH 26/55] Rename wrapper to row_converter --- .../core/src/physical_plan/aggregates/group_values/row.rs | 2 +- datafusion/core/src/physical_plan/mod.rs | 2 +- .../core/src/physical_plan/{wrapper.rs => row_converter.rs} | 0 datafusion/core/src/physical_plan/sorts/stream.rs | 2 +- 4 files changed, 3 insertions(+), 3 deletions(-) rename datafusion/core/src/physical_plan/{wrapper.rs => row_converter.rs} (100%) diff --git a/datafusion/core/src/physical_plan/aggregates/group_values/row.rs b/datafusion/core/src/physical_plan/aggregates/group_values/row.rs index dd3e25bbbb49..52d914cb1066 100644 --- a/datafusion/core/src/physical_plan/aggregates/group_values/row.rs +++ b/datafusion/core/src/physical_plan/aggregates/group_values/row.rs @@ -16,7 +16,7 @@ // under the License. use crate::physical_plan::aggregates::group_values::GroupValues; -use crate::physical_plan::wrapper::CardinalityAwareRowConverter; +use crate::physical_plan::row_converter::CardinalityAwareRowConverter; use ahash::RandomState; use arrow::row::{RowConverter, Rows, SortField}; use arrow_array::ArrayRef; diff --git a/datafusion/core/src/physical_plan/mod.rs b/datafusion/core/src/physical_plan/mod.rs index d10888b64543..6f85d3317d4f 100644 --- a/datafusion/core/src/physical_plan/mod.rs +++ b/datafusion/core/src/physical_plan/mod.rs @@ -402,7 +402,7 @@ pub mod union; pub mod unnest; pub mod values; pub mod windows; -mod wrapper; +mod row_converter; use crate::physical_plan::repartition::RepartitionExec; use crate::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; diff --git a/datafusion/core/src/physical_plan/wrapper.rs b/datafusion/core/src/physical_plan/row_converter.rs similarity index 100% rename from datafusion/core/src/physical_plan/wrapper.rs rename to datafusion/core/src/physical_plan/row_converter.rs diff --git a/datafusion/core/src/physical_plan/sorts/stream.rs b/datafusion/core/src/physical_plan/sorts/stream.rs index aba3835bf013..9b469b3e64a0 100644 --- a/datafusion/core/src/physical_plan/sorts/stream.rs +++ b/datafusion/core/src/physical_plan/sorts/stream.rs @@ -16,7 +16,7 @@ // under the License. use crate::physical_plan::sorts::cursor::{FieldArray, FieldCursor, RowCursor}; -use crate::physical_plan::wrapper::CardinalityAwareRowConverter; +use crate::physical_plan::row_converter::CardinalityAwareRowConverter; use crate::physical_plan::SendableRecordBatchStream; use crate::physical_plan::{PhysicalExpr, PhysicalSortExpr}; use arrow::array::Array; From eb811910ac96e6fd0fc81bdc7b09855400fd3cd3 Mon Sep 17 00:00:00 2001 From: Jayjeet Chakraborty Date: Sat, 9 Sep 2023 12:01:21 -0700 Subject: [PATCH 27/55] Recovert the group values --- .../aggregates/group_values/row.rs | 19 ++++++++++++++----- .../src/physical_plan/aggregates/row_hash.rs | 7 ++++++- 2 files changed, 20 insertions(+), 6 deletions(-) diff --git a/datafusion/core/src/physical_plan/aggregates/group_values/row.rs b/datafusion/core/src/physical_plan/aggregates/group_values/row.rs index 52d914cb1066..559fccc3822c 100644 --- a/datafusion/core/src/physical_plan/aggregates/group_values/row.rs +++ b/datafusion/core/src/physical_plan/aggregates/group_values/row.rs @@ -18,7 +18,7 @@ use crate::physical_plan::aggregates::group_values::GroupValues; use crate::physical_plan::row_converter::CardinalityAwareRowConverter; use ahash::RandomState; -use arrow::row::{RowConverter, Rows, SortField}; +use arrow::row::{Rows, SortField}; use arrow_array::ArrayRef; use arrow_schema::SchemaRef; use datafusion_common::Result; @@ -60,6 +60,9 @@ pub struct GroupValuesRows { /// Random state for creating hashes random_state: RandomState, + + /// Flag indicating whether the group values have been reconverted + done: bool, } impl GroupValuesRows { @@ -82,6 +85,7 @@ impl GroupValuesRows { group_values: Some(group_values), hashes_buffer: Default::default(), random_state: Default::default(), + done: false, }) } } @@ -90,10 +94,15 @@ impl GroupValues for GroupValuesRows { fn intern(&mut self, cols: &[ArrayRef], groups: &mut Vec) -> Result<()> { // Convert the group keys into the row format // Avoid reallocation when https://github.com/apache/arrow-rs/issues/4479 is available - - let group_values_cols = self.row_converter.convert_rows(self.group_values.as_ref().unwrap())?; - let group_rows = self.row_converter.convert_columns(cols)?; - self.group_values = Some(self.row_converter.convert_columns(&group_values_cols)?); + let group_rows: Rows; + if !self.done { + let group_values_cols = self.row_converter.convert_rows(self.group_values.as_ref().unwrap())?; + group_rows = self.row_converter.convert_columns(cols)?; // 1. pd=false, 2. pd=false + self.group_values = Some(self.row_converter.convert_columns(&group_values_cols)?); + self.done = true; + } else { + group_rows = self.row_converter.convert_columns(cols)?; // 1. pd=false, 2. pd=false + } let n_rows = group_rows.num_rows(); diff --git a/datafusion/core/src/physical_plan/aggregates/row_hash.rs b/datafusion/core/src/physical_plan/aggregates/row_hash.rs index 4613a2e46443..68e9a64d3ff2 100644 --- a/datafusion/core/src/physical_plan/aggregates/row_hash.rs +++ b/datafusion/core/src/physical_plan/aggregates/row_hash.rs @@ -474,7 +474,12 @@ impl GroupedHashAggregateStream { } self.update_memory_reservation()?; - let batch = RecordBatch::try_new(self.schema(), output)?; + assert_eq!(output.len(), self.schema().fields().len()); + let mut output_columns: Vec<(String, ArrayRef)> = Vec::new(); + for (i, field) in self.schema().fields().iter().enumerate() { + output_columns.push((field.name().clone(), output[i].clone())); + } + let batch = RecordBatch::try_from_iter(output_columns)?; Ok(batch) } } From d22b6458301d4d313c14c9fb50fddaec6e26bf72 Mon Sep 17 00:00:00 2001 From: Jayjeet Chakraborty Date: Sat, 9 Sep 2023 20:54:48 -0700 Subject: [PATCH 28/55] Convert back to dictionary --- .../src/physical_plan/aggregates/row_hash.rs | 21 +++++++++++++------ 1 file changed, 15 insertions(+), 6 deletions(-) diff --git a/datafusion/core/src/physical_plan/aggregates/row_hash.rs b/datafusion/core/src/physical_plan/aggregates/row_hash.rs index 68e9a64d3ff2..fd662befda1b 100644 --- a/datafusion/core/src/physical_plan/aggregates/row_hash.rs +++ b/datafusion/core/src/physical_plan/aggregates/row_hash.rs @@ -37,6 +37,7 @@ use crate::physical_plan::metrics::{BaselineMetrics, RecordOutput}; use crate::physical_plan::{aggregates, PhysicalExpr}; use crate::physical_plan::{RecordBatchStream, SendableRecordBatchStream}; use arrow::array::*; +use arrow::datatypes::Int32Type; use arrow::{datatypes::SchemaRef, record_batch::RecordBatch}; use datafusion_common::Result; use datafusion_execution::memory_pool::proxy::VecAllocExt; @@ -474,12 +475,20 @@ impl GroupedHashAggregateStream { } self.update_memory_reservation()?; - assert_eq!(output.len(), self.schema().fields().len()); - let mut output_columns: Vec<(String, ArrayRef)> = Vec::new(); - for (i, field) in self.schema().fields().iter().enumerate() { - output_columns.push((field.name().clone(), output[i].clone())); - } - let batch = RecordBatch::try_from_iter(output_columns)?; + + let mut updated_output: Vec> = Vec::new(); + output.iter().for_each(|x| match x.data_type() { + arrow::datatypes::DataType::Utf8 => { + let string_arr: StringArray = StringArray::from(x.to_data()); + let dict_array: DictionaryArray = string_arr.into_iter().collect(); + updated_output.push(Arc::new(dict_array)); + } + _ => { + updated_output.push(x.clone()); + } + }); + + let batch = RecordBatch::try_new(self.schema(), updated_output)?; Ok(batch) } } From aa24717f07aa66e4e1b3367f47654bcbebf8f0dd Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Mon, 11 Sep 2023 09:39:11 -0400 Subject: [PATCH 29/55] fmt --- .../core/src/physical_plan/aggregates/group_values/row.rs | 7 +++++-- datafusion/core/src/physical_plan/aggregates/row_hash.rs | 3 ++- datafusion/core/src/physical_plan/mod.rs | 2 +- datafusion/core/src/physical_plan/sorts/stream.rs | 2 +- 4 files changed, 9 insertions(+), 5 deletions(-) diff --git a/datafusion/core/src/physical_plan/aggregates/group_values/row.rs b/datafusion/core/src/physical_plan/aggregates/group_values/row.rs index 559fccc3822c..bb033e0275cf 100644 --- a/datafusion/core/src/physical_plan/aggregates/group_values/row.rs +++ b/datafusion/core/src/physical_plan/aggregates/group_values/row.rs @@ -96,9 +96,12 @@ impl GroupValues for GroupValuesRows { // Avoid reallocation when https://github.com/apache/arrow-rs/issues/4479 is available let group_rows: Rows; if !self.done { - let group_values_cols = self.row_converter.convert_rows(self.group_values.as_ref().unwrap())?; + let group_values_cols = self + .row_converter + .convert_rows(self.group_values.as_ref().unwrap())?; group_rows = self.row_converter.convert_columns(cols)?; // 1. pd=false, 2. pd=false - self.group_values = Some(self.row_converter.convert_columns(&group_values_cols)?); + self.group_values = + Some(self.row_converter.convert_columns(&group_values_cols)?); self.done = true; } else { group_rows = self.row_converter.convert_columns(cols)?; // 1. pd=false, 2. pd=false diff --git a/datafusion/core/src/physical_plan/aggregates/row_hash.rs b/datafusion/core/src/physical_plan/aggregates/row_hash.rs index fd662befda1b..09b918ade9fb 100644 --- a/datafusion/core/src/physical_plan/aggregates/row_hash.rs +++ b/datafusion/core/src/physical_plan/aggregates/row_hash.rs @@ -480,7 +480,8 @@ impl GroupedHashAggregateStream { output.iter().for_each(|x| match x.data_type() { arrow::datatypes::DataType::Utf8 => { let string_arr: StringArray = StringArray::from(x.to_data()); - let dict_array: DictionaryArray = string_arr.into_iter().collect(); + let dict_array: DictionaryArray = + string_arr.into_iter().collect(); updated_output.push(Arc::new(dict_array)); } _ => { diff --git a/datafusion/core/src/physical_plan/mod.rs b/datafusion/core/src/physical_plan/mod.rs index 6f85d3317d4f..e0603d162f34 100644 --- a/datafusion/core/src/physical_plan/mod.rs +++ b/datafusion/core/src/physical_plan/mod.rs @@ -393,6 +393,7 @@ pub mod memory; pub mod metrics; pub mod projection; pub mod repartition; +mod row_converter; pub mod sorts; pub mod stream; pub mod streaming; @@ -402,7 +403,6 @@ pub mod union; pub mod unnest; pub mod values; pub mod windows; -mod row_converter; use crate::physical_plan::repartition::RepartitionExec; use crate::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; diff --git a/datafusion/core/src/physical_plan/sorts/stream.rs b/datafusion/core/src/physical_plan/sorts/stream.rs index 9b469b3e64a0..b72c49ab53a3 100644 --- a/datafusion/core/src/physical_plan/sorts/stream.rs +++ b/datafusion/core/src/physical_plan/sorts/stream.rs @@ -15,8 +15,8 @@ // specific language governing permissions and limitations // under the License. -use crate::physical_plan::sorts::cursor::{FieldArray, FieldCursor, RowCursor}; use crate::physical_plan::row_converter::CardinalityAwareRowConverter; +use crate::physical_plan::sorts::cursor::{FieldArray, FieldCursor, RowCursor}; use crate::physical_plan::SendableRecordBatchStream; use crate::physical_plan::{PhysicalExpr, PhysicalSortExpr}; use arrow::array::Array; From dbd66f284fa84534254bc7c6641bdbe61f611422 Mon Sep 17 00:00:00 2001 From: Jayjeet Chakraborty Date: Mon, 11 Sep 2023 10:51:07 -0500 Subject: [PATCH 30/55] A fmt pass --- .../core/src/physical_plan/aggregates/group_values/row.rs | 7 +++++-- datafusion/core/src/physical_plan/aggregates/row_hash.rs | 3 ++- datafusion/core/src/physical_plan/mod.rs | 2 +- datafusion/core/src/physical_plan/sorts/stream.rs | 2 +- 4 files changed, 9 insertions(+), 5 deletions(-) diff --git a/datafusion/core/src/physical_plan/aggregates/group_values/row.rs b/datafusion/core/src/physical_plan/aggregates/group_values/row.rs index 559fccc3822c..bb033e0275cf 100644 --- a/datafusion/core/src/physical_plan/aggregates/group_values/row.rs +++ b/datafusion/core/src/physical_plan/aggregates/group_values/row.rs @@ -96,9 +96,12 @@ impl GroupValues for GroupValuesRows { // Avoid reallocation when https://github.com/apache/arrow-rs/issues/4479 is available let group_rows: Rows; if !self.done { - let group_values_cols = self.row_converter.convert_rows(self.group_values.as_ref().unwrap())?; + let group_values_cols = self + .row_converter + .convert_rows(self.group_values.as_ref().unwrap())?; group_rows = self.row_converter.convert_columns(cols)?; // 1. pd=false, 2. pd=false - self.group_values = Some(self.row_converter.convert_columns(&group_values_cols)?); + self.group_values = + Some(self.row_converter.convert_columns(&group_values_cols)?); self.done = true; } else { group_rows = self.row_converter.convert_columns(cols)?; // 1. pd=false, 2. pd=false diff --git a/datafusion/core/src/physical_plan/aggregates/row_hash.rs b/datafusion/core/src/physical_plan/aggregates/row_hash.rs index fd662befda1b..09b918ade9fb 100644 --- a/datafusion/core/src/physical_plan/aggregates/row_hash.rs +++ b/datafusion/core/src/physical_plan/aggregates/row_hash.rs @@ -480,7 +480,8 @@ impl GroupedHashAggregateStream { output.iter().for_each(|x| match x.data_type() { arrow::datatypes::DataType::Utf8 => { let string_arr: StringArray = StringArray::from(x.to_data()); - let dict_array: DictionaryArray = string_arr.into_iter().collect(); + let dict_array: DictionaryArray = + string_arr.into_iter().collect(); updated_output.push(Arc::new(dict_array)); } _ => { diff --git a/datafusion/core/src/physical_plan/mod.rs b/datafusion/core/src/physical_plan/mod.rs index 6f85d3317d4f..e0603d162f34 100644 --- a/datafusion/core/src/physical_plan/mod.rs +++ b/datafusion/core/src/physical_plan/mod.rs @@ -393,6 +393,7 @@ pub mod memory; pub mod metrics; pub mod projection; pub mod repartition; +mod row_converter; pub mod sorts; pub mod stream; pub mod streaming; @@ -402,7 +403,6 @@ pub mod union; pub mod unnest; pub mod values; pub mod windows; -mod row_converter; use crate::physical_plan::repartition::RepartitionExec; use crate::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; diff --git a/datafusion/core/src/physical_plan/sorts/stream.rs b/datafusion/core/src/physical_plan/sorts/stream.rs index 9b469b3e64a0..b72c49ab53a3 100644 --- a/datafusion/core/src/physical_plan/sorts/stream.rs +++ b/datafusion/core/src/physical_plan/sorts/stream.rs @@ -15,8 +15,8 @@ // specific language governing permissions and limitations // under the License. -use crate::physical_plan::sorts::cursor::{FieldArray, FieldCursor, RowCursor}; use crate::physical_plan::row_converter::CardinalityAwareRowConverter; +use crate::physical_plan::sorts::cursor::{FieldArray, FieldCursor, RowCursor}; use crate::physical_plan::SendableRecordBatchStream; use crate::physical_plan::{PhysicalExpr, PhysicalSortExpr}; use arrow::array::Array; From 8a42957a5fcb8028844f217e03c17a4c80d9b8ec Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Mon, 11 Sep 2023 12:11:28 -0400 Subject: [PATCH 31/55] fix: fmt --- .../aggregates/group_values/row.rs | 55 ++++++------- .../core/src/physical_plan/row_converter.rs | 78 +++++++++++++------ 2 files changed, 83 insertions(+), 50 deletions(-) diff --git a/datafusion/core/src/physical_plan/aggregates/group_values/row.rs b/datafusion/core/src/physical_plan/aggregates/group_values/row.rs index bb033e0275cf..d24cc0355258 100644 --- a/datafusion/core/src/physical_plan/aggregates/group_values/row.rs +++ b/datafusion/core/src/physical_plan/aggregates/group_values/row.rs @@ -61,8 +61,8 @@ pub struct GroupValuesRows { /// Random state for creating hashes random_state: RandomState, - /// Flag indicating whether the group values have been reconverted - done: bool, + /// Scratch space for storing per-batch group rows + scratch_group_rows: Option, } impl GroupValuesRows { @@ -76,36 +76,34 @@ impl GroupValuesRows { )?; let map = RawTable::with_capacity(0); - let group_values = row_converter.empty_rows(0, 0); Ok(Self { row_converter, map, map_size: 0, - group_values: Some(group_values), + group_values: None, hashes_buffer: Default::default(), random_state: Default::default(), - done: false, + scratch_group_rows: None, }) } } impl GroupValues for GroupValuesRows { fn intern(&mut self, cols: &[ArrayRef], groups: &mut Vec) -> Result<()> { - // Convert the group keys into the row format - // Avoid reallocation when https://github.com/apache/arrow-rs/issues/4479 is available - let group_rows: Rows; - if !self.done { - let group_values_cols = self - .row_converter - .convert_rows(self.group_values.as_ref().unwrap())?; - group_rows = self.row_converter.convert_columns(cols)?; // 1. pd=false, 2. pd=false - self.group_values = - Some(self.row_converter.convert_columns(&group_values_cols)?); - self.done = true; - } else { - group_rows = self.row_converter.convert_columns(cols)?; // 1. pd=false, 2. pd=false - } + // Convert the group keys into the row format, reusing rows when possible + let group_rows = match self.scratch_group_rows.take() { + Some(mut group_rows) => { + self.row_converter.append(&mut group_rows, cols)?; + group_rows + } + None => self.row_converter.convert_columns(cols)?, + }; + + 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 = group_rows.num_rows(); @@ -123,7 +121,7 @@ impl GroupValues for GroupValuesRows { // verify that a 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) == self.group_values.as_mut().unwrap().row(*group_idx) + group_rows.row(row) == group_values.row(*group_idx) }); let group_idx = match entry { @@ -132,11 +130,8 @@ impl GroupValues for GroupValuesRows { // 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 = self.group_values.as_ref().unwrap().num_rows(); - self.group_values - .as_mut() - .unwrap() - .push(group_rows.row(row)); + let group_idx = group_values.num_rows(); + group_values.push(group_rows.row(row)); // for hasher function, use precomputed hash value self.map.insert_accounted( @@ -150,6 +145,9 @@ impl GroupValues for GroupValuesRows { groups.push(group_idx); } + self.group_values = Some(group_values); + self.scratch_group_rows = Some(group_rows); + Ok(()) } @@ -165,7 +163,10 @@ impl GroupValues for GroupValuesRows { } fn len(&self) -> usize { - self.group_values.as_ref().unwrap().num_rows() + self.group_values + .as_ref() + .map(|group_values| group_values.num_rows()) + .unwrap_or(0) } fn emit(&mut self, emit_to: EmitTo) -> Result> { @@ -180,7 +181,7 @@ impl GroupValues for GroupValuesRows { 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 efficent? - let mut new_group_values = self.row_converter.empty_rows(0, 0); + let mut new_group_values = self.row_converter.empty_rows(0, 0)?; for row in self.group_values.as_ref().unwrap().iter().skip(n) { new_group_values.push(row); } diff --git a/datafusion/core/src/physical_plan/row_converter.rs b/datafusion/core/src/physical_plan/row_converter.rs index cdf76156c6b5..9fb3a3342be5 100644 --- a/datafusion/core/src/physical_plan/row_converter.rs +++ b/datafusion/core/src/physical_plan/row_converter.rs @@ -19,11 +19,11 @@ //! [`arrow_row`] format. use arrow::datatypes::DataType; -use arrow::error::ArrowError; use arrow::row::{Row, RowConverter}; use arrow::row::{Rows, SortField}; use arrow_array::cast::AsArray; use arrow_array::*; +use datafusion_common::{internal_err, DataFusionError, Result}; /// The threshold of the number of values at which to consider a /// [`DictionaryArray`] "high" cardinality. @@ -61,41 +61,55 @@ const LOW_CARDINALITY_THRESHOLD: usize = 10; /// See for /// more details #[derive(Debug)] -pub struct CardinalityAwareRowConverter { - fields: Option>, - inner: Option, +pub enum CardinalityAwareRowConverter { + /// Converter is newly initialized, and hasn't yet seen data + New { fields: Vec }, + /// Converter has seen data and can convert data + Converting { inner: RowConverter }, } impl CardinalityAwareRowConverter { - pub fn new(fields: Vec) -> Result { - let converter = RowConverter::new(fields.clone())?; - Ok(Self { - fields: Some(fields), - inner: Some(converter), - }) + pub fn new(fields: Vec) -> Result { + Ok(Self::New { fields }) } pub fn size(&self) -> usize { - return self.inner.as_ref().unwrap().size(); + match self { + Self::New { .. } => { + // TODO account for size of `fields` + 0 + } + Self::Converting { inner } => inner.size(), + } } - pub fn empty_rows(&self, row_capacity: usize, data_capacity: usize) -> Rows { - return self - .inner - .as_ref() - .unwrap() - .empty_rows(row_capacity, data_capacity); + pub fn empty_rows(&self, row_capacity: usize, data_capacity: usize) -> Result { + let converter = self.converter()?; + Ok(converter.empty_rows(row_capacity, data_capacity)) } - pub fn convert_rows<'a, I>(&self, rows: I) -> Result, ArrowError> + pub fn convert_rows<'a, I>(&self, rows: I) -> Result> where I: IntoIterator>, { - return self.inner.as_ref().unwrap().convert_rows(rows); + let converter = self.converter()?; + Ok(converter.convert_rows(rows)?) + } + + pub fn append(&mut self, rows: &mut Rows, columns: &[ArrayRef]) -> Result<()> { + Ok(self.converter_mut(columns)?.append(rows, columns)?) + } + + /// Calls [`RowConverter::convert_columns`] after first + /// initializing the converter based on cardinalities + pub fn convert_columns(&mut self, columns: &[ArrayRef]) -> Result { + Ok(self.converter_mut(columns)?.convert_columns(columns)?) } - pub fn convert_columns(&mut self, columns: &[ArrayRef]) -> Result { - if let Some(mut updated_fields) = self.fields.take() { + /// Return a mutable reference to the inner converter, creating it if needed + fn converter_mut(&mut self, columns: &[ArrayRef]) -> Result<&mut RowConverter> { + if let Self::New { fields } = self { + let mut updated_fields = fields.clone(); for (i, col) in columns.iter().enumerate() { if let DataType::Dictionary(_, _) = col.data_type() { // see comments on LOW_CARDINALITY_THRESHOLD for @@ -107,9 +121,27 @@ impl CardinalityAwareRowConverter { } } } - self.inner = Some(RowConverter::new(updated_fields)?); + *self = Self::Converting { + inner: RowConverter::new(updated_fields)?, + }; + }; + + match self { + Self::New { .. } => { + unreachable!(); + } + Self::Converting { inner } => Ok(inner), + } + } + + /// Return a reference to the inner converter, erroring if we have not yet converted a row. + fn converter(&self) -> Result<&RowConverter> { + match self { + Self::New { .. } => internal_err!( + "CardinalityAwareRowConverter has not converted any rows yet" + ), + Self::Converting { inner } => Ok(inner), } - self.inner.as_mut().unwrap().convert_columns(columns) } } From 1a9a58d3639a079ad781b3759d30a04c0bb6d39a Mon Sep 17 00:00:00 2001 From: Jayjeet Chakraborty Date: Mon, 11 Sep 2023 11:22:45 -0500 Subject: [PATCH 32/55] Move the reconversion to dict to just consider group by columns --- .../src/physical_plan/aggregates/row_hash.rs | 27 +++++++++---------- 1 file changed, 13 insertions(+), 14 deletions(-) diff --git a/datafusion/core/src/physical_plan/aggregates/row_hash.rs b/datafusion/core/src/physical_plan/aggregates/row_hash.rs index 09b918ade9fb..e391e98ddd8b 100644 --- a/datafusion/core/src/physical_plan/aggregates/row_hash.rs +++ b/datafusion/core/src/physical_plan/aggregates/row_hash.rs @@ -458,24 +458,11 @@ impl GroupedHashAggregateStream { return Ok(RecordBatch::new_empty(self.schema())); } - let mut output = self.group_values.emit(emit_to)?; + let output = self.group_values.emit(emit_to)?; if let EmitTo::First(n) = emit_to { self.group_ordering.remove_groups(n); } - // Next output each aggregate value - for acc in self.accumulators.iter_mut() { - match self.mode { - AggregateMode::Partial => output.extend(acc.state(emit_to)?), - AggregateMode::Final - | AggregateMode::FinalPartitioned - | AggregateMode::Single - | AggregateMode::SinglePartitioned => output.push(acc.evaluate(emit_to)?), - } - } - - self.update_memory_reservation()?; - let mut updated_output: Vec> = Vec::new(); output.iter().for_each(|x| match x.data_type() { arrow::datatypes::DataType::Utf8 => { @@ -489,6 +476,18 @@ impl GroupedHashAggregateStream { } }); + // Next output each aggregate value + for acc in self.accumulators.iter_mut() { + match self.mode { + AggregateMode::Partial => updated_output.extend(acc.state(emit_to)?), + AggregateMode::Final + | AggregateMode::FinalPartitioned + | AggregateMode::Single + | AggregateMode::SinglePartitioned => updated_output.push(acc.evaluate(emit_to)?), + } + } + + self.update_memory_reservation()?; let batch = RecordBatch::try_new(self.schema(), updated_output)?; Ok(batch) } From acf1cd41cbb8cb701a46258ff6ad71776cb123a4 Mon Sep 17 00:00:00 2001 From: Jayjeet Chakraborty Date: Mon, 11 Sep 2023 11:41:25 -0500 Subject: [PATCH 33/55] Reconvert only the correct cols --- .../core/src/physical_plan/aggregates/row_hash.rs | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/datafusion/core/src/physical_plan/aggregates/row_hash.rs b/datafusion/core/src/physical_plan/aggregates/row_hash.rs index e391e98ddd8b..4ad1e2bea3d5 100644 --- a/datafusion/core/src/physical_plan/aggregates/row_hash.rs +++ b/datafusion/core/src/physical_plan/aggregates/row_hash.rs @@ -464,17 +464,17 @@ impl GroupedHashAggregateStream { } let mut updated_output: Vec> = Vec::new(); - output.iter().for_each(|x| match x.data_type() { - arrow::datatypes::DataType::Utf8 => { - let string_arr: StringArray = StringArray::from(x.to_data()); + for (col, field) in output.iter().zip(self.schema().fields().iter()) { + if col.data_type() != field.data_type() { + assert!(*col.data_type() == arrow::datatypes::DataType::Utf8); + let string_arr: StringArray = StringArray::from(col.to_data()); let dict_array: DictionaryArray = string_arr.into_iter().collect(); updated_output.push(Arc::new(dict_array)); + } else { + updated_output.push(col.clone()); } - _ => { - updated_output.push(x.clone()); - } - }); + } // Next output each aggregate value for acc in self.accumulators.iter_mut() { From 6543b9d2f804cc60523aab3aacdca2420f9a5a07 Mon Sep 17 00:00:00 2001 From: Jayjeet Chakraborty Date: Mon, 11 Sep 2023 11:48:43 -0500 Subject: [PATCH 34/55] Use assert eq --- datafusion/core/src/physical_plan/aggregates/row_hash.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/core/src/physical_plan/aggregates/row_hash.rs b/datafusion/core/src/physical_plan/aggregates/row_hash.rs index 4ad1e2bea3d5..1bc1b291f6d2 100644 --- a/datafusion/core/src/physical_plan/aggregates/row_hash.rs +++ b/datafusion/core/src/physical_plan/aggregates/row_hash.rs @@ -466,7 +466,7 @@ impl GroupedHashAggregateStream { let mut updated_output: Vec> = Vec::new(); for (col, field) in output.iter().zip(self.schema().fields().iter()) { if col.data_type() != field.data_type() { - assert!(*col.data_type() == arrow::datatypes::DataType::Utf8); + assert_eq!(*col.data_type(), arrow::datatypes::DataType::Utf8); let string_arr: StringArray = StringArray::from(col.to_data()); let dict_array: DictionaryArray = string_arr.into_iter().collect(); From c6bf41a1b4fc512458edb33bba8ae5ace8055d42 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Mon, 11 Sep 2023 13:04:21 -0400 Subject: [PATCH 35/55] clippy --- .../aggregates/group_values/row.rs | 28 ++++++++++++------- .../core/src/physical_plan/row_converter.rs | 21 +++++++------- 2 files changed, 28 insertions(+), 21 deletions(-) diff --git a/datafusion/core/src/physical_plan/aggregates/group_values/row.rs b/datafusion/core/src/physical_plan/aggregates/group_values/row.rs index d24cc0355258..cff4ca532a69 100644 --- a/datafusion/core/src/physical_plan/aggregates/group_values/row.rs +++ b/datafusion/core/src/physical_plan/aggregates/group_values/row.rs @@ -170,11 +170,19 @@ impl GroupValues for GroupValuesRows { } fn emit(&mut self, emit_to: EmitTo) -> Result> { - Ok(match emit_to { + let mut group_values = match self.group_values.take() { + Some(group_values) => group_values, + None => { + // not clear it is actually unreachable (may have to make empty arrays) + unreachable!() + } + }; + + let output = match emit_to { EmitTo::All => { - // Eventually we may also want to clear the hash table here - self.row_converter - .convert_rows(self.group_values.as_ref().unwrap())? + let output = self.row_converter.convert_rows(&group_values)?; + group_values.clear(); + output } EmitTo::First(n) => { let groups_rows = self.group_values.as_ref().unwrap().iter().take(n); @@ -182,13 +190,10 @@ impl GroupValues for GroupValuesRows { // Clear out first n group keys by copying them to a new Rows. // TODO file some ticket in arrow-rs to make this more efficent? let mut new_group_values = self.row_converter.empty_rows(0, 0)?; - for row in self.group_values.as_ref().unwrap().iter().skip(n) { + for row in group_values.iter().skip(n) { new_group_values.push(row); } - std::mem::swap( - &mut new_group_values, - &mut self.group_values.as_mut().unwrap(), - ); + std::mem::swap(&mut new_group_values, &mut group_values); // SAFETY: self.map outlives iterator and is not modified concurrently unsafe { @@ -204,6 +209,9 @@ impl GroupValues for GroupValuesRows { } output } - }) + }; + + self.group_values = Some(group_values); + Ok(output) } } diff --git a/datafusion/core/src/physical_plan/row_converter.rs b/datafusion/core/src/physical_plan/row_converter.rs index 9fb3a3342be5..bf9d58a9c831 100644 --- a/datafusion/core/src/physical_plan/row_converter.rs +++ b/datafusion/core/src/physical_plan/row_converter.rs @@ -163,12 +163,11 @@ mod tests { // Generate a record batch with a high cardinality dictionary field fn generate_batch_with_cardinality(card: String) -> Result { - let col_a: ArrayRef; - if card == "high" { + let col_a = if card == "high" { // building column `a_dict` let mut values_vector: Vec = Vec::new(); for _i in 1..=20 { - values_vector.push(String::from(Uuid::new_v4().to_string())); + values_vector.push(Uuid::new_v4().to_string()); } let values = StringArray::from(values_vector); @@ -177,7 +176,7 @@ mod tests { keys_vector.push(rand::thread_rng().gen_range(0..20)); } let keys = Int32Array::from(keys_vector); - col_a = Arc::new( + Arc::new( DictionaryArray::::try_new(keys, Arc::new(values)).unwrap(), ); } else { @@ -189,10 +188,10 @@ mod tests { keys_vector.push(rand::thread_rng().gen_range(0..2)); } let keys = Int32Array::from(keys_vector); - col_a = Arc::new( + Arc::new( DictionaryArray::::try_new(keys, Arc::new(values)).unwrap(), - ); - } + ) + }; // building column `b_prim` let mut values: Vec = Vec::new(); @@ -223,12 +222,12 @@ mod tests { // converted to the `Row` format, the dictionary encoding is not preserved and we switch to Utf8 encoding. let mut converter = CardinalityAwareRowConverter::new(sort_fields.clone()).unwrap(); - let rows = converter.convert_columns(&batch.columns()).unwrap(); + let rows = converter.convert_columns(batch.columns()).unwrap(); let converted_batch = converter.convert_rows(&rows).unwrap(); assert_eq!(converted_batch[0].data_type(), &DataType::Utf8); let mut converter = RowConverter::new(sort_fields.clone()).unwrap(); - let rows = converter.convert_columns(&batch.columns()).unwrap(); + let rows = converter.convert_columns(batch.columns()).unwrap(); let converted_batch: Vec> = converter.convert_rows(&rows).unwrap(); // With the `RowConverter`, the dictionary encoding is preserved. assert_eq!( @@ -254,7 +253,7 @@ mod tests { // preserves the dictionary encoding. let mut converter = CardinalityAwareRowConverter::new(sort_fields.clone()).unwrap(); - let rows = converter.convert_columns(&batch.columns()).unwrap(); + let rows = converter.convert_columns(batch.columns()).unwrap(); let converted_batch = converter.convert_rows(&rows).unwrap(); assert_eq!( converted_batch[0].data_type(), @@ -262,7 +261,7 @@ mod tests { ); let mut converter = RowConverter::new(sort_fields.clone()).unwrap(); - let rows = converter.convert_columns(&batch.columns()).unwrap(); + let rows = converter.convert_columns(batch.columns()).unwrap(); let converted_batch = converter.convert_rows(&rows).unwrap(); assert_eq!( converted_batch[0].data_type(), From 08c6f7dc5c1e3e870526c8671e7b648d54340375 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Mon, 11 Sep 2023 13:04:55 -0400 Subject: [PATCH 36/55] clippy --- datafusion/core/src/physical_plan/row_converter.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/core/src/physical_plan/row_converter.rs b/datafusion/core/src/physical_plan/row_converter.rs index bf9d58a9c831..9efb6a02a65d 100644 --- a/datafusion/core/src/physical_plan/row_converter.rs +++ b/datafusion/core/src/physical_plan/row_converter.rs @@ -163,7 +163,7 @@ mod tests { // Generate a record batch with a high cardinality dictionary field fn generate_batch_with_cardinality(card: String) -> Result { - let col_a = if card == "high" { + let col_a: ArrayRef = if card == "high" { // building column `a_dict` let mut values_vector: Vec = Vec::new(); for _i in 1..=20 { From c965bbebb3a3f853ba5459ab0e85cb1a857820c1 Mon Sep 17 00:00:00 2001 From: Jayjeet Chakraborty Date: Mon, 11 Sep 2023 12:16:19 -0500 Subject: [PATCH 37/55] Add comment about the reconversion to dict --- datafusion/core/src/physical_plan/aggregates/row_hash.rs | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/datafusion/core/src/physical_plan/aggregates/row_hash.rs b/datafusion/core/src/physical_plan/aggregates/row_hash.rs index 35164d3dbc91..4af868098e40 100644 --- a/datafusion/core/src/physical_plan/aggregates/row_hash.rs +++ b/datafusion/core/src/physical_plan/aggregates/row_hash.rs @@ -463,6 +463,12 @@ impl GroupedHashAggregateStream { self.group_ordering.remove_groups(n); } + // Since the `CardinalityAwareRowConverter` disables dictionary preserving + // for the high-cardinality `Dictionary` encoded group by columns while converting + // a batch into `Rows` for computation, we get the group by columns as `Utf8` + // in the output of the computation. Since the upstream operaters still expect + // `Dictionary` arrays, we need to convert the `Utf8` arrays back into `Dictionary` + // arrays before passing them on. let mut updated_output: Vec> = Vec::new(); for (col, field) in output.iter().zip(self.schema().fields().iter()) { if col.data_type() != field.data_type() { From 8244c833be83a27396ce6031f7406809f166fd67 Mon Sep 17 00:00:00 2001 From: Jayjeet Chakraborty Date: Mon, 11 Sep 2023 12:25:28 -0500 Subject: [PATCH 38/55] Fix the merge issues --- .../core/src/physical_plan/aggregates/row_hash.rs | 14 -------------- 1 file changed, 14 deletions(-) diff --git a/datafusion/core/src/physical_plan/aggregates/row_hash.rs b/datafusion/core/src/physical_plan/aggregates/row_hash.rs index 4af868098e40..00c30e65de24 100644 --- a/datafusion/core/src/physical_plan/aggregates/row_hash.rs +++ b/datafusion/core/src/physical_plan/aggregates/row_hash.rs @@ -496,20 +496,6 @@ impl GroupedHashAggregateStream { } self.update_memory_reservation()?; - - let mut updated_output: Vec> = Vec::new(); - output.iter().for_each(|x| match x.data_type() { - arrow::datatypes::DataType::Utf8 => { - let string_arr: StringArray = StringArray::from(x.to_data()); - let dict_array: DictionaryArray = - string_arr.into_iter().collect(); - updated_output.push(Arc::new(dict_array)); - } - _ => { - updated_output.push(x.clone()); - } - }); - let batch = RecordBatch::try_new(self.schema(), updated_output)?; Ok(batch) } From 235f3bc00a2e4c4bbe0cd0d73d265b12a20c92ba Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Tue, 12 Sep 2023 12:07:28 -0400 Subject: [PATCH 39/55] move data type conversion --- .../aggregates/group_values/row.rs | 1 + .../src/physical_plan/aggregates/row_hash.rs | 30 ++------- .../core/src/physical_plan/row_converter.rs | 67 +++++++++++++------ 3 files changed, 53 insertions(+), 45 deletions(-) diff --git a/datafusion/core/src/physical_plan/aggregates/group_values/row.rs b/datafusion/core/src/physical_plan/aggregates/group_values/row.rs index cff4ca532a69..00b7eba42a20 100644 --- a/datafusion/core/src/physical_plan/aggregates/group_values/row.rs +++ b/datafusion/core/src/physical_plan/aggregates/group_values/row.rs @@ -94,6 +94,7 @@ impl GroupValues for GroupValuesRows { // Convert the group keys into the row format, reusing rows when possible let group_rows = match self.scratch_group_rows.take() { Some(mut group_rows) => { + group_rows.clear(); self.row_converter.append(&mut group_rows, cols)?; group_rows } diff --git a/datafusion/core/src/physical_plan/aggregates/row_hash.rs b/datafusion/core/src/physical_plan/aggregates/row_hash.rs index 00c30e65de24..4613a2e46443 100644 --- a/datafusion/core/src/physical_plan/aggregates/row_hash.rs +++ b/datafusion/core/src/physical_plan/aggregates/row_hash.rs @@ -37,7 +37,6 @@ use crate::physical_plan::metrics::{BaselineMetrics, RecordOutput}; use crate::physical_plan::{aggregates, PhysicalExpr}; use crate::physical_plan::{RecordBatchStream, SendableRecordBatchStream}; use arrow::array::*; -use arrow::datatypes::Int32Type; use arrow::{datatypes::SchemaRef, record_batch::RecordBatch}; use datafusion_common::Result; use datafusion_execution::memory_pool::proxy::VecAllocExt; @@ -458,45 +457,24 @@ impl GroupedHashAggregateStream { return Ok(RecordBatch::new_empty(self.schema())); } - let output = self.group_values.emit(emit_to)?; + let mut output = self.group_values.emit(emit_to)?; if let EmitTo::First(n) = emit_to { self.group_ordering.remove_groups(n); } - // Since the `CardinalityAwareRowConverter` disables dictionary preserving - // for the high-cardinality `Dictionary` encoded group by columns while converting - // a batch into `Rows` for computation, we get the group by columns as `Utf8` - // in the output of the computation. Since the upstream operaters still expect - // `Dictionary` arrays, we need to convert the `Utf8` arrays back into `Dictionary` - // arrays before passing them on. - let mut updated_output: Vec> = Vec::new(); - for (col, field) in output.iter().zip(self.schema().fields().iter()) { - if col.data_type() != field.data_type() { - assert_eq!(*col.data_type(), arrow::datatypes::DataType::Utf8); - let string_arr: StringArray = StringArray::from(col.to_data()); - let dict_array: DictionaryArray = - string_arr.into_iter().collect(); - updated_output.push(Arc::new(dict_array)); - } else { - updated_output.push(col.clone()); - } - } - // Next output each aggregate value for acc in self.accumulators.iter_mut() { match self.mode { - AggregateMode::Partial => updated_output.extend(acc.state(emit_to)?), + AggregateMode::Partial => output.extend(acc.state(emit_to)?), AggregateMode::Final | AggregateMode::FinalPartitioned | AggregateMode::Single - | AggregateMode::SinglePartitioned => { - updated_output.push(acc.evaluate(emit_to)?) - } + | AggregateMode::SinglePartitioned => output.push(acc.evaluate(emit_to)?), } } self.update_memory_reservation()?; - let batch = RecordBatch::try_new(self.schema(), updated_output)?; + let batch = RecordBatch::try_new(self.schema(), output)?; Ok(batch) } } diff --git a/datafusion/core/src/physical_plan/row_converter.rs b/datafusion/core/src/physical_plan/row_converter.rs index 5e898b196517..5bcdc89ff287 100644 --- a/datafusion/core/src/physical_plan/row_converter.rs +++ b/datafusion/core/src/physical_plan/row_converter.rs @@ -64,8 +64,14 @@ const LOW_CARDINALITY_THRESHOLD: usize = 10; pub enum CardinalityAwareRowConverter { /// Converter is newly initialized, and hasn't yet seen data New { fields: Vec }, - /// Converter has seen data and can convert data - Converting { inner: RowConverter }, + /// Converter has seen data and can convert [`Array`]s to/from + /// [`Rows`] + Converting { + converter: RowConverter, + /// if preserve_dictionaries is disabled, the output type can be + /// different than input. + output_types: Vec, + }, } impl CardinalityAwareRowConverter { @@ -79,21 +85,50 @@ impl CardinalityAwareRowConverter { // TODO account for size of `fields` 0 } - Self::Converting { inner } => inner.size(), + Self::Converting { converter, .. } => converter.size(), } } pub fn empty_rows(&self, row_capacity: usize, data_capacity: usize) -> Result { - let converter = self.converter()?; - Ok(converter.empty_rows(row_capacity, data_capacity)) + match self { + Self::New { .. } => internal_err!( + "CardinalityAwareRowConverter has not converted any rows yet" + ), + Self::Converting { converter, .. } => { + Ok(converter.empty_rows(row_capacity, data_capacity)) + } + } } pub fn convert_rows<'a, I>(&self, rows: I) -> Result> where I: IntoIterator>, { - let converter = self.converter()?; - Ok(converter.convert_rows(rows)?) + match self { + Self::New { .. } => internal_err!( + "CardinalityAwareRowConverter has not converted any rows yet" + ), + Self::Converting { + converter, + output_types, + } => { + // Cast output type if needed + let output = converter + .convert_rows(rows)? + .into_iter() + .zip(output_types.iter()) + .map(|(arr, output_type)| { + if arr.data_type() != output_type { + Ok(arrow::compute::cast(&arr, output_type)?) + } else { + Ok(arr) + } + }) + .collect::>>()?; + + Ok(output) + } + } } pub fn append(&mut self, rows: &mut Rows, columns: &[ArrayRef]) -> Result<()> { @@ -109,8 +144,11 @@ impl CardinalityAwareRowConverter { /// Return a mutable reference to the inner converter, creating it if needed fn converter_mut(&mut self, columns: &[ArrayRef]) -> Result<&mut RowConverter> { if let Self::New { fields } = self { + // TODO clean up the code let mut updated_fields = fields.clone(); + let mut output_types = vec![]; for (i, col) in columns.iter().enumerate() { + output_types.push(col.data_type().clone()); if let DataType::Dictionary(_, _) = col.data_type() { // see comments on LOW_CARDINALITY_THRESHOLD for // the rationale of this calculation @@ -122,7 +160,8 @@ impl CardinalityAwareRowConverter { } } *self = Self::Converting { - inner: RowConverter::new(updated_fields)?, + converter: RowConverter::new(updated_fields)?, + output_types, }; }; @@ -130,17 +169,7 @@ impl CardinalityAwareRowConverter { Self::New { .. } => { unreachable!(); } - Self::Converting { inner } => Ok(inner), - } - } - - /// Return a reference to the inner converter, erroring if we have not yet converted a row. - fn converter(&self) -> Result<&RowConverter> { - match self { - Self::New { .. } => internal_err!( - "CardinalityAwareRowConverter has not converted any rows yet" - ), - Self::Converting { inner } => Ok(inner), + Self::Converting { converter, .. } => Ok(converter), } } } From f3eb44c3584659ecbafe3d2cb81340b01df8cb5b Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Tue, 12 Sep 2023 12:12:51 -0400 Subject: [PATCH 40/55] fix --- .../aggregates/group_values/row.rs | 20 ++++--------------- .../core/src/physical_plan/row_converter.rs | 4 ---- 2 files changed, 4 insertions(+), 20 deletions(-) diff --git a/datafusion/core/src/physical_plan/aggregates/group_values/row.rs b/datafusion/core/src/physical_plan/aggregates/group_values/row.rs index 00b7eba42a20..a4f7fa968765 100644 --- a/datafusion/core/src/physical_plan/aggregates/group_values/row.rs +++ b/datafusion/core/src/physical_plan/aggregates/group_values/row.rs @@ -60,9 +60,6 @@ pub struct GroupValuesRows { /// Random state for creating hashes random_state: RandomState, - - /// Scratch space for storing per-batch group rows - scratch_group_rows: Option, } impl GroupValuesRows { @@ -84,30 +81,22 @@ impl GroupValuesRows { group_values: None, hashes_buffer: Default::default(), random_state: Default::default(), - scratch_group_rows: None, }) } } impl GroupValues for GroupValuesRows { fn intern(&mut self, cols: &[ArrayRef], groups: &mut Vec) -> Result<()> { - // Convert the group keys into the row format, reusing rows when possible - let group_rows = match self.scratch_group_rows.take() { - Some(mut group_rows) => { - group_rows.clear(); - self.row_converter.append(&mut group_rows, cols)?; - group_rows - } - None => self.row_converter.convert_columns(cols)?, - }; + // Convert the group keys into the row format + // Avoid reallocation when https://github.com/apache/arrow-rs/issues/4479 is available + let group_rows = self.row_converter.convert_columns(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 = group_rows.num_rows(); - // tracks to which group each of the input rows belongs groups.clear(); @@ -147,7 +136,6 @@ impl GroupValues for GroupValuesRows { } self.group_values = Some(group_values); - self.scratch_group_rows = Some(group_rows); Ok(()) } diff --git a/datafusion/core/src/physical_plan/row_converter.rs b/datafusion/core/src/physical_plan/row_converter.rs index 5bcdc89ff287..24629559603c 100644 --- a/datafusion/core/src/physical_plan/row_converter.rs +++ b/datafusion/core/src/physical_plan/row_converter.rs @@ -131,10 +131,6 @@ impl CardinalityAwareRowConverter { } } - pub fn append(&mut self, rows: &mut Rows, columns: &[ArrayRef]) -> Result<()> { - Ok(self.converter_mut(columns)?.append(rows, columns)?) - } - /// Calls [`RowConverter::convert_columns`] after first /// initializing the converter based on cardinalities pub fn convert_columns(&mut self, columns: &[ArrayRef]) -> Result { From 664e6a014147be0466f47488ca6bb53b0102eb8a Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Tue, 12 Sep 2023 12:19:38 -0400 Subject: [PATCH 41/55] fix docs --- .../core/src/physical_plan/row_converter.rs | 17 ++++++++++------- 1 file changed, 10 insertions(+), 7 deletions(-) diff --git a/datafusion/core/src/physical_plan/row_converter.rs b/datafusion/core/src/physical_plan/row_converter.rs index 24629559603c..6a1d46924e9c 100644 --- a/datafusion/core/src/physical_plan/row_converter.rs +++ b/datafusion/core/src/physical_plan/row_converter.rs @@ -16,7 +16,7 @@ // under the License. //! [`CardinalityAwareRowConverter`] for converting data to -//! [`arrow_row`] format. +//! [`arrow::row`] format. use arrow::datatypes::DataType; use arrow::row::{Row, RowConverter}; @@ -44,7 +44,7 @@ use datafusion_common::{internal_err, DataFusionError, Result}; /// the time of this writing. const LOW_CARDINALITY_THRESHOLD: usize = 10; -/// Wrapper around an [`arrow_row`] [`RowConverter` that disables +/// Wrapper around an [`arrow::row`] [`RowConverter`] that disables /// dictionary preservation for high cardinality columns, based on the /// observed cardinalities in the first columns converted. /// @@ -63,10 +63,14 @@ const LOW_CARDINALITY_THRESHOLD: usize = 10; #[derive(Debug)] pub enum CardinalityAwareRowConverter { /// Converter is newly initialized, and hasn't yet seen data - New { fields: Vec }, + New { + /// Defines the Row conversion + fields: Vec, + }, /// Converter has seen data and can convert [`Array`]s to/from /// [`Rows`] Converting { + /// Underlying converter converter: RowConverter, /// if preserve_dictionaries is disabled, the output type can be /// different than input. @@ -79,12 +83,11 @@ impl CardinalityAwareRowConverter { Ok(Self::New { fields }) } + /// Returns the memory size of the underlying [`RowConverter`] if + /// any. pub fn size(&self) -> usize { match self { - Self::New { .. } => { - // TODO account for size of `fields` - 0 - } + Self::New { .. } => 0, Self::Converting { converter, .. } => converter.size(), } } From bbc59827ae691fb8221573008aba6bc6b508f023 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Tue, 12 Sep 2023 12:42:50 -0400 Subject: [PATCH 42/55] fix bug --- .../core/src/physical_plan/aggregates/group_values/row.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/core/src/physical_plan/aggregates/group_values/row.rs b/datafusion/core/src/physical_plan/aggregates/group_values/row.rs index a4f7fa968765..b085b3c0a994 100644 --- a/datafusion/core/src/physical_plan/aggregates/group_values/row.rs +++ b/datafusion/core/src/physical_plan/aggregates/group_values/row.rs @@ -174,7 +174,7 @@ impl GroupValues for GroupValuesRows { output } EmitTo::First(n) => { - let groups_rows = self.group_values.as_ref().unwrap().iter().take(n); + 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 efficent? From a1f69a70910601f4c5fb4295ebfb91cedb21efed Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Tue, 12 Sep 2023 14:25:50 -0400 Subject: [PATCH 43/55] Improve tests --- .../core/src/physical_plan/row_converter.rs | 175 +++++++++--------- 1 file changed, 84 insertions(+), 91 deletions(-) diff --git a/datafusion/core/src/physical_plan/row_converter.rs b/datafusion/core/src/physical_plan/row_converter.rs index 6a1d46924e9c..c33b302dcafc 100644 --- a/datafusion/core/src/physical_plan/row_converter.rs +++ b/datafusion/core/src/physical_plan/row_converter.rs @@ -181,7 +181,6 @@ mod tests { use uuid::Uuid; use arrow::datatypes::DataType; - use arrow::error::ArrowError; use arrow_array::{ types::Int32Type, ArrayRef, DictionaryArray, Int32Array, RecordBatch, StringArray, }; @@ -189,111 +188,105 @@ mod tests { use super::*; - // Generate a record batch with a high cardinality dictionary field - fn generate_batch_with_cardinality(card: String) -> Result { - let col_a: ArrayRef = if card == "high" { - // building column `a_dict` - let mut values_vector: Vec = Vec::new(); - for _i in 1..=20 { - values_vector.push(Uuid::new_v4().to_string()); - } - let values = StringArray::from(values_vector); + /// Generate a record batch with two columns: + /// + /// `a_dict`: String Dictionary + /// `b_prim`: Int32Array with random values 0..20 + enum Generator { + /// "High" cardinality (20 distinct values) + High, + /// "Low" cardinality (2 distinct values) + Low, + } - let mut keys_vector: Vec = Vec::new(); - for _i in 1..=20 { - keys_vector.push(rand::thread_rng().gen_range(0..20)); - } - let keys = Int32Array::from(keys_vector); - Arc::new( - DictionaryArray::::try_new(keys, Arc::new(values)).unwrap(), - ) - } else { - let values_vector = vec!["a", "b", "c"]; - let values = StringArray::from(values_vector); + impl Generator { + fn build(&self) -> RecordBatch { + let (keys, values) = match self { + Self::High => { + let values: Vec<_> = + (0..20).map(|_| Uuid::new_v4().to_string()).collect(); + let values = StringArray::from(values); - let mut keys_vector: Vec = Vec::new(); - for _i in 1..=20 { - keys_vector.push(rand::thread_rng().gen_range(0..2)); - } - let keys = Int32Array::from(keys_vector); - Arc::new( - DictionaryArray::::try_new(keys, Arc::new(values)).unwrap(), - ) - }; + let keys: Int32Array = (0..20) + .map(|_| rand::thread_rng().gen_range(0..20)) + .collect(); + (keys, values) + } + Self::Low => { + let values = StringArray::from_iter_values(["a", "b", "c"]); + let keys: Int32Array = (0..20) + .map(|_| rand::thread_rng().gen_range(0..2)) + .collect(); + (keys, values) + } + }; + let dict = + DictionaryArray::::try_new(keys, Arc::new(values)).unwrap(); + let col_a = Arc::new(dict) as ArrayRef; + + // building column `b_prim` + let values: Int32Array = (0..20) + .map(|_| rand::thread_rng().gen_range(0..20)) + .collect(); + let col_b: ArrayRef = Arc::new(values); - // building column `b_prim` - let mut values: Vec = Vec::new(); - for _i in 1..=20 { - values.push(rand::thread_rng().gen_range(0..20)); + RecordBatch::try_from_iter(vec![("a_dict", col_a), ("b_prim", col_b)]) + .unwrap() } - let col_b: ArrayRef = Arc::new(Int32Array::from(values)); - // building the record batch - RecordBatch::try_from_iter(vec![("a_dict", col_a), ("b_prim", col_b)]) + fn sort_fields(&self) -> Vec { + vec![ + SortField::new_with_options( + DataType::Dictionary( + Box::new(DataType::Int32), + Box::new(DataType::Utf8), + ), + SortOptions::default(), + ), + SortField::new_with_options(DataType::Int32, SortOptions::default()), + ] + } } #[tokio::test] async fn test_with_high_card() { - let batch = generate_batch_with_cardinality(String::from("high")).unwrap(); - let sort_fields = vec![ - arrow::row::SortField::new_with_options( - DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)), - SortOptions::default(), - ), - arrow::row::SortField::new_with_options( - DataType::Int32, - SortOptions::default(), - ), - ]; + let generator = Generator::High; + let batch = generator.build(); + + let mut card_converter = + CardinalityAwareRowConverter::new(generator.sort_fields()).unwrap(); + let rows = card_converter.convert_columns(batch.columns()).unwrap(); + let converted_batch = card_converter.convert_rows(&rows).unwrap(); + assert_eq!(converted_batch, batch.columns()); - // With the `CardinalityAwareRowConverter`, when the high cardinality dictionary-encoded sort field is - // converted to the `Row` format, the dictionary encoding is not preserved and we switch to Utf8 encoding. - let mut converter = - CardinalityAwareRowConverter::new(sort_fields.clone()).unwrap(); - let rows = converter.convert_columns(batch.columns()).unwrap(); - let converted_batch = converter.convert_rows(&rows).unwrap(); - assert_eq!(converted_batch[0].data_type(), &DataType::Utf8); + let mut row_converter = RowConverter::new(generator.sort_fields()).unwrap(); + let rows = row_converter.convert_columns(batch.columns()).unwrap(); + let converted_batch = row_converter.convert_rows(&rows).unwrap(); + assert_eq!(converted_batch, batch.columns()); - let mut converter = RowConverter::new(sort_fields.clone()).unwrap(); - let rows = converter.convert_columns(batch.columns()).unwrap(); - let converted_batch: Vec> = converter.convert_rows(&rows).unwrap(); - // With the `RowConverter`, the dictionary encoding is preserved. - assert_eq!( - converted_batch[0].data_type(), - &DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)) - ); + // with high cardinality the cardinality aware converter + // should be lower, as there is no interning of values + assert!(card_converter.size() < row_converter.size()); } #[tokio::test] async fn test_with_low_card() { - let batch = generate_batch_with_cardinality(String::from("low")).unwrap(); - let sort_fields = vec![ - arrow::row::SortField::new_with_options( - DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)), - SortOptions::default(), - ), - arrow::row::SortField::new_with_options( - DataType::Int32, - SortOptions::default(), - ), - ]; - // With low cardinality dictionary-encoded sort fields, both `CardinalityAwareRowConverter` and `RowConverter` - // preserves the dictionary encoding. - let mut converter = - CardinalityAwareRowConverter::new(sort_fields.clone()).unwrap(); - let rows = converter.convert_columns(batch.columns()).unwrap(); - let converted_batch = converter.convert_rows(&rows).unwrap(); - assert_eq!( - converted_batch[0].data_type(), - &DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)) - ); + let generator = Generator::Low; + let batch = generator.build(); + + let mut card_converter = + CardinalityAwareRowConverter::new(generator.sort_fields()).unwrap(); + let rows = card_converter.convert_columns(batch.columns()).unwrap(); + let converted_batch = card_converter.convert_rows(&rows).unwrap(); + assert_eq!(converted_batch, batch.columns()); + + let mut row_converter = RowConverter::new(generator.sort_fields()).unwrap(); + let rows = row_converter.convert_columns(batch.columns()).unwrap(); + let converted_batch = row_converter.convert_rows(&rows).unwrap(); + assert_eq!(converted_batch, batch.columns()); - let mut converter = RowConverter::new(sort_fields.clone()).unwrap(); - let rows = converter.convert_columns(batch.columns()).unwrap(); - let converted_batch = converter.convert_rows(&rows).unwrap(); - assert_eq!( - converted_batch[0].data_type(), - &DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)) - ); + // with low cardinality the row converters sizes should be + // equal as the same converter logic is used + assert_eq!(card_converter.size(), row_converter.size()); } } From 76feb4fef1aa585403e69623a86e9a56ceb12f5b Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Tue, 12 Sep 2023 14:28:59 -0400 Subject: [PATCH 44/55] simplify --- .../src/physical_plan/aggregates/group_values/row.rs | 11 ++++------- 1 file changed, 4 insertions(+), 7 deletions(-) diff --git a/datafusion/core/src/physical_plan/aggregates/group_values/row.rs b/datafusion/core/src/physical_plan/aggregates/group_values/row.rs index b085b3c0a994..2ea1191145d4 100644 --- a/datafusion/core/src/physical_plan/aggregates/group_values/row.rs +++ b/datafusion/core/src/physical_plan/aggregates/group_values/row.rs @@ -159,13 +159,10 @@ impl GroupValues for GroupValuesRows { } fn emit(&mut self, emit_to: EmitTo) -> Result> { - let mut group_values = match self.group_values.take() { - Some(group_values) => group_values, - None => { - // not clear it is actually unreachable (may have to make empty arrays) - unreachable!() - } - }; + let mut group_values = self + .group_values + .take() + .expect("Can not emit from empty rows"); let output = match emit_to { EmitTo::All => { From 137d78ef94c1ecb50c8c0571b240435286c95438 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Tue, 12 Sep 2023 14:32:18 -0400 Subject: [PATCH 45/55] Use cardinality aware row converter in gby order --- .../core/src/physical_plan/aggregates/order/partial.rs | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/datafusion/core/src/physical_plan/aggregates/order/partial.rs b/datafusion/core/src/physical_plan/aggregates/order/partial.rs index 019e61ef2688..9390debc0f98 100644 --- a/datafusion/core/src/physical_plan/aggregates/order/partial.rs +++ b/datafusion/core/src/physical_plan/aggregates/order/partial.rs @@ -15,8 +15,10 @@ // specific language governing permissions and limitations // under the License. -use crate::physical_expr::EmitTo; -use arrow::row::{OwnedRow, RowConverter, Rows, SortField}; +use crate::{ + physical_expr::EmitTo, physical_plan::row_converter::CardinalityAwareRowConverter, +}; +use arrow::row::{OwnedRow, Rows, SortField}; use arrow_array::ArrayRef; use arrow_schema::Schema; use datafusion_common::Result; @@ -70,7 +72,7 @@ pub(crate) struct GroupOrderingPartial { /// Converter for the sort key (used on the group columns /// specified in `order_indexes`) - row_converter: RowConverter, + row_converter: CardinalityAwareRowConverter, } #[derive(Debug, Default)] @@ -124,7 +126,7 @@ impl GroupOrderingPartial { Ok(Self { state: State::Start, order_indices: order_indices.to_vec(), - row_converter: RowConverter::new(fields)?, + row_converter: CardinalityAwareRowConverter::new(fields)?, }) } From 65d31cca6c8d67a8ce0024d594be75add48f9a70 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Tue, 12 Sep 2023 14:32:57 -0400 Subject: [PATCH 46/55] clippy --- datafusion/core/src/physical_plan/aggregates/order/partial.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/core/src/physical_plan/aggregates/order/partial.rs b/datafusion/core/src/physical_plan/aggregates/order/partial.rs index 9390debc0f98..c4e7ba867fe3 100644 --- a/datafusion/core/src/physical_plan/aggregates/order/partial.rs +++ b/datafusion/core/src/physical_plan/aggregates/order/partial.rs @@ -143,7 +143,7 @@ impl GroupOrderingPartial { .map(|&idx| group_values[idx].clone()) .collect(); - Ok(self.row_converter.convert_columns(&sort_values)?) + self.row_converter.convert_columns(&sort_values) } /// How many groups be emitted, or None if no data can be emitted From 9b5681d69f97572837ad57fe6688019ce535994c Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Tue, 12 Sep 2023 15:46:08 -0400 Subject: [PATCH 47/55] Adjust memory test --- datafusion/core/tests/memory_limit.rs | 82 ++++++++++++++++++++------- 1 file changed, 61 insertions(+), 21 deletions(-) diff --git a/datafusion/core/tests/memory_limit.rs b/datafusion/core/tests/memory_limit.rs index 12cde8660038..4297aa0e1f47 100644 --- a/datafusion/core/tests/memory_limit.rs +++ b/datafusion/core/tests/memory_limit.rs @@ -238,7 +238,7 @@ async fn sort_preserving_merge() { #[tokio::test] async fn sort_spill_reservation() { - let partition_size = batches_byte_size(&dict_batches()); + let partition_size = batches_byte_size(&maybe_split_batches(dict_batches(), true)); let base_config = SessionConfig::new() // do not allow the sort to use the 'concat in place' path @@ -248,30 +248,32 @@ async fn sort_spill_reservation() { // purposely sorting data that requires non trivial memory to // sort/merge. let test = TestCase::new() - // This query uses a different order than the input table to - // force a sort. It also needs to have multiple columns to - // force RowFormat / interner that makes merge require - // substantial memory + // This query uses a different order than the input table to + // force a sort. It also needs to have multiple columns to + // force RowFormat / interner that makes merge require + // substantial memory .with_query("select * from t ORDER BY a , b DESC") - // enough memory to sort if we don't try to merge it all at once + // enough memory to sort if we don't try to merge it all at once .with_memory_limit(partition_size) - // use a single partiton so only a sort is needed + // use a single partiton so only a sort is needed .with_scenario(Scenario::DictionaryStrings(1)) .with_disk_manager_config(DiskManagerConfig::NewOs) + // make the batches small enough to avoid triggering CardinalityAwareRowConverter + .with_single_row_batches(true) .with_expected_plan( // It is important that this plan only has a SortExec, not // also merge, so we can ensure the sort could finish // given enough merging memory &[ - "+---------------+--------------------------------------------------------------------------------------------------------+", - "| plan_type | plan |", - "+---------------+--------------------------------------------------------------------------------------------------------+", - "| logical_plan | Sort: t.a ASC NULLS LAST, t.b DESC NULLS FIRST |", - "| | TableScan: t projection=[a, b] |", - "| physical_plan | SortExec: expr=[a@0 ASC NULLS LAST,b@1 DESC] |", - "| | MemoryExec: partitions=1, partition_sizes=[5], output_ordering=a@0 ASC NULLS LAST,b@1 ASC NULLS LAST |", - "| | |", - "+---------------+--------------------------------------------------------------------------------------------------------+", + "+---------------+----------------------------------------------------------------------------------------------------------+", + "| plan_type | plan |", + "+---------------+----------------------------------------------------------------------------------------------------------+", + "| logical_plan | Sort: t.a ASC NULLS LAST, t.b DESC NULLS FIRST |", + "| | TableScan: t projection=[a, b] |", + "| physical_plan | SortExec: expr=[a@0 ASC NULLS LAST,b@1 DESC] |", + "| | MemoryExec: partitions=1, partition_sizes=[245], output_ordering=a@0 ASC NULLS LAST,b@1 ASC NULLS LAST |", + "| | |", + "+---------------+----------------------------------------------------------------------------------------------------------+", ] ); @@ -308,6 +310,8 @@ struct TestCase { memory_limit: usize, config: SessionConfig, scenario: Scenario, + /// If true, splits all input batches into 1 row each + single_row_batches: bool, /// How should the disk manager (that allows spilling) be /// configured? Defaults to `Disabled` disk_manager_config: DiskManagerConfig, @@ -325,6 +329,7 @@ impl TestCase { memory_limit: 0, config: SessionConfig::new(), scenario: Scenario::AccessLog, + single_row_batches: false, disk_manager_config: DiskManagerConfig::Disabled, expected_plan: vec![], expected_success: false, @@ -381,6 +386,12 @@ impl TestCase { self } + /// Should the input be split into 1 row batches? + fn with_single_row_batches(mut self, single_row_batches: bool) -> Self { + self.single_row_batches = single_row_batches; + self + } + /// Specify an expected plan to review pub fn with_expected_plan(mut self, expected_plan: &[&str]) -> Self { self.expected_plan = expected_plan.iter().map(|s| s.to_string()).collect(); @@ -396,11 +407,12 @@ impl TestCase { config, scenario, disk_manager_config, + single_row_batches, expected_plan, expected_success, } = self; - let table = scenario.table(); + let table = scenario.table(single_row_batches); let rt_config = RuntimeConfig::new() // do not allow spilling @@ -477,16 +489,18 @@ enum Scenario { impl Scenario { /// return a TableProvider with data for the test - fn table(&self) -> Arc { + fn table(&self, one_row_batches: bool) -> Arc { match self { Self::AccessLog => { let batches = access_log_batches(); + let batches = maybe_split_batches(batches, one_row_batches); let table = MemTable::try_new(batches[0].schema(), vec![batches]).unwrap(); Arc::new(table) } Self::AccessLogStreaming => { let batches = access_log_batches(); + let batches = maybe_split_batches(batches, one_row_batches); // Create a new streaming table with the generated schema and batches let table = StreamingTable::try_new( @@ -502,9 +516,12 @@ impl Scenario { } Self::DictionaryStrings(num_partitions) => { use datafusion::physical_expr::expressions::col; - let batches: Vec> = std::iter::repeat(dict_batches()) - .take(*num_partitions) - .collect(); + let batches: Vec> = std::iter::repeat(maybe_split_batches( + dict_batches(), + one_row_batches, + )) + .take(*num_partitions) + .collect(); let schema = batches[0][0].schema(); let options = SortOptions { @@ -559,6 +576,29 @@ fn access_log_batches() -> Vec { .collect() } +/// If `one_row_batches` is true, then returns new record batches that +/// are one row in size +fn maybe_split_batches( + batches: Vec, + one_row_batches: bool, +) -> Vec { + if !one_row_batches { + return batches; + } + + batches + .into_iter() + .flat_map(|mut batch| { + let mut batches = vec![]; + while batch.num_rows() > 1 { + batches.push(batch.slice(0, 1)); + batch = batch.slice(1, batch.num_rows() - 1); + } + batches + }) + .collect() +} + static DICT_BATCHES: OnceLock> = OnceLock::new(); /// Returns 5 sorted string dictionary batches each with 50 rows with From b673c09b0ad780cf9d5baf55a2653cb1bbd280b6 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Wed, 13 Sep 2023 11:32:29 -0400 Subject: [PATCH 48/55] Add doc comments about row converter --- datafusion/core/src/physical_plan/row_converter.rs | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/datafusion/core/src/physical_plan/row_converter.rs b/datafusion/core/src/physical_plan/row_converter.rs index c33b302dcafc..db16c69b37cc 100644 --- a/datafusion/core/src/physical_plan/row_converter.rs +++ b/datafusion/core/src/physical_plan/row_converter.rs @@ -115,7 +115,11 @@ impl CardinalityAwareRowConverter { converter, output_types, } => { - // Cast output type if needed + // Cast output type if needed. The input type and + // output type must match exactly (including + // encodings). + // https://github.com/apache/arrow-datafusion/discussions/7421 + // could reduce the need for this. let output = converter .convert_rows(rows)? .into_iter() From 25861a781e0ddb0f9fa22d744d9c2ed0b72aeb12 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Wed, 13 Sep 2023 11:34:07 -0400 Subject: [PATCH 49/55] remove outdated comment --- datafusion/core/src/physical_plan/row_converter.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/datafusion/core/src/physical_plan/row_converter.rs b/datafusion/core/src/physical_plan/row_converter.rs index db16c69b37cc..beb0c7775082 100644 --- a/datafusion/core/src/physical_plan/row_converter.rs +++ b/datafusion/core/src/physical_plan/row_converter.rs @@ -147,7 +147,6 @@ impl CardinalityAwareRowConverter { /// Return a mutable reference to the inner converter, creating it if needed fn converter_mut(&mut self, columns: &[ArrayRef]) -> Result<&mut RowConverter> { if let Self::New { fields } = self { - // TODO clean up the code let mut updated_fields = fields.clone(); let mut output_types = vec![]; for (i, col) in columns.iter().enumerate() { From 355ef7397ea99d8fe4c9e6ff80d3484d7dee3134 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Wed, 13 Sep 2023 11:50:26 -0400 Subject: [PATCH 50/55] Rework partition size calculation to make test clearer --- datafusion/core/tests/memory_limit.rs | 82 ++++++++++++++++++--------- 1 file changed, 56 insertions(+), 26 deletions(-) diff --git a/datafusion/core/tests/memory_limit.rs b/datafusion/core/tests/memory_limit.rs index 4297aa0e1f47..3367a923024f 100644 --- a/datafusion/core/tests/memory_limit.rs +++ b/datafusion/core/tests/memory_limit.rs @@ -200,7 +200,8 @@ async fn symmetric_hash_join() { #[tokio::test] async fn sort_preserving_merge() { - let partition_size = batches_byte_size(&dict_batches()); + let scenario = Scenario::new_dictionary_strings(2); + let partition_size = scenario.partition_size(); TestCase::new() // This query uses the exact same ordering as the input table @@ -213,7 +214,7 @@ async fn sort_preserving_merge() { // provide insufficient memory to merge .with_memory_limit(partition_size / 2) // two partitions of data, so a merge is required - .with_scenario(Scenario::DictionaryStrings(2)) + .with_scenario(scenario) .with_expected_plan( // It is important that this plan only has // SortPreservingMergeExec (not a Sort which would compete @@ -238,7 +239,10 @@ async fn sort_preserving_merge() { #[tokio::test] async fn sort_spill_reservation() { - let partition_size = batches_byte_size(&maybe_split_batches(dict_batches(), true)); + let scenario = Scenario::new_dictionary_strings(1) + // make the batches small enough to avoid triggering CardinalityAwareRowConverter + .with_single_row_batches(true); + let partition_size = scenario.partition_size(); let base_config = SessionConfig::new() // do not allow the sort to use the 'concat in place' path @@ -256,10 +260,8 @@ async fn sort_spill_reservation() { // enough memory to sort if we don't try to merge it all at once .with_memory_limit(partition_size) // use a single partiton so only a sort is needed - .with_scenario(Scenario::DictionaryStrings(1)) + .with_scenario(scenario) .with_disk_manager_config(DiskManagerConfig::NewOs) - // make the batches small enough to avoid triggering CardinalityAwareRowConverter - .with_single_row_batches(true) .with_expected_plan( // It is important that this plan only has a SortExec, not // also merge, so we can ensure the sort could finish @@ -310,8 +312,6 @@ struct TestCase { memory_limit: usize, config: SessionConfig, scenario: Scenario, - /// If true, splits all input batches into 1 row each - single_row_batches: bool, /// How should the disk manager (that allows spilling) be /// configured? Defaults to `Disabled` disk_manager_config: DiskManagerConfig, @@ -329,7 +329,6 @@ impl TestCase { memory_limit: 0, config: SessionConfig::new(), scenario: Scenario::AccessLog, - single_row_batches: false, disk_manager_config: DiskManagerConfig::Disabled, expected_plan: vec![], expected_success: false, @@ -386,12 +385,6 @@ impl TestCase { self } - /// Should the input be split into 1 row batches? - fn with_single_row_batches(mut self, single_row_batches: bool) -> Self { - self.single_row_batches = single_row_batches; - self - } - /// Specify an expected plan to review pub fn with_expected_plan(mut self, expected_plan: &[&str]) -> Self { self.expected_plan = expected_plan.iter().map(|s| s.to_string()).collect(); @@ -407,12 +400,11 @@ impl TestCase { config, scenario, disk_manager_config, - single_row_batches, expected_plan, expected_success, } = self; - let table = scenario.table(single_row_batches); + let table = scenario.table(); let rt_config = RuntimeConfig::new() // do not allow spilling @@ -483,24 +475,59 @@ enum Scenario { /// [`StreamingTable`] AccessLogStreaming, - /// N partitions of of sorted, dictionary encoded strings - DictionaryStrings(usize), + /// N partitions of of sorted, dictionary encoded strings. + DictionaryStrings { + partitions: usize, + /// If true, splits all input batches into 1 row each + single_row_batches: bool, + }, } impl Scenario { + /// Create a new DictionaryStrings scenario with the number of partitions + fn new_dictionary_strings(partitions: usize) -> Self { + Self::DictionaryStrings { + partitions, + single_row_batches: false, + } + } + + /// Should the input be split into 1 row batches? + fn with_single_row_batches(mut self, val: bool) -> Self { + if let Self::DictionaryStrings { + single_row_batches, .. + } = &mut self + { + *single_row_batches = val; + } else { + panic!("Scenario does not support single row batches"); + } + self + } + + /// return the size, in bytes, of each partition + fn partition_size(&self) -> usize { + if let Self::DictionaryStrings { + single_row_batches, .. + } = self + { + batches_byte_size(&maybe_split_batches(dict_batches(), *single_row_batches)) + } else { + panic!("Scenario does not support partition size"); + } + } + /// return a TableProvider with data for the test - fn table(&self, one_row_batches: bool) -> Arc { + fn table(&self) -> Arc { match self { Self::AccessLog => { let batches = access_log_batches(); - let batches = maybe_split_batches(batches, one_row_batches); let table = MemTable::try_new(batches[0].schema(), vec![batches]).unwrap(); Arc::new(table) } Self::AccessLogStreaming => { let batches = access_log_batches(); - let batches = maybe_split_batches(batches, one_row_batches); // Create a new streaming table with the generated schema and batches let table = StreamingTable::try_new( @@ -514,13 +541,16 @@ impl Scenario { .with_infinite_table(true); Arc::new(table) } - Self::DictionaryStrings(num_partitions) => { + Self::DictionaryStrings { + partitions, + single_row_batches, + } => { use datafusion::physical_expr::expressions::col; let batches: Vec> = std::iter::repeat(maybe_split_batches( dict_batches(), - one_row_batches, + *single_row_batches, )) - .take(*num_partitions) + .take(*partitions) .collect(); let schema = batches[0][0].schema(); @@ -561,7 +591,7 @@ impl Scenario { // first Some(vec![Arc::new(JoinSelection::new())]) } - Self::DictionaryStrings(_) => { + Self::DictionaryStrings { .. } => { // Use default rules None } From f48325a397cf456b36ddf6707db8566f893a5e85 Mon Sep 17 00:00:00 2001 From: Jayjeet Chakraborty Date: Fri, 15 Sep 2023 14:22:43 -0700 Subject: [PATCH 51/55] Increase threshold to 512 --- datafusion/core/src/physical_plan/row_converter.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/core/src/physical_plan/row_converter.rs b/datafusion/core/src/physical_plan/row_converter.rs index beb0c7775082..fdd6d18923a6 100644 --- a/datafusion/core/src/physical_plan/row_converter.rs +++ b/datafusion/core/src/physical_plan/row_converter.rs @@ -42,7 +42,7 @@ use datafusion_common::{internal_err, DataFusionError, Result}; /// this would likely only really make sense if the dictionary /// interner itself followed a similar approach, which it did not at /// the time of this writing. -const LOW_CARDINALITY_THRESHOLD: usize = 10; +const LOW_CARDINALITY_THRESHOLD: usize = 512; /// Wrapper around an [`arrow::row`] [`RowConverter`] that disables /// dictionary preservation for high cardinality columns, based on the From 22a90e70a25a2417718e50656f94ee525333e861 Mon Sep 17 00:00:00 2001 From: Jayjeet Chakraborty Date: Fri, 15 Sep 2023 14:44:18 -0700 Subject: [PATCH 52/55] Update row converter tests according to new threshold --- datafusion/core/src/physical_plan/row_converter.rs | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/datafusion/core/src/physical_plan/row_converter.rs b/datafusion/core/src/physical_plan/row_converter.rs index fdd6d18923a6..447cdaf4c1ed 100644 --- a/datafusion/core/src/physical_plan/row_converter.rs +++ b/datafusion/core/src/physical_plan/row_converter.rs @@ -207,17 +207,17 @@ mod tests { let (keys, values) = match self { Self::High => { let values: Vec<_> = - (0..20).map(|_| Uuid::new_v4().to_string()).collect(); + (0..1024).map(|_| Uuid::new_v4().to_string()).collect(); let values = StringArray::from(values); - let keys: Int32Array = (0..20) - .map(|_| rand::thread_rng().gen_range(0..20)) + let keys: Int32Array = (0..1024) + .map(|_| rand::thread_rng().gen_range(0..1024)) .collect(); (keys, values) } Self::Low => { let values = StringArray::from_iter_values(["a", "b", "c"]); - let keys: Int32Array = (0..20) + let keys: Int32Array = (0..1024) .map(|_| rand::thread_rng().gen_range(0..2)) .collect(); (keys, values) @@ -228,7 +228,7 @@ mod tests { let col_a = Arc::new(dict) as ArrayRef; // building column `b_prim` - let values: Int32Array = (0..20) + let values: Int32Array = (0..1024) .map(|_| rand::thread_rng().gen_range(0..20)) .collect(); let col_b: ArrayRef = Arc::new(values); From fdb4835abf4b188c3a2d6d0ca542d5c634174bcf Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Mon, 18 Sep 2023 09:30:53 -0400 Subject: [PATCH 53/55] fix clippy --- datafusion-cli/Cargo.lock | 29 ++++++++++--------- datafusion/physical-plan/Cargo.toml | 1 + .../src/aggregates/group_values/row.rs | 8 +++-- .../src/aggregates/order/partial.rs | 2 +- 4 files changed, 22 insertions(+), 18 deletions(-) diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index d7d1357a659d..5b613cc2b71d 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -347,7 +347,7 @@ checksum = "bc00ceb34980c03614e35a3a4e218276a0a824e911d07651cd0d858a51e8c0f0" dependencies = [ "proc-macro2", "quote", - "syn 2.0.33", + "syn 2.0.37", ] [[package]] @@ -1031,7 +1031,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1f34ba9a9bcb8645379e9de8cb3ecfcf4d1c85ba66d90deb3259206fa5aa193b" dependencies = [ "quote", - "syn 2.0.33", + "syn 2.0.37", ] [[package]] @@ -1248,6 +1248,7 @@ dependencies = [ "tempfile", "termtree", "tokio", + "uuid", ] [[package]] @@ -1535,7 +1536,7 @@ checksum = "89ca545a94061b6365f2c7355b4b32bd20df3ff95f02da9329b34ccc3bd6ee72" dependencies = [ "proc-macro2", "quote", - "syn 2.0.33", + "syn 2.0.37", ] [[package]] @@ -2421,7 +2422,7 @@ checksum = "4359fd9c9171ec6e8c62926d6faaf553a8dc3f64e1507e76da7911b4f6a04405" dependencies = [ "proc-macro2", "quote", - "syn 2.0.33", + "syn 2.0.37", ] [[package]] @@ -2748,7 +2749,7 @@ dependencies = [ "regex", "relative-path", "rustc_version", - "syn 2.0.33", + "syn 2.0.37", "unicode-ident", ] @@ -2956,7 +2957,7 @@ checksum = "4eca7ac642d82aa35b60049a6eccb4be6be75e599bd2e9adb5f875a737654af2" dependencies = [ "proc-macro2", "quote", - "syn 2.0.33", + "syn 2.0.37", ] [[package]] @@ -3145,7 +3146,7 @@ dependencies = [ "proc-macro2", "quote", "rustversion", - "syn 2.0.33", + "syn 2.0.37", ] [[package]] @@ -3167,9 +3168,9 @@ dependencies = [ [[package]] name = "syn" -version = "2.0.33" +version = "2.0.37" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9caece70c63bfba29ec2fed841a09851b14a235c60010fa4de58089b6c025668" +checksum = "7303ef2c05cd654186cb250d29049a24840ca25d2747c25c0381c8d9e2f582e8" dependencies = [ "proc-macro2", "quote", @@ -3227,7 +3228,7 @@ checksum = "49922ecae66cc8a249b77e68d1d0623c1b2c514f0060c27cdc68bd62a1219d35" dependencies = [ "proc-macro2", "quote", - "syn 2.0.33", + "syn 2.0.37", ] [[package]] @@ -3318,7 +3319,7 @@ checksum = "630bdcf245f78637c13ec01ffae6187cca34625e8c63150d424b59e55af2675e" dependencies = [ "proc-macro2", "quote", - "syn 2.0.33", + "syn 2.0.37", ] [[package]] @@ -3416,7 +3417,7 @@ checksum = "5f4f31f56159e98206da9efd823404b79b6ef3143b4a7ab76e67b1751b25a4ab" dependencies = [ "proc-macro2", "quote", - "syn 2.0.33", + "syn 2.0.37", ] [[package]] @@ -3588,7 +3589,7 @@ dependencies = [ "once_cell", "proc-macro2", "quote", - "syn 2.0.33", + "syn 2.0.37", "wasm-bindgen-shared", ] @@ -3622,7 +3623,7 @@ checksum = "54681b18a46765f095758388f2d0cf16eb8d4169b639ab575a8f5693af210c7b" dependencies = [ "proc-macro2", "quote", - "syn 2.0.33", + "syn 2.0.37", "wasm-bindgen-backend", "wasm-bindgen-shared", ] diff --git a/datafusion/physical-plan/Cargo.toml b/datafusion/physical-plan/Cargo.toml index 02f154762760..8549ad6fdd1f 100644 --- a/datafusion/physical-plan/Cargo.toml +++ b/datafusion/physical-plan/Cargo.toml @@ -60,3 +60,4 @@ tempfile = "3" #[dev-dependencies] termtree = "0.4.1" tokio = { version = "1.28", features = ["macros", "rt", "rt-multi-thread", "sync", "fs", "parking_lot"] } +uuid = { version = "^1.2", features = ["v4"] } diff --git a/datafusion/physical-plan/src/aggregates/group_values/row.rs b/datafusion/physical-plan/src/aggregates/group_values/row.rs index 5f3cd0e37fbf..690d7740a5c2 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/row.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/row.rs @@ -19,7 +19,7 @@ use crate::aggregates::group_values::GroupValues; use crate::row_converter::CardinalityAwareRowConverter; use ahash::RandomState; use arrow::record_batch::RecordBatch; -use arrow::row::{RowConverter, Rows, SortField}; +use arrow::row::{Rows, SortField}; use arrow_array::ArrayRef; use arrow_schema::SchemaRef; use datafusion_common::Result; @@ -204,8 +204,10 @@ impl GroupValues for GroupValuesRows { fn clear_shrink(&mut self, batch: &RecordBatch) { let count = batch.num_rows(); - // FIXME: there is no good way to clear_shrink for self.group_values - self.group_values = self.row_converter.empty_rows(count, 0); + self.group_values = self.group_values.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)>(); diff --git a/datafusion/physical-plan/src/aggregates/order/partial.rs b/datafusion/physical-plan/src/aggregates/order/partial.rs index a75162b086a9..bd8688820d93 100644 --- a/datafusion/physical-plan/src/aggregates/order/partial.rs +++ b/datafusion/physical-plan/src/aggregates/order/partial.rs @@ -16,7 +16,7 @@ // under the License. use crate::row_converter::CardinalityAwareRowConverter; -use arrow::row::{OwnedRow, RowConverter, Rows, SortField}; +use arrow::row::{OwnedRow, Rows, SortField}; use arrow_array::ArrayRef; use arrow_schema::Schema; use datafusion_common::Result; From efca2b2e1cc5368131f0e95f76f9136c04dab29f Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Mon, 18 Sep 2023 10:28:19 -0400 Subject: [PATCH 54/55] fix panic --- datafusion/physical-plan/src/aggregates/group_values/row.rs | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/datafusion/physical-plan/src/aggregates/group_values/row.rs b/datafusion/physical-plan/src/aggregates/group_values/row.rs index 690d7740a5c2..346f15fd1fe9 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/row.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/row.rs @@ -142,8 +142,9 @@ impl GroupValues for GroupValuesRows { } fn size(&self) -> usize { + let group_values_size = self.group_values.as_ref().map(|v| v.size()).unwrap_or(0); self.row_converter.size() - + self.group_values.as_ref().unwrap().size() + + group_values_size + self.map_size + self.hashes_buffer.allocated_size() } From 8498d043dfacbb8d4f78a5641abac1e99d5031c1 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Mon, 18 Sep 2023 11:15:52 -0400 Subject: [PATCH 55/55] Adjust constant for test --- datafusion/physical-plan/src/aggregates/mod.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 2916786a8072..7c7a593c48fc 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -1664,7 +1664,7 @@ mod tests { ))]; let task_ctx = if spill { - new_spill_ctx(2, 2144) + new_spill_ctx(2, 2500) } else { Arc::new(TaskContext::default()) };