From 7e8717dfbc2f5fae60d26ead7f7a058b80e290c1 Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Tue, 5 Mar 2024 23:57:18 +0800 Subject: [PATCH 01/24] enable stage table as non-local-table --- src/query/storages/stage/src/stage_table.rs | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/query/storages/stage/src/stage_table.rs b/src/query/storages/stage/src/stage_table.rs index 8a8b00d420afc..54a084aa01d73 100644 --- a/src/query/storages/stage/src/stage_table.rs +++ b/src/query/storages/stage/src/stage_table.rs @@ -167,6 +167,10 @@ impl Table for StageTable { } } + fn is_local(&self) -> bool { + false + } + fn read_data( &self, ctx: Arc, From e9010e3404b204c0d05371b349343f94aae3d046 Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Fri, 15 Mar 2024 19:50:29 +0800 Subject: [PATCH 02/24] add bloom index for merge into runtime filter --- Cargo.lock | 1 + src/query/catalog/Cargo.toml | 1 + .../src/plan/datasource/datasource_plan.rs | 3 + src/query/catalog/src/runtime_filter_info.rs | 10 ++++ .../hash_join/hash_join_build_state.rs | 59 +++++++++++++++++++ src/query/settings/src/settings_default.rs | 6 ++ .../settings/src/settings_getter_setter.rs | 4 ++ .../sql/src/planner/format/display_plan.rs | 2 +- .../fuse/src/operations/read/fuse_source.rs | 30 +++++++++- 9 files changed, 114 insertions(+), 2 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index c16cab78a4b6f..cf5fd8e601e8a 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2705,6 +2705,7 @@ dependencies = [ "async-trait-fn", "chrono", "dashmap", + "databend-common-arrow", "databend-common-base", "databend-common-config", "databend-common-exception", diff --git a/src/query/catalog/Cargo.toml b/src/query/catalog/Cargo.toml index 9150214ccc299..d226cd98256a0 100644 --- a/src/query/catalog/Cargo.toml +++ b/src/query/catalog/Cargo.toml @@ -10,6 +10,7 @@ test = false [dependencies] databend-common-base = { path = "../../common/base" } +databend-common-arrow = { path = "../../common/arrow" } databend-common-config = { path = "../config" } databend-common-exception = { path = "../../common/exception" } databend-common-expression = { path = "../expression" } diff --git a/src/query/catalog/src/plan/datasource/datasource_plan.rs b/src/query/catalog/src/plan/datasource/datasource_plan.rs index 99d0c27023536..59dd914dc82a9 100644 --- a/src/query/catalog/src/plan/datasource/datasource_plan.rs +++ b/src/query/catalog/src/plan/datasource/datasource_plan.rs @@ -13,12 +13,14 @@ // limitations under the License. use std::collections::BTreeMap; +use std::collections::HashMap; use databend_common_expression::FieldIndex; use databend_common_expression::RemoteExpr; use databend_common_expression::Scalar; use databend_common_expression::TableSchemaRef; use databend_common_meta_app::schema::CatalogInfo; +use databend_storages_common_table_meta::meta::BlockMeta; use crate::plan::datasource::datasource_info::DataSourceInfo; use crate::plan::PartStatistics; @@ -49,6 +51,7 @@ pub struct DataSourcePlan { pub data_mask_policy: Option>, pub table_index: usize, + // pub merge_into_target_table_block_meta: Option>, } impl DataSourcePlan { diff --git a/src/query/catalog/src/runtime_filter_info.rs b/src/query/catalog/src/runtime_filter_info.rs index 7c29f193879f0..b20201d92e3fa 100644 --- a/src/query/catalog/src/runtime_filter_info.rs +++ b/src/query/catalog/src/runtime_filter_info.rs @@ -12,6 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. +use databend_common_arrow::arrow::bitmap::Bitmap; +use databend_common_arrow::arrow::buffer::Buffer; use databend_common_expression::Expr; use xorf::BinaryFuse16; @@ -20,6 +22,7 @@ pub struct RuntimeFilterInfo { inlist: Vec>, min_max: Vec>, bloom: Vec<(String, BinaryFuse16)>, + siphashes: Vec<(String, (Buffer, Option))>, } impl RuntimeFilterInfo { @@ -31,6 +34,13 @@ impl RuntimeFilterInfo { self.bloom.push(bloom); } + pub fn add_merge_into_source_build_siphashkeys( + &mut self, + digests: (String, (Buffer, Option)), + ) { + self.siphashes.push(digests); + } + pub fn add_min_max(&mut self, expr: Expr) { self.min_max.push(expr); } 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 9e0c978acc0b1..2cc06d2f9d5f9 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 @@ -21,7 +21,9 @@ use std::sync::atomic::Ordering; use std::sync::Arc; use databend_common_arrow::arrow::bitmap::Bitmap; +use databend_common_arrow::arrow::buffer::Buffer; use databend_common_base::base::tokio::sync::Barrier; +use databend_common_catalog::merge_into_join::MergeIntoJoinType; use databend_common_catalog::runtime_filter_info::RuntimeFilterInfo; use databend_common_catalog::table_context::TableContext; use databend_common_exception::ErrorCode; @@ -55,6 +57,7 @@ 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_storages_common_index::BloomIndex; use ethnum::U256; use itertools::Itertools; use log::info; @@ -845,9 +848,19 @@ impl HashJoinBuildState { probe_key, )?; } + + // add BloomIndex hash keys for merge into source build. + self.build_merge_into_runtime_filter_siphashes( + build_chunks, + &mut runtime_filter, + build_key, + probe_key, + )?; + if self.enable_bloom_runtime_filter { self.bloom_runtime_filter(build_chunks, &mut runtime_filter, build_key, probe_key)?; } + if self.enable_min_max_runtime_filter { self.min_max_runtime_filter( build_chunks, @@ -912,6 +925,52 @@ impl HashJoinBuildState { Ok(()) } + // for merge into source build cases, like below: + // merge into `t1` using `t2` on xxx when matched xx when not matched xxx, if merge_into_optimizer + // gives `t2` as source build side, we can build source join keys `siphashes`, that's because we use + // siphash to build target table's bloom index block. + // in this way, we can avoid current `runtime_filter()` func's performance cost, especially for large + // target table case, the `runtime_filter()`'s cost is even higer than disable `runtime_filter()`. + // However, for `build_runtime_filter_siphashes()` usages, we currently just used for merge into, + // we doesn't support join query, and it's only for `source build` cases. In fact, source build is the + // main case in most time. + fn build_merge_into_runtime_filter_siphashes( + &self, + data_blocks: &[DataBlock], + runtime_filter: &mut RuntimeFilterInfo, + build_key: &Expr, + probe_key: &Expr, + ) -> Result<()> { + // `calculate_nullable_column_digest`, `apply_bloom_pruning` + let merge_type = self.ctx.get_merge_into_join(); + if matches!(merge_type.merge_into_join_type, MergeIntoJoinType::Right) { + if let Expr::ColumnRef { id, .. } = probe_key { + let mut columns = Vec::with_capacity(data_blocks.len()); + for block in data_blocks.iter() { + if block.num_columns() == 0 { + continue; + } + let evaluator = Evaluator::new(block, &self.func_ctx, &BUILTIN_FUNCTIONS); + let column = evaluator + .run(build_key)? + .convert_to_full_column(build_key.data_type(), block.num_rows()); + columns.push(column); + } + if columns.is_empty() { + return Ok(()); + } + let build_key_column = Column::concat_columns(columns.into_iter())?; + let digests = BloomIndex::calculate_nullable_column_digest( + &self.func_ctx, + &build_key_column, + &build_key_column.data_type(), + )?; + runtime_filter.add_merge_into_source_build_siphashkeys((id.to_string(), digests)); + } + } + Ok(()) + } + fn inlist_runtime_filter( &self, runtime_filter: &mut RuntimeFilterInfo, diff --git a/src/query/settings/src/settings_default.rs b/src/query/settings/src/settings_default.rs index 6de6e92f47d19..7d05f709c474c 100644 --- a/src/query/settings/src/settings_default.rs +++ b/src/query/settings/src/settings_default.rs @@ -485,6 +485,12 @@ impl DefaultSettings { mode: SettingMode::Both, range: Some(SettingRange::Numeric(0..=1)), }), + ("enable_merge_into_source_build_bloom", DefaultSettingValue { + value: UserSettingValue::UInt64(0), + desc: "Enable merge into source build bloom.", + mode: SettingMode::Both, + range: Some(SettingRange::Numeric(0..=1)), + }), ("enable_distributed_merge_into", DefaultSettingValue { value: UserSettingValue::UInt64(0), desc: "Enable distributed merge into.", diff --git a/src/query/settings/src/settings_getter_setter.rs b/src/query/settings/src/settings_getter_setter.rs index 3c5bd488853e3..3b324e5c76cf1 100644 --- a/src/query/settings/src/settings_getter_setter.rs +++ b/src/query/settings/src/settings_getter_setter.rs @@ -438,6 +438,10 @@ impl Settings { Ok(self.try_get_u64("enable_experimental_merge_into")? != 0) } + pub fn get_enable_merge_into_source_build_bloom(&self) -> Result { + Ok(self.try_get_u64("enable_merge_into_source_build_bloom")? != 0) + } + pub fn get_enable_distributed_merge_into(&self) -> Result { Ok(self.try_get_u64("enable_distributed_merge_into")? != 0) } diff --git a/src/query/sql/src/planner/format/display_plan.rs b/src/query/sql/src/planner/format/display_plan.rs index 055e24e5fae9c..cf37d082f71a7 100644 --- a/src/query/sql/src/planner/format/display_plan.rs +++ b/src/query/sql/src/planner/format/display_plan.rs @@ -324,7 +324,7 @@ fn format_merge_into(merge_into: &MergeInto) -> Result { )); // add macthed clauses let mut matched_children = Vec::with_capacity(merge_into.matched_evaluators.len()); - let taregt_schema = table_entry.table().schema(); + let taregt_schema = table_entry.table().schema_with_stream(); for evaluator in &merge_into.matched_evaluators { let condition_format = evaluator.condition.as_ref().map_or_else( || "condition: None".to_string(), diff --git a/src/query/storages/fuse/src/operations/read/fuse_source.rs b/src/query/storages/fuse/src/operations/read/fuse_source.rs index acab2afb038ae..fc022ca937cbe 100644 --- a/src/query/storages/fuse/src/operations/read/fuse_source.rs +++ b/src/query/storages/fuse/src/operations/read/fuse_source.rs @@ -15,6 +15,8 @@ use std::collections::VecDeque; use std::sync::Arc; +use databend_common_catalog::merge_into_join::MergeIntoJoinType; +use databend_common_catalog::plan::DataSourceInfo; use databend_common_catalog::plan::DataSourcePlan; use databend_common_catalog::plan::PartInfoPtr; use databend_common_catalog::plan::StealablePartitions; @@ -148,8 +150,34 @@ pub fn build_fuse_parquet_source_pipeline( (max_threads, max_io_requests) = adjust_threads_and_request(false, max_threads, max_io_requests, plan); - let mut source_builder = SourcePipeBuilder::create(); + let merge_into_join = ctx.get_merge_into_join(); + if ctx + .get_settings() + .get_enable_merge_into_source_build_bloom()? + && matches!( + merge_into_join.merge_into_join_type, + MergeIntoJoinType::Right + ) + && merge_into_join.target_tbl_idx == plan.table_index + { + // we can add block_metas info for merge into runtime filter, they will + // be used for bloom prune for target table block. + assert!(matches!(plan.source_info, DataSourceInfo::TableSource(_))); + // if let DataSourceInfo::TableSource(table_info) = plan.source_info { + // let table = ctx + // .get_table(table_info.catalog(), table_info.name(), 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 mut source_builder = SourcePipeBuilder::create(); match block_reader.support_blocking_api() { true => { let partitions = dispatch_partitions(ctx.clone(), plan, max_threads); From f1ad0cd084d6dc0fa1f0c582cea4e65ed785ba38 Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Wed, 20 Mar 2024 02:45:25 +0800 Subject: [PATCH 03/24] add merge_into_source_build_bloom_info --- src/query/catalog/src/merge_into_join.rs | 41 ++++++++- .../src/plan/datasource/datasource_plan.rs | 3 - src/query/catalog/src/runtime_filter_info.rs | 6 ++ src/query/catalog/src/table_context.rs | 13 +++ .../interpreters/interpreter_merge_into.rs | 23 ++++- src/query/service/src/sessions/query_ctx.rs | 27 ++++++ .../service/src/sessions/query_ctx_shared.rs | 5 ++ .../sql/src/planner/optimizer/optimizer.rs | 88 ++++++++++++++----- src/query/storages/common/index/src/lib.rs | 1 + .../storages/fuse/src/operations/read/mod.rs | 1 + .../read/native_data_source_reader.rs | 36 ++++++++ .../read/parquet_data_source_reader.rs | 33 +++++++ .../operations/read/runtime_filter_prunner.rs | 88 ++++++++++++++++++- .../storages/fuse/src/operations/read/util.rs | 37 +++++++- .../mutator/merge_into_mutator.rs | 2 +- 15 files changed, 372 insertions(+), 32 deletions(-) diff --git a/src/query/catalog/src/merge_into_join.rs b/src/query/catalog/src/merge_into_join.rs index db48f84b66a5b..4f2efe0cb4e8e 100644 --- a/src/query/catalog/src/merge_into_join.rs +++ b/src/query/catalog/src/merge_into_join.rs @@ -12,6 +12,12 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::sync::Arc; + +use databend_common_meta_app::schema::CatalogInfo; +use databend_common_meta_app::schema::TableInfo; +use databend_storages_common_table_meta::meta::Location; + #[derive(Clone)] pub enum MergeIntoJoinType { Left, @@ -23,14 +29,38 @@ 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 type MergeIntoSourceBuildSegments = Arc>; + +// MergeIntoJoin is used in two cases: +// I. target build optimization: +// we should support MergeIntoJoinType::Left(need to support LeftInner,LeftAnti) to use MergeIntoBlockInfoHashTable in two situations: +// 1. distributed broadcast join and target table as build side (don't support this now). +// 2. in standalone mode and target table as build side (supported). +// 3. native(not-supported) and parquet(supported) format +// for the `target build optimization`, merge_into_join_type is only Left for current implementation in fact.And we +// don't support distributed mode and parquet. So only if it is a LeftJoin and non-dirtributed-merge-into(even if in distributed environment +// but the merge_into_optimizer can also give a non-dirtributed-merge-into) and uses parquet, the `target build optimization` can be enabled. +// II. source build runtime bloom filter. +// only if it's a RightJoin((need to support RightInner,RightAnti)),target_tbl_idx is not invalid. We can make sure it enables +// `source build runtime bloom filter`. And in this case the table_info and catalog_info must be some. We give an `assert` for this judge. +// +// So let's do a summary: +// In most cases, the MergeIntoJoin's `merge_into_join_type` is NormalJoin (even this is a real merge into join),because +// MergeIntoJoin is used to judge whether we enable target_build_optimizations or source build runtime bloom filter. +// Both of these optimizations all will check `merge_into_join_type` to judge if the correlated optimization is enabled. +// So we only `set_merge_into_join` when we use these two optimziations (of course there is an exception, the `merge_into_optimzier` can't distinct +// parquet and native, so it will think it's using parquet format in default, so if it's a native format in fact, the `interpreter_merge_into` will rollback it. +// But the modification is a little tricky. Because the `MergeIntoJoinType` doesn't contain any info about parquet or native,we set the target_table_index as +// DUMMY_TABLE_INDEX, but in fact it has a target_table_index. However it's ok to do this. Firstly we can do this to make sure the `target build optimization` is +// closed. And it won't cause a conflict with `source build runtime bloom filter`. Because for the `target build optimization`, it's target build, and for +// `source build runtime bloom filter`, it's source build). pub struct MergeIntoJoin { pub merge_into_join_type: MergeIntoJoinType, pub is_distributed: bool, pub target_tbl_idx: usize, + pub table_info: Option, + pub catalog_info: Option, + pub database_name: String, } impl Default for MergeIntoJoin { @@ -40,6 +70,9 @@ impl Default for MergeIntoJoin { is_distributed: false, // Invalid Index target_tbl_idx: usize::MAX, + table_info: None, + catalog_info: None, + database_name: Default::default(), } } } diff --git a/src/query/catalog/src/plan/datasource/datasource_plan.rs b/src/query/catalog/src/plan/datasource/datasource_plan.rs index 59dd914dc82a9..99d0c27023536 100644 --- a/src/query/catalog/src/plan/datasource/datasource_plan.rs +++ b/src/query/catalog/src/plan/datasource/datasource_plan.rs @@ -13,14 +13,12 @@ // limitations under the License. use std::collections::BTreeMap; -use std::collections::HashMap; use databend_common_expression::FieldIndex; use databend_common_expression::RemoteExpr; use databend_common_expression::Scalar; use databend_common_expression::TableSchemaRef; use databend_common_meta_app::schema::CatalogInfo; -use databend_storages_common_table_meta::meta::BlockMeta; use crate::plan::datasource::datasource_info::DataSourceInfo; use crate::plan::PartStatistics; @@ -51,7 +49,6 @@ pub struct DataSourcePlan { pub data_mask_policy: Option>, pub table_index: usize, - // pub merge_into_target_table_block_meta: Option>, } impl DataSourcePlan { diff --git a/src/query/catalog/src/runtime_filter_info.rs b/src/query/catalog/src/runtime_filter_info.rs index b20201d92e3fa..f2c9911b20f13 100644 --- a/src/query/catalog/src/runtime_filter_info.rs +++ b/src/query/catalog/src/runtime_filter_info.rs @@ -34,6 +34,12 @@ impl RuntimeFilterInfo { self.bloom.push(bloom); } + pub fn get_merge_into_source_build_siphashkeys( + &mut self, + ) -> Vec<(String, (Buffer, Option))> { + self.siphashes.clone() + } + pub fn add_merge_into_source_build_siphashkeys( &mut self, digests: (String, (Buffer, Option)), diff --git a/src/query/catalog/src/table_context.rs b/src/query/catalog/src/table_context.rs index 0fda26eb7335c..fb45c5e48649c 100644 --- a/src/query/catalog/src/table_context.rs +++ b/src/query/catalog/src/table_context.rs @@ -22,6 +22,8 @@ use std::sync::Arc; use std::time::SystemTime; use dashmap::DashMap; +use databend_common_arrow::arrow::bitmap::Bitmap; +use databend_common_arrow::arrow::buffer::Buffer; use databend_common_base::base::Progress; use databend_common_base::base::ProgressValues; use databend_common_base::runtime::profile::Profile; @@ -55,6 +57,7 @@ use xorf::BinaryFuse16; use crate::catalog::Catalog; use crate::cluster_info::Cluster; use crate::merge_into_join::MergeIntoJoin; +use crate::merge_into_join::MergeIntoSourceBuildSegments; use crate::plan::DataSourcePlan; use crate::plan::PartInfoPtr; use crate::plan::Partitions; @@ -254,11 +257,16 @@ pub trait TableContext: Send + Sync { fn get_query_profiles(&self) -> Vec; + // fn set_runtime_filter(&self, filters: (usize, RuntimeFilterInfo)); fn set_merge_into_join(&self, join: MergeIntoJoin); fn get_merge_into_join(&self) -> MergeIntoJoin; + // set the target table's segments + fn set_merge_into_source_build_segments(&self, segments: MergeIntoSourceBuildSegments); + // get the target table's segments + fn get_merge_into_source_build_segments(&self) -> MergeIntoSourceBuildSegments; fn get_bloom_runtime_filter_with_id(&self, id: usize) -> Vec<(String, BinaryFuse16)>; @@ -266,6 +274,11 @@ pub trait TableContext: Send + Sync { fn get_min_max_runtime_filter_with_id(&self, id: usize) -> Vec>; + fn get_merge_into_source_build_siphashkeys_with_id( + &self, + id: usize, + ) -> Vec<(String, (Buffer, Option))>; + fn has_bloom_runtime_filters(&self, id: usize) -> bool; fn txn_mgr(&self) -> TxnManagerRef; } diff --git a/src/query/service/src/interpreters/interpreter_merge_into.rs b/src/query/service/src/interpreters/interpreter_merge_into.rs index 2741d6518e444..f0a691e94e18e 100644 --- a/src/query/service/src/interpreters/interpreter_merge_into.rs +++ b/src/query/service/src/interpreters/interpreter_merge_into.rs @@ -17,6 +17,7 @@ use std::sync::Arc; use std::u64::MAX; use databend_common_catalog::merge_into_join::MergeIntoJoin; +use databend_common_catalog::merge_into_join::MergeIntoJoinType; use databend_common_catalog::table::TableExt; use databend_common_exception::ErrorCode; use databend_common_exception::Result; @@ -181,7 +182,7 @@ impl MergeIntoInterpreter { // 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 + // II. target build optimization for this pr. the MergeIntoType is FullOperation let mut target_build_optimization = matches!(self.plan.merge_type, MergeIntoType::FullOperation) && !self.plan.columns_set.contains(&self.plan.row_id_index); @@ -199,6 +200,9 @@ impl MergeIntoInterpreter { target_tbl_idx: DUMMY_TABLE_INDEX, is_distributed: merge_into_join.is_distributed, merge_into_join_type: merge_into_join.merge_into_join_type, + table_info: merge_into_join.table_info.clone(), + catalog_info: merge_into_join.catalog_info.clone(), + database_name: merge_into_join.database_name.clone(), }); } } @@ -287,7 +291,22 @@ impl MergeIntoInterpreter { let table_info = fuse_table.get_table_info().clone(); let catalog_ = self.ctx.get_catalog(catalog).await?; - + // try add catalog_info and table_info for `source_build_bloom_filter` + let merge_into_join = self.ctx.get_merge_into_join(); + let source_build_bloom_filter = matches!( + merge_into_join.merge_into_join_type, + MergeIntoJoinType::Right + ) && merge_into_join.target_tbl_idx != DUMMY_TABLE_INDEX; + if source_build_bloom_filter { + self.ctx.set_merge_into_join(MergeIntoJoin { + target_tbl_idx: merge_into_join.target_tbl_idx, + is_distributed: merge_into_join.is_distributed, + merge_into_join_type: merge_into_join.merge_into_join_type, + table_info: Some(table_info.clone()), + catalog_info: Some(catalog_.info()), + database_name: merge_into_join.database_name.clone(), + }) + } // merge_into_source is used to recv join's datablocks and split them into macthed and not matched // datablocks. let merge_into_source = if !*distributed && extract_exchange { diff --git a/src/query/service/src/sessions/query_ctx.rs b/src/query/service/src/sessions/query_ctx.rs index 85235c40e5f38..5b4a7c725d0f1 100644 --- a/src/query/service/src/sessions/query_ctx.rs +++ b/src/query/service/src/sessions/query_ctx.rs @@ -32,6 +32,8 @@ use std::time::UNIX_EPOCH; use chrono_tz::Tz; use dashmap::mapref::multiple::RefMulti; use dashmap::DashMap; +use databend_common_arrow::arrow::bitmap::Bitmap; +use databend_common_arrow::arrow::buffer::Buffer; use databend_common_base::base::tokio::task::JoinHandle; use databend_common_base::base::Progress; use databend_common_base::base::ProgressValues; @@ -39,6 +41,7 @@ use databend_common_base::runtime::profile::Profile; use databend_common_base::runtime::profile::ProfileStatisticsName; use databend_common_base::runtime::TrySpawn; use databend_common_catalog::merge_into_join::MergeIntoJoin; +use databend_common_catalog::merge_into_join::MergeIntoSourceBuildSegments; use databend_common_catalog::plan::DataSourceInfo; use databend_common_catalog::plan::DataSourcePlan; use databend_common_catalog::plan::PartInfoPtr; @@ -1008,6 +1011,9 @@ impl TableContext for QueryContext { 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, + catalog_info: merge_into_join.catalog_info.clone(), + table_info: merge_into_join.table_info.clone(), + database_name: merge_into_join.database_name.clone(), } } @@ -1019,6 +1025,17 @@ impl TableContext for QueryContext { } } + fn get_merge_into_source_build_siphashkeys_with_id( + &self, + id: IndexType, + ) -> Vec<(String, (Buffer, Option))> { + let runtime_filters = self.shared.runtime_filters.read(); + match runtime_filters.get(&id) { + Some(v) => v.get_merge_into_source_build_siphashkeys(), + None => vec![], + } + } + fn get_inlist_runtime_filter_with_id(&self, id: IndexType) -> Vec> { let runtime_filters = self.shared.runtime_filters.read(); match runtime_filters.get(&id) { @@ -1045,6 +1062,16 @@ impl TableContext for QueryContext { fn txn_mgr(&self) -> TxnManagerRef { self.shared.session.session_ctx.txn_mgr() } + + fn set_merge_into_source_build_segments(&self, segments: MergeIntoSourceBuildSegments) { + let mut merge_into_source_build_segments = + self.shared.merge_into_source_build_segments.write(); + *merge_into_source_build_segments = segments; + } + + fn get_merge_into_source_build_segments(&self) -> MergeIntoSourceBuildSegments { + self.shared.merge_into_source_build_segments.read().clone() + } } impl TrySpawn for QueryContext { diff --git a/src/query/service/src/sessions/query_ctx_shared.rs b/src/query/service/src/sessions/query_ctx_shared.rs index 6573451eccfe2..00ece389a2b59 100644 --- a/src/query/service/src/sessions/query_ctx_shared.rs +++ b/src/query/service/src/sessions/query_ctx_shared.rs @@ -26,6 +26,7 @@ use databend_common_base::runtime::drop_guard; use databend_common_base::runtime::Runtime; use databend_common_catalog::catalog::CatalogManager; use databend_common_catalog::merge_into_join::MergeIntoJoin; +use databend_common_catalog::merge_into_join::MergeIntoSourceBuildSegments; use databend_common_catalog::query_kind::QueryKind; use databend_common_catalog::runtime_filter_info::RuntimeFilterInfo; use databend_common_catalog::statistics::data_cache_statistics::DataCacheMetrics; @@ -121,6 +122,9 @@ pub struct QueryContextShared { pub(in crate::sessions) merge_into_join: Arc>, + pub(in crate::sessions) merge_into_source_build_segments: + Arc>, + // Records query level data cache metrics pub(in crate::sessions) query_cache_metrics: DataCacheMetrics, } @@ -170,6 +174,7 @@ impl QueryContextShared { query_profiles: Arc::new(RwLock::new(HashMap::new())), runtime_filters: Default::default(), merge_into_join: Default::default(), + merge_into_source_build_segments: Default::default(), })) } diff --git a/src/query/sql/src/planner/optimizer/optimizer.rs b/src/query/sql/src/planner/optimizer/optimizer.rs index 82c78c1c80fdd..17827b0bfb0b2 100644 --- a/src/query/sql/src/planner/optimizer/optimizer.rs +++ b/src/query/sql/src/planner/optimizer/optimizer.rs @@ -406,35 +406,39 @@ fn optimize_merge_into(opt_ctx: OptimizerContext, plan: Box) -> Resul false }; - // 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`. - let flag = plan.matched_evaluators.len() == 1 - && plan.matched_evaluators[0].condition.is_none() - && 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 + // try add source_build bloom filter + if !change_join_order && matches!(plan.merge_type, MergeIntoType::FullOperation) && opt_ctx .table_ctx .get_settings() - .get_join_spilling_memory_ratio()? - == 0 - && flag + .get_enable_merge_into_source_build_bloom()? { - new_columns_set.remove(&plan.row_id_index); + let merge_into_join = opt_ctx.table_ctx.get_merge_into_join(); + // this is the first time set, so it must be none, and we will set it in `interpreter_merge_into` + assert!(matches!(merge_into_join.catalog_info, None)); + assert!(matches!(merge_into_join.table_info, None)); + assert!(merge_into_join.database_name.as_str() == ""); opt_ctx.table_ctx.set_merge_into_join(MergeIntoJoin { - merge_into_join_type: MergeIntoJoinType::Left, - is_distributed: false, + // we will set catalog_info and table_info in `intepreter_merge_into` + catalog_info: None, + table_info: None, target_tbl_idx: plan.target_table_idx, + is_distributed: false, // we will set it after later optimization. + merge_into_join_type: MergeIntoJoinType::Right, + database_name: plan.database.clone(), }) } + // 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`. + 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(); + // try to optimize distributed join, only if // - distributed optimization is enabled // - no local table scan @@ -494,6 +498,27 @@ fn optimize_merge_into(opt_ctx: OptimizerContext, plan: Box) -> Resul // III. (merge_into_join_sexpr.clone(), false) }; + // try add target_build_optimizations + if change_join_order + && matches!(plan.merge_type, MergeIntoType::FullOperation) + && opt_ctx + .table_ctx + .get_settings() + .get_join_spilling_memory_ratio()? + == 0 + && flag + { + new_columns_set.remove(&plan.row_id_index); + let merge_into_join = opt_ctx.table_ctx.get_merge_into_join(); + opt_ctx.table_ctx.set_merge_into_join(MergeIntoJoin { + merge_into_join_type: MergeIntoJoinType::Left, + is_distributed: distributed, + target_tbl_idx: plan.target_table_idx, + catalog_info: merge_into_join.catalog_info.clone(), + table_info: merge_into_join.table_info.clone(), + database_name: merge_into_join.database_name.clone(), + }) + } Ok(Plan::MergeInto(Box::new(MergeInto { input: Box::new(optimized_distributed_merge_into_join_sexpr), @@ -503,6 +528,27 @@ fn optimize_merge_into(opt_ctx: OptimizerContext, plan: Box) -> Resul ..*plan }))) } else { + // try add target_build_optimizations + if change_join_order + && matches!(plan.merge_type, MergeIntoType::FullOperation) + && opt_ctx + .table_ctx + .get_settings() + .get_join_spilling_memory_ratio()? + == 0 + && flag + { + new_columns_set.remove(&plan.row_id_index); + let merge_into_join = opt_ctx.table_ctx.get_merge_into_join(); + 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, + catalog_info: merge_into_join.catalog_info.clone(), + table_info: merge_into_join.table_info.clone(), + database_name: merge_into_join.database_name, + }) + } Ok(Plan::MergeInto(Box::new(MergeInto { input: join_sexpr, change_join_order, @@ -512,6 +558,7 @@ fn optimize_merge_into(opt_ctx: OptimizerContext, plan: Box) -> Resul } } +// Todo(JackTan25): This method is useless for now, it will be used for distributed target build optimization in the future. fn try_to_change_as_broadcast_join( merge_into_join_sexpr: SExpr, _change_join_order: bool, @@ -523,7 +570,8 @@ fn try_to_change_as_broadcast_join( 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() { - let join: Join = merge_into_join_sexpr.child(0)?.plan().clone().try_into()?; + let mut join: Join = merge_into_join_sexpr.child(0)?.plan().clone().try_into()?; + let join_s_expr = merge_into_join_sexpr .child(0)? .replace_plan(Arc::new(RelOperator::Join(join))); diff --git a/src/query/storages/common/index/src/lib.rs b/src/query/storages/common/index/src/lib.rs index 88894ad8f7a40..cec3ca37a8604 100644 --- a/src/query/storages/common/index/src/lib.rs +++ b/src/query/storages/common/index/src/lib.rs @@ -24,6 +24,7 @@ mod range_index; pub use bloom_index::BloomIndex; pub use bloom_index::BloomIndexMeta; pub use bloom_index::FilterEvalResult; +pub use filters::Xor8Filter; pub use index::Index; pub use page_index::PageIndex; pub use range_index::statistics_to_domain; diff --git a/src/query/storages/fuse/src/operations/read/mod.rs b/src/query/storages/fuse/src/operations/read/mod.rs index be309849c7fb3..16ccca3cab0c0 100644 --- a/src/query/storages/fuse/src/operations/read/mod.rs +++ b/src/query/storages/fuse/src/operations/read/mod.rs @@ -32,4 +32,5 @@ 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::can_merge_into_target_build_bloom_filter; pub use util::need_reserve_block_info; diff --git a/src/query/storages/fuse/src/operations/read/native_data_source_reader.rs b/src/query/storages/fuse/src/operations/read/native_data_source_reader.rs index f7d059bfd49ea..2fb17557863d3 100644 --- a/src/query/storages/fuse/src/operations/read/native_data_source_reader.rs +++ b/src/query/storages/fuse/src/operations/read/native_data_source_reader.rs @@ -15,6 +15,7 @@ use std::any::Any; use std::sync::Arc; +use databend_common_catalog::merge_into_join; use databend_common_catalog::plan::PartInfoPtr; use databend_common_catalog::plan::StealablePartitions; use databend_common_catalog::table_context::TableContext; @@ -32,7 +33,9 @@ use databend_common_pipeline_sources::SyncSourcer; use databend_common_sql::IndexType; use log::debug; +use super::can_merge_into_target_build_bloom_filter; use super::native_data_source::NativeDataSource; +use super::util::MergeIntoSourceBuildBloomInfo; use crate::io::AggIndexReader; use crate::io::BlockReader; use crate::io::TableMetaLocationGenerator; @@ -57,6 +60,7 @@ pub struct ReadNativeDataSource { table_schema: Arc, table_index: IndexType, + merge_into_source_build_bloom_info: MergeIntoSourceBuildBloomInfo, } impl ReadNativeDataSource { @@ -73,6 +77,7 @@ impl ReadNativeDataSource { ) -> Result { let batch_size = ctx.get_settings().get_storage_fetch_part_num()? as usize; let func_ctx = ctx.get_function_context()?; + let merge_into_join = ctx.get_merge_into_join(); SyncSourcer::create(ctx.clone(), output.clone(), ReadNativeDataSource:: { func_ctx, id, @@ -86,6 +91,16 @@ impl ReadNativeDataSource { virtual_reader, table_schema, table_index, + merge_into_source_build_bloom_info: MergeIntoSourceBuildBloomInfo { + can_do_merge_into_rumtime_filter_bloom: can_merge_into_target_build_bloom_filter( + ctx.clone(), + table_index, + )?, + segment_infos: Default::default(), + catalog_info: merge_into_join.catalog_info.clone(), + table_info: merge_into_join.table_info.clone(), + database_name: merge_into_join.database_name.clone(), + }, }) } } @@ -104,6 +119,7 @@ impl ReadNativeDataSource { ) -> Result { let batch_size = ctx.get_settings().get_storage_fetch_part_num()? as usize; let func_ctx = ctx.get_function_context()?; + let merge_into_join = ctx.get_merge_into_join(); Ok(ProcessorPtr::create(Box::new(ReadNativeDataSource::< false, > { @@ -119,6 +135,16 @@ impl ReadNativeDataSource { virtual_reader, table_schema, table_index, + merge_into_source_build_bloom_info: MergeIntoSourceBuildBloomInfo { + can_do_merge_into_rumtime_filter_bloom: can_merge_into_target_build_bloom_filter( + ctx.clone(), + table_index, + )?, + segment_infos: Default::default(), + catalog_info: merge_into_join.catalog_info.clone(), + table_info: merge_into_join.table_info.clone(), + database_name: merge_into_join.database_name.clone(), + }, }))) } } @@ -144,6 +170,11 @@ impl SyncSource for ReadNativeDataSource { &part, &filters, &self.func_ctx, + self.merge_into_source_build_bloom_info + .can_do_merge_into_rumtime_filter_bloom, + self.partitions.ctx.clone(), + self.table_index, + &mut self.merge_into_source_build_bloom_info, )? { return Ok(Some(DataBlock::empty())); } @@ -251,6 +282,11 @@ impl Processor for ReadNativeDataSource { &part, &filters, &self.func_ctx, + self.merge_into_source_build_bloom_info + .can_do_merge_into_rumtime_filter_bloom, + self.partitions.ctx.clone(), + self.table_index, + &mut self.merge_into_source_build_bloom_info, )? { continue; } 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 6cc06bcbe6cfa..ac67c717a523b 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 @@ -32,7 +32,9 @@ use databend_common_pipeline_sources::SyncSourcer; use databend_common_sql::IndexType; use log::debug; +use super::can_merge_into_target_build_bloom_filter; use super::parquet_data_source::ParquetDataSource; +use super::util::MergeIntoSourceBuildBloomInfo; use crate::fuse_part::FusePartInfo; use crate::io::AggIndexReader; use crate::io::BlockReader; @@ -45,6 +47,7 @@ use crate::operations::read::runtime_filter_prunner::runtime_filter_pruner; pub struct ReadParquetDataSource { func_ctx: FunctionContext, id: usize, + // build table index, not probe side table index table_index: IndexType, finished: bool, batch_size: usize, @@ -58,6 +61,8 @@ pub struct ReadParquetDataSource { virtual_reader: Arc>, table_schema: Arc, + + merge_into_source_build_bloom_info: MergeIntoSourceBuildBloomInfo, } impl ReadParquetDataSource { @@ -76,6 +81,7 @@ impl ReadParquetDataSource { let batch_size = ctx.get_settings().get_storage_fetch_part_num()? as usize; let func_ctx = ctx.get_function_context()?; if BLOCKING_IO { + let merge_into_join = ctx.get_merge_into_join(); SyncSourcer::create(ctx.clone(), output.clone(), ReadParquetDataSource:: { func_ctx, id, @@ -89,8 +95,17 @@ impl ReadParquetDataSource { index_reader, virtual_reader, table_schema, + merge_into_source_build_bloom_info: MergeIntoSourceBuildBloomInfo { + can_do_merge_into_rumtime_filter_bloom: + can_merge_into_target_build_bloom_filter(ctx.clone(), table_index)?, + segment_infos: Default::default(), + catalog_info: merge_into_join.catalog_info.clone(), + table_info: merge_into_join.table_info.clone(), + database_name: merge_into_join.database_name.clone(), + }, }) } else { + let merge_into_join = ctx.get_merge_into_join(); Ok(ProcessorPtr::create(Box::new(ReadParquetDataSource::< false, > { @@ -106,6 +121,14 @@ impl ReadParquetDataSource { index_reader, virtual_reader, table_schema, + merge_into_source_build_bloom_info: MergeIntoSourceBuildBloomInfo { + can_do_merge_into_rumtime_filter_bloom: + can_merge_into_target_build_bloom_filter(ctx.clone(), table_index)?, + segment_infos: Default::default(), + catalog_info: merge_into_join.catalog_info.clone(), + table_info: merge_into_join.table_info.clone(), + database_name: merge_into_join.database_name.clone(), + }, }))) } } @@ -132,6 +155,11 @@ impl SyncSource for ReadParquetDataSource { &part, &filters, &self.func_ctx, + self.merge_into_source_build_bloom_info + .can_do_merge_into_rumtime_filter_bloom, + self.partitions.ctx.clone(), + self.table_index, + &mut self.merge_into_source_build_bloom_info, )? { return Ok(Some(DataBlock::empty())); } @@ -248,6 +276,11 @@ impl Processor for ReadParquetDataSource { &part, &filters, &self.func_ctx, + self.merge_into_source_build_bloom_info + .can_do_merge_into_rumtime_filter_bloom, + self.partitions.ctx.clone(), + self.table_index, + &mut self.merge_into_source_build_bloom_info, )? { continue; } diff --git a/src/query/storages/fuse/src/operations/read/runtime_filter_prunner.rs b/src/query/storages/fuse/src/operations/read/runtime_filter_prunner.rs index b261319185dee..84e53e30f05eb 100644 --- a/src/query/storages/fuse/src/operations/read/runtime_filter_prunner.rs +++ b/src/query/storages/fuse/src/operations/read/runtime_filter_prunner.rs @@ -12,13 +12,17 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::collections::hash_map::Entry; use std::collections::HashMap; use std::sync::Arc; use databend_common_arrow::arrow::bitmap::MutableBitmap; +use databend_common_base::runtime::block_on; use databend_common_base::runtime::profile::Profile; use databend_common_base::runtime::profile::ProfileStatisticsName; use databend_common_catalog::plan::PartInfoPtr; +use databend_common_catalog::table_context::TableContext; +use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::types::NumberColumn; use databend_common_expression::Column; @@ -35,20 +39,30 @@ use databend_common_expression::Scalar; use databend_common_expression::TableSchema; use databend_common_functions::BUILTIN_FUNCTIONS; use databend_common_hashtable::FastHash; +use databend_storages_common_cache::LoadParams; use databend_storages_common_index::statistics_to_domain; +use databend_storages_common_table_meta::meta::SegmentInfo; use log::info; use xorf::BinaryFuse16; use xorf::Filter; +use super::util::MergeIntoSourceBuildBloomInfo; +use crate::io::MetaReaders; +use crate::operations::SegmentIndex; use crate::FusePartInfo; +use crate::FuseTable; pub fn runtime_filter_pruner( table_schema: Arc, part: &PartInfoPtr, filters: &[Expr], func_ctx: &FunctionContext, + can_do_merge_into_target_build_bloom_filter: bool, + ctx: Arc, + id: usize, + merge_into_source_build_bloom_info: &mut MergeIntoSourceBuildBloomInfo, ) -> Result { - if filters.is_empty() { + if filters.is_empty() && !can_do_merge_into_target_build_bloom_filter { return Ok(false); } let part = FusePartInfo::from_part(part)?; @@ -99,6 +113,78 @@ pub fn runtime_filter_pruner( return Ok(true); } + // if we can't pruned this block, we can try get siphashkeys if this is a merge into source build + if can_do_merge_into_target_build_bloom_filter { + assert!(matches!(part.block_meta_index(), Some(_))); + let block_meta_index = part.block_meta_index().unwrap(); + let hash_keys = ctx.get_merge_into_source_build_siphashkeys_with_id(id); + let segment_idx = block_meta_index.segment_idx; + let block_idx = block_meta_index.block_idx; + let target_table_segments = ctx.get_merge_into_source_build_segments(); + if let Entry::Vacant(e) = merge_into_source_build_bloom_info + .segment_infos + .entry(segment_idx) + { + let (_,(path, ver)) = target_table_segments.get(segment_idx).ok_or_else(|| { + return ErrorCode::Internal(format!( + "unexpected, segment (idx {}) not found, during do merge into source build bloom filter", + segment_idx + )) + })?; + + let load_param = LoadParams { + location: path.clone(), + len_hint: None, + ver: *ver, + put_cache: true, + }; + let catalog_info = merge_into_source_build_bloom_info + .catalog_info + .as_ref() + .unwrap(); + let table_info = merge_into_source_build_bloom_info + .table_info + .as_ref() + .unwrap(); + let table = block_on(async { + ctx.get_table( + catalog_info.catalog_name(), + &merge_into_source_build_bloom_info.database_name, + &table_info.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 target_table_schema = table.schema_with_stream(); + let data_accessor = fuse_table.get_operator(); + let segment_reader = MetaReaders::segment_info_reader( + data_accessor.clone(), + target_table_schema.clone(), + ); + let compact_segment_info = block_on(async { segment_reader.read(&load_param).await })?; + let segment_info: SegmentInfo = compact_segment_info.try_into()?; + e.insert(segment_info); + } + // load bloom filter + let segment_info = merge_into_source_build_bloom_info + .segment_infos + .get(&segment_idx) + .unwrap(); + assert!(block_idx < segment_info.blocks.len()); + info!( + "merge into source build runtime bloom filter: segment_idx:{},blk_idx:{}", + segment_idx, block_idx + ); + // 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(); + } + Ok(false) } diff --git a/src/query/storages/fuse/src/operations/read/util.rs b/src/query/storages/fuse/src/operations/read/util.rs index 53a06909fc56f..dcb876da7677e 100644 --- a/src/query/storages/fuse/src/operations/read/util.rs +++ b/src/query/storages/fuse/src/operations/read/util.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::collections::HashMap; use std::sync::Arc; use databend_common_catalog::merge_into_join::MergeIntoJoinType; @@ -22,21 +23,55 @@ use databend_common_exception::Result; use databend_common_expression::BlockMetaInfoPtr; use databend_common_expression::DataBlock; use databend_common_expression::Scalar; +use databend_common_meta_app::schema::CatalogInfo; +use databend_common_meta_app::schema::TableInfo; +use databend_storages_common_table_meta::meta::SegmentInfo; use crate::operations::BlockMetaIndex; +use crate::operations::SegmentIndex; use crate::FusePartInfo; +pub struct MergeIntoSourceBuildBloomInfo { + pub can_do_merge_into_rumtime_filter_bloom: bool, + pub segment_infos: HashMap, + pub table_info: Option, + pub catalog_info: Option, + pub database_name: String, +} + 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, + ) && merge_into_join.target_tbl_idx == table_idx + && !merge_into_join.is_distributed, /* we don't support distributed mod for target build optimization for now, */ merge_into_join.is_distributed, ) } +pub fn can_merge_into_target_build_bloom_filter( + ctx: Arc, + table_idx: usize, +) -> Result { + let merge_into_join = ctx.get_merge_into_join(); + let enabled = matches!( + merge_into_join.merge_into_join_type, + MergeIntoJoinType::Right + ) && merge_into_join.target_tbl_idx == table_idx; + if enabled { + assert!( + ctx.get_settings() + .get_enable_merge_into_source_build_bloom()? + ); + assert!(merge_into_join.database_name.as_str() != ""); + assert!(matches!(merge_into_join.catalog_info, Some(_))); + assert!(matches!(merge_into_join.table_info, Some(_))); + } + Ok(enabled) +} + pub(crate) fn add_data_block_meta( block: DataBlock, fuse_part: &FusePartInfo, 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 8d428c9318bd2..46461f50b8b81 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 @@ -673,7 +673,7 @@ impl AggregationContext { } // return true if the block is pruned, otherwise false - async fn apply_bloom_pruning( + pub async fn apply_bloom_pruning( &self, block_meta: &BlockMeta, input_hashes: &[Vec], From d03501c2cae8cacd79c91cc3ff0b6f18c2ce7324 Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Wed, 20 Mar 2024 23:42:34 +0800 Subject: [PATCH 04/24] refactor hashed logic --- src/query/catalog/src/runtime_filter_info.rs | 13 ++---- .../hash_join/hash_join_build_state.rs | 46 +++++++++++++++---- .../tests/it/sql/exec/get_table_bind_test.rs | 18 ++++++++ src/query/storages/fuse/src/operations/mod.rs | 1 + .../operations/read/runtime_filter_prunner.rs | 1 - 5 files changed, 62 insertions(+), 17 deletions(-) diff --git a/src/query/catalog/src/runtime_filter_info.rs b/src/query/catalog/src/runtime_filter_info.rs index f2c9911b20f13..fdc1844162d41 100644 --- a/src/query/catalog/src/runtime_filter_info.rs +++ b/src/query/catalog/src/runtime_filter_info.rs @@ -12,6 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::sync::Arc; + use databend_common_arrow::arrow::bitmap::Bitmap; use databend_common_arrow::arrow::buffer::Buffer; use databend_common_expression::Expr; @@ -22,7 +24,7 @@ pub struct RuntimeFilterInfo { inlist: Vec>, min_max: Vec>, bloom: Vec<(String, BinaryFuse16)>, - siphashes: Vec<(String, (Buffer, Option))>, + siphashes: Vec<(String, Arc>)>, } impl RuntimeFilterInfo { @@ -34,16 +36,11 @@ impl RuntimeFilterInfo { self.bloom.push(bloom); } - pub fn get_merge_into_source_build_siphashkeys( - &mut self, - ) -> Vec<(String, (Buffer, Option))> { + pub fn get_merge_into_source_build_siphashkeys(&self) -> Vec<(String, Arc>)> { self.siphashes.clone() } - pub fn add_merge_into_source_build_siphashkeys( - &mut self, - digests: (String, (Buffer, Option)), - ) { + pub fn add_merge_into_source_build_siphashkeys(&mut self, digests: (String, Arc>)) { self.siphashes.push(digests); } 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 5838b108de327..e956b72104516 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 @@ -57,6 +57,7 @@ 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::can_merge_into_target_build_bloom_filter; use databend_storages_common_index::BloomIndex; use ethnum::U256; use itertools::Itertools; @@ -851,12 +852,14 @@ impl HashJoinBuildState { } // add BloomIndex hash keys for merge into source build. - self.build_merge_into_runtime_filter_siphashes( - build_chunks, - &mut runtime_filter, - build_key, - probe_key, - )?; + if can_merge_into_target_build_bloom_filter(self.ctx.clone(), *table_index)? { + self.build_merge_into_runtime_filter_siphashes( + build_chunks, + &mut runtime_filter, + build_key, + probe_key, + )?; + } if self.enable_bloom_runtime_filter { self.bloom_runtime_filter(build_chunks, &mut runtime_filter, build_key, probe_key)?; @@ -961,12 +964,39 @@ impl HashJoinBuildState { return Ok(()); } let build_key_column = Column::concat_columns(columns.into_iter())?; - let digests = BloomIndex::calculate_nullable_column_digest( + // mabye there will be null values here, so we use nullable column, the null value will be treat as default + // value for the sepcified type, like String -> "", int -> 0. so we need to remove the null hash values here. + let (hashes, bitmap_op) = BloomIndex::calculate_nullable_column_digest( &self.func_ctx, &build_key_column, &build_key_column.data_type(), )?; - runtime_filter.add_merge_into_source_build_siphashkeys((id.to_string(), digests)); + if let Some(bitmap) = bitmap_op { + // no null values + let digests = if bitmap.unset_bits() == 0 { + hashes.to_vec() + } else { + let new_hashes = Vec::with_capacity(bitmap.len()); + assert_eq!(hashes.len(), bitmap.len()); + for row_idx in 0..bitmap.len() { + if bitmap.get_bit(row_idx) { + new_hashes.push(hashes[row_idx]) + } + } + new_hashes.to_vec() + }; + // id is probe key name + runtime_filter.add_merge_into_source_build_siphashkeys(( + id.to_string(), + Arc::new(digests), + )); + } else { + // id is probe key name + runtime_filter.add_merge_into_source_build_siphashkeys(( + id.to_string(), + Arc::new(hashes), + )); + } } } Ok(()) 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 7e8c9c2816f08..9597119884039 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 @@ -18,6 +18,8 @@ use std::sync::atomic::AtomicUsize; use std::sync::Arc; use dashmap::DashMap; +use databend_common_arrow::arrow::bitmap::Bitmap; +use databend_common_arrow::arrow::buffer::Buffer; use databend_common_base::base::tokio; use databend_common_base::base::Progress; use databend_common_base::base::ProgressValues; @@ -26,6 +28,7 @@ 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::merge_into_join::MergeIntoSourceBuildSegments; use databend_common_catalog::plan::DataSourcePlan; use databend_common_catalog::plan::PartInfoPtr; use databend_common_catalog::plan::Partitions; @@ -826,6 +829,21 @@ impl TableContext for CtxDelegation { fn set_read_block_thresholds(&self, _thresholds: BlockThresholds) { todo!() } + + fn get_merge_into_source_build_siphashkeys_with_id( + &self, + id: usize, + ) -> Vec<(String, (Buffer, Option))> { + todo!() + } + + fn set_merge_into_source_build_segments(&self, segments: MergeIntoSourceBuildSegments) { + todo!() + } + + fn get_merge_into_source_build_segments(&self) -> MergeIntoSourceBuildSegments { + todo!() + } } #[tokio::test(flavor = "multi_thread")] diff --git a/src/query/storages/fuse/src/operations/mod.rs b/src/query/storages/fuse/src/operations/mod.rs index b89d0e57db9c7..ad5ad84e5620b 100644 --- a/src/query/storages/fuse/src/operations/mod.rs +++ b/src/query/storages/fuse/src/operations/mod.rs @@ -42,6 +42,7 @@ pub use delete::MutationBlockPruningContext; pub use merge_into::*; pub use mutation::*; pub use read::build_row_fetcher_pipeline; +pub use read::can_merge_into_target_build_bloom_filter; pub use read::need_reserve_block_info; pub use replace_into::*; pub use util::acquire_task_permit; diff --git a/src/query/storages/fuse/src/operations/read/runtime_filter_prunner.rs b/src/query/storages/fuse/src/operations/read/runtime_filter_prunner.rs index 84e53e30f05eb..fcb7e94825dfd 100644 --- a/src/query/storages/fuse/src/operations/read/runtime_filter_prunner.rs +++ b/src/query/storages/fuse/src/operations/read/runtime_filter_prunner.rs @@ -181,7 +181,6 @@ pub fn runtime_filter_pruner( "merge into source build runtime bloom filter: segment_idx:{},blk_idx:{}", segment_idx, block_idx ); - // 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(); } From 7a41e4e5707daf70c292d4d4d2aaab9dbf90e521 Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Thu, 21 Mar 2024 01:17:03 +0800 Subject: [PATCH 05/24] refactot build bloom info --- src/query/catalog/src/merge_into_join.rs | 3 ++ src/query/catalog/src/runtime_filter_info.rs | 2 - src/query/catalog/src/table_context.rs | 4 +- src/query/service/src/sessions/query_ctx.rs | 15 +++++++- .../tests/it/sql/exec/get_table_bind_test.rs | 6 ++- .../sql/src/planner/binder/merge_into.rs | 1 + .../sql/src/planner/optimizer/optimizer.rs | 3 ++ src/query/sql/src/planner/plans/merge_into.rs | 4 ++ .../read/native_data_source_reader.rs | 32 ++++++---------- .../read/parquet_data_source_reader.rs | 28 ++++++-------- .../operations/read/runtime_filter_prunner.rs | 1 + .../storages/fuse/src/operations/read/util.rs | 38 +++++++++++++++++++ 12 files changed, 94 insertions(+), 43 deletions(-) diff --git a/src/query/catalog/src/merge_into_join.rs b/src/query/catalog/src/merge_into_join.rs index 4f2efe0cb4e8e..19b21b5d9b289 100644 --- a/src/query/catalog/src/merge_into_join.rs +++ b/src/query/catalog/src/merge_into_join.rs @@ -14,6 +14,7 @@ use std::sync::Arc; +use databend_common_expression::TableSchemaRef; use databend_common_meta_app::schema::CatalogInfo; use databend_common_meta_app::schema::TableInfo; use databend_storages_common_table_meta::meta::Location; @@ -61,6 +62,7 @@ pub struct MergeIntoJoin { pub table_info: Option, pub catalog_info: Option, pub database_name: String, + pub table_schema: Option, } impl Default for MergeIntoJoin { @@ -73,6 +75,7 @@ impl Default for MergeIntoJoin { table_info: None, catalog_info: None, database_name: Default::default(), + table_schema: None, } } } diff --git a/src/query/catalog/src/runtime_filter_info.rs b/src/query/catalog/src/runtime_filter_info.rs index fdc1844162d41..7a674267ee20d 100644 --- a/src/query/catalog/src/runtime_filter_info.rs +++ b/src/query/catalog/src/runtime_filter_info.rs @@ -14,8 +14,6 @@ use std::sync::Arc; -use databend_common_arrow::arrow::bitmap::Bitmap; -use databend_common_arrow::arrow::buffer::Buffer; use databend_common_expression::Expr; use xorf::BinaryFuse16; diff --git a/src/query/catalog/src/table_context.rs b/src/query/catalog/src/table_context.rs index df4daffa4da7a..2bcf90d471f15 100644 --- a/src/query/catalog/src/table_context.rs +++ b/src/query/catalog/src/table_context.rs @@ -278,7 +278,9 @@ pub trait TableContext: Send + Sync { fn get_merge_into_source_build_siphashkeys_with_id( &self, id: usize, - ) -> Vec<(String, (Buffer, Option))>; + ) -> Vec<(String, Arc>)>; + + fn get_merge_into_source_build_bloom_probe_keys(&self, id: usize) -> Vec; fn has_bloom_runtime_filters(&self, id: usize) -> bool; fn txn_mgr(&self) -> TxnManagerRef; diff --git a/src/query/service/src/sessions/query_ctx.rs b/src/query/service/src/sessions/query_ctx.rs index 812b2ab614946..263e0d74d7061 100644 --- a/src/query/service/src/sessions/query_ctx.rs +++ b/src/query/service/src/sessions/query_ctx.rs @@ -1017,6 +1017,7 @@ impl TableContext for QueryContext { catalog_info: merge_into_join.catalog_info.clone(), table_info: merge_into_join.table_info.clone(), database_name: merge_into_join.database_name.clone(), + table_schema: merge_into_join.table_schema.clone(), } } @@ -1031,7 +1032,7 @@ impl TableContext for QueryContext { fn get_merge_into_source_build_siphashkeys_with_id( &self, id: IndexType, - ) -> Vec<(String, (Buffer, Option))> { + ) -> Vec<(String, Arc>)> { let runtime_filters = self.shared.runtime_filters.read(); match runtime_filters.get(&id) { Some(v) => v.get_merge_into_source_build_siphashkeys(), @@ -1039,6 +1040,18 @@ impl TableContext for QueryContext { } } + fn get_merge_into_source_build_bloom_probe_keys(&self, id: usize) -> Vec { + let runtime_filters = self.shared.runtime_filters.read(); + match runtime_filters.get(&id) { + Some(v) => v + .get_merge_into_source_build_siphashkeys() + .iter() + .map(|key| key.0) + .collect(), + None => vec![], + } + } + fn get_inlist_runtime_filter_with_id(&self, id: IndexType) -> Vec> { 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 9597119884039..cabc09109cde0 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 @@ -833,7 +833,11 @@ impl TableContext for CtxDelegation { fn get_merge_into_source_build_siphashkeys_with_id( &self, id: usize, - ) -> Vec<(String, (Buffer, Option))> { + ) -> Vec<(String, Arc>)> { + todo!() + } + + fn get_merge_into_source_build_bloom_probe_keys(&self, id: usize) -> Vec { todo!() } diff --git a/src/query/sql/src/planner/binder/merge_into.rs b/src/query/sql/src/planner/binder/merge_into.rs index ba231338b0a99..37edbd357ccc2 100644 --- a/src/query/sql/src/planner/binder/merge_into.rs +++ b/src/query/sql/src/planner/binder/merge_into.rs @@ -488,6 +488,7 @@ impl Binder { row_id_index: column_binding.index, split_idx, can_try_update_column_only: self.can_try_update_column_only(&matched_clauses), + table_schema, }) } diff --git a/src/query/sql/src/planner/optimizer/optimizer.rs b/src/query/sql/src/planner/optimizer/optimizer.rs index 17827b0bfb0b2..62a40b35c8caa 100644 --- a/src/query/sql/src/planner/optimizer/optimizer.rs +++ b/src/query/sql/src/planner/optimizer/optimizer.rs @@ -427,6 +427,7 @@ fn optimize_merge_into(opt_ctx: OptimizerContext, plan: Box) -> Resul is_distributed: false, // we will set it after later optimization. merge_into_join_type: MergeIntoJoinType::Right, database_name: plan.database.clone(), + table_schema: Some(plan.table_schema.clone()), }) } @@ -517,6 +518,7 @@ fn optimize_merge_into(opt_ctx: OptimizerContext, plan: Box) -> Resul catalog_info: merge_into_join.catalog_info.clone(), table_info: merge_into_join.table_info.clone(), database_name: merge_into_join.database_name.clone(), + table_schema: merge_into_join.table_schema.clone(), }) } @@ -547,6 +549,7 @@ fn optimize_merge_into(opt_ctx: OptimizerContext, plan: Box) -> Resul catalog_info: merge_into_join.catalog_info.clone(), table_info: merge_into_join.table_info.clone(), database_name: merge_into_join.database_name, + table_schema: merge_into_join.table_schema.clone(), }) } Ok(Plan::MergeInto(Box::new(MergeInto { diff --git a/src/query/sql/src/planner/plans/merge_into.rs b/src/query/sql/src/planner/plans/merge_into.rs index 9986e5f561437..370a9c27ce2a6 100644 --- a/src/query/sql/src/planner/plans/merge_into.rs +++ b/src/query/sql/src/planner/plans/merge_into.rs @@ -14,6 +14,7 @@ use std::collections::HashMap; use std::collections::HashSet; +use std::sync::Arc; use databend_common_ast::ast::TableAlias; use databend_common_exception::ErrorCode; @@ -24,6 +25,7 @@ use databend_common_expression::DataField; use databend_common_expression::DataSchemaRef; use databend_common_expression::DataSchemaRefExt; use databend_common_expression::FieldIndex; +use databend_common_expression::TableSchema; use databend_common_meta_types::MetaId; use crate::binder::MergeIntoType; @@ -77,6 +79,8 @@ pub struct MergeInto { // `update *`` or `update set t1.a = t2.a ...`, the right expr on the `=` must be only a column, // we don't support complex expressions. pub can_try_update_column_only: bool, + + pub table_schema: Arc, } impl std::fmt::Debug for MergeInto { diff --git a/src/query/storages/fuse/src/operations/read/native_data_source_reader.rs b/src/query/storages/fuse/src/operations/read/native_data_source_reader.rs index 2fb17557863d3..3df8846bfbf9c 100644 --- a/src/query/storages/fuse/src/operations/read/native_data_source_reader.rs +++ b/src/query/storages/fuse/src/operations/read/native_data_source_reader.rs @@ -33,8 +33,8 @@ use databend_common_pipeline_sources::SyncSourcer; use databend_common_sql::IndexType; use log::debug; -use super::can_merge_into_target_build_bloom_filter; use super::native_data_source::NativeDataSource; +use super::util::build_merge_into_source_build_bloom_info; use super::util::MergeIntoSourceBuildBloomInfo; use crate::io::AggIndexReader; use crate::io::BlockReader; @@ -91,16 +91,11 @@ impl ReadNativeDataSource { virtual_reader, table_schema, table_index, - merge_into_source_build_bloom_info: MergeIntoSourceBuildBloomInfo { - can_do_merge_into_rumtime_filter_bloom: can_merge_into_target_build_bloom_filter( - ctx.clone(), - table_index, - )?, - segment_infos: Default::default(), - catalog_info: merge_into_join.catalog_info.clone(), - table_info: merge_into_join.table_info.clone(), - database_name: merge_into_join.database_name.clone(), - }, + merge_into_source_build_bloom_info: build_merge_into_source_build_bloom_info( + ctx, + table_index, + merge_into_join, + )?, }) } } @@ -135,16 +130,11 @@ impl ReadNativeDataSource { virtual_reader, table_schema, table_index, - merge_into_source_build_bloom_info: MergeIntoSourceBuildBloomInfo { - can_do_merge_into_rumtime_filter_bloom: can_merge_into_target_build_bloom_filter( - ctx.clone(), - table_index, - )?, - segment_infos: Default::default(), - catalog_info: merge_into_join.catalog_info.clone(), - table_info: merge_into_join.table_info.clone(), - database_name: merge_into_join.database_name.clone(), - }, + merge_into_source_build_bloom_info: build_merge_into_source_build_bloom_info( + ctx, + table_index, + merge_into_join, + )?, }))) } } 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 ac67c717a523b..c2af6bc955d5c 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 @@ -32,8 +32,8 @@ use databend_common_pipeline_sources::SyncSourcer; use databend_common_sql::IndexType; use log::debug; -use super::can_merge_into_target_build_bloom_filter; use super::parquet_data_source::ParquetDataSource; +use super::util::build_merge_into_source_build_bloom_info; use super::util::MergeIntoSourceBuildBloomInfo; use crate::fuse_part::FusePartInfo; use crate::io::AggIndexReader; @@ -95,14 +95,11 @@ impl ReadParquetDataSource { index_reader, virtual_reader, table_schema, - merge_into_source_build_bloom_info: MergeIntoSourceBuildBloomInfo { - can_do_merge_into_rumtime_filter_bloom: - can_merge_into_target_build_bloom_filter(ctx.clone(), table_index)?, - segment_infos: Default::default(), - catalog_info: merge_into_join.catalog_info.clone(), - table_info: merge_into_join.table_info.clone(), - database_name: merge_into_join.database_name.clone(), - }, + merge_into_source_build_bloom_info: build_merge_into_source_build_bloom_info( + ctx, + table_index, + merge_into_join, + )?, }) } else { let merge_into_join = ctx.get_merge_into_join(); @@ -121,14 +118,11 @@ impl ReadParquetDataSource { index_reader, virtual_reader, table_schema, - merge_into_source_build_bloom_info: MergeIntoSourceBuildBloomInfo { - can_do_merge_into_rumtime_filter_bloom: - can_merge_into_target_build_bloom_filter(ctx.clone(), table_index)?, - segment_infos: Default::default(), - catalog_info: merge_into_join.catalog_info.clone(), - table_info: merge_into_join.table_info.clone(), - database_name: merge_into_join.database_name.clone(), - }, + merge_into_source_build_bloom_info: build_merge_into_source_build_bloom_info( + ctx, + table_index, + merge_into_join, + )?, }))) } } diff --git a/src/query/storages/fuse/src/operations/read/runtime_filter_prunner.rs b/src/query/storages/fuse/src/operations/read/runtime_filter_prunner.rs index fcb7e94825dfd..1dfcc574badd9 100644 --- a/src/query/storages/fuse/src/operations/read/runtime_filter_prunner.rs +++ b/src/query/storages/fuse/src/operations/read/runtime_filter_prunner.rs @@ -182,6 +182,7 @@ pub fn runtime_filter_pruner( segment_idx, block_idx ); let block_meta = segment_info.blocks[block_idx].clone(); + // iterate every probe key name to do prune } Ok(false) diff --git a/src/query/storages/fuse/src/operations/read/util.rs b/src/query/storages/fuse/src/operations/read/util.rs index dcb876da7677e..a6fafb7d407c2 100644 --- a/src/query/storages/fuse/src/operations/read/util.rs +++ b/src/query/storages/fuse/src/operations/read/util.rs @@ -15,13 +15,16 @@ use std::collections::HashMap; use std::sync::Arc; +use databend_common_catalog::merge_into_join::MergeIntoJoin; use databend_common_catalog::merge_into_join::MergeIntoJoinType; use databend_common_catalog::plan::gen_mutation_stream_meta; use databend_common_catalog::plan::InternalColumnMeta; use databend_common_catalog::table_context::TableContext; +use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::BlockMetaInfoPtr; use databend_common_expression::DataBlock; +use databend_common_expression::FieldIndex; use databend_common_expression::Scalar; use databend_common_meta_app::schema::CatalogInfo; use databend_common_meta_app::schema::TableInfo; @@ -37,6 +40,7 @@ pub struct MergeIntoSourceBuildBloomInfo { pub table_info: Option, pub catalog_info: Option, pub database_name: String, + pub bloom_indexes: Vec, } pub fn need_reserve_block_info(ctx: Arc, table_idx: usize) -> (bool, bool) { @@ -68,6 +72,7 @@ pub fn can_merge_into_target_build_bloom_filter( assert!(merge_into_join.database_name.as_str() != ""); assert!(matches!(merge_into_join.catalog_info, Some(_))); assert!(matches!(merge_into_join.table_info, Some(_))); + assert!(matches!(merge_into_join.table_schema, Some(_))); } Ok(enabled) } @@ -116,3 +121,36 @@ pub(crate) fn add_data_block_meta( } block.add_meta(meta) } + +pub fn build_merge_into_source_build_bloom_info( + ctx: Arc, + table_index: usize, + merge_into_join: MergeIntoJoin, +) -> Result { + let enabled_bloom_filter = can_merge_into_target_build_bloom_filter(ctx.clone(), table_index)?; + let bloom_indexes = if enabled_bloom_filter { + ctx.get_merge_into_source_build_bloom_probe_keys(table_index) + .iter() + .try_fold(Vec::new(), |mut acc, probe_key_name| { + let table_schema = merge_into_join.table_schema.as_ref().ok_or_else(|| { + ErrorCode::Internal( + "can't get merge into target table schema when build bloom info, it's a bug", + ) + })?; + let index = table_schema.index_of(probe_key_name)?; + acc.push(index); + Ok::<_, ErrorCode>(acc) + })? + } else { + vec![] + }; + + Ok(MergeIntoSourceBuildBloomInfo { + can_do_merge_into_rumtime_filter_bloom: enabled_bloom_filter, + segment_infos: Default::default(), + catalog_info: merge_into_join.catalog_info.clone(), + table_info: merge_into_join.table_info.clone(), + database_name: merge_into_join.database_name.clone(), + bloom_indexes, + }) +} From 3778c67f05c3ef24800147f5ad9b285da81dbdc7 Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Thu, 21 Mar 2024 01:18:45 +0800 Subject: [PATCH 06/24] fix typos --- src/query/sql/src/planner/optimizer/optimizer.rs | 2 +- .../fuse/src/operations/read/native_data_source_reader.rs | 4 ++-- .../fuse/src/operations/read/parquet_data_source_reader.rs | 4 ++-- src/query/storages/fuse/src/operations/read/util.rs | 4 ++-- 4 files changed, 7 insertions(+), 7 deletions(-) diff --git a/src/query/sql/src/planner/optimizer/optimizer.rs b/src/query/sql/src/planner/optimizer/optimizer.rs index 62a40b35c8caa..df1191b0ffa33 100644 --- a/src/query/sql/src/planner/optimizer/optimizer.rs +++ b/src/query/sql/src/planner/optimizer/optimizer.rs @@ -420,7 +420,7 @@ fn optimize_merge_into(opt_ctx: OptimizerContext, plan: Box) -> Resul assert!(matches!(merge_into_join.table_info, None)); assert!(merge_into_join.database_name.as_str() == ""); opt_ctx.table_ctx.set_merge_into_join(MergeIntoJoin { - // we will set catalog_info and table_info in `intepreter_merge_into` + // we will set catalog_info and table_info in `interpreter_merge_into` catalog_info: None, table_info: None, target_tbl_idx: plan.target_table_idx, diff --git a/src/query/storages/fuse/src/operations/read/native_data_source_reader.rs b/src/query/storages/fuse/src/operations/read/native_data_source_reader.rs index 3df8846bfbf9c..05fb70b567051 100644 --- a/src/query/storages/fuse/src/operations/read/native_data_source_reader.rs +++ b/src/query/storages/fuse/src/operations/read/native_data_source_reader.rs @@ -161,7 +161,7 @@ impl SyncSource for ReadNativeDataSource { &filters, &self.func_ctx, self.merge_into_source_build_bloom_info - .can_do_merge_into_rumtime_filter_bloom, + .can_do_merge_into_runtime_filter_bloom, self.partitions.ctx.clone(), self.table_index, &mut self.merge_into_source_build_bloom_info, @@ -273,7 +273,7 @@ impl Processor for ReadNativeDataSource { &filters, &self.func_ctx, self.merge_into_source_build_bloom_info - .can_do_merge_into_rumtime_filter_bloom, + .can_do_merge_into_runtime_filter_bloom, self.partitions.ctx.clone(), self.table_index, &mut self.merge_into_source_build_bloom_info, 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 c2af6bc955d5c..20e65f326563d 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 @@ -150,7 +150,7 @@ impl SyncSource for ReadParquetDataSource { &filters, &self.func_ctx, self.merge_into_source_build_bloom_info - .can_do_merge_into_rumtime_filter_bloom, + .can_do_merge_into_runtime_filter_bloom, self.partitions.ctx.clone(), self.table_index, &mut self.merge_into_source_build_bloom_info, @@ -271,7 +271,7 @@ impl Processor for ReadParquetDataSource { &filters, &self.func_ctx, self.merge_into_source_build_bloom_info - .can_do_merge_into_rumtime_filter_bloom, + .can_do_merge_into_runtime_filter_bloom, self.partitions.ctx.clone(), self.table_index, &mut self.merge_into_source_build_bloom_info, diff --git a/src/query/storages/fuse/src/operations/read/util.rs b/src/query/storages/fuse/src/operations/read/util.rs index a6fafb7d407c2..99dfc98f5c1d9 100644 --- a/src/query/storages/fuse/src/operations/read/util.rs +++ b/src/query/storages/fuse/src/operations/read/util.rs @@ -35,7 +35,7 @@ use crate::operations::SegmentIndex; use crate::FusePartInfo; pub struct MergeIntoSourceBuildBloomInfo { - pub can_do_merge_into_rumtime_filter_bloom: bool, + pub can_do_merge_into_runtime_filter_bloom: bool, pub segment_infos: HashMap, pub table_info: Option, pub catalog_info: Option, @@ -146,7 +146,7 @@ pub fn build_merge_into_source_build_bloom_info( }; Ok(MergeIntoSourceBuildBloomInfo { - can_do_merge_into_rumtime_filter_bloom: enabled_bloom_filter, + can_do_merge_into_runtime_filter_bloom: enabled_bloom_filter, segment_infos: Default::default(), catalog_info: merge_into_join.catalog_info.clone(), table_info: merge_into_join.table_info.clone(), From 69a5675f4e1a3e5a630991e7c529a39e3165727a Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Thu, 21 Mar 2024 04:23:04 +0800 Subject: [PATCH 07/24] add runtime filter,need tests --- src/query/catalog/src/runtime_filter_info.rs | 13 +- src/query/catalog/src/table_context.rs | 5 +- .../interpreters/interpreter_merge_into.rs | 2 + .../hash_join/hash_join_build_state.rs | 17 +- src/query/service/src/sessions/query_ctx.rs | 20 +- .../tests/it/sql/exec/get_table_bind_test.rs | 11 +- .../it/storages/fuse/operations/commit.rs | 27 +++ .../sql/src/planner/optimizer/optimizer.rs | 6 +- .../read/native_data_source_reader.rs | 1 - .../operations/read/runtime_filter_prunner.rs | 160 +++++++++------- .../storages/fuse/src/operations/read/util.rs | 21 +- .../mutator/merge_into_mutator.rs | 181 +++++++++--------- .../operations/replace_into/mutator/mod.rs | 2 + 13 files changed, 265 insertions(+), 201 deletions(-) diff --git a/src/query/catalog/src/runtime_filter_info.rs b/src/query/catalog/src/runtime_filter_info.rs index 7a674267ee20d..549049d343a34 100644 --- a/src/query/catalog/src/runtime_filter_info.rs +++ b/src/query/catalog/src/runtime_filter_info.rs @@ -15,14 +15,17 @@ use std::sync::Arc; use databend_common_expression::Expr; +use parking_lot::RwLock; use xorf::BinaryFuse16; +pub type MergeIntoSourceBuildSiphashkeys = (Vec, Arc>>>); + #[derive(Clone, Debug, Default)] pub struct RuntimeFilterInfo { inlist: Vec>, min_max: Vec>, bloom: Vec<(String, BinaryFuse16)>, - siphashes: Vec<(String, Arc>)>, + siphashes: MergeIntoSourceBuildSiphashkeys, } impl RuntimeFilterInfo { @@ -34,12 +37,14 @@ impl RuntimeFilterInfo { self.bloom.push(bloom); } - pub fn get_merge_into_source_build_siphashkeys(&self) -> Vec<(String, Arc>)> { + pub fn get_merge_into_source_build_siphashkeys(&self) -> MergeIntoSourceBuildSiphashkeys { self.siphashes.clone() } - pub fn add_merge_into_source_build_siphashkeys(&mut self, digests: (String, Arc>)) { - self.siphashes.push(digests); + pub fn add_merge_into_source_build_siphashkeys(&mut self, digests: (String, Vec)) { + self.siphashes.0.push(digests.0); + let mut borrow_hash_keys = self.siphashes.1.write(); + borrow_hash_keys.push(digests.1) } pub fn add_min_max(&mut self, expr: Expr) { diff --git a/src/query/catalog/src/table_context.rs b/src/query/catalog/src/table_context.rs index 2bcf90d471f15..01b0b73c4ece1 100644 --- a/src/query/catalog/src/table_context.rs +++ b/src/query/catalog/src/table_context.rs @@ -22,8 +22,6 @@ use std::sync::Arc; use std::time::SystemTime; use dashmap::DashMap; -use databend_common_arrow::arrow::bitmap::Bitmap; -use databend_common_arrow::arrow::buffer::Buffer; use databend_common_base::base::Progress; use databend_common_base::base::ProgressValues; use databend_common_base::runtime::profile::Profile; @@ -63,6 +61,7 @@ use crate::plan::DataSourcePlan; use crate::plan::PartInfoPtr; use crate::plan::Partitions; use crate::query_kind::QueryKind; +use crate::runtime_filter_info::MergeIntoSourceBuildSiphashkeys; use crate::runtime_filter_info::RuntimeFilterInfo; use crate::statistics::data_cache_statistics::DataCacheMetrics; use crate::table::Table; @@ -278,7 +277,7 @@ pub trait TableContext: Send + Sync { fn get_merge_into_source_build_siphashkeys_with_id( &self, id: usize, - ) -> Vec<(String, Arc>)>; + ) -> Option; fn get_merge_into_source_build_bloom_probe_keys(&self, id: usize) -> Vec; diff --git a/src/query/service/src/interpreters/interpreter_merge_into.rs b/src/query/service/src/interpreters/interpreter_merge_into.rs index f0a691e94e18e..37d485b1c4ae3 100644 --- a/src/query/service/src/interpreters/interpreter_merge_into.rs +++ b/src/query/service/src/interpreters/interpreter_merge_into.rs @@ -203,6 +203,7 @@ impl MergeIntoInterpreter { table_info: merge_into_join.table_info.clone(), catalog_info: merge_into_join.catalog_info.clone(), database_name: merge_into_join.database_name.clone(), + table_schema: merge_into_join.table_schema.clone(), }); } } @@ -305,6 +306,7 @@ impl MergeIntoInterpreter { table_info: Some(table_info.clone()), catalog_info: Some(catalog_.info()), database_name: merge_into_join.database_name.clone(), + table_schema: merge_into_join.table_schema.clone(), }) } // merge_into_source is used to recv join's datablocks and split them into macthed and not matched 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 e956b72104516..de3241147528c 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 @@ -21,7 +21,6 @@ use std::sync::atomic::Ordering; use std::sync::Arc; use databend_common_arrow::arrow::bitmap::Bitmap; -use databend_common_arrow::arrow::buffer::Buffer; use databend_common_base::base::tokio::sync::Barrier; use databend_common_catalog::merge_into_join::MergeIntoJoinType; use databend_common_catalog::runtime_filter_info::RuntimeFilterInfo; @@ -964,7 +963,7 @@ impl HashJoinBuildState { return Ok(()); } let build_key_column = Column::concat_columns(columns.into_iter())?; - // mabye there will be null values here, so we use nullable column, the null value will be treat as default + // maybe there will be null values here, so we use nullable column, the null value will be treat as default // value for the sepcified type, like String -> "", int -> 0. so we need to remove the null hash values here. let (hashes, bitmap_op) = BloomIndex::calculate_nullable_column_digest( &self.func_ctx, @@ -976,7 +975,7 @@ impl HashJoinBuildState { let digests = if bitmap.unset_bits() == 0 { hashes.to_vec() } else { - let new_hashes = Vec::with_capacity(bitmap.len()); + let mut new_hashes = Vec::with_capacity(bitmap.len()); assert_eq!(hashes.len(), bitmap.len()); for row_idx in 0..bitmap.len() { if bitmap.get_bit(row_idx) { @@ -986,16 +985,12 @@ impl HashJoinBuildState { new_hashes.to_vec() }; // id is probe key name - runtime_filter.add_merge_into_source_build_siphashkeys(( - id.to_string(), - Arc::new(digests), - )); + runtime_filter + .add_merge_into_source_build_siphashkeys((id.to_string(), digests)); } else { // id is probe key name - runtime_filter.add_merge_into_source_build_siphashkeys(( - id.to_string(), - Arc::new(hashes), - )); + runtime_filter + .add_merge_into_source_build_siphashkeys((id.to_string(), hashes.to_vec())); } } } diff --git a/src/query/service/src/sessions/query_ctx.rs b/src/query/service/src/sessions/query_ctx.rs index 263e0d74d7061..052e0d151d35f 100644 --- a/src/query/service/src/sessions/query_ctx.rs +++ b/src/query/service/src/sessions/query_ctx.rs @@ -32,8 +32,6 @@ use std::time::UNIX_EPOCH; use chrono_tz::Tz; use dashmap::mapref::multiple::RefMulti; use dashmap::DashMap; -use databend_common_arrow::arrow::bitmap::Bitmap; -use databend_common_arrow::arrow::buffer::Buffer; use databend_common_base::base::tokio::task::JoinHandle; use databend_common_base::base::Progress; use databend_common_base::base::ProgressValues; @@ -48,6 +46,7 @@ use databend_common_catalog::plan::PartInfoPtr; use databend_common_catalog::plan::Partitions; use databend_common_catalog::plan::StageTableInfo; use databend_common_catalog::query_kind::QueryKind; +use databend_common_catalog::runtime_filter_info::MergeIntoSourceBuildSiphashkeys; use databend_common_catalog::runtime_filter_info::RuntimeFilterInfo; use databend_common_catalog::statistics::data_cache_statistics::DataCacheMetrics; use databend_common_catalog::table_args::TableArgs; @@ -1031,23 +1030,18 @@ impl TableContext for QueryContext { fn get_merge_into_source_build_siphashkeys_with_id( &self, - id: IndexType, - ) -> Vec<(String, Arc>)> { + id: usize, + ) -> Option { let runtime_filters = self.shared.runtime_filters.read(); - match runtime_filters.get(&id) { - Some(v) => v.get_merge_into_source_build_siphashkeys(), - None => vec![], - } + runtime_filters + .get(&id) + .map(|v| v.get_merge_into_source_build_siphashkeys()) } fn get_merge_into_source_build_bloom_probe_keys(&self, id: usize) -> Vec { let runtime_filters = self.shared.runtime_filters.read(); match runtime_filters.get(&id) { - Some(v) => v - .get_merge_into_source_build_siphashkeys() - .iter() - .map(|key| key.0) - .collect(), + Some(v) => v.get_merge_into_source_build_siphashkeys().0.clone(), None => vec![], } } 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 cabc09109cde0..cf74b58ae26ac 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 @@ -18,8 +18,6 @@ use std::sync::atomic::AtomicUsize; use std::sync::Arc; use dashmap::DashMap; -use databend_common_arrow::arrow::bitmap::Bitmap; -use databend_common_arrow::arrow::buffer::Buffer; use databend_common_base::base::tokio; use databend_common_base::base::Progress; use databend_common_base::base::ProgressValues; @@ -33,6 +31,7 @@ use databend_common_catalog::plan::DataSourcePlan; use databend_common_catalog::plan::PartInfoPtr; use databend_common_catalog::plan::Partitions; use databend_common_catalog::query_kind::QueryKind; +use databend_common_catalog::runtime_filter_info::MergeIntoSourceBuildSiphashkeys; use databend_common_catalog::runtime_filter_info::RuntimeFilterInfo; use databend_common_catalog::statistics::data_cache_statistics::DataCacheMetrics; use databend_common_catalog::table::Table; @@ -832,16 +831,16 @@ impl TableContext for CtxDelegation { fn get_merge_into_source_build_siphashkeys_with_id( &self, - id: usize, - ) -> Vec<(String, Arc>)> { + _id: usize, + ) -> Option { todo!() } - fn get_merge_into_source_build_bloom_probe_keys(&self, id: usize) -> Vec { + fn get_merge_into_source_build_bloom_probe_keys(&self, _id: usize) -> Vec { todo!() } - fn set_merge_into_source_build_segments(&self, segments: MergeIntoSourceBuildSegments) { + fn set_merge_into_source_build_segments(&self, _segments: MergeIntoSourceBuildSegments) { 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 222801ff0f4cb..c77d9d57f3443 100644 --- a/src/query/service/tests/it/storages/fuse/operations/commit.rs +++ b/src/query/service/tests/it/storages/fuse/operations/commit.rs @@ -375,6 +375,33 @@ impl TableContext for CtxDelegation { self } + fn get_merge_into_source_build_bloom_probe_keys(&self, _: usize) -> Vec { + todo!() + } + + fn set_merge_into_source_build_segments( + &self, + _: Arc>, + ) { + todo!() + } + + fn get_merge_into_source_build_siphashkeys_with_id( + &self, + _: usize, + ) -> std::option::Option<( + Vec, + Arc>>>, + )> { + todo!() + } + + fn get_merge_into_source_build_segments( + &self, + ) -> Arc> { + todo!() + } + fn build_table_from_source_plan(&self, _plan: &DataSourcePlan) -> Result> { todo!() } diff --git a/src/query/sql/src/planner/optimizer/optimizer.rs b/src/query/sql/src/planner/optimizer/optimizer.rs index df1191b0ffa33..6cbc677926ece 100644 --- a/src/query/sql/src/planner/optimizer/optimizer.rs +++ b/src/query/sql/src/planner/optimizer/optimizer.rs @@ -416,8 +416,8 @@ fn optimize_merge_into(opt_ctx: OptimizerContext, plan: Box) -> Resul { let merge_into_join = opt_ctx.table_ctx.get_merge_into_join(); // this is the first time set, so it must be none, and we will set it in `interpreter_merge_into` - assert!(matches!(merge_into_join.catalog_info, None)); - assert!(matches!(merge_into_join.table_info, None)); + assert!(merge_into_join.catalog_info.is_none()); + assert!(merge_into_join.table_info.is_none()); assert!(merge_into_join.database_name.as_str() == ""); opt_ctx.table_ctx.set_merge_into_join(MergeIntoJoin { // we will set catalog_info and table_info in `interpreter_merge_into` @@ -573,7 +573,7 @@ fn try_to_change_as_broadcast_join( 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() { - let mut join: Join = merge_into_join_sexpr.child(0)?.plan().clone().try_into()?; + let join: Join = merge_into_join_sexpr.child(0)?.plan().clone().try_into()?; let join_s_expr = merge_into_join_sexpr .child(0)? diff --git a/src/query/storages/fuse/src/operations/read/native_data_source_reader.rs b/src/query/storages/fuse/src/operations/read/native_data_source_reader.rs index 05fb70b567051..b058668dbd30e 100644 --- a/src/query/storages/fuse/src/operations/read/native_data_source_reader.rs +++ b/src/query/storages/fuse/src/operations/read/native_data_source_reader.rs @@ -15,7 +15,6 @@ use std::any::Any; use std::sync::Arc; -use databend_common_catalog::merge_into_join; use databend_common_catalog::plan::PartInfoPtr; use databend_common_catalog::plan::StealablePartitions; use databend_common_catalog::table_context::TableContext; diff --git a/src/query/storages/fuse/src/operations/read/runtime_filter_prunner.rs b/src/query/storages/fuse/src/operations/read/runtime_filter_prunner.rs index 1dfcc574badd9..3d8c608c06ce4 100644 --- a/src/query/storages/fuse/src/operations/read/runtime_filter_prunner.rs +++ b/src/query/storages/fuse/src/operations/read/runtime_filter_prunner.rs @@ -48,7 +48,8 @@ use xorf::Filter; use super::util::MergeIntoSourceBuildBloomInfo; use crate::io::MetaReaders; -use crate::operations::SegmentIndex; +use crate::operations::load_bloom_filter; +use crate::operations::try_prune_use_bloom_filter; use crate::FusePartInfo; use crate::FuseTable; @@ -115,77 +116,104 @@ pub fn runtime_filter_pruner( // if we can't pruned this block, we can try get siphashkeys if this is a merge into source build if can_do_merge_into_target_build_bloom_filter { - assert!(matches!(part.block_meta_index(), Some(_))); - let block_meta_index = part.block_meta_index().unwrap(); - let hash_keys = ctx.get_merge_into_source_build_siphashkeys_with_id(id); - let segment_idx = block_meta_index.segment_idx; - let block_idx = block_meta_index.block_idx; - let target_table_segments = ctx.get_merge_into_source_build_segments(); - if let Entry::Vacant(e) = merge_into_source_build_bloom_info - .segment_infos - .entry(segment_idx) - { - let (_,(path, ver)) = target_table_segments.get(segment_idx).ok_or_else(|| { - return ErrorCode::Internal(format!( + try_prune_merge_into_target_table(ctx.clone(), part, merge_into_source_build_bloom_info, id) + } else { + Ok(false) + } +} + +pub(crate) fn try_prune_merge_into_target_table( + ctx: Arc, + part: &FusePartInfo, + merge_into_source_build_bloom_info: &mut MergeIntoSourceBuildBloomInfo, + id: usize, +) -> Result { + assert!(part.block_meta_index().is_some()); + let block_meta_index = part.block_meta_index().unwrap(); + + let segment_idx = block_meta_index.segment_idx; + let block_idx = block_meta_index.block_idx; + let target_table_segments = ctx.get_merge_into_source_build_segments(); + let catalog_info = merge_into_source_build_bloom_info + .catalog_info + .as_ref() + .unwrap(); + let table_info = merge_into_source_build_bloom_info + .table_info + .as_ref() + .unwrap(); + let table = block_on(async { + ctx.get_table( + catalog_info.catalog_name(), + &merge_into_source_build_bloom_info.database_name, + &table_info.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(), + )) + })?; + if let Entry::Vacant(e) = merge_into_source_build_bloom_info + .segment_infos + .entry(segment_idx) + { + let (_,(path, ver)) = target_table_segments.get(segment_idx).ok_or_else(|| { + ErrorCode::Internal(format!( "unexpected, segment (idx {}) not found, during do merge into source build bloom filter", segment_idx )) })?; - let load_param = LoadParams { - location: path.clone(), - len_hint: None, - ver: *ver, - put_cache: true, - }; - let catalog_info = merge_into_source_build_bloom_info - .catalog_info - .as_ref() - .unwrap(); - let table_info = merge_into_source_build_bloom_info - .table_info - .as_ref() - .unwrap(); - let table = block_on(async { - ctx.get_table( - catalog_info.catalog_name(), - &merge_into_source_build_bloom_info.database_name, - &table_info.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 target_table_schema = table.schema_with_stream(); - let data_accessor = fuse_table.get_operator(); - let segment_reader = MetaReaders::segment_info_reader( - data_accessor.clone(), - target_table_schema.clone(), - ); - let compact_segment_info = block_on(async { segment_reader.read(&load_param).await })?; - let segment_info: SegmentInfo = compact_segment_info.try_into()?; - e.insert(segment_info); - } - // load bloom filter - let segment_info = merge_into_source_build_bloom_info - .segment_infos - .get(&segment_idx) - .unwrap(); - assert!(block_idx < segment_info.blocks.len()); - info!( - "merge into source build runtime bloom filter: segment_idx:{},blk_idx:{}", - segment_idx, block_idx - ); - let block_meta = segment_info.blocks[block_idx].clone(); - // iterate every probe key name to do prune + let load_param = LoadParams { + location: path.clone(), + len_hint: None, + ver: *ver, + put_cache: true, + }; + let target_table_schema = table.schema_with_stream(); + let data_accessor = fuse_table.get_operator(); + let segment_reader = + MetaReaders::segment_info_reader(data_accessor.clone(), target_table_schema.clone()); + let compact_segment_info = block_on(async { segment_reader.read(&load_param).await })?; + let segment_info: SegmentInfo = compact_segment_info.try_into()?; + e.insert(segment_info); + } + // load bloom filter + let segment_info = merge_into_source_build_bloom_info + .segment_infos + .get(&segment_idx) + .unwrap(); + assert!(block_idx < segment_info.blocks.len()); + info!( + "merge into source build runtime bloom filter: segment_idx:{},blk_idx:{}", + segment_idx, block_idx + ); + let block_meta = segment_info.blocks[block_idx].clone(); + if let Some(index_location) = block_meta.bloom_filter_index_location.as_ref() { + let filters = block_on(async { + load_bloom_filter( + fuse_table.get_operator(), + &merge_into_source_build_bloom_info.bloom_fields, + index_location, + block_meta.bloom_filter_index_size, + &merge_into_source_build_bloom_info.bloom_indexes, + ) + .await + }); + Ok(try_prune_use_bloom_filter( + filters, + &ctx.get_merge_into_source_build_siphashkeys_with_id(id) + .unwrap() + .1 + .read(), + )) + } else { + Ok(false) } - - Ok(false) } pub(crate) fn update_bitmap_with_bloom_filter( diff --git a/src/query/storages/fuse/src/operations/read/util.rs b/src/query/storages/fuse/src/operations/read/util.rs index 99dfc98f5c1d9..c2d9324990b7a 100644 --- a/src/query/storages/fuse/src/operations/read/util.rs +++ b/src/query/storages/fuse/src/operations/read/util.rs @@ -28,6 +28,7 @@ use databend_common_expression::FieldIndex; use databend_common_expression::Scalar; use databend_common_meta_app::schema::CatalogInfo; use databend_common_meta_app::schema::TableInfo; +use databend_common_sql::executor::physical_plans::OnConflictField; use databend_storages_common_table_meta::meta::SegmentInfo; use crate::operations::BlockMetaIndex; @@ -41,6 +42,7 @@ pub struct MergeIntoSourceBuildBloomInfo { pub catalog_info: Option, pub database_name: String, pub bloom_indexes: Vec, + pub bloom_fields: Vec, } pub fn need_reserve_block_info(ctx: Arc, table_idx: usize) -> (bool, bool) { @@ -70,9 +72,9 @@ pub fn can_merge_into_target_build_bloom_filter( .get_enable_merge_into_source_build_bloom()? ); assert!(merge_into_join.database_name.as_str() != ""); - assert!(matches!(merge_into_join.catalog_info, Some(_))); - assert!(matches!(merge_into_join.table_info, Some(_))); - assert!(matches!(merge_into_join.table_schema, Some(_))); + assert!(merge_into_join.catalog_info.is_some()); + assert!(merge_into_join.table_info.is_some()); + assert!(merge_into_join.table_schema.is_some()); } Ok(enabled) } @@ -128,23 +130,25 @@ pub fn build_merge_into_source_build_bloom_info( merge_into_join: MergeIntoJoin, ) -> Result { let enabled_bloom_filter = can_merge_into_target_build_bloom_filter(ctx.clone(), table_index)?; - let bloom_indexes = if enabled_bloom_filter { + + let (bloom_indexes, bloom_fields) = if enabled_bloom_filter { ctx.get_merge_into_source_build_bloom_probe_keys(table_index) .iter() - .try_fold(Vec::new(), |mut acc, probe_key_name| { + .try_fold((Vec::new(),Vec::new()), |mut acc, probe_key_name| { let table_schema = merge_into_join.table_schema.as_ref().ok_or_else(|| { ErrorCode::Internal( "can't get merge into target table schema when build bloom info, it's a bug", ) })?; let index = table_schema.index_of(probe_key_name)?; - acc.push(index); + acc.0.push(index); + acc.1.push(OnConflictField { table_field: table_schema.field(index).clone(), field_index: index }); Ok::<_, ErrorCode>(acc) })? } else { - vec![] + (vec![], vec![]) }; - + assert_eq!(bloom_fields.len(), bloom_indexes.len()); Ok(MergeIntoSourceBuildBloomInfo { can_do_merge_into_runtime_filter_bloom: enabled_bloom_filter, segment_infos: Default::default(), @@ -152,5 +156,6 @@ pub fn build_merge_into_source_build_bloom_info( table_info: merge_into_join.table_info.clone(), database_name: merge_into_join.database_name.clone(), bloom_indexes, + bloom_fields, }) } 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 46461f50b8b81..c5d114719ec20 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 @@ -683,107 +683,116 @@ impl AggregationContext { return false; } if let Some(loc) = &block_meta.bloom_filter_index_location { - match self - .load_bloom_filter( - loc, - block_meta.bloom_filter_index_size, - bloom_on_conflict_field_index, - ) - .await - { - Ok(filters) => { - // the caller ensures that the input_hashes is not empty - let row_count = input_hashes[0].len(); - - // let assume that the target block is prunable - let mut block_pruned = true; - for row in 0..row_count { - // for each row, by default, assume that columns of this row do have conflict with the target block. - let mut row_not_prunable = true; - for (col_idx, col_hash) in input_hashes.iter().enumerate() { - // For each column of current row, check if the corresponding bloom - // filter contains the digest of the column. - // - // Any one of the columns NOT contains by the corresponding bloom filter, - // indicates that the row is prunable(thus, we do not stop on the first column that - // the bloom filter contains). - - // - if bloom filter presents, check if the column is contained - // - if bloom filter absents, do nothing(since by default, we assume that the row is not-prunable) - if let Some(col_filter) = &filters[col_idx] { - let hash = col_hash[row]; - if hash == 0 || !col_filter.contains_digest(hash) { - // - hash == 0 indicates that the column value is null, which equals nothing. - // - NOT `contains_digest`, indicates that this column of row does not match - row_not_prunable = false; - // if one column not match, we do not need to check other columns - break; - } - } - } - if row_not_prunable { - // any row not prunable indicates that the target block is not prunable - block_pruned = false; + let filters = load_bloom_filter( + self.data_accessor.clone(), + &self.on_conflict_fields, + loc, + block_meta.bloom_filter_index_size, + bloom_on_conflict_field_index, + ) + .await; + try_prune_use_bloom_filter(filters, input_hashes) + } else { + // no bloom filter, no pruning + false + } + } +} + +pub fn try_prune_use_bloom_filter( + filters: Result>>>, + input_hashes: &[Vec], +) -> bool { + match filters { + Ok(filters) => { + // the caller ensures that the input_hashes is not empty + let row_count = input_hashes[0].len(); + + // let assume that the target block is prunable + let mut block_pruned = true; + for row in 0..row_count { + // for each row, by default, assume that columns of this row do have conflict with the target block. + let mut row_not_prunable = true; + for (col_idx, col_hash) in input_hashes.iter().enumerate() { + // For each column of current row, check if the corresponding bloom + // filter contains the digest of the column. + // + // Any one of the columns NOT contains by the corresponding bloom filter, + // indicates that the row is prunable(thus, we do not stop on the first column that + // the bloom filter contains). + + // - if bloom filter presents, check if the column is contained + // - if bloom filter absents, do nothing(since by default, we assume that the row is not-prunable) + if let Some(col_filter) = &filters[col_idx] { + let hash = col_hash[row]; + if hash == 0 || !col_filter.contains_digest(hash) { + // - hash == 0 indicates that the column value is null, which equals nothing. + // - NOT `contains_digest`, indicates that this column of row does not match + row_not_prunable = false; + // if one column not match, we do not need to check other columns break; } } - block_pruned } - Err(e) => { - // broken index should not stop us: - warn!("failed to build bloom index column name: {}", e); - // failed to load bloom filter, do not prune - false + if row_not_prunable { + // any row not prunable indicates that the target block is not prunable + block_pruned = false; + break; } } - } else { - // no bloom filter, no pruning + block_pruned + } + Err(e) => { + // broken index should not stop us: + warn!("failed to build bloom index column name: {}", e); + // failed to load bloom filter, do not prune false } } +} - async fn load_bloom_filter( - &self, - location: &Location, - index_len: u64, - bloom_on_conflict_field_index: &[FieldIndex], - ) -> Result>>> { - // different block may have different version of bloom filter index - let mut col_names = Vec::with_capacity(bloom_on_conflict_field_index.len()); - - for idx in bloom_on_conflict_field_index { - let bloom_column_name = BloomIndex::build_filter_column_name( - location.1, - &self.on_conflict_fields[*idx].table_field, - )?; - col_names.push(bloom_column_name); - } +pub async fn load_bloom_filter( + data_accessor: Operator, + on_conflict_fields: &[OnConflictField], + location: &Location, + index_len: u64, + bloom_on_conflict_field_index: &[FieldIndex], +) -> Result>>> { + // different block may have different version of bloom filter index + let mut col_names = Vec::with_capacity(bloom_on_conflict_field_index.len()); + + for idx in bloom_on_conflict_field_index { + let bloom_column_name = BloomIndex::build_filter_column_name( + location.1, + &on_conflict_fields[*idx].table_field, + )?; + col_names.push(bloom_column_name); + } - // using load_bloom_filter_by_columns is attractive, - // but it do not care about the version of the bloom filter index - let block_filter = location - .read_block_filter(self.data_accessor.clone(), &col_names, index_len) - .await?; + // using load_bloom_filter_by_columns is attractive, + // but it do not care about the version of the bloom filter index + let block_filter = location + .read_block_filter(data_accessor.clone(), &col_names, index_len) + .await?; - // reorder the filter according to the order of bloom_on_conflict_field - let mut filters = Vec::with_capacity(bloom_on_conflict_field_index.len()); - for filter_col_name in &col_names { - match block_filter.filter_schema.index_of(filter_col_name) { - Ok(idx) => { - filters.push(Some(block_filter.filters[idx].clone())); - } - Err(_) => { - info!( - "bloom filter column {} not found for block {}", - filter_col_name, location.0 - ); - filters.push(None); - } + // reorder the filter according to the order of bloom_on_conflict_field + let mut filters = Vec::with_capacity(bloom_on_conflict_field_index.len()); + for filter_col_name in &col_names { + match block_filter.filter_schema.index_of(filter_col_name) { + Ok(idx) => { + filters.push(Some(block_filter.filters[idx].clone())); + } + Err(_) => { + info!( + "bloom filter column {} not found for block {}", + filter_col_name, location.0 + ); + filters.push(None); } } - - Ok(filters) } + + Ok(filters) } #[cfg(test)] diff --git a/src/query/storages/fuse/src/operations/replace_into/mutator/mod.rs b/src/query/storages/fuse/src/operations/replace_into/mutator/mod.rs index 1763ea1fb74ba..06b99c7d760df 100644 --- a/src/query/storages/fuse/src/operations/replace_into/mutator/mod.rs +++ b/src/query/storages/fuse/src/operations/replace_into/mutator/mod.rs @@ -20,5 +20,7 @@ mod mutator_replace_into; pub use column_hash::row_hash_of_columns; pub use deletion_accumulator::BlockDeletionKeys; pub use deletion_accumulator::DeletionAccumulator; +pub use merge_into_mutator::load_bloom_filter; +pub use merge_into_mutator::try_prune_use_bloom_filter; pub use merge_into_mutator::MergeIntoOperationAggregator; pub use mutator_replace_into::ReplaceIntoMutator; From 60e963eea65e47ec27ddc0ca6a25810a7b14a4ac Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Thu, 21 Mar 2024 04:25:19 +0800 Subject: [PATCH 08/24] fix lint --- .../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 de3241147528c..c55ed5dec02f9 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 @@ -933,7 +933,7 @@ impl HashJoinBuildState { // gives `t2` as source build side, we can build source join keys `siphashes`, that's because we use // siphash to build target table's bloom index block. // in this way, we can avoid current `runtime_filter()` func's performance cost, especially for large - // target table case, the `runtime_filter()`'s cost is even higer than disable `runtime_filter()`. + // target table case, the `runtime_filter()`'s cost is even higher than disable `runtime_filter()`. // However, for `build_runtime_filter_siphashes()` usages, we currently just used for merge into, // we doesn't support join query, and it's only for `source build` cases. In fact, source build is the // main case in most time. @@ -964,7 +964,7 @@ impl HashJoinBuildState { } let build_key_column = Column::concat_columns(columns.into_iter())?; // maybe there will be null values here, so we use nullable column, the null value will be treat as default - // value for the sepcified type, like String -> "", int -> 0. so we need to remove the null hash values here. + // value for the specified type, like String -> "", int -> 0. so we need to remove the null hash values here. let (hashes, bitmap_op) = BloomIndex::calculate_nullable_column_digest( &self.func_ctx, &build_key_column, From a86e64f71f1a8d88a0e59cefe29f33b4df66419f Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Thu, 21 Mar 2024 18:10:48 +0800 Subject: [PATCH 09/24] add profile and tests --- Cargo.lock | 1 - src/common/base/src/runtime/profile/profiles.rs | 8 ++++++++ src/query/catalog/Cargo.toml | 1 - src/query/sql/src/planner/format/display_plan.rs | 6 ++++++ src/query/sql/src/planner/optimizer/optimizer.rs | 6 +++++- src/query/sql/src/planner/plans/merge_into.rs | 2 +- .../src/operations/read/runtime_filter_prunner.rs | 14 +++++++++++++- 7 files changed, 33 insertions(+), 5 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 0211e6e843814..1d3f82406c268 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2822,7 +2822,6 @@ dependencies = [ "async-trait-fn", "chrono", "dashmap", - "databend-common-arrow", "databend-common-base", "databend-common-config", "databend-common-exception", diff --git a/src/common/base/src/runtime/profile/profiles.rs b/src/common/base/src/runtime/profile/profiles.rs index 2aa2a7d2db850..eb2d1bb3a7801 100644 --- a/src/common/base/src/runtime/profile/profiles.rs +++ b/src/common/base/src/runtime/profile/profiles.rs @@ -44,6 +44,7 @@ pub enum ProfileStatisticsName { SpillReadBytes, SpillReadTime, RuntimeFilterPruneParts, + RuntimeFilterMergeIntoSourceBuildBloomPruneParts, MemoryUsage, } @@ -236,6 +237,13 @@ pub fn get_statistics_desc() -> Arc unit: StatisticsUnit::Count, plain_statistics: true, }), + (ProfileStatisticsName::RuntimeFilterMergeIntoSourceBuildBloomPruneParts, ProfileDesc { + display_name: "parts pruned by merge into target table bloom filter", + desc: "The partitions pruned by merge into target table bloom filter", + index: ProfileStatisticsName::RuntimeFilterMergeIntoSourceBuildBloomPruneParts as usize, + unit: StatisticsUnit::Count, + plain_statistics: true, + }), (ProfileStatisticsName::MemoryUsage, ProfileDesc { display_name: "memory usage", desc: "The real time memory usage", diff --git a/src/query/catalog/Cargo.toml b/src/query/catalog/Cargo.toml index d226cd98256a0..9150214ccc299 100644 --- a/src/query/catalog/Cargo.toml +++ b/src/query/catalog/Cargo.toml @@ -10,7 +10,6 @@ test = false [dependencies] databend-common-base = { path = "../../common/base" } -databend-common-arrow = { path = "../../common/arrow" } databend-common-config = { path = "../config" } databend-common-exception = { path = "../../common/exception" } databend-common-expression = { path = "../expression" } diff --git a/src/query/sql/src/planner/format/display_plan.rs b/src/query/sql/src/planner/format/display_plan.rs index e5d7d4801753d..5dc30bb7d2b9c 100644 --- a/src/query/sql/src/planner/format/display_plan.rs +++ b/src/query/sql/src/planner/format/display_plan.rs @@ -311,6 +311,11 @@ fn format_merge_into(merge_into: &MergeInto) -> Result { table_entry.name(), ); + let can_merge_into_source_build_bloom_format = FormatTreeNode::new(format!( + "can_merge_into_source_build_bloom: {}", + merge_into.can_merge_into_source_build_bloom + )); + 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(format!( @@ -388,6 +393,7 @@ fn format_merge_into(merge_into: &MergeInto) -> Result { vec![distributed_format], vec![target_build_optimization_format], vec![can_try_update_column_only_format], + vec![can_merge_into_source_build_bloom_format], matched_children, unmatched_children, vec![input_format_child], diff --git a/src/query/sql/src/planner/optimizer/optimizer.rs b/src/query/sql/src/planner/optimizer/optimizer.rs index 5c0ec0be2bb06..019e3e4a9736d 100644 --- a/src/query/sql/src/planner/optimizer/optimizer.rs +++ b/src/query/sql/src/planner/optimizer/optimizer.rs @@ -411,6 +411,7 @@ fn optimize_merge_into(opt_ctx: OptimizerContext, plan: Box) -> Resul }; // try add source_build bloom filter + let mut enable_merge_into_source_build_bloom = false; if !change_join_order && matches!(plan.merge_type, MergeIntoType::FullOperation) && opt_ctx @@ -432,7 +433,8 @@ fn optimize_merge_into(opt_ctx: OptimizerContext, plan: Box) -> Resul merge_into_join_type: MergeIntoJoinType::Right, database_name: plan.database.clone(), table_schema: Some(plan.table_schema.clone()), - }) + }); + enable_merge_into_source_build_bloom = true; } // we just support left join to use MergeIntoBlockInfoHashTable, we @@ -531,6 +533,7 @@ fn optimize_merge_into(opt_ctx: OptimizerContext, plan: Box) -> Resul distributed, change_join_order, columns_set: new_columns_set.clone(), + can_merge_into_source_build_bloom: enable_merge_into_source_build_bloom, ..*plan }))) } else { @@ -560,6 +563,7 @@ fn optimize_merge_into(opt_ctx: OptimizerContext, plan: Box) -> Resul input: join_sexpr, change_join_order, columns_set: new_columns_set, + can_merge_into_source_build_bloom: enable_merge_into_source_build_bloom, ..*plan }))) } diff --git a/src/query/sql/src/planner/plans/merge_into.rs b/src/query/sql/src/planner/plans/merge_into.rs index 370a9c27ce2a6..a1155f48e96e5 100644 --- a/src/query/sql/src/planner/plans/merge_into.rs +++ b/src/query/sql/src/planner/plans/merge_into.rs @@ -79,7 +79,7 @@ pub struct MergeInto { // `update *`` or `update set t1.a = t2.a ...`, the right expr on the `=` must be only a column, // we don't support complex expressions. pub can_try_update_column_only: bool, - + pub can_merge_into_source_build_bloom: bool, pub table_schema: Arc, } diff --git a/src/query/storages/fuse/src/operations/read/runtime_filter_prunner.rs b/src/query/storages/fuse/src/operations/read/runtime_filter_prunner.rs index 3d8c608c06ce4..b8a4bc1d58b95 100644 --- a/src/query/storages/fuse/src/operations/read/runtime_filter_prunner.rs +++ b/src/query/storages/fuse/src/operations/read/runtime_filter_prunner.rs @@ -116,7 +116,19 @@ pub fn runtime_filter_pruner( // if we can't pruned this block, we can try get siphashkeys if this is a merge into source build if can_do_merge_into_target_build_bloom_filter { - try_prune_merge_into_target_table(ctx.clone(), part, merge_into_source_build_bloom_info, id) + let pruned = try_prune_merge_into_target_table( + ctx.clone(), + part, + merge_into_source_build_bloom_info, + id, + )?; + if pruned { + Profile::record_usize_profile( + ProfileStatisticsName::RuntimeFilterMergeIntoSourceBuildBloomPruneParts, + 1, + ); + } + Ok(pruned) } else { Ok(false) } From 4291c8253608de1c2c80635296bea2170a86eb05 Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Thu, 21 Mar 2024 18:53:41 +0800 Subject: [PATCH 10/24] fix bugs --- .../interpreters/interpreter_merge_into.rs | 17 ----------------- .../sql/src/planner/binder/merge_into.rs | 1 + .../sql/src/planner/optimizer/optimizer.rs | 19 +++++++++++++++++-- 3 files changed, 18 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 37d485b1c4ae3..248f44e3e263b 100644 --- a/src/query/service/src/interpreters/interpreter_merge_into.rs +++ b/src/query/service/src/interpreters/interpreter_merge_into.rs @@ -292,23 +292,6 @@ impl MergeIntoInterpreter { let table_info = fuse_table.get_table_info().clone(); let catalog_ = self.ctx.get_catalog(catalog).await?; - // try add catalog_info and table_info for `source_build_bloom_filter` - let merge_into_join = self.ctx.get_merge_into_join(); - let source_build_bloom_filter = matches!( - merge_into_join.merge_into_join_type, - MergeIntoJoinType::Right - ) && merge_into_join.target_tbl_idx != DUMMY_TABLE_INDEX; - if source_build_bloom_filter { - self.ctx.set_merge_into_join(MergeIntoJoin { - target_tbl_idx: merge_into_join.target_tbl_idx, - is_distributed: merge_into_join.is_distributed, - merge_into_join_type: merge_into_join.merge_into_join_type, - table_info: Some(table_info.clone()), - catalog_info: Some(catalog_.info()), - database_name: merge_into_join.database_name.clone(), - table_schema: merge_into_join.table_schema.clone(), - }) - } // merge_into_source is used to recv join's datablocks and split them into macthed and not matched // datablocks. let merge_into_source = if !*distributed && extract_exchange { diff --git a/src/query/sql/src/planner/binder/merge_into.rs b/src/query/sql/src/planner/binder/merge_into.rs index 37edbd357ccc2..bb4aa06e85e2e 100644 --- a/src/query/sql/src/planner/binder/merge_into.rs +++ b/src/query/sql/src/planner/binder/merge_into.rs @@ -489,6 +489,7 @@ impl Binder { split_idx, can_try_update_column_only: self.can_try_update_column_only(&matched_clauses), table_schema, + can_merge_into_source_build_bloom: false, }) } diff --git a/src/query/sql/src/planner/optimizer/optimizer.rs b/src/query/sql/src/planner/optimizer/optimizer.rs index 019e3e4a9736d..c3b074ac6c606 100644 --- a/src/query/sql/src/planner/optimizer/optimizer.rs +++ b/src/query/sql/src/planner/optimizer/optimizer.rs @@ -16,6 +16,7 @@ use std::collections::HashSet; use std::sync::Arc; use databend_common_ast::ast::ExplainKind; +use databend_common_base::runtime::block_on; use databend_common_catalog::merge_into_join::MergeIntoJoin; use databend_common_catalog::merge_into_join::MergeIntoJoinType; use databend_common_catalog::table_context::TableContext; @@ -424,10 +425,24 @@ fn optimize_merge_into(opt_ctx: OptimizerContext, plan: Box) -> Resul assert!(merge_into_join.catalog_info.is_none()); assert!(merge_into_join.table_info.is_none()); assert!(merge_into_join.database_name.as_str() == ""); + let table = block_on(async { + opt_ctx + .table_ctx + .get_table( + plan.catalog.as_str(), + plan.database.as_str(), + plan.table.as_str(), + ) + .await + })?; + + let table_info = table.get_table_info().clone(); + let catalog_info = + block_on(async { opt_ctx.table_ctx.get_catalog(&plan.catalog).await })?.info(); opt_ctx.table_ctx.set_merge_into_join(MergeIntoJoin { // we will set catalog_info and table_info in `interpreter_merge_into` - catalog_info: None, - table_info: None, + catalog_info: Some(catalog_info), + table_info: Some(table_info), target_tbl_idx: plan.target_table_idx, is_distributed: false, // we will set it after later optimization. merge_into_join_type: MergeIntoJoinType::Right, From dfdaa6440fe0de46f04faad291833bc0afd83a40 Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Thu, 21 Mar 2024 18:56:29 +0800 Subject: [PATCH 11/24] fix explain --- .../mode/cluster/merge_into_non_equal_distributed.test | 2 ++ .../explain/09_0039_target_build_merge_into_standalone.test | 4 ++++ .../suites/mode/standalone/explain/merge_into.test | 6 ++++++ 3 files changed, 12 insertions(+) diff --git a/tests/sqllogictests/suites/mode/cluster/merge_into_non_equal_distributed.test b/tests/sqllogictests/suites/mode/cluster/merge_into_non_equal_distributed.test index 9a94c8c0ddc23..e1fcede45ac84 100644 --- a/tests/sqllogictests/suites/mode/cluster/merge_into_non_equal_distributed.test +++ b/tests/sqllogictests/suites/mode/cluster/merge_into_non_equal_distributed.test @@ -61,6 +61,7 @@ target_table: default.default.t1 ├── distributed: false ├── target_build_optimization: false ├── can_try_update_column_only: true +├── can_merge_into_source_build_bloom: false ├── matched update: [condition: None,update set a = a (#0)] ├── unmatched insert: [condition: None,insert into (a) values(CAST(a (#0) AS Int32 NULL))] └── Join(Right) @@ -140,6 +141,7 @@ target_table: default.default.t1 ├── distributed: false ├── target_build_optimization: false ├── can_try_update_column_only: true +├── can_merge_into_source_build_bloom: false ├── matched update: [condition: None,update set a = a (#0)] ├── unmatched insert: [condition: None,insert into (a) values(CAST(a (#0) AS Int32 NULL))] └── Exchange(Merge) 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 3780971e67508..c3a8a207d0681 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 @@ -62,6 +62,7 @@ target_table: default.default.target_build_optimization ├── distributed: false ├── target_build_optimization: true ├── can_try_update_column_only: true +├── can_merge_into_source_build_bloom: false ├── 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))] └── Join(Left) @@ -124,6 +125,7 @@ target_table: default.default.target_build_optimization ├── distributed: false ├── target_build_optimization: true ├── can_try_update_column_only: true +├── can_merge_into_source_build_bloom: false ├── 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))] └── Join(Left) @@ -203,6 +205,7 @@ target_table: default.default.target_build_optimization ├── distributed: false ├── target_build_optimization: false ├── can_try_update_column_only: true +├── can_merge_into_source_build_bloom: false ├── 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))] └── Join(Left) @@ -286,6 +289,7 @@ target_table: default.default.target_build_optimization ├── distributed: false ├── target_build_optimization: false ├── can_try_update_column_only: true +├── can_merge_into_source_build_bloom: false ├── 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))] └── Join(Left) diff --git a/tests/sqllogictests/suites/mode/standalone/explain/merge_into.test b/tests/sqllogictests/suites/mode/standalone/explain/merge_into.test index e8098921e116d..b4e2da2fbe06e 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/merge_into.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/merge_into.test @@ -26,6 +26,7 @@ target_table: default.default.salaries2 ├── distributed: false ├── target_build_optimization: false ├── can_try_update_column_only: false +├── can_merge_into_source_build_bloom: 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))] @@ -57,6 +58,7 @@ target_table: default.default.salaries2 ├── distributed: false ├── target_build_optimization: false ├── can_try_update_column_only: false +├── can_merge_into_source_build_bloom: 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))] @@ -102,6 +104,7 @@ target_table: default.default.column_only_optimization_target ├── distributed: false ├── target_build_optimization: true ├── can_try_update_column_only: true +├── can_merge_into_source_build_bloom: false ├── matched update: [condition: None,update set b = t2.b (#1)] ├── unmatched insert: [condition: None,insert into (a,b) values(CAST(a (#0) AS Int32 NULL),CAST(b (#1) AS String NULL))] └── Join(Left) @@ -128,6 +131,7 @@ target_table: default.default.column_only_optimization_target ├── distributed: false ├── target_build_optimization: true ├── can_try_update_column_only: true +├── can_merge_into_source_build_bloom: false ├── matched update: [condition: None,update set a = a (#0),b = b (#1)] ├── unmatched insert: [condition: None,insert into (a,b) values(CAST(a (#0) AS Int32 NULL),CAST(b (#1) AS String NULL))] └── Join(Left) @@ -157,6 +161,7 @@ target_table: default.default.column_only_optimization_target ├── distributed: false ├── target_build_optimization: false ├── can_try_update_column_only: false +├── can_merge_into_source_build_bloom: false ├── matched update: [condition: None,update set b = 'test'] ├── unmatched insert: [condition: None,insert into (a,b) values(CAST(a (#0) AS Int32 NULL),CAST(b (#1) AS String NULL))] └── Join(Left) @@ -183,6 +188,7 @@ target_table: default.default.column_only_optimization_target ├── distributed: false ├── target_build_optimization: false ├── can_try_update_column_only: false +├── can_merge_into_source_build_bloom: false ├── matched update: [condition: None,update set b = concat(t2.b (#1), 'test')] ├── unmatched insert: [condition: None,insert into (a,b) values(CAST(a (#0) AS Int32 NULL),CAST(b (#1) AS String NULL))] └── Join(Left) From 60bf6b8b574cd47b88787d30cca6464b2e6fca6f Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Thu, 21 Mar 2024 19:58:44 +0800 Subject: [PATCH 12/24] fix runtime --- src/query/catalog/src/merge_into_join.rs | 12 ++++------ .../interpreters/interpreter_merge_into.rs | 6 +---- src/query/service/src/sessions/query_ctx.rs | 5 +--- .../sql/src/planner/optimizer/optimizer.rs | 23 ++++--------------- .../operations/read/runtime_filter_prunner.rs | 18 ++------------- .../storages/fuse/src/operations/read/util.rs | 20 +++++----------- 6 files changed, 18 insertions(+), 66 deletions(-) diff --git a/src/query/catalog/src/merge_into_join.rs b/src/query/catalog/src/merge_into_join.rs index 19b21b5d9b289..3680b36d2c738 100644 --- a/src/query/catalog/src/merge_into_join.rs +++ b/src/query/catalog/src/merge_into_join.rs @@ -19,6 +19,8 @@ use databend_common_meta_app::schema::CatalogInfo; use databend_common_meta_app::schema::TableInfo; use databend_storages_common_table_meta::meta::Location; +use crate::table::Table; + #[derive(Clone)] pub enum MergeIntoJoinType { Left, @@ -59,10 +61,7 @@ pub struct MergeIntoJoin { pub merge_into_join_type: MergeIntoJoinType, pub is_distributed: bool, pub target_tbl_idx: usize, - pub table_info: Option, - pub catalog_info: Option, - pub database_name: String, - pub table_schema: Option, + pub table: Option>, } impl Default for MergeIntoJoin { @@ -72,10 +71,7 @@ impl Default for MergeIntoJoin { is_distributed: false, // Invalid Index target_tbl_idx: usize::MAX, - table_info: None, - catalog_info: None, - database_name: Default::default(), - table_schema: None, + table: None, } } } diff --git a/src/query/service/src/interpreters/interpreter_merge_into.rs b/src/query/service/src/interpreters/interpreter_merge_into.rs index 248f44e3e263b..e52978c978775 100644 --- a/src/query/service/src/interpreters/interpreter_merge_into.rs +++ b/src/query/service/src/interpreters/interpreter_merge_into.rs @@ -17,7 +17,6 @@ use std::sync::Arc; use std::u64::MAX; use databend_common_catalog::merge_into_join::MergeIntoJoin; -use databend_common_catalog::merge_into_join::MergeIntoJoinType; use databend_common_catalog::table::TableExt; use databend_common_exception::ErrorCode; use databend_common_exception::Result; @@ -200,10 +199,7 @@ impl MergeIntoInterpreter { target_tbl_idx: DUMMY_TABLE_INDEX, is_distributed: merge_into_join.is_distributed, merge_into_join_type: merge_into_join.merge_into_join_type, - table_info: merge_into_join.table_info.clone(), - catalog_info: merge_into_join.catalog_info.clone(), - database_name: merge_into_join.database_name.clone(), - table_schema: merge_into_join.table_schema.clone(), + table: merge_into_join.table.clone(), }); } } diff --git a/src/query/service/src/sessions/query_ctx.rs b/src/query/service/src/sessions/query_ctx.rs index 052e0d151d35f..e0f21e30a5588 100644 --- a/src/query/service/src/sessions/query_ctx.rs +++ b/src/query/service/src/sessions/query_ctx.rs @@ -1013,10 +1013,7 @@ impl TableContext for QueryContext { 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, - catalog_info: merge_into_join.catalog_info.clone(), - table_info: merge_into_join.table_info.clone(), - database_name: merge_into_join.database_name.clone(), - table_schema: merge_into_join.table_schema.clone(), + table: merge_into_join.table.clone(), } } diff --git a/src/query/sql/src/planner/optimizer/optimizer.rs b/src/query/sql/src/planner/optimizer/optimizer.rs index c3b074ac6c606..0ca503c2cb49b 100644 --- a/src/query/sql/src/planner/optimizer/optimizer.rs +++ b/src/query/sql/src/planner/optimizer/optimizer.rs @@ -422,9 +422,7 @@ fn optimize_merge_into(opt_ctx: OptimizerContext, plan: Box) -> Resul { let merge_into_join = opt_ctx.table_ctx.get_merge_into_join(); // this is the first time set, so it must be none, and we will set it in `interpreter_merge_into` - assert!(merge_into_join.catalog_info.is_none()); - assert!(merge_into_join.table_info.is_none()); - assert!(merge_into_join.database_name.as_str() == ""); + assert!(merge_into_join.table.is_none()); let table = block_on(async { opt_ctx .table_ctx @@ -436,18 +434,11 @@ fn optimize_merge_into(opt_ctx: OptimizerContext, plan: Box) -> Resul .await })?; - let table_info = table.get_table_info().clone(); - let catalog_info = - block_on(async { opt_ctx.table_ctx.get_catalog(&plan.catalog).await })?.info(); opt_ctx.table_ctx.set_merge_into_join(MergeIntoJoin { - // we will set catalog_info and table_info in `interpreter_merge_into` - catalog_info: Some(catalog_info), - table_info: Some(table_info), target_tbl_idx: plan.target_table_idx, is_distributed: false, // we will set it after later optimization. merge_into_join_type: MergeIntoJoinType::Right, - database_name: plan.database.clone(), - table_schema: Some(plan.table_schema.clone()), + table: Some(table), }); enable_merge_into_source_build_bloom = true; } @@ -536,10 +527,7 @@ fn optimize_merge_into(opt_ctx: OptimizerContext, plan: Box) -> Resul merge_into_join_type: MergeIntoJoinType::Left, is_distributed: distributed, target_tbl_idx: plan.target_table_idx, - catalog_info: merge_into_join.catalog_info.clone(), - table_info: merge_into_join.table_info.clone(), - database_name: merge_into_join.database_name.clone(), - table_schema: merge_into_join.table_schema.clone(), + table: merge_into_join.table.clone(), }) } @@ -568,10 +556,7 @@ fn optimize_merge_into(opt_ctx: OptimizerContext, plan: Box) -> Resul merge_into_join_type: MergeIntoJoinType::Left, is_distributed: false, target_tbl_idx: plan.target_table_idx, - catalog_info: merge_into_join.catalog_info.clone(), - table_info: merge_into_join.table_info.clone(), - database_name: merge_into_join.database_name, - table_schema: merge_into_join.table_schema.clone(), + table: merge_into_join.table.clone(), }) } Ok(Plan::MergeInto(Box::new(MergeInto { diff --git a/src/query/storages/fuse/src/operations/read/runtime_filter_prunner.rs b/src/query/storages/fuse/src/operations/read/runtime_filter_prunner.rs index b8a4bc1d58b95..c30eb00b0a764 100644 --- a/src/query/storages/fuse/src/operations/read/runtime_filter_prunner.rs +++ b/src/query/storages/fuse/src/operations/read/runtime_filter_prunner.rs @@ -146,22 +146,8 @@ pub(crate) fn try_prune_merge_into_target_table( let segment_idx = block_meta_index.segment_idx; let block_idx = block_meta_index.block_idx; let target_table_segments = ctx.get_merge_into_source_build_segments(); - let catalog_info = merge_into_source_build_bloom_info - .catalog_info - .as_ref() - .unwrap(); - let table_info = merge_into_source_build_bloom_info - .table_info - .as_ref() - .unwrap(); - let table = block_on(async { - ctx.get_table( - catalog_info.catalog_name(), - &merge_into_source_build_bloom_info.database_name, - &table_info.name, - ) - .await - })?; + + let table = merge_into_source_build_bloom_info.table.as_ref().unwrap(); let fuse_table = table.as_any().downcast_ref::().ok_or_else(|| { ErrorCode::Unimplemented(format!( "table {}, engine type {}, does not support MERGE INTO", diff --git a/src/query/storages/fuse/src/operations/read/util.rs b/src/query/storages/fuse/src/operations/read/util.rs index c2d9324990b7a..366809b7669f7 100644 --- a/src/query/storages/fuse/src/operations/read/util.rs +++ b/src/query/storages/fuse/src/operations/read/util.rs @@ -19,6 +19,7 @@ use databend_common_catalog::merge_into_join::MergeIntoJoin; use databend_common_catalog::merge_into_join::MergeIntoJoinType; use databend_common_catalog::plan::gen_mutation_stream_meta; use databend_common_catalog::plan::InternalColumnMeta; +use databend_common_catalog::table::Table; use databend_common_catalog::table_context::TableContext; use databend_common_exception::ErrorCode; use databend_common_exception::Result; @@ -26,8 +27,6 @@ use databend_common_expression::BlockMetaInfoPtr; use databend_common_expression::DataBlock; use databend_common_expression::FieldIndex; use databend_common_expression::Scalar; -use databend_common_meta_app::schema::CatalogInfo; -use databend_common_meta_app::schema::TableInfo; use databend_common_sql::executor::physical_plans::OnConflictField; use databend_storages_common_table_meta::meta::SegmentInfo; @@ -38,9 +37,7 @@ use crate::FusePartInfo; pub struct MergeIntoSourceBuildBloomInfo { pub can_do_merge_into_runtime_filter_bloom: bool, pub segment_infos: HashMap, - pub table_info: Option, - pub catalog_info: Option, - pub database_name: String, + pub table: Option>, pub bloom_indexes: Vec, pub bloom_fields: Vec, } @@ -71,10 +68,7 @@ pub fn can_merge_into_target_build_bloom_filter( ctx.get_settings() .get_enable_merge_into_source_build_bloom()? ); - assert!(merge_into_join.database_name.as_str() != ""); - assert!(merge_into_join.catalog_info.is_some()); - assert!(merge_into_join.table_info.is_some()); - assert!(merge_into_join.table_schema.is_some()); + assert!(merge_into_join.table.is_some()); } Ok(enabled) } @@ -135,11 +129,11 @@ pub fn build_merge_into_source_build_bloom_info( ctx.get_merge_into_source_build_bloom_probe_keys(table_index) .iter() .try_fold((Vec::new(),Vec::new()), |mut acc, probe_key_name| { - let table_schema = merge_into_join.table_schema.as_ref().ok_or_else(|| { + let table_schema = merge_into_join.table.as_ref().ok_or_else(|| { ErrorCode::Internal( "can't get merge into target table schema when build bloom info, it's a bug", ) - })?; + })?.schema(); let index = table_schema.index_of(probe_key_name)?; acc.0.push(index); acc.1.push(OnConflictField { table_field: table_schema.field(index).clone(), field_index: index }); @@ -152,9 +146,7 @@ pub fn build_merge_into_source_build_bloom_info( Ok(MergeIntoSourceBuildBloomInfo { can_do_merge_into_runtime_filter_bloom: enabled_bloom_filter, segment_infos: Default::default(), - catalog_info: merge_into_join.catalog_info.clone(), - table_info: merge_into_join.table_info.clone(), - database_name: merge_into_join.database_name.clone(), + table: merge_into_join.table.clone(), bloom_indexes, bloom_fields, }) From 6dcf50ae664c105e9a2b16ce3922c966128e1ff3 Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Thu, 21 Mar 2024 22:43:26 +0800 Subject: [PATCH 13/24] ad tests and fix tokio runtime io --- a.sql | 2 + src/query/catalog/src/merge_into_join.rs | 5 +- .../src/interpreters/interpreter_explain.rs | 33 ++++++ .../interpreters/interpreter_merge_into.rs | 12 ++ .../it/storages/fuse/operations/commit.rs | 5 +- .../operations/read/runtime_filter_prunner.rs | 51 ++++++-- .../storages/fuse/src/operations/read/util.rs | 28 ++--- .../merge_into_source_build_bloom.test | 112 ++++++++++++++++++ .../merge_into_source_build_bloom_filter.test | 97 +++++++++++++++ 9 files changed, 306 insertions(+), 39 deletions(-) create mode 100644 a.sql create mode 100644 tests/sqllogictests/suites/mode/cluster/merge_into_source_build_bloom.test create mode 100644 tests/sqllogictests/suites/mode/standalone/explain/merge_into_source_build_bloom_filter.test diff --git a/a.sql b/a.sql new file mode 100644 index 0000000000000..84b9d25d9e7b3 --- /dev/null +++ b/a.sql @@ -0,0 +1,2 @@ + +explain merge into t1 using (SELECT * EXCLUDE (rn) FROM (SELECT *, row_number() OVER (PARTITION BY id ORDER BY t DESC) AS rn FROM tbcc WHERE a in(1,2,3))) as t2 on t1.a = t2.a when matched then update * when not matched then insert *; diff --git a/src/query/catalog/src/merge_into_join.rs b/src/query/catalog/src/merge_into_join.rs index 3680b36d2c738..9866de83c6de3 100644 --- a/src/query/catalog/src/merge_into_join.rs +++ b/src/query/catalog/src/merge_into_join.rs @@ -14,9 +14,6 @@ use std::sync::Arc; -use databend_common_expression::TableSchemaRef; -use databend_common_meta_app::schema::CatalogInfo; -use databend_common_meta_app::schema::TableInfo; use databend_storages_common_table_meta::meta::Location; use crate::table::Table; @@ -32,7 +29,7 @@ pub enum MergeIntoJoinType { NormalJoin, } -pub type MergeIntoSourceBuildSegments = Arc>; +pub type MergeIntoSourceBuildSegments = Arc>; // MergeIntoJoin is used in two cases: // I. target build optimization: diff --git a/src/query/service/src/interpreters/interpreter_explain.rs b/src/query/service/src/interpreters/interpreter_explain.rs index 133455d68ce3b..2b6db45ed43ef 100644 --- a/src/query/service/src/interpreters/interpreter_explain.rs +++ b/src/query/service/src/interpreters/interpreter_explain.rs @@ -25,14 +25,18 @@ use databend_common_expression::DataBlock; use databend_common_expression::FromData; use databend_common_pipeline_core::processors::PlanProfile; use databend_common_sql::binder::ExplainConfig; +use databend_common_sql::binder::MergeIntoType; use databend_common_sql::optimizer::ColumnSet; use databend_common_sql::plans::UpdatePlan; use databend_common_sql::BindContext; use databend_common_sql::InsertInputSource; use databend_common_sql::MetadataRef; +use databend_common_storages_factory::Table; +use databend_common_storages_fuse::FuseTable; use databend_common_storages_result_cache::gen_result_cache_key; use databend_common_storages_result_cache::ResultCacheReader; use databend_common_users::UserApiProvider; +use databend_storages_common_table_meta::meta::TableSnapshot; use super::InterpreterFactory; use super::UpdateInterpreter; @@ -159,6 +163,35 @@ impl Interpreter for ExplainInterpreter { .await? } Plan::MergeInto(plan) => { + // if we enable merge_into_source_build_bloom, we should set the segments here + if !plan.change_join_order + && matches!(plan.merge_type, MergeIntoType::FullOperation) + && self + .ctx + .get_settings() + .get_enable_merge_into_source_build_bloom()? + { + let merge_into_join = self.ctx.get_merge_into_join(); + let table = merge_into_join.table.as_ref().unwrap(); + 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 base_snapshot = + fuse_table.read_table_snapshot().await?.unwrap_or_else(|| { + Arc::new(TableSnapshot::new_empty_snapshot( + fuse_table.schema().as_ref().clone(), + )) + }); + self.ctx.set_merge_into_source_build_segments(Arc::new( + base_snapshot.segments.clone(), + )); + } + self.explain_analyze( &plan.input, &plan.meta_data, diff --git a/src/query/service/src/interpreters/interpreter_merge_into.rs b/src/query/service/src/interpreters/interpreter_merge_into.rs index e52978c978775..c1477e4c77d02 100644 --- a/src/query/service/src/interpreters/interpreter_merge_into.rs +++ b/src/query/service/src/interpreters/interpreter_merge_into.rs @@ -17,6 +17,7 @@ use std::sync::Arc; use std::u64::MAX; use databend_common_catalog::merge_into_join::MergeIntoJoin; +use databend_common_catalog::merge_into_join::MergeIntoJoinType; use databend_common_catalog::table::TableExt; use databend_common_exception::ErrorCode; use databend_common_exception::Result; @@ -434,6 +435,17 @@ impl MergeIntoInterpreter { .enumerate() .collect(); + // try add catalog_info and table_info for `source_build_bloom_filter` + let merge_into_join = self.ctx.get_merge_into_join(); + let source_build_bloom_filter = matches!( + merge_into_join.merge_into_join_type, + MergeIntoJoinType::Right + ) && merge_into_join.target_tbl_idx != DUMMY_TABLE_INDEX; + if source_build_bloom_filter { + self.ctx + .set_merge_into_source_build_segments(Arc::new(base_snapshot.segments.clone())); + } + let commit_input = if !distributed { // recv datablocks from matched upstream and unmatched upstream // transform and append data 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 c77d9d57f3443..3ba18a65a6c7a 100644 --- a/src/query/service/tests/it/storages/fuse/operations/commit.rs +++ b/src/query/service/tests/it/storages/fuse/operations/commit.rs @@ -379,10 +379,7 @@ impl TableContext for CtxDelegation { todo!() } - fn set_merge_into_source_build_segments( - &self, - _: Arc>, - ) { + fn set_merge_into_source_build_segments(&self, _: Arc>) { todo!() } diff --git a/src/query/storages/fuse/src/operations/read/runtime_filter_prunner.rs b/src/query/storages/fuse/src/operations/read/runtime_filter_prunner.rs index c30eb00b0a764..5876a93d84abc 100644 --- a/src/query/storages/fuse/src/operations/read/runtime_filter_prunner.rs +++ b/src/query/storages/fuse/src/operations/read/runtime_filter_prunner.rs @@ -17,9 +17,9 @@ use std::collections::HashMap; use std::sync::Arc; use databend_common_arrow::arrow::bitmap::MutableBitmap; -use databend_common_base::runtime::block_on; use databend_common_base::runtime::profile::Profile; use databend_common_base::runtime::profile::ProfileStatisticsName; +use databend_common_base::runtime::GlobalIORuntime; use databend_common_catalog::plan::PartInfoPtr; use databend_common_catalog::table_context::TableContext; use databend_common_exception::ErrorCode; @@ -39,6 +39,7 @@ use databend_common_expression::Scalar; use databend_common_expression::TableSchema; use databend_common_functions::BUILTIN_FUNCTIONS; use databend_common_hashtable::FastHash; +use databend_common_sql::executor::physical_plans::OnConflictField; use databend_storages_common_cache::LoadParams; use databend_storages_common_index::statistics_to_domain; use databend_storages_common_table_meta::meta::SegmentInfo; @@ -159,7 +160,7 @@ pub(crate) fn try_prune_merge_into_target_table( .segment_infos .entry(segment_idx) { - let (_,(path, ver)) = target_table_segments.get(segment_idx).ok_or_else(|| { + let (path, ver) = target_table_segments.get(segment_idx).ok_or_else(|| { ErrorCode::Internal(format!( "unexpected, segment (idx {}) not found, during do merge into source build bloom filter", segment_idx @@ -176,7 +177,8 @@ pub(crate) fn try_prune_merge_into_target_table( let data_accessor = fuse_table.get_operator(); let segment_reader = MetaReaders::segment_info_reader(data_accessor.clone(), target_table_schema.clone()); - let compact_segment_info = block_on(async { segment_reader.read(&load_param).await })?; + let compact_segment_info = GlobalIORuntime::instance() + .block_on(async move { segment_reader.read(&load_param).await })?; let segment_info: SegmentInfo = compact_segment_info.try_into()?; e.insert(segment_info); } @@ -191,14 +193,43 @@ pub(crate) fn try_prune_merge_into_target_table( segment_idx, block_idx ); let block_meta = segment_info.blocks[block_idx].clone(); - if let Some(index_location) = block_meta.bloom_filter_index_location.as_ref() { - let filters = block_on(async { + let bloom_filter_index_size = block_meta.bloom_filter_index_size; + let index_location = block_meta.bloom_filter_index_location.clone(); + if let Some(index_location) = index_location { + // init bloom info. + if !merge_into_source_build_bloom_info.init_bloom_index_info { + merge_into_source_build_bloom_info.init_bloom_index_info = true; + let merge_into_join = ctx.get_merge_into_join(); + let (bloom_indexes, bloom_fields) = + ctx.get_merge_into_source_build_bloom_probe_keys(merge_into_source_build_bloom_info.target_table_index) + .iter() + .try_fold((Vec::new(),Vec::new()), |mut acc, probe_key_name| { + let table_schema = merge_into_join.table.as_ref().ok_or_else(|| { + ErrorCode::Internal( + "can't get merge into target table schema when build bloom info, it's a bug", + ) + })?.schema(); + let index = table_schema.index_of(probe_key_name)?; + acc.0.push(index); + acc.1.push(OnConflictField { table_field: table_schema.field(index).clone(), field_index: index }); + Ok::<_, ErrorCode>(acc) + })?; + assert_eq!(bloom_fields.len(), bloom_indexes.len()); + merge_into_source_build_bloom_info.bloom_fields = bloom_fields; + merge_into_source_build_bloom_info.bloom_indexes = bloom_indexes; + } + + let bloom_fields = merge_into_source_build_bloom_info.bloom_fields.clone(); + let bloom_indexes = merge_into_source_build_bloom_info.bloom_indexes.clone(); + let operator = fuse_table.get_operator(); + + let filters = GlobalIORuntime::instance().block_on(async move { load_bloom_filter( - fuse_table.get_operator(), - &merge_into_source_build_bloom_info.bloom_fields, - index_location, - block_meta.bloom_filter_index_size, - &merge_into_source_build_bloom_info.bloom_indexes, + operator, + &bloom_fields, + &index_location, + bloom_filter_index_size, + &bloom_indexes, ) .await }); diff --git a/src/query/storages/fuse/src/operations/read/util.rs b/src/query/storages/fuse/src/operations/read/util.rs index 366809b7669f7..b58c4a98eb2d7 100644 --- a/src/query/storages/fuse/src/operations/read/util.rs +++ b/src/query/storages/fuse/src/operations/read/util.rs @@ -21,7 +21,6 @@ use databend_common_catalog::plan::gen_mutation_stream_meta; use databend_common_catalog::plan::InternalColumnMeta; use databend_common_catalog::table::Table; use databend_common_catalog::table_context::TableContext; -use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::BlockMetaInfoPtr; use databend_common_expression::DataBlock; @@ -40,6 +39,8 @@ pub struct MergeIntoSourceBuildBloomInfo { pub table: Option>, pub bloom_indexes: Vec, pub bloom_fields: Vec, + pub init_bloom_index_info: bool, + pub target_table_index: usize, } pub fn need_reserve_block_info(ctx: Arc, table_idx: usize) -> (bool, bool) { @@ -125,29 +126,14 @@ pub fn build_merge_into_source_build_bloom_info( ) -> Result { let enabled_bloom_filter = can_merge_into_target_build_bloom_filter(ctx.clone(), table_index)?; - let (bloom_indexes, bloom_fields) = if enabled_bloom_filter { - ctx.get_merge_into_source_build_bloom_probe_keys(table_index) - .iter() - .try_fold((Vec::new(),Vec::new()), |mut acc, probe_key_name| { - let table_schema = merge_into_join.table.as_ref().ok_or_else(|| { - ErrorCode::Internal( - "can't get merge into target table schema when build bloom info, it's a bug", - ) - })?.schema(); - let index = table_schema.index_of(probe_key_name)?; - acc.0.push(index); - acc.1.push(OnConflictField { table_field: table_schema.field(index).clone(), field_index: index }); - Ok::<_, ErrorCode>(acc) - })? - } else { - (vec![], vec![]) - }; - assert_eq!(bloom_fields.len(), bloom_indexes.len()); Ok(MergeIntoSourceBuildBloomInfo { can_do_merge_into_runtime_filter_bloom: enabled_bloom_filter, segment_infos: Default::default(), table: merge_into_join.table.clone(), - bloom_indexes, - bloom_fields, + // update bloom_indexes and bloom_field after pipeline running. + bloom_indexes: vec![], + bloom_fields: vec![], + init_bloom_index_info: false, + target_table_index: table_index, }) } diff --git a/tests/sqllogictests/suites/mode/cluster/merge_into_source_build_bloom.test b/tests/sqllogictests/suites/mode/cluster/merge_into_source_build_bloom.test new file mode 100644 index 0000000000000..bbf7d7dfdc024 --- /dev/null +++ b/tests/sqllogictests/suites/mode/cluster/merge_into_source_build_bloom.test @@ -0,0 +1,112 @@ +statement ok +set enable_merge_into_source_build_bloom = 1; + +statement ok +set enable_distributed_merge_into = 1; + +statement ok +set join_spilling_memory_ratio = 0; + +statement ok +set enable_experimental_merge_into = 1; + +statement ok +drop table if exists target_bloom_table; + +statement ok +drop table if exists source_bloom_table; + +statement ok +create table target_bloom_table(a string,b int); + +statement ok +create table source_bloom_table(a string,b int); + +statement ok +insert into source_bloom_table values('abc',2),('def',1); + +statement ok +insert into source_bloom_table values('ff',18),('kgt',27); + +query T +select count(*) from fuse_block('default','source_bloom_table'); +---- +2 + +statement ok +insert into target_bloom_table values('ak7',992),('def',213); + +statement ok +insert into target_bloom_table values('abc12',22),('mkgt',73); + +statement ok +insert into target_bloom_table values('ab77c',93),('dqef',107); + +statement ok +insert into target_bloom_table values('falf',189),('krrgt',207); + +query T +select count(*) from fuse_block('default','target_bloom_table'); +---- +4 + +query T +explain merge into target_bloom_table as t1 using source_bloom_table as t2 on t1.a = t2.a when matched then update * when not matched then insert *; +---- +MergeInto: +target_table: default.default.target_bloom_table +├── distributed: true +├── target_build_optimization: false +├── can_try_update_column_only: true +├── can_merge_into_source_build_bloom: true +├── matched update: [condition: None,update set a = a (#0),b = b (#1)] +├── unmatched insert: [condition: None,insert into (a,b) values(CAST(a (#0) AS String NULL),CAST(b (#1) AS Int32 NULL))] +└── Exchange(Merge) + └── Join(Right) + ├── build keys: [t2.a (#0)] + ├── probe keys: [t1.a (#2)] + ├── other filters: [] + ├── Scan + │ ├── table: default.target_bloom_table + │ ├── filters: [] + │ ├── order by: [] + │ └── limit: NONE + └── Exchange(Broadcast) + └── AddRowNumber(AddRowNumber) + └── Scan + ├── table: default.source_bloom_table + ├── filters: [] + ├── order by: [] + └── limit: NONE + +query TT +merge into target_bloom_table as t1 using source_bloom_table as t2 on t1.a = t2.a when matched then update * when not matched then insert *; +---- +3 1 + +query TT +select * from target_bloom_table order by a,b; +---- +ab77c 93 +abc 2 +abc12 22 +ak7 992 +def 1 +dqef 107 +falf 189 +ff 18 +kgt 27 +krrgt 207 +mkgt 73 + +statement ok +set enable_merge_into_source_build_bloom = 0; + +statement ok +set enable_experimental_merge_into = 0; + +statement ok +set join_spilling_memory_ratio = 60; + +statement ok +set enable_distributed_merge_into = 0; \ No newline at end of file diff --git a/tests/sqllogictests/suites/mode/standalone/explain/merge_into_source_build_bloom_filter.test b/tests/sqllogictests/suites/mode/standalone/explain/merge_into_source_build_bloom_filter.test new file mode 100644 index 0000000000000..eac276fd6db30 --- /dev/null +++ b/tests/sqllogictests/suites/mode/standalone/explain/merge_into_source_build_bloom_filter.test @@ -0,0 +1,97 @@ +statement ok +set enable_merge_into_source_build_bloom = 1; + +statement ok +set enable_experimental_merge_into = 1; + +statement ok +drop table if exists target_bloom_table; + +statement ok +drop table if exists source_bloom_table; + +statement ok +create table target_bloom_table(a string,b int); + +statement ok +create table source_bloom_table(a string,b int); + +statement ok +insert into source_bloom_table values('abc',2),('def',1); + +statement ok +insert into source_bloom_table values('ff',18),('kgt',27); + +query T +select count(*) from fuse_block('default','source_bloom_table'); +---- +2 + +statement ok +insert into target_bloom_table values('ak7',992),('def',213); + +statement ok +insert into target_bloom_table values('abc12',22),('mkgt',73); + +statement ok +insert into target_bloom_table values('ab77c',93),('dqef',107); + +statement ok +insert into target_bloom_table values('falf',189),('krrgt',207); + +query T +select count(*) from fuse_block('default','target_bloom_table'); +---- +4 + +query T +explain merge into target_bloom_table as t1 using source_bloom_table as t2 on t1.a = t2.a when matched then update * when not matched then insert *; +---- +MergeInto: +target_table: default.default.target_bloom_table +├── distributed: false +├── target_build_optimization: false +├── can_try_update_column_only: true +├── can_merge_into_source_build_bloom: true +├── matched update: [condition: None,update set a = a (#0),b = b (#1)] +├── unmatched insert: [condition: None,insert into (a,b) values(CAST(a (#0) AS String NULL),CAST(b (#1) AS Int32 NULL))] +└── Join(Left) + ├── build keys: [t1.a (#2)] + ├── probe keys: [t2.a (#0)] + ├── other filters: [] + ├── Scan + │ ├── table: default.source_bloom_table + │ ├── filters: [] + │ ├── order by: [] + │ └── limit: NONE + └── Scan + ├── table: default.target_bloom_table + ├── filters: [] + ├── order by: [] + └── limit: NONE + +query TT +merge into target_bloom_table as t1 using source_bloom_table as t2 on t1.a = t2.a when matched then update * when not matched then insert *; +---- +3 1 + +query TT +select * from target_bloom_table order by a,b; +---- +ab77c 93 +abc 2 +abc12 22 +ak7 992 +def 1 +dqef 107 +falf 189 +ff 18 +kgt 27 +krrgt 207 +mkgt 73 + +statement ok +set enable_merge_into_source_build_bloom = 0; + +statement ok +set enable_experimental_merge_into = 0; From 6a208df755c182340b0686d2e27beb5056a22ed1 Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Thu, 21 Mar 2024 22:47:36 +0800 Subject: [PATCH 14/24] fix lint --- .../service/tests/it/storages/fuse/operations/commit.rs | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) 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 3ba18a65a6c7a..fb10197a73eba 100644 --- a/src/query/service/tests/it/storages/fuse/operations/commit.rs +++ b/src/query/service/tests/it/storages/fuse/operations/commit.rs @@ -25,6 +25,7 @@ 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::merge_into_join::MergeIntoSourceBuildSegments; use databend_common_catalog::plan::DataSourcePlan; use databend_common_catalog::plan::PartInfoPtr; use databend_common_catalog::plan::Partitions; @@ -379,7 +380,7 @@ impl TableContext for CtxDelegation { todo!() } - fn set_merge_into_source_build_segments(&self, _: Arc>) { + fn set_merge_into_source_build_segments(&self, _: Arc>) { todo!() } @@ -393,9 +394,7 @@ impl TableContext for CtxDelegation { todo!() } - fn get_merge_into_source_build_segments( - &self, - ) -> Arc> { + fn get_merge_into_source_build_segments(&self) -> MergeIntoSourceBuildSegments { todo!() } From 946e994d21a7e0bd703962a3e5528b1815301f21 Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Thu, 21 Mar 2024 22:53:33 +0800 Subject: [PATCH 15/24] remove a.sql --- a.sql | 2 -- 1 file changed, 2 deletions(-) delete mode 100644 a.sql diff --git a/a.sql b/a.sql deleted file mode 100644 index 84b9d25d9e7b3..0000000000000 --- a/a.sql +++ /dev/null @@ -1,2 +0,0 @@ - -explain merge into t1 using (SELECT * EXCLUDE (rn) FROM (SELECT *, row_number() OVER (PARTITION BY id ORDER BY t DESC) AS rn FROM tbcc WHERE a in(1,2,3))) as t2 on t1.a = t2.a when matched then update * when not matched then insert *; From 885be2be5c7f80f603295da358c95442a31de4d5 Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Thu, 21 Mar 2024 23:12:02 +0800 Subject: [PATCH 16/24] fix test --- .../explain/merge_into_source_build_bloom_filter.test | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/sqllogictests/suites/mode/standalone/explain/merge_into_source_build_bloom_filter.test b/tests/sqllogictests/suites/mode/standalone/explain/merge_into_source_build_bloom_filter.test index eac276fd6db30..ae867d39d1834 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/merge_into_source_build_bloom_filter.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/merge_into_source_build_bloom_filter.test @@ -55,17 +55,17 @@ target_table: default.default.target_bloom_table ├── can_merge_into_source_build_bloom: true ├── matched update: [condition: None,update set a = a (#0),b = b (#1)] ├── unmatched insert: [condition: None,insert into (a,b) values(CAST(a (#0) AS String NULL),CAST(b (#1) AS Int32 NULL))] -└── Join(Left) - ├── build keys: [t1.a (#2)] - ├── probe keys: [t2.a (#0)] +└── Join(Right) + ├── build keys: [t2.a (#0)] + ├── probe keys: [t1.a (#2)] ├── other filters: [] ├── Scan - │ ├── table: default.source_bloom_table + │ ├── table: default.target_bloom_table │ ├── filters: [] │ ├── order by: [] │ └── limit: NONE └── Scan - ├── table: default.target_bloom_table + ├── table: default.source_bloom_table ├── filters: [] ├── order by: [] └── limit: NONE From 7636c957a5d4c82c57045c1b90fe1a8cdf75f507 Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Thu, 21 Mar 2024 23:24:00 +0800 Subject: [PATCH 17/24] remove useless codes --- .../fuse/src/operations/read/fuse_source.rs | 27 ------------------- 1 file changed, 27 deletions(-) diff --git a/src/query/storages/fuse/src/operations/read/fuse_source.rs b/src/query/storages/fuse/src/operations/read/fuse_source.rs index fc022ca937cbe..0398ec61c871a 100644 --- a/src/query/storages/fuse/src/operations/read/fuse_source.rs +++ b/src/query/storages/fuse/src/operations/read/fuse_source.rs @@ -150,33 +150,6 @@ pub fn build_fuse_parquet_source_pipeline( (max_threads, max_io_requests) = adjust_threads_and_request(false, max_threads, max_io_requests, plan); - let merge_into_join = ctx.get_merge_into_join(); - if ctx - .get_settings() - .get_enable_merge_into_source_build_bloom()? - && matches!( - merge_into_join.merge_into_join_type, - MergeIntoJoinType::Right - ) - && merge_into_join.target_tbl_idx == plan.table_index - { - // we can add block_metas info for merge into runtime filter, they will - // be used for bloom prune for target table block. - assert!(matches!(plan.source_info, DataSourceInfo::TableSource(_))); - // if let DataSourceInfo::TableSource(table_info) = plan.source_info { - // let table = ctx - // .get_table(table_info.catalog(), table_info.name(), 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 mut source_builder = SourcePipeBuilder::create(); match block_reader.support_blocking_api() { true => { From 58e3a37904eb9466082189e2fa4b94d5fbdf41c3 Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Thu, 21 Mar 2024 23:24:15 +0800 Subject: [PATCH 18/24] remove useless codes --- src/query/storages/fuse/src/operations/read/fuse_source.rs | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/query/storages/fuse/src/operations/read/fuse_source.rs b/src/query/storages/fuse/src/operations/read/fuse_source.rs index 0398ec61c871a..0c175c98bed70 100644 --- a/src/query/storages/fuse/src/operations/read/fuse_source.rs +++ b/src/query/storages/fuse/src/operations/read/fuse_source.rs @@ -15,8 +15,6 @@ use std::collections::VecDeque; use std::sync::Arc; -use databend_common_catalog::merge_into_join::MergeIntoJoinType; -use databend_common_catalog::plan::DataSourceInfo; use databend_common_catalog::plan::DataSourcePlan; use databend_common_catalog::plan::PartInfoPtr; use databend_common_catalog::plan::StealablePartitions; From a8a2971aae900de39d10623b935e53670961baf3 Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Fri, 22 Mar 2024 02:14:11 +0800 Subject: [PATCH 19/24] add order test --- .../operations/read/runtime_filter_prunner.rs | 2 +- .../merge_into_source_build_bloom.test | 79 +++++++++++++++++++ .../merge_into_source_build_bloom_filter.test | 76 ++++++++++++++++++ 3 files changed, 156 insertions(+), 1 deletion(-) diff --git a/src/query/storages/fuse/src/operations/read/runtime_filter_prunner.rs b/src/query/storages/fuse/src/operations/read/runtime_filter_prunner.rs index 5876a93d84abc..7b22e0c5efa9e 100644 --- a/src/query/storages/fuse/src/operations/read/runtime_filter_prunner.rs +++ b/src/query/storages/fuse/src/operations/read/runtime_filter_prunner.rs @@ -210,7 +210,7 @@ pub(crate) fn try_prune_merge_into_target_table( ) })?.schema(); let index = table_schema.index_of(probe_key_name)?; - acc.0.push(index); + acc.0.push(acc.0.len()); acc.1.push(OnConflictField { table_field: table_schema.field(index).clone(), field_index: index }); Ok::<_, ErrorCode>(acc) })?; diff --git a/tests/sqllogictests/suites/mode/cluster/merge_into_source_build_bloom.test b/tests/sqllogictests/suites/mode/cluster/merge_into_source_build_bloom.test index bbf7d7dfdc024..5e6503dae2a90 100644 --- a/tests/sqllogictests/suites/mode/cluster/merge_into_source_build_bloom.test +++ b/tests/sqllogictests/suites/mode/cluster/merge_into_source_build_bloom.test @@ -99,6 +99,85 @@ kgt 27 krrgt 207 mkgt 73 + +statement ok +drop table if exists target_bloom_reorder; + +statement ok +drop table if exists source_bloom_reorder; + +statement ok +create table target_bloom_reorder(a int,b string); + +statement ok +create table source_bloom_reorder(a int,b string); + +statement ok +insert into target_bloom_reorder values(1,'ab'),(2,'cd'); + +statement ok +insert into target_bloom_reorder values(1,'gh'),(2,'ij'); + +statement ok +insert into target_bloom_reorder values(3,'abc'); + +statement ok +insert into source_bloom_reorder values(4,'abc'),(5,'ffff'); + +query T +select count(*) from fuse_block('default','target_bloom_reorder'); +---- +3 + +query T +select count(*) from fuse_block('default','source_bloom_reorder'); +---- +1 + +query T +explain merge into target_bloom_reorder as t1 using source_bloom_reorder as t2 on t1.b = t2.b when matched then update * when not matched then insert *; +---- +MergeInto: +target_table: default.default.target_bloom_reorder +├── distributed: true +├── target_build_optimization: false +├── can_try_update_column_only: true +├── can_merge_into_source_build_bloom: true +├── matched update: [condition: None,update set a = a (#0),b = b (#1)] +├── unmatched insert: [condition: None,insert into (a,b) values(CAST(a (#0) AS Int32 NULL),CAST(b (#1) AS String NULL))] +└── Exchange(Merge) + └── Join(Right) + ├── build keys: [t2.b (#1)] + ├── probe keys: [t1.b (#3)] + ├── other filters: [] + ├── Scan + │ ├── table: default.target_bloom_reorder + │ ├── filters: [] + │ ├── order by: [] + │ └── limit: NONE + └── Exchange(Broadcast) + └── AddRowNumber(AddRowNumber) + └── Scan + ├── table: default.source_bloom_reorder + ├── filters: [] + ├── order by: [] + └── limit: NONE + +query T +merge into target_bloom_reorder as t1 using source_bloom_reorder as t2 on t1.b = t2.b when matched then update * when not matched then insert *; +---- +1 1 + +query TT +select b,a from target_bloom_reorder order by b,a; +---- +ab 1 +abc 4 +cd 2 +ffff 5 +gh 1 +ij 2 + statement ok set enable_merge_into_source_build_bloom = 0; diff --git a/tests/sqllogictests/suites/mode/standalone/explain/merge_into_source_build_bloom_filter.test b/tests/sqllogictests/suites/mode/standalone/explain/merge_into_source_build_bloom_filter.test index ae867d39d1834..ff09da807f525 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/merge_into_source_build_bloom_filter.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/merge_into_source_build_bloom_filter.test @@ -90,6 +90,82 @@ kgt 27 krrgt 207 mkgt 73 +## test col order +statement ok +drop table if exists target_bloom_reorder; + +statement ok +drop table if exists source_bloom_reorder; + +statement ok +create table target_bloom_reorder(a int,b string); + +statement ok +create table source_bloom_reorder(a int,b string); + +statement ok +insert into target_bloom_reorder values(1,'ab'),(2,'cd'); + +statement ok +insert into target_bloom_reorder values(1,'gh'),(2,'ij'); + +statement ok +insert into target_bloom_reorder values(3,'abc'); + +statement ok +insert into source_bloom_reorder values(4,'abc'),(5,'ffff'); + +query T +select count(*) from fuse_block('default','target_bloom_reorder'); +---- +3 + +query T +select count(*) from fuse_block('default','source_bloom_reorder'); +---- +1 + +query T +explain merge into target_bloom_reorder as t1 using source_bloom_reorder as t2 on t1.b = t2.b when matched then update * when not matched then insert *; +---- +MergeInto: +target_table: default.default.target_bloom_reorder +├── distributed: false +├── target_build_optimization: false +├── can_try_update_column_only: true +├── can_merge_into_source_build_bloom: true +├── matched update: [condition: None,update set a = a (#0),b = b (#1)] +├── unmatched insert: [condition: None,insert into (a,b) values(CAST(a (#0) AS Int32 NULL),CAST(b (#1) AS String NULL))] +└── Join(Right) + ├── build keys: [t2.b (#1)] + ├── probe keys: [t1.b (#3)] + ├── other filters: [] + ├── Scan + │ ├── table: default.target_bloom_reorder + │ ├── filters: [] + │ ├── order by: [] + │ └── limit: NONE + └── Scan + ├── table: default.source_bloom_reorder + ├── filters: [] + ├── order by: [] + └── limit: NONE + +query T +merge into target_bloom_reorder as t1 using source_bloom_reorder as t2 on t1.b = t2.b when matched then update * when not matched then insert *; +---- +1 1 + +query TT +select b,a from target_bloom_reorder order by b,a; +---- +ab 1 +abc 4 +cd 2 +ffff 5 +gh 1 +ij 2 + statement ok set enable_merge_into_source_build_bloom = 0; From 6fa01905aa040d30605a9d4e5e32bc4265f232f5 Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Fri, 22 Mar 2024 22:45:34 +0800 Subject: [PATCH 20/24] fix empty hashkeys bugs --- src/query/catalog/src/runtime_filter_info.rs | 5 +- .../operations/read/runtime_filter_prunner.rs | 10 +- .../merge_into_source_build_bloom.test | 117 ++++++++++++++++++ .../merge_into_source_build_bloom_filter.test | 117 ++++++++++++++++++ 4 files changed, 246 insertions(+), 3 deletions(-) diff --git a/src/query/catalog/src/runtime_filter_info.rs b/src/query/catalog/src/runtime_filter_info.rs index 549049d343a34..f674df617c0fa 100644 --- a/src/query/catalog/src/runtime_filter_info.rs +++ b/src/query/catalog/src/runtime_filter_info.rs @@ -76,6 +76,9 @@ impl RuntimeFilterInfo { } pub fn is_empty(&self) -> bool { - self.inlist.is_empty() && self.bloom.is_empty() && self.min_max.is_empty() + self.inlist.is_empty() + && self.bloom.is_empty() + && self.min_max.is_empty() + && self.siphashes.0.len() == 0 } } diff --git a/src/query/storages/fuse/src/operations/read/runtime_filter_prunner.rs b/src/query/storages/fuse/src/operations/read/runtime_filter_prunner.rs index 7b22e0c5efa9e..d5f6f8856577f 100644 --- a/src/query/storages/fuse/src/operations/read/runtime_filter_prunner.rs +++ b/src/query/storages/fuse/src/operations/read/runtime_filter_prunner.rs @@ -115,8 +115,14 @@ pub fn runtime_filter_pruner( return Ok(true); } - // if we can't pruned this block, we can try get siphashkeys if this is a merge into source build - if can_do_merge_into_target_build_bloom_filter { + // if we can't pruned this block, we can try get siphashkeys if this is a merge into source build. + // for every probe key expr, if it's a ColumnRef, we can get the build column hash keys,but if not, + // we can't. so even if we enable this bloom filter, we probally can't do bloom filter in fact. + if can_do_merge_into_target_build_bloom_filter + && ctx + .get_merge_into_source_build_siphashkeys_with_id(id) + .is_some_and(|hash_keys| hash_keys.0.len() > 0) + { let pruned = try_prune_merge_into_target_table( ctx.clone(), part, diff --git a/tests/sqllogictests/suites/mode/cluster/merge_into_source_build_bloom.test b/tests/sqllogictests/suites/mode/cluster/merge_into_source_build_bloom.test index 5e6503dae2a90..7b73ce783246e 100644 --- a/tests/sqllogictests/suites/mode/cluster/merge_into_source_build_bloom.test +++ b/tests/sqllogictests/suites/mode/cluster/merge_into_source_build_bloom.test @@ -178,6 +178,123 @@ ffff 5 gh 1 ij 2 +### test not null column, we can't prune now, but the sql run until to be finished. +statement ok +CREATE TABLE IF NOT EXISTS lineitem_target_origin_200_blocks1 ( + l_orderkey BIGINT not null, + l_partkey BIGINT not null, + l_suppkey BIGINT not null, + l_linenumber BIGINT not null, + l_quantity DECIMAL(15, 2) not null, + l_extendedprice DECIMAL(15, 2) not null, + l_discount DECIMAL(15, 2) not null, + l_tax DECIMAL(15, 2) not null, + l_returnflag STRING not null, + l_linestatus STRING not null, + l_shipdate DATE not null, + l_commitdate DATE not null, + l_receiptdate DATE not null, + l_shipinstruct STRING not null, + l_shipmode STRING not null, + l_comment STRING not null +) CLUSTER BY(l_shipdate, l_orderkey); + +statement ok +CREATE TABLE IF NOT EXISTS lineitem_target_origin_400_blocks1 ( + l_orderkey BIGINT not null, + l_partkey BIGINT not null, + l_suppkey BIGINT not null, + l_linenumber BIGINT not null, + l_quantity DECIMAL(15, 2) not null, + l_extendedprice DECIMAL(15, 2) not null, + l_discount DECIMAL(15, 2) not null, + l_tax DECIMAL(15, 2) not null, + l_returnflag STRING not null, + l_linestatus STRING not null, + l_shipdate DATE not null, + l_commitdate DATE not null, + l_receiptdate DATE not null, + l_shipinstruct STRING not null, + l_shipmode STRING not null, + l_comment STRING not null +) CLUSTER BY(l_shipdate, l_orderkey); + +statement ok +CREATE TABLE IF NOT EXISTS lineitem_random ( + l_orderkey BIGINT not null, + l_partkey BIGINT not null, + l_suppkey BIGINT not null, + l_linenumber BIGINT not null, + l_quantity DECIMAL(15, 2) not null, + l_extendedprice DECIMAL(15, 2) not null, + l_discount DECIMAL(15, 2) not null, + l_tax DECIMAL(15, 2) not null, + l_returnflag STRING not null, + l_linestatus STRING not null, + l_shipdate DATE not null, + l_commitdate DATE not null, + l_receiptdate DATE not null, + l_shipinstruct STRING not null, + l_shipmode STRING not null, + l_comment STRING not null +) engine = random; + +statement ok +insert into lineitem_target_origin_400_blocks1 select * from lineitem_random limit 10; + +statement ok +insert into lineitem_target_origin_400_blocks1 select * from lineitem_random limit 10; + +statement ok +insert into lineitem_target_origin_400_blocks1 select * from lineitem_random limit 10; + +statement ok +insert into lineitem_target_origin_200_blocks1 select * from lineitem_random limit 10; + +statement ok +explain analyze MERGE INTO lineitem_target_origin_400_blocks1 as t1 using lineitem_target_origin_200_blocks1 as t2 on + t1.l_orderkey = t2.l_orderkey and + t1.l_partkey = t2.l_partkey and t1.l_suppkey = t2.l_suppkey and + t1.l_linenumber = t2.l_linenumber and + t1.l_quantity = t2.l_quantity and + t1.l_extendedprice = t2.l_extendedprice and + t1.l_discount = t2.l_discount + when matched then update set + t1.l_orderkey = t2.l_orderkey, + t1.l_partkey = t2.l_partkey, + t1.l_suppkey = t2.l_suppkey, + t1.l_linenumber = t2.l_linenumber, + t1.l_quantity = t2.l_quantity, + t1.l_extendedprice = t2.l_extendedprice, + t1.l_discount = t2.l_discount, + t1.l_tax = t2.l_tax, + t1.l_returnflag = t2.l_returnflag, + t1.l_linestatus = t2.l_linestatus, + t1.l_shipdate = t2.l_shipdate, + t1.l_commitdate = t2.l_commitdate, + t1.l_receiptdate = t2.l_receiptdate, + t1.l_shipinstruct = t2.l_shipinstruct, + t1.l_shipmode = t2.l_shipmode, + t1.l_comment = t2.l_comment + when not matched then insert + values( + t2.l_orderkey, + t2.l_partkey, + t2.l_suppkey, + t2.l_linenumber, + t2.l_quantity, + t2.l_extendedprice, + t2.l_discount, + t2.l_tax, + t2.l_returnflag, + t2.l_linestatus, + t2.l_shipdate, + t2.l_commitdate, + t2.l_receiptdate, + t2.l_shipinstruct, + t2.l_shipmode, + t2.l_comment); + statement ok set enable_merge_into_source_build_bloom = 0; diff --git a/tests/sqllogictests/suites/mode/standalone/explain/merge_into_source_build_bloom_filter.test b/tests/sqllogictests/suites/mode/standalone/explain/merge_into_source_build_bloom_filter.test index ff09da807f525..0e1b1a6d70e65 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/merge_into_source_build_bloom_filter.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/merge_into_source_build_bloom_filter.test @@ -166,6 +166,123 @@ ffff 5 gh 1 ij 2 +### test not null column, we can't prune now, but the sql run until to be finished. +statement ok +CREATE TABLE IF NOT EXISTS lineitem_target_origin_200_blocks1 ( + l_orderkey BIGINT not null, + l_partkey BIGINT not null, + l_suppkey BIGINT not null, + l_linenumber BIGINT not null, + l_quantity DECIMAL(15, 2) not null, + l_extendedprice DECIMAL(15, 2) not null, + l_discount DECIMAL(15, 2) not null, + l_tax DECIMAL(15, 2) not null, + l_returnflag STRING not null, + l_linestatus STRING not null, + l_shipdate DATE not null, + l_commitdate DATE not null, + l_receiptdate DATE not null, + l_shipinstruct STRING not null, + l_shipmode STRING not null, + l_comment STRING not null +) CLUSTER BY(l_shipdate, l_orderkey); + +statement ok +CREATE TABLE IF NOT EXISTS lineitem_target_origin_400_blocks1 ( + l_orderkey BIGINT not null, + l_partkey BIGINT not null, + l_suppkey BIGINT not null, + l_linenumber BIGINT not null, + l_quantity DECIMAL(15, 2) not null, + l_extendedprice DECIMAL(15, 2) not null, + l_discount DECIMAL(15, 2) not null, + l_tax DECIMAL(15, 2) not null, + l_returnflag STRING not null, + l_linestatus STRING not null, + l_shipdate DATE not null, + l_commitdate DATE not null, + l_receiptdate DATE not null, + l_shipinstruct STRING not null, + l_shipmode STRING not null, + l_comment STRING not null +) CLUSTER BY(l_shipdate, l_orderkey); + +statement ok +CREATE TABLE IF NOT EXISTS lineitem_random ( + l_orderkey BIGINT not null, + l_partkey BIGINT not null, + l_suppkey BIGINT not null, + l_linenumber BIGINT not null, + l_quantity DECIMAL(15, 2) not null, + l_extendedprice DECIMAL(15, 2) not null, + l_discount DECIMAL(15, 2) not null, + l_tax DECIMAL(15, 2) not null, + l_returnflag STRING not null, + l_linestatus STRING not null, + l_shipdate DATE not null, + l_commitdate DATE not null, + l_receiptdate DATE not null, + l_shipinstruct STRING not null, + l_shipmode STRING not null, + l_comment STRING not null +) engine = random; + +statement ok +insert into lineitem_target_origin_400_blocks1 select * from lineitem_random limit 10; + +statement ok +insert into lineitem_target_origin_400_blocks1 select * from lineitem_random limit 10; + +statement ok +insert into lineitem_target_origin_400_blocks1 select * from lineitem_random limit 10; + +statement ok +insert into lineitem_target_origin_200_blocks1 select * from lineitem_random limit 10; + +statement ok +explain analyze MERGE INTO lineitem_target_origin_400_blocks1 as t1 using lineitem_target_origin_200_blocks1 as t2 on + t1.l_orderkey = t2.l_orderkey and + t1.l_partkey = t2.l_partkey and t1.l_suppkey = t2.l_suppkey and + t1.l_linenumber = t2.l_linenumber and + t1.l_quantity = t2.l_quantity and + t1.l_extendedprice = t2.l_extendedprice and + t1.l_discount = t2.l_discount + when matched then update set + t1.l_orderkey = t2.l_orderkey, + t1.l_partkey = t2.l_partkey, + t1.l_suppkey = t2.l_suppkey, + t1.l_linenumber = t2.l_linenumber, + t1.l_quantity = t2.l_quantity, + t1.l_extendedprice = t2.l_extendedprice, + t1.l_discount = t2.l_discount, + t1.l_tax = t2.l_tax, + t1.l_returnflag = t2.l_returnflag, + t1.l_linestatus = t2.l_linestatus, + t1.l_shipdate = t2.l_shipdate, + t1.l_commitdate = t2.l_commitdate, + t1.l_receiptdate = t2.l_receiptdate, + t1.l_shipinstruct = t2.l_shipinstruct, + t1.l_shipmode = t2.l_shipmode, + t1.l_comment = t2.l_comment + when not matched then insert + values( + t2.l_orderkey, + t2.l_partkey, + t2.l_suppkey, + t2.l_linenumber, + t2.l_quantity, + t2.l_extendedprice, + t2.l_discount, + t2.l_tax, + t2.l_returnflag, + t2.l_linestatus, + t2.l_shipdate, + t2.l_commitdate, + t2.l_receiptdate, + t2.l_shipinstruct, + t2.l_shipmode, + t2.l_comment); + statement ok set enable_merge_into_source_build_bloom = 0; From 8b44a29c28bcd9e368e2062cf1940d34812766c1 Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Fri, 22 Mar 2024 22:57:24 +0800 Subject: [PATCH 21/24] fix lint --- .../storages/fuse/src/operations/read/runtime_filter_prunner.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/query/storages/fuse/src/operations/read/runtime_filter_prunner.rs b/src/query/storages/fuse/src/operations/read/runtime_filter_prunner.rs index d5f6f8856577f..272f86d52948d 100644 --- a/src/query/storages/fuse/src/operations/read/runtime_filter_prunner.rs +++ b/src/query/storages/fuse/src/operations/read/runtime_filter_prunner.rs @@ -121,7 +121,7 @@ pub fn runtime_filter_pruner( if can_do_merge_into_target_build_bloom_filter && ctx .get_merge_into_source_build_siphashkeys_with_id(id) - .is_some_and(|hash_keys| hash_keys.0.len() > 0) + .is_some_and(|hash_keys| !hash_keys.0.is_empty()) { let pruned = try_prune_merge_into_target_table( ctx.clone(), From 3deb681165c5cc3259710667ae41bc35bc6c0ae7 Mon Sep 17 00:00:00 2001 From: dantengsky Date: Mon, 25 Mar 2024 21:05:53 +0800 Subject: [PATCH 22/24] lint --- .../fuse/src/operations/read/parquet_data_source_reader.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 ac20472b606e8..5bd8122f069ef 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,9 +33,9 @@ use databend_common_sql::IndexType; use log::debug; use super::parquet_data_source::ParquetDataSource; -use crate::fuse_part::FuseBlockPartInfo; use super::util::build_merge_into_source_build_bloom_info; use super::util::MergeIntoSourceBuildBloomInfo; +use crate::fuse_part::FuseBlockPartInfo; use crate::io::AggIndexReader; use crate::io::BlockReader; use crate::io::ReadSettings; From 5ba2c66e1cabad949455343991f8a1fb1b8d6558 Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Sat, 30 Mar 2024 19:25:25 +0800 Subject: [PATCH 23/24] fix bloom, thanks @dantengsky --- .../transforms/hash_join/hash_join_build_state.rs | 11 ++++++++++- src/query/service/src/sessions/query_ctx.rs | 7 +++++++ 2 files changed, 17 insertions(+), 1 deletion(-) 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 c55ed5dec02f9..3fd274c3980b1 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 @@ -947,7 +947,16 @@ impl HashJoinBuildState { // `calculate_nullable_column_digest`, `apply_bloom_pruning` let merge_type = self.ctx.get_merge_into_join(); if matches!(merge_type.merge_into_join_type, MergeIntoJoinType::Right) { - if let Expr::ColumnRef { id, .. } = probe_key { + let id = match probe_key { + Expr::ColumnRef { id, .. } => Some(id), + Expr::Cast { + expr: box Expr::ColumnRef { id, .. }, + .. + } => Some(id), + _ => None, + }; + + if let Some(id) = id { let mut columns = Vec::with_capacity(data_blocks.len()); for block in data_blocks.iter() { if block.num_columns() == 0 { diff --git a/src/query/service/src/sessions/query_ctx.rs b/src/query/service/src/sessions/query_ctx.rs index e0f21e30a5588..431d8eeeb0a6f 100644 --- a/src/query/service/src/sessions/query_ctx.rs +++ b/src/query/service/src/sessions/query_ctx.rs @@ -1000,6 +1000,13 @@ impl TableContext for QueryContext { for filter in filters.1.get_min_max() { v.get_mut().add_min_max(filter.clone()); } + let (keys, siphashkeys) = filters.1.get_merge_into_source_build_siphashkeys(); + for (idx, key) in keys.into_iter().enumerate() { + v.get_mut().add_merge_into_source_build_siphashkeys(( + key, + siphashkeys.read()[idx].clone(), + )) + } for filter in filters.1.blooms() { v.get_mut().add_bloom(filter); } From 8aa5d5bf8c90970e9d0214915cf4cb4b0fa5856f Mon Sep 17 00:00:00 2001 From: JackTan25 Date: Sat, 30 Mar 2024 19:46:51 +0800 Subject: [PATCH 24/24] fix lint --- src/query/service/src/interpreters/interpreter_explain.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/src/query/service/src/interpreters/interpreter_explain.rs b/src/query/service/src/interpreters/interpreter_explain.rs index 2b6db45ed43ef..e590622370600 100644 --- a/src/query/service/src/interpreters/interpreter_explain.rs +++ b/src/query/service/src/interpreters/interpreter_explain.rs @@ -185,6 +185,7 @@ impl Interpreter for ExplainInterpreter { fuse_table.read_table_snapshot().await?.unwrap_or_else(|| { Arc::new(TableSnapshot::new_empty_snapshot( fuse_table.schema().as_ref().clone(), + None, )) }); self.ctx.set_merge_into_source_build_segments(Arc::new(