diff --git a/Cargo.lock b/Cargo.lock index d69c13da4644..e01856785c63 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2706,6 +2706,7 @@ dependencies = [ "serde", "serde_json", "simdutf8", + "strength_reduce", "terminal_size", "tonic 0.10.2", "typetag", diff --git a/src/common/base/src/runtime/runtime.rs b/src/common/base/src/runtime/runtime.rs index ad349391a5d7..7d7d42cba65d 100644 --- a/src/common/base/src/runtime/runtime.rs +++ b/src/common/base/src/runtime/runtime.rs @@ -159,6 +159,7 @@ impl Runtime { runtime_builder.thread_stack_size(20 * 1024 * 1024); } + runtime_builder.thread_stack_size(100 * 1024 * 1024); Self::create(None, mem_stat, &mut runtime_builder) } @@ -185,6 +186,7 @@ impl Runtime { runtime_builder.thread_stack_size(20 * 1024 * 1024); } + runtime_builder.thread_stack_size(100 * 1024 * 1024); if let Some(thread_name) = &thread_name { runtime_builder.thread_name(thread_name); diff --git a/src/common/hashtable/src/lib.rs b/src/common/hashtable/src/lib.rs index 9800028411e1..135601ef1e8b 100644 --- a/src/common/hashtable/src/lib.rs +++ b/src/common/hashtable/src/lib.rs @@ -113,5 +113,6 @@ pub use partitioned_hashtable::hash2bucket; pub type HashJoinHashMap = hashjoin_hashtable::HashJoinHashTable; pub type BinaryHashJoinHashMap = hashjoin_string_hashtable::HashJoinStringHashTable; pub use traits::HashJoinHashtableLike; +pub use utils::fast_memcmp; pub use utils::Interval; pub use utils::MergeIntoBlockInfoIndex; diff --git a/src/common/hashtable/src/utils.rs b/src/common/hashtable/src/utils.rs index 92cd091d1612..beeabc80e26a 100644 --- a/src/common/hashtable/src/utils.rs +++ b/src/common/hashtable/src/utils.rs @@ -97,6 +97,18 @@ pub unsafe fn read_le(data: *const u8, len: usize) -> u64 { } } +#[cfg(all(target_arch = "x86_64", target_feature = "sse4.2"))] +#[inline] +pub fn fast_memcmp(a: &[u8], b: &[u8]) -> bool { + unsafe { sse::memcmp_sse(a, b) } +} + +#[cfg(not(all(any(target_arch = "x86_64"), target_feature = "sse4.2")))] +#[inline] +pub fn fast_memcmp(a: &[u8], b: &[u8]) -> bool { + a == b +} + #[cfg(all(target_arch = "x86_64", target_feature = "sse4.2"))] pub mod sse { use std::arch::x86_64::*; @@ -137,6 +149,8 @@ pub mod sse { )) } + /// # Safety + /// This is safe that we compare bytes via addr #[inline(always)] pub unsafe fn memcmp_sse(a: &[u8], b: &[u8]) -> bool { let mut size = a.len(); diff --git a/src/common/metrics/src/metrics/transform.rs b/src/common/metrics/src/metrics/transform.rs index 6e402afdf418..9b069786b957 100644 --- a/src/common/metrics/src/metrics/transform.rs +++ b/src/common/metrics/src/metrics/transform.rs @@ -22,10 +22,16 @@ use crate::Family; use crate::Histogram; use crate::VecLabels; +pub static AGGREGATE_PARTIAL_CELL_COUNT: LazyLock = + LazyLock::new(|| register_counter("transform_aggregate_partial_cell_count")); + pub static AGGREGATE_PARTIAL_SPILL_CELL_COUNT: LazyLock = LazyLock::new(|| register_counter("transform_aggregate_partial_spill_cell_count")); pub static AGGREGATE_PARTIAL_HASHTABLE_ALLOCATED_BYTES: LazyLock = LazyLock::new(|| register_counter("transform_aggregate_partial_hashtable_allocated_bytes")); +pub static AGGREGATE_PARTIAL_HASHTABLE_EXCHANGE_ROWS: LazyLock = + LazyLock::new(|| register_counter("transform_aggregate_partial_hashtable_exchange_rows")); + pub static SPILL_COUNT: LazyLock> = LazyLock::new(|| register_counter_family("transform_spill_count")); pub static SPILL_WRITE_COUNT: LazyLock> = @@ -70,10 +76,27 @@ pub fn metrics_inc_aggregate_partial_spill_cell_count(c: u64) { AGGREGATE_PARTIAL_SPILL_CELL_COUNT.inc_by(c); } +pub fn metrics_inc_aggregate_partial_hashtable_exchange_rows(c: u64) { + AGGREGATE_PARTIAL_HASHTABLE_EXCHANGE_ROWS.inc_by(c); +} + pub fn metrics_inc_aggregate_partial_hashtable_allocated_bytes(c: u64) { AGGREGATE_PARTIAL_HASHTABLE_ALLOCATED_BYTES.inc_by(c); } +pub fn metrics_inc_group_by_partial_spill_count() { + let labels = &vec![("spill", "group_by_partial_spill".to_string())]; + SPILL_COUNT.get_or_create(labels).inc(); +} + +pub fn metrics_inc_group_by_partial_spill_cell_count(c: u64) { + AGGREGATE_PARTIAL_SPILL_CELL_COUNT.inc_by(c); +} + +pub fn metrics_inc_group_by_partial_hashtable_allocated_bytes(c: u64) { + AGGREGATE_PARTIAL_HASHTABLE_ALLOCATED_BYTES.inc_by(c); +} + pub fn metrics_inc_group_by_spill_write_count() { let labels = &vec![("spill", "group_by_spill".to_string())]; SPILL_WRITE_COUNT.get_or_create(labels).inc(); diff --git a/src/meta/app/src/principal/user_privilege.rs b/src/meta/app/src/principal/user_privilege.rs index be40c9d92433..fcfe4af1d60e 100644 --- a/src/meta/app/src/principal/user_privilege.rs +++ b/src/meta/app/src/principal/user_privilege.rs @@ -19,6 +19,7 @@ use enumflags2::bitflags; use enumflags2::make_bitflags; use enumflags2::BitFlags; +// Note: If add new privilege type, need add forward test #[bitflags] #[repr(u64)] #[derive( diff --git a/src/meta/proto-conv/src/share_from_to_protobuf_impl.rs b/src/meta/proto-conv/src/share_from_to_protobuf_impl.rs index d364723d2074..ebfbfbab79dc 100644 --- a/src/meta/proto-conv/src/share_from_to_protobuf_impl.rs +++ b/src/meta/proto-conv/src/share_from_to_protobuf_impl.rs @@ -129,23 +129,19 @@ impl FromToProto for mt::ShareGrantEntry { where Self: Sized { reader_check_msg(p.ver, p.min_reader_ver)?; - let privileges = BitFlags::::from_bits(p.privileges); - match privileges { - Ok(privileges) => Ok(mt::ShareGrantEntry { - object: mt::ShareGrantObject::from_pb(p.object.ok_or_else(|| Incompatible { - reason: "ShareGrantEntry.object can not be None".to_string(), - })?)?, - privileges, - grant_on: DateTime::::from_pb(p.grant_on)?, - update_on: match p.update_on { - Some(t) => Some(DateTime::::from_pb(t)?), - None => None, - }, - }), - Err(e) => Err(Incompatible { - reason: format!("UserPrivilegeType error: {}", e), - }), - } + let privileges = + BitFlags::::from_bits_truncate(p.privileges); + Ok(mt::ShareGrantEntry { + object: mt::ShareGrantObject::from_pb(p.object.ok_or_else(|| Incompatible { + reason: "ShareGrantEntry.object can not be None".to_string(), + })?)?, + privileges, + grant_on: DateTime::::from_pb(p.grant_on)?, + update_on: match p.update_on { + Some(t) => Some(DateTime::::from_pb(t)?), + None => None, + }, + }) } fn to_pb(&self) -> Result { diff --git a/src/meta/proto-conv/src/user_from_to_protobuf_impl.rs b/src/meta/proto-conv/src/user_from_to_protobuf_impl.rs index 859a2eac6fb7..82b5ca76b4d4 100644 --- a/src/meta/proto-conv/src/user_from_to_protobuf_impl.rs +++ b/src/meta/proto-conv/src/user_from_to_protobuf_impl.rs @@ -241,18 +241,14 @@ impl FromToProto for mt::principal::GrantEntry { where Self: Sized { reader_check_msg(p.ver, p.min_reader_ver)?; - let privileges = BitFlags::::from_bits(p.privileges); - match privileges { - Ok(privileges) => Ok(mt::principal::GrantEntry::new( - mt::principal::GrantObject::from_pb(p.object.ok_or_else(|| Incompatible { - reason: "GrantEntry.object can not be None".to_string(), - })?)?, - privileges, - )), - Err(e) => Err(Incompatible { - reason: format!("UserPrivilegeType error: {}", e), - }), - } + let privileges = + BitFlags::::from_bits_truncate(p.privileges); + Ok(mt::principal::GrantEntry::new( + mt::principal::GrantObject::from_pb(p.object.ok_or_else(|| Incompatible { + reason: "GrantEntry.object can not be None".to_string(), + })?)?, + privileges, + )) } fn to_pb(&self) -> Result { diff --git a/src/meta/proto-conv/src/util.rs b/src/meta/proto-conv/src/util.rs index f60c5da2c80f..ba73152864ca 100644 --- a/src/meta/proto-conv/src/util.rs +++ b/src/meta/proto-conv/src/util.rs @@ -107,6 +107,7 @@ const META_CHANGE_LOG: &[(u64, &str)] = &[ (75, "2024-01-15: ADD: user.proto/CsvFileFormatParams add field `binary_format` and `output_header`", ), (76, "2024-01-18: ADD: ownership.proto and role.proto", ), (77, "2024-01-22: Remove: allow_anonymous in S3 Config", ), + (78, "2024-01-29: Refactor: GrantEntry::UserPrivilegeType and ShareGrantEntry::ShareGrantObjectPrivilege use from_bits_truncate deserialize", ), // Dear developer: // If you're gonna add a new metadata version, you'll have to add a test for it. // You could just copy an existing test file(e.g., `../tests/it/v024_table_meta.rs`) diff --git a/src/meta/proto-conv/tests/it/main.rs b/src/meta/proto-conv/tests/it/main.rs index 3b0c0285dc16..0b62ec1ba756 100644 --- a/src/meta/proto-conv/tests/it/main.rs +++ b/src/meta/proto-conv/tests/it/main.rs @@ -81,3 +81,4 @@ mod v074_table_db_meta; mod v075_csv_format_params; mod v076_role_ownership_info; mod v077_s3_remove_allow_anonymous; +mod v078_grantentry; diff --git a/src/meta/proto-conv/tests/it/v078_grantentry.rs b/src/meta/proto-conv/tests/it/v078_grantentry.rs new file mode 100644 index 000000000000..b23a1262e6d6 --- /dev/null +++ b/src/meta/proto-conv/tests/it/v078_grantentry.rs @@ -0,0 +1,72 @@ +// Copyright 2023 Datafuse Labs. +// +// Licensed 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 chrono::TimeZone; +use chrono::Utc; +use databend_common_meta_app as mt; +use databend_common_meta_app::principal::UserPrivilegeType; +use databend_common_meta_app::share::ShareGrantObjectPrivilege; +use enumflags2::make_bitflags; +use minitrace::func_name; + +use crate::common; + +// These bytes are built when a new version in introduced, +// and are kept for backward compatibility test. +// +// ************************************************************* +// * These messages should never be updated, * +// * only be added when a new version is added, * +// * or be removed when an old version is no longer supported. * +// ************************************************************* +// + +#[test] +fn test_decode_v78_grant_entry() -> anyhow::Result<()> { + let grant_entry_v78 = vec![ + 10, 8, 10, 0, 160, 6, 78, 168, 6, 24, 16, 254, 255, 55, 160, 6, 78, 168, 6, 24, + ]; + + let want = || { + mt::principal::GrantEntry::new( + mt::principal::GrantObject::Global, + make_bitflags!(UserPrivilegeType::{Create | Select | Insert | Update | Delete | Drop | Alter | Super | CreateUser | DropUser | CreateRole | DropRole | Grant | CreateStage | Set | CreateDataMask | Read | Write }), + ) + }; + + common::test_pb_from_to(func_name!(), want())?; + common::test_load_old(func_name!(), grant_entry_v78.as_slice(), 78, want())?; + + Ok(()) +} + +#[test] +fn test_decode_v78_share_grant_entry() -> anyhow::Result<()> { + let share_grant_entry_v78 = vec![ + 10, 8, 16, 19, 160, 6, 78, 168, 6, 24, 16, 4, 26, 23, 50, 48, 49, 52, 45, 49, 49, 45, 50, + 56, 32, 49, 50, 58, 48, 48, 58, 48, 57, 32, 85, 84, 67, 160, 6, 78, 168, 6, 24, + ]; + + let want = || mt::share::ShareGrantEntry { + object: mt::share::ShareGrantObject::Table(19), + privileges: make_bitflags!(ShareGrantObjectPrivilege::{Select}), + grant_on: Utc.with_ymd_and_hms(2014, 11, 28, 12, 0, 9).unwrap(), + update_on: None, + }; + + common::test_pb_from_to(func_name!(), want())?; + common::test_load_old(func_name!(), share_grant_entry_v78.as_slice(), 78, want())?; + + Ok(()) +} diff --git a/src/query/expression/Cargo.toml b/src/query/expression/Cargo.toml index 0c81ce624614..ef94506eb0b1 100644 --- a/src/query/expression/Cargo.toml +++ b/src/query/expression/Cargo.toml @@ -51,6 +51,7 @@ rust_decimal = "1.26" serde = { workspace = true } serde_json = { workspace = true } simdutf8 = "0.1.4" +strength_reduce = "0.2.4" terminal_size = "0.2.6" tonic = { workspace = true } typetag = { workspace = true } diff --git a/src/query/expression/src/aggregate/aggregate_hashtable.rs b/src/query/expression/src/aggregate/aggregate_hashtable.rs index f005fb722624..3f118cd8ac09 100644 --- a/src/query/expression/src/aggregate/aggregate_hashtable.rs +++ b/src/query/expression/src/aggregate/aggregate_hashtable.rs @@ -14,104 +14,228 @@ // A new AggregateHashtable which inspired by duckdb's https://duckdb.org/2022/03/07/aggregate-hashtable.html +use std::sync::atomic::Ordering; use std::sync::Arc; +use bumpalo::Bump; use databend_common_exception::Result; -use super::payload::Payload; +use super::partitioned_payload::PartitionedPayload; use super::payload_flush::PayloadFlushState; use super::probe_state::ProbeState; use crate::aggregate::payload_row::row_match_columns; use crate::group_hash_columns; -use crate::load; -use crate::select_vector::SelectVector; +use crate::new_sel; +use crate::read; use crate::types::DataType; use crate::AggregateFunctionRef; use crate::Column; use crate::ColumnBuilder; +use crate::HashTableConfig; +use crate::Payload; use crate::StateAddr; +use crate::BATCH_SIZE; +use crate::LOAD_FACTOR; +use crate::MAX_PAGE_SIZE; -const LOAD_FACTOR: f64 = 1.5; -// hashes layout: -// [SALT][PAGE_NR][PAGE_OFFSET] -// [SALT] are the high bits of the hash value, e.g. 16 for 64 bit hashes -// [PAGE_NR] is the buffer managed payload page index -// [PAGE_OFFSET] is the logical entry offset into said payload page - -#[repr(packed)] -#[derive(Default, Debug, Clone, Copy)] -pub struct Entry { - pub salt: u16, - pub page_offset: u16, - pub page_nr: u32, -} +const BATCH_ADD_SIZE: usize = 2048; + +// The high 16 bits are the salt, the low 48 bits are the pointer address +pub type Entry = u64; pub struct AggregateHashTable { - payload: Payload, + pub payload: PartitionedPayload, + // use for append rows directly during deserialize + pub direct_append: bool, + pub config: HashTableConfig, + current_radix_bits: u64, entries: Vec, + count: usize, capacity: usize, } +unsafe impl Send for AggregateHashTable {} +unsafe impl Sync for AggregateHashTable {} + impl AggregateHashTable { pub fn new( - arena: Arc, group_types: Vec, aggrs: Vec, + config: HashTableConfig, + arena: Arc, + ) -> Self { + let capacity = Self::initial_capacity(); + Self::new_with_capacity(group_types, aggrs, config, capacity, arena) + } + + pub fn new_with_capacity( + group_types: Vec, + aggrs: Vec, + config: HashTableConfig, + capacity: usize, + arena: Arc, ) -> Self { - let capacity = 128; Self { - entries: Self::new_entries(capacity), - payload: Payload::new(arena, group_types, aggrs), + entries: vec![0u64; capacity], + count: 0, + direct_append: false, + current_radix_bits: config.initial_radix_bits, + payload: PartitionedPayload::new( + group_types, + aggrs, + 1 << config.initial_radix_bits, + vec![arena], + ), capacity, + config, } } - // Faster way to create entries - // We don't need to extend N zero elements using u64 after we allocate zero spaces - // due to IsZero Trait(https://stdrs.dev/nightly/x86_64-unknown-linux-gnu/src/alloc/vec/spec_from_elem.rs.html#24) - fn new_entries(capacity: usize) -> Vec { - let entries = vec![0u64; capacity]; - let (ptr, len, cap) = entries.into_raw_parts(); - unsafe { Vec::from_raw_parts(ptr as *mut Entry, len, cap) } + pub fn new_directly( + group_types: Vec, + aggrs: Vec, + config: HashTableConfig, + capacity: usize, + arena: Arc, + need_init_entry: bool, + ) -> Self { + let entries = if need_init_entry { + vec![0u64; capacity] + } else { + vec![] + }; + Self { + entries, + count: 0, + direct_append: !need_init_entry, + current_radix_bits: config.initial_radix_bits, + payload: PartitionedPayload::new( + group_types, + aggrs, + 1 << config.initial_radix_bits, + vec![arena], + ), + capacity, + config, + } } - fn len(&self) -> usize { + pub fn len(&self) -> usize { self.payload.len() } - // Add new groups and combine the states pub fn add_groups( &mut self, state: &mut ProbeState, group_columns: &[Column], params: &[Vec], + agg_states: &[Column], + row_count: usize, + ) -> Result { + if row_count <= BATCH_ADD_SIZE { + self.add_groups_inner(state, group_columns, params, agg_states, row_count) + } else { + let mut new_count = 0; + for start in (0..row_count).step_by(BATCH_ADD_SIZE) { + let end = (start + BATCH_ADD_SIZE).min(row_count); + let step_group_columns = group_columns + .iter() + .map(|c| c.slice(start..end)) + .collect::>(); + + let step_params: Vec> = params + .iter() + .map(|c| c.iter().map(|x| x.slice(start..end)).collect()) + .collect::>(); + let agg_states = agg_states + .iter() + .map(|c| c.slice(start..end)) + .collect::>(); + + new_count += self.add_groups_inner( + state, + &step_group_columns, + &step_params, + &agg_states, + end - start, + )?; + } + Ok(new_count) + } + } + + // Add new groups and combine the states + fn add_groups_inner( + &mut self, + state: &mut ProbeState, + group_columns: &[Column], + params: &[Vec], + agg_states: &[Column], row_count: usize, ) -> Result { - let group_hashes = group_hash_columns(group_columns); - let new_group_count = self.probe_and_create(state, group_columns, row_count, &group_hashes); + state.row_count = row_count; + group_hash_columns(group_columns, &mut state.group_hashes); - for i in 0..row_count { - state.state_places[i] = unsafe { - StateAddr::new( - load::(state.addresses[i].add(self.payload.state_offset)) as usize, - ) - }; + let new_group_count = if self.direct_append { + for idx in 0..row_count { + state.empty_vector[idx] = idx; + } + self.payload.append_rows(state, row_count, group_columns); + row_count + } else { + self.probe_and_create(state, group_columns, row_count) + }; + + if !self.payload.aggrs.is_empty() { + for i in 0..row_count { + state.state_places[i] = unsafe { + StateAddr::new(read::( + state.addresses[i].add(self.payload.state_offset) as _ + ) as usize) + }; + } + + let state_places = &state.state_places.as_slice()[0..row_count]; + + if agg_states.is_empty() { + for ((aggr, params), addr_offset) in self + .payload + .aggrs + .iter() + .zip(params.iter()) + .zip(self.payload.state_addr_offsets.iter()) + { + aggr.accumulate_keys(state_places, *addr_offset, params, row_count)?; + } + } else { + for ((aggr, agg_state), addr_offset) in self + .payload + .aggrs + .iter() + .zip(agg_states.iter()) + .zip(self.payload.state_addr_offsets.iter()) + { + aggr.batch_merge(state_places, *addr_offset, agg_state)?; + } + } } - for ((aggr, params), addr_offset) in self - .payload - .aggrs - .iter() - .zip(params.iter()) - .zip(self.payload.state_addr_offsets.iter()) - { - aggr.accumulate_keys( - &state.state_places.as_slice()[0..row_count], - *addr_offset, - params, - row_count, - )?; + if self.config.partial_agg { + // check size + if self.count + BATCH_ADD_SIZE > self.resize_threshold() + && self.capacity >= self.config.max_partial_capacity + { + self.clear_ht(); + self.reset_count(); + } + + // check maybe_repartition + if self.maybe_repartition() { + self.clear_ht(); + self.reset_count(); + } } + Ok(new_group_count) } @@ -120,26 +244,25 @@ impl AggregateHashTable { state: &mut ProbeState, group_columns: &[Column], row_count: usize, - hashes: &[u64], ) -> usize { - if self.capacity - self.len() <= row_count || self.len() > self.resize_threshold() { - let mut new_capacity = self.capacity * 2; - - while new_capacity - self.len() <= row_count { - new_capacity *= 2; - } - self.resize(new_capacity); + // exceed capacity or should resize + if row_count + self.count > self.resize_threshold() { + self.resize(self.capacity * 2); } - state.adjust_group_columns(group_columns, hashes, row_count, self.capacity); - let mut new_group_count = 0; let mut remaining_entries = row_count; - let mut select_vector = SelectVector::auto_increment(); + let entries = &mut self.entries; - let mut payload_page_offset = self.len() % self.payload.row_per_page; - let mut payload_page_nr = (self.len() / self.payload.row_per_page) + 1; + let mut group_hashes = new_sel(); + let mut hash_salts = [0_u64; BATCH_SIZE]; + let mask = self.capacity - 1; + for i in 0..row_count { + group_hashes[i] = state.group_hashes[i] as usize & mask; + hash_salts[i] = state.group_hashes[i].get_salt(); + state.no_match_vector[i] = i; + } while remaining_entries > 0 { let mut new_entry_count = 0; @@ -148,125 +271,154 @@ impl AggregateHashTable { // 1. inject new_group_count, new_entry_count, need_compare_count, no_match_count for i in 0..remaining_entries { - let index = select_vector.get_index(i); - let entry = &mut self.entries[state.ht_offsets[index]]; - - // cell is empty, could be occupied - if entry.page_nr == 0 { - entry.salt = state.hash_salts[index]; - entry.page_nr = payload_page_nr as u32; - entry.page_offset = payload_page_offset as u16; - - payload_page_offset += 1; - if payload_page_offset == self.payload.row_per_page { - payload_page_offset = 0; - payload_page_nr += 1; + let index = state.no_match_vector[i]; + + let ht_offset = &mut group_hashes[index]; + + let salt = hash_salts[index]; + + loop { + let entry = &mut entries[*ht_offset]; + if entry.is_occupied() { + if entry.get_salt() == salt { + state.group_compare_vector[need_compare_count] = index; + need_compare_count += 1; + break; + } else { + *ht_offset += 1; + if *ht_offset >= self.capacity { + *ht_offset = 0; + } + continue; + } + } else { + entry.set_salt(salt); + state.empty_vector[new_entry_count] = index; + new_entry_count += 1; + break; } - - state.empty_vector.set_index(new_entry_count, index); - state.new_groups.set_index(new_group_count, index); - new_entry_count += 1; - new_group_count += 1; - } else if entry.salt == state.hash_salts[index] { - state - .group_compare_vector - .set_index(need_compare_count, index); - need_compare_count += 1; - } else { - state.no_match_vector.set_index(no_match_count, index); - no_match_count += 1; } } // 2. append new_group_count to payload if new_entry_count != 0 { - self.payload.append_rows( - state, - hashes, - &select_vector, - new_entry_count, - group_columns, - ); - } + new_group_count += new_entry_count; - // 3. handle need_compare_count - for need_compare_idx in 0..need_compare_count { - let index = state.group_compare_vector.get_index(need_compare_idx); - let entry = &mut self.entries[state.ht_offsets[index]]; + self.payload + .append_rows(state, new_entry_count, group_columns); - let page_ptr = self.payload.get_page_ptr((entry.page_nr - 1) as usize); - let page_offset = entry.page_offset as usize * self.payload.tuple_size; + for i in 0..new_entry_count { + let index = state.empty_vector[i]; + let ht_offset = group_hashes[index]; + let entry = &mut entries[ht_offset]; - state.addresses[index] = unsafe { page_ptr.add(page_offset) }; - } + entry.set_pointer(state.addresses[index]); - // 4. compare - unsafe { - row_match_columns( - group_columns, - &state.addresses, - &mut state.group_compare_vector, - need_compare_count, - &self.payload.validity_offsets, - &self.payload.group_offsets, - &mut state.no_match_vector, - &mut no_match_count, - ); + debug_assert_eq!(entry.get_pointer(), state.addresses[index]); + } } - // 5. Linear probing - for i in 0..no_match_count { - let index = state.no_match_vector.get_index(i); - state.ht_offsets[index] += 1; + // 3. set address of compare vector + if need_compare_count > 0 { + for i in 0..need_compare_count { + let index = state.group_compare_vector[i]; + let ht_offset = group_hashes[index]; + let entry = &mut entries[ht_offset]; - if state.ht_offsets[index] >= self.capacity { - state.ht_offsets[index] = 0; + debug_assert!(entry.is_occupied()); + debug_assert_eq!(entry.get_salt(), hash_salts[index]); + state.addresses[index] = entry.get_pointer(); } - } - std::mem::swap(&mut select_vector, &mut state.no_match_vector); - state.no_match_vector.resize(no_match_count); + // 4. compare + unsafe { + row_match_columns( + group_columns, + &state.addresses, + &mut state.group_compare_vector, + &mut state.temp_vector, + need_compare_count, + &self.payload.validity_offsets, + &self.payload.group_offsets, + &mut state.no_match_vector, + &mut no_match_count, + ); + } + } + // 5. Linear probing, just increase iter_times + for i in 0..no_match_count { + let idx = state.no_match_vector[i]; + let ht_offset = &mut group_hashes[idx]; + *ht_offset += 1; + if *ht_offset >= self.capacity { + *ht_offset = 0; + } + } remaining_entries = no_match_count; } - // set state places - for i in 0..row_count { - state.state_places[i] = unsafe { - StateAddr::new( - load::(state.addresses[i].add(self.payload.state_offset)) as usize, - ) - }; - } + self.count += new_group_count; new_group_count } pub fn combine(&mut self, other: Self, flush_state: &mut PayloadFlushState) -> Result<()> { - while other.payload.flush(flush_state) { + self.combine_payloads(&other.payload, flush_state) + } + + pub fn combine_payloads( + &mut self, + payloads: &PartitionedPayload, + flush_state: &mut PayloadFlushState, + ) -> Result<()> { + for payload in payloads.payloads.iter() { + self.combine_payload(payload, flush_state)?; + } + Ok(()) + } + + pub fn combine_payload( + &mut self, + payload: &Payload, + flush_state: &mut PayloadFlushState, + ) -> Result<()> { + flush_state.clear(); + + while payload.flush(flush_state) { let row_count = flush_state.row_count; let _ = self.probe_and_create( &mut flush_state.probe_state, &flush_state.group_columns, row_count, - &flush_state.group_hashes, ); + // set state places + if !self.payload.aggrs.is_empty() { + for i in 0..row_count { + flush_state.probe_state.state_places[i] = unsafe { + StateAddr::new(read::( + flush_state.probe_state.addresses[i].add(self.payload.state_offset) + as _, + ) as usize) + }; + } + } + let state = &mut flush_state.probe_state; + let places = &state.state_places.as_slice()[0..row_count]; + let rhses = &flush_state.state_places.as_slice()[0..row_count]; for (aggr, addr_offset) in self .payload .aggrs .iter() .zip(self.payload.state_addr_offsets.iter()) { - aggr.batch_merge_states( - &state.state_places.as_slice()[0..row_count], - &flush_state.state_places.as_slice()[0..row_count], - *addr_offset, - )?; + aggr.batch_merge_states(places, rhses, *addr_offset)?; } } + Ok(()) } @@ -296,34 +448,187 @@ impl AggregateHashTable { Ok(false) } + fn maybe_repartition(&mut self) -> bool { + // already final stage or the max radix bits + if !self.config.partial_agg || (self.current_radix_bits == self.config.max_radix_bits) { + return false; + } + + let bytes_per_partition = self.payload.memory_size() / self.payload.partition_count(); + + let mut new_radix_bits = self.current_radix_bits; + + if bytes_per_partition > MAX_PAGE_SIZE * self.config.block_fill_factor as usize { + new_radix_bits += self.config.repartition_radix_bits_incr; + } + + loop { + let current_max_radix_bits = self.config.current_max_radix_bits.load(Ordering::SeqCst); + if current_max_radix_bits < new_radix_bits + && self + .config + .current_max_radix_bits + .compare_exchange( + current_max_radix_bits, + new_radix_bits, + Ordering::SeqCst, + Ordering::SeqCst, + ) + .is_err() + { + continue; + } + break; + } + + let current_max_radix_bits = self.config.current_max_radix_bits.load(Ordering::SeqCst); + + if current_max_radix_bits > self.current_radix_bits { + let temp_payload = PartitionedPayload::new( + self.payload.group_types.clone(), + self.payload.aggrs.clone(), + 1, + vec![Arc::new(Bump::new())], + ); + let payload = std::mem::replace(&mut self.payload, temp_payload); + let mut state = PayloadFlushState::default(); + + self.current_radix_bits = current_max_radix_bits; + self.payload = payload.repartition(1 << current_max_radix_bits, &mut state); + return true; + } + false + } + + #[inline] fn resize_threshold(&self) -> usize { (self.capacity as f64 / LOAD_FACTOR) as usize } + // scan payload to reconstruct PointArray pub fn resize(&mut self, new_capacity: usize) { + if self.config.partial_agg { + if self.capacity == self.config.max_partial_capacity { + return; + } + self.entries = vec![0; new_capacity]; + self.reset_count(); + self.capacity = new_capacity; + return; + } + + self.reset_count(); let mask = (new_capacity - 1) as u64; - let mut entries = Self::new_entries(new_capacity); + let mut entries = vec![0; new_capacity]; + // iterate over payloads and copy to new entries - for row in 0..self.len() { - let row_ptr = self.payload.get_row_ptr(row); - let hash: u64 = unsafe { load(row_ptr.add(self.payload.hash_offset)) }; - let mut hash_slot = hash & mask; - - while entries[hash_slot as usize].page_nr != 0 { - hash_slot += 1; - if hash_slot >= self.capacity as u64 { - hash_slot = 0; + for payload in self.payload.payloads.iter() { + for page in payload.pages.iter() { + for idx in 0..page.rows { + let row_ptr: *const u8 = + unsafe { page.data.as_ptr().add(idx * payload.tuple_size) as _ }; + + let hash: u64 = + unsafe { core::ptr::read(row_ptr.add(payload.hash_offset) as _) }; + + let mut hash_slot = hash & mask; + while entries[hash_slot as usize].is_occupied() { + hash_slot += 1; + if hash_slot >= new_capacity as u64 { + hash_slot = 0; + } + } + + let hs = hash_slot as usize; + debug_assert!(!entries[hs].is_occupied()); + // set value + entries[hs].set_salt(hash.get_salt()); + entries[hs].set_pointer(row_ptr); + debug_assert!(entries[hs].is_occupied()); + debug_assert_eq!(entries[hs].get_pointer(), row_ptr); + debug_assert_eq!(entries[hs].get_salt(), hash.get_salt()); + + self.count += 1; } } - let entry = &mut entries[hash_slot as usize]; - - entry.page_nr = (row / self.payload.row_per_page) as u32 + 1; - entry.page_offset = (row % self.payload.row_per_page) as u16; - entry.salt = (hash >> (64 - 16)) as u16; } self.entries = entries; self.capacity = new_capacity; } + + pub fn initial_capacity() -> usize { + 8192 * 4 + } + + pub fn get_capacity_for_count(count: usize) -> usize { + ((count.max(Self::initial_capacity()) as f64 * LOAD_FACTOR) as usize).next_power_of_two() + } + + pub fn clear_ht(&mut self) { + self.payload.mark_min_cardinality(); + self.entries.fill(0); + } + + pub fn reset_count(&mut self) { + self.count = 0; + } + + pub fn allocated_bytes(&self) -> usize { + self.payload.memory_size() + + self + .payload + .arenas + .iter() + .map(|arena| arena.allocated_bytes()) + .sum::() + } +} + +/// Upper 16 bits are salt +const SALT_MASK: u64 = 0xFFFF000000000000; +/// Lower 48 bits are the pointer +const POINTER_MASK: u64 = 0x0000FFFFFFFFFFFF; + +pub(crate) trait EntryLike { + fn get_salt(&self) -> u64; + fn set_salt(&mut self, _salt: u64); + fn is_occupied(&self) -> bool; + + fn get_pointer(&self) -> *const u8; + fn set_pointer(&mut self, ptr: *const u8); +} + +impl EntryLike for u64 { + #[inline] + fn get_salt(&self) -> u64 { + *self | POINTER_MASK + } + + #[inline] + fn set_salt(&mut self, salt: u64) { + *self = salt; + } + + #[inline] + fn is_occupied(&self) -> bool { + *self != 0 + } + + #[inline] + fn get_pointer(&self) -> *const u8 { + (*self & POINTER_MASK) as *const u8 + } + + #[inline] + fn set_pointer(&mut self, ptr: *const u8) { + let ptr_value = ptr as u64; + // Pointer shouldn't use upper bits + debug_assert!(ptr_value & SALT_MASK == 0); + // Value should have all 1's in the pointer area + debug_assert!(*self & POINTER_MASK == POINTER_MASK); + + *self &= ptr_value | SALT_MASK; + } } diff --git a/src/query/expression/src/aggregate/group_hash.rs b/src/query/expression/src/aggregate/group_hash.rs index 9f5ed204dd5c..c1381d9681c0 100644 --- a/src/query/expression/src/aggregate/group_hash.rs +++ b/src/query/expression/src/aggregate/group_hash.rs @@ -12,10 +12,11 @@ // See the License for the specific language governing permissions and // limitations under the License. -use databend_common_hashtable::FastHash; use ethnum::i256; +use ordered_float::OrderedFloat; use crate::types::decimal::DecimalType; +use crate::types::AnyType; use crate::types::ArgType; use crate::types::BinaryType; use crate::types::BitmapType; @@ -31,80 +32,223 @@ use crate::types::ValueType; use crate::types::VariantType; use crate::with_number_mapped_type; use crate::Column; +use crate::ScalarRef; const NULL_HASH_VAL: u64 = 0xd1cefa08eb382d69; -pub fn group_hash_columns(cols: &[Column]) -> Vec { +pub fn group_hash_columns(cols: &[Column], values: &mut [u64]) { debug_assert!(!cols.is_empty()); - - let mut values = group_hash_column(&cols[0]); - + combine_group_hash_column::(&cols[0], values); if cols.len() > 1 { for col in &cols[1..] { - let col_values = group_hash_column(col); - for (val, v) in values.iter_mut().zip(col_values) { - *val = (*val).wrapping_mul(NULL_HASH_VAL) ^ v; - } + combine_group_hash_column::(col, values); } } - values } -pub fn group_hash_column(c: &Column) -> Vec { - let len = c.len(); +pub fn combine_group_hash_column(c: &Column, values: &mut [u64]) { match c.data_type() { - DataType::Null => vec![NULL_HASH_VAL; len], - DataType::EmptyArray => vec![NULL_HASH_VAL; len], - DataType::EmptyMap => vec![NULL_HASH_VAL; len], + DataType::Null => {} + DataType::EmptyArray => {} + DataType::EmptyMap => {} DataType::Number(v) => with_number_mapped_type!(|NUM_TYPE| match v { NumberDataType::NUM_TYPE => { - group_hash_type_column::>(c) + combine_group_hash_type_column::>(c, values) } }), DataType::Decimal(v) => match v { - DecimalDataType::Decimal128(_) => group_hash_type_column::>(c), - DecimalDataType::Decimal256(_) => group_hash_type_column::>(c), + DecimalDataType::Decimal128(_) => { + combine_group_hash_type_column::>(c, values) + } + DecimalDataType::Decimal256(_) => { + combine_group_hash_type_column::>(c, values) + } }, - DataType::Boolean => group_hash_type_column::(c), - DataType::Binary => { - let c = BinaryType::try_downcast_column(c).unwrap(); - BinaryType::iter_column(&c).map(|x| x.fast_hash()).collect() - } - DataType::String => { - let c = StringType::try_downcast_column(c).unwrap(); - StringType::iter_column(&c).map(|x| x.fast_hash()).collect() - } - DataType::Bitmap => { - let c = BitmapType::try_downcast_column(c).unwrap(); - BitmapType::iter_column(&c).map(|x| x.fast_hash()).collect() - } - DataType::Variant => { - let c = VariantType::try_downcast_column(c).unwrap(); - VariantType::iter_column(&c) - .map(|x| x.fast_hash()) - .collect() - } - DataType::Timestamp => group_hash_type_column::(c), - DataType::Date => group_hash_type_column::(c), + DataType::Boolean => combine_group_hash_type_column::(c, values), + DataType::Timestamp => combine_group_hash_type_column::(c, values), + DataType::Date => combine_group_hash_type_column::(c, values), + DataType::Binary => combine_group_hash_string_column::(c, values), + DataType::String => combine_group_hash_string_column::(c, values), + DataType::Bitmap => combine_group_hash_string_column::(c, values), + DataType::Variant => combine_group_hash_string_column::(c, values), DataType::Nullable(_) => { let col = c.as_nullable().unwrap(); - let mut values = group_hash_column(&col.column); - for (index, val) in col.validity.iter().enumerate() { - if !val { - values[index] = NULL_HASH_VAL; + if IS_FIRST { + combine_group_hash_column::(&col.column, values); + for (val, ok) in values.iter_mut().zip(col.validity.iter()) { + if !ok { + *val = NULL_HASH_VAL; + } + } + } else { + let mut values2 = vec![0; c.len()]; + combine_group_hash_column::(&col.column, &mut values2); + + for ((x, val), ok) in values2 + .iter() + .zip(values.iter_mut()) + .zip(col.validity.iter()) + { + if ok { + *val = (*val).wrapping_mul(NULL_HASH_VAL) ^ *x; + } else { + *val = (*val).wrapping_mul(NULL_HASH_VAL) ^ NULL_HASH_VAL; + } } } - values } - DataType::Tuple(_) => todo!(), - DataType::Array(_) => todo!(), - DataType::Map(_) => todo!(), DataType::Generic(_) => unreachable!(), + _ => combine_group_hash_type_column::(c, values), + } +} + +fn combine_group_hash_type_column( + col: &Column, + values: &mut [u64], +) where + for<'a> T::ScalarRef<'a>: AggHash, +{ + let c = T::try_downcast_column(col).unwrap(); + if IS_FIRST { + for (x, val) in T::iter_column(&c).zip(values.iter_mut()) { + *val = x.agg_hash(); + } + } else { + for (x, val) in T::iter_column(&c).zip(values.iter_mut()) { + *val = (*val).wrapping_mul(NULL_HASH_VAL) ^ x.agg_hash(); + } } } -fn group_hash_type_column(col: &Column) -> Vec -where for<'a> T::ScalarRef<'a>: FastHash { +fn combine_group_hash_string_column( + col: &Column, + values: &mut [u64], +) where + for<'a> T::ScalarRef<'a>: AsRef<[u8]>, +{ let c = T::try_downcast_column(col).unwrap(); - T::iter_column(&c).map(|x| x.fast_hash()).collect() + if IS_FIRST { + for (x, val) in T::iter_column(&c).zip(values.iter_mut()) { + *val = x.as_ref().agg_hash(); + } + } else { + for (x, val) in T::iter_column(&c).zip(values.iter_mut()) { + *val = (*val).wrapping_mul(NULL_HASH_VAL) ^ x.as_ref().agg_hash(); + } + } +} + +pub trait AggHash { + fn agg_hash(&self) -> u64; +} + +// MIT License +// Copyright (c) 2018-2021 Martin Ankerl +// https://github.com/martinus/robin-hood-hashing/blob/3.11.5/LICENSE +// Rewrite using chatgpt + +impl AggHash for [u8] { + fn agg_hash(&self) -> u64 { + const M: u64 = 0xc6a4a7935bd1e995; + const SEED: u64 = 0xe17a1465; + const R: u64 = 47; + + let mut h = SEED ^ (self.len() as u64).wrapping_mul(M); + let n_blocks = self.len() / 8; + + for i in 0..n_blocks { + let mut k = unsafe { (&self[i * 8] as *const u8 as *const u64).read_unaligned() }; + + k = k.wrapping_mul(M); + k ^= k >> R; + k = k.wrapping_mul(M); + + h ^= k; + h = h.wrapping_mul(M); + } + + let data8 = &self[n_blocks * 8..]; + for (i, &value) in data8.iter().enumerate() { + h ^= (value as u64) << (8 * (data8.len() - i - 1)); + } + + h ^= h >> R; + h = h.wrapping_mul(M); + h ^= h >> R; + + h + } +} + +macro_rules! impl_agg_hash_for_primitive_types { + ($t: ty) => { + impl AggHash for $t { + #[inline(always)] + fn agg_hash(&self) -> u64 { + let mut x = *self as u64; + x ^= x >> 32; + x = x.wrapping_mul(0xd6e8feb86659fd93); + x ^= x >> 32; + x = x.wrapping_mul(0xd6e8feb86659fd93); + x ^= x >> 32; + x + } + } + }; +} + +impl_agg_hash_for_primitive_types!(u8); +impl_agg_hash_for_primitive_types!(i8); +impl_agg_hash_for_primitive_types!(u16); +impl_agg_hash_for_primitive_types!(i16); +impl_agg_hash_for_primitive_types!(u32); +impl_agg_hash_for_primitive_types!(i32); +impl_agg_hash_for_primitive_types!(u64); +impl_agg_hash_for_primitive_types!(i64); + +impl AggHash for bool { + fn agg_hash(&self) -> u64 { + *self as u64 + } +} + +impl AggHash for i128 { + fn agg_hash(&self) -> u64 { + self.to_le_bytes().agg_hash() + } +} + +impl AggHash for i256 { + fn agg_hash(&self) -> u64 { + self.to_le_bytes().agg_hash() + } +} + +impl AggHash for OrderedFloat { + #[inline(always)] + fn agg_hash(&self) -> u64 { + if self.is_nan() { + f32::NAN.to_bits().agg_hash() + } else { + self.to_bits().agg_hash() + } + } +} + +impl AggHash for OrderedFloat { + #[inline(always)] + fn agg_hash(&self) -> u64 { + if self.is_nan() { + f64::NAN.to_bits().agg_hash() + } else { + self.to_bits().agg_hash() + } + } +} + +impl AggHash for ScalarRef<'_> { + #[inline(always)] + fn agg_hash(&self) -> u64 { + self.to_string().as_bytes().agg_hash() + } } diff --git a/src/query/expression/src/aggregate/mod.rs b/src/query/expression/src/aggregate/mod.rs index d3adb408663c..6911a0efc3cf 100644 --- a/src/query/expression/src/aggregate/mod.rs +++ b/src/query/expression/src/aggregate/mod.rs @@ -19,14 +19,115 @@ mod aggregate_function; mod aggregate_function_state; mod aggregate_hashtable; mod group_hash; +mod partitioned_payload; mod payload; mod payload_flush; mod payload_row; mod probe_state; +use std::sync::atomic::AtomicU64; +use std::sync::atomic::Ordering; +use std::sync::Arc; + pub use aggregate_function::*; pub use aggregate_function_state::*; pub use aggregate_hashtable::*; pub use group_hash::*; +pub use partitioned_payload::*; +pub use payload::*; pub use payload_flush::*; pub use probe_state::*; + +pub type SelectVector = [usize; BATCH_SIZE]; + +pub fn new_sel() -> SelectVector { + [0; BATCH_SIZE] +} + +// A batch size to probe, flush, repartition, etc. +pub(crate) const BATCH_SIZE: usize = 2048; +pub(crate) const LOAD_FACTOR: f64 = 1.5; +pub(crate) const MAX_PAGE_SIZE: usize = 256 * 1024; + +// Assume (1 << 15) = 32KB L1 cache per core, divided by two because hyperthreading +pub(crate) const L1_CACHE_SIZE: usize = 32768 / 2; +// Assume (1 << 20) = 1MB L2 cache per core, divided by two because hyperthreading +pub(crate) const L2_CACHE_SIZE: usize = 1048576 / 2; +// Assume (1 << 20) + (1 << 19) = 1.5MB L3 cache per core (shared), divided by two because hyperthreading +pub(crate) const L3_CACHE_SIZE: usize = 1572864 / 2; + +#[derive(Clone, Debug)] +pub struct HashTableConfig { + // Max radix bits across all threads, this is a hint to repartition + pub current_max_radix_bits: Arc, + pub initial_radix_bits: u64, + pub max_radix_bits: u64, + pub repartition_radix_bits_incr: u64, + pub block_fill_factor: f64, + pub partial_agg: bool, + pub max_partial_capacity: usize, +} + +impl Default for HashTableConfig { + fn default() -> Self { + Self { + current_max_radix_bits: Arc::new(AtomicU64::new(3)), + initial_radix_bits: 3, + max_radix_bits: 7, + repartition_radix_bits_incr: 2, + block_fill_factor: 1.8, + partial_agg: false, + max_partial_capacity: 131072, + } + } +} + +impl HashTableConfig { + pub fn with_initial_radix_bits(mut self, initial_radix_bits: u64) -> Self { + self.initial_radix_bits = initial_radix_bits; + self.current_max_radix_bits = Arc::new(AtomicU64::new(initial_radix_bits)); + self + } + + pub fn with_partial(mut self, partial_agg: bool, active_threads: usize) -> Self { + self.partial_agg = partial_agg; + + // init max_partial_capacity + let total_shared_cache_size = active_threads * L3_CACHE_SIZE; + let cache_per_active_thread = + L1_CACHE_SIZE + L2_CACHE_SIZE + total_shared_cache_size / active_threads; + let size_per_entry = (8_f64 * LOAD_FACTOR) as usize; + let capacity = (cache_per_active_thread / size_per_entry).next_power_of_two(); + self.max_partial_capacity = capacity; + + self + } + + pub fn cluster_with_partial(mut self, partial_agg: bool, node_nums: usize) -> Self { + self.partial_agg = partial_agg; + self.repartition_radix_bits_incr = 4; + self.max_partial_capacity = 131072 * (2 << node_nums); + + self + } + + pub fn update_current_max_radix_bits(&self) { + loop { + let current_max_radix_bits = self.current_max_radix_bits.load(Ordering::SeqCst); + if current_max_radix_bits < self.max_radix_bits + && self + .current_max_radix_bits + .compare_exchange( + current_max_radix_bits, + self.max_radix_bits, + Ordering::SeqCst, + Ordering::SeqCst, + ) + .is_err() + { + continue; + } + break; + } + } +} diff --git a/src/query/expression/src/aggregate/partitioned_payload.rs b/src/query/expression/src/aggregate/partitioned_payload.rs new file mode 100644 index 000000000000..6a0152b7978a --- /dev/null +++ b/src/query/expression/src/aggregate/partitioned_payload.rs @@ -0,0 +1,275 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed 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 std::alloc::Layout; +use std::sync::Arc; + +use bumpalo::Bump; +use itertools::Itertools; + +use super::payload::Payload; +use super::probe_state::ProbeState; +use crate::read; +use crate::types::DataType; +use crate::AggregateFunctionRef; +use crate::Column; +use crate::PayloadFlushState; +use crate::BATCH_SIZE; + +pub struct PartitionedPayload { + pub payloads: Vec, + pub group_types: Vec, + pub aggrs: Vec, + + pub group_sizes: Vec, + pub group_offsets: Vec, + pub validity_offsets: Vec, + pub hash_offset: usize, + pub state_offset: usize, + pub state_addr_offsets: Vec, + pub state_layout: Option, + + pub arenas: Vec>, + + partition_count: u64, + mask_v: u64, + shift_v: u64, +} + +unsafe impl Send for PartitionedPayload {} +unsafe impl Sync for PartitionedPayload {} + +impl PartitionedPayload { + pub fn new( + group_types: Vec, + aggrs: Vec, + partition_count: u64, + arenas: Vec>, + ) -> Self { + let radix_bits = partition_count.trailing_zeros() as u64; + debug_assert_eq!(1 << radix_bits, partition_count); + + let payloads = (0..partition_count) + .map(|_| Payload::new(arenas[0].clone(), group_types.clone(), aggrs.clone())) + .collect_vec(); + + let group_sizes = payloads[0].group_sizes.clone(); + let group_offsets = payloads[0].group_offsets.clone(); + let validity_offsets = payloads[0].validity_offsets.clone(); + let hash_offset = payloads[0].hash_offset; + let state_offset = payloads[0].state_offset; + let state_addr_offsets = payloads[0].state_addr_offsets.clone(); + let state_layout = payloads[0].state_layout; + + PartitionedPayload { + payloads, + group_types, + aggrs, + group_sizes, + group_offsets, + validity_offsets, + hash_offset, + state_offset, + state_addr_offsets, + state_layout, + partition_count, + + arenas, + mask_v: mask(radix_bits), + shift_v: shift(radix_bits), + } + } + + pub fn mark_min_cardinality(&mut self) { + for payload in self.payloads.iter_mut() { + payload.mark_min_cardinality(); + } + } + + pub fn append_rows( + &mut self, + state: &mut ProbeState, + new_group_rows: usize, + group_columns: &[Column], + ) { + if self.payloads.len() == 1 { + self.payloads[0].reserve_append_rows( + &state.empty_vector, + &state.group_hashes, + &mut state.addresses, + new_group_rows, + group_columns, + ); + } else { + // generate partition selection indices + state.reset_partitions(self.partition_count()); + let select_vector = &state.empty_vector; + + for idx in select_vector.iter().take(new_group_rows).copied() { + let hash = state.group_hashes[idx]; + let partition_idx = ((hash & self.mask_v) >> self.shift_v) as usize; + let sel = &mut state.partition_entries[partition_idx]; + + sel[state.partition_count[partition_idx]] = idx; + state.partition_count[partition_idx] += 1; + } + + for partition_index in 0..self.payloads.len() { + let count = state.partition_count[partition_index]; + if count > 0 { + let sel = &state.partition_entries[partition_index]; + + self.payloads[partition_index].reserve_append_rows( + sel, + &state.group_hashes, + &mut state.addresses, + count, + group_columns, + ); + } + } + } + } + + pub fn repartition(self, new_partition_count: usize, state: &mut PayloadFlushState) -> Self { + if self.partition_count() == new_partition_count { + return self; + } + + let mut new_partition_payload = PartitionedPayload::new( + self.group_types.clone(), + self.aggrs.clone(), + new_partition_count as u64, + self.arenas.clone(), + ); + + new_partition_payload.combine(self, state); + new_partition_payload + } + + pub fn combine(&mut self, other: PartitionedPayload, state: &mut PayloadFlushState) { + if other.partition_count == self.partition_count { + for (l, r) in self.payloads.iter_mut().zip(other.payloads.into_iter()) { + l.combine(r); + } + } else { + state.clear(); + + for payload in other.payloads.into_iter() { + self.combine_single(payload, state, None) + } + } + } + + pub fn combine_single( + &mut self, + mut other: Payload, + state: &mut PayloadFlushState, + only_bucket: Option, + ) { + if other.len() == 0 { + return; + } + + if self.partition_count == 1 { + self.payloads[0].combine(other); + } else { + state.clear(); + + // flush for other's each page to correct partition + while self.gather_flush(&other, state) { + // copy rows + for partition in (0..self.partition_count as usize) + .filter(|x| only_bucket.is_none() || only_bucket == Some(*x)) + { + let payload = &mut self.payloads[partition]; + let count = state.probe_state.partition_count[partition]; + + if count > 0 { + let sel = &state.probe_state.partition_entries[partition]; + payload.copy_rows(sel, count, &state.addresses); + } + } + } + other.state_move_out = true; + } + } + + // for each page's row, compute which partition it belongs to + pub fn gather_flush(&self, other: &Payload, state: &mut PayloadFlushState) -> bool { + if state.flush_page >= other.pages.len() { + return false; + } + + let page = &other.pages[state.flush_page]; + + // ToNext + if state.flush_page_row >= page.rows { + state.flush_page += 1; + state.flush_page_row = 0; + state.row_count = 0; + return self.gather_flush(other, state); + } + + let end = (state.flush_page_row + BATCH_SIZE).min(page.rows); + let rows = end - state.flush_page_row; + state.row_count = rows; + + state.probe_state.reset_partitions(self.partition_count()); + + for idx in 0..rows { + state.addresses[idx] = other.data_ptr(page, idx + state.flush_page_row); + + let hash = unsafe { read::(state.addresses[idx].add(self.hash_offset) as _) }; + + let partition_idx = ((hash & self.mask_v) >> self.shift_v) as usize; + + let sel = &mut state.probe_state.partition_entries[partition_idx]; + sel[state.probe_state.partition_count[partition_idx]] = idx; + state.probe_state.partition_count[partition_idx] += 1; + } + state.flush_page_row = end; + true + } + + #[inline] + pub fn len(&self) -> usize { + self.payloads.iter().map(|x| x.len()).sum() + } + + #[inline] + pub fn partition_count(&self) -> usize { + self.partition_count as usize + } + + #[allow(dead_code)] + pub fn page_count(&self) -> usize { + self.payloads.iter().map(|x| x.pages.len()).sum() + } + + #[allow(dead_code)] + pub fn memory_size(&self) -> usize { + self.payloads.iter().map(|x| x.memory_size()).sum() + } +} + +#[inline] +fn shift(radix_bits: u64) -> u64 { + 48 - radix_bits +} + +#[inline] +fn mask(radix_bits: u64) -> u64 { + ((1 << radix_bits) - 1) << shift(radix_bits) +} diff --git a/src/query/expression/src/aggregate/payload.rs b/src/query/expression/src/aggregate/payload.rs index 80153e11aed1..6611b22fb862 100644 --- a/src/query/expression/src/aggregate/payload.rs +++ b/src/query/expression/src/aggregate/payload.rs @@ -13,23 +13,29 @@ // limitations under the License. use std::alloc::Layout; +use std::mem::MaybeUninit; use std::sync::Arc; use bumpalo::Bump; +use itertools::Itertools; +use strength_reduce::StrengthReducedU64; use super::payload_row::rowformat_size; use super::payload_row::serialize_column_to_rowformat; -use super::probe_state::ProbeState; use crate::get_layout_offsets; -use crate::load; -use crate::select_vector::SelectVector; +use crate::read; use crate::store; use crate::types::DataType; use crate::AggregateFunctionRef; use crate::Column; +use crate::ColumnBuilder; +use crate::DataBlock; +use crate::PayloadFlushState; +use crate::SelectVector; use crate::StateAddr; +use crate::BATCH_SIZE; +use crate::MAX_PAGE_SIZE; -const MAX_PAGE_SIZE: usize = 256 * 1024; // payload layout // [VALIDITY][GROUPS][HASH][STATE_ADDRS] // [VALIDITY] is the validity bits of the data columns (including the HASH) @@ -38,13 +44,19 @@ const MAX_PAGE_SIZE: usize = 256 * 1024; // [STATE_ADDRS] is the state_addrs of the aggregate functions, 8 bytes each pub struct Payload { pub arena: Arc, + // if true, the states are moved out of the payload into other payload, and will not be dropped + pub state_move_out: bool, pub group_types: Vec, pub aggrs: Vec, - pub pages: Vec>, + pub pages: Pages, pub tuple_size: usize, pub row_per_page: usize, - pub current_row: usize, + + pub total_rows: usize, + + // Starts from 1, zero means no page allocated + pub current_write_page: usize, pub group_offsets: Vec, pub group_sizes: Vec, @@ -52,9 +64,23 @@ pub struct Payload { pub hash_offset: usize, pub state_offset: usize, pub state_addr_offsets: Vec, - pub state_layout: Layout, + pub state_layout: Option, + + // if set, the payload contains at least duplicate rows + pub min_cardinality: Option, } +unsafe impl Send for Payload {} +unsafe impl Sync for Payload {} + +pub struct Page { + pub(crate) data: Vec>, + pub(crate) rows: usize, + pub(crate) capacity: usize, +} + +pub type Pages = Vec; + // TODO FIXME impl Payload { pub fn new( @@ -63,7 +89,11 @@ impl Payload { aggrs: Vec, ) -> Self { let mut state_addr_offsets = Vec::new(); - let state_layout = get_layout_offsets(&aggrs, &mut state_addr_offsets).unwrap(); + let state_layout = if !aggrs.is_empty() { + Some(get_layout_offsets(&aggrs, &mut state_addr_offsets).unwrap()) + } else { + None + }; let mut tuple_size = 0; let mut validity_offsets = Vec::with_capacity(group_types.len()); @@ -92,16 +122,23 @@ impl Payload { tuple_size += hash_size; let state_offset = tuple_size; - tuple_size += 8; + if !aggrs.is_empty() { + tuple_size += 8; + } + + let row_per_page = (u16::MAX as usize).min(MAX_PAGE_SIZE / tuple_size).max(1); Self { arena, + state_move_out: false, pages: vec![], + current_write_page: 0, group_types, aggrs, tuple_size, - row_per_page: (u16::MAX as usize).min(MAX_PAGE_SIZE / tuple_size).max(1), - current_row: 0, + row_per_page, + min_cardinality: None, + total_rows: 0, group_offsets, group_sizes, validity_offsets, @@ -112,62 +149,106 @@ impl Payload { } } + #[inline] pub fn len(&self) -> usize { - self.current_row + self.total_rows } - pub fn get_page_ptr(&self, page_nr: usize) -> *const u8 { - self.pages[page_nr].as_ptr() + pub fn clear(&mut self) { + self.total_rows = 0; + self.pages.clear(); } - pub fn try_reverse(&mut self, additional_rows: usize) { - let mut row_capacity = self.pages.len() * self.row_per_page - self.current_row; + #[inline] + pub fn memory_size(&self) -> usize { + self.total_rows * self.tuple_size + } - while row_capacity < additional_rows { - self.pages - .push(vec![0; self.row_per_page * self.tuple_size]); - row_capacity += self.row_per_page; + #[inline] + pub fn writable_page(&mut self) -> &mut Page { + if self.current_write_page == 0 + || self.pages[self.current_write_page - 1].rows + == self.pages[self.current_write_page - 1].capacity + { + self.current_write_page += 1; + if self.current_write_page > self.pages.len() { + self.pages.push(Page { + data: Vec::with_capacity(self.row_per_page * self.tuple_size), + rows: 0, + capacity: self.row_per_page, + }); + } } + &mut self.pages[self.current_write_page - 1] } - pub fn get_row_ptr(&self, row: usize) -> *const u8 { - let page = row / self.row_per_page; - let page_ptr = self.get_page_ptr(page); - let row_offset = (row % self.row_per_page) * self.tuple_size; - - unsafe { page_ptr.add(row_offset) } + #[inline] + pub fn data_ptr(&self, page: &Page, row: usize) -> *const u8 { + unsafe { page.data.as_ptr().add(row * self.tuple_size) as _ } } - pub fn append_rows( + pub fn reserve_append_rows( &mut self, - state: &mut ProbeState, - group_hashes: &[u64], select_vector: &SelectVector, + group_hashes: &[u64], + address: &mut [*const u8], new_group_rows: usize, group_columns: &[Column], ) { - self.try_reverse(new_group_rows); + let tuple_size = self.tuple_size; + let mut page = self.writable_page(); + for idx in select_vector.iter().take(new_group_rows).copied() { + address[idx] = unsafe { page.data.as_ptr().add(page.rows * tuple_size) as *const u8 }; + page.rows += 1; + + if page.rows == page.capacity { + page = self.writable_page(); + } + } - for i in 0..new_group_rows { - let idx = select_vector.get_index(i); + self.total_rows += new_group_rows; - state.addresses[idx] = self.get_row_ptr(self.current_row); - self.current_row += 1; - } + debug_assert_eq!( + self.total_rows, + self.pages.iter().map(|x| x.rows).sum::() + ); - let address = state.addresses.as_slice(); + self.append_rows( + select_vector, + group_hashes, + address, + new_group_rows, + group_columns, + ) + } + pub fn append_rows( + &mut self, + select_vector: &SelectVector, + group_hashes: &[u64], + address: &mut [*const u8], + new_group_rows: usize, + group_columns: &[Column], + ) { let mut write_offset = 0; // write validity for col in group_columns { if let Column::Nullable(c) = col { let bitmap = &c.validity; - for i in 0..new_group_rows { - let idx = select_vector.get_index(i); - if bitmap.get_bit(idx) { + if bitmap.unset_bits() == 0 || bitmap.unset_bits() == bitmap.len() { + let val: u8 = if bitmap.unset_bits() == 0 { 1 } else { 0 }; + // faster path + for idx in select_vector.iter().take(new_group_rows).copied() { unsafe { - let dst = address[i].add(write_offset); - store(&1, dst as *mut u8); + let dst = address[idx].add(write_offset); + store::(&val, dst as *mut u8); + } + } + } else { + for idx in select_vector.iter().take(new_group_rows).copied() { + unsafe { + let dst = address[idx].add(write_offset); + store::(&(bitmap.get_bit(idx) as u8), dst as *mut u8); } } } @@ -177,6 +258,8 @@ impl Payload { let mut scratch = vec![]; for (idx, col) in group_columns.iter().enumerate() { + debug_assert!(write_offset == self.group_offsets[idx]); + unsafe { serialize_column_to_rowformat( &self.arena, @@ -192,45 +275,152 @@ impl Payload { } // write group hashes - for i in 0..new_group_rows { - let idx = select_vector.get_index(i); + debug_assert!(write_offset == self.hash_offset); + for idx in select_vector.iter().take(new_group_rows).copied() { unsafe { - let dst = address[i].add(write_offset); - store(&group_hashes[idx], dst as *mut u8); + let dst = address[idx].add(write_offset); + store::(&group_hashes[idx], dst as *mut u8); } } write_offset += 8; + debug_assert!(write_offset == self.state_offset); + if let Some(layout) = self.state_layout { + // write states + for idx in select_vector.iter().take(new_group_rows).copied() { + let place = self.arena.alloc_layout(layout); + unsafe { + let dst = address[idx].add(write_offset); + store::(&(place.as_ptr() as u64), dst as *mut u8); + } + + let place = StateAddr::from(place); + for (aggr, offset) in self.aggrs.iter().zip(self.state_addr_offsets.iter()) { + aggr.init_state(place.next(*offset)); + } + } + } + } + + pub fn combine(&mut self, mut other: Payload) { + debug_assert_eq!( + other.total_rows, + other.pages.iter().map(|x| x.rows).sum::() + ); + + self.total_rows += other.total_rows; + self.pages.append(other.pages.as_mut()); + } + + pub fn mark_min_cardinality(&mut self) { + if self.min_cardinality.is_none() { + self.min_cardinality = Some(self.total_rows); + } + } + + pub fn copy_rows( + &mut self, + select_vector: &SelectVector, + row_count: usize, + address: &[*const u8], + ) { + let tuple_size = self.tuple_size; + let mut page = self.writable_page(); + for i in 0..row_count { + let index = select_vector[i]; - // write states - for i in 0..new_group_rows { - let place = self.arena.alloc_layout(self.state_layout); - let idx = select_vector.get_index(i); unsafe { - let dst = address[idx].add(write_offset); - store(&(place.as_ptr() as u64), dst as *mut u8); + std::ptr::copy_nonoverlapping( + address[index], + page.data.as_mut_ptr().add(page.rows * tuple_size) as _, + tuple_size, + ) } + page.rows += 1; - let place = StateAddr::from(place); - for (aggr, offset) in self.aggrs.iter().zip(self.state_addr_offsets.iter()) { - aggr.init_state(place.next(*offset)); + if page.rows == page.capacity { + page = self.writable_page(); } } + + self.total_rows += row_count; + + debug_assert_eq!( + self.total_rows, + self.pages.iter().map(|x| x.rows).sum::() + ); + } + + pub fn scatter(&self, state: &mut PayloadFlushState, partition_count: usize) -> bool { + if state.flush_page >= self.pages.len() { + return false; + } + + let page = &self.pages[state.flush_page]; + + // ToNext + if state.flush_page_row >= page.rows { + state.flush_page += 1; + state.flush_page_row = 0; + state.row_count = 0; + return self.scatter(state, partition_count); + } + + let end = (state.flush_page_row + BATCH_SIZE).min(page.rows); + let rows = end - state.flush_page_row; + state.row_count = rows; + + state.probe_state.reset_partitions(partition_count); + + let mods: StrengthReducedU64 = StrengthReducedU64::new(partition_count as u64); + for idx in 0..rows { + state.addresses[idx] = self.data_ptr(page, idx + state.flush_page_row); + + let hash = unsafe { read::(state.addresses[idx].add(self.hash_offset) as _) }; + + let partition_idx = (hash % mods) as usize; + + let sel = &mut state.probe_state.partition_entries[partition_idx]; + sel[state.probe_state.partition_count[partition_idx]] = idx; + state.probe_state.partition_count[partition_idx] += 1; + } + state.flush_page_row = end; + true + } + + pub fn empty_block(&self) -> DataBlock { + let columns = self + .aggrs + .iter() + .map(|f| ColumnBuilder::with_capacity(&f.return_type().unwrap(), 0).build()) + .chain( + self.group_types + .iter() + .map(|t| ColumnBuilder::with_capacity(t, 0).build()), + ) + .collect_vec(); + DataBlock::new_from_columns(columns) } } impl Drop for Payload { fn drop(&mut self) { // drop states - for (aggr, addr_offset) in self.aggrs.iter().zip(self.state_addr_offsets.iter()) { - if aggr.need_manual_drop_state() { - for row in 0..self.len() { - let row_ptr = self.get_row_ptr(row); - - unsafe { - let state_addr: u64 = load(row_ptr.add(self.state_offset)); - aggr.drop_state(StateAddr::new(state_addr as usize + *addr_offset)) - }; + if !self.state_move_out { + for (aggr, addr_offset) in self.aggrs.iter().zip(self.state_addr_offsets.iter()) { + if aggr.need_manual_drop_state() { + for page in self.pages.iter() { + for row in 0..page.rows { + unsafe { + let state_place = StateAddr::new(read::( + self.data_ptr(page, row).add(self.state_offset) as _, + ) + as usize); + + aggr.drop_state(state_place.next(*addr_offset)); + } + } + } } } } diff --git a/src/query/expression/src/aggregate/payload_flush.rs b/src/query/expression/src/aggregate/payload_flush.rs index e7ed245e7d06..163949157127 100644 --- a/src/query/expression/src/aggregate/payload_flush.rs +++ b/src/query/expression/src/aggregate/payload_flush.rs @@ -12,13 +12,17 @@ // See the License for the specific language governing permissions and // limitations under the License. +use databend_common_exception::Result; +use databend_common_io::prelude::bincode_deserialize_from_slice; use ethnum::i256; +use super::partitioned_payload::PartitionedPayload; use super::payload::Payload; use super::probe_state::ProbeState; -use crate::load; +use crate::read; use crate::types::binary::BinaryColumn; use crate::types::binary::BinaryColumnBuilder; +use crate::types::decimal::Decimal; use crate::types::decimal::DecimalType; use crate::types::nullable::NullableColumn; use crate::types::string::StringColumn; @@ -26,76 +30,199 @@ use crate::types::ArgType; use crate::types::BooleanType; use crate::types::DataType; use crate::types::DateType; +use crate::types::DecimalSize; use crate::types::NumberDataType; use crate::types::NumberType; use crate::types::TimestampType; +use crate::types::ValueType; use crate::with_number_mapped_type; +use crate::AggregateFunctionRef; use crate::Column; +use crate::ColumnBuilder; +use crate::DataBlock; +use crate::Scalar; use crate::StateAddr; +use crate::BATCH_SIZE; -const FLUSH_BATCH_SIZE: usize = 8192; - -#[derive(Default)] pub struct PayloadFlushState { pub probe_state: ProbeState, - pub group_hashes: Vec, pub group_columns: Vec, pub aggregate_results: Vec, pub row_count: usize, - pub flush_offset: usize, - pub addresses: Vec<*const u8>, - pub state_places: Vec, + pub flush_partition: usize, + pub flush_page: usize, + pub flush_page_row: usize, + + pub addresses: [*const u8; BATCH_SIZE], + pub state_places: [StateAddr; BATCH_SIZE], +} + +impl Default for PayloadFlushState { + fn default() -> Self { + PayloadFlushState { + probe_state: ProbeState::default(), + group_columns: Vec::new(), + aggregate_results: Vec::new(), + row_count: 0, + flush_partition: 0, + flush_page: 0, + flush_page_row: 0, + addresses: [std::ptr::null::(); BATCH_SIZE], + state_places: [StateAddr::new(0); BATCH_SIZE], + } + } +} + +unsafe impl Send for PayloadFlushState {} +unsafe impl Sync for PayloadFlushState {} + +impl PayloadFlushState { + pub fn clear(&mut self) { + self.row_count = 0; + self.flush_partition = 0; + self.flush_page = 0; + self.flush_page_row = 0; + } + + pub fn take_group_columns(&mut self) -> Vec { + std::mem::take(&mut self.group_columns) + } + pub fn take_aggregate_results(&mut self) -> Vec { + std::mem::take(&mut self.aggregate_results) + } +} + +impl PartitionedPayload { + pub fn flush(&mut self, state: &mut PayloadFlushState) -> bool { + if state.flush_partition >= self.payloads.len() { + return false; + } + + let p = &self.payloads[state.flush_partition]; + if p.flush(state) { + true + } else { + let partition_idx = state.flush_partition + 1; + state.clear(); + state.flush_partition = partition_idx; + self.flush(state) + } + } } impl Payload { - pub fn flush(&self, state: &mut PayloadFlushState) -> bool { - let flush_end = (state.flush_offset + FLUSH_BATCH_SIZE).min(self.len()); + pub fn aggregate_flush_all(&self) -> Result { + let mut state = PayloadFlushState::default(); + let mut blocks = vec![]; + + while let Some(block) = self.aggregate_flush(&mut state)? { + blocks.push(block); + } + + if blocks.is_empty() { + return Ok(self.empty_block()); + } + DataBlock::concat(&blocks) + } + + pub fn aggregate_flush(&self, state: &mut PayloadFlushState) -> Result> { + if self.flush(state) { + let row_count = state.row_count; + + let mut state_builders: Vec = self + .aggrs + .iter() + .map(|agg| state_serializer(agg, row_count)) + .collect(); + + for place in state.state_places.as_slice()[0..row_count].iter() { + for (idx, (addr_offset, aggr)) in self + .state_addr_offsets + .iter() + .zip(self.aggrs.iter()) + .enumerate() + { + let arg_place = place.next(*addr_offset); + aggr.serialize(arg_place, &mut state_builders[idx].data) + .unwrap(); + state_builders[idx].commit_row(); + } + } + + let mut cols = Vec::with_capacity(self.aggrs.len() + self.group_types.len()); + for builder in state_builders.into_iter() { + let col = Column::Binary(builder.build()); + cols.push(col); + } + + cols.extend_from_slice(&state.take_group_columns()); + return Ok(Some(DataBlock::new_from_columns(cols))); + } + + Ok(None) + } + + pub fn group_by_flush_all(&self) -> Result { + let mut state = PayloadFlushState::default(); + let mut blocks = vec![]; + + while self.flush(&mut state) { + let cols = state.take_group_columns(); + blocks.push(DataBlock::new_from_columns(cols)); + } - let rows = flush_end - state.flush_offset; - if rows == 0 { + if blocks.is_empty() { + return Ok(self.empty_block()); + } + + DataBlock::concat(&blocks) + } + + pub fn flush(&self, state: &mut PayloadFlushState) -> bool { + if state.flush_page >= self.pages.len() { return false; } - if state.row_count < rows { - state.group_hashes.resize(rows, 0); - state.addresses.resize(rows, std::ptr::null::()); - state.state_places.resize(rows, StateAddr::new(0)); + let page = &self.pages[state.flush_page]; + + if state.flush_page_row >= page.rows { + state.flush_page += 1; + state.flush_page_row = 0; + state.row_count = 0; + + return self.flush(state); } + let end = (state.flush_page_row + BATCH_SIZE).min(page.rows); + let rows = end - state.flush_page_row; state.group_columns.clear(); state.row_count = rows; - state.probe_state.adjust_row_count(rows); + state.probe_state.row_count = rows; + + for idx in 0..rows { + state.addresses[idx] = self.data_ptr(page, idx + state.flush_page_row); + state.probe_state.group_hashes[idx] = + unsafe { read::(state.addresses[idx].add(self.hash_offset) as _) }; - for row in state.flush_offset..flush_end { - state.addresses[row - state.flush_offset] = self.get_row_ptr(row); + if !self.aggrs.is_empty() { + state.state_places[idx] = unsafe { + StateAddr::new( + read::(state.addresses[idx].add(self.state_offset) as _) as usize, + ) + }; + } } - self.flush_hashes(state); for col_index in 0..self.group_types.len() { let col = self.flush_column(col_index, state); state.group_columns.push(col); } - for i in 0..rows { - state.state_places[i] = unsafe { - StateAddr::new(load::(state.addresses[i].add(self.state_offset)) as usize) - }; - } - - state.flush_offset = flush_end; + state.flush_page_row = end; true } - fn flush_hashes(&self, state: &mut PayloadFlushState) { - let len = state.probe_state.row_count; - - for i in 0..len { - state.group_hashes[i] = - unsafe { load::(state.addresses[i].add(self.hash_offset)) }; - } - } - fn flush_column(&self, col_index: usize, state: &mut PayloadFlushState) -> Column { let len = state.probe_state.row_count; @@ -110,11 +237,11 @@ impl Payload { self.flush_type_column::>(col_offset, state), }), DataType::Decimal(v) => match v { - crate::types::DecimalDataType::Decimal128(_) => { - self.flush_type_column::>(col_offset, state) + crate::types::DecimalDataType::Decimal128(s) => { + self.flush_decimal_column::(col_offset, state, s) } - crate::types::DecimalDataType::Decimal256(_) => { - self.flush_type_column::>(col_offset, state) + crate::types::DecimalDataType::Decimal256(s) => { + self.flush_decimal_column::(col_offset, state, s) } }, DataType::Timestamp => self.flush_type_column::(col_offset, state), @@ -124,10 +251,7 @@ impl Payload { DataType::Bitmap => Column::Bitmap(self.flush_binary_column(col_offset, state)), DataType::Variant => Column::Variant(self.flush_binary_column(col_offset, state)), DataType::Nullable(_) => unreachable!(), - DataType::Array(_) => todo!(), - DataType::Map(_) => todo!(), - DataType::Tuple(_) => todo!(), - DataType::Generic(_) => unreachable!(), + other => self.flush_generic_column(&other, col_offset, state), }; let validity_offset = self.validity_offsets[col_index]; @@ -150,12 +274,28 @@ impl Payload { state: &mut PayloadFlushState, ) -> Column { let len = state.probe_state.row_count; - let iter = - (0..len).map(|idx| unsafe { load::(state.addresses[idx].add(col_offset)) }); + let iter = (0..len) + .map(|idx| unsafe { read::(state.addresses[idx].add(col_offset) as _) }); let col = T::column_from_iter(iter, &[]); T::upcast_column(col) } + fn flush_decimal_column( + &self, + col_offset: usize, + state: &mut PayloadFlushState, + decimal_size: DecimalSize, + ) -> Column { + let len = state.probe_state.row_count; + let iter = (0..len).map(|idx| unsafe { + read::< as ValueType>::Scalar>( + state.addresses[idx].add(col_offset) as _ + ) + }); + let col = DecimalType::::column_from_iter(iter, &[]); + Num::upcast_column(col, decimal_size) + } + fn flush_binary_column( &self, col_offset: usize, @@ -166,9 +306,9 @@ impl Payload { unsafe { for idx in 0..len { - let str_len = load::(state.addresses[idx].add(col_offset)) as usize; - let data_address = - load::(state.addresses[idx].add(col_offset + 4)) as usize as *const u8; + let str_len = read::(state.addresses[idx].add(col_offset) as _) as usize; + let data_address = read::(state.addresses[idx].add(col_offset + 4) as _) + as usize as *const u8; let scalar = std::slice::from_raw_parts(data_address, str_len); @@ -186,4 +326,33 @@ impl Payload { ) -> StringColumn { unsafe { StringColumn::from_binary_unchecked(self.flush_binary_column(col_offset, state)) } } + + fn flush_generic_column( + &self, + data_type: &DataType, + col_offset: usize, + state: &mut PayloadFlushState, + ) -> Column { + let len = state.probe_state.row_count; + let mut builder = ColumnBuilder::with_capacity(data_type, len); + + unsafe { + for idx in 0..len { + let str_len = read::(state.addresses[idx].add(col_offset) as _) as usize; + let data_address = read::(state.addresses[idx].add(col_offset + 4) as _) + as usize as *const u8; + + let scalar = std::slice::from_raw_parts(data_address, str_len); + let scalar: Scalar = bincode_deserialize_from_slice(scalar).unwrap(); + + builder.push(scalar.as_ref()); + } + } + builder.build() + } +} + +fn state_serializer(func: &AggregateFunctionRef, row: usize) -> BinaryColumnBuilder { + let size = func.serialize_size_per_row().unwrap_or(4); + BinaryColumnBuilder::with_capacity(row, row * size) } diff --git a/src/query/expression/src/aggregate/payload_row.rs b/src/query/expression/src/aggregate/payload_row.rs index 97ded71a4d58..a7cc52721788 100644 --- a/src/query/expression/src/aggregate/payload_row.rs +++ b/src/query/expression/src/aggregate/payload_row.rs @@ -14,14 +14,16 @@ use bumpalo::Bump; use databend_common_arrow::arrow::bitmap::Bitmap; +use databend_common_io::prelude::bincode_deserialize_from_slice; +use databend_common_io::prelude::bincode_serialize_into_buf; use ethnum::i256; -use crate::load; -use crate::select_vector::SelectVector; +use crate::read; use crate::store; use crate::types::binary::BinaryColumn; use crate::types::decimal::DecimalColumn; use crate::types::decimal::DecimalType; +use crate::types::AnyType; use crate::types::ArgType; use crate::types::BinaryType; use crate::types::BooleanType; @@ -34,13 +36,13 @@ use crate::types::ValueType; use crate::with_decimal_mapped_type; use crate::with_number_mapped_type; use crate::Column; +use crate::Scalar; +use crate::SelectVector; pub fn rowformat_size(data_type: &DataType) -> usize { match data_type { DataType::Null | DataType::EmptyArray | DataType::EmptyMap => 0, DataType::Boolean => 1, - // use address instead - DataType::Binary | DataType::String | DataType::Bitmap | DataType::Variant => 4 + 8, /* u32 len + address */ DataType::Number(n) => n.bit_width() as usize / 8, DataType::Decimal(n) => match n { crate::types::DecimalDataType::Decimal128(_) => 16, @@ -48,10 +50,10 @@ pub fn rowformat_size(data_type: &DataType) -> usize { }, DataType::Timestamp => 8, DataType::Date => 4, - DataType::Nullable(_) => 4, - DataType::Array(_) => todo!(), - DataType::Map(_) => todo!(), - DataType::Tuple(_) => todo!(), + // use address instead + DataType::Binary | DataType::String | DataType::Bitmap | DataType::Variant => 4 + 8, /* u32 len + address */ + DataType::Nullable(x) => rowformat_size(x), + DataType::Array(_) | DataType::Map(_) | DataType::Tuple(_) => 4 + 8, DataType::Generic(_) => unreachable!(), } } @@ -60,18 +62,17 @@ pub fn rowformat_size(data_type: &DataType) -> usize { pub unsafe fn serialize_column_to_rowformat( arena: &Bump, column: &Column, - select_index: &SelectVector, + select_vector: &SelectVector, rows: usize, address: &[*const u8], offset: usize, - _scratch: &mut Vec, + scratch: &mut Vec, ) { match column { Column::Null { .. } | Column::EmptyArray { .. } | Column::EmptyMap { .. } => {} Column::Number(v) => with_number_mapped_type!(|NUM_TYPE| match v { NumberColumn::NUM_TYPE(buffer) => { - for i in 0..rows { - let index = select_index.get_index(i); + for index in select_vector.iter().take(rows).copied() { store(&buffer[index], address[index].add(offset) as *mut u8); } } @@ -79,26 +80,32 @@ pub unsafe fn serialize_column_to_rowformat( Column::Decimal(v) => { with_decimal_mapped_type!(|DECIMAL_TYPE| match v { DecimalColumn::DECIMAL_TYPE(buffer, _) => { - for i in 0..rows { - let index = select_index.get_index(i); + for index in select_vector.iter().take(rows).copied() { store(&buffer[index], address[index].add(offset) as *mut u8); } } }) } Column::Boolean(v) => { - for i in 0..rows { - let index = select_index.get_index(i); - store(&v.get_bit(index), address[index].add(offset) as *mut u8); + if v.unset_bits() == 0 || v.unset_bits() == v.len() { + let val: u8 = if v.unset_bits() == 0 { 1 } else { 0 }; + // faster path + for index in select_vector.iter().take(rows).copied() { + store(&val, address[index].add(offset) as *mut u8); + } + } else { + for index in select_vector.iter().take(rows).copied() { + store( + &(v.get_bit(index) as u8), + address[index].add(offset) as *mut u8, + ); + } } } Column::Binary(v) | Column::Bitmap(v) | Column::Variant(v) => { - for i in 0..rows { - let index = select_index.get_index(i); + for index in select_vector.iter().take(rows).copied() { let data = arena.alloc_slice_copy(v.index_unchecked(index)); - store(&(data.len() as u32), address[index].add(offset) as *mut u8); - store( &(data.as_ptr() as u64), address[index].add(offset + 4) as *mut u8, @@ -106,12 +113,9 @@ pub unsafe fn serialize_column_to_rowformat( } } Column::String(v) => { - for i in 0..rows { - let index = select_index.get_index(i); - let data = arena.alloc_slice_copy(v.index_unchecked(index).as_bytes()); - + for index in select_vector.iter().take(rows).copied() { + let data = arena.alloc_str(v.index_unchecked(index)); store(&(data.len() as u32), address[index].add(offset) as *mut u8); - store( &(data.as_ptr() as u64), address[index].add(offset + 4) as *mut u8, @@ -119,32 +123,39 @@ pub unsafe fn serialize_column_to_rowformat( } } Column::Timestamp(buffer) => { - for i in 0..rows { - let index = select_index.get_index(i); + for index in select_vector.iter().take(rows).copied() { store(&buffer[index], address[index].add(offset) as *mut u8); } } Column::Date(buffer) => { - for i in 0..rows { - let index = select_index.get_index(i); + for index in select_vector.iter().take(rows).copied() { store(&buffer[index], address[index].add(offset) as *mut u8); } } Column::Nullable(c) => serialize_column_to_rowformat( arena, &c.column, - select_index, + select_vector, rows, address, offset, - _scratch, + scratch, ), - Column::Array(_array) | Column::Map(_array) => { - todo!("nested tuple/array/map not supported yet") - } - Column::Tuple(_fields) => { - todo!("nested tuple/array/map not supported yet") + // for complex column + other => { + for index in select_vector.iter().take(rows).copied() { + let s = other.index_unchecked(index).to_owned(); + scratch.clear(); + bincode_serialize_into_buf(scratch, &s).unwrap(); + + let data = arena.alloc_slice_copy(scratch); + store(&(data.len() as u32), address[index].add(offset) as *mut u8); + store( + &(data.as_ptr() as u64), + address[index].add(offset + 4) as *mut u8, + ); + } } } } @@ -152,7 +163,8 @@ pub unsafe fn serialize_column_to_rowformat( pub unsafe fn row_match_columns( cols: &[Column], address: &[*const u8], - select_index: &mut SelectVector, + select_vector: &mut SelectVector, + temp_vector: &mut SelectVector, count: usize, validity_offset: &[usize], col_offsets: &[usize], @@ -168,20 +180,27 @@ pub unsafe fn row_match_columns( row_match_column( col, address, - select_index, + select_vector, + temp_vector, &mut count, *validity_offset, *col_offset, no_match, no_match_count, - ) + ); + + // no row matches + if count == 0 { + return; + } } } pub unsafe fn row_match_column( col: &Column, address: &[*const u8], - select_index: &mut SelectVector, + select_vector: &mut SelectVector, + temp_vector: &mut SelectVector, count: &mut usize, validity_offset: usize, col_offset: usize, @@ -205,7 +224,8 @@ pub unsafe fn row_match_column( col, validity, address, - select_index, + select_vector, + temp_vector, count, validity_offset, col_offset, @@ -219,7 +239,8 @@ pub unsafe fn row_match_column( col, validity, address, - select_index, + select_vector, + temp_vector, count, validity_offset, col_offset, @@ -230,7 +251,8 @@ pub unsafe fn row_match_column( col, validity, address, - select_index, + select_vector, + temp_vector, count, validity_offset, col_offset, @@ -242,7 +264,8 @@ pub unsafe fn row_match_column( col, validity, address, - select_index, + select_vector, + temp_vector, count, validity_offset, col_offset, @@ -253,7 +276,8 @@ pub unsafe fn row_match_column( col, validity, address, - select_index, + select_vector, + temp_vector, count, validity_offset, col_offset, @@ -264,39 +288,52 @@ pub unsafe fn row_match_column( col, validity, address, - select_index, + select_vector, + temp_vector, count, validity_offset, col_offset, no_match, no_match_count, ), - Column::Binary(v) | Column::Bitmap(v) | Column::Variant(v) => row_match_binary_column( + Column::Bitmap(v) | Column::Binary(v) | Column::Variant(v) => row_match_binary_column( v, validity, address, - select_index, + select_vector, + temp_vector, count, validity_offset, col_offset, no_match, no_match_count, ), - Column::String(v) => row_match_binary_column( - &v.clone().into(), - validity, + Column::String(v) => { + let v = &BinaryColumn::from(v.clone()); + row_match_binary_column( + v, + validity, + address, + select_vector, + temp_vector, + count, + validity_offset, + col_offset, + no_match, + no_match_count, + ) + } + Column::Nullable(_) => unreachable!("nullable is unwrapped"), + other => row_match_generic_column( + other, address, - select_index, + select_vector, + temp_vector, count, - validity_offset, col_offset, no_match, no_match_count, ), - Column::Nullable(_) => unreachable!(), - Column::Array(_) => todo!(), - Column::Map(_) => todo!(), - Column::Tuple(_) => todo!(), } } @@ -304,7 +341,8 @@ unsafe fn row_match_binary_column( col: &BinaryColumn, validity: Option<&Bitmap>, address: &[*const u8], - select_index: &mut SelectVector, + select_vector: &mut SelectVector, + temp_vector: &mut SelectVector, count: &mut usize, validity_offset: usize, col_offset: usize, @@ -315,65 +353,68 @@ unsafe fn row_match_binary_column( let mut equal: bool; if let Some(validity) = validity { - for i in 0..*count { - let idx = select_index.get_index(i); - let isnull = !validity.get_bit(idx); - + let is_all_set = validity.unset_bits() == 0; + for idx in select_vector[..*count].iter() { + let idx = *idx; let validity_address = address[idx].add(validity_offset); - let isnull2 = load::(validity_address) != 0; + let is_set2 = read::(validity_address as _) != 0; + let is_set = is_all_set || validity.get_bit_unchecked(idx); - equal = isnull == isnull2; - if !isnull && !isnull2 { + if is_set && is_set2 { let len_address = address[idx].add(col_offset); let address = address[idx].add(col_offset + 4); - let len = load::(len_address) as usize; + let len = read::(len_address as _) as usize; let value = BinaryType::index_column_unchecked(col, idx); if len != value.len() { equal = false; } else { - let data_address = load::(address) as usize as *const u8; + let data_address = read::(address as _) as usize as *const u8; let scalar = std::slice::from_raw_parts(data_address, len); - equal = scalar.eq(value); + equal = databend_common_hashtable::fast_memcmp(scalar, value); } + } else { + equal = is_set == is_set2; } if equal { - select_index.set_index(match_count, idx); + temp_vector[match_count] = idx; match_count += 1; } else { - no_match.set_index(*no_match_count, idx); + no_match[*no_match_count] = idx; *no_match_count += 1; } } } else { - for i in 0..*count { - let idx = select_index.get_index(i); - + for idx in select_vector[..*count].iter() { + let idx = *idx; let len_address = address[idx].add(col_offset); let address = address[idx].add(col_offset + 4); - let len = load::(len_address) as usize; + let len = read::(len_address as _) as usize; let value = BinaryType::index_column_unchecked(col, idx); if len != value.len() { equal = false; } else { - let data_address = load::(address) as usize as *const u8; + let data_address = read::(address as _) as usize as *const u8; let scalar = std::slice::from_raw_parts(data_address, len); - equal = scalar.eq(value); + + equal = databend_common_hashtable::fast_memcmp(scalar, value); } if equal { - select_index.set_index(match_count, idx); + temp_vector[match_count] = idx; match_count += 1; } else { - no_match.set_index(*no_match_count, idx); + no_match[*no_match_count] = idx; *no_match_count += 1; } } } + select_vector.clone_from_slice(temp_vector); + *count = match_count; } @@ -381,7 +422,8 @@ unsafe fn row_match_column_type( col: &Column, validity: Option<&Bitmap>, address: &[*const u8], - select_index: &mut SelectVector, + select_vector: &mut SelectVector, + temp_vector: &mut SelectVector, count: &mut usize, validity_offset: usize, col_offset: usize, @@ -392,49 +434,87 @@ unsafe fn row_match_column_type( let mut match_count = 0; let mut equal: bool; - if let Some(validity) = validity { - for i in 0..*count { - let idx = select_index.get_index(i); - let isnull = !validity.get_bit(idx); - + let is_all_set = validity.unset_bits() == 0; + for idx in select_vector[..*count].iter() { + let idx = *idx; let validity_address = address[idx].add(validity_offset); - let isnull2 = load::(validity_address) != 0; - - equal = isnull == isnull2; - if !isnull && !isnull2 { + let is_set2 = read::(validity_address as _) != 0; + let is_set = is_all_set || validity.get_bit_unchecked(idx); + if is_set && is_set2 { let address = address[idx].add(col_offset); - let scalar = load::<::Scalar>(address); + let scalar = read::<::Scalar>(address as _); let value = T::index_column_unchecked(&col, idx); let value = T::to_owned_scalar(value); + equal = scalar.eq(&value); + } else { + equal = is_set == is_set2; } if equal { - select_index.set_index(match_count, idx); + temp_vector[match_count] = idx; match_count += 1; } else { - no_match.set_index(*no_match_count, idx); + no_match[*no_match_count] = idx; *no_match_count += 1; } } } else { - for i in 0..*count { - let idx = select_index.get_index(i); + for idx in select_vector[..*count].iter() { + let idx = *idx; let value = T::index_column_unchecked(&col, idx); let address = address[idx].add(col_offset); - let scalar = load::<::Scalar>(address); + let scalar = read::<::Scalar>(address as _); let value = T::to_owned_scalar(value); if scalar.eq(&value) { - select_index.set_index(match_count, idx); + temp_vector[match_count] = idx; match_count += 1; } else { - no_match.set_index(*no_match_count, idx); + no_match[*no_match_count] = idx; *no_match_count += 1; } } } + select_vector.clone_from_slice(temp_vector); + *count = match_count; +} + +unsafe fn row_match_generic_column( + col: &Column, + address: &[*const u8], + select_vector: &mut SelectVector, + temp_vector: &mut SelectVector, + count: &mut usize, + col_offset: usize, + no_match: &mut SelectVector, + no_match_count: &mut usize, +) { + let mut match_count = 0; + + for idx in select_vector[..*count].iter() { + let idx = *idx; + let len_address = address[idx].add(col_offset); + let len = read::(len_address as _) as usize; + + let address = address[idx].add(col_offset + 4); + + let value = AnyType::index_column_unchecked(col, idx); + let data_address = read::(address as _) as usize as *const u8; + + let scalar = std::slice::from_raw_parts(data_address, len); + let scalar: Scalar = bincode_deserialize_from_slice(scalar).unwrap(); + + if scalar.as_ref() == value { + temp_vector[match_count] = idx; + match_count += 1; + } else { + no_match[*no_match_count] = idx; + *no_match_count += 1; + } + } + select_vector.clone_from_slice(temp_vector); *count = match_count; } diff --git a/src/query/expression/src/aggregate/probe_state.rs b/src/query/expression/src/aggregate/probe_state.rs index 16a1b83c10da..bdeb5f0854a9 100644 --- a/src/query/expression/src/aggregate/probe_state.rs +++ b/src/query/expression/src/aggregate/probe_state.rs @@ -12,60 +12,62 @@ // See the License for the specific language governing permissions and // limitations under the License. -use crate::select_vector::SelectVector; -use crate::Column; +use crate::new_sel; +use crate::SelectVector; use crate::StateAddr; +use crate::BATCH_SIZE; /// ProbeState is the state to probe HT /// It could be reuse during multiple probe process -#[derive(Default, Debug)] pub struct ProbeState { - pub ht_offsets: Vec, - pub hash_salts: Vec, - pub addresses: Vec<*const u8>, - pub state_places: Vec, + pub group_hashes: [u64; BATCH_SIZE], + pub addresses: [*const u8; BATCH_SIZE], + pub state_places: [StateAddr; BATCH_SIZE], pub group_compare_vector: SelectVector, pub no_match_vector: SelectVector, pub empty_vector: SelectVector, - pub new_groups: SelectVector, - - pub group_columns: Vec, + pub temp_vector: SelectVector, pub row_count: usize, + + pub partition_entries: Vec, + pub partition_count: Vec, } -impl ProbeState { - pub fn adjust_group_columns( - &mut self, - group_columns: &[Column], - hashes: &[u64], - row_count: usize, - ht_size: usize, - ) { - self.group_columns = group_columns.to_owned(); - self.adjust_row_count(row_count); +impl Default for ProbeState { + fn default() -> Self { + Self { + group_hashes: [0_u64; BATCH_SIZE], + addresses: [std::ptr::null::(); BATCH_SIZE], + state_places: [StateAddr::new(0); BATCH_SIZE], + group_compare_vector: new_sel(), + no_match_vector: new_sel(), + empty_vector: new_sel(), + temp_vector: new_sel(), + partition_entries: vec![], + partition_count: vec![], + row_count: 0, + } + } +} + +unsafe impl Send for ProbeState {} +unsafe impl Sync for ProbeState {} - for ((hash, salt), ht_offset) in hashes - .iter() - .zip(self.hash_salts.iter_mut()) - .zip(self.ht_offsets.iter_mut()) - { - *salt = (*hash >> (64 - 16)) as u16; - *ht_offset = (*hash & (ht_size as u64 - 1)) as usize; +impl ProbeState { + pub fn set_incr_empty_vector(&mut self, row_count: usize) { + for i in 0..row_count { + self.empty_vector[i] = i; } } - pub fn adjust_row_count(&mut self, row_count: usize) { - if self.row_count < row_count { - self.ht_offsets.resize(row_count, 0); - self.hash_salts.resize(row_count, 0); - self.addresses.resize(row_count, std::ptr::null::()); - self.state_places.resize(row_count, StateAddr::new(0)); + pub fn reset_partitions(&mut self, partition_count: usize) { + if self.partition_entries.len() < partition_count { + self.partition_entries.resize(partition_count, new_sel()); + self.partition_count.resize(partition_count, 0); + } - self.group_compare_vector.resize(row_count); - self.no_match_vector.resize(row_count); - self.empty_vector.resize(row_count); - self.new_groups.resize(row_count); + for i in 0..partition_count { + self.partition_count[i] = 0; } - self.row_count = row_count; } } diff --git a/src/query/expression/src/kernels/utils.rs b/src/query/expression/src/kernels/utils.rs index 4b23be7ba0a9..d74bb15ba2d9 100644 --- a/src/query/expression/src/kernels/utils.rs +++ b/src/query/expression/src/kernels/utils.rs @@ -66,19 +66,17 @@ pub unsafe fn set_vec_len_by_ptr(vec: &mut Vec, ptr: *const T) { } /// # Safety -/// # As: copy_nonoverlapping +/// # As: core::ptr::copy_nonoverlapping #[inline] -pub unsafe fn store(val: &T, ptr: *mut u8) { - std::ptr::copy_nonoverlapping(val as *const T as *const u8, ptr, std::mem::size_of::()); +pub unsafe fn store(val: &T, ptr: *mut u8) { + core::ptr::copy_nonoverlapping(val as *const T as *const u8, ptr, std::mem::size_of::()); } /// # Safety -/// # As: copy_nonoverlapping +/// # As: core::ptr::read_unaligned #[inline] -pub unsafe fn load(ptr: *const u8) -> T { - let mut ret: T = std::mem::zeroed(); - std::ptr::copy_nonoverlapping(ptr as *const T, &mut ret, 1); - ret +pub unsafe fn read(ptr: *const u8) -> T { + core::ptr::read_unaligned::(ptr as _) } /// Iterates over an arbitrarily aligned byte buffer diff --git a/src/query/expression/src/lib.rs b/src/query/expression/src/lib.rs index 37b2decd8823..20cabe4a971f 100755 --- a/src/query/expression/src/lib.rs +++ b/src/query/expression/src/lib.rs @@ -14,6 +14,7 @@ #![allow(clippy::uninlined_format_args)] #![allow(clippy::len_without_is_empty)] +#![allow(clippy::arc_with_non_send_sync)] // FIXME: we should avoid this by implementing Ord correctly. #![allow(clippy::non_canonical_partial_ord_impl)] #![allow(incomplete_features)] diff --git a/src/query/expression/src/types/decimal.rs b/src/query/expression/src/types/decimal.rs index 13b613459a11..78d91023104d 100644 --- a/src/query/expression/src/types/decimal.rs +++ b/src/query/expression/src/types/decimal.rs @@ -342,6 +342,8 @@ pub trait Decimal: fn checked_mul(self, rhs: Self) -> Option; fn checked_rem(self, rhs: Self) -> Option; + fn do_round_div(self, rhs: Self, mul: Self) -> Option; + fn min_for_precision(precision: u8) -> Self; fn max_for_precision(precision: u8) -> Self; @@ -436,6 +438,16 @@ impl Decimal for i128 { self.checked_rem(rhs) } + fn do_round_div(self, rhs: Self, mul: Self) -> Option { + if self.is_negative() == rhs.is_negative() { + let res = (i256::from(self) * i256::from(mul) + i256::from(rhs) / 2) / i256::from(rhs); + Some(*res.low()) + } else { + let res = (i256::from(self) * i256::from(mul) - i256::from(rhs) / 2) / i256::from(rhs); + Some(*res.low()) + } + } + fn min_for_precision(to_precision: u8) -> Self { MIN_DECIMAL_FOR_EACH_PRECISION[to_precision as usize - 1] } @@ -636,6 +648,14 @@ impl Decimal for i256 { self.checked_rem(rhs) } + fn do_round_div(self, rhs: Self, mul: Self) -> Option { + if self.is_negative() == rhs.is_negative() { + self.checked_mul(mul).map(|x| (x + rhs / 2) / rhs) + } else { + self.checked_mul(mul).map(|x| (x - rhs / 2) / rhs) + } + } + fn min_for_precision(to_precision: u8) -> Self { MIN_DECIMAL256_BYTES_FOR_EACH_PRECISION[to_precision as usize - 1] } diff --git a/src/query/expression/src/types/string.rs b/src/query/expression/src/types/string.rs index 2b3063bd5bf2..0aac805d5b4b 100644 --- a/src/query/expression/src/types/string.rs +++ b/src/query/expression/src/types/string.rs @@ -110,7 +110,7 @@ impl ValueType for StringType { col.index(index) } - #[inline(always)] + #[inline] unsafe fn index_column_unchecked(col: &Self::Column, index: usize) -> Self::ScalarRef<'_> { col.index_unchecked(index) } diff --git a/src/query/expression/src/utils/block_debug.rs b/src/query/expression/src/utils/block_debug.rs index d6be3d8afd48..ac0902844c53 100644 --- a/src/query/expression/src/utils/block_debug.rs +++ b/src/query/expression/src/utils/block_debug.rs @@ -60,6 +60,19 @@ pub fn assert_block_value_eq(a: &DataBlock, b: &DataBlock) { } } +pub fn assert_block_value_sort_eq(a: &DataBlock, b: &DataBlock) { + assert!(a.num_columns() == b.num_columns()); + assert!(a.num_rows() == b.num_rows()); + + let a = pretty_format_blocks(&[a.clone()]).unwrap(); + let b = pretty_format_blocks(&[b.clone()]).unwrap(); + + let a: Vec<&str> = get_lines(&a); + let b: Vec<&str> = get_lines(&b); + + assert_eq!(a, b); +} + /// Assert with order insensitive. /// ['a', 'b'] equals ['b', 'a'] pub fn assert_blocks_sorted_eq_with_name(test_name: &str, expect: Vec<&str>, blocks: &[DataBlock]) { @@ -72,6 +85,16 @@ pub fn assert_blocks_sorted_eq_with_name(test_name: &str, expect: Vec<&str>, blo } let formatted = pretty_format_blocks(blocks).unwrap(); + let actual_lines: Vec<&str> = get_lines(&formatted); + + assert_eq!( + expected_lines, actual_lines, + "{:#?}\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", + test_name, expected_lines, actual_lines + ); +} + +fn get_lines(formatted: &str) -> Vec<&str> { let mut actual_lines: Vec<&str> = formatted.trim().lines().collect(); // sort except for header + footer @@ -79,12 +102,7 @@ pub fn assert_blocks_sorted_eq_with_name(test_name: &str, expect: Vec<&str>, blo if num_lines > 3 { actual_lines.as_mut_slice()[2..num_lines - 1].sort_unstable() } - - assert_eq!( - expected_lines, actual_lines, - "{:#?}\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - test_name, expected_lines, actual_lines - ); + actual_lines } /// Assert with order insensitive. diff --git a/src/query/expression/src/utils/mod.rs b/src/query/expression/src/utils/mod.rs index 7da770b977ae..53a64b849d91 100644 --- a/src/query/expression/src/utils/mod.rs +++ b/src/query/expression/src/utils/mod.rs @@ -20,7 +20,6 @@ mod column_from; pub mod date_helper; pub mod display; pub mod filter_helper; -pub mod select_vector; pub mod serialize; pub mod udf_client; pub mod variant_transform; diff --git a/src/query/functions/src/scalars/decimal/arithmetic.rs b/src/query/functions/src/scalars/decimal/arithmetic.rs index 7bc9b63b20b9..088595e88e95 100644 --- a/src/query/functions/src/scalars/decimal/arithmetic.rs +++ b/src/query/functions/src/scalars/decimal/arithmetic.rs @@ -88,7 +88,6 @@ macro_rules! binary_decimal { let scale_a = $left.scale(); let scale_b = $right.scale(); - // Note: the result scale is always larger than the left scale let scale_mul = scale_b + $size.scale - scale_a; let multiplier = T::e(scale_mul as u32); @@ -102,10 +101,17 @@ macro_rules! binary_decimal { if std::intrinsics::unlikely(b == zero) { ctx.set_error(result.len(), "divided by zero"); result.push(one); - } else if a.is_negative() == b.is_negative() { - result.push((a * multiplier + b / 2).div(b)); } else { - result.push((a * multiplier - b / 2).div(b)); + match a.do_round_div(b, multiplier) { + Some(t) => result.push(t), + None => { + ctx.set_error( + result.len(), + concat!("Decimal overflow at line : ", line!()), + ); + result.push(one); + } + } } }; diff --git a/src/query/functions/tests/it/aggregates/agg_hashtable.rs b/src/query/functions/tests/it/aggregates/agg_hashtable.rs index 4e92c0ef2f8b..6f00b119c004 100644 --- a/src/query/functions/tests/it/aggregates/agg_hashtable.rs +++ b/src/query/functions/tests/it/aggregates/agg_hashtable.rs @@ -29,6 +29,7 @@ use std::sync::Arc; use bumpalo::Bump; +use databend_common_expression::block_debug::assert_block_value_sort_eq; use databend_common_expression::types::ArgType; use databend_common_expression::types::BooleanType; use databend_common_expression::types::Float32Type; @@ -45,12 +46,13 @@ use databend_common_expression::AggregateHashTable; use databend_common_expression::Column; use databend_common_expression::DataBlock; use databend_common_expression::FromData; +use databend_common_expression::HashTableConfig; use databend_common_expression::PayloadFlushState; use databend_common_expression::ProbeState; use databend_common_functions::aggregates::AggregateFunctionFactory; use itertools::Itertools; -// cargo test --package common-functions --test it -- aggregates::agg_hashtable::test_agg_hashtable --exact --nocapture +// cargo test --package databend-common-functions --test it -- aggregates::agg_hashtable::test_agg_hashtable --exact --nocapture #[test] fn test_agg_hashtable() { let factory = AggregateFunctionFactory::instance(); @@ -87,20 +89,29 @@ fn test_agg_hashtable() { let params: Vec> = aggrs.iter().map(|_| vec![columns[1].clone()]).collect(); - let arena1 = Arc::new(Bump::new()); - let mut hashtable = AggregateHashTable::new(arena1, group_types.clone(), aggrs.clone()); + let config = HashTableConfig::default(); + let mut hashtable = AggregateHashTable::new( + group_types.clone(), + aggrs.clone(), + config.clone(), + Arc::new(Bump::new()), + ); let mut state = ProbeState::default(); let _ = hashtable - .add_groups(&mut state, &group_columns, ¶ms, n) + .add_groups(&mut state, &group_columns, ¶ms, &[], n) .unwrap(); - let arena2 = Arc::new(Bump::new()); - let mut hashtable2 = AggregateHashTable::new(arena2, group_types.clone(), aggrs.clone()); + let mut hashtable2 = AggregateHashTable::new( + group_types.clone(), + aggrs.clone(), + config.clone(), + Arc::new(Bump::new()), + ); let mut state2 = ProbeState::default(); let _ = hashtable2 - .add_groups(&mut state2, &group_columns, ¶ms, n) + .add_groups(&mut state2, &group_columns, ¶ms, &[], n) .unwrap(); let mut flush_state = PayloadFlushState::default(); @@ -112,8 +123,8 @@ fn test_agg_hashtable() { loop { match hashtable.merge_result(&mut merge_state) { Ok(true) => { - let mut columns = merge_state.group_columns.clone(); - columns.extend_from_slice(&merge_state.aggregate_results); + let mut columns = merge_state.take_group_columns(); + columns.extend_from_slice(&merge_state.take_aggregate_results()); let block = DataBlock::new_from_columns(columns); blocks.push(block); @@ -145,9 +156,8 @@ fn test_agg_hashtable() { UInt64Type::from_data(vec![urows / 2, urows / 2, urows / 2, urows / 2]), ]); - for (column, expected) in block.columns().iter().zip(expected_results.iter()) { - let column = column.value.as_column().unwrap(); - assert_eq!(column, expected) - } + let block_expected = DataBlock::new_from_columns(expected_results.clone()); + + assert_block_value_sort_eq(&block, &block_expected); } } diff --git a/src/query/pipeline/core/src/processors/profiles.rs b/src/query/pipeline/core/src/processors/profiles.rs index b9119343417c..66b157a18972 100644 --- a/src/query/pipeline/core/src/processors/profiles.rs +++ b/src/query/pipeline/core/src/processors/profiles.rs @@ -41,6 +41,15 @@ pub enum ProfileStatisticsName { SpillReadTime, } +#[derive(Clone, Hash, Eq, PartialEq, serde::Serialize, serde::Deserialize, Debug)] +pub enum StatisticsUnit { + Rows, + Bytes, + NanoSeconds, + MillisSeconds, + Count, +} + impl Display for ProfileStatisticsName { fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { write!(f, "{:?}", self) @@ -67,6 +76,8 @@ pub struct ProfileDesc { desc: &'static str, display_name: &'static str, index: usize, + unit: StatisticsUnit, + plain_statistics: bool, } pub static PROFILES_DESC: OnceCell>> = @@ -99,76 +110,106 @@ pub fn get_statistics_desc() -> Arc> display_name: "cpu time", desc: "The time spent to process in nanoseconds", index: ProfileStatisticsName::CpuTime as usize, + unit: StatisticsUnit::NanoSeconds, + plain_statistics: false, }), (ProfileStatisticsName::WaitTime, ProfileDesc { display_name: "wait time", desc: "The time spent to wait in nanoseconds, usually used to measure the time spent on waiting for I/O", index: ProfileStatisticsName::WaitTime as usize, + unit: StatisticsUnit::NanoSeconds, + plain_statistics: false, }), (ProfileStatisticsName::ExchangeRows, ProfileDesc { display_name: "exchange rows", desc: "The number of data rows exchange between nodes in cluster mode", index: ProfileStatisticsName::ExchangeRows as usize, + unit: StatisticsUnit::Rows, + plain_statistics: true, }), (ProfileStatisticsName::ExchangeBytes, ProfileDesc { display_name: "exchange bytes", desc: "The number of data bytes exchange between nodes in cluster mode", index: ProfileStatisticsName::ExchangeBytes as usize, + unit: StatisticsUnit::Bytes, + plain_statistics: true, }), (ProfileStatisticsName::OutputRows, ProfileDesc { display_name: "output rows", desc: "The number of rows from the physical plan output to the next physical plan", index: ProfileStatisticsName::OutputRows as usize, + unit: StatisticsUnit::Rows, + plain_statistics: true, }), (ProfileStatisticsName::OutputBytes, ProfileDesc { display_name: "output bytes", desc: "The number of bytes from the physical plan output to the next physical plan", index: ProfileStatisticsName::OutputBytes as usize, + unit: StatisticsUnit::Bytes, + plain_statistics: true, }), (ProfileStatisticsName::ScanBytes, ProfileDesc { display_name: "bytes scanned", desc: "The bytes scanned of query", index: ProfileStatisticsName::ScanBytes as usize, + unit: StatisticsUnit::Bytes, + plain_statistics: true, }), (ProfileStatisticsName::ScanCacheBytes, ProfileDesc { display_name: "bytes scanned from cache", desc: "The bytes scanned from cache of query", index: ProfileStatisticsName::ScanCacheBytes as usize, + unit: StatisticsUnit::Bytes, + plain_statistics: true, }), (ProfileStatisticsName::ScanPartitions, ProfileDesc { display_name: "partitions scanned", desc: "The partitions scanned of query", index: ProfileStatisticsName::ScanPartitions as usize, + unit: StatisticsUnit::Count, + plain_statistics: true, }), (ProfileStatisticsName::SpillWriteCount, ProfileDesc { display_name: "numbers spilled by write", desc: "The number of spilled by write", index: ProfileStatisticsName::SpillWriteCount as usize, + unit: StatisticsUnit::Count, + plain_statistics: true, }), (ProfileStatisticsName::SpillWriteBytes, ProfileDesc { display_name: "bytes spilled by write", desc: "The bytes spilled by write", index: ProfileStatisticsName::SpillWriteBytes as usize, + unit: StatisticsUnit::Bytes, + plain_statistics: true, }), (ProfileStatisticsName::SpillWriteTime, ProfileDesc { display_name: "spilled time by write", desc: "The time spent to write spill in millisecond", index: ProfileStatisticsName::SpillWriteTime as usize, + unit: StatisticsUnit::MillisSeconds, + plain_statistics: true, }), (ProfileStatisticsName::SpillReadCount, ProfileDesc { display_name: "numbers spilled by read", desc: "The number of spilled by read", index: ProfileStatisticsName::SpillReadCount as usize, + unit: StatisticsUnit::Count, + plain_statistics: true, }), (ProfileStatisticsName::SpillReadBytes, ProfileDesc { display_name: "bytes spilled by read", desc: "The bytes spilled by read", index: ProfileStatisticsName::SpillReadBytes as usize, + unit: StatisticsUnit::Bytes, + plain_statistics: true, }), (ProfileStatisticsName::SpillReadTime, ProfileDesc { display_name: "spilled time by read", desc: "The time spent to read spill in millisecond", index: ProfileStatisticsName::SpillReadTime as usize, + unit: StatisticsUnit::MillisSeconds, + plain_statistics: true, }), ])) }).clone() diff --git a/src/query/service/src/pipelines/builders/builder_aggregate.rs b/src/query/service/src/pipelines/builders/builder_aggregate.rs index d79e905ea637..13db1fdd6be3 100644 --- a/src/query/service/src/pipelines/builders/builder_aggregate.rs +++ b/src/query/service/src/pipelines/builders/builder_aggregate.rs @@ -22,6 +22,7 @@ use databend_common_expression::AggregateFunctionRef; use databend_common_expression::DataBlock; use databend_common_expression::DataSchemaRef; use databend_common_expression::HashMethodKind; +use databend_common_expression::HashTableConfig; use databend_common_functions::aggregates::AggregateFunctionFactory; use databend_common_pipeline_core::processors::ProcessorPtr; use databend_common_pipeline_core::query_spill_prefix; @@ -98,10 +99,22 @@ impl PipelineBuilder { pub(crate) fn build_aggregate_partial(&mut self, aggregate: &AggregatePartial) -> Result<()> { self.build_pipeline(&aggregate.input)?; + let max_block_size = self.settings.get_max_block_size()?; + let max_threads = self.settings.get_max_threads()?; + + let enable_experimental_aggregate_hashtable = self + .settings + .get_enable_experimental_aggregate_hashtable()?; + + let in_cluster = !self.ctx.get_cluster().is_empty(); + let params = Self::build_aggregator_params( aggregate.input.output_schema()?, &aggregate.group_by, &aggregate.agg_funcs, + enable_experimental_aggregate_hashtable, + in_cluster, + max_block_size as usize, None, )?; @@ -120,6 +133,14 @@ impl PipelineBuilder { let sample_block = DataBlock::empty_with_schema(schema_before_group_by); let method = DataBlock::choose_hash_method(&sample_block, group_cols, efficiently_memory)?; + // Need a global atomic to read the max current radix bits hint + let partial_agg_config = if self.ctx.get_cluster().is_empty() { + HashTableConfig::default().with_partial(true, max_threads as usize) + } else { + HashTableConfig::default() + .cluster_with_partial(true, self.ctx.get_cluster().nodes.len()) + }; + self.main_pipeline.add_transform(|input, output| { Ok(ProcessorPtr::create( match params.aggregate_functions.is_empty() { @@ -129,7 +150,8 @@ impl PipelineBuilder { method, input, output, - params.clone() + params.clone(), + partial_agg_config.clone() ), }), false => with_mappedhash_method!(|T| match method.clone() { @@ -138,7 +160,8 @@ impl PipelineBuilder { method, input, output, - params.clone() + params.clone(), + partial_agg_config.clone() ), }), }?, @@ -193,10 +216,18 @@ impl PipelineBuilder { } pub(crate) fn build_aggregate_final(&mut self, aggregate: &AggregateFinal) -> Result<()> { + let max_block_size = self.settings.get_max_block_size()?; + let enable_experimental_aggregate_hashtable = self + .settings + .get_enable_experimental_aggregate_hashtable()?; + let in_cluster = !self.ctx.get_cluster().is_empty(); let params = Self::build_aggregator_params( aggregate.before_group_by_schema.clone(), &aggregate.group_by, &aggregate.agg_funcs, + enable_experimental_aggregate_hashtable, + in_cluster, + max_block_size as usize, aggregate.limit, )?; @@ -260,6 +291,9 @@ impl PipelineBuilder { input_schema: DataSchemaRef, group_by: &[IndexType], agg_funcs: &[AggregateFunctionDesc], + enable_experimental_aggregate_hashtable: bool, + in_cluster: bool, + max_block_size: usize, limit: Option, ) -> Result> { let mut agg_args = Vec::with_capacity(agg_funcs.len()); @@ -299,6 +333,9 @@ impl PipelineBuilder { &group_by, &aggs, &agg_args, + enable_experimental_aggregate_hashtable, + in_cluster, + max_block_size, limit, )?; diff --git a/src/query/service/src/pipelines/executor/executor_graph.rs b/src/query/service/src/pipelines/executor/executor_graph.rs index e2d416fba33e..806157100e3f 100644 --- a/src/query/service/src/pipelines/executor/executor_graph.rs +++ b/src/query/service/src/pipelines/executor/executor_graph.rs @@ -15,6 +15,7 @@ use std::collections::VecDeque; use std::fmt::Debug; use std::fmt::Formatter; +use std::sync::atomic::AtomicU64; use std::sync::atomic::AtomicUsize; use std::sync::atomic::Ordering; use std::sync::Arc; @@ -37,11 +38,14 @@ use petgraph::prelude::NodeIndex; use petgraph::prelude::StableGraph; use petgraph::Direction; +use crate::pipelines::executor::processor_async_task::ExecutorTasksQueue; use crate::pipelines::executor::ExecutorTask; -use crate::pipelines::executor::ExecutorTasksQueue; use crate::pipelines::executor::ExecutorWorkerContext; -use crate::pipelines::executor::PipelineExecutor; use crate::pipelines::executor::ProcessorAsyncTask; +use crate::pipelines::executor::QueriesExecutorTasksQueue; +use crate::pipelines::executor::QueriesPipelineExecutor; +use crate::pipelines::executor::QueryExecutorTasksQueue; +use crate::pipelines::executor::QueryPipelineExecutor; use crate::pipelines::executor::WorkersCondvar; use crate::pipelines::processors::connect; use crate::pipelines::processors::DirectedEdge; @@ -102,24 +106,36 @@ impl Node { } } +const POINTS_MASK: u64 = 0xFFFFFFFF00000000; +const EPOCH_MASK: u64 = 0x00000000FFFFFFFF; + +// TODO: Replace with a variable, not a const value +const MAX_POINTS: u64 = 3; + struct ExecutingGraph { finished_nodes: AtomicUsize, graph: StableGraph, EdgeInfo>, + /// points store two values + /// + /// - the high 32 bit store the number of points that can be consumed + /// - the low 32 bit store this points belong to which epoch + points: AtomicU64, } type StateLockGuard = ExecutingGraph; impl ExecutingGraph { - pub fn create(mut pipeline: Pipeline) -> Result { + pub fn create(mut pipeline: Pipeline, init_epoch: u32) -> Result { let mut graph = StableGraph::new(); Self::init_graph(&mut pipeline, &mut graph); Ok(ExecutingGraph { graph, finished_nodes: AtomicUsize::new(0), + points: AtomicU64::new((MAX_POINTS << 32) | init_epoch as u64), }) } - pub fn from_pipelines(mut pipelines: Vec) -> Result { + pub fn from_pipelines(mut pipelines: Vec, init_epoch: u32) -> Result { let mut graph = StableGraph::new(); for pipeline in &mut pipelines { @@ -129,6 +145,7 @@ impl ExecutingGraph { Ok(ExecutingGraph { finished_nodes: AtomicUsize::new(0), graph, + points: AtomicU64::new((MAX_POINTS << 32) | init_epoch as u64), }) } @@ -228,10 +245,11 @@ impl ExecutingGraph { pub unsafe fn init_schedule_queue( locker: &StateLockGuard, capacity: usize, + graph: &Arc, ) -> Result { let mut schedule_queue = ScheduleQueue::with_capacity(capacity); for sink_index in locker.graph.externals(Direction::Outgoing) { - ExecutingGraph::schedule_queue(locker, sink_index, &mut schedule_queue)?; + ExecutingGraph::schedule_queue(locker, sink_index, &mut schedule_queue, graph)?; } Ok(schedule_queue) @@ -244,6 +262,7 @@ impl ExecutingGraph { locker: &StateLockGuard, index: NodeIndex, schedule_queue: &mut ScheduleQueue, + graph: &Arc, ) -> Result<()> { let mut need_schedule_nodes = VecDeque::new(); let mut need_schedule_edges = VecDeque::new(); @@ -304,11 +323,17 @@ impl ExecutingGraph { } Event::NeedData | Event::NeedConsume => State::Idle, Event::Sync => { - schedule_queue.push_sync(node.processor.clone()); + schedule_queue.push_sync(ProcessorWrapper { + processor: node.processor.clone(), + graph: graph.clone(), + }); State::Processing } Event::Async => { - schedule_queue.push_async(node.processor.clone()); + schedule_queue.push_async(ProcessorWrapper { + processor: node.processor.clone(), + graph: graph.clone(), + }); State::Processing } }; @@ -320,11 +345,48 @@ impl ExecutingGraph { Ok(()) } + + /// Checks if a task can be performed in the current epoch, consuming a point if possible. + pub fn can_perform_task(&self, global_epoch: u32, max_points: u64) -> bool { + let mut expected_value = 0; + let mut desired_value = 0; + loop { + match self.points.compare_exchange_weak( + expected_value, + desired_value, + Ordering::SeqCst, + Ordering::Relaxed, + ) { + Ok(old_value) => { + return (old_value & EPOCH_MASK) as u32 == global_epoch; + } + Err(new_expected) => { + let remain_points = (new_expected & POINTS_MASK) >> 32; + let epoch = new_expected & EPOCH_MASK; + + expected_value = new_expected; + if epoch != global_epoch as u64 { + desired_value = new_expected; + } else if remain_points >= 1 { + desired_value = (remain_points - 1) << 32 | epoch; + } else { + desired_value = max_points << 32 | (epoch + 1); + } + } + } + } + } +} + +#[derive(Clone)] +pub struct ProcessorWrapper { + pub processor: ProcessorPtr, + pub graph: Arc, } pub struct ScheduleQueue { - pub sync_queue: VecDeque, - pub async_queue: VecDeque, + pub sync_queue: VecDeque, + pub async_queue: VecDeque, } impl ScheduleQueue { @@ -336,30 +398,20 @@ impl ScheduleQueue { } #[inline] - pub fn push_sync(&mut self, processor: ProcessorPtr) { + pub fn push_sync(&mut self, processor: ProcessorWrapper) { self.sync_queue.push_back(processor); } #[inline] - pub fn push_async(&mut self, processor: ProcessorPtr) { + pub fn push_async(&mut self, processor: ProcessorWrapper) { self.async_queue.push_back(processor); } - pub fn schedule_tail(mut self, global: &ExecutorTasksQueue, ctx: &mut ExecutorWorkerContext) { - let mut tasks = VecDeque::with_capacity(self.sync_queue.len()); - - while let Some(processor) = self.sync_queue.pop_front() { - tasks.push_back(ExecutorTask::Sync(processor)); - } - - global.push_tasks(ctx, tasks) - } - pub fn schedule( mut self, - global: &Arc, + global: &Arc, context: &mut ExecutorWorkerContext, - executor: &Arc, + executor: &Arc, ) { debug_assert!(!context.has_task()); @@ -384,28 +436,28 @@ impl ScheduleQueue { } pub fn schedule_async_task( - proc: ProcessorPtr, + proc: ProcessorWrapper, query_id: Arc, - executor: &Arc, + executor: &Arc, wakeup_worker_id: usize, workers_condvar: Arc, - global_queue: Arc, + global_queue: Arc, ) { unsafe { workers_condvar.inc_active_async_worker(); - let weak_executor = Arc::downgrade(executor); - let node_profile = executor.graph.get_node_profile(proc.id()).clone(); - let process_future = proc.async_process(); + let graph = proc.graph; + let node_profile = executor.graph.get_node_profile(proc.processor.id()).clone(); + let process_future = proc.processor.async_process(); executor.async_runtime.spawn( query_id.as_ref().clone(), TrackedFuture::create(ProcessorAsyncTask::create( query_id, wakeup_worker_id, - proc.clone(), - global_queue, + proc.processor.clone(), + Arc::new(ExecutorTasksQueue::QueryExecutorTasksQueue(global_queue)), workers_condvar, - weak_executor, node_profile, + graph, process_future, )) .in_span(Span::enter_with_local_parent(std::any::type_name::< @@ -415,41 +467,149 @@ impl ScheduleQueue { } } - fn schedule_sync(&mut self, _: &ExecutorTasksQueue, ctx: &mut ExecutorWorkerContext) { + fn schedule_sync(&mut self, _: &QueryExecutorTasksQueue, ctx: &mut ExecutorWorkerContext) { if let Some(processor) = self.sync_queue.pop_front() { ctx.set_task(ExecutorTask::Sync(processor)); } } + + pub fn schedule_tail( + mut self, + global: &QueryExecutorTasksQueue, + ctx: &mut ExecutorWorkerContext, + ) { + let mut tasks = VecDeque::with_capacity(self.sync_queue.len()); + + while let Some(processor) = self.sync_queue.pop_front() { + tasks.push_back(ExecutorTask::Sync(processor)); + } + + global.push_tasks(ctx, tasks) + } + + pub fn schedule_with_condition( + mut self, + global: &Arc, + context: &mut ExecutorWorkerContext, + executor: &Arc, + ) { + debug_assert!(!context.has_task()); + + while let Some(processor) = self.async_queue.pop_front() { + if processor + .graph + .can_perform_task(executor.epoch.load(Ordering::SeqCst), MAX_POINTS) + { + Self::schedule_async_task_with_condition( + processor, + context.query_id.clone(), + executor, + context.get_worker_id(), + context.get_workers_condvar().clone(), + global.clone(), + ) + } else { + let mut tasks = VecDeque::with_capacity(1); + tasks.push_back(ExecutorTask::Async(processor)); + global.push_tasks(context.get_worker_id(), None, tasks); + } + } + + if !self.sync_queue.is_empty() { + while let Some(processor) = self.sync_queue.pop_front() { + if processor + .graph + .can_perform_task(executor.epoch.load(Ordering::SeqCst), MAX_POINTS) + { + context.set_task(ExecutorTask::Sync(processor)); + break; + } else { + let mut tasks = VecDeque::with_capacity(1); + tasks.push_back(ExecutorTask::Sync(processor)); + global.push_tasks(context.get_worker_id(), None, tasks); + } + } + } + + if !self.sync_queue.is_empty() { + let mut current_tasks = VecDeque::with_capacity(self.sync_queue.len()); + let mut next_tasks = VecDeque::with_capacity(self.sync_queue.len()); + while let Some(processor) = self.sync_queue.pop_front() { + if processor + .graph + .can_perform_task(executor.epoch.load(Ordering::SeqCst), MAX_POINTS) + { + current_tasks.push_back(ExecutorTask::Sync(processor)); + } else { + next_tasks.push_back(ExecutorTask::Sync(processor)); + } + } + let worker_id = context.get_worker_id(); + global.push_tasks(worker_id, Some(current_tasks), next_tasks); + } + } + + pub fn schedule_async_task_with_condition( + proc: ProcessorWrapper, + query_id: Arc, + executor: &Arc, + wakeup_worker_id: usize, + workers_condvar: Arc, + global_queue: Arc, + ) { + unsafe { + workers_condvar.inc_active_async_worker(); + let graph = proc.graph; + let node_profile = executor.graph.get_node_profile(proc.processor.id()).clone(); + let process_future = proc.processor.async_process(); + executor.async_runtime.spawn( + query_id.as_ref().clone(), + ProcessorAsyncTask::create( + query_id, + wakeup_worker_id, + proc.processor.clone(), + Arc::new(ExecutorTasksQueue::QueriesExecutorTasksQueue(global_queue)), + workers_condvar, + node_profile, + graph, + process_future, + ) + .in_span(Span::enter_with_local_parent(std::any::type_name::< + ProcessorAsyncTask, + >())), + ); + } + } } pub struct RunningGraph(ExecutingGraph); impl RunningGraph { - pub fn create(pipeline: Pipeline) -> Result { - let graph_state = ExecutingGraph::create(pipeline)?; + pub fn create(pipeline: Pipeline, init_epoch: u32) -> Result> { + let graph_state = ExecutingGraph::create(pipeline, init_epoch)?; debug!("Create running graph:{:?}", graph_state); - Ok(RunningGraph(graph_state)) + Ok(Arc::new(RunningGraph(graph_state))) } - pub fn from_pipelines(pipelines: Vec) -> Result { - let graph_state = ExecutingGraph::from_pipelines(pipelines)?; + pub fn from_pipelines(pipelines: Vec, init_epoch: u32) -> Result> { + let graph_state = ExecutingGraph::from_pipelines(pipelines, init_epoch)?; debug!("Create running graph:{:?}", graph_state); - Ok(RunningGraph(graph_state)) + Ok(Arc::new(RunningGraph(graph_state))) } /// # Safety /// /// Method is thread unsafe and require thread safe call - pub unsafe fn init_schedule_queue(&self, capacity: usize) -> Result { - ExecutingGraph::init_schedule_queue(&self.0, capacity) + pub unsafe fn init_schedule_queue(self: Arc, capacity: usize) -> Result { + ExecutingGraph::init_schedule_queue(&self.0, capacity, &self) } /// # Safety /// /// Method is thread unsafe and require thread safe call - pub unsafe fn schedule_queue(&self, node_index: NodeIndex) -> Result { + pub unsafe fn schedule_queue(self: Arc, node_index: NodeIndex) -> Result { let mut schedule_queue = ScheduleQueue::with_capacity(0); - ExecutingGraph::schedule_queue(&self.0, node_index, &mut schedule_queue)?; + ExecutingGraph::schedule_queue(&self.0, node_index, &mut schedule_queue, &self)?; Ok(schedule_queue) } @@ -485,6 +645,11 @@ impl RunningGraph { } } + /// Checks if a task can be performed in the current epoch, consuming a point if possible. + pub fn can_perform_task(&self, global_epoch: u32, max_points: u64) -> bool { + self.0.can_perform_task(global_epoch, max_points) + } + pub fn format_graph_nodes(&self) -> String { pub struct NodeDisplay { id: usize, @@ -627,15 +792,15 @@ impl Debug for ScheduleQueue { for item in &self.sync_queue { sync_queue.push(QueueItem { - id: item.id().index(), - name: item.name().to_string(), + id: item.processor.id().index(), + name: item.processor.name().to_string(), }) } for item in &self.async_queue { async_queue.push(QueueItem { - id: item.id().index(), - name: item.name().to_string(), + id: item.processor.id().index(), + name: item.processor.name().to_string(), }) } diff --git a/src/query/service/src/pipelines/executor/executor_settings.rs b/src/query/service/src/pipelines/executor/executor_settings.rs index b45bc8465582..d78d391f2d79 100644 --- a/src/query/service/src/pipelines/executor/executor_settings.rs +++ b/src/query/service/src/pipelines/executor/executor_settings.rs @@ -20,6 +20,7 @@ use databend_common_settings::Settings; #[derive(Clone)] pub struct ExecutorSettings { + pub enable_new_executor: bool, pub query_id: Arc, pub max_execute_time_in_seconds: Duration, } @@ -28,6 +29,7 @@ impl ExecutorSettings { pub fn try_create(settings: &Settings, query_id: String) -> Result { let max_execute_time_in_seconds = settings.get_max_execute_time_in_seconds()?; Ok(ExecutorSettings { + enable_new_executor: settings.get_enable_experimental_new_executor()?, query_id: Arc::new(query_id), max_execute_time_in_seconds: Duration::from_secs(max_execute_time_in_seconds), }) diff --git a/src/query/service/src/pipelines/executor/executor_worker_context.rs b/src/query/service/src/pipelines/executor/executor_worker_context.rs index 9b430dc302e3..8b003f7d5a20 100644 --- a/src/query/service/src/pipelines/executor/executor_worker_context.rs +++ b/src/query/service/src/pipelines/executor/executor_worker_context.rs @@ -24,17 +24,40 @@ use databend_common_pipeline_core::processors::Profile; use databend_common_pipeline_core::processors::ProfileStatisticsName; use petgraph::prelude::NodeIndex; -use crate::pipelines::executor::CompletedAsyncTask; +use crate::pipelines::executor::executor_graph::ProcessorWrapper; use crate::pipelines::executor::RunningGraph; use crate::pipelines::executor::WorkersCondvar; -use crate::pipelines::processors::ProcessorPtr; pub enum ExecutorTask { None, - Sync(ProcessorPtr), + Sync(ProcessorWrapper), + Async(ProcessorWrapper), AsyncCompleted(CompletedAsyncTask), } +pub struct CompletedAsyncTask { + pub id: NodeIndex, + pub worker_id: usize, + pub res: Result<()>, + pub graph: Arc, +} + +impl CompletedAsyncTask { + pub fn create( + id: NodeIndex, + worker_id: usize, + res: Result<()>, + graph: Arc, + ) -> Self { + CompletedAsyncTask { + id, + worker_id, + res, + graph, + } + } +} + pub struct ExecutorWorkerContext { pub query_id: Arc, worker_id: usize, @@ -73,33 +96,33 @@ impl ExecutorWorkerContext { } /// # Safety - pub unsafe fn execute_task(&mut self, graph: &RunningGraph) -> Result { + pub unsafe fn execute_task(&mut self) -> Result)>> { match std::mem::replace(&mut self.task, ExecutorTask::None) { ExecutorTask::None => Err(ErrorCode::Internal("Execute none task.")), - ExecutorTask::Sync(processor) => self.execute_sync_task(processor, graph), + ExecutorTask::Sync(processor) => self.execute_sync_task(processor), ExecutorTask::AsyncCompleted(task) => match task.res { - Ok(_) => Ok(task.id), + Ok(_) => Ok(Some((task.id, task.graph))), Err(cause) => Err(cause), }, + ExecutorTask::Async(_) => unreachable!("used for new executor"), } } /// # Safety unsafe fn execute_sync_task( &mut self, - proc: ProcessorPtr, - graph: &RunningGraph, - ) -> Result { - Profile::track_profile(graph.get_node_profile(proc.id())); + proc: ProcessorWrapper, + ) -> Result)>> { + Profile::track_profile(proc.graph.get_node_profile(proc.processor.id())); let instant = Instant::now(); - proc.process()?; + proc.processor.process()?; let nanos = instant.elapsed().as_nanos(); assume(nanos < 18446744073709551615_u128); Profile::record_usize_profile(ProfileStatisticsName::CpuTime, nanos as usize); - Ok(proc.id()) + Ok(Some((proc.processor.id(), proc.graph))) } pub fn get_workers_condvar(&self) -> &Arc { @@ -115,8 +138,14 @@ impl Debug for ExecutorTask { ExecutorTask::Sync(p) => write!( f, "ExecutorTask::Sync {{ id: {}, name: {}}}", - p.id().index(), - p.name() + p.processor.id().index(), + p.processor.name() + ), + ExecutorTask::Async(p) => write!( + f, + "ExecutorTask::Async {{ id: {}, name: {}}}", + p.processor.id().index(), + p.processor.name() ), ExecutorTask::AsyncCompleted(_) => write!(f, "ExecutorTask::CompletedAsync"), } diff --git a/src/query/service/src/pipelines/executor/mod.rs b/src/query/service/src/pipelines/executor/mod.rs index ff21c2f3d82f..ae51827d83da 100644 --- a/src/query/service/src/pipelines/executor/mod.rs +++ b/src/query/service/src/pipelines/executor/mod.rs @@ -12,30 +12,36 @@ // See the License for the specific language governing permissions and // limitations under the License. -mod pipeline_executor; +mod query_pipeline_executor; mod executor_condvar; mod executor_graph; mod executor_settings; -mod executor_tasks; mod executor_worker_context; mod pipeline_complete_executor; +mod pipeline_executor; mod pipeline_pulling_executor; mod pipeline_pushing_executor; mod processor_async_task; +mod queries_executor_tasks; +mod queries_pipeline_executor; +mod query_executor_tasks; pub use databend_common_base::base::WatchNotify; pub use executor_condvar::WorkersCondvar; pub use executor_condvar::WorkersWaitingStatus; pub use executor_graph::RunningGraph; pub use executor_settings::ExecutorSettings; -pub use executor_tasks::CompletedAsyncTask; -pub use executor_tasks::ExecutorTasksQueue; +pub use executor_worker_context::CompletedAsyncTask; pub use executor_worker_context::ExecutorTask; pub use executor_worker_context::ExecutorWorkerContext; pub use pipeline_complete_executor::PipelineCompleteExecutor; -pub use pipeline_executor::FinishedCallback; pub use pipeline_executor::PipelineExecutor; pub use pipeline_pulling_executor::PipelinePullingExecutor; pub use pipeline_pushing_executor::PipelinePushingExecutor; pub use processor_async_task::ProcessorAsyncTask; +pub use queries_executor_tasks::QueriesExecutorTasksQueue; +pub use queries_pipeline_executor::QueriesPipelineExecutor; +pub use query_executor_tasks::QueryExecutorTasksQueue; +pub use query_pipeline_executor::FinishedCallback; +pub use query_pipeline_executor::QueryPipelineExecutor; diff --git a/src/query/service/src/pipelines/executor/pipeline_complete_executor.rs b/src/query/service/src/pipelines/executor/pipeline_complete_executor.rs index 60d970fa3a81..761865fe38ff 100644 --- a/src/query/service/src/pipelines/executor/pipeline_complete_executor.rs +++ b/src/query/service/src/pipelines/executor/pipeline_complete_executor.rs @@ -23,6 +23,8 @@ use minitrace::prelude::*; use crate::pipelines::executor::ExecutorSettings; use crate::pipelines::executor::PipelineExecutor; +use crate::pipelines::executor::QueriesPipelineExecutor; +use crate::pipelines::executor::QueryPipelineExecutor; pub struct PipelineCompleteExecutor { executor: Arc, @@ -39,9 +41,19 @@ impl PipelineCompleteExecutor { "Logical error, PipelineCompleteExecutor can only work on complete pipeline.", )); } + let executor = if settings.enable_new_executor { + PipelineExecutor::QueriesPipelineExecutor(QueriesPipelineExecutor::create( + pipeline, settings, + )?) + } else { + PipelineExecutor::QueryPipelineExecutor(QueryPipelineExecutor::create( + pipeline, settings, + )?) + }; - let executor = PipelineExecutor::create(pipeline, settings)?; - Ok(PipelineCompleteExecutor { executor }) + Ok(PipelineCompleteExecutor { + executor: Arc::new(executor), + }) } pub fn from_pipelines( @@ -55,9 +67,18 @@ impl PipelineCompleteExecutor { )); } } - - let executor = PipelineExecutor::from_pipelines(pipelines, settings)?; - Ok(Arc::new(PipelineCompleteExecutor { executor })) + let executor = if settings.enable_new_executor { + PipelineExecutor::QueriesPipelineExecutor(QueriesPipelineExecutor::from_pipelines( + pipelines, settings, + )?) + } else { + PipelineExecutor::QueryPipelineExecutor(QueryPipelineExecutor::from_pipelines( + pipelines, settings, + )?) + }; + Ok(Arc::new(PipelineCompleteExecutor { + executor: Arc::new(executor), + })) } pub fn get_inner(&self) -> Arc { diff --git a/src/query/service/src/pipelines/executor/pipeline_executor.rs b/src/query/service/src/pipelines/executor/pipeline_executor.rs index 027a865bbb17..681760384ceb 100644 --- a/src/query/service/src/pipelines/executor/pipeline_executor.rs +++ b/src/query/service/src/pipelines/executor/pipeline_executor.rs @@ -13,430 +13,52 @@ // limitations under the License. use std::sync::Arc; -use std::time::Instant; -use databend_common_base::base::tokio; -use databend_common_base::runtime::catch_unwind; -use databend_common_base::runtime::GlobalIORuntime; -use databend_common_base::runtime::Runtime; -use databend_common_base::runtime::Thread; -use databend_common_base::runtime::ThreadJoinHandle; -use databend_common_base::runtime::TrySpawn; -use databend_common_base::GLOBAL_TASK; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_pipeline_core::processors::Profile; -use databend_common_pipeline_core::LockGuard; -use databend_common_pipeline_core::Pipeline; -use futures::future::select; -use futures_util::future::Either; -use log::info; -use log::warn; -use log::LevelFilter; -use minitrace::full_name; -use minitrace::prelude::*; -use parking_lot::Mutex; -use petgraph::matrix_graph::Zero; -use crate::pipelines::executor::executor_graph::ScheduleQueue; -use crate::pipelines::executor::ExecutorSettings; -use crate::pipelines::executor::ExecutorTasksQueue; -use crate::pipelines::executor::ExecutorWorkerContext; -use crate::pipelines::executor::RunningGraph; -use crate::pipelines::executor::WatchNotify; -use crate::pipelines::executor::WorkersCondvar; +use crate::pipelines::executor::QueriesPipelineExecutor; +use crate::pipelines::executor::QueryPipelineExecutor; -pub type InitCallback = Box Result<()> + Send + Sync + 'static>; - -pub type FinishedCallback = - Box>, ErrorCode>) -> Result<()> + Send + Sync + 'static>; - -pub struct PipelineExecutor { - threads_num: usize, - pub(crate) graph: RunningGraph, - workers_condvar: Arc, - pub async_runtime: Arc, - pub global_tasks_queue: Arc, - on_init_callback: Mutex>, - on_finished_callback: Mutex>, - settings: ExecutorSettings, - finished_notify: Arc, - finished_error: Mutex>, - #[allow(unused)] - lock_guards: Vec, +pub enum PipelineExecutor { + QueryPipelineExecutor(Arc), + QueriesPipelineExecutor(Arc), } impl PipelineExecutor { - pub fn create( - mut pipeline: Pipeline, - settings: ExecutorSettings, - ) -> Result> { - let threads_num = pipeline.get_max_threads(); - - if threads_num.is_zero() { - return Err(ErrorCode::Internal( - "Pipeline max threads cannot equals zero.", - )); - } - - let on_init_callback = pipeline.take_on_init(); - let on_finished_callback = pipeline.take_on_finished(); - let lock_guards = pipeline.take_lock_guards(); - - match RunningGraph::create(pipeline) { - Err(cause) => { - let _ = on_finished_callback(&Err(cause.clone())); - Err(cause) - } - Ok(running_graph) => Self::try_create( - running_graph, - threads_num, - Mutex::new(Some(on_init_callback)), - Mutex::new(Some(on_finished_callback)), - settings, - lock_guards, - ), - } - } - - #[minitrace::trace] - pub fn from_pipelines( - mut pipelines: Vec, - settings: ExecutorSettings, - ) -> Result> { - if pipelines.is_empty() { - return Err(ErrorCode::Internal("Executor Pipelines is empty.")); - } - - let threads_num = pipelines - .iter() - .map(|x| x.get_max_threads()) - .max() - .unwrap_or(0); - - if threads_num.is_zero() { - return Err(ErrorCode::Internal( - "Pipeline max threads cannot equals zero.", - )); - } - - let on_init_callback = { - let pipelines_callback = pipelines - .iter_mut() - .map(|x| x.take_on_init()) - .collect::>(); - - pipelines_callback.into_iter().reduce(|left, right| { - Box::new(move || { - left()?; - right() - }) - }) - }; - - let on_finished_callback = { - let pipelines_callback = pipelines - .iter_mut() - .map(|x| x.take_on_finished()) - .collect::>(); - - pipelines_callback.into_iter().reduce(|left, right| { - Box::new(move |arg| { - left(arg)?; - right(arg) - }) - }) - }; - - let lock_guards = pipelines - .iter_mut() - .flat_map(|x| x.take_lock_guards()) - .collect::>(); - - match RunningGraph::from_pipelines(pipelines) { - Err(cause) => { - if let Some(on_finished_callback) = on_finished_callback { - let _ = on_finished_callback(&Err(cause.clone())); - } - - Err(cause) - } - Ok(running_graph) => Self::try_create( - running_graph, - threads_num, - Mutex::new(on_init_callback), - Mutex::new(on_finished_callback), - settings, - lock_guards, - ), + pub fn execute(&self) -> Result<()> { + match self { + PipelineExecutor::QueryPipelineExecutor(executor) => executor.execute(), + PipelineExecutor::QueriesPipelineExecutor(executor) => executor.execute(), } } - fn try_create( - graph: RunningGraph, - threads_num: usize, - on_init_callback: Mutex>, - on_finished_callback: Mutex>, - settings: ExecutorSettings, - lock_guards: Vec, - ) -> Result> { - let workers_condvar = WorkersCondvar::create(threads_num); - let global_tasks_queue = ExecutorTasksQueue::create(threads_num); - - Ok(Arc::new(PipelineExecutor { - graph, - threads_num, - workers_condvar, - global_tasks_queue, - on_init_callback, - on_finished_callback, - async_runtime: GlobalIORuntime::instance(), - settings, - finished_error: Mutex::new(None), - finished_notify: Arc::new(WatchNotify::new()), - lock_guards, - })) - } - - fn on_finished(&self, error: &Result>, ErrorCode>) -> Result<()> { - let mut guard = self.on_finished_callback.lock(); - if let Some(on_finished_callback) = guard.take() { - drop(guard); - catch_unwind(move || on_finished_callback(error))??; - } - Ok(()) - } - pub fn finish(&self, cause: Option) { - if let Some(cause) = cause { - let mut finished_error = self.finished_error.lock(); - - // We only save the cause of the first error. - if finished_error.is_none() { - *finished_error = Some(cause); - } + match self { + PipelineExecutor::QueryPipelineExecutor(executor) => executor.finish(cause), + PipelineExecutor::QueriesPipelineExecutor(executor) => executor.finish(cause), } - - self.global_tasks_queue.finish(self.workers_condvar.clone()); - self.graph.interrupt_running_nodes(); - self.finished_notify.notify_waiters(); } pub fn is_finished(&self) -> bool { - self.global_tasks_queue.is_finished() - } - - #[minitrace::trace] - pub fn execute(self: &Arc) -> Result<()> { - self.init()?; - - self.start_executor_daemon()?; - - let mut thread_join_handles = self.execute_threads(self.threads_num); - - while let Some(join_handle) = thread_join_handles.pop() { - let thread_res = join_handle.join().flatten(); - - { - let finished_error_guard = self.finished_error.lock(); - if let Some(error) = finished_error_guard.as_ref() { - let may_error = error.clone(); - drop(finished_error_guard); - - self.on_finished(&Err(may_error.clone()))?; - return Err(may_error); - } - } - - // We will ignore the abort query error, because returned by finished_error if abort query. - if matches!(&thread_res, Err(error) if error.code() != ErrorCode::ABORTED_QUERY) { - let may_error = thread_res.unwrap_err(); - self.on_finished(&Err(may_error.clone()))?; - return Err(may_error); - } - } - - if let Err(error) = self.graph.assert_finished_graph() { - self.on_finished(&Err(error.clone()))?; - return Err(error); - } - - self.on_finished(&Ok(self.graph.get_proc_profiles()))?; - Ok(()) - } - - fn init(self: &Arc) -> Result<()> { - unsafe { - // TODO: the on init callback cannot be killed. - { - let instant = Instant::now(); - let mut guard = self.on_init_callback.lock(); - if let Some(callback) = guard.take() { - drop(guard); - if let Err(cause) = Result::flatten(catch_unwind(callback)) { - return Err(cause.add_message_back("(while in query pipeline init)")); - } - } - - info!( - "Init pipeline successfully, query_id: {:?}, elapsed: {:?}", - self.settings.query_id, - instant.elapsed() - ); - } - - let mut init_schedule_queue = self.graph.init_schedule_queue(self.threads_num)?; - - let mut wakeup_worker_id = 0; - while let Some(proc) = init_schedule_queue.async_queue.pop_front() { - ScheduleQueue::schedule_async_task( - proc.clone(), - self.settings.query_id.clone(), - self, - wakeup_worker_id, - self.workers_condvar.clone(), - self.global_tasks_queue.clone(), - ); - wakeup_worker_id += 1; - - if wakeup_worker_id == self.threads_num { - wakeup_worker_id = 0; - } - } - - let sync_queue = std::mem::take(&mut init_schedule_queue.sync_queue); - self.global_tasks_queue.init_sync_tasks(sync_queue); - Ok(()) + match self { + PipelineExecutor::QueryPipelineExecutor(executor) => executor.is_finished(), + PipelineExecutor::QueriesPipelineExecutor(executor) => executor.is_finished(), } } - fn start_executor_daemon(self: &Arc) -> Result<()> { - if !self.settings.max_execute_time_in_seconds.is_zero() { - // NOTE(wake ref): When runtime scheduling is blocked, holding executor strong ref may cause the executor can not stop. - let this = Arc::downgrade(self); - let max_execute_time_in_seconds = self.settings.max_execute_time_in_seconds; - let finished_notify = self.finished_notify.clone(); - self.async_runtime.spawn(GLOBAL_TASK, async move { - let finished_future = Box::pin(finished_notify.notified()); - let max_execute_future = Box::pin(tokio::time::sleep(max_execute_time_in_seconds)); - if let Either::Left(_) = select(max_execute_future, finished_future).await { - if let Some(executor) = this.upgrade() { - executor.finish(Some(ErrorCode::AbortedQuery( - "Aborted query, because the execution time exceeds the maximum execution time limit", - ))); - } - } - }); - } - - Ok(()) - } - - fn execute_threads(self: &Arc, threads: usize) -> Vec>> { - let mut thread_join_handles = Vec::with_capacity(threads); - - for thread_num in 0..threads { - let this = self.clone(); - #[allow(unused_mut)] - let mut name = format!("PipelineExecutor-{}", thread_num); - - #[cfg(debug_assertions)] - { - // We need to pass the thread name in the unit test, because the thread name is the test name - if matches!(std::env::var("UNIT_TEST"), Ok(var_value) if var_value == "TRUE") { - if let Some(cur_thread_name) = std::thread::current().name() { - name = cur_thread_name.to_string(); - } - } - } - - let span = Span::enter_with_local_parent(full_name!()) - .with_property(|| ("thread_name", name.clone())); - thread_join_handles.push(Thread::named_spawn(Some(name), move || unsafe { - let _g = span.set_local_parent(); - let this_clone = this.clone(); - let try_result = catch_unwind(move || -> Result<()> { - match this_clone.execute_single_thread(thread_num) { - Ok(_) => Ok(()), - Err(cause) => { - if log::max_level() == LevelFilter::Trace { - Err(cause.add_message_back(format!( - " (while in processor thread {})", - thread_num - ))) - } else { - Err(cause) - } - } - } - }); - - // finish the pipeline executor when has error or panic - if let Err(cause) = try_result.flatten() { - this.finish(Some(cause)); - } - - Ok(()) - })); - } - thread_join_handles - } - - /// # Safety - /// - /// Method is thread unsafe and require thread safe call - pub unsafe fn execute_single_thread(self: &Arc, thread_num: usize) -> Result<()> { - let workers_condvar = self.workers_condvar.clone(); - let mut context = ExecutorWorkerContext::create( - thread_num, - workers_condvar, - self.settings.query_id.clone(), - ); - - while !self.global_tasks_queue.is_finished() { - // When there are not enough tasks, the thread will be blocked, so we need loop check. - while !self.global_tasks_queue.is_finished() && !context.has_task() { - self.global_tasks_queue.steal_task_to_context(&mut context); - } - - while !self.global_tasks_queue.is_finished() && context.has_task() { - let executed_pid = context.execute_task(&self.graph)?; - - // Not scheduled graph if pipeline is finished. - if !self.global_tasks_queue.is_finished() { - // We immediately schedule the processor again. - let schedule_queue = self.graph.schedule_queue(executed_pid)?; - schedule_queue.schedule(&self.global_tasks_queue, &mut context, self); - } - } - } - - Ok(()) - } - pub fn format_graph_nodes(&self) -> String { - self.graph.format_graph_nodes() + match self { + PipelineExecutor::QueryPipelineExecutor(executor) => executor.format_graph_nodes(), + PipelineExecutor::QueriesPipelineExecutor(executor) => executor.format_graph_nodes(), + } } pub fn get_profiles(&self) -> Vec> { - self.graph.get_proc_profiles() - } -} - -impl Drop for PipelineExecutor { - fn drop(&mut self) { - self.finish(None); - - let mut guard = self.on_finished_callback.lock(); - if let Some(on_finished_callback) = guard.take() { - drop(guard); - let cause = match self.finished_error.lock().as_ref() { - Some(cause) => cause.clone(), - None => ErrorCode::Internal("Pipeline illegal state: not successfully shutdown."), - }; - if let Err(cause) = catch_unwind(move || on_finished_callback(&Err(cause))).flatten() { - warn!("Pipeline executor shutdown failure, {:?}", cause); - } + match self { + PipelineExecutor::QueryPipelineExecutor(executor) => executor.get_profiles(), + PipelineExecutor::QueriesPipelineExecutor(executor) => executor.get_profiles(), } } } diff --git a/src/query/service/src/pipelines/executor/pipeline_pulling_executor.rs b/src/query/service/src/pipelines/executor/pipeline_pulling_executor.rs index 4c8697ea4292..3d6ff4d5968e 100644 --- a/src/query/service/src/pipelines/executor/pipeline_pulling_executor.rs +++ b/src/query/service/src/pipelines/executor/pipeline_pulling_executor.rs @@ -36,6 +36,8 @@ use parking_lot::Mutex; use crate::pipelines::executor::ExecutorSettings; use crate::pipelines::executor::PipelineExecutor; +use crate::pipelines::executor::QueriesPipelineExecutor; +use crate::pipelines::executor::QueryPipelineExecutor; use crate::pipelines::processors::InputPort; use crate::pipelines::processors::ProcessorPtr; use crate::pipelines::PipelineBuildResult; @@ -122,11 +124,19 @@ impl PipelinePullingExecutor { let (sender, receiver) = std::sync::mpsc::sync_channel(pipeline.output_len()); Self::wrap_pipeline(&mut pipeline, sender)?; + let executor = if settings.enable_new_executor { + PipelineExecutor::QueriesPipelineExecutor(QueriesPipelineExecutor::create( + pipeline, settings, + )?) + } else { + PipelineExecutor::QueryPipelineExecutor(QueryPipelineExecutor::create( + pipeline, settings, + )?) + }; - let executor = PipelineExecutor::create(pipeline, settings)?; Ok(PipelinePullingExecutor { receiver, - executor, + executor: Arc::new(executor), state: State::create(), }) } @@ -142,11 +152,19 @@ impl PipelinePullingExecutor { let mut pipelines = build_res.sources_pipelines; pipelines.push(main_pipeline); - + let executor = if settings.enable_new_executor { + PipelineExecutor::QueriesPipelineExecutor(QueriesPipelineExecutor::from_pipelines( + pipelines, settings, + )?) + } else { + PipelineExecutor::QueryPipelineExecutor(QueryPipelineExecutor::from_pipelines( + pipelines, settings, + )?) + }; Ok(PipelinePullingExecutor { receiver, state: State::create(), - executor: PipelineExecutor::from_pipelines(pipelines, settings)?, + executor: Arc::new(executor), }) } diff --git a/src/query/service/src/pipelines/executor/pipeline_pushing_executor.rs b/src/query/service/src/pipelines/executor/pipeline_pushing_executor.rs index f6caffbf04ea..92a1d3fced87 100644 --- a/src/query/service/src/pipelines/executor/pipeline_pushing_executor.rs +++ b/src/query/service/src/pipelines/executor/pipeline_pushing_executor.rs @@ -32,6 +32,8 @@ use parking_lot::Mutex; use crate::pipelines::executor::ExecutorSettings; use crate::pipelines::executor::PipelineExecutor; +use crate::pipelines::executor::QueriesPipelineExecutor; +use crate::pipelines::executor::QueryPipelineExecutor; use crate::pipelines::processors::OutputPort; use crate::pipelines::processors::ProcessorPtr; use crate::sessions::QueryContext; @@ -96,11 +98,19 @@ impl PipelinePushingExecutor { ) -> Result { let state = State::create(); let sender = Self::wrap_pipeline(ctx, &mut pipeline)?; - let executor = PipelineExecutor::create(pipeline, settings)?; + let executor = if settings.enable_new_executor { + PipelineExecutor::QueriesPipelineExecutor(QueriesPipelineExecutor::create( + pipeline, settings, + )?) + } else { + PipelineExecutor::QueryPipelineExecutor(QueryPipelineExecutor::create( + pipeline, settings, + )?) + }; Ok(PipelinePushingExecutor { state, sender, - executor, + executor: Arc::new(executor), }) } diff --git a/src/query/service/src/pipelines/executor/processor_async_task.rs b/src/query/service/src/pipelines/executor/processor_async_task.rs index 696acd10cfc0..3fb4495cf798 100644 --- a/src/query/service/src/pipelines/executor/processor_async_task.rs +++ b/src/query/service/src/pipelines/executor/processor_async_task.rs @@ -16,13 +16,13 @@ use std::future::Future; use std::intrinsics::assume; use std::pin::Pin; use std::sync::Arc; -use std::sync::Weak; use std::task::Context; use std::task::Poll; use std::time::Duration; use std::time::Instant; use databend_common_base::base::tokio::time::sleep; +use databend_common_base::base::WatchNotify; use databend_common_base::runtime::catch_unwind; use databend_common_exception::ErrorCode; use databend_common_exception::Result; @@ -37,10 +37,49 @@ use log::warn; use petgraph::prelude::NodeIndex; use crate::pipelines::executor::CompletedAsyncTask; -use crate::pipelines::executor::ExecutorTasksQueue; -use crate::pipelines::executor::PipelineExecutor; +use crate::pipelines::executor::QueriesExecutorTasksQueue; +use crate::pipelines::executor::QueryExecutorTasksQueue; +use crate::pipelines::executor::RunningGraph; use crate::pipelines::executor::WorkersCondvar; +pub enum ExecutorTasksQueue { + QueryExecutorTasksQueue(Arc), + QueriesExecutorTasksQueue(Arc), +} + +impl ExecutorTasksQueue { + pub fn get_finished_notify(&self) -> Arc { + match &self { + ExecutorTasksQueue::QueryExecutorTasksQueue(executor) => executor.get_finished_notify(), + ExecutorTasksQueue::QueriesExecutorTasksQueue(executor) => { + executor.get_finished_notify() + } + } + } + pub fn active_workers(&self) -> usize { + match &self { + ExecutorTasksQueue::QueryExecutorTasksQueue(executor) => executor.active_workers(), + ExecutorTasksQueue::QueriesExecutorTasksQueue(executor) => executor.active_workers(), + } + } + pub fn is_finished(&self) -> bool { + match &self { + ExecutorTasksQueue::QueryExecutorTasksQueue(executor) => executor.is_finished(), + ExecutorTasksQueue::QueriesExecutorTasksQueue(executor) => executor.is_finished(), + } + } + pub fn completed_async_task(&self, condvar: Arc, task: CompletedAsyncTask) { + match &self { + ExecutorTasksQueue::QueryExecutorTasksQueue(executor) => { + executor.completed_async_task(condvar, task) + } + ExecutorTasksQueue::QueriesExecutorTasksQueue(executor) => { + executor.completed_async_task(condvar, task) + } + } + } +} + pub struct ProcessorAsyncTask { worker_id: usize, processor_id: NodeIndex, @@ -49,6 +88,7 @@ pub struct ProcessorAsyncTask { profile: Arc, instant: Instant, last_nanos: usize, + graph: Arc, inner: BoxFuture<'static, Result<()>>, } @@ -59,8 +99,8 @@ impl ProcessorAsyncTask { processor: ProcessorPtr, queue: Arc, workers_condvar: Arc, - weak_executor: Weak, profile: Arc, + graph: Arc, inner: Inner, ) -> ProcessorAsyncTask { let finished_notify = queue.get_finished_notify(); @@ -79,6 +119,7 @@ impl ProcessorAsyncTask { let processor_id = unsafe { processor.id() }; let processor_name = unsafe { processor.name() }; let queue_clone = queue.clone(); + let graph_clone = graph.clone(); let inner = async move { let start = Instant::now(); let mut inner = inner.boxed(); @@ -103,17 +144,15 @@ impl ProcessorAsyncTask { } true => { log_graph = true; - if let Some(executor) = weak_executor.upgrade() { - error!( - "Very slow processor async task, query_id:{:?}, processor id: {:?}, name: {:?}, elapsed: {:?}, active sync workers: {:?}, {}", - query_id, - processor_id, - processor_name, - elapsed, - active_workers, - executor.graph.format_graph_nodes() - ); - } + error!( + "Very slow processor async task, query_id:{:?}, processor id: {:?}, name: {:?}, elapsed: {:?}, active sync workers: {:?}, {}", + query_id, + processor_id, + processor_name, + elapsed, + active_workers, + graph_clone.format_graph_nodes() + ); } }; } @@ -133,6 +172,7 @@ impl ProcessorAsyncTask { profile, last_nanos: instant.elapsed().as_nanos() as usize, instant, + graph, inner: inner.boxed(), } } @@ -172,14 +212,24 @@ impl Future for ProcessorAsyncTask { Ok(Poll::Ready(res)) => { self.queue.completed_async_task( self.workers_condvar.clone(), - CompletedAsyncTask::create(self.processor_id, self.worker_id, res), + CompletedAsyncTask::create( + self.processor_id, + self.worker_id, + res, + self.graph.clone(), + ), ); Poll::Ready(()) } Err(cause) => { self.queue.completed_async_task( self.workers_condvar.clone(), - CompletedAsyncTask::create(self.processor_id, self.worker_id, Err(cause)), + CompletedAsyncTask::create( + self.processor_id, + self.worker_id, + Err(cause), + self.graph.clone(), + ), ); Poll::Ready(()) diff --git a/src/query/service/src/pipelines/executor/queries_executor_tasks.rs b/src/query/service/src/pipelines/executor/queries_executor_tasks.rs new file mode 100644 index 000000000000..1250b459acf5 --- /dev/null +++ b/src/query/service/src/pipelines/executor/queries_executor_tasks.rs @@ -0,0 +1,330 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed 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 std::collections::VecDeque; +use std::mem; +use std::sync::atomic::AtomicBool; +use std::sync::atomic::Ordering; +use std::sync::Arc; + +use parking_lot::Mutex; + +use crate::pipelines::executor::executor_graph::ProcessorWrapper; +use crate::pipelines::executor::executor_worker_context::CompletedAsyncTask; +use crate::pipelines::executor::ExecutorTask; +use crate::pipelines::executor::ExecutorWorkerContext; +use crate::pipelines::executor::QueriesPipelineExecutor; +use crate::pipelines::executor::WatchNotify; +use crate::pipelines::executor::WorkersCondvar; +use crate::pipelines::executor::WorkersWaitingStatus; + +struct WorkersTasks { + workers_waiting_status: WorkersWaitingStatus, + /// current_tasks is the task queue we are using in current time slice + current_tasks: ExecutorTasks, + /// next_tasks is the task queue we will use in next time slice + next_tasks: ExecutorTasks, +} + +impl WorkersTasks { + pub fn create(workers_size: usize) -> Self { + WorkersTasks { + workers_waiting_status: WorkersWaitingStatus::create(workers_size), + current_tasks: ExecutorTasks::create(workers_size), + next_tasks: ExecutorTasks::create(workers_size), + } + } + + pub fn swap_tasks(&mut self) { + mem::swap(&mut self.current_tasks, &mut self.next_tasks); + } +} + +pub struct QueriesExecutorTasksQueue { + finished: Arc, + finished_notify: Arc, + workers_tasks: Mutex, +} + +impl QueriesExecutorTasksQueue { + pub fn create(workers_size: usize) -> Arc { + Arc::new(QueriesExecutorTasksQueue { + finished: Arc::new(AtomicBool::new(false)), + finished_notify: Arc::new(WatchNotify::new()), + workers_tasks: Mutex::new(WorkersTasks::create(workers_size)), + }) + } + + pub fn finish(&self, workers_condvar: Arc) { + self.finished.store(true, Ordering::SeqCst); + self.finished_notify.notify_waiters(); + + let mut workers_tasks = self.workers_tasks.lock(); + let mut wakeup_workers = + Vec::with_capacity(workers_tasks.workers_waiting_status.waiting_size()); + + while workers_tasks.workers_waiting_status.waiting_size() != 0 { + let worker_id = workers_tasks.workers_waiting_status.wakeup_any_worker(); + wakeup_workers.push(worker_id); + } + + drop(workers_tasks); + for wakeup_worker in wakeup_workers { + workers_condvar.wakeup(wakeup_worker); + } + } + + pub fn is_finished(&self) -> bool { + self.finished.load(Ordering::SeqCst) + } + + /// Pull task from the global task queue + /// Method is thread unsafe and require thread safe call + pub fn steal_task_to_context( + self: &Arc, + context: &mut ExecutorWorkerContext, + executor: &Arc, + ) { + let mut workers_tasks = self.workers_tasks.lock(); + if !workers_tasks.current_tasks.is_empty() { + let task = workers_tasks + .current_tasks + .pop_task(context.get_worker_id()); + context.set_task(task); + + let workers_condvar = context.get_workers_condvar(); + if !workers_tasks.current_tasks.is_empty() + && workers_tasks.workers_waiting_status.waiting_size() != 0 + { + let worker_id = context.get_worker_id(); + let mut wakeup_worker_id = + workers_tasks.current_tasks.best_worker_id(worker_id + 1); + + if workers_tasks + .workers_waiting_status + .is_waiting(wakeup_worker_id) + { + workers_tasks + .workers_waiting_status + .wakeup_worker(wakeup_worker_id); + } else { + wakeup_worker_id = workers_tasks.workers_waiting_status.wakeup_any_worker(); + } + + drop(workers_tasks); + workers_condvar.wakeup(wakeup_worker_id); + } + + return; + } + + if workers_tasks.current_tasks.is_empty() && !workers_tasks.next_tasks.is_empty() { + workers_tasks.swap_tasks(); + executor.increase_global_epoch(); + return; + } + + let workers_condvar = context.get_workers_condvar(); + if !workers_condvar.has_waiting_async_task() + && workers_tasks.workers_waiting_status.is_last_active_worker() + { + drop(workers_tasks); + self.finish(workers_condvar.clone()); + return; + } + + let worker_id = context.get_worker_id(); + workers_tasks.workers_waiting_status.wait_worker(worker_id); + drop(workers_tasks); + workers_condvar.wait(worker_id, self.finished.clone()); + } + + pub fn init_sync_tasks(&self, tasks: VecDeque) { + let mut workers_tasks = self.workers_tasks.lock(); + + let mut worker_id = 0; + for proc in tasks.into_iter() { + workers_tasks + .next_tasks + .push_task(worker_id, ExecutorTask::Sync(proc)); + + worker_id += 1; + if worker_id == workers_tasks.next_tasks.workers_sync_tasks.len() { + worker_id = 0; + } + } + } + + pub fn completed_async_task(&self, condvar: Arc, task: CompletedAsyncTask) { + let mut workers_tasks = self.workers_tasks.lock(); + let mut worker_id = task.worker_id; + workers_tasks.current_tasks.tasks_size += 1; + workers_tasks.current_tasks.workers_completed_async_tasks[worker_id].push_back(task); + + condvar.dec_active_async_worker(); + + if workers_tasks.workers_waiting_status.waiting_size() != 0 { + if workers_tasks.workers_waiting_status.is_waiting(worker_id) { + workers_tasks + .workers_waiting_status + .wakeup_worker(worker_id); + } else { + worker_id = workers_tasks.workers_waiting_status.wakeup_any_worker(); + } + + drop(workers_tasks); + condvar.wakeup(worker_id); + } + } + + pub fn get_finished_notify(&self) -> Arc { + self.finished_notify.clone() + } + + pub fn active_workers(&self) -> usize { + let workers_tasks = self.workers_tasks.lock(); + workers_tasks.workers_waiting_status.total_size() + - workers_tasks.workers_waiting_status.waiting_size() + } + + pub fn push_tasks( + &self, + worker_id: usize, + current_tasks: Option>, + mut next_tasks: VecDeque, + ) { + let mut workers_tasks = self.workers_tasks.lock(); + + if let Some(mut tasks) = current_tasks { + while let Some(task) = tasks.pop_front() { + workers_tasks.current_tasks.push_task(worker_id, task); + } + } + while let Some(task) = next_tasks.pop_front() { + workers_tasks.next_tasks.push_task(worker_id, task); + } + } +} + +struct ExecutorTasks { + tasks_size: usize, + workers_sync_tasks: Vec>, + workers_async_tasks: Vec>, + workers_completed_async_tasks: Vec>, +} + +unsafe impl Send for ExecutorTasks {} + +impl ExecutorTasks { + pub fn create(workers_size: usize) -> ExecutorTasks { + let mut workers_sync_tasks = Vec::with_capacity(workers_size); + let mut workers_completed_async_tasks = Vec::with_capacity(workers_size); + let mut workers_async_tasks = Vec::with_capacity(workers_size); + for _index in 0..workers_size { + workers_sync_tasks.push(VecDeque::new()); + workers_async_tasks.push(VecDeque::new()); + workers_completed_async_tasks.push(VecDeque::new()); + } + + ExecutorTasks { + tasks_size: 0, + workers_sync_tasks, + workers_async_tasks, + workers_completed_async_tasks, + } + } + + pub fn is_empty(&self) -> bool { + self.tasks_size == 0 + } + + #[inline] + fn pop_worker_task(&mut self, worker_id: usize) -> ExecutorTask { + if let Some(processor) = self.workers_async_tasks[worker_id].pop_front() { + return ExecutorTask::Async(processor); + } + + if let Some(task) = self.workers_completed_async_tasks[worker_id].pop_front() { + return ExecutorTask::AsyncCompleted(task); + } + + if let Some(processor) = self.workers_sync_tasks[worker_id].pop_front() { + return ExecutorTask::Sync(processor); + } + + ExecutorTask::None + } + + pub fn best_worker_id(&self, mut worker_id: usize) -> usize { + for _index in 0..self.workers_sync_tasks.len() { + if worker_id >= self.workers_sync_tasks.len() { + worker_id = 0; + } + + if !self.workers_sync_tasks[worker_id].is_empty() { + return worker_id; + } + + if !self.workers_completed_async_tasks[worker_id].is_empty() { + return worker_id; + } + + if !self.workers_async_tasks[worker_id].is_empty() { + return worker_id; + } + + worker_id += 1; + } + + worker_id + } + + pub fn pop_task(&mut self, mut worker_id: usize) -> ExecutorTask { + for _index in 0..self.workers_sync_tasks.len() { + match self.pop_worker_task(worker_id) { + ExecutorTask::None => { + worker_id += 1; + if worker_id >= self.workers_sync_tasks.len() { + worker_id = 0; + } + } + other => { + self.tasks_size -= 1; + return other; + } + } + } + + ExecutorTask::None + } + + pub fn push_task(&mut self, worker_id: usize, task: ExecutorTask) { + self.tasks_size += 1; + debug_assert!( + worker_id < self.workers_sync_tasks.len(), + "out of index, {}, {}", + worker_id, + self.workers_sync_tasks.len() + ); + let sync_queue = &mut self.workers_sync_tasks[worker_id]; + let completed_queue = &mut self.workers_completed_async_tasks[worker_id]; + let async_queue = &mut self.workers_async_tasks[worker_id]; + match task { + ExecutorTask::None => unreachable!(), + ExecutorTask::Sync(processor) => sync_queue.push_back(processor), + ExecutorTask::Async(processor) => async_queue.push_back(processor), + ExecutorTask::AsyncCompleted(task) => completed_queue.push_back(task), + } + } +} diff --git a/src/query/service/src/pipelines/executor/queries_pipeline_executor.rs b/src/query/service/src/pipelines/executor/queries_pipeline_executor.rs new file mode 100644 index 000000000000..70a5637a73b1 --- /dev/null +++ b/src/query/service/src/pipelines/executor/queries_pipeline_executor.rs @@ -0,0 +1,463 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed 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 std::collections::VecDeque; +use std::sync::atomic::AtomicU32; +use std::sync::atomic::Ordering; +use std::sync::Arc; +use std::time::Instant; + +use databend_common_base::base::tokio; +use databend_common_base::runtime::catch_unwind; +use databend_common_base::runtime::GlobalIORuntime; +use databend_common_base::runtime::Runtime; +use databend_common_base::runtime::Thread; +use databend_common_base::runtime::ThreadJoinHandle; +use databend_common_base::runtime::TrySpawn; +use databend_common_base::GLOBAL_TASK; +use databend_common_exception::ErrorCode; +use databend_common_exception::Result; +use databend_common_pipeline_core::processors::Profile; +use databend_common_pipeline_core::LockGuard; +use databend_common_pipeline_core::Pipeline; +use futures::future::select; +use futures_util::future::Either; +use log::info; +use log::warn; +use log::LevelFilter; +use minitrace::full_name; +use minitrace::prelude::*; +use parking_lot::Mutex; +use petgraph::matrix_graph::Zero; +use tokio::time; + +use crate::pipelines::executor::ExecutorSettings; +use crate::pipelines::executor::ExecutorTask; +use crate::pipelines::executor::ExecutorWorkerContext; +use crate::pipelines::executor::QueriesExecutorTasksQueue; +use crate::pipelines::executor::RunningGraph; +use crate::pipelines::executor::WatchNotify; +use crate::pipelines::executor::WorkersCondvar; + +pub type InitCallback = Box Result<()> + Send + Sync + 'static>; + +pub type FinishedCallback = + Box>, ErrorCode>) -> Result<()> + Send + Sync + 'static>; + +pub struct QueriesPipelineExecutor { + threads_num: usize, + workers_condvar: Arc, + pub async_runtime: Arc, + pub global_tasks_queue: Arc, + on_init_callback: Mutex>, + on_finished_callback: Mutex>, + settings: ExecutorSettings, + finished_notify: Arc, + finished_error: Mutex>, + #[allow(unused)] + lock_guards: Vec, + pub epoch: AtomicU32, + + // TODO: will remove it after refactoring Executor into a 1:n pattern + pub graph: Arc, +} + +impl QueriesPipelineExecutor { + pub fn create( + mut pipeline: Pipeline, + settings: ExecutorSettings, + ) -> Result> { + let threads_num = pipeline.get_max_threads(); + + if threads_num.is_zero() { + return Err(ErrorCode::Internal( + "Pipeline max threads cannot equals zero.", + )); + } + + let on_init_callback = pipeline.take_on_init(); + let on_finished_callback = pipeline.take_on_finished(); + let lock_guards = pipeline.take_lock_guards(); + + match RunningGraph::create(pipeline, 1) { + Err(cause) => { + let _ = on_finished_callback(&Err(cause.clone())); + Err(cause) + } + Ok(running_graph) => Self::try_create( + running_graph, + threads_num, + Mutex::new(Some(on_init_callback)), + Mutex::new(Some(on_finished_callback)), + settings, + lock_guards, + ), + } + } + + #[minitrace::trace] + pub fn from_pipelines( + mut pipelines: Vec, + settings: ExecutorSettings, + ) -> Result> { + if pipelines.is_empty() { + return Err(ErrorCode::Internal("Executor Pipelines is empty.")); + } + + let threads_num = pipelines + .iter() + .map(|x| x.get_max_threads()) + .max() + .unwrap_or(0); + + if threads_num.is_zero() { + return Err(ErrorCode::Internal( + "Pipeline max threads cannot equals zero.", + )); + } + + let on_init_callback = { + let pipelines_callback = pipelines + .iter_mut() + .map(|x| x.take_on_init()) + .collect::>(); + + pipelines_callback.into_iter().reduce(|left, right| { + Box::new(move || { + left()?; + right() + }) + }) + }; + + let on_finished_callback = { + let pipelines_callback = pipelines + .iter_mut() + .map(|x| x.take_on_finished()) + .collect::>(); + + pipelines_callback.into_iter().reduce(|left, right| { + Box::new(move |arg| { + left(arg)?; + right(arg) + }) + }) + }; + + let lock_guards = pipelines + .iter_mut() + .flat_map(|x| x.take_lock_guards()) + .collect::>(); + + match RunningGraph::from_pipelines(pipelines, 1) { + Err(cause) => { + if let Some(on_finished_callback) = on_finished_callback { + let _ = on_finished_callback(&Err(cause.clone())); + } + + Err(cause) + } + Ok(running_graph) => Self::try_create( + running_graph, + threads_num, + Mutex::new(on_init_callback), + Mutex::new(on_finished_callback), + settings, + lock_guards, + ), + } + } + + fn try_create( + graph: Arc, + threads_num: usize, + on_init_callback: Mutex>, + on_finished_callback: Mutex>, + settings: ExecutorSettings, + lock_guards: Vec, + ) -> Result> { + let workers_condvar = WorkersCondvar::create(threads_num); + let global_tasks_queue = QueriesExecutorTasksQueue::create(threads_num); + + Ok(Arc::new(QueriesPipelineExecutor { + graph, + threads_num, + workers_condvar, + global_tasks_queue, + on_init_callback, + on_finished_callback, + async_runtime: GlobalIORuntime::instance(), + settings, + finished_error: Mutex::new(None), + finished_notify: Arc::new(WatchNotify::new()), + lock_guards, + epoch: AtomicU32::new(0), + })) + } + + fn on_finished(&self, error: &Result>, ErrorCode>) -> Result<()> { + let mut guard = self.on_finished_callback.lock(); + if let Some(on_finished_callback) = guard.take() { + drop(guard); + catch_unwind(move || on_finished_callback(error))??; + } + Ok(()) + } + + pub fn finish(&self, cause: Option) { + if let Some(cause) = cause { + let mut finished_error = self.finished_error.lock(); + + // We only save the cause of the first error. + if finished_error.is_none() { + *finished_error = Some(cause); + } + } + + self.global_tasks_queue.finish(self.workers_condvar.clone()); + self.graph.interrupt_running_nodes(); + self.finished_notify.notify_waiters(); + } + + pub fn is_finished(&self) -> bool { + self.global_tasks_queue.is_finished() + } + + #[minitrace::trace] + pub fn execute(self: &Arc) -> Result<()> { + // TODO: will remove this in the future + self.init(self.graph.clone())?; + + self.start_time_limit_daemon()?; + + let mut thread_join_handles = self.execute_threads(self.threads_num); + + while let Some(join_handle) = thread_join_handles.pop() { + let thread_res = join_handle.join().flatten(); + + { + let finished_error_guard = self.finished_error.lock(); + if let Some(error) = finished_error_guard.as_ref() { + let may_error = error.clone(); + drop(finished_error_guard); + + self.on_finished(&Err(may_error.clone()))?; + return Err(may_error); + } + } + + // We will ignore the abort query error, because returned by finished_error if abort query. + if matches!(&thread_res, Err(error) if error.code() != ErrorCode::ABORTED_QUERY) { + let may_error = thread_res.unwrap_err(); + self.on_finished(&Err(may_error.clone()))?; + return Err(may_error); + } + } + + if let Err(error) = self.graph.assert_finished_graph() { + self.on_finished(&Err(error.clone()))?; + return Err(error); + } + + self.on_finished(&Ok(self.graph.get_proc_profiles()))?; + Ok(()) + } + + fn init(self: &Arc, graph: Arc) -> Result<()> { + unsafe { + // TODO: the on init callback cannot be killed. + { + let instant = Instant::now(); + let mut guard = self.on_init_callback.lock(); + if let Some(callback) = guard.take() { + drop(guard); + if let Err(cause) = Result::flatten(catch_unwind(callback)) { + return Err(cause.add_message_back("(while in query pipeline init)")); + } + } + + info!( + "Init pipeline successfully, query_id: {:?}, elapsed: {:?}", + self.settings.query_id, + instant.elapsed() + ); + } + + let mut init_schedule_queue = graph.init_schedule_queue(self.threads_num)?; + + let mut wakeup_worker_id = 0; + while let Some(proc) = init_schedule_queue.async_queue.pop_front() { + let mut tasks = VecDeque::with_capacity(1); + tasks.push_back(ExecutorTask::Async(proc)); + self.global_tasks_queue + .push_tasks(wakeup_worker_id, None, tasks); + + wakeup_worker_id += 1; + if wakeup_worker_id == self.threads_num { + wakeup_worker_id = 0; + } + } + + let sync_queue = std::mem::take(&mut init_schedule_queue.sync_queue); + self.global_tasks_queue.init_sync_tasks(sync_queue); + Ok(()) + } + } + + /// Used to abort the query when the execution time exceeds the maximum execution time limit + fn start_time_limit_daemon(self: &Arc) -> Result<()> { + if !self.settings.max_execute_time_in_seconds.is_zero() { + // NOTE(wake ref): When runtime scheduling is blocked, holding executor strong ref may cause the executor can not stop. + let this = Arc::downgrade(self); + let max_execute_time_in_seconds = self.settings.max_execute_time_in_seconds; + let finished_notify = self.finished_notify.clone(); + self.async_runtime.spawn(GLOBAL_TASK, async move { + let finished_future = Box::pin(finished_notify.notified()); + let max_execute_future = Box::pin(time::sleep(max_execute_time_in_seconds)); + + // This waits for either of two futures to complete: + // 1. The 'finished_future', which gets triggered when an external event signals that the task is finished. + // 2. The 'max_execute_future', which gets triggered when the maximum execution time as set in 'max_execute_time_in_seconds' elapses. + // When either future completes, the executor is finished. + if let Either::Left(_) = select(max_execute_future, finished_future).await { + if let Some(executor) = this.upgrade() { + executor.finish(Some(ErrorCode::AbortedQuery( + "Aborted query, because the execution time exceeds the maximum execution time limit", + ))); + } + } + }); + } + + Ok(()) + } + + fn execute_threads(self: &Arc, threads: usize) -> Vec>> { + let mut thread_join_handles = Vec::with_capacity(threads); + + for thread_num in 0..threads { + let this = self.clone(); + #[allow(unused_mut)] + let mut name = format!("PipelineExecutor-{}", thread_num); + + #[cfg(debug_assertions)] + { + // We need to pass the thread name in the unit test, because the thread name is the test name + if matches!(std::env::var("UNIT_TEST"), Ok(var_value) if var_value == "TRUE") { + if let Some(cur_thread_name) = std::thread::current().name() { + name = cur_thread_name.to_string(); + } + } + } + + let span = Span::enter_with_local_parent(full_name!()) + .with_property(|| ("thread_name", name.clone())); + thread_join_handles.push(Thread::named_spawn(Some(name), move || unsafe { + let _g = span.set_local_parent(); + let this_clone = this.clone(); + let try_result = catch_unwind(move || -> Result<()> { + match this_clone.execute_single_thread(thread_num) { + Ok(_) => Ok(()), + Err(cause) => { + if log::max_level() == LevelFilter::Trace { + Err(cause.add_message_back(format!( + " (while in processor thread {})", + thread_num + ))) + } else { + Err(cause) + } + } + } + }); + + // finish the pipeline executor when has error or panic + if let Err(cause) = try_result.flatten() { + this.finish(Some(cause)); + } + + Ok(()) + })); + } + thread_join_handles + } + + /// # Safety + /// + /// Method is thread unsafe and require thread safe call + pub unsafe fn execute_single_thread(self: &Arc, thread_num: usize) -> Result<()> { + let workers_condvar = self.workers_condvar.clone(); + let mut context = ExecutorWorkerContext::create( + thread_num, + workers_condvar, + self.settings.query_id.clone(), + ); + + while !self.global_tasks_queue.is_finished() { + // When there are not enough tasks, the thread will be blocked, so we need loop check. + while !self.global_tasks_queue.is_finished() && !context.has_task() { + self.global_tasks_queue + .steal_task_to_context(&mut context, self); + } + + while !self.global_tasks_queue.is_finished() && context.has_task() { + if let Some((executed_pid, graph)) = context.execute_task()? { + // Not scheduled graph if pipeline is finished. + if !self.global_tasks_queue.is_finished() { + // We immediately schedule the processor again. + let schedule_queue = graph.schedule_queue(executed_pid)?; + schedule_queue.schedule_with_condition( + &self.global_tasks_queue, + &mut context, + self, + ); + } + } + } + } + + Ok(()) + } + + #[inline] + pub(crate) fn increase_global_epoch(&self) { + self.epoch.fetch_add(1, Ordering::SeqCst); + } + + pub fn format_graph_nodes(&self) -> String { + self.graph.format_graph_nodes() + } + + pub fn get_profiles(&self) -> Vec> { + self.graph.get_proc_profiles() + } +} + +impl Drop for QueriesPipelineExecutor { + fn drop(&mut self) { + self.finish(None); + + let mut guard = self.on_finished_callback.lock(); + if let Some(on_finished_callback) = guard.take() { + drop(guard); + let cause = match self.finished_error.lock().as_ref() { + Some(cause) => cause.clone(), + None => ErrorCode::Internal("Pipeline illegal state: not successfully shutdown."), + }; + if let Err(cause) = catch_unwind(move || on_finished_callback(&Err(cause))).flatten() { + warn!("Pipeline executor shutdown failure, {:?}", cause); + } + } + } +} diff --git a/src/query/service/src/pipelines/executor/executor_tasks.rs b/src/query/service/src/pipelines/executor/query_executor_tasks.rs similarity index 93% rename from src/query/service/src/pipelines/executor/executor_tasks.rs rename to src/query/service/src/pipelines/executor/query_executor_tasks.rs index cc70fcd199ee..335b16bfa323 100644 --- a/src/query/service/src/pipelines/executor/executor_tasks.rs +++ b/src/query/service/src/pipelines/executor/query_executor_tasks.rs @@ -17,26 +17,25 @@ use std::sync::atomic::AtomicBool; use std::sync::atomic::Ordering; use std::sync::Arc; -use databend_common_exception::Result; use parking_lot::Mutex; -use petgraph::prelude::NodeIndex; +use crate::pipelines::executor::executor_graph::ProcessorWrapper; +use crate::pipelines::executor::executor_worker_context::CompletedAsyncTask; use crate::pipelines::executor::ExecutorTask; use crate::pipelines::executor::ExecutorWorkerContext; use crate::pipelines::executor::WatchNotify; use crate::pipelines::executor::WorkersCondvar; use crate::pipelines::executor::WorkersWaitingStatus; -use crate::pipelines::processors::ProcessorPtr; -pub struct ExecutorTasksQueue { +pub struct QueryExecutorTasksQueue { finished: Arc, finished_notify: Arc, workers_tasks: Mutex, } -impl ExecutorTasksQueue { - pub fn create(workers_size: usize) -> Arc { - Arc::new(ExecutorTasksQueue { +impl QueryExecutorTasksQueue { + pub fn create(workers_size: usize) -> Arc { + Arc::new(QueryExecutorTasksQueue { finished: Arc::new(AtomicBool::new(false)), finished_notify: Arc::new(WatchNotify::new()), workers_tasks: Mutex::new(ExecutorTasks::create(workers_size)), @@ -117,7 +116,7 @@ impl ExecutorTasksQueue { workers_condvar.wait(worker_id, self.finished.clone()); } - pub fn init_sync_tasks(&self, tasks: VecDeque) { + pub fn init_sync_tasks(&self, tasks: VecDeque) { let mut workers_tasks = self.workers_tasks.lock(); let mut worker_id = 0; @@ -193,22 +192,10 @@ impl ExecutorTasksQueue { } } -pub struct CompletedAsyncTask { - pub id: NodeIndex, - pub worker_id: usize, - pub res: Result<()>, -} - -impl CompletedAsyncTask { - pub fn create(id: NodeIndex, worker_id: usize, res: Result<()>) -> Self { - CompletedAsyncTask { id, worker_id, res } - } -} - struct ExecutorTasks { tasks_size: usize, workers_waiting_status: WorkersWaitingStatus, - workers_sync_tasks: Vec>, + workers_sync_tasks: Vec>, workers_completed_async_tasks: Vec>, } @@ -302,6 +289,7 @@ impl ExecutorTasks { match task { ExecutorTask::None => unreachable!(), ExecutorTask::Sync(processor) => sync_queue.push_back(processor), + ExecutorTask::Async(_) => unreachable!("used for new executor"), ExecutorTask::AsyncCompleted(task) => completed_queue.push_back(task), } } diff --git a/src/query/service/src/pipelines/executor/query_pipeline_executor.rs b/src/query/service/src/pipelines/executor/query_pipeline_executor.rs new file mode 100644 index 000000000000..b965835b9f6d --- /dev/null +++ b/src/query/service/src/pipelines/executor/query_pipeline_executor.rs @@ -0,0 +1,442 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed 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 std::sync::Arc; +use std::time::Instant; + +use databend_common_base::base::tokio; +use databend_common_base::runtime::catch_unwind; +use databend_common_base::runtime::GlobalIORuntime; +use databend_common_base::runtime::Runtime; +use databend_common_base::runtime::Thread; +use databend_common_base::runtime::ThreadJoinHandle; +use databend_common_base::runtime::TrySpawn; +use databend_common_base::GLOBAL_TASK; +use databend_common_exception::ErrorCode; +use databend_common_exception::Result; +use databend_common_pipeline_core::processors::Profile; +use databend_common_pipeline_core::LockGuard; +use databend_common_pipeline_core::Pipeline; +use futures::future::select; +use futures_util::future::Either; +use log::info; +use log::warn; +use log::LevelFilter; +use minitrace::full_name; +use minitrace::prelude::*; +use parking_lot::Mutex; +use petgraph::matrix_graph::Zero; + +use crate::pipelines::executor::executor_graph::ScheduleQueue; +use crate::pipelines::executor::ExecutorSettings; +use crate::pipelines::executor::ExecutorWorkerContext; +use crate::pipelines::executor::QueryExecutorTasksQueue; +use crate::pipelines::executor::RunningGraph; +use crate::pipelines::executor::WatchNotify; +use crate::pipelines::executor::WorkersCondvar; + +pub type InitCallback = Box Result<()> + Send + Sync + 'static>; + +pub type FinishedCallback = + Box>, ErrorCode>) -> Result<()> + Send + Sync + 'static>; + +pub struct QueryPipelineExecutor { + threads_num: usize, + pub(crate) graph: Arc, + workers_condvar: Arc, + pub async_runtime: Arc, + pub global_tasks_queue: Arc, + on_init_callback: Mutex>, + on_finished_callback: Mutex>, + settings: ExecutorSettings, + finished_notify: Arc, + finished_error: Mutex>, + #[allow(unused)] + lock_guards: Vec, +} + +impl QueryPipelineExecutor { + pub fn create( + mut pipeline: Pipeline, + settings: ExecutorSettings, + ) -> Result> { + let threads_num = pipeline.get_max_threads(); + + if threads_num.is_zero() { + return Err(ErrorCode::Internal( + "Pipeline max threads cannot equals zero.", + )); + } + + let on_init_callback = pipeline.take_on_init(); + let on_finished_callback = pipeline.take_on_finished(); + let lock_guards = pipeline.take_lock_guards(); + + match RunningGraph::create(pipeline, 1) { + Err(cause) => { + let _ = on_finished_callback(&Err(cause.clone())); + Err(cause) + } + Ok(running_graph) => Self::try_create( + running_graph, + threads_num, + Mutex::new(Some(on_init_callback)), + Mutex::new(Some(on_finished_callback)), + settings, + lock_guards, + ), + } + } + + #[minitrace::trace] + pub fn from_pipelines( + mut pipelines: Vec, + settings: ExecutorSettings, + ) -> Result> { + if pipelines.is_empty() { + return Err(ErrorCode::Internal("Executor Pipelines is empty.")); + } + + let threads_num = pipelines + .iter() + .map(|x| x.get_max_threads()) + .max() + .unwrap_or(0); + + if threads_num.is_zero() { + return Err(ErrorCode::Internal( + "Pipeline max threads cannot equals zero.", + )); + } + + let on_init_callback = { + let pipelines_callback = pipelines + .iter_mut() + .map(|x| x.take_on_init()) + .collect::>(); + + pipelines_callback.into_iter().reduce(|left, right| { + Box::new(move || { + left()?; + right() + }) + }) + }; + + let on_finished_callback = { + let pipelines_callback = pipelines + .iter_mut() + .map(|x| x.take_on_finished()) + .collect::>(); + + pipelines_callback.into_iter().reduce(|left, right| { + Box::new(move |arg| { + left(arg)?; + right(arg) + }) + }) + }; + + let lock_guards = pipelines + .iter_mut() + .flat_map(|x| x.take_lock_guards()) + .collect::>(); + + match RunningGraph::from_pipelines(pipelines, 1) { + Err(cause) => { + if let Some(on_finished_callback) = on_finished_callback { + let _ = on_finished_callback(&Err(cause.clone())); + } + + Err(cause) + } + Ok(running_graph) => Self::try_create( + running_graph, + threads_num, + Mutex::new(on_init_callback), + Mutex::new(on_finished_callback), + settings, + lock_guards, + ), + } + } + + fn try_create( + graph: Arc, + threads_num: usize, + on_init_callback: Mutex>, + on_finished_callback: Mutex>, + settings: ExecutorSettings, + lock_guards: Vec, + ) -> Result> { + let workers_condvar = WorkersCondvar::create(threads_num); + let global_tasks_queue = QueryExecutorTasksQueue::create(threads_num); + + Ok(Arc::new(QueryPipelineExecutor { + graph, + threads_num, + workers_condvar, + global_tasks_queue, + on_init_callback, + on_finished_callback, + async_runtime: GlobalIORuntime::instance(), + settings, + finished_error: Mutex::new(None), + finished_notify: Arc::new(WatchNotify::new()), + lock_guards, + })) + } + + fn on_finished(&self, error: &Result>, ErrorCode>) -> Result<()> { + let mut guard = self.on_finished_callback.lock(); + if let Some(on_finished_callback) = guard.take() { + drop(guard); + catch_unwind(move || on_finished_callback(error))??; + } + Ok(()) + } + + pub fn finish(&self, cause: Option) { + if let Some(cause) = cause { + let mut finished_error = self.finished_error.lock(); + + // We only save the cause of the first error. + if finished_error.is_none() { + *finished_error = Some(cause); + } + } + + self.global_tasks_queue.finish(self.workers_condvar.clone()); + self.graph.interrupt_running_nodes(); + self.finished_notify.notify_waiters(); + } + + pub fn is_finished(&self) -> bool { + self.global_tasks_queue.is_finished() + } + + #[minitrace::trace] + pub fn execute(self: &Arc) -> Result<()> { + self.init(self.graph.clone())?; + + self.start_executor_daemon()?; + + let mut thread_join_handles = self.execute_threads(self.threads_num); + + while let Some(join_handle) = thread_join_handles.pop() { + let thread_res = join_handle.join().flatten(); + + { + let finished_error_guard = self.finished_error.lock(); + if let Some(error) = finished_error_guard.as_ref() { + let may_error = error.clone(); + drop(finished_error_guard); + + self.on_finished(&Err(may_error.clone()))?; + return Err(may_error); + } + } + + // We will ignore the abort query error, because returned by finished_error if abort query. + if matches!(&thread_res, Err(error) if error.code() != ErrorCode::ABORTED_QUERY) { + let may_error = thread_res.unwrap_err(); + self.on_finished(&Err(may_error.clone()))?; + return Err(may_error); + } + } + + if let Err(error) = self.graph.assert_finished_graph() { + self.on_finished(&Err(error.clone()))?; + return Err(error); + } + + self.on_finished(&Ok(self.graph.get_proc_profiles()))?; + Ok(()) + } + + fn init(self: &Arc, graph: Arc) -> Result<()> { + unsafe { + // TODO: the on init callback cannot be killed. + { + let instant = Instant::now(); + let mut guard = self.on_init_callback.lock(); + if let Some(callback) = guard.take() { + drop(guard); + if let Err(cause) = Result::flatten(catch_unwind(callback)) { + return Err(cause.add_message_back("(while in query pipeline init)")); + } + } + + info!( + "Init pipeline successfully, query_id: {:?}, elapsed: {:?}", + self.settings.query_id, + instant.elapsed() + ); + } + + let mut init_schedule_queue = graph.init_schedule_queue(self.threads_num)?; + + let mut wakeup_worker_id = 0; + while let Some(proc) = init_schedule_queue.async_queue.pop_front() { + ScheduleQueue::schedule_async_task( + proc.clone(), + self.settings.query_id.clone(), + self, + wakeup_worker_id, + self.workers_condvar.clone(), + self.global_tasks_queue.clone(), + ); + wakeup_worker_id += 1; + + if wakeup_worker_id == self.threads_num { + wakeup_worker_id = 0; + } + } + + let sync_queue = std::mem::take(&mut init_schedule_queue.sync_queue); + self.global_tasks_queue.init_sync_tasks(sync_queue); + Ok(()) + } + } + + fn start_executor_daemon(self: &Arc) -> Result<()> { + if !self.settings.max_execute_time_in_seconds.is_zero() { + // NOTE(wake ref): When runtime scheduling is blocked, holding executor strong ref may cause the executor can not stop. + let this = Arc::downgrade(self); + let max_execute_time_in_seconds = self.settings.max_execute_time_in_seconds; + let finished_notify = self.finished_notify.clone(); + self.async_runtime.spawn(GLOBAL_TASK, async move { + let finished_future = Box::pin(finished_notify.notified()); + let max_execute_future = Box::pin(tokio::time::sleep(max_execute_time_in_seconds)); + if let Either::Left(_) = select(max_execute_future, finished_future).await { + if let Some(executor) = this.upgrade() { + executor.finish(Some(ErrorCode::AbortedQuery( + "Aborted query, because the execution time exceeds the maximum execution time limit", + ))); + } + } + }); + } + + Ok(()) + } + + fn execute_threads(self: &Arc, threads: usize) -> Vec>> { + let mut thread_join_handles = Vec::with_capacity(threads); + + for thread_num in 0..threads { + let this = self.clone(); + #[allow(unused_mut)] + let mut name = format!("PipelineExecutor-{}", thread_num); + + #[cfg(debug_assertions)] + { + // We need to pass the thread name in the unit test, because the thread name is the test name + if matches!(std::env::var("UNIT_TEST"), Ok(var_value) if var_value == "TRUE") { + if let Some(cur_thread_name) = std::thread::current().name() { + name = cur_thread_name.to_string(); + } + } + } + + let span = Span::enter_with_local_parent(full_name!()) + .with_property(|| ("thread_name", name.clone())); + thread_join_handles.push(Thread::named_spawn(Some(name), move || unsafe { + let _g = span.set_local_parent(); + let this_clone = this.clone(); + let try_result = catch_unwind(move || -> Result<()> { + match this_clone.execute_single_thread(thread_num) { + Ok(_) => Ok(()), + Err(cause) => { + if log::max_level() == LevelFilter::Trace { + Err(cause.add_message_back(format!( + " (while in processor thread {})", + thread_num + ))) + } else { + Err(cause) + } + } + } + }); + + // finish the pipeline executor when has error or panic + if let Err(cause) = try_result.flatten() { + this.finish(Some(cause)); + } + + Ok(()) + })); + } + thread_join_handles + } + + /// # Safety + /// + /// Method is thread unsafe and require thread safe call + pub unsafe fn execute_single_thread(self: &Arc, thread_num: usize) -> Result<()> { + let workers_condvar = self.workers_condvar.clone(); + let mut context = ExecutorWorkerContext::create( + thread_num, + workers_condvar, + self.settings.query_id.clone(), + ); + + while !self.global_tasks_queue.is_finished() { + // When there are not enough tasks, the thread will be blocked, so we need loop check. + while !self.global_tasks_queue.is_finished() && !context.has_task() { + self.global_tasks_queue.steal_task_to_context(&mut context); + } + + while !self.global_tasks_queue.is_finished() && context.has_task() { + if let Some((executed_pid, graph)) = context.execute_task()? { + // Not scheduled graph if pipeline is finished. + if !self.global_tasks_queue.is_finished() { + // We immediately schedule the processor again. + let schedule_queue = graph.schedule_queue(executed_pid)?; + schedule_queue.schedule(&self.global_tasks_queue, &mut context, self); + } + } + } + } + + Ok(()) + } + + pub fn format_graph_nodes(&self) -> String { + self.graph.format_graph_nodes() + } + + pub fn get_profiles(&self) -> Vec> { + self.graph.get_proc_profiles() + } +} + +impl Drop for QueryPipelineExecutor { + fn drop(&mut self) { + self.finish(None); + + let mut guard = self.on_finished_callback.lock(); + if let Some(on_finished_callback) = guard.take() { + drop(guard); + let cause = match self.finished_error.lock().as_ref() { + Some(cause) => cause.clone(), + None => ErrorCode::Internal("Pipeline illegal state: not successfully shutdown."), + }; + if let Err(cause) = catch_unwind(move || on_finished_callback(&Err(cause))).flatten() { + warn!("Pipeline executor shutdown failure, {:?}", cause); + } + } + } +} diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/aggregate_exchange_injector.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/aggregate_exchange_injector.rs index 9c523cfefa50..25621a322f29 100644 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/aggregate_exchange_injector.rs +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/aggregate_exchange_injector.rs @@ -21,6 +21,9 @@ use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::BlockMetaInfoDowncast; use databend_common_expression::DataBlock; +use databend_common_expression::PartitionedPayload; +use databend_common_expression::Payload; +use databend_common_expression::PayloadFlushState; use databend_common_hashtable::FastHash; use databend_common_hashtable::HashtableEntryMutRefLike; use databend_common_hashtable::HashtableEntryRefLike; @@ -61,6 +64,10 @@ struct AggregateExchangeSorting, } +pub fn compute_block_number(bucket: isize, max_partition_count: usize) -> Result { + Ok(max_partition_count as isize * 1000 + bucket) +} + impl ExchangeSorting for AggregateExchangeSorting { @@ -75,11 +82,17 @@ impl ExchangeSorting ))), Some(meta_info) => match meta_info { AggregateMeta::Partitioned { .. } => unreachable!(), - AggregateMeta::Serialized(v) => Ok(v.bucket), + AggregateMeta::Serialized(v) => { + compute_block_number(v.bucket, v.max_partition_count) + } AggregateMeta::HashTable(v) => Ok(v.bucket), - AggregateMeta::Spilled(_) - | AggregateMeta::Spilling(_) - | AggregateMeta::BucketSpilled(_) => Ok(-1), + AggregateMeta::AggregatePayload(v) => { + compute_block_number(v.bucket, v.max_partition_count) + } + AggregateMeta::AggregateSpilling(_) + | AggregateMeta::Spilled(_) + | AggregateMeta::BucketSpilled(_) + | AggregateMeta::Spilling(_) => Ok(-1), }, } } @@ -138,6 +151,89 @@ fn scatter( Ok(res) } +fn scatter_payload(mut payload: Payload, buckets: usize) -> Result> { + let mut buckets = Vec::with_capacity(buckets); + + let group_types = payload.group_types.clone(); + let aggrs = payload.aggrs.clone(); + let mut state = PayloadFlushState::default(); + + for _ in 0..buckets.capacity() { + let p = Payload::new(payload.arena.clone(), group_types.clone(), aggrs.clone()); + buckets.push(p); + } + + // scatter each page of the payload. + while payload.scatter(&mut state, buckets.len()) { + // copy to the corresponding bucket. + for (idx, bucket) in buckets.iter_mut().enumerate() { + let count = state.probe_state.partition_count[idx]; + + if count > 0 { + let sel = &state.probe_state.partition_entries[idx]; + bucket.copy_rows(sel, count, &state.addresses); + } + } + } + payload.state_move_out = true; + + Ok(buckets) +} + +fn scatter_partitioned_payload( + partitioned_payload: PartitionedPayload, + buckets: usize, +) -> Result> { + let mut buckets = Vec::with_capacity(buckets); + + let group_types = partitioned_payload.group_types.clone(); + let aggrs = partitioned_payload.aggrs.clone(); + let partition_count = partitioned_payload.partition_count() as u64; + let mut state = PayloadFlushState::default(); + + for _ in 0..buckets.capacity() { + buckets.push(PartitionedPayload::new( + group_types.clone(), + aggrs.clone(), + partition_count, + partitioned_payload.arenas.clone(), + )); + } + + let mut payloads = Vec::with_capacity(buckets.len()); + + for _ in 0..payloads.capacity() { + payloads.push(Payload::new( + Arc::new(Bump::new()), + group_types.clone(), + aggrs.clone(), + )); + } + + for mut payload in partitioned_payload.payloads.into_iter() { + // scatter each page of the payload. + while payload.scatter(&mut state, buckets.len()) { + // copy to the corresponding bucket. + for (idx, bucket) in payloads.iter_mut().enumerate() { + let count = state.probe_state.partition_count[idx]; + + if count > 0 { + let sel = &state.probe_state.partition_entries[idx]; + bucket.copy_rows(sel, count, &state.addresses); + } + } + } + state.clear(); + payload.state_move_out = true; + } + + for (idx, payload) in payloads.into_iter().enumerate() { + buckets[idx].combine_single(payload, &mut state, None); + } + + Ok(buckets) +} + impl FlightScatter for HashTableHashScatter { @@ -161,6 +257,16 @@ impl FlightScatter }); } } + AggregateMeta::AggregateSpilling(payload) => { + for p in scatter_partitioned_payload(payload, self.buckets)? { + blocks.push(match p.len() == 0 { + true => DataBlock::empty(), + false => DataBlock::empty_with_meta( + AggregateMeta::::create_agg_spilling(p), + ), + }); + } + } AggregateMeta::HashTable(payload) => { let bucket = payload.bucket; for hashtable_cell in scatter(payload, self.buckets, &self.method)? { @@ -175,6 +281,20 @@ impl FlightScatter }); } } + AggregateMeta::AggregatePayload(p) => { + for payload in scatter_payload(p.payload, self.buckets)? { + blocks.push(match payload.len() == 0 { + true => DataBlock::empty(), + false => DataBlock::empty_with_meta( + AggregateMeta::::create_agg_payload( + p.bucket, + payload, + p.max_partition_count, + ), + ), + }); + } + } }; return Ok(blocks); diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/aggregate_meta.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/aggregate_meta.rs index 10824e7b7add..b6fd9186fa17 100644 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/aggregate_meta.rs +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/aggregate_meta.rs @@ -15,11 +15,21 @@ use std::fmt::Debug; use std::fmt::Formatter; use std::ops::Range; +use std::sync::Arc; +use bumpalo::Bump; +use databend_common_exception::Result; +use databend_common_expression::types::DataType; +use databend_common_expression::AggregateFunction; +use databend_common_expression::AggregateHashTable; use databend_common_expression::BlockMetaInfo; use databend_common_expression::BlockMetaInfoPtr; use databend_common_expression::Column; use databend_common_expression::DataBlock; +use databend_common_expression::HashTableConfig; +use databend_common_expression::PartitionedPayload; +use databend_common_expression::Payload; +use databend_common_expression::ProbeState; use crate::pipelines::processors::transforms::aggregator::HashTableCell; use crate::pipelines::processors::transforms::group_by::HashMethodBounds; @@ -33,6 +43,8 @@ pub struct HashTablePayload { pub struct SerializedPayload { pub bucket: isize, pub data_block: DataBlock, + // use for new agg_hashtable + pub max_partition_count: usize, } impl SerializedPayload { @@ -40,6 +52,69 @@ impl SerializedPayload { let entry = self.data_block.columns().last().unwrap(); entry.value.as_column().unwrap() } + + pub fn convert_to_aggregate_table( + &self, + group_types: Vec, + aggrs: Vec>, + radix_bits: u64, + arena: Arc, + need_init_entry: bool, + ) -> Result { + let rows_num = self.data_block.num_rows(); + let capacity = AggregateHashTable::get_capacity_for_count(rows_num); + let config = HashTableConfig::default().with_initial_radix_bits(radix_bits); + let mut state = ProbeState::default(); + let agg_len = aggrs.len(); + let group_len = group_types.len(); + let mut hashtable = AggregateHashTable::new_directly( + group_types, + aggrs, + config, + capacity, + arena, + need_init_entry, + ); + + let agg_states = (0..agg_len) + .map(|i| { + self.data_block + .get_by_offset(i) + .value + .as_column() + .unwrap() + .clone() + }) + .collect::>(); + let group_columns = (agg_len..(agg_len + group_len)) + .map(|i| { + self.data_block + .get_by_offset(i) + .value + .as_column() + .unwrap() + .clone() + }) + .collect::>(); + + let _ = + hashtable.add_groups(&mut state, &group_columns, &[vec![]], &agg_states, rows_num)?; + + hashtable.payload.mark_min_cardinality(); + Ok(hashtable) + } + + pub fn convert_to_partitioned_payload( + &self, + group_types: Vec, + aggrs: Vec>, + radix_bits: u64, + arena: Arc, + ) -> Result { + let hashtable = + self.convert_to_aggregate_table(group_types, aggrs, radix_bits, arena, false)?; + Ok(hashtable.payload) + } } pub struct BucketSpilledPayload { @@ -47,11 +122,21 @@ pub struct BucketSpilledPayload { pub location: String, pub data_range: Range, pub columns_layout: Vec, + pub max_partition_count: usize, +} + +pub struct AggregatePayload { + pub bucket: isize, + pub payload: Payload, + // use for new agg_hashtable + pub max_partition_count: usize, } pub enum AggregateMeta { Serialized(SerializedPayload), HashTable(HashTablePayload), + AggregatePayload(AggregatePayload), + AggregateSpilling(PartitionedPayload), BucketSpilled(BucketSpilledPayload), Spilled(Vec), Spilling(HashTablePayload, V>), @@ -67,10 +152,33 @@ impl AggregateMeta BlockMetaInfoPtr { + pub fn create_agg_payload( + bucket: isize, + payload: Payload, + max_partition_count: usize, + ) -> BlockMetaInfoPtr { + Box::new(AggregateMeta::::AggregatePayload( + AggregatePayload { + bucket, + payload, + max_partition_count, + }, + )) + } + + pub fn create_agg_spilling(payload: PartitionedPayload) -> BlockMetaInfoPtr { + Box::new(AggregateMeta::::AggregateSpilling(payload)) + } + + pub fn create_serialized( + bucket: isize, + block: DataBlock, + max_partition_count: usize, + ) -> BlockMetaInfoPtr { Box::new(AggregateMeta::::Serialized(SerializedPayload { bucket, data_block: block, + max_partition_count, })) } @@ -127,6 +235,12 @@ impl Debug for AggregateMeta AggregateMeta::Spilling(_) => f.debug_struct("Aggregate::Spilling").finish(), AggregateMeta::Spilled(_) => f.debug_struct("Aggregate::Spilled").finish(), AggregateMeta::BucketSpilled(_) => f.debug_struct("Aggregate::BucketSpilled").finish(), + AggregateMeta::AggregatePayload(_) => { + f.debug_struct("AggregateMeta:AggregatePayload").finish() + } + AggregateMeta::AggregateSpilling(_) => { + f.debug_struct("AggregateMeta:AggregateSpilling").finish() + } } } } diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/aggregator_params.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/aggregator_params.rs index 75204c68ce55..e30a1a5addb5 100644 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/aggregator_params.rs +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/aggregator_params.rs @@ -17,11 +17,14 @@ use std::sync::Arc; use databend_common_exception::Result; use databend_common_expression::types::DataType; +use databend_common_expression::ColumnBuilder; +use databend_common_expression::DataBlock; use databend_common_expression::DataSchemaRef; use databend_common_functions::aggregates::get_layout_offsets; use databend_common_functions::aggregates::AggregateFunctionRef; use databend_common_functions::aggregates::StateAddr; use databend_common_sql::IndexType; +use itertools::Itertools; use crate::pipelines::processors::transforms::group_by::Area; @@ -38,6 +41,9 @@ pub struct AggregatorParams { pub layout: Option, pub offsets_aggregate_states: Vec, + pub enable_experimental_aggregate_hashtable: bool, + pub in_cluster: bool, + pub max_block_size: usize, // Limit is push down to AggregatorTransform pub limit: Option, } @@ -49,6 +55,9 @@ impl AggregatorParams { group_columns: &[usize], agg_funcs: &[AggregateFunctionRef], agg_args: &[Vec], + enable_experimental_aggregate_hashtable: bool, + in_cluster: bool, + max_block_size: usize, limit: Option, ) -> Result> { let mut states_offsets: Vec = Vec::with_capacity(agg_funcs.len()); @@ -66,6 +75,9 @@ impl AggregatorParams { aggregate_functions_arguments: agg_args.to_vec(), layout: states_layout, offsets_aggregate_states: states_offsets, + enable_experimental_aggregate_hashtable, + in_cluster, + max_block_size, limit, })) } @@ -87,4 +99,18 @@ impl AggregatorParams { .iter() .any(|f| f.name().contains("DistinctCombinator")) } + + pub fn empty_result_block(&self) -> DataBlock { + let columns = self + .aggregate_functions + .iter() + .map(|f| ColumnBuilder::with_capacity(&f.return_type().unwrap(), 0).build()) + .chain( + self.group_data_types + .iter() + .map(|t| ColumnBuilder::with_capacity(t, 0).build()), + ) + .collect_vec(); + DataBlock::new_from_columns(columns) + } } diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/mod.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/mod.rs index 8197fd8d552a..2a7b442229d0 100644 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/mod.rs +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/mod.rs @@ -16,6 +16,7 @@ mod aggregate_cell; mod aggregate_exchange_injector; mod aggregate_meta; mod aggregator_params; +mod new_transform_partition_bucket; mod serde; mod transform_aggregate_expand; mod transform_aggregate_final; diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/new_transform_partition_bucket.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/new_transform_partition_bucket.rs new file mode 100644 index 000000000000..216007252714 --- /dev/null +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/new_transform_partition_bucket.rs @@ -0,0 +1,576 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed 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 std::any::Any; +use std::collections::btree_map::Entry; +use std::collections::BTreeMap; +use std::marker::PhantomData; +use std::sync::Arc; + +use bumpalo::Bump; +use databend_common_exception::ErrorCode; +use databend_common_exception::Result; +use databend_common_expression::BlockMetaInfoDowncast; +use databend_common_expression::DataBlock; +use databend_common_expression::PartitionedPayload; +use databend_common_expression::PayloadFlushState; +use databend_common_pipeline_core::processors::Event; +use databend_common_pipeline_core::processors::InputPort; +use databend_common_pipeline_core::processors::OutputPort; +use databend_common_pipeline_core::processors::Processor; + +use super::AggregatePayload; +use crate::pipelines::processors::transforms::aggregator::aggregate_meta::AggregateMeta; +use crate::pipelines::processors::transforms::aggregator::aggregate_meta::SerializedPayload; +use crate::pipelines::processors::transforms::aggregator::AggregatorParams; +use crate::pipelines::processors::transforms::group_by::HashMethodBounds; + +static SINGLE_LEVEL_BUCKET_NUM: isize = -1; +static MAX_PARTITION_COUNT: usize = 128; + +struct InputPortState { + port: Arc, + bucket: isize, + max_partition_count: usize, +} +pub struct NewTransformPartitionBucket { + output: Arc, + inputs: Vec, + params: Arc, + working_bucket: isize, + pushing_bucket: isize, + initialized_all_inputs: bool, + all_inputs_init: bool, + buckets_blocks: BTreeMap>, + flush_state: PayloadFlushState, + unpartitioned_blocks: Vec, + max_partition_count: usize, + _phantom: PhantomData<(Method, V)>, +} + +impl + NewTransformPartitionBucket +{ + pub fn create(input_nums: usize, params: Arc) -> Result { + let mut inputs = Vec::with_capacity(input_nums); + + for _index in 0..input_nums { + inputs.push(InputPortState { + bucket: -1, + port: InputPort::create(), + max_partition_count: 0, + }); + } + + Ok(NewTransformPartitionBucket { + params, + inputs, + working_bucket: 0, + pushing_bucket: 0, + output: OutputPort::create(), + buckets_blocks: BTreeMap::new(), + unpartitioned_blocks: vec![], + flush_state: PayloadFlushState::default(), + initialized_all_inputs: false, + all_inputs_init: false, + max_partition_count: 0, + _phantom: Default::default(), + }) + } + + pub fn get_inputs(&self) -> Vec> { + let mut inputs = Vec::with_capacity(self.inputs.len()); + + for input_state in &self.inputs { + inputs.push(input_state.port.clone()); + } + + inputs + } + + pub fn get_output(&self) -> Arc { + self.output.clone() + } + + fn initialize_all_inputs(&mut self) -> Result { + self.initialized_all_inputs = true; + // in a cluster where partitions are only 8 and 128, + // we need to pull all data where the partition equals 8 until the partition changes to 128 or there is no data available. + if self.params.in_cluster { + for index in 0..self.inputs.len() { + if self.inputs[index].port.is_finished() { + continue; + } + + // We pull all the data that are not the max_partition_count and all spill data + if self.inputs[index].max_partition_count == MAX_PARTITION_COUNT + && self.inputs[index].bucket > SINGLE_LEVEL_BUCKET_NUM + { + continue; + } + + if !self.inputs[index].port.has_data() { + self.inputs[index].port.set_need_data(); + self.initialized_all_inputs = false; + continue; + } + + let data_block = self.inputs[index].port.pull_data().unwrap()?; + + ( + self.inputs[index].bucket, + self.inputs[index].max_partition_count, + ) = self.add_bucket(data_block)?; + + // we need pull all spill data in init, and data less than max partition + if self.inputs[index].bucket <= SINGLE_LEVEL_BUCKET_NUM + || self.inputs[index].max_partition_count < MAX_PARTITION_COUNT + { + self.inputs[index].port.set_need_data(); + self.initialized_all_inputs = false; + } + } + } else { + // in singleton, the partition is 8, 32, 128. + // We pull the first data to ensure the max partition, + // and then pull all data that is less than the max partition + let mut refresh_index = 0; + for index in 0..self.inputs.len() { + if self.inputs[index].port.is_finished() { + continue; + } + + // We pull all the data that are not the max_partition_count + if self.inputs[index].max_partition_count > 0 + && self.inputs[index].bucket > SINGLE_LEVEL_BUCKET_NUM + && self.inputs[index].max_partition_count == self.max_partition_count + { + continue; + } + + if !self.inputs[index].port.has_data() { + self.inputs[index].port.set_need_data(); + self.initialized_all_inputs = false; + continue; + } + + let data_block = self.inputs[index].port.pull_data().unwrap()?; + + let before_max_partition_count = self.max_partition_count; + ( + self.inputs[index].bucket, + self.inputs[index].max_partition_count, + ) = self.add_bucket(data_block)?; + + // we need pull all spill data in init, and data less than max partition + if self.inputs[index].bucket <= SINGLE_LEVEL_BUCKET_NUM + || self.inputs[index].max_partition_count < self.max_partition_count + { + self.inputs[index].port.set_need_data(); + self.initialized_all_inputs = false; + } + + // max partition count change + if before_max_partition_count > 0 + && before_max_partition_count != self.max_partition_count + { + // set need data for inputs which is less than the max partition + for i in refresh_index..index { + if !self.inputs[i].port.is_finished() + && !self.inputs[i].port.has_data() + && self.inputs[i].max_partition_count != self.max_partition_count + { + self.inputs[i].port.set_need_data(); + self.initialized_all_inputs = false; + } + } + refresh_index = index; + } + } + } + + if self.initialized_all_inputs { + self.all_inputs_init = true; + } + + Ok(self.initialized_all_inputs) + } + + #[allow(unused_assignments)] + fn add_bucket(&mut self, mut data_block: DataBlock) -> Result<(isize, usize)> { + let (mut bucket, mut partition_count) = (0, 0); + if let Some(block_meta) = data_block.get_meta() { + if let Some(block_meta) = AggregateMeta::::downcast_ref_from(block_meta) { + (bucket, partition_count) = match block_meta { + AggregateMeta::Spilling(_) => unreachable!(), + AggregateMeta::Partitioned { .. } => unreachable!(), + AggregateMeta::AggregateSpilling(_) => unreachable!(), + AggregateMeta::HashTable(_) => unreachable!(), + AggregateMeta::BucketSpilled(_) => { + let meta = data_block.take_meta().unwrap(); + + if let Some(AggregateMeta::BucketSpilled(payload)) = + AggregateMeta::::downcast_from(meta) + { + let bucket = payload.bucket; + let partition_count = payload.max_partition_count; + self.max_partition_count = + self.max_partition_count.max(partition_count); + + let data_block = DataBlock::empty_with_meta( + AggregateMeta::::create_bucket_spilled(payload), + ); + match self.buckets_blocks.entry(bucket) { + Entry::Vacant(v) => { + v.insert(vec![data_block]); + } + Entry::Occupied(mut v) => { + v.get_mut().push(data_block); + } + }; + + return Ok((SINGLE_LEVEL_BUCKET_NUM, partition_count)); + } + unreachable!() + } + AggregateMeta::Spilled(_) => { + let meta = data_block.take_meta().unwrap(); + + if let Some(AggregateMeta::Spilled(buckets_payload)) = + AggregateMeta::::downcast_from(meta) + { + let partition_count = buckets_payload[0].max_partition_count; + self.max_partition_count = + self.max_partition_count.max(partition_count); + + for bucket_payload in buckets_payload { + let bucket = bucket_payload.bucket; + let data_block = DataBlock::empty_with_meta(AggregateMeta::< + Method, + V, + >::create_bucket_spilled( + bucket_payload + )); + match self.buckets_blocks.entry(bucket) { + Entry::Vacant(v) => { + v.insert(vec![data_block]); + } + Entry::Occupied(mut v) => { + v.get_mut().push(data_block); + } + }; + } + + return Ok((SINGLE_LEVEL_BUCKET_NUM, partition_count)); + } + unreachable!() + } + AggregateMeta::Serialized(payload) => { + self.max_partition_count = + self.max_partition_count.max(payload.max_partition_count); + + (payload.bucket, payload.max_partition_count) + } + AggregateMeta::AggregatePayload(payload) => { + self.max_partition_count = + self.max_partition_count.max(payload.max_partition_count); + + (payload.bucket, payload.max_partition_count) + } + }; + } else { + return Err(ErrorCode::Internal(format!( + "Internal, TransformPartitionBucket only recv AggregateMeta, but got {:?}", + block_meta + ))); + } + } else { + return Err(ErrorCode::Internal( + "Internal, TransformPartitionBucket only recv DataBlock with meta.", + )); + } + + if self.all_inputs_init { + if partition_count != self.max_partition_count { + return Err(ErrorCode::Internal( + "Internal, the partition count does not equal the max partition count on TransformPartitionBucket. + ", + )); + } + match self.buckets_blocks.entry(bucket) { + Entry::Vacant(v) => { + v.insert(vec![data_block]); + } + Entry::Occupied(mut v) => { + v.get_mut().push(data_block); + } + }; + } else { + self.unpartitioned_blocks.push(data_block); + } + + Ok((bucket, partition_count)) + } + + fn try_push_data_block(&mut self) -> bool { + while self.pushing_bucket < self.working_bucket { + if let Some(bucket_blocks) = self.buckets_blocks.remove(&self.pushing_bucket) { + let data_block = Self::convert_blocks(self.pushing_bucket, bucket_blocks); + self.output.push_data(Ok(data_block)); + self.pushing_bucket += 1; + return true; + } + + self.pushing_bucket += 1; + } + + false + } + + fn partition_block(&mut self, payload: SerializedPayload) -> Result>> { + // already is max partition + if payload.max_partition_count == self.max_partition_count { + let bucket = payload.bucket; + let data_block = DataBlock::empty_with_meta(Box::new( + AggregateMeta::::Serialized(payload), + )); + match self.buckets_blocks.entry(bucket) { + Entry::Vacant(v) => { + v.insert(vec![data_block]); + } + Entry::Occupied(mut v) => { + v.get_mut().push(data_block); + } + }; + return Ok(vec![]); + } + + // need repartition + let mut blocks = Vec::with_capacity(self.max_partition_count); + let p = payload.convert_to_partitioned_payload( + self.params.group_data_types.clone(), + self.params.aggregate_functions.clone(), + 0, + Arc::new(Bump::new()), + )?; + + let mut partitioned_payload = PartitionedPayload::new( + self.params.group_data_types.clone(), + self.params.aggregate_functions.clone(), + self.max_partition_count as u64, + p.arenas.clone(), + ); + partitioned_payload.combine(p, &mut self.flush_state); + + for (bucket, payload) in partitioned_payload.payloads.into_iter().enumerate() { + blocks.push(Some(DataBlock::empty_with_meta( + AggregateMeta::::create_agg_payload( + bucket as isize, + payload, + self.max_partition_count, + ), + ))); + } + + Ok(blocks) + } + + fn partition_payload(&mut self, payload: AggregatePayload) -> Result>> { + // already is max partition + if payload.max_partition_count == self.max_partition_count { + let bucket = payload.bucket; + let data_block = DataBlock::empty_with_meta(Box::new( + AggregateMeta::::AggregatePayload(payload), + )); + match self.buckets_blocks.entry(bucket) { + Entry::Vacant(v) => { + v.insert(vec![data_block]); + } + Entry::Occupied(mut v) => { + v.get_mut().push(data_block); + } + }; + return Ok(vec![]); + } + + // need repartition + let mut blocks = Vec::with_capacity(self.max_partition_count); + let mut partitioned_payload = PartitionedPayload::new( + self.params.group_data_types.clone(), + self.params.aggregate_functions.clone(), + self.max_partition_count as u64, + vec![payload.payload.arena.clone()], + ); + + partitioned_payload.combine_single(payload.payload, &mut self.flush_state, None); + + for (bucket, payload) in partitioned_payload.payloads.into_iter().enumerate() { + blocks.push(Some(DataBlock::empty_with_meta( + AggregateMeta::::create_agg_payload( + bucket as isize, + payload, + self.max_partition_count, + ), + ))); + } + + Ok(blocks) + } + + fn convert_blocks(bucket: isize, data_blocks: Vec) -> DataBlock { + let mut data = Vec::with_capacity(data_blocks.len()); + for mut data_block in data_blocks.into_iter() { + if let Some(block_meta) = data_block.take_meta() { + if let Some(block_meta) = AggregateMeta::::downcast_from(block_meta) { + data.push(block_meta); + } + } + } + + DataBlock::empty_with_meta(AggregateMeta::::create_partitioned(bucket, data)) + } +} + +#[async_trait::async_trait] +impl Processor + for NewTransformPartitionBucket +{ + fn name(&self) -> String { + String::from("TransformPartitionBucket") + } + + fn as_any(&mut self) -> &mut dyn Any { + self + } + + fn event(&mut self) -> Result { + if self.output.is_finished() { + for input_state in &self.inputs { + input_state.port.finish(); + } + + self.buckets_blocks.clear(); + return Ok(Event::Finished); + } + + // We pull the first unsplitted data block + if !self.initialized_all_inputs && !self.initialize_all_inputs()? { + return Ok(Event::NeedData); + } + + if !self.unpartitioned_blocks.is_empty() { + // Split data blocks if it's unsplitted. + return Ok(Event::Sync); + } + + if !self.output.can_push() { + for input_state in &self.inputs { + input_state.port.set_not_need_data(); + } + + return Ok(Event::NeedConsume); + } + + let pushed_data_block = self.try_push_data_block(); + + loop { + // Try to pull the next data or until the port is closed + let mut all_inputs_is_finished = true; + let mut all_port_prepared_data = true; + for index in 0..self.inputs.len() { + if self.inputs[index].port.is_finished() { + continue; + } + + all_inputs_is_finished = false; + if self.inputs[index].bucket > self.working_bucket { + continue; + } + + if !self.inputs[index].port.has_data() { + all_port_prepared_data = false; + self.inputs[index].port.set_need_data(); + continue; + } + + let data_block = self.inputs[index].port.pull_data().unwrap()?; + (self.inputs[index].bucket, _) = self.add_bucket(data_block)?; + + if self.inputs[index].bucket <= self.working_bucket { + all_port_prepared_data = false; + self.inputs[index].port.set_need_data(); + } + } + + if all_inputs_is_finished { + break; + } + + if !all_port_prepared_data { + return Ok(Event::NeedData); + } + + self.working_bucket += 1; + } + + if pushed_data_block || self.try_push_data_block() { + return Ok(Event::NeedConsume); + } + + if let Some((bucket, bucket_blocks)) = self.buckets_blocks.pop_first() { + let data_block = Self::convert_blocks(bucket, bucket_blocks); + self.output.push_data(Ok(data_block)); + return Ok(Event::NeedConsume); + } + + self.output.finish(); + Ok(Event::Finished) + } + + fn process(&mut self) -> Result<()> { + let block_meta = self + .unpartitioned_blocks + .pop() + .and_then(|mut block| block.take_meta()) + .and_then(AggregateMeta::::downcast_from); + + if let Some(agg_block_meta) = block_meta { + let data_blocks = match agg_block_meta { + AggregateMeta::Spilled(_) => unreachable!(), + AggregateMeta::Spilling(_) => unreachable!(), + AggregateMeta::Partitioned { .. } => unreachable!(), + AggregateMeta::HashTable(_) => unreachable!(), + AggregateMeta::AggregateSpilling(_) => unreachable!(), + AggregateMeta::BucketSpilled(_) => unreachable!(), + AggregateMeta::Serialized(payload) => self.partition_block(payload)?, + AggregateMeta::AggregatePayload(payload) => self.partition_payload(payload)?, + }; + + for (bucket, block) in data_blocks.into_iter().enumerate() { + if let Some(data_block) = block { + match self.buckets_blocks.entry(bucket as isize) { + Entry::Vacant(v) => { + v.insert(vec![data_block]); + } + Entry::Occupied(mut v) => { + v.get_mut().push(data_block); + } + }; + } + } + } + + Ok(()) + } +} diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/serde/serde_meta.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/serde/serde_meta.rs index 9e10af17b7c0..731ec4e1b104 100644 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/serde/serde_meta.rs +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/serde/serde_meta.rs @@ -29,6 +29,8 @@ pub struct AggregateSerdeMeta { pub location: Option, pub data_range: Option>, pub columns_layout: Vec, + // use for new agg hashtable + pub max_partition_count: usize, } impl AggregateSerdeMeta { @@ -39,6 +41,18 @@ impl AggregateSerdeMeta { location: None, data_range: None, columns_layout: vec![], + max_partition_count: 0, + }) + } + + pub fn create_agg_payload(bucket: isize, max_partition_count: usize) -> BlockMetaInfoPtr { + Box::new(AggregateSerdeMeta { + typ: BUCKET_TYPE, + bucket, + location: None, + data_range: None, + columns_layout: vec![], + max_partition_count, }) } @@ -54,6 +68,24 @@ impl AggregateSerdeMeta { columns_layout, location: Some(location), data_range: Some(data_range), + max_partition_count: 0, + }) + } + + pub fn create_agg_spilled( + bucket: isize, + location: String, + data_range: Range, + columns_layout: Vec, + max_partition_count: usize, + ) -> BlockMetaInfoPtr { + Box::new(AggregateSerdeMeta { + typ: SPILLED_TYPE, + bucket, + columns_layout, + location: Some(location), + data_range: Some(data_range), + max_partition_count, }) } } diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_aggregate_serializer.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_aggregate_serializer.rs index 67654e0e47b0..27d721233087 100644 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_aggregate_serializer.rs +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_aggregate_serializer.rs @@ -22,6 +22,7 @@ use databend_common_expression::types::binary::BinaryColumnBuilder; use databend_common_expression::BlockMetaInfoDowncast; use databend_common_expression::Column; use databend_common_expression::DataBlock; +use databend_common_expression::PayloadFlushState; use databend_common_functions::aggregates::StateAddr; use databend_common_hashtable::HashtableEntryRefLike; use databend_common_hashtable::HashtableLike; @@ -31,15 +32,14 @@ use databend_common_pipeline_core::processors::OutputPort; use databend_common_pipeline_core::processors::Processor; use databend_common_pipeline_core::processors::ProcessorPtr; +use super::SerializePayload; use crate::pipelines::processors::transforms::aggregator::create_state_serializer; use crate::pipelines::processors::transforms::aggregator::estimated_key_size; use crate::pipelines::processors::transforms::aggregator::AggregateMeta; use crate::pipelines::processors::transforms::aggregator::AggregateSerdeMeta; use crate::pipelines::processors::transforms::aggregator::AggregatorParams; -use crate::pipelines::processors::transforms::aggregator::HashTablePayload; use crate::pipelines::processors::transforms::group_by::HashMethodBounds; use crate::pipelines::processors::transforms::group_by::KeysColumnBuilder; - pub struct TransformAggregateSerializer { method: Method, params: Arc, @@ -137,11 +137,20 @@ impl TransformAggregateSerializer { AggregateMeta::Serialized(_) => unreachable!(), AggregateMeta::BucketSpilled(_) => unreachable!(), AggregateMeta::Partitioned { .. } => unreachable!(), + AggregateMeta::AggregateSpilling(_) => unreachable!(), AggregateMeta::HashTable(payload) => { self.input_data = Some(SerializeAggregateStream::create( &self.method, &self.params, - payload, + SerializePayload::::HashTablePayload(payload), + )); + return Ok(Event::Sync); + } + AggregateMeta::AggregatePayload(p) => { + self.input_data = Some(SerializeAggregateStream::create( + &self.method, + &self.params, + SerializePayload::::AggregatePayload(p), )); return Ok(Event::Sync); } @@ -197,8 +206,10 @@ pub fn serialize_aggregate( pub struct SerializeAggregateStream { method: Method, params: Arc, - pub payload: Pin>>, - iter: as HashtableLike>::Iterator<'static>, + pub payload: Pin>>, + // old hashtable' iter + iter: Option< as HashtableLike>::Iterator<'static>>, + flush_state: Option, end_iter: bool, } @@ -210,18 +221,29 @@ impl SerializeAggregateStream { pub fn create( method: &Method, params: &Arc, - payload: HashTablePayload, + payload: SerializePayload, ) -> Self { unsafe { let payload = Box::pin(payload); - let point = NonNull::from(&payload.cell.hashtable); - let iter = point.as_ref().iter(); + let iter = if let SerializePayload::HashTablePayload(p) = payload.as_ref().get_ref() { + Some(NonNull::from(&p.cell.hashtable).as_ref().iter()) + } else { + None + }; + + let flush_state = + if let SerializePayload::AggregatePayload(_) = payload.as_ref().get_ref() { + Some(PayloadFlushState::default()) + } else { + None + }; SerializeAggregateStream:: { iter, payload, end_iter: false, + flush_state, method: method.clone(), params: params.clone(), } @@ -243,49 +265,65 @@ impl SerializeAggregateStream { return Ok(None); } - let max_block_rows = std::cmp::min(8192, self.payload.cell.hashtable.len()); - let max_block_bytes = std::cmp::min( - 8 * 1024 * 1024 + 1024, - self.payload - .cell - .hashtable - .unsize_key_size() - .unwrap_or(usize::MAX), - ); - - let funcs = &self.params.aggregate_functions; - let offsets_aggregate_states = &self.params.offsets_aggregate_states; - - let mut state_builders: Vec = funcs - .iter() - .map(|func| create_state_serializer(func, max_block_rows)) - .collect(); - - let mut group_key_builder = self - .method - .keys_column_builder(max_block_rows, max_block_bytes); - - #[allow(clippy::while_let_on_iterator)] - while let Some(group_entity) = self.iter.next() { - let mut bytes = 0; - let place = Into::::into(*group_entity.get()); - - for (idx, func) in funcs.iter().enumerate() { - let arg_place = place.next(offsets_aggregate_states[idx]); - func.serialize(arg_place, &mut state_builders[idx].data)?; - state_builders[idx].commit_row(); - bytes += state_builders[idx].memory_size(); + match self.payload.as_ref().get_ref() { + SerializePayload::HashTablePayload(p) => { + let max_block_rows = std::cmp::min(8192, p.cell.hashtable.len()); + let max_block_bytes = std::cmp::min( + 8 * 1024 * 1024 + 1024, + p.cell.hashtable.unsize_key_size().unwrap_or(usize::MAX), + ); + + let funcs = &self.params.aggregate_functions; + let offsets_aggregate_states = &self.params.offsets_aggregate_states; + + let mut state_builders: Vec = funcs + .iter() + .map(|func| create_state_serializer(func, max_block_rows)) + .collect(); + + let mut group_key_builder = self + .method + .keys_column_builder(max_block_rows, max_block_bytes); + + let mut bytes = 0; + + #[allow(clippy::while_let_on_iterator)] + while let Some(group_entity) = self.iter.as_mut().and_then(|iter| iter.next()) { + let place = Into::::into(*group_entity.get()); + + for (idx, func) in funcs.iter().enumerate() { + let arg_place = place.next(offsets_aggregate_states[idx]); + func.serialize(arg_place, &mut state_builders[idx].data)?; + state_builders[idx].commit_row(); + bytes += state_builders[idx].memory_size(); + } + + group_key_builder.append_value(group_entity.key()); + + if bytes + group_key_builder.bytes_size() >= 8 * 1024 * 1024 { + return self.finish(state_builders, group_key_builder); + } + } + + self.end_iter = true; + self.finish(state_builders, group_key_builder) } + SerializePayload::AggregatePayload(p) => { + let state = self.flush_state.as_mut().unwrap(); + let block = p.payload.aggregate_flush(state)?; - group_key_builder.append_value(group_entity.key()); + if block.is_none() { + self.end_iter = true; + } - if bytes >= 8 * 1024 * 1024 { - return self.finish(state_builders, group_key_builder); + match block { + Some(block) => Ok(Some(block.add_meta(Some( + AggregateSerdeMeta::create_agg_payload(p.bucket, p.max_partition_count), + ))?)), + None => Ok(None), + } } } - - self.end_iter = true; - self.finish(state_builders, group_key_builder) } fn finish( @@ -299,7 +337,12 @@ impl SerializeAggregateStream { columns.push(Column::Binary(builder.build())); } - let bucket = self.payload.bucket; + let bucket = if let SerializePayload::HashTablePayload(p) = self.payload.as_ref().get_ref() + { + p.bucket + } else { + 0 + }; columns.push(group_key_builder.finish()); let block = DataBlock::new_from_columns(columns); Ok(Some( diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_aggregate_spill_writer.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_aggregate_spill_writer.rs index cfb2be722203..4b903ca2eb38 100644 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_aggregate_spill_writer.rs +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_aggregate_spill_writer.rs @@ -24,6 +24,7 @@ use databend_common_exception::Result; use databend_common_expression::arrow::serialize_column; use databend_common_expression::BlockMetaInfoDowncast; use databend_common_expression::DataBlock; +use databend_common_expression::PartitionedPayload; use databend_common_hashtable::HashtableLike; use databend_common_metrics::transform::*; use databend_common_pipeline_core::processors::Event; @@ -135,6 +136,13 @@ impl Processor for TransformAggregateSpillWriter::downcast_from(block_meta); return Ok(Event::Sync); } + + if matches!(block_meta, AggregateMeta::AggregateSpilling(_)) { + self.input.set_not_need_data(); + let block_meta = data_block.take_meta().unwrap(); + self.spilling_meta = AggregateMeta::::downcast_from(block_meta); + return Ok(Event::Sync); + } } self.output.push_data(Ok(data_block)); @@ -152,20 +160,34 @@ impl Processor for TransformAggregateSpillWriter Result<()> { if let Some(spilling_meta) = self.spilling_meta.take() { - if let AggregateMeta::Spilling(payload) = spilling_meta { - self.spilling_future = Some(spilling_aggregate_payload( - self.ctx.clone(), - self.operator.clone(), - &self.method, - &self.location_prefix, - &self.params, - payload, - )?); - - return Ok(()); - } + match spilling_meta { + AggregateMeta::Spilling(payload) => { + self.spilling_future = Some(spilling_aggregate_payload( + self.ctx.clone(), + self.operator.clone(), + &self.method, + &self.location_prefix, + &self.params, + payload, + )?); + + return Ok(()); + } + AggregateMeta::AggregateSpilling(payload) => { + self.spilling_future = Some(agg_spilling_aggregate_payload::( + self.ctx.clone(), + self.operator.clone(), + &self.location_prefix, + payload, + )?); + + return Ok(()); + } - return Err(ErrorCode::Internal("")); + _ => { + return Err(ErrorCode::Internal("")); + } + } } Ok(()) @@ -181,6 +203,113 @@ impl Processor for TransformAggregateSpillWriter( + ctx: Arc, + operator: Operator, + location_prefix: &str, + partitioned_payload: PartitionedPayload, +) -> Result>> { + let unique_name = GlobalUniqName::unique(); + let location = format!("{}/{}", location_prefix, unique_name); + + let mut write_size = 0; + let partition_count = partitioned_payload.partition_count(); + let mut write_data = Vec::with_capacity(partition_count); + let mut spilled_buckets_payloads = Vec::with_capacity(partition_count); + // Record how many rows are spilled. + let mut rows = 0; + for (bucket, payload) in partitioned_payload.payloads.into_iter().enumerate() { + if payload.len() == 0 { + continue; + } + + let now = Instant::now(); + let data_block = payload.aggregate_flush_all()?; + rows += data_block.num_rows(); + + let begin = write_size; + let columns = data_block.columns().to_vec(); + let mut columns_data = Vec::with_capacity(columns.len()); + let mut columns_layout = Vec::with_capacity(columns.len()); + for column in columns.into_iter() { + let column = column.value.as_column().unwrap(); + let column_data = serialize_column(column); + write_size += column_data.len() as u64; + columns_layout.push(column_data.len() as u64); + columns_data.push(column_data); + } + + // perf + { + metrics_inc_aggregate_spill_data_serialize_milliseconds( + now.elapsed().as_millis() as u64 + ); + } + + write_data.push(columns_data); + spilled_buckets_payloads.push(BucketSpilledPayload { + bucket: bucket as isize, + location: location.clone(), + data_range: begin..write_size, + columns_layout, + max_partition_count: partition_count, + }); + } + + Ok(Box::pin(async move { + let instant = Instant::now(); + + let mut write_bytes = 0; + + if !write_data.is_empty() { + let mut writer = operator + .writer_with(&location) + .buffer(8 * 1024 * 1024) + .await?; + for write_bucket_data in write_data.into_iter() { + for data in write_bucket_data.into_iter() { + write_bytes += data.len(); + writer.write(data).await?; + } + } + + writer.close().await?; + } + + // perf + { + metrics_inc_aggregate_spill_write_count(); + metrics_inc_aggregate_spill_write_bytes(write_bytes as u64); + metrics_inc_aggregate_spill_write_milliseconds(instant.elapsed().as_millis() as u64); + + Profile::record_usize_profile(ProfileStatisticsName::SpillWriteCount, 1); + Profile::record_usize_profile(ProfileStatisticsName::SpillWriteBytes, write_bytes); + Profile::record_usize_profile( + ProfileStatisticsName::SpillWriteTime, + instant.elapsed().as_millis() as usize, + ); + } + + { + let progress_val = ProgressValues { + rows, + bytes: write_bytes, + }; + ctx.get_aggregate_spill_progress().incr(&progress_val); + } + + info!( + "Write aggregate spill {} successfully, elapsed: {:?}", + location, + instant.elapsed() + ); + + Ok(DataBlock::empty_with_meta( + AggregateMeta::::create_spilled(spilled_buckets_payloads), + )) + })) +} + pub fn spilling_aggregate_payload( ctx: Arc, operator: Operator, @@ -232,6 +361,7 @@ pub fn spilling_aggregate_payload( location: location.clone(), data_range: begin..write_size, columns_layout, + max_partition_count: 0, }); } diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_deserializer.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_deserializer.rs index 01a2a3a50c06..514e1a037aee 100644 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_deserializer.rs +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_deserializer.rs @@ -118,6 +118,7 @@ impl TransformDeserializer { @@ -163,6 +164,7 @@ impl TransformDeserializer BlockMetaTransform }, )); } - + Some(AggregateMeta::AggregateSpilling(payload)) => { + serialized_blocks.push(FlightSerialized::Future( + match index == self.local_pos { + true => local_agg_spilling_aggregate_payload::( + self.ctx.clone(), + self.operator.clone(), + &self.location_prefix, + payload, + )?, + false => agg_spilling_aggregate_payload::( + self.ctx.clone(), + self.operator.clone(), + &self.location_prefix, + payload, + )?, + }, + )); + } Some(AggregateMeta::HashTable(payload)) => { if index == self.local_pos { serialized_blocks.push(FlightSerialized::DataBlock(block.add_meta( @@ -165,15 +186,56 @@ impl BlockMetaTransform continue; } - let mut stream = - SerializeAggregateStream::create(&self.method, &self.params, payload); - let bucket = stream.payload.bucket; - serialized_blocks.push(FlightSerialized::DataBlock(match stream.next() { - None => DataBlock::empty(), - Some(data_block) => { - serialize_block(bucket, data_block?, &self.ipc_fields, &self.options)? + let bucket = payload.bucket; + let stream = SerializeAggregateStream::create( + &self.method, + &self.params, + SerializePayload::::HashTablePayload(payload), + ); + let mut stream_blocks = stream.into_iter().collect::>>()?; + + if stream_blocks.is_empty() { + serialized_blocks.push(FlightSerialized::DataBlock(DataBlock::empty())); + } else { + let mut c = DataBlock::concat(&stream_blocks)?; + if let Some(meta) = stream_blocks[0].take_meta() { + c.replace_meta(meta); } - })); + metrics_inc_aggregate_partial_hashtable_exchange_rows(c.num_rows() as u64); + let c = serialize_block(bucket, c, &self.ipc_fields, &self.options)?; + serialized_blocks.push(FlightSerialized::DataBlock(c)); + } + } + Some(AggregateMeta::AggregatePayload(p)) => { + if index == self.local_pos { + serialized_blocks.push(FlightSerialized::DataBlock(block.add_meta( + Some(Box::new(AggregateMeta::::AggregatePayload( + p, + ))), + )?)); + continue; + } + + let bucket = compute_block_number(p.bucket, p.max_partition_count)?; + let stream = SerializeAggregateStream::create( + &self.method, + &self.params, + SerializePayload::::AggregatePayload(p), + ); + let mut stream_blocks = stream.into_iter().collect::>>()?; + + if stream_blocks.is_empty() { + serialized_blocks.push(FlightSerialized::DataBlock(DataBlock::empty())); + } else { + let mut c = DataBlock::concat(&stream_blocks)?; + if let Some(meta) = stream_blocks[0].take_meta() { + c.replace_meta(meta); + } + metrics_inc_aggregate_partial_hashtable_exchange_rows(c.num_rows() as u64); + + let c = serialize_block(bucket, c, &self.ipc_fields, &self.options)?; + serialized_blocks.push(FlightSerialized::DataBlock(c)); + } } }; } @@ -184,6 +246,138 @@ impl BlockMetaTransform } } +fn agg_spilling_aggregate_payload( + ctx: Arc, + operator: Operator, + location_prefix: &str, + partitioned_payload: PartitionedPayload, +) -> Result>> { + let unique_name = GlobalUniqName::unique(); + let location = format!("{}/{}", location_prefix, unique_name); + + let partition_count = partitioned_payload.partition_count(); + let mut write_size = 0; + let mut write_data = Vec::with_capacity(partition_count); + let mut buckets_column_data = Vec::with_capacity(partition_count); + let mut data_range_start_column_data = Vec::with_capacity(partition_count); + let mut data_range_end_column_data = Vec::with_capacity(partition_count); + let mut columns_layout_column_data = Vec::with_capacity(partition_count); + // Record how many rows are spilled. + let mut rows = 0; + + for (bucket, payload) in partitioned_payload.payloads.into_iter().enumerate() { + if payload.len() == 0 { + continue; + } + + let now = Instant::now(); + let data_block = payload.aggregate_flush_all()?; + rows += data_block.num_rows(); + + let old_write_size = write_size; + let columns = data_block.columns().to_vec(); + let mut columns_data = Vec::with_capacity(columns.len()); + let mut columns_layout = Vec::with_capacity(columns.len()); + + for column in columns.into_iter() { + let column = column.value.as_column().unwrap(); + let column_data = serialize_column(column); + write_size += column_data.len() as u64; + columns_layout.push(column_data.len() as u64); + columns_data.push(column_data); + } + + // perf + { + metrics_inc_aggregate_spill_data_serialize_milliseconds( + now.elapsed().as_millis() as u64 + ); + } + + write_data.push(columns_data); + buckets_column_data.push(bucket as i64); + data_range_end_column_data.push(write_size); + columns_layout_column_data.push(columns_layout); + data_range_start_column_data.push(old_write_size); + } + + Ok(Box::pin(async move { + if !write_data.is_empty() { + let instant = Instant::now(); + + let mut write_bytes = 0; + let mut writer = operator + .writer_with(&location) + .buffer(8 * 1024 * 1024) + .await?; + for write_bucket_data in write_data.into_iter() { + for data in write_bucket_data.into_iter() { + write_bytes += data.len(); + writer.write(data).await?; + } + } + + writer.close().await?; + + // perf + { + metrics_inc_aggregate_spill_write_count(); + metrics_inc_aggregate_spill_write_bytes(write_bytes as u64); + metrics_inc_aggregate_spill_write_milliseconds(instant.elapsed().as_millis() as u64); + + Profile::record_usize_profile(ProfileStatisticsName::SpillWriteCount, 1); + Profile::record_usize_profile(ProfileStatisticsName::SpillWriteBytes, write_bytes); + Profile::record_usize_profile( + ProfileStatisticsName::SpillWriteTime, + instant.elapsed().as_millis() as usize, + ); + } + + { + { + let progress_val = ProgressValues { + rows, + bytes: write_bytes, + }; + ctx.get_aggregate_spill_progress().incr(&progress_val); + } + } + + info!( + "Write aggregate spill {} successfully, elapsed: {:?}", + location, + instant.elapsed() + ); + + let data_block = DataBlock::new_from_columns(vec![ + Int64Type::from_data(buckets_column_data), + UInt64Type::from_data(data_range_start_column_data), + UInt64Type::from_data(data_range_end_column_data), + ArrayType::upcast_column(ArrayType::::column_from_iter( + columns_layout_column_data + .into_iter() + .map(|x| UInt64Type::column_from_iter(x.into_iter(), &[])), + &[], + )), + ]); + + let data_block = data_block.add_meta(Some(AggregateSerdeMeta::create_agg_spilled( + -1, + location.clone(), + 0..0, + vec![], + partition_count, + )))?; + + let ipc_fields = exchange_defines::spilled_ipc_fields(); + let write_options = exchange_defines::spilled_write_options(); + return serialize_block(-1, data_block, ipc_fields, write_options); + } + + Ok(DataBlock::empty()) + })) +} + fn spilling_aggregate_payload( ctx: Arc, operator: Operator, diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_exchange_group_by_serializer.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_exchange_group_by_serializer.rs index afc4fa7a333c..a4545a2bd14a 100644 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_exchange_group_by_serializer.rs +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_exchange_group_by_serializer.rs @@ -39,6 +39,7 @@ use databend_common_expression::BlockMetaInfoPtr; use databend_common_expression::DataBlock; use databend_common_expression::DataSchemaRef; use databend_common_expression::FromData; +use databend_common_expression::PartitionedPayload; use databend_common_hashtable::HashtableLike; use databend_common_metrics::transform::*; use databend_common_pipeline_core::processors::InputPort; @@ -54,8 +55,11 @@ use futures_util::future::BoxFuture; use log::info; use opendal::Operator; +use super::SerializePayload; use crate::api::serialize_block; use crate::api::ExchangeShuffleMeta; +use crate::pipelines::processors::transforms::aggregator::agg_spilling_group_by_payload as local_agg_spilling_group_by_payload; +use crate::pipelines::processors::transforms::aggregator::aggregate_exchange_injector::compute_block_number; use crate::pipelines::processors::transforms::aggregator::exchange_defines; use crate::pipelines::processors::transforms::aggregator::serialize_group_by; use crate::pipelines::processors::transforms::aggregator::spilling_group_by_payload as local_spilling_group_by_payload; @@ -207,6 +211,24 @@ impl BlockMetaTransform }, )); } + Some(AggregateMeta::AggregateSpilling(payload)) => { + serialized_blocks.push(FlightSerialized::Future( + match index == self.local_pos { + true => local_agg_spilling_group_by_payload::( + self.ctx.clone(), + self.operator.clone(), + &self.location_prefix, + payload, + )?, + false => agg_spilling_group_by_payload::( + self.ctx.clone(), + self.operator.clone(), + &self.location_prefix, + payload, + )?, + }, + )); + } Some(AggregateMeta::HashTable(payload)) => { if index == self.local_pos { serialized_blocks.push(FlightSerialized::DataBlock(block.add_meta( @@ -215,14 +237,55 @@ impl BlockMetaTransform continue; } - let mut stream = SerializeGroupByStream::create(&self.method, payload); - let bucket = stream.payload.bucket; - serialized_blocks.push(FlightSerialized::DataBlock(match stream.next() { - None => DataBlock::empty(), - Some(data_block) => { - serialize_block(bucket, data_block?, &self.ipc_fields, &self.options)? + let bucket = payload.bucket; + let stream = SerializeGroupByStream::create( + &self.method, + SerializePayload::::HashTablePayload(payload), + ); + + let mut stream_blocks = stream.into_iter().collect::>>()?; + + if stream_blocks.is_empty() { + serialized_blocks.push(FlightSerialized::DataBlock(DataBlock::empty())); + } else { + let mut c = DataBlock::concat(&stream_blocks)?; + if let Some(meta) = stream_blocks[0].take_meta() { + c.replace_meta(meta); } - })); + + metrics_inc_aggregate_partial_hashtable_exchange_rows(c.num_rows() as u64); + let c = serialize_block(bucket, c, &self.ipc_fields, &self.options)?; + serialized_blocks.push(FlightSerialized::DataBlock(c)); + } + } + Some(AggregateMeta::AggregatePayload(p)) => { + if index == self.local_pos { + serialized_blocks.push(FlightSerialized::DataBlock(block.add_meta( + Some(Box::new(AggregateMeta::::AggregatePayload(p))), + )?)); + continue; + } + + let bucket = compute_block_number(p.bucket, p.max_partition_count)?; + let stream = SerializeGroupByStream::create( + &self.method, + SerializePayload::::AggregatePayload(p), + ); + + let mut stream_blocks = stream.into_iter().collect::>>()?; + + if stream_blocks.is_empty() { + serialized_blocks.push(FlightSerialized::DataBlock(DataBlock::empty())); + } else { + let mut c = DataBlock::concat(&stream_blocks)?; + if let Some(meta) = stream_blocks[0].take_meta() { + c.replace_meta(meta); + } + + metrics_inc_aggregate_partial_hashtable_exchange_rows(c.num_rows() as u64); + let c = serialize_block(bucket, c, &self.ipc_fields, &self.options)?; + serialized_blocks.push(FlightSerialized::DataBlock(c)); + } } }; } @@ -237,6 +300,136 @@ fn get_columns(data_block: DataBlock) -> Vec { data_block.columns().to_vec() } +fn agg_spilling_group_by_payload( + ctx: Arc, + operator: Operator, + location_prefix: &str, + partitioned_payload: PartitionedPayload, +) -> Result>> { + let unique_name = GlobalUniqName::unique(); + let location = format!("{}/{}", location_prefix, unique_name); + + let mut write_size = 0; + let partition_count = partitioned_payload.partition_count(); + let mut write_data = Vec::with_capacity(partition_count); + let mut buckets_column_data = Vec::with_capacity(partition_count); + let mut data_range_start_column_data = Vec::with_capacity(partition_count); + let mut data_range_end_column_data = Vec::with_capacity(partition_count); + let mut columns_layout_column_data = Vec::with_capacity(partition_count); + // Record how many rows are spilled + let mut rows = 0; + + for (bucket, payload) in partitioned_payload.payloads.into_iter().enumerate() { + if payload.len() == 0 { + continue; + } + + let now = Instant::now(); + let data_block = payload.group_by_flush_all()?; + rows += data_block.num_rows(); + + let old_write_size = write_size; + let columns = get_columns(data_block); + let mut columns_data = Vec::with_capacity(columns.len()); + let mut columns_layout = Vec::with_capacity(columns.len()); + + for column in columns.into_iter() { + let column = column.value.as_column().unwrap(); + let column_data = serialize_column(column); + write_size += column_data.len() as u64; + columns_layout.push(column_data.len() as u64); + columns_data.push(column_data); + } + + // perf + { + metrics_inc_aggregate_spill_data_serialize_milliseconds( + now.elapsed().as_millis() as u64 + ); + } + + write_data.push(columns_data); + buckets_column_data.push(bucket as i64); + data_range_end_column_data.push(write_size); + columns_layout_column_data.push(columns_layout); + data_range_start_column_data.push(old_write_size); + } + + Ok(Box::pin(async move { + let instant = Instant::now(); + + if !write_data.is_empty() { + let mut write_bytes = 0; + let mut writer = operator + .writer_with(&location) + .buffer(8 * 1024 * 1024) + .await?; + for write_bucket_data in write_data.into_iter() { + for data in write_bucket_data.into_iter() { + write_bytes += data.len(); + writer.write(data).await?; + } + } + + writer.close().await?; + + // perf + { + metrics_inc_group_by_spill_write_count(); + metrics_inc_group_by_spill_write_bytes(write_bytes as u64); + metrics_inc_group_by_spill_write_milliseconds(instant.elapsed().as_millis() as u64); + + Profile::record_usize_profile(ProfileStatisticsName::SpillWriteCount, 1); + Profile::record_usize_profile(ProfileStatisticsName::SpillWriteBytes, write_bytes); + Profile::record_usize_profile( + ProfileStatisticsName::SpillWriteTime, + instant.elapsed().as_millis() as usize, + ); + } + + { + let progress_val = ProgressValues { + rows, + bytes: write_bytes, + }; + ctx.get_group_by_spill_progress().incr(&progress_val); + } + + info!( + "Write aggregate spill {} successfully, elapsed: {:?}", + location, + instant.elapsed() + ); + + let data_block = DataBlock::new_from_columns(vec![ + Int64Type::from_data(buckets_column_data), + UInt64Type::from_data(data_range_start_column_data), + UInt64Type::from_data(data_range_end_column_data), + ArrayType::upcast_column(ArrayType::::column_from_iter( + columns_layout_column_data + .into_iter() + .map(|x| UInt64Type::column_from_iter(x.into_iter(), &[])), + &[], + )), + ]); + + let data_block = data_block.add_meta(Some(AggregateSerdeMeta::create_agg_spilled( + -1, + location.clone(), + 0..0, + vec![], + partition_count, + )))?; + + let ipc_fields = exchange_defines::spilled_ipc_fields(); + let write_options = exchange_defines::spilled_write_options(); + return serialize_block(-1, data_block, ipc_fields, write_options); + } + + Ok(DataBlock::empty()) + })) +} + fn spilling_group_by_payload( ctx: Arc, operator: Operator, diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_group_by_serializer.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_group_by_serializer.rs index 5ea397ca8b8e..4e8520e63915 100644 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_group_by_serializer.rs +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_group_by_serializer.rs @@ -20,6 +20,7 @@ use std::sync::Arc; use databend_common_exception::Result; use databend_common_expression::BlockMetaInfoDowncast; use databend_common_expression::DataBlock; +use databend_common_expression::PayloadFlushState; use databend_common_hashtable::HashtableEntryRefLike; use databend_common_hashtable::HashtableLike; use databend_common_pipeline_core::processors::Event; @@ -30,6 +31,7 @@ use databend_common_pipeline_core::processors::ProcessorPtr; use crate::pipelines::processors::transforms::aggregator::estimated_key_size; use crate::pipelines::processors::transforms::aggregator::AggregateMeta; +use crate::pipelines::processors::transforms::aggregator::AggregatePayload; use crate::pipelines::processors::transforms::aggregator::AggregateSerdeMeta; use crate::pipelines::processors::transforms::aggregator::HashTablePayload; use crate::pipelines::processors::transforms::group_by::HashMethodBounds; @@ -126,9 +128,19 @@ impl TransformGroupBySerializer { AggregateMeta::Serialized(_) => unreachable!(), AggregateMeta::BucketSpilled(_) => unreachable!(), AggregateMeta::Partitioned { .. } => unreachable!(), + AggregateMeta::AggregateSpilling(_) => unreachable!(), + AggregateMeta::AggregatePayload(p) => { + self.input_data = Some(SerializeGroupByStream::create( + &self.method, + SerializePayload::::AggregatePayload(p), + )); + return Ok(Event::Sync); + } AggregateMeta::HashTable(payload) => { - self.input_data = - Some(SerializeGroupByStream::create(&self.method, payload)); + self.input_data = Some(SerializeGroupByStream::create( + &self.method, + SerializePayload::::HashTablePayload(payload), + )); return Ok(Event::Sync); } } @@ -156,10 +168,17 @@ pub fn serialize_group_by( ])) } +pub enum SerializePayload { + HashTablePayload(HashTablePayload), + AggregatePayload(AggregatePayload), +} + pub struct SerializeGroupByStream { method: Method, - pub payload: Pin>>, - iter: as HashtableLike>::Iterator<'static>, + pub payload: Pin>>, + // old hashtable' iter + iter: Option< as HashtableLike>::Iterator<'static>>, + flush_state: Option, end_iter: bool, } @@ -168,14 +187,27 @@ unsafe impl Send for SerializeGroupByStream {} unsafe impl Sync for SerializeGroupByStream {} impl SerializeGroupByStream { - pub fn create(method: &Method, payload: HashTablePayload) -> Self { + pub fn create(method: &Method, payload: SerializePayload) -> Self { unsafe { let payload = Box::pin(payload); - let iter = NonNull::from(&payload.cell.hashtable).as_ref().iter(); + + let iter = if let SerializePayload::HashTablePayload(p) = payload.as_ref().get_ref() { + Some(NonNull::from(&p.cell.hashtable).as_ref().iter()) + } else { + None + }; + + let flush_state = + if let SerializePayload::AggregatePayload(_) = payload.as_ref().get_ref() { + Some(PayloadFlushState::default()) + } else { + None + }; SerializeGroupByStream:: { iter, payload, + flush_state, method: method.clone(), end_iter: false, } @@ -191,34 +223,50 @@ impl Iterator for SerializeGroupByStream { return None; } - let max_block_rows = std::cmp::min(8192, self.payload.cell.hashtable.len()); - let max_block_bytes = std::cmp::min( - 8 * 1024 * 1024 + 1024, - self.payload - .cell - .hashtable - .unsize_key_size() - .unwrap_or(usize::MAX), - ); - - let mut group_key_builder = self - .method - .keys_column_builder(max_block_rows, max_block_bytes); - - #[allow(clippy::while_let_on_iterator)] - while let Some(group_entity) = self.iter.next() { - group_key_builder.append_value(group_entity.key()); - - if group_key_builder.bytes_size() >= 8 * 1024 * 1024 { - let bucket = self.payload.bucket; + match self.payload.as_ref().get_ref() { + SerializePayload::HashTablePayload(p) => { + let max_block_rows = std::cmp::min(8192, p.cell.hashtable.len()); + let max_block_bytes = std::cmp::min( + 8 * 1024 * 1024 + 1024, + p.cell.hashtable.unsize_key_size().unwrap_or(usize::MAX), + ); + + let mut group_key_builder = self + .method + .keys_column_builder(max_block_rows, max_block_bytes); + + #[allow(clippy::while_let_on_iterator)] + while let Some(group_entity) = self.iter.as_mut()?.next() { + group_key_builder.append_value(group_entity.key()); + + if group_key_builder.bytes_size() >= 8 * 1024 * 1024 { + let bucket = p.bucket; + let data_block = + DataBlock::new_from_columns(vec![group_key_builder.finish()]); + return Some(data_block.add_meta(Some(AggregateSerdeMeta::create(bucket)))); + } + } + + self.end_iter = true; + let bucket = p.bucket; let data_block = DataBlock::new_from_columns(vec![group_key_builder.finish()]); - return Some(data_block.add_meta(Some(AggregateSerdeMeta::create(bucket)))); + Some(data_block.add_meta(Some(AggregateSerdeMeta::create(bucket)))) } - } + SerializePayload::AggregatePayload(p) => { + let state = self.flush_state.as_mut().unwrap(); + let block = p.payload.aggregate_flush(state).unwrap(); - self.end_iter = true; - let bucket = self.payload.bucket; - let data_block = DataBlock::new_from_columns(vec![group_key_builder.finish()]); - Some(data_block.add_meta(Some(AggregateSerdeMeta::create(bucket)))) + if block.is_none() { + self.end_iter = true; + } + + block.map(|block| { + block.add_meta(Some(AggregateSerdeMeta::create_agg_payload( + p.bucket, + p.max_partition_count, + ))) + }) + } + } } } diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_group_by_spill_writer.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_group_by_spill_writer.rs index 1ddb32550494..7a386e1603bd 100644 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_group_by_spill_writer.rs +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_group_by_spill_writer.rs @@ -24,6 +24,7 @@ use databend_common_exception::Result; use databend_common_expression::arrow::serialize_column; use databend_common_expression::BlockMetaInfoDowncast; use databend_common_expression::DataBlock; +use databend_common_expression::PartitionedPayload; use databend_common_hashtable::HashtableLike; use databend_common_metrics::transform::*; use databend_common_pipeline_core::processors::Event; @@ -131,6 +132,13 @@ impl Processor for TransformGroupBySpillWriter self.spilling_meta = AggregateMeta::::downcast_from(block_meta); return Ok(Event::Sync); } + + if matches!(block_meta, AggregateMeta::AggregateSpilling(_)) { + self.input.set_not_need_data(); + let block_meta = data_block.take_meta().unwrap(); + self.spilling_meta = AggregateMeta::::downcast_from(block_meta); + return Ok(Event::Sync); + } } self.output.push_data(Ok(data_block)); @@ -148,21 +156,35 @@ impl Processor for TransformGroupBySpillWriter fn process(&mut self) -> Result<()> { if let Some(spilling_meta) = self.spilling_meta.take() { - if let AggregateMeta::Spilling(payload) = spilling_meta { - self.spilling_future = Some(spilling_group_by_payload( - self.ctx.clone(), - self.operator.clone(), - &self.method, - &self.location_prefix, - payload, - )?); - - return Ok(()); - } + match spilling_meta { + AggregateMeta::Spilling(payload) => { + self.spilling_future = Some(spilling_group_by_payload( + self.ctx.clone(), + self.operator.clone(), + &self.method, + &self.location_prefix, + payload, + )?); + + return Ok(()); + } + AggregateMeta::AggregateSpilling(payload) => { + self.spilling_future = Some(agg_spilling_group_by_payload::( + self.ctx.clone(), + self.operator.clone(), + &self.location_prefix, + payload, + )?); + + return Ok(()); + } - return Err(ErrorCode::Internal( - "TransformGroupBySpillWriter only recv AggregateMeta", - )); + _ => { + return Err(ErrorCode::Internal( + "TransformGroupBySpillWriter only recv AggregateMeta", + )); + } + } } Ok(()) @@ -178,6 +200,112 @@ impl Processor for TransformGroupBySpillWriter } } +pub fn agg_spilling_group_by_payload( + ctx: Arc, + operator: Operator, + location_prefix: &str, + partitioned_payload: PartitionedPayload, +) -> Result>> { + let unique_name = GlobalUniqName::unique(); + let location = format!("{}/{}", location_prefix, unique_name); + + let mut write_size = 0; + let partition_count = partitioned_payload.partition_count(); + let mut write_data = Vec::with_capacity(partition_count); + let mut spilled_buckets_payloads = Vec::with_capacity(partition_count); + let mut rows = 0; + + for (bucket, payload) in partitioned_payload.payloads.into_iter().enumerate() { + if payload.len() == 0 { + continue; + } + + let now = Instant::now(); + let data_block = payload.group_by_flush_all()?; + rows += data_block.num_rows(); + + let begin = write_size; + let columns = data_block.columns().to_vec(); + let mut columns_data = Vec::with_capacity(columns.len()); + let mut columns_layout = Vec::with_capacity(columns.len()); + for column in columns.into_iter() { + let column = column.value.as_column().unwrap(); + let column_data = serialize_column(column); + write_size += column_data.len() as u64; + columns_layout.push(column_data.len() as u64); + columns_data.push(column_data); + } + + // perf + { + metrics_inc_aggregate_spill_data_serialize_milliseconds( + now.elapsed().as_millis() as u64 + ); + } + + write_data.push(columns_data); + spilled_buckets_payloads.push(BucketSpilledPayload { + bucket: bucket as isize, + location: location.clone(), + data_range: begin..write_size, + columns_layout, + max_partition_count: partition_count, + }); + } + + Ok(Box::pin(async move { + let instant = Instant::now(); + + let mut write_bytes = 0; + if !write_data.is_empty() { + let mut writer = operator + .writer_with(&location) + .buffer(8 * 1024 * 1024) + .await?; + for write_bucket_data in write_data.into_iter() { + for data in write_bucket_data.into_iter() { + write_bytes += data.len(); + writer.write(data).await?; + } + } + + writer.close().await?; + } + + // perf + { + metrics_inc_group_by_spill_write_count(); + metrics_inc_group_by_spill_write_bytes(write_bytes as u64); + metrics_inc_group_by_spill_write_milliseconds(instant.elapsed().as_millis() as u64); + + Profile::record_usize_profile(ProfileStatisticsName::SpillWriteCount, 1); + Profile::record_usize_profile(ProfileStatisticsName::SpillWriteBytes, write_bytes); + Profile::record_usize_profile( + ProfileStatisticsName::SpillWriteTime, + instant.elapsed().as_millis() as usize, + ); + } + + { + let progress_val = ProgressValues { + rows, + bytes: write_bytes, + }; + ctx.get_group_by_spill_progress().incr(&progress_val); + } + + info!( + "Write aggregate spill {} successfully, elapsed: {:?}", + location, + instant.elapsed() + ); + + Ok(DataBlock::empty_with_meta( + AggregateMeta::::create_spilled(spilled_buckets_payloads), + )) + })) +} + pub fn spilling_group_by_payload( ctx: Arc, operator: Operator, @@ -226,6 +354,7 @@ pub fn spilling_group_by_payload( location: location.clone(), data_range: begin..write_size, columns_layout, + max_partition_count: 0, }); } diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_spill_reader.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_spill_reader.rs index f0cfc0a1b1ab..67eea4268223 100644 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_spill_reader.rs +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_spill_reader.rs @@ -136,6 +136,8 @@ impl Processor match meta { AggregateMeta::Spilled(_) => unreachable!(), AggregateMeta::Spilling(_) => unreachable!(), + AggregateMeta::AggregatePayload(_) => unreachable!(), + AggregateMeta::AggregateSpilling(_) => unreachable!(), AggregateMeta::HashTable(_) => unreachable!(), AggregateMeta::Serialized(_) => unreachable!(), AggregateMeta::BucketSpilled(payload) => { @@ -177,6 +179,8 @@ impl Processor AggregateMeta::Spilled(_) => unreachable!(), AggregateMeta::Spilling(_) => unreachable!(), AggregateMeta::HashTable(_) => unreachable!(), + AggregateMeta::AggregatePayload(_) => unreachable!(), + AggregateMeta::AggregateSpilling(_) => unreachable!(), AggregateMeta::Serialized(_) => unreachable!(), AggregateMeta::BucketSpilled(payload) => { let instant = Instant::now(); @@ -299,6 +303,7 @@ impl TransformSpillReader::Serialized(SerializedPayload { bucket: payload.bucket, data_block: DataBlock::new_from_columns(columns), + max_partition_count: payload.max_partition_count, }) } } diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/transform_aggregate_final.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/transform_aggregate_final.rs index d5e74a84303c..d7a0e500ae5c 100644 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/transform_aggregate_final.rs +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/transform_aggregate_final.rs @@ -18,8 +18,11 @@ use std::sync::Arc; use bumpalo::Bump; use databend_common_exception::ErrorCode; use databend_common_exception::Result; +use databend_common_expression::AggregateHashTable; use databend_common_expression::ColumnBuilder; use databend_common_expression::DataBlock; +use databend_common_expression::HashTableConfig; +use databend_common_expression::PayloadFlushState; use databend_common_functions::aggregates::StateAddr; use databend_common_hashtable::HashtableEntryMutRefLike; use databend_common_hashtable::HashtableEntryRefLike; @@ -42,6 +45,8 @@ use crate::pipelines::processors::transforms::group_by::KeysColumnIter; pub struct TransformFinalAggregate { method: Method, params: Arc, + flush_state: PayloadFlushState, + reach_limit: bool, } impl TransformFinalAggregate { @@ -54,9 +59,101 @@ impl TransformFinalAggregate { Ok(Box::new(BlockMetaTransformer::create( input, output, - TransformFinalAggregate:: { method, params }, + TransformFinalAggregate:: { + method, + params, + flush_state: PayloadFlushState::default(), + reach_limit: false, + }, ))) } + + fn transform_agg_hashtable(&mut self, meta: AggregateMeta) -> Result { + let mut agg_hashtable: Option = None; + if let AggregateMeta::Partitioned { bucket, data } = meta { + for bucket_data in data { + match bucket_data { + AggregateMeta::Serialized(payload) => match agg_hashtable.as_mut() { + Some(ht) => { + debug_assert!(bucket == payload.bucket); + let payload = payload.convert_to_partitioned_payload( + self.params.group_data_types.clone(), + self.params.aggregate_functions.clone(), + 0, + Arc::new(Bump::new()), + )?; + ht.combine_payloads(&payload, &mut self.flush_state)?; + } + None => { + debug_assert!(bucket == payload.bucket); + agg_hashtable = Some(payload.convert_to_aggregate_table( + self.params.group_data_types.clone(), + self.params.aggregate_functions.clone(), + 0, + Arc::new(Bump::new()), + true, + )?); + } + }, + AggregateMeta::AggregatePayload(payload) => match agg_hashtable.as_mut() { + Some(ht) => { + debug_assert!(bucket == payload.bucket); + ht.combine_payload(&payload.payload, &mut self.flush_state)?; + } + None => { + debug_assert!(bucket == payload.bucket); + let capacity = + AggregateHashTable::get_capacity_for_count(payload.payload.len()); + let mut hashtable = AggregateHashTable::new_with_capacity( + self.params.group_data_types.clone(), + self.params.aggregate_functions.clone(), + HashTableConfig::default().with_initial_radix_bits(0), + capacity, + Arc::new(Bump::new()), + ); + hashtable.combine_payload(&payload.payload, &mut self.flush_state)?; + agg_hashtable = Some(hashtable); + } + }, + _ => unreachable!(), + } + } + } + + if let Some(mut ht) = agg_hashtable { + let mut blocks = vec![]; + self.flush_state.clear(); + + let mut rows = 0; + loop { + if ht.merge_result(&mut self.flush_state)? { + let mut cols = self.flush_state.take_aggregate_results(); + cols.extend_from_slice(&self.flush_state.take_group_columns()); + rows += cols[0].len(); + blocks.push(DataBlock::new_from_columns(cols)); + + if rows >= self.params.limit.unwrap_or(usize::MAX) { + log::info!( + "reach limit optimization in flush agg hashtable, current {}, total {}", + rows, + ht.len(), + ); + self.reach_limit = true; + break; + } + } else { + break; + } + } + + if blocks.is_empty() { + return Ok(self.params.empty_result_block()); + } + return DataBlock::concat(&blocks); + } + + Ok(self.params.empty_result_block()) + } } impl BlockMetaTransform> for TransformFinalAggregate @@ -65,8 +162,15 @@ where Method: HashMethodBounds const NAME: &'static str = "TransformFinalAggregate"; fn transform(&mut self, meta: AggregateMeta) -> Result { + if self.reach_limit { + return Ok(self.params.empty_result_block()); + } + + if self.params.enable_experimental_aggregate_hashtable { + return self.transform_agg_hashtable(meta); + } + if let AggregateMeta::Partitioned { bucket, data } = meta { - let mut reach_limit = false; let arena = Arc::new(Bump::new()); let hashtable = self.method.create_hash_table::(arena)?; let _dropper = AggregateHashTableDropper::create(self.params.clone()); @@ -95,7 +199,7 @@ where Method: HashMethodBounds let mut current_len = hash_cell.hashtable.len(); unsafe { for key in keys_iter { - if reach_limit { + if self.reach_limit { let entry = hash_cell.hashtable.entry(key); if let Some(entry) = entry { let place = Into::::into(*entry.get()); @@ -115,7 +219,7 @@ where Method: HashMethodBounds if let Some(limit) = self.params.limit { current_len += 1; if current_len >= limit { - reach_limit = true; + self.reach_limit = true; } } } @@ -176,6 +280,8 @@ where Method: HashMethodBounds } } }, + AggregateMeta::AggregatePayload(_) => unreachable!(), + AggregateMeta::AggregateSpilling(_) => unreachable!(), } } diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/transform_aggregate_partial.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/transform_aggregate_partial.rs index f72e6e244e23..ca02470caab3 100644 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/transform_aggregate_partial.rs +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/transform_aggregate_partial.rs @@ -23,9 +23,13 @@ use databend_common_catalog::plan::AggIndexMeta; use databend_common_catalog::table_context::TableContext; use databend_common_exception::ErrorCode; use databend_common_exception::Result; +use databend_common_expression::AggregateHashTable; use databend_common_expression::BlockMetaInfoDowncast; use databend_common_expression::Column; use databend_common_expression::DataBlock; +use databend_common_expression::HashTableConfig; +use databend_common_expression::PayloadFlushState; +use databend_common_expression::ProbeState; use databend_common_functions::aggregates::StateAddr; use databend_common_functions::aggregates::StateAddrs; use databend_common_hashtable::HashtableEntryMutRefLike; @@ -47,11 +51,11 @@ use crate::pipelines::processors::transforms::group_by::HashMethodBounds; use crate::pipelines::processors::transforms::group_by::PartitionedHashMethod; use crate::pipelines::processors::transforms::group_by::PolymorphicKeysHelper; use crate::sessions::QueryContext; - #[allow(clippy::enum_variant_names)] enum HashTable { MovedOut, HashTable(HashTableCell), + AggregateHashTable(AggregateHashTable), PartitionedHashTable(HashTableCell, usize>), } @@ -106,7 +110,7 @@ pub struct TransformPartialAggregate { method: Method, settings: AggregateSettings, hash_table: HashTable, - + probe_state: ProbeState, params: Arc, } @@ -117,17 +121,39 @@ impl TransformPartialAggregate { input: Arc, output: Arc, params: Arc, + config: HashTableConfig, ) -> Result> { - let arena = Arc::new(Bump::new()); - let hashtable = method.create_hash_table(arena)?; - let _dropper = AggregateHashTableDropper::create(params.clone()); - let hashtable = HashTableCell::create(hashtable, _dropper); - - let hash_table = match !Method::SUPPORT_PARTITIONED || !params.has_distinct_combinator() { - true => HashTable::HashTable(hashtable), - false => HashTable::PartitionedHashTable(PartitionedHashMethod::convert_hashtable( - &method, hashtable, - )?), + let hash_table = if !params.enable_experimental_aggregate_hashtable { + let arena = Arc::new(Bump::new()); + let hashtable = method.create_hash_table(arena)?; + let _dropper = AggregateHashTableDropper::create(params.clone()); + let hashtable = HashTableCell::create(hashtable, _dropper); + + match !Method::SUPPORT_PARTITIONED || !params.has_distinct_combinator() { + true => HashTable::HashTable(hashtable), + false => HashTable::PartitionedHashTable(PartitionedHashMethod::convert_hashtable( + &method, hashtable, + )?), + } + } else { + let arena = Arc::new(Bump::new()); + match !params.has_distinct_combinator() { + true => HashTable::AggregateHashTable(AggregateHashTable::new( + params.group_data_types.clone(), + params.aggregate_functions.clone(), + config, + arena, + )), + false => { + let max_radix_bits = config.max_radix_bits; + HashTable::AggregateHashTable(AggregateHashTable::new( + params.group_data_types.clone(), + params.aggregate_functions.clone(), + config.with_initial_radix_bits(max_radix_bits), + arena, + )) + } + } }; Ok(AccumulatingTransformer::create( @@ -137,6 +163,7 @@ impl TransformPartialAggregate { method, params, hash_table, + probe_state: ProbeState::default(), settings: AggregateSettings::try_from(ctx)?, }, )) @@ -234,11 +261,11 @@ impl TransformPartialAggregate { unsafe { let rows_num = block.num_rows(); - let state = self.method.build_keys_state(&group_columns, rows_num)?; match &mut self.hash_table { HashTable::MovedOut => unreachable!(), HashTable::HashTable(hashtable) => { + let state = self.method.build_keys_state(&group_columns, rows_num)?; let mut places = Vec::with_capacity(rows_num); for key in self.method.build_keys_iter(&state)? { @@ -259,6 +286,7 @@ impl TransformPartialAggregate { } } HashTable::PartitionedHashTable(hashtable) => { + let state = self.method.build_keys_state(&group_columns, rows_num)?; let mut places = Vec::with_capacity(rows_num); for key in self.method.build_keys_iter(&state)? { @@ -278,6 +306,41 @@ impl TransformPartialAggregate { Self::execute(&self.params, &block, &places) } } + HashTable::AggregateHashTable(hashtable) => { + let group_columns: Vec = + group_columns.into_iter().map(|c| c.0).collect(); + + let (params_columns, agg_states) = if is_agg_index_block { + ( + vec![], + (0..self.params.aggregate_functions.len()) + .map(|index| { + block + .get_by_offset( + block.num_columns() + - self.params.aggregate_functions.len() + + index, + ) + .value + .as_column() + .cloned() + .unwrap() + }) + .collect(), + ) + } else { + (Self::aggregate_arguments(&block, &self.params)?, vec![]) + }; + + let _ = hashtable.add_groups( + &mut self.probe_state, + &group_columns, + ¶ms_columns, + &agg_states, + rows_num, + )?; + Ok(()) + } } } } @@ -289,13 +352,15 @@ impl AccumulatingTransform for TransformPartialAggrega fn transform(&mut self, block: DataBlock) -> Result> { self.execute_one_block(block)?; + let is_new_agg = self.params.enable_experimental_aggregate_hashtable; #[allow(clippy::collapsible_if)] if Method::SUPPORT_PARTITIONED { - if matches!(&self.hash_table, HashTable::HashTable(cell) - if cell.len() >= self.settings.convert_threshold || - cell.allocated_bytes() >= self.settings.spilling_bytes_threshold_per_proc || - GLOBAL_MEM_STAT.get_memory_usage() as usize >= self.settings.max_memory_usage - ) { + if !is_new_agg + && (matches!(&self.hash_table, HashTable::HashTable(cell) + if cell.len() >= self.settings.convert_threshold || + cell.allocated_bytes() >= self.settings.spilling_bytes_threshold_per_proc || + GLOBAL_MEM_STAT.get_memory_usage() as usize >= self.settings.max_memory_usage)) + { if let HashTable::HashTable(cell) = std::mem::take(&mut self.hash_table) { self.hash_table = HashTable::PartitionedHashTable( PartitionedHashMethod::convert_hashtable(&self.method, cell)?, @@ -303,8 +368,10 @@ impl AccumulatingTransform for TransformPartialAggrega } } - if matches!(&self.hash_table, HashTable::PartitionedHashTable(cell) if cell.allocated_bytes() > self.settings.spilling_bytes_threshold_per_proc) - || GLOBAL_MEM_STAT.get_memory_usage() as usize >= self.settings.max_memory_usage + if !is_new_agg + && (matches!(&self.hash_table, HashTable::PartitionedHashTable(cell) if cell.allocated_bytes() > self.settings.spilling_bytes_threshold_per_proc) + || GLOBAL_MEM_STAT.get_memory_usage() as usize + >= self.settings.max_memory_usage) { if let HashTable::PartitionedHashTable(v) = std::mem::take(&mut self.hash_table) { // perf @@ -335,6 +402,43 @@ impl AccumulatingTransform for TransformPartialAggrega } } + if is_new_agg + && (matches!(&self.hash_table, HashTable::AggregateHashTable(cell) if cell.allocated_bytes() > self.settings.spilling_bytes_threshold_per_proc + || GLOBAL_MEM_STAT.get_memory_usage() as usize >= self.settings.max_memory_usage)) + { + if let HashTable::AggregateHashTable(v) = std::mem::take(&mut self.hash_table) { + let group_types = v.payload.group_types.clone(); + let aggrs = v.payload.aggrs.clone(); + v.config.update_current_max_radix_bits(); + let config = v + .config + .clone() + .with_initial_radix_bits(v.config.max_radix_bits); + + let mut state = PayloadFlushState::default(); + + // repartition to max for normalization + let partitioned_payload = v + .payload + .repartition(1 << config.max_radix_bits, &mut state); + + let blocks = vec![DataBlock::empty_with_meta( + AggregateMeta::::create_agg_spilling(partitioned_payload), + )]; + + let arena = Arc::new(Bump::new()); + self.hash_table = HashTable::AggregateHashTable(AggregateHashTable::new( + group_types, + aggrs, + config, + arena, + )); + return Ok(blocks); + } + + unreachable!() + } + Ok(vec![]) } @@ -343,9 +447,14 @@ impl AccumulatingTransform for TransformPartialAggrega HashTable::MovedOut => unreachable!(), HashTable::HashTable(v) => match v.hashtable.len() == 0 { true => vec![], - false => vec![DataBlock::empty_with_meta( - AggregateMeta::::create_hashtable(-1, v), - )], + false => { + metrics_inc_aggregate_partial_hashtable_allocated_bytes( + v.allocated_bytes() as u64 + ); + vec![DataBlock::empty_with_meta( + AggregateMeta::::create_hashtable(-1, v), + )] + } }, HashTable::PartitionedHashTable(v) => { info!( @@ -354,6 +463,8 @@ impl AccumulatingTransform for TransformPartialAggrega convert_byte_size(v.allocated_bytes() as f64) ); + metrics_inc_aggregate_partial_hashtable_allocated_bytes(v.allocated_bytes() as u64); + let cells = PartitionedHashTableDropper::split_cell(v); let mut blocks = Vec::with_capacity(cells.len()); for (bucket, cell) in cells.into_iter().enumerate() { @@ -364,6 +475,27 @@ impl AccumulatingTransform for TransformPartialAggrega } } + blocks + } + HashTable::AggregateHashTable(hashtable) => { + metrics_inc_aggregate_partial_hashtable_allocated_bytes( + hashtable.allocated_bytes() as u64, + ); + + let partition_count = hashtable.payload.partition_count(); + let mut blocks = Vec::with_capacity(partition_count); + for (bucket, payload) in hashtable.payload.payloads.into_iter().enumerate() { + if payload.len() != 0 { + blocks.push(DataBlock::empty_with_meta( + AggregateMeta::::create_agg_payload( + bucket as isize, + payload, + partition_count, + ), + )); + } + } + blocks } }) diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/transform_group_by_final.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/transform_group_by_final.rs index f1f4b0985422..f8e598d354a2 100644 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/transform_group_by_final.rs +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/transform_group_by_final.rs @@ -17,7 +17,10 @@ use std::sync::Arc; use bumpalo::Bump; use databend_common_exception::ErrorCode; use databend_common_exception::Result; +use databend_common_expression::AggregateHashTable; use databend_common_expression::DataBlock; +use databend_common_expression::HashTableConfig; +use databend_common_expression::PayloadFlushState; use databend_common_hashtable::HashtableEntryRefLike; use databend_common_hashtable::HashtableLike; use databend_common_pipeline_core::processors::InputPort; @@ -36,6 +39,8 @@ use crate::pipelines::processors::transforms::group_by::KeysColumnIter; pub struct TransformFinalGroupBy { method: Method, params: Arc, + flush_state: PayloadFlushState, + reach_limit: bool, } impl TransformFinalGroupBy { @@ -48,9 +53,100 @@ impl TransformFinalGroupBy { Ok(Box::new(BlockMetaTransformer::create( input, output, - TransformFinalGroupBy:: { method, params }, + TransformFinalGroupBy:: { + method, + params, + flush_state: PayloadFlushState::default(), + reach_limit: false, + }, ))) } + + fn transform_agg_hashtable(&mut self, meta: AggregateMeta) -> Result { + let mut agg_hashtable: Option = None; + if let AggregateMeta::Partitioned { bucket, data } = meta { + for bucket_data in data { + match bucket_data { + AggregateMeta::Serialized(payload) => match agg_hashtable.as_mut() { + Some(ht) => { + debug_assert!(bucket == payload.bucket); + let payload = payload.convert_to_partitioned_payload( + self.params.group_data_types.clone(), + self.params.aggregate_functions.clone(), + 0, + Arc::new(Bump::new()), + )?; + ht.combine_payloads(&payload, &mut self.flush_state)?; + } + None => { + debug_assert!(bucket == payload.bucket); + agg_hashtable = Some(payload.convert_to_aggregate_table( + self.params.group_data_types.clone(), + self.params.aggregate_functions.clone(), + 0, + Arc::new(Bump::new()), + true, + )?); + } + }, + AggregateMeta::AggregatePayload(payload) => match agg_hashtable.as_mut() { + Some(ht) => { + debug_assert!(bucket == payload.bucket); + ht.combine_payload(&payload.payload, &mut self.flush_state)?; + } + None => { + debug_assert!(bucket == payload.bucket); + let capacity = + AggregateHashTable::get_capacity_for_count(payload.payload.len()); + let mut hashtable = AggregateHashTable::new_with_capacity( + self.params.group_data_types.clone(), + self.params.aggregate_functions.clone(), + HashTableConfig::default().with_initial_radix_bits(0), + capacity, + Arc::new(Bump::new()), + ); + hashtable.combine_payload(&payload.payload, &mut self.flush_state)?; + agg_hashtable = Some(hashtable); + } + }, + _ => unreachable!(), + } + } + } + + if let Some(mut ht) = agg_hashtable { + let mut blocks = vec![]; + self.flush_state.clear(); + + let mut rows = 0; + loop { + if ht.merge_result(&mut self.flush_state)? { + let cols = self.flush_state.take_group_columns(); + rows += cols[0].len(); + blocks.push(DataBlock::new_from_columns(cols)); + + if rows >= self.params.limit.unwrap_or(usize::MAX) { + log::info!( + "reach limit optimization in flush agg hashtable, current {}, total {}", + rows, + ht.len(), + ); + self.reach_limit = true; + break; + } + } else { + break; + } + } + + if blocks.is_empty() { + return Ok(self.params.empty_result_block()); + } + + return DataBlock::concat(&blocks); + } + Ok(self.params.empty_result_block()) + } } impl BlockMetaTransform> for TransformFinalGroupBy @@ -59,9 +155,18 @@ where Method: HashMethodBounds const NAME: &'static str = "TransformFinalGroupBy"; fn transform(&mut self, meta: AggregateMeta) -> Result { + if self.reach_limit { + return Ok(self.params.empty_result_block()); + } + + if self.params.enable_experimental_aggregate_hashtable { + return self.transform_agg_hashtable(meta); + } + if let AggregateMeta::Partitioned { bucket, data } = meta { let arena = Arc::new(Bump::new()); let mut hashtable = self.method.create_hash_table::<()>(arena)?; + 'merge_hashtable: for bucket_data in data { match bucket_data { AggregateMeta::Spilled(_) => unreachable!(), @@ -80,6 +185,7 @@ where Method: HashMethodBounds if let Some(limit) = self.params.limit { if hashtable.len() >= limit { + self.reach_limit = true; break 'merge_hashtable; } } @@ -98,6 +204,8 @@ where Method: HashMethodBounds } } }, + AggregateMeta::AggregatePayload(_) => unreachable!(), + AggregateMeta::AggregateSpilling(_) => unreachable!(), } } diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/transform_group_by_partial.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/transform_group_by_partial.rs index 1080996ff58b..e302e4906770 100644 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/transform_group_by_partial.rs +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/transform_group_by_partial.rs @@ -22,14 +22,19 @@ use databend_common_base::runtime::GLOBAL_MEM_STAT; use databend_common_catalog::table_context::TableContext; use databend_common_exception::ErrorCode; use databend_common_exception::Result; +use databend_common_expression::AggregateHashTable; +use databend_common_expression::Column; use databend_common_expression::DataBlock; +use databend_common_expression::HashTableConfig; +use databend_common_expression::PayloadFlushState; +use databend_common_expression::ProbeState; use databend_common_hashtable::HashtableLike; +use databend_common_metrics::transform::*; use databend_common_pipeline_core::processors::InputPort; use databend_common_pipeline_core::processors::OutputPort; use databend_common_pipeline_core::processors::Processor; use databend_common_pipeline_transforms::processors::AccumulatingTransform; use databend_common_pipeline_transforms::processors::AccumulatingTransformer; -use databend_common_sql::IndexType; use log::info; use crate::pipelines::processors::transforms::aggregator::aggregate_cell::GroupByHashTableDropper; @@ -46,6 +51,7 @@ use crate::sessions::QueryContext; enum HashTable { MovedOut, HashTable(HashTableCell), + AggregateHashTable(AggregateHashTable), PartitionedHashTable(HashTableCell, ()>), } @@ -99,8 +105,9 @@ impl TryFrom> for GroupBySettings { pub struct TransformPartialGroupBy { method: Method, hash_table: HashTable, - group_columns: Vec, + probe_state: ProbeState, settings: GroupBySettings, + params: Arc, } impl TransformPartialGroupBy { @@ -110,11 +117,22 @@ impl TransformPartialGroupBy { input: Arc, output: Arc, params: Arc, + config: HashTableConfig, ) -> Result> { - let arena = Arc::new(Bump::new()); - let hashtable = method.create_hash_table(arena)?; - let _dropper = GroupByHashTableDropper::::create(); - let hash_table = HashTable::HashTable(HashTableCell::create(hashtable, _dropper)); + let hash_table = if !params.enable_experimental_aggregate_hashtable { + let arena = Arc::new(Bump::new()); + let hashtable = method.create_hash_table(arena)?; + let _dropper = GroupByHashTableDropper::::create(); + HashTable::HashTable(HashTableCell::create(hashtable, _dropper)) + } else { + let arena = Arc::new(Bump::new()); + HashTable::AggregateHashTable(AggregateHashTable::new( + params.group_data_types.clone(), + params.aggregate_functions.clone(), + config, + arena, + )) + }; Ok(AccumulatingTransformer::create( input, @@ -122,7 +140,8 @@ impl TransformPartialGroupBy { TransformPartialGroupBy:: { method, hash_table, - group_columns: params.group_columns.clone(), + probe_state: ProbeState::default(), + params, settings: GroupBySettings::try_from(ctx)?, }, )) @@ -135,6 +154,7 @@ impl AccumulatingTransform for TransformPartialGroupBy fn transform(&mut self, block: DataBlock) -> Result> { let block = block.convert_to_full(); let group_columns = self + .params .group_columns .iter() .map(|&index| block.get_by_offset(index)) @@ -147,29 +167,44 @@ impl AccumulatingTransform for TransformPartialGroupBy unsafe { let rows_num = block.num_rows(); - let state = self.method.build_keys_state(&group_columns, rows_num)?; match &mut self.hash_table { HashTable::MovedOut => unreachable!(), HashTable::HashTable(cell) => { + let state = self.method.build_keys_state(&group_columns, rows_num)?; for key in self.method.build_keys_iter(&state)? { let _ = cell.hashtable.insert_and_entry(key); } } HashTable::PartitionedHashTable(cell) => { + let state = self.method.build_keys_state(&group_columns, rows_num)?; for key in self.method.build_keys_iter(&state)? { let _ = cell.hashtable.insert_and_entry(key); } } + HashTable::AggregateHashTable(hashtable) => { + let group_columns: Vec = + group_columns.into_iter().map(|c| c.0).collect(); + let _ = hashtable.add_groups( + &mut self.probe_state, + &group_columns, + &[vec![]], + &[], + rows_num, + )?; + } }; + let is_new_agg = self.params.enable_experimental_aggregate_hashtable; + #[allow(clippy::collapsible_if)] if Method::SUPPORT_PARTITIONED { - if matches!(&self.hash_table, HashTable::HashTable(cell) - if cell.len() >= self.settings.convert_threshold || - cell.allocated_bytes() >= self.settings.spilling_bytes_threshold_per_proc || - GLOBAL_MEM_STAT.get_memory_usage() as usize >= self.settings.max_memory_usage - ) { + if !is_new_agg + && (matches!(&self.hash_table, HashTable::HashTable(cell) + if cell.len() >= self.settings.convert_threshold || + cell.allocated_bytes() >= self.settings.spilling_bytes_threshold_per_proc || + GLOBAL_MEM_STAT.get_memory_usage() as usize >= self.settings.max_memory_usage)) + { if let HashTable::HashTable(cell) = std::mem::take(&mut self.hash_table) { self.hash_table = HashTable::PartitionedHashTable( PartitionedHashMethod::convert_hashtable(&self.method, cell)?, @@ -177,11 +212,22 @@ impl AccumulatingTransform for TransformPartialGroupBy } } - if matches!(&self.hash_table, HashTable::PartitionedHashTable(cell) if cell.allocated_bytes() > self.settings.spilling_bytes_threshold_per_proc) - || GLOBAL_MEM_STAT.get_memory_usage() as usize >= self.settings.max_memory_usage + if !is_new_agg + && (matches!(&self.hash_table, HashTable::PartitionedHashTable(cell) if cell.allocated_bytes() > self.settings.spilling_bytes_threshold_per_proc) + || GLOBAL_MEM_STAT.get_memory_usage() as usize + >= self.settings.max_memory_usage) { if let HashTable::PartitionedHashTable(v) = std::mem::take(&mut self.hash_table) { + // perf + { + metrics_inc_group_by_partial_spill_count(); + metrics_inc_group_by_partial_spill_cell_count(1); + metrics_inc_group_by_partial_hashtable_allocated_bytes( + v.allocated_bytes() as u64, + ); + } + let _dropper = v._dropper.clone(); let blocks = vec![DataBlock::empty_with_meta( AggregateMeta::::create_spilling(v), @@ -200,6 +246,41 @@ impl AccumulatingTransform for TransformPartialGroupBy unreachable!() } } + + if is_new_agg + && (matches!(&self.hash_table, HashTable::AggregateHashTable(cell) if cell.allocated_bytes() > self.settings.spilling_bytes_threshold_per_proc + || GLOBAL_MEM_STAT.get_memory_usage() as usize >= self.settings.max_memory_usage)) + { + if let HashTable::AggregateHashTable(v) = std::mem::take(&mut self.hash_table) { + let group_types = v.payload.group_types.clone(); + let aggrs = v.payload.aggrs.clone(); + v.config.update_current_max_radix_bits(); + let config = v + .config + .clone() + .with_initial_radix_bits(v.config.max_radix_bits); + let mut state = PayloadFlushState::default(); + + // repartition to max for normalization + let partitioned_payload = v + .payload + .repartition(1 << config.max_radix_bits, &mut state); + let blocks = vec![DataBlock::empty_with_meta( + AggregateMeta::::create_agg_spilling(partitioned_payload), + )]; + + let arena = Arc::new(Bump::new()); + self.hash_table = HashTable::AggregateHashTable(AggregateHashTable::new( + group_types, + aggrs, + config, + arena, + )); + return Ok(blocks); + } + + unreachable!() + } } Ok(vec![]) @@ -210,9 +291,14 @@ impl AccumulatingTransform for TransformPartialGroupBy HashTable::MovedOut => unreachable!(), HashTable::HashTable(cell) => match cell.hashtable.len() == 0 { true => vec![], - false => vec![DataBlock::empty_with_meta( - AggregateMeta::::create_hashtable(-1, cell), - )], + false => { + metrics_inc_aggregate_partial_hashtable_allocated_bytes( + cell.allocated_bytes() as u64 + ); + vec![DataBlock::empty_with_meta( + AggregateMeta::::create_hashtable(-1, cell), + )] + } }, HashTable::PartitionedHashTable(v) => { info!( @@ -220,6 +306,9 @@ impl AccumulatingTransform for TransformPartialGroupBy convert_number_size(v.len() as f64), convert_byte_size(v.allocated_bytes() as f64) ); + + metrics_inc_aggregate_partial_hashtable_allocated_bytes(v.allocated_bytes() as u64); + let _ = v.hashtable.unsize_key_size(); let cells = PartitionedHashTableDropper::split_cell(v); let mut blocks = Vec::with_capacity(cells.len()); @@ -231,6 +320,27 @@ impl AccumulatingTransform for TransformPartialGroupBy } } + blocks + } + HashTable::AggregateHashTable(hashtable) => { + metrics_inc_aggregate_partial_hashtable_allocated_bytes( + hashtable.allocated_bytes() as u64, + ); + + let partition_count = hashtable.payload.partition_count(); + let mut blocks = Vec::with_capacity(partition_count); + for (bucket, payload) in hashtable.payload.payloads.into_iter().enumerate() { + if payload.len() != 0 { + blocks.push(DataBlock::empty_with_meta( + AggregateMeta::::create_agg_payload( + bucket as isize, + payload, + partition_count, + ), + )); + } + } + blocks } }) diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_bucket.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_bucket.rs index 8f1b414a7997..f7fe175dcfb4 100644 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_bucket.rs +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_bucket.rs @@ -38,6 +38,7 @@ use databend_common_storage::DataOperator; use crate::pipelines::processors::transforms::aggregator::aggregate_meta::AggregateMeta; use crate::pipelines::processors::transforms::aggregator::aggregate_meta::HashTablePayload; use crate::pipelines::processors::transforms::aggregator::aggregate_meta::SerializedPayload; +use crate::pipelines::processors::transforms::aggregator::new_transform_partition_bucket::NewTransformPartitionBucket; use crate::pipelines::processors::transforms::aggregator::AggregatorParams; use crate::pipelines::processors::transforms::aggregator::PartitionedHashTableDropper; use crate::pipelines::processors::transforms::aggregator::TransformAggregateSpillReader; @@ -58,7 +59,6 @@ struct InputPortState { pub struct TransformPartitionBucket { output: Arc, inputs: Vec, - method: Method, working_bucket: isize, pushing_bucket: isize, @@ -83,7 +83,6 @@ impl Ok(TransformPartitionBucket { method, - // params, inputs, working_bucket: 0, pushing_bucket: 0, @@ -129,7 +128,7 @@ impl } let data_block = self.inputs[index].port.pull_data().unwrap()?; - self.inputs[index].bucket = self.add_bucket(data_block); + self.inputs[index].bucket = self.add_bucket(data_block)?; if self.inputs[index].bucket <= SINGLE_LEVEL_BUCKET_NUM { self.inputs[index].port.set_need_data(); @@ -140,12 +139,14 @@ impl Ok(self.initialized_all_inputs) } - fn add_bucket(&mut self, mut data_block: DataBlock) -> isize { + fn add_bucket(&mut self, mut data_block: DataBlock) -> Result { if let Some(block_meta) = data_block.get_meta() { if let Some(block_meta) = AggregateMeta::::downcast_ref_from(block_meta) { let (bucket, res) = match block_meta { AggregateMeta::Spilling(_) => unreachable!(), AggregateMeta::Partitioned { .. } => unreachable!(), + AggregateMeta::AggregatePayload(_) => unreachable!(), + AggregateMeta::AggregateSpilling(_) => unreachable!(), AggregateMeta::BucketSpilled(payload) => { (payload.bucket, SINGLE_LEVEL_BUCKET_NUM) } @@ -176,7 +177,7 @@ impl }; } - return SINGLE_LEVEL_BUCKET_NUM; + return Ok(SINGLE_LEVEL_BUCKET_NUM); } unreachable!() @@ -193,13 +194,13 @@ impl } }; - return res; + return Ok(res); } } } self.unsplitted_blocks.push(data_block); - SINGLE_LEVEL_BUCKET_NUM + Ok(SINGLE_LEVEL_BUCKET_NUM) } fn try_push_data_block(&mut self) -> bool { @@ -268,7 +269,7 @@ impl blocks.push(match data_block.is_empty() { true => None, false => Some(DataBlock::empty_with_meta( - AggregateMeta::::create_serialized(bucket as isize, data_block), + AggregateMeta::::create_serialized(bucket as isize, data_block, 0), )), }); } @@ -361,7 +362,7 @@ impl Processor } let data_block = self.inputs[index].port.pull_data().unwrap()?; - self.inputs[index].bucket = self.add_bucket(data_block); + self.inputs[index].bucket = self.add_bucket(data_block)?; debug_assert!(self.unsplitted_blocks.is_empty()); if self.inputs[index].bucket <= self.working_bucket { @@ -414,6 +415,8 @@ impl Processor AggregateMeta::Partitioned { .. } => unreachable!(), AggregateMeta::Serialized(payload) => self.partition_block(payload)?, AggregateMeta::HashTable(payload) => self.partition_hashtable(payload)?, + AggregateMeta::AggregatePayload(_) => unreachable!(), + AggregateMeta::AggregateSpilling(_) => unreachable!(), }; for (bucket, block) in data_blocks.into_iter().enumerate() { @@ -440,47 +443,92 @@ pub fn build_partition_bucket, ) -> Result<()> { - let input_nums = pipeline.output_len(); - let transform = TransformPartitionBucket::::create(method.clone(), input_nums)?; - - let output = transform.get_output(); - let inputs_port = transform.get_inputs(); - - pipeline.add_pipe(Pipe::create(inputs_port.len(), 1, vec![PipeItem::create( - ProcessorPtr::create(Box::new(transform)), - inputs_port, - vec![output], - )])); - - pipeline.try_resize(input_nums)?; - - let operator = DataOperator::instance().operator(); - pipeline.add_transform(|input, output| { - let operator = operator.clone(); - match params.aggregate_functions.is_empty() { - true => TransformGroupBySpillReader::::create(input, output, operator), - false => TransformAggregateSpillReader::::create(input, output, operator), - } - })?; + if params.enable_experimental_aggregate_hashtable { + let input_nums = pipeline.output_len(); + let transform = + NewTransformPartitionBucket::::create(input_nums, params.clone())?; + + let output = transform.get_output(); + let inputs_port = transform.get_inputs(); + + pipeline.add_pipe(Pipe::create(inputs_port.len(), 1, vec![PipeItem::create( + ProcessorPtr::create(Box::new(transform)), + inputs_port, + vec![output], + )])); + + pipeline.try_resize(input_nums)?; - pipeline.add_transform(|input, output| { - Ok(ProcessorPtr::create( + let operator = DataOperator::instance().operator(); + pipeline.add_transform(|input, output| { + let operator = operator.clone(); match params.aggregate_functions.is_empty() { - true => TransformFinalGroupBy::try_create( - input, - output, - method.clone(), - params.clone(), - )?, - false => TransformFinalAggregate::try_create( - input, - output, - method.clone(), - params.clone(), - )?, - }, - )) - })?; + true => TransformGroupBySpillReader::::create(input, output, operator), + false => TransformAggregateSpillReader::::create(input, output, operator), + } + })?; + + pipeline.add_transform(|input, output| { + Ok(ProcessorPtr::create( + match params.aggregate_functions.is_empty() { + true => TransformFinalGroupBy::try_create( + input, + output, + method.clone(), + params.clone(), + )?, + false => TransformFinalAggregate::try_create( + input, + output, + method.clone(), + params.clone(), + )?, + }, + )) + })?; + } else { + let input_nums = pipeline.output_len(); + let transform = TransformPartitionBucket::::create(method.clone(), input_nums)?; + + let output = transform.get_output(); + let inputs_port = transform.get_inputs(); + + pipeline.add_pipe(Pipe::create(inputs_port.len(), 1, vec![PipeItem::create( + ProcessorPtr::create(Box::new(transform)), + inputs_port, + vec![output], + )])); + + pipeline.try_resize(input_nums)?; + + let operator = DataOperator::instance().operator(); + pipeline.add_transform(|input, output| { + let operator = operator.clone(); + match params.aggregate_functions.is_empty() { + true => TransformGroupBySpillReader::::create(input, output, operator), + false => TransformAggregateSpillReader::::create(input, output, operator), + } + })?; + + pipeline.add_transform(|input, output| { + Ok(ProcessorPtr::create( + match params.aggregate_functions.is_empty() { + true => TransformFinalGroupBy::try_create( + input, + output, + method.clone(), + params.clone(), + )?, + false => TransformFinalAggregate::try_create( + input, + output, + method.clone(), + params.clone(), + )?, + }, + )) + })?; + } Ok(()) } diff --git a/src/query/service/tests/it/pipelines/executor/executor_graph.rs b/src/query/service/tests/it/pipelines/executor/executor_graph.rs index e7e436cf6e2d..4b21e1b307d3 100644 --- a/src/query/service/tests/it/pipelines/executor/executor_graph.rs +++ b/src/query/service/tests/it/pipelines/executor/executor_graph.rs @@ -30,7 +30,7 @@ use databend_common_pipeline_sources::BlocksSource; use databend_common_pipeline_transforms::processors::TransformDummy; use databend_query::pipelines::executor::ExecutorSettings; use databend_query::pipelines::executor::ExecutorWorkerContext; -use databend_query::pipelines::executor::PipelineExecutor; +use databend_query::pipelines::executor::QueryPipelineExecutor; use databend_query::pipelines::executor::RunningGraph; use databend_query::pipelines::executor::WorkersCondvar; use databend_query::pipelines::processors::InputPort; @@ -187,9 +187,9 @@ async fn test_simple_schedule_queue() -> Result<()> { let pipeline = create_simple_pipeline(ctx)?; // init queue and result should be sink node - let init_queue = unsafe { pipeline.init_schedule_queue(0)? }; + let init_queue = unsafe { pipeline.clone().init_schedule_queue(0)? }; unsafe { - let _ = init_queue.sync_queue.front().unwrap().process(); + let _ = init_queue.sync_queue.front().unwrap().processor.process(); } // node_indices is input of schedule_queue @@ -198,7 +198,7 @@ async fn test_simple_schedule_queue() -> Result<()> { let scheduled_result = [1, 0, 1, 2]; for (i, &index) in node_indices.iter().enumerate() { - let scheduled = unsafe { pipeline.schedule_queue(NodeIndex::new(index))? }; + let scheduled = unsafe { pipeline.clone().schedule_queue(NodeIndex::new(index))? }; assert_eq!(scheduled.sync_queue.len(), if i == 4 { 0 } else { 1 }); assert_eq!(scheduled.async_queue.len(), 0); @@ -207,9 +207,9 @@ async fn test_simple_schedule_queue() -> Result<()> { continue; } unsafe { - let _ = scheduled.sync_queue.front().unwrap().process(); + let _ = scheduled.sync_queue.front().unwrap().processor.process(); assert_eq!( - scheduled.sync_queue.front().unwrap().id().index(), + scheduled.sync_queue.front().unwrap().processor.id().index(), scheduled_result[i] ); } @@ -225,12 +225,12 @@ async fn test_parallel_schedule_queue() -> Result<()> { let pipeline = create_parallel_simple_pipeline(ctx)?; // init queue and result should be two sink nodes - let init_queue = unsafe { pipeline.init_schedule_queue(0)? }; + let init_queue = unsafe { pipeline.clone().init_schedule_queue(0)? }; unsafe { - let _ = init_queue.sync_queue[0].process(); + let _ = init_queue.sync_queue[0].processor.process(); } unsafe { - let _ = init_queue.sync_queue[1].process(); + let _ = init_queue.sync_queue[1].processor.process(); } // node_indices is input of schedule_queue @@ -239,7 +239,7 @@ async fn test_parallel_schedule_queue() -> Result<()> { let scheduled_result = [2, 3, 0, 1, 2, 3, 4, 5]; for (i, &index) in node_indices.iter().enumerate() { - let scheduled = unsafe { pipeline.schedule_queue(NodeIndex::new(index))? }; + let scheduled = unsafe { pipeline.clone().schedule_queue(NodeIndex::new(index))? }; assert_eq!( scheduled.sync_queue.len(), @@ -251,9 +251,9 @@ async fn test_parallel_schedule_queue() -> Result<()> { continue; } unsafe { - let _ = scheduled.sync_queue.front().unwrap().process(); + let _ = scheduled.sync_queue.front().unwrap().processor.process(); assert_eq!( - scheduled.sync_queue.front().unwrap().id().index(), + scheduled.sync_queue.front().unwrap().processor.id().index(), scheduled_result[i] ); } @@ -269,10 +269,10 @@ async fn test_resize_schedule_queue() -> Result<()> { let pipeline = create_resize_pipeline(ctx)?; // init queue and result should be two sink nodes - let init_queue = unsafe { pipeline.init_schedule_queue(0)? }; + let init_queue = unsafe { pipeline.clone().init_schedule_queue(0)? }; unsafe { - let _ = init_queue.sync_queue[0].process(); - let _ = init_queue.sync_queue[1].process(); + let _ = init_queue.sync_queue[0].processor.process(); + let _ = init_queue.sync_queue[1].processor.process(); } // node_indices is input of schedule_queue @@ -283,26 +283,29 @@ async fn test_resize_schedule_queue() -> Result<()> { let scheduled_result = [5, 2, 3, 0, 2, 3, 5, 7, 8]; let mut acc = 0; for (i, &index) in node_indices.iter().enumerate() { - let scheduled = unsafe { pipeline.schedule_queue(NodeIndex::new(index))? }; + let scheduled = unsafe { pipeline.clone().schedule_queue(NodeIndex::new(index))? }; assert_eq!(scheduled.sync_queue.len(), sync_length[i]); assert_eq!(scheduled.async_queue.len(), 0); match sync_length[i] { 0 => continue, 1 => unsafe { - let _ = scheduled.sync_queue.front().unwrap().process(); + let _ = scheduled.sync_queue.front().unwrap().processor.process(); assert_eq!( - scheduled.sync_queue.front().unwrap().id().index(), + scheduled.sync_queue.front().unwrap().processor.id().index(), scheduled_result[acc] ); acc += 1; }, 2 => unsafe { - let _ = scheduled.sync_queue[0].process(); - let _ = scheduled.sync_queue[1].process(); - assert_eq!(scheduled.sync_queue[0].id().index(), scheduled_result[acc]); + let _ = scheduled.sync_queue[0].processor.process(); + let _ = scheduled.sync_queue[1].processor.process(); assert_eq!( - scheduled.sync_queue[1].id().index(), + scheduled.sync_queue[0].processor.id().index(), + scheduled_result[acc] + ); + assert_eq!( + scheduled.sync_queue[1].processor.id().index(), scheduled_result[acc + 1] ); acc += 2; @@ -320,16 +323,16 @@ async fn test_schedule_queue_twice_without_processing() -> Result<()> { let ctx = fixture.new_query_ctx().await?; let pipeline = create_simple_pipeline(ctx)?; - let init_queue = unsafe { pipeline.init_schedule_queue(0)? }; + let init_queue = unsafe { pipeline.clone().init_schedule_queue(0)? }; unsafe { - let _ = init_queue.sync_queue.front().unwrap().process(); + let _ = init_queue.sync_queue.front().unwrap().processor.process(); } - let scheduled = unsafe { pipeline.schedule_queue(NodeIndex::new(2))? }; + let scheduled = unsafe { pipeline.clone().schedule_queue(NodeIndex::new(2))? }; assert_eq!(scheduled.sync_queue.len(), 1); // schedule a need data node twice, the second time should be ignored and return empty queue - let scheduled = unsafe { pipeline.schedule_queue(NodeIndex::new(2))? }; + let scheduled = unsafe { pipeline.clone().schedule_queue(NodeIndex::new(2))? }; assert_eq!(scheduled.sync_queue.len(), 0); Ok(()) @@ -347,7 +350,7 @@ async fn test_schedule_with_one_tasks() -> Result<()> { let mut context = ExecutorWorkerContext::create(1, WorkersCondvar::create(1), Arc::new("".to_string())); - let init_queue = unsafe { graph.init_schedule_queue(0)? }; + let init_queue = unsafe { graph.clone().init_schedule_queue(0)? }; assert_eq!(init_queue.sync_queue.len(), 1); init_queue.schedule(&executor.global_tasks_queue, &mut context, &executor); assert!(context.has_task()); @@ -371,7 +374,7 @@ async fn test_schedule_with_two_tasks() -> Result<()> { let mut context = ExecutorWorkerContext::create(1, WorkersCondvar::create(1), Arc::new("".to_string())); - let init_queue = unsafe { graph.init_schedule_queue(0)? }; + let init_queue = unsafe { graph.clone().init_schedule_queue(0)? }; assert_eq!(init_queue.sync_queue.len(), 2); init_queue.schedule(&executor.global_tasks_queue, &mut context, &executor); assert!(context.has_task()); @@ -383,7 +386,7 @@ async fn test_schedule_with_two_tasks() -> Result<()> { Ok(()) } -fn create_simple_pipeline(ctx: Arc) -> Result { +fn create_simple_pipeline(ctx: Arc) -> Result> { let (_rx, sink_pipe) = create_sink_pipe(1)?; let (_tx, source_pipe) = create_source_pipe(ctx, 1)?; let mut pipeline = Pipeline::create(); @@ -391,10 +394,10 @@ fn create_simple_pipeline(ctx: Arc) -> Result { pipeline.add_pipe(create_transform_pipe(1)?); pipeline.add_pipe(sink_pipe); - RunningGraph::create(pipeline) + RunningGraph::create(pipeline, 1) } -fn create_parallel_simple_pipeline(ctx: Arc) -> Result { +fn create_parallel_simple_pipeline(ctx: Arc) -> Result> { let (_rx, sink_pipe) = create_sink_pipe(2)?; let (_tx, source_pipe) = create_source_pipe(ctx, 2)?; @@ -403,10 +406,10 @@ fn create_parallel_simple_pipeline(ctx: Arc) -> Result) -> Result { +fn create_resize_pipeline(ctx: Arc) -> Result> { let (_rx, sink_pipe) = create_sink_pipe(2)?; let (_tx, source_pipe) = create_source_pipe(ctx, 1)?; @@ -419,7 +422,7 @@ fn create_resize_pipeline(ctx: Arc) -> Result { pipeline.try_resize(2)?; pipeline.add_pipe(sink_pipe); - RunningGraph::create(pipeline) + RunningGraph::create(pipeline, 1) } fn create_source_pipe( @@ -483,7 +486,7 @@ fn create_sink_pipe(size: usize) -> Result<(Vec>>, Pi async fn create_executor_with_simple_pipeline( ctx: Arc, size: usize, -) -> Result> { +) -> Result> { let (_rx, sink_pipe) = create_sink_pipe(size)?; let (_tx, source_pipe) = create_source_pipe(ctx, size)?; let mut pipeline = Pipeline::create(); @@ -494,6 +497,7 @@ async fn create_executor_with_simple_pipeline( let settings = ExecutorSettings { query_id: Arc::new("".to_string()), max_execute_time_in_seconds: Default::default(), + enable_new_executor: false, }; - PipelineExecutor::create(pipeline, settings) + QueryPipelineExecutor::create(pipeline, settings) } diff --git a/src/query/service/tests/it/pipelines/executor/pipeline_executor.rs b/src/query/service/tests/it/pipelines/executor/pipeline_executor.rs index 907c37840c03..18494f0e2b54 100644 --- a/src/query/service/tests/it/pipelines/executor/pipeline_executor.rs +++ b/src/query/service/tests/it/pipelines/executor/pipeline_executor.rs @@ -32,7 +32,7 @@ use databend_common_pipeline_core::Pipeline; use databend_common_pipeline_sinks::SyncSenderSink; use databend_common_pipeline_sources::SyncReceiverSource; use databend_query::pipelines::executor::ExecutorSettings; -use databend_query::pipelines::executor::PipelineExecutor; +use databend_query::pipelines::executor::QueryPipelineExecutor; use databend_query::sessions::QueryContext; use databend_query::test_kits::TestFixture; @@ -43,12 +43,13 @@ async fn test_always_call_on_finished() -> Result<()> { let settings = ExecutorSettings { query_id: Arc::new("".to_string()), max_execute_time_in_seconds: Default::default(), + enable_new_executor: false, }; { let (called_finished, pipeline) = create_pipeline(); - match PipelineExecutor::create(pipeline, settings.clone()) { + match QueryPipelineExecutor::create(pipeline, settings.clone()) { Ok(_) => unreachable!(), Err(error) => { assert_eq!(error.code(), 1001); @@ -70,7 +71,7 @@ async fn test_always_call_on_finished() -> Result<()> { pipeline.add_pipe(sink_pipe); pipeline.set_max_threads(1); - let executor = PipelineExecutor::create(pipeline, settings.clone())?; + let executor = QueryPipelineExecutor::create(pipeline, settings.clone())?; match executor.execute() { Ok(_) => unreachable!(), diff --git a/src/query/settings/src/settings_default.rs b/src/query/settings/src/settings_default.rs index 74b15449d293..18488758089a 100644 --- a/src/query/settings/src/settings_default.rs +++ b/src/query/settings/src/settings_default.rs @@ -595,6 +595,12 @@ impl DefaultSettings { mode: SettingMode::Both, range: Some(SettingRange::Numeric(0..=1)), }), + ("enable_experimental_aggregate_hashtable", DefaultSettingValue { + value: UserSettingValue::UInt64(1), + desc: "Enables experimental aggregate hashtable", + mode: SettingMode::Both, + range: Some(SettingRange::Numeric(0..=1)), + }), ("numeric_cast_option", DefaultSettingValue { value: UserSettingValue::String("rounding".to_string()), desc: "Set numeric cast mode as \"rounding\" or \"truncating\".", @@ -631,6 +637,12 @@ impl DefaultSettings { mode: SettingMode::Both, range: Some(SettingRange::Numeric(0..=1)), }), + ("enable_experimental_new_executor", DefaultSettingValue { + value: UserSettingValue::UInt64(0), + desc: "Enables experimental new executor", + mode: SettingMode::Both, + range: None, + }), ]); Ok(Arc::new(DefaultSettings { diff --git a/src/query/settings/src/settings_getter_setter.rs b/src/query/settings/src/settings_getter_setter.rs index 3690d6e31708..3333eea3004e 100644 --- a/src/query/settings/src/settings_getter_setter.rs +++ b/src/query/settings/src/settings_getter_setter.rs @@ -370,6 +370,10 @@ impl Settings { Ok(self.try_get_u64("efficiently_memory_group_by")? == 1) } + pub fn get_enable_experimental_aggregate_hashtable(&self) -> Result { + Ok(self.try_get_u64("enable_experimental_aggregate_hashtable")? == 1) + } + pub fn get_lazy_read_threshold(&self) -> Result { self.try_get_u64("lazy_read_threshold") } @@ -569,4 +573,8 @@ impl Settings { u64::from(val), ) } + + pub fn get_enable_experimental_new_executor(&self) -> Result { + Ok(self.try_get_u64("enable_experimental_new_executor")? == 1) + } } diff --git a/src/query/sql/src/executor/format.rs b/src/query/sql/src/executor/format.rs index 8222f9a5c809..577f0b8aed77 100644 --- a/src/query/sql/src/executor/format.rs +++ b/src/query/sql/src/executor/format.rs @@ -587,10 +587,6 @@ fn aggregate_partial_to_format_tree( .join(", "); let mut children = vec![ - FormatTreeNode::new(format!( - "output columns: [{}]", - format_output_columns(plan.output_schema()?, metadata, true) - )), FormatTreeNode::new(format!("group by: [{group_by}]")), FormatTreeNode::new(format!("aggregate functions: [{agg_funcs}]")), ]; @@ -679,11 +675,8 @@ fn window_to_format_tree( let order_by = plan .order_by .iter() - .map(|v| { - let name = metadata.column(v.order_by).name(); - Ok(name) - }) - .collect::>>()? + .map(|v| v.display_name.clone()) + .collect::>() .join(", "); let frame = plan.window_frame.to_string(); @@ -727,10 +720,9 @@ fn sort_to_format_tree( .order_by .iter() .map(|sort_key| { - let index = sort_key.order_by; Ok(format!( "{} {} {}", - metadata.column(index).name(), + sort_key.display_name, if sort_key.asc { "ASC" } else { "DESC" }, if sort_key.nulls_first { "NULLS FIRST" diff --git a/src/query/sql/src/executor/physical_plan.rs b/src/query/sql/src/executor/physical_plan.rs index bfb5c1c7ad4d..6f16263bf965 100644 --- a/src/query/sql/src/executor/physical_plan.rs +++ b/src/query/sql/src/executor/physical_plan.rs @@ -413,43 +413,19 @@ impl PhysicalPlan { true => String::new(), false => v.predicates[0].as_expr(&BUILTIN_FUNCTIONS).sql_display(), }, - PhysicalPlan::AggregatePartial(v) => v - .agg_funcs - .iter() - .map(|x| { - format!( - "{}({})", - x.sig.name, - x.arg_indices - .iter() - .map(|x| x.to_string()) - .collect::>() - .join(", ") - ) - }) - .join(", "), - PhysicalPlan::AggregateFinal(v) => v - .agg_funcs - .iter() - .map(|x| { - format!( - "{}({})", - x.sig.name, - x.arg_indices - .iter() - .map(|x| x.to_string()) - .collect::>() - .join(", ") - ) - }) - .join(", "), + PhysicalPlan::AggregatePartial(v) => { + v.agg_funcs.iter().map(|x| x.display.clone()).join(", ") + } + PhysicalPlan::AggregateFinal(v) => { + v.agg_funcs.iter().map(|x| x.display.clone()).join(", ") + } PhysicalPlan::Sort(v) => v .order_by .iter() .map(|x| { format!( "{}{}{}", - x.order_by, + x.display_name, if x.asc { "" } else { " DESC" }, if x.nulls_first { " NULLS FIRST" } else { "" }, ) @@ -471,22 +447,26 @@ impl PhysicalPlan { .map(|(x, _)| x.as_expr(&BUILTIN_FUNCTIONS).sql_display()) .join(", "), PhysicalPlan::HashJoin(v) => { - format!( - "{} AND {}", - v.build_keys - .iter() - .zip(v.probe_keys.iter()) - .map(|(l, r)| format!( + let mut conditions = v + .build_keys + .iter() + .zip(v.probe_keys.iter()) + .map(|(l, r)| { + format!( "({} = {})", l.as_expr(&BUILTIN_FUNCTIONS).sql_display(), r.as_expr(&BUILTIN_FUNCTIONS).sql_display() - )) - .join(" AND "), + ) + }) + .collect::>(); + + conditions.extend( v.non_equi_conditions .iter() - .map(|x| x.as_expr(&BUILTIN_FUNCTIONS).sql_display()) - .join(" AND ") - ) + .map(|x| x.as_expr(&BUILTIN_FUNCTIONS).sql_display()), + ); + + conditions.join(" AND ") } PhysicalPlan::ProjectSet(v) => v .srf_exprs @@ -520,7 +500,7 @@ impl PhysicalPlan { .map(|x| { format!( "{}{}{}", - x.order_by, + x.display_name, if x.asc { "" } else { " DESC" }, if x.nulls_first { " NULLS FIRST" } else { "" }, ) @@ -536,27 +516,29 @@ impl PhysicalPlan { projected_schema.fields.iter().map(|f| f.name()).join(", ") } PhysicalPlan::RangeJoin(v) => { - format!( - "{} AND {}", - v.conditions - .iter() - .map(|condition| { - let left = condition - .left_expr - .as_expr(&BUILTIN_FUNCTIONS) - .sql_display(); - let right = condition - .right_expr - .as_expr(&BUILTIN_FUNCTIONS) - .sql_display(); - format!("{left} {:?} {right}", condition.operator) - }) - .join(" AND "), + let mut condition = v + .conditions + .iter() + .map(|condition| { + let left = condition + .left_expr + .as_expr(&BUILTIN_FUNCTIONS) + .sql_display(); + let right = condition + .right_expr + .as_expr(&BUILTIN_FUNCTIONS) + .sql_display(); + format!("{left} {:?} {right}", condition.operator) + }) + .collect::>(); + + condition.extend( v.other_conditions .iter() - .map(|x| x.as_expr(&BUILTIN_FUNCTIONS).sql_display()) - .join(" AND ") - ) + .map(|x| x.as_expr(&BUILTIN_FUNCTIONS).sql_display()), + ); + + condition.join(" AND ") } PhysicalPlan::Udf(v) => v .udf_funcs @@ -569,135 +551,131 @@ impl PhysicalPlan { PhysicalPlan::UnionAll(v) => v .pairs .iter() - .map(|(l, r)| format!("{} <- {}", l, r)) + .map(|(l, r)| format!("#{} <- #{}", l, r)) .join(", "), _ => String::new(), }) } pub fn get_labels(&self) -> Result>> { - Ok(match self { + let mut labels = HashMap::with_capacity(16); + + match self { PhysicalPlan::TableScan(v) => { - let output_schema = v.output_schema()?; - let source_schema = v.source.source_info.schema(); - let columns_name = format!( - "Columns ({} / {})", - output_schema.num_fields(), - source_schema.num_fields() + labels.insert(String::from("Full table name"), vec![format!( + "{}.{}", + v.source.catalog_info.name_ident.catalog_name, + v.source.source_info.desc() + )]); + + labels.insert( + format!( + "Columns ({} / {})", + v.output_schema()?.num_fields(), + v.source.source_info.schema().num_fields() + ), + v.name_mapping.keys().cloned().collect(), ); - HashMap::from([ - (String::from("Full table name"), vec![format!( - "{}.{}", - v.source.catalog_info.name_ident.catalog_name, - v.source.source_info.desc() - )]), - (columns_name, v.name_mapping.keys().cloned().collect()), - (String::from("Total partitions"), vec![ - v.source.statistics.partitions_total.to_string(), - ]), - ]) + labels.insert(String::from("Total partitions"), vec![ + v.source.statistics.partitions_total.to_string(), + ]); } - PhysicalPlan::Filter(v) => HashMap::from([( - String::from("Filter condition"), - v.predicates - .iter() - .map(|x| x.as_expr(&BUILTIN_FUNCTIONS).sql_display()) - .collect(), - )]), - PhysicalPlan::Limit(v) => match v.limit { - Some(limit) => HashMap::from([ - (String::from("Number of rows"), vec![limit.to_string()]), - (String::from("Offset"), vec![v.offset.to_string()]), - ]), - None => HashMap::from([(String::from("Offset"), vec![v.offset.to_string()])]), - }, - PhysicalPlan::EvalScalar(v) => HashMap::from([( - String::from("List of Expressions"), - v.exprs - .iter() - .map(|(x, _)| x.as_expr(&BUILTIN_FUNCTIONS).sql_display()) - .collect(), - )]), - PhysicalPlan::Project(v) => HashMap::from([( - String::from("List of Expressions"), - v.output_schema()? - .fields - .iter() - .map(|x| x.name()) - .cloned() - .collect(), - )]), - PhysicalPlan::AggregatePartial(v) => HashMap::from([ - ( - String::from("Grouping keys"), - v.group_by.iter().map(|x| x.to_string()).collect(), - ), - ( - String::from("Aggregate Functions"), - v.agg_funcs - .iter() - .map(|x| { - format!( - "{}({})", - x.sig.name, - x.arg_indices - .iter() - .map(|x| x.to_string()) - .collect::>() - .join(", ") - ) - }) - .collect(), - ), - ]), - PhysicalPlan::AggregateFinal(v) => HashMap::from([ - ( - String::from("Grouping keys"), - v.group_by.iter().map(|x| x.to_string()).collect(), - ), - ( - String::from("Aggregate Functions"), - v.agg_funcs - .iter() - .map(|x| { - format!( - "{}({})", - x.sig.name, - x.arg_indices - .iter() - .map(|x| x.to_string()) - .collect::>() - .join(", ") - ) - }) - .collect(), - ), - ]), - PhysicalPlan::HashJoin(v) => HashMap::from([ - (String::from("Join Type"), vec![v.join_type.to_string()]), - ( - String::from("Join Build Side Keys"), - v.build_keys + PhysicalPlan::Filter(v) => { + labels.insert( + String::from("Filter condition"), + v.predicates .iter() .map(|x| x.as_expr(&BUILTIN_FUNCTIONS).sql_display()) .collect(), - ), - ( - String::from("Join Probe Side Keys"), - v.probe_keys + ); + } + PhysicalPlan::Limit(v) => { + labels.insert(String::from("Offset"), vec![v.offset.to_string()]); + + if let Some(limit) = v.limit { + labels.insert(String::from("Number of rows"), vec![limit.to_string()]); + } + } + PhysicalPlan::EvalScalar(v) => { + labels.insert( + String::from("List of Expressions"), + v.exprs .iter() - .map(|x| x.as_expr(&BUILTIN_FUNCTIONS).sql_display()) + .map(|(x, _)| x.as_expr(&BUILTIN_FUNCTIONS).sql_display()) .collect(), - ), - ( - String::from("Join Conditions"), - v.non_equi_conditions + ); + } + PhysicalPlan::Project(v) => { + labels.insert( + String::from("List of Expressions"), + v.output_schema()? + .fields .iter() - .map(|x| x.as_expr(&BUILTIN_FUNCTIONS).sql_display()) + .map(|x| x.name()) + .cloned() .collect(), - ), - ]), - _ => HashMap::new(), - }) + ); + } + PhysicalPlan::AggregatePartial(v) => { + if !v.group_by_display.is_empty() { + labels.insert(String::from("Grouping keys"), v.group_by_display.clone()); + } + + if !v.agg_funcs.is_empty() { + labels.insert( + String::from("Aggregate Functions"), + v.agg_funcs.iter().map(|x| x.display.clone()).collect(), + ); + } + } + PhysicalPlan::AggregateFinal(v) => { + if !v.group_by_display.is_empty() { + labels.insert(String::from("Grouping keys"), v.group_by_display.clone()); + } + + if !v.agg_funcs.is_empty() { + labels.insert( + String::from("Aggregate Functions"), + v.agg_funcs.iter().map(|x| x.display.clone()).collect(), + ); + } + } + PhysicalPlan::HashJoin(v) => { + labels.insert(String::from("Join Type"), vec![v.join_type.to_string()]); + + if !v.build_keys.is_empty() { + labels.insert( + String::from("Join Build Side Keys"), + v.build_keys + .iter() + .map(|x| x.as_expr(&BUILTIN_FUNCTIONS).sql_display()) + .collect(), + ); + } + + if !v.probe_keys.is_empty() { + labels.insert( + String::from("Join Probe Side Keys"), + v.probe_keys + .iter() + .map(|x| x.as_expr(&BUILTIN_FUNCTIONS).sql_display()) + .collect(), + ); + } + + if !v.non_equi_conditions.is_empty() { + labels.insert( + String::from("Join Conditions"), + v.non_equi_conditions + .iter() + .map(|x| x.as_expr(&BUILTIN_FUNCTIONS).sql_display()) + .collect(), + ); + } + } + _ => {} + }; + + Ok(labels) } } diff --git a/src/query/sql/src/executor/physical_plan_visitor.rs b/src/query/sql/src/executor/physical_plan_visitor.rs index 297d641a43e4..03bf39867f8b 100644 --- a/src/query/sql/src/executor/physical_plan_visitor.rs +++ b/src/query/sql/src/executor/physical_plan_visitor.rs @@ -178,7 +178,9 @@ pub trait PhysicalPlanReplacer { Ok(PhysicalPlan::AggregatePartial(AggregatePartial { plan_id: plan.plan_id, input: Box::new(input), + enable_experimental_aggregate_hashtable: plan.enable_experimental_aggregate_hashtable, group_by: plan.group_by.clone(), + group_by_display: plan.group_by_display.clone(), agg_funcs: plan.agg_funcs.clone(), stat_info: plan.stat_info.clone(), })) @@ -193,6 +195,7 @@ pub trait PhysicalPlanReplacer { before_group_by_schema: plan.before_group_by_schema.clone(), group_by: plan.group_by.clone(), agg_funcs: plan.agg_funcs.clone(), + group_by_display: plan.group_by_display.clone(), stat_info: plan.stat_info.clone(), limit: plan.limit, })) diff --git a/src/query/sql/src/executor/physical_plans/common.rs b/src/query/sql/src/executor/physical_plans/common.rs index 5381de4a35b9..d3ce06c16bc0 100644 --- a/src/query/sql/src/executor/physical_plans/common.rs +++ b/src/query/sql/src/executor/physical_plans/common.rs @@ -43,6 +43,7 @@ pub struct AggregateFunctionDesc { pub output_column: IndexType, /// Bound indices of arguments. Only used in partial aggregation. pub arg_indices: Vec, + pub display: String, } #[derive(Clone, Debug, Eq, PartialEq, serde::Serialize, serde::Deserialize)] @@ -50,6 +51,7 @@ pub struct SortDesc { pub asc: bool, pub nulls_first: bool, pub order_by: IndexType, + pub display_name: String, } #[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] diff --git a/src/query/sql/src/executor/physical_plans/physical_aggregate_final.rs b/src/query/sql/src/executor/physical_plans/physical_aggregate_final.rs index 3c4c47d5ee5e..7347ab0729cd 100644 --- a/src/query/sql/src/executor/physical_plans/physical_aggregate_final.rs +++ b/src/query/sql/src/executor/physical_plans/physical_aggregate_final.rs @@ -47,6 +47,8 @@ pub struct AggregateFinal { pub before_group_by_schema: DataSchemaRef, pub limit: Option, + pub group_by_display: Vec, + // Only used for explain pub stat_info: Option, } @@ -114,6 +116,12 @@ impl PhysicalPlanBuilder { let result = match &agg.mode { AggregateMode::Partial => { + let group_by_display = agg + .group_items + .iter() + .map(|item| Ok(item.scalar.as_expr()?.sql_display())) + .collect::>>()?; + let mut agg_funcs: Vec = agg.aggregate_functions.iter().map(|v| { if let ScalarExpr::AggregateFunction(agg) = &v.scalar { Ok(AggregateFunctionDesc { @@ -140,6 +148,7 @@ impl PhysicalPlanBuilder { )) } }).collect::>()?, + display: v.scalar.as_expr()?.sql_display(), }) } else { Err(ErrorCode::Internal("Expected aggregate function".to_string())) @@ -148,6 +157,8 @@ impl PhysicalPlanBuilder { let settings = self.ctx.get_settings(); let group_by_shuffle_mode = settings.get_group_by_shuffle_mode()?; + let enable_experimental_aggregate_hashtable = + settings.get_enable_experimental_aggregate_hashtable()?; if let Some(grouping_sets) = agg.grouping_sets.as_ref() { assert_eq!(grouping_sets.dup_group_items.len(), group_items.len() - 1); // ignore `_grouping_id`. @@ -182,6 +193,8 @@ impl PhysicalPlanBuilder { plan_id: self.next_plan_id(), input: Box::new(PhysicalPlan::AggregateExpand(expand)), agg_funcs, + enable_experimental_aggregate_hashtable, + group_by_display, group_by: group_items, stat_info: Some(stat_info), } @@ -190,6 +203,8 @@ impl PhysicalPlanBuilder { plan_id: self.next_plan_id(), input, agg_funcs, + enable_experimental_aggregate_hashtable, + group_by_display, group_by: group_items, stat_info: Some(stat_info), } @@ -197,17 +212,42 @@ impl PhysicalPlanBuilder { let settings = self.ctx.get_settings(); let efficiently_memory = settings.get_efficiently_memory_group_by()?; + let enable_experimental_aggregate_hashtable = + settings.get_enable_experimental_aggregate_hashtable()?; - let group_by_key_index = - aggregate_partial.output_schema()?.num_fields() - 1; - let group_by_key_data_type = DataBlock::choose_hash_method_with_types( - &agg.group_items - .iter() - .map(|v| v.scalar.data_type()) - .collect::>>()?, - efficiently_memory, - )? - .data_type(); + let keys = if enable_experimental_aggregate_hashtable { + let schema = aggregate_partial.output_schema()?; + let start = aggregate_partial.agg_funcs.len(); + let end = schema.num_fields(); + let mut groups = Vec::with_capacity(end - start); + for idx in start..end { + let group_key = RemoteExpr::ColumnRef { + span: None, + id: idx, + data_type: schema.field(idx).data_type().clone(), + display_name: (idx - start).to_string(), + }; + groups.push(group_key); + } + groups + } else { + let group_by_key_index = + aggregate_partial.output_schema()?.num_fields() - 1; + let group_by_key_data_type = DataBlock::choose_hash_method_with_types( + &agg.group_items + .iter() + .map(|v| v.scalar.data_type()) + .collect::>>()?, + efficiently_memory, + )? + .data_type(); + vec![RemoteExpr::ColumnRef { + span: None, + id: group_by_key_index, + data_type: group_by_key_data_type, + display_name: "_group_by_key".to_string(), + }] + }; PhysicalPlan::Exchange(Exchange { plan_id: self.next_plan_id(), @@ -215,12 +255,7 @@ impl PhysicalPlanBuilder { allow_adjust_parallelism: true, ignore_exchange: false, input: Box::new(PhysicalPlan::AggregatePartial(aggregate_partial)), - keys: vec![RemoteExpr::ColumnRef { - span: None, - id: group_by_key_index, - data_type: group_by_key_data_type, - display_name: "_group_by_key".to_string(), - }], + keys, }) } _ => { @@ -235,6 +270,8 @@ impl PhysicalPlanBuilder { PhysicalPlan::AggregatePartial(AggregatePartial { plan_id: self.next_plan_id(), agg_funcs, + enable_experimental_aggregate_hashtable, + group_by_display, group_by: group_items, input: Box::new(PhysicalPlan::AggregateExpand(expand)), stat_info: Some(stat_info), @@ -243,6 +280,8 @@ impl PhysicalPlanBuilder { PhysicalPlan::AggregatePartial(AggregatePartial { plan_id: self.next_plan_id(), agg_funcs, + enable_experimental_aggregate_hashtable, + group_by_display, group_by: group_items, input: Box::new(input), stat_info: Some(stat_info), @@ -296,6 +335,7 @@ impl PhysicalPlanBuilder { )) } }).collect::>()?, + display: v.scalar.as_expr()?.sql_display(), }) } else { Err(ErrorCode::Internal("Expected aggregate function".to_string())) @@ -321,6 +361,7 @@ impl PhysicalPlanBuilder { let limit = agg.limit; PhysicalPlan::AggregateFinal(AggregateFinal { plan_id: self.next_plan_id(), + group_by_display: partial.group_by_display.clone(), input: Box::new(input), group_by: group_items, agg_funcs, @@ -340,6 +381,7 @@ impl PhysicalPlanBuilder { PhysicalPlan::AggregateFinal(AggregateFinal { plan_id: self.next_plan_id(), + group_by_display: partial.group_by_display.clone(), input: Box::new(input), group_by: group_items, agg_funcs, diff --git a/src/query/sql/src/executor/physical_plans/physical_aggregate_partial.rs b/src/query/sql/src/executor/physical_plans/physical_aggregate_partial.rs index f6121a1a2458..91953f4a2555 100644 --- a/src/query/sql/src/executor/physical_plans/physical_aggregate_partial.rs +++ b/src/query/sql/src/executor/physical_plans/physical_aggregate_partial.rs @@ -14,6 +14,7 @@ use databend_common_exception::Result; use databend_common_expression::types::DataType; +#[allow(unused_imports)] use databend_common_expression::DataBlock; use databend_common_expression::DataField; use databend_common_expression::DataSchemaRef; @@ -31,6 +32,8 @@ pub struct AggregatePartial { pub input: Box, pub group_by: Vec, pub agg_funcs: Vec, + pub enable_experimental_aggregate_hashtable: bool, + pub group_by_display: Vec, // Only used for explain pub stat_info: Option, @@ -39,6 +42,33 @@ pub struct AggregatePartial { impl AggregatePartial { pub fn output_schema(&self) -> Result { let input_schema = self.input.output_schema()?; + + if self.enable_experimental_aggregate_hashtable { + let mut fields = Vec::with_capacity(self.agg_funcs.len() + self.group_by.len()); + for agg in self.agg_funcs.iter() { + fields.push(DataField::new( + &agg.output_column.to_string(), + DataType::Binary, + )); + } + + let group_types = self + .group_by + .iter() + .map(|index| { + Ok(input_schema + .field_with_name(&index.to_string())? + .data_type() + .clone()) + }) + .collect::>>()?; + + for (idx, data_type) in self.group_by.iter().zip(group_types.iter()) { + fields.push(DataField::new(&idx.to_string(), data_type.clone())); + } + return Ok(DataSchemaRefExt::create(fields)); + } + let mut fields = Vec::with_capacity(self.agg_funcs.len() + self.group_by.is_empty() as usize); for agg in self.agg_funcs.iter() { @@ -63,6 +93,7 @@ impl AggregatePartial { )?; fields.push(DataField::new("_group_by_key", method.data_type())); } + Ok(DataSchemaRefExt::create(fields)) } } diff --git a/src/query/sql/src/executor/physical_plans/physical_sort.rs b/src/query/sql/src/executor/physical_plans/physical_sort.rs index 6e2ae7821ac2..d9c9dc182300 100644 --- a/src/query/sql/src/executor/physical_plans/physical_sort.rs +++ b/src/query/sql/src/executor/physical_plans/physical_sort.rs @@ -137,6 +137,7 @@ impl PhysicalPlanBuilder { asc: v.asc, nulls_first: v.nulls_first, order_by: v.index, + display_name: self.metadata.read().column(v.index).name(), }) .collect(), limit: sort.limit, diff --git a/src/query/sql/src/executor/physical_plans/physical_window.rs b/src/query/sql/src/executor/physical_plans/physical_window.rs index 042d65422ec7..59a1a1cde9b8 100644 --- a/src/query/sql/src/executor/physical_plans/physical_window.rs +++ b/src/query/sql/src/executor/physical_plans/physical_window.rs @@ -247,6 +247,7 @@ impl PhysicalPlanBuilder { asc: v.asc.unwrap_or(true), nulls_first: v.nulls_first.unwrap_or(false), order_by: v.order_by_item.index, + display_name: self.metadata.read().column(v.order_by_item.index).name(), }) .collect::>(); let partition_items = w.partition_by.iter().map(|v| v.index).collect::>(); @@ -276,6 +277,9 @@ impl PhysicalPlanBuilder { } }) .collect::>()?, + display: ScalarExpr::AggregateFunction(agg.clone()) + .as_expr()? + .sql_display(), }), WindowFuncType::LagLead(lag_lead) => { let new_default = match &lag_lead.default { diff --git a/tests/fuse-compat/README.md b/tests/fuse-compat/README.md index bbaa8b1a312b..71b1d7e965b7 100644 --- a/tests/fuse-compat/README.md +++ b/tests/fuse-compat/README.md @@ -7,11 +7,15 @@ by a older version databend-query. ```shell tests/fuse-compat/test-fuse-compat.sh +tests/fuse-compat/test-fuse-forward-compat.sh ``` E.g. `tests/fuse-compat/test-fuse-compat.sh 0.7.151` tests if the fuse-table written by **databend-query-0.7.151** can be read by **current** version databend-query. +`tests/fuse-compat/test-fuse-forward-compat.sh 1.2.307` tests if the fuse-table written +by **current** can be read by **databend-query-0.7.151** version databend-query. + ## Prerequisites - Current version of databend-query and databend-meta must reside in `./bins`: diff --git a/tests/fuse-compat/test-fuse-compat.sh b/tests/fuse-compat/test-fuse-compat.sh index a7055c302b12..cee53d0c5515 100755 --- a/tests/fuse-compat/test-fuse-compat.sh +++ b/tests/fuse-compat/test-fuse-compat.sh @@ -53,7 +53,7 @@ download_query_config "$old_query_ver" old_config download_binary "$old_query_ver" old_config_path="old_config/$query_config_path" -run_test $old_query_ver $old_config_path $logictest_path +run_test $old_query_ver $old_config_path $logictest_path "backward" if [ -n "$stateless_test_path" ]; then diff --git a/tests/fuse-compat/test-fuse-forward-compat.sh b/tests/fuse-compat/test-fuse-forward-compat.sh new file mode 100644 index 000000000000..ea11cb7d799a --- /dev/null +++ b/tests/fuse-compat/test-fuse-forward-compat.sh @@ -0,0 +1,64 @@ +#!/bin/bash + +set -o errexit +SCRIPT_PATH="$(cd "$(dirname "$0")" >/dev/null 2>&1 && pwd)" +echo " === SCRIPT_PATH: $SCRIPT_PATH" +# go to work tree root +cd "$SCRIPT_PATH/../../" +ROOT="$(pwd)" +pwd + +export RUST_BACKTRACE=full + +BUILD_PROFILE="${BUILD_PROFILE:-debug}" + +query_config_path="scripts/ci/deploy/config/databend-query-node-1.toml" + +usage() { + echo " === Assert that an old version query being compatible with lastest version query on fuse-table format" + echo " === Expect ./bins/current contains current version binaries" + echo " === Usage: $0 " +} + +source "${SCRIPT_PATH}/util.sh" + + +# -- main -- + +# The previous version to assert compatibility with +# e.g. old_query_ver="0.7.151" +old_query_ver="$1" + +# default sqllogic test suite is "testlogictest_path=${2:-"./base"}s/fuse-forward-compat/compat-logictest/" +logictest_path=${2:-"./base"} + +# supplementary stateless test suite if provided (optional), which will be searched under "tests/fuse-forward-compat/compat-stateless" +stateless_test_path="$3" + +echo " === old query ver : ${old_query_ver}" +echo " === sql logic test path: ${logictest_path}" +echo " === supplementary stateless test path: ${stateless_test_path}" + + +chmod +x ./bins/current/* + +echo " === current metasrv ver: $(./bins/current/databend-meta --single --cmd ver | tr '\n' ' ')" +echo " === current query ver: $(./bins/current/databend-query --cmd ver | tr '\n' ' ')" +echo " === old query ver: $old_query_ver" + + +mkdir -p ./target/${BUILD_PROFILE}/ + +download_query_config "$old_query_ver" old_config +download_binary "$old_query_ver" + +echo "=== Now test forward compat ===" + +old_config_path="old_config/$query_config_path" +run_test $old_query_ver $old_config_path $logictest_path "forward" + +if [ -n "$stateless_test_path" ]; +then + echo "=== ruing supplementary stateless test: ${stateless_test_path}" + run_stateless $stateless_test_path +fi diff --git a/tests/fuse-compat/util.sh b/tests/fuse-compat/util.sh index 96e39f0d02e6..5dc2e8f98db3 100755 --- a/tests/fuse-compat/util.sh +++ b/tests/fuse-compat/util.sh @@ -117,6 +117,7 @@ run_test() { local query_old_ver="$1" local old_config_path="$2" local logictest_path="tests/fuse-compat/compat-logictest/$3" + local forward="$4" echo " === Test with query-$query_old_ver and current query" @@ -150,12 +151,20 @@ run_test() { export RUST_BACKTRACE=1 - echo ' === Start old databend-meta...' + if [ "$forward" == "forward" ] + then + echo ' === Start new databend-meta and databend-query...' + config_path="scripts/ci/deploy/config/databend-query-node-1.toml" + log="query-current.log" + start $metasrv_new $query_new $config_path $log - nohup "$metasrv_old" --single --log-level=DEBUG & - python3 scripts/ci/wait_tcp.py --timeout 10 --port 9191 + echo " === Run test: fuse_compat_write with current query" + else + echo ' === Start old databend-meta...' - echo ' === Start old databend-query...' + echo ' === Start old databend-meta and databend-query...' + log="query-old.log" + start "$metasrv_old" "$query_old" "$old_config_path" $log # TODO clean up data? echo " === bring up $query_old" @@ -166,7 +175,10 @@ run_test() { echo " === Run test: fuse_compat_write with old query" # download_logictest $query_old_ver old_logictest/$query_old_ver - # run_logictest old_logictest/$query_old_ver fuse_compat_write + # if backward + # run_logictest new_logictest/$query_old_ver fuse_compat_write + # if forward + # run_logictest new_logictest/$query_new_ver fuse_compat_write $sqllogictests --handlers mysql --suites "$logictest_path" --run_file fuse_compat_write kill_proc databend-query @@ -191,6 +203,24 @@ run_test() { $sqllogictests --handlers mysql --suites "$logictest_path" --run_file fuse_compat_read } +start() { + local metasrv="$1" + local query="$2" + local config_path="$3" + local log="$4" + export RUST_BACKTRACE=1 + echo " === Start $metasrv databend-meta..." + + nohup "$metasrv" --single --log-level=DEBUG & + python3 scripts/ci/wait_tcp.py --timeout 20 --port 9191 + + echo " === Start $query databend-query..." + + echo "databend config path: $config_path" + + nohup "$query" -c "$config_path" --log-level DEBUG --meta-endpoints "0.0.0.0:9191" > "$log" & + python3 scripts/ci/wait_tcp.py --timeout 30 --port 3307 +} # Run suppelmentary stateless tests run_stateless() { local case_path="$1" diff --git a/tests/sqllogictests/suites/base/03_common/03_0043_new_agg_hashtable.test b/tests/sqllogictests/suites/base/03_common/03_0043_new_agg_hashtable.test new file mode 100644 index 000000000000..b151c71f5771 --- /dev/null +++ b/tests/sqllogictests/suites/base/03_common/03_0043_new_agg_hashtable.test @@ -0,0 +1,230 @@ +# test new agg hashtable +statement ok +set enable_experimental_aggregate_hashtable=1 + +statement ok +SELECT number % 5,max(number),min(number),sum(number) FROM numbers_mt(0) GROUP BY number % 5 + +statement ok +SELECT max(number) FROM numbers_mt (10) WHERE number > 999999 GROUP BY number % 4 + +statement ok +select number, count(DISTINCT number) as a from numbers(5000000) GROUP BY number HAVING (a > 2) LIMIT 10 + +query III +SELECT number%5 as c1, number%4 as c2, number%3 as c3 FROM numbers_mt(1000000) where number > 4 group by number%5, number%4, number%3 order by c1,c2,c3 limit 3 +---- +0 0 0 +0 0 1 +0 0 2 + +query II +SELECT number%3 as c1, sum(c1) FROM numbers_mt(10) where number > 2 group by number%3 order by c1 +---- +0 0 +1 2 +2 4 + +query IIII +SELECT a,b,sum(a),sum(b),count() from (SELECT cast((number%6) AS bigint) as a, cast((number%15) AS bigint) as b from numbers(1000)) group by a,b order by a,b limit 3 +---- +0 0 0 0 34 +0 3 0 99 33 +0 6 0 204 34 + +statement ok +DROP TABLE if EXISTS t + +statement ok +CREATE TABLE t(a UInt64 null, b UInt32 null, c UInt32) Engine = Fuse + +statement ok +INSERT INTO t(a,b, c) SELECT if (number % 3 = 2, null, number) as a, number + 5 as b, number + 6 as c FROM numbers(10) + +query II +SELECT a%3 as a1, count(1) as ct from t GROUP BY a1 ORDER BY a1 NULLS FIRST,ct +---- +NULL 3 +0 4 +1 3 + +query III +SELECT a%2 as a1, a%3 as a2, count(0) as ct FROM t GROUP BY a1, a2 ORDER BY a1 NULLS FIRST, a2 NULLS FIRST +---- +NULL NULL 3 +0 0 2 +0 1 1 +1 0 2 +1 1 2 + +query III +SELECT a%2 as a1, to_uint64(c % 3) as c1, count(0) as ct FROM t GROUP BY a1, c1 ORDER BY a1 NULLS FIRST, c1, ct +---- +NULL 2 3 +0 0 2 +0 1 1 +1 0 2 +1 1 2 + +query III +SELECT to_uint64(c % 3) as c1, a%2 as a1, count(0) as ct FROM t GROUP BY a1, c1 ORDER BY a1 NULLS FIRST, c1 NULLS FIRST, ct +---- +2 NULL 3 +0 0 2 +1 0 1 +0 1 2 +1 1 2 + +statement ok +DROP table t + +statement ok +drop table if exists t_datetime all + +statement ok +CREATE TABLE t_datetime(created_at Date, created_time DateTime, count Int32) + +statement ok +insert into t_datetime select to_date('2024-04-01') + number % 3, to_datetime('2024-04-01 00:00:00') + number % 3, 1 from numbers(10) + +query TI +select created_at, sum(count) from t_datetime group by created_at order by created_at +---- +2024-04-01 4 +2024-04-02 3 +2024-04-03 3 + +query TI +select created_time, sum(count) from t_datetime group by created_time order by created_time +---- +2024-04-01 00:00:00.000000 4 +2024-04-01 00:00:00.000001 3 +2024-04-01 00:00:00.000002 3 + +statement ok +drop table t_datetime + +query II +SELECT number, count(*) FROM numbers_mt(10) group by number order by number limit 5 +---- +0 1 +1 1 +2 1 +3 1 +4 1 + +query IIIF +SELECT number % 3 as a, number%4 as b, sum(a),avg(b) FROM numbers_mt(10000000) group by a,b order by a,b limit 5 +---- +0 0 0 0.0 +0 1 0 1.0 +0 2 0 2.0 +0 3 0 3.0 +1 0 833333 0.0 + +statement ok +CREATE TABLE IF NOT EXISTS t_variant(id Int null, var Variant null) Engine = Fuse + +statement ok +INSERT INTO t_variant VALUES(1, parse_json('{"k":"v"}')), (2, parse_json('{"k":"v"}')), (3, parse_json('"abcd"')), (4, parse_json('"abcd"')), (5, parse_json('12')), (6, parse_json('12')), (7, parse_json('[1,2,3]')), (8, parse_json('[1,2,3]')) + +query IIT +SELECT max(id) as n, min(id), var FROM t_variant GROUP BY var ORDER BY n ASC +---- +2 1 {"k":"v"} +4 3 "abcd" +6 5 12 +8 7 [1,2,3] + +statement ok +DROP TABLE t_variant + +statement ok +CREATE TABLE IF NOT EXISTS t_array(id Int null, arr Array(Int32)) Engine = Fuse + +statement ok +INSERT INTO t_array VALUES(1, []), (2, []), (3, [1,2,3]), (4, [1,2,3]), (5, [4,5,6]), (6, [4,5,6]) + +query I +select id % 3 as a from t_array where array_sum(arr) > 1 group by a order by a +---- +0 +1 +2 + +statement ok +DROP TABLE t_array + +query I +select count() from numbers(10) group by 'ab' +---- +10 + +query I +select count() from numbers(10) group by to_nullable('ab') +---- +10 + +## Decimal + +query III +select (number % 3)::Decimal(19, 2) a ,(number % 4)::Decimal(36, 4) b , count() from numbers(100) group by a,b order by a,b limit 5 +---- +0.00 0.0000 9 +0.00 1.0000 8 +0.00 2.0000 8 +0.00 3.0000 9 +1.00 0.0000 8 + +query ITI +select (number % 3)::Decimal(19, 2) c, to_string(number % 3) d, count() from numbers(100) group by c,d order by c, d ; +---- +0.00 0 34 +1.00 1 33 +2.00 2 33 + +query TI +select number::Variant a, count() from numbers(3) group by a order by a +---- +0 1 +1 1 +2 1 + +query TI +select number::Variant a, number as b, count() from numbers(3) group by a, b order by a +---- +0 0 1 +1 1 1 +2 2 1 + +query IIII +select number % 3 a, max(number) - 10, number % 2 b, sum(number) + 10 from numbers(1000000) group by all order by a, b +---- +0 999986 0 83333166676 +0 999989 1 83333666677 +1 999984 0 83332833344 +1 999987 1 83333333343 +2 999988 0 83333500010 +2 999985 1 83333000010 + +query TTT rowsort +SELECT ( null, to_hour(to_timestamp(3501857592331)), number::Date) from numbers(3) group by all +---- +(NULL,18,'1970-01-01') +(NULL,18,'1970-01-02') +(NULL,18,'1970-01-03') + +query TT rowsort +select to_string(to_bitmap(number)), to_string(to_bitmap(number+3)) FROM numbers(3) GROUP BY GROUPING SETS ((to_bitmap(number), to_bitmap(number+3))) +---- +0 3 +1 4 +2 5 + +# group by aggregate functions + +statement error (?s)1065.*GROUP BY items can't contain aggregate functions or window functions +select sum(number + 3 ), number % 3 from numbers(10) group by sum(number + 3 ), number % 3 + +statement error (?s)1065.*GROUP BY items can't contain aggregate functions or window functions +select sum(number + 3 ), number % 3 from numbers(10) group by 1, 2 diff --git a/tests/sqllogictests/suites/base/11_data_type/11_0006_data_type_decimal.test b/tests/sqllogictests/suites/base/11_data_type/11_0006_data_type_decimal.test index 1fcb4ed91d28..c86828ce6dc5 100644 --- a/tests/sqllogictests/suites/base/11_data_type/11_0006_data_type_decimal.test +++ b/tests/sqllogictests/suites/base/11_data_type/11_0006_data_type_decimal.test @@ -259,6 +259,11 @@ SELECT CAST(987654321.34 AS DECIMAL(76, 2)) / CAST(1.23 AS DECIMAL(76, 2)) AS re ---- 802970992.95934959 +query I +SELECT 404.754480000000000000000001 / 563.653044520000000000000001, 404.754480000000000000000000 / 563.653044520000000000000000; +---- +0.718091535094401799683905 0.718091535094401799683905 + ## negative query I @@ -1095,5 +1100,16 @@ select cast(b as int), cast(c as int), cast(d as int) from t -1 -1 -1 -2 -2 -2 +statement ok +create table decimal_test2(a decimal(28,8), b decimal(24,16)); + +statement ok +insert into decimal_test2 values(300.07878791,5325.0000000000000000),(2.00000000,10491.0000000000000000); + +query I +select sum(a * b) / sum(a * b), sum(a + b) / sum(a + b), sum(a - b) / sum(a - b) from decimal_test2; +---- +1.000000000000000000000000 1.0000000000000000 1.0000000000000000 + statement ok drop database decimal_t; diff --git a/tests/sqllogictests/suites/ee/02_ee_aggregating_index/02_0000_async_agg_index_base.test b/tests/sqllogictests/suites/ee/02_ee_aggregating_index/02_0000_async_agg_index_base.test index 6d0769e50543..3398c62e7d9b 100644 --- a/tests/sqllogictests/suites/ee/02_ee_aggregating_index/02_0000_async_agg_index_base.test +++ b/tests/sqllogictests/suites/ee/02_ee_aggregating_index/02_0000_async_agg_index_base.test @@ -386,7 +386,6 @@ EvalScalar ├── aggregate functions: [max(user_id), sum(id), count(id)] ├── estimated rows: 8.00 └── AggregatePartial - ├── output columns: [max(user_id) (#3), sum(id) (#4), count(id) (#5), #_group_by_key] ├── group by: [event_name, user_id] ├── aggregate functions: [max(user_id), sum(id), count(id)] ├── estimated rows: 8.00 @@ -427,7 +426,6 @@ EvalScalar ├── aggregate functions: [max(user_id), sum(id), count(id)] ├── estimated rows: 7.11 └── AggregatePartial - ├── output columns: [max(user_id) (#3), sum(id) (#4), count(id) (#5), #_group_by_key] ├── group by: [event_name, user_id] ├── aggregate functions: [max(user_id), sum(id), count(id)] ├── estimated rows: 7.11 @@ -474,7 +472,6 @@ EvalScalar ├── aggregate functions: [max(user_id), sum(id), count(id)] ├── estimated rows: 7.11 └── AggregatePartial - ├── output columns: [max(user_id) (#3), sum(id) (#4), count(id) (#5), #_group_by_key] ├── group by: [event_name, user_id] ├── aggregate functions: [max(user_id), sum(id), count(id)] ├── estimated rows: 7.11 diff --git a/tests/sqllogictests/suites/mode/cluster/exchange.test b/tests/sqllogictests/suites/mode/cluster/exchange.test index abefb62726df..749e7f77cf6a 100644 --- a/tests/sqllogictests/suites/mode/cluster/exchange.test +++ b/tests/sqllogictests/suites/mode/cluster/exchange.test @@ -167,10 +167,9 @@ Exchange │ ├── aggregate functions: [sum(number)] │ ├── estimated rows: 1.00 │ └── Exchange - │ ├── output columns: [sum(number) (#2), #_group_by_key] - │ ├── exchange type: Hash(_group_by_key) + │ ├── output columns: [sum(number) (#2), numbers.number (#0)] + │ ├── exchange type: Hash(0) │ └── AggregatePartial - │ ├── output columns: [sum(number) (#2), #_group_by_key] │ ├── group by: [number] │ ├── aggregate functions: [sum(number)] │ ├── estimated rows: 1.00 @@ -199,10 +198,9 @@ explain fragments select * from (select sum(number) as number from numbers(1) gr Fragment 0: DataExchange: Shuffle ExchangeSink - ├── output columns: [sum(number) (#2), #_group_by_key] + ├── output columns: [sum(number) (#2), numbers.number (#0)] ├── destination fragment: [1] └── AggregatePartial - ├── output columns: [sum(number) (#2), #_group_by_key] ├── group by: [number] ├── aggregate functions: [sum(number)] ├── estimated rows: 1.00 @@ -228,7 +226,7 @@ Fragment 1: ├── aggregate functions: [sum(number)] ├── estimated rows: 1.00 └── ExchangeSource - ├── output columns: [sum(number) (#2), #_group_by_key] + ├── output columns: [sum(number) (#2), numbers.number (#0)] └── source fragment: [0] (empty) (empty) @@ -285,7 +283,6 @@ AggregateFinal ├── output columns: [sum(a) (#4)] ├── exchange type: Merge └── AggregatePartial - ├── output columns: [sum(a) (#4)] ├── group by: [] ├── aggregate functions: [sum(number)] ├── estimated rows: 1.00 @@ -368,7 +365,6 @@ AggregateFinal ├── aggregate functions: [sum(a)] ├── estimated rows: 1.00 └── AggregatePartial - ├── output columns: [sum(a) (#6)] ├── group by: [] ├── aggregate functions: [sum(a)] ├── estimated rows: 1.00 @@ -419,7 +415,6 @@ AggregateFinal ├── output columns: [sum(t1.a) (#4)] ├── exchange type: Merge └── AggregatePartial - ├── output columns: [sum(t1.a) (#4)] ├── group by: [] ├── aggregate functions: [sum(number)] ├── estimated rows: 1.00 @@ -473,7 +468,6 @@ AggregateFinal ├── output columns: [sum(a) (#3)] ├── exchange type: Merge └── AggregatePartial - ├── output columns: [sum(a) (#3)] ├── group by: [] ├── aggregate functions: [sum(number)] ├── estimated rows: 1.00 diff --git a/tests/sqllogictests/suites/mode/cluster/explain_v2.test b/tests/sqllogictests/suites/mode/cluster/explain_v2.test index 899706f2e0c0..360022aaee7b 100644 --- a/tests/sqllogictests/suites/mode/cluster/explain_v2.test +++ b/tests/sqllogictests/suites/mode/cluster/explain_v2.test @@ -184,7 +184,6 @@ Limit ├── output columns: [count(1) (#2), max(a) (#4)] ├── exchange type: Merge └── AggregatePartial - ├── output columns: [count(1) (#2), max(a) (#4)] ├── group by: [] ├── aggregate functions: [count(), max(a)] ├── estimated rows: 1.00 @@ -337,7 +336,6 @@ AggregateFinal ├── output columns: [sum(a) (#2)] ├── exchange type: Merge └── AggregatePartial - ├── output columns: [sum(a) (#2)] ├── group by: [] ├── aggregate functions: [sum(number)] ├── estimated rows: 1.00 diff --git a/tests/sqllogictests/suites/mode/cluster/group_shuffle.test b/tests/sqllogictests/suites/mode/cluster/group_shuffle.test index ab6149d1b812..656b56ec9208 100644 --- a/tests/sqllogictests/suites/mode/cluster/group_shuffle.test +++ b/tests/sqllogictests/suites/mode/cluster/group_shuffle.test @@ -13,7 +13,6 @@ Exchange ├── aggregate functions: [] ├── estimated rows: 100000.00 └── AggregatePartial - ├── output columns: [#_group_by_key] ├── group by: [number] ├── aggregate functions: [] ├── estimated rows: 100000.00 @@ -46,10 +45,9 @@ Exchange ├── aggregate functions: [] ├── estimated rows: 100000.00 └── Exchange - ├── output columns: [#_group_by_key] - ├── exchange type: Hash(_group_by_key) + ├── output columns: [numbers_mt.number (#0)] + ├── exchange type: Hash(0) └── AggregatePartial - ├── output columns: [#_group_by_key] ├── group by: [number] ├── aggregate functions: [] ├── estimated rows: 100000.00 diff --git a/tests/sqllogictests/suites/mode/standalone/explain/aggregate.test b/tests/sqllogictests/suites/mode/standalone/explain/aggregate.test index 41578ca3c461..392bd1a6c12a 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/aggregate.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/aggregate.test @@ -7,7 +7,6 @@ AggregateFinal ├── aggregate functions: [] ├── estimated rows: 10.00 └── AggregatePartial - ├── output columns: [#_group_by_key] ├── group by: [number] ├── aggregate functions: [] ├── estimated rows: 10.00 @@ -31,7 +30,6 @@ AggregateFinal ├── aggregate functions: [] ├── estimated rows: 10.00 └── AggregatePartial - ├── output columns: [#_group_by_key] ├── group by: [number] ├── aggregate functions: [] ├── estimated rows: 10.00 @@ -78,7 +76,6 @@ EvalScalar ├── aggregate functions: [] ├── estimated rows: 10.00 └── AggregatePartial - ├── output columns: [#_group_by_key] ├── group by: [number] ├── aggregate functions: [] ├── estimated rows: 10.00 @@ -106,7 +103,6 @@ EvalScalar ├── aggregate functions: [count()] ├── estimated rows: 0.00 └── AggregatePartial - ├── output columns: [count(3) (#13), #_group_by_key] ├── group by: [name, type] ├── aggregate functions: [count()] ├── estimated rows: 0.00 @@ -133,7 +129,6 @@ AggregateFinal ├── aggregate functions: [] ├── estimated rows: 0.00 └── AggregatePartial - ├── output columns: [#_group_by_key] ├── group by: [a] ├── aggregate functions: [] ├── estimated rows: 0.00 @@ -160,7 +155,6 @@ AggregateFinal ├── aggregate functions: [] ├── estimated rows: 0.00 └── AggregatePartial - ├── output columns: [#_group_by_key] ├── group by: [a] ├── aggregate functions: [] ├── estimated rows: 0.00 @@ -187,7 +181,6 @@ AggregateFinal ├── aggregate functions: [max(b)] ├── estimated rows: 0.00 └── AggregatePartial - ├── output columns: [max(b) (#2), #_group_by_key] ├── group by: [a] ├── aggregate functions: [max(b)] ├── estimated rows: 0.00 @@ -222,7 +215,6 @@ EvalScalar ├── aggregate functions: [sum(b), count(), max(b)] ├── estimated rows: 0.00 └── AggregatePartial - ├── output columns: [sum(b) (#2), count(b) (#3), max(b) (#5), #_group_by_key] ├── group by: [a] ├── aggregate functions: [sum(b), count(), max(b)] ├── estimated rows: 0.00 @@ -261,7 +253,6 @@ EvalScalar ├── aggregate functions: [sum(b), count()] ├── estimated rows: 0.00 └── AggregatePartial - ├── output columns: [sum(b) (#2), count(b) (#3), #_group_by_key] ├── group by: [a] ├── aggregate functions: [sum(b), count()] ├── estimated rows: 0.00 @@ -297,7 +288,6 @@ EvalScalar ├── aggregate functions: [sum(b), count()] ├── estimated rows: 0.00 └── AggregatePartial - ├── output columns: [sum(b) (#2), count(b) (#3), #_group_by_key] ├── group by: [a] ├── aggregate functions: [sum(b), count()] ├── estimated rows: 0.00 @@ -332,7 +322,6 @@ EvalScalar ├── aggregate functions: [sum(b), count()] ├── estimated rows: 0.00 └── AggregatePartial - ├── output columns: [sum(b) (#2), count(b) (#3), #_group_by_key] ├── group by: [a] ├── aggregate functions: [sum(b), count()] ├── estimated rows: 0.00 @@ -367,7 +356,6 @@ EvalScalar ├── aggregate functions: [sum(b), count()] ├── estimated rows: 0.00 └── AggregatePartial - ├── output columns: [sum(b) (#2), count(b) (#3), #_group_by_key] ├── group by: [a] ├── aggregate functions: [sum(b), count()] ├── estimated rows: 0.00 @@ -402,7 +390,6 @@ EvalScalar ├── aggregate functions: [sum(b), count()] ├── estimated rows: 0.00 └── AggregatePartial - ├── output columns: [sum(b) (#2), count(b) (#3), #_group_by_key] ├── group by: [a] ├── aggregate functions: [sum(b), count()] ├── estimated rows: 0.00 @@ -440,7 +427,6 @@ AggregateFinal ├── aggregate functions: [count()] ├── estimated rows: 1.00 └── AggregatePartial - ├── output columns: [count() (#2)] ├── group by: [] ├── aggregate functions: [count()] ├── estimated rows: 1.00 diff --git a/tests/sqllogictests/suites/mode/standalone/explain/explain.test b/tests/sqllogictests/suites/mode/standalone/explain/explain.test index 8bbd4b423df2..49361ac46e0b 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/explain.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/explain.test @@ -864,7 +864,7 @@ Limit ├── push downs: [filters: [t2.a (#2) > 2 OR t2.b (#3) < 4], limit: NONE] └── estimated rows: 5.00 -query +query explain select * from t1,t2 where (t1.a > 1 or t1.b < 2) and (t1.a > 1 or t1.b < 2) ---- HashJoin @@ -899,7 +899,7 @@ HashJoin ├── push downs: [filters: [], limit: NONE] └── estimated rows: 5.00 -query +query explain select count(distinct a) from t1; ---- AggregateFinal @@ -908,7 +908,6 @@ AggregateFinal ├── aggregate functions: [count()] ├── estimated rows: 1.00 └── AggregatePartial - ├── output columns: [count(_1) (#2)] ├── group by: [] ├── aggregate functions: [count()] ├── estimated rows: 1.00 @@ -918,7 +917,6 @@ AggregateFinal ├── aggregate functions: [] ├── estimated rows: 1.00 └── AggregatePartial - ├── output columns: [#_group_by_key] ├── group by: [a] ├── aggregate functions: [] ├── estimated rows: 1.00 @@ -933,7 +931,7 @@ AggregateFinal ├── push downs: [filters: [], limit: NONE] └── estimated rows: 1.00 -query +query explain select count_distinct(a) from t1; ---- AggregateFinal @@ -942,7 +940,6 @@ AggregateFinal ├── aggregate functions: [count()] ├── estimated rows: 1.00 └── AggregatePartial - ├── output columns: [count(_1) (#2)] ├── group by: [] ├── aggregate functions: [count()] ├── estimated rows: 1.00 @@ -952,7 +949,6 @@ AggregateFinal ├── aggregate functions: [] ├── estimated rows: 1.00 └── AggregatePartial - ├── output columns: [#_group_by_key] ├── group by: [a] ├── aggregate functions: [] ├── estimated rows: 1.00 @@ -967,7 +963,7 @@ AggregateFinal ├── push downs: [filters: [], limit: NONE] └── estimated rows: 1.00 -query +query explain select * from (values(1, 'a'),(2, 'b')) t(c1,c2) ---- ConstantTableScan @@ -981,28 +977,28 @@ drop table t1 statement ok drop table t2 -query +query explain syntax select * from read_parquet('p1', 'p2', 'p3'); ---- SELECT * FROM read_parquet('p1', 'p2', 'p3') -query +query explain syntax select * from read_parquet(prune_page=>true, refresh_meta_cache=>true); ---- SELECT * FROM read_parquet(prune_page=>TRUE, refresh_meta_cache=>TRUE) -query +query explain syntax select * from read_parquet('p1', 'p2', 'p3', prune_page=>true, refresh_meta_cache=>true); ---- SELECT * FROM read_parquet('p1', 'p2', 'p3', prune_page=>TRUE, refresh_meta_cache=>TRUE) -query +query explain ast select * from read_parquet('p1', 'p2', 'p3', prune_page=>true, refresh_meta_cache=>true); ---- Query (children 1) @@ -1024,7 +1020,7 @@ drop table if exists t4 statement ok create table t4(a int, b string); -query +query explain select * from t4 where a = 1 and try_cast(get(try_parse_json(b),'bb') as varchar) = 'xx'; ---- Filter @@ -1047,7 +1043,7 @@ drop view if exists v4 statement ok create view v4 as select a as a, try_cast(get(try_parse_json(b), 'bb') as varchar) as b from t4; -query +query explain select * from v4 where b = 'xx'; ---- EvalScalar @@ -1311,7 +1307,6 @@ HashJoin ├── aggregate functions: [] ├── estimated rows: 1301.00 └── AggregatePartial - ├── output columns: [#_group_by_key] ├── group by: [col0] ├── aggregate functions: [] ├── estimated rows: 1301.00 @@ -1349,7 +1344,6 @@ Filter ├── aggregate functions: [] ├── estimated rows: 1301.00 └── AggregatePartial - ├── output columns: [#_group_by_key] ├── group by: [col0] ├── aggregate functions: [] ├── estimated rows: 1301.00 @@ -1623,7 +1617,6 @@ Filter │ ├── aggregate functions: [count(a)] │ ├── estimated rows: 0.00 │ └── AggregatePartial - │ ├── output columns: [count(a) (#6), #_group_by_key] │ ├── group by: [c, a] │ ├── aggregate functions: [count(a)] │ ├── estimated rows: 0.00 @@ -1678,7 +1671,6 @@ Filter │ ├── aggregate functions: [count()] │ ├── estimated rows: 0.00 │ └── AggregatePartial - │ ├── output columns: [COUNT(*) (#6), #_group_by_key] │ ├── group by: [c, a] │ ├── aggregate functions: [count()] │ ├── estimated rows: 0.00 diff --git a/tests/sqllogictests/suites/mode/standalone/explain/explain_grouping_sets.test b/tests/sqllogictests/suites/mode/standalone/explain/explain_grouping_sets.test index e378af9d4f79..bd0b11ab01db 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/explain_grouping_sets.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/explain_grouping_sets.test @@ -11,7 +11,6 @@ EvalScalar ├── aggregate functions: [] ├── estimated rows: 1.00 └── AggregatePartial - ├── output columns: [#_group_by_key] ├── group by: [a, b, c, _grouping_id] ├── aggregate functions: [] ├── estimated rows: 1.00 @@ -46,7 +45,6 @@ EvalScalar ├── aggregate functions: [] ├── estimated rows: 1.00 └── AggregatePartial - ├── output columns: [#_group_by_key] ├── group by: [a, b, c, _grouping_id] ├── aggregate functions: [] ├── estimated rows: 1.00 diff --git a/tests/sqllogictests/suites/mode/standalone/explain/fold_count.test b/tests/sqllogictests/suites/mode/standalone/explain/fold_count.test index 8d1b381b5f29..018815b67b33 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/fold_count.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/fold_count.test @@ -34,7 +34,6 @@ AggregateFinal ├── aggregate functions: [count()] ├── estimated rows: 1.00 └── AggregatePartial - ├── output columns: [COUNT(*) (#1)] ├── group by: [] ├── aggregate functions: [count()] ├── estimated rows: 1.00 @@ -62,7 +61,6 @@ AggregateFinal ├── aggregate functions: [count()] ├── estimated rows: 1000.00 └── AggregatePartial - ├── output columns: [COUNT(*) (#1), #_group_by_key] ├── group by: [number] ├── aggregate functions: [count()] ├── estimated rows: 1000.00 diff --git a/tests/sqllogictests/suites/mode/standalone/explain/join_reorder/mark.test b/tests/sqllogictests/suites/mode/standalone/explain/join_reorder/mark.test index 8740e9732bf1..956a08f73369 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/join_reorder/mark.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/join_reorder/mark.test @@ -115,7 +115,6 @@ EvalScalar │ │ │ ├── aggregate functions: [] │ │ │ ├── estimated rows: 0.00 │ │ │ └── AggregatePartial - │ │ │ ├── output columns: [#_group_by_key] │ │ │ ├── group by: [c] │ │ │ ├── aggregate functions: [] │ │ │ ├── estimated rows: 0.00 diff --git a/tests/sqllogictests/suites/mode/standalone/explain/lateral.test b/tests/sqllogictests/suites/mode/standalone/explain/lateral.test index e4466bada019..9232f46356f7 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/lateral.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/lateral.test @@ -54,7 +54,6 @@ HashJoin │ │ ├── aggregate functions: [] │ │ ├── estimated rows: 0.00 │ │ └── AggregatePartial -│ │ ├── output columns: [#_group_by_key] │ │ ├── group by: [number] │ │ ├── aggregate functions: [] │ │ ├── estimated rows: 0.00 @@ -113,7 +112,6 @@ HashJoin │ │ ├── aggregate functions: [] │ │ ├── estimated rows: 0.00 │ │ └── AggregatePartial -│ │ ├── output columns: [#_group_by_key] │ │ ├── group by: [number] │ │ ├── aggregate functions: [] │ │ ├── estimated rows: 0.00 diff --git a/tests/sqllogictests/suites/mode/standalone/explain/limit.test b/tests/sqllogictests/suites/mode/standalone/explain/limit.test index c33a8410dee6..19b22ed6f9d0 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/limit.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/limit.test @@ -91,7 +91,6 @@ Limit ├── aggregate functions: [] ├── estimated rows: 0.20 └── AggregatePartial - ├── output columns: [#_group_by_key] ├── group by: [number] ├── aggregate functions: [] ├── estimated rows: 0.20 @@ -112,7 +111,6 @@ Limit │ ├── aggregate functions: [count()] │ ├── estimated rows: 1.00 │ └── AggregatePartial - │ ├── output columns: [COUNT(*) (#4), #_group_by_key] │ ├── group by: [number] │ ├── aggregate functions: [count()] │ ├── estimated rows: 1.00 @@ -192,7 +190,6 @@ Limit │ ├── aggregate functions: [count()] │ ├── estimated rows: 1.00 │ └── AggregatePartial - │ ├── output columns: [count(t1.number) (#2), #_group_by_key] │ ├── group by: [number] │ ├── aggregate functions: [count()] │ ├── estimated rows: 1.00 @@ -211,7 +208,6 @@ Limit ├── aggregate functions: [count()] ├── estimated rows: 2.00 └── AggregatePartial - ├── output columns: [count(t.number) (#5), #_group_by_key] ├── group by: [number] ├── aggregate functions: [count()] ├── estimated rows: 2.00 diff --git a/tests/sqllogictests/suites/mode/standalone/explain/project_set.test b/tests/sqllogictests/suites/mode/standalone/explain/project_set.test index 068e16a862ba..18507110bf27 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/project_set.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/project_set.test @@ -19,7 +19,6 @@ AggregateFinal ├── aggregate functions: [count()] ├── estimated rows: 1.00 └── AggregatePartial - ├── output columns: [count() (#4)] ├── group by: [] ├── aggregate functions: [count()] ├── estimated rows: 1.00 diff --git a/tests/sqllogictests/suites/mode/standalone/explain/prune_column.test b/tests/sqllogictests/suites/mode/standalone/explain/prune_column.test index d67f409f36af..b1b88ea4b243 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/prune_column.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/prune_column.test @@ -20,7 +20,6 @@ AggregateFinal ├── aggregate functions: [] ├── estimated rows: 1.00 └── AggregatePartial - ├── output columns: [#_group_by_key] ├── group by: [number] ├── aggregate functions: [] ├── estimated rows: 1.00 @@ -52,7 +51,6 @@ Limit ├── aggregate functions: [] ├── estimated rows: 0.00 └── AggregatePartial - ├── output columns: [#_group_by_key] ├── group by: [number, number, number, number] ├── aggregate functions: [] ├── estimated rows: 0.00 @@ -137,7 +135,6 @@ HashJoin │ ├── limit: 1 │ ├── estimated rows: 1.00 │ └── AggregatePartial -│ ├── output columns: [_count_scalar_subquery (#13), _any_scalar_subquery (#14)] │ ├── group by: [] │ ├── aggregate functions: [count(), any(COUNT(*))] │ ├── estimated rows: 1.00 @@ -147,7 +144,6 @@ HashJoin │ ├── aggregate functions: [count()] │ ├── estimated rows: 1.00 │ └── AggregatePartial -│ ├── output columns: [COUNT(*) (#12)] │ ├── group by: [] │ ├── aggregate functions: [count()] │ ├── estimated rows: 1.00 @@ -240,7 +236,6 @@ HashJoin │ ├── aggregate functions: [count()] │ ├── estimated rows: 1.00 │ └── AggregatePartial -│ ├── output columns: [count(*) (#2)] │ ├── group by: [] │ ├── aggregate functions: [count()] │ ├── estimated rows: 1.00 @@ -287,7 +282,6 @@ AggregateFinal ├── aggregate functions: [count()] ├── estimated rows: 1.00 └── AggregatePartial - ├── output columns: [COUNT(*) (#2)] ├── group by: [] ├── aggregate functions: [count()] ├── estimated rows: 1.00 diff --git a/tests/sqllogictests/suites/mode/standalone/explain/push_down_filter/push_down_filter_eval_scalar.test b/tests/sqllogictests/suites/mode/standalone/explain/push_down_filter/push_down_filter_eval_scalar.test index 43a459a500c7..0a05959af25d 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/push_down_filter/push_down_filter_eval_scalar.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/push_down_filter/push_down_filter_eval_scalar.test @@ -57,7 +57,6 @@ AggregateFinal ├── aggregate functions: [] ├── estimated rows: 0.00 └── AggregatePartial - ├── output columns: [#_group_by_key] ├── group by: [id] ├── aggregate functions: [] ├── estimated rows: 0.00 @@ -74,7 +73,6 @@ AggregateFinal │ ├── aggregate functions: [sum(sum(coalesce(t3.val, 0)))] │ ├── estimated rows: 0.00 │ └── AggregatePartial - │ ├── output columns: [sum(tb.de) (#7), #_group_by_key] │ ├── group by: [id] │ ├── aggregate functions: [sum(sum(coalesce(t3.val, 0)))] │ ├── estimated rows: 0.00 @@ -91,7 +89,6 @@ AggregateFinal │ │ ├── aggregate functions: [sum(sum_arg_0)] │ │ ├── estimated rows: 0.00 │ │ └── AggregatePartial - │ │ ├── output columns: [sum(coalesce(t3.val, 0)) (#5), #_group_by_key] │ │ ├── group by: [sid] │ │ ├── aggregate functions: [sum(sum_arg_0)] │ │ ├── estimated rows: 0.00 @@ -139,7 +136,6 @@ AggregateFinal ├── aggregate functions: [] ├── estimated rows: 0.00 └── AggregatePartial - ├── output columns: [#_group_by_key] ├── group by: [id] ├── aggregate functions: [] ├── estimated rows: 0.00 diff --git a/tests/sqllogictests/suites/mode/standalone/explain/subquery.test b/tests/sqllogictests/suites/mode/standalone/explain/subquery.test index 7632039b318a..c35a936a1714 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/subquery.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/subquery.test @@ -18,7 +18,6 @@ Filter │ ├── aggregate functions: [count()] │ ├── estimated rows: 1.00 │ └── AggregatePartial - │ ├── output columns: [COUNT(*) (#4), #_group_by_key] │ ├── group by: [number] │ ├── aggregate functions: [count()] │ ├── estimated rows: 1.00 @@ -126,7 +125,6 @@ HashJoin │ ├── aggregate functions: [count()] │ ├── estimated rows: 1.00 │ └── AggregatePartial -│ ├── output columns: [count(*) (#2)] │ ├── group by: [] │ ├── aggregate functions: [count()] │ ├── estimated rows: 1.00 @@ -184,7 +182,6 @@ HashJoin │ ├── limit: 1 │ ├── estimated rows: 1.00 │ └── AggregatePartial -│ ├── output columns: [_count_scalar_subquery (#2), _any_scalar_subquery (#3)] │ ├── group by: [] │ ├── aggregate functions: [count(), any(number)] │ ├── estimated rows: 1.00 @@ -413,7 +410,6 @@ HashJoin │ ├── aggregate functions: [count()] │ ├── estimated rows: 1.00 │ └── AggregatePartial -│ ├── output columns: [COUNT(*) (#3), #_group_by_key] │ ├── group by: [number] │ ├── aggregate functions: [count()] │ ├── estimated rows: 1.00 @@ -547,7 +543,6 @@ EvalScalar │ ├── aggregate functions: [count()] │ ├── estimated rows: 1.00 │ └── AggregatePartial - │ ├── output columns: [count(*) (#3)] │ ├── group by: [] │ ├── aggregate functions: [count()] │ ├── estimated rows: 1.00 diff --git a/tests/sqllogictests/suites/mode/standalone/explain_native/aggregate.test b/tests/sqllogictests/suites/mode/standalone/explain_native/aggregate.test index 6f66bf98afec..7606c8f7bccc 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain_native/aggregate.test +++ b/tests/sqllogictests/suites/mode/standalone/explain_native/aggregate.test @@ -7,7 +7,6 @@ AggregateFinal ├── aggregate functions: [] ├── estimated rows: 10.00 └── AggregatePartial - ├── output columns: [#_group_by_key] ├── group by: [number] ├── aggregate functions: [] ├── estimated rows: 10.00 @@ -31,7 +30,6 @@ AggregateFinal ├── aggregate functions: [] ├── estimated rows: 10.00 └── AggregatePartial - ├── output columns: [#_group_by_key] ├── group by: [number] ├── aggregate functions: [] ├── estimated rows: 10.00 @@ -78,7 +76,6 @@ EvalScalar ├── aggregate functions: [] ├── estimated rows: 10.00 └── AggregatePartial - ├── output columns: [#_group_by_key] ├── group by: [number] ├── aggregate functions: [] ├── estimated rows: 10.00 @@ -106,7 +103,6 @@ EvalScalar ├── aggregate functions: [count()] ├── estimated rows: 0.00 └── AggregatePartial - ├── output columns: [count(3) (#13), #_group_by_key] ├── group by: [name, type] ├── aggregate functions: [count()] ├── estimated rows: 0.00 @@ -133,7 +129,6 @@ AggregateFinal ├── aggregate functions: [] ├── estimated rows: 0.00 └── AggregatePartial - ├── output columns: [#_group_by_key] ├── group by: [a] ├── aggregate functions: [] ├── estimated rows: 0.00 @@ -156,7 +151,6 @@ AggregateFinal ├── aggregate functions: [] ├── estimated rows: 0.00 └── AggregatePartial - ├── output columns: [#_group_by_key] ├── group by: [a] ├── aggregate functions: [] ├── estimated rows: 0.00 @@ -179,7 +173,6 @@ AggregateFinal ├── aggregate functions: [max(b)] ├── estimated rows: 0.00 └── AggregatePartial - ├── output columns: [max(b) (#2), #_group_by_key] ├── group by: [a] ├── aggregate functions: [max(b)] ├── estimated rows: 0.00 @@ -210,7 +203,6 @@ EvalScalar ├── aggregate functions: [sum(b), count(), max(b)] ├── estimated rows: 0.00 └── AggregatePartial - ├── output columns: [sum(b) (#2), count(b) (#3), max(b) (#5), #_group_by_key] ├── group by: [a] ├── aggregate functions: [sum(b), count(), max(b)] ├── estimated rows: 0.00 @@ -241,7 +233,6 @@ EvalScalar ├── aggregate functions: [sum(b), count(), max(b)] ├── estimated rows: 0.00 └── AggregatePartial - ├── output columns: [sum(b) (#2), count(b) (#3), max(b) (#5), #_group_by_key] ├── group by: [a] ├── aggregate functions: [sum(b), count(), max(b)] ├── estimated rows: 0.00 @@ -279,7 +270,6 @@ AggregateFinal ├── aggregate functions: [count()] ├── estimated rows: 1.00 └── AggregatePartial - ├── output columns: [count() (#2)] ├── group by: [] ├── aggregate functions: [count()] ├── estimated rows: 1.00 diff --git a/tests/sqllogictests/suites/mode/standalone/explain_native/explain.test b/tests/sqllogictests/suites/mode/standalone/explain_native/explain.test index 1e366ae86672..c1461e75fa94 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain_native/explain.test +++ b/tests/sqllogictests/suites/mode/standalone/explain_native/explain.test @@ -860,7 +860,6 @@ AggregateFinal ├── aggregate functions: [count()] ├── estimated rows: 1.00 └── AggregatePartial - ├── output columns: [count(_1) (#2)] ├── group by: [] ├── aggregate functions: [count()] ├── estimated rows: 1.00 @@ -870,7 +869,6 @@ AggregateFinal ├── aggregate functions: [] ├── estimated rows: 1.00 └── AggregatePartial - ├── output columns: [#_group_by_key] ├── group by: [a] ├── aggregate functions: [] ├── estimated rows: 1.00 @@ -894,7 +892,6 @@ AggregateFinal ├── aggregate functions: [count()] ├── estimated rows: 1.00 └── AggregatePartial - ├── output columns: [count(_1) (#2)] ├── group by: [] ├── aggregate functions: [count()] ├── estimated rows: 1.00 @@ -904,7 +901,6 @@ AggregateFinal ├── aggregate functions: [] ├── estimated rows: 1.00 └── AggregatePartial - ├── output columns: [#_group_by_key] ├── group by: [a] ├── aggregate functions: [] ├── estimated rows: 1.00 @@ -919,7 +915,7 @@ AggregateFinal ├── push downs: [filters: [], limit: NONE] └── estimated rows: 1.00 -query +query explain select * from (values(1, 'a'),(2, 'b')) t(c1,c2) ---- ConstantTableScan diff --git a/tests/sqllogictests/suites/mode/standalone/explain_native/explain_grouping_sets.test b/tests/sqllogictests/suites/mode/standalone/explain_native/explain_grouping_sets.test index e378af9d4f79..bd0b11ab01db 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain_native/explain_grouping_sets.test +++ b/tests/sqllogictests/suites/mode/standalone/explain_native/explain_grouping_sets.test @@ -11,7 +11,6 @@ EvalScalar ├── aggregate functions: [] ├── estimated rows: 1.00 └── AggregatePartial - ├── output columns: [#_group_by_key] ├── group by: [a, b, c, _grouping_id] ├── aggregate functions: [] ├── estimated rows: 1.00 @@ -46,7 +45,6 @@ EvalScalar ├── aggregate functions: [] ├── estimated rows: 1.00 └── AggregatePartial - ├── output columns: [#_group_by_key] ├── group by: [a, b, c, _grouping_id] ├── aggregate functions: [] ├── estimated rows: 1.00 diff --git a/tests/sqllogictests/suites/mode/standalone/explain_native/fold_count.test b/tests/sqllogictests/suites/mode/standalone/explain_native/fold_count.test index ed75e0538f98..d321ac50bad9 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain_native/fold_count.test +++ b/tests/sqllogictests/suites/mode/standalone/explain_native/fold_count.test @@ -34,7 +34,6 @@ AggregateFinal ├── aggregate functions: [count()] ├── estimated rows: 1.00 └── AggregatePartial - ├── output columns: [COUNT(*) (#1)] ├── group by: [] ├── aggregate functions: [count()] ├── estimated rows: 1.00 diff --git a/tests/sqllogictests/suites/mode/standalone/explain_native/limit.test b/tests/sqllogictests/suites/mode/standalone/explain_native/limit.test index c33a8410dee6..19b22ed6f9d0 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain_native/limit.test +++ b/tests/sqllogictests/suites/mode/standalone/explain_native/limit.test @@ -91,7 +91,6 @@ Limit ├── aggregate functions: [] ├── estimated rows: 0.20 └── AggregatePartial - ├── output columns: [#_group_by_key] ├── group by: [number] ├── aggregate functions: [] ├── estimated rows: 0.20 @@ -112,7 +111,6 @@ Limit │ ├── aggregate functions: [count()] │ ├── estimated rows: 1.00 │ └── AggregatePartial - │ ├── output columns: [COUNT(*) (#4), #_group_by_key] │ ├── group by: [number] │ ├── aggregate functions: [count()] │ ├── estimated rows: 1.00 @@ -192,7 +190,6 @@ Limit │ ├── aggregate functions: [count()] │ ├── estimated rows: 1.00 │ └── AggregatePartial - │ ├── output columns: [count(t1.number) (#2), #_group_by_key] │ ├── group by: [number] │ ├── aggregate functions: [count()] │ ├── estimated rows: 1.00 @@ -211,7 +208,6 @@ Limit ├── aggregate functions: [count()] ├── estimated rows: 2.00 └── AggregatePartial - ├── output columns: [count(t.number) (#5), #_group_by_key] ├── group by: [number] ├── aggregate functions: [count()] ├── estimated rows: 2.00 diff --git a/tests/sqllogictests/suites/mode/standalone/explain_native/project_set.test b/tests/sqllogictests/suites/mode/standalone/explain_native/project_set.test index 59de879ece6a..1cdf3e351e22 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain_native/project_set.test +++ b/tests/sqllogictests/suites/mode/standalone/explain_native/project_set.test @@ -19,7 +19,6 @@ AggregateFinal ├── aggregate functions: [count()] ├── estimated rows: 1.00 └── AggregatePartial - ├── output columns: [count() (#4)] ├── group by: [] ├── aggregate functions: [count()] ├── estimated rows: 1.00 diff --git a/tests/sqllogictests/suites/mode/standalone/explain_native/prune_column.test b/tests/sqllogictests/suites/mode/standalone/explain_native/prune_column.test index 9c6f8a75331f..16d86f308eb4 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain_native/prune_column.test +++ b/tests/sqllogictests/suites/mode/standalone/explain_native/prune_column.test @@ -20,7 +20,6 @@ AggregateFinal ├── aggregate functions: [] ├── estimated rows: 1.00 └── AggregatePartial - ├── output columns: [#_group_by_key] ├── group by: [number] ├── aggregate functions: [] ├── estimated rows: 1.00 @@ -52,7 +51,6 @@ Limit ├── aggregate functions: [] ├── estimated rows: 0.00 └── AggregatePartial - ├── output columns: [#_group_by_key] ├── group by: [number, number, number, number] ├── aggregate functions: [] ├── estimated rows: 0.00 @@ -137,7 +135,6 @@ HashJoin │ ├── limit: 1 │ ├── estimated rows: 1.00 │ └── AggregatePartial -│ ├── output columns: [_count_scalar_subquery (#13), _any_scalar_subquery (#14)] │ ├── group by: [] │ ├── aggregate functions: [count(), any(COUNT(*))] │ ├── estimated rows: 1.00 @@ -147,7 +144,6 @@ HashJoin │ ├── aggregate functions: [count()] │ ├── estimated rows: 1.00 │ └── AggregatePartial -│ ├── output columns: [COUNT(*) (#12)] │ ├── group by: [] │ ├── aggregate functions: [count()] │ ├── estimated rows: 1.00 @@ -239,7 +235,6 @@ HashJoin │ ├── aggregate functions: [count()] │ ├── estimated rows: 1.00 │ └── AggregatePartial -│ ├── output columns: [count(*) (#2)] │ ├── group by: [] │ ├── aggregate functions: [count()] │ ├── estimated rows: 1.00 @@ -286,7 +281,6 @@ AggregateFinal ├── aggregate functions: [count()] ├── estimated rows: 1.00 └── AggregatePartial - ├── output columns: [COUNT(*) (#2)] ├── group by: [] ├── aggregate functions: [count()] ├── estimated rows: 1.00 diff --git a/tests/sqllogictests/suites/mode/standalone/explain_native/push_down_filter/push_down_filter_eval_scalar.test b/tests/sqllogictests/suites/mode/standalone/explain_native/push_down_filter/push_down_filter_eval_scalar.test index 8aff143dfa86..adaadb448191 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain_native/push_down_filter/push_down_filter_eval_scalar.test +++ b/tests/sqllogictests/suites/mode/standalone/explain_native/push_down_filter/push_down_filter_eval_scalar.test @@ -57,7 +57,6 @@ AggregateFinal ├── aggregate functions: [] ├── estimated rows: 0.00 └── AggregatePartial - ├── output columns: [#_group_by_key] ├── group by: [id] ├── aggregate functions: [] ├── estimated rows: 0.00 @@ -74,7 +73,6 @@ AggregateFinal │ ├── aggregate functions: [sum(sum(coalesce(t3.val, 0)))] │ ├── estimated rows: 0.00 │ └── AggregatePartial - │ ├── output columns: [sum(tb.de) (#7), #_group_by_key] │ ├── group by: [id] │ ├── aggregate functions: [sum(sum(coalesce(t3.val, 0)))] │ ├── estimated rows: 0.00 @@ -91,7 +89,6 @@ AggregateFinal │ │ ├── aggregate functions: [sum(sum_arg_0)] │ │ ├── estimated rows: 0.00 │ │ └── AggregatePartial - │ │ ├── output columns: [sum(coalesce(t3.val, 0)) (#5), #_group_by_key] │ │ ├── group by: [sid] │ │ ├── aggregate functions: [sum(sum_arg_0)] │ │ ├── estimated rows: 0.00 @@ -131,7 +128,6 @@ AggregateFinal ├── aggregate functions: [] ├── estimated rows: 0.00 └── AggregatePartial - ├── output columns: [#_group_by_key] ├── group by: [id] ├── aggregate functions: [] ├── estimated rows: 0.00 diff --git a/tests/sqllogictests/suites/mode/standalone/explain_native/subquery.test b/tests/sqllogictests/suites/mode/standalone/explain_native/subquery.test index 58726b387909..ae95f5e8cc3e 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain_native/subquery.test +++ b/tests/sqllogictests/suites/mode/standalone/explain_native/subquery.test @@ -18,7 +18,6 @@ Filter │ ├── aggregate functions: [count()] │ ├── estimated rows: 1.00 │ └── AggregatePartial - │ ├── output columns: [COUNT(*) (#4), #_group_by_key] │ ├── group by: [number] │ ├── aggregate functions: [count()] │ ├── estimated rows: 1.00 @@ -126,7 +125,6 @@ HashJoin │ ├── aggregate functions: [count()] │ ├── estimated rows: 1.00 │ └── AggregatePartial -│ ├── output columns: [count(*) (#2)] │ ├── group by: [] │ ├── aggregate functions: [count()] │ ├── estimated rows: 1.00 @@ -184,7 +182,6 @@ HashJoin │ ├── limit: 1 │ ├── estimated rows: 1.00 │ └── AggregatePartial -│ ├── output columns: [_count_scalar_subquery (#2), _any_scalar_subquery (#3)] │ ├── group by: [] │ ├── aggregate functions: [count(), any(number)] │ ├── estimated rows: 1.00 @@ -413,7 +410,6 @@ HashJoin │ ├── aggregate functions: [count()] │ ├── estimated rows: 1.00 │ └── AggregatePartial -│ ├── output columns: [COUNT(*) (#3), #_group_by_key] │ ├── group by: [number] │ ├── aggregate functions: [count()] │ ├── estimated rows: 1.00 @@ -530,7 +526,6 @@ EvalScalar │ ├── aggregate functions: [count()] │ ├── estimated rows: 1.00 │ └── AggregatePartial - │ ├── output columns: [count(*) (#3)] │ ├── group by: [] │ ├── aggregate functions: [count()] │ ├── estimated rows: 1.00 diff --git a/tests/suites/1_stateful/04_mini_dataset/04_0000_mini_ontime.result b/tests/suites/1_stateful/04_mini_dataset/04_0000_mini_ontime.result index 6d067bb0e3e0..5467925d36fd 100644 --- a/tests/suites/1_stateful/04_mini_dataset/04_0000_mini_ontime.result +++ b/tests/suites/1_stateful/04_mini_dataset/04_0000_mini_ontime.result @@ -35,6 +35,7 @@ NW 11370.336373327018 OH 12599.259602036094 2006 12.16979169791698 99999 +2006 1 San Diego, CA Los Angeles, CA 622 Los Angeles, CA San Diego, CA 619 Kahului, HI Honolulu, HI 605 diff --git a/tests/suites/1_stateful/04_mini_dataset/04_0000_mini_ontime.sh b/tests/suites/1_stateful/04_mini_dataset/04_0000_mini_ontime.sh index 3b6998d88bff..17b6b13ab816 100755 --- a/tests/suites/1_stateful/04_mini_dataset/04_0000_mini_ontime.sh +++ b/tests/suites/1_stateful/04_mini_dataset/04_0000_mini_ontime.sh @@ -20,6 +20,7 @@ ontime_statements=( "SELECT IATA_CODE_Reporting_Airline AS Carrier, avg(DepDelay) * 1000 AS c3 FROM ontime_mini WHERE (Year >= 2000) AND (Year <= 2008) GROUP BY Carrier ORDER BY Carrier;" "SELECT Year, avg(DepDelay) FROM ontime_mini GROUP BY Year;" "SELECT avg(c1) FROM ( SELECT Year, Month, count(*) AS c1 FROM ontime_mini GROUP BY Year, Month ) AS a;" + "SELECT sum(Year), count(Month) FROM ( SELECT Year, Month FROM ontime_mini GROUP BY Year, Month ) AS a;" "SELECT OriginCityName, DestCityName, count(*) AS c FROM ontime_mini GROUP BY OriginCityName, DestCityName ORDER BY c DESC LIMIT 10;" )