From c600104cfc66add52684405c37d194a39325ad88 Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Tue, 2 Jan 2024 17:39:27 +0800 Subject: [PATCH 01/47] init blockinfo hashtable --- .../src/hashjoin_block_info_hashtable.rs | 90 +++++++++++++++++++ src/common/hashtable/src/lib.rs | 8 +- 2 files changed, 94 insertions(+), 4 deletions(-) create mode 100644 src/common/hashtable/src/hashjoin_block_info_hashtable.rs diff --git a/src/common/hashtable/src/hashjoin_block_info_hashtable.rs b/src/common/hashtable/src/hashjoin_block_info_hashtable.rs new file mode 100644 index 000000000000..252d37ad98d7 --- /dev/null +++ b/src/common/hashtable/src/hashjoin_block_info_hashtable.rs @@ -0,0 +1,90 @@ +// 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::Allocator; +use std::marker::PhantomData; +use std::sync::atomic::AtomicU64; + +use databend_common_base::mem_allocator::MmapAllocator; + +use super::traits::Keyable; +use crate::HashJoinHashtableLike; + +// This hashtable is only used for target build merge into (both standalone and distributed mode). +// Advantages: +// 1. Reduces redundant I/O operations, enhancing performance. +// 2. Lowers the maintenance overhead of deduplicating row_id. +// 3. Allows the scheduling of the subsequent mutation pipeline to be entirely allocated to not matched append operations. +// Disadvantages: +// 1. This solution is likely to be a one-time approach (especially if there are not matched insert operations involved), +// potentially leading to the target table being unsuitable for use as a build table in the future. +// 2. Requires a significant amount of memory to be efficient and currently does not support spill operations. +#[allow(unused)] +pub struct HashJoinBlockInfoHashTable { + pub(crate) pointers: Box<[u64], A>, + pub(crate) atomic_pointers: *mut AtomicU64, + pub(crate) hash_shift: usize, + pub(crate) phantom: PhantomData, +} + +impl HashJoinHashtableLike for HashJoinBlockInfoHashTable +where + K: Keyable, + A: Allocator + Clone + 'static, +{ + type Key = K; + #[allow(unused)] + fn probe( + &self, + hashes: &mut [u64], + bitmap: Option, + ) -> usize { + todo!() + } + #[allow(unused)] + fn early_filtering_probe( + &self, + hashes: &mut [u64], + bitmap: Option, + ) -> usize { + todo!() + } + + #[allow(unused)] + fn early_filtering_probe_with_selection( + &self, + hashes: &mut [u64], + valids: Option, + selection: &mut [u32], + ) -> usize { + todo!() + } + + #[allow(unused)] + fn next_contains(&self, key: &Self::Key, ptr: u64) -> bool { + todo!() + } + + #[allow(unused)] + fn next_probe( + &self, + key: &Self::Key, + ptr: u64, + vec_ptr: *mut crate::RowPtr, + occupied: usize, + capacity: usize, + ) -> (usize, u64) { + todo!() + } +} diff --git a/src/common/hashtable/src/lib.rs b/src/common/hashtable/src/lib.rs index 2f80f9d17cc9..09627e370789 100644 --- a/src/common/hashtable/src/lib.rs +++ b/src/common/hashtable/src/lib.rs @@ -23,18 +23,18 @@ extern crate core; mod container; +mod dictionary_string_hashtable; +mod hashjoin_block_info_hashtable; mod hashjoin_hashtable; mod hashjoin_string_hashtable; mod hashtable; mod keys_ref; mod lookup_hashtable; -mod stack_hashtable; -mod table0; - -mod dictionary_string_hashtable; mod partitioned_hashtable; mod short_string_hashtable; +mod stack_hashtable; mod string_hashtable; +mod table0; #[allow(dead_code)] mod table1; mod table_empty; From c28de64e3876f36dfb22a0d80618b9b8b845cb6e Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Mon, 8 Jan 2024 14:04:08 +0800 Subject: [PATCH 02/47] add some comments --- .../hashtable/src/hashjoin_block_info_hashtable.rs | 3 ++- src/common/hashtable/src/traits.rs | 7 +++++++ src/query/service/src/pipelines/builders/builder_join.rs | 2 +- src/query/service/src/pipelines/pipeline_builder.rs | 2 +- .../pipelines/processors/transforms/hash_join/common.rs | 1 + .../pipelines/processors/transforms/hash_join/desc.rs | 3 +++ .../transforms/hash_join/hash_join_build_state.rs | 2 +- .../transforms/hash_join/hash_join_probe_state.rs | 1 + .../transforms/hash_join/probe_join/left_join.rs | 2 +- .../processors/transforms/hash_join/probe_state.rs | 9 +++++++++ .../processors/transforms/hash_join/result_blocks.rs | 2 +- src/query/sql/src/planner/binder/merge_into.rs | 2 -- 12 files changed, 28 insertions(+), 8 deletions(-) diff --git a/src/common/hashtable/src/hashjoin_block_info_hashtable.rs b/src/common/hashtable/src/hashjoin_block_info_hashtable.rs index 252d37ad98d7..13e5a5152b32 100644 --- a/src/common/hashtable/src/hashjoin_block_info_hashtable.rs +++ b/src/common/hashtable/src/hashjoin_block_info_hashtable.rs @@ -24,7 +24,7 @@ use crate::HashJoinHashtableLike; // This hashtable is only used for target build merge into (both standalone and distributed mode). // Advantages: // 1. Reduces redundant I/O operations, enhancing performance. -// 2. Lowers the maintenance overhead of deduplicating row_id. +// 2. Lowers the maintenance overhead of deduplicating row_id.(But in distributed design, we also need to give rowid) // 3. Allows the scheduling of the subsequent mutation pipeline to be entirely allocated to not matched append operations. // Disadvantages: // 1. This solution is likely to be a one-time approach (especially if there are not matched insert operations involved), @@ -36,6 +36,7 @@ pub struct HashJoinBlockInfoHashTable, + pub(crate) is_distributed: bool, } impl HashJoinHashtableLike for HashJoinBlockInfoHashTable diff --git a/src/common/hashtable/src/traits.rs b/src/common/hashtable/src/traits.rs index 6dc828bcd7e1..cf23c4700a11 100644 --- a/src/common/hashtable/src/traits.rs +++ b/src/common/hashtable/src/traits.rs @@ -522,6 +522,13 @@ pub trait HashJoinHashtableLike { fn next_contains(&self, key: &Self::Key, ptr: u64) -> bool; + /// 1. `key` is the serialize build key from one row + /// 2. `ptr` pointers to the *RawEntry for of the bucket correlated to key.So before this method, + /// we will do a round probe firstly. If the ptr is zero, it means there is no correlated bucket + /// for key + /// 3. `vec_ptr` is RowPtr Array, we use this one to record the matched row in chunks + /// 4. `occupied` is the length for vec_ptr + /// 5. `capacity` is the capacity of vec_ptr fn next_probe( &self, key: &Self::Key, diff --git a/src/query/service/src/pipelines/builders/builder_join.rs b/src/query/service/src/pipelines/builders/builder_join.rs index a415b0cd9b20..0a8f10d3ee28 100644 --- a/src/query/service/src/pipelines/builders/builder_join.rs +++ b/src/query/service/src/pipelines/builders/builder_join.rs @@ -197,7 +197,7 @@ impl PipelineBuilder { Ok(ProcessorPtr::create(transform)) } }; - // for merge into + // for distributed merge into when source as build side. if hash_join_plan.need_hold_hash_table { self.join_state = Some(build_state.clone()) } diff --git a/src/query/service/src/pipelines/pipeline_builder.rs b/src/query/service/src/pipelines/pipeline_builder.rs index d37c8f90cb5f..9843e2994315 100644 --- a/src/query/service/src/pipelines/pipeline_builder.rs +++ b/src/query/service/src/pipelines/pipeline_builder.rs @@ -43,7 +43,7 @@ pub struct PipelineBuilder { pub pipelines: Vec, - // probe data_fields for merge into + // probe data_fields for distributed merge into when source build pub probe_data_fields: Option>, pub join_state: Option>, diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/common.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/common.rs index 1571683d8f1f..3c53de4d7077 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/common.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/common.rs @@ -139,6 +139,7 @@ impl HashJoinProbeState { } impl HashJoinState { + /// if all cols in the same row are all null, we mark this row as null. pub(crate) fn init_markers( &self, cols: &[(Column, DataType)], diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/desc.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/desc.rs index ab38e842c40f..701c35e39e32 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/desc.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/desc.rs @@ -35,6 +35,9 @@ pub struct HashJoinDesc { pub(crate) build_keys: Vec, pub(crate) probe_keys: Vec, pub(crate) join_type: JoinType, + /// when we have non-equal conditions for hash join, + /// for exmaple `a = b and c = d`, we will use `and_filters` + /// to wrap them as a other_predicate to do next step's check. pub(crate) other_predicate: Option, pub(crate) marker_join_desc: MarkJoinDesc, /// Whether the Join are derived from correlated subquery. diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_build_state.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_build_state.rs index 7ded55547232..25213f95ddb5 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_build_state.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_build_state.rs @@ -211,7 +211,7 @@ impl HashJoinBuildState { vec![] }; - let block_mark_scan_map = if self.hash_join_state.need_outer_scan() { + let block_mark_scan_map = if self.hash_join_state.need_mark_scan() { vec![MARKER_KIND_FALSE; data_block.num_rows()] } else { vec![] diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs index 10738a67e6f6..da948d253525 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs @@ -273,6 +273,7 @@ impl HashJoinProbeState { } else { input_num_rows as u64 }; + // We use the information from the probed data to predict the matching state of this probe. let prefer_early_filtering = (probe_state.num_keys_hash_matched as f64) / (probe_state.num_keys as f64) < 0.8; diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/probe_join/left_join.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/probe_join/left_join.rs index 42c494382cc0..caea9cc16bd2 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/probe_join/left_join.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/probe_join/left_join.rs @@ -207,7 +207,7 @@ impl HashJoinProbeState { // Probe hash table and fill `build_indexes`. let (mut match_count, mut incomplete_ptr) = hash_table.next_probe(key, ptr, build_indexes_ptr, matched_idx, max_block_size); - + // `total_probe_matched` is used to record the matched rows count for current `idx` row from probe_block let mut total_probe_matched = 0; if match_count > 0 { total_probe_matched += match_count; diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/probe_state.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/probe_state.rs index c5e00e3e7ac5..f4215a541e8d 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/probe_state.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/probe_state.rs @@ -138,8 +138,17 @@ impl MutableIndexes { } pub struct ProbeBlockGenerationState { + /// in fact, it means whether we need to output some probe blocks's columns, + /// we use probe_projections to check whether we can get a non-empty result + /// block. pub(crate) is_probe_projected: bool, + /// for Right/Full/RightSingle we use true_validity to reduce memory, because + /// we need to wrap probe block's all column type as nullable(if they are not). + /// But when we need to wrap this way, the validity is all true, so we use this + /// one to share the memory. pub(crate) true_validity: Bitmap, + /// we use `string_items_buf` for Binary/String/Bitmap/Variant Column + /// to store the (pointer,length). So we can reuse the memory for all take. pub(crate) string_items_buf: Option>, } diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/result_blocks.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/result_blocks.rs index 5cd8f3ffb2bc..cf4af55b6b0c 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/result_blocks.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/result_blocks.rs @@ -37,7 +37,7 @@ impl HashJoinProbeState { /// non-equi-condition is subquery's child expr with subquery's output column. /// for example: select * from t1 where t1.a = ANY (select t2.a from t2 where t2.b = t1.b); [t1: a, b], [t2: a, b] /// subquery's outer columns: t1.b, and it'll derive a new column: subquery_5 when subquery cross join t1; - /// so equi-condition is t1.b = subquery_5, and non-equi-condition is t1.a = t2.a. + /// so equi-condition is t2.b = subquery_5, and non-equi-condition is t1.a = t2.a. /// 3. Correlated Exists subquery: only have one kind of join condition, equi-condition. /// equi-condition is subquery's outer columns with subquery's derived columns. (see the above example in correlated ANY subquery) pub(crate) fn result_blocks<'a, H: HashJoinHashtableLike>( diff --git a/src/query/sql/src/planner/binder/merge_into.rs b/src/query/sql/src/planner/binder/merge_into.rs index 3ff2d3f8b028..5248e9c3d893 100644 --- a/src/query/sql/src/planner/binder/merge_into.rs +++ b/src/query/sql/src/planner/binder/merge_into.rs @@ -73,8 +73,6 @@ pub enum MergeIntoType { // right outer // 3. matched only: // inner join -// we will import optimizer for these join type in the future. - impl Binder { #[allow(warnings)] #[async_backtrace::framed] From e4a18143353d1d6ab61dc4c84a3a5d13a738c715 Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Mon, 8 Jan 2024 20:47:27 +0800 Subject: [PATCH 03/47] add more comments for hash_table interface --- src/common/hashtable/src/traits.rs | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/src/common/hashtable/src/traits.rs b/src/common/hashtable/src/traits.rs index dafbdc540ff4..63cd39dd6119 100644 --- a/src/common/hashtable/src/traits.rs +++ b/src/common/hashtable/src/traits.rs @@ -508,12 +508,22 @@ pub trait HashJoinHashtableLike { type Key: ?Sized; // Using hashes to probe hash table and converting them in-place to pointers for memory reuse. + // same with `early_filtering_probe`, but we don't use early_filter fn probe(&self, hashes: &mut [u64], bitmap: Option) -> usize; // Using hashes to probe hash table and converting them in-place to pointers for memory reuse. + // 1. same with `early_filtering_probe_with_selection`, but we don't use selection to preserve the + // unfiltered indexes, we just set the filtered hashes as zero. + // 2. return the unfilterd counts. fn early_filtering_probe(&self, hashes: &mut [u64], bitmap: Option) -> usize; // Using hashes to probe hash table and converting them in-place to pointers for memory reuse. + // we use `early_filtering_probe_with_selection` to do the first round probe. + // 1. `hashes` is the hash value of probe block's rows. we will use this one to + // do early filtering. if we can't early filter one row(at idx), we will assign pointer in + // the bucket to hashes[idx] to reuse the memory. + // 2. `selection` is used to preserved the indexes which can't be early_filterd. + // 3. return the count of preserved the indexes in `selection` fn early_filtering_probe_with_selection( &self, hashes: &mut [u64], @@ -521,6 +531,8 @@ pub trait HashJoinHashtableLike { selection: &mut [u32], ) -> usize; + // we use `next_contains` to see whether we can find a matched row in the link. + // the ptr is the link header. fn next_contains(&self, key: &Self::Key, ptr: u64) -> bool; /// 1. `key` is the serialize build key from one row @@ -530,6 +542,8 @@ pub trait HashJoinHashtableLike { /// 3. `vec_ptr` is RowPtr Array, we use this one to record the matched row in chunks /// 4. `occupied` is the length for vec_ptr /// 5. `capacity` is the capacity of vec_ptr + /// 6. return macthed rows count and next ptr which need to test in the future. + /// if the capacity is enougth, the next ptr is zero, otherwise next ptr is valid. fn next_probe( &self, key: &Self::Key, From ce199d40fa7df201d9079e9487015ea703883808 Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Tue, 9 Jan 2024 14:07:50 +0800 Subject: [PATCH 04/47] add merge_into_join_type info and block_info index --- .../src/hashjoin_block_info_hashtable.rs | 3 +- src/common/hashtable/src/utils.rs | 107 ++++++++++++++++++ src/query/catalog/src/lib.rs | 1 + src/query/catalog/src/merge_into_join.rs | 37 ++++++ src/query/catalog/src/table_context.rs | 3 + .../transforms/hash_join/build_state.rs | 2 + src/query/service/src/sessions/query_ctx.rs | 6 + .../service/src/sessions/query_ctx_shared.rs | 4 + .../it/storages/fuse/operations/commit.rs | 6 + 9 files changed, 168 insertions(+), 1 deletion(-) create mode 100644 src/query/catalog/src/merge_into_join.rs diff --git a/src/common/hashtable/src/hashjoin_block_info_hashtable.rs b/src/common/hashtable/src/hashjoin_block_info_hashtable.rs index 13e5a5152b32..1fedb1b44201 100644 --- a/src/common/hashtable/src/hashjoin_block_info_hashtable.rs +++ b/src/common/hashtable/src/hashjoin_block_info_hashtable.rs @@ -35,8 +35,9 @@ pub struct HashJoinBlockInfoHashTable, pub(crate) atomic_pointers: *mut AtomicU64, pub(crate) hash_shift: usize, - pub(crate) phantom: PhantomData, pub(crate) is_distributed: bool, + pub(crate) matched: Box<[u64]>, + pub(crate) phantom: PhantomData, } impl HashJoinHashtableLike for HashJoinBlockInfoHashTable diff --git a/src/common/hashtable/src/utils.rs b/src/common/hashtable/src/utils.rs index 083255f31129..ce1ee58b3937 100644 --- a/src/common/hashtable/src/utils.rs +++ b/src/common/hashtable/src/utils.rs @@ -185,3 +185,110 @@ pub mod sse { } } } + +// for merge into: +// we use BlockInfoIndex to maintain an index +// for the block info in chunks. +pub struct BlockInfoIndex { + // the intervals will be like below: + // (0,10)(11,29),(30,38). it's ordered. + intervals: Vec, + prefixs: Vec, + length: usize, +} + +pub type Interval = (u32, u32); + +/// the segment blocks are not sequential,because we do parallel hashtable build. +/// the block lay out in chunks could be like belows: +/// segment0_block1 | +/// segment1_block0 | chunk0 +/// segment0_block0 | +/// +/// segment0_block3 | +/// segment1_block1 | chunk1 +/// segment2_block0 | +/// +/// ......... +impl BlockInfoIndex { + pub fn new_with_capacity(capacity: usize) -> Self { + BlockInfoIndex { + intervals: Vec::with_capacity(capacity), + prefixs: Vec::with_capacity(capacity), + length: 0, + } + } + + /// 1.interval stands for the (start,end) in chunks for one block. + /// 2.prefix is the segment_id_block_id composition. + /// we can promise the orderd insert from outside. + pub fn insert_block_offsets(&mut self, interval: Interval, prefix: u64) { + self.intervals.push(interval); + self.prefixs.push(prefix); + self.length += 1; + } + + /// we do a binary search to get the partial modified offsets + /// we will return the Interval and prefix. For example: + /// intervals: (0,10)(11,22),(23,40)(41,55) + /// interval: (8,27) + /// we will give (8,10),(23,27), we don't give the (11,12),because it's updated all. + pub fn get_block_info(&self, interval: Interval) -> Vec<(Interval, u64)> { + let mut res = Vec::<(Interval, u64)>::with_capacity(2); + let left_idx = self.search_idx(interval.0); + let right_idx = self.search_idx(interval.1); + let left_interval = &self.intervals[left_idx]; + let right_interval = &self.intervals[right_idx]; + if left_interval.0 < interval.0 { + res.push(((interval.0, left_interval.1), self.prefixs[left_idx])) + } + if right_interval.1 > interval.1 { + res.push(((right_interval.0, interval.1), self.prefixs[right_idx])) + } + res + } + + /// search idx help us to find out the intervals idx which contain offset. + /// It must contain offset. + fn search_idx(&self, offset: u32) -> usize { + let mut l = 0; + let mut r = self.length - 1; + while l < r { + let mid = (l + r + 1) / 2; + if self.intervals[mid].0 <= offset { + l = mid; + } else { + r = mid - 1; + } + } + l + } +} + +/// we think the build blocks count is about 1024 at most time. +impl Default for BlockInfoIndex { + fn default() -> Self { + Self { + intervals: Vec::with_capacity(1024), + prefixs: Vec::with_capacity(1024), + length: 0, + } + } +} + +#[test] +fn test_block_info_index() { + // let's build [0,10][11,20][21,30],[31,40],and then find [10,37]. + // we should get [10,10],[31,37] + let intevals: Vec = vec![(0, 10), (11, 20), (21, 30), (31, 40)]; + let find_interval: Interval = (10, 37); + let mut block_info_index = BlockInfoIndex::new_with_capacity(10); + for (idx, interval) in intevals.iter().enumerate() { + block_info_index.insert_block_offsets(interval.clone(), idx as u64) + } + let result = block_info_index.get_block_info(find_interval); + assert_eq!(result[0].0, (10, 10)); + assert_eq!(result[0].1, 0); + assert_eq!(result[1].0, (31, 37)); + assert_eq!(result[1].1, 3); +} diff --git a/src/query/catalog/src/lib.rs b/src/query/catalog/src/lib.rs index 98c76eb544ae..57536d2dcba8 100644 --- a/src/query/catalog/src/lib.rs +++ b/src/query/catalog/src/lib.rs @@ -26,5 +26,6 @@ pub mod table_args; pub mod table_context; pub mod table_function; +pub mod merge_into_join; pub mod runtime_filter_info; pub mod table; diff --git a/src/query/catalog/src/merge_into_join.rs b/src/query/catalog/src/merge_into_join.rs new file mode 100644 index 000000000000..39315b788c75 --- /dev/null +++ b/src/query/catalog/src/merge_into_join.rs @@ -0,0 +1,37 @@ +// 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. + +pub enum MergeIntoJoinType { + Left, + Right, + Inner, + LeftAnti, + RightAnti, + // it means this join is not a merge into join + NormalJoin, +} + +pub struct MergeIntoJoin { + merge_into_join_type: MergeIntoJoinType, + is_distributed: bool, +} + +impl Default for MergeIntoJoin { + fn default() -> Self { + Self { + merge_into_join_type: MergeIntoJoinType::NormalJoin, + is_distributed: Default::default(), + } + } +} diff --git a/src/query/catalog/src/table_context.rs b/src/query/catalog/src/table_context.rs index a32dedd6ce63..57611ee71a12 100644 --- a/src/query/catalog/src/table_context.rs +++ b/src/query/catalog/src/table_context.rs @@ -52,6 +52,7 @@ use xorf::BinaryFuse16; use crate::catalog::Catalog; use crate::cluster_info::Cluster; +use crate::merge_into_join::MergeIntoJoin; use crate::plan::DataSourcePlan; use crate::plan::PartInfoPtr; use crate::plan::Partitions; @@ -243,6 +244,8 @@ pub trait TableContext: Send + Sync { fn set_runtime_filter(&self, filters: (usize, RuntimeFilterInfo)); + fn set_merge_into_join_type(&self, join: MergeIntoJoin); + fn get_bloom_runtime_filter_with_id(&self, id: usize) -> Vec<(String, BinaryFuse16)>; fn get_inlist_runtime_filter_with_id(&self, id: usize) -> Vec>; diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/build_state.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/build_state.rs index 28e8538795b7..be9e050b8886 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/build_state.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/build_state.rs @@ -39,8 +39,10 @@ pub struct BuildBlockGenerationState { pub(crate) build_num_rows: usize, /// Data of the build side. pub(crate) chunks: Vec, + // we converted all chunks into ColumnVec for every column. pub(crate) build_columns: Vec, pub(crate) build_columns_data_type: Vec, + // after projectd by build_projection, whether we still have data. pub(crate) is_build_projected: bool, } diff --git a/src/query/service/src/sessions/query_ctx.rs b/src/query/service/src/sessions/query_ctx.rs index aac9b25b2de2..64e13c86b360 100644 --- a/src/query/service/src/sessions/query_ctx.rs +++ b/src/query/service/src/sessions/query_ctx.rs @@ -36,6 +36,7 @@ use databend_common_base::base::tokio::task::JoinHandle; use databend_common_base::base::Progress; use databend_common_base::base::ProgressValues; use databend_common_base::runtime::TrySpawn; +use databend_common_catalog::merge_into_join::MergeIntoJoin; use databend_common_catalog::plan::DataSourceInfo; use databend_common_catalog::plan::DataSourcePlan; use databend_common_catalog::plan::PartInfoPtr; @@ -910,6 +911,11 @@ impl TableContext for QueryContext { queries_profile } + fn set_merge_into_join_type(&self, join: MergeIntoJoin) { + let mut merge_into_join = self.shared.merge_into_join.write(); + *merge_into_join = join; + } + fn set_runtime_filter(&self, filters: (IndexType, RuntimeFilterInfo)) { let mut runtime_filters = self.shared.runtime_filters.write(); match runtime_filters.entry(filters.0) { diff --git a/src/query/service/src/sessions/query_ctx_shared.rs b/src/query/service/src/sessions/query_ctx_shared.rs index fd7f50f3b460..d5b6607c6168 100644 --- a/src/query/service/src/sessions/query_ctx_shared.rs +++ b/src/query/service/src/sessions/query_ctx_shared.rs @@ -24,6 +24,7 @@ use dashmap::DashMap; use databend_common_base::base::Progress; use databend_common_base::runtime::Runtime; use databend_common_catalog::catalog::CatalogManager; +use databend_common_catalog::merge_into_join::MergeIntoJoin; use databend_common_catalog::query_kind::QueryKind; use databend_common_catalog::runtime_filter_info::RuntimeFilterInfo; use databend_common_catalog::table_context::MaterializedCtesBlocks; @@ -113,6 +114,8 @@ pub struct QueryContextShared { pub(in crate::sessions) query_profiles: Arc, PlanProfile>>>, pub(in crate::sessions) runtime_filters: Arc>>, + + pub(in crate::sessions) merge_into_join: Arc>, } impl QueryContextShared { @@ -158,6 +161,7 @@ impl QueryContextShared { group_by_spill_progress: Arc::new(Progress::create()), query_profiles: Arc::new(RwLock::new(HashMap::new())), runtime_filters: Default::default(), + merge_into_join: Default::default(), })) } diff --git a/src/query/service/tests/it/storages/fuse/operations/commit.rs b/src/query/service/tests/it/storages/fuse/operations/commit.rs index 597e0ef042db..8d5d2221481b 100644 --- a/src/query/service/tests/it/storages/fuse/operations/commit.rs +++ b/src/query/service/tests/it/storages/fuse/operations/commit.rs @@ -24,6 +24,7 @@ use databend_common_base::base::ProgressValues; use databend_common_catalog::catalog::Catalog; use databend_common_catalog::cluster_info::Cluster; use databend_common_catalog::database::Database; +use databend_common_catalog::merge_into_join::MergeIntoJoin; use databend_common_catalog::plan::DataSourcePlan; use databend_common_catalog::plan::PartInfoPtr; use databend_common_catalog::plan::Partitions; @@ -699,6 +700,11 @@ impl TableContext for CtxDelegation { fn get_query_profiles(&self) -> Vec { todo!() } + + fn set_merge_into_join_type(&self, join: MergeIntoJoin) { + todo!() + } + fn set_runtime_filter(&self, _filters: (IndexType, RuntimeFilterInfo)) { todo!() } From 45f9099ce127ed5ccd8414119e0af9def94b5d96 Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Tue, 9 Jan 2024 14:31:16 +0800 Subject: [PATCH 05/47] add block info hashtable basic implementation --- .../src/hashjoin_block_info_hashtable.rs | 260 +++++++++++++-- .../hashjoin_block_info_string_hashtable.rs | 301 ++++++++++++++++++ src/common/hashtable/src/lib.rs | 4 + 3 files changed, 540 insertions(+), 25 deletions(-) create mode 100644 src/common/hashtable/src/hashjoin_block_info_string_hashtable.rs diff --git a/src/common/hashtable/src/hashjoin_block_info_hashtable.rs b/src/common/hashtable/src/hashjoin_block_info_hashtable.rs index 1fedb1b44201..a3f099a88fd0 100644 --- a/src/common/hashtable/src/hashjoin_block_info_hashtable.rs +++ b/src/common/hashtable/src/hashjoin_block_info_hashtable.rs @@ -15,11 +15,20 @@ use std::alloc::Allocator; use std::marker::PhantomData; use std::sync::atomic::AtomicU64; +use std::sync::atomic::Ordering; +use databend_common_arrow::arrow::bitmap::Bitmap; use databend_common_base::mem_allocator::MmapAllocator; use super::traits::Keyable; +use crate::hashjoin_hashtable::combine_header; +use crate::hashjoin_hashtable::early_filtering; +use crate::hashjoin_hashtable::hash_bits; +use crate::hashjoin_hashtable::new_header; +use crate::hashjoin_hashtable::remove_header_tag; use crate::HashJoinHashtableLike; +use crate::RawEntry; +use crate::RowPtr; // This hashtable is only used for target build merge into (both standalone and distributed mode). // Advantages: @@ -40,53 +49,254 @@ pub struct HashJoinBlockInfoHashTable, } +unsafe impl Send + for HashJoinBlockInfoHashTable +{ +} + +unsafe impl Sync + for HashJoinBlockInfoHashTable +{ +} + +impl HashJoinBlockInfoHashTable { + pub fn with_build_row_num(row_num: usize) -> Self { + let capacity = std::cmp::max((row_num * 2).next_power_of_two(), 1 << 10); + let mut hashtable = Self { + pointers: unsafe { + Box::new_zeroed_slice_in(capacity, Default::default()).assume_init() + }, + atomic_pointers: std::ptr::null_mut(), + hash_shift: (hash_bits() - capacity.trailing_zeros()) as usize, + phantom: PhantomData, + is_distributed: false, + matched: unsafe { Box::new_zeroed_slice_in(row_num, Default::default()).assume_init() }, + }; + hashtable.atomic_pointers = unsafe { + std::mem::transmute::<*mut u64, *mut AtomicU64>(hashtable.pointers.as_mut_ptr()) + }; + hashtable + } + + pub fn insert(&mut self, key: K, entry_ptr: *mut RawEntry) { + let hash = key.hash(); + let index = (hash >> self.hash_shift) as usize; + let new_header = new_header(entry_ptr as u64, hash); + // # Safety + // `index` is less than the capacity of hash table. + let mut old_header = unsafe { (*self.atomic_pointers.add(index)).load(Ordering::Relaxed) }; + loop { + let res = unsafe { + (*self.atomic_pointers.add(index)).compare_exchange_weak( + old_header, + combine_header(new_header, old_header), + Ordering::SeqCst, + Ordering::SeqCst, + ) + }; + match res { + Ok(_) => break, + Err(x) => old_header = x, + }; + } + unsafe { (*entry_ptr).next = remove_header_tag(old_header) }; + } +} + impl HashJoinHashtableLike for HashJoinBlockInfoHashTable where K: Keyable, A: Allocator + Clone + 'static, { type Key = K; - #[allow(unused)] - fn probe( - &self, - hashes: &mut [u64], - bitmap: Option, - ) -> usize { - todo!() + + // Using hashes to probe hash table and converting them in-place to pointers for memory reuse. + fn probe(&self, hashes: &mut [u64], bitmap: Option) -> usize { + let mut valids = None; + if let Some(bitmap) = bitmap { + if bitmap.unset_bits() == bitmap.len() { + hashes.iter_mut().for_each(|hash| { + *hash = 0; + }); + return 0; + } else if bitmap.unset_bits() > 0 { + valids = Some(bitmap); + } + } + let mut count = 0; + match valids { + Some(valids) => { + valids + .iter() + .zip(hashes.iter_mut()) + .for_each(|(valid, hash)| { + if valid { + let header = self.pointers[(*hash >> self.hash_shift) as usize]; + if header != 0 { + *hash = remove_header_tag(header); + count += 1; + } else { + *hash = 0; + } + } else { + *hash = 0; + } + }); + } + None => { + hashes.iter_mut().for_each(|hash| { + let header = self.pointers[(*hash >> self.hash_shift) as usize]; + if header != 0 { + *hash = remove_header_tag(header); + count += 1; + } else { + *hash = 0; + } + }); + } + } + count } - #[allow(unused)] - fn early_filtering_probe( - &self, - hashes: &mut [u64], - bitmap: Option, - ) -> usize { - todo!() + + // Using hashes to probe hash table and converting them in-place to pointers for memory reuse. + fn early_filtering_probe(&self, hashes: &mut [u64], bitmap: Option) -> usize { + let mut valids = None; + if let Some(bitmap) = bitmap { + if bitmap.unset_bits() == bitmap.len() { + hashes.iter_mut().for_each(|hash| { + *hash = 0; + }); + return 0; + } else if bitmap.unset_bits() > 0 { + valids = Some(bitmap); + } + } + let mut count = 0; + match valids { + Some(valids) => { + valids + .iter() + .zip(hashes.iter_mut()) + .for_each(|(valid, hash)| { + if valid { + let header = self.pointers[(*hash >> self.hash_shift) as usize]; + if header != 0 && early_filtering(header, *hash) { + *hash = remove_header_tag(header); + count += 1; + } else { + *hash = 0; + } + } else { + *hash = 0; + } + }); + } + None => { + hashes.iter_mut().for_each(|hash| { + let header = self.pointers[(*hash >> self.hash_shift) as usize]; + if header != 0 && early_filtering(header, *hash) { + *hash = remove_header_tag(header); + count += 1; + } else { + *hash = 0; + } + }); + } + } + count } - #[allow(unused)] + // Using hashes to probe hash table and converting them in-place to pointers for memory reuse. fn early_filtering_probe_with_selection( &self, hashes: &mut [u64], - valids: Option, + bitmap: Option, selection: &mut [u32], ) -> usize { - todo!() + let mut valids = None; + if let Some(bitmap) = bitmap { + if bitmap.unset_bits() == bitmap.len() { + return 0; + } else if bitmap.unset_bits() > 0 { + valids = Some(bitmap); + } + } + let mut count = 0; + match valids { + Some(valids) => { + valids.iter().zip(hashes.iter_mut().enumerate()).for_each( + |(valid, (idx, hash))| { + if valid { + let header = self.pointers[(*hash >> self.hash_shift) as usize]; + if header != 0 && early_filtering(header, *hash) { + *hash = remove_header_tag(header); + unsafe { *selection.get_unchecked_mut(count) = idx as u32 }; + count += 1; + } + } + }, + ); + } + None => { + hashes.iter_mut().enumerate().for_each(|(idx, hash)| { + let header = self.pointers[(*hash >> self.hash_shift) as usize]; + if header != 0 && early_filtering(header, *hash) { + *hash = remove_header_tag(header); + unsafe { *selection.get_unchecked_mut(count) = idx as u32 }; + count += 1; + } + }); + } + } + count } - #[allow(unused)] - fn next_contains(&self, key: &Self::Key, ptr: u64) -> bool { - todo!() + fn next_contains(&self, key: &Self::Key, mut ptr: u64) -> bool { + loop { + if ptr == 0 { + break; + } + let raw_entry = unsafe { &*(ptr as *mut RawEntry) }; + if key == &raw_entry.key { + return true; + } + ptr = raw_entry.next; + } + false } - #[allow(unused)] fn next_probe( &self, key: &Self::Key, - ptr: u64, - vec_ptr: *mut crate::RowPtr, - occupied: usize, + mut ptr: u64, + vec_ptr: *mut RowPtr, + mut occupied: usize, capacity: usize, ) -> (usize, u64) { - todo!() + let origin = occupied; + loop { + if ptr == 0 || occupied >= capacity { + break; + } + let raw_entry = unsafe { &*(ptr as *mut RawEntry) }; + if key == &raw_entry.key { + // # Safety + // occupied is less than the capacity of vec_ptr. + unsafe { + std::ptr::copy_nonoverlapping( + &raw_entry.row_ptr as *const RowPtr, + vec_ptr.add(occupied), + 1, + ) + }; + occupied += 1; + } + ptr = raw_entry.next; + } + if occupied > origin { + (occupied - origin, ptr) + } else { + (0, 0) + } } } diff --git a/src/common/hashtable/src/hashjoin_block_info_string_hashtable.rs b/src/common/hashtable/src/hashjoin_block_info_string_hashtable.rs new file mode 100644 index 000000000000..c74cf35b0c96 --- /dev/null +++ b/src/common/hashtable/src/hashjoin_block_info_string_hashtable.rs @@ -0,0 +1,301 @@ +// 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::Allocator; +use std::sync::atomic::AtomicU64; +use std::sync::atomic::Ordering; + +use databend_common_arrow::arrow::bitmap::Bitmap; +use databend_common_base::mem_allocator::MmapAllocator; + +use crate::hash_join_fast_string_hash; +use crate::hashjoin_hashtable::combine_header; +use crate::hashjoin_hashtable::early_filtering; +use crate::hashjoin_hashtable::hash_bits; +use crate::hashjoin_hashtable::new_header; +use crate::hashjoin_hashtable::remove_header_tag; +use crate::HashJoinHashtableLike; +use crate::RowPtr; +use crate::StringRawEntry; +use crate::STRING_EARLY_SIZE; + +pub struct HashJoinBlockInfoStringHashTable { + pub(crate) pointers: Box<[u64], A>, + pub(crate) atomic_pointers: *mut AtomicU64, + pub(crate) hash_shift: usize, + pub(crate) is_distributed: bool, + pub(crate) matched: Box<[u64]>, +} + +unsafe impl Send for HashJoinBlockInfoStringHashTable {} + +unsafe impl Sync for HashJoinBlockInfoStringHashTable {} + +impl HashJoinBlockInfoStringHashTable { + pub fn with_build_row_num(row_num: usize) -> Self { + let capacity = std::cmp::max((row_num * 2).next_power_of_two(), 1 << 10); + let mut hashtable = Self { + pointers: unsafe { + Box::new_zeroed_slice_in(capacity, Default::default()).assume_init() + }, + atomic_pointers: std::ptr::null_mut(), + hash_shift: (hash_bits() - capacity.trailing_zeros()) as usize, + is_distributed: false, + matched: unsafe { Box::new_zeroed_slice_in(row_num, Default::default()).assume_init() }, + }; + hashtable.atomic_pointers = unsafe { + std::mem::transmute::<*mut u64, *mut AtomicU64>(hashtable.pointers.as_mut_ptr()) + }; + hashtable + } + + pub fn insert(&mut self, key: &[u8], entry_ptr: *mut StringRawEntry) { + let hash = hash_join_fast_string_hash(key); + let index = (hash >> self.hash_shift) as usize; + let new_header = new_header(entry_ptr as u64, hash); + // # Safety + // `index` is less than the capacity of hash table. + let mut old_header = unsafe { (*self.atomic_pointers.add(index)).load(Ordering::Relaxed) }; + loop { + let res = unsafe { + (*self.atomic_pointers.add(index)).compare_exchange_weak( + old_header, + combine_header(new_header, old_header), + Ordering::SeqCst, + Ordering::SeqCst, + ) + }; + match res { + Ok(_) => break, + Err(x) => old_header = x, + }; + } + unsafe { (*entry_ptr).next = remove_header_tag(old_header) }; + } +} + +impl HashJoinHashtableLike for HashJoinBlockInfoStringHashTable +where A: Allocator + Clone + 'static +{ + type Key = [u8]; + + // Using hashes to probe hash table and converting them in-place to pointers for memory reuse. + fn probe(&self, hashes: &mut [u64], bitmap: Option) -> usize { + let mut valids = None; + if let Some(bitmap) = bitmap { + if bitmap.unset_bits() == bitmap.len() { + hashes.iter_mut().for_each(|hash| { + *hash = 0; + }); + return 0; + } else if bitmap.unset_bits() > 0 { + valids = Some(bitmap); + } + } + let mut count = 0; + match valids { + Some(valids) => { + hashes.iter_mut().enumerate().for_each(|(idx, hash)| { + if unsafe { valids.get_bit_unchecked(idx) } { + let header = self.pointers[(*hash >> self.hash_shift) as usize]; + if header != 0 { + *hash = remove_header_tag(header); + count += 1; + } else { + *hash = 0; + } + } else { + *hash = 0; + }; + }); + } + None => { + hashes.iter_mut().for_each(|hash| { + let header = self.pointers[(*hash >> self.hash_shift) as usize]; + if header != 0 { + *hash = remove_header_tag(header); + count += 1; + } else { + *hash = 0; + } + }); + } + } + count + } + + // Using hashes to probe hash table and converting them in-place to pointers for memory reuse. + fn early_filtering_probe(&self, hashes: &mut [u64], bitmap: Option) -> usize { + let mut valids = None; + if let Some(bitmap) = bitmap { + if bitmap.unset_bits() == bitmap.len() { + hashes.iter_mut().for_each(|hash| { + *hash = 0; + }); + return 0; + } else if bitmap.unset_bits() > 0 { + valids = Some(bitmap); + } + } + let mut count = 0; + match valids { + Some(valids) => { + hashes.iter_mut().enumerate().for_each(|(idx, hash)| { + if unsafe { valids.get_bit_unchecked(idx) } { + let header = self.pointers[(*hash >> self.hash_shift) as usize]; + if header != 0 && early_filtering(header, *hash) { + *hash = remove_header_tag(header); + count += 1; + } else { + *hash = 0; + } + } else { + *hash = 0; + }; + }); + } + None => { + hashes.iter_mut().for_each(|hash| { + let header = self.pointers[(*hash >> self.hash_shift) as usize]; + if header != 0 && early_filtering(header, *hash) { + *hash = remove_header_tag(header); + count += 1; + } else { + *hash = 0; + } + }); + } + } + count + } + + // Using hashes to probe hash table and converting them in-place to pointers for memory reuse. + fn early_filtering_probe_with_selection( + &self, + hashes: &mut [u64], + bitmap: Option, + selection: &mut [u32], + ) -> usize { + let mut valids = None; + if let Some(bitmap) = bitmap { + if bitmap.unset_bits() == bitmap.len() { + return 0; + } else if bitmap.unset_bits() > 0 { + valids = Some(bitmap); + } + } + let mut count = 0; + match valids { + Some(valids) => { + hashes.iter_mut().enumerate().for_each(|(idx, hash)| { + if unsafe { valids.get_bit_unchecked(idx) } { + let header = self.pointers[(*hash >> self.hash_shift) as usize]; + if header != 0 && early_filtering(header, *hash) { + *hash = remove_header_tag(header); + unsafe { *selection.get_unchecked_mut(count) = idx as u32 }; + count += 1; + } + } + }); + } + None => { + hashes.iter_mut().enumerate().for_each(|(idx, hash)| { + let header = self.pointers[(*hash >> self.hash_shift) as usize]; + if header != 0 && early_filtering(header, *hash) { + *hash = remove_header_tag(header); + unsafe { *selection.get_unchecked_mut(count) = idx as u32 }; + count += 1; + } + }); + } + } + count + } + + fn next_contains(&self, key: &Self::Key, mut ptr: u64) -> bool { + loop { + if ptr == 0 { + break; + } + let raw_entry = unsafe { &*(ptr as *mut StringRawEntry) }; + // Compare `early` and the length of the string, the size of `early` is 4. + let min_len = std::cmp::min( + STRING_EARLY_SIZE, + std::cmp::min(key.len(), raw_entry.length as usize), + ); + if raw_entry.length as usize == key.len() + && key[0..min_len] == raw_entry.early[0..min_len] + { + let key_ref = unsafe { + std::slice::from_raw_parts( + raw_entry.key as *const u8, + raw_entry.length as usize, + ) + }; + if key == key_ref { + return true; + } + } + ptr = raw_entry.next; + } + false + } + + fn next_probe( + &self, + key: &Self::Key, + mut ptr: u64, + vec_ptr: *mut RowPtr, + mut occupied: usize, + capacity: usize, + ) -> (usize, u64) { + let origin = occupied; + loop { + if ptr == 0 || occupied >= capacity { + break; + } + let raw_entry = unsafe { &*(ptr as *mut StringRawEntry) }; + // Compare `early` and the length of the string, the size of `early` is 4. + let min_len = std::cmp::min(STRING_EARLY_SIZE, key.len()); + if raw_entry.length as usize == key.len() + && key[0..min_len] == raw_entry.early[0..min_len] + { + let key_ref = unsafe { + std::slice::from_raw_parts( + raw_entry.key as *const u8, + raw_entry.length as usize, + ) + }; + if key == key_ref { + // # Safety + // occupied is less than the capacity of vec_ptr. + unsafe { + std::ptr::copy_nonoverlapping( + &raw_entry.row_ptr as *const RowPtr, + vec_ptr.add(occupied), + 1, + ) + }; + occupied += 1; + } + } + ptr = raw_entry.next; + } + if occupied > origin { + (occupied - origin, ptr) + } else { + (0, 0) + } + } +} diff --git a/src/common/hashtable/src/lib.rs b/src/common/hashtable/src/lib.rs index 09627e370789..4414b7a977f2 100644 --- a/src/common/hashtable/src/lib.rs +++ b/src/common/hashtable/src/lib.rs @@ -25,6 +25,7 @@ extern crate core; mod container; mod dictionary_string_hashtable; mod hashjoin_block_info_hashtable; +mod hashjoin_block_info_string_hashtable; mod hashjoin_hashtable; mod hashjoin_string_hashtable; mod hashtable; @@ -112,4 +113,7 @@ pub use keys_ref::KeysRef; pub use partitioned_hashtable::hash2bucket; pub type HashJoinHashMap = hashjoin_hashtable::HashJoinHashTable; pub type StringHashJoinHashMap = hashjoin_string_hashtable::HashJoinStringHashTable; +pub type BlockInfoStringJoinHashMap = + hashjoin_block_info_string_hashtable::HashJoinBlockInfoStringHashTable; +pub type BlockInfoJoinHashMap = hashjoin_block_info_hashtable::HashJoinBlockInfoHashTable; pub use traits::HashJoinHashtableLike; From f24d32870c1d2260092c7bfd9c0190ddc52133b6 Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Tue, 9 Jan 2024 16:19:57 +0800 Subject: [PATCH 06/47] fix typos --- .../hashjoin_block_info_string_hashtable.rs | 1 + src/common/hashtable/src/traits.rs | 6 ++-- src/common/hashtable/src/utils.rs | 19 ++++++---- src/query/catalog/src/merge_into_join.rs | 10 ++++-- .../transforms/hash_join/build_state.rs | 2 +- .../processors/transforms/hash_join/desc.rs | 2 +- .../tests/it/sql/exec/get_table_bind_test.rs | 6 +++- .../it/storages/fuse/operations/commit.rs | 2 +- .../sql/src/planner/optimizer/optimizer.rs | 36 +++++++++++++++++-- 9 files changed, 67 insertions(+), 17 deletions(-) diff --git a/src/common/hashtable/src/hashjoin_block_info_string_hashtable.rs b/src/common/hashtable/src/hashjoin_block_info_string_hashtable.rs index c74cf35b0c96..1dfa2d0ff985 100644 --- a/src/common/hashtable/src/hashjoin_block_info_string_hashtable.rs +++ b/src/common/hashtable/src/hashjoin_block_info_string_hashtable.rs @@ -30,6 +30,7 @@ use crate::RowPtr; use crate::StringRawEntry; use crate::STRING_EARLY_SIZE; +#[allow(unused)] pub struct HashJoinBlockInfoStringHashTable { pub(crate) pointers: Box<[u64], A>, pub(crate) atomic_pointers: *mut AtomicU64, diff --git a/src/common/hashtable/src/traits.rs b/src/common/hashtable/src/traits.rs index 63cd39dd6119..8ee0a99e8ef0 100644 --- a/src/common/hashtable/src/traits.rs +++ b/src/common/hashtable/src/traits.rs @@ -514,7 +514,7 @@ pub trait HashJoinHashtableLike { // Using hashes to probe hash table and converting them in-place to pointers for memory reuse. // 1. same with `early_filtering_probe_with_selection`, but we don't use selection to preserve the // unfiltered indexes, we just set the filtered hashes as zero. - // 2. return the unfilterd counts. + // 2. return the unfiltered counts. fn early_filtering_probe(&self, hashes: &mut [u64], bitmap: Option) -> usize; // Using hashes to probe hash table and converting them in-place to pointers for memory reuse. @@ -522,7 +522,7 @@ pub trait HashJoinHashtableLike { // 1. `hashes` is the hash value of probe block's rows. we will use this one to // do early filtering. if we can't early filter one row(at idx), we will assign pointer in // the bucket to hashes[idx] to reuse the memory. - // 2. `selection` is used to preserved the indexes which can't be early_filterd. + // 2. `selection` is used to preserved the indexes which can't be early_filtered. // 3. return the count of preserved the indexes in `selection` fn early_filtering_probe_with_selection( &self, @@ -543,7 +543,7 @@ pub trait HashJoinHashtableLike { /// 4. `occupied` is the length for vec_ptr /// 5. `capacity` is the capacity of vec_ptr /// 6. return macthed rows count and next ptr which need to test in the future. - /// if the capacity is enougth, the next ptr is zero, otherwise next ptr is valid. + /// if the capacity is enough, the next ptr is zero, otherwise next ptr is valid. fn next_probe( &self, key: &Self::Key, diff --git a/src/common/hashtable/src/utils.rs b/src/common/hashtable/src/utils.rs index ce1ee58b3937..d925d00dccca 100644 --- a/src/common/hashtable/src/utils.rs +++ b/src/common/hashtable/src/utils.rs @@ -187,13 +187,15 @@ pub mod sse { } // for merge into: -// we use BlockInfoIndex to maintain an index -// for the block info in chunks. +// we use BlockInfoIndex to maintain an index for the block info in chunks. pub struct BlockInfoIndex { // the intervals will be like below: // (0,10)(11,29),(30,38). it's ordered. + #[allow(dead_code)] intervals: Vec, + #[allow(dead_code)] prefixs: Vec, + #[allow(dead_code)] length: usize, } @@ -210,7 +212,9 @@ pub type Interval = (u32, u32); /// segment2_block0 | /// /// ......... + impl BlockInfoIndex { + #[allow(dead_code)] pub fn new_with_capacity(capacity: usize) -> Self { BlockInfoIndex { intervals: Vec::with_capacity(capacity), @@ -221,7 +225,8 @@ impl BlockInfoIndex { /// 1.interval stands for the (start,end) in chunks for one block. /// 2.prefix is the segment_id_block_id composition. - /// we can promise the orderd insert from outside. + /// we can promise the ordered insert from outside. + #[allow(dead_code)] pub fn insert_block_offsets(&mut self, interval: Interval, prefix: u64) { self.intervals.push(interval); self.prefixs.push(prefix); @@ -233,6 +238,7 @@ impl BlockInfoIndex { /// intervals: (0,10)(11,22),(23,40)(41,55) /// interval: (8,27) /// we will give (8,10),(23,27), we don't give the (11,12),because it's updated all. + #[allow(dead_code)] pub fn get_block_info(&self, interval: Interval) -> Vec<(Interval, u64)> { let mut res = Vec::<(Interval, u64)>::with_capacity(2); let left_idx = self.search_idx(interval.0); @@ -250,6 +256,7 @@ impl BlockInfoIndex { /// search idx help us to find out the intervals idx which contain offset. /// It must contain offset. + #[allow(dead_code)] fn search_idx(&self, offset: u32) -> usize { let mut l = 0; let mut r = self.length - 1; @@ -280,11 +287,11 @@ impl Default for BlockInfoIndex { fn test_block_info_index() { // let's build [0,10][11,20][21,30],[31,40],and then find [10,37]. // we should get [10,10],[31,37] - let intevals: Vec = vec![(0, 10), (11, 20), (21, 30), (31, 40)]; + let intervals: Vec = vec![(0, 10), (11, 20), (21, 30), (31, 40)]; let find_interval: Interval = (10, 37); let mut block_info_index = BlockInfoIndex::new_with_capacity(10); - for (idx, interval) in intevals.iter().enumerate() { - block_info_index.insert_block_offsets(interval.clone(), idx as u64) + for (idx, interval) in intervals.iter().enumerate() { + block_info_index.insert_block_offsets(*interval, idx as u64) } let result = block_info_index.get_block_info(find_interval); assert_eq!(result[0].0, (10, 10)); diff --git a/src/query/catalog/src/merge_into_join.rs b/src/query/catalog/src/merge_into_join.rs index 39315b788c75..26a55c438c6e 100644 --- a/src/query/catalog/src/merge_into_join.rs +++ b/src/query/catalog/src/merge_into_join.rs @@ -22,9 +22,14 @@ pub enum MergeIntoJoinType { NormalJoin, } +// for now, we just support MergeIntoJoinType::Left to use MergeIntoBlockInfoHashTable in two situations: +// 1. distributed broadcast join and target table as build side. +// 2. in standalone mode and target table as build side. +// we will support Inner next, so the merge_into_join_type is only Left for current implementation in fact. pub struct MergeIntoJoin { - merge_into_join_type: MergeIntoJoinType, - is_distributed: bool, + pub merge_into_join_type: MergeIntoJoinType, + pub is_distributed: bool, + pub target_tbl_idx: usize, } impl Default for MergeIntoJoin { @@ -32,6 +37,7 @@ impl Default for MergeIntoJoin { Self { merge_into_join_type: MergeIntoJoinType::NormalJoin, is_distributed: Default::default(), + target_tbl_idx: Default::default(), } } } diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/build_state.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/build_state.rs index be9e050b8886..bddb87c0cf2a 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/build_state.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/build_state.rs @@ -42,7 +42,7 @@ pub struct BuildBlockGenerationState { // we converted all chunks into ColumnVec for every column. pub(crate) build_columns: Vec, pub(crate) build_columns_data_type: Vec, - // after projectd by build_projection, whether we still have data. + // after projected by build_projection, whether we still have data. pub(crate) is_build_projected: bool, } diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/desc.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/desc.rs index 701c35e39e32..ced2fe9dbe46 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/desc.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/desc.rs @@ -36,7 +36,7 @@ pub struct HashJoinDesc { pub(crate) probe_keys: Vec, pub(crate) join_type: JoinType, /// when we have non-equal conditions for hash join, - /// for exmaple `a = b and c = d`, we will use `and_filters` + /// for example `a = b and c = d`, we will use `and_filters` /// to wrap them as a other_predicate to do next step's check. pub(crate) other_predicate: Option, pub(crate) marker_join_desc: MarkJoinDesc, diff --git a/src/query/service/tests/it/sql/exec/get_table_bind_test.rs b/src/query/service/tests/it/sql/exec/get_table_bind_test.rs index cc8e313ef5fc..fa31f97b6f8f 100644 --- a/src/query/service/tests/it/sql/exec/get_table_bind_test.rs +++ b/src/query/service/tests/it/sql/exec/get_table_bind_test.rs @@ -11,7 +11,6 @@ // 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::HashMap; use std::collections::HashSet; @@ -25,6 +24,7 @@ use databend_common_base::base::ProgressValues; use databend_common_catalog::catalog::Catalog; use databend_common_catalog::cluster_info::Cluster; use databend_common_catalog::database::Database; +use databend_common_catalog::merge_into_join::MergeIntoJoin; use databend_common_catalog::plan::DataSourcePlan; use databend_common_catalog::plan::PartInfoPtr; use databend_common_catalog::plan::Partitions; @@ -749,6 +749,10 @@ impl TableContext for CtxDelegation { fn get_query_profiles(&self) -> Vec { todo!() } + fn set_merge_into_join_type(&self, _join: MergeIntoJoin) { + todo!() + } + fn set_runtime_filter(&self, _filters: (IndexType, RuntimeFilterInfo)) { todo!() } diff --git a/src/query/service/tests/it/storages/fuse/operations/commit.rs b/src/query/service/tests/it/storages/fuse/operations/commit.rs index 8d5d2221481b..134fbaf83695 100644 --- a/src/query/service/tests/it/storages/fuse/operations/commit.rs +++ b/src/query/service/tests/it/storages/fuse/operations/commit.rs @@ -701,7 +701,7 @@ impl TableContext for CtxDelegation { todo!() } - fn set_merge_into_join_type(&self, join: MergeIntoJoin) { + fn set_merge_into_join_type(&self, _join: MergeIntoJoin) { todo!() } diff --git a/src/query/sql/src/planner/optimizer/optimizer.rs b/src/query/sql/src/planner/optimizer/optimizer.rs index fcb86e068348..3d7441ca08c4 100644 --- a/src/query/sql/src/planner/optimizer/optimizer.rs +++ b/src/query/sql/src/planner/optimizer/optimizer.rs @@ -16,6 +16,8 @@ use std::collections::HashMap; use std::sync::Arc; use databend_common_ast::ast::ExplainKind; +use databend_common_catalog::merge_into_join::MergeIntoJoin; +use databend_common_catalog::merge_into_join::MergeIntoJoinType; use databend_common_catalog::table_context::TableContext; use databend_common_exception::ErrorCode; use databend_common_exception::Result; @@ -26,6 +28,7 @@ use super::cost::CostContext; use super::distributed::MergeSourceOptimizer; use super::format::display_memo; use super::Memo; +use crate::binder::MergeIntoType; use crate::optimizer::cascades::CascadesOptimizer; use crate::optimizer::decorrelate::decorrelate_subquery; use crate::optimizer::distributed::optimize_distributed_query; @@ -322,6 +325,14 @@ fn optimize_merge_into(opt_ctx: OptimizerContext, plan: Box) -> Resul false }; + // we just support left join to use MergeIntoBlockInfoHashTable + if change_join_order && matches!(plan.merge_type, MergeIntoType::FullOperation) { + opt_ctx.table_ctx.set_merge_into_join_type(MergeIntoJoin { + merge_into_join_type: MergeIntoJoinType::Left, + is_distributed: false, + target_tbl_idx: plan.target_table_idx, + }) + } // try to optimize distributed join, only if // - distributed optimization is enabled // - no local table scan @@ -349,7 +360,13 @@ fn optimize_merge_into(opt_ctx: OptimizerContext, plan: Box) -> Resul || change_join_order { // we need to judge whether it'a broadcast join to support runtime filter. - merge_into_join_sexpr = try_to_change_as_broadcast_join(merge_into_join_sexpr)?; + merge_into_join_sexpr = try_to_change_as_broadcast_join( + merge_into_join_sexpr, + change_join_order, + opt_ctx.table_ctx.clone(), + plan.merge_type.clone(), + plan.target_table_idx, + )?; (merge_into_join_sexpr.clone(), false) } else { ( @@ -373,7 +390,13 @@ fn optimize_merge_into(opt_ctx: OptimizerContext, plan: Box) -> Resul } } -fn try_to_change_as_broadcast_join(merge_into_join_sexpr: SExpr) -> Result { +fn try_to_change_as_broadcast_join( + merge_into_join_sexpr: SExpr, + change_join_order: bool, + table_ctx: Arc, + merge_into_type: MergeIntoType, + target_tbl_idx: usize, +) -> Result { if let RelOperator::Exchange(Exchange::Merge) = merge_into_join_sexpr.plan.as_ref() { let right_exchange = merge_into_join_sexpr.child(0)?.child(1)?; if let RelOperator::Exchange(Exchange::Broadcast) = right_exchange.plan.as_ref() { @@ -382,6 +405,15 @@ fn try_to_change_as_broadcast_join(merge_into_join_sexpr: SExpr) -> Result Date: Tue, 9 Jan 2024 21:28:30 +0800 Subject: [PATCH 07/47] add RowPrefix for native_deserialize and parquet_deserialize --- src/query/catalog/src/merge_into_join.rs | 2 +- src/query/catalog/src/table_context.rs | 4 +- .../processors/transforms/hash_join/desc.rs | 4 +- src/query/service/src/sessions/query_ctx.rs | 11 +++- .../tests/it/sql/exec/get_table_bind_test.rs | 6 +- .../it/storages/fuse/operations/commit.rs | 6 +- .../sql/src/planner/optimizer/optimizer.rs | 16 +++-- .../storages/common/pruner/src/block_meta.rs | 22 +++++++ src/query/storages/common/pruner/src/lib.rs | 2 + .../io/read/block/block_reader_deserialize.rs | 1 + .../storages/fuse/src/operations/read/mod.rs | 2 +- .../read/native_data_source_deserializer.rs | 24 ++++++-- .../read/parquet_data_source_deserializer.rs | 14 ++++- .../read/parquet_data_source_reader.rs | 7 ++- .../storages/fuse/src/operations/read/util.rs | 58 +++++++++++++++++++ 15 files changed, 159 insertions(+), 20 deletions(-) create mode 100644 src/query/storages/fuse/src/operations/read/util.rs diff --git a/src/query/catalog/src/merge_into_join.rs b/src/query/catalog/src/merge_into_join.rs index 26a55c438c6e..dd1190b810d6 100644 --- a/src/query/catalog/src/merge_into_join.rs +++ b/src/query/catalog/src/merge_into_join.rs @@ -11,7 +11,7 @@ // 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. - +#[derive(Clone)] pub enum MergeIntoJoinType { Left, Right, diff --git a/src/query/catalog/src/table_context.rs b/src/query/catalog/src/table_context.rs index 57611ee71a12..c62aa0d3b706 100644 --- a/src/query/catalog/src/table_context.rs +++ b/src/query/catalog/src/table_context.rs @@ -244,7 +244,9 @@ pub trait TableContext: Send + Sync { fn set_runtime_filter(&self, filters: (usize, RuntimeFilterInfo)); - fn set_merge_into_join_type(&self, join: MergeIntoJoin); + fn set_merge_into_join(&self, join: MergeIntoJoin); + + fn get_merge_into_join(&self) -> MergeIntoJoin; fn get_bloom_runtime_filter_with_id(&self, id: usize) -> Vec<(String, BinaryFuse16)>; diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/desc.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/desc.rs index ced2fe9dbe46..1d02554e2545 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/desc.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/desc.rs @@ -36,8 +36,8 @@ pub struct HashJoinDesc { pub(crate) probe_keys: Vec, pub(crate) join_type: JoinType, /// when we have non-equal conditions for hash join, - /// for example `a = b and c = d`, we will use `and_filters` - /// to wrap them as a other_predicate to do next step's check. + /// for example `a = b and c = d and e > f`, we will use `and_filters` + /// to wrap `e > f` as a other_predicate to do next step's check. pub(crate) other_predicate: Option, pub(crate) marker_join_desc: MarkJoinDesc, /// Whether the Join are derived from correlated subquery. diff --git a/src/query/service/src/sessions/query_ctx.rs b/src/query/service/src/sessions/query_ctx.rs index 64e13c86b360..7c6b1e98de5f 100644 --- a/src/query/service/src/sessions/query_ctx.rs +++ b/src/query/service/src/sessions/query_ctx.rs @@ -911,7 +911,7 @@ impl TableContext for QueryContext { queries_profile } - fn set_merge_into_join_type(&self, join: MergeIntoJoin) { + fn set_merge_into_join(&self, join: MergeIntoJoin) { let mut merge_into_join = self.shared.merge_into_join.write(); *merge_into_join = join; } @@ -936,6 +936,15 @@ impl TableContext for QueryContext { } } + fn get_merge_into_join(&self) -> MergeIntoJoin { + let merge_into_join = self.shared.merge_into_join.read(); + MergeIntoJoin { + merge_into_join_type: merge_into_join.merge_into_join_type.clone(), + is_distributed: merge_into_join.is_distributed, + target_tbl_idx: merge_into_join.target_tbl_idx, + } + } + fn get_bloom_runtime_filter_with_id(&self, id: IndexType) -> Vec<(String, BinaryFuse16)> { let runtime_filters = self.shared.runtime_filters.read(); match runtime_filters.get(&id) { diff --git a/src/query/service/tests/it/sql/exec/get_table_bind_test.rs b/src/query/service/tests/it/sql/exec/get_table_bind_test.rs index fa31f97b6f8f..144fdd62d087 100644 --- a/src/query/service/tests/it/sql/exec/get_table_bind_test.rs +++ b/src/query/service/tests/it/sql/exec/get_table_bind_test.rs @@ -749,7 +749,11 @@ impl TableContext for CtxDelegation { fn get_query_profiles(&self) -> Vec { todo!() } - fn set_merge_into_join_type(&self, _join: MergeIntoJoin) { + fn set_merge_into_join(&self, _join: MergeIntoJoin) { + todo!() + } + + fn get_merge_into_join(&self) -> MergeIntoJoin { todo!() } diff --git a/src/query/service/tests/it/storages/fuse/operations/commit.rs b/src/query/service/tests/it/storages/fuse/operations/commit.rs index 134fbaf83695..94554d64a5ad 100644 --- a/src/query/service/tests/it/storages/fuse/operations/commit.rs +++ b/src/query/service/tests/it/storages/fuse/operations/commit.rs @@ -701,7 +701,11 @@ impl TableContext for CtxDelegation { todo!() } - fn set_merge_into_join_type(&self, _join: MergeIntoJoin) { + fn set_merge_into_join(&self, _join: MergeIntoJoin) { + todo!() + } + + fn get_merge_into_join(&self) -> MergeIntoJoin { todo!() } diff --git a/src/query/sql/src/planner/optimizer/optimizer.rs b/src/query/sql/src/planner/optimizer/optimizer.rs index 3d7441ca08c4..96888cb7f209 100644 --- a/src/query/sql/src/planner/optimizer/optimizer.rs +++ b/src/query/sql/src/planner/optimizer/optimizer.rs @@ -325,9 +325,17 @@ fn optimize_merge_into(opt_ctx: OptimizerContext, plan: Box) -> Resul false }; - // we just support left join to use MergeIntoBlockInfoHashTable - if change_join_order && matches!(plan.merge_type, MergeIntoType::FullOperation) { - opt_ctx.table_ctx.set_merge_into_join_type(MergeIntoJoin { + // we just support left join to use MergeIntoBlockInfoHashTable, we + // don't support spill for now. + if change_join_order + && matches!(plan.merge_type, MergeIntoType::FullOperation) + && opt_ctx + .table_ctx + .get_settings() + .get_join_spilling_threshold()? + == 0 + { + opt_ctx.table_ctx.set_merge_into_join(MergeIntoJoin { merge_into_join_type: MergeIntoJoinType::Left, is_distributed: false, target_tbl_idx: plan.target_table_idx, @@ -408,7 +416,7 @@ fn try_to_change_as_broadcast_join( // for now, when we use target table as build side and it's a broadcast join, // we will use merge_into_block_info_hashtable to reduce i/o operations. if change_join_order && matches!(merge_into_type, MergeIntoType::FullOperation) { - table_ctx.set_merge_into_join_type(MergeIntoJoin { + table_ctx.set_merge_into_join(MergeIntoJoin { merge_into_join_type: MergeIntoJoinType::Left, is_distributed: true, target_tbl_idx, diff --git a/src/query/storages/common/pruner/src/block_meta.rs b/src/query/storages/common/pruner/src/block_meta.rs index 901e743c2df7..f5362f93c16b 100644 --- a/src/query/storages/common/pruner/src/block_meta.rs +++ b/src/query/storages/common/pruner/src/block_meta.rs @@ -56,3 +56,25 @@ impl BlockMetaIndex { }) } } + +#[derive(serde::Serialize, serde::Deserialize, Clone, Debug, Default, PartialEq, Eq)] +pub struct RowPrefix { + // segment_idx_block_id + pub prefix: u64, + pub inner: Option, +} + +#[typetag::serde(name = "row_prefix")] +impl BlockMetaInfo for RowPrefix { + fn equals(&self, info: &Box) -> bool { + RowPrefix::downcast_ref_from(info).is_some_and(|other| self == other) + } + + fn clone_self(&self) -> Box { + Box::new(self.clone()) + } +} + +pub fn gen_row_prefix(inner: Option, prefix: u64) -> RowPrefix { + RowPrefix { prefix, inner } +} diff --git a/src/query/storages/common/pruner/src/lib.rs b/src/query/storages/common/pruner/src/lib.rs index 04bc0341fc43..b55adb6f89f4 100644 --- a/src/query/storages/common/pruner/src/lib.rs +++ b/src/query/storages/common/pruner/src/lib.rs @@ -21,7 +21,9 @@ mod page_pruner; mod range_pruner; mod topn_pruner; +pub use block_meta::gen_row_prefix; pub use block_meta::BlockMetaIndex; +pub use block_meta::RowPrefix; pub use internal_column_pruner::InternalColumnPruner; pub use limiter_pruner::Limiter; pub use limiter_pruner::LimiterPruner; diff --git a/src/query/storages/fuse/src/io/read/block/block_reader_deserialize.rs b/src/query/storages/fuse/src/io/read/block/block_reader_deserialize.rs index 0b526abe33d0..096a015ec11d 100644 --- a/src/query/storages/fuse/src/io/read/block/block_reader_deserialize.rs +++ b/src/query/storages/fuse/src/io/read/block/block_reader_deserialize.rs @@ -58,6 +58,7 @@ impl BlockReader { storage_format: &FuseStorageFormat, ) -> Result { let part = FusePartInfo::from_part(&part)?; + self.deserialize_chunks( &part.location, part.nums_rows, diff --git a/src/query/storages/fuse/src/operations/read/mod.rs b/src/query/storages/fuse/src/operations/read/mod.rs index c619d474f657..08468b855739 100644 --- a/src/query/storages/fuse/src/operations/read/mod.rs +++ b/src/query/storages/fuse/src/operations/read/mod.rs @@ -25,7 +25,7 @@ mod parquet_rows_fetcher; mod runtime_filter_prunner; mod data_source_with_meta; - +mod util; pub use fuse_rows_fetcher::build_row_fetcher_pipeline; pub use fuse_source::build_fuse_parquet_source_pipeline; pub use native_data_source_deserializer::NativeDeserializeDataTransform; diff --git a/src/query/storages/fuse/src/operations/read/native_data_source_deserializer.rs b/src/query/storages/fuse/src/operations/read/native_data_source_deserializer.rs index 04eca2224fdc..196cbdbc7a85 100644 --- a/src/query/storages/fuse/src/operations/read/native_data_source_deserializer.rs +++ b/src/query/storages/fuse/src/operations/read/native_data_source_deserializer.rs @@ -25,6 +25,7 @@ use databend_common_arrow::native::read::ArrayIter; use databend_common_arrow::parquet::metadata::ColumnDescriptor; use databend_common_base::base::Progress; use databend_common_base::base::ProgressValues; +use databend_common_catalog::plan::compute_row_id_prefix; use databend_common_catalog::plan::gen_mutation_stream_meta; use databend_common_catalog::plan::DataSourcePlan; use databend_common_catalog::plan::PartInfoPtr; @@ -61,16 +62,19 @@ use databend_common_pipeline_core::processors::OutputPort; use databend_common_pipeline_core::processors::Processor; use databend_common_pipeline_core::processors::ProcessorPtr; use databend_common_sql::IndexType; +use databend_storages_common_pruner::gen_row_prefix; use xorf::BinaryFuse16; use super::fuse_source::fill_internal_column_meta; use super::native_data_source::NativeDataSource; +use super::util::need_reserve_block_info; use crate::fuse_part::FusePartInfo; use crate::io::AggIndexReader; use crate::io::BlockReader; use crate::io::VirtualColumnReader; use crate::operations::read::data_source_with_meta::DataSourceWithMeta; use crate::operations::read::runtime_filter_prunner::update_bitmap_with_bloom_filter; +use crate::operations::read::util::add_row_prefix_meta; use crate::DEFAULT_ROW_PER_PAGE; /// A helper struct to store the intermediate state while reading a native partition. @@ -235,6 +239,9 @@ pub struct NativeDeserializeDataTransform { /// Record how many sets of pages have been skipped. /// It's used for metrics. skipped_pages: usize, + + // for merge_into target build. + need_reserve_block_info: bool, } impl NativeDeserializeDataTransform { @@ -250,7 +257,7 @@ impl NativeDeserializeDataTransform { virtual_reader: Arc>, ) -> Result { let scan_progress = ctx.get_scan_progress(); - + let need_reserve_block_info = need_reserve_block_info(ctx.clone(), plan.table_index); let mut src_schema: DataSchema = (block_reader.schema().as_ref()).into(); let mut prewhere_columns: Vec = @@ -381,6 +388,7 @@ impl NativeDeserializeDataTransform { base_block_ids: plan.base_block_ids.clone(), bloom_runtime_filter: None, read_state: ReadPartState::new(), + need_reserve_block_info, }, ))) } @@ -946,7 +954,9 @@ impl NativeDeserializeDataTransform { // All columns are default values, not need to read. let part = self.parts.front().unwrap(); let fuse_part = FusePartInfo::from_part(part)?; - let block = self.build_default_block(fuse_part)?; + let mut block = self.build_default_block(fuse_part)?; + // for merge into target build + block = add_row_prefix_meta(self.need_reserve_block_info, fuse_part, block)?; self.add_output_block(block); self.finish_partition(); return Ok(()); @@ -968,6 +978,7 @@ impl NativeDeserializeDataTransform { // Fill `InternalColumnMeta` as `DataBlock.meta` if query internal columns, // `TransformAddInternalColumns` will generate internal columns using `InternalColumnMeta` in next pipeline. let mut block = block.resort(&self.src_schema, &self.output_schema)?; + let fuse_part = FusePartInfo::from_part(&self.parts[0])?; if self.block_reader.query_internal_columns() { let offset = self.read_state.offset; let offsets = if let Some(count) = self.read_state.filtered_count { @@ -980,7 +991,6 @@ impl NativeDeserializeDataTransform { (offset..offset + origin_num_rows).collect() }; - let fuse_part = FusePartInfo::from_part(&self.parts[0])?; block = fill_internal_column_meta( block, fuse_part, @@ -989,13 +999,16 @@ impl NativeDeserializeDataTransform { )?; } + // we will do recluster for stream here. if self.block_reader.update_stream_columns() { let inner_meta = block.take_meta(); - let fuse_part = FusePartInfo::from_part(&self.parts[0])?; let meta = gen_mutation_stream_meta(inner_meta, &fuse_part.location)?; block = block.add_meta(Some(Box::new(meta)))?; } + // for merge into target build + block = add_row_prefix_meta(self.need_reserve_block_info, fuse_part, block)?; + self.read_state.offset += origin_num_rows; Ok(block) @@ -1088,6 +1101,9 @@ impl Processor for NativeDeserializeDataTransform { self.base_block_ids.clone(), )?; } + data_block = + add_row_prefix_meta(self.need_reserve_block_info, fuse_part, data_block)?; + self.finish_partition(); self.add_output_block(data_block); return Ok(()); diff --git a/src/query/storages/fuse/src/operations/read/parquet_data_source_deserializer.rs b/src/query/storages/fuse/src/operations/read/parquet_data_source_deserializer.rs index 4d732af5fa92..404be8f04e36 100644 --- a/src/query/storages/fuse/src/operations/read/parquet_data_source_deserializer.rs +++ b/src/query/storages/fuse/src/operations/read/parquet_data_source_deserializer.rs @@ -21,6 +21,7 @@ use databend_common_arrow::arrow::bitmap::Bitmap; use databend_common_arrow::arrow::bitmap::MutableBitmap; use databend_common_base::base::Progress; use databend_common_base::base::ProgressValues; +use databend_common_catalog::plan::compute_row_id_prefix; use databend_common_catalog::plan::gen_mutation_stream_meta; use databend_common_catalog::plan::DataSourcePlan; use databend_common_catalog::plan::PartInfoPtr; @@ -40,10 +41,13 @@ use databend_common_pipeline_core::processors::OutputPort; use databend_common_pipeline_core::processors::Processor; use databend_common_pipeline_core::processors::ProcessorPtr; use databend_common_sql::IndexType; +use databend_storages_common_pruner::gen_row_prefix; use xorf::BinaryFuse16; use super::fuse_source::fill_internal_column_meta; use super::parquet_data_source::ParquetDataSource; +use super::util::add_row_prefix_meta; +use super::util::need_reserve_block_info; use crate::fuse_part::FusePartInfo; use crate::io::AggIndexReader; use crate::io::BlockReader; @@ -72,6 +76,8 @@ pub struct DeserializeDataTransform { base_block_ids: Option, cached_runtime_filter: Option>, + // for merge_into target build. + need_reserve_block_info: bool, } unsafe impl Send for DeserializeDataTransform {} @@ -105,7 +111,7 @@ impl DeserializeDataTransform { let mut output_schema = plan.schema().as_ref().clone(); output_schema.remove_internal_fields(); let output_schema: DataSchema = (&output_schema).into(); - + let need_reserve_block_info = need_reserve_block_info(ctx.clone(), plan.table_index); Ok(ProcessorPtr::create(Box::new(DeserializeDataTransform { ctx, table_index: plan.table_index, @@ -123,6 +129,7 @@ impl DeserializeDataTransform { virtual_reader, base_block_ids: plan.base_block_ids.clone(), cached_runtime_filter: None, + need_reserve_block_info, }))) } @@ -314,12 +321,17 @@ impl Processor for DeserializeDataTransform { )?; } + // we will do recluster for stream here. if self.block_reader.update_stream_columns() { let inner_meta = data_block.take_meta(); let meta = gen_mutation_stream_meta(inner_meta, &part.location)?; data_block = data_block.add_meta(Some(Box::new(meta)))?; } + // for merge into target build + data_block = + add_row_prefix_meta(self.need_reserve_block_info, part, data_block)?; + self.output_data = Some(data_block); } } diff --git a/src/query/storages/fuse/src/operations/read/parquet_data_source_reader.rs b/src/query/storages/fuse/src/operations/read/parquet_data_source_reader.rs index e132e8cd009c..e7cc841b0ff4 100644 --- a/src/query/storages/fuse/src/operations/read/parquet_data_source_reader.rs +++ b/src/query/storages/fuse/src/operations/read/parquet_data_source_reader.rs @@ -33,6 +33,7 @@ use databend_common_pipeline_sources::SyncSourcer; use databend_common_sql::IndexType; use super::parquet_data_source::ParquetDataSource; +use super::util::need_reserve_block_info; use crate::fuse_part::FusePartInfo; use crate::io::AggIndexReader; use crate::io::BlockReader; @@ -75,6 +76,7 @@ impl ReadParquetDataSource { ) -> Result { let batch_size = ctx.get_settings().get_storage_fetch_part_num()? as usize; let func_ctx = ctx.get_function_context()?; + let need_reserve_block_info = need_reserve_block_info(ctx.clone(), table_index); if BLOCKING_IO { SyncSourcer::create(ctx.clone(), output.clone(), ReadParquetDataSource:: { func_ctx, @@ -176,7 +178,7 @@ impl SyncSource for ReadParquetDataSource { &None }; - let source = self.block_reader.sync_read_columns_data_by_merge_io( + let mut source = self.block_reader.sync_read_columns_data_by_merge_io( &ReadSettings::from_ctx(&self.partitions.ctx)?, &part, ignore_column_ids, @@ -242,7 +244,6 @@ impl Processor for ReadParquetDataSource { .ctx .get_min_max_runtime_filter_with_id(self.table_index), ); - let mut fuse_part_infos = Vec::with_capacity(parts.len()); for part in parts.into_iter() { if runtime_filter_pruner( @@ -298,7 +299,7 @@ impl Processor for ReadParquetDataSource { &None }; - let source = block_reader + let mut source = block_reader .read_columns_data_by_merge_io( &settings, &part.location, diff --git a/src/query/storages/fuse/src/operations/read/util.rs b/src/query/storages/fuse/src/operations/read/util.rs new file mode 100644 index 000000000000..ac0061e0785d --- /dev/null +++ b/src/query/storages/fuse/src/operations/read/util.rs @@ -0,0 +1,58 @@ +// 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 databend_common_catalog::merge_into_join::MergeIntoJoinType; +use databend_common_catalog::plan::compute_row_id_prefix; +use databend_common_catalog::table_context::TableContext; +use databend_common_exception::Result; +use databend_common_expression::DataBlock; +use databend_storages_common_pruner::gen_row_prefix; + +use crate::FusePartInfo; + +pub(crate) fn need_reserve_block_info(ctx: Arc, table_idx: usize) -> bool { + let merge_into_join = ctx.get_merge_into_join(); + if matches!( + merge_into_join.merge_into_join_type, + MergeIntoJoinType::Left + ) && merge_into_join.target_tbl_idx == table_idx + { + true + } else { + false + } +} + +// for merge into target build, in this situation, we don't need rowid +pub(crate) fn add_row_prefix_meta( + need_reserve_block_info: bool, + fuse_part: &FusePartInfo, + mut block: DataBlock, +) -> Result { + if need_reserve_block_info && fuse_part.block_meta_index.is_some() { + let block_meta_index = fuse_part.block_meta_index.as_ref().unwrap(); + let prefix = compute_row_id_prefix( + block_meta_index.segment_idx as u64, + block_meta_index.block_id as u64, + ); + // in fact, inner_meta is none for now, for merge into target build, we don't need + // to get row_id. + let inner_meta = block.take_meta(); + block.add_meta(Some(Box::new(gen_row_prefix(inner_meta, prefix)))) + } else { + Ok(block) + } +} From c4bdedbbf3dc8bc2ffd42049989e11b5748b00b6 Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Tue, 9 Jan 2024 21:56:22 +0800 Subject: [PATCH 08/47] fix lint --- .../src/operations/read/native_data_source_deserializer.rs | 2 -- .../operations/read/parquet_data_source_deserializer.rs | 2 -- .../fuse/src/operations/read/parquet_data_source_reader.rs | 6 ++---- src/query/storages/fuse/src/operations/read/util.rs | 7 +------ 4 files changed, 3 insertions(+), 14 deletions(-) diff --git a/src/query/storages/fuse/src/operations/read/native_data_source_deserializer.rs b/src/query/storages/fuse/src/operations/read/native_data_source_deserializer.rs index 196cbdbc7a85..6d20632caa93 100644 --- a/src/query/storages/fuse/src/operations/read/native_data_source_deserializer.rs +++ b/src/query/storages/fuse/src/operations/read/native_data_source_deserializer.rs @@ -25,7 +25,6 @@ use databend_common_arrow::native::read::ArrayIter; use databend_common_arrow::parquet::metadata::ColumnDescriptor; use databend_common_base::base::Progress; use databend_common_base::base::ProgressValues; -use databend_common_catalog::plan::compute_row_id_prefix; use databend_common_catalog::plan::gen_mutation_stream_meta; use databend_common_catalog::plan::DataSourcePlan; use databend_common_catalog::plan::PartInfoPtr; @@ -62,7 +61,6 @@ use databend_common_pipeline_core::processors::OutputPort; use databend_common_pipeline_core::processors::Processor; use databend_common_pipeline_core::processors::ProcessorPtr; use databend_common_sql::IndexType; -use databend_storages_common_pruner::gen_row_prefix; use xorf::BinaryFuse16; use super::fuse_source::fill_internal_column_meta; diff --git a/src/query/storages/fuse/src/operations/read/parquet_data_source_deserializer.rs b/src/query/storages/fuse/src/operations/read/parquet_data_source_deserializer.rs index 404be8f04e36..0e011ca428bb 100644 --- a/src/query/storages/fuse/src/operations/read/parquet_data_source_deserializer.rs +++ b/src/query/storages/fuse/src/operations/read/parquet_data_source_deserializer.rs @@ -21,7 +21,6 @@ use databend_common_arrow::arrow::bitmap::Bitmap; use databend_common_arrow::arrow::bitmap::MutableBitmap; use databend_common_base::base::Progress; use databend_common_base::base::ProgressValues; -use databend_common_catalog::plan::compute_row_id_prefix; use databend_common_catalog::plan::gen_mutation_stream_meta; use databend_common_catalog::plan::DataSourcePlan; use databend_common_catalog::plan::PartInfoPtr; @@ -41,7 +40,6 @@ use databend_common_pipeline_core::processors::OutputPort; use databend_common_pipeline_core::processors::Processor; use databend_common_pipeline_core::processors::ProcessorPtr; use databend_common_sql::IndexType; -use databend_storages_common_pruner::gen_row_prefix; use xorf::BinaryFuse16; use super::fuse_source::fill_internal_column_meta; diff --git a/src/query/storages/fuse/src/operations/read/parquet_data_source_reader.rs b/src/query/storages/fuse/src/operations/read/parquet_data_source_reader.rs index e7cc841b0ff4..dbc0f4baf1b9 100644 --- a/src/query/storages/fuse/src/operations/read/parquet_data_source_reader.rs +++ b/src/query/storages/fuse/src/operations/read/parquet_data_source_reader.rs @@ -33,7 +33,6 @@ use databend_common_pipeline_sources::SyncSourcer; use databend_common_sql::IndexType; use super::parquet_data_source::ParquetDataSource; -use super::util::need_reserve_block_info; use crate::fuse_part::FusePartInfo; use crate::io::AggIndexReader; use crate::io::BlockReader; @@ -76,7 +75,6 @@ impl ReadParquetDataSource { ) -> Result { let batch_size = ctx.get_settings().get_storage_fetch_part_num()? as usize; let func_ctx = ctx.get_function_context()?; - let need_reserve_block_info = need_reserve_block_info(ctx.clone(), table_index); if BLOCKING_IO { SyncSourcer::create(ctx.clone(), output.clone(), ReadParquetDataSource:: { func_ctx, @@ -178,7 +176,7 @@ impl SyncSource for ReadParquetDataSource { &None }; - let mut source = self.block_reader.sync_read_columns_data_by_merge_io( + let source = self.block_reader.sync_read_columns_data_by_merge_io( &ReadSettings::from_ctx(&self.partitions.ctx)?, &part, ignore_column_ids, @@ -299,7 +297,7 @@ impl Processor for ReadParquetDataSource { &None }; - let mut source = block_reader + let source = block_reader .read_columns_data_by_merge_io( &settings, &part.location, diff --git a/src/query/storages/fuse/src/operations/read/util.rs b/src/query/storages/fuse/src/operations/read/util.rs index ac0061e0785d..5e34f613d506 100644 --- a/src/query/storages/fuse/src/operations/read/util.rs +++ b/src/query/storages/fuse/src/operations/read/util.rs @@ -25,15 +25,10 @@ use crate::FusePartInfo; pub(crate) fn need_reserve_block_info(ctx: Arc, table_idx: usize) -> bool { let merge_into_join = ctx.get_merge_into_join(); - if matches!( + matches!( merge_into_join.merge_into_join_type, MergeIntoJoinType::Left ) && merge_into_join.target_tbl_idx == table_idx - { - true - } else { - false - } } // for merge into target build, in this situation, we don't need rowid From 81c47bb26b0596c2f5d6282c1019a421cc5a5eb7 Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Wed, 10 Jan 2024 00:37:01 +0800 Subject: [PATCH 09/47] add gather_partial_modified and reduce_false_matched --- .../src/hashjoin_block_info_hashtable.rs | 23 ++++++++++++-- .../hashjoin_block_info_string_hashtable.rs | 31 +++++++++++++++++-- .../hashtable/src/hashjoin_hashtable.rs | 11 +++++++ .../src/hashjoin_string_hashtable.rs | 11 +++++++ src/common/hashtable/src/traits.rs | 7 +++++ 5 files changed, 79 insertions(+), 4 deletions(-) diff --git a/src/common/hashtable/src/hashjoin_block_info_hashtable.rs b/src/common/hashtable/src/hashjoin_block_info_hashtable.rs index a3f099a88fd0..37c823e4f6d6 100644 --- a/src/common/hashtable/src/hashjoin_block_info_hashtable.rs +++ b/src/common/hashtable/src/hashjoin_block_info_hashtable.rs @@ -26,6 +26,8 @@ use crate::hashjoin_hashtable::early_filtering; use crate::hashjoin_hashtable::hash_bits; use crate::hashjoin_hashtable::new_header; use crate::hashjoin_hashtable::remove_header_tag; +use crate::utils::BlockInfoIndex; +use crate::utils::Interval; use crate::HashJoinHashtableLike; use crate::RawEntry; use crate::RowPtr; @@ -45,7 +47,9 @@ pub struct HashJoinBlockInfoHashTable, + pub(crate) block_info_index: BlockInfoIndex, pub(crate) phantom: PhantomData, } @@ -60,7 +64,11 @@ unsafe impl Sync } impl HashJoinBlockInfoHashTable { - pub fn with_build_row_num(row_num: usize) -> Self { + pub fn with_build_row_num( + row_num: usize, + is_distributed: bool, + block_info_index: BlockInfoIndex, + ) -> Self { let capacity = std::cmp::max((row_num * 2).next_power_of_two(), 1 << 10); let mut hashtable = Self { pointers: unsafe { @@ -69,8 +77,9 @@ impl HashJoinBlockInfoHashTable(hashtable.pointers.as_mut_ptr()) @@ -299,4 +308,14 @@ where (0, 0) } } + + // for merge into block info hash table + fn gather_partial_modified_block(&self) -> (Interval, u64) { + unreachable!() + } + + // for merge into block info hash table + fn reduce_false_matched_for_conjuct(&mut self) { + unreachable!() + } } diff --git a/src/common/hashtable/src/hashjoin_block_info_string_hashtable.rs b/src/common/hashtable/src/hashjoin_block_info_string_hashtable.rs index 1dfa2d0ff985..5f861dcc7b59 100644 --- a/src/common/hashtable/src/hashjoin_block_info_string_hashtable.rs +++ b/src/common/hashtable/src/hashjoin_block_info_string_hashtable.rs @@ -25,11 +25,22 @@ use crate::hashjoin_hashtable::early_filtering; use crate::hashjoin_hashtable::hash_bits; use crate::hashjoin_hashtable::new_header; use crate::hashjoin_hashtable::remove_header_tag; +use crate::utils::BlockInfoIndex; +use crate::utils::Interval; use crate::HashJoinHashtableLike; use crate::RowPtr; use crate::StringRawEntry; use crate::STRING_EARLY_SIZE; +// This hashtable is only used for target build merge into (both standalone and distributed mode). +// Advantages: +// 1. Reduces redundant I/O operations, enhancing performance. +// 2. Lowers the maintenance overhead of deduplicating row_id.(But in distributed design, we also need to give rowid) +// 3. Allows the scheduling of the subsequent mutation pipeline to be entirely allocated to not matched append operations. +// Disadvantages: +// 1. This solution is likely to be a one-time approach (especially if there are not matched insert operations involved), +// potentially leading to the target table being unsuitable for use as a build table in the future. +// 2. Requires a significant amount of memory to be efficient and currently does not support spill operations. #[allow(unused)] pub struct HashJoinBlockInfoStringHashTable { pub(crate) pointers: Box<[u64], A>, @@ -37,6 +48,7 @@ pub struct HashJoinBlockInfoStringHashTable, + pub(crate) block_info_index: BlockInfoIndex, } unsafe impl Send for HashJoinBlockInfoStringHashTable {} @@ -44,7 +56,11 @@ unsafe impl Send for HashJoinBlockInfoStringHashTab unsafe impl Sync for HashJoinBlockInfoStringHashTable {} impl HashJoinBlockInfoStringHashTable { - pub fn with_build_row_num(row_num: usize) -> Self { + pub fn with_build_row_num( + row_num: usize, + is_distributed: bool, + block_info_index: BlockInfoIndex, + ) -> Self { let capacity = std::cmp::max((row_num * 2).next_power_of_two(), 1 << 10); let mut hashtable = Self { pointers: unsafe { @@ -52,8 +68,9 @@ impl HashJoinBlockInfoStringHashTable { }, atomic_pointers: std::ptr::null_mut(), hash_shift: (hash_bits() - capacity.trailing_zeros()) as usize, - is_distributed: false, + is_distributed, matched: unsafe { Box::new_zeroed_slice_in(row_num, Default::default()).assume_init() }, + block_info_index, }; hashtable.atomic_pointers = unsafe { std::mem::transmute::<*mut u64, *mut AtomicU64>(hashtable.pointers.as_mut_ptr()) @@ -299,4 +316,14 @@ where A: Allocator + Clone + 'static (0, 0) } } + + // for merge into block info hash table + fn gather_partial_modified_block(&self) -> (Interval, u64) { + unreachable!() + } + + // for merge into block info hash table + fn reduce_false_matched_for_conjuct(&mut self) { + unreachable!() + } } diff --git a/src/common/hashtable/src/hashjoin_hashtable.rs b/src/common/hashtable/src/hashjoin_hashtable.rs index 7c126159fa14..4877a917e4ec 100644 --- a/src/common/hashtable/src/hashjoin_hashtable.rs +++ b/src/common/hashtable/src/hashjoin_hashtable.rs @@ -22,6 +22,7 @@ use databend_common_base::mem_allocator::MmapAllocator; use super::traits::HashJoinHashtableLike; use super::traits::Keyable; +use crate::utils::Interval; #[derive(Clone, Copy, Debug)] pub struct RowPtr { @@ -350,4 +351,14 @@ where (0, 0) } } + + // for merge into block info hash table + fn gather_partial_modified_block(&self) -> (Interval, u64) { + unreachable!() + } + + // for merge into block info hash table + fn reduce_false_matched_for_conjuct(&mut self) { + unreachable!() + } } diff --git a/src/common/hashtable/src/hashjoin_string_hashtable.rs b/src/common/hashtable/src/hashjoin_string_hashtable.rs index c7ca141e7f03..3b23a229c32f 100644 --- a/src/common/hashtable/src/hashjoin_string_hashtable.rs +++ b/src/common/hashtable/src/hashjoin_string_hashtable.rs @@ -26,6 +26,7 @@ use crate::hashjoin_hashtable::hash_bits; use crate::hashjoin_hashtable::new_header; use crate::hashjoin_hashtable::remove_header_tag; use crate::traits::hash_join_fast_string_hash; +use crate::utils::Interval; use crate::RowPtr; pub const STRING_EARLY_SIZE: usize = 4; @@ -301,4 +302,14 @@ where A: Allocator + Clone + 'static (0, 0) } } + + // for merge into block info hash table + fn gather_partial_modified_block(&self) -> (Interval, u64) { + unreachable!() + } + + // for merge into block info hash table + fn reduce_false_matched_for_conjuct(&mut self) { + unreachable!() + } } diff --git a/src/common/hashtable/src/traits.rs b/src/common/hashtable/src/traits.rs index 8ee0a99e8ef0..215c41d31f03 100644 --- a/src/common/hashtable/src/traits.rs +++ b/src/common/hashtable/src/traits.rs @@ -25,6 +25,7 @@ use ethnum::i256; use ethnum::U256; use ordered_float::OrderedFloat; +use crate::utils::Interval; use crate::RowPtr; /// # Safety @@ -552,4 +553,10 @@ pub trait HashJoinHashtableLike { occupied: usize, capacity: usize, ) -> (usize, u64); + + // for merge into block info hash table + fn gather_partial_modified_block(&self) -> (Interval, u64); + + // for merge into block info hash table + fn reduce_false_matched_for_conjuct(&mut self); } From 1676466a670736239e969a1c148ea2e6ef16570e Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Wed, 10 Jan 2024 21:07:22 +0800 Subject: [PATCH 10/47] refactor: remove block info hashtable and build blockinfo index outside, add check multirows conflict --- .../src/hashjoin_block_info_hashtable.rs | 321 ----------------- .../hashjoin_block_info_string_hashtable.rs | 329 ------------------ src/common/hashtable/src/lib.rs | 7 +- .../src/pipelines/builders/builder_join.rs | 29 +- .../pipelines/builders/builder_merge_into.rs | 1 + .../builders/builder_replace_into.rs | 1 + .../hash_join/hash_join_build_state.rs | 47 ++- .../transforms/hash_join/hash_join_state.rs | 30 ++ .../hash_join/probe_join/left_join.rs | 53 +++ .../sql/src/planner/optimizer/optimizer.rs | 11 +- .../storages/common/pruner/src/block_meta.rs | 22 -- src/query/storages/common/pruner/src/lib.rs | 2 - .../src/operations/common/mutation_log.rs | 13 + .../merge_into/mutator/matched_mutator.rs | 2 + src/query/storages/fuse/src/operations/mod.rs | 1 + .../mutation/mutator/block_compact_mutator.rs | 1 + .../mutation/processors/mutation_source.rs | 1 + .../storages/fuse/src/operations/read/mod.rs | 1 + .../read/native_data_source_deserializer.rs | 2 +- .../read/parquet_data_source_deserializer.rs | 2 +- .../storages/fuse/src/operations/read/util.rs | 26 +- .../mutator/merge_into_mutator.rs | 2 + 22 files changed, 206 insertions(+), 698 deletions(-) delete mode 100644 src/common/hashtable/src/hashjoin_block_info_hashtable.rs delete mode 100644 src/common/hashtable/src/hashjoin_block_info_string_hashtable.rs diff --git a/src/common/hashtable/src/hashjoin_block_info_hashtable.rs b/src/common/hashtable/src/hashjoin_block_info_hashtable.rs deleted file mode 100644 index 37c823e4f6d6..000000000000 --- a/src/common/hashtable/src/hashjoin_block_info_hashtable.rs +++ /dev/null @@ -1,321 +0,0 @@ -// 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::Allocator; -use std::marker::PhantomData; -use std::sync::atomic::AtomicU64; -use std::sync::atomic::Ordering; - -use databend_common_arrow::arrow::bitmap::Bitmap; -use databend_common_base::mem_allocator::MmapAllocator; - -use super::traits::Keyable; -use crate::hashjoin_hashtable::combine_header; -use crate::hashjoin_hashtable::early_filtering; -use crate::hashjoin_hashtable::hash_bits; -use crate::hashjoin_hashtable::new_header; -use crate::hashjoin_hashtable::remove_header_tag; -use crate::utils::BlockInfoIndex; -use crate::utils::Interval; -use crate::HashJoinHashtableLike; -use crate::RawEntry; -use crate::RowPtr; - -// This hashtable is only used for target build merge into (both standalone and distributed mode). -// Advantages: -// 1. Reduces redundant I/O operations, enhancing performance. -// 2. Lowers the maintenance overhead of deduplicating row_id.(But in distributed design, we also need to give rowid) -// 3. Allows the scheduling of the subsequent mutation pipeline to be entirely allocated to not matched append operations. -// Disadvantages: -// 1. This solution is likely to be a one-time approach (especially if there are not matched insert operations involved), -// potentially leading to the target table being unsuitable for use as a build table in the future. -// 2. Requires a significant amount of memory to be efficient and currently does not support spill operations. -#[allow(unused)] -pub struct HashJoinBlockInfoHashTable { - pub(crate) pointers: Box<[u64], A>, - pub(crate) atomic_pointers: *mut AtomicU64, - pub(crate) hash_shift: usize, - pub(crate) is_distributed: bool, - // matched[idx] means - pub(crate) matched: Box<[u64]>, - pub(crate) block_info_index: BlockInfoIndex, - pub(crate) phantom: PhantomData, -} - -unsafe impl Send - for HashJoinBlockInfoHashTable -{ -} - -unsafe impl Sync - for HashJoinBlockInfoHashTable -{ -} - -impl HashJoinBlockInfoHashTable { - pub fn with_build_row_num( - row_num: usize, - is_distributed: bool, - block_info_index: BlockInfoIndex, - ) -> Self { - let capacity = std::cmp::max((row_num * 2).next_power_of_two(), 1 << 10); - let mut hashtable = Self { - pointers: unsafe { - Box::new_zeroed_slice_in(capacity, Default::default()).assume_init() - }, - atomic_pointers: std::ptr::null_mut(), - hash_shift: (hash_bits() - capacity.trailing_zeros()) as usize, - phantom: PhantomData, - is_distributed, - matched: unsafe { Box::new_zeroed_slice_in(row_num, Default::default()).assume_init() }, - block_info_index, - }; - hashtable.atomic_pointers = unsafe { - std::mem::transmute::<*mut u64, *mut AtomicU64>(hashtable.pointers.as_mut_ptr()) - }; - hashtable - } - - pub fn insert(&mut self, key: K, entry_ptr: *mut RawEntry) { - let hash = key.hash(); - let index = (hash >> self.hash_shift) as usize; - let new_header = new_header(entry_ptr as u64, hash); - // # Safety - // `index` is less than the capacity of hash table. - let mut old_header = unsafe { (*self.atomic_pointers.add(index)).load(Ordering::Relaxed) }; - loop { - let res = unsafe { - (*self.atomic_pointers.add(index)).compare_exchange_weak( - old_header, - combine_header(new_header, old_header), - Ordering::SeqCst, - Ordering::SeqCst, - ) - }; - match res { - Ok(_) => break, - Err(x) => old_header = x, - }; - } - unsafe { (*entry_ptr).next = remove_header_tag(old_header) }; - } -} - -impl HashJoinHashtableLike for HashJoinBlockInfoHashTable -where - K: Keyable, - A: Allocator + Clone + 'static, -{ - type Key = K; - - // Using hashes to probe hash table and converting them in-place to pointers for memory reuse. - fn probe(&self, hashes: &mut [u64], bitmap: Option) -> usize { - let mut valids = None; - if let Some(bitmap) = bitmap { - if bitmap.unset_bits() == bitmap.len() { - hashes.iter_mut().for_each(|hash| { - *hash = 0; - }); - return 0; - } else if bitmap.unset_bits() > 0 { - valids = Some(bitmap); - } - } - let mut count = 0; - match valids { - Some(valids) => { - valids - .iter() - .zip(hashes.iter_mut()) - .for_each(|(valid, hash)| { - if valid { - let header = self.pointers[(*hash >> self.hash_shift) as usize]; - if header != 0 { - *hash = remove_header_tag(header); - count += 1; - } else { - *hash = 0; - } - } else { - *hash = 0; - } - }); - } - None => { - hashes.iter_mut().for_each(|hash| { - let header = self.pointers[(*hash >> self.hash_shift) as usize]; - if header != 0 { - *hash = remove_header_tag(header); - count += 1; - } else { - *hash = 0; - } - }); - } - } - count - } - - // Using hashes to probe hash table and converting them in-place to pointers for memory reuse. - fn early_filtering_probe(&self, hashes: &mut [u64], bitmap: Option) -> usize { - let mut valids = None; - if let Some(bitmap) = bitmap { - if bitmap.unset_bits() == bitmap.len() { - hashes.iter_mut().for_each(|hash| { - *hash = 0; - }); - return 0; - } else if bitmap.unset_bits() > 0 { - valids = Some(bitmap); - } - } - let mut count = 0; - match valids { - Some(valids) => { - valids - .iter() - .zip(hashes.iter_mut()) - .for_each(|(valid, hash)| { - if valid { - let header = self.pointers[(*hash >> self.hash_shift) as usize]; - if header != 0 && early_filtering(header, *hash) { - *hash = remove_header_tag(header); - count += 1; - } else { - *hash = 0; - } - } else { - *hash = 0; - } - }); - } - None => { - hashes.iter_mut().for_each(|hash| { - let header = self.pointers[(*hash >> self.hash_shift) as usize]; - if header != 0 && early_filtering(header, *hash) { - *hash = remove_header_tag(header); - count += 1; - } else { - *hash = 0; - } - }); - } - } - count - } - - // Using hashes to probe hash table and converting them in-place to pointers for memory reuse. - fn early_filtering_probe_with_selection( - &self, - hashes: &mut [u64], - bitmap: Option, - selection: &mut [u32], - ) -> usize { - let mut valids = None; - if let Some(bitmap) = bitmap { - if bitmap.unset_bits() == bitmap.len() { - return 0; - } else if bitmap.unset_bits() > 0 { - valids = Some(bitmap); - } - } - let mut count = 0; - match valids { - Some(valids) => { - valids.iter().zip(hashes.iter_mut().enumerate()).for_each( - |(valid, (idx, hash))| { - if valid { - let header = self.pointers[(*hash >> self.hash_shift) as usize]; - if header != 0 && early_filtering(header, *hash) { - *hash = remove_header_tag(header); - unsafe { *selection.get_unchecked_mut(count) = idx as u32 }; - count += 1; - } - } - }, - ); - } - None => { - hashes.iter_mut().enumerate().for_each(|(idx, hash)| { - let header = self.pointers[(*hash >> self.hash_shift) as usize]; - if header != 0 && early_filtering(header, *hash) { - *hash = remove_header_tag(header); - unsafe { *selection.get_unchecked_mut(count) = idx as u32 }; - count += 1; - } - }); - } - } - count - } - - fn next_contains(&self, key: &Self::Key, mut ptr: u64) -> bool { - loop { - if ptr == 0 { - break; - } - let raw_entry = unsafe { &*(ptr as *mut RawEntry) }; - if key == &raw_entry.key { - return true; - } - ptr = raw_entry.next; - } - false - } - - fn next_probe( - &self, - key: &Self::Key, - mut ptr: u64, - vec_ptr: *mut RowPtr, - mut occupied: usize, - capacity: usize, - ) -> (usize, u64) { - let origin = occupied; - loop { - if ptr == 0 || occupied >= capacity { - break; - } - let raw_entry = unsafe { &*(ptr as *mut RawEntry) }; - if key == &raw_entry.key { - // # Safety - // occupied is less than the capacity of vec_ptr. - unsafe { - std::ptr::copy_nonoverlapping( - &raw_entry.row_ptr as *const RowPtr, - vec_ptr.add(occupied), - 1, - ) - }; - occupied += 1; - } - ptr = raw_entry.next; - } - if occupied > origin { - (occupied - origin, ptr) - } else { - (0, 0) - } - } - - // for merge into block info hash table - fn gather_partial_modified_block(&self) -> (Interval, u64) { - unreachable!() - } - - // for merge into block info hash table - fn reduce_false_matched_for_conjuct(&mut self) { - unreachable!() - } -} diff --git a/src/common/hashtable/src/hashjoin_block_info_string_hashtable.rs b/src/common/hashtable/src/hashjoin_block_info_string_hashtable.rs deleted file mode 100644 index 5f861dcc7b59..000000000000 --- a/src/common/hashtable/src/hashjoin_block_info_string_hashtable.rs +++ /dev/null @@ -1,329 +0,0 @@ -// 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::Allocator; -use std::sync::atomic::AtomicU64; -use std::sync::atomic::Ordering; - -use databend_common_arrow::arrow::bitmap::Bitmap; -use databend_common_base::mem_allocator::MmapAllocator; - -use crate::hash_join_fast_string_hash; -use crate::hashjoin_hashtable::combine_header; -use crate::hashjoin_hashtable::early_filtering; -use crate::hashjoin_hashtable::hash_bits; -use crate::hashjoin_hashtable::new_header; -use crate::hashjoin_hashtable::remove_header_tag; -use crate::utils::BlockInfoIndex; -use crate::utils::Interval; -use crate::HashJoinHashtableLike; -use crate::RowPtr; -use crate::StringRawEntry; -use crate::STRING_EARLY_SIZE; - -// This hashtable is only used for target build merge into (both standalone and distributed mode). -// Advantages: -// 1. Reduces redundant I/O operations, enhancing performance. -// 2. Lowers the maintenance overhead of deduplicating row_id.(But in distributed design, we also need to give rowid) -// 3. Allows the scheduling of the subsequent mutation pipeline to be entirely allocated to not matched append operations. -// Disadvantages: -// 1. This solution is likely to be a one-time approach (especially if there are not matched insert operations involved), -// potentially leading to the target table being unsuitable for use as a build table in the future. -// 2. Requires a significant amount of memory to be efficient and currently does not support spill operations. -#[allow(unused)] -pub struct HashJoinBlockInfoStringHashTable { - pub(crate) pointers: Box<[u64], A>, - pub(crate) atomic_pointers: *mut AtomicU64, - pub(crate) hash_shift: usize, - pub(crate) is_distributed: bool, - pub(crate) matched: Box<[u64]>, - pub(crate) block_info_index: BlockInfoIndex, -} - -unsafe impl Send for HashJoinBlockInfoStringHashTable {} - -unsafe impl Sync for HashJoinBlockInfoStringHashTable {} - -impl HashJoinBlockInfoStringHashTable { - pub fn with_build_row_num( - row_num: usize, - is_distributed: bool, - block_info_index: BlockInfoIndex, - ) -> Self { - let capacity = std::cmp::max((row_num * 2).next_power_of_two(), 1 << 10); - let mut hashtable = Self { - pointers: unsafe { - Box::new_zeroed_slice_in(capacity, Default::default()).assume_init() - }, - atomic_pointers: std::ptr::null_mut(), - hash_shift: (hash_bits() - capacity.trailing_zeros()) as usize, - is_distributed, - matched: unsafe { Box::new_zeroed_slice_in(row_num, Default::default()).assume_init() }, - block_info_index, - }; - hashtable.atomic_pointers = unsafe { - std::mem::transmute::<*mut u64, *mut AtomicU64>(hashtable.pointers.as_mut_ptr()) - }; - hashtable - } - - pub fn insert(&mut self, key: &[u8], entry_ptr: *mut StringRawEntry) { - let hash = hash_join_fast_string_hash(key); - let index = (hash >> self.hash_shift) as usize; - let new_header = new_header(entry_ptr as u64, hash); - // # Safety - // `index` is less than the capacity of hash table. - let mut old_header = unsafe { (*self.atomic_pointers.add(index)).load(Ordering::Relaxed) }; - loop { - let res = unsafe { - (*self.atomic_pointers.add(index)).compare_exchange_weak( - old_header, - combine_header(new_header, old_header), - Ordering::SeqCst, - Ordering::SeqCst, - ) - }; - match res { - Ok(_) => break, - Err(x) => old_header = x, - }; - } - unsafe { (*entry_ptr).next = remove_header_tag(old_header) }; - } -} - -impl HashJoinHashtableLike for HashJoinBlockInfoStringHashTable -where A: Allocator + Clone + 'static -{ - type Key = [u8]; - - // Using hashes to probe hash table and converting them in-place to pointers for memory reuse. - fn probe(&self, hashes: &mut [u64], bitmap: Option) -> usize { - let mut valids = None; - if let Some(bitmap) = bitmap { - if bitmap.unset_bits() == bitmap.len() { - hashes.iter_mut().for_each(|hash| { - *hash = 0; - }); - return 0; - } else if bitmap.unset_bits() > 0 { - valids = Some(bitmap); - } - } - let mut count = 0; - match valids { - Some(valids) => { - hashes.iter_mut().enumerate().for_each(|(idx, hash)| { - if unsafe { valids.get_bit_unchecked(idx) } { - let header = self.pointers[(*hash >> self.hash_shift) as usize]; - if header != 0 { - *hash = remove_header_tag(header); - count += 1; - } else { - *hash = 0; - } - } else { - *hash = 0; - }; - }); - } - None => { - hashes.iter_mut().for_each(|hash| { - let header = self.pointers[(*hash >> self.hash_shift) as usize]; - if header != 0 { - *hash = remove_header_tag(header); - count += 1; - } else { - *hash = 0; - } - }); - } - } - count - } - - // Using hashes to probe hash table and converting them in-place to pointers for memory reuse. - fn early_filtering_probe(&self, hashes: &mut [u64], bitmap: Option) -> usize { - let mut valids = None; - if let Some(bitmap) = bitmap { - if bitmap.unset_bits() == bitmap.len() { - hashes.iter_mut().for_each(|hash| { - *hash = 0; - }); - return 0; - } else if bitmap.unset_bits() > 0 { - valids = Some(bitmap); - } - } - let mut count = 0; - match valids { - Some(valids) => { - hashes.iter_mut().enumerate().for_each(|(idx, hash)| { - if unsafe { valids.get_bit_unchecked(idx) } { - let header = self.pointers[(*hash >> self.hash_shift) as usize]; - if header != 0 && early_filtering(header, *hash) { - *hash = remove_header_tag(header); - count += 1; - } else { - *hash = 0; - } - } else { - *hash = 0; - }; - }); - } - None => { - hashes.iter_mut().for_each(|hash| { - let header = self.pointers[(*hash >> self.hash_shift) as usize]; - if header != 0 && early_filtering(header, *hash) { - *hash = remove_header_tag(header); - count += 1; - } else { - *hash = 0; - } - }); - } - } - count - } - - // Using hashes to probe hash table and converting them in-place to pointers for memory reuse. - fn early_filtering_probe_with_selection( - &self, - hashes: &mut [u64], - bitmap: Option, - selection: &mut [u32], - ) -> usize { - let mut valids = None; - if let Some(bitmap) = bitmap { - if bitmap.unset_bits() == bitmap.len() { - return 0; - } else if bitmap.unset_bits() > 0 { - valids = Some(bitmap); - } - } - let mut count = 0; - match valids { - Some(valids) => { - hashes.iter_mut().enumerate().for_each(|(idx, hash)| { - if unsafe { valids.get_bit_unchecked(idx) } { - let header = self.pointers[(*hash >> self.hash_shift) as usize]; - if header != 0 && early_filtering(header, *hash) { - *hash = remove_header_tag(header); - unsafe { *selection.get_unchecked_mut(count) = idx as u32 }; - count += 1; - } - } - }); - } - None => { - hashes.iter_mut().enumerate().for_each(|(idx, hash)| { - let header = self.pointers[(*hash >> self.hash_shift) as usize]; - if header != 0 && early_filtering(header, *hash) { - *hash = remove_header_tag(header); - unsafe { *selection.get_unchecked_mut(count) = idx as u32 }; - count += 1; - } - }); - } - } - count - } - - fn next_contains(&self, key: &Self::Key, mut ptr: u64) -> bool { - loop { - if ptr == 0 { - break; - } - let raw_entry = unsafe { &*(ptr as *mut StringRawEntry) }; - // Compare `early` and the length of the string, the size of `early` is 4. - let min_len = std::cmp::min( - STRING_EARLY_SIZE, - std::cmp::min(key.len(), raw_entry.length as usize), - ); - if raw_entry.length as usize == key.len() - && key[0..min_len] == raw_entry.early[0..min_len] - { - let key_ref = unsafe { - std::slice::from_raw_parts( - raw_entry.key as *const u8, - raw_entry.length as usize, - ) - }; - if key == key_ref { - return true; - } - } - ptr = raw_entry.next; - } - false - } - - fn next_probe( - &self, - key: &Self::Key, - mut ptr: u64, - vec_ptr: *mut RowPtr, - mut occupied: usize, - capacity: usize, - ) -> (usize, u64) { - let origin = occupied; - loop { - if ptr == 0 || occupied >= capacity { - break; - } - let raw_entry = unsafe { &*(ptr as *mut StringRawEntry) }; - // Compare `early` and the length of the string, the size of `early` is 4. - let min_len = std::cmp::min(STRING_EARLY_SIZE, key.len()); - if raw_entry.length as usize == key.len() - && key[0..min_len] == raw_entry.early[0..min_len] - { - let key_ref = unsafe { - std::slice::from_raw_parts( - raw_entry.key as *const u8, - raw_entry.length as usize, - ) - }; - if key == key_ref { - // # Safety - // occupied is less than the capacity of vec_ptr. - unsafe { - std::ptr::copy_nonoverlapping( - &raw_entry.row_ptr as *const RowPtr, - vec_ptr.add(occupied), - 1, - ) - }; - occupied += 1; - } - } - ptr = raw_entry.next; - } - if occupied > origin { - (occupied - origin, ptr) - } else { - (0, 0) - } - } - - // for merge into block info hash table - fn gather_partial_modified_block(&self) -> (Interval, u64) { - unreachable!() - } - - // for merge into block info hash table - fn reduce_false_matched_for_conjuct(&mut self) { - unreachable!() - } -} diff --git a/src/common/hashtable/src/lib.rs b/src/common/hashtable/src/lib.rs index 4414b7a977f2..d35eeba01a85 100644 --- a/src/common/hashtable/src/lib.rs +++ b/src/common/hashtable/src/lib.rs @@ -24,8 +24,7 @@ extern crate core; mod container; mod dictionary_string_hashtable; -mod hashjoin_block_info_hashtable; -mod hashjoin_block_info_string_hashtable; + mod hashjoin_hashtable; mod hashjoin_string_hashtable; mod hashtable; @@ -113,7 +112,5 @@ pub use keys_ref::KeysRef; pub use partitioned_hashtable::hash2bucket; pub type HashJoinHashMap = hashjoin_hashtable::HashJoinHashTable; pub type StringHashJoinHashMap = hashjoin_string_hashtable::HashJoinStringHashTable; -pub type BlockInfoStringJoinHashMap = - hashjoin_block_info_string_hashtable::HashJoinBlockInfoStringHashTable; -pub type BlockInfoJoinHashMap = hashjoin_block_info_hashtable::HashJoinBlockInfoHashTable; pub use traits::HashJoinHashtableLike; +pub use utils::BlockInfoIndex; diff --git a/src/query/service/src/pipelines/builders/builder_join.rs b/src/query/service/src/pipelines/builders/builder_join.rs index 0a8f10d3ee28..544fa5bcbe7c 100644 --- a/src/query/service/src/pipelines/builders/builder_join.rs +++ b/src/query/service/src/pipelines/builders/builder_join.rs @@ -27,6 +27,8 @@ use databend_common_sql::executor::physical_plans::RangeJoin; use databend_common_sql::executor::PhysicalPlan; use databend_common_sql::ColumnBinding; use databend_common_sql::IndexType; +use databend_common_sql::DUMMY_TABLE_INDEX; +use databend_common_storages_fuse::operations::need_reserve_block_info; use crate::pipelines::processors::transforms::range_join::RangeJoinState; use crate::pipelines::processors::transforms::range_join::TransformRangeJoinLeft; @@ -125,12 +127,33 @@ impl PipelineBuilder { pub(crate) fn build_join(&mut self, join: &HashJoin) -> Result<()> { let id = join.probe.get_table_index(); - let state = self.build_join_state(join, id)?; + // for merge into target table as build side. + let (build_table_index, is_distributed_merge_into) = + if matches!(&*join.build, PhysicalPlan::TableScan(_)) { + let (need_block_info, is_distributed) = + need_reserve_block_info(self.ctx.clone(), join.build.get_table_index()); + if need_block_info { + (join.build.get_table_index(), is_distributed) + } else { + (DUMMY_TABLE_INDEX, false) + } + } else { + (DUMMY_TABLE_INDEX, false) + }; + + let state = + self.build_join_state(join, id, build_table_index, is_distributed_merge_into)?; self.expand_build_side_pipeline(&join.build, join, state.clone())?; self.build_join_probe(join, state) } - fn build_join_state(&mut self, join: &HashJoin, id: IndexType) -> Result> { + fn build_join_state( + &mut self, + join: &HashJoin, + id: IndexType, + merge_into_target_table_index: IndexType, + is_distributed_merge_into: bool, + ) -> Result> { HashJoinState::try_create( self.ctx.clone(), join.build.output_schema()?, @@ -138,6 +161,8 @@ impl PipelineBuilder { HashJoinDesc::create(join)?, &join.probe_to_build, id, + merge_into_target_table_index, + is_distributed_merge_into, ) } diff --git a/src/query/service/src/pipelines/builders/builder_merge_into.rs b/src/query/service/src/pipelines/builders/builder_merge_into.rs index 44c341d39877..fc6847ea6098 100644 --- a/src/query/service/src/pipelines/builders/builder_merge_into.rs +++ b/src/query/service/src/pipelines/builders/builder_merge_into.rs @@ -375,6 +375,7 @@ impl PipelineBuilder { } // build merge into pipeline. + // the block rows is limitd by join (65536 rows), but we don't promise the block size. pub(crate) fn build_merge_into(&mut self, merge_into: &MergeInto) -> Result<()> { let MergeInto { input, diff --git a/src/query/service/src/pipelines/builders/builder_replace_into.rs b/src/query/service/src/pipelines/builders/builder_replace_into.rs index 1c9c04e55493..ebfc51441b9e 100644 --- a/src/query/service/src/pipelines/builders/builder_replace_into.rs +++ b/src/query/service/src/pipelines/builders/builder_replace_into.rs @@ -158,6 +158,7 @@ impl PipelineBuilder { return Ok(()); } + // The Block Size and Rows is promised by DataSouce by user. if segment_partition_num == 0 { let dummy_item = create_dummy_item(); // ┌──────────────────────┐ ┌──────────────────┐ diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_build_state.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_build_state.rs index 25213f95ddb5..25a008343016 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_build_state.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_build_state.rs @@ -22,6 +22,7 @@ use std::sync::Arc; use databend_common_arrow::arrow::bitmap::Bitmap; use databend_common_base::base::tokio::sync::Barrier; +use databend_common_catalog::plan::compute_row_id_prefix; use databend_common_catalog::runtime_filter_info::RuntimeFilterInfo; use databend_common_catalog::table_context::TableContext; use databend_common_exception::ErrorCode; @@ -29,6 +30,7 @@ use databend_common_exception::Result; use databend_common_expression::arrow::and_validities; use databend_common_expression::types::DataType; use databend_common_expression::types::NumberDomain; +use databend_common_expression::BlockMetaInfoDowncast; use databend_common_expression::Column; use databend_common_expression::ColumnBuilder; use databend_common_expression::ColumnVec; @@ -53,6 +55,8 @@ use databend_common_hashtable::StringRawEntry; use databend_common_hashtable::STRING_EARLY_SIZE; use databend_common_sql::plans::JoinType; use databend_common_sql::ColumnSet; +use databend_common_sql::DUMMY_TABLE_INDEX; +use databend_common_storages_fuse::operations::BlockMetaIndex; use ethnum::U256; use itertools::Itertools; use log::info; @@ -60,6 +64,7 @@ use parking_lot::Mutex; use parking_lot::RwLock; use xorf::BinaryFuse16; +use super::MatchedPtr; use crate::pipelines::processors::transforms::hash_join::common::wrap_true_validity; use crate::pipelines::processors::transforms::hash_join::desc::MARKER_KIND_FALSE; use crate::pipelines::processors::transforms::hash_join::util::dedup_build_key_column; @@ -156,6 +161,7 @@ impl HashJoinBuildState { } } let chunk_size_limit = ctx.get_settings().get_max_block_size()? as usize * 16; + Ok(Arc::new(Self { ctx: ctx.clone(), func_ctx, @@ -181,14 +187,39 @@ impl HashJoinBuildState { /// Add input `DataBlock` to `hash_join_state.row_space`. pub fn build(&self, input: DataBlock) -> Result<()> { let mut buffer = self.hash_join_state.row_space.buffer.write(); + let input_rows = input.num_rows(); - buffer.push(input); + buffer.push(input.clone()); let old_size = self .hash_join_state .row_space .buffer_row_size .fetch_add(input_rows, Ordering::Relaxed); + let build_state = unsafe { &*self.hash_join_state.build_state.get() }; + let start_offset = build_state.generation_state.build_num_rows + old_size; + let end_offset = start_offset + input_rows - 1; + // merge into target table as build side. + if self.hash_join_state.block_info_index.is_some() { + assert!(input.get_meta().is_some()); + let block_meta_index = + BlockMetaIndex::downcast_ref_from(input.get_meta().unwrap()).unwrap(); + let row_prefix = compute_row_id_prefix( + block_meta_index.segment_idx as u64, + block_meta_index.block_idx as u64, + ); + let block_info_index = unsafe { + &mut *self + .hash_join_state + .block_info_index + .as_ref() + .unwrap() + .get() + }; + block_info_index + .insert_block_offsets((start_offset as u32, end_offset as u32), row_prefix); + } + if old_size + input_rows < self.chunk_size_limit { return Ok(()); } @@ -227,6 +258,10 @@ impl HashJoinBuildState { if self.hash_join_state.need_mark_scan() { build_state.mark_scan_map.push(block_mark_scan_map); } + if self.hash_join_state.merge_into_target_table_index != DUMMY_TABLE_INDEX { + let chunk_offsets = unsafe { &mut *self.hash_join_state.chunk_offsets.get() }; + chunk_offsets.push(build_state.generation_state.build_num_rows as u64); + } build_state.generation_state.build_num_rows += data_block.num_rows(); build_state.generation_state.chunks.push(data_block); } @@ -386,6 +421,16 @@ impl HashJoinBuildState { }; let hashtable = unsafe { &mut *self.hash_join_state.hash_table.get() }; *hashtable = hashjoin_hashtable; + // generate macthed offsets memory. + if self.hash_join_state.merge_into_target_table_index != DUMMY_TABLE_INDEX { + let matched = unsafe { &mut *self.hash_join_state.matched.get() }; + let build_state = unsafe { &*self.hash_join_state.build_state.get() }; + let atomic_pointer = unsafe { &mut *self.hash_join_state.atomic_pointer.get() }; + *matched = vec![0; build_state.generation_state.build_num_rows]; + let pointer = + unsafe { std::mem::transmute::<*mut u8, *mut AtomicU8>(matched.as_mut_ptr()) }; + *atomic_pointer = MatchedPtr(pointer); + } } Ok(()) } diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_state.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_state.rs index 4e39c0240aec..bf71c4d0f0c2 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_state.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_state.rs @@ -16,6 +16,7 @@ use std::cell::SyncUnsafeCell; use std::collections::HashSet; use std::sync::atomic::AtomicBool; use std::sync::atomic::AtomicI8; +use std::sync::atomic::AtomicU8; use std::sync::atomic::AtomicUsize; use std::sync::atomic::Ordering; use std::sync::Arc; @@ -30,6 +31,7 @@ use databend_common_expression::DataSchemaRef; use databend_common_expression::HashMethodFixedKeys; use databend_common_expression::HashMethodSerializer; use databend_common_expression::HashMethodSingleString; +use databend_common_hashtable::BlockInfoIndex; use databend_common_hashtable::HashJoinHashMap; use databend_common_hashtable::HashtableKeyable; use databend_common_hashtable::StringHashJoinHashMap; @@ -72,6 +74,11 @@ pub enum HashJoinHashTable { KeysU256(FixedKeyHashJoinHashTable), } +pub struct MatchedPtr(pub *mut AtomicU8); + +unsafe impl Send for MatchedPtr {} +unsafe impl Sync for MatchedPtr {} + /// Define some shared states for hash join build and probe. /// It will like a bridge to connect build and probe. /// Such as build side will pass hash table to probe side by it @@ -121,6 +128,21 @@ pub struct HashJoinState { /// If the join node generate runtime filters, the scan node will use it to do prune. pub(crate) table_index: IndexType, + /// If we use target table as build side for merge into, we use this to track target table + /// and extract partial modified blocks from hashtable + pub(crate) merge_into_target_table_index: IndexType, + pub(crate) is_distributed_merge_into: bool, + + /// FOR MERGE INTO TARGET TABLE AS BUILD SIDE + /// When merge into target table as build side, we should preseve block info index. + pub(crate) block_info_index: Option>, + /// we use matched to tag the matched offset in chunks. + pub(crate) matched: SyncUnsafeCell>, + /// the matched will be modified concurrently, so we use + /// atomic_pointers to pointer to matched + pub(crate) atomic_pointer: SyncUnsafeCell, + /// chunk_offsets[chunk_idx] stands for the offset of chunk_idx_th chunk in chunks. + pub(crate) chunk_offsets: SyncUnsafeCell>, } impl HashJoinState { @@ -131,6 +153,8 @@ impl HashJoinState { hash_join_desc: HashJoinDesc, probe_to_build: &[(usize, (bool, bool))], table_index: IndexType, + merge_into_target_table_index: IndexType, + is_distributed_merge_into: bool, ) -> Result> { if matches!( hash_join_desc.join_type, @@ -166,6 +190,12 @@ impl HashJoinState { partition_id: AtomicI8::new(-2), enable_spill, table_index, + merge_into_target_table_index, + is_distributed_merge_into, + block_info_index: None, + matched: SyncUnsafeCell::new(Vec::new()), + atomic_pointer: SyncUnsafeCell::new(MatchedPtr(std::ptr::null_mut())), + chunk_offsets: SyncUnsafeCell::new(Vec::with_capacity(100)), })) } diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/probe_join/left_join.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/probe_join/left_join.rs index caea9cc16bd2..32087a94a3e0 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/probe_join/left_join.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/probe_join/left_join.rs @@ -14,6 +14,7 @@ use std::sync::atomic::Ordering; +use databend_common_arrow::arrow::bitmap::Bitmap; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::BlockEntry; @@ -24,6 +25,7 @@ use databend_common_expression::Scalar; use databend_common_expression::Value; use databend_common_hashtable::HashJoinHashtableLike; use databend_common_hashtable::RowPtr; +use databend_common_sql::DUMMY_TABLE_INDEX; use crate::pipelines::processors::transforms::hash_join::build_state::BuildBlockGenerationState; use crate::pipelines::processors::transforms::hash_join::common::wrap_true_validity; @@ -370,6 +372,54 @@ impl HashJoinProbeState { Ok(self.merge_eq_block(probe_block, build_block, matched_idx)) } + #[inline] + #[allow(clippy::too_many_arguments)] + fn check_and_set_matched( + &self, + build_indexes: &[RowPtr], + matched_idx: usize, + valids: &Bitmap, + ) -> Result<()> { + // merge into target table as build side. + if self.hash_join_state.merge_into_target_table_index != DUMMY_TABLE_INDEX { + let chunk_offsets = unsafe { &*self.hash_join_state.chunk_offsets.get() }; + let pointer = unsafe { &*self.hash_join_state.atomic_pointer.get() }; + // add matched indexes. + for (idx, row_ptr) in build_indexes[0..matched_idx].iter().enumerate() { + unsafe { + if !valids.get_bit_unchecked(idx) { + continue; + } + } + let offset = chunk_offsets[row_ptr.chunk_index as usize] as usize + + row_ptr.row_index as usize; + let mut old_mactehd_counts = + unsafe { (*pointer.0.add(offset)).load(Ordering::Relaxed) }; + let new_matched_count = old_mactehd_counts + 1; + if old_mactehd_counts > 0 { + return Err(ErrorCode::UnresolvableConflict( + "multi rows from source match one and the same row in the target_table multi times", + )); + } + loop { + let res = unsafe { + (*pointer.0.add(offset)).compare_exchange_weak( + old_mactehd_counts, + new_matched_count, + Ordering::SeqCst, + Ordering::SeqCst, + ) + }; + match res { + Ok(_) => break, + Err(x) => old_mactehd_counts = x, + }; + } + } + } + Ok(()) + } + #[inline] #[allow(clippy::too_many_arguments)] fn process_left_or_full_join_block( @@ -454,6 +504,7 @@ impl HashJoinProbeState { }; } } + self.check_and_set_matched(build_indexes, matched_idx, &probe_state.true_validity)?; return Ok(()); } @@ -480,6 +531,7 @@ impl HashJoinProbeState { }; } } + self.check_and_set_matched(build_indexes, matched_idx, &probe_state.true_validity)?; } else if all_false { let mut idx = 0; while idx < matched_idx { @@ -510,6 +562,7 @@ impl HashJoinProbeState { } } else { let mut idx = 0; + self.check_and_set_matched(build_indexes, matched_idx, &validity); while idx < matched_idx { unsafe { let valid = validity.get_bit_unchecked(idx); diff --git a/src/query/sql/src/planner/optimizer/optimizer.rs b/src/query/sql/src/planner/optimizer/optimizer.rs index 96888cb7f209..edb17c50c0fd 100644 --- a/src/query/sql/src/planner/optimizer/optimizer.rs +++ b/src/query/sql/src/planner/optimizer/optimizer.rs @@ -326,7 +326,8 @@ fn optimize_merge_into(opt_ctx: OptimizerContext, plan: Box) -> Resul }; // we just support left join to use MergeIntoBlockInfoHashTable, we - // don't support spill for now. + // don't support spill for now, and we need the macthed cluases' count + // is one if change_join_order && matches!(plan.merge_type, MergeIntoType::FullOperation) && opt_ctx @@ -334,6 +335,7 @@ fn optimize_merge_into(opt_ctx: OptimizerContext, plan: Box) -> Resul .get_settings() .get_join_spilling_threshold()? == 0 + && plan.matched_evaluators.len() == 1 { opt_ctx.table_ctx.set_merge_into_join(MergeIntoJoin { merge_into_join_type: MergeIntoJoinType::Left, @@ -374,6 +376,7 @@ fn optimize_merge_into(opt_ctx: OptimizerContext, plan: Box) -> Resul opt_ctx.table_ctx.clone(), plan.merge_type.clone(), plan.target_table_idx, + plan.matched_evaluators.len() == 1, )?; (merge_into_join_sexpr.clone(), false) } else { @@ -404,6 +407,7 @@ fn try_to_change_as_broadcast_join( table_ctx: Arc, merge_into_type: MergeIntoType, target_tbl_idx: usize, + only_one_matched_clause: bool, ) -> Result { if let RelOperator::Exchange(Exchange::Merge) = merge_into_join_sexpr.plan.as_ref() { let right_exchange = merge_into_join_sexpr.child(0)?.child(1)?; @@ -415,7 +419,10 @@ fn try_to_change_as_broadcast_join( .replace_plan(Arc::new(RelOperator::Join(join))); // for now, when we use target table as build side and it's a broadcast join, // we will use merge_into_block_info_hashtable to reduce i/o operations. - if change_join_order && matches!(merge_into_type, MergeIntoType::FullOperation) { + if change_join_order + && matches!(merge_into_type, MergeIntoType::FullOperation) + && only_one_matched_clause + { table_ctx.set_merge_into_join(MergeIntoJoin { merge_into_join_type: MergeIntoJoinType::Left, is_distributed: true, diff --git a/src/query/storages/common/pruner/src/block_meta.rs b/src/query/storages/common/pruner/src/block_meta.rs index f5362f93c16b..901e743c2df7 100644 --- a/src/query/storages/common/pruner/src/block_meta.rs +++ b/src/query/storages/common/pruner/src/block_meta.rs @@ -56,25 +56,3 @@ impl BlockMetaIndex { }) } } - -#[derive(serde::Serialize, serde::Deserialize, Clone, Debug, Default, PartialEq, Eq)] -pub struct RowPrefix { - // segment_idx_block_id - pub prefix: u64, - pub inner: Option, -} - -#[typetag::serde(name = "row_prefix")] -impl BlockMetaInfo for RowPrefix { - fn equals(&self, info: &Box) -> bool { - RowPrefix::downcast_ref_from(info).is_some_and(|other| self == other) - } - - fn clone_self(&self) -> Box { - Box::new(self.clone()) - } -} - -pub fn gen_row_prefix(inner: Option, prefix: u64) -> RowPrefix { - RowPrefix { prefix, inner } -} diff --git a/src/query/storages/common/pruner/src/lib.rs b/src/query/storages/common/pruner/src/lib.rs index b55adb6f89f4..04bc0341fc43 100644 --- a/src/query/storages/common/pruner/src/lib.rs +++ b/src/query/storages/common/pruner/src/lib.rs @@ -21,9 +21,7 @@ mod page_pruner; mod range_pruner; mod topn_pruner; -pub use block_meta::gen_row_prefix; pub use block_meta::BlockMetaIndex; -pub use block_meta::RowPrefix; pub use internal_column_pruner::InternalColumnPruner; pub use limiter_pruner::Limiter; pub use limiter_pruner::LimiterPruner; diff --git a/src/query/storages/fuse/src/operations/common/mutation_log.rs b/src/query/storages/fuse/src/operations/common/mutation_log.rs index c0abe13f993d..a87341350bfb 100644 --- a/src/query/storages/fuse/src/operations/common/mutation_log.rs +++ b/src/query/storages/fuse/src/operations/common/mutation_log.rs @@ -18,6 +18,7 @@ use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::BlockMetaInfo; use databend_common_expression::BlockMetaInfoDowncast; +use databend_common_expression::BlockMetaInfoPtr; use databend_common_expression::DataBlock; use databend_common_pipeline_transforms::processors::AccumulatingTransform; use databend_storages_common_table_meta::meta::BlockMeta; @@ -66,10 +67,22 @@ pub enum MutationLogEntry { pub struct BlockMetaIndex { pub segment_idx: SegmentIndex, pub block_idx: BlockIndex, + pub inner: Option, // range is unused for now. // pub range: Option>, } +#[typetag::serde(name = "block_meta_index")] +impl BlockMetaInfo for BlockMetaIndex { + fn equals(&self, info: &Box) -> bool { + BlockMetaIndex::downcast_ref_from(info).is_some_and(|other| self == other) + } + + fn clone_self(&self) -> Box { + Box::new(self.clone()) + } +} + #[typetag::serde(name = "mutation_logs_meta")] impl BlockMetaInfo for MutationLogs { fn equals(&self, info: &Box) -> bool { diff --git a/src/query/storages/fuse/src/operations/merge_into/mutator/matched_mutator.rs b/src/query/storages/fuse/src/operations/merge_into/mutator/matched_mutator.rs index 035095a0e7d4..f149b73a43b3 100644 --- a/src/query/storages/fuse/src/operations/merge_into/mutator/matched_mutator.rs +++ b/src/query/storages/fuse/src/operations/merge_into/mutator/matched_mutator.rs @@ -334,6 +334,7 @@ impl AggregationContext { index: BlockMetaIndex { segment_idx, block_idx, + inner: None, }, })); } @@ -370,6 +371,7 @@ impl AggregationContext { index: BlockMetaIndex { segment_idx, block_idx, + inner: None, }, block_meta: Arc::new(new_block_meta), }; diff --git a/src/query/storages/fuse/src/operations/mod.rs b/src/query/storages/fuse/src/operations/mod.rs index 2cb5d826fd14..fbbad4b46af1 100644 --- a/src/query/storages/fuse/src/operations/mod.rs +++ b/src/query/storages/fuse/src/operations/mod.rs @@ -41,6 +41,7 @@ pub use delete::MutationBlockPruningContext; pub use merge_into::*; pub use mutation::*; pub use read::build_row_fetcher_pipeline; +pub use read::need_reserve_block_info; pub use replace_into::*; pub use util::acquire_task_permit; pub use util::column_parquet_metas; diff --git a/src/query/storages/fuse/src/operations/mutation/mutator/block_compact_mutator.rs b/src/query/storages/fuse/src/operations/mutation/mutator/block_compact_mutator.rs index 5579e2814494..ea2760582f37 100644 --- a/src/query/storages/fuse/src/operations/mutation/mutator/block_compact_mutator.rs +++ b/src/query/storages/fuse/src/operations/mutation/mutator/block_compact_mutator.rs @@ -575,6 +575,7 @@ impl CompactTaskBuilder { CompactTaskInfo::create(blocks, BlockMetaIndex { segment_idx, block_idx, + inner: None, }), )))); } diff --git a/src/query/storages/fuse/src/operations/mutation/processors/mutation_source.rs b/src/query/storages/fuse/src/operations/mutation/processors/mutation_source.rs index e170f460ff70..a3c2b7074cdc 100644 --- a/src/query/storages/fuse/src/operations/mutation/processors/mutation_source.rs +++ b/src/query/storages/fuse/src/operations/mutation/processors/mutation_source.rs @@ -380,6 +380,7 @@ impl Processor for MutationSource { self.index = BlockMetaIndex { segment_idx: part.index.segment_idx, block_idx: part.index.block_idx, + inner: None, }; if matches!(self.action, MutationAction::Deletion) { self.stats_type = diff --git a/src/query/storages/fuse/src/operations/read/mod.rs b/src/query/storages/fuse/src/operations/read/mod.rs index 08468b855739..be309849c7fb 100644 --- a/src/query/storages/fuse/src/operations/read/mod.rs +++ b/src/query/storages/fuse/src/operations/read/mod.rs @@ -32,3 +32,4 @@ pub use native_data_source_deserializer::NativeDeserializeDataTransform; pub use native_data_source_reader::ReadNativeDataSource; pub use parquet_data_source_deserializer::DeserializeDataTransform; pub use parquet_data_source_reader::ReadParquetDataSource; +pub use util::need_reserve_block_info; diff --git a/src/query/storages/fuse/src/operations/read/native_data_source_deserializer.rs b/src/query/storages/fuse/src/operations/read/native_data_source_deserializer.rs index 6d20632caa93..e18333a51c0d 100644 --- a/src/query/storages/fuse/src/operations/read/native_data_source_deserializer.rs +++ b/src/query/storages/fuse/src/operations/read/native_data_source_deserializer.rs @@ -255,7 +255,7 @@ impl NativeDeserializeDataTransform { virtual_reader: Arc>, ) -> Result { let scan_progress = ctx.get_scan_progress(); - let need_reserve_block_info = need_reserve_block_info(ctx.clone(), plan.table_index); + let (need_reserve_block_info, _) = need_reserve_block_info(ctx.clone(), plan.table_index); let mut src_schema: DataSchema = (block_reader.schema().as_ref()).into(); let mut prewhere_columns: Vec = diff --git a/src/query/storages/fuse/src/operations/read/parquet_data_source_deserializer.rs b/src/query/storages/fuse/src/operations/read/parquet_data_source_deserializer.rs index 0e011ca428bb..db6438159bc3 100644 --- a/src/query/storages/fuse/src/operations/read/parquet_data_source_deserializer.rs +++ b/src/query/storages/fuse/src/operations/read/parquet_data_source_deserializer.rs @@ -109,7 +109,7 @@ impl DeserializeDataTransform { let mut output_schema = plan.schema().as_ref().clone(); output_schema.remove_internal_fields(); let output_schema: DataSchema = (&output_schema).into(); - let need_reserve_block_info = need_reserve_block_info(ctx.clone(), plan.table_index); + let (need_reserve_block_info, _) = need_reserve_block_info(ctx.clone(), plan.table_index); Ok(ProcessorPtr::create(Box::new(DeserializeDataTransform { ctx, table_index: plan.table_index, diff --git a/src/query/storages/fuse/src/operations/read/util.rs b/src/query/storages/fuse/src/operations/read/util.rs index 5e34f613d506..01dd9471b9b6 100644 --- a/src/query/storages/fuse/src/operations/read/util.rs +++ b/src/query/storages/fuse/src/operations/read/util.rs @@ -15,20 +15,22 @@ use std::sync::Arc; use databend_common_catalog::merge_into_join::MergeIntoJoinType; -use databend_common_catalog::plan::compute_row_id_prefix; use databend_common_catalog::table_context::TableContext; use databend_common_exception::Result; use databend_common_expression::DataBlock; -use databend_storages_common_pruner::gen_row_prefix; +use crate::operations::BlockMetaIndex; use crate::FusePartInfo; -pub(crate) fn need_reserve_block_info(ctx: Arc, table_idx: usize) -> bool { +pub fn need_reserve_block_info(ctx: Arc, table_idx: usize) -> (bool, bool) { let merge_into_join = ctx.get_merge_into_join(); - matches!( - merge_into_join.merge_into_join_type, - MergeIntoJoinType::Left - ) && merge_into_join.target_tbl_idx == table_idx + ( + matches!( + merge_into_join.merge_into_join_type, + MergeIntoJoinType::Left + ) && merge_into_join.target_tbl_idx == table_idx, + merge_into_join.is_distributed, + ) } // for merge into target build, in this situation, we don't need rowid @@ -39,14 +41,14 @@ pub(crate) fn add_row_prefix_meta( ) -> Result { if need_reserve_block_info && fuse_part.block_meta_index.is_some() { let block_meta_index = fuse_part.block_meta_index.as_ref().unwrap(); - let prefix = compute_row_id_prefix( - block_meta_index.segment_idx as u64, - block_meta_index.block_id as u64, - ); // in fact, inner_meta is none for now, for merge into target build, we don't need // to get row_id. let inner_meta = block.take_meta(); - block.add_meta(Some(Box::new(gen_row_prefix(inner_meta, prefix)))) + block.add_meta(Some(Box::new(BlockMetaIndex { + segment_idx: block_meta_index.segment_idx, + block_idx: block_meta_index.block_id, + inner: inner_meta, + }))) } else { Ok(block) } diff --git a/src/query/storages/fuse/src/operations/replace_into/mutator/merge_into_mutator.rs b/src/query/storages/fuse/src/operations/replace_into/mutator/merge_into_mutator.rs index 7e162b2a14a4..7ff4b623af9b 100644 --- a/src/query/storages/fuse/src/operations/replace_into/mutator/merge_into_mutator.rs +++ b/src/query/storages/fuse/src/operations/replace_into/mutator/merge_into_mutator.rs @@ -458,6 +458,7 @@ impl AggregationContext { index: BlockMetaIndex { segment_idx: segment_index, block_idx: block_index, + inner: None, }, }; @@ -534,6 +535,7 @@ impl AggregationContext { index: BlockMetaIndex { segment_idx: segment_index, block_idx: block_index, + inner: None, }, block_meta: Arc::new(new_block_meta), }; From 6a3f722caa0463ef4ff8a2d363049ab6bbb394ab Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Wed, 10 Jan 2024 21:53:38 +0800 Subject: [PATCH 11/47] fix blockinfo index --- .../hashtable/src/hashjoin_hashtable.rs | 10 ---- .../src/hashjoin_string_hashtable.rs | 10 ---- src/common/hashtable/src/traits.rs | 6 --- src/common/hashtable/src/utils.rs | 51 +++++++++++++++++++ .../sql/src/planner/optimizer/optimizer.rs | 10 ++-- 5 files changed, 58 insertions(+), 29 deletions(-) diff --git a/src/common/hashtable/src/hashjoin_hashtable.rs b/src/common/hashtable/src/hashjoin_hashtable.rs index 4877a917e4ec..18aabaef2969 100644 --- a/src/common/hashtable/src/hashjoin_hashtable.rs +++ b/src/common/hashtable/src/hashjoin_hashtable.rs @@ -351,14 +351,4 @@ where (0, 0) } } - - // for merge into block info hash table - fn gather_partial_modified_block(&self) -> (Interval, u64) { - unreachable!() - } - - // for merge into block info hash table - fn reduce_false_matched_for_conjuct(&mut self) { - unreachable!() - } } diff --git a/src/common/hashtable/src/hashjoin_string_hashtable.rs b/src/common/hashtable/src/hashjoin_string_hashtable.rs index 3b23a229c32f..bd5f3e1fd0d3 100644 --- a/src/common/hashtable/src/hashjoin_string_hashtable.rs +++ b/src/common/hashtable/src/hashjoin_string_hashtable.rs @@ -302,14 +302,4 @@ where A: Allocator + Clone + 'static (0, 0) } } - - // for merge into block info hash table - fn gather_partial_modified_block(&self) -> (Interval, u64) { - unreachable!() - } - - // for merge into block info hash table - fn reduce_false_matched_for_conjuct(&mut self) { - unreachable!() - } } diff --git a/src/common/hashtable/src/traits.rs b/src/common/hashtable/src/traits.rs index 215c41d31f03..265c4beb153d 100644 --- a/src/common/hashtable/src/traits.rs +++ b/src/common/hashtable/src/traits.rs @@ -553,10 +553,4 @@ pub trait HashJoinHashtableLike { occupied: usize, capacity: usize, ) -> (usize, u64); - - // for merge into block info hash table - fn gather_partial_modified_block(&self) -> (Interval, u64); - - // for merge into block info hash table - fn reduce_false_matched_for_conjuct(&mut self); } diff --git a/src/common/hashtable/src/utils.rs b/src/common/hashtable/src/utils.rs index d925d00dccca..ea947c79cd55 100644 --- a/src/common/hashtable/src/utils.rs +++ b/src/common/hashtable/src/utils.rs @@ -238,6 +238,14 @@ impl BlockInfoIndex { /// intervals: (0,10)(11,22),(23,40)(41,55) /// interval: (8,27) /// we will give (8,10),(23,27), we don't give the (11,12),because it's updated all. + /// case1: |-----|------|------| + /// |-----------| + /// case2: |-----|------|------| + /// |------| + /// case3: |-----|------|------| + /// |--| + /// case4: |-----|------|------| + /// |--------| #[allow(dead_code)] pub fn get_block_info(&self, interval: Interval) -> Vec<(Interval, u64)> { let mut res = Vec::<(Interval, u64)>::with_capacity(2); @@ -245,6 +253,15 @@ impl BlockInfoIndex { let right_idx = self.search_idx(interval.1); let left_interval = &self.intervals[left_idx]; let right_interval = &self.intervals[right_idx]; + // empty cases + if left_interval.0 == interval.0 && right_interval.1 == interval.1 { + return res; + } + // only one result + if self.prefixs[left_idx] == self.prefixs[right_idx] { + res.push(((interval.0, interval.1), self.prefixs[left_idx])); + return res; + } if left_interval.0 < interval.0 { res.push(((interval.0, left_interval.1), self.prefixs[left_idx])) } @@ -294,8 +311,42 @@ fn test_block_info_index() { block_info_index.insert_block_offsets(*interval, idx as u64) } let result = block_info_index.get_block_info(find_interval); + assert_eq!(result.len(), 2); assert_eq!(result[0].0, (10, 10)); assert_eq!(result[0].1, 0); assert_eq!(result[1].0, (31, 37)); assert_eq!(result[1].1, 3); + + // we find [3,7], and should get [3,7] + let find_interval: Interval = (3, 7); + let result = block_info_index.get_block_info(find_interval); + assert_eq!(result.len(), 1); + assert_eq!(result[0].0, (3, 7)); + assert_eq!(result[0].1, 0); + + // we find [11,20], and should get empty + let find_interval: Interval = (11, 20); + let result = block_info_index.get_block_info(find_interval); + assert_eq!(result.len(), 0); + + // we find [11,20], and should get empty + let find_interval: Interval = (11, 30); + let result = block_info_index.get_block_info(find_interval); + assert_eq!(result.len(), 0); + + // we find [8,13], and should get (8,10),(11,13) + let find_interval: Interval = (8, 13); + let result = block_info_index.get_block_info(find_interval); + assert_eq!(result.len(), 2); + assert_eq!(result[0].0, (8, 10)); + assert_eq!(result[0].1, 0); + assert_eq!(result[1].0, (11, 13)); + assert_eq!(result[1].1, 1); + + // we find [11,23], and should get (20,23) + let find_interval: Interval = (11, 23); + let result = block_info_index.get_block_info(find_interval); + assert_eq!(result.len(), 1); + assert_eq!(result[0].0, (21, 23)); + assert_eq!(result[0].1, 2); } diff --git a/src/query/sql/src/planner/optimizer/optimizer.rs b/src/query/sql/src/planner/optimizer/optimizer.rs index edb17c50c0fd..bdd7a038dbab 100644 --- a/src/query/sql/src/planner/optimizer/optimizer.rs +++ b/src/query/sql/src/planner/optimizer/optimizer.rs @@ -327,7 +327,11 @@ fn optimize_merge_into(opt_ctx: OptimizerContext, plan: Box) -> Resul // we just support left join to use MergeIntoBlockInfoHashTable, we // don't support spill for now, and we need the macthed cluases' count - // is one + // is one, just support merge into t using source when matched then + // update xx when not matched then insert xx. + let flag = plan.matched_evaluators.len() == 1 + && plan.matched_evaluators[0].condition.is_none() + && plan.matched_evaluators[0].update.is_some(); if change_join_order && matches!(plan.merge_type, MergeIntoType::FullOperation) && opt_ctx @@ -335,7 +339,7 @@ fn optimize_merge_into(opt_ctx: OptimizerContext, plan: Box) -> Resul .get_settings() .get_join_spilling_threshold()? == 0 - && plan.matched_evaluators.len() == 1 + && flag { opt_ctx.table_ctx.set_merge_into_join(MergeIntoJoin { merge_into_join_type: MergeIntoJoinType::Left, @@ -376,7 +380,7 @@ fn optimize_merge_into(opt_ctx: OptimizerContext, plan: Box) -> Resul opt_ctx.table_ctx.clone(), plan.merge_type.clone(), plan.target_table_idx, - plan.matched_evaluators.len() == 1, + flag, )?; (merge_into_join_sexpr.clone(), false) } else { From 8d8a42389ec95fdd0e183ab718e723e5ebf1d6c1 Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Wed, 10 Jan 2024 23:18:06 +0800 Subject: [PATCH 12/47] gather partial modified blocks and fix lint --- .../hashtable/src/hashjoin_hashtable.rs | 1 - .../src/hashjoin_string_hashtable.rs | 1 - src/common/hashtable/src/utils.rs | 78 +++++++++++++++++-- .../builders/builder_replace_into.rs | 2 +- .../transforms/hash_join/hash_join_state.rs | 4 +- .../hash_join/probe_join/left_join.rs | 2 +- .../sql/src/planner/optimizer/optimizer.rs | 4 +- 7 files changed, 80 insertions(+), 12 deletions(-) diff --git a/src/common/hashtable/src/hashjoin_hashtable.rs b/src/common/hashtable/src/hashjoin_hashtable.rs index 18aabaef2969..7c126159fa14 100644 --- a/src/common/hashtable/src/hashjoin_hashtable.rs +++ b/src/common/hashtable/src/hashjoin_hashtable.rs @@ -22,7 +22,6 @@ use databend_common_base::mem_allocator::MmapAllocator; use super::traits::HashJoinHashtableLike; use super::traits::Keyable; -use crate::utils::Interval; #[derive(Clone, Copy, Debug)] pub struct RowPtr { diff --git a/src/common/hashtable/src/hashjoin_string_hashtable.rs b/src/common/hashtable/src/hashjoin_string_hashtable.rs index bd5f3e1fd0d3..c7ca141e7f03 100644 --- a/src/common/hashtable/src/hashjoin_string_hashtable.rs +++ b/src/common/hashtable/src/hashjoin_string_hashtable.rs @@ -26,7 +26,6 @@ use crate::hashjoin_hashtable::hash_bits; use crate::hashjoin_hashtable::new_header; use crate::hashjoin_hashtable::remove_header_tag; use crate::traits::hash_join_fast_string_hash; -use crate::utils::Interval; use crate::RowPtr; pub const STRING_EARLY_SIZE: usize = 4; diff --git a/src/common/hashtable/src/utils.rs b/src/common/hashtable/src/utils.rs index ea947c79cd55..cc5037d7cc78 100644 --- a/src/common/hashtable/src/utils.rs +++ b/src/common/hashtable/src/utils.rs @@ -186,16 +186,25 @@ pub mod sse { } } +// This Index is only used for target build merge into (both standalone and distributed mode). +// Advantages: +// 1. Reduces redundant I/O operations, enhancing performance. +// 2. Lowers the maintenance overhead of deduplicating row_id.(But in distributed design, we also need to give rowid) +// 3. Allows the scheduling of the subsequent mutation pipeline to be entirely allocated to not matched append operations. +// Disadvantages: +// 1. This solution is likely to be a one-time approach (especially if there are not matched insert operations involved), +// potentially leading to the target table being unsuitable for use as a build table in the future. +// 2. Requires a significant amount of memory to be efficient and currently does not support spill operations. +// for now we just support sql like below: +// `merge into t using source on xxx when matched then update xxx when not macthed then insert xxx. // for merge into: // we use BlockInfoIndex to maintain an index for the block info in chunks. +#[allow(dead_code)] pub struct BlockInfoIndex { // the intervals will be like below: // (0,10)(11,29),(30,38). it's ordered. - #[allow(dead_code)] intervals: Vec, - #[allow(dead_code)] prefixs: Vec, - #[allow(dead_code)] length: usize, } @@ -287,6 +296,34 @@ impl BlockInfoIndex { } l } + + #[allow(dead_code)] + fn gather_all_partial_block_offsets(&self, hits: &[u8]) -> Vec<(Interval, u64)> { + let mut res = Vec::with_capacity(10); + let mut step = 0; + while step < hits.len() { + if hits[step] == 0 { + break; + } + step += 1; + } + if step == hits.len() { + return res; + } + let mut start = step; + let mut end = step; + while start < hits.len() { + while end < hits.len() && hits[end] == 0 { + end += 1; + } + res.extend(self.get_block_info((start as u32, (end - 1) as u32))); + while end < hits.len() && hits[end] == 1 { + end += 1; + } + start = end; + } + res + } } /// we think the build blocks count is about 1024 at most time. @@ -302,9 +339,9 @@ impl Default for BlockInfoIndex { #[test] fn test_block_info_index() { - // let's build [0,10][11,20][21,30],[31,40],and then find [10,37]. + // let's build [0,10][11,20][21,30],[31,39],and then find [10,37]. // we should get [10,10],[31,37] - let intervals: Vec = vec![(0, 10), (11, 20), (21, 30), (31, 40)]; + let intervals: Vec = vec![(0, 10), (11, 20), (21, 30), (31, 39)]; let find_interval: Interval = (10, 37); let mut block_info_index = BlockInfoIndex::new_with_capacity(10); for (idx, interval) in intervals.iter().enumerate() { @@ -349,4 +386,35 @@ fn test_block_info_index() { assert_eq!(result.len(), 1); assert_eq!(result[0].0, (21, 23)); assert_eq!(result[0].1, 2); + + // test `gather_all_partial_block_offsets` + let mut hits = vec![0; 40]; + // [0,9][28,39] + for item in hits.iter_mut().take(27 + 1).skip(10) { + *item = 1; + } + let result = block_info_index.gather_all_partial_block_offsets(&hits); + assert_eq!(result.len(), 2); + assert_eq!(result[0].0, (0, 9)); + assert_eq!(result[0].1, 0); + assert_eq!(result[1].0, (28, 30)); + assert_eq!(result[1].1, 2); + + let mut hits = vec![0; 40]; + // [0,9] + for item in hits.iter_mut().take(30 + 1).skip(10) { + *item = 1; + } + let result = block_info_index.gather_all_partial_block_offsets(&hits); + assert_eq!(result.len(), 1); + assert_eq!(result[0].0, (0, 9)); + assert_eq!(result[0].1, 0); + + let mut hits = vec![0; 40]; + // [0,10] + for item in hits.iter_mut().take(30 + 1).skip(11) { + *item = 1; + } + let result = block_info_index.gather_all_partial_block_offsets(&hits); + assert_eq!(result.len(), 0); } diff --git a/src/query/service/src/pipelines/builders/builder_replace_into.rs b/src/query/service/src/pipelines/builders/builder_replace_into.rs index ebfc51441b9e..9d75d6db565d 100644 --- a/src/query/service/src/pipelines/builders/builder_replace_into.rs +++ b/src/query/service/src/pipelines/builders/builder_replace_into.rs @@ -158,7 +158,7 @@ impl PipelineBuilder { return Ok(()); } - // The Block Size and Rows is promised by DataSouce by user. + // The Block Size and Rows is promised by DataSource by user. if segment_partition_num == 0 { let dummy_item = create_dummy_item(); // ┌──────────────────────┐ ┌──────────────────┐ diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_state.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_state.rs index bf71c4d0f0c2..3224eb2c6b8c 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_state.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_state.rs @@ -131,10 +131,12 @@ pub struct HashJoinState { /// If we use target table as build side for merge into, we use this to track target table /// and extract partial modified blocks from hashtable pub(crate) merge_into_target_table_index: IndexType, + /// for now we don't support distributed, we will support in the next pr. + #[allow(unused)] pub(crate) is_distributed_merge_into: bool, /// FOR MERGE INTO TARGET TABLE AS BUILD SIDE - /// When merge into target table as build side, we should preseve block info index. + /// When merge into target table as build side, we should preserve block info index. pub(crate) block_info_index: Option>, /// we use matched to tag the matched offset in chunks. pub(crate) matched: SyncUnsafeCell>, diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/probe_join/left_join.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/probe_join/left_join.rs index 32087a94a3e0..5fde4aeab466 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/probe_join/left_join.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/probe_join/left_join.rs @@ -562,7 +562,7 @@ impl HashJoinProbeState { } } else { let mut idx = 0; - self.check_and_set_matched(build_indexes, matched_idx, &validity); + self.check_and_set_matched(build_indexes, matched_idx, &validity)?; while idx < matched_idx { unsafe { let valid = validity.get_bit_unchecked(idx); diff --git a/src/query/sql/src/planner/optimizer/optimizer.rs b/src/query/sql/src/planner/optimizer/optimizer.rs index bdd7a038dbab..0d7892d4690b 100644 --- a/src/query/sql/src/planner/optimizer/optimizer.rs +++ b/src/query/sql/src/planner/optimizer/optimizer.rs @@ -326,7 +326,7 @@ fn optimize_merge_into(opt_ctx: OptimizerContext, plan: Box) -> Resul }; // we just support left join to use MergeIntoBlockInfoHashTable, we - // don't support spill for now, and we need the macthed cluases' count + // don't support spill for now, and we need the macthed clauses' count // is one, just support merge into t using source when matched then // update xx when not matched then insert xx. let flag = plan.matched_evaluators.len() == 1 @@ -380,7 +380,7 @@ fn optimize_merge_into(opt_ctx: OptimizerContext, plan: Box) -> Resul opt_ctx.table_ctx.clone(), plan.merge_type.clone(), plan.target_table_idx, - flag, + false, // we will open it, but for now we don't support distributed )?; (merge_into_join_sexpr.clone(), false) } else { From fc6780f95e596a9141f82975a618e98eac55e935 Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Thu, 11 Jan 2024 13:19:15 +0800 Subject: [PATCH 13/47] remove rowid when use target table as build side --- src/query/catalog/src/merge_into_join.rs | 1 + .../sql/src/planner/binder/merge_into.rs | 1 + .../sql/src/planner/optimizer/optimizer.rs | 19 +++++++++++++------ src/query/sql/src/planner/plans/merge_into.rs | 2 ++ 4 files changed, 17 insertions(+), 6 deletions(-) diff --git a/src/query/catalog/src/merge_into_join.rs b/src/query/catalog/src/merge_into_join.rs index dd1190b810d6..4421b6a9143c 100644 --- a/src/query/catalog/src/merge_into_join.rs +++ b/src/query/catalog/src/merge_into_join.rs @@ -11,6 +11,7 @@ // 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. + #[derive(Clone)] pub enum MergeIntoJoinType { Left, diff --git a/src/query/sql/src/planner/binder/merge_into.rs b/src/query/sql/src/planner/binder/merge_into.rs index 5248e9c3d893..b1fe349f214e 100644 --- a/src/query/sql/src/planner/binder/merge_into.rs +++ b/src/query/sql/src/planner/binder/merge_into.rs @@ -407,6 +407,7 @@ impl Binder { merge_type, distributed: false, change_join_order: false, + row_id_index: column_binding.index, }) } diff --git a/src/query/sql/src/planner/optimizer/optimizer.rs b/src/query/sql/src/planner/optimizer/optimizer.rs index 0d7892d4690b..6f48ccd880e1 100644 --- a/src/query/sql/src/planner/optimizer/optimizer.rs +++ b/src/query/sql/src/planner/optimizer/optimizer.rs @@ -13,6 +13,7 @@ // limitations under the License. use std::collections::HashMap; +use std::collections::HashSet; use std::sync::Arc; use databend_common_ast::ast::ExplainKind; @@ -332,6 +333,7 @@ fn optimize_merge_into(opt_ctx: OptimizerContext, plan: Box) -> Resul let flag = plan.matched_evaluators.len() == 1 && plan.matched_evaluators[0].condition.is_none() && plan.matched_evaluators[0].update.is_some(); + let mut new_columns_set = plan.columns_set.clone(); if change_join_order && matches!(plan.merge_type, MergeIntoType::FullOperation) && opt_ctx @@ -341,6 +343,7 @@ fn optimize_merge_into(opt_ctx: OptimizerContext, plan: Box) -> Resul == 0 && flag { + new_columns_set.remove(&plan.row_id_index); opt_ctx.table_ctx.set_merge_into_join(MergeIntoJoin { merge_into_join_type: MergeIntoJoinType::Left, is_distributed: false, @@ -378,9 +381,9 @@ fn optimize_merge_into(opt_ctx: OptimizerContext, plan: Box) -> Resul merge_into_join_sexpr, change_join_order, opt_ctx.table_ctx.clone(), - plan.merge_type.clone(), - plan.target_table_idx, + plan.as_ref(), false, // we will open it, but for now we don't support distributed + new_columns_set.as_mut(), )?; (merge_into_join_sexpr.clone(), false) } else { @@ -394,12 +397,14 @@ fn optimize_merge_into(opt_ctx: OptimizerContext, plan: Box) -> Resul input: Box::new(optimized_distributed_merge_into_join_sexpr), distributed, change_join_order, + columns_set: new_columns_set.clone(), ..*plan }))) } else { Ok(Plan::MergeInto(Box::new(MergeInto { input: join_sexpr, change_join_order, + columns_set: new_columns_set, ..*plan }))) } @@ -409,9 +414,9 @@ fn try_to_change_as_broadcast_join( merge_into_join_sexpr: SExpr, change_join_order: bool, table_ctx: Arc, - merge_into_type: MergeIntoType, - target_tbl_idx: usize, + plan: &MergeInto, only_one_matched_clause: bool, + new_columns_set: &mut HashSet, ) -> Result { if let RelOperator::Exchange(Exchange::Merge) = merge_into_join_sexpr.plan.as_ref() { let right_exchange = merge_into_join_sexpr.child(0)?.child(1)?; @@ -424,13 +429,15 @@ fn try_to_change_as_broadcast_join( // for now, when we use target table as build side and it's a broadcast join, // we will use merge_into_block_info_hashtable to reduce i/o operations. if change_join_order - && matches!(merge_into_type, MergeIntoType::FullOperation) + && matches!(plan.merge_type, MergeIntoType::FullOperation) && only_one_matched_clause { + // remove rowid + new_columns_set.remove(&plan.row_id_index); table_ctx.set_merge_into_join(MergeIntoJoin { merge_into_join_type: MergeIntoJoinType::Left, is_distributed: true, - target_tbl_idx, + target_tbl_idx: plan.target_table_idx, }) } return Ok(merge_into_join_sexpr.replace_children(vec![Arc::new(join_s_expr)])); diff --git a/src/query/sql/src/planner/plans/merge_into.rs b/src/query/sql/src/planner/plans/merge_into.rs index a4df99c73593..4c683c565135 100644 --- a/src/query/sql/src/planner/plans/merge_into.rs +++ b/src/query/sql/src/planner/plans/merge_into.rs @@ -68,6 +68,8 @@ pub struct MergeInto { pub merge_type: MergeIntoType, pub distributed: bool, pub change_join_order: bool, + // when we use target table as build side, we need to remove rowid columns. + pub row_id_index: IndexType, } impl std::fmt::Debug for MergeInto { From 859f0e195142d16f7d5fd7f57db1303d60595b8d Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Thu, 11 Jan 2024 15:35:19 +0800 Subject: [PATCH 14/47] support target_build_optimization for merge into pipeline in standalone mode --- src/query/catalog/src/merge_into_join.rs | 5 +- .../interpreters/interpreter_merge_into.rs | 4 +- .../pipelines/builders/builder_merge_into.rs | 2 + .../physical_plans/physical_merge_into.rs | 1 + .../storages/fuse/src/operations/merge.rs | 2 + .../merge_into/mutator/matched_mutator.rs | 70 +++++++++++++++++-- .../processor_merge_into_matched_and_split.rs | 10 +++ .../processors/processor_merge_into_split.rs | 21 +++++- ...sor_merge_into_split_row_number_and_log.rs | 1 + 9 files changed, 106 insertions(+), 10 deletions(-) diff --git a/src/query/catalog/src/merge_into_join.rs b/src/query/catalog/src/merge_into_join.rs index 4421b6a9143c..db48f84b66a5 100644 --- a/src/query/catalog/src/merge_into_join.rs +++ b/src/query/catalog/src/merge_into_join.rs @@ -37,8 +37,9 @@ impl Default for MergeIntoJoin { fn default() -> Self { Self { merge_into_join_type: MergeIntoJoinType::NormalJoin, - is_distributed: Default::default(), - target_tbl_idx: Default::default(), + is_distributed: false, + // Invalid Index + target_tbl_idx: usize::MAX, } } } diff --git a/src/query/service/src/interpreters/interpreter_merge_into.rs b/src/query/service/src/interpreters/interpreter_merge_into.rs index dfae695fb898..038e03f00eff 100644 --- a/src/query/service/src/interpreters/interpreter_merge_into.rs +++ b/src/query/service/src/interpreters/interpreter_merge_into.rs @@ -373,7 +373,7 @@ impl MergeIntoInterpreter { .into_iter() .enumerate() .collect(); - + let target_build_optimization = !self.plan.columns_set.contains(&self.plan.row_id_index); let commit_input = if !distributed { // recv datablocks from matched upstream and unmatched upstream // transform and append dat @@ -390,6 +390,7 @@ impl MergeIntoInterpreter { output_schema: DataSchemaRef::default(), merge_type: merge_type.clone(), change_join_order: *change_join_order, + target_build_optimization, })) } else { let merge_append = PhysicalPlan::MergeInto(Box::new(MergeInto { @@ -415,6 +416,7 @@ impl MergeIntoInterpreter { }, merge_type: merge_type.clone(), change_join_order: *change_join_order, + target_build_optimization: false, // we don't support for distributed mode for now.. })); // if change_join_order = true, it means the target is build side, // in this way, we will do matched operation and not matched operation diff --git a/src/query/service/src/pipelines/builders/builder_merge_into.rs b/src/query/service/src/pipelines/builders/builder_merge_into.rs index fc6847ea6098..85b73bcc53c5 100644 --- a/src/query/service/src/pipelines/builders/builder_merge_into.rs +++ b/src/query/service/src/pipelines/builders/builder_merge_into.rs @@ -308,6 +308,7 @@ impl PipelineBuilder { block_builder, io_request_semaphore, segments.clone(), + false, // we don't support for distributed mode. )?, create_dummy_item(), ])); @@ -768,6 +769,7 @@ impl PipelineBuilder { block_builder, io_request_semaphore, segments.clone(), + merge_into.target_build_optimization, )?); } } diff --git a/src/query/sql/src/executor/physical_plans/physical_merge_into.rs b/src/query/sql/src/executor/physical_plans/physical_merge_into.rs index 7b177506e8e8..b51f445c5aa3 100644 --- a/src/query/sql/src/executor/physical_plans/physical_merge_into.rs +++ b/src/query/sql/src/executor/physical_plans/physical_merge_into.rs @@ -52,6 +52,7 @@ pub struct MergeInto { pub distributed: bool, pub merge_type: MergeIntoType, pub change_join_order: bool, + pub target_build_optimization: bool, } #[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] diff --git a/src/query/storages/fuse/src/operations/merge.rs b/src/query/storages/fuse/src/operations/merge.rs index 450a3b96c286..fc967f3e1a6b 100644 --- a/src/query/storages/fuse/src/operations/merge.rs +++ b/src/query/storages/fuse/src/operations/merge.rs @@ -77,6 +77,7 @@ impl FuseTable { block_builder: BlockBuilder, io_request_semaphore: Arc, segment_locations: Vec<(SegmentIndex, Location)>, + target_build_optimization: bool, ) -> Result { let read_settings = ReadSettings::from_ctx(&ctx)?; let aggregator = MatchedAggregator::create( @@ -88,6 +89,7 @@ impl FuseTable { block_builder, io_request_semaphore, segment_locations, + target_build_optimization, )?; Ok(aggregator.into_pipe_item()) } diff --git a/src/query/storages/fuse/src/operations/merge_into/mutator/matched_mutator.rs b/src/query/storages/fuse/src/operations/merge_into/mutator/matched_mutator.rs index f149b73a43b3..df4eaa9443f9 100644 --- a/src/query/storages/fuse/src/operations/merge_into/mutator/matched_mutator.rs +++ b/src/query/storages/fuse/src/operations/merge_into/mutator/matched_mutator.rs @@ -78,6 +78,8 @@ pub struct MatchedAggregator { segment_locations: AHashMap, block_mutation_row_offset: HashMap, HashSet)>, aggregation_ctx: Arc, + target_build_optimization: bool, + meta_indexes: HashSet<(SegmentIndex, BlockIndex)>, } impl MatchedAggregator { @@ -91,6 +93,7 @@ impl MatchedAggregator { block_builder: BlockBuilder, io_request_semaphore: Arc, segment_locations: Vec<(SegmentIndex, Location)>, + target_build_optimization: bool, ) -> Result { let segment_reader = MetaReaders::segment_info_reader(data_accessor.clone(), target_table_schema.clone()); @@ -123,11 +126,31 @@ impl MatchedAggregator { block_mutation_row_offset: HashMap::new(), segment_locations: AHashMap::from_iter(segment_locations), ctx: ctx.clone(), + target_build_optimization, + meta_indexes: HashSet::new(), }) } #[async_backtrace::framed] pub async fn accumulate(&mut self, data_block: DataBlock) -> Result<()> { + // An optimization: If we use target table as build side, the deduplicate will be done + // in hashtable probe phase.In this case, We don't support delete for now, so we + // don't to add MergeStatus here. + if data_block.get_meta().is_some() && data_block.is_empty() { + let meta_index = BlockMetaIndex::downcast_ref_from(data_block.get_meta().unwrap()); + if meta_index.is_some() { + let meta_index = meta_index.unwrap(); + if !self + .meta_indexes + .insert((meta_index.segment_idx, meta_index.block_idx)) + { + return Err(ErrorCode::Internal( + "merge into get duplicated block for target build unmodified optimization", + )); + } + } + return Ok(()); + } if data_block.is_empty() { return Ok(()); } @@ -192,10 +215,23 @@ impl MatchedAggregator { let start = Instant::now(); // 1.get modified segments let mut segment_infos = HashMap::::new(); + let segment_indexes = if self.target_build_optimization { + let mut vecs = Vec::with_capacity(self.meta_indexes.len()); + for prefix in &self.meta_indexes { + vecs.push(prefix.0); + } + vecs + } else { + let mut vecs = Vec::with_capacity(self.block_mutation_row_offset.len()); + for prefix in self.block_mutation_row_offset.keys() { + let (segment_idx, _) = split_prefix(*prefix); + let segment_idx = segment_idx as usize; + vecs.push(segment_idx); + } + vecs + }; - for prefix in self.block_mutation_row_offset.keys() { - let (segment_idx, _) = split_prefix(*prefix); - let segment_idx = segment_idx as usize; + for segment_idx in segment_indexes { if let Entry::Vacant(e) = segment_infos.entry(segment_idx) { let (path, ver) = self.segment_locations.get(&segment_idx).ok_or_else(|| { ErrorCode::Internal(format!( @@ -220,6 +256,30 @@ impl MatchedAggregator { } } + if self.target_build_optimization { + let mut mutation_logs = Vec::with_capacity(self.meta_indexes.len()); + for item in &self.meta_indexes { + let segment_idx = item.0; + let block_idx = item.1; + let segment_info = segment_infos.get(&item.0).unwrap(); + let block_idx = segment_info.blocks.len() - block_idx - 1; + info!( + "merge into apply: segment_idx:{},blk_idx:{}", + segment_idx, block_idx + ); + mutation_logs.push(MutationLogEntry::DeletedBlock { + index: BlockMetaIndex { + segment_idx, + block_idx, + inner: None, + }, + }) + } + return Ok(Some(MutationLogs { + entries: mutation_logs, + })); + } + let io_runtime = GlobalIORuntime::instance(); let mut mutation_log_handlers = Vec::with_capacity(self.block_mutation_row_offset.len()); @@ -229,12 +289,12 @@ impl MatchedAggregator { let permit = acquire_task_permit(self.io_request_semaphore.clone()).await?; let aggregation_ctx = self.aggregation_ctx.clone(); let segment_info = segment_infos.get(&segment_idx).unwrap(); + let block_idx = segment_info.blocks.len() - block_idx as usize - 1; + assert!(block_idx < segment_info.blocks.len()); info!( "merge into apply: segment_idx:{},blk_idx:{}", segment_idx, block_idx ); - let block_idx = segment_info.blocks.len() - block_idx as usize - 1; - assert!(block_idx < segment_info.blocks.len()); // the row_id is generated by block_id, not block_idx,reference to fill_internal_column_meta() let block_meta = segment_info.blocks[block_idx].clone(); diff --git a/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_matched_and_split.rs b/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_matched_and_split.rs index 306fe9d88d70..1092400e1849 100644 --- a/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_matched_and_split.rs +++ b/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_matched_and_split.rs @@ -42,6 +42,7 @@ use databend_common_storage::MergeStatus; use crate::operations::common::MutationLogs; use crate::operations::merge_into::mutator::DeleteByExprMutator; use crate::operations::merge_into::mutator::UpdateByExprMutator; +use crate::operations::BlockMetaIndex; #[derive(serde::Serialize, serde::Deserialize, Clone, Debug, PartialEq)] pub struct SourceFullMatched; @@ -262,6 +263,15 @@ impl Processor for MatchedSplitProcessor { fn process(&mut self) -> Result<()> { if let Some(data_block) = self.input_data.take() { + // we receive a partial unmodified block data meta. + if data_block.get_meta().is_some() && data_block.is_empty() { + let meta_index = BlockMetaIndex::downcast_ref_from(data_block.get_meta().unwrap()); + if meta_index.is_some() { + self.output_data_row_id_data.push(data_block); + return Ok(()); + } + } + if data_block.is_empty() { return Ok(()); } diff --git a/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_split.rs b/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_split.rs index ab3c5fa03d38..3701e2238c75 100644 --- a/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_split.rs +++ b/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_split.rs @@ -17,6 +17,7 @@ use std::sync::Arc; use std::time::Instant; use databend_common_exception::Result; +use databend_common_expression::BlockMetaInfoDowncast; use databend_common_expression::DataBlock; use databend_common_metrics::storage::*; use databend_common_pipeline_core::processors::Event; @@ -29,7 +30,13 @@ use databend_common_pipeline_core::PipeItem; use super::processor_merge_into_matched_and_split::SourceFullMatched; use crate::operations::merge_into::mutator::MergeIntoSplitMutator; +use crate::operations::BlockMetaIndex; +// There are two kinds of usage for this processor: +// 1. we will receive a probed datablock from join, and split it by rowid into matched block and unmatched block +// 2. we will receive a unmatched datablock, but this is an optimization for target table as build side. The unmatched +// datablock is a physical block's partial unmodified block. And its meta is a prefix(segment_id_block_id). +// we use the meta to distinct 1 and 2. pub struct MergeIntoSplitProcessor { input_port: Arc, output_port_matched: Arc, @@ -143,9 +150,19 @@ impl Processor for MergeIntoSplitProcessor { } } - // Todo:(JackTan25) accutally, we should do insert-only optimization in the future. fn process(&mut self) -> Result<()> { - if let Some(data_block) = self.input_data.take() { + if let Some(mut data_block) = self.input_data.take() { + // we receive a partial unmodified block data. please see details at the top of this file. + if data_block.get_meta().is_some() { + let meta_index = BlockMetaIndex::downcast_from(data_block.take_meta().unwrap()); + if meta_index.is_some() { + self.output_data_not_matched_data = Some(data_block.clone()); + // if the downstream receive this, it should just treat this as a DeletedLog. + self.output_data_matched_data = + Some(DataBlock::empty_with_meta(Box::new(meta_index.unwrap()))); + return Ok(()); + } + } // for distributed execution, if one node matched all source data. // if we use right join, we will receive a empty block, but we must // give it to downstream. diff --git a/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_split_row_number_and_log.rs b/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_split_row_number_and_log.rs index b458d068af21..ac85c711b2e6 100644 --- a/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_split_row_number_and_log.rs +++ b/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_split_row_number_and_log.rs @@ -29,6 +29,7 @@ use databend_common_pipeline_core::PipeItem; use super::processor_merge_into_matched_and_split::SourceFullMatched; use crate::operations::merge_into::processors::RowIdKind; +// for distributed merge into (source as build and it will be broadcast) pub struct RowNumberAndLogSplitProcessor { input_port: Arc, output_port_row_number: Arc, From 58810d66cc4df6b3f296acbfbcd45e23778dad73 Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Thu, 11 Jan 2024 19:07:39 +0800 Subject: [PATCH 15/47] add more tests, and enhance explain merge into, add fix add merge status when target table build optimization is triggered --- .../interpreters/interpreter_merge_into.rs | 4 +- .../sql/src/planner/format/display_plan.rs | 21 +- .../processor_merge_into_not_matched.rs | 18 +- .../processors/processor_merge_into_split.rs | 9 +- ...39t_arget_build_merge_into_standalone.test | 259 ++++++++++++++++++ .../mode/standalone/explain/merge_into.test | 4 + 6 files changed, 301 insertions(+), 14 deletions(-) create mode 100644 tests/sqllogictests/suites/base/09_fuse_engine/09_0039t_arget_build_merge_into_standalone.test diff --git a/src/query/service/src/interpreters/interpreter_merge_into.rs b/src/query/service/src/interpreters/interpreter_merge_into.rs index 038e03f00eff..9ff350d2c06b 100644 --- a/src/query/service/src/interpreters/interpreter_merge_into.rs +++ b/src/query/service/src/interpreters/interpreter_merge_into.rs @@ -373,7 +373,9 @@ impl MergeIntoInterpreter { .into_iter() .enumerate() .collect(); - let target_build_optimization = !self.plan.columns_set.contains(&self.plan.row_id_index); + let target_build_optimization = + matches!(self.plan.merge_type, MergeIntoType::FullOperation) + && !self.plan.columns_set.contains(&self.plan.row_id_index); let commit_input = if !distributed { // recv datablocks from matched upstream and unmatched upstream // transform and append dat diff --git a/src/query/sql/src/planner/format/display_plan.rs b/src/query/sql/src/planner/format/display_plan.rs index 516ee0492de6..f13dae5c1931 100644 --- a/src/query/sql/src/planner/format/display_plan.rs +++ b/src/query/sql/src/planner/format/display_plan.rs @@ -23,6 +23,7 @@ use databend_common_expression::ROW_ID_COL_NAME; use itertools::Itertools; use crate::binder::ColumnBindingBuilder; +use crate::binder::MergeIntoType; use crate::format_scalar; use crate::optimizer::SExpr; use crate::planner::format::display_rel_operator::FormatContext; @@ -292,6 +293,16 @@ fn format_merge_into(merge_into: &MergeInto) -> Result { table_entry.database(), table_entry.name(), )); + let target_build_optimization = matches!(merge_into.merge_type, MergeIntoType::FullOperation) + && !merge_into.columns_set.contains(&merge_into.row_id_index); + let target_build_optimization_format = FormatTreeNode::new(FormatContext::Text(format!( + "target_build_optimization: {}", + target_build_optimization + ))); + let distributed_format = FormatTreeNode::new(FormatContext::Text(format!( + "distributed: {}", + merge_into.distributed + ))); // add macthed clauses let mut matched_children = Vec::with_capacity(merge_into.matched_evaluators.len()); @@ -351,9 +362,13 @@ fn format_merge_into(merge_into: &MergeInto) -> Result { } let s_expr = merge_into.input.as_ref(); let input_format_child = s_expr.to_format_tree(&merge_into.meta_data); - let all_children = [matched_children, unmatched_children, vec![ - input_format_child, - ]] + let all_children = [ + vec![distributed_format], + vec![target_build_optimization_format], + matched_children, + unmatched_children, + vec![input_format_child], + ] .concat(); let res = FormatTreeNode::with_children(target_table_format, all_children).format_pretty()?; Ok(format!("MergeInto:\n{res}")) diff --git a/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_not_matched.rs b/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_not_matched.rs index 17a8958353e5..2debb6006ad8 100644 --- a/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_not_matched.rs +++ b/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_not_matched.rs @@ -20,6 +20,7 @@ use std::time::Instant; use databend_common_catalog::table_context::TableContext; use databend_common_exception::Result; +use databend_common_expression::BlockMetaInfoDowncast; use databend_common_expression::DataBlock; use databend_common_expression::DataSchemaRef; use databend_common_expression::FunctionContext; @@ -37,6 +38,7 @@ use databend_common_storage::MergeStatus; use itertools::Itertools; use crate::operations::merge_into::mutator::SplitByExprMutator; +use crate::operations::BlockMetaIndex; // (source_schema,condition,values_exprs) type UnMatchedExprs = Vec<(DataSchemaRef, Option, Vec)>; @@ -157,6 +159,9 @@ impl Processor for MergeIntoNotMatchedProcessor { if data_block.is_empty() { return Ok(()); } + // target build optimization + let no_need_add_status = data_block.get_meta().is_some() + && BlockMetaIndex::downcast_ref_from(data_block.get_meta().unwrap()).is_some(); let start = Instant::now(); let mut current_block = data_block; for (idx, op) in self.ops.iter().enumerate() { @@ -169,12 +174,13 @@ impl Processor for MergeIntoNotMatchedProcessor { metrics_inc_merge_into_append_blocks_rows_counter( satisfied_block.num_rows() as u32 ); - - self.ctx.add_merge_status(MergeStatus { - insert_rows: satisfied_block.num_rows(), - update_rows: 0, - deleted_rows: 0, - }); + if !no_need_add_status { + self.ctx.add_merge_status(MergeStatus { + insert_rows: satisfied_block.num_rows(), + update_rows: 0, + deleted_rows: 0, + }); + } self.output_data .push(op.op.execute(&self.func_ctx, satisfied_block)?) diff --git a/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_split.rs b/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_split.rs index 3701e2238c75..c4b0f710edbd 100644 --- a/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_split.rs +++ b/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_split.rs @@ -151,15 +151,16 @@ impl Processor for MergeIntoSplitProcessor { } fn process(&mut self) -> Result<()> { - if let Some(mut data_block) = self.input_data.take() { + if let Some(data_block) = self.input_data.take() { // we receive a partial unmodified block data. please see details at the top of this file. if data_block.get_meta().is_some() { - let meta_index = BlockMetaIndex::downcast_from(data_block.take_meta().unwrap()); + let meta_index = BlockMetaIndex::downcast_ref_from(data_block.get_meta().unwrap()); if meta_index.is_some() { self.output_data_not_matched_data = Some(data_block.clone()); // if the downstream receive this, it should just treat this as a DeletedLog. - self.output_data_matched_data = - Some(DataBlock::empty_with_meta(Box::new(meta_index.unwrap()))); + self.output_data_matched_data = Some(DataBlock::empty_with_meta(Box::new( + meta_index.unwrap().clone(), + ))); return Ok(()); } } diff --git a/tests/sqllogictests/suites/base/09_fuse_engine/09_0039t_arget_build_merge_into_standalone.test b/tests/sqllogictests/suites/base/09_fuse_engine/09_0039t_arget_build_merge_into_standalone.test new file mode 100644 index 000000000000..6489167d0157 --- /dev/null +++ b/tests/sqllogictests/suites/base/09_fuse_engine/09_0039t_arget_build_merge_into_standalone.test @@ -0,0 +1,259 @@ +statement ok +set enable_experimental_merge_into = 1; + +## Target Build Optimization Test +statement ok +create table target_build_optimization(a int,b string,c string); + +statement ok +create table source_optimization(a int,b string,c string); + +### 1. add 4 blocks for target_build_optimization +statement ok +insert into target_build_optimization values(1,'b1','c1'),(2,'b2','c2'); + +statement ok +insert into target_build_optimization values(3,'b3','c3'),(4,'b4','c4'); + +statement ok +insert into target_build_optimization values(5,'b5','c5'),(6,'b6','c6'); + +statement ok +insert into target_build_optimization values(7,'b7','c7'),(8,'b8','c8'); + +### 2. we need to make source_optimization is larger than target_build_optimization +### 2.1 test multi columns matched, +statement ok +insert into source_optimization values(1,'b1','c_1'),(1,'b1','c_2'); + +statement ok +insert into source_optimization values(3,'b3','c3'),(4,'b4','c4'); + +statement ok +insert into source_optimization values(5,'b5','c5'),(6,'b6','c6'); + +statement ok +insert into source_optimization values(7,'b7','c7'),(8,'b8','c8'); + +statement ok +insert into source_optimization values(7,'b7','c7'),(8,'b8','c8'); + +statement ok +insert into source_optimization values(5,'b5','c5'),(6,'b6','c6'); + +statement error 4001 +merge into target_build_optimization as t1 using source_optimization as t2 on t1.a = t2.a and t1.b = t2.b when matched then update * when not matched then insert *; + +### 2.2 make sure the plan is expected +query T +explain merge into target_build_optimization as t1 using source_optimization as t2 on t1.a = t2.a and t1.b = t2.b when matched then update * when not matched then insert *; +---- +MergeInto: +target_table: default.default.target_build_optimization +├── distributed: false +├── target_build_optimization: true +├── matched update: [condition: None,update set c = c (#2),b = b (#1),a = a (#0)] +├── unmatched insert: [condition: None,insert into (a,b,c) values(CAST(a (#0) AS Int32 NULL),CAST(b (#1) AS String NULL),CAST(c (#2) AS String NULL))] +└── HashJoin: LEFT OUTER + ├── equi conditions: [and(eq(t2.a (#0), t1.a (#3)), eq(t2.b (#1), t1.b (#4)))] + ├── non-equi conditions: [] + ├── LogicalGet + │ ├── table: default.default.source_optimization + │ ├── filters: [] + │ ├── order by: [] + │ └── limit: NONE + └── LogicalGet + ├── table: default.default.target_build_optimization + ├── filters: [] + ├── order by: [] + └── limit: NONE + +### 3. test with conjunct and without conjunct +statement ok +truncate table source_optimization; + +## partial updated +statement ok +insert into source_optimization values(1,'b1','c_1') + +statement ok +insert into source_optimization values(3,'b3','c_3') + +## all updated +statement ok +insert into source_optimization values(5,'b5','c_5'),(6,'b6','c_6'); + +statement ok +insert into source_optimization values(7,'b7','c_7'),(8,'b8','c_8'); + +## insert +statement ok +insert into source_optimization values(9,'b9','c_9'),(10,'b10','c_10'); + +statement ok +insert into source_optimization values(11,'b11','c_11'),(12,'b12','c_12'); + +### test block counts +query T +select count(*) from fuse_block('default','source_optimization'); +---- +6 + +query T +select count(*) from fuse_block('default','target_build_optimization'); +---- +4 + +### make sure the plan is expected +query T +explain merge into target_build_optimization as t1 using source_optimization as t2 on t1.a = t2.a and t1.b = t2.b when matched then update * when not matched then insert *; +---- +MergeInto: +target_table: default.default.target_build_optimization +├── distributed: false +├── target_build_optimization: true +├── matched update: [condition: None,update set c = c (#2),b = b (#1),a = a (#0)] +├── unmatched insert: [condition: None,insert into (a,b,c) values(CAST(a (#0) AS Int32 NULL),CAST(b (#1) AS String NULL),CAST(c (#2) AS String NULL))] +└── HashJoin: LEFT OUTER + ├── equi conditions: [and(eq(t2.a (#0), t1.a (#3)), eq(t2.b (#1), t1.b (#4)))] + ├── non-equi conditions: [] + ├── LogicalGet + │ ├── table: default.default.source_optimization + │ ├── filters: [] + │ ├── order by: [] + │ └── limit: NONE + └── LogicalGet + ├── table: default.default.target_build_optimization + ├── filters: [] + ├── order by: [] + └── limit: NONE + +## test without conjunct +query TT +merge into target_build_optimization as t1 using source_optimization as t2 on t1.a = t2.a and t1.b = t2.b when matched then update * when not matched then insert *; +---- +4 6 + +query TTT +select * from target_build_optimization order by a,b,c; +---- +1 b1 c_1 +2 b2 c2 +3 b3 c_3 +4 b4 c4 +5 b5 c_5 +6 b6 c_6 +7 b7 c_7 +8 b8 c_8 +9 b9 c_9 +10 b10 c_10 +11 b11 c_11 +12 b12 c_12 + +### test with conjunct +#### we need to make sure the blocks count and layout, so we should truncate and insert again. +statement ok +truncate table target_build_optimization; + +statement ok +insert into target_build_optimization values(1,'b1','c1'),(3,'b2','c2'); + +statement ok +insert into target_build_optimization values(3,'b3','c3'),(5,'b4','c4'); + +statement ok +insert into target_build_optimization values(7,'b5','c5'),(8,'b6','c6'); + +statement ok +insert into target_build_optimization values(7,'b7','c7'),(8,'b8','c8'); + +### test block counts +query T +select count(*) from fuse_block('default','source_optimization'); +---- +6 + +query T +select count(*) from fuse_block('default','target_build_optimization'); +---- +4 + +### make sure the plan is expected +query T +explain merge into target_build_optimization as t1 using source_optimization as t2 on t1.a = t2.a and t1.b = t2.b when matched then update * when not matched then insert *; +---- +MergeInto: +target_table: default.default.target_build_optimization +├── distributed: false +├── target_build_optimization: true +├── matched update: [condition: None,update set b = b (#1),a = a (#0),c = c (#2)] +├── unmatched insert: [condition: None,insert into (a,b,c) values(CAST(a (#0) AS Int32 NULL),CAST(b (#1) AS String NULL),CAST(c (#2) AS String NULL))] +└── HashJoin: LEFT OUTER + ├── equi conditions: [eq(t2.b (#1), t1.b (#4))] + ├── non-equi conditions: [gt(t1.a (#3), t2.a (#0))] + ├── LogicalGet + │ ├── table: default.default.source_optimization + │ ├── filters: [] + │ ├── order by: [] + │ └── limit: NONE + └── LogicalGet + ├── table: default.default.target_build_optimization + ├── filters: [] + ├── order by: [] + └── limit: NONE + +statement ok +update source_optimization set a = 2,b = 'b2' where a = 3 and b = 'b3'; + +query TTT +select * from source_optimization order by a,b,c; +---- +1 b1 c_1 +2 b2 c_3 +5 b5 c_5 +6 b6 c_6 +7 b7 c_7 +8 b8 c_8 +9 b9 c_9 +10 b10 c_10 +11 b11 c_11 +12 b12 c_12 + +query TTT +select * from target_build_optimization order by a,b,c; +---- +1 b1 c1 +3 b2 c2 +3 b3 c3 +5 b4 c4 +7 b5 c5 +7 b7 c7 +8 b6 c6 +8 b8 c8 + +query TT +merge into target_build_optimization as t1 using source_optimization as t2 on t1.a > t2.a and t1.b = t2.b when matched then update * when not matched then insert *; +---- +7 3 + +query TTT +select * from target_build_optimization order by a,b,c; +---- +1 b1 c1 +1 b1 c_1 +2 b2 c_3 +3 b3 c3 +5 b4 c4 +5 b5 c_5 +6 b6 c_6 +7 b7 c7 +7 b7 c_7 +8 b8 c8 +8 b8 c_8 +9 b9 c_9 +10 b10 c_10 +11 b11 c_11 +12 b12 c_12 + +statement ok +set enable_experimental_merge_into = 0; \ No newline at end of file diff --git a/tests/sqllogictests/suites/mode/standalone/explain/merge_into.test b/tests/sqllogictests/suites/mode/standalone/explain/merge_into.test index f713b4831cca..e42f6752844d 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/merge_into.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/merge_into.test @@ -23,6 +23,8 @@ explain MERGE INTO salaries2 USING (SELECT * FROM employees2) as employees2 ON s ---- MergeInto: target_table: default.default.salaries2 +├── distributed: false | +├── target_build_optimization: false ├── matched update: [condition: eq(employees2.department (#2), 'HR'),update set salary = plus(salaries2.salary (#4), 1000.00)] ├── matched update: [condition: None,update set salary = plus(salaries2.salary (#4), 500.00)] ├── unmatched insert: [condition: None,insert into (employee_id,salary) values(CAST(employees2.employee_id (#0) AS Int32 NULL),CAST(55000.00 AS Decimal(10, 2) NULL))] @@ -50,6 +52,8 @@ explain MERGE INTO salaries2 USING (SELECT * FROM employees2) as employees2 ON s ---- MergeInto: target_table: default.default.salaries2 +├── distributed: false | +├── target_build_optimization: false ├── matched update: [condition: eq(employees2.department (#2), 'HR'),update set salary = plus(salaries2.salary (#4), 1000.00)] ├── matched update: [condition: None,update set salary = plus(salaries2.salary (#4), 500.00)] ├── unmatched insert: [condition: None,insert into (employee_id,salary) values(CAST(employees2.employee_id (#0) AS Int32 NULL),CAST(55000.00 AS Decimal(10, 2) NULL))] From d316c8ab50d2fe644fa556cd03a33979999e28dd Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Fri, 12 Jan 2024 00:37:33 +0800 Subject: [PATCH 16/47] add probe done output logic and add more tests --- src/common/hashtable/src/lib.rs | 1 + src/common/hashtable/src/utils.rs | 48 +++++++++++++- .../hash_join/hash_join_build_state.rs | 8 ++- .../hash_join/hash_join_probe_state.rs | 38 ++++++++++++ .../transforms/hash_join/hash_join_state.rs | 7 ++- .../hash_join/transform_hash_join_probe.rs | 62 ++++++++++++++++++- ...39t_arget_build_merge_into_standalone.test | 47 ++++++++++++++ 7 files changed, 203 insertions(+), 8 deletions(-) diff --git a/src/common/hashtable/src/lib.rs b/src/common/hashtable/src/lib.rs index d35eeba01a85..efddeefed660 100644 --- a/src/common/hashtable/src/lib.rs +++ b/src/common/hashtable/src/lib.rs @@ -114,3 +114,4 @@ pub type HashJoinHashMap = hashjoin_hashtable::HashJoinHashTable; pub type StringHashJoinHashMap = hashjoin_string_hashtable::HashJoinStringHashTable; pub use traits::HashJoinHashtableLike; pub use utils::BlockInfoIndex; +pub use utils::Interval; diff --git a/src/common/hashtable/src/utils.rs b/src/common/hashtable/src/utils.rs index cc5037d7cc78..e8de092b9365 100644 --- a/src/common/hashtable/src/utils.rs +++ b/src/common/hashtable/src/utils.rs @@ -298,7 +298,7 @@ impl BlockInfoIndex { } #[allow(dead_code)] - fn gather_all_partial_block_offsets(&self, hits: &[u8]) -> Vec<(Interval, u64)> { + pub fn gather_all_partial_block_offsets(&self, hits: &[u8]) -> Vec<(Interval, u64)> { let mut res = Vec::with_capacity(10); let mut step = 0; while step < hits.len() { @@ -324,6 +324,33 @@ impl BlockInfoIndex { } res } + + /// return [{(Interval,prefix),(Interval,prefix)},chunk_idx] + pub fn chunk_offsets( + &self, + partial_unmodified: &Vec<(Interval, u64)>, + chunks_offsets: &Vec, + ) -> Vec<(Vec<(Interval, u64)>, u32)> { + let mut res = Vec::with_capacity(chunks_offsets.len()); + let mut chunk_idx = 0; + let mut partial_idx = 0; + let mut offset = 0; + while chunk_idx < chunks_offsets.len() && partial_idx < partial_unmodified.len() { + // here is '<', not '<=', chunks_offsets[chunk_idx] is the count of chunks[chunk_idx] + if partial_unmodified[partial_idx].0.1 < chunks_offsets[chunk_idx] { + if offset >= res.len() { + res.push((Vec::new(), chunk_idx as u32)); + } + res[offset].0.push(partial_unmodified[partial_idx]) + } else { + offset += 1; + chunk_idx += 1; + partial_idx -= 1; + } + partial_idx += 1; + } + res + } } /// we think the build blocks count is about 1024 at most time. @@ -417,4 +444,23 @@ fn test_block_info_index() { } let result = block_info_index.gather_all_partial_block_offsets(&hits); assert_eq!(result.len(), 0); + + // test chunk_offsets + // blocks: [0,10][11,20][21,30],[31,39] + // chunks: [0,20],[21,39] + // chunks_offsets: [21],[40] + // partial_unmodified: [((8,10),0),((13,16),1),((33,36),3)] + let partial_unmodified = vec![((8, 10), 0), ((13, 16), 1), ((33, 36), 3)]; + let chunks_offsets = vec![21, 40]; + let res = block_info_index.chunk_offsets(&partial_unmodified, &chunks_offsets); + assert_eq!(res.len(), 2); + + assert_eq!(res[0].0.len(), 2); + assert_eq!(res[0].1, 0); // chunk_idx + assert_eq!(res[0].0[0], ((8, 10), 0)); + assert_eq!(res[0].0[1], ((13, 16), 1)); + + assert_eq!(res[1].0.len(), 1); + assert_eq!(res[1].1, 1); // chunk_idx + assert_eq!(res[1].0[0], ((33, 36), 3)); } diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_build_state.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_build_state.rs index edff284e31b9..6dc1438f664f 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_build_state.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_build_state.rs @@ -258,12 +258,14 @@ impl HashJoinBuildState { if self.hash_join_state.need_mark_scan() { build_state.mark_scan_map.push(block_mark_scan_map); } + + build_state.generation_state.build_num_rows += data_block.num_rows(); + build_state.generation_state.chunks.push(data_block); + if self.hash_join_state.merge_into_target_table_index != DUMMY_TABLE_INDEX { let chunk_offsets = unsafe { &mut *self.hash_join_state.chunk_offsets.get() }; - chunk_offsets.push(build_state.generation_state.build_num_rows as u64); + chunk_offsets.push(build_state.generation_state.build_num_rows as u32); } - build_state.generation_state.build_num_rows += data_block.num_rows(); - build_state.generation_state.chunks.push(data_block); } Ok(()) } diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs index da948d253525..f729573af7f8 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs @@ -39,6 +39,7 @@ use databend_common_expression::Scalar; use databend_common_expression::Value; use databend_common_functions::BUILTIN_FUNCTIONS; use databend_common_hashtable::HashJoinHashtableLike; +use databend_common_hashtable::Interval; use databend_common_sql::ColumnSet; use itertools::Itertools; use log::info; @@ -56,6 +57,7 @@ use crate::pipelines::processors::HashJoinState; use crate::sessions::QueryContext; use crate::sql::planner::plans::JoinType; +pub type ChunkPartialUnmodified = (Vec<(Interval, u64)>, u32); /// Define some shared states for all hash join probe threads. pub struct HashJoinProbeState { pub(crate) ctx: Arc, @@ -80,6 +82,9 @@ pub struct HashJoinProbeState { /// Todo(xudong): add more detailed comments for the following fields. /// Final scan tasks pub(crate) final_scan_tasks: RwLock>, + /// for merge into target as build side. + pub(crate) final_merge_into_partial_unmodified_scan_tasks: + RwLock>, pub(crate) mark_scan_map_lock: Mutex<()>, /// Hash method pub(crate) hash_method: HashMethodKind, @@ -138,6 +143,7 @@ impl HashJoinProbeState { probe_schema, probe_projections: probe_projections.clone(), final_scan_tasks: RwLock::new(VecDeque::new()), + final_merge_into_partial_unmodified_scan_tasks: RwLock::new(VecDeque::new()), mark_scan_map_lock: Mutex::new(()), hash_method: method, spill_partitions: Default::default(), @@ -422,6 +428,15 @@ impl HashJoinProbeState { Ok(()) } + pub fn probe_merge_into_partial_modified_done(&self) -> Result<()> { + let old_count = self.probe_workers.fetch_sub(1, Ordering::Relaxed); + if old_count == 1 { + // Divide the final scan phase into multiple tasks. + self.generate_merge_into_final_scan_task()?; + } + Ok(()) + } + pub fn finish_spill(&self) -> Result<()> { self.final_probe_workers.fetch_sub(1, Ordering::Relaxed); let old_count = self.spill_workers.fetch_sub(1, Ordering::Relaxed); @@ -463,11 +478,34 @@ impl HashJoinProbeState { Ok(()) } + pub fn generate_merge_into_final_scan_task(&self) -> Result<()> { + let block_info_index = unsafe { + &*self + .hash_join_state + .block_info_index + .as_ref() + .unwrap() + .get() + }; + let matched = unsafe { &*self.hash_join_state.matched.get() }; + let chunks_offsets = unsafe { &*self.hash_join_state.chunk_offsets.get() }; + let partial_unmodified = block_info_index.gather_all_partial_block_offsets(matched); + // generate chunks + let tasks = block_info_index.chunk_offsets(&partial_unmodified, chunks_offsets); + *self.final_merge_into_partial_unmodified_scan_tasks.write() = tasks.into(); + Ok(()) + } + pub fn final_scan_task(&self) -> Option { let mut tasks = self.final_scan_tasks.write(); tasks.pop_front() } + pub fn final_merge_into_partial_unmodified_scan_task(&self) -> Option { + let mut tasks = self.final_merge_into_partial_unmodified_scan_tasks.write(); + tasks.pop_front() + } + pub fn final_scan(&self, task: usize, state: &mut ProbeState) -> Result> { match &self.hash_join_state.hash_join_desc.join_type { JoinType::Right | JoinType::RightSingle | JoinType::Full => { diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_state.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_state.rs index 3224eb2c6b8c..1785a3595098 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_state.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_state.rs @@ -38,6 +38,7 @@ use databend_common_hashtable::StringHashJoinHashMap; use databend_common_sql::plans::JoinType; use databend_common_sql::ColumnSet; use databend_common_sql::IndexType; +use databend_common_sql::DUMMY_TABLE_INDEX; use ethnum::U256; use parking_lot::RwLock; @@ -144,7 +145,7 @@ pub struct HashJoinState { /// atomic_pointers to pointer to matched pub(crate) atomic_pointer: SyncUnsafeCell, /// chunk_offsets[chunk_idx] stands for the offset of chunk_idx_th chunk in chunks. - pub(crate) chunk_offsets: SyncUnsafeCell>, + pub(crate) chunk_offsets: SyncUnsafeCell>, } impl HashJoinState { @@ -249,6 +250,10 @@ impl HashJoinState { matches!(self.hash_join_desc.join_type, JoinType::LeftMark) } + pub fn need_merge_into_target_partial_modified_scan(&self) -> bool { + self.merge_into_target_table_index != DUMMY_TABLE_INDEX + } + pub fn set_spilled_partition(&self, partitions: &HashSet) { let mut spill_partition = self.build_spilled_partitions.write(); spill_partition.extend(partitions); diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/transform_hash_join_probe.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/transform_hash_join_probe.rs index 732a39952431..d2411b181b2f 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/transform_hash_join_probe.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/transform_hash_join_probe.rs @@ -17,14 +17,18 @@ use std::collections::VecDeque; use std::sync::atomic::Ordering; use std::sync::Arc; +use databend_common_catalog::plan::split_prefix; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::DataBlock; use databend_common_expression::FunctionContext; use databend_common_sql::optimizer::ColumnSet; use databend_common_sql::plans::JoinType; +use databend_common_sql::DUMMY_TABLE_INDEX; +use databend_common_storages_fuse::operations::BlockMetaIndex; use log::info; +use super::hash_join_probe_state::ChunkPartialUnmodified; use crate::pipelines::processors::transforms::hash_join::probe_spill::ProbeSpillState; use crate::pipelines::processors::transforms::hash_join::HashJoinProbeState; use crate::pipelines::processors::transforms::hash_join::ProbeState; @@ -121,6 +125,36 @@ impl TransformHashJoinProbe { Ok(()) } + fn final_merge_into_partial_unmodified_scan( + &mut self, + item: ChunkPartialUnmodified, + ) -> Result<()> { + let build_state = unsafe { &*self.join_probe_state.hash_join_state.build_state.get() }; + let chunk_block = &build_state.generation_state.chunks[item.1 as usize]; + for (interval, prefix) in item.0 { + let indices = (interval.0..=interval.1) + .collect::>() + .chunks(self.max_block_size) + .map(|chunk| chunk.to_vec()) + .collect::>>(); + for range in indices.iter() { + let data_block = chunk_block.take( + range, + &mut self.probe_state.generation_state.string_items_buf, + )?; + assert!(!data_block.is_empty()); + let (segment_idx, block_idx) = split_prefix(prefix); + let data_block = data_block.add_meta(Some(Box::new(BlockMetaIndex { + segment_idx: segment_idx as usize, + block_idx: block_idx as usize, + inner: None, + })))?; + self.output_data_blocks.push_back(data_block); + } + } + Ok(()) + } + fn final_scan(&mut self, task: usize) -> Result<()> { let data_blocks = self .join_probe_state @@ -203,6 +237,14 @@ impl TransformHashJoinProbe { { self.join_probe_state.probe_done()?; Ok(Event::Async) + } else if self + .join_probe_state + .hash_join_state + .need_merge_into_target_partial_modified_scan() + { + self.join_probe_state + .probe_merge_into_partial_modified_done()?; + Ok(Event::Async) } else { if !self.join_probe_state.spill_partitions.read().is_empty() { self.join_probe_state.finish_final_probe()?; @@ -369,11 +411,25 @@ impl Processor for TransformHashJoinProbe { Ok(()) } HashJoinProbeStep::FinalScan => { - if let Some(task) = self.join_probe_state.final_scan_task() { + // if self.join_probe_state.h + if self + .join_probe_state + .hash_join_state + .merge_into_target_table_index + != DUMMY_TABLE_INDEX + { + if let Some(item) = self + .join_probe_state + .final_merge_into_partial_unmodified_scan_task() + { + self.final_merge_into_partial_unmodified_scan(item)?; + return Ok(()); + } + } else if let Some(task) = self.join_probe_state.final_scan_task() { self.final_scan(task)?; - } else { - self.outer_scan_finished = true; + return Ok(()); } + self.outer_scan_finished = true; Ok(()) } HashJoinProbeStep::FastReturn diff --git a/tests/sqllogictests/suites/base/09_fuse_engine/09_0039t_arget_build_merge_into_standalone.test b/tests/sqllogictests/suites/base/09_fuse_engine/09_0039t_arget_build_merge_into_standalone.test index 6489167d0157..fee00bd0b9ff 100644 --- a/tests/sqllogictests/suites/base/09_fuse_engine/09_0039t_arget_build_merge_into_standalone.test +++ b/tests/sqllogictests/suites/base/09_fuse_engine/09_0039t_arget_build_merge_into_standalone.test @@ -255,5 +255,52 @@ select * from target_build_optimization order by a,b,c; 11 b11 c_11 12 b12 c_12 +### test target_table is empty +statement ok +truncate table target_build_optimization; + +### make sure the plan is expected +query T +explain merge into target_build_optimization as t1 using source_optimization as t2 on t1.a = t2.a and t1.b = t2.b when matched then update * when not matched then insert *; +---- +MergeInto: +target_table: default.default.target_build_optimization +├── distributed: false +├── target_build_optimization: true +├── matched update: [condition: None,update set b = b (#1),a = a (#0),c = c (#2)] +├── unmatched insert: [condition: None,insert into (a,b,c) values(CAST(a (#0) AS Int32 NULL),CAST(b (#1) AS String NULL),CAST(c (#2) AS String NULL))] +└── HashJoin: LEFT OUTER + ├── equi conditions: [eq(t2.b (#1), t1.b (#4))] + ├── non-equi conditions: [gt(t1.a (#3), t2.a (#0))] + ├── LogicalGet + │ ├── table: default.default.source_optimization + │ ├── filters: [] + │ ├── order by: [] + │ └── limit: NONE + └── LogicalGet + ├── table: default.default.target_build_optimization + ├── filters: [] + ├── order by: [] + └── limit: NONE + +query TT +merge into target_build_optimization as t1 using source_optimization as t2 on t1.a > t2.a and t1.b = t2.b when matched then update * when not matched then insert *; +---- +12 0 + +query TTT +select * from target_build_optimization order by a,b,c; +---- +1 b1 c_1 +2 b2 c_3 +5 b5 c_5 +6 b6 c_6 +7 b7 c_7 +8 b8 c_8 +9 b9 c_9 +10 b10 c_10 +11 b11 c_11 +12 b12 c_12 + statement ok set enable_experimental_merge_into = 0; \ No newline at end of file From f9c8876a03af4339d201af856e73d522fca41308 Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Fri, 12 Jan 2024 00:41:12 +0800 Subject: [PATCH 17/47] add one chunk ut test for block_info_index --- src/common/hashtable/src/utils.rs | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/src/common/hashtable/src/utils.rs b/src/common/hashtable/src/utils.rs index e8de092b9365..dfb4d8fa53ae 100644 --- a/src/common/hashtable/src/utils.rs +++ b/src/common/hashtable/src/utils.rs @@ -463,4 +463,18 @@ fn test_block_info_index() { assert_eq!(res[1].0.len(), 1); assert_eq!(res[1].1, 1); // chunk_idx assert_eq!(res[1].0[0], ((33, 36), 3)); + + // test only one chunk + // blocks: [0,10][11,20][21,30],[31,39] + // chunks: [0,20],[21,39] + // chunks_offsets: [21],[40] + // partial_unmodified: [((8,10),0),((13,16),1),((33,36),3)] + let partial_unmodified = vec![((13, 16), 1)]; + let chunks_offsets = vec![21, 40]; + let res = block_info_index.chunk_offsets(&partial_unmodified, &chunks_offsets); + assert_eq!(res.len(), 1); + + assert_eq!(res[0].0.len(), 1); + assert_eq!(res[0].1, 0); // chunk_idx + assert_eq!(res[0].0[0], ((13, 16), 1)); } From 404f9c89a66cbd04223fa88ef6a0e89f32124f17 Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Fri, 12 Jan 2024 12:30:14 +0800 Subject: [PATCH 18/47] fix test result --- .../suites/mode/standalone/explain/merge_into.test | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/sqllogictests/suites/mode/standalone/explain/merge_into.test b/tests/sqllogictests/suites/mode/standalone/explain/merge_into.test index e42f6752844d..e372a9a329e6 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/merge_into.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/merge_into.test @@ -23,8 +23,8 @@ explain MERGE INTO salaries2 USING (SELECT * FROM employees2) as employees2 ON s ---- MergeInto: target_table: default.default.salaries2 -├── distributed: false | -├── target_build_optimization: false +├── distributed: false +├── target_build_optimization: false ├── matched update: [condition: eq(employees2.department (#2), 'HR'),update set salary = plus(salaries2.salary (#4), 1000.00)] ├── matched update: [condition: None,update set salary = plus(salaries2.salary (#4), 500.00)] ├── unmatched insert: [condition: None,insert into (employee_id,salary) values(CAST(employees2.employee_id (#0) AS Int32 NULL),CAST(55000.00 AS Decimal(10, 2) NULL))] @@ -52,8 +52,8 @@ explain MERGE INTO salaries2 USING (SELECT * FROM employees2) as employees2 ON s ---- MergeInto: target_table: default.default.salaries2 -├── distributed: false | -├── target_build_optimization: false +├── distributed: false +├── target_build_optimization: false ├── matched update: [condition: eq(employees2.department (#2), 'HR'),update set salary = plus(salaries2.salary (#4), 1000.00)] ├── matched update: [condition: None,update set salary = plus(salaries2.salary (#4), 500.00)] ├── unmatched insert: [condition: None,insert into (employee_id,salary) values(CAST(employees2.employee_id (#0) AS Int32 NULL),CAST(55000.00 AS Decimal(10, 2) NULL))] From 7f991cbbd76f7b6fb02ab95536359e9e0bdfc9d4 Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Fri, 12 Jan 2024 14:46:54 +0800 Subject: [PATCH 19/47] add more commnnts for merge into strategies, and fix rowid read --- .../interpreters/interpreter_merge_into.rs | 33 ++++++++++++--- .../sql/src/planner/optimizer/optimizer.rs | 42 ++++++++++++------- 2 files changed, 56 insertions(+), 19 deletions(-) diff --git a/src/query/service/src/interpreters/interpreter_merge_into.rs b/src/query/service/src/interpreters/interpreter_merge_into.rs index 9ff350d2c06b..e01864a18bae 100644 --- a/src/query/service/src/interpreters/interpreter_merge_into.rs +++ b/src/query/service/src/interpreters/interpreter_merge_into.rs @@ -199,12 +199,37 @@ impl MergeIntoInterpreter { break; } } - + // attentation!! for now we have some strategies: + // 1. target_build_optimization, this is enabled in standalone mode and in this case we don't need rowid column anymore. + // but we just support for `merge into xx using source on xxx when matched then update xxx when not matched then insert xxx`. + // 2. merge into join strategies: + // Left,Right,Inner,Left Anti, Right Anti + // important flag: + // I. change join order: if true, target table as build side, if false, source as build side. + // II. distributed: this merge into is executed at a distributed stargety. + // 2.1 Left: there are macthed and not macthed, and change join order is false. + // 2.2 Left Anti: change join order is true, but it's insert-only. + // 2.3 Inner: this is matched only case. + // 2.3.1 change join order is true, + // 2.3.2 change join order is false. + // 2.4 Right: change join order is false, there are macthed and not macthed + // 2.5 Right Anti: change join order is false, but it's insert-only. + // distributed execution stargeties: + // I. change join order is true, we use the `optimize_distributed_query`'s result. + // II. change join order is false and match_pattern and not enable spill, we use right outer join with rownumber distributed strategies. + // III otherwise, use `merge_into_join_sexpr` as standalone execution(so if change join order is false,but doesn't match_pattern, we don't support distributed,in fact. case I + // can take this at most time, if that's a hash shuffle, the I can take it. We think source is always very small). + let target_build_optimization = + matches!(self.plan.merge_type, MergeIntoType::FullOperation) + && !self.plan.columns_set.contains(&self.plan.row_id_index); + if target_build_optimization { + assert!(*change_join_order && !*distributed); + } if *distributed && !*change_join_order { row_number_idx = Some(join_output_schema.index_of(ROW_NUMBER_COL_NAME)?); } - if !insert_only && !found_row_id { + if !target_build_optimization && !insert_only && !found_row_id { // we can't get row_id_idx, throw an exception return Err(ErrorCode::InvalidRowIdIndex( "can't get internal row_id_idx when running merge into", @@ -373,9 +398,7 @@ impl MergeIntoInterpreter { .into_iter() .enumerate() .collect(); - let target_build_optimization = - matches!(self.plan.merge_type, MergeIntoType::FullOperation) - && !self.plan.columns_set.contains(&self.plan.row_id_index); + let commit_input = if !distributed { // recv datablocks from matched upstream and unmatched upstream // transform and append dat diff --git a/src/query/sql/src/planner/optimizer/optimizer.rs b/src/query/sql/src/planner/optimizer/optimizer.rs index 6f48ccd880e1..a37d70dc82ef 100644 --- a/src/query/sql/src/planner/optimizer/optimizer.rs +++ b/src/query/sql/src/planner/optimizer/optimizer.rs @@ -354,28 +354,38 @@ fn optimize_merge_into(opt_ctx: OptimizerContext, plan: Box) -> Resul // - distributed optimization is enabled // - no local table scan // - distributed merge-into is enabled - // - join spilling is disabled if opt_ctx.enable_distributed_optimization && !contains_local_table_scan(&join_sexpr, &opt_ctx.metadata) && opt_ctx .table_ctx .get_settings() .get_enable_distributed_merge_into()? - && opt_ctx - .table_ctx - .get_settings() - .get_join_spilling_threshold()? - == 0 { + // distributed execution stargeties: + // I. change join order is true, we use the `optimize_distributed_query`'s result. + // II. change join order is false and match_pattern and not enable spill, we use right outer join with rownumber distributed strategies. + // III otherwise, use `merge_into_join_sexpr` as standalone execution(so if change join order is false,but doesn't match_pattern, we don't support distributed,in fact. case I + // can take this at most time, if that's a hash shuffle, the I can take it. We think source is always very small). // input is a Join_SExpr let mut merge_into_join_sexpr = optimize_distributed_query(opt_ctx.table_ctx.clone(), &join_sexpr)?; - // after optimize source, we need to add let merge_source_optimizer = MergeSourceOptimizer::create(); - let (optimized_distributed_merge_into_join_sexpr, distributed) = if !merge_into_join_sexpr - .match_pattern(&merge_source_optimizer.merge_source_pattern) - || change_join_order + // II. + // - join spilling is disabled + let (optimized_distributed_merge_into_join_sexpr, distributed) = if opt_ctx + .table_ctx + .get_settings() + .get_join_spilling_threshold()? + == 0 + && !change_join_order + && merge_into_join_sexpr.match_pattern(&merge_source_optimizer.merge_source_pattern) { + ( + merge_source_optimizer.optimize(&merge_into_join_sexpr)?, + true, + ) + } else if change_join_order { + // I // we need to judge whether it'a broadcast join to support runtime filter. merge_into_join_sexpr = try_to_change_as_broadcast_join( merge_into_join_sexpr, @@ -385,12 +395,16 @@ fn optimize_merge_into(opt_ctx: OptimizerContext, plan: Box) -> Resul false, // we will open it, but for now we don't support distributed new_columns_set.as_mut(), )?; - (merge_into_join_sexpr.clone(), false) - } else { ( - merge_source_optimizer.optimize(&merge_into_join_sexpr)?, - true, + merge_into_join_sexpr.clone(), + matches!( + merge_into_join_sexpr.plan.as_ref(), + RelOperator::Exchange(_) + ), ) + } else { + // III. + (merge_into_join_sexpr.clone(), false) }; Ok(Plan::MergeInto(Box::new(MergeInto { From 575d14c218d147a66351b4dadc89856a747d1c83 Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Fri, 12 Jan 2024 17:06:29 +0800 Subject: [PATCH 20/47] fix test --- .../sql/src/planner/optimizer/optimizer.rs | 41 +++++++++++-------- 1 file changed, 23 insertions(+), 18 deletions(-) diff --git a/src/query/sql/src/planner/optimizer/optimizer.rs b/src/query/sql/src/planner/optimizer/optimizer.rs index a37d70dc82ef..f7d7e90eee3d 100644 --- a/src/query/sql/src/planner/optimizer/optimizer.rs +++ b/src/query/sql/src/planner/optimizer/optimizer.rs @@ -332,7 +332,11 @@ fn optimize_merge_into(opt_ctx: OptimizerContext, plan: Box) -> Resul // update xx when not matched then insert xx. let flag = plan.matched_evaluators.len() == 1 && plan.matched_evaluators[0].condition.is_none() - && plan.matched_evaluators[0].update.is_some(); + && plan.matched_evaluators[0].update.is_some() + && !opt_ctx + .table_ctx + .get_settings() + .get_enable_distributed_merge_into()?; let mut new_columns_set = plan.columns_set.clone(); if change_join_order && matches!(plan.merge_type, MergeIntoType::FullOperation) @@ -426,11 +430,11 @@ fn optimize_merge_into(opt_ctx: OptimizerContext, plan: Box) -> Resul fn try_to_change_as_broadcast_join( merge_into_join_sexpr: SExpr, - change_join_order: bool, - table_ctx: Arc, - plan: &MergeInto, - only_one_matched_clause: bool, - new_columns_set: &mut HashSet, + _change_join_order: bool, + _table_ctx: Arc, + _plan: &MergeInto, + _only_one_matched_clause: bool, + _new_columns_set: &mut HashSet, ) -> Result { if let RelOperator::Exchange(Exchange::Merge) = merge_into_join_sexpr.plan.as_ref() { let right_exchange = merge_into_join_sexpr.child(0)?.child(1)?; @@ -442,18 +446,19 @@ fn try_to_change_as_broadcast_join( .replace_plan(Arc::new(RelOperator::Join(join))); // for now, when we use target table as build side and it's a broadcast join, // we will use merge_into_block_info_hashtable to reduce i/o operations. - if change_join_order - && matches!(plan.merge_type, MergeIntoType::FullOperation) - && only_one_matched_clause - { - // remove rowid - new_columns_set.remove(&plan.row_id_index); - table_ctx.set_merge_into_join(MergeIntoJoin { - merge_into_join_type: MergeIntoJoinType::Left, - is_distributed: true, - target_tbl_idx: plan.target_table_idx, - }) - } + // Todo(JackTan25): we don't support in distributed mod for target build optimization for now. we will enable in next pr. + // if change_join_order + // && matches!(plan.merge_type, MergeIntoType::FullOperation) + // && only_one_matched_clause + // { + // remove rowid + // new_columns_set.remove(&plan.row_id_index); + // table_ctx.set_merge_into_join(MergeIntoJoin { + // merge_into_join_type: MergeIntoJoinType::Left, + // is_distributed: true, + // target_tbl_idx: plan.target_table_idx, + // }) + // } return Ok(merge_into_join_sexpr.replace_children(vec![Arc::new(join_s_expr)])); } } From 4e503ffd84e1778f603635533aeb4ad7e28b2bd1 Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Fri, 12 Jan 2024 18:34:13 +0800 Subject: [PATCH 21/47] fix split --- .../interpreters/interpreter_merge_into.rs | 69 ++++++++++++------- .../pipelines/builders/builder_merge_into.rs | 6 +- .../physical_plans/physical_merge_into.rs | 1 + .../sql/src/planner/binder/merge_into.rs | 13 +++- src/query/sql/src/planner/plans/merge_into.rs | 1 + .../mutator/merge_into_split_mutator.rs | 13 ++-- .../processors/processor_merge_into_split.rs | 4 +- 7 files changed, 69 insertions(+), 38 deletions(-) diff --git a/src/query/service/src/interpreters/interpreter_merge_into.rs b/src/query/service/src/interpreters/interpreter_merge_into.rs index e01864a18bae..cf4bc84052bc 100644 --- a/src/query/service/src/interpreters/interpreter_merge_into.rs +++ b/src/query/service/src/interpreters/interpreter_merge_into.rs @@ -50,6 +50,7 @@ use databend_common_sql::plans::UpdatePlan; use databend_common_sql::IndexType; use databend_common_sql::ScalarExpr; use databend_common_sql::TypeCheck; +use databend_common_sql::DUMMY_COLUMN_INDEX; use databend_common_storages_factory::Table; use databend_common_storages_fuse::FuseTable; use databend_common_storages_fuse::TableContext; @@ -138,9 +139,37 @@ impl MergeIntoInterpreter { merge_type, distributed, change_join_order, + split_idx, .. } = &self.plan; + // attentation!! for now we have some strategies: + // 1. target_build_optimization, this is enabled in standalone mode and in this case we don't need rowid column anymore. + // but we just support for `merge into xx using source on xxx when matched then update xxx when not matched then insert xxx`. + // 2. merge into join strategies: + // Left,Right,Inner,Left Anti, Right Anti + // important flag: + // I. change join order: if true, target table as build side, if false, source as build side. + // II. distributed: this merge into is executed at a distributed stargety. + // 2.1 Left: there are macthed and not macthed, and change join order is false. + // 2.2 Left Anti: change join order is true, but it's insert-only. + // 2.3 Inner: this is matched only case. + // 2.3.1 change join order is true, + // 2.3.2 change join order is false. + // 2.4 Right: change join order is false, there are macthed and not macthed + // 2.5 Right Anti: change join order is false, but it's insert-only. + // distributed execution stargeties: + // I. change join order is true, we use the `optimize_distributed_query`'s result. + // II. change join order is false and match_pattern and not enable spill, we use right outer join with rownumber distributed strategies. + // III otherwise, use `merge_into_join_sexpr` as standalone execution(so if change join order is false,but doesn't match_pattern, we don't support distributed,in fact. case I + // can take this at most time, if that's a hash shuffle, the I can take it. We think source is always very small). + let target_build_optimization = + matches!(self.plan.merge_type, MergeIntoType::FullOperation) + && !self.plan.columns_set.contains(&self.plan.row_id_index); + + if target_build_optimization { + assert!(*change_join_order && !*distributed); + } // check mutability let check_table = self.ctx.get_table(catalog, database, table_name).await?; check_table.check_mutable()?; @@ -174,7 +203,7 @@ impl MergeIntoInterpreter { let insert_only = matches!(merge_type, MergeIntoType::InsertOnly); - let mut row_id_idx = if !insert_only { + let mut row_id_idx = if !insert_only && !target_build_optimization { match meta_data .read() .row_id_index_by_table_index(*target_table_idx) @@ -199,32 +228,18 @@ impl MergeIntoInterpreter { break; } } - // attentation!! for now we have some strategies: - // 1. target_build_optimization, this is enabled in standalone mode and in this case we don't need rowid column anymore. - // but we just support for `merge into xx using source on xxx when matched then update xxx when not matched then insert xxx`. - // 2. merge into join strategies: - // Left,Right,Inner,Left Anti, Right Anti - // important flag: - // I. change join order: if true, target table as build side, if false, source as build side. - // II. distributed: this merge into is executed at a distributed stargety. - // 2.1 Left: there are macthed and not macthed, and change join order is false. - // 2.2 Left Anti: change join order is true, but it's insert-only. - // 2.3 Inner: this is matched only case. - // 2.3.1 change join order is true, - // 2.3.2 change join order is false. - // 2.4 Right: change join order is false, there are macthed and not macthed - // 2.5 Right Anti: change join order is false, but it's insert-only. - // distributed execution stargeties: - // I. change join order is true, we use the `optimize_distributed_query`'s result. - // II. change join order is false and match_pattern and not enable spill, we use right outer join with rownumber distributed strategies. - // III otherwise, use `merge_into_join_sexpr` as standalone execution(so if change join order is false,but doesn't match_pattern, we don't support distributed,in fact. case I - // can take this at most time, if that's a hash shuffle, the I can take it. We think source is always very small). - let target_build_optimization = - matches!(self.plan.merge_type, MergeIntoType::FullOperation) - && !self.plan.columns_set.contains(&self.plan.row_id_index); - if target_build_optimization { - assert!(*change_join_order && !*distributed); + + let mut merge_into_split_idx = DUMMY_COLUMN_INDEX; + if matches!(merge_type, MergeIntoType::FullOperation) { + for (idx, data_field) in join_output_schema.fields().iter().enumerate() { + if *data_field.name() == split_idx.to_string() { + merge_into_split_idx = idx; + break; + } + } + assert!(merge_into_split_idx != DUMMY_COLUMN_INDEX); } + if *distributed && !*change_join_order { row_number_idx = Some(join_output_schema.index_of(ROW_NUMBER_COL_NAME)?); } @@ -270,12 +285,14 @@ impl MergeIntoInterpreter { input: Box::new(rollback_join_input), row_id_idx: row_id_idx as u32, merge_type: merge_type.clone(), + merge_into_split_idx: merge_into_split_idx as u32, }) } else { PhysicalPlan::MergeIntoSource(MergeIntoSource { input: Box::new(join_input), row_id_idx: row_id_idx as u32, merge_type: merge_type.clone(), + merge_into_split_idx: merge_into_split_idx as u32, }) }; diff --git a/src/query/service/src/pipelines/builders/builder_merge_into.rs b/src/query/service/src/pipelines/builders/builder_merge_into.rs index 85b73bcc53c5..1820b38b402c 100644 --- a/src/query/service/src/pipelines/builders/builder_merge_into.rs +++ b/src/query/service/src/pipelines/builders/builder_merge_into.rs @@ -319,14 +319,16 @@ impl PipelineBuilder { Ok(()) } + // Optimization Todo(@JackTan25): If insert only, we can reduce the target columns after join. pub(crate) fn build_merge_into_source( &mut self, merge_into_source: &MergeIntoSource, ) -> Result<()> { let MergeIntoSource { input, - row_id_idx, merge_type, + merge_into_split_idx, + .. } = merge_into_source; self.build_pipeline(input)?; @@ -341,7 +343,7 @@ impl PipelineBuilder { let output_len = self.main_pipeline.output_len(); for _ in 0..output_len { let merge_into_split_processor = - MergeIntoSplitProcessor::create(*row_id_idx, false)?; + MergeIntoSplitProcessor::create(*merge_into_split_idx, false)?; items.push(merge_into_split_processor.into_pipe_item()); } diff --git a/src/query/sql/src/executor/physical_plans/physical_merge_into.rs b/src/query/sql/src/executor/physical_plans/physical_merge_into.rs index b51f445c5aa3..9c44a11d4853 100644 --- a/src/query/sql/src/executor/physical_plans/physical_merge_into.rs +++ b/src/query/sql/src/executor/physical_plans/physical_merge_into.rs @@ -32,6 +32,7 @@ pub struct MergeIntoSource { pub input: Box, pub row_id_idx: u32, pub merge_type: MergeIntoType, + pub merge_into_split_idx: u32, } #[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] diff --git a/src/query/sql/src/planner/binder/merge_into.rs b/src/query/sql/src/planner/binder/merge_into.rs index b1fe349f214e..98a45696045b 100644 --- a/src/query/sql/src/planner/binder/merge_into.rs +++ b/src/query/sql/src/planner/binder/merge_into.rs @@ -57,6 +57,7 @@ use crate::Metadata; use crate::ScalarBinder; use crate::ScalarExpr; use crate::Visibility; +use crate::DUMMY_COLUMN_INDEX; #[derive(Clone, Debug, PartialEq, serde::Serialize, serde::Deserialize)] pub enum MergeIntoType { @@ -389,7 +390,16 @@ impl Binder { .await?, ); } - + let mut split_idx = DUMMY_COLUMN_INDEX; + // find any target table column index for merge_into_split + for column in self.metadata.read().columns() { + if column.table_index().is_some() + && *column.table_index().as_ref().unwrap() == table_index + { + split_idx = column.index() + } + } + assert!(split_idx != DUMMY_COLUMN_INDEX); Ok(MergeInto { catalog: catalog_name.to_string(), database: database_name.to_string(), @@ -408,6 +418,7 @@ impl Binder { distributed: false, change_join_order: false, row_id_index: column_binding.index, + split_idx, }) } diff --git a/src/query/sql/src/planner/plans/merge_into.rs b/src/query/sql/src/planner/plans/merge_into.rs index 4c683c565135..dc60b21221f2 100644 --- a/src/query/sql/src/planner/plans/merge_into.rs +++ b/src/query/sql/src/planner/plans/merge_into.rs @@ -70,6 +70,7 @@ pub struct MergeInto { pub change_join_order: bool, // when we use target table as build side, we need to remove rowid columns. pub row_id_index: IndexType, + pub split_idx: IndexType, } impl std::fmt::Debug for MergeInto { diff --git a/src/query/storages/fuse/src/operations/merge_into/mutator/merge_into_split_mutator.rs b/src/query/storages/fuse/src/operations/merge_into/mutator/merge_into_split_mutator.rs index 929c1d05c330..0a36809879fe 100644 --- a/src/query/storages/fuse/src/operations/merge_into/mutator/merge_into_split_mutator.rs +++ b/src/query/storages/fuse/src/operations/merge_into/mutator/merge_into_split_mutator.rs @@ -22,25 +22,24 @@ use databend_common_expression::types::NumberDataType; use databend_common_expression::DataBlock; pub struct MergeIntoSplitMutator { - pub row_id_idx: u32, + pub split_idx: u32, } impl MergeIntoSplitMutator { - #[allow(dead_code)] - pub fn try_create(row_id_idx: u32) -> Self { - Self { row_id_idx } + pub fn try_create(split_idx: u32) -> Self { + Self { split_idx } } // (matched_block,not_matched_block) pub fn split_data_block(&mut self, block: &DataBlock) -> Result<(DataBlock, DataBlock)> { - let row_id_column = &block.columns()[self.row_id_idx as usize]; + let split_column = &block.columns()[self.split_idx as usize]; assert_eq!( - row_id_column.data_type, + split_column.data_type, DataType::Nullable(Box::new(DataType::Number(NumberDataType::UInt64))), ); // get row_id do check duplicate and get filter - let filter: Bitmap = match &row_id_column.value { + let filter: Bitmap = match &split_column.value { databend_common_expression::Value::Scalar(scalar) => { // fast judge if scalar.is_null() { diff --git a/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_split.rs b/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_split.rs index c4b0f710edbd..31e83a8473fa 100644 --- a/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_split.rs +++ b/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_split.rs @@ -52,8 +52,8 @@ pub struct MergeIntoSplitProcessor { } impl MergeIntoSplitProcessor { - pub fn create(row_id_idx: u32, target_table_empty: bool) -> Result { - let merge_into_split_mutator = MergeIntoSplitMutator::try_create(row_id_idx); + pub fn create(split_idx: u32, target_table_empty: bool) -> Result { + let merge_into_split_mutator = MergeIntoSplitMutator::try_create(split_idx); let input_port = InputPort::create(); let output_port_matched = OutputPort::create(); let output_port_not_matched = OutputPort::create(); From 612cebfe16c4147d569530ed5f8628c31167e2ac Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Sat, 13 Jan 2024 00:03:42 +0800 Subject: [PATCH 22/47] fix block_info_index init, matched offsets update and add target_table_schema for partial unmodified blocks to append directly, add probe attach for target_build_optimization, fix merge intp explain update order --- .../pipelines/builders/builder_merge_into.rs | 6 ++++ .../hash_join/hash_join_build_state.rs | 15 ++++------ .../hash_join/hash_join_probe_state.rs | 16 +++++----- .../transforms/hash_join/hash_join_state.rs | 8 +++-- .../hash_join/probe_join/left_join.rs | 9 ++++-- .../hash_join/transform_hash_join_probe.rs | 1 + .../sql/src/planner/binder/merge_into.rs | 4 ++- .../sql/src/planner/format/display_plan.rs | 12 ++++---- .../mutator/delete_by_expr_mutator.rs | 19 ++++++++---- .../mutator/merge_into_split_mutator.rs | 6 +--- .../processor_merge_into_matched_and_split.rs | 20 +++++++++---- .../processor_merge_into_not_matched.rs | 30 ++++++++++++------- .../processors/processor_merge_into_split.rs | 1 + ...9_target_build_merge_into_standalone.test} | 16 +++++----- 14 files changed, 102 insertions(+), 61 deletions(-) rename tests/sqllogictests/suites/{base/09_fuse_engine/09_0039t_arget_build_merge_into_standalone.test => mode/standalone/explain/09_0039_target_build_merge_into_standalone.test} (92%) diff --git a/src/query/service/src/pipelines/builders/builder_merge_into.rs b/src/query/service/src/pipelines/builders/builder_merge_into.rs index 1820b38b402c..21e247a35ccf 100644 --- a/src/query/service/src/pipelines/builders/builder_merge_into.rs +++ b/src/query/service/src/pipelines/builders/builder_merge_into.rs @@ -134,6 +134,7 @@ impl PipelineBuilder { .ctx .build_table_by_table_info(catalog_info, table_info, None)?; let table = FuseTable::try_from_table(tbl.as_ref())?; + // case 1 if !*change_join_order { if let MergeIntoType::MatechedOnly = merge_type { @@ -169,11 +170,13 @@ impl PipelineBuilder { self.main_pipeline.add_pipe(Pipe::create(2, 2, pipe_items)); // not macthed operation + let table_default_schema = &tbl.schema().remove_computed_fields(); let merge_into_not_matched_processor = MergeIntoNotMatchedProcessor::create( unmatched.clone(), input_schema.clone(), self.func_ctx.clone(), self.ctx.clone(), + Arc::new(DataSchema::from(table_default_schema)), )?; let pipe_items = vec![ merge_into_not_matched_processor.into_pipe_item(), @@ -471,6 +474,7 @@ impl PipelineBuilder { field_index_of_input_schema.clone(), input.output_schema()?, Arc::new(DataSchema::from(tbl.schema())), + merge_into.target_build_optimization, )?; pipe_items.push(matched_split_processor.into_pipe_item()); } @@ -480,11 +484,13 @@ impl PipelineBuilder { // (distributed,change join order):(true,true) target is build side, we // need to support insert in local node. if !*distributed || *change_join_order { + let table_default_schema = &tbl.schema().remove_computed_fields(); let merge_into_not_matched_processor = MergeIntoNotMatchedProcessor::create( unmatched.clone(), input.output_schema()?, self.func_ctx.clone(), self.ctx.clone(), + Arc::new(DataSchema::from(table_default_schema)), )?; pipe_items.push(merge_into_not_matched_processor.into_pipe_item()); } else { diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_build_state.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_build_state.rs index 6dc1438f664f..5c42450360fe 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_build_state.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_build_state.rs @@ -199,8 +199,12 @@ impl HashJoinBuildState { let build_state = unsafe { &*self.hash_join_state.build_state.get() }; let start_offset = build_state.generation_state.build_num_rows + old_size; let end_offset = start_offset + input_rows - 1; + // merge into target table as build side. - if self.hash_join_state.block_info_index.is_some() { + if self + .hash_join_state + .need_merge_into_target_partial_modified_scan() + { assert!(input.get_meta().is_some()); let block_meta_index = BlockMetaIndex::downcast_ref_from(input.get_meta().unwrap()).unwrap(); @@ -208,14 +212,7 @@ impl HashJoinBuildState { block_meta_index.segment_idx as u64, block_meta_index.block_idx as u64, ); - let block_info_index = unsafe { - &mut *self - .hash_join_state - .block_info_index - .as_ref() - .unwrap() - .get() - }; + let block_info_index = unsafe { &mut *self.hash_join_state.block_info_index.get() }; block_info_index .insert_block_offsets((start_offset as u32, end_offset as u32), row_prefix); } diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs index f729573af7f8..c9cd39ac37b9 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs @@ -380,7 +380,12 @@ impl HashJoinProbeState { pub fn probe_attach(&self) -> Result { let mut worker_id = 0; - if self.hash_join_state.need_outer_scan() || self.hash_join_state.need_mark_scan() { + if self.hash_join_state.need_outer_scan() + || self.hash_join_state.need_mark_scan() + || self + .hash_join_state + .need_merge_into_target_partial_modified_scan() + { worker_id = self.probe_workers.fetch_add(1, Ordering::Relaxed); } if self.hash_join_state.enable_spill { @@ -479,14 +484,7 @@ impl HashJoinProbeState { } pub fn generate_merge_into_final_scan_task(&self) -> Result<()> { - let block_info_index = unsafe { - &*self - .hash_join_state - .block_info_index - .as_ref() - .unwrap() - .get() - }; + let block_info_index = unsafe { &*self.hash_join_state.block_info_index.get() }; let matched = unsafe { &*self.hash_join_state.matched.get() }; let chunks_offsets = unsafe { &*self.hash_join_state.chunk_offsets.get() }; let partial_unmodified = block_info_index.gather_all_partial_block_offsets(matched); diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_state.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_state.rs index 1785a3595098..c81dca765100 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_state.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_state.rs @@ -138,7 +138,7 @@ pub struct HashJoinState { /// FOR MERGE INTO TARGET TABLE AS BUILD SIDE /// When merge into target table as build side, we should preserve block info index. - pub(crate) block_info_index: Option>, + pub(crate) block_info_index: SyncUnsafeCell, /// we use matched to tag the matched offset in chunks. pub(crate) matched: SyncUnsafeCell>, /// the matched will be modified concurrently, so we use @@ -195,7 +195,11 @@ impl HashJoinState { table_index, merge_into_target_table_index, is_distributed_merge_into, - block_info_index: None, + block_info_index: if merge_into_target_table_index == DUMMY_TABLE_INDEX { + SyncUnsafeCell::new(BlockInfoIndex::new_with_capacity(0)) + } else { + SyncUnsafeCell::new(Default::default()) + }, matched: SyncUnsafeCell::new(Vec::new()), atomic_pointer: SyncUnsafeCell::new(MatchedPtr(std::ptr::null_mut())), chunk_offsets: SyncUnsafeCell::new(Vec::with_capacity(100)), diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/probe_join/left_join.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/probe_join/left_join.rs index 5fde4aeab466..b2074b35ed82 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/probe_join/left_join.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/probe_join/left_join.rs @@ -391,8 +391,13 @@ impl HashJoinProbeState { continue; } } - let offset = chunk_offsets[row_ptr.chunk_index as usize] as usize - + row_ptr.row_index as usize; + let offset = if row_ptr.chunk_index == 0 { + row_ptr.row_index as usize + } else { + (chunk_offsets[(row_ptr.chunk_index - 1) as usize] - 1) as usize + + row_ptr.row_index as usize + }; + let mut old_mactehd_counts = unsafe { (*pointer.0.add(offset)).load(Ordering::Relaxed) }; let new_matched_count = old_mactehd_counts + 1; diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/transform_hash_join_probe.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/transform_hash_join_probe.rs index d2411b181b2f..d8b9fa6821c6 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/transform_hash_join_probe.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/transform_hash_join_probe.rs @@ -149,6 +149,7 @@ impl TransformHashJoinProbe { block_idx: block_idx as usize, inner: None, })))?; + // add null columns self.output_data_blocks.push_back(data_block); } } diff --git a/src/query/sql/src/planner/binder/merge_into.rs b/src/query/sql/src/planner/binder/merge_into.rs index 98a45696045b..3c9e7f04e9a6 100644 --- a/src/query/sql/src/planner/binder/merge_into.rs +++ b/src/query/sql/src/planner/binder/merge_into.rs @@ -395,8 +395,10 @@ impl Binder { for column in self.metadata.read().columns() { if column.table_index().is_some() && *column.table_index().as_ref().unwrap() == table_index + && column.index() != column_binding.index { - split_idx = column.index() + split_idx = column.index(); + break; } } assert!(split_idx != DUMMY_COLUMN_INDEX); diff --git a/src/query/sql/src/planner/format/display_plan.rs b/src/query/sql/src/planner/format/display_plan.rs index f13dae5c1931..311331578702 100644 --- a/src/query/sql/src/planner/format/display_plan.rs +++ b/src/query/sql/src/planner/format/display_plan.rs @@ -318,12 +318,14 @@ fn format_merge_into(merge_into: &MergeInto) -> Result { condition_format )))); } else { - let update_format = evaluator - .update - .as_ref() - .unwrap() + let map = evaluator.update.as_ref().unwrap(); + let mut field_indexes: Vec = + map.iter().map(|(field_idx, _)| *field_idx).collect(); + field_indexes.sort(); + let update_format = field_indexes .iter() - .map(|(field_idx, expr)| { + .map(|field_idx| { + let expr = map.get(field_idx).unwrap(); format!( "{} = {}", taregt_schema.field(*field_idx).name(), diff --git a/src/query/storages/fuse/src/operations/merge_into/mutator/delete_by_expr_mutator.rs b/src/query/storages/fuse/src/operations/merge_into/mutator/delete_by_expr_mutator.rs index e66c6c73d508..9feecd26002c 100644 --- a/src/query/storages/fuse/src/operations/merge_into/mutator/delete_by_expr_mutator.rs +++ b/src/query/storages/fuse/src/operations/merge_into/mutator/delete_by_expr_mutator.rs @@ -32,6 +32,8 @@ pub struct DeleteByExprMutator { row_id_idx: usize, func_ctx: FunctionContext, origin_input_columns: usize, + // if use target_build_optimization, we don't need to give row ids to `matched mutator` + target_build_optimization: bool, } impl DeleteByExprMutator { @@ -40,12 +42,14 @@ impl DeleteByExprMutator { func_ctx: FunctionContext, row_id_idx: usize, origin_input_columns: usize, + target_build_optimization: bool, ) -> Self { Self { expr, row_id_idx, func_ctx, origin_input_columns, + target_build_optimization, } } @@ -72,16 +76,20 @@ impl DeleteByExprMutator { } pub(crate) fn get_row_id_block(&self, block: DataBlock) -> DataBlock { - DataBlock::new( - vec![block.get_by_offset(self.row_id_idx).clone()], - block.num_rows(), - ) + if self.target_build_optimization { + DataBlock::empty() + } else { + DataBlock::new( + vec![block.get_by_offset(self.row_id_idx).clone()], + block.num_rows(), + ) + } } fn get_result_block( &self, predicate: &Value, - predicate_not: &Value, + predicate_not: &Value, // the rows which can be processed at this time. data_block: DataBlock, ) -> Result<(DataBlock, DataBlock)> { let res_block = data_block.clone().filter_boolean_value(predicate)?; @@ -121,6 +129,7 @@ impl DeleteByExprMutator { &self.func_ctx, data_block.num_rows(), )?; + // the rows can be processed by this time let res: Value = res.try_downcast().unwrap(); let (res_not, _) = get_not(res.clone(), &self.func_ctx, data_block.num_rows())?; diff --git a/src/query/storages/fuse/src/operations/merge_into/mutator/merge_into_split_mutator.rs b/src/query/storages/fuse/src/operations/merge_into/mutator/merge_into_split_mutator.rs index 0a36809879fe..0d303cc5115c 100644 --- a/src/query/storages/fuse/src/operations/merge_into/mutator/merge_into_split_mutator.rs +++ b/src/query/storages/fuse/src/operations/merge_into/mutator/merge_into_split_mutator.rs @@ -18,7 +18,6 @@ use databend_common_arrow::arrow::bitmap::Bitmap; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::types::DataType; -use databend_common_expression::types::NumberDataType; use databend_common_expression::DataBlock; pub struct MergeIntoSplitMutator { @@ -33,10 +32,7 @@ impl MergeIntoSplitMutator { // (matched_block,not_matched_block) pub fn split_data_block(&mut self, block: &DataBlock) -> Result<(DataBlock, DataBlock)> { let split_column = &block.columns()[self.split_idx as usize]; - assert_eq!( - split_column.data_type, - DataType::Nullable(Box::new(DataType::Number(NumberDataType::UInt64))), - ); + assert!(matches!(split_column.data_type, DataType::Nullable(_)),); // get row_id do check duplicate and get filter let filter: Bitmap = match &split_column.value { diff --git a/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_matched_and_split.rs b/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_matched_and_split.rs index 1092400e1849..6315c27b454e 100644 --- a/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_matched_and_split.rs +++ b/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_matched_and_split.rs @@ -122,6 +122,7 @@ pub struct MatchedSplitProcessor { output_data_row_id_data: Vec, output_data_updated_data: Option, target_table_schema: DataSchemaRef, + target_build_optimization: bool, } impl MatchedSplitProcessor { @@ -132,6 +133,7 @@ impl MatchedSplitProcessor { field_index_of_input_schema: HashMap, input_schema: DataSchemaRef, target_table_schema: DataSchemaRef, + target_build_optimization: bool, ) -> Result { let mut ops = Vec::::new(); for item in matched.iter() { @@ -144,6 +146,7 @@ impl MatchedSplitProcessor { ctx.get_function_context()?, row_id_idx, input_schema.num_fields(), + target_build_optimization, ), })) } else { @@ -183,6 +186,7 @@ impl MatchedSplitProcessor { row_id_idx, update_projections, target_table_schema, + target_build_optimization, }) } @@ -265,6 +269,7 @@ impl Processor for MatchedSplitProcessor { if let Some(data_block) = self.input_data.take() { // we receive a partial unmodified block data meta. if data_block.get_meta().is_some() && data_block.is_empty() { + assert!(self.target_build_optimization); let meta_index = BlockMetaIndex::downcast_ref_from(data_block.get_meta().unwrap()); if meta_index.is_some() { self.output_data_row_id_data.push(data_block); @@ -323,11 +328,16 @@ impl Processor for MatchedSplitProcessor { update_rows: current_block.num_rows(), deleted_rows: 0, }); - self.output_data_row_id_data.push(DataBlock::new_with_meta( - vec![current_block.get_by_offset(self.row_id_idx).clone()], - current_block.num_rows(), - Some(Box::new(RowIdKind::Update)), - )); + + // for target build optimization, there is only one matched clause without condition. we won't read rowid. + if !self.target_build_optimization { + self.output_data_row_id_data.push(DataBlock::new_with_meta( + vec![current_block.get_by_offset(self.row_id_idx).clone()], + current_block.num_rows(), + Some(Box::new(RowIdKind::Update)), + )); + } + let op = BlockOperator::Project { projection: self.update_projections.clone(), }; diff --git a/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_not_matched.rs b/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_not_matched.rs index 2debb6006ad8..4fa753a005b5 100644 --- a/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_not_matched.rs +++ b/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_not_matched.rs @@ -58,6 +58,8 @@ pub struct MergeIntoNotMatchedProcessor { func_ctx: FunctionContext, // data_schemas[i] means the i-th op's result block's schema. data_schemas: HashMap, + // for target table build optimization + target_table_schema: DataSchemaRef, ctx: Arc, } @@ -67,6 +69,7 @@ impl MergeIntoNotMatchedProcessor { input_schema: DataSchemaRef, func_ctx: FunctionContext, ctx: Arc, + target_table_schema: DataSchemaRef, ) -> Result { let mut ops = Vec::::with_capacity(unmatched.len()); let mut data_schemas = HashMap::with_capacity(unmatched.len()); @@ -99,6 +102,7 @@ impl MergeIntoNotMatchedProcessor { output_data: Vec::new(), func_ctx, data_schemas, + target_table_schema, ctx, }) } @@ -155,13 +159,21 @@ impl Processor for MergeIntoNotMatchedProcessor { } fn process(&mut self) -> Result<()> { - if let Some(data_block) = self.input_data.take() { + if let Some(mut data_block) = self.input_data.take() { if data_block.is_empty() { return Ok(()); } - // target build optimization + // target build optimization, we `take_meta` not `get_meta`, because the `BlockMetaIndex` is + // just used to judge whether we need to update `merge_status`, we shouldn't pass it through. + // no_need_add_status means this the origin data block from targe table, and we can push it directly. let no_need_add_status = data_block.get_meta().is_some() - && BlockMetaIndex::downcast_ref_from(data_block.get_meta().unwrap()).is_some(); + && BlockMetaIndex::downcast_from(data_block.take_meta().unwrap()).is_some(); + if no_need_add_status { + data_block = + data_block.add_meta(Some(Box::new(self.target_table_schema.clone())))?; + self.output_data.push(data_block); + return Ok(()); + } let start = Instant::now(); let mut current_block = data_block; for (idx, op) in self.ops.iter().enumerate() { @@ -174,13 +186,11 @@ impl Processor for MergeIntoNotMatchedProcessor { metrics_inc_merge_into_append_blocks_rows_counter( satisfied_block.num_rows() as u32 ); - if !no_need_add_status { - self.ctx.add_merge_status(MergeStatus { - insert_rows: satisfied_block.num_rows(), - update_rows: 0, - deleted_rows: 0, - }); - } + self.ctx.add_merge_status(MergeStatus { + insert_rows: satisfied_block.num_rows(), + update_rows: 0, + deleted_rows: 0, + }); self.output_data .push(op.op.execute(&self.func_ctx, satisfied_block)?) diff --git a/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_split.rs b/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_split.rs index 31e83a8473fa..e39a6ac32819 100644 --- a/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_split.rs +++ b/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_split.rs @@ -156,6 +156,7 @@ impl Processor for MergeIntoSplitProcessor { if data_block.get_meta().is_some() { let meta_index = BlockMetaIndex::downcast_ref_from(data_block.get_meta().unwrap()); if meta_index.is_some() { + // we reserve the meta in data_block to avoid adding insert `merge_status` in `merge_into_not_matched` by mistake. self.output_data_not_matched_data = Some(data_block.clone()); // if the downstream receive this, it should just treat this as a DeletedLog. self.output_data_matched_data = Some(DataBlock::empty_with_meta(Box::new( diff --git a/tests/sqllogictests/suites/base/09_fuse_engine/09_0039t_arget_build_merge_into_standalone.test b/tests/sqllogictests/suites/mode/standalone/explain/09_0039_target_build_merge_into_standalone.test similarity index 92% rename from tests/sqllogictests/suites/base/09_fuse_engine/09_0039t_arget_build_merge_into_standalone.test rename to tests/sqllogictests/suites/mode/standalone/explain/09_0039_target_build_merge_into_standalone.test index fee00bd0b9ff..fd44f340bc81 100644 --- a/tests/sqllogictests/suites/base/09_fuse_engine/09_0039t_arget_build_merge_into_standalone.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/09_0039_target_build_merge_into_standalone.test @@ -46,13 +46,13 @@ merge into target_build_optimization as t1 using source_optimization as t2 on t1 ### 2.2 make sure the plan is expected query T -explain merge into target_build_optimization as t1 using source_optimization as t2 on t1.a = t2.a and t1.b = t2.b when matched then update * when not matched then insert *; +explain merge into target_build_optimization as t1 using source_optimization as t2 on t1.a = t2.a and t1.b = t2.b when matched then update set t1.a = t2.a,t1.b = t2.b,t1.c = t2.c when not matched then insert *; ---- MergeInto: target_table: default.default.target_build_optimization ├── distributed: false ├── target_build_optimization: true -├── matched update: [condition: None,update set c = c (#2),b = b (#1),a = a (#0)] +├── matched update: [condition: None,update set a = t2.a (#0),b = t2.b (#1),c = t2.c (#2)] ├── unmatched insert: [condition: None,insert into (a,b,c) values(CAST(a (#0) AS Int32 NULL),CAST(b (#1) AS String NULL),CAST(c (#2) AS String NULL))] └── HashJoin: LEFT OUTER ├── equi conditions: [and(eq(t2.a (#0), t1.a (#3)), eq(t2.b (#1), t1.b (#4)))] @@ -106,13 +106,13 @@ select count(*) from fuse_block('default','target_build_optimization'); ### make sure the plan is expected query T -explain merge into target_build_optimization as t1 using source_optimization as t2 on t1.a = t2.a and t1.b = t2.b when matched then update * when not matched then insert *; +explain merge into target_build_optimization as t1 using source_optimization as t2 on t1.a = t2.a and t1.b = t2.b when matched then update set t1.a = t2.a,t1.b = t2.b,t1.c = t2.c when not matched then insert *; ---- MergeInto: target_table: default.default.target_build_optimization ├── distributed: false ├── target_build_optimization: true -├── matched update: [condition: None,update set c = c (#2),b = b (#1),a = a (#0)] +├── matched update: [condition: None,update set a = t2.a (#0),b = t2.b (#1),c = t2.c (#2)] ├── unmatched insert: [condition: None,insert into (a,b,c) values(CAST(a (#0) AS Int32 NULL),CAST(b (#1) AS String NULL),CAST(c (#2) AS String NULL))] └── HashJoin: LEFT OUTER ├── equi conditions: [and(eq(t2.a (#0), t1.a (#3)), eq(t2.b (#1), t1.b (#4)))] @@ -180,13 +180,13 @@ select count(*) from fuse_block('default','target_build_optimization'); ### make sure the plan is expected query T -explain merge into target_build_optimization as t1 using source_optimization as t2 on t1.a = t2.a and t1.b = t2.b when matched then update * when not matched then insert *; +explain merge into target_build_optimization as t1 using source_optimization as t2 on t1.a = t2.a and t1.b = t2.b when matched then update set t1.a = t2.a,t1.b = t2.b,t1.c = t2.c when not matched then insert *; ---- MergeInto: target_table: default.default.target_build_optimization ├── distributed: false ├── target_build_optimization: true -├── matched update: [condition: None,update set b = b (#1),a = a (#0),c = c (#2)] +├── matched update: [condition: None,update set a = t2.a (#0),b = t2.b (#1),c = t2.c (#2)] ├── unmatched insert: [condition: None,insert into (a,b,c) values(CAST(a (#0) AS Int32 NULL),CAST(b (#1) AS String NULL),CAST(c (#2) AS String NULL))] └── HashJoin: LEFT OUTER ├── equi conditions: [eq(t2.b (#1), t1.b (#4))] @@ -261,13 +261,13 @@ truncate table target_build_optimization; ### make sure the plan is expected query T -explain merge into target_build_optimization as t1 using source_optimization as t2 on t1.a = t2.a and t1.b = t2.b when matched then update * when not matched then insert *; +explain merge into target_build_optimization as t1 using source_optimization as t2 on t1.a = t2.a and t1.b = t2.b when matched then update set t1.a = t2.a,t1.b = t2.b,t1.c = t2.c when not matched then insert *; ---- MergeInto: target_table: default.default.target_build_optimization ├── distributed: false ├── target_build_optimization: true -├── matched update: [condition: None,update set b = b (#1),a = a (#0),c = c (#2)] +├── matched update: [condition: None,update set a = t2.a (#0),b = t2.b (#1),c = t2.c (#2)] ├── unmatched insert: [condition: None,insert into (a,b,c) values(CAST(a (#0) AS Int32 NULL),CAST(b (#1) AS String NULL),CAST(c (#2) AS String NULL))] └── HashJoin: LEFT OUTER ├── equi conditions: [eq(t2.b (#1), t1.b (#4))] From 47c44eee4cecb611cf0e4044ab0ae13ec677972d Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Sat, 13 Jan 2024 02:36:43 +0800 Subject: [PATCH 23/47] fix all matched delete for target build optimization --- src/common/hashtable/src/utils.rs | 121 ++++++++++++++++-- .../hash_join/hash_join_probe_state.rs | 9 +- .../hash_join/transform_hash_join_probe.rs | 13 +- .../processors/processor_merge_into_split.rs | 5 +- 4 files changed, 134 insertions(+), 14 deletions(-) diff --git a/src/common/hashtable/src/utils.rs b/src/common/hashtable/src/utils.rs index dfb4d8fa53ae..c4d4adea2b63 100644 --- a/src/common/hashtable/src/utils.rs +++ b/src/common/hashtable/src/utils.rs @@ -199,7 +199,7 @@ pub mod sse { // `merge into t using source on xxx when matched then update xxx when not macthed then insert xxx. // for merge into: // we use BlockInfoIndex to maintain an index for the block info in chunks. -#[allow(dead_code)] + pub struct BlockInfoIndex { // the intervals will be like below: // (0,10)(11,29),(30,38). it's ordered. @@ -221,9 +221,7 @@ pub type Interval = (u32, u32); /// segment2_block0 | /// /// ......... - impl BlockInfoIndex { - #[allow(dead_code)] pub fn new_with_capacity(capacity: usize) -> Self { BlockInfoIndex { intervals: Vec::with_capacity(capacity), @@ -235,7 +233,6 @@ impl BlockInfoIndex { /// 1.interval stands for the (start,end) in chunks for one block. /// 2.prefix is the segment_id_block_id composition. /// we can promise the ordered insert from outside. - #[allow(dead_code)] pub fn insert_block_offsets(&mut self, interval: Interval, prefix: u64) { self.intervals.push(interval); self.prefixs.push(prefix); @@ -255,8 +252,7 @@ impl BlockInfoIndex { /// |--| /// case4: |-----|------|------| /// |--------| - #[allow(dead_code)] - pub fn get_block_info(&self, interval: Interval) -> Vec<(Interval, u64)> { + fn get_block_info(&self, interval: Interval) -> Vec<(Interval, u64)> { let mut res = Vec::<(Interval, u64)>::with_capacity(2); let left_idx = self.search_idx(interval.0); let right_idx = self.search_idx(interval.1); @@ -282,7 +278,6 @@ impl BlockInfoIndex { /// search idx help us to find out the intervals idx which contain offset. /// It must contain offset. - #[allow(dead_code)] fn search_idx(&self, offset: u32) -> usize { let mut l = 0; let mut r = self.length - 1; @@ -297,7 +292,55 @@ impl BlockInfoIndex { l } - #[allow(dead_code)] + pub fn gather_matched_all_blocks(&self, hits: &[u8]) -> Vec { + let mut res = Vec::with_capacity(10); + let mut step = 0; + while step < hits.len() { + if hits[step] == 1 { + break; + } + step += 1; + } + if step == hits.len() { + return res; + } + let mut start = step; + let mut end = step; + while start < hits.len() { + while end < hits.len() && hits[end] == 1 { + end += 1; + } + let left = self.search_idx(start as u32); + let right = self.search_idx((end - 1) as u32); + if left == right { + // macthed only one block. + if self.intervals[left].0 == (start as u32) + && self.intervals[right].1 == (end - 1) as u32 + { + res.push(self.prefixs[left]); + } + } else { + assert!(right > left); + // 1. left most side. + if self.intervals[left].0 == start as u32 { + res.push(self.prefixs[left]); + } + for idx in left + 1..right { + res.push(self.prefixs[idx]); + } + // 2. right most side. + if self.intervals[right].1 == (end - 1) as u32 { + res.push(self.prefixs[right]); + } + } + while end < hits.len() && hits[end] == 0 { + end += 1; + } + start = end; + } + res + } + pub fn gather_all_partial_block_offsets(&self, hits: &[u8]) -> Vec<(Interval, u64)> { let mut res = Vec::with_capacity(10); let mut step = 0; @@ -330,7 +373,7 @@ impl BlockInfoIndex { &self, partial_unmodified: &Vec<(Interval, u64)>, chunks_offsets: &Vec, - ) -> Vec<(Vec<(Interval, u64)>, u32)> { + ) -> Vec<(Vec<(Interval, u64)>, u64)> { let mut res = Vec::with_capacity(chunks_offsets.len()); let mut chunk_idx = 0; let mut partial_idx = 0; @@ -339,7 +382,7 @@ impl BlockInfoIndex { // here is '<', not '<=', chunks_offsets[chunk_idx] is the count of chunks[chunk_idx] if partial_unmodified[partial_idx].0.1 < chunks_offsets[chunk_idx] { if offset >= res.len() { - res.push((Vec::new(), chunk_idx as u32)); + res.push((Vec::new(), chunk_idx as u64)); } res[offset].0.push(partial_unmodified[partial_idx]) } else { @@ -349,6 +392,10 @@ impl BlockInfoIndex { } partial_idx += 1; } + // check + for chunk in &res { + assert!(!chunk.0.is_empty()); + } res } } @@ -477,4 +524,58 @@ fn test_block_info_index() { assert_eq!(res[0].0.len(), 1); assert_eq!(res[0].1, 0); // chunk_idx assert_eq!(res[0].0[0], ((13, 16), 1)); + + // test matched all blocks + // blocks: [0,10][11,20][21,30],[31,39] + + // 1.empty + let mut hits = vec![0; 40]; + // set [11,19] + for item in hits.iter_mut().take(19 + 1).skip(11) { + *item = 1; + } + let res = block_info_index.gather_matched_all_blocks(&hits); + assert!(res.is_empty()); + + let mut hits = vec![0; 40]; + // set [13,28] + for item in hits.iter_mut().take(28 + 1).skip(13) { + *item = 1; + } + let res = block_info_index.gather_matched_all_blocks(&hits); + assert!(res.is_empty()); + + // 2.one + let mut hits = vec![0; 40]; + // set [11,20] + for item in hits.iter_mut().take(20 + 1).skip(11) { + *item = 1; + } + let res = block_info_index.gather_matched_all_blocks(&hits); + assert!(res.len() == 1 && res[0] == 1); + + let mut hits = vec![0; 40]; + // set [13,33] + for item in hits.iter_mut().take(33 + 1).skip(13) { + *item = 1; + } + let res = block_info_index.gather_matched_all_blocks(&hits); + assert!(res.len() == 1 && res[0] == 2); + + // 3.multi blocks + let mut hits = vec![0; 40]; + // set [11,30] + for item in hits.iter_mut().take(30 + 1).skip(11) { + *item = 1; + } + let res = block_info_index.gather_matched_all_blocks(&hits); + assert!(res.len() == 2 && res[0] == 1 && res[1] == 2); + + let mut hits = vec![0; 40]; + // set [10,31] + for item in hits.iter_mut().take(31 + 1).skip(11) { + *item = 1; + } + let res = block_info_index.gather_matched_all_blocks(&hits); + assert!(res.len() == 2 && res[0] == 1 && res[1] == 2); } diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs index c9cd39ac37b9..721fd5088fdf 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs @@ -57,7 +57,7 @@ use crate::pipelines::processors::HashJoinState; use crate::sessions::QueryContext; use crate::sql::planner::plans::JoinType; -pub type ChunkPartialUnmodified = (Vec<(Interval, u64)>, u32); +pub type ChunkPartialUnmodified = (Vec<(Interval, u64)>, u64); /// Define some shared states for all hash join probe threads. pub struct HashJoinProbeState { pub(crate) ctx: Arc, @@ -488,8 +488,13 @@ impl HashJoinProbeState { let matched = unsafe { &*self.hash_join_state.matched.get() }; let chunks_offsets = unsafe { &*self.hash_join_state.chunk_offsets.get() }; let partial_unmodified = block_info_index.gather_all_partial_block_offsets(matched); + let all_matched_blocks = block_info_index.gather_matched_all_blocks(matched); // generate chunks - let tasks = block_info_index.chunk_offsets(&partial_unmodified, chunks_offsets); + let mut tasks = block_info_index.chunk_offsets(&partial_unmodified, chunks_offsets); + for prefix in all_matched_blocks { + // deleted block + tasks.push((Vec::new(), prefix)); + } *self.final_merge_into_partial_unmodified_scan_tasks.write() = tasks.into(); Ok(()) } diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/transform_hash_join_probe.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/transform_hash_join_probe.rs index d8b9fa6821c6..d694f01222d8 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/transform_hash_join_probe.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/transform_hash_join_probe.rs @@ -129,6 +129,18 @@ impl TransformHashJoinProbe { &mut self, item: ChunkPartialUnmodified, ) -> Result<()> { + // matched whole block, need to delete + if item.0.is_empty() { + let prefix = item.1; + let (segment_idx, block_idx) = split_prefix(prefix); + let data_block = DataBlock::empty_with_meta(Box::new(BlockMetaIndex { + segment_idx: segment_idx as usize, + block_idx: block_idx as usize, + inner: None, + })); + self.output_data_blocks.push_back(data_block); + return Ok(()); + } let build_state = unsafe { &*self.join_probe_state.hash_join_state.build_state.get() }; let chunk_block = &build_state.generation_state.chunks[item.1 as usize]; for (interval, prefix) in item.0 { @@ -149,7 +161,6 @@ impl TransformHashJoinProbe { block_idx: block_idx as usize, inner: None, })))?; - // add null columns self.output_data_blocks.push_back(data_block); } } diff --git a/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_split.rs b/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_split.rs index e39a6ac32819..54c7716d5260 100644 --- a/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_split.rs +++ b/src/query/storages/fuse/src/operations/merge_into/processors/processor_merge_into_split.rs @@ -157,7 +157,10 @@ impl Processor for MergeIntoSplitProcessor { let meta_index = BlockMetaIndex::downcast_ref_from(data_block.get_meta().unwrap()); if meta_index.is_some() { // we reserve the meta in data_block to avoid adding insert `merge_status` in `merge_into_not_matched` by mistake. - self.output_data_not_matched_data = Some(data_block.clone()); + // if `is_empty`, it's a whole block matched, we need to delete. + if !data_block.is_empty() { + self.output_data_not_matched_data = Some(data_block.clone()); + } // if the downstream receive this, it should just treat this as a DeletedLog. self.output_data_matched_data = Some(DataBlock::empty_with_meta(Box::new( meta_index.unwrap().clone(), From eb16d7da540024c1bab3bd58813a59446f252138 Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Sat, 13 Jan 2024 03:11:19 +0800 Subject: [PATCH 24/47] fix test --- .../explain/09_0039_target_build_merge_into_standalone.test | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/sqllogictests/suites/mode/standalone/explain/09_0039_target_build_merge_into_standalone.test b/tests/sqllogictests/suites/mode/standalone/explain/09_0039_target_build_merge_into_standalone.test index fd44f340bc81..b8143581b3fe 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/09_0039_target_build_merge_into_standalone.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/09_0039_target_build_merge_into_standalone.test @@ -180,7 +180,7 @@ select count(*) from fuse_block('default','target_build_optimization'); ### make sure the plan is expected query T -explain merge into target_build_optimization as t1 using source_optimization as t2 on t1.a = t2.a and t1.b = t2.b when matched then update set t1.a = t2.a,t1.b = t2.b,t1.c = t2.c when not matched then insert *; +explain merge into target_build_optimization as t1 using source_optimization as t2 on t1.a = t2.a and t1.b > t2.b when matched then update set t1.a = t2.a,t1.b = t2.b,t1.c = t2.c when not matched then insert *; ---- MergeInto: target_table: default.default.target_build_optimization @@ -261,7 +261,7 @@ truncate table target_build_optimization; ### make sure the plan is expected query T -explain merge into target_build_optimization as t1 using source_optimization as t2 on t1.a = t2.a and t1.b = t2.b when matched then update set t1.a = t2.a,t1.b = t2.b,t1.c = t2.c when not matched then insert *; +explain merge into target_build_optimization as t1 using source_optimization as t2 on t1.a = t2.a and t1.b > t2.b when matched then update set t1.a = t2.a,t1.b = t2.b,t1.c = t2.c when not matched then insert *; ---- MergeInto: target_table: default.default.target_build_optimization From 10879f52b3b34ea8457cdf9e72244d45020bdb02 Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Sat, 13 Jan 2024 13:47:04 +0800 Subject: [PATCH 25/47] add info log --- .../transforms/hash_join/probe_join/left_join.rs | 2 +- .../transforms/hash_join/transform_hash_join_probe.rs | 8 ++++++++ .../src/operations/merge_into/mutator/matched_mutator.rs | 4 ++++ .../09_0039_target_build_merge_into_standalone.test | 2 +- 4 files changed, 14 insertions(+), 2 deletions(-) diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/probe_join/left_join.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/probe_join/left_join.rs index b2074b35ed82..11e9e4bac56a 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/probe_join/left_join.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/probe_join/left_join.rs @@ -403,7 +403,7 @@ impl HashJoinProbeState { let new_matched_count = old_mactehd_counts + 1; if old_mactehd_counts > 0 { return Err(ErrorCode::UnresolvableConflict( - "multi rows from source match one and the same row in the target_table multi times", + "multi rows from source match one and the same row in the target_table multi times in probe phase", )); } loop { diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/transform_hash_join_probe.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/transform_hash_join_probe.rs index d694f01222d8..dbb33b1ec91b 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/transform_hash_join_probe.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/transform_hash_join_probe.rs @@ -133,6 +133,10 @@ impl TransformHashJoinProbe { if item.0.is_empty() { let prefix = item.1; let (segment_idx, block_idx) = split_prefix(prefix); + info!( + "matched whole block: segment_idx: {}, block_idx: {}", + segment_idx, block_idx + ); let data_block = DataBlock::empty_with_meta(Box::new(BlockMetaIndex { segment_idx: segment_idx as usize, block_idx: block_idx as usize, @@ -156,6 +160,10 @@ impl TransformHashJoinProbe { )?; assert!(!data_block.is_empty()); let (segment_idx, block_idx) = split_prefix(prefix); + info!( + "matched partial block: segment_idx: {}, block_idx: {}", + segment_idx, block_idx + ); let data_block = data_block.add_meta(Some(Box::new(BlockMetaIndex { segment_idx: segment_idx as usize, block_idx: block_idx as usize, diff --git a/src/query/storages/fuse/src/operations/merge_into/mutator/matched_mutator.rs b/src/query/storages/fuse/src/operations/merge_into/mutator/matched_mutator.rs index df4eaa9443f9..d9b3b63b737f 100644 --- a/src/query/storages/fuse/src/operations/merge_into/mutator/matched_mutator.rs +++ b/src/query/storages/fuse/src/operations/merge_into/mutator/matched_mutator.rs @@ -144,6 +144,10 @@ impl MatchedAggregator { .meta_indexes .insert((meta_index.segment_idx, meta_index.block_idx)) { + info!( + "duplicated block: segment_idx: {}, block_idx: {}", + meta_index.segment_idx, meta_index.block_idx + ); return Err(ErrorCode::Internal( "merge into get duplicated block for target build unmodified optimization", )); diff --git a/tests/sqllogictests/suites/mode/standalone/explain/09_0039_target_build_merge_into_standalone.test b/tests/sqllogictests/suites/mode/standalone/explain/09_0039_target_build_merge_into_standalone.test index b31b20b9083a..e9cdadf2b9a9 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/09_0039_target_build_merge_into_standalone.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/09_0039_target_build_merge_into_standalone.test @@ -286,7 +286,7 @@ target_table: default.default.target_build_optimization query TT merge into target_build_optimization as t1 using source_optimization as t2 on t1.a > t2.a and t1.b = t2.b when matched then update * when not matched then insert *; ---- -12 0 +10 0 query TTT select * from target_build_optimization order by a,b,c; From 33c0c4d90d45d48c90450bc13e06e19fa7c9fe0a Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Sat, 13 Jan 2024 16:07:59 +0800 Subject: [PATCH 26/47] add logs --- .../processors/transforms/hash_join/hash_join_probe_state.rs | 5 +++++ .../src/operations/merge_into/mutator/matched_mutator.rs | 2 +- 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs index 721fd5088fdf..c75307d5b207 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs @@ -491,10 +491,15 @@ impl HashJoinProbeState { let all_matched_blocks = block_info_index.gather_matched_all_blocks(matched); // generate chunks let mut tasks = block_info_index.chunk_offsets(&partial_unmodified, chunks_offsets); + info!("partial unmodified blocks num: {}", tasks.len()); for prefix in all_matched_blocks { // deleted block tasks.push((Vec::new(), prefix)); } + info!( + "partial unmodified blocks and matched whole blocks num: {}", + tasks.len() + ); *self.final_merge_into_partial_unmodified_scan_tasks.write() = tasks.into(); Ok(()) } diff --git a/src/query/storages/fuse/src/operations/merge_into/mutator/matched_mutator.rs b/src/query/storages/fuse/src/operations/merge_into/mutator/matched_mutator.rs index d9b3b63b737f..710a79ce228b 100644 --- a/src/query/storages/fuse/src/operations/merge_into/mutator/matched_mutator.rs +++ b/src/query/storages/fuse/src/operations/merge_into/mutator/matched_mutator.rs @@ -268,7 +268,7 @@ impl MatchedAggregator { let segment_info = segment_infos.get(&item.0).unwrap(); let block_idx = segment_info.blocks.len() - block_idx - 1; info!( - "merge into apply: segment_idx:{},blk_idx:{}", + "target_build_optimization, merge into apply: segment_idx:{},blk_idx:{}", segment_idx, block_idx ); mutation_logs.push(MutationLogEntry::DeletedBlock { From 1b3ed9aa529b0ed5ed6e5e370e4c8e2c05777871 Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Sat, 13 Jan 2024 17:27:34 +0800 Subject: [PATCH 27/47] add debug logs --- src/common/hashtable/src/utils.rs | 3 +++ .../transforms/hash_join/hash_join_probe_state.rs | 12 ++++++++++++ 2 files changed, 15 insertions(+) diff --git a/src/common/hashtable/src/utils.rs b/src/common/hashtable/src/utils.rs index c4d4adea2b63..c863a0203499 100644 --- a/src/common/hashtable/src/utils.rs +++ b/src/common/hashtable/src/utils.rs @@ -230,6 +230,9 @@ impl BlockInfoIndex { } } + pub fn print_intervals(&self) -> Vec { + self.intervals.clone() + } /// 1.interval stands for the (start,end) in chunks for one block. /// 2.prefix is the segment_id_block_id composition. /// we can promise the ordered insert from outside. diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs index c75307d5b207..137f75c650d4 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs @@ -20,6 +20,7 @@ use std::sync::Arc; use databend_common_arrow::arrow::bitmap::Bitmap; use databend_common_arrow::arrow::bitmap::MutableBitmap; +use databend_common_arrow::arrow::chunk; use databend_common_base::base::tokio::sync::Barrier; use databend_common_exception::ErrorCode; use databend_common_exception::Result; @@ -490,6 +491,17 @@ impl HashJoinProbeState { let partial_unmodified = block_info_index.gather_all_partial_block_offsets(matched); let all_matched_blocks = block_info_index.gather_matched_all_blocks(matched); // generate chunks + info!("chunk len: {}", chunks_offsets.len()); + for chunk_offset in chunks_offsets { + info!("chunk offset: {}", chunk_offset); + } + let intervals = block_info_index.print_intervals(); + for interval in intervals { + info!("interval :{} ", interval); + } + for (row_idx, hit) in matched.iter().enumerate() { + info!("row_idx :{}, hit: {} ", row_idx, hit); + } let mut tasks = block_info_index.chunk_offsets(&partial_unmodified, chunks_offsets); info!("partial unmodified blocks num: {}", tasks.len()); for prefix in all_matched_blocks { From 01d8099a819a087fd96c0cd8836a0a06c9511590 Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Sat, 13 Jan 2024 17:28:32 +0800 Subject: [PATCH 28/47] add debug logs --- .../processors/transforms/hash_join/hash_join_probe_state.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs index 137f75c650d4..860ab9ee31ba 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs @@ -497,7 +497,7 @@ impl HashJoinProbeState { } let intervals = block_info_index.print_intervals(); for interval in intervals { - info!("interval :{} ", interval); + info!("interval :({},{}) ", interval.0, interval.1); } for (row_idx, hit) in matched.iter().enumerate() { info!("row_idx :{}, hit: {} ", row_idx, hit); From 8acc5f7e9df8e6e9c6775c07f90d03e453c5824c Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Sat, 13 Jan 2024 18:12:06 +0800 Subject: [PATCH 29/47] fix lint --- .../processors/transforms/hash_join/hash_join_probe_state.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs index 860ab9ee31ba..389a85188472 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs @@ -20,7 +20,6 @@ use std::sync::Arc; use databend_common_arrow::arrow::bitmap::Bitmap; use databend_common_arrow::arrow::bitmap::MutableBitmap; -use databend_common_arrow::arrow::chunk; use databend_common_base::base::tokio::sync::Barrier; use databend_common_exception::ErrorCode; use databend_common_exception::Result; From c0a6927ed3b39d0c5063814c0b6ed4c23c6ff3d5 Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Sat, 13 Jan 2024 20:22:29 +0800 Subject: [PATCH 30/47] forbiden native engine for target build optimization --- src/common/hashtable/src/utils.rs | 5 +-- .../interpreters/interpreter_merge_into.rs | 39 +++++++++++++------ .../hash_join/hash_join_probe_state.rs | 11 +----- .../sql/src/planner/binder/merge_into.rs | 1 + .../sql/src/planner/optimizer/optimizer.rs | 4 +- ...merge_into_without_distributed_enable.test | 2 + 6 files changed, 35 insertions(+), 27 deletions(-) diff --git a/src/common/hashtable/src/utils.rs b/src/common/hashtable/src/utils.rs index c863a0203499..82c6c50642db 100644 --- a/src/common/hashtable/src/utils.rs +++ b/src/common/hashtable/src/utils.rs @@ -203,7 +203,7 @@ pub mod sse { pub struct BlockInfoIndex { // the intervals will be like below: // (0,10)(11,29),(30,38). it's ordered. - intervals: Vec, + pub intervals: Vec, prefixs: Vec, length: usize, } @@ -230,9 +230,6 @@ impl BlockInfoIndex { } } - pub fn print_intervals(&self) -> Vec { - self.intervals.clone() - } /// 1.interval stands for the (start,end) in chunks for one block. /// 2.prefix is the segment_id_block_id composition. /// we can promise the ordered insert from outside. diff --git a/src/query/service/src/interpreters/interpreter_merge_into.rs b/src/query/service/src/interpreters/interpreter_merge_into.rs index cf4bc84052bc..3d24f8f231df 100644 --- a/src/query/service/src/interpreters/interpreter_merge_into.rs +++ b/src/query/service/src/interpreters/interpreter_merge_into.rs @@ -16,6 +16,7 @@ use std::collections::HashMap; use std::sync::Arc; use std::u64::MAX; +use databend_common_catalog::merge_into_join::MergeIntoJoin; use databend_common_catalog::table::TableExt; use databend_common_exception::ErrorCode; use databend_common_exception::Result; @@ -51,6 +52,7 @@ use databend_common_sql::IndexType; use databend_common_sql::ScalarExpr; use databend_common_sql::TypeCheck; use databend_common_sql::DUMMY_COLUMN_INDEX; +use databend_common_sql::DUMMY_TABLE_INDEX; use databend_common_storages_factory::Table; use databend_common_storages_fuse::FuseTable; use databend_common_storages_fuse::TableContext; @@ -140,8 +142,18 @@ impl MergeIntoInterpreter { distributed, change_join_order, split_idx, + row_id_index, .. } = &self.plan; + let mut columns_set = columns_set.clone(); + let table = self.ctx.get_table(catalog, database, table_name).await?; + let fuse_table = table.as_any().downcast_ref::().ok_or_else(|| { + ErrorCode::Unimplemented(format!( + "table {}, engine type {}, does not support MERGE INTO", + table.name(), + table.get_table_info().engine(), + )) + })?; // attentation!! for now we have some strategies: // 1. target_build_optimization, this is enabled in standalone mode and in this case we don't need rowid column anymore. @@ -163,13 +175,27 @@ impl MergeIntoInterpreter { // II. change join order is false and match_pattern and not enable spill, we use right outer join with rownumber distributed strategies. // III otherwise, use `merge_into_join_sexpr` as standalone execution(so if change join order is false,but doesn't match_pattern, we don't support distributed,in fact. case I // can take this at most time, if that's a hash shuffle, the I can take it. We think source is always very small). - let target_build_optimization = + let mut target_build_optimization = matches!(self.plan.merge_type, MergeIntoType::FullOperation) && !self.plan.columns_set.contains(&self.plan.row_id_index); - if target_build_optimization { assert!(*change_join_order && !*distributed); + // so if `target_build_optimization` is true, it means the optimizer enable this rule. + // but we need to check if it's parquet format or native format. for now,we just support + // parquet. (we will support native in the next pr). + if fuse_table.is_native() { + target_build_optimization = false; + // and we need to add row_id back and forbidden target_build_optimization + columns_set.insert(*row_id_index); + let merge_into_join = self.ctx.get_merge_into_join(); + self.ctx.set_merge_into_join(MergeIntoJoin { + target_tbl_idx: DUMMY_TABLE_INDEX, + is_distributed: merge_into_join.is_distributed, + merge_into_join_type: merge_into_join.merge_into_join_type, + }); + } } + // check mutability let check_table = self.ctx.get_table(catalog, database, table_name).await?; check_table.check_mutable()?; @@ -257,15 +283,6 @@ impl MergeIntoInterpreter { )); } - let table = self.ctx.get_table(catalog, database, &table_name).await?; - let fuse_table = table.as_any().downcast_ref::().ok_or_else(|| { - ErrorCode::Unimplemented(format!( - "table {}, engine type {}, does not support MERGE INTO", - table.name(), - table.get_table_info().engine(), - )) - })?; - let table_info = fuse_table.get_table_info().clone(); let catalog_ = self.ctx.get_catalog(catalog).await?; diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs index 389a85188472..9a6ea75b90fe 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs @@ -491,16 +491,7 @@ impl HashJoinProbeState { let all_matched_blocks = block_info_index.gather_matched_all_blocks(matched); // generate chunks info!("chunk len: {}", chunks_offsets.len()); - for chunk_offset in chunks_offsets { - info!("chunk offset: {}", chunk_offset); - } - let intervals = block_info_index.print_intervals(); - for interval in intervals { - info!("interval :({},{}) ", interval.0, interval.1); - } - for (row_idx, hit) in matched.iter().enumerate() { - info!("row_idx :{}, hit: {} ", row_idx, hit); - } + info!("intervals len: {} ", block_info_index.intervals.len()); let mut tasks = block_info_index.chunk_offsets(&partial_unmodified, chunks_offsets); info!("partial unmodified blocks num: {}", tasks.len()); for prefix in all_matched_blocks { diff --git a/src/query/sql/src/planner/binder/merge_into.rs b/src/query/sql/src/planner/binder/merge_into.rs index 3c9e7f04e9a6..9bc0254f4547 100644 --- a/src/query/sql/src/planner/binder/merge_into.rs +++ b/src/query/sql/src/planner/binder/merge_into.rs @@ -402,6 +402,7 @@ impl Binder { } } assert!(split_idx != DUMMY_COLUMN_INDEX); + Ok(MergeInto { catalog: catalog_name.to_string(), database: database_name.to_string(), diff --git a/src/query/sql/src/planner/optimizer/optimizer.rs b/src/query/sql/src/planner/optimizer/optimizer.rs index f7d7e90eee3d..d3dd0e5d7312 100644 --- a/src/query/sql/src/planner/optimizer/optimizer.rs +++ b/src/query/sql/src/planner/optimizer/optimizer.rs @@ -328,8 +328,8 @@ fn optimize_merge_into(opt_ctx: OptimizerContext, plan: Box) -> Resul // we just support left join to use MergeIntoBlockInfoHashTable, we // don't support spill for now, and we need the macthed clauses' count - // is one, just support merge into t using source when matched then - // update xx when not matched then insert xx. + // is one, just support `merge into t using source when matched then + // update xx when not matched then insert xx`. let flag = plan.matched_evaluators.len() == 1 && plan.matched_evaluators[0].condition.is_none() && plan.matched_evaluators[0].update.is_some() diff --git a/tests/sqllogictests/suites/base/09_fuse_engine/09_0036_merge_into_without_distributed_enable.test b/tests/sqllogictests/suites/base/09_fuse_engine/09_0036_merge_into_without_distributed_enable.test index 05bdf380266c..9cc961bec218 100644 --- a/tests/sqllogictests/suites/base/09_fuse_engine/09_0036_merge_into_without_distributed_enable.test +++ b/tests/sqllogictests/suites/base/09_fuse_engine/09_0036_merge_into_without_distributed_enable.test @@ -849,6 +849,8 @@ CREATE TABLE orders CLUSTER BY (to_yyyymmddhh(created_at), user_id) AS SELECT date_add('day', floor(rand() * 10 % 365)::int, '2021-01-01') AS updated_at FROM numbers(5000); +### for now, we disable target_table_optimization for native. Native will +### spilt one block into multi pages. We should fix this one in the future. statement ok MERGE INTO orders USING ( From ababc1acdcbd77bb113cfdd5461d4bc44ea97ac6 Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Mon, 15 Jan 2024 00:12:59 +0800 Subject: [PATCH 31/47] add logs --- src/common/hashtable/src/utils.rs | 7 +++++++ .../transforms/hash_join/hash_join_probe_state.rs | 14 +++++++++----- 2 files changed, 16 insertions(+), 5 deletions(-) diff --git a/src/common/hashtable/src/utils.rs b/src/common/hashtable/src/utils.rs index 82c6c50642db..1a5352b578fa 100644 --- a/src/common/hashtable/src/utils.rs +++ b/src/common/hashtable/src/utils.rs @@ -375,6 +375,12 @@ impl BlockInfoIndex { chunks_offsets: &Vec, ) -> Vec<(Vec<(Interval, u64)>, u64)> { let mut res = Vec::with_capacity(chunks_offsets.len()); + if chunks_offsets.len() == 0 { + assert!(partial_unmodified.is_empty()); + } + if partial_unmodified.is_empty() || chunks_offsets.is_empty() { + return res; + } let mut chunk_idx = 0; let mut partial_idx = 0; let mut offset = 0; @@ -383,6 +389,7 @@ impl BlockInfoIndex { if partial_unmodified[partial_idx].0.1 < chunks_offsets[chunk_idx] { if offset >= res.len() { res.push((Vec::new(), chunk_idx as u64)); + assert_eq!(offset + 1, res.len()); } res[offset].0.push(partial_unmodified[partial_idx]) } else { diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs index 9a6ea75b90fe..56309cef3fbb 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs @@ -492,16 +492,20 @@ impl HashJoinProbeState { // generate chunks info!("chunk len: {}", chunks_offsets.len()); info!("intervals len: {} ", block_info_index.intervals.len()); + info!( + "partial unmodified blocks num: {}", + partial_unmodified.len() + ); + info!( + "all_matched_blocks blocks num: {}", + all_matched_blocks.len() + ); let mut tasks = block_info_index.chunk_offsets(&partial_unmodified, chunks_offsets); - info!("partial unmodified blocks num: {}", tasks.len()); + info!("partial unmodified chunk num: {}", tasks.len()); for prefix in all_matched_blocks { // deleted block tasks.push((Vec::new(), prefix)); } - info!( - "partial unmodified blocks and matched whole blocks num: {}", - tasks.len() - ); *self.final_merge_into_partial_unmodified_scan_tasks.write() = tasks.into(); Ok(()) } From 0901be5b828001261654da4a3b6566dd18fff0cc Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Mon, 15 Jan 2024 00:24:29 +0800 Subject: [PATCH 32/47] add more log --- .../transforms/hash_join/hash_join_probe_state.rs | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs index 56309cef3fbb..dbbe830c4e85 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs @@ -489,6 +489,19 @@ impl HashJoinProbeState { let chunks_offsets = unsafe { &*self.hash_join_state.chunk_offsets.get() }; let partial_unmodified = block_info_index.gather_all_partial_block_offsets(matched); let all_matched_blocks = block_info_index.gather_matched_all_blocks(matched); + let mut chunks_offsets_string = String::from("["); + for offset in chunks_offsets { + chunks_offsets_string.push_str(&format!("{},", offset).to_string()); + } + chunks_offsets_string.push_str("]"); + info!("chunks_offsets_string: {}", chunks_offsets_string); + let mut partial_unmodified_string = String::from("["); + for item in &partial_unmodified { + partial_unmodified_string + .push_str(&format!("({},{}),{}\n", item.0.0, item.0.1, item.1).to_string()); + } + partial_unmodified_string.push_str("]"); + info!("partial_unmodified_string: {}", partial_unmodified_string); // generate chunks info!("chunk len: {}", chunks_offsets.len()); info!("intervals len: {} ", block_info_index.intervals.len()); From 16dde6c9354f788c8acc1e7ac6af1cb0a6984d52 Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Mon, 15 Jan 2024 01:20:28 +0800 Subject: [PATCH 33/47] add debug log --- src/common/hashtable/src/utils.rs | 2 +- .../hash_join/hash_join_probe_state.rs | 19 ++++++++++++++----- 2 files changed, 15 insertions(+), 6 deletions(-) diff --git a/src/common/hashtable/src/utils.rs b/src/common/hashtable/src/utils.rs index 1a5352b578fa..0e7c84c98dc5 100644 --- a/src/common/hashtable/src/utils.rs +++ b/src/common/hashtable/src/utils.rs @@ -375,7 +375,7 @@ impl BlockInfoIndex { chunks_offsets: &Vec, ) -> Vec<(Vec<(Interval, u64)>, u64)> { let mut res = Vec::with_capacity(chunks_offsets.len()); - if chunks_offsets.len() == 0 { + if chunks_offsets.is_empty() { assert!(partial_unmodified.is_empty()); } if partial_unmodified.is_empty() || chunks_offsets.is_empty() { diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs index dbbe830c4e85..1861103f206d 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs @@ -489,28 +489,37 @@ impl HashJoinProbeState { let chunks_offsets = unsafe { &*self.hash_join_state.chunk_offsets.get() }; let partial_unmodified = block_info_index.gather_all_partial_block_offsets(matched); let all_matched_blocks = block_info_index.gather_matched_all_blocks(matched); + + let mut intervals_string = String::from("["); + for offset in &block_info_index.intervals { + intervals_string.push_str(&format!("({},{}) ", offset.0, offset.1).to_string()); + } + intervals_string.push_str("]"); + info!("intervals_string: {}", intervals_string); + let mut chunks_offsets_string = String::from("["); for offset in chunks_offsets { chunks_offsets_string.push_str(&format!("{},", offset).to_string()); } chunks_offsets_string.push_str("]"); info!("chunks_offsets_string: {}", chunks_offsets_string); + let mut partial_unmodified_string = String::from("["); for item in &partial_unmodified { partial_unmodified_string - .push_str(&format!("({},{}),{}\n", item.0.0, item.0.1, item.1).to_string()); + .push_str(&format!("({},{}),{} ", item.0.0, item.0.1, item.1).to_string()); } partial_unmodified_string.push_str("]"); info!("partial_unmodified_string: {}", partial_unmodified_string); // generate chunks - info!("chunk len: {}", chunks_offsets.len()); - info!("intervals len: {} ", block_info_index.intervals.len()); + info!("chunk len: {}", chunks_offsets.len()); // 100 + info!("intervals len: {} ", block_info_index.intervals.len()); // 206 info!( - "partial unmodified blocks num: {}", + "partial unmodified blocks num: {}", // 105 partial_unmodified.len() ); info!( - "all_matched_blocks blocks num: {}", + "all_matched_blocks blocks num: {}", // 0 all_matched_blocks.len() ); let mut tasks = block_info_index.chunk_offsets(&partial_unmodified, chunks_offsets); From 2e29d9e6d8769cfbe41213924370a0eea945d010 Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Mon, 15 Jan 2024 02:24:20 +0800 Subject: [PATCH 34/47] fix multi chunks start offset and add skip chunk ut test --- src/common/hashtable/src/utils.rs | 36 +++++++++++++++++-- .../hash_join/hash_join_probe_state.rs | 21 ----------- .../hash_join/transform_hash_join_probe.rs | 9 ++++- 3 files changed, 41 insertions(+), 25 deletions(-) diff --git a/src/common/hashtable/src/utils.rs b/src/common/hashtable/src/utils.rs index 0e7c84c98dc5..238322cd11af 100644 --- a/src/common/hashtable/src/utils.rs +++ b/src/common/hashtable/src/utils.rs @@ -384,16 +384,18 @@ impl BlockInfoIndex { let mut chunk_idx = 0; let mut partial_idx = 0; let mut offset = 0; + let mut new_chunk = true; while chunk_idx < chunks_offsets.len() && partial_idx < partial_unmodified.len() { // here is '<', not '<=', chunks_offsets[chunk_idx] is the count of chunks[chunk_idx] if partial_unmodified[partial_idx].0.1 < chunks_offsets[chunk_idx] { - if offset >= res.len() { + if new_chunk { res.push((Vec::new(), chunk_idx as u64)); - assert_eq!(offset + 1, res.len()); + offset = res.len() - 1; + new_chunk = false; } res[offset].0.push(partial_unmodified[partial_idx]) } else { - offset += 1; + new_chunk = true; chunk_idx += 1; partial_idx -= 1; } @@ -586,3 +588,31 @@ fn test_block_info_index() { let res = block_info_index.gather_matched_all_blocks(&hits); assert!(res.len() == 2 && res[0] == 1 && res[1] == 2); } + +#[test] +fn test_chunk_offsets_skip_chunk() { + // test chunk_offsets + // blocks: [0,10],[11,20],[21,30],[31,39],[40,50],[51,60] + // chunks: [0,20],[21,39],[40,60] + // chunks_offsets: [21],[40],[61] + // partial_unmodified: [((8,10),0),((40,46),4),((51,55),5)] + let partial_unmodified = vec![((8, 10), 0), ((40, 46), 4), ((51, 55), 5)]; + let chunks_offsets = vec![21, 40, 61]; + let intervals: Vec = vec![(0, 10), (11, 20), (21, 30), (31, 39), (40, 50), (51, 60)]; + let mut block_info_index = BlockInfoIndex::new_with_capacity(10); + for (idx, interval) in intervals.iter().enumerate() { + block_info_index.insert_block_offsets(*interval, idx as u64) + } + let res = block_info_index.chunk_offsets(&partial_unmodified, &chunks_offsets); + assert_eq!(res.len(), 2); + assert_eq!(res[0].0.len(), 1); + assert_eq!(res[0].0[0].0.0, 8); + assert_eq!(res[0].0[0].0.1, 10); + + assert_eq!(res[1].0.len(), 2); + assert_eq!(res[1].0[0].0.0, 40); + assert_eq!(res[1].0[0].0.1, 46); + + assert_eq!(res[1].0[1].0.0, 51); + assert_eq!(res[1].0[1].0.1, 55); +} diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs index 1861103f206d..db161eb286f0 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs @@ -490,27 +490,6 @@ impl HashJoinProbeState { let partial_unmodified = block_info_index.gather_all_partial_block_offsets(matched); let all_matched_blocks = block_info_index.gather_matched_all_blocks(matched); - let mut intervals_string = String::from("["); - for offset in &block_info_index.intervals { - intervals_string.push_str(&format!("({},{}) ", offset.0, offset.1).to_string()); - } - intervals_string.push_str("]"); - info!("intervals_string: {}", intervals_string); - - let mut chunks_offsets_string = String::from("["); - for offset in chunks_offsets { - chunks_offsets_string.push_str(&format!("{},", offset).to_string()); - } - chunks_offsets_string.push_str("]"); - info!("chunks_offsets_string: {}", chunks_offsets_string); - - let mut partial_unmodified_string = String::from("["); - for item in &partial_unmodified { - partial_unmodified_string - .push_str(&format!("({},{}),{} ", item.0.0, item.0.1, item.1).to_string()); - } - partial_unmodified_string.push_str("]"); - info!("partial_unmodified_string: {}", partial_unmodified_string); // generate chunks info!("chunk len: {}", chunks_offsets.len()); // 100 info!("intervals len: {} ", block_info_index.intervals.len()); // 206 diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/transform_hash_join_probe.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/transform_hash_join_probe.rs index dbb33b1ec91b..f3053cfd5e35 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/transform_hash_join_probe.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/transform_hash_join_probe.rs @@ -145,10 +145,17 @@ impl TransformHashJoinProbe { self.output_data_blocks.push_back(data_block); return Ok(()); } + + let chunks_offsets = unsafe { &*self.join_probe_state.hash_join_state.chunk_offsets.get() }; let build_state = unsafe { &*self.join_probe_state.hash_join_state.build_state.get() }; let chunk_block = &build_state.generation_state.chunks[item.1 as usize]; + let chunk_start = if item.1 == 0 { + 0 + } else { + chunks_offsets[(item.1 - 1) as usize] + }; for (interval, prefix) in item.0 { - let indices = (interval.0..=interval.1) + let indices = ((interval.0 - chunk_start)..=(interval.1 - chunk_start)) .collect::>() .chunks(self.max_block_size) .map(|chunk| chunk.to_vec()) From bea75aaf8cbe066c7da0f1673ceeceb11975dd9e Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Mon, 15 Jan 2024 03:18:23 +0800 Subject: [PATCH 35/47] support recieve duplicated block for matched_mutator --- .../src/operations/merge_into/mutator/matched_mutator.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/query/storages/fuse/src/operations/merge_into/mutator/matched_mutator.rs b/src/query/storages/fuse/src/operations/merge_into/mutator/matched_mutator.rs index 710a79ce228b..8b224560b5d4 100644 --- a/src/query/storages/fuse/src/operations/merge_into/mutator/matched_mutator.rs +++ b/src/query/storages/fuse/src/operations/merge_into/mutator/matched_mutator.rs @@ -144,13 +144,13 @@ impl MatchedAggregator { .meta_indexes .insert((meta_index.segment_idx, meta_index.block_idx)) { + // we can get duplicated partial unmodified blocks,this is not an error + // |----------------------------block----------------------------------------| + // |----partial-unmodified----|-----macthed------|----partial-unmodified-----| info!( "duplicated block: segment_idx: {}, block_idx: {}", meta_index.segment_idx, meta_index.block_idx ); - return Err(ErrorCode::Internal( - "merge into get duplicated block for target build unmodified optimization", - )); } } return Ok(()); From 8178714e48ebf09d7eb3bc8d4324ad04b04d47ef Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Mon, 15 Jan 2024 13:25:56 +0800 Subject: [PATCH 36/47] move logic code --- .../transforms/hash_join/hash_join_build_state.rs | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_build_state.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_build_state.rs index d3721f1a6b88..264109e1bbd6 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_build_state.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_build_state.rs @@ -196,16 +196,15 @@ impl HashJoinBuildState { .buffer_row_size .fetch_add(input_rows, Ordering::Relaxed); - let build_state = unsafe { &*self.hash_join_state.build_state.get() }; - let start_offset = build_state.generation_state.build_num_rows + old_size; - let end_offset = start_offset + input_rows - 1; - // merge into target table as build side. if self .hash_join_state .need_merge_into_target_partial_modified_scan() { assert!(input.get_meta().is_some()); + let build_state = unsafe { &*self.hash_join_state.build_state.get() }; + let start_offset = build_state.generation_state.build_num_rows + old_size; + let end_offset = start_offset + input_rows - 1; let block_meta_index = BlockMetaIndex::downcast_ref_from(input.get_meta().unwrap()).unwrap(); let row_prefix = compute_row_id_prefix( From 9131116efffba0be0cf1cdb244355bc2aa54a6a9 Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Mon, 15 Jan 2024 23:36:40 +0800 Subject: [PATCH 37/47] fix flaky matched and fix offset for pointer (chunk_offsets shouldn't minus one) --- .../hash_join/hash_join_probe_state.rs | 8 +++---- .../hash_join/probe_join/left_join.rs | 21 ++++++++++++------- 2 files changed, 17 insertions(+), 12 deletions(-) diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs index db161eb286f0..e9b9850e1dcf 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs @@ -491,14 +491,14 @@ impl HashJoinProbeState { let all_matched_blocks = block_info_index.gather_matched_all_blocks(matched); // generate chunks - info!("chunk len: {}", chunks_offsets.len()); // 100 - info!("intervals len: {} ", block_info_index.intervals.len()); // 206 + info!("chunk len: {}", chunks_offsets.len()); + info!("intervals len: {} ", block_info_index.intervals.len()); info!( - "partial unmodified blocks num: {}", // 105 + "partial unmodified blocks num: {}", partial_unmodified.len() ); info!( - "all_matched_blocks blocks num: {}", // 0 + "all_matched_blocks blocks num: {}", all_matched_blocks.len() ); let mut tasks = block_info_index.chunk_offsets(&partial_unmodified, chunks_offsets); diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/probe_join/left_join.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/probe_join/left_join.rs index 11e9e4bac56a..9868b8313034 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/probe_join/left_join.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/probe_join/left_join.rs @@ -394,19 +394,20 @@ impl HashJoinProbeState { let offset = if row_ptr.chunk_index == 0 { row_ptr.row_index as usize } else { - (chunk_offsets[(row_ptr.chunk_index - 1) as usize] - 1) as usize + chunk_offsets[(row_ptr.chunk_index - 1) as usize] as usize + row_ptr.row_index as usize }; let mut old_mactehd_counts = unsafe { (*pointer.0.add(offset)).load(Ordering::Relaxed) }; - let new_matched_count = old_mactehd_counts + 1; - if old_mactehd_counts > 0 { - return Err(ErrorCode::UnresolvableConflict( - "multi rows from source match one and the same row in the target_table multi times in probe phase", - )); - } + let mut new_matched_count = old_mactehd_counts + 1; loop { + if old_mactehd_counts > 0 { + return Err(ErrorCode::UnresolvableConflict( + "multi rows from source match one and the same row in the target_table multi times in probe phase", + )); + } + let res = unsafe { (*pointer.0.add(offset)).compare_exchange_weak( old_mactehd_counts, @@ -415,9 +416,13 @@ impl HashJoinProbeState { Ordering::SeqCst, ) }; + match res { Ok(_) => break, - Err(x) => old_mactehd_counts = x, + Err(x) => { + old_mactehd_counts = x; + new_matched_count = old_mactehd_counts + 1; + } }; } } From 0e0e4dbab9f5a6bdfb17197ad2868a08d6aca6a6 Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Tue, 16 Jan 2024 13:16:47 +0800 Subject: [PATCH 38/47] add merge_state --- .../hash_join/hash_join_build_state.rs | 22 ++++--- .../hash_join/hash_join_probe_state.rs | 7 ++- .../transforms/hash_join/hash_join_state.rs | 58 ++++++++++--------- .../hash_join/probe_join/left_join.rs | 11 ++-- .../hash_join/transform_hash_join_probe.rs | 13 +++-- 5 files changed, 67 insertions(+), 44 deletions(-) diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_build_state.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_build_state.rs index 264109e1bbd6..a58fa320c7ad 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_build_state.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_build_state.rs @@ -55,7 +55,6 @@ use databend_common_hashtable::StringRawEntry; use databend_common_hashtable::STRING_EARLY_SIZE; use databend_common_sql::plans::JoinType; use databend_common_sql::ColumnSet; -use databend_common_sql::DUMMY_TABLE_INDEX; use databend_common_storages_fuse::operations::BlockMetaIndex; use ethnum::U256; use itertools::Itertools; @@ -211,7 +210,8 @@ impl HashJoinBuildState { block_meta_index.segment_idx as u64, block_meta_index.block_idx as u64, ); - let block_info_index = unsafe { &mut *self.hash_join_state.block_info_index.get() }; + let block_info_index = + unsafe { &mut *self.hash_join_state.merge_into_state.block_info_index.get() }; block_info_index .insert_block_offsets((start_offset as u32, end_offset as u32), row_prefix); } @@ -258,8 +258,12 @@ impl HashJoinBuildState { build_state.generation_state.build_num_rows += data_block.num_rows(); build_state.generation_state.chunks.push(data_block); - if self.hash_join_state.merge_into_target_table_index != DUMMY_TABLE_INDEX { - let chunk_offsets = unsafe { &mut *self.hash_join_state.chunk_offsets.get() }; + if self + .hash_join_state + .need_merge_into_target_partial_modified_scan() + { + let chunk_offsets = + unsafe { &mut *self.hash_join_state.merge_into_state.chunk_offsets.get() }; chunk_offsets.push(build_state.generation_state.build_num_rows as u32); } } @@ -420,10 +424,14 @@ impl HashJoinBuildState { let hashtable = unsafe { &mut *self.hash_join_state.hash_table.get() }; *hashtable = hashjoin_hashtable; // generate macthed offsets memory. - if self.hash_join_state.merge_into_target_table_index != DUMMY_TABLE_INDEX { - let matched = unsafe { &mut *self.hash_join_state.matched.get() }; + if self + .hash_join_state + .need_merge_into_target_partial_modified_scan() + { + let matched = unsafe { &mut *self.hash_join_state.merge_into_state.matched.get() }; let build_state = unsafe { &*self.hash_join_state.build_state.get() }; - let atomic_pointer = unsafe { &mut *self.hash_join_state.atomic_pointer.get() }; + let atomic_pointer = + unsafe { &mut *self.hash_join_state.merge_into_state.atomic_pointer.get() }; *matched = vec![0; build_state.generation_state.build_num_rows]; let pointer = unsafe { std::mem::transmute::<*mut u8, *mut AtomicU8>(matched.as_mut_ptr()) }; diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs index e9b9850e1dcf..ebf7651c60a5 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs @@ -484,9 +484,10 @@ impl HashJoinProbeState { } pub fn generate_merge_into_final_scan_task(&self) -> Result<()> { - let block_info_index = unsafe { &*self.hash_join_state.block_info_index.get() }; - let matched = unsafe { &*self.hash_join_state.matched.get() }; - let chunks_offsets = unsafe { &*self.hash_join_state.chunk_offsets.get() }; + let block_info_index = + unsafe { &*self.hash_join_state.merge_into_state.block_info_index.get() }; + let matched = unsafe { &*self.hash_join_state.merge_into_state.matched.get() }; + let chunks_offsets = unsafe { &*self.hash_join_state.merge_into_state.chunk_offsets.get() }; let partial_unmodified = block_info_index.gather_all_partial_block_offsets(matched); let all_matched_blocks = block_info_index.gather_matched_all_blocks(matched); diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_state.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_state.rs index 09574d93c232..084e520bc156 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_state.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_state.rs @@ -80,6 +80,25 @@ pub struct MatchedPtr(pub *mut AtomicU8); unsafe impl Send for MatchedPtr {} unsafe impl Sync for MatchedPtr {} +pub struct MergeIntoState { + /// If we use target table as build side for merge into, we use this to track target table + /// and extract partial modified blocks from hashtable + pub(crate) merge_into_target_table_index: IndexType, + /// for now we don't support distributed, we will support in the next pr. + #[allow(unused)] + pub(crate) is_distributed_merge_into: bool, + + /// FOR MERGE INTO TARGET TABLE AS BUILD SIDE + /// When merge into target table as build side, we should preserve block info index. + pub(crate) block_info_index: SyncUnsafeCell, + /// we use matched to tag the matched offset in chunks. + pub(crate) matched: SyncUnsafeCell>, + /// the matched will be modified concurrently, so we use + /// atomic_pointers to pointer to matched + pub(crate) atomic_pointer: SyncUnsafeCell, + /// chunk_offsets[chunk_idx] stands for the offset of chunk_idx_th chunk in chunks. + pub(crate) chunk_offsets: SyncUnsafeCell>, +} /// Define some shared states for hash join build and probe. /// It will like a bridge to connect build and probe. /// Such as build side will pass hash table to probe side by it @@ -129,23 +148,8 @@ pub struct HashJoinState { /// If the join node generate runtime filters, the scan node will use it to do prune. pub(crate) table_index: IndexType, - /// If we use target table as build side for merge into, we use this to track target table - /// and extract partial modified blocks from hashtable - pub(crate) merge_into_target_table_index: IndexType, - /// for now we don't support distributed, we will support in the next pr. - #[allow(unused)] - pub(crate) is_distributed_merge_into: bool, - /// FOR MERGE INTO TARGET TABLE AS BUILD SIDE - /// When merge into target table as build side, we should preserve block info index. - pub(crate) block_info_index: SyncUnsafeCell, - /// we use matched to tag the matched offset in chunks. - pub(crate) matched: SyncUnsafeCell>, - /// the matched will be modified concurrently, so we use - /// atomic_pointers to pointer to matched - pub(crate) atomic_pointer: SyncUnsafeCell, - /// chunk_offsets[chunk_idx] stands for the offset of chunk_idx_th chunk in chunks. - pub(crate) chunk_offsets: SyncUnsafeCell>, + pub(crate) merge_into_state: MergeIntoState, } impl HashJoinState { @@ -193,16 +197,18 @@ impl HashJoinState { partition_id: AtomicI8::new(-2), enable_spill, table_index, - merge_into_target_table_index, - is_distributed_merge_into, - block_info_index: if merge_into_target_table_index == DUMMY_TABLE_INDEX { - SyncUnsafeCell::new(BlockInfoIndex::new_with_capacity(0)) - } else { - SyncUnsafeCell::new(Default::default()) + merge_into_state: MergeIntoState { + merge_into_target_table_index, + is_distributed_merge_into, + block_info_index: if merge_into_target_table_index == DUMMY_TABLE_INDEX { + SyncUnsafeCell::new(BlockInfoIndex::new_with_capacity(0)) + } else { + SyncUnsafeCell::new(Default::default()) + }, + matched: SyncUnsafeCell::new(Vec::new()), + atomic_pointer: SyncUnsafeCell::new(MatchedPtr(std::ptr::null_mut())), + chunk_offsets: SyncUnsafeCell::new(Vec::with_capacity(100)), }, - matched: SyncUnsafeCell::new(Vec::new()), - atomic_pointer: SyncUnsafeCell::new(MatchedPtr(std::ptr::null_mut())), - chunk_offsets: SyncUnsafeCell::new(Vec::with_capacity(100)), })) } @@ -255,7 +261,7 @@ impl HashJoinState { } pub fn need_merge_into_target_partial_modified_scan(&self) -> bool { - self.merge_into_target_table_index != DUMMY_TABLE_INDEX + self.merge_into_state.merge_into_target_table_index != DUMMY_TABLE_INDEX } pub fn set_spilled_partition(&self, partitions: &HashSet) { diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/probe_join/left_join.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/probe_join/left_join.rs index 9868b8313034..98d04e82faa2 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/probe_join/left_join.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/probe_join/left_join.rs @@ -25,7 +25,6 @@ use databend_common_expression::Scalar; use databend_common_expression::Value; use databend_common_hashtable::HashJoinHashtableLike; use databend_common_hashtable::RowPtr; -use databend_common_sql::DUMMY_TABLE_INDEX; use crate::pipelines::processors::transforms::hash_join::build_state::BuildBlockGenerationState; use crate::pipelines::processors::transforms::hash_join::common::wrap_true_validity; @@ -381,9 +380,13 @@ impl HashJoinProbeState { valids: &Bitmap, ) -> Result<()> { // merge into target table as build side. - if self.hash_join_state.merge_into_target_table_index != DUMMY_TABLE_INDEX { - let chunk_offsets = unsafe { &*self.hash_join_state.chunk_offsets.get() }; - let pointer = unsafe { &*self.hash_join_state.atomic_pointer.get() }; + if self + .hash_join_state + .need_merge_into_target_partial_modified_scan() + { + let chunk_offsets = + unsafe { &*self.hash_join_state.merge_into_state.chunk_offsets.get() }; + let pointer = unsafe { &*self.hash_join_state.merge_into_state.atomic_pointer.get() }; // add matched indexes. for (idx, row_ptr) in build_indexes[0..matched_idx].iter().enumerate() { unsafe { diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/transform_hash_join_probe.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/transform_hash_join_probe.rs index f3053cfd5e35..ecf8eecef959 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/transform_hash_join_probe.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/transform_hash_join_probe.rs @@ -24,7 +24,6 @@ use databend_common_expression::DataBlock; use databend_common_expression::FunctionContext; use databend_common_sql::optimizer::ColumnSet; use databend_common_sql::plans::JoinType; -use databend_common_sql::DUMMY_TABLE_INDEX; use databend_common_storages_fuse::operations::BlockMetaIndex; use log::info; @@ -146,7 +145,14 @@ impl TransformHashJoinProbe { return Ok(()); } - let chunks_offsets = unsafe { &*self.join_probe_state.hash_join_state.chunk_offsets.get() }; + let chunks_offsets = unsafe { + &*self + .join_probe_state + .hash_join_state + .merge_into_state + .chunk_offsets + .get() + }; let build_state = unsafe { &*self.join_probe_state.hash_join_state.build_state.get() }; let chunk_block = &build_state.generation_state.chunks[item.1 as usize]; let chunk_start = if item.1 == 0 { @@ -442,8 +448,7 @@ impl Processor for TransformHashJoinProbe { if self .join_probe_state .hash_join_state - .merge_into_target_table_index - != DUMMY_TABLE_INDEX + .need_merge_into_target_partial_modified_scan() { if let Some(item) = self .join_probe_state From 234961c110e74e29521ea8574539bdf387814fad Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Tue, 16 Jan 2024 16:02:00 +0800 Subject: [PATCH 39/47] refactor codes --- .../hash_join/hash_join_build_state.rs | 65 +++++++++++++------ .../hash_join/hash_join_probe_state.rs | 15 +++-- .../transforms/hash_join/hash_join_state.rs | 36 +++++----- .../hash_join/probe_join/left_join.rs | 15 +++-- .../hash_join/transform_hash_join_probe.rs | 20 +++--- 5 files changed, 93 insertions(+), 58 deletions(-) diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_build_state.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_build_state.rs index a58fa320c7ad..315b1ab12c9d 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_build_state.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_build_state.rs @@ -183,38 +183,49 @@ impl HashJoinBuildState { })) } - /// Add input `DataBlock` to `hash_join_state.row_space`. - pub fn build(&self, input: DataBlock) -> Result<()> { - let mut buffer = self.hash_join_state.row_space.buffer.write(); - - let input_rows = input.num_rows(); - buffer.push(input.clone()); - let old_size = self - .hash_join_state - .row_space - .buffer_row_size - .fetch_add(input_rows, Ordering::Relaxed); - + fn build_merge_into_block_info_index(&self, input: DataBlock, old_size: usize) { // merge into target table as build side. if self .hash_join_state .need_merge_into_target_partial_modified_scan() { assert!(input.get_meta().is_some()); + let merge_into_state = unsafe { + &mut *self + .hash_join_state + .merge_into_state + .as_ref() + .unwrap() + .get() + }; let build_state = unsafe { &*self.hash_join_state.build_state.get() }; let start_offset = build_state.generation_state.build_num_rows + old_size; - let end_offset = start_offset + input_rows - 1; + let end_offset = start_offset + input.num_rows() - 1; let block_meta_index = BlockMetaIndex::downcast_ref_from(input.get_meta().unwrap()).unwrap(); let row_prefix = compute_row_id_prefix( block_meta_index.segment_idx as u64, block_meta_index.block_idx as u64, ); - let block_info_index = - unsafe { &mut *self.hash_join_state.merge_into_state.block_info_index.get() }; + let block_info_index = &mut merge_into_state.block_info_index; block_info_index .insert_block_offsets((start_offset as u32, end_offset as u32), row_prefix); } + } + + /// Add input `DataBlock` to `hash_join_state.row_space`. + pub fn build(&self, input: DataBlock) -> Result<()> { + let mut buffer = self.hash_join_state.row_space.buffer.write(); + + let input_rows = input.num_rows(); + buffer.push(input.clone()); + let old_size = self + .hash_join_state + .row_space + .buffer_row_size + .fetch_add(input_rows, Ordering::Relaxed); + + self.build_merge_into_block_info_index(input.clone(), old_size); if old_size + input_rows < self.chunk_size_limit { return Ok(()); @@ -262,8 +273,15 @@ impl HashJoinBuildState { .hash_join_state .need_merge_into_target_partial_modified_scan() { - let chunk_offsets = - unsafe { &mut *self.hash_join_state.merge_into_state.chunk_offsets.get() }; + let merge_into_state = unsafe { + &mut *self + .hash_join_state + .merge_into_state + .as_ref() + .unwrap() + .get() + }; + let chunk_offsets = &mut merge_into_state.chunk_offsets; chunk_offsets.push(build_state.generation_state.build_num_rows as u32); } } @@ -428,10 +446,17 @@ impl HashJoinBuildState { .hash_join_state .need_merge_into_target_partial_modified_scan() { - let matched = unsafe { &mut *self.hash_join_state.merge_into_state.matched.get() }; + let merge_into_state = unsafe { + &mut *self + .hash_join_state + .merge_into_state + .as_ref() + .unwrap() + .get() + }; + let matched = &mut merge_into_state.matched; let build_state = unsafe { &*self.hash_join_state.build_state.get() }; - let atomic_pointer = - unsafe { &mut *self.hash_join_state.merge_into_state.atomic_pointer.get() }; + let atomic_pointer = &mut merge_into_state.atomic_pointer; *matched = vec![0; build_state.generation_state.build_num_rows]; let pointer = unsafe { std::mem::transmute::<*mut u8, *mut AtomicU8>(matched.as_mut_ptr()) }; diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs index ebf7651c60a5..f4b201e79e5f 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs @@ -484,10 +484,17 @@ impl HashJoinProbeState { } pub fn generate_merge_into_final_scan_task(&self) -> Result<()> { - let block_info_index = - unsafe { &*self.hash_join_state.merge_into_state.block_info_index.get() }; - let matched = unsafe { &*self.hash_join_state.merge_into_state.matched.get() }; - let chunks_offsets = unsafe { &*self.hash_join_state.merge_into_state.chunk_offsets.get() }; + let merge_into_state = unsafe { + &*self + .hash_join_state + .merge_into_state + .as_ref() + .unwrap() + .get() + }; + let block_info_index = &merge_into_state.block_info_index; + let matched = &merge_into_state.matched; + let chunks_offsets = &merge_into_state.chunk_offsets; let partial_unmodified = block_info_index.gather_all_partial_block_offsets(matched); let all_matched_blocks = block_info_index.gather_matched_all_blocks(matched); diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_state.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_state.rs index 084e520bc156..597cad3f81ee 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_state.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_state.rs @@ -81,23 +81,20 @@ unsafe impl Send for MatchedPtr {} unsafe impl Sync for MatchedPtr {} pub struct MergeIntoState { - /// If we use target table as build side for merge into, we use this to track target table - /// and extract partial modified blocks from hashtable - pub(crate) merge_into_target_table_index: IndexType, /// for now we don't support distributed, we will support in the next pr. #[allow(unused)] pub(crate) is_distributed_merge_into: bool, /// FOR MERGE INTO TARGET TABLE AS BUILD SIDE /// When merge into target table as build side, we should preserve block info index. - pub(crate) block_info_index: SyncUnsafeCell, + pub(crate) block_info_index: BlockInfoIndex, /// we use matched to tag the matched offset in chunks. - pub(crate) matched: SyncUnsafeCell>, + pub(crate) matched: Vec, /// the matched will be modified concurrently, so we use /// atomic_pointers to pointer to matched - pub(crate) atomic_pointer: SyncUnsafeCell, + pub(crate) atomic_pointer: MatchedPtr, /// chunk_offsets[chunk_idx] stands for the offset of chunk_idx_th chunk in chunks. - pub(crate) chunk_offsets: SyncUnsafeCell>, + pub(crate) chunk_offsets: Vec, } /// Define some shared states for hash join build and probe. /// It will like a bridge to connect build and probe. @@ -149,7 +146,7 @@ pub struct HashJoinState { /// If the join node generate runtime filters, the scan node will use it to do prune. pub(crate) table_index: IndexType, - pub(crate) merge_into_state: MergeIntoState, + pub(crate) merge_into_state: Option>, } impl HashJoinState { @@ -197,17 +194,16 @@ impl HashJoinState { partition_id: AtomicI8::new(-2), enable_spill, table_index, - merge_into_state: MergeIntoState { - merge_into_target_table_index, - is_distributed_merge_into, - block_info_index: if merge_into_target_table_index == DUMMY_TABLE_INDEX { - SyncUnsafeCell::new(BlockInfoIndex::new_with_capacity(0)) - } else { - SyncUnsafeCell::new(Default::default()) - }, - matched: SyncUnsafeCell::new(Vec::new()), - atomic_pointer: SyncUnsafeCell::new(MatchedPtr(std::ptr::null_mut())), - chunk_offsets: SyncUnsafeCell::new(Vec::with_capacity(100)), + merge_into_state: if merge_into_target_table_index != DUMMY_TABLE_INDEX { + Some(SyncUnsafeCell::new(MergeIntoState { + is_distributed_merge_into, + block_info_index: Default::default(), + matched: Vec::new(), + atomic_pointer: MatchedPtr(std::ptr::null_mut()), + chunk_offsets: Vec::with_capacity(100), + })) + } else { + None }, })) } @@ -261,7 +257,7 @@ impl HashJoinState { } pub fn need_merge_into_target_partial_modified_scan(&self) -> bool { - self.merge_into_state.merge_into_target_table_index != DUMMY_TABLE_INDEX + self.merge_into_state.is_some() } pub fn set_spilled_partition(&self, partitions: &HashSet) { diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/probe_join/left_join.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/probe_join/left_join.rs index 98d04e82faa2..6f889f4d3af0 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/probe_join/left_join.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/probe_join/left_join.rs @@ -372,7 +372,6 @@ impl HashJoinProbeState { } #[inline] - #[allow(clippy::too_many_arguments)] fn check_and_set_matched( &self, build_indexes: &[RowPtr], @@ -384,9 +383,17 @@ impl HashJoinProbeState { .hash_join_state .need_merge_into_target_partial_modified_scan() { - let chunk_offsets = - unsafe { &*self.hash_join_state.merge_into_state.chunk_offsets.get() }; - let pointer = unsafe { &*self.hash_join_state.merge_into_state.atomic_pointer.get() }; + let merge_into_state = unsafe { + &*self + .hash_join_state + .merge_into_state + .as_ref() + .unwrap() + .get() + }; + let chunk_offsets = &merge_into_state.chunk_offsets; + + let pointer = &merge_into_state.atomic_pointer; // add matched indexes. for (idx, row_ptr) in build_indexes[0..matched_idx].iter().enumerate() { unsafe { diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/transform_hash_join_probe.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/transform_hash_join_probe.rs index ecf8eecef959..a66edf500dfe 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/transform_hash_join_probe.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/transform_hash_join_probe.rs @@ -144,15 +144,16 @@ impl TransformHashJoinProbe { self.output_data_blocks.push_back(data_block); return Ok(()); } - - let chunks_offsets = unsafe { + let merge_into_state = unsafe { &*self .join_probe_state .hash_join_state .merge_into_state - .chunk_offsets + .as_ref() + .unwrap() .get() }; + let chunks_offsets = &merge_into_state.chunk_offsets; let build_state = unsafe { &*self.join_probe_state.hash_join_state.build_state.get() }; let chunk_block = &build_state.generation_state.chunks[item.1 as usize]; let chunk_start = if item.1 == 0 { @@ -161,14 +162,13 @@ impl TransformHashJoinProbe { chunks_offsets[(item.1 - 1) as usize] }; for (interval, prefix) in item.0 { - let indices = ((interval.0 - chunk_start)..=(interval.1 - chunk_start)) - .collect::>() - .chunks(self.max_block_size) - .map(|chunk| chunk.to_vec()) - .collect::>>(); - for range in indices.iter() { + for start in ((interval.0 - chunk_start)..=(interval.1 - chunk_start)) + .step_by(self.max_block_size) + { + let end = (interval.1 - chunk_start).min(start + self.max_block_size as u32 - 1); + let range = (start..=end).collect::>(); let data_block = chunk_block.take( - range, + &range, &mut self.probe_state.generation_state.string_items_buf, )?; assert!(!data_block.is_empty()); From 70d42c94dd07300e1206897401dfca45901e5c59 Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Tue, 16 Jan 2024 21:21:35 +0800 Subject: [PATCH 40/47] add more commnets --- src/common/hashtable/src/traits.rs | 4 +--- .../src/interpreters/interpreter_merge_into.rs | 12 +++++++++--- 2 files changed, 10 insertions(+), 6 deletions(-) diff --git a/src/common/hashtable/src/traits.rs b/src/common/hashtable/src/traits.rs index 265c4beb153d..f49ad9472b8b 100644 --- a/src/common/hashtable/src/traits.rs +++ b/src/common/hashtable/src/traits.rs @@ -13,9 +13,8 @@ // limitations under the License. // To avoid RUSTFLAGS="-C target-feature=+sse4.2" warning. -#![allow(unused_imports)] + use std::hash::BuildHasher; -use std::hash::Hasher; use std::iter::TrustedLen; use std::mem::MaybeUninit; use std::num::NonZeroU64; @@ -25,7 +24,6 @@ use ethnum::i256; use ethnum::U256; use ordered_float::OrderedFloat; -use crate::utils::Interval; use crate::RowPtr; /// # Safety diff --git a/src/query/service/src/interpreters/interpreter_merge_into.rs b/src/query/service/src/interpreters/interpreter_merge_into.rs index 3d24f8f231df..930a24c5cdc5 100644 --- a/src/query/service/src/interpreters/interpreter_merge_into.rs +++ b/src/query/service/src/interpreters/interpreter_merge_into.rs @@ -163,11 +163,11 @@ impl MergeIntoInterpreter { // important flag: // I. change join order: if true, target table as build side, if false, source as build side. // II. distributed: this merge into is executed at a distributed stargety. - // 2.1 Left: there are macthed and not macthed, and change join order is false. + // 2.1 Left: there are macthed and not macthed, and change join order is true. // 2.2 Left Anti: change join order is true, but it's insert-only. // 2.3 Inner: this is matched only case. - // 2.3.1 change join order is true, - // 2.3.2 change join order is false. + // 2.3.1 change join order is true, target table as build side,it's matched-only. + // 2.3.2 change join order is false, source data as build side,it's matched-only. // 2.4 Right: change join order is false, there are macthed and not macthed // 2.5 Right Anti: change join order is false, but it's insert-only. // distributed execution stargeties: @@ -175,6 +175,10 @@ impl MergeIntoInterpreter { // II. change join order is false and match_pattern and not enable spill, we use right outer join with rownumber distributed strategies. // III otherwise, use `merge_into_join_sexpr` as standalone execution(so if change join order is false,but doesn't match_pattern, we don't support distributed,in fact. case I // can take this at most time, if that's a hash shuffle, the I can take it. We think source is always very small). + + // for `target_build_optimization` we don't need to read rowId column. for now, there are two cases we don't read rowid: + // I. InsertOnly, the MergeIntoType is InsertOnly + // II. target build optimization for this pr. the MergeIntoType is MergeIntoType let mut target_build_optimization = matches!(self.plan.merge_type, MergeIntoType::FullOperation) && !self.plan.columns_set.contains(&self.plan.row_id_index); @@ -255,6 +259,8 @@ impl MergeIntoInterpreter { } } + // we use `merge_into_split_idx` to specify a column from target table to spilt a block + // from join into macthed part and unmacthed part. let mut merge_into_split_idx = DUMMY_COLUMN_INDEX; if matches!(merge_type, MergeIntoType::FullOperation) { for (idx, data_field) in join_output_schema.fields().iter().enumerate() { From 0711dc7fb93d6963882ea1aad3e8cd374cb6aba1 Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Tue, 16 Jan 2024 23:17:29 +0800 Subject: [PATCH 41/47] refactor codes, split merge into optimziation codes into other files --- a.txt | 7 + .../src/pipelines/builders/builder_join.rs | 14 +- .../builders/merge_into_join_optimizations.rs | 40 ++ .../service/src/pipelines/builders/mod.rs | 2 +- .../hash_join/hash_join_build_state.rs | 74 +--- .../hash_join/hash_join_probe_state.rs | 54 +-- .../transforms/hash_join/hash_join_state.rs | 44 +-- .../merge_into_hash_join_optimization.rs | 350 ++++++++++++++++++ .../processors/transforms/hash_join/mod.rs | 2 +- .../hash_join/probe_join/left_join.rs | 70 ---- .../hash_join/transform_hash_join_probe.rs | 83 +---- 11 files changed, 421 insertions(+), 319 deletions(-) create mode 100644 a.txt create mode 100644 src/query/service/src/pipelines/builders/merge_into_join_optimizations.rs create mode 100644 src/query/service/src/pipelines/processors/transforms/hash_join/merge_into_hash_join_optimization.rs diff --git a/a.txt b/a.txt new file mode 100644 index 000000000000..55a507fac890 --- /dev/null +++ b/a.txt @@ -0,0 +1,7 @@ +intervals_string: [(0,237533) (237534,811068) (811069,1165420) (1165421,1738750) (1738751,2312789) (2312790,2887196) (2887197,3460092) (3460093,4034056) (4034057,4607072) (4607073,5181204) (5181205,5754438) (5754439,6327660) (6327661,6900860) (6900861,7474879) (7474880,8048824) (8048825,8622265) (8622266,9212089) (9212090,9785116) (9785117,10359017) (10359018,10948841) (10948842,11521663) (11521664,12094634) (12094635,12668909) (12668910,13242207) (13242208,13815000) (13815001,14388437) (14388438,14961724) (14961725,15535530) (15535531,16109636) (16109637,16682669) (16682670,17256957) (17256958,17830626) (17830627,18403272) (18403273,18977533) (18977534,19551437) (19551438,20125106) (20125107,20698854) (20698855,21272069) (21272070,21845219) (21845220,22419288) (22419289,22561286) (22561287,23135682) (23135683,23709204) (23709205,24282399) (24282400,24855599) (24855600,25430163) (25430164,26003400) (26003401,26577381) (26577382,27151897) (27151898,27724679) (27724680,28269351) (28269352,28842727) (28842728,29417305) (29417306,29991576) (29991577,30214112) (30214113,30787411) (30787412,31360395) (31360396,31950219) (31950220,32523734) (32523735,33097407) (33097408,33671105) (33671106,33972279) (33972280,34546266) (34546267,35119282) (35119283,35692524) (35692525,36266016) (36266017,36839196) (36839197,37044870) (37044871,37617683) (37617684,38191653) (38191654,38765968) (38765969,39338780) (39338781,39912197) (39912198,40487432) (40487433,41060652) (41060653,41634348) (41634349,42224172) (42224173,42797938) (42797939,43371720) (43371721,43945281) (43945282,44519037) (44519038,45092484) (45092485,45666675) (45666676,46256499) (46256500,46830243) (46830244,47403729) (47403730,47978550) (47978551,48551303) (48551304,49124574) (49124575,49697844) (49697845,50271914) (50271915,50845216) (50845217,51418598) (51418599,51991349) (51991350,52565143) (52565144,53139411) (53139412,53712648) (53712649,54286724) (54286725,54859983) (54859984,55433700) (55433701,56007050) (56007051,56596874) (56596875,57170024) (57170025,57744711) (57744712,58317999) (58318000,58892731) (58892732,59466281) (59466282,60040707) (60040708,60614999) (60615000,61188189) (61188190,61761456) (61761457,62336197) (62336198,62909662) (62909663,63483120) (63483121,64055843) (64055844,64629576) (64629577,65202830) (65202831,65776524) (65776525,66350998) (66350999,66924565) (66924566,67281639) (67281640,67855929) (67855930,68429199) (68429200,69002918) (69002919,69576444) (69576445,70150256) (70150257,70724219) (70724220,71298505) (71298506,71873951) (71873952,72447446) (72447447,73020448) (73020449,73594288) (73594289,74169041) (74169042,74742398) (74742399,75316427) (75316428,75889506) (75889507,76462744) (76462745,77038390) (77038391,77611612) (77611613,78185851) (78185852,78760666) (78760667,79334714) (79334715,79907979) (79907980,80481938) (80481939,81054792) (81054793,81628150) (81628151,82201928) (82201929,82764043) (82764044,83337806) (83337807,83911308) (83911309,84485024) (84485025,85059685) (85059686,85633227) (85633228,86206692) (86206693,86779426) (86779427,87353195) (87353196,87927979) (87927980,88501912) (88501913,89075177) (89075178,89649118) (89649119,90223331) (90223332,90798077) (90798078,91371589) (91371590,91945390) (91945391,92519248) (92519249,93093438) (93093439,93667712) (93667713,94242440) (94242441,94815905) (94815906,95390195) (95390196,95963663) (95963664,96537859) (96537860,97112484) (97112485,97686050) (97686051,98259326) (98259327,98832637) (98832638,99406186) (99406187,99979479) (99979480,100552833) (100552834,101127605) (101127606,101701375) (101701376,102274143) (102274144,102847640) (102847641,103421011) (103421012,103993814) (103993815,104567211) (104567212,105140770) (105140771,105714775) (105714776,106289053) (106289054,106862621) (106862622,107435824) (107435825,108009383) (108009384,108582932) (108582933,109156905) (109156906,109730400) (109730401,110304291) (110304292,110878350) (110878351,111451744) (111451745,112024456) (112024457,112598573) (112598574,113172848) (113172849,113747214) (113747215,114320708) (114320709,114894482) (114894483,115468021) (115468022,116041280) ] +chunks_offsets_string: [1165421,2312790,3460093,4607073,5754439,6900861,8048825,9212090,10359018,11521664,12668910,13815001,14961725,16109637,17256958,18403273,19551438,20698855,21845220,23135683,24282400,25430164,26577382,27724680,28842728,29991577,31360396,32523735,33671106,35119283,36266017,37617684,38765969,39912198,41060653,42224173,43371721,44519038,45666676,46830244,47978551,49124575,50271915,51418599,52565144,53712649,54859984,56007051,57170025,58318000,59466282,60615000,61761457,62909663,64055844,65202831,66350999,67855930,69002919,70150257,71298506,72447447,73594289,74742399,75889507,77038391,78185852,79334715,80481939,81628151,82764044,83911309,85059686,86206693,87353196,88501913,89649119,90798078,91945391,93093439,94242441,95390196,96537860,97686051,98832638,99979480,101127606,102274144,103421012,104567212,105714776,106862622,108009384,109156906,110304292,111451745,112598574,113747215,114894483,116041281] +partial_unmodified_string: [(3460093,4034055),8589875200 (4549765,4607072),8589930496 (7474880,8048823),8589914112 (8564920,8622265),8589750272 (10359018,10948840),8589643777 (11464379,11521663),8589608960 (11521664,12094633),8589834240 (12611477,12668909),8589887488 (13815001,14388436),8589932544 (14904389,14961724),8589660160 (18977534,19551436),8589854720 (20067731,20125106),8589871104 (20125107,20698853),8589799424 (21214743,21272069),8589815808 (21845220,22419287),8589758464 (22488102,22561285),8589533184 (23078237,23135682),8589563904 (23135683,23709203),8589666304 (24225075,24282399),8589862912 (24855600,25430162),8589893632 (25946070,26003400),8589535232 (27724680,28269350),8589856768 (28785384,28842726),8589873152 (29359840,29417304),8589801472 (29934148,29991575),8589817856 (30132876,30214111),8589668352 (30730073,30787410),8589557760 (31303093,31360394),8589760512 (32466378,32523733),8589565952 (33040037,33097407),8589864960 (33972280,34546265),8589895680 (35061975,35119282),8589537280 (36839197,37044869),8589844480 (37560399,37617683),8589580288 (37617684,38191652),8589840384 (38708532,38765968),8589602816 (38765969,39338779),8589676544 (39854849,39912197),8589824000 (43371721,43945280),8589725696 (44461656,44519037),8589836288 (44519038,45092483),8589531136 (45609255,45666675),8589647872 (46830244,47403728),8589842432 (47921067,47978549),8589604864 (48494025,48551302),8589678592 (49067246,49124574),8589731840 (49124575,49697843),8589826048 (50214507,50271914),8589694976 (52565144,53139410),8589727744 (53655318,53712647),8589611008 (54229311,54286724),8589838336 (55433701,56007049),8589770752 (56596874,56596874),8589668353 (60040708,60614998),8589746176 (61130870,61188188),8589719552 (61704123,61761455),8589524992 (62278722,62336196),8589819904 (62852311,62909661),8589613056 (63425767,63483120),8589701120 (63483121,64055842),8589674496 (64572200,64629576),8589740032 (65202831,65776523),8589772800 (66293547,66350998),8589617152 (66924566,67281638),8589574144 (67798500,67855929),8589903872 (70150257,70724218),8589527040 (71241071,71298505),8589821952 (71873952,72447445),8589703168 (72963146,73020447),8589623296 (73536904,73594288),8589709312 (75316428,75889505),8589697024 (76405413,76462744),8589897728 (76462745,77038389),8589742080 (77554288,77611612),8589762560 (86206693,86779425),8589764608 (87295810,87353195),8589629440 (87353196,87927978),8589907968 (88444516,88501912),8589778944 (88501913,89075176),8589651968 (89591723,89649118),8589850624 (93093439,93667711),8589910016 (94184960,94242439),8589635584 (94758554,94815905),8589791232 (97112485,97686049),8589780992 (98201993,98259326),8589713408 (100552834,101127604),8589928448 (101643998,101701375),8589891584 (102274144,102847639),8589912064 (103363673,103421011),8589637632 (104567212,105140769),8589860864 (105657370,105714775),8589682688 (107435825,108009382),8589920256 (108525569,108582931),8589598720 (109099505,109156905),8589922304 (109156906,109730399),8589883392 (110246901,110304291),8589721600 (112598574,113172847),8589600768 (113689772,113747214),8589924352 (113747215,114320707),8589723648 (114837101,114894481),8589848576 (115410659,115468021),8589830144 ] + +error: error happens after fetched 0 rows: APIError: ResponseError with 1104: assertion `left == right` failed + left: 4 + right: 1 \ No newline at end of file diff --git a/src/query/service/src/pipelines/builders/builder_join.rs b/src/query/service/src/pipelines/builders/builder_join.rs index 544fa5bcbe7c..2933ccf4cd6a 100644 --- a/src/query/service/src/pipelines/builders/builder_join.rs +++ b/src/query/service/src/pipelines/builders/builder_join.rs @@ -27,8 +27,6 @@ use databend_common_sql::executor::physical_plans::RangeJoin; use databend_common_sql::executor::PhysicalPlan; use databend_common_sql::ColumnBinding; use databend_common_sql::IndexType; -use databend_common_sql::DUMMY_TABLE_INDEX; -use databend_common_storages_fuse::operations::need_reserve_block_info; use crate::pipelines::processors::transforms::range_join::RangeJoinState; use crate::pipelines::processors::transforms::range_join::TransformRangeJoinLeft; @@ -129,17 +127,7 @@ impl PipelineBuilder { let id = join.probe.get_table_index(); // for merge into target table as build side. let (build_table_index, is_distributed_merge_into) = - if matches!(&*join.build, PhysicalPlan::TableScan(_)) { - let (need_block_info, is_distributed) = - need_reserve_block_info(self.ctx.clone(), join.build.get_table_index()); - if need_block_info { - (join.build.get_table_index(), is_distributed) - } else { - (DUMMY_TABLE_INDEX, false) - } - } else { - (DUMMY_TABLE_INDEX, false) - }; + self.get_merge_into_optimization_flag(join); let state = self.build_join_state(join, id, build_table_index, is_distributed_merge_into)?; diff --git a/src/query/service/src/pipelines/builders/merge_into_join_optimizations.rs b/src/query/service/src/pipelines/builders/merge_into_join_optimizations.rs new file mode 100644 index 000000000000..eeb06a4f9b3b --- /dev/null +++ b/src/query/service/src/pipelines/builders/merge_into_join_optimizations.rs @@ -0,0 +1,40 @@ +// 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 databend_common_sql::executor::physical_plans::HashJoin; +use databend_common_sql::executor::PhysicalPlan; +use databend_common_sql::IndexType; +use databend_common_sql::DUMMY_TABLE_INDEX; +use databend_common_storages_fuse::operations::need_reserve_block_info; + +use crate::pipelines::PipelineBuilder; + +impl PipelineBuilder { + pub(crate) fn get_merge_into_optimization_flag(&self, join: &HashJoin) -> (IndexType, bool) { + // for merge into target table as build side. + let (build_table_index, is_distributed_merge_into) = + if matches!(&*join.build, PhysicalPlan::TableScan(_)) { + let (need_block_info, is_distributed) = + need_reserve_block_info(self.ctx.clone(), join.build.get_table_index()); + if need_block_info { + (join.build.get_table_index(), is_distributed) + } else { + (DUMMY_TABLE_INDEX, false) + } + } else { + (DUMMY_TABLE_INDEX, false) + }; + (build_table_index, is_distributed_merge_into) + } +} diff --git a/src/query/service/src/pipelines/builders/mod.rs b/src/query/service/src/pipelines/builders/mod.rs index 0677001b9d45..1d7a709aff9d 100644 --- a/src/query/service/src/pipelines/builders/mod.rs +++ b/src/query/service/src/pipelines/builders/mod.rs @@ -11,7 +11,6 @@ // 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. - mod builder_aggregate; mod builder_append_table; mod builder_commit; @@ -37,6 +36,7 @@ mod builder_udf; mod builder_union_all; mod builder_update; mod builder_window; +mod merge_into_join_optimizations; pub use builder_replace_into::ValueSource; pub use builder_sort::SortPipelineBuilder; diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_build_state.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_build_state.rs index 315b1ab12c9d..6216e01baa09 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_build_state.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_build_state.rs @@ -22,7 +22,6 @@ use std::sync::Arc; use databend_common_arrow::arrow::bitmap::Bitmap; use databend_common_base::base::tokio::sync::Barrier; -use databend_common_catalog::plan::compute_row_id_prefix; use databend_common_catalog::runtime_filter_info::RuntimeFilterInfo; use databend_common_catalog::table_context::TableContext; use databend_common_exception::ErrorCode; @@ -30,7 +29,6 @@ use databend_common_exception::Result; use databend_common_expression::arrow::and_validities; use databend_common_expression::types::DataType; use databend_common_expression::types::NumberDomain; -use databend_common_expression::BlockMetaInfoDowncast; use databend_common_expression::Column; use databend_common_expression::ColumnBuilder; use databend_common_expression::ColumnVec; @@ -55,7 +53,6 @@ use databend_common_hashtable::StringRawEntry; use databend_common_hashtable::STRING_EARLY_SIZE; use databend_common_sql::plans::JoinType; use databend_common_sql::ColumnSet; -use databend_common_storages_fuse::operations::BlockMetaIndex; use ethnum::U256; use itertools::Itertools; use log::info; @@ -63,7 +60,6 @@ use parking_lot::Mutex; use parking_lot::RwLock; use xorf::BinaryFuse16; -use super::MatchedPtr; use crate::pipelines::processors::transforms::hash_join::common::wrap_true_validity; use crate::pipelines::processors::transforms::hash_join::desc::MARKER_KIND_FALSE; use crate::pipelines::processors::transforms::hash_join::util::dedup_build_key_column; @@ -183,36 +179,6 @@ impl HashJoinBuildState { })) } - fn build_merge_into_block_info_index(&self, input: DataBlock, old_size: usize) { - // merge into target table as build side. - if self - .hash_join_state - .need_merge_into_target_partial_modified_scan() - { - assert!(input.get_meta().is_some()); - let merge_into_state = unsafe { - &mut *self - .hash_join_state - .merge_into_state - .as_ref() - .unwrap() - .get() - }; - let build_state = unsafe { &*self.hash_join_state.build_state.get() }; - let start_offset = build_state.generation_state.build_num_rows + old_size; - let end_offset = start_offset + input.num_rows() - 1; - let block_meta_index = - BlockMetaIndex::downcast_ref_from(input.get_meta().unwrap()).unwrap(); - let row_prefix = compute_row_id_prefix( - block_meta_index.segment_idx as u64, - block_meta_index.block_idx as u64, - ); - let block_info_index = &mut merge_into_state.block_info_index; - block_info_index - .insert_block_offsets((start_offset as u32, end_offset as u32), row_prefix); - } - } - /// Add input `DataBlock` to `hash_join_state.row_space`. pub fn build(&self, input: DataBlock) -> Result<()> { let mut buffer = self.hash_join_state.row_space.buffer.write(); @@ -225,7 +191,7 @@ impl HashJoinBuildState { .buffer_row_size .fetch_add(input_rows, Ordering::Relaxed); - self.build_merge_into_block_info_index(input.clone(), old_size); + self.merge_into_try_build_merge_into_block_info_index(input.clone(), old_size); if old_size + input_rows < self.chunk_size_limit { return Ok(()); @@ -269,21 +235,7 @@ impl HashJoinBuildState { build_state.generation_state.build_num_rows += data_block.num_rows(); build_state.generation_state.chunks.push(data_block); - if self - .hash_join_state - .need_merge_into_target_partial_modified_scan() - { - let merge_into_state = unsafe { - &mut *self - .hash_join_state - .merge_into_state - .as_ref() - .unwrap() - .get() - }; - let chunk_offsets = &mut merge_into_state.chunk_offsets; - chunk_offsets.push(build_state.generation_state.build_num_rows as u32); - } + self.merge_into_try_add_chunk_offset(build_state); } Ok(()) } @@ -441,27 +393,7 @@ impl HashJoinBuildState { }; let hashtable = unsafe { &mut *self.hash_join_state.hash_table.get() }; *hashtable = hashjoin_hashtable; - // generate macthed offsets memory. - if self - .hash_join_state - .need_merge_into_target_partial_modified_scan() - { - let merge_into_state = unsafe { - &mut *self - .hash_join_state - .merge_into_state - .as_ref() - .unwrap() - .get() - }; - let matched = &mut merge_into_state.matched; - let build_state = unsafe { &*self.hash_join_state.build_state.get() }; - let atomic_pointer = &mut merge_into_state.atomic_pointer; - *matched = vec![0; build_state.generation_state.build_num_rows]; - let pointer = - unsafe { std::mem::transmute::<*mut u8, *mut AtomicU8>(matched.as_mut_ptr()) }; - *atomic_pointer = MatchedPtr(pointer); - } + self.merge_into_try_generate_matched_memory(); } Ok(()) } diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs index f4b201e79e5f..dfa86f8db66d 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs @@ -57,7 +57,7 @@ use crate::pipelines::processors::HashJoinState; use crate::sessions::QueryContext; use crate::sql::planner::plans::JoinType; -pub type ChunkPartialUnmodified = (Vec<(Interval, u64)>, u64); +pub type MergeIntoChunkPartialUnmodified = (Vec<(Interval, u64)>, u64); /// Define some shared states for all hash join probe threads. pub struct HashJoinProbeState { pub(crate) ctx: Arc, @@ -84,7 +84,7 @@ pub struct HashJoinProbeState { pub(crate) final_scan_tasks: RwLock>, /// for merge into target as build side. pub(crate) final_merge_into_partial_unmodified_scan_tasks: - RwLock>, + RwLock>, pub(crate) mark_scan_map_lock: Mutex<()>, /// Hash method pub(crate) hash_method: HashMethodKind, @@ -433,15 +433,6 @@ impl HashJoinProbeState { Ok(()) } - pub fn probe_merge_into_partial_modified_done(&self) -> Result<()> { - let old_count = self.probe_workers.fetch_sub(1, Ordering::Relaxed); - if old_count == 1 { - // Divide the final scan phase into multiple tasks. - self.generate_merge_into_final_scan_task()?; - } - Ok(()) - } - pub fn finish_spill(&self) -> Result<()> { self.final_probe_workers.fetch_sub(1, Ordering::Relaxed); let old_count = self.spill_workers.fetch_sub(1, Ordering::Relaxed); @@ -483,52 +474,11 @@ impl HashJoinProbeState { Ok(()) } - pub fn generate_merge_into_final_scan_task(&self) -> Result<()> { - let merge_into_state = unsafe { - &*self - .hash_join_state - .merge_into_state - .as_ref() - .unwrap() - .get() - }; - let block_info_index = &merge_into_state.block_info_index; - let matched = &merge_into_state.matched; - let chunks_offsets = &merge_into_state.chunk_offsets; - let partial_unmodified = block_info_index.gather_all_partial_block_offsets(matched); - let all_matched_blocks = block_info_index.gather_matched_all_blocks(matched); - - // generate chunks - info!("chunk len: {}", chunks_offsets.len()); - info!("intervals len: {} ", block_info_index.intervals.len()); - info!( - "partial unmodified blocks num: {}", - partial_unmodified.len() - ); - info!( - "all_matched_blocks blocks num: {}", - all_matched_blocks.len() - ); - let mut tasks = block_info_index.chunk_offsets(&partial_unmodified, chunks_offsets); - info!("partial unmodified chunk num: {}", tasks.len()); - for prefix in all_matched_blocks { - // deleted block - tasks.push((Vec::new(), prefix)); - } - *self.final_merge_into_partial_unmodified_scan_tasks.write() = tasks.into(); - Ok(()) - } - pub fn final_scan_task(&self) -> Option { let mut tasks = self.final_scan_tasks.write(); tasks.pop_front() } - pub fn final_merge_into_partial_unmodified_scan_task(&self) -> Option { - let mut tasks = self.final_merge_into_partial_unmodified_scan_tasks.write(); - tasks.pop_front() - } - pub fn final_scan(&self, task: usize, state: &mut ProbeState) -> Result> { match &self.hash_join_state.hash_join_desc.join_type { JoinType::Right | JoinType::RightSingle | JoinType::Full => { diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_state.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_state.rs index 597cad3f81ee..fa8969336f44 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_state.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_state.rs @@ -16,7 +16,6 @@ use std::cell::SyncUnsafeCell; use std::collections::HashSet; use std::sync::atomic::AtomicBool; use std::sync::atomic::AtomicI8; -use std::sync::atomic::AtomicU8; use std::sync::atomic::AtomicUsize; use std::sync::atomic::Ordering; use std::sync::Arc; @@ -32,16 +31,15 @@ use databend_common_expression::HashMethodFixedKeys; use databend_common_expression::HashMethodSerializer; use databend_common_expression::HashMethodSingleBinary; use databend_common_hashtable::BinaryHashJoinHashMap; -use databend_common_hashtable::BlockInfoIndex; use databend_common_hashtable::HashJoinHashMap; use databend_common_hashtable::HashtableKeyable; use databend_common_sql::plans::JoinType; use databend_common_sql::ColumnSet; use databend_common_sql::IndexType; -use databend_common_sql::DUMMY_TABLE_INDEX; use ethnum::U256; use parking_lot::RwLock; +use super::merge_into_hash_join_optimization::MergeIntoState; use crate::pipelines::processors::transforms::hash_join::build_state::BuildState; use crate::pipelines::processors::transforms::hash_join::row::RowSpace; use crate::pipelines::processors::transforms::hash_join::util::build_schema_wrap_nullable; @@ -75,27 +73,6 @@ pub enum HashJoinHashTable { KeysU256(FixedKeyHashJoinHashTable), } -pub struct MatchedPtr(pub *mut AtomicU8); - -unsafe impl Send for MatchedPtr {} -unsafe impl Sync for MatchedPtr {} - -pub struct MergeIntoState { - /// for now we don't support distributed, we will support in the next pr. - #[allow(unused)] - pub(crate) is_distributed_merge_into: bool, - - /// FOR MERGE INTO TARGET TABLE AS BUILD SIDE - /// When merge into target table as build side, we should preserve block info index. - pub(crate) block_info_index: BlockInfoIndex, - /// we use matched to tag the matched offset in chunks. - pub(crate) matched: Vec, - /// the matched will be modified concurrently, so we use - /// atomic_pointers to pointer to matched - pub(crate) atomic_pointer: MatchedPtr, - /// chunk_offsets[chunk_idx] stands for the offset of chunk_idx_th chunk in chunks. - pub(crate) chunk_offsets: Vec, -} /// Define some shared states for hash join build and probe. /// It will like a bridge to connect build and probe. /// Such as build side will pass hash table to probe side by it @@ -194,17 +171,10 @@ impl HashJoinState { partition_id: AtomicI8::new(-2), enable_spill, table_index, - merge_into_state: if merge_into_target_table_index != DUMMY_TABLE_INDEX { - Some(SyncUnsafeCell::new(MergeIntoState { - is_distributed_merge_into, - block_info_index: Default::default(), - matched: Vec::new(), - atomic_pointer: MatchedPtr(std::ptr::null_mut()), - chunk_offsets: Vec::with_capacity(100), - })) - } else { - None - }, + merge_into_state: MergeIntoState::try_create_merge_into_state( + merge_into_target_table_index, + is_distributed_merge_into, + ), })) } @@ -256,10 +226,6 @@ impl HashJoinState { matches!(self.hash_join_desc.join_type, JoinType::LeftMark) } - pub fn need_merge_into_target_partial_modified_scan(&self) -> bool { - self.merge_into_state.is_some() - } - pub fn set_spilled_partition(&self, partitions: &HashSet) { let mut spill_partition = self.build_spilled_partitions.write(); spill_partition.extend(partitions); diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/merge_into_hash_join_optimization.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/merge_into_hash_join_optimization.rs new file mode 100644 index 000000000000..4f7cdf763df5 --- /dev/null +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/merge_into_hash_join_optimization.rs @@ -0,0 +1,350 @@ +use std::cell::SyncUnsafeCell; +// 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::atomic::AtomicU8; +use std::sync::atomic::Ordering; + +use databend_common_arrow::arrow::bitmap::Bitmap; +use databend_common_catalog::plan::compute_row_id_prefix; +use databend_common_catalog::plan::split_prefix; +use databend_common_exception::ErrorCode; +use databend_common_exception::Result; +use databend_common_expression::BlockMetaInfoDowncast; +use databend_common_expression::DataBlock; +use databend_common_hashtable::BlockInfoIndex; +use databend_common_hashtable::RowPtr; +use databend_common_sql::IndexType; +use databend_common_sql::DUMMY_TABLE_INDEX; +use databend_common_storages_fuse::operations::BlockMetaIndex; +use log::info; + +use super::build_state::BuildState; +use super::hash_join_probe_state::MergeIntoChunkPartialUnmodified; +use super::HashJoinBuildState; +use super::HashJoinProbeState; +use super::HashJoinState; +use super::TransformHashJoinProbe; +pub struct MatchedPtr(pub *mut AtomicU8); + +unsafe impl Send for MatchedPtr {} +unsafe impl Sync for MatchedPtr {} + +pub struct MergeIntoState { + /// for now we don't support distributed, we will support in the next pr. + #[allow(unused)] + pub(crate) is_distributed_merge_into: bool, + + /// FOR MERGE INTO TARGET TABLE AS BUILD SIDE + /// When merge into target table as build side, we should preserve block info index. + pub(crate) block_info_index: BlockInfoIndex, + /// we use matched to tag the matched offset in chunks. + pub(crate) matched: Vec, + /// the matched will be modified concurrently, so we use + /// atomic_pointers to pointer to matched + pub(crate) atomic_pointer: MatchedPtr, + /// chunk_offsets[chunk_idx] stands for the offset of chunk_idx_th chunk in chunks. + pub(crate) chunk_offsets: Vec, +} + +impl MergeIntoState { + pub(crate) fn try_create_merge_into_state( + merge_into_target_table_index: IndexType, + is_distributed_merge_into: bool, + ) -> Option> { + if merge_into_target_table_index != DUMMY_TABLE_INDEX { + Some(SyncUnsafeCell::new(MergeIntoState { + is_distributed_merge_into, + block_info_index: Default::default(), + matched: Vec::new(), + atomic_pointer: MatchedPtr(std::ptr::null_mut()), + chunk_offsets: Vec::with_capacity(100), + })) + } else { + None + } + } +} + +impl HashJoinBuildState { + pub(crate) fn merge_into_try_build_merge_into_block_info_index( + &self, + input: DataBlock, + old_size: usize, + ) { + // merge into target table as build side. + if self + .hash_join_state + .need_merge_into_target_partial_modified_scan() + { + assert!(input.get_meta().is_some()); + let merge_into_state = unsafe { + &mut *self + .hash_join_state + .merge_into_state + .as_ref() + .unwrap() + .get() + }; + let build_state = unsafe { &*self.hash_join_state.build_state.get() }; + let start_offset = build_state.generation_state.build_num_rows + old_size; + let end_offset = start_offset + input.num_rows() - 1; + let block_meta_index = + BlockMetaIndex::downcast_ref_from(input.get_meta().unwrap()).unwrap(); + let row_prefix = compute_row_id_prefix( + block_meta_index.segment_idx as u64, + block_meta_index.block_idx as u64, + ); + let block_info_index = &mut merge_into_state.block_info_index; + block_info_index + .insert_block_offsets((start_offset as u32, end_offset as u32), row_prefix); + } + } + + pub(crate) fn merge_into_try_add_chunk_offset(&self, build_state: &mut BuildState) { + if self + .hash_join_state + .need_merge_into_target_partial_modified_scan() + { + let merge_into_state = unsafe { + &mut *self + .hash_join_state + .merge_into_state + .as_ref() + .unwrap() + .get() + }; + let chunk_offsets = &mut merge_into_state.chunk_offsets; + chunk_offsets.push(build_state.generation_state.build_num_rows as u32); + } + } + + pub(crate) fn merge_into_try_generate_matched_memory(&self) { + // generate macthed offsets memory. + if self + .hash_join_state + .need_merge_into_target_partial_modified_scan() + { + let merge_into_state = unsafe { + &mut *self + .hash_join_state + .merge_into_state + .as_ref() + .unwrap() + .get() + }; + let matched = &mut merge_into_state.matched; + let build_state = unsafe { &*self.hash_join_state.build_state.get() }; + let atomic_pointer = &mut merge_into_state.atomic_pointer; + *matched = vec![0; build_state.generation_state.build_num_rows]; + let pointer = + unsafe { std::mem::transmute::<*mut u8, *mut AtomicU8>(matched.as_mut_ptr()) }; + *atomic_pointer = MatchedPtr(pointer); + } + } +} + +impl HashJoinProbeState { + #[inline] + pub(crate) fn check_and_set_matched( + &self, + build_indexes: &[RowPtr], + matched_idx: usize, + valids: &Bitmap, + ) -> Result<()> { + // merge into target table as build side. + if self + .hash_join_state + .need_merge_into_target_partial_modified_scan() + { + let merge_into_state = unsafe { + &*self + .hash_join_state + .merge_into_state + .as_ref() + .unwrap() + .get() + }; + let chunk_offsets = &merge_into_state.chunk_offsets; + + let pointer = &merge_into_state.atomic_pointer; + // add matched indexes. + for (idx, row_ptr) in build_indexes[0..matched_idx].iter().enumerate() { + unsafe { + if !valids.get_bit_unchecked(idx) { + continue; + } + } + let offset = if row_ptr.chunk_index == 0 { + row_ptr.row_index as usize + } else { + chunk_offsets[(row_ptr.chunk_index - 1) as usize] as usize + + row_ptr.row_index as usize + }; + + let mut old_mactehd_counts = + unsafe { (*pointer.0.add(offset)).load(Ordering::Relaxed) }; + let mut new_matched_count = old_mactehd_counts + 1; + loop { + if old_mactehd_counts > 0 { + return Err(ErrorCode::UnresolvableConflict( + "multi rows from source match one and the same row in the target_table multi times in probe phase", + )); + } + + let res = unsafe { + (*pointer.0.add(offset)).compare_exchange_weak( + old_mactehd_counts, + new_matched_count, + Ordering::SeqCst, + Ordering::SeqCst, + ) + }; + + match res { + Ok(_) => break, + Err(x) => { + old_mactehd_counts = x; + new_matched_count = old_mactehd_counts + 1; + } + }; + } + } + } + Ok(()) + } + + pub(crate) fn probe_merge_into_partial_modified_done(&self) -> Result<()> { + let old_count = self.probe_workers.fetch_sub(1, Ordering::Relaxed); + if old_count == 1 { + // Divide the final scan phase into multiple tasks. + self.generate_merge_into_final_scan_task()?; + } + Ok(()) + } + + pub(crate) fn generate_merge_into_final_scan_task(&self) -> Result<()> { + let merge_into_state = unsafe { + &*self + .hash_join_state + .merge_into_state + .as_ref() + .unwrap() + .get() + }; + let block_info_index = &merge_into_state.block_info_index; + let matched = &merge_into_state.matched; + let chunks_offsets = &merge_into_state.chunk_offsets; + let partial_unmodified = block_info_index.gather_all_partial_block_offsets(matched); + let all_matched_blocks = block_info_index.gather_matched_all_blocks(matched); + + // generate chunks + info!("chunk len: {}", chunks_offsets.len()); + info!("intervals len: {} ", block_info_index.intervals.len()); + info!( + "partial unmodified blocks num: {}", + partial_unmodified.len() + ); + info!( + "all_matched_blocks blocks num: {}", + all_matched_blocks.len() + ); + let mut tasks = block_info_index.chunk_offsets(&partial_unmodified, chunks_offsets); + info!("partial unmodified chunk num: {}", tasks.len()); + for prefix in all_matched_blocks { + // deleted block + tasks.push((Vec::new(), prefix)); + } + *self.final_merge_into_partial_unmodified_scan_tasks.write() = tasks.into(); + Ok(()) + } + + pub(crate) fn final_merge_into_partial_unmodified_scan_task( + &self, + ) -> Option { + let mut tasks = self.final_merge_into_partial_unmodified_scan_tasks.write(); + tasks.pop_front() + } +} + +impl HashJoinState { + pub(crate) fn need_merge_into_target_partial_modified_scan(&self) -> bool { + self.merge_into_state.is_some() + } +} + +impl TransformHashJoinProbe { + pub(crate) fn final_merge_into_partial_unmodified_scan( + &mut self, + item: MergeIntoChunkPartialUnmodified, + ) -> Result<()> { + // matched whole block, need to delete + if item.0.is_empty() { + let prefix = item.1; + let (segment_idx, block_idx) = split_prefix(prefix); + info!( + "matched whole block: segment_idx: {}, block_idx: {}", + segment_idx, block_idx + ); + let data_block = DataBlock::empty_with_meta(Box::new(BlockMetaIndex { + segment_idx: segment_idx as usize, + block_idx: block_idx as usize, + inner: None, + })); + self.output_data_blocks.push_back(data_block); + return Ok(()); + } + let merge_into_state = unsafe { + &*self + .join_probe_state + .hash_join_state + .merge_into_state + .as_ref() + .unwrap() + .get() + }; + let chunks_offsets = &merge_into_state.chunk_offsets; + let build_state = unsafe { &*self.join_probe_state.hash_join_state.build_state.get() }; + let chunk_block = &build_state.generation_state.chunks[item.1 as usize]; + let chunk_start = if item.1 == 0 { + 0 + } else { + chunks_offsets[(item.1 - 1) as usize] + }; + for (interval, prefix) in item.0 { + for start in ((interval.0 - chunk_start)..=(interval.1 - chunk_start)) + .step_by(self.max_block_size) + { + let end = (interval.1 - chunk_start).min(start + self.max_block_size as u32 - 1); + let range = (start..=end).collect::>(); + let data_block = chunk_block.take( + &range, + &mut self.probe_state.generation_state.string_items_buf, + )?; + assert!(!data_block.is_empty()); + let (segment_idx, block_idx) = split_prefix(prefix); + info!( + "matched partial block: segment_idx: {}, block_idx: {}", + segment_idx, block_idx + ); + let data_block = data_block.add_meta(Some(Box::new(BlockMetaIndex { + segment_idx: segment_idx as usize, + block_idx: block_idx as usize, + inner: None, + })))?; + self.output_data_blocks.push_back(data_block); + } + } + Ok(()) + } +} diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/mod.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/mod.rs index f5713ff83073..aaa005493c5e 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/mod.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/mod.rs @@ -11,7 +11,6 @@ // 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. - mod build_spill; mod build_state; mod common; @@ -19,6 +18,7 @@ mod desc; mod hash_join_build_state; mod hash_join_probe_state; mod hash_join_state; +mod merge_into_hash_join_optimization; mod probe_join; mod probe_spill; mod probe_state; diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/probe_join/left_join.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/probe_join/left_join.rs index 6f889f4d3af0..da582b4fff4c 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/probe_join/left_join.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/probe_join/left_join.rs @@ -14,7 +14,6 @@ use std::sync::atomic::Ordering; -use databend_common_arrow::arrow::bitmap::Bitmap; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::BlockEntry; @@ -371,75 +370,6 @@ impl HashJoinProbeState { Ok(self.merge_eq_block(probe_block, build_block, matched_idx)) } - #[inline] - fn check_and_set_matched( - &self, - build_indexes: &[RowPtr], - matched_idx: usize, - valids: &Bitmap, - ) -> Result<()> { - // merge into target table as build side. - if self - .hash_join_state - .need_merge_into_target_partial_modified_scan() - { - let merge_into_state = unsafe { - &*self - .hash_join_state - .merge_into_state - .as_ref() - .unwrap() - .get() - }; - let chunk_offsets = &merge_into_state.chunk_offsets; - - let pointer = &merge_into_state.atomic_pointer; - // add matched indexes. - for (idx, row_ptr) in build_indexes[0..matched_idx].iter().enumerate() { - unsafe { - if !valids.get_bit_unchecked(idx) { - continue; - } - } - let offset = if row_ptr.chunk_index == 0 { - row_ptr.row_index as usize - } else { - chunk_offsets[(row_ptr.chunk_index - 1) as usize] as usize - + row_ptr.row_index as usize - }; - - let mut old_mactehd_counts = - unsafe { (*pointer.0.add(offset)).load(Ordering::Relaxed) }; - let mut new_matched_count = old_mactehd_counts + 1; - loop { - if old_mactehd_counts > 0 { - return Err(ErrorCode::UnresolvableConflict( - "multi rows from source match one and the same row in the target_table multi times in probe phase", - )); - } - - let res = unsafe { - (*pointer.0.add(offset)).compare_exchange_weak( - old_mactehd_counts, - new_matched_count, - Ordering::SeqCst, - Ordering::SeqCst, - ) - }; - - match res { - Ok(_) => break, - Err(x) => { - old_mactehd_counts = x; - new_matched_count = old_mactehd_counts + 1; - } - }; - } - } - } - Ok(()) - } - #[inline] #[allow(clippy::too_many_arguments)] fn process_left_or_full_join_block( diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/transform_hash_join_probe.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/transform_hash_join_probe.rs index a66edf500dfe..15889b5145a7 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/transform_hash_join_probe.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/transform_hash_join_probe.rs @@ -17,17 +17,14 @@ use std::collections::VecDeque; use std::sync::atomic::Ordering; use std::sync::Arc; -use databend_common_catalog::plan::split_prefix; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::DataBlock; use databend_common_expression::FunctionContext; use databend_common_sql::optimizer::ColumnSet; use databend_common_sql::plans::JoinType; -use databend_common_storages_fuse::operations::BlockMetaIndex; use log::info; -use super::hash_join_probe_state::ChunkPartialUnmodified; use crate::pipelines::processors::transforms::hash_join::probe_spill::ProbeSpillState; use crate::pipelines::processors::transforms::hash_join::HashJoinProbeState; use crate::pipelines::processors::transforms::hash_join::ProbeState; @@ -57,13 +54,13 @@ pub struct TransformHashJoinProbe { output_port: Arc, input_data: VecDeque, - output_data_blocks: VecDeque, + pub(crate) output_data_blocks: VecDeque, projections: ColumnSet, step: HashJoinProbeStep, step_logs: Vec, - join_probe_state: Arc, - probe_state: ProbeState, - max_block_size: usize, + pub(crate) join_probe_state: Arc, + pub(crate) probe_state: ProbeState, + pub(crate) max_block_size: usize, outer_scan_finished: bool, processor_id: usize, @@ -124,70 +121,6 @@ impl TransformHashJoinProbe { Ok(()) } - fn final_merge_into_partial_unmodified_scan( - &mut self, - item: ChunkPartialUnmodified, - ) -> Result<()> { - // matched whole block, need to delete - if item.0.is_empty() { - let prefix = item.1; - let (segment_idx, block_idx) = split_prefix(prefix); - info!( - "matched whole block: segment_idx: {}, block_idx: {}", - segment_idx, block_idx - ); - let data_block = DataBlock::empty_with_meta(Box::new(BlockMetaIndex { - segment_idx: segment_idx as usize, - block_idx: block_idx as usize, - inner: None, - })); - self.output_data_blocks.push_back(data_block); - return Ok(()); - } - let merge_into_state = unsafe { - &*self - .join_probe_state - .hash_join_state - .merge_into_state - .as_ref() - .unwrap() - .get() - }; - let chunks_offsets = &merge_into_state.chunk_offsets; - let build_state = unsafe { &*self.join_probe_state.hash_join_state.build_state.get() }; - let chunk_block = &build_state.generation_state.chunks[item.1 as usize]; - let chunk_start = if item.1 == 0 { - 0 - } else { - chunks_offsets[(item.1 - 1) as usize] - }; - for (interval, prefix) in item.0 { - for start in ((interval.0 - chunk_start)..=(interval.1 - chunk_start)) - .step_by(self.max_block_size) - { - let end = (interval.1 - chunk_start).min(start + self.max_block_size as u32 - 1); - let range = (start..=end).collect::>(); - let data_block = chunk_block.take( - &range, - &mut self.probe_state.generation_state.string_items_buf, - )?; - assert!(!data_block.is_empty()); - let (segment_idx, block_idx) = split_prefix(prefix); - info!( - "matched partial block: segment_idx: {}, block_idx: {}", - segment_idx, block_idx - ); - let data_block = data_block.add_meta(Some(Box::new(BlockMetaIndex { - segment_idx: segment_idx as usize, - block_idx: block_idx as usize, - inner: None, - })))?; - self.output_data_blocks.push_back(data_block); - } - } - Ok(()) - } - fn final_scan(&mut self, task: usize) -> Result<()> { let data_blocks = self .join_probe_state @@ -275,6 +208,13 @@ impl TransformHashJoinProbe { .hash_join_state .need_merge_into_target_partial_modified_scan() { + assert!(matches!( + self.join_probe_state + .hash_join_state + .hash_join_desc + .join_type, + JoinType::Left + )); self.join_probe_state .probe_merge_into_partial_modified_done()?; Ok(Event::Async) @@ -444,7 +384,6 @@ impl Processor for TransformHashJoinProbe { Ok(()) } HashJoinProbeStep::FinalScan => { - // if self.join_probe_state.h if self .join_probe_state .hash_join_state From 865ea8b3585ecdb6ad82a72c7b9ec808492e2e02 Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Tue, 16 Jan 2024 23:18:39 +0800 Subject: [PATCH 42/47] remove a.txt --- a.txt | 7 ------- 1 file changed, 7 deletions(-) delete mode 100644 a.txt diff --git a/a.txt b/a.txt deleted file mode 100644 index 55a507fac890..000000000000 --- a/a.txt +++ /dev/null @@ -1,7 +0,0 @@ -intervals_string: [(0,237533) (237534,811068) (811069,1165420) (1165421,1738750) (1738751,2312789) (2312790,2887196) (2887197,3460092) (3460093,4034056) (4034057,4607072) (4607073,5181204) (5181205,5754438) (5754439,6327660) (6327661,6900860) (6900861,7474879) (7474880,8048824) (8048825,8622265) (8622266,9212089) (9212090,9785116) (9785117,10359017) (10359018,10948841) (10948842,11521663) (11521664,12094634) (12094635,12668909) (12668910,13242207) (13242208,13815000) (13815001,14388437) (14388438,14961724) (14961725,15535530) (15535531,16109636) (16109637,16682669) (16682670,17256957) (17256958,17830626) (17830627,18403272) (18403273,18977533) (18977534,19551437) (19551438,20125106) (20125107,20698854) (20698855,21272069) (21272070,21845219) (21845220,22419288) (22419289,22561286) (22561287,23135682) (23135683,23709204) (23709205,24282399) (24282400,24855599) (24855600,25430163) (25430164,26003400) (26003401,26577381) (26577382,27151897) (27151898,27724679) (27724680,28269351) (28269352,28842727) (28842728,29417305) (29417306,29991576) (29991577,30214112) (30214113,30787411) (30787412,31360395) (31360396,31950219) (31950220,32523734) (32523735,33097407) (33097408,33671105) (33671106,33972279) (33972280,34546266) (34546267,35119282) (35119283,35692524) (35692525,36266016) (36266017,36839196) (36839197,37044870) (37044871,37617683) (37617684,38191653) (38191654,38765968) (38765969,39338780) (39338781,39912197) (39912198,40487432) (40487433,41060652) (41060653,41634348) (41634349,42224172) (42224173,42797938) (42797939,43371720) (43371721,43945281) (43945282,44519037) (44519038,45092484) (45092485,45666675) (45666676,46256499) (46256500,46830243) (46830244,47403729) (47403730,47978550) (47978551,48551303) (48551304,49124574) (49124575,49697844) (49697845,50271914) (50271915,50845216) (50845217,51418598) (51418599,51991349) (51991350,52565143) (52565144,53139411) (53139412,53712648) (53712649,54286724) (54286725,54859983) (54859984,55433700) (55433701,56007050) (56007051,56596874) (56596875,57170024) (57170025,57744711) (57744712,58317999) (58318000,58892731) (58892732,59466281) (59466282,60040707) (60040708,60614999) (60615000,61188189) (61188190,61761456) (61761457,62336197) (62336198,62909662) (62909663,63483120) (63483121,64055843) (64055844,64629576) (64629577,65202830) (65202831,65776524) (65776525,66350998) (66350999,66924565) (66924566,67281639) (67281640,67855929) (67855930,68429199) (68429200,69002918) (69002919,69576444) (69576445,70150256) (70150257,70724219) (70724220,71298505) (71298506,71873951) (71873952,72447446) (72447447,73020448) (73020449,73594288) (73594289,74169041) (74169042,74742398) (74742399,75316427) (75316428,75889506) (75889507,76462744) (76462745,77038390) (77038391,77611612) (77611613,78185851) (78185852,78760666) (78760667,79334714) (79334715,79907979) (79907980,80481938) (80481939,81054792) (81054793,81628150) (81628151,82201928) (82201929,82764043) (82764044,83337806) (83337807,83911308) (83911309,84485024) (84485025,85059685) (85059686,85633227) (85633228,86206692) (86206693,86779426) (86779427,87353195) (87353196,87927979) (87927980,88501912) (88501913,89075177) (89075178,89649118) (89649119,90223331) (90223332,90798077) (90798078,91371589) (91371590,91945390) (91945391,92519248) (92519249,93093438) (93093439,93667712) (93667713,94242440) (94242441,94815905) (94815906,95390195) (95390196,95963663) (95963664,96537859) (96537860,97112484) (97112485,97686050) (97686051,98259326) (98259327,98832637) (98832638,99406186) (99406187,99979479) (99979480,100552833) (100552834,101127605) (101127606,101701375) (101701376,102274143) (102274144,102847640) (102847641,103421011) (103421012,103993814) (103993815,104567211) (104567212,105140770) (105140771,105714775) (105714776,106289053) (106289054,106862621) (106862622,107435824) (107435825,108009383) (108009384,108582932) (108582933,109156905) (109156906,109730400) (109730401,110304291) (110304292,110878350) (110878351,111451744) (111451745,112024456) (112024457,112598573) (112598574,113172848) (113172849,113747214) (113747215,114320708) (114320709,114894482) (114894483,115468021) (115468022,116041280) ] -chunks_offsets_string: [1165421,2312790,3460093,4607073,5754439,6900861,8048825,9212090,10359018,11521664,12668910,13815001,14961725,16109637,17256958,18403273,19551438,20698855,21845220,23135683,24282400,25430164,26577382,27724680,28842728,29991577,31360396,32523735,33671106,35119283,36266017,37617684,38765969,39912198,41060653,42224173,43371721,44519038,45666676,46830244,47978551,49124575,50271915,51418599,52565144,53712649,54859984,56007051,57170025,58318000,59466282,60615000,61761457,62909663,64055844,65202831,66350999,67855930,69002919,70150257,71298506,72447447,73594289,74742399,75889507,77038391,78185852,79334715,80481939,81628151,82764044,83911309,85059686,86206693,87353196,88501913,89649119,90798078,91945391,93093439,94242441,95390196,96537860,97686051,98832638,99979480,101127606,102274144,103421012,104567212,105714776,106862622,108009384,109156906,110304292,111451745,112598574,113747215,114894483,116041281] -partial_unmodified_string: [(3460093,4034055),8589875200 (4549765,4607072),8589930496 (7474880,8048823),8589914112 (8564920,8622265),8589750272 (10359018,10948840),8589643777 (11464379,11521663),8589608960 (11521664,12094633),8589834240 (12611477,12668909),8589887488 (13815001,14388436),8589932544 (14904389,14961724),8589660160 (18977534,19551436),8589854720 (20067731,20125106),8589871104 (20125107,20698853),8589799424 (21214743,21272069),8589815808 (21845220,22419287),8589758464 (22488102,22561285),8589533184 (23078237,23135682),8589563904 (23135683,23709203),8589666304 (24225075,24282399),8589862912 (24855600,25430162),8589893632 (25946070,26003400),8589535232 (27724680,28269350),8589856768 (28785384,28842726),8589873152 (29359840,29417304),8589801472 (29934148,29991575),8589817856 (30132876,30214111),8589668352 (30730073,30787410),8589557760 (31303093,31360394),8589760512 (32466378,32523733),8589565952 (33040037,33097407),8589864960 (33972280,34546265),8589895680 (35061975,35119282),8589537280 (36839197,37044869),8589844480 (37560399,37617683),8589580288 (37617684,38191652),8589840384 (38708532,38765968),8589602816 (38765969,39338779),8589676544 (39854849,39912197),8589824000 (43371721,43945280),8589725696 (44461656,44519037),8589836288 (44519038,45092483),8589531136 (45609255,45666675),8589647872 (46830244,47403728),8589842432 (47921067,47978549),8589604864 (48494025,48551302),8589678592 (49067246,49124574),8589731840 (49124575,49697843),8589826048 (50214507,50271914),8589694976 (52565144,53139410),8589727744 (53655318,53712647),8589611008 (54229311,54286724),8589838336 (55433701,56007049),8589770752 (56596874,56596874),8589668353 (60040708,60614998),8589746176 (61130870,61188188),8589719552 (61704123,61761455),8589524992 (62278722,62336196),8589819904 (62852311,62909661),8589613056 (63425767,63483120),8589701120 (63483121,64055842),8589674496 (64572200,64629576),8589740032 (65202831,65776523),8589772800 (66293547,66350998),8589617152 (66924566,67281638),8589574144 (67798500,67855929),8589903872 (70150257,70724218),8589527040 (71241071,71298505),8589821952 (71873952,72447445),8589703168 (72963146,73020447),8589623296 (73536904,73594288),8589709312 (75316428,75889505),8589697024 (76405413,76462744),8589897728 (76462745,77038389),8589742080 (77554288,77611612),8589762560 (86206693,86779425),8589764608 (87295810,87353195),8589629440 (87353196,87927978),8589907968 (88444516,88501912),8589778944 (88501913,89075176),8589651968 (89591723,89649118),8589850624 (93093439,93667711),8589910016 (94184960,94242439),8589635584 (94758554,94815905),8589791232 (97112485,97686049),8589780992 (98201993,98259326),8589713408 (100552834,101127604),8589928448 (101643998,101701375),8589891584 (102274144,102847639),8589912064 (103363673,103421011),8589637632 (104567212,105140769),8589860864 (105657370,105714775),8589682688 (107435825,108009382),8589920256 (108525569,108582931),8589598720 (109099505,109156905),8589922304 (109156906,109730399),8589883392 (110246901,110304291),8589721600 (112598574,113172847),8589600768 (113689772,113747214),8589924352 (113747215,114320707),8589723648 (114837101,114894481),8589848576 (115410659,115468021),8589830144 ] - -error: error happens after fetched 0 rows: APIError: ResponseError with 1104: assertion `left == right` failed - left: 4 - right: 1 \ No newline at end of file From 7bbedcd8939acfeda62a5526651c232b58a0c6eb Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Wed, 17 Jan 2024 12:29:25 +0800 Subject: [PATCH 43/47] fix check --- src/query/service/src/pipelines/builders/mod.rs | 1 + .../transforms/hash_join/merge_into_hash_join_optimization.rs | 3 ++- .../src/pipelines/processors/transforms/hash_join/mod.rs | 1 + 3 files changed, 4 insertions(+), 1 deletion(-) diff --git a/src/query/service/src/pipelines/builders/mod.rs b/src/query/service/src/pipelines/builders/mod.rs index 1d7a709aff9d..cf1c1f33d205 100644 --- a/src/query/service/src/pipelines/builders/mod.rs +++ b/src/query/service/src/pipelines/builders/mod.rs @@ -11,6 +11,7 @@ // 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. + mod builder_aggregate; mod builder_append_table; mod builder_commit; diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/merge_into_hash_join_optimization.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/merge_into_hash_join_optimization.rs index 4f7cdf763df5..e65b2e3f5c44 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/merge_into_hash_join_optimization.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/merge_into_hash_join_optimization.rs @@ -1,4 +1,3 @@ -use std::cell::SyncUnsafeCell; // Copyright 2021 Datafuse Labs // // Licensed under the Apache License, Version 2.0 (the "License"); @@ -12,6 +11,8 @@ use std::cell::SyncUnsafeCell; // 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::cell::SyncUnsafeCell; use std::sync::atomic::AtomicU8; use std::sync::atomic::Ordering; diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/mod.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/mod.rs index aaa005493c5e..2c7660191737 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/mod.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/mod.rs @@ -11,6 +11,7 @@ // 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. + mod build_spill; mod build_state; mod common; From f5ca491394a1f0172d456dbfc8b66a66bf9b40ab Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Wed, 17 Jan 2024 13:44:56 +0800 Subject: [PATCH 44/47] chore: modify function name --- .../transforms/hash_join/hash_join_build_state.rs | 2 +- .../hash_join/merge_into_hash_join_optimization.rs | 6 +----- 2 files changed, 2 insertions(+), 6 deletions(-) diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_build_state.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_build_state.rs index 6216e01baa09..c86ced3fe191 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_build_state.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_build_state.rs @@ -191,7 +191,7 @@ impl HashJoinBuildState { .buffer_row_size .fetch_add(input_rows, Ordering::Relaxed); - self.merge_into_try_build_merge_into_block_info_index(input.clone(), old_size); + self.merge_into_try_build_block_info_index(input.clone(), old_size); if old_size + input_rows < self.chunk_size_limit { return Ok(()); diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/merge_into_hash_join_optimization.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/merge_into_hash_join_optimization.rs index e65b2e3f5c44..323b776a8e87 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/merge_into_hash_join_optimization.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/merge_into_hash_join_optimization.rs @@ -78,11 +78,7 @@ impl MergeIntoState { } impl HashJoinBuildState { - pub(crate) fn merge_into_try_build_merge_into_block_info_index( - &self, - input: DataBlock, - old_size: usize, - ) { + pub(crate) fn merge_into_try_build_block_info_index(&self, input: DataBlock, old_size: usize) { // merge into target table as build side. if self .hash_join_state From f4bb2cf74bdeea9edb77476a41d578be9dcd35fe Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Wed, 17 Jan 2024 15:22:02 +0800 Subject: [PATCH 45/47] rename variables with merge_into prefix --- src/common/hashtable/src/lib.rs | 2 +- src/common/hashtable/src/utils.rs | 14 +++++------ .../pipelines/builders/builder_exchange.rs | 2 +- .../src/pipelines/builders/builder_join.rs | 18 ++++++++------ .../pipelines/builders/builder_merge_into.rs | 4 ++-- .../builders/merge_into_join_optimizations.rs | 6 ++--- .../service/src/pipelines/pipeline_builder.rs | 6 ++--- .../hash_join/hash_join_probe_state.rs | 6 ++--- .../transforms/hash_join/hash_join_state.rs | 4 ++-- .../merge_into_hash_join_optimization.rs | 24 +++++++++---------- .../hash_join/transform_hash_join_probe.rs | 4 ++-- ...cessor_extract_hash_table_by_row_number.rs | 8 +++---- 12 files changed, 51 insertions(+), 47 deletions(-) diff --git a/src/common/hashtable/src/lib.rs b/src/common/hashtable/src/lib.rs index 777ef95e872c..9800028411e1 100644 --- a/src/common/hashtable/src/lib.rs +++ b/src/common/hashtable/src/lib.rs @@ -113,5 +113,5 @@ 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::BlockInfoIndex; 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 238322cd11af..92cd091d1612 100644 --- a/src/common/hashtable/src/utils.rs +++ b/src/common/hashtable/src/utils.rs @@ -198,9 +198,9 @@ pub mod sse { // for now we just support sql like below: // `merge into t using source on xxx when matched then update xxx when not macthed then insert xxx. // for merge into: -// we use BlockInfoIndex to maintain an index for the block info in chunks. +// we use MergeIntoBlockInfoIndex to maintain an index for the block info in chunks. -pub struct BlockInfoIndex { +pub struct MergeIntoBlockInfoIndex { // the intervals will be like below: // (0,10)(11,29),(30,38). it's ordered. pub intervals: Vec, @@ -221,9 +221,9 @@ pub type Interval = (u32, u32); /// segment2_block0 | /// /// ......... -impl BlockInfoIndex { +impl MergeIntoBlockInfoIndex { pub fn new_with_capacity(capacity: usize) -> Self { - BlockInfoIndex { + MergeIntoBlockInfoIndex { intervals: Vec::with_capacity(capacity), prefixs: Vec::with_capacity(capacity), length: 0, @@ -410,7 +410,7 @@ impl BlockInfoIndex { } /// we think the build blocks count is about 1024 at most time. -impl Default for BlockInfoIndex { +impl Default for MergeIntoBlockInfoIndex { fn default() -> Self { Self { intervals: Vec::with_capacity(1024), @@ -426,7 +426,7 @@ fn test_block_info_index() { // we should get [10,10],[31,37] let intervals: Vec = vec![(0, 10), (11, 20), (21, 30), (31, 39)]; let find_interval: Interval = (10, 37); - let mut block_info_index = BlockInfoIndex::new_with_capacity(10); + let mut block_info_index = MergeIntoBlockInfoIndex::new_with_capacity(10); for (idx, interval) in intervals.iter().enumerate() { block_info_index.insert_block_offsets(*interval, idx as u64) } @@ -599,7 +599,7 @@ fn test_chunk_offsets_skip_chunk() { let partial_unmodified = vec![((8, 10), 0), ((40, 46), 4), ((51, 55), 5)]; let chunks_offsets = vec![21, 40, 61]; let intervals: Vec = vec![(0, 10), (11, 20), (21, 30), (31, 39), (40, 50), (51, 60)]; - let mut block_info_index = BlockInfoIndex::new_with_capacity(10); + let mut block_info_index = MergeIntoBlockInfoIndex::new_with_capacity(10); for (idx, interval) in intervals.iter().enumerate() { block_info_index.insert_block_offsets(*interval, idx as u64) } diff --git a/src/query/service/src/pipelines/builders/builder_exchange.rs b/src/query/service/src/pipelines/builders/builder_exchange.rs index 733debac29d7..66d13b358d6a 100644 --- a/src/query/service/src/pipelines/builders/builder_exchange.rs +++ b/src/query/service/src/pipelines/builders/builder_exchange.rs @@ -29,7 +29,7 @@ impl PipelineBuilder { )?; // add sharing data self.join_state = build_res.builder_data.input_join_state; - self.probe_data_fields = build_res.builder_data.input_probe_schema; + self.merge_into_probe_data_fields = build_res.builder_data.input_probe_schema; self.main_pipeline = build_res.main_pipeline; self.pipelines.extend(build_res.sources_pipelines); diff --git a/src/query/service/src/pipelines/builders/builder_join.rs b/src/query/service/src/pipelines/builders/builder_join.rs index 2933ccf4cd6a..3db9373560dc 100644 --- a/src/query/service/src/pipelines/builders/builder_join.rs +++ b/src/query/service/src/pipelines/builders/builder_join.rs @@ -126,11 +126,15 @@ impl PipelineBuilder { pub(crate) fn build_join(&mut self, join: &HashJoin) -> Result<()> { let id = join.probe.get_table_index(); // for merge into target table as build side. - let (build_table_index, is_distributed_merge_into) = - self.get_merge_into_optimization_flag(join); + let (merge_into_build_table_index, merge_into_is_distributed) = + self.merge_into_get_optimization_flag(join); - let state = - self.build_join_state(join, id, build_table_index, is_distributed_merge_into)?; + let state = self.build_join_state( + join, + id, + merge_into_build_table_index, + merge_into_is_distributed, + )?; self.expand_build_side_pipeline(&join.build, join, state.clone())?; self.build_join_probe(join, state) } @@ -140,7 +144,7 @@ impl PipelineBuilder { join: &HashJoin, id: IndexType, merge_into_target_table_index: IndexType, - is_distributed_merge_into: bool, + merge_into_is_distributed: bool, ) -> Result> { HashJoinState::try_create( self.ctx.clone(), @@ -150,7 +154,7 @@ impl PipelineBuilder { &join.probe_to_build, id, merge_into_target_table_index, - is_distributed_merge_into, + merge_into_is_distributed, ) } @@ -297,7 +301,7 @@ impl PipelineBuilder { projected_probe_fields.push(field.clone()); } } - self.probe_data_fields = Some(projected_probe_fields); + self.merge_into_probe_data_fields = Some(projected_probe_fields); } Ok(()) diff --git a/src/query/service/src/pipelines/builders/builder_merge_into.rs b/src/query/service/src/pipelines/builders/builder_merge_into.rs index 21e247a35ccf..0b7f554f5979 100644 --- a/src/query/service/src/pipelines/builders/builder_merge_into.rs +++ b/src/query/service/src/pipelines/builders/builder_merge_into.rs @@ -142,7 +142,7 @@ impl PipelineBuilder { return Ok(()); } assert!(self.join_state.is_some()); - assert!(self.probe_data_fields.is_some()); + assert!(self.merge_into_probe_data_fields.is_some()); let join_state = self.join_state.clone().unwrap(); // split row_number and log @@ -161,7 +161,7 @@ impl PipelineBuilder { let pipe_items = vec![ ExtractHashTableByRowNumber::create( join_state, - self.probe_data_fields.clone().unwrap(), + self.merge_into_probe_data_fields.clone().unwrap(), merge_type.clone(), )? .into_pipe_item(), diff --git a/src/query/service/src/pipelines/builders/merge_into_join_optimizations.rs b/src/query/service/src/pipelines/builders/merge_into_join_optimizations.rs index eeb06a4f9b3b..024aa5e38b6f 100644 --- a/src/query/service/src/pipelines/builders/merge_into_join_optimizations.rs +++ b/src/query/service/src/pipelines/builders/merge_into_join_optimizations.rs @@ -21,9 +21,9 @@ use databend_common_storages_fuse::operations::need_reserve_block_info; use crate::pipelines::PipelineBuilder; impl PipelineBuilder { - pub(crate) fn get_merge_into_optimization_flag(&self, join: &HashJoin) -> (IndexType, bool) { + pub(crate) fn merge_into_get_optimization_flag(&self, join: &HashJoin) -> (IndexType, bool) { // for merge into target table as build side. - let (build_table_index, is_distributed_merge_into) = + let (merge_into_build_table_index, merge_into_is_distributed) = if matches!(&*join.build, PhysicalPlan::TableScan(_)) { let (need_block_info, is_distributed) = need_reserve_block_info(self.ctx.clone(), join.build.get_table_index()); @@ -35,6 +35,6 @@ impl PipelineBuilder { } else { (DUMMY_TABLE_INDEX, false) }; - (build_table_index, is_distributed_merge_into) + (merge_into_build_table_index, merge_into_is_distributed) } } diff --git a/src/query/service/src/pipelines/pipeline_builder.rs b/src/query/service/src/pipelines/pipeline_builder.rs index 9843e2994315..d5ca7c0af540 100644 --- a/src/query/service/src/pipelines/pipeline_builder.rs +++ b/src/query/service/src/pipelines/pipeline_builder.rs @@ -44,7 +44,7 @@ pub struct PipelineBuilder { pub pipelines: Vec, // probe data_fields for distributed merge into when source build - pub probe_data_fields: Option>, + pub merge_into_probe_data_fields: Option>, pub join_state: Option>, // Cte -> state, each cte has it's own state @@ -74,7 +74,7 @@ impl PipelineBuilder { proc_profs: prof_span_set, exchange_injector: DefaultExchangeInjector::create(), cte_state: HashMap::new(), - probe_data_fields: None, + merge_into_probe_data_fields: None, join_state: None, } } @@ -97,7 +97,7 @@ impl PipelineBuilder { exchange_injector: self.exchange_injector, builder_data: PipelineBuilderData { input_join_state: self.join_state, - input_probe_schema: self.probe_data_fields, + input_probe_schema: self.merge_into_probe_data_fields, }, }) } diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs index dfa86f8db66d..4af0cca200c5 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs @@ -83,7 +83,7 @@ pub struct HashJoinProbeState { /// Final scan tasks pub(crate) final_scan_tasks: RwLock>, /// for merge into target as build side. - pub(crate) final_merge_into_partial_unmodified_scan_tasks: + pub(crate) merge_into_final_partial_unmodified_scan_tasks: RwLock>, pub(crate) mark_scan_map_lock: Mutex<()>, /// Hash method @@ -143,7 +143,7 @@ impl HashJoinProbeState { probe_schema, probe_projections: probe_projections.clone(), final_scan_tasks: RwLock::new(VecDeque::new()), - final_merge_into_partial_unmodified_scan_tasks: RwLock::new(VecDeque::new()), + merge_into_final_partial_unmodified_scan_tasks: RwLock::new(VecDeque::new()), mark_scan_map_lock: Mutex::new(()), hash_method: method, spill_partitions: Default::default(), @@ -384,7 +384,7 @@ impl HashJoinProbeState { || self.hash_join_state.need_mark_scan() || self .hash_join_state - .need_merge_into_target_partial_modified_scan() + .merge_into_need_target_partial_modified_scan() { worker_id = self.probe_workers.fetch_add(1, Ordering::Relaxed); } diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_state.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_state.rs index fa8969336f44..706de842b75b 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_state.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_state.rs @@ -135,7 +135,7 @@ impl HashJoinState { probe_to_build: &[(usize, (bool, bool))], table_index: IndexType, merge_into_target_table_index: IndexType, - is_distributed_merge_into: bool, + merge_into_is_distributed: bool, ) -> Result> { if matches!( hash_join_desc.join_type, @@ -173,7 +173,7 @@ impl HashJoinState { table_index, merge_into_state: MergeIntoState::try_create_merge_into_state( merge_into_target_table_index, - is_distributed_merge_into, + merge_into_is_distributed, ), })) } diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/merge_into_hash_join_optimization.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/merge_into_hash_join_optimization.rs index 323b776a8e87..c6f883d88620 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/merge_into_hash_join_optimization.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/merge_into_hash_join_optimization.rs @@ -23,7 +23,7 @@ use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::BlockMetaInfoDowncast; use databend_common_expression::DataBlock; -use databend_common_hashtable::BlockInfoIndex; +use databend_common_hashtable::MergeIntoBlockInfoIndex; use databend_common_hashtable::RowPtr; use databend_common_sql::IndexType; use databend_common_sql::DUMMY_TABLE_INDEX; @@ -44,11 +44,11 @@ unsafe impl Sync for MatchedPtr {} pub struct MergeIntoState { /// for now we don't support distributed, we will support in the next pr. #[allow(unused)] - pub(crate) is_distributed_merge_into: bool, + pub(crate) merge_into_is_distributed: bool, /// FOR MERGE INTO TARGET TABLE AS BUILD SIDE /// When merge into target table as build side, we should preserve block info index. - pub(crate) block_info_index: BlockInfoIndex, + pub(crate) block_info_index: MergeIntoBlockInfoIndex, /// we use matched to tag the matched offset in chunks. pub(crate) matched: Vec, /// the matched will be modified concurrently, so we use @@ -61,11 +61,11 @@ pub struct MergeIntoState { impl MergeIntoState { pub(crate) fn try_create_merge_into_state( merge_into_target_table_index: IndexType, - is_distributed_merge_into: bool, + merge_into_is_distributed: bool, ) -> Option> { if merge_into_target_table_index != DUMMY_TABLE_INDEX { Some(SyncUnsafeCell::new(MergeIntoState { - is_distributed_merge_into, + merge_into_is_distributed, block_info_index: Default::default(), matched: Vec::new(), atomic_pointer: MatchedPtr(std::ptr::null_mut()), @@ -82,7 +82,7 @@ impl HashJoinBuildState { // merge into target table as build side. if self .hash_join_state - .need_merge_into_target_partial_modified_scan() + .merge_into_need_target_partial_modified_scan() { assert!(input.get_meta().is_some()); let merge_into_state = unsafe { @@ -111,7 +111,7 @@ impl HashJoinBuildState { pub(crate) fn merge_into_try_add_chunk_offset(&self, build_state: &mut BuildState) { if self .hash_join_state - .need_merge_into_target_partial_modified_scan() + .merge_into_need_target_partial_modified_scan() { let merge_into_state = unsafe { &mut *self @@ -130,7 +130,7 @@ impl HashJoinBuildState { // generate macthed offsets memory. if self .hash_join_state - .need_merge_into_target_partial_modified_scan() + .merge_into_need_target_partial_modified_scan() { let merge_into_state = unsafe { &mut *self @@ -162,7 +162,7 @@ impl HashJoinProbeState { // merge into target table as build side. if self .hash_join_state - .need_merge_into_target_partial_modified_scan() + .merge_into_need_target_partial_modified_scan() { let merge_into_state = unsafe { &*self @@ -262,20 +262,20 @@ impl HashJoinProbeState { // deleted block tasks.push((Vec::new(), prefix)); } - *self.final_merge_into_partial_unmodified_scan_tasks.write() = tasks.into(); + *self.merge_into_final_partial_unmodified_scan_tasks.write() = tasks.into(); Ok(()) } pub(crate) fn final_merge_into_partial_unmodified_scan_task( &self, ) -> Option { - let mut tasks = self.final_merge_into_partial_unmodified_scan_tasks.write(); + let mut tasks = self.merge_into_final_partial_unmodified_scan_tasks.write(); tasks.pop_front() } } impl HashJoinState { - pub(crate) fn need_merge_into_target_partial_modified_scan(&self) -> bool { + pub(crate) fn merge_into_need_target_partial_modified_scan(&self) -> bool { self.merge_into_state.is_some() } } diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/transform_hash_join_probe.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/transform_hash_join_probe.rs index 15889b5145a7..d7f49f74ea77 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/transform_hash_join_probe.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/transform_hash_join_probe.rs @@ -206,7 +206,7 @@ impl TransformHashJoinProbe { } else if self .join_probe_state .hash_join_state - .need_merge_into_target_partial_modified_scan() + .merge_into_need_target_partial_modified_scan() { assert!(matches!( self.join_probe_state @@ -387,7 +387,7 @@ impl Processor for TransformHashJoinProbe { if self .join_probe_state .hash_join_state - .need_merge_into_target_partial_modified_scan() + .merge_into_need_target_partial_modified_scan() { if let Some(item) = self .join_probe_state diff --git a/src/query/service/src/pipelines/processors/transforms/processor_extract_hash_table_by_row_number.rs b/src/query/service/src/pipelines/processors/transforms/processor_extract_hash_table_by_row_number.rs index 9d1cd00e1fae..2a98320aa8a0 100644 --- a/src/query/service/src/pipelines/processors/transforms/processor_extract_hash_table_by_row_number.rs +++ b/src/query/service/src/pipelines/processors/transforms/processor_extract_hash_table_by_row_number.rs @@ -41,7 +41,7 @@ pub struct ExtractHashTableByRowNumber { output_port: Arc, input_data: Option, output_data: Vec, - probe_data_fields: Vec, + merge_into_probe_data_fields: Vec, hashstate: Arc, // if insert only, we don't need to // fill null BlockEntries @@ -51,14 +51,14 @@ pub struct ExtractHashTableByRowNumber { impl ExtractHashTableByRowNumber { pub fn create( hashstate: Arc, - probe_data_fields: Vec, + merge_into_probe_data_fields: Vec, merge_type: MergeIntoType, ) -> Result { Ok(Self { input_port: InputPort::create(), output_port: OutputPort::create(), hashstate, - probe_data_fields, + merge_into_probe_data_fields, input_data: None, output_data: Vec::new(), merge_type, @@ -151,7 +151,7 @@ impl Processor for ExtractHashTableByRowNumber { } else { // Create null chunk for unmatched rows in probe side let mut null_block = DataBlock::new( - self.probe_data_fields + self.merge_into_probe_data_fields .iter() .map(|df| { BlockEntry::new( From fbd7cbac13585ffe28d65051df624b6e23d22cfc Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Wed, 17 Jan 2024 16:59:57 +0800 Subject: [PATCH 46/47] rename function --- .../transforms/hash_join/hash_join_build_state.rs | 2 +- .../transforms/hash_join/hash_join_probe_state.rs | 4 ++++ .../hash_join/merge_into_hash_join_optimization.rs | 2 +- .../transforms/hash_join/probe_join/left_join.rs | 14 +++++++++++--- 4 files changed, 17 insertions(+), 5 deletions(-) diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_build_state.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_build_state.rs index c86ced3fe191..d967cf90412b 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_build_state.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_build_state.rs @@ -191,7 +191,7 @@ impl HashJoinBuildState { .buffer_row_size .fetch_add(input_rows, Ordering::Relaxed); - self.merge_into_try_build_block_info_index(input.clone(), old_size); + self.merge_into_try_build_block_info_index(input, old_size); if old_size + input_rows < self.chunk_size_limit { return Ok(()); diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs index 4af0cca200c5..534386ad8655 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs @@ -57,6 +57,10 @@ use crate::pipelines::processors::HashJoinState; use crate::sessions::QueryContext; use crate::sql::planner::plans::JoinType; +// ({(Interval,prefix),(Interval,repfix),...},chunk_idx) +// 1.The Interval is the partial unmodified interval offset in chunks. +// 2.Prefix is segment_idx_block_id +// 3.chunk_idx: the index of correlated chunk in chunks. pub type MergeIntoChunkPartialUnmodified = (Vec<(Interval, u64)>, u64); /// Define some shared states for all hash join probe threads. pub struct HashJoinProbeState { diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/merge_into_hash_join_optimization.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/merge_into_hash_join_optimization.rs index c6f883d88620..08984a78b56e 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/merge_into_hash_join_optimization.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/merge_into_hash_join_optimization.rs @@ -153,7 +153,7 @@ impl HashJoinBuildState { impl HashJoinProbeState { #[inline] - pub(crate) fn check_and_set_matched( + pub(crate) fn merge_into_check_and_set_matched( &self, build_indexes: &[RowPtr], matched_idx: usize, diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/probe_join/left_join.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/probe_join/left_join.rs index da582b4fff4c..bdf26a1dc984 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/probe_join/left_join.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/probe_join/left_join.rs @@ -454,7 +454,11 @@ impl HashJoinProbeState { }; } } - self.check_and_set_matched(build_indexes, matched_idx, &probe_state.true_validity)?; + self.merge_into_check_and_set_matched( + build_indexes, + matched_idx, + &probe_state.true_validity, + )?; return Ok(()); } @@ -481,7 +485,11 @@ impl HashJoinProbeState { }; } } - self.check_and_set_matched(build_indexes, matched_idx, &probe_state.true_validity)?; + self.merge_into_check_and_set_matched( + build_indexes, + matched_idx, + &probe_state.true_validity, + )?; } else if all_false { let mut idx = 0; while idx < matched_idx { @@ -512,7 +520,7 @@ impl HashJoinProbeState { } } else { let mut idx = 0; - self.check_and_set_matched(build_indexes, matched_idx, &validity)?; + self.merge_into_check_and_set_matched(build_indexes, matched_idx, &validity)?; while idx < matched_idx { unsafe { let valid = validity.get_bit_unchecked(idx); From c5cb3a94c25d2e08584c8c18981ccf2204f51d44 Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Wed, 17 Jan 2024 18:37:47 +0800 Subject: [PATCH 47/47] move merge_into_try_build_block_info_index to front --- .../processors/transforms/hash_join/hash_join_build_state.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_build_state.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_build_state.rs index d967cf90412b..d1d0ec7c5ec7 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_build_state.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_build_state.rs @@ -184,14 +184,14 @@ impl HashJoinBuildState { let mut buffer = self.hash_join_state.row_space.buffer.write(); let input_rows = input.num_rows(); - buffer.push(input.clone()); let old_size = self .hash_join_state .row_space .buffer_row_size .fetch_add(input_rows, Ordering::Relaxed); - self.merge_into_try_build_block_info_index(input, old_size); + self.merge_into_try_build_block_info_index(input.clone(), old_size); + buffer.push(input); if old_size + input_rows < self.chunk_size_limit { return Ok(());