diff --git a/src/query/catalog/src/runtime_filter_info.rs b/src/query/catalog/src/runtime_filter_info.rs index 5687f8f19e32e..344e85f9ebafc 100644 --- a/src/query/catalog/src/runtime_filter_info.rs +++ b/src/query/catalog/src/runtime_filter_info.rs @@ -14,6 +14,9 @@ use std::fmt::Debug; use std::fmt::Formatter; +use std::sync::atomic::AtomicU64; +use std::sync::atomic::Ordering; +use std::sync::Arc; use databend_common_base::base::tokio::sync::watch; use databend_common_base::base::tokio::sync::watch::Receiver; @@ -23,42 +26,108 @@ use xorf::BinaryFuse16; #[derive(Clone, Default)] pub struct RuntimeFilterInfo { - pub inlist: Vec>, - pub min_max: Vec>, - pub bloom: Vec<(String, BinaryFuse16)>, + pub filters: Vec, } impl Debug for RuntimeFilterInfo { fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { write!( f, - "RuntimeFilterInfo {{ inlist: {}, min_max: {}, bloom: {:?} }}", - self.inlist + "RuntimeFilterInfo {{ filters: [{}] }}", + self.filters .iter() - .map(|e| e.sql_display()) + .map(|entry| format!("#{}(probe:{})", entry.id, entry.probe_expr.sql_display())) .collect::>() - .join(","), - self.min_max - .iter() - .map(|e| e.sql_display()) - .collect::>() - .join(","), - self.bloom - .iter() - .map(|(name, _)| name) - .collect::>() + .join(",") ) } } impl RuntimeFilterInfo { pub fn is_empty(&self) -> bool { - self.inlist.is_empty() && self.bloom.is_empty() && self.min_max.is_empty() + self.filters.is_empty() } pub fn is_blooms_empty(&self) -> bool { - self.bloom.is_empty() + self.filters.iter().all(|entry| entry.bloom.is_none()) + } +} + +#[derive(Clone)] +pub struct RuntimeFilterEntry { + pub id: usize, + pub probe_expr: Expr, + pub bloom: Option, + pub inlist: Option>, + pub min_max: Option>, + pub stats: Arc, + pub build_rows: usize, + pub build_table_rows: Option, + pub enabled: bool, +} + +#[derive(Clone)] +pub struct RuntimeFilterBloom { + pub column_name: String, + pub filter: BinaryFuse16, +} + +#[derive(Default)] +pub struct RuntimeFilterStats { + bloom_time_ns: AtomicU64, + bloom_rows_filtered: AtomicU64, + inlist_min_max_time_ns: AtomicU64, + min_max_rows_filtered: AtomicU64, + min_max_partitions_pruned: AtomicU64, +} + +impl RuntimeFilterStats { + pub fn new() -> Self { + Self::default() } + + pub fn record_bloom(&self, time_ns: u64, rows_filtered: u64) { + self.bloom_time_ns.fetch_add(time_ns, Ordering::Relaxed); + self.bloom_rows_filtered + .fetch_add(rows_filtered, Ordering::Relaxed); + } + + pub fn record_inlist_min_max(&self, time_ns: u64, rows_filtered: u64, partitions_pruned: u64) { + self.inlist_min_max_time_ns + .fetch_add(time_ns, Ordering::Relaxed); + self.min_max_rows_filtered + .fetch_add(rows_filtered, Ordering::Relaxed); + self.min_max_partitions_pruned + .fetch_add(partitions_pruned, Ordering::Relaxed); + } + + pub fn snapshot(&self) -> RuntimeFilterStatsSnapshot { + RuntimeFilterStatsSnapshot { + bloom_time_ns: self.bloom_time_ns.load(Ordering::Relaxed), + bloom_rows_filtered: self.bloom_rows_filtered.load(Ordering::Relaxed), + inlist_min_max_time_ns: self.inlist_min_max_time_ns.load(Ordering::Relaxed), + min_max_rows_filtered: self.min_max_rows_filtered.load(Ordering::Relaxed), + min_max_partitions_pruned: self.min_max_partitions_pruned.load(Ordering::Relaxed), + } + } +} + +#[derive(Default, Clone, Debug)] +pub struct RuntimeFilterStatsSnapshot { + pub bloom_time_ns: u64, + pub bloom_rows_filtered: u64, + pub inlist_min_max_time_ns: u64, + pub min_max_rows_filtered: u64, + pub min_max_partitions_pruned: u64, +} + +#[derive(Clone, Debug)] +pub struct RuntimeFilterReport { + pub filter_id: usize, + pub has_bloom: bool, + pub has_inlist: bool, + pub has_min_max: bool, + pub stats: RuntimeFilterStatsSnapshot, } pub struct RuntimeFilterReady { diff --git a/src/query/catalog/src/table_context.rs b/src/query/catalog/src/table_context.rs index da3ad982e6e62..805a7d85def47 100644 --- a/src/query/catalog/src/table_context.rs +++ b/src/query/catalog/src/table_context.rs @@ -78,8 +78,10 @@ use crate::plan::PartInfoPtr; use crate::plan::PartStatistics; use crate::plan::Partitions; use crate::query_kind::QueryKind; +use crate::runtime_filter_info::RuntimeFilterEntry; use crate::runtime_filter_info::RuntimeFilterInfo; use crate::runtime_filter_info::RuntimeFilterReady; +use crate::runtime_filter_info::RuntimeFilterReport; use crate::session_type::SessionType; use crate::statistics::data_cache_statistics::DataCacheMetrics; use crate::table::Table; @@ -357,22 +359,25 @@ pub trait TableContext: Send + Sync { fn get_runtime_filter_ready(&self, table_index: usize) -> Vec>; - fn set_wait_runtime_filter(&self, table_index: usize, need_to_wait: bool); - - fn get_wait_runtime_filter(&self, table_index: usize) -> bool; - fn clear_runtime_filter(&self); + fn assert_no_runtime_filter_state(&self) -> Result<()> { + unimplemented!() + } fn set_merge_into_join(&self, join: MergeIntoJoin); fn get_merge_into_join(&self) -> MergeIntoJoin; + fn get_runtime_filters(&self, id: usize) -> Vec; + fn get_bloom_runtime_filter_with_id(&self, id: usize) -> Vec<(String, BinaryFuse16)>; fn get_inlist_runtime_filter_with_id(&self, id: usize) -> Vec>; fn get_min_max_runtime_filter_with_id(&self, id: usize) -> Vec>; + fn runtime_filter_reports(&self) -> HashMap>; + fn has_bloom_runtime_filters(&self, id: usize) -> bool; fn txn_mgr(&self) -> TxnManagerRef; fn get_table_meta_timestamps( diff --git a/src/query/service/src/interpreters/interpreter_explain.rs b/src/query/service/src/interpreters/interpreter_explain.rs index bbb482ce75987..9244fde6dce10 100644 --- a/src/query/service/src/interpreters/interpreter_explain.rs +++ b/src/query/service/src/interpreters/interpreter_explain.rs @@ -180,6 +180,7 @@ impl Interpreter for ExplainInterpreter { profs: HashMap::new(), metadata: &metadata, scan_id_to_runtime_filters: HashMap::new(), + runtime_filter_reports: HashMap::new(), }; let formatter = plan.formatter()?; @@ -483,6 +484,8 @@ impl ExplainInterpreter { plan.set_pruning_stats(&mut pruned_partitions_stats); } + let runtime_filter_reports = self.ctx.runtime_filter_reports(); + let result = match self.partial { true => { let metadata = metadata.read(); @@ -490,6 +493,7 @@ impl ExplainInterpreter { profs: query_profiles.clone(), metadata: &metadata, scan_id_to_runtime_filters: HashMap::new(), + runtime_filter_reports: runtime_filter_reports.clone(), }; let formatter = plan.formatter()?; @@ -498,8 +502,15 @@ impl ExplainInterpreter { } false => { let metadata = metadata.read(); - plan.format(&metadata, query_profiles.clone())? - .format_pretty()? + let mut context = FormatContext { + profs: query_profiles.clone(), + metadata: &metadata, + scan_id_to_runtime_filters: HashMap::new(), + runtime_filter_reports: runtime_filter_reports.clone(), + }; + let formatter = plan.formatter()?; + let format_node = formatter.format(&mut context)?; + format_node.format_pretty()? } }; diff --git a/src/query/service/src/physical_plans/format/common.rs b/src/query/service/src/physical_plans/format/common.rs index a37e79902c893..f4e28144d7c28 100644 --- a/src/query/service/src/physical_plans/format/common.rs +++ b/src/query/service/src/physical_plans/format/common.rs @@ -18,6 +18,7 @@ use databend_common_ast::ast::FormatTreeNode; use databend_common_base::base::format_byte_size; use databend_common_base::runtime::profile::get_statistics_desc; use databend_common_catalog::plan::PartStatistics; +use databend_common_catalog::runtime_filter_info::RuntimeFilterReport; use databend_common_expression::DataSchemaRef; use databend_common_sql::executor::physical_plans::AggregateFunctionDesc; use databend_common_sql::IndexType; @@ -32,6 +33,7 @@ pub struct FormatContext<'a> { pub metadata: &'a Metadata, pub profs: HashMap, pub scan_id_to_runtime_filters: HashMap>, + pub runtime_filter_reports: HashMap>, } pub fn pretty_display_agg_desc(desc: &AggregateFunctionDesc, metadata: &Metadata) -> String { diff --git a/src/query/service/src/physical_plans/format/format_hash_join.rs b/src/query/service/src/physical_plans/format/format_hash_join.rs index 4711d6b573ad4..bbfe12af626b6 100644 --- a/src/query/service/src/physical_plans/format/format_hash_join.rs +++ b/src/query/service/src/physical_plans/format/format_hash_join.rs @@ -43,11 +43,14 @@ impl<'a> PhysicalFormat for HashJoinFormatter<'a> { #[recursive::recursive] fn format(&self, ctx: &mut FormatContext<'_>) -> Result> { + // Register runtime filters for all probe targets for rf in self.inner.runtime_filter.filters.iter() { - ctx.scan_id_to_runtime_filters - .entry(rf.scan_id) - .or_default() - .push(rf.clone()); + for (_probe_key, scan_id) in &rf.probe_targets { + ctx.scan_id_to_runtime_filters + .entry(*scan_id) + .or_default() + .push(rf.clone()); + } } let build_keys = self @@ -83,11 +86,25 @@ impl<'a> PhysicalFormat for HashJoinFormatter<'a> { let mut build_runtime_filters = vec![]; for rf in self.inner.runtime_filter.filters.iter() { + // Format all probe targets + let probe_targets_str = rf + .probe_targets + .iter() + .map(|(probe_key, scan_id)| { + format!( + "{}@scan{}", + probe_key.as_expr(&BUILTIN_FUNCTIONS).sql_display(), + scan_id + ) + }) + .collect::>() + .join(", "); + let mut s = format!( - "filter id:{}, build key:{}, probe key:{}, filter type:", + "filter id:{}, build key:{}, probe targets:[{}], filter type:", rf.id, rf.build_key.as_expr(&BUILTIN_FUNCTIONS).sql_display(), - rf.probe_key.as_expr(&BUILTIN_FUNCTIONS).sql_display(), + probe_targets_str, ); if rf.enable_bloom_runtime_filter { s += "bloom,"; diff --git a/src/query/service/src/physical_plans/format/format_table_scan.rs b/src/query/service/src/physical_plans/format/format_table_scan.rs index 4a3a4b8fd33da..83515e96e89b8 100644 --- a/src/query/service/src/physical_plans/format/format_table_scan.rs +++ b/src/query/service/src/physical_plans/format/format_table_scan.rs @@ -105,6 +105,7 @@ impl<'a> PhysicalFormat for TableScanFormatter<'a> { let mut children = vec![ FormatTreeNode::new(format!("table: {table_name}")), + FormatTreeNode::new(format!("scan id: {}", self.inner.scan_id)), FormatTreeNode::new(format!( "output columns: [{}]", format_output_columns(self.inner.output_schema()?, ctx.metadata, false) diff --git a/src/query/service/src/physical_plans/mod.rs b/src/query/service/src/physical_plans/mod.rs index f8661a3f16632..c2901b8a81c3d 100644 --- a/src/query/service/src/physical_plans/mod.rs +++ b/src/query/service/src/physical_plans/mod.rs @@ -35,7 +35,6 @@ mod physical_expression_scan; mod physical_filter; mod physical_hash_join; mod physical_join; -mod physical_join_filter; mod physical_limit; mod physical_multi_table_insert; mod physical_mutation; @@ -59,6 +58,7 @@ mod physical_udf; mod physical_union_all; mod physical_window; mod physical_window_partition; +mod runtime_filter; pub use physical_add_stream_column::AddStreamColumn; pub use physical_aggregate_expand::AggregateExpand; @@ -82,9 +82,6 @@ pub use physical_exchange_sink::ExchangeSink; pub use physical_exchange_source::ExchangeSource; pub use physical_filter::Filter; pub use physical_hash_join::HashJoin; -pub use physical_join_filter::JoinRuntimeFilter; -pub use physical_join_filter::PhysicalRuntimeFilter; -pub use physical_join_filter::PhysicalRuntimeFilters; pub use physical_limit::Limit; pub use physical_materialized_cte::*; pub use physical_multi_table_insert::*; @@ -110,6 +107,8 @@ pub use physical_udf::UdfFunctionDesc; pub use physical_union_all::UnionAll; pub use physical_window::*; pub use physical_window_partition::*; +pub use runtime_filter::PhysicalRuntimeFilter; +pub use runtime_filter::PhysicalRuntimeFilters; pub mod explain; mod format; diff --git a/src/query/service/src/physical_plans/physical_aggregate_partial.rs b/src/query/service/src/physical_plans/physical_aggregate_partial.rs index b97d0c4712558..6c52a14171f92 100644 --- a/src/query/service/src/physical_plans/physical_aggregate_partial.rs +++ b/src/query/service/src/physical_plans/physical_aggregate_partial.rs @@ -167,7 +167,6 @@ impl IPhysicalPlan for AggregatePartial { } fn build_pipeline2(&self, builder: &mut PipelineBuilder) -> Result<()> { - builder.contain_sink_processor = true; self.input.build_pipeline(builder)?; let max_block_size = builder.settings.get_max_block_size()?; diff --git a/src/query/service/src/physical_plans/physical_hash_join.rs b/src/query/service/src/physical_plans/physical_hash_join.rs index 192f0a36f0ba1..fa15700cfac96 100644 --- a/src/query/service/src/physical_plans/physical_hash_join.rs +++ b/src/query/service/src/physical_plans/physical_hash_join.rs @@ -44,8 +44,7 @@ use databend_common_sql::ScalarExpr; use databend_common_sql::TypeCheck; use tokio::sync::Barrier; -use super::physical_join_filter::PhysicalRuntimeFilters; -use super::JoinRuntimeFilter; +use super::runtime_filter::PhysicalRuntimeFilters; use super::PhysicalPlanCast; use crate::physical_plans::explain::PlanStatsInfo; use crate::physical_plans::format::HashJoinFormatter; @@ -53,6 +52,7 @@ use crate::physical_plans::format::PhysicalFormat; use crate::physical_plans::physical_plan::IPhysicalPlan; use crate::physical_plans::physical_plan::PhysicalPlan; use crate::physical_plans::physical_plan::PhysicalPlanMeta; +use crate::physical_plans::runtime_filter::build_runtime_filter; use crate::physical_plans::Exchange; use crate::physical_plans::PhysicalPlanBuilder; use crate::pipelines::processors::transforms::HashJoinFactory; @@ -72,8 +72,9 @@ type JoinConditionsResult = ( Vec, Vec, Vec, - Vec, usize, usize)>>, + Vec, usize, usize, IndexType)>>, Vec<((usize, bool), usize)>, + Vec>, ); type ProjectionsResult = ( @@ -82,6 +83,15 @@ type ProjectionsResult = ( Option<(usize, HashMap)>, ); +/// Type alias for runtime filter expression result +/// Contains: (Expr, scan_id, table_index, column_idx) +type RuntimeFilterExpr = Option<( + databend_common_expression::Expr, + usize, + usize, + IndexType, +)>; + type MergedFieldsResult = ( Vec, Vec, @@ -668,7 +678,7 @@ impl PhysicalPlanBuilder { fn prepare_runtime_filter_expr( &self, left_condition: &ScalarExpr, - ) -> Result, usize, usize)>> { + ) -> Result { // Runtime filter only supports columns in base tables if left_condition.used_columns().iter().all(|idx| { matches!( @@ -697,6 +707,7 @@ impl PhysicalPlanBuilder { .project_column_ref(|col| Ok(col.column_name.clone()))?, scan_id, table_index, + *column_idx, ))); } } @@ -780,6 +791,7 @@ impl PhysicalPlanBuilder { let mut is_null_equal = Vec::new(); let mut left_join_conditions_rt = Vec::new(); let mut probe_to_build_index = Vec::new(); + let mut build_table_indexes = Vec::new(); let cast_rules = &BUILTIN_FUNCTIONS.get_auto_cast_rules("eq"); for condition in join.equi_conditions.iter() { @@ -797,6 +809,20 @@ impl PhysicalPlanBuilder { // Prepare runtime filter expression let left_expr_for_runtime_filter = self.prepare_runtime_filter_expr(left_condition)?; + let build_table_index = if right_condition.used_columns().len() == 1 { + let column_idx = *right_condition.used_columns().iter().next().unwrap(); + if matches!( + self.metadata.read().column(column_idx), + ColumnEntry::BaseTableColumn(_) + ) { + self.metadata.read().column(column_idx).table_index() + } else { + None + } + } else { + None + }; + // Handle inner join column optimization if matches!(join.join_type, JoinType::Inner | JoinType::InnerAny) { self.handle_inner_join_column_optimization( @@ -839,9 +865,9 @@ impl PhysicalPlanBuilder { // Process runtime filter expressions let left_expr_for_runtime_filter = left_expr_for_runtime_filter - .map(|(expr, scan_id, table_index)| { + .map(|(expr, scan_id, table_index, column_idx)| { check_cast(expr.span(), false, expr, &common_ty, &BUILTIN_FUNCTIONS) - .map(|casted_expr| (casted_expr, scan_id, table_index)) + .map(|casted_expr| (casted_expr, scan_id, table_index, column_idx)) }) .transpose()?; @@ -852,11 +878,12 @@ impl PhysicalPlanBuilder { ConstantFolder::fold(&right_expr, &self.func_ctx, &BUILTIN_FUNCTIONS); let left_expr_for_runtime_filter = - left_expr_for_runtime_filter.map(|(expr, scan_id, table_index)| { + left_expr_for_runtime_filter.map(|(expr, scan_id, table_index, column_idx)| { ( ConstantFolder::fold(&expr, &self.func_ctx, &BUILTIN_FUNCTIONS).0, scan_id, table_index, + column_idx, ) }); @@ -864,11 +891,12 @@ impl PhysicalPlanBuilder { left_join_conditions.push(left_expr.as_remote_expr()); right_join_conditions.push(right_expr.as_remote_expr()); is_null_equal.push(condition.is_null_equal); - left_join_conditions_rt.push( - left_expr_for_runtime_filter.map(|(expr, scan_id, table_index)| { - (expr.as_remote_expr(), scan_id, table_index) - }), - ); + left_join_conditions_rt.push(left_expr_for_runtime_filter.map( + |(expr, scan_id, table_index, column_idx)| { + (expr.as_remote_expr(), scan_id, table_index, column_idx) + }, + )); + build_table_indexes.push(build_table_index); } Ok(( @@ -877,6 +905,7 @@ impl PhysicalPlanBuilder { is_null_equal, left_join_conditions_rt, probe_to_build_index, + build_table_indexes, )) } @@ -1262,6 +1291,7 @@ impl PhysicalPlanBuilder { is_null_equal, left_join_conditions_rt, mut probe_to_build_index, + build_table_indexes, ) = self.process_equi_conditions( join, &probe_schema, @@ -1301,14 +1331,16 @@ impl PhysicalPlanBuilder { let non_equi_conditions = self.process_non_equi_conditions(join, &merged_schema)?; // Step 11: Build runtime filter - let runtime_filter = self - .build_runtime_filter( - join, - s_expr, - &right_join_conditions, - left_join_conditions_rt, - ) - .await?; + let runtime_filter = build_runtime_filter( + self.ctx.clone(), + &self.metadata, + join, + s_expr, + &right_join_conditions, + left_join_conditions_rt, + build_table_indexes, + ) + .await?; // Step 12: Create and return the HashJoin self.create_hash_join( @@ -1330,22 +1362,4 @@ impl PhysicalPlanBuilder { stat_info, ) } - - async fn build_runtime_filter( - &self, - join: &Join, - s_expr: &SExpr, - build_keys: &[RemoteExpr], - probe_keys: Vec, usize, usize)>>, - ) -> Result { - JoinRuntimeFilter::build_runtime_filter( - self.ctx.clone(), - &self.metadata, - join, - s_expr, - build_keys, - probe_keys, - ) - .await - } } diff --git a/src/query/service/src/physical_plans/physical_join_filter.rs b/src/query/service/src/physical_plans/physical_join_filter.rs deleted file mode 100644 index ec1ed306e0c3b..0000000000000 --- a/src/query/service/src/physical_plans/physical_join_filter.rs +++ /dev/null @@ -1,216 +0,0 @@ -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::sync::Arc; - -use databend_common_catalog::table_context::TableContext; -use databend_common_catalog::TableStatistics; -use databend_common_exception::Result; -use databend_common_expression::types::DataType; -use databend_common_expression::RemoteExpr; -use databend_common_functions::BUILTIN_FUNCTIONS; -use databend_common_sql::optimizer::ir::RelExpr; -use databend_common_sql::optimizer::ir::SExpr; -use databend_common_sql::plans::Exchange; -use databend_common_sql::plans::Join; -use databend_common_sql::plans::JoinType; -use databend_common_sql::IndexType; -use databend_common_sql::MetadataRef; -use databend_storages_common_table_meta::table::get_change_type; - -#[derive(Clone, Debug, serde::Serialize, serde::Deserialize, Default)] -pub struct PhysicalRuntimeFilters { - pub filters: Vec, -} - -#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] -pub struct PhysicalRuntimeFilter { - pub id: usize, - pub build_key: RemoteExpr, - pub probe_key: RemoteExpr, - pub scan_id: usize, - pub enable_bloom_runtime_filter: bool, - pub enable_inlist_runtime_filter: bool, - pub enable_min_max_runtime_filter: bool, -} - -/// JoinRuntimeFilter handles the creation and adjustment of runtime filters for join operations -pub struct JoinRuntimeFilter; - -impl JoinRuntimeFilter { - /// Adjust the bloom runtime filter based on the table statistics - async fn adjust_bloom_runtime_filter( - ctx: Arc, - metadata: &MetadataRef, - table_index: Option, - s_expr: &SExpr, - ) -> Result { - // Early return if bloom runtime filter is disabled in settings - if !Self::is_bloom_filter_enabled(ctx.clone())? { - return Ok(false); - } - - // Check if we have a valid table index - if let Some(table_index) = table_index { - return Self::evaluate_bloom_filter_for_table(ctx, metadata, table_index, s_expr).await; - } - - Ok(false) - } - - /// Check if bloom runtime filter is enabled in settings - fn is_bloom_filter_enabled(ctx: Arc) -> Result { - ctx.get_settings().get_bloom_runtime_filter() - } - - /// Evaluate if bloom filter should be used based on table statistics - async fn evaluate_bloom_filter_for_table( - ctx: Arc, - metadata: &MetadataRef, - table_index: IndexType, - s_expr: &SExpr, - ) -> Result { - let table_entry = metadata.read().table(table_index).clone(); - let change_type = get_change_type(table_entry.alias_name()); - let table = table_entry.table(); - - // Get table statistics - if let Some(stats) = table - .table_statistics(ctx.clone(), true, change_type) - .await? - { - return Self::compare_cardinality_with_stats(stats, s_expr); - } - - Ok(false) - } - - /// Compare join cardinality with table statistics to determine if bloom filter is beneficial - fn compare_cardinality_with_stats(stats: TableStatistics, s_expr: &SExpr) -> Result { - const BLOOM_FILTER_SIZE_REDUCTION_FACTOR: u64 = 1000; - - if let Some(num_rows) = stats.num_rows { - let join_cardinality = RelExpr::with_s_expr(s_expr) - .derive_cardinality()? - .cardinality; - - // If the filtered data reduces to less than 1/BLOOM_FILTER_SIZE_REDUCTION_FACTOR of the original dataset, - // we will enable bloom runtime filter. - return Ok(join_cardinality <= (num_rows / BLOOM_FILTER_SIZE_REDUCTION_FACTOR) as f64); - } - - Ok(false) - } - - /// Check if a data type is supported for bloom filter - fn is_type_supported_for_bloom_filter(data_type: &DataType) -> bool { - data_type.is_number() || data_type.is_string() - } - - /// Check if a data type is supported for min-max filter - fn is_type_supported_for_min_max_filter(data_type: &DataType) -> bool { - data_type.is_number() || data_type.is_date() || data_type.is_string() - } - - /// Check if the join type is supported for runtime filter - fn supported_join_type_for_runtime_filter(join_type: &JoinType) -> bool { - matches!( - join_type, - JoinType::Inner - | JoinType::Right - | JoinType::RightSemi - | JoinType::RightAnti - | JoinType::LeftMark - ) - } - - /// Build runtime filters for a join operation - pub async fn build_runtime_filter( - ctx: Arc, - metadata: &MetadataRef, - join: &Join, - s_expr: &SExpr, - build_keys: &[RemoteExpr], - probe_keys: Vec, usize, usize)>>, - ) -> Result { - if !ctx.get_settings().get_enable_join_runtime_filter()? { - return Ok(Default::default()); - } - - if !Self::supported_join_type_for_runtime_filter(&join.join_type) { - return Ok(Default::default()); - } - - let mut filters = Vec::new(); - - let build_side_data_distribution = s_expr.build_side_child().get_data_distribution()?; - - // Process each probe key that has runtime filter information - for (build_key, probe_key, scan_id, table_index) in build_keys - .iter() - .zip(probe_keys.into_iter()) - .filter_map(|(b, p)| p.map(|(p, scan_id, table_index)| (b, p, scan_id, table_index))) - { - // Skip if not a column reference - if probe_key.as_column_ref().is_none() { - continue; - } - - let data_type = build_key - .as_expr(&BUILTIN_FUNCTIONS) - .data_type() - .remove_nullable(); - let id = metadata.write().next_runtime_filter_id(); - - // Determine which filter types to enable based on data type and statistics - let enable_bloom_runtime_filter = { - let enable_in_cluster = build_side_data_distribution - .as_ref() - .is_none_or(|e| matches!(e, Exchange::Broadcast)); - let is_supported_type = Self::is_type_supported_for_bloom_filter(&data_type); - let enable_bloom_runtime_filter_based_on_stats = Self::adjust_bloom_runtime_filter( - ctx.clone(), - metadata, - Some(table_index), - s_expr, - ) - .await?; - enable_in_cluster && is_supported_type && enable_bloom_runtime_filter_based_on_stats - }; - - let enable_min_max_runtime_filter = build_side_data_distribution - .as_ref() - .is_none_or(|e| matches!(e, Exchange::Broadcast | Exchange::Hash(_))) - && Self::is_type_supported_for_min_max_filter(&data_type); - - let enable_inlist_runtime_filter = build_side_data_distribution - .as_ref() - .is_none_or(|e| matches!(e, Exchange::Broadcast | Exchange::Hash(_))); - - // Create and add the runtime filter - let runtime_filter = PhysicalRuntimeFilter { - id, - build_key: build_key.clone(), - probe_key, - scan_id, - enable_bloom_runtime_filter, - enable_inlist_runtime_filter, - enable_min_max_runtime_filter, - }; - filters.push(runtime_filter); - } - - Ok(PhysicalRuntimeFilters { filters }) - } -} diff --git a/src/query/service/src/physical_plans/physical_plan.rs b/src/query/service/src/physical_plans/physical_plan.rs index c9d693670ca5d..19a06f1b4fc29 100644 --- a/src/query/service/src/physical_plans/physical_plan.rs +++ b/src/query/service/src/physical_plans/physical_plan.rs @@ -359,6 +359,7 @@ impl PhysicalPlan { profs, metadata, scan_id_to_runtime_filters: HashMap::new(), + runtime_filter_reports: HashMap::new(), }; self.formatter()?.format(&mut context) diff --git a/src/query/service/src/physical_plans/physical_table_scan.rs b/src/query/service/src/physical_plans/physical_table_scan.rs index edac9872b471b..8162ba6685568 100644 --- a/src/query/service/src/physical_plans/physical_table_scan.rs +++ b/src/query/service/src/physical_plans/physical_table_scan.rs @@ -206,9 +206,6 @@ impl IPhysicalPlan for TableScan { fn build_pipeline2(&self, builder: &mut PipelineBuilder) -> Result<()> { let table = builder.ctx.build_table_from_source_plan(&self.source)?; builder.ctx.set_partitions(self.source.parts.clone())?; - builder - .ctx - .set_wait_runtime_filter(self.scan_id, builder.contain_sink_processor); if builder.ctx.get_settings().get_enable_prune_pipeline()? { if let Some(prune_pipeline) = table.build_prune_pipeline( diff --git a/src/query/service/src/physical_plans/runtime_filter/builder.rs b/src/query/service/src/physical_plans/runtime_filter/builder.rs new file mode 100644 index 0000000000000..6bf0d8191339a --- /dev/null +++ b/src/query/service/src/physical_plans/runtime_filter/builder.rs @@ -0,0 +1,305 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::collections::HashMap; +use std::sync::Arc; + +use databend_common_catalog::table_context::TableContext; +use databend_common_exception::Result; +use databend_common_expression::types::DataType; +use databend_common_expression::RemoteExpr; +use databend_common_functions::BUILTIN_FUNCTIONS; +use databend_common_sql::optimizer::ir::SExpr; +use databend_common_sql::plans::Exchange; +use databend_common_sql::plans::Join; +use databend_common_sql::plans::JoinEquiCondition; +use databend_common_sql::plans::JoinType; +use databend_common_sql::plans::RelOperator; +use databend_common_sql::plans::ScalarExpr; +use databend_common_sql::ColumnEntry; +use databend_common_sql::IndexType; +use databend_common_sql::MetadataRef; +use databend_common_sql::TypeCheck; + +use super::types::PhysicalRuntimeFilter; +use super::types::PhysicalRuntimeFilters; + +/// Type alias for probe keys with runtime filter information +/// Contains: (RemoteExpr, scan_id, table_index, column_idx) +type ProbeKeysWithRuntimeFilter = Vec, usize, usize, IndexType)>>; + +/// Check if a data type is supported for bloom filter +/// +/// Currently supports: numbers and strings +pub fn is_type_supported_for_bloom_filter(data_type: &DataType) -> bool { + data_type.is_number() || data_type.is_string() +} + +/// Check if a data type is supported for min-max filter +/// +/// Currently supports: numbers, dates, and strings +pub fn is_type_supported_for_min_max_filter(data_type: &DataType) -> bool { + data_type.is_number() || data_type.is_date() || data_type.is_string() +} + +/// Check if the join type is supported for runtime filter +/// +/// Runtime filters are only applicable to certain join types where +/// filtering the probe side can reduce processing +pub fn supported_join_type_for_runtime_filter(join_type: &JoinType) -> bool { + matches!( + join_type, + JoinType::Inner + | JoinType::Right + | JoinType::RightSemi + | JoinType::RightAnti + | JoinType::LeftMark + ) +} + +/// Build runtime filters for a join operation +/// +/// This is the legacy method that creates one runtime filter per probe key. +/// For equivalence class propagation, use the enhanced version in physical_hash_join.rs +/// +/// # Arguments +/// * `ctx` - Table context +/// * `metadata` - Metadata reference +/// * `join` - Join plan +/// * `s_expr` - SExpr for the join +/// * `build_keys` - Build side keys +/// * `probe_keys` - Probe side keys with scan_id, table_index, and column_idx +/// +/// # Returns +/// Collection of runtime filters to be applied +pub async fn build_runtime_filter( + ctx: Arc, + metadata: &MetadataRef, + join: &Join, + s_expr: &SExpr, + build_keys: &[RemoteExpr], + probe_keys: ProbeKeysWithRuntimeFilter, + build_table_indexes: Vec>, +) -> Result { + if !ctx.get_settings().get_enable_join_runtime_filter()? { + return Ok(Default::default()); + } + + if !supported_join_type_for_runtime_filter(&join.join_type) { + return Ok(Default::default()); + } + + let build_side = s_expr.build_side_child(); + let build_side_data_distribution = build_side.get_data_distribution()?; + if !build_side_data_distribution + .as_ref() + .is_none_or(|e| matches!(e, Exchange::Broadcast | Exchange::Hash(_))) + { + return Ok(Default::default()); + } + + let mut filters = Vec::new(); + + let probe_side = s_expr.probe_side_child(); + + // Process each probe key that has runtime filter information + for (build_key, probe_key, scan_id, _table_index, column_idx, build_table_index) in build_keys + .iter() + .zip(probe_keys.into_iter()) + .zip(build_table_indexes.into_iter()) + .filter_map(|((b, p), table_idx)| { + p.map(|(p, scan_id, table_index, column_idx)| { + (b, p, scan_id, table_index, column_idx, table_idx) + }) + }) + { + // Skip if not a column reference + if probe_key.as_column_ref().is_none() { + continue; + } + + let probe_targets = + find_probe_targets(metadata, probe_side, &probe_key, scan_id, column_idx)?; + + let build_table_rows = + get_build_table_rows(ctx.clone(), metadata, build_table_index).await?; + + let data_type = build_key + .as_expr(&BUILTIN_FUNCTIONS) + .data_type() + .remove_nullable(); + let id = metadata.write().next_runtime_filter_id(); + + let enable_bloom_runtime_filter = is_type_supported_for_bloom_filter(&data_type); + + let enable_min_max_runtime_filter = is_type_supported_for_min_max_filter(&data_type); + + // Create and add the runtime filter + let runtime_filter = PhysicalRuntimeFilter { + id, + build_key: build_key.clone(), + probe_targets, + build_table_rows, + enable_bloom_runtime_filter, + enable_inlist_runtime_filter: true, + enable_min_max_runtime_filter, + }; + filters.push(runtime_filter); + } + + Ok(PhysicalRuntimeFilters { filters }) +} + +async fn get_build_table_rows( + ctx: Arc, + metadata: &MetadataRef, + build_table_index: Option, +) -> Result> { + if let Some(table_index) = build_table_index { + let table = { + let metadata_read = metadata.read(); + metadata_read.table(table_index).table().clone() + }; + + let table_stats = table.table_statistics(ctx, false, None).await?; + return Ok(table_stats.and_then(|s| s.num_rows)); + } + + Ok(None) +} + +fn find_probe_targets( + metadata: &MetadataRef, + s_expr: &SExpr, + probe_key: &RemoteExpr, + probe_scan_id: usize, + probe_key_col_idx: IndexType, +) -> Result, usize)>> { + let mut uf = UnionFind::new(); + let mut column_to_remote: HashMap, usize)> = HashMap::new(); + column_to_remote.insert(probe_key_col_idx, (probe_key.clone(), probe_scan_id)); + + let equi_conditions = collect_equi_conditions(s_expr)?; + for cond in equi_conditions { + if let ( + Some((left_remote, left_scan_id, left_idx)), + Some((right_remote, right_scan_id, right_idx)), + ) = ( + scalar_to_remote_expr(metadata, &cond.left)?, + scalar_to_remote_expr(metadata, &cond.right)?, + ) { + uf.union(left_idx, right_idx); + column_to_remote.insert(left_idx, (left_remote, left_scan_id)); + column_to_remote.insert(right_idx, (right_remote, right_scan_id)); + } + } + + let equiv_class = uf.get_equivalence_class(probe_key_col_idx); + + let mut result = Vec::new(); + for idx in equiv_class { + if let Some((remote_expr, scan_id)) = column_to_remote.get(&idx) { + result.push((remote_expr.clone(), *scan_id)); + } + } + + Ok(result) +} + +fn collect_equi_conditions(s_expr: &SExpr) -> Result> { + let mut conditions = Vec::new(); + + if let RelOperator::Join(join) = s_expr.plan() { + if matches!(join.join_type, JoinType::Inner) { + conditions.extend(join.equi_conditions.clone()); + } + } + + for child in s_expr.children() { + conditions.extend(collect_equi_conditions(child)?); + } + + Ok(conditions) +} + +fn scalar_to_remote_expr( + metadata: &MetadataRef, + scalar: &ScalarExpr, +) -> Result, usize, IndexType)>> { + if scalar.used_columns().iter().all(|idx| { + matches!( + metadata.read().column(*idx), + ColumnEntry::BaseTableColumn(_) + ) + }) { + if let Some(column_idx) = scalar.used_columns().iter().next() { + let scan_id = metadata.read().base_column_scan_id(*column_idx); + + if let Some(scan_id) = scan_id { + let remote_expr = scalar + .as_raw_expr() + .type_check(&*metadata.read())? + .project_column_ref(|col| Ok(col.column_name.clone()))? + .as_remote_expr(); + + return Ok(Some((remote_expr, scan_id, *column_idx))); + } + } + } + + Ok(None) +} + +struct UnionFind { + parent: HashMap, +} + +impl UnionFind { + fn new() -> Self { + Self { + parent: HashMap::new(), + } + } + + fn find(&mut self, x: IndexType) -> IndexType { + if !self.parent.contains_key(&x) { + self.parent.insert(x, x); + return x; + } + + let parent = *self.parent.get(&x).unwrap(); + if parent != x { + let root = self.find(parent); + self.parent.insert(x, root); + } + *self.parent.get(&x).unwrap() + } + + fn union(&mut self, x: IndexType, y: IndexType) { + let root_x = self.find(x); + let root_y = self.find(y); + if root_x != root_y { + self.parent.insert(root_x, root_y); + } + } + + fn get_equivalence_class(&mut self, x: IndexType) -> Vec { + let root = self.find(x); + let all_keys: Vec = self.parent.keys().copied().collect(); + all_keys + .into_iter() + .filter(|&k| self.find(k) == root) + .collect() + } +} diff --git a/src/query/service/src/physical_plans/runtime_filter/mod.rs b/src/query/service/src/physical_plans/runtime_filter/mod.rs new file mode 100644 index 0000000000000..d28e5ff6a8968 --- /dev/null +++ b/src/query/service/src/physical_plans/runtime_filter/mod.rs @@ -0,0 +1,19 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +mod builder; +mod types; + +pub use builder::build_runtime_filter; +pub use types::*; diff --git a/src/query/service/src/physical_plans/runtime_filter/types.rs b/src/query/service/src/physical_plans/runtime_filter/types.rs new file mode 100644 index 0000000000000..11a7a9992f5c6 --- /dev/null +++ b/src/query/service/src/physical_plans/runtime_filter/types.rs @@ -0,0 +1,55 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use databend_common_expression::RemoteExpr; + +// pub type ProbeKeyWithEquivalents = Option, usize, usize)>>; + +/// Collection of runtime filters for a join operation +#[derive(Clone, Debug, serde::Serialize, serde::Deserialize, Default)] +pub struct PhysicalRuntimeFilters { + pub filters: Vec, +} + +/// A runtime filter that is built once and applied to multiple probe targets +/// +/// # Design +/// A single runtime filter is constructed once from the build side and then +/// pushed down to multiple table scans on the probe side. This is particularly +/// useful when join columns form equivalence classes (e.g., t1.c1 = t2.c1 = t3.c1), +/// allowing one filter to be applied to multiple tables. +#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] +pub struct PhysicalRuntimeFilter { + /// Unique identifier for this runtime filter + pub id: usize, + + /// The build key expression used to construct the filter + pub build_key: RemoteExpr, + + /// List of (probe_key, scan_id) pairs that this filter should be applied to + /// A single filter is built once and then pushed down to multiple scans + /// All probe targets in this list are in the same equivalence class + pub probe_targets: Vec<(RemoteExpr, usize)>, + + pub build_table_rows: Option, + + /// Enable bloom filter for this runtime filter + pub enable_bloom_runtime_filter: bool, + + /// Enable inlist filter for this runtime filter + pub enable_inlist_runtime_filter: bool, + + /// Enable min-max filter for this runtime filter + pub enable_min_max_runtime_filter: bool, +} diff --git a/src/query/service/src/pipelines/mod.rs b/src/query/service/src/pipelines/mod.rs index 584001f76b1f4..0c56b4247d0dd 100644 --- a/src/query/service/src/pipelines/mod.rs +++ b/src/query/service/src/pipelines/mod.rs @@ -24,5 +24,6 @@ pub use builders::RawValueSource; pub use builders::ValueSource; pub use pipeline_build_res::PipelineBuildResult; pub use pipeline_build_res::PipelineBuilderData; +pub use pipeline_builder::attach_runtime_filter_logger; pub use pipeline_builder::HashJoinStateRef; pub use pipeline_builder::PipelineBuilder; diff --git a/src/query/service/src/pipelines/pipeline_builder.rs b/src/query/service/src/pipelines/pipeline_builder.rs index 863338c1dd25a..bc923ea19aedd 100644 --- a/src/query/service/src/pipelines/pipeline_builder.rs +++ b/src/query/service/src/pipelines/pipeline_builder.rs @@ -59,8 +59,6 @@ pub struct PipelineBuilder { pub r_cte_scan_interpreters: Vec, pub(crate) is_exchange_stack: Vec, - - pub contain_sink_processor: bool, } impl PipelineBuilder { @@ -80,7 +78,6 @@ impl PipelineBuilder { join_state: None, hash_join_states: HashMap::new(), r_cte_scan_interpreters: vec![], - contain_sink_processor: false, is_exchange_stack: vec![], } } @@ -130,3 +127,10 @@ impl PipelineBuilder { plan.build_pipeline(self) } } + +pub fn attach_runtime_filter_logger(ctx: Arc, pipeline: &mut Pipeline) { + pipeline.set_on_finished(always_callback(move |_info: &ExecutionInfo| { + ctx.log_runtime_filter_stats(); + Ok(()) + })); +} diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/desc.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/desc.rs index ee191e992b35e..6de68d563cba2 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/desc.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/desc.rs @@ -69,8 +69,8 @@ pub struct HashJoinDesc { pub struct RuntimeFilterDesc { pub id: usize, pub build_key: Expr, - pub probe_key: Expr, - pub scan_id: usize, + pub probe_targets: Vec<(Expr, usize)>, + pub build_table_rows: Option, pub enable_bloom_runtime_filter: bool, pub enable_inlist_runtime_filter: bool, pub enable_min_max_runtime_filter: bool, @@ -93,8 +93,12 @@ impl From<&PhysicalRuntimeFilter> for RuntimeFilterDesc { Self { id: runtime_filter.id, build_key: runtime_filter.build_key.as_expr(&BUILTIN_FUNCTIONS), - probe_key: runtime_filter.probe_key.as_expr(&BUILTIN_FUNCTIONS), - scan_id: runtime_filter.scan_id, + probe_targets: runtime_filter + .probe_targets + .iter() + .map(|(probe_key, scan_id)| (probe_key.as_expr(&BUILTIN_FUNCTIONS), *scan_id)) + .collect(), + build_table_rows: runtime_filter.build_table_rows, enable_bloom_runtime_filter: runtime_filter.enable_bloom_runtime_filter, enable_inlist_runtime_filter: runtime_filter.enable_inlist_runtime_filter, enable_min_max_runtime_filter: runtime_filter.enable_min_max_runtime_filter, 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 6fe0ce8beca7e..8838fdb0462eb 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 @@ -840,21 +840,19 @@ impl HashJoinBuildState { } pub fn add_runtime_filter_ready(&self) { - if self.ctx.get_cluster().is_empty() { - return; - } - - let mut wait_runtime_filter_table_indexes = HashSet::new(); + let mut scan_ids = HashSet::new(); for rf in self.runtime_filter_desc() { - wait_runtime_filter_table_indexes.insert(rf.scan_id); + for (_probe_key, scan_id) in &rf.probe_targets { + scan_ids.insert(*scan_id); + } } let build_state = unsafe { &mut *self.hash_join_state.build_state.get() }; let runtime_filter_ready = &mut build_state.runtime_filter_ready; - for table_index in wait_runtime_filter_table_indexes.into_iter() { + for scan_id in scan_ids.into_iter() { let ready = Arc::new(RuntimeFilterReady::default()); runtime_filter_ready.push(ready.clone()); - self.ctx.set_runtime_filter_ready(table_index, ready); + self.ctx.set_runtime_filter_ready(scan_id, ready); } } diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/runtime_filter/builder.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/runtime_filter/builder.rs index a2e13f41df1d9..caf69ce7e9781 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/runtime_filter/builder.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/runtime_filter/builder.rs @@ -42,6 +42,8 @@ struct JoinRuntimeFilterPacketBuilder<'a> { inlist_threshold: usize, bloom_threshold: usize, min_max_threshold: usize, + is_spill_happened: bool, + selectivity_threshold: u64, } impl<'a> JoinRuntimeFilterPacketBuilder<'a> { @@ -52,6 +54,8 @@ impl<'a> JoinRuntimeFilterPacketBuilder<'a> { inlist_threshold: usize, bloom_threshold: usize, min_max_threshold: usize, + selectivity_threshold: u64, + is_spill_happened: bool, ) -> Result { let build_key_column = Self::eval_build_key_column(data_blocks, func_ctx, build_key)?; Ok(Self { @@ -60,9 +64,25 @@ impl<'a> JoinRuntimeFilterPacketBuilder<'a> { inlist_threshold, bloom_threshold, min_max_threshold, + selectivity_threshold, + is_spill_happened, }) } fn build(&self, desc: &RuntimeFilterDesc) -> Result { + if self.is_spill_happened + || !should_enable_runtime_filter( + desc, + self.build_key_column.len(), + self.selectivity_threshold, + ) + { + return Ok(RuntimeFilterPacket { + id: desc.id, + inlist: None, + min_max: None, + bloom: None, + }); + } let start = Instant::now(); let min_max_start = Instant::now(); @@ -115,7 +135,15 @@ impl<'a> JoinRuntimeFilterPacketBuilder<'a> { } fn enable_bloom(&self, desc: &RuntimeFilterDesc) -> bool { - desc.enable_bloom_runtime_filter && self.build_key_column.len() < self.bloom_threshold + if !desc.enable_bloom_runtime_filter { + return false; + } + + if self.build_key_column.len() >= self.bloom_threshold { + return false; + } + + true } fn build_min_max(&self) -> Result { @@ -175,6 +203,48 @@ impl<'a> JoinRuntimeFilterPacketBuilder<'a> { } } +pub(super) fn should_enable_runtime_filter( + desc: &RuntimeFilterDesc, + build_num_rows: usize, + selectivity_threshold: u64, +) -> bool { + if build_num_rows == 0 { + return false; + } + + let Some(build_table_rows) = desc.build_table_rows else { + log::info!( + "RUNTIME-FILTER: Disable runtime filter {} - no build table statistics available", + desc.id + ); + return false; + }; + + let selectivity_pct = (build_num_rows as f64 / build_table_rows as f64) * 100.0; + + if selectivity_pct < selectivity_threshold as f64 { + log::info!( + "RUNTIME-FILTER: Enable runtime filter {} - low selectivity: {:.2}% < {}% (build_rows={}, build_table_rows={})", + desc.id, + selectivity_pct, + selectivity_threshold, + build_num_rows, + build_table_rows + ); + true + } else { + log::info!( + "RUNTIME-FILTER: Disable runtime filter {} - high selectivity: {:.2}% >= {}% (build_rows={}, build_table_rows={})", + desc.id, + selectivity_pct, + selectivity_threshold, + build_num_rows, + build_table_rows + ); + false + } +} + pub fn build_runtime_filter_packet( build_chunks: &[DataBlock], build_num_rows: usize, @@ -183,9 +253,14 @@ pub fn build_runtime_filter_packet( inlist_threshold: usize, bloom_threshold: usize, min_max_threshold: usize, + selectivity_threshold: u64, + is_spill_happened: bool, ) -> Result { if build_num_rows == 0 { - return Ok(JoinRuntimeFilterPacket::default()); + return Ok(JoinRuntimeFilterPacket { + packets: None, + build_rows: build_num_rows, + }); } let mut runtime_filters = HashMap::new(); for rf in runtime_filter_desc { @@ -198,11 +273,14 @@ pub fn build_runtime_filter_packet( inlist_threshold, bloom_threshold, min_max_threshold, + selectivity_threshold, + is_spill_happened, )? .build(rf)?, ); } Ok(JoinRuntimeFilterPacket { packets: Some(runtime_filters), + build_rows: build_num_rows, }) } diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/runtime_filter/convert.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/runtime_filter/convert.rs index 9316608552dbc..374bcec25c7cf 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/runtime_filter/convert.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/runtime_filter/convert.rs @@ -14,8 +14,12 @@ use std::collections::HashMap; use std::collections::HashSet; +use std::sync::Arc; +use databend_common_catalog::runtime_filter_info::RuntimeFilterBloom; +use databend_common_catalog::runtime_filter_info::RuntimeFilterEntry; use databend_common_catalog::runtime_filter_info::RuntimeFilterInfo; +use databend_common_catalog::runtime_filter_info::RuntimeFilterStats; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::type_check; @@ -31,6 +35,7 @@ use databend_common_expression::Scalar; use databend_common_functions::BUILTIN_FUNCTIONS; use xorf::BinaryFuse16; +use super::builder::should_enable_runtime_filter; use super::packet::JoinRuntimeFilterPacket; use super::packet::SerializableDomain; use crate::pipelines::processors::transforms::hash_join::desc::RuntimeFilterDesc; @@ -40,33 +45,71 @@ use crate::pipelines::processors::transforms::hash_join::util::min_max_filter; /// /// The key in the resulting [`HashMap`] is the scan_id, which identifies the scan operator /// where the runtime filter will be applied. This is different from the runtime filter's own id, +/// +/// Each runtime filter (identified by packet.id) is built once and then applied to multiple scans. +/// The probe_targets in RuntimeFilterDesc specify all (probe_key, scan_id) pairs where this filter should be applied. pub fn build_runtime_filter_infos( packet: JoinRuntimeFilterPacket, runtime_filter_descs: HashMap, + selectivity_threshold: u64, ) -> Result> { + let total_build_rows = packet.build_rows; let Some(packets) = packet.packets else { return Ok(HashMap::new()); }; let mut filters: HashMap = HashMap::new(); + + // Iterate over all runtime filter packets for packet in packets.into_values() { let desc = runtime_filter_descs.get(&packet.id).unwrap(); - let entry = filters.entry(desc.scan_id).or_default(); - if let Some(inlist) = packet.inlist { - entry - .inlist - .push(build_inlist_filter(inlist, &desc.probe_key)?); - } - if let Some(min_max) = packet.min_max { - entry.min_max.push(build_min_max_filter( - min_max, - &desc.probe_key, - &desc.build_key, - )?); - } - if let Some(bloom) = packet.bloom { - entry - .bloom - .push(build_bloom_filter(bloom, &desc.probe_key)?); + let enabled = should_enable_runtime_filter(desc, total_build_rows, selectivity_threshold); + + // Apply this single runtime filter to all probe targets (scan_id, probe_key pairs) + // This implements the design goal: "one runtime filter built once, pushed down to multiple scans" + for (probe_key, scan_id) in &desc.probe_targets { + let entry = filters.entry(*scan_id).or_default(); + + let runtime_entry = RuntimeFilterEntry { + id: desc.id, + probe_expr: probe_key.clone(), + bloom: if enabled { + if let Some(ref bloom) = packet.bloom { + Some(build_bloom_filter(bloom.clone(), probe_key)?) + } else { + None + } + } else { + None + }, + inlist: if enabled { + if let Some(ref inlist) = packet.inlist { + Some(build_inlist_filter(inlist.clone(), probe_key)?) + } else { + None + } + } else { + None + }, + min_max: if enabled { + if let Some(ref min_max) = packet.min_max { + Some(build_min_max_filter( + min_max.clone(), + probe_key, + &desc.build_key, + )?) + } else { + None + } + } else { + None + }, + stats: Arc::new(RuntimeFilterStats::new()), + build_rows: total_build_rows, + build_table_rows: desc.build_table_rows, + enabled, + }; + + entry.filters.push(runtime_entry); } } Ok(filters) @@ -201,14 +244,14 @@ fn build_min_max_filter( Ok(min_max_filter) } -fn build_bloom_filter( - bloom: HashSet, - probe_key: &Expr, -) -> Result<(String, BinaryFuse16)> { +fn build_bloom_filter(bloom: HashSet, probe_key: &Expr) -> Result { let probe_key = probe_key.as_column_ref().unwrap(); let hashes_vec = bloom.into_iter().collect::>(); let filter = BinaryFuse16::try_from(&hashes_vec)?; - Ok((probe_key.id.to_string(), filter)) + Ok(RuntimeFilterBloom { + column_name: probe_key.id.to_string(), + filter, + }) } #[cfg(test)] diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/runtime_filter/interface.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/runtime_filter/interface.rs index 413032de20f2d..7aa8536f98faa 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/runtime_filter/interface.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/runtime_filter/interface.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::sync::atomic::Ordering; use std::time::Instant; use databend_common_exception::Result; @@ -30,6 +31,12 @@ pub async fn build_and_push_down_runtime_filter( ) -> Result<()> { let overall_start = Instant::now(); + let is_spill_happened = join.hash_join_state.need_next_round.load(Ordering::Acquire) + || join + .hash_join_state + .is_spill_happened + .load(Ordering::Acquire); + let inlist_threshold = join .ctx .get_settings() @@ -42,6 +49,10 @@ pub async fn build_and_push_down_runtime_filter( .ctx .get_settings() .get_min_max_runtime_filter_threshold()? as usize; + let selectivity_threshold = join + .ctx + .get_settings() + .get_join_runtime_filter_selectivity_threshold()?; let build_start = Instant::now(); let mut packet = build_runtime_filter_packet( @@ -52,6 +63,8 @@ pub async fn build_and_push_down_runtime_filter( inlist_threshold, bloom_threshold, min_max_threshold, + selectivity_threshold, + is_spill_happened, )?; let build_time = build_start.elapsed(); @@ -72,7 +85,8 @@ pub async fn build_and_push_down_runtime_filter( .iter() .map(|r| (r.id, r)) .collect(); - let runtime_filter_infos = build_runtime_filter_infos(packet, runtime_filter_descs)?; + let runtime_filter_infos = + build_runtime_filter_infos(packet, runtime_filter_descs, selectivity_threshold)?; let total_time = overall_start.elapsed(); let filter_count = runtime_filter_infos.len(); diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/runtime_filter/merge.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/runtime_filter/merge.rs index 7f125e9a19bd2..6847d7b847c80 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/runtime_filter/merge.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/runtime_filter/merge.rs @@ -29,6 +29,7 @@ pub fn merge_join_runtime_filter_packets( "RUNTIME-FILTER: merge_join_runtime_filter_packets input: {:?}", packets ); + let total_build_rows: usize = packets.iter().map(|packet| packet.build_rows).sum(); // Skip packets that `JoinRuntimeFilterPacket::packets` is `None` let packets = packets .into_iter() @@ -36,7 +37,10 @@ pub fn merge_join_runtime_filter_packets( .collect::>(); if packets.is_empty() { - return Ok(JoinRuntimeFilterPacket::default()); + return Ok(JoinRuntimeFilterPacket { + packets: None, + build_rows: total_build_rows, + }); } let mut result = HashMap::new(); @@ -55,6 +59,7 @@ pub fn merge_join_runtime_filter_packets( ); Ok(JoinRuntimeFilterPacket { packets: Some(result), + build_rows: total_build_rows, }) } diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/runtime_filter/packet.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/runtime_filter/packet.rs index 12b11230d02e7..14450b5508aba 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/runtime_filter/packet.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/runtime_filter/packet.rs @@ -56,9 +56,13 @@ impl Debug for RuntimeFilterPacket { /// # Fields /// /// * `packets` - A map of runtime filter packets, keyed by their unique identifier `RuntimeFilterPacket::id`. When `packets` is `None`, it means that `build_num_rows` is zero. +/// * `build_rows` - Total number of rows used when building the runtime filters. #[derive(serde::Serialize, serde::Deserialize, Clone, Debug, Default, PartialEq)] pub struct JoinRuntimeFilterPacket { + #[serde(default)] pub packets: Option>, + #[serde(default)] + pub build_rows: usize, } #[typetag::serde(name = "join_runtime_filter_packet")] diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/transform_hash_join_build.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/transform_hash_join_build.rs index dab322eacbd3d..8123682b4e062 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/transform_hash_join_build.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/transform_hash_join_build.rs @@ -300,6 +300,7 @@ impl Processor for TransformHashJoinBuild { .chunks .clone() }; + let build_num_rows = unsafe { (*self.build_state.hash_join_state.build_state.get()) .generation_state diff --git a/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/inner_join.rs b/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/inner_join.rs index 0988f95be0696..3b1dab84b39a7 100644 --- a/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/inner_join.rs +++ b/src/query/service/src/pipelines/processors/transforms/new_hash_join/memory/inner_join.rs @@ -102,6 +102,8 @@ impl Join for InnerHashJoin { desc.inlist_threshold, desc.bloom_threshold, desc.min_max_threshold, + desc.selectivity_threshold, + false, ) } diff --git a/src/query/service/src/pipelines/processors/transforms/new_hash_join/runtime_filter.rs b/src/query/service/src/pipelines/processors/transforms/new_hash_join/runtime_filter.rs index 17e0a64af4311..51a07ef78368e 100644 --- a/src/query/service/src/pipelines/processors/transforms/new_hash_join/runtime_filter.rs +++ b/src/query/service/src/pipelines/processors/transforms/new_hash_join/runtime_filter.rs @@ -32,6 +32,7 @@ pub struct RuntimeFiltersDesc { pub bloom_threshold: usize, pub inlist_threshold: usize, pub min_max_threshold: usize, + pub selectivity_threshold: u64, broadcast_id: Option, pub filters_desc: Vec, @@ -44,6 +45,7 @@ impl RuntimeFiltersDesc { let bloom_threshold = settings.get_bloom_runtime_filter_threshold()? as usize; let inlist_threshold = settings.get_inlist_runtime_filter_threshold()? as usize; let min_max_threshold = settings.get_min_max_runtime_filter_threshold()? as usize; + let selectivity_threshold = settings.get_join_runtime_filter_selectivity_threshold()?; let mut filters_desc = Vec::with_capacity(join.runtime_filter.filters.len()); let mut runtime_filters_ready = Vec::with_capacity(join.runtime_filter.filters.len()); @@ -52,9 +54,12 @@ impl RuntimeFiltersDesc { let filter_desc = RuntimeFilterDesc::from(filter_desc); if !ctx.get_cluster().is_empty() { - let ready = Arc::new(RuntimeFilterReady::default()); - runtime_filters_ready.push(ready.clone()); - ctx.set_runtime_filter_ready(filter_desc.scan_id, ready); + // Set runtime filter ready for all probe targets + for (_probe_key, scan_id) in &filter_desc.probe_targets { + let ready = Arc::new(RuntimeFilterReady::default()); + runtime_filters_ready.push(ready.clone()); + ctx.set_runtime_filter_ready(*scan_id, ready); + } } filters_desc.push(filter_desc); @@ -65,6 +70,7 @@ impl RuntimeFiltersDesc { bloom_threshold, inlist_threshold, min_max_threshold, + selectivity_threshold, runtime_filters_ready, ctx: ctx.clone(), broadcast_id: join.broadcast_id, @@ -77,7 +83,8 @@ impl RuntimeFiltersDesc { } let runtime_filter_descs = self.filters_desc.iter().map(|r| (r.id, r)).collect(); - let runtime_filter_infos = build_runtime_filter_infos(packet, runtime_filter_descs)?; + let runtime_filter_infos = + build_runtime_filter_infos(packet, runtime_filter_descs, self.selectivity_threshold)?; self.ctx.set_runtime_filter(runtime_filter_infos); diff --git a/src/query/service/src/schedulers/scheduler.rs b/src/query/service/src/schedulers/scheduler.rs index e5811b8232258..78e47a2714e12 100644 --- a/src/query/service/src/schedulers/scheduler.rs +++ b/src/query/service/src/schedulers/scheduler.rs @@ -24,6 +24,7 @@ use futures_util::TryStreamExt; use crate::interpreters::InterpreterFactory; use crate::physical_plans::build_broadcast_plans; use crate::physical_plans::PhysicalPlan; +use crate::pipelines::attach_runtime_filter_logger; use crate::pipelines::executor::ExecutorSettings; use crate::pipelines::executor::PipelinePullingExecutor; use crate::pipelines::PipelineBuildResult; @@ -63,7 +64,7 @@ pub async fn build_query_pipeline_without_render_result_set( ctx: &Arc, plan: &PhysicalPlan, ) -> Result { - let build_res = if !plan.is_distributed_plan() { + let mut build_res = if !plan.is_distributed_plan() { build_local_pipeline(ctx, plan).await } else { if plan.is_warehouse_distributed_plan() { @@ -72,6 +73,7 @@ pub async fn build_query_pipeline_without_render_result_set( build_distributed_pipeline(ctx, plan).await }?; + attach_runtime_filter_logger(ctx.clone(), &mut build_res.main_pipeline); Ok(build_res) } @@ -126,7 +128,6 @@ pub async fn build_distributed_pipeline( } } } - pub struct ServiceQueryExecutor { ctx: Arc, } @@ -142,25 +143,37 @@ impl ServiceQueryExecutor { &self, plan: &PhysicalPlan, ) -> Result> { - let build_res = build_query_pipeline_without_render_result_set(&self.ctx, plan).await?; - let settings = ExecutorSettings::try_create(self.ctx.clone())?; - let pulling_executor = PipelinePullingExecutor::from_pipelines(build_res, settings)?; - self.ctx.set_executor(pulling_executor.get_inner())?; - - PullingExecutorStream::create(pulling_executor)? - .try_collect::>() - .await + let result = async { + let build_res = build_query_pipeline_without_render_result_set(&self.ctx, plan).await?; + let settings = ExecutorSettings::try_create(self.ctx.clone())?; + let pulling_executor = PipelinePullingExecutor::from_pipelines(build_res, settings)?; + self.ctx.set_executor(pulling_executor.get_inner())?; + + PullingExecutorStream::create(pulling_executor)? + .try_collect::>() + .await + } + .await; + + self.ctx.clear_runtime_filter(); + result } } #[async_trait] impl QueryExecutor for ServiceQueryExecutor { async fn execute_query_with_sql_string(&self, query_sql: &str) -> Result> { - let mut planner = Planner::new(self.ctx.clone()); - let (plan, _) = planner.plan_sql(query_sql).await?; - let interpreter = InterpreterFactory::get(self.ctx.clone(), &plan).await?; - let stream = interpreter.execute(self.ctx.clone()).await?; - let blocks = stream.try_collect::>().await?; - Ok(blocks) + let result = async { + let mut planner = Planner::new(self.ctx.clone()); + let (plan, _) = planner.plan_sql(query_sql).await?; + let interpreter = InterpreterFactory::get(self.ctx.clone(), &plan).await?; + let stream = interpreter.execute(self.ctx.clone()).await?; + let blocks = stream.try_collect::>().await?; + Ok(blocks) + } + .await; + + self.ctx.clear_runtime_filter(); + result } } diff --git a/src/query/service/src/servers/flight/v1/exchange/exchange_manager.rs b/src/query/service/src/servers/flight/v1/exchange/exchange_manager.rs index 1d6b335446691..b1b801e03dbf3 100644 --- a/src/query/service/src/servers/flight/v1/exchange/exchange_manager.rs +++ b/src/query/service/src/servers/flight/v1/exchange/exchange_manager.rs @@ -57,6 +57,7 @@ use super::statistics_sender::StatisticsSender; use crate::clusters::ClusterHelper; use crate::clusters::FlightParams; use crate::physical_plans::PhysicalPlan; +use crate::pipelines::attach_runtime_filter_logger; use crate::pipelines::executor::ExecutorSettings; use crate::pipelines::executor::PipelineCompleteExecutor; use crate::pipelines::PipelineBuildResult; @@ -1083,10 +1084,11 @@ impl FragmentCoordinator { let pipeline_builder = PipelineBuilder::create( pipeline_ctx.get_function_context()?, pipeline_ctx.get_settings(), - pipeline_ctx, + pipeline_ctx.clone(), ); - let res = pipeline_builder.finalize(&self.physical_plan)?; + let mut res = pipeline_builder.finalize(&self.physical_plan)?; + attach_runtime_filter_logger(pipeline_ctx, &mut res.main_pipeline); self.pipeline_build_res = Some(res); } diff --git a/src/query/service/src/sessions/query_ctx.rs b/src/query/service/src/sessions/query_ctx.rs index 7a9966a31aebf..560a63f27326a 100644 --- a/src/query/service/src/sessions/query_ctx.rs +++ b/src/query/service/src/sessions/query_ctx.rs @@ -33,6 +33,7 @@ use async_channel::Sender; use chrono_tz::Tz; use dashmap::mapref::multiple::RefMulti; use dashmap::DashMap; +use databend_common_ast::ast::FormatTreeNode; use databend_common_base::base::Progress; use databend_common_base::base::ProgressValues; use databend_common_base::base::SpillProgress; @@ -57,8 +58,11 @@ use databend_common_catalog::plan::PartStatistics; 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::RuntimeFilterEntry; use databend_common_catalog::runtime_filter_info::RuntimeFilterInfo; use databend_common_catalog::runtime_filter_info::RuntimeFilterReady; +use databend_common_catalog::runtime_filter_info::RuntimeFilterReport; +use databend_common_catalog::runtime_filter_info::RuntimeFilterStatsSnapshot; use databend_common_catalog::session_type::SessionType; use databend_common_catalog::statistics::data_cache_statistics::DataCacheMetrics; use databend_common_catalog::table_args::TableArgs; @@ -517,6 +521,168 @@ impl QueryContext { .fetch_or(true, Ordering::SeqCst) } + pub fn should_log_runtime_filters(&self) -> bool { + !self + .shared + .runtime_filter_logged + .swap(true, Ordering::SeqCst) + } + + pub fn log_runtime_filter_stats(&self) { + struct FilterLogEntry { + filter_id: usize, + probe_expr: String, + bloom_column: Option, + has_bloom: bool, + has_inlist: bool, + has_min_max: bool, + stats: RuntimeFilterStatsSnapshot, + build_rows: usize, + build_table_rows: Option, + enabled: bool, + } + + let runtime_filters = self.shared.runtime_filters.read(); + let mut snapshots: Vec<(IndexType, Vec)> = Vec::new(); + for (scan_id, info) in runtime_filters.iter() { + if info.filters.is_empty() { + continue; + } + + let mut filters = Vec::with_capacity(info.filters.len()); + for entry in &info.filters { + filters.push(FilterLogEntry { + filter_id: entry.id, + probe_expr: entry.probe_expr.sql_display(), + bloom_column: entry.bloom.as_ref().map(|bloom| bloom.column_name.clone()), + has_bloom: entry.bloom.is_some(), + has_inlist: entry.inlist.is_some(), + has_min_max: entry.min_max.is_some(), + stats: entry.stats.snapshot(), + build_rows: entry.build_rows, + build_table_rows: entry.build_table_rows, + enabled: entry.enabled, + }); + } + + if !filters.is_empty() { + snapshots.push((*scan_id, filters)); + } + } + drop(runtime_filters); + + if snapshots.is_empty() { + return; + } + + if !self.should_log_runtime_filters() { + return; + } + + let query_id = self.get_id(); + + for (scan_id, filters) in snapshots { + let mut filter_nodes = Vec::new(); + for filter in filters { + let FilterLogEntry { + filter_id, + probe_expr, + bloom_column, + has_bloom, + has_inlist, + has_min_max, + stats, + build_rows, + build_table_rows, + enabled, + } = filter; + + let mut types = Vec::new(); + if has_bloom { + types.push("bloom"); + } + if has_inlist { + types.push("inlist"); + } + if has_min_max { + types.push("min_max"); + } + let type_text = if types.is_empty() { + "none".to_string() + } else { + types.join(",") + }; + + let mut detail_children = vec![ + FormatTreeNode::new(format!("probe expr: {}", probe_expr)), + FormatTreeNode::new(format!("types: [{}]", type_text)), + FormatTreeNode::new(format!("enabled: {}", enabled)), + FormatTreeNode::new(format!("build rows: {}", build_rows)), + FormatTreeNode::new(format!( + "build table rows: {}", + build_table_rows + .map(|v| v.to_string()) + .unwrap_or_else(|| "unknown".to_string()) + )), + ]; + + if let Some(column) = bloom_column { + detail_children.push(FormatTreeNode::new(format!("bloom column: {}", column))); + } + + if has_bloom { + detail_children.push(FormatTreeNode::new(format!( + "bloom rows filtered: {}", + stats.bloom_rows_filtered + ))); + detail_children.push(FormatTreeNode::new(format!( + "bloom time: {:?}", + Duration::from_nanos(stats.bloom_time_ns) + ))); + } + + if has_inlist || has_min_max { + detail_children.push(FormatTreeNode::new(format!( + "inlist/min-max time: {:?}", + Duration::from_nanos(stats.inlist_min_max_time_ns) + ))); + detail_children.push(FormatTreeNode::new(format!( + "min-max rows filtered: {}", + stats.min_max_rows_filtered + ))); + detail_children.push(FormatTreeNode::new(format!( + "min-max partitions pruned: {}", + stats.min_max_partitions_pruned + ))); + } + + filter_nodes.push(FormatTreeNode::with_children( + format!("filter id:{}", filter_id), + detail_children, + )); + } + + if filter_nodes.is_empty() { + continue; + } + + let root = FormatTreeNode::with_children(format!("Scan {}", scan_id), vec![ + FormatTreeNode::with_children("runtime filters".to_string(), filter_nodes), + ]); + + match root.format_pretty() { + Ok(text) => info!( + "runtime filter stats (query_id={}, scan_id={}):\n{}", + query_id, scan_id, text + ), + Err(err) => info!( + "runtime filter stats (query_id={}, scan_id={}): failed to format: {}", + query_id, scan_id, err + ), + } + } + } + pub fn unload_spill_meta(&self) { const SPILL_META_SUFFIX: &str = ".list"; let r = self.shared.spilled_files.read(); @@ -1506,15 +1672,39 @@ impl TableContext for QueryContext { fn clear_runtime_filter(&self) { let mut runtime_filters = self.shared.runtime_filters.write(); runtime_filters.clear(); + self.shared.runtime_filter_ready.write().clear(); + self.shared + .runtime_filter_logged + .store(false, Ordering::SeqCst); + } + + fn assert_no_runtime_filter_state(&self) -> Result<()> { + let query_id = self.get_id(); + if !self.shared.runtime_filters.read().is_empty() { + return Err(ErrorCode::Internal(format!( + "Runtime filters should be empty for query {query_id}" + ))); + } + if !self.shared.runtime_filter_ready.read().is_empty() { + return Err(ErrorCode::Internal(format!( + "Runtime filter ready set should be empty for query {query_id}" + ))); + } + if self.shared.runtime_filter_logged.load(Ordering::Relaxed) { + return Err(ErrorCode::Internal(format!( + "Runtime filter logged flag should be reset for query {query_id}" + ))); + } + Ok(()) } fn set_runtime_filter(&self, filters: HashMap) { let mut runtime_filters = self.shared.runtime_filters.write(); for (scan_id, filter) in filters { let entry = runtime_filters.entry(scan_id).or_default(); - entry.inlist.extend(filter.inlist); - entry.min_max.extend(filter.min_max); - entry.bloom.extend(filter.bloom); + for new_filter in filter.filters { + entry.filters.push(new_filter); + } } } @@ -1530,27 +1720,14 @@ impl TableContext for QueryContext { } } - fn get_runtime_filter_ready(&self, table_index: usize) -> Vec> { + fn get_runtime_filter_ready(&self, scan_id: usize) -> Vec> { let runtime_filter_ready = self.shared.runtime_filter_ready.read(); - match runtime_filter_ready.get(&table_index) { + match runtime_filter_ready.get(&scan_id) { Some(v) => v.to_vec(), None => vec![], } } - fn set_wait_runtime_filter(&self, table_index: usize, need_to_wait: bool) { - let mut wait_runtime_filter = self.shared.wait_runtime_filter.write(); - wait_runtime_filter.insert(table_index, need_to_wait); - } - - fn get_wait_runtime_filter(&self, table_index: usize) -> bool { - let wait_runtime_filter = self.shared.wait_runtime_filter.read(); - match wait_runtime_filter.get(&table_index) { - Some(v) => *v, - None => false, - } - } - fn get_merge_into_join(&self) -> MergeIntoJoin { let merge_into_join = self.shared.merge_into_join.read(); MergeIntoJoin { @@ -1560,33 +1737,62 @@ impl TableContext for QueryContext { } } - fn get_bloom_runtime_filter_with_id(&self, id: IndexType) -> Vec<(String, BinaryFuse16)> { + fn get_runtime_filters(&self, id: IndexType) -> Vec { let runtime_filters = self.shared.runtime_filters.read(); - match runtime_filters.get(&id) { - Some(v) => v.bloom.clone(), - None => vec![], - } + runtime_filters + .get(&id) + .map(|v| v.filters.clone()) + .unwrap_or_default() + } + + fn get_bloom_runtime_filter_with_id(&self, id: IndexType) -> Vec<(String, BinaryFuse16)> { + self.get_runtime_filters(id) + .into_iter() + .filter_map(|entry| entry.bloom.map(|bloom| (bloom.column_name, bloom.filter))) + .collect() } fn get_inlist_runtime_filter_with_id(&self, id: IndexType) -> Vec> { - let runtime_filters = self.shared.runtime_filters.read(); - match runtime_filters.get(&id) { - Some(v) => v.inlist.clone(), - None => vec![], - } + self.get_runtime_filters(id) + .into_iter() + .filter_map(|entry| entry.inlist) + .collect() } fn get_min_max_runtime_filter_with_id(&self, id: IndexType) -> Vec> { + self.get_runtime_filters(id) + .into_iter() + .filter_map(|entry| entry.min_max) + .collect() + } + + fn runtime_filter_reports(&self) -> HashMap> { let runtime_filters = self.shared.runtime_filters.read(); - match runtime_filters.get(&id) { - Some(v) => v.min_max.clone(), - None => vec![], - } + runtime_filters + .iter() + .map(|(scan_id, info)| { + let reports = info + .filters + .iter() + .map(|entry| RuntimeFilterReport { + filter_id: entry.id, + has_bloom: entry.bloom.is_some(), + has_inlist: entry.inlist.is_some(), + has_min_max: entry.min_max.is_some(), + stats: entry.stats.snapshot(), + }) + .collect(); + (*scan_id, reports) + }) + .collect() } fn has_bloom_runtime_filters(&self, id: usize) -> bool { if let Some(runtime_filter) = self.shared.runtime_filters.read().get(&id) { - return !runtime_filter.bloom.is_empty(); + return runtime_filter + .filters + .iter() + .any(|entry| entry.bloom.is_some()); } false } diff --git a/src/query/service/src/sessions/query_ctx_shared.rs b/src/query/service/src/sessions/query_ctx_shared.rs index 8d8c18627b5b5..eb8d331e0ce52 100644 --- a/src/query/service/src/sessions/query_ctx_shared.rs +++ b/src/query/service/src/sessions/query_ctx_shared.rs @@ -159,8 +159,6 @@ pub struct QueryContextShared { pub(super) runtime_filter_ready: Arc>>>>, - pub(super) wait_runtime_filter: Arc>>, - pub(super) merge_into_join: Arc>, // Records query level data cache metrics @@ -173,6 +171,7 @@ pub struct QueryContextShared { pub(super) cluster_spill_progress: Arc>>, pub(super) spilled_files: Arc>>, pub(super) unload_callbacked: AtomicBool, + pub(super) runtime_filter_logged: AtomicBool, pub(super) mem_stat: Arc>>>, pub(super) node_memory_usage: Arc>>>, @@ -253,7 +252,6 @@ impl QueryContextShared { query_profiles: Arc::new(RwLock::new(HashMap::new())), runtime_filters: Default::default(), runtime_filter_ready: Default::default(), - wait_runtime_filter: Default::default(), merge_into_join: Default::default(), multi_table_insert_status: Default::default(), query_queued_duration: Arc::new(RwLock::new(Duration::from_secs(0))), @@ -261,6 +259,7 @@ impl QueryContextShared { cluster_spill_progress: Default::default(), spilled_files: Default::default(), unload_callbacked: AtomicBool::new(false), + runtime_filter_logged: AtomicBool::new(false), warehouse_cache: Arc::new(RwLock::new(None)), mem_stat: Arc::new(RwLock::new(None)), node_memory_usage: Arc::new(RwLock::new(HashMap::new())), 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 4ffe8ccf4d8ef..32030373b1d53 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 @@ -33,7 +33,9 @@ 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::RuntimeFilterEntry; use databend_common_catalog::runtime_filter_info::RuntimeFilterReady; +use databend_common_catalog::runtime_filter_info::RuntimeFilterReport; use databend_common_catalog::session_type::SessionType; use databend_common_catalog::statistics::data_cache_statistics::DataCacheMetrics; use databend_common_catalog::table::Table; @@ -950,16 +952,16 @@ impl TableContext for CtxDelegation { todo!() } - fn set_wait_runtime_filter(&self, _table_index: usize, _need_to_wait: bool) { + fn clear_runtime_filter(&self) { todo!() } - fn get_wait_runtime_filter(&self, _table_index: usize) -> bool { - todo!() + fn get_runtime_filters(&self, _id: usize) -> Vec { + Vec::::new() } - fn clear_runtime_filter(&self) { - todo!() + fn runtime_filter_reports(&self) -> HashMap> { + HashMap::new() } fn get_bloom_runtime_filter_with_id(&self, _id: usize) -> Vec<(String, BinaryFuse16)> { @@ -1059,6 +1061,10 @@ impl TableContext for CtxDelegation { fn get_abort_notify(&self) -> Arc { self.ctx.get_abort_notify() } + + fn assert_no_runtime_filter_state(&self) -> Result<()> { + Ok(()) + } } #[tokio::test(flavor = "multi_thread")] diff --git a/src/query/service/tests/it/sql/planner/optimizer/data/results/basic/01_cross_join_aggregation_physical.txt b/src/query/service/tests/it/sql/planner/optimizer/data/results/basic/01_cross_join_aggregation_physical.txt index ea759d005b16e..2edcbdd0463eb 100644 --- a/src/query/service/tests/it/sql/planner/optimizer/data/results/basic/01_cross_join_aggregation_physical.txt +++ b/src/query/service/tests/it/sql/planner/optimizer/data/results/basic/01_cross_join_aggregation_physical.txt @@ -23,6 +23,7 @@ AggregateFinal │ ├── exchange type: Broadcast │ └── TableScan │ ├── table: default.default.integers + │ ├── scan id: 1 │ ├── output columns: [i (#1)] │ ├── read rows: 5000 │ ├── read size: 20.15 KiB @@ -33,6 +34,7 @@ AggregateFinal │ └── estimated rows: 5000.00 └── TableScan(Probe) ├── table: default.default.integers + ├── scan id: 0 ├── output columns: [i (#0)] ├── read rows: 5000 ├── read size: 20.15 KiB 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 3e364c13017c9..b9ef74130d3ec 100644 --- a/src/query/service/tests/it/storages/fuse/operations/commit.rs +++ b/src/query/service/tests/it/storages/fuse/operations/commit.rs @@ -32,7 +32,9 @@ 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::RuntimeFilterEntry; use databend_common_catalog::runtime_filter_info::RuntimeFilterReady; +use databend_common_catalog::runtime_filter_info::RuntimeFilterReport; use databend_common_catalog::statistics::data_cache_statistics::DataCacheMetrics; use databend_common_catalog::table::Table; use databend_common_catalog::table_context::ContextError; @@ -815,16 +817,16 @@ impl TableContext for CtxDelegation { todo!() } - fn set_wait_runtime_filter(&self, _table_index: usize, _need_to_wait: bool) { + fn clear_runtime_filter(&self) { todo!() } - fn get_wait_runtime_filter(&self, _table_index: usize) -> bool { - todo!() + fn get_runtime_filters(&self, _id: usize) -> Vec { + Vec::::new() } - fn clear_runtime_filter(&self) { - todo!() + fn runtime_filter_reports(&self) -> HashMap> { + HashMap::new() } fn get_bloom_runtime_filter_with_id(&self, _id: usize) -> Vec<(String, BinaryFuse16)> { diff --git a/src/query/settings/src/settings_default.rs b/src/query/settings/src/settings_default.rs index ce0042a36505d..4f4e7477da877 100644 --- a/src/query/settings/src/settings_default.rs +++ b/src/query/settings/src/settings_default.rs @@ -509,6 +509,13 @@ impl DefaultSettings { scope: SettingScope::Both, range: Some(SettingRange::Numeric(0..=1)), }), + ("join_runtime_filter_selectivity_threshold", DefaultSettingValue { + value: UserSettingValue::UInt64(10), + desc: "Selectivity threshold (percentage) for join runtime filters. Filters are enabled when (build_rows / build_table_rows * 100) < threshold. Default 10 means 10%.", + mode: SettingMode::Both, + scope: SettingScope::Both, + range: Some(SettingRange::Numeric(1..=u64::MAX)), + }), ("max_execute_time_in_seconds", DefaultSettingValue { value: UserSettingValue::UInt64(0), desc: "Sets the maximum query execution time in seconds. Setting it to 0 means no limit.", diff --git a/src/query/settings/src/settings_getter_setter.rs b/src/query/settings/src/settings_getter_setter.rs index 0203cf7b19b5c..6057058270e25 100644 --- a/src/query/settings/src/settings_getter_setter.rs +++ b/src/query/settings/src/settings_getter_setter.rs @@ -388,6 +388,10 @@ impl Settings { Ok(self.try_get_u64("enable_join_runtime_filter")? != 0) } + pub fn get_join_runtime_filter_selectivity_threshold(&self) -> Result { + self.try_get_u64("join_runtime_filter_selectivity_threshold") + } + pub fn get_prefer_broadcast_join(&self) -> Result { Ok(self.try_get_u64("prefer_broadcast_join")? != 0) } 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 b7c481582e121..ba6d8c9d7ec68 100644 --- a/src/query/storages/fuse/src/operations/read/fuse_source.rs +++ b/src/query/storages/fuse/src/operations/read/fuse_source.rs @@ -42,6 +42,7 @@ use crate::operations::read::native_data_transform_reader::ReadNativeDataTransfo use crate::operations::read::parquet_data_transform_reader::ReadParquetDataTransform; use crate::operations::read::DeserializeDataTransform; use crate::operations::read::NativeDeserializeDataTransform; +use crate::operations::read::TransformRuntimeFilterWait; #[allow(clippy::too_many_arguments)] pub fn build_fuse_native_source_pipeline( @@ -82,6 +83,14 @@ pub fn build_fuse_native_source_pipeline( pipeline.add_pipe(pipe); } } + pipeline.add_transform(|input, output| { + Ok(TransformRuntimeFilterWait::create( + ctx.clone(), + plan.scan_id, + input, + output, + )) + })?; pipeline.add_transform(|input, output| { ReadNativeDataTransform::::create( plan.scan_id, @@ -118,6 +127,15 @@ pub fn build_fuse_native_source_pipeline( } } + pipeline.add_transform(|input, output| { + Ok(TransformRuntimeFilterWait::create( + ctx.clone(), + plan.scan_id, + input, + output, + )) + })?; + pipeline.add_transform(|input, output| { ReadNativeDataTransform::::create( plan.scan_id, @@ -190,6 +208,15 @@ pub fn build_fuse_parquet_source_pipeline( let unfinished_processors_count = Arc::new(AtomicU64::new(pipeline.output_len() as u64)); + pipeline.add_transform(|input, output| { + Ok(TransformRuntimeFilterWait::create( + ctx.clone(), + plan.scan_id, + input, + output, + )) + })?; + pipeline.add_transform(|input, output| { ReadParquetDataTransform::::create( plan.scan_id, @@ -233,6 +260,15 @@ pub fn build_fuse_parquet_source_pipeline( let unfinished_processors_count = Arc::new(AtomicU64::new(pipeline.output_len() as u64)); + pipeline.add_transform(|input, output| { + Ok(TransformRuntimeFilterWait::create( + ctx.clone(), + plan.scan_id, + input, + output, + )) + })?; + pipeline.add_transform(|input, output| { ReadParquetDataTransform::::create( plan.table_index, diff --git a/src/query/storages/fuse/src/operations/read/mod.rs b/src/query/storages/fuse/src/operations/read/mod.rs index aa2555b654273..f0df63e6d77ee 100644 --- a/src/query/storages/fuse/src/operations/read/mod.rs +++ b/src/query/storages/fuse/src/operations/read/mod.rs @@ -21,6 +21,7 @@ mod parquet_data_source; mod parquet_data_source_deserializer; mod parquet_data_transform_reader; mod parquet_rows_fetcher; +mod runtime_filter_wait; mod block_partition_meta; mod block_partition_receiver_source; @@ -32,4 +33,5 @@ pub use fuse_rows_fetcher::row_fetch_processor; pub use fuse_source::build_fuse_parquet_source_pipeline; pub use native_data_source_deserializer::NativeDeserializeDataTransform; pub use parquet_data_source_deserializer::DeserializeDataTransform; +pub use runtime_filter_wait::TransformRuntimeFilterWait; pub use util::need_reserve_block_info; diff --git a/src/query/storages/fuse/src/operations/read/native_data_source_deserializer.rs b/src/query/storages/fuse/src/operations/read/native_data_source_deserializer.rs index 05567bd76b39a..1d5084b124f66 100644 --- a/src/query/storages/fuse/src/operations/read/native_data_source_deserializer.rs +++ b/src/query/storages/fuse/src/operations/read/native_data_source_deserializer.rs @@ -27,6 +27,8 @@ use databend_common_catalog::plan::DataSourcePlan; use databend_common_catalog::plan::PartInfoPtr; use databend_common_catalog::plan::PushDownInfo; use databend_common_catalog::plan::TopK; +use databend_common_catalog::runtime_filter_info::RuntimeFilterEntry; +use databend_common_catalog::runtime_filter_info::RuntimeFilterStats; use databend_common_catalog::table_context::TableContext; use databend_common_exception::Result; use databend_common_expression::filter_helper::FilterHelpers; @@ -56,7 +58,6 @@ use databend_common_pipeline::core::InputPort; use databend_common_pipeline::core::OutputPort; use databend_common_pipeline::core::Processor; use databend_common_pipeline::core::ProcessorPtr; -use databend_common_sql::IndexType; use roaring::RoaringTreemap; use xorf::BinaryFuse16; @@ -198,7 +199,7 @@ pub struct NativeDeserializeDataTransform { scan_progress: Arc, // Structures for table scan information: - table_index: IndexType, + scan_id: usize, block_reader: Arc, src_schema: DataSchema, output_schema: DataSchema, @@ -214,7 +215,7 @@ pub struct NativeDeserializeDataTransform { // Structures for the bloom runtime filter: ctx: Arc, - bloom_runtime_filter: Option>, + bloom_runtime_filter: Option>, // Structures for aggregating index: index_reader: Arc>, @@ -232,6 +233,14 @@ pub struct NativeDeserializeDataTransform { need_reserve_block_info: bool, } +#[derive(Clone)] +struct BloomRuntimeFilterRef { + column_index: FieldIndex, + filter_id: usize, + filter: BinaryFuse16, + stats: Arc, +} + impl NativeDeserializeDataTransform { #[allow(clippy::too_many_arguments)] pub fn create( @@ -300,11 +309,10 @@ impl NativeDeserializeDataTransform { let mut output_schema = plan.schema().as_ref().clone(); output_schema.remove_internal_fields(); let output_schema: DataSchema = (&output_schema).into(); - Ok(ProcessorPtr::create(Box::new( NativeDeserializeDataTransform { ctx, - table_index: plan.table_index, + scan_id: plan.scan_id, func_ctx, scan_progress, block_reader, @@ -635,28 +643,37 @@ impl NativeDeserializeDataTransform { fn read_and_check_bloom_runtime_filter(&mut self) -> Result { if let Some(bloom_runtime_filter) = self.bloom_runtime_filter.as_ref() { let mut bitmaps = Vec::with_capacity(bloom_runtime_filter.len()); - for (idx, filter) in bloom_runtime_filter.iter() { - let column = if let Some((_, column)) = - self.read_state.columns.iter().find(|(i, _)| i == idx) + for runtime_filter in bloom_runtime_filter.iter() { + let start = std::time::Instant::now(); + let column = if let Some((_, column)) = self + .read_state + .columns + .iter() + .find(|(i, _)| i == &runtime_filter.column_index) { - (*idx, column.clone()) - } else if !self.read_state.read_page(*idx)? { + (runtime_filter.column_index, column.clone()) + } else if !self.read_state.read_page(runtime_filter.column_index)? { debug_assert!(self.read_state.is_finished()); return Ok(false); } else { // The runtime filter column must be the last column to read. let (i, column) = self.read_state.columns.last().unwrap(); - debug_assert_eq!(i, idx); - (*idx, column.clone()) + debug_assert_eq!(i, &runtime_filter.column_index); + (runtime_filter.column_index, column.clone()) }; let probe_block = self.block_reader.build_block(&[column], None)?; let mut bitmap = MutableBitmap::from_len_zeroed(probe_block.num_rows()); let probe_column = probe_block.get_last_column().clone(); // Apply the filter to the probe column. - ExprBloomFilter::new(filter.clone()).apply(probe_column, &mut bitmap)?; + ExprBloomFilter::new(runtime_filter.filter.clone()) + .apply(probe_column, &mut bitmap)?; let unset_bits = bitmap.null_count(); + let elapsed = start.elapsed(); + runtime_filter + .stats + .record_bloom(elapsed.as_nanos() as u64, unset_bits as u64); if unset_bits == bitmap.len() { // skip current page. return Ok(false); @@ -733,19 +750,35 @@ impl NativeDeserializeDataTransform { /// Try to get bloom runtime filter from context. fn try_init_bloom_runtime_filter(&mut self) { if self.bloom_runtime_filter.is_none() { - let bloom_filters = self.ctx.get_bloom_runtime_filter_with_id(self.table_index); - let bloom_filters = bloom_filters + let bloom_filters = self + .ctx + .get_runtime_filters(self.scan_id) .into_iter() - .filter_map(|filter| { - let name = filter.0.as_str(); - // Some probe keys are not in the schema, they are derived from expressions. - self.src_schema - .index_of(name) - .ok() - .map(|idx| (idx, filter.1.clone())) + .filter_map(|entry| { + let filter_id = entry.id; + let RuntimeFilterEntry { bloom, stats, .. } = entry; + let bloom = bloom?; + let column_index = self.src_schema.index_of(bloom.column_name.as_str()).ok()?; + Some(BloomRuntimeFilterRef { + column_index, + filter_id, + filter: bloom.filter.clone(), + stats, + }) }) .collect::>(); if !bloom_filters.is_empty() { + let mut filter_ids = bloom_filters + .iter() + .map(|f| f.filter_id) + .collect::>(); + filter_ids.sort_unstable(); + log::info!( + "RUNTIME-FILTER: scan_id={} bloom_filters={} filter_ids={:?}", + self.scan_id, + bloom_filters.len(), + filter_ids + ); self.bloom_runtime_filter = Some(bloom_filters); if self.filter_executor.is_none() { self.filter_executor = Some(new_dummy_filter_executor(self.func_ctx.clone())); @@ -830,6 +863,7 @@ impl NativeDeserializeDataTransform { } } +#[async_trait::async_trait] impl Processor for NativeDeserializeDataTransform { fn name(&self) -> String { String::from("NativeDeserializeDataTransform") @@ -885,6 +919,11 @@ impl Processor for NativeDeserializeDataTransform { Ok(Event::NeedData) } + #[async_backtrace::framed] + async fn async_process(&mut self) -> Result<()> { + Ok(()) + } + fn process(&mut self) -> Result<()> { // Try to get the bloom runtime filter from the context if existed. self.try_init_bloom_runtime_filter(); diff --git a/src/query/storages/fuse/src/operations/read/native_data_transform_reader.rs b/src/query/storages/fuse/src/operations/read/native_data_transform_reader.rs index a1827767ba056..97e00d06e9a7e 100644 --- a/src/query/storages/fuse/src/operations/read/native_data_transform_reader.rs +++ b/src/query/storages/fuse/src/operations/read/native_data_transform_reader.rs @@ -38,6 +38,7 @@ use crate::io::TableMetaLocationGenerator; use crate::operations::read::block_partition_meta::BlockPartitionMeta; use crate::operations::read::data_source_with_meta::DataSourceWithMeta; use crate::pruning::ExprRuntimePruner; +use crate::pruning::RuntimeFilterExpr; use crate::FuseBlockPartInfo; pub struct ReadNativeDataTransform { @@ -112,13 +113,30 @@ impl Transform for ReadNativeDataTransform { let mut partitions = block_part_meta.part_ptr.clone(); debug_assert!(partitions.len() == 1); let part = partitions.pop().unwrap(); - let mut filters = self.context.get_inlist_runtime_filter_with_id(self.scan_id); - filters.extend( + let runtime_filter = ExprRuntimePruner::new( self.context - .get_min_max_runtime_filter_with_id(self.scan_id), + .get_runtime_filters(self.scan_id) + .into_iter() + .flat_map(|entry| { + let mut exprs = Vec::new(); + if let Some(expr) = entry.inlist.clone() { + exprs.push(RuntimeFilterExpr { + filter_id: entry.id, + expr, + stats: entry.stats.clone(), + }); + } + if let Some(expr) = entry.min_max.clone() { + exprs.push(RuntimeFilterExpr { + filter_id: entry.id, + expr, + stats: entry.stats.clone(), + }); + } + exprs + }) + .collect(), ); - - let runtime_filter = ExprRuntimePruner::new(filters.clone()); if runtime_filter.prune(&self.func_ctx, self.table_schema.clone(), &part)? { return Ok(DataBlock::empty()); } @@ -165,13 +183,31 @@ impl AsyncTransform for ReadNativeDataTransform { let parts = block_part_meta.part_ptr.clone(); if !parts.is_empty() { let mut chunks = Vec::with_capacity(parts.len()); - let mut filters = self.context.get_inlist_runtime_filter_with_id(self.scan_id); - filters.extend( + let mut native_part_infos = Vec::with_capacity(parts.len()); + let runtime_filter = ExprRuntimePruner::new( self.context - .get_min_max_runtime_filter_with_id(self.scan_id), + .get_runtime_filters(self.scan_id) + .into_iter() + .flat_map(|entry| { + let mut exprs = Vec::new(); + if let Some(expr) = entry.inlist.clone() { + exprs.push(RuntimeFilterExpr { + filter_id: entry.id, + expr, + stats: entry.stats.clone(), + }); + } + if let Some(expr) = entry.min_max.clone() { + exprs.push(RuntimeFilterExpr { + filter_id: entry.id, + expr, + stats: entry.stats.clone(), + }); + } + exprs + }) + .collect(), ); - let mut native_part_infos = Vec::with_capacity(parts.len()); - let runtime_filter = ExprRuntimePruner::new(filters.clone()); for part in parts.into_iter() { if runtime_filter.prune(&self.func_ctx, self.table_schema.clone(), &part)? { continue; diff --git a/src/query/storages/fuse/src/operations/read/parquet_data_source_deserializer.rs b/src/query/storages/fuse/src/operations/read/parquet_data_source_deserializer.rs index 87921fd09f2c8..724b0ce3aa293 100644 --- a/src/query/storages/fuse/src/operations/read/parquet_data_source_deserializer.rs +++ b/src/query/storages/fuse/src/operations/read/parquet_data_source_deserializer.rs @@ -23,9 +23,9 @@ use databend_common_base::runtime::profile::Profile; use databend_common_base::runtime::profile::ProfileStatisticsName; use databend_common_catalog::plan::DataSourcePlan; use databend_common_catalog::plan::PartInfoPtr; -use databend_common_catalog::runtime_filter_info::RuntimeFilterReady; +use databend_common_catalog::runtime_filter_info::RuntimeFilterEntry; +use databend_common_catalog::runtime_filter_info::RuntimeFilterStats; use databend_common_catalog::table_context::TableContext; -use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::types::Bitmap; use databend_common_expression::types::DataType; @@ -42,7 +42,6 @@ use databend_common_pipeline::core::InputPort; use databend_common_pipeline::core::OutputPort; use databend_common_pipeline::core::Processor; use databend_common_pipeline::core::ProcessorPtr; -use databend_common_sql::IndexType; use roaring::RoaringTreemap; use xorf::BinaryFuse16; @@ -58,7 +57,6 @@ use crate::pruning::ExprBloomFilter; pub struct DeserializeDataTransform { ctx: Arc, - table_index: IndexType, scan_id: usize, scan_progress: Arc, block_reader: Arc, @@ -75,11 +73,16 @@ pub struct DeserializeDataTransform { virtual_reader: Arc>, base_block_ids: Option, - cached_runtime_filter: Option>, - // for merge_into target build. + cached_runtime_filter: Option>, need_reserve_block_info: bool, - need_wait_runtime_filter: bool, - runtime_filter_ready: Option>, +} + +#[derive(Clone)] +struct BloomRuntimeFilterRef { + column_index: FieldIndex, + filter_id: usize, + filter: BinaryFuse16, + stats: Arc, } unsafe impl Send for DeserializeDataTransform {} @@ -95,8 +98,6 @@ impl DeserializeDataTransform { virtual_reader: Arc>, ) -> Result { let scan_progress = ctx.get_scan_progress(); - let need_wait_runtime_filter = - !ctx.get_cluster().is_empty() && ctx.get_wait_runtime_filter(plan.scan_id); let mut src_schema: DataSchema = (block_reader.schema().as_ref()).into(); if let Some(virtual_reader) = virtual_reader.as_ref() { @@ -116,8 +117,7 @@ impl DeserializeDataTransform { let output_schema: DataSchema = (&output_schema).into(); let (need_reserve_block_info, _) = need_reserve_block_info(ctx.clone(), plan.table_index); Ok(ProcessorPtr::create(Box::new(DeserializeDataTransform { - ctx, - table_index: plan.table_index, + ctx: ctx.clone(), scan_id: plan.scan_id, scan_progress, block_reader, @@ -133,40 +133,60 @@ impl DeserializeDataTransform { base_block_ids: plan.base_block_ids.clone(), cached_runtime_filter: None, need_reserve_block_info, - need_wait_runtime_filter, - runtime_filter_ready: None, }))) } fn runtime_filter(&mut self, data_block: DataBlock) -> Result> { // Check if already cached runtime filters if self.cached_runtime_filter.is_none() { - let bloom_filters = self.ctx.get_bloom_runtime_filter_with_id(self.table_index); - let bloom_filters = bloom_filters + let bloom_filters = self + .ctx + .get_runtime_filters(self.scan_id) .into_iter() - .filter_map(|filter| { - let name = filter.0.as_str(); - // Some probe keys are not in the schema, they are derived from expressions. - self.src_schema - .index_of(name) - .ok() - .map(|idx| (idx, filter.1.clone())) + .filter_map(|entry| { + let filter_id = entry.id; + let RuntimeFilterEntry { bloom, stats, .. } = entry; + let bloom = bloom?; + let column_index = self.src_schema.index_of(bloom.column_name.as_str()).ok()?; + Some(BloomRuntimeFilterRef { + column_index, + filter_id, + filter: bloom.filter.clone(), + stats, + }) }) - .collect::>(); + .collect::>(); if bloom_filters.is_empty() { return Ok(None); } + let mut filter_ids = bloom_filters + .iter() + .map(|f| f.filter_id) + .collect::>(); + filter_ids.sort_unstable(); + log::info!( + "RUNTIME-FILTER: scan_id={} bloom_filters={} filter_ids={:?}", + self.scan_id, + bloom_filters.len(), + filter_ids + ); self.cached_runtime_filter = Some(bloom_filters); } let mut bitmaps = vec![]; - for (idx, filter) in self.cached_runtime_filter.as_ref().unwrap().iter() { + for runtime_filter in self.cached_runtime_filter.as_ref().unwrap().iter() { let mut bitmap = MutableBitmap::from_len_zeroed(data_block.num_rows()); - let probe_block_entry = data_block.get_by_offset(*idx); + let probe_block_entry = data_block.get_by_offset(runtime_filter.column_index); let probe_column = probe_block_entry.to_column(); // Apply bloom filter - ExprBloomFilter::new(filter.clone()).apply(probe_column, &mut bitmap)?; + let start = Instant::now(); + ExprBloomFilter::new(runtime_filter.filter.clone()).apply(probe_column, &mut bitmap)?; + let elapsed = start.elapsed(); + let unset_bits = bitmap.null_count(); + runtime_filter + .stats + .record_bloom(elapsed.as_nanos() as u64, unset_bits as u64); bitmaps.push(bitmap); } if !bitmaps.is_empty() { @@ -180,20 +200,6 @@ impl DeserializeDataTransform { Ok(None) } } - - fn need_wait_runtime_filter(&mut self) -> bool { - if !self.need_wait_runtime_filter { - return false; - } - self.need_wait_runtime_filter = false; - let runtime_filter_ready = self.ctx.get_runtime_filter_ready(self.scan_id); - if runtime_filter_ready.len() == 1 { - self.runtime_filter_ready = Some(runtime_filter_ready[0].clone()); - true - } else { - false - } - } } #[async_trait::async_trait] @@ -207,10 +213,6 @@ impl Processor for DeserializeDataTransform { } fn event(&mut self) -> Result { - if self.need_wait_runtime_filter() { - return Ok(Event::Async); - } - if self.output.is_finished() { self.input.finish(); return Ok(Event::Finished); @@ -293,24 +295,23 @@ impl Processor for DeserializeDataTransform { let origin_num_rows = data_block.num_rows(); let mut filter = None; - if self.ctx.has_bloom_runtime_filters(self.table_index) { - let start = Instant::now(); - let rows_before = data_block.num_rows(); - if let Some(bitmap) = self.runtime_filter(data_block.clone())? { - data_block = data_block.filter_with_bitmap(&bitmap)?; - filter = Some(bitmap); - let rows_after = data_block.num_rows(); - let bloom_duration = start.elapsed(); + let bloom_start = Instant::now(); + + let rows_before = data_block.num_rows(); + if let Some(bitmap) = self.runtime_filter(data_block.clone())? { + data_block = data_block.filter_with_bitmap(&bitmap)?; + filter = Some(bitmap); + let rows_after = data_block.num_rows(); + let bloom_duration = bloom_start.elapsed(); + Profile::record_usize_profile( + ProfileStatisticsName::RuntimeFilterBloomTime, + bloom_duration.as_nanos() as usize, + ); + if rows_before > rows_after { Profile::record_usize_profile( - ProfileStatisticsName::RuntimeFilterBloomTime, - bloom_duration.as_nanos() as usize, + ProfileStatisticsName::RuntimeFilterBloomRowsFiltered, + rows_before - rows_after, ); - if rows_before > rows_after { - Profile::record_usize_profile( - ProfileStatisticsName::RuntimeFilterBloomRowsFiltered, - rows_before - rows_after, - ); - } } } @@ -372,17 +373,4 @@ impl Processor for DeserializeDataTransform { Ok(()) } - - #[async_backtrace::framed] - async fn async_process(&mut self) -> Result<()> { - let runtime_filter_ready = self.runtime_filter_ready.as_mut().unwrap(); - let mut rx = runtime_filter_ready.runtime_filter_watcher.subscribe(); - if (*rx.borrow()).is_some() { - return Ok(()); - } - rx.changed() - .await - .map_err(|_| ErrorCode::TokioError("watcher's sender is dropped"))?; - Ok(()) - } } diff --git a/src/query/storages/fuse/src/operations/read/parquet_data_transform_reader.rs b/src/query/storages/fuse/src/operations/read/parquet_data_transform_reader.rs index 17e7918af1fcb..f08aeda7ec43f 100644 --- a/src/query/storages/fuse/src/operations/read/parquet_data_transform_reader.rs +++ b/src/query/storages/fuse/src/operations/read/parquet_data_transform_reader.rs @@ -47,6 +47,7 @@ use crate::io::VirtualColumnReader; use crate::operations::read::block_partition_meta::BlockPartitionMeta; use crate::operations::read::data_source_with_meta::DataSourceWithMeta; use crate::pruning::ExprRuntimePruner; +use crate::pruning::RuntimeFilterExpr; pub struct ReadStats { pub blocks_total: AtomicU64, @@ -141,11 +142,29 @@ impl Transform for ReadParquetDataTransform { debug_assert!(partitions.len() == 1); let part = partitions.pop().unwrap(); let prune_start = Instant::now(); - let mut filters = self.context.get_inlist_runtime_filter_with_id(self.scan_id); - filters.extend( - self.context - .get_min_max_runtime_filter_with_id(self.scan_id), - ); + let filters = self + .context + .get_runtime_filters(self.scan_id) + .into_iter() + .flat_map(|entry| { + let mut exprs = Vec::new(); + if let Some(expr) = entry.inlist.clone() { + exprs.push(RuntimeFilterExpr { + filter_id: entry.id, + expr, + stats: entry.stats.clone(), + }); + } + if let Some(expr) = entry.min_max.clone() { + exprs.push(RuntimeFilterExpr { + filter_id: entry.id, + expr, + stats: entry.stats.clone(), + }); + } + exprs + }) + .collect::>(); let exists_runtime_filter = !filters.is_empty(); @@ -246,14 +265,32 @@ impl AsyncTransform for ReadParquetDataTransform { let parts = block_part_meta.part_ptr.clone(); if !parts.is_empty() { let mut chunks = Vec::with_capacity(parts.len()); - let mut filters = self.context.get_inlist_runtime_filter_with_id(self.scan_id); - filters.extend( - self.context - .get_min_max_runtime_filter_with_id(self.scan_id), - ); let mut fuse_part_infos = Vec::with_capacity(parts.len()); - let runtime_filter = ExprRuntimePruner::new(filters.clone()); + let runtime_filter = ExprRuntimePruner::new( + self.context + .get_runtime_filters(self.scan_id) + .into_iter() + .flat_map(|entry| { + let mut exprs = Vec::new(); + if let Some(expr) = entry.inlist.clone() { + exprs.push(RuntimeFilterExpr { + filter_id: entry.id, + expr, + stats: entry.stats.clone(), + }); + } + if let Some(expr) = entry.min_max.clone() { + exprs.push(RuntimeFilterExpr { + filter_id: entry.id, + expr, + stats: entry.stats.clone(), + }); + } + exprs + }) + .collect(), + ); for part in parts.into_iter() { let prune_start = Instant::now(); self.stats.blocks_total.fetch_add(1, Ordering::Relaxed); diff --git a/src/query/storages/fuse/src/operations/read/runtime_filter_wait.rs b/src/query/storages/fuse/src/operations/read/runtime_filter_wait.rs new file mode 100644 index 0000000000000..d11a313c7b929 --- /dev/null +++ b/src/query/storages/fuse/src/operations/read/runtime_filter_wait.rs @@ -0,0 +1,144 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::any::Any; +use std::sync::Arc; +use std::time::Duration; + +use databend_common_base::base::tokio::time::timeout; +use databend_common_catalog::runtime_filter_info::RuntimeFilterReady; +use databend_common_catalog::table_context::TableContext; +use databend_common_exception::ErrorCode; +use databend_common_exception::Result; +use databend_common_pipeline::core::Event; +use databend_common_pipeline::core::InputPort; +use databend_common_pipeline::core::OutputPort; +use databend_common_pipeline::core::Processor; +use databend_common_pipeline::core::ProcessorPtr; +use databend_common_sql::IndexType; + +pub struct TransformRuntimeFilterWait { + ctx: Arc, + scan_id: IndexType, + input: Arc, + output: Arc, + runtime_filter_ready: Vec>, + wait_finished: bool, +} + +impl TransformRuntimeFilterWait { + pub fn create( + ctx: Arc, + scan_id: IndexType, + input: Arc, + output: Arc, + ) -> ProcessorPtr { + ProcessorPtr::create(Box::new(TransformRuntimeFilterWait { + ctx, + scan_id, + input, + output, + runtime_filter_ready: Vec::new(), + wait_finished: false, + })) + } +} + +#[async_trait::async_trait] +impl Processor for TransformRuntimeFilterWait { + fn name(&self) -> String { + String::from("TransformRuntimeFilterWait") + } + + fn as_any(&mut self) -> &mut dyn Any { + self + } + + fn event(&mut self) -> Result { + if !self.wait_finished { + return Ok(Event::Async); + } + + if self.output.is_finished() { + self.input.finish(); + return Ok(Event::Finished); + } + + if !self.output.can_push() { + self.input.set_not_need_data(); + return Ok(Event::NeedConsume); + } + + if let Some(data) = self.input.pull_data() { + self.output.push_data(data); + return Ok(Event::NeedConsume); + } + + if self.input.is_finished() { + self.output.finish(); + return Ok(Event::Finished); + } + + self.input.set_need_data(); + Ok(Event::NeedData) + } + + #[async_backtrace::framed] + async fn async_process(&mut self) -> Result<()> { + if self.runtime_filter_ready.is_empty() { + self.runtime_filter_ready = self.ctx.get_runtime_filter_ready(self.scan_id); + } + + if self.runtime_filter_ready.is_empty() { + log::info!( + "RUNTIME-FILTER: scan_id={} no runtime filters found, skipping wait", + self.scan_id + ); + self.wait_finished = true; + return Ok(()); + } + + log::info!( + "RUNTIME-FILTER: scan_id={} waiting for {} runtime filters", + self.scan_id, + self.runtime_filter_ready.len() + ); + + let timeout_duration = Duration::from_secs(30); + for runtime_filter_ready in &self.runtime_filter_ready { + let mut rx = runtime_filter_ready.runtime_filter_watcher.subscribe(); + if (*rx.borrow()).is_some() { + continue; + } + + match timeout(timeout_duration, rx.changed()).await { + Ok(Ok(())) => {} + Ok(Err(_)) => { + return Err(ErrorCode::TokioError("watcher's sender is dropped")); + } + Err(_) => { + log::warn!( + "Runtime filter wait timeout after {:?} for scan_id: {}", + timeout_duration, + self.scan_id + ); + } + } + } + + self.runtime_filter_ready.clear(); + self.wait_finished = true; + Ok(()) + } +} diff --git a/src/query/storages/fuse/src/pruning/expr_runtime_pruner.rs b/src/query/storages/fuse/src/pruning/expr_runtime_pruner.rs index 424c7db422ee7..4dba9b1d48364 100644 --- a/src/query/storages/fuse/src/pruning/expr_runtime_pruner.rs +++ b/src/query/storages/fuse/src/pruning/expr_runtime_pruner.rs @@ -14,10 +14,12 @@ use std::collections::HashMap; use std::sync::Arc; +use std::time::Instant; use databend_common_base::runtime::profile::Profile; use databend_common_base::runtime::profile::ProfileStatisticsName; use databend_common_catalog::plan::PartInfoPtr; +use databend_common_catalog::runtime_filter_info::RuntimeFilterStats; use databend_common_exception::Result; use databend_common_expression::Constant; use databend_common_expression::ConstantFolder; @@ -34,12 +36,19 @@ use crate::FuseBlockPartInfo; /// Runtime pruner that uses expressions to prune partitions. pub struct ExprRuntimePruner { - exprs: Vec>, + exprs: Vec, +} + +#[derive(Clone)] +pub struct RuntimeFilterExpr { + pub filter_id: usize, + pub expr: Expr, + pub stats: Arc, } impl ExprRuntimePruner { /// Create a new expression runtime pruner. - pub fn new(exprs: Vec>) -> Self { + pub fn new(exprs: Vec) -> Self { Self { exprs } } @@ -56,56 +65,84 @@ impl ExprRuntimePruner { } let part = FuseBlockPartInfo::from_part(part)?; - let pruned = self.exprs.iter().any(|filter| { + let mut partition_pruned = false; + for entry in self.exprs.iter() { + let start = Instant::now(); + let filter = &entry.expr; + let mut should_prune = false; // If the filter is a constant false, we can prune the partition. - if matches!(filter, Expr::Constant(Constant { scalar: Scalar::Boolean(false), .. })) { - return true; - } - let column_refs = filter.column_refs(); - // Currently only support filter with one column (probe key). - debug_assert!(column_refs.len() == 1); - let ty = column_refs.values().last().unwrap(); - let name = column_refs.keys().last().unwrap(); + if matches!( + filter, + Expr::Constant(Constant { + scalar: Scalar::Boolean(false), + .. + }) + ) { + should_prune = true; + } else { + let column_refs = filter.column_refs(); + // Currently only support filter with one column (probe key). + debug_assert!(column_refs.len() == 1); + let ty = column_refs.values().last().unwrap(); + let name = column_refs.keys().last().unwrap(); - if let Some(stats) = &part.columns_stat { - let column_ids = table_schema.leaf_columns_of(name); - if column_ids.len() != 1 { - return false; - } - debug_assert!(column_ids.len() == 1); - if let Some(stat) = stats.get(&column_ids[0]) { - let stats = vec![stat]; - let domain = statistics_to_domain(stats, ty); + if let Some(stats) = &part.columns_stat { + let column_ids = table_schema.leaf_columns_of(name); + if column_ids.len() == 1 { + if let Some(stat) = stats.get(&column_ids[0]) { + let stats = vec![stat]; + let domain = statistics_to_domain(stats, ty); - let mut input_domains = HashMap::new(); - input_domains.insert(name.to_string(), domain.clone()); + let mut input_domains = HashMap::new(); + input_domains.insert(name.to_string(), domain.clone()); - let (new_expr, _) = ConstantFolder::fold_with_domain( - filter, - &input_domains, - func_ctx, - &BUILTIN_FUNCTIONS, - ); - debug!("Runtime filter after constant fold is {:?}", new_expr.sql_display()); - return matches!(new_expr, Expr::Constant(Constant { - scalar: Scalar::Boolean(false), - .. - })); + let (new_expr, _) = ConstantFolder::fold_with_domain( + filter, + &input_domains, + func_ctx, + &BUILTIN_FUNCTIONS, + ); + debug!( + "Runtime filter after constant fold is {:?}", + new_expr.sql_display() + ); + if matches!( + new_expr, + Expr::Constant(Constant { + scalar: Scalar::Boolean(false), + .. + }) + ) { + should_prune = true; + } + } + } + } else { + info!("Can't prune the partition by runtime filter, because there is no statistics for the partition"); } } - info!("Can't prune the partition by runtime filter, because there is no statistics for the partition"); - false - }); - - if pruned { - info!( - "Pruned partition with {:?} rows by runtime filter", - part.nums_rows + let elapsed = start.elapsed(); + entry.stats.record_inlist_min_max( + elapsed.as_nanos() as u64, + if should_prune { + part.nums_rows as u64 + } else { + 0 + }, + if should_prune { 1 } else { 0 }, ); + + if should_prune { + partition_pruned = true; + break; + } + } + + if partition_pruned { Profile::record_usize_profile(ProfileStatisticsName::RuntimeFilterPruneParts, 1); } - Ok(pruned) + Ok(partition_pruned) } } diff --git a/src/query/storages/fuse/src/pruning/mod.rs b/src/query/storages/fuse/src/pruning/mod.rs index 650e11aba4c7b..353e2e12eb0a7 100644 --- a/src/query/storages/fuse/src/pruning/mod.rs +++ b/src/query/storages/fuse/src/pruning/mod.rs @@ -29,6 +29,7 @@ pub use bloom_pruner::BloomPruner; pub use bloom_pruner::BloomPrunerCreator; pub use expr_bloom_filter::ExprBloomFilter; pub use expr_runtime_pruner::ExprRuntimePruner; +pub use expr_runtime_pruner::RuntimeFilterExpr; pub use fuse_pruner::table_sample; pub use fuse_pruner::FusePruner; pub use fuse_pruner::PruningContext; diff --git a/tests/sqllogictests/src/client/http_client.rs b/tests/sqllogictests/src/client/http_client.rs index 9a704a02c937b..1aeb05448cc12 100644 --- a/tests/sqllogictests/src/client/http_client.rs +++ b/tests/sqllogictests/src/client/http_client.rs @@ -164,23 +164,25 @@ impl HttpClient { let mut schema = std::mem::take(&mut response.schema); let mut parsed_rows = vec![]; - self.handle_response(&response, &mut parsed_rows)?; - while let Some(next_uri) = &response.next_uri { - let url = format!("http://127.0.0.1:{port}{next_uri}"); - let mut new_response = self.poll_query_result(&url).await?; - if schema.is_empty() && !new_response.schema.is_empty() { - schema = std::mem::take(&mut new_response.schema); + + loop { + self.handle_response(&response, &mut parsed_rows)?; + if let Some(error) = &response.error { + return Err(format_error(error.clone()).into()); } - if new_response.next_uri.is_some() { - self.handle_response(&new_response, &mut parsed_rows)?; - response = new_response; - } else { - break; + + match &response.next_uri { + Some(next_uri) => { + let url = format!("http://127.0.0.1:{port}{next_uri}"); + let mut new_response = self.poll_query_result(&url).await?; + if schema.is_empty() && !new_response.schema.is_empty() { + schema = std::mem::take(&mut new_response.schema); + } + response = new_response; + } + None => break, } } - if let Some(error) = response.error { - return Err(format_error(error).into()); - } if self.debug { println!( diff --git a/tests/sqllogictests/suites/mode/cluster/create_table.test b/tests/sqllogictests/suites/mode/cluster/create_table.test index d6216f4a11e81..99280bd872287 100644 --- a/tests/sqllogictests/suites/mode/cluster/create_table.test +++ b/tests/sqllogictests/suites/mode/cluster/create_table.test @@ -41,6 +41,7 @@ EvalScalar ├── rank limit: 3 └── TableScan ├── table: default.system.numbers + ├── scan id: 0 ├── output columns: [number (#0)] ├── read rows: 10000000 ├── read size: 76.29 MiB diff --git a/tests/sqllogictests/suites/mode/cluster/distributed_sort.test b/tests/sqllogictests/suites/mode/cluster/distributed_sort.test index 1545b080f0f3e..77f7548c20524 100644 --- a/tests/sqllogictests/suites/mode/cluster/distributed_sort.test +++ b/tests/sqllogictests/suites/mode/cluster/distributed_sort.test @@ -22,6 +22,7 @@ Sort(Final) ├── estimated rows: 0.00 └── TableScan ├── table: default.default.t_distributed_sort + ├── scan id: 0 ├── output columns: [a (#0), b (#1), c (#2), d (#3), e (#6)] ├── read rows: 0 ├── read size: 0 @@ -56,6 +57,7 @@ Limit ├── estimated rows: 0.00 └── TableScan ├── table: default.default.t_distributed_sort + ├── scan id: 0 ├── output columns: [a (#0), b (#1), c (#2), d (#3), e (#6)] ├── read rows: 0 ├── read size: 0 @@ -92,6 +94,7 @@ RowFetch ├── estimated rows: 0.00 └── TableScan ├── table: default.default.t_distributed_sort + ├── scan id: 0 ├── output columns: [a (#0), _row_id (#7)] ├── read rows: 0 ├── read size: 0 @@ -114,6 +117,7 @@ Limit ├── exchange type: Merge └── TableScan ├── table: default.system.numbers + ├── scan id: 0 ├── output columns: [number (#0)] ├── read rows: 1000 ├── read size: 7.81 KiB @@ -140,6 +144,7 @@ Limit ├── estimated rows: 110.00 └── TableScan ├── table: default.system.numbers + ├── scan id: 0 ├── output columns: [number (#0)] ├── read rows: 110 ├── read size: < 1 KiB @@ -166,6 +171,7 @@ Limit ├── estimated rows: 110.00 └── TableScan ├── table: default.system.numbers + ├── scan id: 0 ├── output columns: [number (#0)] ├── read rows: 110 ├── read size: < 1 KiB diff --git a/tests/sqllogictests/suites/mode/cluster/enfore_local.test b/tests/sqllogictests/suites/mode/cluster/enfore_local.test index 048128d5fbfe6..c7c5050e321c1 100644 --- a/tests/sqllogictests/suites/mode/cluster/enfore_local.test +++ b/tests/sqllogictests/suites/mode/cluster/enfore_local.test @@ -15,6 +15,7 @@ AggregateFinal ├── estimated rows: 33333.33 └── TableScan ├── table: default.system.numbers_mt + ├── scan id: 0 ├── output columns: [number (#0)] ├── read rows: 100000 ├── read size: 781.25 KiB diff --git a/tests/sqllogictests/suites/mode/cluster/exchange.test b/tests/sqllogictests/suites/mode/cluster/exchange.test index a4e0d9b41ddfc..0b95a85d1b1fe 100644 --- a/tests/sqllogictests/suites/mode/cluster/exchange.test +++ b/tests/sqllogictests/suites/mode/cluster/exchange.test @@ -15,13 +15,14 @@ Exchange ├── keys is null equal: [false] ├── filters: [] ├── build join filters: - │ └── filter id:0, build key:t.number (#0), probe key:t1.number (#1), filter type:inlist,min_max + │ └── filter id:0, build key:t.number (#0), probe targets:[t1.number (#1)@scan1], filter type:bloom,inlist,min_max ├── estimated rows: 2.00 ├── Exchange(Build) │ ├── output columns: [t.number (#0)] │ ├── exchange type: Broadcast │ └── TableScan │ ├── table: default.system.numbers + │ ├── scan id: 0 │ ├── output columns: [number (#0)] │ ├── read rows: 1 │ ├── read size: < 1 KiB @@ -31,6 +32,7 @@ Exchange │ └── estimated rows: 1.00 └── TableScan(Probe) ├── table: default.system.numbers + ├── scan id: 1 ├── output columns: [number (#1)] ├── read rows: 2 ├── read size: < 1 KiB @@ -54,7 +56,7 @@ Exchange ├── keys is null equal: [false] ├── filters: [] ├── build join filters: - │ └── filter id:1, build key:t.number (#0), probe key:t2.number (#2), filter type:inlist,min_max + │ └── filter id:1, build key:t.number (#0), probe targets:[t2.number (#2)@scan2], filter type:bloom,inlist,min_max ├── estimated rows: 6.00 ├── Exchange(Build) │ ├── output columns: [t1.number (#1), t.number (#0)] @@ -67,13 +69,14 @@ Exchange │ ├── keys is null equal: [false] │ ├── filters: [] │ ├── build join filters: - │ │ └── filter id:0, build key:t.number (#0), probe key:t1.number (#1), filter type:inlist,min_max + │ │ └── filter id:0, build key:t.number (#0), probe targets:[t1.number (#1)@scan1], filter type:bloom,inlist,min_max │ ├── estimated rows: 2.00 │ ├── Exchange(Build) │ │ ├── output columns: [t.number (#0)] │ │ ├── exchange type: Broadcast │ │ └── TableScan │ │ ├── table: default.system.numbers + │ │ ├── scan id: 0 │ │ ├── output columns: [number (#0)] │ │ ├── read rows: 1 │ │ ├── read size: < 1 KiB @@ -83,6 +86,7 @@ Exchange │ │ └── estimated rows: 1.00 │ └── TableScan(Probe) │ ├── table: default.system.numbers + │ ├── scan id: 1 │ ├── output columns: [number (#1)] │ ├── read rows: 2 │ ├── read size: < 1 KiB @@ -93,6 +97,7 @@ Exchange │ └── estimated rows: 2.00 └── TableScan(Probe) ├── table: default.system.numbers + ├── scan id: 2 ├── output columns: [number (#2)] ├── read rows: 3 ├── read size: < 1 KiB @@ -116,7 +121,7 @@ Exchange ├── keys is null equal: [false] ├── filters: [] ├── build join filters: - │ └── filter id:1, build key:t.b (#1), probe key:t2.number (#3), filter type:inlist,min_max + │ └── filter id:1, build key:t.b (#1), probe targets:[t2.number (#3)@scan2], filter type:bloom,inlist,min_max ├── estimated rows: 6.00 ├── Exchange(Build) │ ├── output columns: [t1.number (#2), b (#1), numbers.number (#0)] @@ -129,7 +134,7 @@ Exchange │ ├── keys is null equal: [false] │ ├── filters: [] │ ├── build join filters: - │ │ └── filter id:0, build key:t.a (#0), probe key:t1.number (#2), filter type:inlist,min_max + │ │ └── filter id:0, build key:t.a (#0), probe targets:[t1.number (#2)@scan1], filter type:bloom,inlist,min_max │ ├── estimated rows: 2.00 │ ├── Exchange(Build) │ │ ├── output columns: [numbers.number (#0), b (#1)] @@ -140,6 +145,7 @@ Exchange │ │ ├── estimated rows: 1.00 │ │ └── TableScan │ │ ├── table: default.system.numbers + │ │ ├── scan id: 0 │ │ ├── output columns: [number (#0)] │ │ ├── read rows: 1 │ │ ├── read size: < 1 KiB @@ -149,6 +155,7 @@ Exchange │ │ └── estimated rows: 1.00 │ └── TableScan(Probe) │ ├── table: default.system.numbers + │ ├── scan id: 1 │ ├── output columns: [number (#2)] │ ├── read rows: 2 │ ├── read size: < 1 KiB @@ -159,6 +166,7 @@ Exchange │ └── estimated rows: 2.00 └── TableScan(Probe) ├── table: default.system.numbers + ├── scan id: 2 ├── output columns: [number (#3)] ├── read rows: 3 ├── read size: < 1 KiB @@ -199,6 +207,7 @@ Exchange │ ├── estimated rows: 1.00 │ └── TableScan │ ├── table: default.system.numbers + │ ├── scan id: 0 │ ├── output columns: [number (#0)] │ ├── read rows: 1 │ ├── read size: < 1 KiB @@ -208,6 +217,7 @@ Exchange │ └── estimated rows: 1.00 └── TableScan(Probe) ├── table: default.system.numbers + ├── scan id: 1 ├── output columns: [number (#2)] ├── read rows: 2 ├── read size: < 1 KiB @@ -230,6 +240,7 @@ Fragment 0: ├── estimated rows: 1.00 └── TableScan ├── table: default.system.numbers + ├── scan id: 0 ├── output columns: [number (#0)] ├── read rows: 1 ├── read size: < 1 KiB @@ -272,6 +283,7 @@ Fragment 2: │ └── source fragment: [1] └── TableScan(Probe) ├── table: default.system.numbers + ├── scan id: 1 ├── output columns: [number (#2)] ├── read rows: 2 ├── read size: < 1 KiB @@ -322,13 +334,14 @@ AggregateFinal │ ├── keys is null equal: [false] │ ├── filters: [] │ ├── build join filters: - │ │ └── filter id:0, build key:t1.a (#0), probe key:t2.a (#1), filter type:inlist,min_max + │ │ └── filter id:0, build key:t1.a (#0), probe targets:[t2.a (#1)@scan1], filter type:bloom,inlist,min_max │ ├── estimated rows: 10000.00 │ ├── Exchange(Build) │ │ ├── output columns: [numbers.number (#0)] │ │ ├── exchange type: Broadcast │ │ └── TableScan │ │ ├── table: default.system.numbers + │ │ ├── scan id: 0 │ │ ├── output columns: [number (#0)] │ │ ├── read rows: 10 │ │ ├── read size: < 1 KiB @@ -338,6 +351,7 @@ AggregateFinal │ │ └── estimated rows: 10.00 │ └── TableScan(Probe) │ ├── table: default.system.numbers + │ ├── scan id: 1 │ ├── output columns: [number (#1)] │ ├── read rows: 1000 │ ├── read size: 7.81 KiB @@ -354,13 +368,14 @@ AggregateFinal ├── keys is null equal: [false] ├── filters: [] ├── build join filters: - │ └── filter id:1, build key:t3.a (#3), probe key:t1.a (#2), filter type:inlist,min_max + │ └── filter id:1, build key:t3.a (#3), probe targets:[t1.a (#2)@scan2], filter type:bloom,inlist,min_max ├── estimated rows: 100.00 ├── Exchange(Build) │ ├── output columns: [numbers.number (#3)] │ ├── exchange type: Broadcast │ └── TableScan │ ├── table: default.system.numbers + │ ├── scan id: 3 │ ├── output columns: [number (#3)] │ ├── read rows: 10 │ ├── read size: < 1 KiB @@ -370,6 +385,7 @@ AggregateFinal │ └── estimated rows: 10.00 └── TableScan(Probe) ├── table: default.system.numbers + ├── scan id: 2 ├── output columns: [number (#2)] ├── read rows: 10 ├── read size: < 1 KiB @@ -415,13 +431,14 @@ AggregateFinal │ ├── keys is null equal: [false] │ ├── filters: [] │ ├── build join filters: - │ │ └── filter id:0, build key:t1.a (#0), probe key:t2.a (#1), filter type:inlist,min_max + │ │ └── filter id:0, build key:t1.a (#0), probe targets:[t2.a (#1)@scan1], filter type:bloom,inlist,min_max │ ├── estimated rows: 10000.00 │ ├── Exchange(Build) │ │ ├── output columns: [numbers.number (#0)] │ │ ├── exchange type: Broadcast │ │ └── TableScan │ │ ├── table: default.system.numbers + │ │ ├── scan id: 0 │ │ ├── output columns: [number (#0)] │ │ ├── read rows: 10 │ │ ├── read size: < 1 KiB @@ -431,6 +448,7 @@ AggregateFinal │ │ └── estimated rows: 10.00 │ └── TableScan(Probe) │ ├── table: default.system.numbers + │ ├── scan id: 1 │ ├── output columns: [number (#1)] │ ├── read rows: 1000 │ ├── read size: 7.81 KiB @@ -459,13 +477,14 @@ AggregateFinal ├── keys is null equal: [false] ├── filters: [] ├── build join filters: - │ └── filter id:1, build key:t3.a (#3), probe key:t1.a (#2), filter type:inlist,min_max + │ └── filter id:1, build key:t3.a (#3), probe targets:[t1.a (#2)@scan2], filter type:bloom,inlist,min_max ├── estimated rows: 100.00 ├── Exchange(Build) │ ├── output columns: [numbers.number (#3)] │ ├── exchange type: Broadcast │ └── TableScan │ ├── table: default.system.numbers + │ ├── scan id: 3 │ ├── output columns: [number (#3)] │ ├── read rows: 10 │ ├── read size: < 1 KiB @@ -475,6 +494,7 @@ AggregateFinal │ └── estimated rows: 10.00 └── TableScan(Probe) ├── table: default.system.numbers + ├── scan id: 2 ├── output columns: [number (#2)] ├── read rows: 10 ├── read size: < 1 KiB @@ -519,13 +539,14 @@ AggregateFinal │ ├── keys is null equal: [false] │ ├── filters: [] │ ├── build join filters: - │ │ └── filter id:0, build key:t1.a (#0), probe key:t2.a (#1), filter type:inlist,min_max + │ │ └── filter id:0, build key:t1.a (#0), probe targets:[t2.a (#1)@scan1], filter type:bloom,inlist,min_max │ ├── estimated rows: 10000.00 │ ├── Exchange(Build) │ │ ├── output columns: [numbers.number (#0)] │ │ ├── exchange type: Broadcast │ │ └── TableScan │ │ ├── table: default.system.numbers + │ │ ├── scan id: 0 │ │ ├── output columns: [number (#0)] │ │ ├── read rows: 10 │ │ ├── read size: < 1 KiB @@ -535,6 +556,7 @@ AggregateFinal │ │ └── estimated rows: 10.00 │ └── TableScan(Probe) │ ├── table: default.system.numbers + │ ├── scan id: 1 │ ├── output columns: [number (#1)] │ ├── read rows: 1000 │ ├── read size: 7.81 KiB @@ -545,6 +567,7 @@ AggregateFinal │ └── estimated rows: 1000.00 └── TableScan ├── table: default.system.numbers + ├── scan id: 2 ├── output columns: [number (#2)] ├── read rows: 10 ├── read size: < 1 KiB @@ -577,6 +600,7 @@ Exchange │ ├── exchange type: Broadcast │ └── TableScan │ ├── table: default.system.numbers + │ ├── scan id: 2 │ ├── output columns: [number (#2)] │ ├── read rows: 10 │ ├── read size: < 1 KiB @@ -597,6 +621,7 @@ Exchange │ ├── exchange type: Broadcast │ └── TableScan │ ├── table: default.system.numbers + │ ├── scan id: 0 │ ├── output columns: [number (#0)] │ ├── read rows: 10 │ ├── read size: < 1 KiB @@ -606,6 +631,7 @@ Exchange │ └── estimated rows: 10.00 └── TableScan(Probe) ├── table: default.system.numbers + ├── scan id: 1 ├── output columns: [number (#1)] ├── read rows: 1000 ├── read size: 7.81 KiB @@ -631,13 +657,14 @@ Exchange ├── keys is null equal: [false] ├── filters: [] ├── build join filters(distributed): - │ └── filter id:0, build key:t2.number (#1), probe key:t1.number (#0), filter type:inlist,min_max + │ └── filter id:0, build key:t2.number (#1), probe targets:[t1.number (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 200.00 ├── Exchange(Build) │ ├── output columns: [t2.number (#1)] │ ├── exchange type: Hash(t2.number (#1)) │ └── TableScan │ ├── table: default.system.numbers + │ ├── scan id: 1 │ ├── output columns: [number (#1)] │ ├── read rows: 20 │ ├── read size: < 1 KiB @@ -650,6 +677,7 @@ Exchange ├── exchange type: Hash(t1.number (#0)) └── TableScan ├── table: default.system.numbers + ├── scan id: 0 ├── output columns: [number (#0)] ├── read rows: 10 ├── read size: < 1 KiB diff --git a/tests/sqllogictests/suites/mode/cluster/explain_analyze.test b/tests/sqllogictests/suites/mode/cluster/explain_analyze.test index 10d0ff7e0e9dd..8dcf7189b25b3 100644 --- a/tests/sqllogictests/suites/mode/cluster/explain_analyze.test +++ b/tests/sqllogictests/suites/mode/cluster/explain_analyze.test @@ -43,6 +43,7 @@ EvalScalar ├── output bytes: 7.63 MiB ├── bytes scanned: 7.63 MiB ├── table: default.system.numbers + ├── scan id: 0 ├── output columns: [number (#0)] ├── read rows: 1000000 ├── read size: 7.63 MiB @@ -104,6 +105,7 @@ Exchange ├── exchange type: Merge └── TableScan ├── table: default.default.article + ├── scan id: 0 ├── output columns: [article_id (#0), author_id (#1), viewer_id (#2), view_date (#3)] ├── read rows: 6 ├── read size: < 1 KiB @@ -127,6 +129,7 @@ Exchange ├── output bytes: 120.00 B ├── bytes scanned: 120.00 B ├── table: default.default.article + ├── scan id: 0 ├── output columns: [article_id (#0), author_id (#1), viewer_id (#2), view_date (#3)] ├── read rows: 6 ├── read size: < 1 KiB @@ -156,6 +159,7 @@ Exchange ├── output bytes: 80.00 B ├── bytes scanned: 80.00 B ├── table: default.default.article + ├── scan id: 0 ├── output columns: [article_id (#0), author_id (#1), viewer_id (#2), view_date (#3)] ├── read rows: 4 ├── read size: < 1 KiB @@ -177,7 +181,6 @@ Exchange ├── exchange bytes: 1.09 KiB ├── output rows: 1 ├── output bytes: 42.00 B - ├── runtime filter build time: ├── output columns: [article.article_id (#0), article.author_id (#1), article.viewer_id (#2), article.view_date (#3), author.name (#5), author.id (#4)] ├── join type: INNER ├── build keys: [author.id (#4)] @@ -185,7 +188,7 @@ Exchange ├── keys is null equal: [false] ├── filters: [] ├── build join filters: - │ └── filter id:0, build key:author.id (#4), probe key:article.author_id (#1), filter type:inlist,min_max + │ └── filter id:0, build key:author.id (#4), probe targets:[article.author_id (#1)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 2.50 ├── Exchange(Build) │ ├── output columns: [author.id (#4), author.name (#5)] @@ -204,6 +207,7 @@ Exchange │ ├── output bytes: 31.00 B │ ├── bytes scanned: 31.00 B │ ├── table: default.default.author + │ ├── scan id: 1 │ ├── output columns: [id (#4), name (#5)] │ ├── read rows: 1 │ ├── read size: < 1 KiB @@ -215,12 +219,11 @@ Exchange └── TableScan(Probe) ├── cpu time: ├── wait time: - ├── output rows: 1 - ├── output bytes: 20.00 B - ├── bytes scanned: 20.00 B - ├── parts pruned by runtime filter: 5 - ├── runtime filter inlist/min-max time: + ├── output rows: 6 + ├── output bytes: 120.00 B + ├── bytes scanned: 120.00 B ├── table: default.default.article + ├── scan id: 0 ├── output columns: [article_id (#0), author_id (#1), viewer_id (#2), view_date (#3)] ├── read rows: 6 ├── read size: < 1 KiB diff --git a/tests/sqllogictests/suites/mode/cluster/explain_v2.test b/tests/sqllogictests/suites/mode/cluster/explain_v2.test index bbba528e20c00..47d87c3c925de 100644 --- a/tests/sqllogictests/suites/mode/cluster/explain_v2.test +++ b/tests/sqllogictests/suites/mode/cluster/explain_v2.test @@ -28,6 +28,7 @@ Exchange ├── estimated rows: 99.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 100 ├── read size: < 1 KiB @@ -55,7 +56,7 @@ Exchange ├── keys is null equal: [false] ├── filters: [] ├── build join filters: - │ └── filter id:0, build key:t2.a (#2), probe key:t1.a (#0), filter type:inlist,min_max + │ └── filter id:0, build key:t2.a (#2), probe targets:[t1.a (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 99.92 ├── Exchange(Build) │ ├── output columns: [t2.a (#2), t2.b (#3)] @@ -66,6 +67,7 @@ Exchange │ ├── estimated rows: 99.92 │ └── TableScan │ ├── table: default.default.t2 + │ ├── scan id: 1 │ ├── output columns: [a (#2), b (#3)] │ ├── read rows: 100 │ ├── read size: < 1 KiB @@ -80,6 +82,7 @@ Exchange ├── estimated rows: 99.92 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 100 ├── read size: < 1 KiB @@ -104,13 +107,14 @@ Exchange ├── keys is null equal: [false] ├── filters: [] ├── build join filters: - │ └── filter id:0, build key:t2.a (#2), probe key:t1.a (#0), filter type:inlist,min_max + │ └── filter id:0, build key:t2.a (#2), probe targets:[t1.a (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 100.00 ├── Exchange(Build) │ ├── output columns: [t2.a (#2), t2.b (#3)] │ ├── exchange type: Broadcast │ └── TableScan │ ├── table: default.default.t2 + │ ├── scan id: 1 │ ├── output columns: [a (#2), b (#3)] │ ├── read rows: 100 │ ├── read size: < 1 KiB @@ -121,6 +125,7 @@ Exchange │ └── estimated rows: 100.00 └── TableScan(Probe) ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 100 ├── read size: < 1 KiB @@ -232,13 +237,14 @@ Limit ├── keys is null equal: [false] ├── filters: [] ├── build join filters: - │ └── filter id:0, build key:t2.a (#2), probe key:t1.a (#0), filter type:inlist,min_max + │ └── filter id:0, build key:t2.a (#2), probe targets:[t1.a (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 100.00 ├── Exchange(Build) │ ├── output columns: [t2.a (#2)] │ ├── exchange type: Broadcast │ └── TableScan │ ├── table: default.default.t2 + │ ├── scan id: 1 │ ├── output columns: [a (#2)] │ ├── read rows: 100 │ ├── read size: < 1 KiB @@ -249,6 +255,7 @@ Limit │ └── estimated rows: 100.00 └── TableScan(Probe) ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 100 ├── read size: < 1 KiB @@ -276,13 +283,14 @@ Exchange ├── keys is null equal: [false] ├── filters: [] ├── build join filters: - │ └── filter id:0, build key:t2.a (#2), probe key:t1.a (#0), filter type:inlist,min_max + │ └── filter id:0, build key:t2.a (#2), probe targets:[t1.a (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 100.00 ├── Exchange(Build) │ ├── output columns: [t2.a (#2), t2.b (#3)] │ ├── exchange type: Broadcast │ └── TableScan │ ├── table: default.default.t2 + │ ├── scan id: 1 │ ├── output columns: [a (#2), b (#3)] │ ├── read rows: 100 │ ├── read size: < 1 KiB @@ -293,6 +301,7 @@ Exchange │ └── estimated rows: 100.00 └── TableScan(Probe) ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 100 ├── read size: < 1 KiB @@ -327,6 +336,7 @@ Limit ├── estimated rows: 10.00 └── TableScan ├── table: default.system.numbers + ├── scan id: 0 ├── output columns: [number (#0)] ├── read rows: 10 ├── read size: < 1 KiB @@ -363,6 +373,7 @@ AggregateFinal │ ├── exchange type: Hash(CAST(y.a (#1) AS UInt64 NULL)) │ └── TableScan │ ├── table: default.system.numbers + │ ├── scan id: 1 │ ├── output columns: [number (#1)] │ ├── read rows: 5 │ ├── read size: < 1 KiB @@ -379,6 +390,7 @@ AggregateFinal ├── estimated rows: 10.00 └── TableScan ├── table: default.system.numbers + ├── scan id: 0 ├── output columns: [number (#0)] ├── read rows: 10 ├── read size: < 1 KiB @@ -416,7 +428,7 @@ Exchange ├── keys is null equal: [false] ├── filters: [] ├── build join filters: - │ └── filter id:0, build key:CAST(CAST(subquery_2 (#2) AS UInt8 NULL) AS Int32 NULL), probe key:t1.a (#0), filter type:inlist,min_max + │ └── filter id:0, build key:CAST(CAST(subquery_2 (#2) AS UInt8 NULL) AS Int32 NULL), probe targets:[t1.a (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 2.00 ├── Exchange(Build) │ ├── output columns: [col0 (#2)] @@ -438,6 +450,7 @@ Exchange │ └── column 0: [1, 2] └── TableScan(Probe) ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 3 ├── read size: < 1 KiB @@ -497,7 +510,7 @@ Exchange ├── keys is null equal: [false] ├── filters: [] ├── build join filters(distributed): - │ └── filter id:0, build key:t1.a0f (#28), probe key:t2.a0f (#2), filter type:inlist,min_max + │ └── filter id:0, build key:t1.a0f (#28), probe targets:[t2.a0f (#2)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 0.00 ├── Filter(Build) │ ├── output columns: [a00c.a0f (#28)] @@ -519,6 +532,7 @@ Exchange │ ├── exchange type: Hash(a00c.a0f (#28)) │ └── TableScan │ ├── table: default.default.a00c + │ ├── scan id: 1 │ ├── output columns: [a0f (#28), a0t (#42)] │ ├── read rows: 0 │ ├── read size: 0 @@ -551,6 +565,7 @@ Exchange ├── estimated rows: 0.00 └── TableScan ├── table: default.default.a00c + ├── scan id: 0 ├── output columns: [a0f (#2), a0t (#16), a0w (#19)] ├── read rows: 0 ├── read size: 0 diff --git a/tests/sqllogictests/suites/mode/cluster/filter_nulls.test b/tests/sqllogictests/suites/mode/cluster/filter_nulls.test index 9eeb396265168..41b2df2ad51d8 100644 --- a/tests/sqllogictests/suites/mode/cluster/filter_nulls.test +++ b/tests/sqllogictests/suites/mode/cluster/filter_nulls.test @@ -46,7 +46,7 @@ Exchange ├── keys is null equal: [false] ├── filters: [] ├── build join filters: - │ └── filter id:0, build key:table2.value (#1), probe key:table1.value (#0), filter type:inlist,min_max + │ └── filter id:0, build key:table2.value (#1), probe targets:[table1.value (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 250.00 ├── Exchange(Build) │ ├── output columns: [table2.value (#1)] @@ -57,6 +57,7 @@ Exchange │ ├── estimated rows: 1000.00 │ └── TableScan │ ├── table: default.default.table2 + │ ├── scan id: 1 │ ├── output columns: [value (#1)] │ ├── read rows: 1000 │ ├── read size: 2.30 KiB @@ -71,6 +72,7 @@ Exchange ├── estimated rows: 2000.00 └── TableScan ├── table: default.default.table1 + ├── scan id: 0 ├── output columns: [value (#0)] ├── read rows: 2000 ├── read size: 3.94 KiB @@ -95,7 +97,7 @@ Exchange ├── keys is null equal: [false] ├── filters: [] ├── build join filters: - │ └── filter id:1, build key:table3.value (#2), probe key:table1.value (#0), filter type:inlist,min_max + │ └── filter id:1, build key:table3.value (#2), probe targets:[table1.value (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 200.00 ├── Exchange(Build) │ ├── output columns: [table3.value (#2), table2.value (#1)] @@ -108,7 +110,7 @@ Exchange │ ├── keys is null equal: [false] │ ├── filters: [] │ ├── build join filters: - │ │ └── filter id:0, build key:table2.value (#1), probe key:table3.value (#2), filter type:inlist,min_max + │ │ └── filter id:0, build key:table2.value (#1), probe targets:[table3.value (#2)@scan2], filter type:bloom,inlist,min_max │ ├── estimated rows: 250.00 │ ├── Exchange(Build) │ │ ├── output columns: [table2.value (#1)] @@ -119,6 +121,7 @@ Exchange │ │ ├── estimated rows: 1000.00 │ │ └── TableScan │ │ ├── table: default.default.table2 + │ │ ├── scan id: 1 │ │ ├── output columns: [value (#1)] │ │ ├── read rows: 1000 │ │ ├── read size: 2.30 KiB @@ -133,6 +136,7 @@ Exchange │ ├── estimated rows: 2000.00 │ └── TableScan │ ├── table: default.default.table3 + │ ├── scan id: 2 │ ├── output columns: [value (#2)] │ ├── read rows: 2000 │ ├── read size: 3.94 KiB @@ -148,6 +152,7 @@ Exchange ├── estimated rows: 2000.00 └── TableScan ├── table: default.default.table1 + ├── scan id: 0 ├── output columns: [value (#0)] ├── read rows: 2000 ├── read size: 3.94 KiB @@ -182,6 +187,7 @@ Exchange │ ├── estimated rows: 1000.00 │ └── TableScan │ ├── table: default.default.table2 + │ ├── scan id: 1 │ ├── output columns: [value (#1)] │ ├── read rows: 1000 │ ├── read size: 2.30 KiB @@ -196,6 +202,7 @@ Exchange ├── estimated rows: 2000.00 └── TableScan ├── table: default.default.table1 + ├── scan id: 0 ├── output columns: [value (#0)] ├── read rows: 2000 ├── read size: 3.94 KiB @@ -219,7 +226,7 @@ Exchange ├── keys is null equal: [false] ├── filters: [] ├── build join filters(distributed): - │ └── filter id:0, build key:table2.value (#1), probe key:table1.value (#0), filter type:inlist,min_max + │ └── filter id:0, build key:table2.value (#1), probe targets:[table1.value (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 250.00 ├── Exchange(Build) │ ├── output columns: [table2.value (#1)] @@ -230,6 +237,7 @@ Exchange │ ├── estimated rows: 1000.00 │ └── TableScan │ ├── table: default.default.table2 + │ ├── scan id: 1 │ ├── output columns: [value (#1)] │ ├── read rows: 1000 │ ├── read size: 2.30 KiB @@ -247,6 +255,7 @@ Exchange ├── estimated rows: 2000.00 └── TableScan ├── table: default.default.table1 + ├── scan id: 0 ├── output columns: [value (#0)] ├── read rows: 2000 ├── read size: 3.94 KiB diff --git a/tests/sqllogictests/suites/mode/cluster/group_shuffle.test b/tests/sqllogictests/suites/mode/cluster/group_shuffle.test index 737ad799ac5f8..f191ff7ec6d76 100644 --- a/tests/sqllogictests/suites/mode/cluster/group_shuffle.test +++ b/tests/sqllogictests/suites/mode/cluster/group_shuffle.test @@ -21,6 +21,7 @@ Exchange ├── exchange type: Hash(numbers_mt.number (#0)) └── TableScan ├── table: default.system.numbers_mt + ├── scan id: 0 ├── output columns: [number (#0)] ├── read rows: 100000 ├── read size: 781.25 KiB @@ -53,6 +54,7 @@ Exchange ├── estimated rows: 33333.33 └── TableScan ├── table: default.system.numbers_mt + ├── scan id: 0 ├── output columns: [number (#0)] ├── read rows: 100000 ├── read size: 781.25 KiB diff --git a/tests/sqllogictests/suites/mode/cluster/lazy_read.test b/tests/sqllogictests/suites/mode/cluster/lazy_read.test index e99115ddaa48b..5e6ab6557034e 100644 --- a/tests/sqllogictests/suites/mode/cluster/lazy_read.test +++ b/tests/sqllogictests/suites/mode/cluster/lazy_read.test @@ -46,6 +46,7 @@ RowFetch ├── estimated rows: 300.00 └── TableScan ├── table: default.default.t_lazy + ├── scan id: 0 ├── output columns: [a (#0), _row_id (#7)] ├── read rows: 200 ├── read size: < 1 KiB @@ -81,6 +82,7 @@ RowFetch ├── estimated rows: 0.00 └── TableScan ├── table: default.default.t_lazy + ├── scan id: 0 ├── output columns: [a (#0), _row_id (#7)] ├── read rows: 100 ├── read size: < 1 KiB @@ -114,6 +116,7 @@ Limit ├── estimated rows: 300.00 └── TableScan ├── table: default.default.t_lazy + ├── scan id: 0 ├── output columns: [a (#0), b (#1), c (#2), d (#3), e (#6)] ├── read rows: 200 ├── read size: 1.12 KiB @@ -145,6 +148,7 @@ Limit ├── estimated rows: 0.00 └── TableScan ├── table: default.default.t_lazy + ├── scan id: 0 ├── output columns: [a (#0), b (#1), c (#2), d (#3), e (#6)] ├── read rows: 100 ├── read size: < 1 KiB @@ -172,6 +176,7 @@ Limit ├── estimated rows: 2.00 └── TableScan ├── table: default.default.t_lazy + ├── scan id: 0 ├── output columns: [a (#0), b (#1), c (#2), d (#3), e (#6)] ├── read rows: 100 ├── read size: < 1 KiB 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 420e845124972..4344f2f97db8f 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 @@ -82,6 +82,7 @@ CommitSink ├── exchange type: Hash() └── TableScan ├── table: default.default.t2 + ├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 1 ├── read size: < 1 KiB @@ -116,6 +117,7 @@ CommitSink ├── estimated rows: 15.00 ├── TableScan(Build) │ ├── table: default.default.t2 + │ ├── scan id: 0 │ ├── output columns: [a (#0)] │ ├── read rows: 1 │ ├── read size: < 1 KiB @@ -126,6 +128,7 @@ CommitSink │ └── estimated rows: 1.00 └── TableScan(Probe) ├── table: default.default.t1 + ├── scan id: 1 ├── output columns: [a (#1), _row_id (#2)] ├── read rows: 15 ├── read size: < 1 KiB @@ -155,6 +158,7 @@ CommitSink ├── estimated rows: 15.00 ├── TableScan(Build) │ ├── table: default.default.t2 + │ ├── scan id: 0 │ ├── output columns: [a (#0)] │ ├── read rows: 1 │ ├── read size: < 1 KiB @@ -165,6 +169,7 @@ CommitSink │ └── estimated rows: 1.00 └── TableScan(Probe) ├── table: default.default.t1 + ├── scan id: 1 ├── output columns: [a (#1), _row_id (#2)] ├── read rows: 15 ├── read size: < 1 KiB @@ -257,6 +262,7 @@ CommitSink │ ├── exchange type: Hash(CAST(t2.a (#0) AS Decimal(38, 5) NULL)) │ └── TableScan │ ├── table: default.system.stage + │ ├── scan id: 0 │ ├── output columns: [_$1 (#0)] │ ├── read rows: 6 │ ├── read size: < 1 KiB @@ -269,6 +275,7 @@ CommitSink ├── exchange type: Hash(CAST(t1.a (#1) AS Decimal(38, 5) NULL)) └── TableScan ├── table: default.default.t1 + ├── scan id: 1 ├── output columns: [a (#1), _row_id (#2)] ├── read rows: 2 ├── read size: < 1 KiB diff --git a/tests/sqllogictests/suites/mode/cluster/subquery.test b/tests/sqllogictests/suites/mode/cluster/subquery.test index 18d4c87f6c0ed..55cc19470cbd0 100644 --- a/tests/sqllogictests/suites/mode/cluster/subquery.test +++ b/tests/sqllogictests/suites/mode/cluster/subquery.test @@ -45,6 +45,7 @@ Exchange │ ├── estimated rows: 0.40 │ └── TableScan │ ├── table: default.d_subquery.t2 + │ ├── scan id: 2 │ ├── output columns: [a (#3)] │ ├── read rows: 2 │ ├── read size: < 1 KiB @@ -73,6 +74,7 @@ Exchange │ ├── estimated rows: 0.40 │ └── TableScan │ ├── table: default.d_subquery.t2 + │ ├── scan id: 1 │ ├── output columns: [a (#1)] │ ├── read rows: 2 │ ├── read size: < 1 KiB @@ -86,6 +88,7 @@ Exchange ├── exchange type: Hash(a (#0)) └── TableScan ├── table: default.d_subquery.t1 + ├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 3 ├── read size: < 1 KiB @@ -122,6 +125,7 @@ Exchange │ ├── exchange type: Broadcast │ └── TableScan │ ├── table: default.d_subquery.t2 + │ ├── scan id: 2 │ ├── output columns: [a (#2)] │ ├── read rows: 2 │ ├── read size: < 1 KiB @@ -146,6 +150,7 @@ Exchange │ ├── exchange type: Broadcast │ └── TableScan │ ├── table: default.d_subquery.t2 + │ ├── scan id: 1 │ ├── output columns: [a (#1)] │ ├── read rows: 2 │ ├── read size: < 1 KiB @@ -159,6 +164,7 @@ Exchange ├── exchange type: Hash(t1.a (#0)) └── TableScan ├── table: default.d_subquery.t1 + ├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 3 ├── read size: < 1 KiB diff --git a/tests/sqllogictests/suites/mode/cluster/window.test b/tests/sqllogictests/suites/mode/cluster/window.test index 3014ffd53e9fb..d0713f33caaf5 100644 --- a/tests/sqllogictests/suites/mode/cluster/window.test +++ b/tests/sqllogictests/suites/mode/cluster/window.test @@ -99,13 +99,14 @@ Exchange ├── keys is null equal: [false] ├── filters: [] ├── build join filters: - │ └── filter id:0, build key:d.department_id (#4), probe key:e.department_id (#2), filter type:inlist,min_max + │ └── filter id:0, build key:d.department_id (#4), probe targets:[e.department_id (#2)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 8.00 ├── Exchange(Build) │ ├── output columns: [d.department_id (#4), d.department_name (#5)] │ ├── exchange type: Broadcast │ └── TableScan │ ├── table: default.default.departments + │ ├── scan id: 1 │ ├── output columns: [department_id (#4), department_name (#5)] │ ├── read rows: 4 │ ├── read size: < 1 KiB @@ -116,6 +117,7 @@ Exchange │ └── estimated rows: 4.00 └── TableScan(Probe) ├── table: default.default.employees + ├── scan id: 0 ├── output columns: [name (#1), department_id (#2), salary (#3)] ├── read rows: 10 ├── read size: < 1 KiB @@ -167,6 +169,7 @@ Exchange ├── estimated rows: 50.00 └── TableScan ├── table: default.system.numbers + ├── scan id: 0 ├── output columns: [number (#0)] ├── read rows: 50 ├── read size: < 1 KiB @@ -232,6 +235,7 @@ Limit ├── exchange type: Hash(sales.customer_id (#2)) └── TableScan ├── table: default.default.sales + ├── scan id: 0 ├── output columns: [customer_id (#2), net_paid (#5)] ├── read rows: 0 ├── read size: 0 diff --git a/tests/sqllogictests/suites/mode/standalone/ee/explain_hilbert_clustering.test b/tests/sqllogictests/suites/mode/standalone/ee/explain_hilbert_clustering.test index 5655ef5a940ce..91fde062132ec 100644 --- a/tests/sqllogictests/suites/mode/standalone/ee/explain_hilbert_clustering.test +++ b/tests/sqllogictests/suites/mode/standalone/ee/explain_hilbert_clustering.test @@ -35,6 +35,7 @@ Filter ├── estimated rows: 2.00 └── TableScan ├── table: default.default.test_hilbert + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 4 ├── read size: < 1 KiB @@ -53,6 +54,7 @@ Filter ├── estimated rows: 2.00 └── TableScan ├── table: default.default.test_hilbert + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 2 ├── read size: < 1 KiB diff --git a/tests/sqllogictests/suites/mode/standalone/explain/09_0041_merge_into_issue_15669.test b/tests/sqllogictests/suites/mode/standalone/explain/09_0041_merge_into_issue_15669.test index 8bd6d027a2704..47199bdc86689 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/09_0041_merge_into_issue_15669.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/09_0041_merge_into_issue_15669.test @@ -21,6 +21,7 @@ Filter ├── estimated rows: 1.04 └── TableScan ├── table: default.test_15669.t + ├── scan id: 0 ├── output columns: [c (#0)] ├── read rows: 0 ├── read size: 0 diff --git a/tests/sqllogictests/suites/mode/standalone/explain/aggregate.test b/tests/sqllogictests/suites/mode/standalone/explain/aggregate.test index b077751f5a975..6554cac8d43fa 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/aggregate.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/aggregate.test @@ -12,6 +12,7 @@ AggregateFinal ├── estimated rows: 3.33 └── TableScan ├── table: default.system.numbers + ├── scan id: 0 ├── output columns: [number (#0)] ├── read rows: 10 ├── read size: < 1 KiB @@ -35,6 +36,7 @@ AggregateFinal ├── estimated rows: 3.33 └── TableScan ├── table: default.system.numbers + ├── scan id: 0 ├── output columns: [number (#0)] ├── read rows: 10 ├── read size: < 1 KiB @@ -67,6 +69,7 @@ EvalScalar ├── estimated rows: 10.00 └── TableScan ├── table: default.system.numbers + ├── scan id: 0 ├── output columns: [] ├── read rows: 10 ├── read size: < 1 KiB @@ -94,6 +97,7 @@ EvalScalar ├── estimated rows: 3.33 └── TableScan ├── table: default.system.numbers + ├── scan id: 0 ├── output columns: [number (#0)] ├── read rows: 10 ├── read size: < 1 KiB @@ -121,6 +125,7 @@ EvalScalar ├── estimated rows: 1.00 └── TableScan ├── table: default.system.columns + ├── scan id: 0 ├── output columns: [name (#0), type (#3)] ├── read rows: 0 ├── read size: 0 @@ -156,6 +161,7 @@ AggregateFinal ├── estimated rows: 0.00 └── TableScan ├── table: default.default.explain_agg_t1 + ├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 0 ├── read size: 0 @@ -182,6 +188,7 @@ AggregateFinal ├── estimated rows: 0.00 └── TableScan ├── table: default.default.explain_agg_t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -216,6 +223,7 @@ EvalScalar ├── estimated rows: 0.00 └── TableScan ├── table: default.default.explain_agg_t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -246,6 +254,7 @@ Sort(Single) ├── estimated rows: 1.00 └── TableScan ├── table: default.default.explain_agg_t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -277,6 +286,7 @@ Sort(Single) ├── estimated rows: 1.00 └── TableScan ├── table: default.default.explain_agg_t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -307,6 +317,7 @@ Sort(Single) ├── estimated rows: 1.00 └── TableScan ├── table: default.default.explain_agg_t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -337,6 +348,7 @@ Sort(Single) ├── estimated rows: 1.00 └── TableScan ├── table: default.default.explain_agg_t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -367,6 +379,7 @@ Sort(Single) ├── estimated rows: 1.00 └── TableScan ├── table: default.default.explain_agg_t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -410,6 +423,7 @@ AggregateFinal ├── estimated rows: 1000.00 ├── TableScan(Left) │ ├── table: default.default.t1 + │ ├── scan id: 0 │ ├── output columns: [a (#0)] │ ├── read rows: 10 │ ├── read size: < 1 KiB @@ -420,6 +434,7 @@ AggregateFinal │ └── estimated rows: 10.00 └── TableScan(Right) ├── table: default.default.t2 + ├── scan id: 1 ├── output columns: [a (#1)] ├── read rows: 100 ├── read size: < 1 KiB @@ -456,6 +471,7 @@ EvalScalar ├── estimated rows: 1.00 └── TableScan ├── table: default.default.t + ├── scan id: 0 ├── output columns: [referer (#0), isrefresh (#1)] ├── read rows: 0 ├── read size: 0 @@ -488,6 +504,7 @@ Limit ├── rank limit: 10 └── TableScan ├── table: default.default.t + ├── scan id: 0 ├── output columns: [referer (#0), isrefresh (#1)] ├── read rows: 0 ├── read size: 0 @@ -520,6 +537,7 @@ Limit ├── rank limit: 13 └── TableScan ├── table: default.default.t + ├── scan id: 0 ├── output columns: [referer (#0), isrefresh (#1)] ├── read rows: 0 ├── read size: 0 diff --git a/tests/sqllogictests/suites/mode/standalone/explain/bloom_filter.test b/tests/sqllogictests/suites/mode/standalone/explain/bloom_filter.test index 68f426d8f3548..c9d823849bd6c 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/bloom_filter.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/bloom_filter.test @@ -27,6 +27,7 @@ EvalScalar ├── estimated rows: 1.50 └── TableScan ├── table: default.default.bloom_test_t + ├── scan id: 0 ├── output columns: [c1 (#0)] ├── read rows: 3 ├── read size: < 1 KiB @@ -113,6 +114,7 @@ EvalScalar ├── estimated rows: 0.00 └── TableScan ├── table: default.default.bloom_test_t + ├── scan id: 0 ├── output columns: [c2 (#1)] ├── read rows: 0 ├── read size: 0 @@ -135,6 +137,7 @@ EvalScalar ├── estimated rows: 4.00 └── TableScan ├── table: default.default.bloom_test_t + ├── scan id: 0 ├── output columns: [c2 (#1)] ├── read rows: 0 ├── read size: 0 @@ -157,6 +160,7 @@ EvalScalar ├── estimated rows: 4.00 └── TableScan ├── table: default.default.bloom_test_t + ├── scan id: 0 ├── output columns: [c3 (#2)] ├── read rows: 0 ├── read size: 0 @@ -199,6 +203,7 @@ EvalScalar ├── estimated rows: 1.50 └── TableScan ├── table: default.default.bloom_test_alter_t1 + ├── scan id: 0 ├── output columns: [c1 (#0)] ├── read rows: 6 ├── read size: < 1 KiB @@ -233,6 +238,7 @@ EvalScalar ├── estimated rows: 1.50 └── TableScan ├── table: default.default.bloom_test_alter_t2 + ├── scan id: 0 ├── output columns: [c1 (#0)] ├── read rows: 6 ├── read size: < 1 KiB @@ -264,6 +270,7 @@ EvalScalar ├── estimated rows: 1.80 └── TableScan ├── table: default.default.bloom_test_alter_t2 + ├── scan id: 0 ├── output columns: [c1 (#0)] ├── read rows: 6 ├── read size: < 1 KiB @@ -290,6 +297,7 @@ EvalScalar ├── estimated rows: 1.80 └── TableScan ├── table: default.default.bloom_test_alter_t2 + ├── scan id: 0 ├── output columns: [c1 (#0)] ├── read rows: 9 ├── read size: < 1 KiB @@ -327,6 +335,7 @@ EvalScalar ├── estimated rows: 1.50 └── TableScan ├── table: default.default.bloom_test_nullable_t + ├── scan id: 0 ├── output columns: [c1 (#0), c2 (#1)] ├── read rows: 3 ├── read size: < 1 KiB @@ -349,6 +358,7 @@ EvalScalar ├── estimated rows: 1.20 └── TableScan ├── table: default.default.bloom_test_nullable_t + ├── scan id: 0 ├── output columns: [c2 (#1)] ├── read rows: 0 ├── read size: 0 @@ -379,6 +389,7 @@ Filter ├── estimated rows: 1.60 └── TableScan ├── table: default.default.bloom_test_nullable_t2 + ├── scan id: 0 ├── output columns: [c0 (#0), c1 (#1), c2 (#2)] ├── read rows: 2 ├── read size: < 1 KiB @@ -397,6 +408,7 @@ Filter ├── estimated rows: 2.00 └── TableScan ├── table: default.default.bloom_test_nullable_t2 + ├── scan id: 0 ├── output columns: [c0 (#0), c1 (#1), c2 (#2)] ├── read rows: 2 ├── read size: < 1 KiB diff --git a/tests/sqllogictests/suites/mode/standalone/explain/clustering.test b/tests/sqllogictests/suites/mode/standalone/explain/clustering.test index 9a927d09fdf93..c78842e2f5314 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/clustering.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/clustering.test @@ -27,6 +27,7 @@ Filter ├── estimated rows: 2.00 └── TableScan ├── table: default.default.test_linear + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 2 ├── read size: < 1 KiB @@ -45,6 +46,7 @@ Filter ├── estimated rows: 2.00 └── TableScan ├── table: default.default.test_linear + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 4 ├── read size: < 1 KiB diff --git a/tests/sqllogictests/suites/mode/standalone/explain/cte_filter_pushdown.test b/tests/sqllogictests/suites/mode/standalone/explain/cte_filter_pushdown.test index c22ceeecb8497..0413c8732468b 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/cte_filter_pushdown.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/cte_filter_pushdown.test @@ -21,6 +21,7 @@ Filter ├── estimated rows: 1.67 └── TableScan ├── table: default.default.cte1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 5 ├── read size: < 1 KiB @@ -45,6 +46,7 @@ Filter ├── estimated rows: 1.00 └── TableScan ├── table: default.default.cte1 + ├── scan id: 0 ├── output columns: [a (#0), cnt (#1), sum_b (#2)] ├── read rows: 3 ├── read size: < 1 KiB @@ -69,6 +71,7 @@ Filter ├── estimated rows: 3.00 └── TableScan ├── table: default.default.cte1 + ├── scan id: 0 ├── output columns: [a (#0), cnt (#1), sum_b (#2)] ├── read rows: 3 ├── read size: < 1 KiB @@ -93,6 +96,7 @@ Filter ├── estimated rows: 2.00 └── TableScan ├── table: default.default.cte1 + ├── scan id: 0 ├── output columns: [a (#0), cnt (#1), sum_b (#2)] ├── read rows: 3 ├── read size: < 1 KiB @@ -123,6 +127,7 @@ UnionAll │ ├── estimated rows: 1.00 │ └── TableScan │ ├── table: default.default.cte1 +│ ├── scan id: 0 │ ├── output columns: [a (#0), cnt (#1)] │ ├── read rows: 3 │ ├── read size: < 1 KiB @@ -137,6 +142,7 @@ UnionAll ├── estimated rows: 1.00 └── TableScan ├── table: default.default.cte1 + ├── scan id: 1 ├── output columns: [a (#2), cnt (#3)] ├── read rows: 3 ├── read size: < 1 KiB @@ -172,6 +178,7 @@ UnionAll │ │ ├── estimated rows: 1.00 │ │ └── TableScan │ │ ├── table: default.default.cte1 +│ │ ├── scan id: 0 │ │ ├── output columns: [a (#0), cnt (#1)] │ │ ├── read rows: 3 │ │ ├── read size: < 1 KiB @@ -186,6 +193,7 @@ UnionAll │ ├── estimated rows: 1.00 │ └── TableScan │ ├── table: default.default.cte1 +│ ├── scan id: 1 │ ├── output columns: [a (#2), cnt (#3)] │ ├── read rows: 3 │ ├── read size: < 1 KiB @@ -200,6 +208,7 @@ UnionAll ├── estimated rows: 1.00 └── TableScan ├── table: default.default.cte1 + ├── scan id: 2 ├── output columns: [a (#6), cnt (#7)] ├── read rows: 3 ├── read size: < 1 KiB @@ -222,6 +231,7 @@ Filter ├── estimated rows: 1.67 └── TableScan ├── table: default.default.cte1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 5 ├── read size: < 1 KiB @@ -247,6 +257,7 @@ Filter ├── estimated rows: 2.00 └── TableScan ├── table: default.default.cte1 + ├── scan id: 0 ├── output columns: [a (#0), cnt (#1), sum_b (#2)] ├── read rows: 3 ├── read size: < 1 KiB @@ -271,6 +282,7 @@ Filter ├── estimated rows: 1.67 └── TableScan ├── table: default.default.cte1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1), cnt (#2)] ├── read rows: 5 ├── read size: < 1 KiB @@ -298,6 +310,7 @@ Filter ├── estimated rows: 1.00 └── TableScan ├── table: default.default.cte1 + ├── scan id: 0 ├── output columns: [a (#0), cnt (#1), sum_b (#2)] ├── read rows: 3 ├── read size: < 1 KiB @@ -322,6 +335,7 @@ Filter ├── estimated rows: 1.67 └── TableScan ├── table: default.default.cte1 + ├── scan id: 0 ├── output columns: [a (#0), cnt (#1)] ├── read rows: 3 ├── read size: < 1 KiB @@ -346,6 +360,7 @@ Filter ├── estimated rows: 2.78 └── TableScan ├── table: default.default.cte1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1), cnt (#2)] ├── read rows: 5 ├── read size: < 1 KiB @@ -370,6 +385,7 @@ Filter ├── estimated rows: 2.00 └── TableScan ├── table: default.default.cte1 + ├── scan id: 0 ├── output columns: [a (#0), cnt (#1)] ├── read rows: 3 ├── read size: < 1 KiB @@ -393,6 +409,7 @@ Filter ├── estimated rows: 1.67 └── TableScan ├── table: default.default.cte1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1), rn (#2)] ├── read rows: 5 ├── read size: < 1 KiB diff --git a/tests/sqllogictests/suites/mode/standalone/explain/delete.test b/tests/sqllogictests/suites/mode/standalone/explain/delete.test index a5b809e897503..4bc88ec033902 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/delete.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/delete.test @@ -47,6 +47,7 @@ CommitSink ├── estimated rows: 2.00 ├── TableScan(Build) │ ├── table: default.default.t2 + │ ├── scan id: 1 │ ├── output columns: [a (#2)] │ ├── read rows: 2 │ ├── read size: < 1 KiB @@ -57,6 +58,7 @@ CommitSink │ └── estimated rows: 2.00 └── TableScan(Probe) ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), _row_id (#4)] ├── read rows: 4 ├── read size: < 1 KiB @@ -98,6 +100,7 @@ CommitSink ├── estimated rows: 2.00 ├── TableScan(Build) │ ├── table: default.default.t2 + │ ├── scan id: 1 │ ├── output columns: [a (#2)] │ ├── read rows: 2 │ ├── read size: < 1 KiB @@ -112,6 +115,7 @@ CommitSink ├── estimated rows: 3.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1), _row_id (#4)] ├── read rows: 4 ├── read size: < 1 KiB diff --git a/tests/sqllogictests/suites/mode/standalone/explain/eliminate_outer_join.test b/tests/sqllogictests/suites/mode/standalone/explain/eliminate_outer_join.test index bd2710caa97b8..88d7eea30809c 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/eliminate_outer_join.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/eliminate_outer_join.test @@ -23,6 +23,7 @@ HashJoin ├── estimated rows: 10.00 ├── TableScan(Build) │ ├── table: default.eliminate_outer_join.t +│ ├── scan id: 1 │ ├── output columns: [a (#1)] │ ├── read rows: 10 │ ├── read size: < 1 KiB @@ -33,6 +34,7 @@ HashJoin │ └── estimated rows: 10.00 └── TableScan(Probe) ├── table: default.eliminate_outer_join.t + ├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 10 ├── read size: < 1 KiB @@ -55,6 +57,7 @@ HashJoin ├── estimated rows: 10.00 ├── TableScan(Build) │ ├── table: default.eliminate_outer_join.t +│ ├── scan id: 0 │ ├── output columns: [a (#0)] │ ├── read rows: 10 │ ├── read size: < 1 KiB @@ -65,6 +68,7 @@ HashJoin │ └── estimated rows: 10.00 └── TableScan(Probe) ├── table: default.eliminate_outer_join.t + ├── scan id: 1 ├── output columns: [a (#1)] ├── read rows: 10 ├── read size: < 1 KiB @@ -87,6 +91,7 @@ HashJoin ├── estimated rows: 10.00 ├── TableScan(Build) │ ├── table: default.eliminate_outer_join.t +│ ├── scan id: 1 │ ├── output columns: [a (#1)] │ ├── read rows: 10 │ ├── read size: < 1 KiB @@ -97,6 +102,7 @@ HashJoin │ └── estimated rows: 10.00 └── TableScan(Probe) ├── table: default.eliminate_outer_join.t + ├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 10 ├── read size: < 1 KiB @@ -117,7 +123,7 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:0, build key:t1.a (#1), probe key:t.a (#0), filter type:inlist,min_max +│ └── filter id:0, build key:t1.a (#1), probe targets:[t.a (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 10.00 ├── Filter(Build) │ ├── output columns: [t1.a (#1)] @@ -125,6 +131,7 @@ HashJoin │ ├── estimated rows: 10.00 │ └── TableScan │ ├── table: default.eliminate_outer_join.t +│ ├── scan id: 1 │ ├── output columns: [a (#1)] │ ├── read rows: 10 │ ├── read size: < 1 KiB @@ -139,6 +146,7 @@ HashJoin ├── estimated rows: 10.00 └── TableScan ├── table: default.eliminate_outer_join.t + ├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 10 ├── read size: < 1 KiB @@ -160,7 +168,7 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:0, build key:t1.a (#1), probe key:t.a (#0), filter type:inlist,min_max +│ └── filter id:0, build key:t1.a (#1), probe targets:[t.a (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 10.00 ├── Filter(Build) │ ├── output columns: [t1.a (#1)] @@ -168,6 +176,7 @@ HashJoin │ ├── estimated rows: 10.00 │ └── TableScan │ ├── table: default.eliminate_outer_join.t +│ ├── scan id: 1 │ ├── output columns: [a (#1)] │ ├── read rows: 10 │ ├── read size: < 1 KiB @@ -182,6 +191,7 @@ HashJoin ├── estimated rows: 10.00 └── TableScan ├── table: default.eliminate_outer_join.t + ├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 10 ├── read size: < 1 KiB @@ -209,6 +219,7 @@ HashJoin │ ├── estimated rows: 10.00 │ └── TableScan │ ├── table: default.eliminate_outer_join.t +│ ├── scan id: 1 │ ├── output columns: [a (#1)] │ ├── read rows: 10 │ ├── read size: < 1 KiB @@ -223,6 +234,7 @@ HashJoin ├── estimated rows: 10.00 └── TableScan ├── table: default.eliminate_outer_join.t + ├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 10 ├── read size: < 1 KiB @@ -249,6 +261,7 @@ HashJoin │ ├── estimated rows: 10.00 │ └── TableScan │ ├── table: default.eliminate_outer_join.t +│ ├── scan id: 0 │ ├── output columns: [a (#0)] │ ├── read rows: 10 │ ├── read size: < 1 KiB @@ -263,6 +276,7 @@ HashJoin ├── estimated rows: 10.00 └── TableScan ├── table: default.eliminate_outer_join.t + ├── scan id: 1 ├── output columns: [a (#1)] ├── read rows: 10 ├── read size: < 1 KiB @@ -283,7 +297,7 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:0, build key:t1.a (#1), probe key:t.a (#0), filter type:inlist,min_max +│ └── filter id:0, build key:t1.a (#1), probe targets:[t.a (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 10.00 ├── Filter(Build) │ ├── output columns: [t1.a (#1)] @@ -291,6 +305,7 @@ HashJoin │ ├── estimated rows: 10.00 │ └── TableScan │ ├── table: default.eliminate_outer_join.t +│ ├── scan id: 1 │ ├── output columns: [a (#1)] │ ├── read rows: 10 │ ├── read size: < 1 KiB @@ -305,6 +320,7 @@ HashJoin ├── estimated rows: 10.00 └── TableScan ├── table: default.eliminate_outer_join.t + ├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 10 ├── read size: < 1 KiB @@ -326,7 +342,7 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:0, build key:t1.a (#1), probe key:t.a (#0), filter type:inlist,min_max +│ └── filter id:0, build key:t1.a (#1), probe targets:[t.a (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 1.00 ├── Filter(Build) │ ├── output columns: [t1.a (#1)] @@ -334,6 +350,7 @@ HashJoin │ ├── estimated rows: 1.00 │ └── TableScan │ ├── table: default.eliminate_outer_join.t +│ ├── scan id: 1 │ ├── output columns: [a (#1)] │ ├── read rows: 10 │ ├── read size: < 1 KiB @@ -348,6 +365,7 @@ HashJoin ├── estimated rows: 1.00 └── TableScan ├── table: default.eliminate_outer_join.t + ├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 10 ├── read size: < 1 KiB @@ -369,7 +387,7 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:0, build key:t1.a (#1), probe key:t.a (#0), filter type:inlist,min_max +│ └── filter id:0, build key:t1.a (#1), probe targets:[t.a (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 8.00 ├── Filter(Build) │ ├── output columns: [t1.a (#1)] @@ -377,6 +395,7 @@ HashJoin │ ├── estimated rows: 8.00 │ └── TableScan │ ├── table: default.eliminate_outer_join.t +│ ├── scan id: 1 │ ├── output columns: [a (#1)] │ ├── read rows: 10 │ ├── read size: < 1 KiB @@ -391,6 +410,7 @@ HashJoin ├── estimated rows: 8.00 └── TableScan ├── table: default.eliminate_outer_join.t + ├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 10 ├── read size: < 1 KiB @@ -412,7 +432,7 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:0, build key:t1.a (#1), probe key:t.a (#0), filter type:inlist,min_max +│ └── filter id:0, build key:t1.a (#1), probe targets:[t.a (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 1.00 ├── Filter(Build) │ ├── output columns: [t1.a (#1)] @@ -420,6 +440,7 @@ HashJoin │ ├── estimated rows: 1.00 │ └── TableScan │ ├── table: default.eliminate_outer_join.t +│ ├── scan id: 1 │ ├── output columns: [a (#1)] │ ├── read rows: 10 │ ├── read size: < 1 KiB @@ -434,6 +455,7 @@ HashJoin ├── estimated rows: 1.00 └── TableScan ├── table: default.eliminate_outer_join.t + ├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 10 ├── read size: < 1 KiB @@ -455,7 +477,7 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:0, build key:t1.a (#1), probe key:t.a (#0), filter type:inlist,min_max +│ └── filter id:0, build key:t1.a (#1), probe targets:[t.a (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 9.00 ├── Filter(Build) │ ├── output columns: [t1.a (#1)] @@ -463,6 +485,7 @@ HashJoin │ ├── estimated rows: 9.00 │ └── TableScan │ ├── table: default.eliminate_outer_join.t +│ ├── scan id: 1 │ ├── output columns: [a (#1)] │ ├── read rows: 10 │ ├── read size: < 1 KiB @@ -477,6 +500,7 @@ HashJoin ├── estimated rows: 9.00 └── TableScan ├── table: default.eliminate_outer_join.t + ├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 10 ├── read size: < 1 KiB @@ -498,7 +522,7 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:0, build key:t1.a (#1), probe key:t.a (#0), filter type:inlist,min_max +│ └── filter id:0, build key:t1.a (#1), probe targets:[t.a (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 9.00 ├── Filter(Build) │ ├── output columns: [t1.a (#1)] @@ -506,6 +530,7 @@ HashJoin │ ├── estimated rows: 9.00 │ └── TableScan │ ├── table: default.eliminate_outer_join.t +│ ├── scan id: 1 │ ├── output columns: [a (#1)] │ ├── read rows: 10 │ ├── read size: < 1 KiB @@ -520,6 +545,7 @@ HashJoin ├── estimated rows: 9.00 └── TableScan ├── table: default.eliminate_outer_join.t + ├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 10 ├── read size: < 1 KiB @@ -541,7 +567,7 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:0, build key:t1.a (#1), probe key:t.a (#0), filter type:inlist,min_max +│ └── filter id:0, build key:t1.a (#1), probe targets:[t.a (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 2.00 ├── Filter(Build) │ ├── output columns: [t1.a (#1)] @@ -549,6 +575,7 @@ HashJoin │ ├── estimated rows: 2.00 │ └── TableScan │ ├── table: default.eliminate_outer_join.t +│ ├── scan id: 1 │ ├── output columns: [a (#1)] │ ├── read rows: 10 │ ├── read size: < 1 KiB @@ -563,6 +590,7 @@ HashJoin ├── estimated rows: 2.00 └── TableScan ├── table: default.eliminate_outer_join.t + ├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 10 ├── read size: < 1 KiB @@ -591,6 +619,7 @@ Filter ├── estimated rows: 10.00 ├── TableScan(Build) │ ├── table: default.eliminate_outer_join.t + │ ├── scan id: 1 │ ├── output columns: [a (#1)] │ ├── read rows: 10 │ ├── read size: < 1 KiB @@ -601,6 +630,7 @@ Filter │ └── estimated rows: 10.00 └── TableScan(Probe) ├── table: default.eliminate_outer_join.t + ├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 10 ├── read size: < 1 KiB @@ -625,7 +655,7 @@ Filter ├── keys is null equal: [false] ├── filters: [] ├── build join filters: - │ └── filter id:0, build key:t1.a (#1), probe key:t.a (#0), filter type:inlist,min_max + │ └── filter id:0, build key:t1.a (#1), probe targets:[t.a (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 8.40 ├── Filter(Build) │ ├── output columns: [t1.a (#1)] @@ -633,6 +663,7 @@ Filter │ ├── estimated rows: 8.40 │ └── TableScan │ ├── table: default.eliminate_outer_join.t + │ ├── scan id: 1 │ ├── output columns: [a (#1)] │ ├── read rows: 10 │ ├── read size: < 1 KiB @@ -647,6 +678,7 @@ Filter ├── estimated rows: 8.40 └── TableScan ├── table: default.eliminate_outer_join.t + ├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 10 ├── read size: < 1 KiB @@ -674,6 +706,7 @@ Filter ├── estimated rows: 10.00 ├── TableScan(Build) │ ├── table: default.eliminate_outer_join.t + │ ├── scan id: 1 │ ├── output columns: [a (#1)] │ ├── read rows: 10 │ ├── read size: < 1 KiB @@ -684,6 +717,7 @@ Filter │ └── estimated rows: 10.00 └── TableScan(Probe) ├── table: default.eliminate_outer_join.t + ├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 10 ├── read size: < 1 KiB @@ -707,7 +741,7 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:0, build key:t1.a (#1), probe key:t.a (#0), filter type:inlist +│ └── filter id:0, build key:t1.a (#1), probe targets:[t.a (#0)@scan0], filter type:inlist ├── estimated rows: 0.00 ├── Filter(Build) │ ├── output columns: [t1.a (#1)] @@ -715,6 +749,7 @@ HashJoin │ ├── estimated rows: 0.00 │ └── TableScan │ ├── table: default.eliminate_outer_join.time_table +│ ├── scan id: 1 │ ├── output columns: [a (#1)] │ ├── read rows: 0 │ ├── read size: 0 @@ -728,6 +763,7 @@ HashJoin ├── estimated rows: 0.00 └── TableScan ├── table: default.eliminate_outer_join.time_table + ├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 0 ├── read size: 0 diff --git a/tests/sqllogictests/suites/mode/standalone/explain/eliminate_sort.test b/tests/sqllogictests/suites/mode/standalone/explain/eliminate_sort.test index d9f6b0b547fb9..433f60250a98c 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/eliminate_sort.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/eliminate_sort.test @@ -7,6 +7,7 @@ Sort(Single) ├── estimated rows: 10.00 └── TableScan ├── table: default.system.numbers + ├── scan id: 0 ├── output columns: [number (#0)] ├── read rows: 10 ├── read size: < 1 KiB @@ -24,6 +25,7 @@ Sort(Single) ├── estimated rows: 10.00 └── TableScan ├── table: default.system.numbers + ├── scan id: 0 ├── output columns: [number (#0)] ├── read rows: 10 ├── read size: < 1 KiB @@ -45,6 +47,7 @@ Sort(Single) ├── estimated rows: 10.00 └── TableScan ├── table: default.system.numbers + ├── scan id: 0 ├── output columns: [number (#0)] ├── read rows: 10 ├── read size: < 1 KiB @@ -66,6 +69,7 @@ Sort(Single) ├── estimated rows: 10.00 └── TableScan ├── table: default.system.numbers + ├── scan id: 0 ├── output columns: [number (#0)] ├── read rows: 10 ├── read size: < 1 KiB @@ -91,6 +95,7 @@ Sort(Single) ├── estimated rows: 10.00 └── TableScan ├── table: default.system.numbers + ├── scan id: 0 ├── output columns: [number (#0)] ├── read rows: 10 ├── read size: < 1 KiB diff --git a/tests/sqllogictests/suites/mode/standalone/explain/explain_analyze.test b/tests/sqllogictests/suites/mode/standalone/explain/explain_analyze.test index 63556be4ccbb7..419f3dd95afa0 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/explain_analyze.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/explain_analyze.test @@ -39,6 +39,7 @@ EvalScalar ├── output bytes: 7.63 MiB ├── bytes scanned: 7.63 MiB ├── table: default.system.numbers + ├── scan id: 0 ├── output columns: [number (#0)] ├── read rows: 1000000 ├── read size: 7.63 MiB @@ -97,6 +98,7 @@ EXPLAIN SELECT * FROM article; ---- TableScan ├── table: default.default.article +├── scan id: 0 ├── output columns: [article_id (#0), author_id (#1), viewer_id (#2), view_date (#3)] ├── read rows: 6 ├── read size: < 1 KiB @@ -112,6 +114,7 @@ EXPLAIN ANALYZE SELECT * FROM article; ---- TableScan ├── table: default.default.article +├── scan id: 0 ├── output columns: [article_id (#0), author_id (#1), viewer_id (#2), view_date (#3)] ├── read rows: 6 ├── read size: < 1 KiB @@ -136,6 +139,7 @@ Filter ├── output bytes: 80.00 B ├── bytes scanned: 80.00 B ├── table: default.default.article + ├── scan id: 0 ├── output columns: [article_id (#0), author_id (#1), viewer_id (#2), view_date (#3)] ├── read rows: 4 ├── read size: < 1 KiB @@ -156,7 +160,7 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:0, build key:author.id (#4), probe key:article.author_id (#1), filter type:inlist,min_max +│ └── filter id:0, build key:author.id (#4), probe targets:[article.author_id (#1)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 2.50 ├── Filter(Build) │ ├── cpu time: @@ -172,6 +176,7 @@ HashJoin │ ├── output bytes: 31.00 B │ ├── bytes scanned: 31.00 B │ ├── table: default.default.author +│ ├── scan id: 1 │ ├── output columns: [id (#4), name (#5)] │ ├── read rows: 1 │ ├── read size: < 1 KiB @@ -183,12 +188,11 @@ HashJoin └── TableScan(Probe) ├── cpu time: ├── wait time: - ├── output rows: 1 - ├── output bytes: 20.00 B - ├── bytes scanned: 20.00 B - ├── parts pruned by runtime filter: 5 - ├── runtime filter inlist/min-max time: + ├── output rows: 6 + ├── output bytes: 120.00 B + ├── bytes scanned: 120.00 B ├── table: default.default.article + ├── scan id: 0 ├── output columns: [article_id (#0), author_id (#1), viewer_id (#2), view_date (#3)] ├── read rows: 6 ├── read size: < 1 KiB diff --git a/tests/sqllogictests/suites/mode/standalone/explain/explain_ddl.test b/tests/sqllogictests/suites/mode/standalone/explain/explain_ddl.test index 39985aedee83c..23c24767d5a52 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/explain_ddl.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/explain_ddl.test @@ -9,6 +9,7 @@ Filter ├── estimated rows: 5.00 └── TableScan ├── table: default.system.numbers + ├── scan id: 0 ├── output columns: [number (#0)] ├── read rows: 10 ├── read size: < 1 KiB diff --git a/tests/sqllogictests/suites/mode/standalone/explain/explain_deduplicate_sort.test b/tests/sqllogictests/suites/mode/standalone/explain/explain_deduplicate_sort.test index 547e485a79c7f..3f2244100b3eb 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/explain_deduplicate_sort.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/explain_deduplicate_sort.test @@ -12,6 +12,7 @@ Sort(Single) ├── estimated rows: 0.00 └── TableScan ├── table: default.default.emp + ├── scan id: 0 ├── output columns: [deptno (#0), job (#1), sal (#2)] ├── read rows: 0 ├── read size: 0 diff --git a/tests/sqllogictests/suites/mode/standalone/explain/explain_grouping_sets.test b/tests/sqllogictests/suites/mode/standalone/explain/explain_grouping_sets.test index 01532c673564e..263e8b6021f59 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/explain_grouping_sets.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/explain_grouping_sets.test @@ -24,6 +24,7 @@ EvalScalar ├── estimated rows: 1.00 └── TableScan ├── table: default.system.numbers + ├── scan id: 0 ├── output columns: [number (#0)] ├── read rows: 1 ├── read size: < 1 KiB @@ -58,6 +59,7 @@ EvalScalar ├── estimated rows: 1.00 └── TableScan ├── table: default.system.numbers + ├── scan id: 0 ├── output columns: [number (#0)] ├── read rows: 1 ├── read size: < 1 KiB @@ -81,6 +83,7 @@ Sequence │ ├── estimated rows: 100.00 │ └── TableScan │ ├── table: default.system.numbers +│ ├── scan id: 0 │ ├── output columns: [number (#0)] │ ├── read rows: 100 │ ├── read size: < 1 KiB @@ -267,6 +270,7 @@ Sequence │ ├── estimated rows: 100.00 │ └── TableScan │ ├── table: default.system.numbers +│ ├── scan id: 0 │ ├── output columns: [number (#0)] │ ├── read rows: 100 │ ├── read size: < 1 KiB diff --git a/tests/sqllogictests/suites/mode/standalone/explain/explain_like.test b/tests/sqllogictests/suites/mode/standalone/explain/explain_like.test index 6f638534c5404..719c5263c4bad 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/explain_like.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/explain_like.test @@ -23,6 +23,7 @@ Filter ├── estimated rows: 5.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [s (#0)] ├── read rows: 5 ├── read size: < 1 KiB @@ -45,6 +46,7 @@ Sort(Single) ├── estimated rows: 1.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [s (#0)] ├── read rows: 5 ├── read size: < 1 KiB @@ -72,6 +74,7 @@ Sort(Single) ├── estimated rows: 1.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [s (#0)] ├── read rows: 5 ├── read size: < 1 KiB @@ -99,6 +102,7 @@ Filter ├── estimated rows: 1.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [s (#0)] ├── read rows: 5 ├── read size: < 1 KiB @@ -125,6 +129,7 @@ Filter ├── estimated rows: 1.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [s (#0)] ├── read rows: 5 ├── read size: < 1 KiB @@ -148,6 +153,7 @@ Filter ├── estimated rows: 2.50 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [s (#0)] ├── read rows: 5 ├── read size: < 1 KiB @@ -171,6 +177,7 @@ Filter ├── estimated rows: 1.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [s (#0)] ├── read rows: 5 ├── read size: < 1 KiB @@ -192,6 +199,7 @@ explain select c from t1 where c like '%' ---- TableScan ├── table: default.default.t1 +├── scan id: 0 ├── output columns: [c (#0)] ├── read rows: 2 ├── read size: < 1 KiB diff --git a/tests/sqllogictests/suites/mode/standalone/explain/explain_pipeline.test b/tests/sqllogictests/suites/mode/standalone/explain/explain_pipeline.test index d8ed52cf50439..678d1bc144980 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/explain_pipeline.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/explain_pipeline.test @@ -17,12 +17,14 @@ explain pipeline select a from t1 ignore_result ---- digraph { 0 [ label = "BlockPartitionSource" ] - 1 [ label = "SyncReadParquetDataTransform" ] - 2 [ label = "DeserializeDataTransform" ] - 3 [ label = "EmptySink" ] + 1 [ label = "TransformRuntimeFilterWait" ] + 2 [ label = "SyncReadParquetDataTransform" ] + 3 [ label = "DeserializeDataTransform" ] + 4 [ label = "EmptySink" ] 0 -> 1 [ label = "" ] 1 -> 2 [ label = "" ] 2 -> 3 [ label = "" ] + 3 -> 4 [ label = "" ] } statement ok diff --git a/tests/sqllogictests/suites/mode/standalone/explain/explain_substr.test b/tests/sqllogictests/suites/mode/standalone/explain/explain_substr.test index 103d21b767e5c..05c68ffb1b69e 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/explain_substr.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/explain_substr.test @@ -17,6 +17,7 @@ Filter ├── estimated rows: 0.80 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [c (#0)] ├── read rows: 2 ├── read size: < 1 KiB @@ -35,6 +36,7 @@ Filter ├── estimated rows: 0.80 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [c (#0)] ├── read rows: 4 ├── read size: < 1 KiB @@ -54,6 +56,7 @@ Filter ├── estimated rows: 0.80 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [c (#0)] ├── read rows: 2 ├── read size: < 1 KiB diff --git a/tests/sqllogictests/suites/mode/standalone/explain/explain_table_function_obfuscate.test b/tests/sqllogictests/suites/mode/standalone/explain/explain_table_function_obfuscate.test index a6e6458a2236a..fc36bdc4bb4e9 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/explain_table_function_obfuscate.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/explain_table_function_obfuscate.test @@ -18,6 +18,7 @@ EvalScalar ├── estimated rows: 0.00 ├── TableScan(Build) │ ├── table: default.default.t1 + │ ├── scan id: 0 │ ├── output columns: [a (#0), b (#1), c (#2), d (#3), e (#4), f (#5)] │ ├── read rows: 0 │ ├── read size: 0 @@ -36,6 +37,7 @@ EvalScalar ├── estimated rows: 1.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 1 ├── output columns: [a (#6), b (#7)] ├── read rows: 0 ├── read size: 0 diff --git a/tests/sqllogictests/suites/mode/standalone/explain/expression_scan.test b/tests/sqllogictests/suites/mode/standalone/explain/expression_scan.test index 0ea5a57905978..74b26dcb55dd0 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/expression_scan.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/expression_scan.test @@ -31,6 +31,7 @@ HashJoin ├── estimated rows: 3.00 ├── TableScan(Build) │ ├── table: default.default.t1 +│ ├── scan id: 0 │ ├── output columns: [a (#0), b (#1)] │ ├── read rows: 3 │ ├── read size: < 1 KiB @@ -71,6 +72,7 @@ HashJoin ├── estimated rows: 6.00 ├── TableScan(Build) │ ├── table: default.default.t1 +│ ├── scan id: 0 │ ├── output columns: [a (#0), b (#1), c (#2)] │ ├── read rows: 3 │ ├── read size: < 1 KiB @@ -112,6 +114,7 @@ HashJoin ├── estimated rows: 6.00 ├── TableScan(Build) │ ├── table: default.default.t1 +│ ├── scan id: 0 │ ├── output columns: [a (#0), b (#1), c (#2)] │ ├── read rows: 3 │ ├── read size: < 1 KiB @@ -167,6 +170,7 @@ HashJoin │ ├── estimated rows: 6.00 │ ├── TableScan(Build) │ │ ├── table: default.default.t1 +│ │ ├── scan id: 0 │ │ ├── output columns: [a (#0), b (#1), c (#2)] │ │ ├── read rows: 3 │ │ ├── read size: < 1 KiB @@ -239,6 +243,7 @@ HashJoin │ ├── estimated rows: 6.00 │ ├── TableScan(Build) │ │ ├── table: default.default.t1 +│ │ ├── scan id: 0 │ │ ├── output columns: [a (#0), b (#1), c (#2)] │ │ ├── read rows: 3 │ │ ├── read size: < 1 KiB diff --git a/tests/sqllogictests/suites/mode/standalone/explain/filter.test b/tests/sqllogictests/suites/mode/standalone/explain/filter.test index 466e0c1684d05..3ef1503722e0b 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/filter.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/filter.test @@ -24,6 +24,7 @@ Filter ├── estimated rows: 0.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -41,6 +42,7 @@ Filter ├── estimated rows: 0.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -76,6 +78,7 @@ Filter ├── estimated rows: 40.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [d (#0), a (#1)] ├── read rows: 0 ├── read size: 0 @@ -94,6 +97,7 @@ Filter ├── estimated rows: 40.00 └── TableScan ├── table: default.default.t2 + ├── scan id: 0 ├── output columns: [d (#0), a (#1)] ├── read rows: 0 ├── read size: 0 @@ -112,6 +116,7 @@ Filter ├── estimated rows: 5.25 └── TableScan ├── table: default.system.numbers + ├── scan id: 0 ├── output columns: [number (#0)] ├── read rows: 6 ├── read size: < 1 KiB @@ -147,6 +152,7 @@ Filter ├── estimated rows: 40.00 └── TableScan ├── table: default.default.t1_tz + ├── scan id: 0 ├── output columns: [d (#0), a (#1)] ├── read rows: 200 ├── read size: < 1 KiB @@ -165,6 +171,7 @@ Filter ├── estimated rows: 40.00 └── TableScan ├── table: default.default.t2_tz + ├── scan id: 0 ├── output columns: [d (#0), a (#1)] ├── read rows: 200 ├── read size: < 1 KiB diff --git a/tests/sqllogictests/suites/mode/standalone/explain/fold_agg.test b/tests/sqllogictests/suites/mode/standalone/explain/fold_agg.test index 6c3844847c3ba..3e62e99924091 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/fold_agg.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/fold_agg.test @@ -83,6 +83,7 @@ AggregateFinal ├── estimated rows: 2983.50 └── TableScan ├── table: default.default.t + ├── scan id: 0 ├── output columns: [number (#0)] ├── read rows: 3000 ├── read size: 5.89 KiB @@ -106,6 +107,7 @@ AggregateFinal ├── estimated rows: 2000.00 └── TableScan ├── table: default.default.t + ├── scan id: 0 ├── output columns: [number (#0)] ├── read rows: 3000 ├── read size: 5.89 KiB @@ -158,6 +160,7 @@ EvalScalar ├── estimated rows: 3000.00 └── TableScan ├── table: default.default.t + ├── scan id: 0 ├── output columns: [number (#0), money (#2)] ├── read rows: 3000 ├── read size: 10.51 KiB @@ -203,6 +206,7 @@ EvalScalar ├── estimated rows: 1.00 └── TableScan ├── table: default.default.t + ├── scan id: 0 ├── output columns: [number (#0), money (#2)] ├── read rows: 3000 ├── read size: 10.51 KiB diff --git a/tests/sqllogictests/suites/mode/standalone/explain/index/explain_inverted_index.test b/tests/sqllogictests/suites/mode/standalone/explain/index/explain_inverted_index.test index 7e0bd3b8544e6..35bd2a292d0e0 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/index/explain_inverted_index.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/index/explain_inverted_index.test @@ -38,6 +38,7 @@ Filter ├── estimated rows: 10.00 └── TableScan ├── table: default.test_inverted_index_db.t1 + ├── scan id: 0 ├── output columns: [id (#0), content (#1), _search_matched (#2)] ├── read rows: 2 ├── read size: < 1 KiB @@ -78,6 +79,7 @@ Filter ├── estimated rows: 10.00 └── TableScan ├── table: default.test_inverted_index_db.t2 + ├── scan id: 0 ├── output columns: [id (#0), content (#1), _search_matched (#2)] ├── read rows: 2 ├── read size: < 1 KiB diff --git a/tests/sqllogictests/suites/mode/standalone/explain/index/explain_ngram_index.test b/tests/sqllogictests/suites/mode/standalone/explain/index/explain_ngram_index.test index b3e0984f3f7ec..3bde648dc0d6a 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/index/explain_ngram_index.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/index/explain_ngram_index.test @@ -44,6 +44,7 @@ Filter ├── estimated rows: 8.00 └── TableScan ├── table: default.test_ngram_index_db.t1 + ├── scan id: 0 ├── output columns: [id (#0), content (#1)] ├── read rows: 16 ├── read size: 1.03 KiB @@ -62,6 +63,7 @@ Filter ├── estimated rows: 0.06 └── TableScan ├── table: default.test_ngram_index_db.t1 + ├── scan id: 0 ├── output columns: [id (#0), content (#1)] ├── read rows: 2 ├── read size: < 1 KiB @@ -80,6 +82,7 @@ Filter ├── estimated rows: 0.03 └── TableScan ├── table: default.test_ngram_index_db.t1 + ├── scan id: 0 ├── output columns: [id (#0), content (#1)] ├── read rows: 2 ├── read size: < 1 KiB @@ -98,6 +101,7 @@ Filter ├── estimated rows: 8.00 └── TableScan ├── table: default.test_ngram_index_db.t1 + ├── scan id: 0 ├── output columns: [id (#0), content (#1)] ├── read rows: 16 ├── read size: 1.03 KiB @@ -116,6 +120,7 @@ Filter ├── estimated rows: 4.00 └── TableScan ├── table: default.test_ngram_index_db.t1 + ├── scan id: 0 ├── output columns: [id (#0), content (#1)] ├── read rows: 2 ├── read size: < 1 KiB @@ -163,6 +168,7 @@ EXPLAIN SELECT id, content FROM t2 WHERE content LIKE '%your eggs%' ---- TableScan ├── table: default.test_ngram_index_db.t2 +├── scan id: 0 ├── output columns: [id (#0), content (#1)] ├── read rows: 2 ├── read size: < 1 KiB @@ -177,6 +183,7 @@ EXPLAIN SELECT id, content FROM t2 WHERE content LIKE '%your eggs' ---- TableScan ├── table: default.test_ngram_index_db.t2 +├── scan id: 0 ├── output columns: [id (#0), content (#1)] ├── read rows: 2 ├── read size: < 1 KiB @@ -196,6 +203,7 @@ EXPLAIN SELECT id, content FROM t2 WHERE content LIKE '%yo%' ---- TableScan ├── table: default.test_ngram_index_db.t2 +├── scan id: 0 ├── output columns: [id (#0), content (#1)] ├── read rows: 16 ├── read size: 1.26 KiB @@ -210,6 +218,7 @@ EXPLAIN SELECT id, content FROM t2 WHERE content LIKE '%风来%' ---- TableScan ├── table: default.test_ngram_index_db.t2 +├── scan id: 0 ├── output columns: [id (#0), content (#1)] ├── read rows: 16 ├── read size: 1.26 KiB @@ -224,6 +233,7 @@ EXPLAIN SELECT id, content FROM t2 WHERE content LIKE '%月无声%' ---- TableScan ├── table: default.test_ngram_index_db.t2 +├── scan id: 0 ├── output columns: [id (#0), content (#1)] ├── read rows: 2 ├── read size: < 1 KiB @@ -262,6 +272,7 @@ EXPLAIN SELECT id, content FROM t2 WHERE content LIKE '%your eggs%' ---- TableScan ├── table: default.test_ngram_index_db.t2 +├── scan id: 0 ├── output columns: [id (#0), content (#1)] ├── read rows: 10 ├── read size: < 1 KiB @@ -282,6 +293,7 @@ EXPLAIN SELECT id, content FROM t2 WHERE content LIKE '%your eggs%' ---- TableScan ├── table: default.test_ngram_index_db.t2 +├── scan id: 0 ├── output columns: [id (#0), content (#1)] ├── read rows: 10 ├── read size: < 1 KiB @@ -336,6 +348,7 @@ Filter ├── estimated rows: 0.25 └── TableScan ├── table: default.test_ngram_index_db.t3 + ├── scan id: 0 ├── output columns: [id (#0), content1 (#1), content2 (#2)] ├── read rows: 2 ├── read size: < 1 KiB @@ -359,6 +372,7 @@ Filter ├── estimated rows: 0.25 └── TableScan ├── table: default.test_ngram_index_db.t3 + ├── scan id: 0 ├── output columns: [id (#0), content1 (#1), content2 (#2)] ├── read rows: 2 ├── read size: < 1 KiB diff --git a/tests/sqllogictests/suites/mode/standalone/explain/infer_filter.test b/tests/sqllogictests/suites/mode/standalone/explain/infer_filter.test index 8da11d6c0c16b..35e7386952d55 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/infer_filter.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/infer_filter.test @@ -32,6 +32,7 @@ Filter ├── estimated rows: 0.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -62,6 +63,7 @@ Filter ├── estimated rows: 0.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -80,6 +82,7 @@ Filter ├── estimated rows: 0.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -104,6 +107,7 @@ Filter ├── estimated rows: 0.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -122,6 +126,7 @@ Filter ├── estimated rows: 0.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -140,6 +145,7 @@ Filter ├── estimated rows: 0.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -164,6 +170,7 @@ Filter ├── estimated rows: 0.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -182,6 +189,7 @@ Filter ├── estimated rows: 0.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -207,6 +215,7 @@ Filter ├── estimated rows: 0.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -225,6 +234,7 @@ Filter ├── estimated rows: 0.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -243,6 +253,7 @@ Filter ├── estimated rows: 0.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -261,6 +272,7 @@ Filter ├── estimated rows: 0.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -279,6 +291,7 @@ Filter ├── estimated rows: 0.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -297,6 +310,7 @@ Filter ├── estimated rows: 0.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -315,6 +329,7 @@ Filter ├── estimated rows: 0.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -333,6 +348,7 @@ Filter ├── estimated rows: 0.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -351,6 +367,7 @@ Filter ├── estimated rows: 0.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -369,6 +386,7 @@ Filter ├── estimated rows: 0.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -387,6 +405,7 @@ Filter ├── estimated rows: 0.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -411,6 +430,7 @@ Filter ├── estimated rows: 0.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -429,6 +449,7 @@ Filter ├── estimated rows: 0.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -447,6 +468,7 @@ Filter ├── estimated rows: 0.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -465,6 +487,7 @@ Filter ├── estimated rows: 0.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -489,6 +512,7 @@ Filter ├── estimated rows: 0.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -507,6 +531,7 @@ Filter ├── estimated rows: 0.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -527,7 +552,7 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:0, build key:t2.a (#2), probe key:t1.a (#0), filter type:bloom,inlist,min_max +│ └── filter id:0, build key:t2.a (#2), probe targets:[t1.a (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 0.00 ├── Filter(Build) │ ├── output columns: [t2.a (#2), t2.b (#3)] @@ -535,6 +560,7 @@ HashJoin │ ├── estimated rows: 0.00 │ └── TableScan │ ├── table: default.default.t2 +│ ├── scan id: 1 │ ├── output columns: [a (#2), b (#3)] │ ├── read rows: 0 │ ├── read size: 0 @@ -548,6 +574,7 @@ HashJoin ├── estimated rows: 0.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -569,7 +596,7 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:1, build key:t3.a (#4), probe key:t1.a (#0), filter type:bloom,inlist,min_max +│ └── filter id:1, build key:t3.a (#4), probe targets:[t1.a (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 0.00 ├── HashJoin(Build) │ ├── output columns: [t2.a (#2), t2.b (#3), t3.b (#5), t3.a (#4)] @@ -579,7 +606,7 @@ HashJoin │ ├── keys is null equal: [false] │ ├── filters: [] │ ├── build join filters: -│ │ └── filter id:0, build key:t3.a (#4), probe key:t2.a (#2), filter type:bloom,inlist,min_max +│ │ └── filter id:0, build key:t3.a (#4), probe targets:[t2.a (#2)@scan1], filter type:bloom,inlist,min_max │ ├── estimated rows: 0.00 │ ├── Filter(Build) │ │ ├── output columns: [t3.a (#4), t3.b (#5)] @@ -587,6 +614,7 @@ HashJoin │ │ ├── estimated rows: 0.00 │ │ └── TableScan │ │ ├── table: default.default.t3 +│ │ ├── scan id: 2 │ │ ├── output columns: [a (#4), b (#5)] │ │ ├── read rows: 0 │ │ ├── read size: 0 @@ -600,6 +628,7 @@ HashJoin │ ├── estimated rows: 0.00 │ └── TableScan │ ├── table: default.default.t2 +│ ├── scan id: 1 │ ├── output columns: [a (#2), b (#3)] │ ├── read rows: 0 │ ├── read size: 0 @@ -614,6 +643,7 @@ HashJoin ├── estimated rows: 0.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -639,6 +669,7 @@ MergeJoin │ ├── estimated rows: 0.00 │ └── TableScan │ ├── table: default.default.t2 +│ ├── scan id: 1 │ ├── output columns: [a (#2), b (#3)] │ ├── read rows: 0 │ ├── read size: 0 @@ -652,6 +683,7 @@ MergeJoin ├── estimated rows: 0.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -672,6 +704,7 @@ MergeJoin ├── estimated rows: 0.00 ├── TableScan(Left) │ ├── table: default.default.t2 +│ ├── scan id: 1 │ ├── output columns: [a (#2), b (#3)] │ ├── read rows: 0 │ ├── read size: 0 @@ -685,6 +718,7 @@ MergeJoin ├── estimated rows: 0.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -717,7 +751,7 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:1, build key:t3.a (#4), probe key:t1.a (#0), filter type:bloom,inlist,min_max +│ └── filter id:1, build key:t3.a (#4), probe targets:[t1.a (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 0.00 ├── HashJoin(Build) │ ├── output columns: [t2.a (#2), t2.b (#3), t3.b (#5), t3.a (#4)] @@ -727,10 +761,11 @@ HashJoin │ ├── keys is null equal: [false] │ ├── filters: [] │ ├── build join filters: -│ │ └── filter id:0, build key:t3.a (#4), probe key:t2.a (#2), filter type:bloom,inlist,min_max +│ │ └── filter id:0, build key:t3.a (#4), probe targets:[t2.a (#2)@scan1], filter type:bloom,inlist,min_max │ ├── estimated rows: 0.00 │ ├── TableScan(Build) │ │ ├── table: default.default.t3 +│ │ ├── scan id: 2 │ │ ├── output columns: [a (#4), b (#5)] │ │ ├── read rows: 0 │ │ ├── read size: 0 @@ -740,6 +775,7 @@ HashJoin │ │ └── estimated rows: 0.00 │ └── TableScan(Probe) │ ├── table: default.default.t2 +│ ├── scan id: 1 │ ├── output columns: [a (#2), b (#3)] │ ├── read rows: 0 │ ├── read size: 0 @@ -750,6 +786,7 @@ HashJoin │ └── estimated rows: 0.00 └── TableScan(Probe) ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -768,6 +805,7 @@ Filter ├── estimated rows: 0.00 └── TableScan ├── table: default.default.t4 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -787,7 +825,7 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:0, build key:CAST(t4.b (#3) AS Int32 NULL), probe key:t3.b (#1), filter type:bloom,inlist,min_max +│ └── filter id:0, build key:CAST(t4.b (#3) AS Int32 NULL), probe targets:[t3.b (#1)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 0.00 ├── Filter(Build) │ ├── output columns: [t4.a (#2), t4.b (#3)] @@ -795,6 +833,7 @@ HashJoin │ ├── estimated rows: 0.00 │ └── TableScan │ ├── table: default.default.t4 +│ ├── scan id: 1 │ ├── output columns: [a (#2), b (#3)] │ ├── read rows: 0 │ ├── read size: 0 @@ -804,6 +843,7 @@ HashJoin │ └── estimated rows: 0.00 └── TableScan(Probe) ├── table: default.default.t3 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -849,6 +889,7 @@ HashJoin │ ├── estimated rows: 1.00 │ └── TableScan │ ├── table: default.default.t2 +│ ├── scan id: 1 │ ├── output columns: [id (#1)] │ ├── read rows: 1 │ ├── read size: < 1 KiB @@ -863,6 +904,7 @@ HashJoin ├── estimated rows: 1.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [id (#0)] ├── read rows: 1 ├── read size: < 1 KiB diff --git a/tests/sqllogictests/suites/mode/standalone/explain/join.test b/tests/sqllogictests/suites/mode/standalone/explain/join.test index d27ed7c1e4e6a..712e466391c8b 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/join.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/join.test @@ -27,10 +27,11 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:0, build key:t.number (#0), probe key:t1.number (#1), filter type:inlist,min_max +│ └── filter id:0, build key:t.number (#0), probe targets:[t1.number (#1)@scan1], filter type:bloom,inlist,min_max ├── estimated rows: 1.00 ├── TableScan(Build) │ ├── table: default.default.t +│ ├── scan id: 0 │ ├── output columns: [number (#0)] │ ├── read rows: 1 │ ├── read size: < 1 KiB @@ -41,6 +42,7 @@ HashJoin │ └── estimated rows: 1.00 └── TableScan(Probe) ├── table: default.default.t1 + ├── scan id: 1 ├── output columns: [number (#1)] ├── read rows: 10 ├── read size: < 1 KiB @@ -62,10 +64,11 @@ HashJoin ├── keys is null equal: [false, false] ├── filters: [] ├── build join filters: -│ └── filter id:0, build key:t.number (#0), probe key:t1.number (#1), filter type:inlist,min_max +│ └── filter id:0, build key:t.number (#0), probe targets:[t1.number (#1)@scan1], filter type:bloom,inlist,min_max ├── estimated rows: 1.00 ├── TableScan(Build) │ ├── table: default.default.t +│ ├── scan id: 0 │ ├── output columns: [number (#0)] │ ├── read rows: 1 │ ├── read size: < 1 KiB @@ -80,6 +83,7 @@ HashJoin ├── estimated rows: 2.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 1 ├── output columns: [number (#1)] ├── read rows: 10 ├── read size: < 1 KiB @@ -107,6 +111,7 @@ HashJoin │ ├── estimated rows: 0.00 │ └── TableScan │ ├── table: default.default.t +│ ├── scan id: 0 │ ├── output columns: [number (#0)] │ ├── read rows: 0 │ ├── read size: 0 @@ -121,6 +126,7 @@ HashJoin ├── estimated rows: 8.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 1 ├── output columns: [number (#1)] ├── read rows: 10 ├── read size: < 1 KiB @@ -147,6 +153,7 @@ Filter ├── estimated rows: 10.00 ├── TableScan(Build) │ ├── table: default.default.t + │ ├── scan id: 0 │ ├── output columns: [number (#0)] │ ├── read rows: 1 │ ├── read size: < 1 KiB @@ -157,6 +164,7 @@ Filter │ └── estimated rows: 1.00 └── TableScan(Probe) ├── table: default.default.t1 + ├── scan id: 1 ├── output columns: [number (#1)] ├── read rows: 10 ├── read size: < 1 KiB @@ -178,7 +186,7 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:0, build key:t1.number (#1), probe key:t2.number (#2), filter type:bloom,inlist,min_max +│ └── filter id:0, build key:t1.number (#1), probe targets:[t2.number (#2)@scan2], filter type:bloom,inlist,min_max ├── estimated rows: 0.00 ├── HashJoin(Build) │ ├── output columns: [t1.number (#1), t.number (#0)] @@ -194,6 +202,7 @@ HashJoin │ │ ├── estimated rows: 0.00 │ │ └── TableScan │ │ ├── table: default.default.t +│ │ ├── scan id: 0 │ │ ├── output columns: [number (#0)] │ │ ├── read rows: 0 │ │ ├── read size: 0 @@ -204,6 +213,7 @@ HashJoin │ │ └── estimated rows: 1.00 │ └── TableScan(Probe) │ ├── table: default.default.t1 +│ ├── scan id: 1 │ ├── output columns: [number (#1)] │ ├── read rows: 10 │ ├── read size: < 1 KiB @@ -214,6 +224,7 @@ HashJoin │ └── estimated rows: 10.00 └── TableScan(Probe) ├── table: default.default.t2 + ├── scan id: 2 ├── output columns: [number (#2)] ├── read rows: 100 ├── read size: < 1 KiB @@ -255,7 +266,7 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:0, build key:a.x (#0), probe key:b.x (#1), filter type:inlist,min_max +│ └── filter id:0, build key:a.x (#0), probe targets:[b.x (#1)@scan1], filter type:bloom,inlist,min_max ├── estimated rows: 2.67 ├── Filter(Build) │ ├── output columns: [a.x (#0)] @@ -263,6 +274,7 @@ HashJoin │ ├── estimated rows: 2.00 │ └── TableScan │ ├── table: default.default.onecolumn +│ ├── scan id: 0 │ ├── output columns: [x (#0)] │ ├── read rows: 4 │ ├── read size: < 1 KiB @@ -277,6 +289,7 @@ HashJoin ├── estimated rows: 2.67 └── TableScan ├── table: default.default.twocolumn + ├── scan id: 1 ├── output columns: [x (#1), y (#2)] ├── read rows: 4 ├── read size: < 1 KiB @@ -298,7 +311,7 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:0, build key:a.x (#0), probe key:b.x (#1), filter type:inlist,min_max +│ └── filter id:0, build key:a.x (#0), probe targets:[b.x (#1)@scan1], filter type:bloom,inlist,min_max ├── estimated rows: 1.33 ├── Filter(Build) │ ├── output columns: [a.x (#0)] @@ -306,6 +319,7 @@ HashJoin │ ├── estimated rows: 1.33 │ └── TableScan │ ├── table: default.default.onecolumn +│ ├── scan id: 0 │ ├── output columns: [x (#0)] │ ├── read rows: 4 │ ├── read size: < 1 KiB @@ -320,6 +334,7 @@ HashJoin ├── estimated rows: 2.00 └── TableScan ├── table: default.default.twocolumn + ├── scan id: 1 ├── output columns: [x (#1), y (#2)] ├── read rows: 4 ├── read size: < 1 KiB @@ -341,7 +356,7 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:0, build key:b.x (#1), probe key:a.x (#0), filter type:inlist,min_max +│ └── filter id:0, build key:b.x (#1), probe targets:[a.x (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 4.00 ├── Filter(Build) │ ├── output columns: [b.x (#1), b.y (#2)] @@ -349,6 +364,7 @@ HashJoin │ ├── estimated rows: 2.00 │ └── TableScan │ ├── table: default.default.twocolumn +│ ├── scan id: 1 │ ├── output columns: [x (#1), y (#2)] │ ├── read rows: 4 │ ├── read size: < 1 KiB @@ -363,6 +379,7 @@ HashJoin ├── estimated rows: 2.00 └── TableScan ├── table: default.default.onecolumn + ├── scan id: 0 ├── output columns: [x (#0)] ├── read rows: 4 ├── read size: < 1 KiB @@ -392,6 +409,7 @@ Filter ├── estimated rows: 5.33 ├── TableScan(Build) │ ├── table: default.default.twocolumn + │ ├── scan id: 1 │ ├── output columns: [x (#1), y (#2)] │ ├── read rows: 4 │ ├── read size: < 1 KiB @@ -402,6 +420,7 @@ Filter │ └── estimated rows: 4.00 └── TableScan(Probe) ├── table: default.default.onecolumn + ├── scan id: 0 ├── output columns: [x (#0)] ├── read rows: 4 ├── read size: < 1 KiB @@ -428,6 +447,7 @@ HashJoin │ ├── estimated rows: 2.00 │ └── TableScan │ ├── table: default.default.onecolumn +│ ├── scan id: 0 │ ├── output columns: [x (#0)] │ ├── read rows: 4 │ ├── read size: < 1 KiB @@ -442,6 +462,7 @@ HashJoin ├── estimated rows: 2.00 └── TableScan ├── table: default.default.twocolumn + ├── scan id: 1 ├── output columns: [x (#1), y (#2)] ├── read rows: 4 ├── read size: < 1 KiB @@ -477,7 +498,7 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:0, build key:t2.a (#2), probe key:t1.a (#0), filter type:bloom,inlist,min_max +│ └── filter id:0, build key:t2.a (#2), probe targets:[t1.a (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 0.00 ├── Filter(Build) │ ├── output columns: [t2.a (#2), t2.b (#3)] @@ -485,6 +506,7 @@ HashJoin │ ├── estimated rows: 0.00 │ └── TableScan │ ├── table: default.default.t2 +│ ├── scan id: 1 │ ├── output columns: [a (#2), b (#3)] │ ├── read rows: 0 │ ├── read size: 0 @@ -498,6 +520,7 @@ HashJoin ├── estimated rows: 0.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -518,10 +541,11 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:0, build key:t2.a (#2), probe key:t1.a (#0), filter type:bloom,inlist,min_max +│ └── filter id:0, build key:t2.a (#2), probe targets:[t1.a (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 0.00 ├── TableScan(Build) │ ├── table: default.default.t2 +│ ├── scan id: 1 │ ├── output columns: [a (#2), b (#3)] │ ├── read rows: 0 │ ├── read size: 0 @@ -535,6 +559,7 @@ HashJoin ├── estimated rows: 0.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -555,8 +580,8 @@ HashJoin ├── keys is null equal: [false, false] ├── filters: [] ├── build join filters: -│ ├── filter id:0, build key:t2.a (#2), probe key:t1.a (#0), filter type:bloom,inlist,min_max -│ └── filter id:1, build key:t2.b (#3), probe key:t1.b (#1), filter type:bloom,inlist,min_max +│ ├── filter id:0, build key:t2.a (#2), probe targets:[t1.a (#0)@scan0], filter type:bloom,inlist,min_max +│ └── filter id:1, build key:t2.b (#3), probe targets:[t1.b (#1)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 0.00 ├── Filter(Build) │ ├── output columns: [t2.a (#2), t2.b (#3)] @@ -564,6 +589,7 @@ HashJoin │ ├── estimated rows: 0.00 │ └── TableScan │ ├── table: default.default.t2 +│ ├── scan id: 1 │ ├── output columns: [a (#2), b (#3)] │ ├── read rows: 0 │ ├── read size: 0 @@ -577,6 +603,7 @@ HashJoin ├── estimated rows: 0.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -597,8 +624,8 @@ HashJoin ├── keys is null equal: [false, false] ├── filters: [] ├── build join filters: -│ ├── filter id:0, build key:t2.a (#2), probe key:t1.a (#0), filter type:bloom,inlist,min_max -│ └── filter id:1, build key:t2.b (#3), probe key:t1.b (#1), filter type:bloom,inlist,min_max +│ ├── filter id:0, build key:t2.a (#2), probe targets:[t1.a (#0)@scan0], filter type:bloom,inlist,min_max +│ └── filter id:1, build key:t2.b (#3), probe targets:[t1.b (#1)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 0.00 ├── Filter(Build) │ ├── output columns: [t2.a (#2), t2.b (#3)] @@ -606,6 +633,7 @@ HashJoin │ ├── estimated rows: 0.00 │ └── TableScan │ ├── table: default.default.t2 +│ ├── scan id: 1 │ ├── output columns: [a (#2), b (#3)] │ ├── read rows: 0 │ ├── read size: 0 @@ -619,6 +647,7 @@ HashJoin ├── estimated rows: 0.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -639,8 +668,8 @@ HashJoin ├── keys is null equal: [false, false] ├── filters: [] ├── build join filters: -│ ├── filter id:0, build key:t2.a (#2), probe key:t1.a (#0), filter type:bloom,inlist,min_max -│ └── filter id:1, build key:t2.b (#3), probe key:t1.b (#1), filter type:bloom,inlist,min_max +│ ├── filter id:0, build key:t2.a (#2), probe targets:[t1.a (#0)@scan0], filter type:bloom,inlist,min_max +│ └── filter id:1, build key:t2.b (#3), probe targets:[t1.b (#1)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 0.00 ├── Filter(Build) │ ├── output columns: [t2.a (#2), t2.b (#3)] @@ -648,6 +677,7 @@ HashJoin │ ├── estimated rows: 0.00 │ └── TableScan │ ├── table: default.default.t2 +│ ├── scan id: 1 │ ├── output columns: [a (#2), b (#3)] │ ├── read rows: 0 │ ├── read size: 0 @@ -661,6 +691,7 @@ HashJoin ├── estimated rows: 0.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -682,10 +713,11 @@ HashJoin ├── keys is null equal: [false] ├── filters: [t1.b (#1) > t2.b (#3)] ├── build join filters: -│ └── filter id:0, build key:t2.a (#2), probe key:t1.a (#0), filter type:bloom,inlist,min_max +│ └── filter id:0, build key:t2.a (#2), probe targets:[t1.a (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 0.00 ├── TableScan(Build) │ ├── table: default.default.t2 +│ ├── scan id: 1 │ ├── output columns: [a (#2), b (#3)] │ ├── read rows: 0 │ ├── read size: 0 @@ -695,6 +727,7 @@ HashJoin │ └── estimated rows: 0.00 └── TableScan(Probe) ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -799,7 +832,7 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:0, build key:t.x (#1), probe key:o.x (#0), filter type:inlist,min_max +│ └── filter id:0, build key:t.x (#1), probe targets:[o.x (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 7.00 ├── Filter(Build) │ ├── output columns: [t.x (#1), t.y (#2)] @@ -807,6 +840,7 @@ HashJoin │ ├── estimated rows: 2.00 │ └── TableScan │ ├── table: default.default.twocolumn +│ ├── scan id: 1 │ ├── output columns: [x (#1), y (#2)] │ ├── read rows: 8 │ ├── read size: < 1 KiB @@ -817,6 +851,7 @@ HashJoin │ └── estimated rows: 8.00 └── TableScan(Probe) ├── table: default.default.onecolumn + ├── scan id: 0 ├── output columns: [x (#0)] ├── read rows: 7 ├── read size: < 1 KiB @@ -849,10 +884,11 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:0, build key:t2.number (#1), probe key:t1.number (#0), filter type:inlist,min_max +│ └── filter id:0, build key:t2.number (#1), probe targets:[t1.number (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 200.00 ├── TableScan(Build) │ ├── table: default.system.numbers +│ ├── scan id: 1 │ ├── output columns: [number (#1)] │ ├── read rows: 20 │ ├── read size: < 1 KiB @@ -862,6 +898,7 @@ HashJoin │ └── estimated rows: 20.00 └── TableScan(Probe) ├── table: default.system.numbers + ├── scan id: 0 ├── output columns: [number (#0)] ├── read rows: 10 ├── read size: < 1 KiB @@ -904,6 +941,7 @@ EvalScalar │ ├── estimated rows: 0.00 │ └── TableScan │ ├── table: default.default.t2 + │ ├── scan id: 1 │ ├── output columns: [wms_id (#4)] │ ├── read rows: 0 │ ├── read size: 0 @@ -917,6 +955,7 @@ EvalScalar ├── estimated rows: 0.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [ext (#1)] ├── read rows: 0 ├── read size: 0 @@ -946,6 +985,7 @@ EvalScalar │ ├── estimated rows: 0.00 │ └── TableScan │ ├── table: default.default.t2 + │ ├── scan id: 1 │ ├── output columns: [wms_id (#4)] │ ├── read rows: 0 │ ├── read size: 0 @@ -959,6 +999,7 @@ EvalScalar ├── estimated rows: 0.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [ext (#1)] ├── read rows: 0 ├── read size: 0 @@ -984,6 +1025,7 @@ EvalScalar ├── estimated rows: 0.00 ├── TableScan(Build) │ ├── table: default.default.t2 + │ ├── scan id: 1 │ ├── output columns: [wms_id (#4)] │ ├── read rows: 0 │ ├── read size: 0 @@ -997,6 +1039,7 @@ EvalScalar ├── estimated rows: 0.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [ext (#1)] ├── read rows: 0 ├── read size: 0 @@ -1044,6 +1087,7 @@ HashJoin │ └── DummyTableScan └── TableScan(Probe) ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 4 ├── read size: < 1 KiB @@ -1070,6 +1114,7 @@ HashJoin │ ├── estimated rows: 1.00 │ └── TableScan │ ├── table: default.default.t2 +│ ├── scan id: 1 │ ├── output columns: [a (#1)] │ ├── read rows: 1 │ ├── read size: < 1 KiB @@ -1080,6 +1125,7 @@ HashJoin │ └── estimated rows: 1.00 └── TableScan(Probe) ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 4 ├── read size: < 1 KiB @@ -1102,6 +1148,7 @@ HashJoin ├── estimated rows: 12.00 ├── TableScan(Build) │ ├── table: default.default.t3 +│ ├── scan id: 1 │ ├── output columns: [a (#1), b (#2)] │ ├── read rows: 3 │ ├── read size: < 1 KiB @@ -1112,6 +1159,7 @@ HashJoin │ └── estimated rows: 3.00 └── TableScan(Probe) ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 4 ├── read size: < 1 KiB @@ -1132,10 +1180,11 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:0, build key:t3.b (#2), probe key:t1.a (#0), filter type:inlist,min_max +│ └── filter id:0, build key:t3.b (#2), probe targets:[t1.a (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 3.00 ├── TableScan(Build) │ ├── table: default.default.t3 +│ ├── scan id: 1 │ ├── output columns: [a (#1), b (#2)] │ ├── read rows: 3 │ ├── read size: < 1 KiB @@ -1146,6 +1195,7 @@ HashJoin │ └── estimated rows: 3.00 └── TableScan(Probe) ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 4 ├── read size: < 1 KiB @@ -1182,7 +1232,7 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:0, build key:t2.a (#2), probe key:t1.a (#0), filter type:bloom,inlist,min_max +│ └── filter id:0, build key:t2.a (#2), probe targets:[t1.a (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 0.00 ├── Filter(Build) │ ├── output columns: [t2.a (#2), t2.b (#3)] @@ -1190,6 +1240,7 @@ HashJoin │ ├── estimated rows: 0.00 │ └── TableScan │ ├── table: default.default.t2 +│ ├── scan id: 1 │ ├── output columns: [a (#2), b (#3)] │ ├── read rows: 0 │ ├── read size: 0 @@ -1203,6 +1254,7 @@ HashJoin ├── estimated rows: 0.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 diff --git a/tests/sqllogictests/suites/mode/standalone/explain/join_reorder/chain.test b/tests/sqllogictests/suites/mode/standalone/explain/join_reorder/chain.test index 111394ff82c38..5c7043feb1408 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/join_reorder/chain.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/join_reorder/chain.test @@ -39,7 +39,7 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:1, build key:t.a (#0), probe key:t2.a (#2), filter type:inlist,min_max +│ └── filter id:1, build key:t.a (#0), probe targets:[t2.a (#2)@scan2], filter type:bloom,inlist,min_max ├── estimated rows: 1.00 ├── HashJoin(Build) │ ├── output columns: [t1.a (#1), t.a (#0)] @@ -49,10 +49,11 @@ HashJoin │ ├── keys is null equal: [false] │ ├── filters: [] │ ├── build join filters: -│ │ └── filter id:0, build key:t.a (#0), probe key:t1.a (#1), filter type:inlist,min_max +│ │ └── filter id:0, build key:t.a (#0), probe targets:[t1.a (#1)@scan1], filter type:bloom,inlist,min_max │ ├── estimated rows: 1.00 │ ├── TableScan(Build) │ │ ├── table: default.join_reorder.t +│ │ ├── scan id: 0 │ │ ├── output columns: [a (#0)] │ │ ├── read rows: 1 │ │ ├── read size: < 1 KiB @@ -63,6 +64,7 @@ HashJoin │ │ └── estimated rows: 1.00 │ └── TableScan(Probe) │ ├── table: default.join_reorder.t1 +│ ├── scan id: 1 │ ├── output columns: [a (#1)] │ ├── read rows: 10 │ ├── read size: < 1 KiB @@ -74,6 +76,7 @@ HashJoin │ └── estimated rows: 10.00 └── TableScan(Probe) ├── table: default.join_reorder.t2 + ├── scan id: 2 ├── output columns: [a (#2)] ├── read rows: 100 ├── read size: < 1 KiB @@ -108,6 +111,7 @@ HashJoin │ ├── estimated rows: 1.00 │ ├── TableScan(Build) │ │ ├── table: default.join_reorder.t +│ │ ├── scan id: 0 │ │ ├── output columns: [a (#0)] │ │ ├── read rows: 1 │ │ ├── read size: < 1 KiB @@ -118,6 +122,7 @@ HashJoin │ │ └── estimated rows: 1.00 │ └── TableScan(Probe) │ ├── table: default.join_reorder.t1 +│ ├── scan id: 1 │ ├── output columns: [a (#1)] │ ├── read rows: 10 │ ├── read size: < 1 KiB @@ -128,6 +133,7 @@ HashJoin │ └── estimated rows: 10.00 └── TableScan(Probe) ├── table: default.join_reorder.t2 + ├── scan id: 2 ├── output columns: [a (#2)] ├── read rows: 100 ├── read size: < 1 KiB @@ -151,7 +157,7 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:1, build key:t.a (#0), probe key:t1.a (#2), filter type:inlist,min_max +│ └── filter id:1, build key:t.a (#0), probe targets:[t1.a (#2)@scan2], filter type:bloom,inlist,min_max ├── estimated rows: 1.00 ├── HashJoin(Build) │ ├── output columns: [t2.a (#1), t.a (#0)] @@ -161,10 +167,11 @@ HashJoin │ ├── keys is null equal: [false] │ ├── filters: [] │ ├── build join filters: -│ │ └── filter id:0, build key:t.a (#0), probe key:t2.a (#1), filter type:inlist,min_max +│ │ └── filter id:0, build key:t.a (#0), probe targets:[t2.a (#1)@scan1], filter type:bloom,inlist,min_max │ ├── estimated rows: 1.00 │ ├── TableScan(Build) │ │ ├── table: default.join_reorder.t +│ │ ├── scan id: 0 │ │ ├── output columns: [a (#0)] │ │ ├── read rows: 1 │ │ ├── read size: < 1 KiB @@ -175,6 +182,7 @@ HashJoin │ │ └── estimated rows: 1.00 │ └── TableScan(Probe) │ ├── table: default.join_reorder.t2 +│ ├── scan id: 1 │ ├── output columns: [a (#1)] │ ├── read rows: 100 │ ├── read size: < 1 KiB @@ -186,6 +194,7 @@ HashJoin │ └── estimated rows: 100.00 └── TableScan(Probe) ├── table: default.join_reorder.t1 + ├── scan id: 2 ├── output columns: [a (#2)] ├── read rows: 10 ├── read size: < 1 KiB @@ -207,7 +216,7 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:1, build key:t2.a (#2), probe key:t1.a (#0), filter type:inlist,min_max +│ └── filter id:1, build key:t2.a (#2), probe targets:[t1.a (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 1.00 ├── HashJoin(Build) │ ├── output columns: [t2.a (#2), t.a (#1)] @@ -217,10 +226,11 @@ HashJoin │ ├── keys is null equal: [false] │ ├── filters: [] │ ├── build join filters: -│ │ └── filter id:0, build key:t.a (#1), probe key:t2.a (#2), filter type:inlist,min_max +│ │ └── filter id:0, build key:t.a (#1), probe targets:[t2.a (#2)@scan2], filter type:bloom,inlist,min_max │ ├── estimated rows: 1.00 │ ├── TableScan(Build) │ │ ├── table: default.join_reorder.t +│ │ ├── scan id: 1 │ │ ├── output columns: [a (#1)] │ │ ├── read rows: 1 │ │ ├── read size: < 1 KiB @@ -231,6 +241,7 @@ HashJoin │ │ └── estimated rows: 1.00 │ └── TableScan(Probe) │ ├── table: default.join_reorder.t2 +│ ├── scan id: 2 │ ├── output columns: [a (#2)] │ ├── read rows: 100 │ ├── read size: < 1 KiB @@ -242,6 +253,7 @@ HashJoin │ └── estimated rows: 100.00 └── TableScan(Probe) ├── table: default.join_reorder.t1 + ├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 10 ├── read size: < 1 KiB @@ -263,7 +275,7 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:1, build key:t.a (#2), probe key:t1.a (#0), filter type:inlist,min_max +│ └── filter id:1, build key:t.a (#2), probe targets:[t1.a (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 1.00 ├── HashJoin(Build) │ ├── output columns: [t2.a (#1), t.a (#2)] @@ -273,10 +285,11 @@ HashJoin │ ├── keys is null equal: [false] │ ├── filters: [] │ ├── build join filters: -│ │ └── filter id:0, build key:t.a (#2), probe key:t2.a (#1), filter type:inlist,min_max +│ │ └── filter id:0, build key:t.a (#2), probe targets:[t2.a (#1)@scan1], filter type:bloom,inlist,min_max │ ├── estimated rows: 1.00 │ ├── TableScan(Build) │ │ ├── table: default.join_reorder.t +│ │ ├── scan id: 2 │ │ ├── output columns: [a (#2)] │ │ ├── read rows: 1 │ │ ├── read size: < 1 KiB @@ -287,6 +300,7 @@ HashJoin │ │ └── estimated rows: 1.00 │ └── TableScan(Probe) │ ├── table: default.join_reorder.t2 +│ ├── scan id: 1 │ ├── output columns: [a (#1)] │ ├── read rows: 100 │ ├── read size: < 1 KiB @@ -298,6 +312,7 @@ HashJoin │ └── estimated rows: 100.00 └── TableScan(Probe) ├── table: default.join_reorder.t1 + ├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 10 ├── read size: < 1 KiB @@ -319,7 +334,7 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:1, build key:t.a (#2), probe key:t2.a (#0), filter type:inlist,min_max +│ └── filter id:1, build key:t.a (#2), probe targets:[t2.a (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 1.00 ├── HashJoin(Build) │ ├── output columns: [t1.a (#1), t.a (#2)] @@ -329,10 +344,11 @@ HashJoin │ ├── keys is null equal: [false] │ ├── filters: [] │ ├── build join filters: -│ │ └── filter id:0, build key:t.a (#2), probe key:t1.a (#1), filter type:inlist,min_max +│ │ └── filter id:0, build key:t.a (#2), probe targets:[t1.a (#1)@scan1], filter type:bloom,inlist,min_max │ ├── estimated rows: 1.00 │ ├── TableScan(Build) │ │ ├── table: default.join_reorder.t +│ │ ├── scan id: 2 │ │ ├── output columns: [a (#2)] │ │ ├── read rows: 1 │ │ ├── read size: < 1 KiB @@ -343,6 +359,7 @@ HashJoin │ │ └── estimated rows: 1.00 │ └── TableScan(Probe) │ ├── table: default.join_reorder.t1 +│ ├── scan id: 1 │ ├── output columns: [a (#1)] │ ├── read rows: 10 │ ├── read size: < 1 KiB @@ -354,6 +371,7 @@ HashJoin │ └── estimated rows: 10.00 └── TableScan(Probe) ├── table: default.join_reorder.t2 + ├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 100 ├── read size: < 1 KiB @@ -375,7 +393,7 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:1, build key:t1.a (#2), probe key:t2.a (#0), filter type:inlist,min_max +│ └── filter id:1, build key:t1.a (#2), probe targets:[t2.a (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 1.00 ├── HashJoin(Build) │ ├── output columns: [t1.a (#2), t.a (#1)] @@ -385,10 +403,11 @@ HashJoin │ ├── keys is null equal: [false] │ ├── filters: [] │ ├── build join filters: -│ │ └── filter id:0, build key:t.a (#1), probe key:t1.a (#2), filter type:inlist,min_max +│ │ └── filter id:0, build key:t.a (#1), probe targets:[t1.a (#2)@scan2], filter type:bloom,inlist,min_max │ ├── estimated rows: 1.00 │ ├── TableScan(Build) │ │ ├── table: default.join_reorder.t +│ │ ├── scan id: 1 │ │ ├── output columns: [a (#1)] │ │ ├── read rows: 1 │ │ ├── read size: < 1 KiB @@ -399,6 +418,7 @@ HashJoin │ │ └── estimated rows: 1.00 │ └── TableScan(Probe) │ ├── table: default.join_reorder.t1 +│ ├── scan id: 2 │ ├── output columns: [a (#2)] │ ├── read rows: 10 │ ├── read size: < 1 KiB @@ -410,6 +430,7 @@ HashJoin │ └── estimated rows: 10.00 └── TableScan(Probe) ├── table: default.join_reorder.t2 + ├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 100 ├── read size: < 1 KiB @@ -433,6 +454,7 @@ HashJoin ├── estimated rows: 10.00 ├── TableScan(Build) │ ├── table: default.join_reorder.t +│ ├── scan id: 0 │ ├── output columns: [a (#0)] │ ├── read rows: 1 │ ├── read size: < 1 KiB @@ -447,6 +469,7 @@ HashJoin ├── estimated rows: 10.00 └── TableScan ├── table: default.join_reorder.t1 + ├── scan id: 1 ├── output columns: [a (#1)] ├── read rows: 10 ├── read size: < 1 KiB @@ -473,6 +496,7 @@ HashJoin │ ├── estimated rows: 1.00 │ └── TableScan │ ├── table: default.join_reorder.t +│ ├── scan id: 0 │ ├── output columns: [a (#0)] │ ├── read rows: 1 │ ├── read size: < 1 KiB @@ -483,6 +507,7 @@ HashJoin │ └── estimated rows: 1.00 └── TableScan(Probe) ├── table: default.join_reorder.t1 + ├── scan id: 1 ├── output columns: [a (#1)] ├── read rows: 10 ├── read size: < 1 KiB @@ -503,10 +528,11 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:0, build key:t.a (#0), probe key:t1.a (#1), filter type:inlist,min_max +│ └── filter id:0, build key:t.a (#0), probe targets:[t1.a (#1)@scan1], filter type:bloom,inlist,min_max ├── estimated rows: 1.00 ├── TableScan(Build) │ ├── table: default.join_reorder.t +│ ├── scan id: 0 │ ├── output columns: [a (#0)] │ ├── read rows: 1 │ ├── read size: < 1 KiB @@ -517,6 +543,7 @@ HashJoin │ └── estimated rows: 1.00 └── TableScan(Probe) ├── table: default.join_reorder.t1 + ├── scan id: 1 ├── output columns: [a (#1)] ├── read rows: 10 ├── read size: < 1 KiB @@ -540,6 +567,7 @@ HashJoin ├── estimated rows: 1.00 ├── TableScan(Build) │ ├── table: default.join_reorder.t +│ ├── scan id: 0 │ ├── output columns: [a (#0)] │ ├── read rows: 1 │ ├── read size: < 1 KiB @@ -550,6 +578,7 @@ HashJoin │ └── estimated rows: 1.00 └── TableScan(Probe) ├── table: default.join_reorder.t1 + ├── scan id: 1 ├── output columns: [a (#1)] ├── read rows: 10 ├── read size: < 1 KiB @@ -570,10 +599,11 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:0, build key:t.a (#0), probe key:t1.a (#1), filter type:inlist,min_max +│ └── filter id:0, build key:t.a (#0), probe targets:[t1.a (#1)@scan1], filter type:bloom,inlist,min_max ├── estimated rows: 1.00 ├── TableScan(Build) │ ├── table: default.join_reorder.t +│ ├── scan id: 0 │ ├── output columns: [a (#0)] │ ├── read rows: 1 │ ├── read size: < 1 KiB @@ -584,6 +614,7 @@ HashJoin │ └── estimated rows: 1.00 └── TableScan(Probe) ├── table: default.join_reorder.t1 + ├── scan id: 1 ├── output columns: [a (#1)] ├── read rows: 10 ├── read size: < 1 KiB @@ -607,6 +638,7 @@ HashJoin ├── estimated rows: 10.00 ├── TableScan(Build) │ ├── table: default.join_reorder.t +│ ├── scan id: 0 │ ├── output columns: [a (#0)] │ ├── read rows: 1 │ ├── read size: < 1 KiB @@ -617,6 +649,7 @@ HashJoin │ └── estimated rows: 1.00 └── TableScan(Probe) ├── table: default.join_reorder.t1 + ├── scan id: 1 ├── output columns: [a (#1)] ├── read rows: 10 ├── read size: < 1 KiB diff --git a/tests/sqllogictests/suites/mode/standalone/explain/join_reorder/cycles.test b/tests/sqllogictests/suites/mode/standalone/explain/join_reorder/cycles.test index e46e983860c78..cbb01953da8bc 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/join_reorder/cycles.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/join_reorder/cycles.test @@ -27,7 +27,7 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:1, build key:t.a (#0), probe key:t2.a (#2), filter type:inlist,min_max +│ └── filter id:1, build key:t.a (#0), probe targets:[t2.a (#2)@scan2], filter type:bloom,inlist,min_max ├── estimated rows: 1.00 ├── HashJoin(Build) │ ├── output columns: [t1.a (#1), t.a (#0)] @@ -37,10 +37,11 @@ HashJoin │ ├── keys is null equal: [false] │ ├── filters: [] │ ├── build join filters: -│ │ └── filter id:0, build key:t.a (#0), probe key:t1.a (#1), filter type:inlist,min_max +│ │ └── filter id:0, build key:t.a (#0), probe targets:[t1.a (#1)@scan1], filter type:bloom,inlist,min_max │ ├── estimated rows: 1.00 │ ├── TableScan(Build) │ │ ├── table: default.join_reorder.t +│ │ ├── scan id: 0 │ │ ├── output columns: [a (#0)] │ │ ├── read rows: 1 │ │ ├── read size: < 1 KiB @@ -51,6 +52,7 @@ HashJoin │ │ └── estimated rows: 1.00 │ └── TableScan(Probe) │ ├── table: default.join_reorder.t1 +│ ├── scan id: 1 │ ├── output columns: [a (#1)] │ ├── read rows: 10 │ ├── read size: < 1 KiB @@ -62,6 +64,7 @@ HashJoin │ └── estimated rows: 10.00 └── TableScan(Probe) ├── table: default.join_reorder.t2 + ├── scan id: 2 ├── output columns: [a (#2)] ├── read rows: 100 ├── read size: < 1 KiB @@ -83,7 +86,7 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:1, build key:t.a (#0), probe key:t1.a (#2), filter type:inlist,min_max +│ └── filter id:1, build key:t.a (#0), probe targets:[t1.a (#2)@scan2], filter type:bloom,inlist,min_max ├── estimated rows: 1.00 ├── HashJoin(Build) │ ├── output columns: [t2.a (#1), t.a (#0)] @@ -93,10 +96,11 @@ HashJoin │ ├── keys is null equal: [false] │ ├── filters: [] │ ├── build join filters: -│ │ └── filter id:0, build key:t.a (#0), probe key:t2.a (#1), filter type:inlist,min_max +│ │ └── filter id:0, build key:t.a (#0), probe targets:[t2.a (#1)@scan1], filter type:bloom,inlist,min_max │ ├── estimated rows: 1.00 │ ├── TableScan(Build) │ │ ├── table: default.join_reorder.t +│ │ ├── scan id: 0 │ │ ├── output columns: [a (#0)] │ │ ├── read rows: 1 │ │ ├── read size: < 1 KiB @@ -107,6 +111,7 @@ HashJoin │ │ └── estimated rows: 1.00 │ └── TableScan(Probe) │ ├── table: default.join_reorder.t2 +│ ├── scan id: 1 │ ├── output columns: [a (#1)] │ ├── read rows: 100 │ ├── read size: < 1 KiB @@ -118,6 +123,7 @@ HashJoin │ └── estimated rows: 100.00 └── TableScan(Probe) ├── table: default.join_reorder.t1 + ├── scan id: 2 ├── output columns: [a (#2)] ├── read rows: 10 ├── read size: < 1 KiB @@ -139,7 +145,7 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:1, build key:t2.a (#2), probe key:t1.a (#0), filter type:inlist,min_max +│ └── filter id:1, build key:t2.a (#2), probe targets:[t1.a (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 1.00 ├── HashJoin(Build) │ ├── output columns: [t2.a (#2), t.a (#1)] @@ -149,10 +155,11 @@ HashJoin │ ├── keys is null equal: [false] │ ├── filters: [] │ ├── build join filters: -│ │ └── filter id:0, build key:t.a (#1), probe key:t2.a (#2), filter type:inlist,min_max +│ │ └── filter id:0, build key:t.a (#1), probe targets:[t2.a (#2)@scan2], filter type:bloom,inlist,min_max │ ├── estimated rows: 1.00 │ ├── TableScan(Build) │ │ ├── table: default.join_reorder.t +│ │ ├── scan id: 1 │ │ ├── output columns: [a (#1)] │ │ ├── read rows: 1 │ │ ├── read size: < 1 KiB @@ -163,6 +170,7 @@ HashJoin │ │ └── estimated rows: 1.00 │ └── TableScan(Probe) │ ├── table: default.join_reorder.t2 +│ ├── scan id: 2 │ ├── output columns: [a (#2)] │ ├── read rows: 100 │ ├── read size: < 1 KiB @@ -174,6 +182,7 @@ HashJoin │ └── estimated rows: 100.00 └── TableScan(Probe) ├── table: default.join_reorder.t1 + ├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 10 ├── read size: < 1 KiB @@ -195,7 +204,7 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:1, build key:t.a (#2), probe key:t1.a (#0), filter type:inlist,min_max +│ └── filter id:1, build key:t.a (#2), probe targets:[t1.a (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 1.00 ├── HashJoin(Build) │ ├── output columns: [t2.a (#1), t.a (#2)] @@ -205,10 +214,11 @@ HashJoin │ ├── keys is null equal: [false] │ ├── filters: [] │ ├── build join filters: -│ │ └── filter id:0, build key:t.a (#2), probe key:t2.a (#1), filter type:inlist,min_max +│ │ └── filter id:0, build key:t.a (#2), probe targets:[t2.a (#1)@scan1], filter type:bloom,inlist,min_max │ ├── estimated rows: 1.00 │ ├── TableScan(Build) │ │ ├── table: default.join_reorder.t +│ │ ├── scan id: 2 │ │ ├── output columns: [a (#2)] │ │ ├── read rows: 1 │ │ ├── read size: < 1 KiB @@ -219,6 +229,7 @@ HashJoin │ │ └── estimated rows: 1.00 │ └── TableScan(Probe) │ ├── table: default.join_reorder.t2 +│ ├── scan id: 1 │ ├── output columns: [a (#1)] │ ├── read rows: 100 │ ├── read size: < 1 KiB @@ -230,6 +241,7 @@ HashJoin │ └── estimated rows: 100.00 └── TableScan(Probe) ├── table: default.join_reorder.t1 + ├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 10 ├── read size: < 1 KiB @@ -251,7 +263,7 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:1, build key:t.a (#2), probe key:t2.a (#0), filter type:inlist,min_max +│ └── filter id:1, build key:t.a (#2), probe targets:[t2.a (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 1.00 ├── HashJoin(Build) │ ├── output columns: [t1.a (#1), t.a (#2)] @@ -261,10 +273,11 @@ HashJoin │ ├── keys is null equal: [false] │ ├── filters: [] │ ├── build join filters: -│ │ └── filter id:0, build key:t.a (#2), probe key:t1.a (#1), filter type:inlist,min_max +│ │ └── filter id:0, build key:t.a (#2), probe targets:[t1.a (#1)@scan1], filter type:bloom,inlist,min_max │ ├── estimated rows: 1.00 │ ├── TableScan(Build) │ │ ├── table: default.join_reorder.t +│ │ ├── scan id: 2 │ │ ├── output columns: [a (#2)] │ │ ├── read rows: 1 │ │ ├── read size: < 1 KiB @@ -275,6 +288,7 @@ HashJoin │ │ └── estimated rows: 1.00 │ └── TableScan(Probe) │ ├── table: default.join_reorder.t1 +│ ├── scan id: 1 │ ├── output columns: [a (#1)] │ ├── read rows: 10 │ ├── read size: < 1 KiB @@ -286,6 +300,7 @@ HashJoin │ └── estimated rows: 10.00 └── TableScan(Probe) ├── table: default.join_reorder.t2 + ├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 100 ├── read size: < 1 KiB @@ -307,7 +322,7 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:1, build key:t1.a (#2), probe key:t2.a (#0), filter type:inlist,min_max +│ └── filter id:1, build key:t1.a (#2), probe targets:[t2.a (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 1.00 ├── HashJoin(Build) │ ├── output columns: [t1.a (#2), t.a (#1)] @@ -317,10 +332,11 @@ HashJoin │ ├── keys is null equal: [false] │ ├── filters: [] │ ├── build join filters: -│ │ └── filter id:0, build key:t.a (#1), probe key:t1.a (#2), filter type:inlist,min_max +│ │ └── filter id:0, build key:t.a (#1), probe targets:[t1.a (#2)@scan2], filter type:bloom,inlist,min_max │ ├── estimated rows: 1.00 │ ├── TableScan(Build) │ │ ├── table: default.join_reorder.t +│ │ ├── scan id: 1 │ │ ├── output columns: [a (#1)] │ │ ├── read rows: 1 │ │ ├── read size: < 1 KiB @@ -331,6 +347,7 @@ HashJoin │ │ └── estimated rows: 1.00 │ └── TableScan(Probe) │ ├── table: default.join_reorder.t1 +│ ├── scan id: 2 │ ├── output columns: [a (#2)] │ ├── read rows: 10 │ ├── read size: < 1 KiB @@ -342,6 +359,7 @@ HashJoin │ └── estimated rows: 10.00 └── TableScan(Probe) ├── table: default.join_reorder.t2 + ├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 100 ├── read size: < 1 KiB diff --git a/tests/sqllogictests/suites/mode/standalone/explain/join_reorder/mark.test b/tests/sqllogictests/suites/mode/standalone/explain/join_reorder/mark.test index efa34dee43aa3..3790e9faaf8e8 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/join_reorder/mark.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/join_reorder/mark.test @@ -11,6 +11,7 @@ HashJoin ├── estimated rows: 10000.00 ├── TableScan(Build) │ ├── table: default.system.numbers +│ ├── scan id: 1 │ ├── output columns: [number (#1)] │ ├── read rows: 1000 │ ├── read size: 7.81 KiB @@ -20,6 +21,7 @@ HashJoin │ └── estimated rows: 1000.00 └── TableScan(Probe) ├── table: default.system.numbers + ├── scan id: 0 ├── output columns: [number (#0)] ├── read rows: 10000 ├── read size: 78.12 KiB @@ -39,10 +41,11 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:0, build key:numbers.number (#0), probe key:subquery_1 (#1), filter type:inlist,min_max +│ └── filter id:0, build key:numbers.number (#0), probe targets:[subquery_1 (#1)@scan1], filter type:bloom,inlist,min_max ├── estimated rows: 1000.00 ├── TableScan(Build) │ ├── table: default.system.numbers +│ ├── scan id: 0 │ ├── output columns: [number (#0)] │ ├── read rows: 1000 │ ├── read size: 7.81 KiB @@ -52,6 +55,7 @@ HashJoin │ └── estimated rows: 1000.00 └── TableScan(Probe) ├── table: default.system.numbers + ├── scan id: 1 ├── output columns: [number (#1)] ├── read rows: 10000 ├── read size: 78.12 KiB @@ -115,6 +119,7 @@ EvalScalar │ │ ├── estimated rows: 0.60 │ │ └── TableScan │ │ ├── table: default.default.t1 + │ │ ├── scan id: 1 │ │ ├── output columns: [a (#2)] │ │ ├── read rows: 3 │ │ ├── read size: < 1 KiB @@ -143,6 +148,7 @@ EvalScalar │ ├── estimated rows: 1.33 │ └── TableScan │ ├── table: default.default.t2 + │ ├── scan id: 2 │ ├── output columns: [c (#8)] │ ├── read rows: 4 │ ├── read size: < 1 KiB @@ -153,6 +159,7 @@ EvalScalar │ └── estimated rows: 4.00 └── TableScan(Probe) ├── table: default.default.t2 + ├── scan id: 0 ├── output columns: [c (#0)] ├── read rows: 4 ├── read size: < 1 KiB diff --git a/tests/sqllogictests/suites/mode/standalone/explain/join_reorder/star.test b/tests/sqllogictests/suites/mode/standalone/explain/join_reorder/star.test index f02aff118b009..6b10db4a1194b 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/join_reorder/star.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/join_reorder/star.test @@ -27,7 +27,7 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:1, build key:t.a (#0), probe key:t2.a (#2), filter type:inlist,min_max +│ └── filter id:1, build key:t.a (#0), probe targets:[t2.a (#2)@scan2], filter type:bloom,inlist,min_max ├── estimated rows: 1.00 ├── HashJoin(Build) │ ├── output columns: [t1.a (#1), t.a (#0)] @@ -37,10 +37,11 @@ HashJoin │ ├── keys is null equal: [false] │ ├── filters: [] │ ├── build join filters: -│ │ └── filter id:0, build key:t.a (#0), probe key:t1.a (#1), filter type:inlist,min_max +│ │ └── filter id:0, build key:t.a (#0), probe targets:[t1.a (#1)@scan1], filter type:bloom,inlist,min_max │ ├── estimated rows: 1.00 │ ├── TableScan(Build) │ │ ├── table: default.join_reorder.t +│ │ ├── scan id: 0 │ │ ├── output columns: [a (#0)] │ │ ├── read rows: 1 │ │ ├── read size: < 1 KiB @@ -51,6 +52,7 @@ HashJoin │ │ └── estimated rows: 1.00 │ └── TableScan(Probe) │ ├── table: default.join_reorder.t1 +│ ├── scan id: 1 │ ├── output columns: [a (#1)] │ ├── read rows: 10 │ ├── read size: < 1 KiB @@ -62,6 +64,7 @@ HashJoin │ └── estimated rows: 10.00 └── TableScan(Probe) ├── table: default.join_reorder.t2 + ├── scan id: 2 ├── output columns: [a (#2)] ├── read rows: 100 ├── read size: < 1 KiB @@ -83,7 +86,7 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:1, build key:t.a (#0), probe key:t1.a (#2), filter type:inlist,min_max +│ └── filter id:1, build key:t.a (#0), probe targets:[t1.a (#2)@scan2], filter type:bloom,inlist,min_max ├── estimated rows: 1.00 ├── HashJoin(Build) │ ├── output columns: [t2.a (#1), t.a (#0)] @@ -93,10 +96,11 @@ HashJoin │ ├── keys is null equal: [false] │ ├── filters: [] │ ├── build join filters: -│ │ └── filter id:0, build key:t.a (#0), probe key:t2.a (#1), filter type:inlist,min_max +│ │ └── filter id:0, build key:t.a (#0), probe targets:[t2.a (#1)@scan1], filter type:bloom,inlist,min_max │ ├── estimated rows: 1.00 │ ├── TableScan(Build) │ │ ├── table: default.join_reorder.t +│ │ ├── scan id: 0 │ │ ├── output columns: [a (#0)] │ │ ├── read rows: 1 │ │ ├── read size: < 1 KiB @@ -107,6 +111,7 @@ HashJoin │ │ └── estimated rows: 1.00 │ └── TableScan(Probe) │ ├── table: default.join_reorder.t2 +│ ├── scan id: 1 │ ├── output columns: [a (#1)] │ ├── read rows: 100 │ ├── read size: < 1 KiB @@ -118,6 +123,7 @@ HashJoin │ └── estimated rows: 100.00 └── TableScan(Probe) ├── table: default.join_reorder.t1 + ├── scan id: 2 ├── output columns: [a (#2)] ├── read rows: 10 ├── read size: < 1 KiB @@ -139,7 +145,7 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:1, build key:t2.a (#2), probe key:t1.a (#0), filter type:inlist,min_max +│ └── filter id:1, build key:t2.a (#2), probe targets:[t1.a (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 1.00 ├── HashJoin(Build) │ ├── output columns: [t2.a (#2), t.a (#1)] @@ -149,10 +155,11 @@ HashJoin │ ├── keys is null equal: [false] │ ├── filters: [] │ ├── build join filters: -│ │ └── filter id:0, build key:t.a (#1), probe key:t2.a (#2), filter type:inlist,min_max +│ │ └── filter id:0, build key:t.a (#1), probe targets:[t2.a (#2)@scan2], filter type:bloom,inlist,min_max │ ├── estimated rows: 1.00 │ ├── TableScan(Build) │ │ ├── table: default.join_reorder.t +│ │ ├── scan id: 1 │ │ ├── output columns: [a (#1)] │ │ ├── read rows: 1 │ │ ├── read size: < 1 KiB @@ -163,6 +170,7 @@ HashJoin │ │ └── estimated rows: 1.00 │ └── TableScan(Probe) │ ├── table: default.join_reorder.t2 +│ ├── scan id: 2 │ ├── output columns: [a (#2)] │ ├── read rows: 100 │ ├── read size: < 1 KiB @@ -174,6 +182,7 @@ HashJoin │ └── estimated rows: 100.00 └── TableScan(Probe) ├── table: default.join_reorder.t1 + ├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 10 ├── read size: < 1 KiB @@ -195,7 +204,7 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:1, build key:t.a (#2), probe key:t1.a (#0), filter type:inlist,min_max +│ └── filter id:1, build key:t.a (#2), probe targets:[t1.a (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 1.00 ├── HashJoin(Build) │ ├── output columns: [t2.a (#1), t.a (#2)] @@ -205,10 +214,11 @@ HashJoin │ ├── keys is null equal: [false] │ ├── filters: [] │ ├── build join filters: -│ │ └── filter id:0, build key:t.a (#2), probe key:t2.a (#1), filter type:inlist,min_max +│ │ └── filter id:0, build key:t.a (#2), probe targets:[t2.a (#1)@scan1], filter type:bloom,inlist,min_max │ ├── estimated rows: 1.00 │ ├── TableScan(Build) │ │ ├── table: default.join_reorder.t +│ │ ├── scan id: 2 │ │ ├── output columns: [a (#2)] │ │ ├── read rows: 1 │ │ ├── read size: < 1 KiB @@ -219,6 +229,7 @@ HashJoin │ │ └── estimated rows: 1.00 │ └── TableScan(Probe) │ ├── table: default.join_reorder.t2 +│ ├── scan id: 1 │ ├── output columns: [a (#1)] │ ├── read rows: 100 │ ├── read size: < 1 KiB @@ -230,6 +241,7 @@ HashJoin │ └── estimated rows: 100.00 └── TableScan(Probe) ├── table: default.join_reorder.t1 + ├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 10 ├── read size: < 1 KiB @@ -251,7 +263,7 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:1, build key:t.a (#2), probe key:t2.a (#0), filter type:inlist,min_max +│ └── filter id:1, build key:t.a (#2), probe targets:[t2.a (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 1.00 ├── HashJoin(Build) │ ├── output columns: [t1.a (#1), t.a (#2)] @@ -261,10 +273,11 @@ HashJoin │ ├── keys is null equal: [false] │ ├── filters: [] │ ├── build join filters: -│ │ └── filter id:0, build key:t.a (#2), probe key:t1.a (#1), filter type:inlist,min_max +│ │ └── filter id:0, build key:t.a (#2), probe targets:[t1.a (#1)@scan1], filter type:bloom,inlist,min_max │ ├── estimated rows: 1.00 │ ├── TableScan(Build) │ │ ├── table: default.join_reorder.t +│ │ ├── scan id: 2 │ │ ├── output columns: [a (#2)] │ │ ├── read rows: 1 │ │ ├── read size: < 1 KiB @@ -275,6 +288,7 @@ HashJoin │ │ └── estimated rows: 1.00 │ └── TableScan(Probe) │ ├── table: default.join_reorder.t1 +│ ├── scan id: 1 │ ├── output columns: [a (#1)] │ ├── read rows: 10 │ ├── read size: < 1 KiB @@ -286,6 +300,7 @@ HashJoin │ └── estimated rows: 10.00 └── TableScan(Probe) ├── table: default.join_reorder.t2 + ├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 100 ├── read size: < 1 KiB @@ -307,7 +322,7 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:1, build key:t1.a (#2), probe key:t2.a (#0), filter type:inlist,min_max +│ └── filter id:1, build key:t1.a (#2), probe targets:[t2.a (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 1.00 ├── HashJoin(Build) │ ├── output columns: [t1.a (#2), t.a (#1)] @@ -317,10 +332,11 @@ HashJoin │ ├── keys is null equal: [false] │ ├── filters: [] │ ├── build join filters: -│ │ └── filter id:0, build key:t.a (#1), probe key:t1.a (#2), filter type:inlist,min_max +│ │ └── filter id:0, build key:t.a (#1), probe targets:[t1.a (#2)@scan2], filter type:bloom,inlist,min_max │ ├── estimated rows: 1.00 │ ├── TableScan(Build) │ │ ├── table: default.join_reorder.t +│ │ ├── scan id: 1 │ │ ├── output columns: [a (#1)] │ │ ├── read rows: 1 │ │ ├── read size: < 1 KiB @@ -331,6 +347,7 @@ HashJoin │ │ └── estimated rows: 1.00 │ └── TableScan(Probe) │ ├── table: default.join_reorder.t1 +│ ├── scan id: 2 │ ├── output columns: [a (#2)] │ ├── read rows: 10 │ ├── read size: < 1 KiB @@ -342,6 +359,7 @@ HashJoin │ └── estimated rows: 10.00 └── TableScan(Probe) ├── table: default.join_reorder.t2 + ├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 100 ├── read size: < 1 KiB diff --git a/tests/sqllogictests/suites/mode/standalone/explain/lazy_read.test b/tests/sqllogictests/suites/mode/standalone/explain/lazy_read.test index 481529c97f2c6..4a022cb88df24 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/lazy_read.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/lazy_read.test @@ -25,6 +25,7 @@ RowFetch ├── estimated rows: 0.00 └── TableScan ├── table: default.default.t_lazy + ├── scan id: 0 ├── output columns: [a (#0), _row_id (#7)] ├── read rows: 0 ├── read size: 0 @@ -51,6 +52,7 @@ RowFetch ├── estimated rows: 0.00 └── TableScan ├── table: default.default.t_lazy + ├── scan id: 0 ├── output columns: [a (#0), _row_id (#7)] ├── read rows: 0 ├── read size: 0 @@ -69,6 +71,7 @@ Limit ├── estimated rows: 0.00 └── TableScan ├── table: default.default.t_lazy + ├── scan id: 0 ├── output columns: [a (#0), b (#1), c (#2), d (#3), e (#6)] ├── read rows: 0 ├── read size: 0 @@ -104,6 +107,7 @@ RowFetch ├── estimated rows: 0.00 └── TableScan ├── table: default.default.t_11831 + ├── scan id: 0 ├── output columns: [uid (#0), time (#3), _row_id (#4)] ├── read rows: 0 ├── read size: 0 @@ -132,6 +136,7 @@ Limit ├── estimated rows: 0.00 └── TableScan ├── table: default.default.t_lazy + ├── scan id: 0 ├── output columns: [a (#0), b (#1), c (#2), d (#3), e (#6)] ├── read rows: 0 ├── read size: 0 @@ -154,6 +159,7 @@ Limit ├── estimated rows: 0.00 └── TableScan ├── table: default.default.t_lazy + ├── scan id: 0 ├── output columns: [a (#0), b (#1), c (#2), d (#3), e (#6)] ├── read rows: 0 ├── read size: 0 @@ -172,6 +178,7 @@ Limit ├── estimated rows: 0.00 └── TableScan ├── table: default.default.t_lazy + ├── scan id: 0 ├── output columns: [a (#0), b (#1), c (#2), d (#3), e (#6)] ├── read rows: 0 ├── read size: 0 @@ -204,6 +211,7 @@ Limit ├── estimated rows: 0.00 └── TableScan ├── table: default.default.t_lazy + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -253,6 +261,7 @@ Limit ├── estimated rows: 0.00 └── TableScan ├── table: default.default.t_lazy + ├── scan id: 0 ├── output columns: [a (#0), b (#1), _row_id (#9)] ├── read rows: 0 ├── read size: 0 diff --git a/tests/sqllogictests/suites/mode/standalone/explain/materialized_cte.test b/tests/sqllogictests/suites/mode/standalone/explain/materialized_cte.test index 96297507145c5..5c322adddcb14 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/materialized_cte.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/materialized_cte.test @@ -9,10 +9,11 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:0, build key:t2.b (#1), probe key:t1.a (#0), filter type:inlist,min_max +│ └── filter id:0, build key:t2.b (#1), probe targets:[t1.a (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 10.00 ├── TableScan(Build) │ ├── table: default.default.t1 +│ ├── scan id: 1 │ ├── output columns: [a (#1)] │ ├── read rows: 10 │ ├── read size: < 1 KiB @@ -23,6 +24,7 @@ HashJoin │ └── estimated rows: 10.00 └── TableScan(Probe) ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 10 ├── read size: < 1 KiB @@ -44,10 +46,11 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:0, build key:t2.b (#1), probe key:t1.a (#0), filter type:inlist,min_max +│ └── filter id:0, build key:t2.b (#1), probe targets:[t1.a (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 10.00 ├── TableScan(Build) │ ├── table: default.default.t2 +│ ├── scan id: 1 │ ├── output columns: [b (#1)] │ ├── read rows: 10 │ ├── read size: < 1 KiB @@ -58,6 +61,7 @@ HashJoin │ └── estimated rows: 10.00 └── TableScan(Probe) ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 10 ├── read size: < 1 KiB diff --git a/tests/sqllogictests/suites/mode/standalone/explain/merge_into.test b/tests/sqllogictests/suites/mode/standalone/explain/merge_into.test index 9347617b8254a..b784f46d3f030 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/merge_into.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/merge_into.test @@ -42,6 +42,7 @@ CommitSink ├── EmptyResultScan(Build) └── TableScan(Probe) ├── table: default.default.employees2 + ├── scan id: 0 ├── output columns: [employee_id (#0), employee_name (#1), department (#2)] ├── read rows: 4 ├── read size: < 1 KiB @@ -79,6 +80,7 @@ CommitSink ├── estimated rows: 4.00 ├── TableScan(Build) │ ├── table: default.default.salaries2 + │ ├── scan id: 1 │ ├── output columns: [employee_id (#3), _row_id (#5)] │ ├── read rows: 4 │ ├── read size: < 1 KiB @@ -89,6 +91,7 @@ CommitSink │ └── estimated rows: 4.00 └── TableScan(Probe) ├── table: default.default.employees2 + ├── scan id: 0 ├── output columns: [employee_id (#0), employee_name (#1), department (#2)] ├── read rows: 4 ├── read size: < 1 KiB @@ -122,10 +125,11 @@ CommitSink ├── keys is null equal: [false] ├── filters: [] ├── build join filters: - │ └── filter id:0, build key:employees2.employee_id (#0), probe key:salaries2.employee_id (#3), filter type:inlist,min_max + │ └── filter id:0, build key:employees2.employee_id (#0), probe targets:[salaries2.employee_id (#3)@scan1], filter type:bloom,inlist,min_max ├── estimated rows: 4.00 ├── TableScan(Build) │ ├── table: default.default.employees2 + │ ├── scan id: 0 │ ├── output columns: [employee_id (#0), employee_name (#1), department (#2)] │ ├── read rows: 4 │ ├── read size: < 1 KiB @@ -136,6 +140,7 @@ CommitSink │ └── estimated rows: 4.00 └── TableScan(Probe) ├── table: default.default.salaries2 + ├── scan id: 1 ├── output columns: [employee_id (#3), _row_id (#5)] ├── read rows: 6 ├── read size: < 1 KiB @@ -167,10 +172,11 @@ CommitSink ├── keys is null equal: [false] ├── filters: [] ├── build join filters: - │ └── filter id:0, build key:employees2.employee_id (#0), probe key:salaries2.employee_id (#3), filter type:inlist,min_max + │ └── filter id:0, build key:employees2.employee_id (#0), probe targets:[salaries2.employee_id (#3)@scan1], filter type:bloom,inlist,min_max ├── estimated rows: 4.00 ├── TableScan(Build) │ ├── table: default.default.employees2 + │ ├── scan id: 0 │ ├── output columns: [employee_id (#0), employee_name (#1), department (#2)] │ ├── read rows: 4 │ ├── read size: < 1 KiB @@ -181,6 +187,7 @@ CommitSink │ └── estimated rows: 4.00 └── TableScan(Probe) ├── table: default.default.salaries2 + ├── scan id: 1 ├── output columns: [employee_id (#3), _row_id (#5)] ├── read rows: 6 ├── read size: < 1 KiB @@ -230,6 +237,7 @@ CommitSink ├── estimated rows: 0.00 ├── TableScan(Build) │ ├── table: default.default.column_only_optimization_target + │ ├── scan id: 1 │ ├── output columns: [a (#2), _row_id (#4)] │ ├── read rows: 0 │ ├── read size: 0 @@ -239,6 +247,7 @@ CommitSink │ └── estimated rows: 0.00 └── TableScan(Probe) ├── table: default.default.column_only_optimization_source + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -270,6 +279,7 @@ CommitSink ├── estimated rows: 0.00 ├── TableScan(Build) │ ├── table: default.default.column_only_optimization_target + │ ├── scan id: 1 │ ├── output columns: [a (#2), _row_id (#4)] │ ├── read rows: 0 │ ├── read size: 0 @@ -279,6 +289,7 @@ CommitSink │ └── estimated rows: 0.00 └── TableScan(Probe) ├── table: default.default.column_only_optimization_source + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -313,6 +324,7 @@ CommitSink ├── estimated rows: 0.00 ├── TableScan(Build) │ ├── table: default.default.column_only_optimization_target + │ ├── scan id: 1 │ ├── output columns: [a (#2), _row_id (#4)] │ ├── read rows: 0 │ ├── read size: 0 @@ -322,6 +334,7 @@ CommitSink │ └── estimated rows: 0.00 └── TableScan(Probe) ├── table: default.default.column_only_optimization_source + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -353,6 +366,7 @@ CommitSink ├── estimated rows: 0.00 ├── TableScan(Build) │ ├── table: default.default.column_only_optimization_target + │ ├── scan id: 1 │ ├── output columns: [a (#2), _row_id (#4)] │ ├── read rows: 0 │ ├── read size: 0 @@ -362,6 +376,7 @@ CommitSink │ └── estimated rows: 0.00 └── TableScan(Probe) ├── table: default.default.column_only_optimization_source + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 diff --git a/tests/sqllogictests/suites/mode/standalone/explain/merge_limit.test b/tests/sqllogictests/suites/mode/standalone/explain/merge_limit.test index cdc30b6110c9f..b4e91bc9a823a 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/merge_limit.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/merge_limit.test @@ -15,6 +15,7 @@ Limit ├── estimated rows: 0.00 └── TableScan ├── table: default.default.test_table + ├── scan id: 0 ├── output columns: [id (#0), name (#1), value (#2)] ├── read rows: 0 ├── read size: 0 @@ -34,6 +35,7 @@ Limit ├── estimated rows: 0.00 └── TableScan ├── table: default.default.test_table + ├── scan id: 0 ├── output columns: [id (#0), name (#1), value (#2)] ├── read rows: 0 ├── read size: 0 @@ -53,6 +55,7 @@ Limit ├── estimated rows: 0.00 └── TableScan ├── table: default.default.test_table + ├── scan id: 0 ├── output columns: [id (#0), name (#1), value (#2)] ├── read rows: 0 ├── read size: 0 @@ -76,6 +79,7 @@ Limit ├── estimated rows: 0.00 └── TableScan ├── table: default.default.test_table + ├── scan id: 0 ├── output columns: [id (#0), name (#1), value (#2)] ├── read rows: 0 ├── read size: 0 @@ -115,6 +119,7 @@ Limit ├── estimated rows: 0.00 └── TableScan ├── table: default.default.test_table + ├── scan id: 0 ├── output columns: [id (#0), _row_id (#3)] ├── read rows: 0 ├── read size: 0 @@ -144,6 +149,7 @@ RowFetch ├── estimated rows: 0.00 └── TableScan ├── table: default.default.test_table + ├── scan id: 0 ├── output columns: [id (#0), _row_id (#3)] ├── read rows: 0 ├── read size: 0 diff --git a/tests/sqllogictests/suites/mode/standalone/explain/multi_table_insert.test b/tests/sqllogictests/suites/mode/standalone/explain/multi_table_insert.test index 26607dd350242..2dc647c169d7b 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/multi_table_insert.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/multi_table_insert.test @@ -38,6 +38,7 @@ Commit ├── Duplicate data to 4 branch └── TableScan ├── table: default.default.orders_placed + ├── scan id: 0 ├── output columns: [order_id (#0), order_amount (#2)] ├── read rows: 5 ├── read size: < 1 KiB diff --git a/tests/sqllogictests/suites/mode/standalone/explain/nullable_prune.test b/tests/sqllogictests/suites/mode/standalone/explain/nullable_prune.test index 7fbfed14105fb..1f8e17fde0de6 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/nullable_prune.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/nullable_prune.test @@ -18,6 +18,7 @@ explain select * from t_nullable_prune ---- TableScan ├── table: default.default.t_nullable_prune +├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 6 ├── read size: < 1 KiB @@ -36,6 +37,7 @@ Filter ├── estimated rows: 3.00 └── TableScan ├── table: default.default.t_nullable_prune + ├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 3 ├── read size: < 1 KiB @@ -54,6 +56,7 @@ Filter ├── estimated rows: 3.00 └── TableScan ├── table: default.default.t_nullable_prune + ├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 3 ├── read size: < 1 KiB diff --git a/tests/sqllogictests/suites/mode/standalone/explain/outer_to_inner.test b/tests/sqllogictests/suites/mode/standalone/explain/outer_to_inner.test index ade787692b268..e2668d5f652e4 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/outer_to_inner.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/outer_to_inner.test @@ -22,7 +22,7 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:0, build key:t2.b (#3), probe key:t1.b (#1), filter type:bloom,inlist,min_max +│ └── filter id:0, build key:t2.b (#3), probe targets:[t1.b (#1)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 0.00 ├── Filter(Build) │ ├── output columns: [t2.a (#2), t2.b (#3)] @@ -30,6 +30,7 @@ HashJoin │ ├── estimated rows: 0.00 │ └── TableScan │ ├── table: default.default.t2 +│ ├── scan id: 1 │ ├── output columns: [a (#2), b (#3)] │ ├── read rows: 0 │ ├── read size: 0 @@ -39,6 +40,7 @@ HashJoin │ └── estimated rows: 0.00 └── TableScan(Probe) ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -73,7 +75,7 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:0, build key:t2.b (#3), probe key:t1.b (#1), filter type:inlist,min_max +│ └── filter id:0, build key:t2.b (#3), probe targets:[t1.b (#1)@scan0], filter type:inlist,min_max ├── estimated rows: 0.00 ├── Filter(Build) │ ├── output columns: [t2.a (#2), t2.b (#3)] @@ -81,6 +83,7 @@ HashJoin │ ├── estimated rows: 0.00 │ └── TableScan │ ├── table: default.default.t2 +│ ├── scan id: 1 │ ├── output columns: [a (#2), b (#3)] │ ├── read rows: 0 │ ├── read size: 0 @@ -90,6 +93,7 @@ HashJoin │ └── estimated rows: 0.00 └── TableScan(Probe) ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -123,7 +127,7 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:0, build key:t2.b (#3), probe key:t1.b (#1), filter type:inlist +│ └── filter id:0, build key:t2.b (#3), probe targets:[t1.b (#1)@scan0], filter type:inlist ├── estimated rows: 0.00 ├── Filter(Build) │ ├── output columns: [t2.a (#2), t2.b (#3)] @@ -131,6 +135,7 @@ HashJoin │ ├── estimated rows: 0.00 │ └── TableScan │ ├── table: default.default.t2 +│ ├── scan id: 1 │ ├── output columns: [a (#2), b (#3)] │ ├── read rows: 0 │ ├── read size: 0 @@ -140,6 +145,7 @@ HashJoin │ └── estimated rows: 0.00 └── TableScan(Probe) ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -173,7 +179,7 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:0, build key:t2.b (#3), probe key:t1.b (#1), filter type:bloom,inlist,min_max +│ └── filter id:0, build key:t2.b (#3), probe targets:[t1.b (#1)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 0.00 ├── Filter(Build) │ ├── output columns: [t2.a (#2), t2.b (#3)] @@ -181,6 +187,7 @@ HashJoin │ ├── estimated rows: 0.00 │ └── TableScan │ ├── table: default.default.t2 +│ ├── scan id: 1 │ ├── output columns: [a (#2), b (#3)] │ ├── read rows: 0 │ ├── read size: 0 @@ -190,6 +197,7 @@ HashJoin │ └── estimated rows: 0.00 └── TableScan(Probe) ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 diff --git a/tests/sqllogictests/suites/mode/standalone/explain/prewhere_optimization.test b/tests/sqllogictests/suites/mode/standalone/explain/prewhere_optimization.test index 7540375271ee4..0e0f60151552d 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/prewhere_optimization.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/prewhere_optimization.test @@ -13,6 +13,7 @@ Filter ├── estimated rows: 0.00 └── TableScan ├── table: default.default.t_where_optimizer + ├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 0 ├── read size: 0 @@ -30,6 +31,7 @@ Filter ├── estimated rows: 0.00 └── TableScan ├── table: default.default.t_where_optimizer + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -47,6 +49,7 @@ Filter ├── estimated rows: 0.00 └── TableScan ├── table: default.default.t_where_optimizer + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -64,6 +67,7 @@ Filter ├── estimated rows: 0.00 └── TableScan ├── table: default.default.t_where_optimizer + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -81,6 +85,7 @@ Filter ├── estimated rows: 0.00 └── TableScan ├── table: default.default.t_where_optimizer + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -98,6 +103,7 @@ Filter ├── estimated rows: 0.00 └── TableScan ├── table: default.default.t_where_optimizer + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -121,6 +127,7 @@ Filter ├── estimated rows: 0.00 └── TableScan ├── table: default.default.t_where_optimizer + ├── scan id: 0 ├── output columns: [id (#0), s (#1), s:a (#3)] ├── read rows: 0 ├── read size: 0 diff --git a/tests/sqllogictests/suites/mode/standalone/explain/project_set.test b/tests/sqllogictests/suites/mode/standalone/explain/project_set.test index ab6f6d1a5ae17..42c1561d8c404 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/project_set.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/project_set.test @@ -28,6 +28,7 @@ AggregateFinal ├── set returning functions: unnest(fold_count.id (#0)) └── TableScan ├── table: default.default.fold_count + ├── scan id: 0 ├── output columns: [id (#0)] ├── read rows: 1 ├── read size: < 1 KiB @@ -58,6 +59,7 @@ ProjectSet ├── set returning functions: unnest([1, 2, 3]) └── TableScan ├── table: default.system.numbers + ├── scan id: 0 ├── output columns: [number (#0)] ├── read rows: 10 ├── read size: < 1 KiB @@ -75,6 +77,7 @@ ProjectSet ├── set returning functions: unnest(CAST(array(1, 2, 3, numbers.number (#0)) AS Array(UInt64 NULL))) └── TableScan ├── table: default.system.numbers + ├── scan id: 0 ├── output columns: [number (#0)] ├── read rows: 10 ├── read size: < 1 KiB @@ -102,6 +105,7 @@ EvalScalar ├── set returning functions: flatten(1, 5)(t.b (#1)) └── TableScan ├── table: default.project_set.t + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -127,6 +131,7 @@ EvalScalar ├── set returning functions: flatten(1, 2, 5)(t.b (#1)) └── TableScan ├── table: default.project_set.t + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -148,6 +153,7 @@ EvalScalar ├── set returning functions: json_each(t.b (#1)), unnest(t.b (#1)) └── TableScan ├── table: default.project_set.t + ├── scan id: 0 ├── output columns: [b (#1)] ├── read rows: 0 ├── read size: 0 @@ -169,6 +175,7 @@ HashJoin ├── estimated rows: 450.00 ├── TableScan(Build) │ ├── table: default.system.numbers +│ ├── scan id: 1 │ ├── output columns: [number (#3)] │ ├── read rows: 15 │ ├── read size: < 1 KiB @@ -186,6 +193,7 @@ HashJoin ├── set returning functions: unnest([1, 2, 3]) └── TableScan ├── table: default.system.numbers + ├── scan id: 0 ├── output columns: [number (#0)] ├── read rows: 10 ├── read size: < 1 KiB diff --git a/tests/sqllogictests/suites/mode/standalone/explain/prune_column.test b/tests/sqllogictests/suites/mode/standalone/explain/prune_column.test index 70e426098def0..7ad598f507679 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/prune_column.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/prune_column.test @@ -3,6 +3,7 @@ explain select * from (select a from (select number as a, number + 1 as b from n ---- TableScan ├── table: default.system.numbers +├── scan id: 0 ├── output columns: [number (#0)] ├── read rows: 1 ├── read size: < 1 KiB @@ -25,6 +26,7 @@ AggregateFinal ├── estimated rows: 1.00 └── TableScan ├── table: default.system.numbers + ├── scan id: 0 ├── output columns: [number (#0)] ├── read rows: 1 ├── read size: < 1 KiB @@ -61,6 +63,7 @@ Limit ├── estimated rows: 0.50 └── TableScan ├── table: default.system.numbers + ├── scan id: 0 ├── output columns: [number (#0)] ├── read rows: 1 ├── read size: < 1 KiB @@ -90,6 +93,7 @@ HashJoin │ ├── estimated rows: 0.20 │ └── TableScan │ ├── table: default.system.numbers +│ ├── scan id: 0 │ ├── output columns: [number (#0)] │ ├── read rows: 1 │ ├── read size: < 1 KiB @@ -103,6 +107,7 @@ HashJoin ├── estimated rows: 1.00 └── TableScan ├── table: default.system.numbers + ├── scan id: 1 ├── output columns: [number (#5)] ├── read rows: 1 ├── read size: < 1 KiB @@ -149,6 +154,7 @@ HashJoin │ │ ├── estimated rows: 0.20 │ │ └── TableScan │ │ ├── table: default.system.numbers +│ │ ├── scan id: 1 │ │ ├── output columns: [number (#3)] │ │ ├── read rows: 1 │ │ ├── read size: < 1 KiB @@ -162,6 +168,7 @@ HashJoin │ ├── estimated rows: 1.00 │ └── TableScan │ ├── table: default.system.numbers +│ ├── scan id: 2 │ ├── output columns: [number (#8)] │ ├── read rows: 1 │ ├── read size: < 1 KiB @@ -175,6 +182,7 @@ HashJoin ├── estimated rows: 2.00 └── TableScan ├── table: default.system.numbers + ├── scan id: 0 ├── output columns: [number (#0)] ├── read rows: 2 ├── read size: < 1 KiB @@ -192,6 +200,7 @@ Sort(Single) ├── estimated rows: 0.00 └── TableScan ├── table: default.system.functions + ├── scan id: 0 ├── output columns: [name (#0), example (#4)] ├── read rows: 0 ├── read size: 0 @@ -218,6 +227,7 @@ HashJoin │ └── DummyTableScan └── TableScan(Probe) ├── table: default.system.numbers + ├── scan id: 0 ├── output columns: [number (#0)] ├── read rows: 10 ├── read size: < 1 KiB @@ -254,6 +264,7 @@ AggregateFinal ├── estimated rows: 1.00 └── TableScan ├── table: default.default.t + ├── scan id: 0 ├── output columns: [b (#1)] ├── read rows: 2 ├── read size: < 1 KiB diff --git a/tests/sqllogictests/suites/mode/standalone/explain/push_down_filter/push_down_filter_join/push_down_filter_join_full_outer.test b/tests/sqllogictests/suites/mode/standalone/explain/push_down_filter/push_down_filter_join/push_down_filter_join_full_outer.test index 42a3fba6676d0..0814a631f57a7 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/push_down_filter/push_down_filter_join/push_down_filter_join_full_outer.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/push_down_filter/push_down_filter_join/push_down_filter_join_full_outer.test @@ -35,6 +35,7 @@ HashJoin │ ├── estimated rows: 3.00 │ └── TableScan │ ├── table: default.default.t2 +│ ├── scan id: 1 │ ├── output columns: [a (#2), b (#3)] │ ├── read rows: 3 │ ├── read size: < 1 KiB @@ -49,6 +50,7 @@ HashJoin ├── estimated rows: 4.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 4 ├── read size: < 1 KiB @@ -70,7 +72,7 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:0, build key:t2.a (#2), probe key:t1.a (#0), filter type:inlist,min_max +│ └── filter id:0, build key:t2.a (#2), probe targets:[t1.a (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 4.00 ├── Filter(Build) │ ├── output columns: [t2.a (#2), t2.b (#3)] @@ -78,6 +80,7 @@ HashJoin │ ├── estimated rows: 3.00 │ └── TableScan │ ├── table: default.default.t2 +│ ├── scan id: 1 │ ├── output columns: [a (#2), b (#3)] │ ├── read rows: 3 │ ├── read size: < 1 KiB @@ -92,6 +95,7 @@ HashJoin ├── estimated rows: 4.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 4 ├── read size: < 1 KiB @@ -116,6 +120,7 @@ HashJoin ├── estimated rows: 4.00 ├── TableScan(Build) │ ├── table: default.default.t2 +│ ├── scan id: 1 │ ├── output columns: [a (#2), b (#3)] │ ├── read rows: 3 │ ├── read size: < 1 KiB @@ -130,6 +135,7 @@ HashJoin ├── estimated rows: 4.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 4 ├── read size: < 1 KiB @@ -151,7 +157,7 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:0, build key:t2.a (#2), probe key:t1.a (#0), filter type:inlist,min_max +│ └── filter id:0, build key:t2.a (#2), probe targets:[t1.a (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 4.00 ├── Filter(Build) │ ├── output columns: [t2.a (#2), t2.b (#3)] @@ -159,6 +165,7 @@ HashJoin │ ├── estimated rows: 3.00 │ └── TableScan │ ├── table: default.default.t2 +│ ├── scan id: 1 │ ├── output columns: [a (#2), b (#3)] │ ├── read rows: 3 │ ├── read size: < 1 KiB @@ -169,6 +176,7 @@ HashJoin │ └── estimated rows: 3.00 └── TableScan(Probe) ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 4 ├── read size: < 1 KiB @@ -191,7 +199,7 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:0, build key:t2.a (#2), probe key:t1.a (#0), filter type:inlist,min_max +│ └── filter id:0, build key:t2.a (#2), probe targets:[t1.a (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 4.00 ├── Filter(Build) │ ├── output columns: [t2.a (#2), t2.b (#3)] @@ -199,6 +207,7 @@ HashJoin │ ├── estimated rows: 3.00 │ └── TableScan │ ├── table: default.default.t2 +│ ├── scan id: 1 │ ├── output columns: [a (#2), b (#3)] │ ├── read rows: 3 │ ├── read size: < 1 KiB @@ -213,6 +222,7 @@ HashJoin ├── estimated rows: 4.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 4 ├── read size: < 1 KiB @@ -235,7 +245,7 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:0, build key:t2.a (#2), probe key:t1.a (#0), filter type:inlist,min_max +│ └── filter id:0, build key:t2.a (#2), probe targets:[t1.a (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 4.00 ├── Filter(Build) │ ├── output columns: [t2.a (#2), t2.b (#3)] @@ -243,6 +253,7 @@ HashJoin │ ├── estimated rows: 3.00 │ └── TableScan │ ├── table: default.default.t2 +│ ├── scan id: 1 │ ├── output columns: [a (#2), b (#3)] │ ├── read rows: 3 │ ├── read size: < 1 KiB @@ -257,6 +268,7 @@ HashJoin ├── estimated rows: 4.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 4 ├── read size: < 1 KiB @@ -281,6 +293,7 @@ HashJoin ├── estimated rows: 4.00 ├── TableScan(Build) │ ├── table: default.default.t2 +│ ├── scan id: 1 │ ├── output columns: [a (#2), b (#3)] │ ├── read rows: 3 │ ├── read size: < 1 KiB @@ -291,6 +304,7 @@ HashJoin │ └── estimated rows: 3.00 └── TableScan(Probe) ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 4 ├── read size: < 1 KiB diff --git a/tests/sqllogictests/suites/mode/standalone/explain/push_down_filter/push_down_filter_join/push_down_filter_join_inner.test b/tests/sqllogictests/suites/mode/standalone/explain/push_down_filter/push_down_filter_join/push_down_filter_join_inner.test index e797a5ea6e62b..434c15403dafb 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/push_down_filter/push_down_filter_join/push_down_filter_join_inner.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/push_down_filter/push_down_filter_join/push_down_filter_join_inner.test @@ -29,7 +29,7 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:0, build key:t2.a (#2), probe key:t1.a (#0), filter type:bloom,inlist,min_max +│ └── filter id:0, build key:t2.a (#2), probe targets:[t1.a (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 0.00 ├── Filter(Build) │ ├── output columns: [t2.a (#2), t2.b (#3)] @@ -37,6 +37,7 @@ HashJoin │ ├── estimated rows: 0.00 │ └── TableScan │ ├── table: default.default.t2 +│ ├── scan id: 1 │ ├── output columns: [a (#2), b (#3)] │ ├── read rows: 0 │ ├── read size: 0 @@ -51,6 +52,7 @@ HashJoin ├── estimated rows: 0.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -77,7 +79,7 @@ Filter ├── keys is null equal: [false] ├── filters: [] ├── build join filters: - │ └── filter id:0, build key:t2.a (#2), probe key:t1.a (#0), filter type:inlist,min_max + │ └── filter id:0, build key:t2.a (#2), probe targets:[t1.a (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 3.56 ├── Filter(Build) │ ├── output columns: [t2.a (#2), t2.b (#3)] @@ -85,6 +87,7 @@ Filter │ ├── estimated rows: 3.00 │ └── TableScan │ ├── table: default.default.t2 + │ ├── scan id: 1 │ ├── output columns: [a (#2), b (#3)] │ ├── read rows: 3 │ ├── read size: < 1 KiB @@ -99,6 +102,7 @@ Filter ├── estimated rows: 3.56 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 4 ├── read size: < 1 KiB diff --git a/tests/sqllogictests/suites/mode/standalone/explain/push_down_filter/push_down_filter_join/push_down_filter_join_left_outer.test b/tests/sqllogictests/suites/mode/standalone/explain/push_down_filter/push_down_filter_join/push_down_filter_join_left_outer.test index fe098ad396845..df62054ee9451 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/push_down_filter/push_down_filter_join/push_down_filter_join_left_outer.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/push_down_filter/push_down_filter_join/push_down_filter_join_left_outer.test @@ -38,6 +38,7 @@ HashJoin │ ├── estimated rows: 3.00 │ └── TableScan │ ├── table: default.default.t2 +│ ├── scan id: 1 │ ├── output columns: [a (#2), b (#3)] │ ├── read rows: 3 │ ├── read size: < 1 KiB @@ -52,6 +53,7 @@ HashJoin ├── estimated rows: 4.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 4 ├── read size: < 1 KiB @@ -75,6 +77,7 @@ HashJoin ├── estimated rows: 4.00 ├── TableScan(Build) │ ├── table: default.default.t2 +│ ├── scan id: 1 │ ├── output columns: [a (#2), b (#3)] │ ├── read rows: 3 │ ├── read size: < 1 KiB @@ -85,6 +88,7 @@ HashJoin │ └── estimated rows: 3.00 └── TableScan(Probe) ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 4 ├── read size: < 1 KiB @@ -108,6 +112,7 @@ HashJoin ├── estimated rows: 4.00 ├── TableScan(Build) │ ├── table: default.default.t2 +│ ├── scan id: 1 │ ├── output columns: [a (#2), b (#3)] │ ├── read rows: 3 │ ├── read size: < 1 KiB @@ -122,6 +127,7 @@ HashJoin ├── estimated rows: 4.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 4 ├── read size: < 1 KiB @@ -143,7 +149,7 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:0, build key:t2.a (#2), probe key:t1.a (#0), filter type:inlist,min_max +│ └── filter id:0, build key:t2.a (#2), probe targets:[t1.a (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 4.00 ├── Filter(Build) │ ├── output columns: [t2.a (#2), t2.b (#3)] @@ -151,6 +157,7 @@ HashJoin │ ├── estimated rows: 3.00 │ └── TableScan │ ├── table: default.default.t2 +│ ├── scan id: 1 │ ├── output columns: [a (#2), b (#3)] │ ├── read rows: 3 │ ├── read size: < 1 KiB @@ -165,6 +172,7 @@ HashJoin ├── estimated rows: 4.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 4 ├── read size: < 1 KiB @@ -187,7 +195,7 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:0, build key:t2.a (#2), probe key:t1.a (#0), filter type:inlist,min_max +│ └── filter id:0, build key:t2.a (#2), probe targets:[t1.a (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 4.00 ├── Filter(Build) │ ├── output columns: [t2.a (#2), t2.b (#3)] @@ -195,6 +203,7 @@ HashJoin │ ├── estimated rows: 3.00 │ └── TableScan │ ├── table: default.default.t2 +│ ├── scan id: 1 │ ├── output columns: [a (#2), b (#3)] │ ├── read rows: 3 │ ├── read size: < 1 KiB @@ -205,6 +214,7 @@ HashJoin │ └── estimated rows: 3.00 └── TableScan(Probe) ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 4 ├── read size: < 1 KiB @@ -247,6 +257,7 @@ HashJoin ├── estimated rows: 0.00 ├── TableScan(Build) │ ├── table: default.default.j1 +│ ├── scan id: 1 │ ├── output columns: [] │ ├── read rows: 0 │ ├── read size: 0 @@ -260,6 +271,7 @@ HashJoin ├── estimated rows: 0.00 └── TableScan ├── table: default.default.m1 + ├── scan id: 0 ├── output columns: [id (#0), context (#1)] ├── read rows: 0 ├── read size: 0 @@ -298,7 +310,7 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:0, build key:t2.a (#0), probe key:t1.a (#2), filter type:bloom,inlist,min_max +│ └── filter id:0, build key:t2.a (#0), probe targets:[t1.a (#2)@scan1], filter type:bloom,inlist,min_max ├── estimated rows: 0.00 ├── EvalScalar(Build) │ ├── output columns: [t.a (#0), action (#1)] @@ -310,6 +322,7 @@ HashJoin │ ├── estimated rows: 0.00 │ └── TableScan │ ├── table: default.default.t +│ ├── scan id: 0 │ ├── output columns: [a (#0)] │ ├── read rows: 0 │ ├── read size: 0 @@ -323,6 +336,7 @@ HashJoin ├── estimated rows: 5.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 1 ├── output columns: [a (#2)] ├── read rows: 5 ├── read size: < 1 KiB diff --git a/tests/sqllogictests/suites/mode/standalone/explain/push_down_filter/push_down_filter_join/push_down_filter_join_semi_anti.test b/tests/sqllogictests/suites/mode/standalone/explain/push_down_filter/push_down_filter_join/push_down_filter_join_semi_anti.test index 87a7b0a47ee02..aeeef34564f21 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/push_down_filter/push_down_filter_join/push_down_filter_join_semi_anti.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/push_down_filter/push_down_filter_join/push_down_filter_join_semi_anti.test @@ -35,6 +35,7 @@ HashJoin │ ├── estimated rows: 0.00 │ └── TableScan │ ├── table: default.default.t2 +│ ├── scan id: 1 │ ├── output columns: [a (#2)] │ ├── read rows: 0 │ ├── read size: 0 @@ -49,6 +50,7 @@ HashJoin ├── estimated rows: 0.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 0 ├── read size: 0 @@ -76,6 +78,7 @@ HashJoin │ ├── estimated rows: 0.00 │ └── TableScan │ ├── table: default.default.t2 +│ ├── scan id: 1 │ ├── output columns: [a (#2)] │ ├── read rows: 0 │ ├── read size: 0 @@ -90,6 +93,7 @@ HashJoin ├── estimated rows: 0.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 0 ├── read size: 0 diff --git a/tests/sqllogictests/suites/mode/standalone/explain/push_down_filter/push_down_filter_join/push_down_filter_self_join.test b/tests/sqllogictests/suites/mode/standalone/explain/push_down_filter/push_down_filter_join/push_down_filter_self_join.test index 5ce0f1171d2e7..2dad546edfe88 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/push_down_filter/push_down_filter_join/push_down_filter_self_join.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/push_down_filter/push_down_filter_join/push_down_filter_self_join.test @@ -15,6 +15,7 @@ Sequence ├── MaterializedCTE: b │ └── TableScan │ ├── table: default.default.a +│ ├── scan id: 0 │ ├── output columns: [a (#0), b (#1)] │ ├── read rows: 20 │ ├── read size: < 1 KiB @@ -31,7 +32,7 @@ Sequence ├── keys is null equal: [false] ├── filters: [d.b (#3) < d.b (#5)] ├── build join filters: - │ └── filter id:0, build key:b2.a (#4), probe key:b1.a (#2), filter type:inlist,min_max + │ └── filter id:0, build key:b2.a (#4), probe targets:[b1.a (#2)@scan1], filter type:bloom,inlist,min_max ├── estimated rows: 40.00 ├── MaterializeCTERef(Build) │ ├── cte_name: b diff --git a/tests/sqllogictests/suites/mode/standalone/explain/push_down_filter/push_down_filter_project_set.test b/tests/sqllogictests/suites/mode/standalone/explain/push_down_filter/push_down_filter_project_set.test index 62ad447a5e32a..61c8a9fa734bc 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/push_down_filter/push_down_filter_project_set.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/push_down_filter/push_down_filter_project_set.test @@ -29,6 +29,7 @@ EvalScalar ├── estimated rows: 0.60 └── TableScan ├── table: default.default.products + ├── scan id: 0 ├── output columns: [name (#0), details (#1)] ├── read rows: 3 ├── read size: < 1 KiB @@ -60,6 +61,7 @@ EvalScalar ├── estimated rows: 0.60 └── TableScan ├── table: default.default.products + ├── scan id: 0 ├── output columns: [name (#0), details (#1)] ├── read rows: 3 ├── read size: < 1 KiB diff --git a/tests/sqllogictests/suites/mode/standalone/explain/push_down_filter/push_down_filter_scan.test b/tests/sqllogictests/suites/mode/standalone/explain/push_down_filter/push_down_filter_scan.test index 7a26bcef438f8..923e52da12edd 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/push_down_filter/push_down_filter_scan.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/push_down_filter/push_down_filter_scan.test @@ -17,6 +17,7 @@ Filter ├── estimated rows: 1.00 └── TableScan ├── table: default.default.t + ├── scan id: 0 ├── output columns: [x (#0)] ├── read rows: 2 ├── read size: < 1 KiB diff --git a/tests/sqllogictests/suites/mode/standalone/explain/range_pruner.test b/tests/sqllogictests/suites/mode/standalone/explain/range_pruner.test index 64120cba11dbe..db3c92d40280e 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/range_pruner.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/range_pruner.test @@ -17,6 +17,7 @@ EvalScalar ├── estimated rows: 0.40 └── TableScan ├── table: default.default.range_t + ├── scan id: 0 ├── output columns: [c (#0)] ├── read rows: 0 ├── read size: 0 @@ -45,6 +46,7 @@ EvalScalar ├── estimated rows: 0.00 └── TableScan ├── table: default.default.range_t + ├── scan id: 0 ├── output columns: [i (#1)] ├── read rows: 0 ├── read size: 0 diff --git a/tests/sqllogictests/suites/mode/standalone/explain/select_limit_offset.test b/tests/sqllogictests/suites/mode/standalone/explain/select_limit_offset.test index 314a118a96bf4..6127a9d2d9f69 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/select_limit_offset.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/select_limit_offset.test @@ -139,7 +139,7 @@ Limit ├── keys is null equal: [false] ├── filters: [] ├── build join filters: - │ └── filter id:0, build key:t.a (#0), probe key:t1.a (#1), filter type:inlist,min_max + │ └── filter id:0, build key:t.a (#0), probe targets:[t1.a (#1)@scan1], filter type:bloom,inlist,min_max ├── estimated rows: 2.00 ├── Limit(Build) │ ├── output columns: [t.a (#0)] @@ -148,6 +148,7 @@ Limit │ ├── estimated rows: 1.00 │ └── TableScan │ ├── table: default.default.t + │ ├── scan id: 0 │ ├── output columns: [a (#0)] │ ├── read rows: 1 │ ├── read size: < 1 KiB @@ -158,6 +159,7 @@ Limit │ └── estimated rows: 1.00 └── TableScan(Probe) ├── table: default.default.t1 + ├── scan id: 1 ├── output columns: [a (#1)] ├── read rows: 2 ├── read size: < 1 KiB @@ -184,7 +186,7 @@ Limit ├── keys is null equal: [false] ├── filters: [] ├── build join filters: - │ └── filter id:0, build key:t.a (#1), probe key:t1.a (#0), filter type:inlist,min_max + │ └── filter id:0, build key:t.a (#1), probe targets:[t1.a (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 2.00 ├── Limit(Build) │ ├── output columns: [t.a (#1)] @@ -193,6 +195,7 @@ Limit │ ├── estimated rows: 1.00 │ └── TableScan │ ├── table: default.default.t + │ ├── scan id: 1 │ ├── output columns: [a (#1)] │ ├── read rows: 1 │ ├── read size: < 1 KiB @@ -203,6 +206,7 @@ Limit │ └── estimated rows: 1.00 └── TableScan(Probe) ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 2 ├── read size: < 1 KiB diff --git a/tests/sqllogictests/suites/mode/standalone/explain/selectivity/filter.test b/tests/sqllogictests/suites/mode/standalone/explain/selectivity/filter.test index 6c6a877e662eb..1b1b92f909685 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/selectivity/filter.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/selectivity/filter.test @@ -10,6 +10,7 @@ Filter ├── estimated rows: 6.00 └── TableScan ├── table: default.default.t + ├── scan id: 0 ├── output columns: [number (#0)] ├── read rows: 738 ├── read size: 1.10 KiB @@ -28,6 +29,7 @@ Filter ├── estimated rows: 0.00 └── TableScan ├── table: default.default.t + ├── scan id: 0 ├── output columns: [number (#0)] ├── read rows: 0 ├── read size: 0 @@ -46,6 +48,7 @@ Filter ├── estimated rows: 37.00 └── TableScan ├── table: default.default.t + ├── scan id: 0 ├── output columns: [number (#0)] ├── read rows: 738 ├── read size: 1.10 KiB diff --git a/tests/sqllogictests/suites/mode/standalone/explain/selectivity/is_not_null.test b/tests/sqllogictests/suites/mode/standalone/explain/selectivity/is_not_null.test index 5c41e4a9a5e4c..e08e228e7812d 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/selectivity/is_not_null.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/selectivity/is_not_null.test @@ -13,6 +13,7 @@ Filter ├── estimated rows: 3.00 └── TableScan ├── table: default.default.twocolumn + ├── scan id: 0 ├── output columns: [x (#0), y (#1)] ├── read rows: 4 ├── read size: < 1 KiB @@ -40,6 +41,7 @@ Filter ├── estimated rows: 2.00 └── TableScan ├── table: default.default.t_user + ├── scan id: 0 ├── output columns: [id (#0), name (#1)] ├── read rows: 2 ├── read size: < 1 KiB diff --git a/tests/sqllogictests/suites/mode/standalone/explain/selectivity/modulo.test b/tests/sqllogictests/suites/mode/standalone/explain/selectivity/modulo.test index 8fc0eb31ab75c..0e667114c50fc 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/selectivity/modulo.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/selectivity/modulo.test @@ -20,6 +20,7 @@ AggregateFinal ├── estimated rows: 0.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [number (#0)] ├── read rows: 1000 ├── read size: 1.40 KiB @@ -48,6 +49,7 @@ AggregateFinal ├── estimated rows: 100.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [number (#0)] ├── read rows: 1000 ├── read size: 1.40 KiB diff --git a/tests/sqllogictests/suites/mode/standalone/explain/selectivity/pr_16069.test b/tests/sqllogictests/suites/mode/standalone/explain/selectivity/pr_16069.test index eb6ca932993ca..4283efa23588a 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/selectivity/pr_16069.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/selectivity/pr_16069.test @@ -13,6 +13,7 @@ Filter ├── estimated rows: 97.69 └── TableScan ├── table: default.default.t + ├── scan id: 0 ├── output columns: [number (#0)] ├── read rows: 100 ├── read size: < 1 KiB diff --git a/tests/sqllogictests/suites/mode/standalone/explain/sort.test b/tests/sqllogictests/suites/mode/standalone/explain/sort.test index 8ac291342a258..fea7fbe1fb7d0 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/sort.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/sort.test @@ -14,6 +14,7 @@ Sort(Single) ├── estimated rows: 0.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 0 ├── read size: 0 @@ -35,6 +36,7 @@ Sort(Single) ├── estimated rows: 0.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 0 ├── read size: 0 @@ -56,6 +58,7 @@ Sort(Single) ├── estimated rows: 0.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 0 ├── read size: 0 @@ -82,6 +85,7 @@ Limit ├── estimated rows: 0.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 0 ├── read size: 0 @@ -102,48 +106,50 @@ explain pipeline select a, b from t1 order by a; ---- digraph { 0 [ label = "BlockPartitionSource" ] - 1 [ label = "SyncReadParquetDataTransform" ] - 2 [ label = "DeserializeDataTransform" ] - 3 [ label = "Resize" ] - 4 [ label = "SortPartialTransform" ] + 1 [ label = "TransformRuntimeFilterWait" ] + 2 [ label = "SyncReadParquetDataTransform" ] + 3 [ label = "DeserializeDataTransform" ] + 4 [ label = "Resize" ] 5 [ label = "SortPartialTransform" ] 6 [ label = "SortPartialTransform" ] 7 [ label = "SortPartialTransform" ] - 8 [ label = "TransformSortMerge" ] + 8 [ label = "SortPartialTransform" ] 9 [ label = "TransformSortMerge" ] 10 [ label = "TransformSortMerge" ] 11 [ label = "TransformSortMerge" ] - 12 [ label = "KWayMergePartitioner" ] - 13 [ label = "KWayMergeWorker" ] + 12 [ label = "TransformSortMerge" ] + 13 [ label = "KWayMergePartitioner" ] 14 [ label = "KWayMergeWorker" ] 15 [ label = "KWayMergeWorker" ] 16 [ label = "KWayMergeWorker" ] - 17 [ label = "KWayMergeCombiner" ] - 18 [ label = "CompoundBlockOperator(Project)" ] + 17 [ label = "KWayMergeWorker" ] + 18 [ label = "KWayMergeCombiner" ] + 19 [ label = "CompoundBlockOperator(Project)" ] 0 -> 1 [ label = "" ] 1 -> 2 [ label = "" ] 2 -> 3 [ label = "" ] - 3 -> 4 [ label = "from: 0, to: 0" ] - 3 -> 5 [ label = "from: 1, to: 0" ] - 3 -> 6 [ label = "from: 2, to: 0" ] - 3 -> 7 [ label = "from: 3, to: 0" ] - 4 -> 8 [ label = "" ] + 3 -> 4 [ label = "" ] + 4 -> 5 [ label = "from: 0, to: 0" ] + 4 -> 6 [ label = "from: 1, to: 0" ] + 4 -> 7 [ label = "from: 2, to: 0" ] + 4 -> 8 [ label = "from: 3, to: 0" ] 5 -> 9 [ label = "" ] 6 -> 10 [ label = "" ] 7 -> 11 [ label = "" ] - 8 -> 12 [ label = "from: 0, to: 0" ] - 9 -> 12 [ label = "from: 0, to: 1" ] - 10 -> 12 [ label = "from: 0, to: 2" ] - 11 -> 12 [ label = "from: 0, to: 3" ] - 12 -> 13 [ label = "from: 0, to: 0" ] - 12 -> 14 [ label = "from: 1, to: 0" ] - 12 -> 15 [ label = "from: 2, to: 0" ] - 12 -> 16 [ label = "from: 3, to: 0" ] - 13 -> 17 [ label = "from: 0, to: 0" ] - 14 -> 17 [ label = "from: 0, to: 1" ] - 15 -> 17 [ label = "from: 0, to: 2" ] - 16 -> 17 [ label = "from: 0, to: 3" ] - 17 -> 18 [ label = "" ] + 8 -> 12 [ label = "" ] + 9 -> 13 [ label = "from: 0, to: 0" ] + 10 -> 13 [ label = "from: 0, to: 1" ] + 11 -> 13 [ label = "from: 0, to: 2" ] + 12 -> 13 [ label = "from: 0, to: 3" ] + 13 -> 14 [ label = "from: 0, to: 0" ] + 13 -> 15 [ label = "from: 1, to: 0" ] + 13 -> 16 [ label = "from: 2, to: 0" ] + 13 -> 17 [ label = "from: 3, to: 0" ] + 14 -> 18 [ label = "from: 0, to: 0" ] + 15 -> 18 [ label = "from: 0, to: 1" ] + 16 -> 18 [ label = "from: 0, to: 2" ] + 17 -> 18 [ label = "from: 0, to: 3" ] + 18 -> 19 [ label = "" ] } @@ -156,48 +162,50 @@ explain pipeline select a, b from t1 order by a; ---- digraph { 0 [ label = "BlockPartitionSource" ] - 1 [ label = "SyncReadParquetDataTransform" ] - 2 [ label = "DeserializeDataTransform" ] - 3 [ label = "Resize" ] - 4 [ label = "SortPartialTransform" ] + 1 [ label = "TransformRuntimeFilterWait" ] + 2 [ label = "SyncReadParquetDataTransform" ] + 3 [ label = "DeserializeDataTransform" ] + 4 [ label = "Resize" ] 5 [ label = "SortPartialTransform" ] 6 [ label = "SortPartialTransform" ] 7 [ label = "SortPartialTransform" ] - 8 [ label = "TransformSortMerge" ] + 8 [ label = "SortPartialTransform" ] 9 [ label = "TransformSortMerge" ] 10 [ label = "TransformSortMerge" ] 11 [ label = "TransformSortMerge" ] - 12 [ label = "KWayMergePartitioner" ] - 13 [ label = "KWayMergeWorker" ] + 12 [ label = "TransformSortMerge" ] + 13 [ label = "KWayMergePartitioner" ] 14 [ label = "KWayMergeWorker" ] 15 [ label = "KWayMergeWorker" ] 16 [ label = "KWayMergeWorker" ] - 17 [ label = "KWayMergeCombiner" ] - 18 [ label = "CompoundBlockOperator(Project)" ] + 17 [ label = "KWayMergeWorker" ] + 18 [ label = "KWayMergeCombiner" ] + 19 [ label = "CompoundBlockOperator(Project)" ] 0 -> 1 [ label = "" ] 1 -> 2 [ label = "" ] 2 -> 3 [ label = "" ] - 3 -> 4 [ label = "from: 0, to: 0" ] - 3 -> 5 [ label = "from: 1, to: 0" ] - 3 -> 6 [ label = "from: 2, to: 0" ] - 3 -> 7 [ label = "from: 3, to: 0" ] - 4 -> 8 [ label = "" ] + 3 -> 4 [ label = "" ] + 4 -> 5 [ label = "from: 0, to: 0" ] + 4 -> 6 [ label = "from: 1, to: 0" ] + 4 -> 7 [ label = "from: 2, to: 0" ] + 4 -> 8 [ label = "from: 3, to: 0" ] 5 -> 9 [ label = "" ] 6 -> 10 [ label = "" ] 7 -> 11 [ label = "" ] - 8 -> 12 [ label = "from: 0, to: 0" ] - 9 -> 12 [ label = "from: 0, to: 1" ] - 10 -> 12 [ label = "from: 0, to: 2" ] - 11 -> 12 [ label = "from: 0, to: 3" ] - 12 -> 13 [ label = "from: 0, to: 0" ] - 12 -> 14 [ label = "from: 1, to: 0" ] - 12 -> 15 [ label = "from: 2, to: 0" ] - 12 -> 16 [ label = "from: 3, to: 0" ] - 13 -> 17 [ label = "from: 0, to: 0" ] - 14 -> 17 [ label = "from: 0, to: 1" ] - 15 -> 17 [ label = "from: 0, to: 2" ] - 16 -> 17 [ label = "from: 0, to: 3" ] - 17 -> 18 [ label = "" ] + 8 -> 12 [ label = "" ] + 9 -> 13 [ label = "from: 0, to: 0" ] + 10 -> 13 [ label = "from: 0, to: 1" ] + 11 -> 13 [ label = "from: 0, to: 2" ] + 12 -> 13 [ label = "from: 0, to: 3" ] + 13 -> 14 [ label = "from: 0, to: 0" ] + 13 -> 15 [ label = "from: 1, to: 0" ] + 13 -> 16 [ label = "from: 2, to: 0" ] + 13 -> 17 [ label = "from: 3, to: 0" ] + 14 -> 18 [ label = "from: 0, to: 0" ] + 15 -> 18 [ label = "from: 0, to: 1" ] + 16 -> 18 [ label = "from: 0, to: 2" ] + 17 -> 18 [ label = "from: 0, to: 3" ] + 18 -> 19 [ label = "" ] } statement ok @@ -209,50 +217,52 @@ explain pipeline select a + 1, b from t1 order by a + 1; ---- digraph { 0 [ label = "BlockPartitionSource" ] - 1 [ label = "SyncReadParquetDataTransform" ] - 2 [ label = "DeserializeDataTransform" ] - 3 [ label = "CompoundBlockOperator(Map)" ] - 4 [ label = "Resize" ] - 5 [ label = "SortPartialTransform" ] + 1 [ label = "TransformRuntimeFilterWait" ] + 2 [ label = "SyncReadParquetDataTransform" ] + 3 [ label = "DeserializeDataTransform" ] + 4 [ label = "CompoundBlockOperator(Map)" ] + 5 [ label = "Resize" ] 6 [ label = "SortPartialTransform" ] 7 [ label = "SortPartialTransform" ] 8 [ label = "SortPartialTransform" ] - 9 [ label = "TransformSortMerge" ] + 9 [ label = "SortPartialTransform" ] 10 [ label = "TransformSortMerge" ] 11 [ label = "TransformSortMerge" ] 12 [ label = "TransformSortMerge" ] - 13 [ label = "KWayMergePartitioner" ] - 14 [ label = "KWayMergeWorker" ] + 13 [ label = "TransformSortMerge" ] + 14 [ label = "KWayMergePartitioner" ] 15 [ label = "KWayMergeWorker" ] 16 [ label = "KWayMergeWorker" ] 17 [ label = "KWayMergeWorker" ] - 18 [ label = "KWayMergeCombiner" ] - 19 [ label = "CompoundBlockOperator(Project)" ] + 18 [ label = "KWayMergeWorker" ] + 19 [ label = "KWayMergeCombiner" ] + 20 [ label = "CompoundBlockOperator(Project)" ] 0 -> 1 [ label = "" ] 1 -> 2 [ label = "" ] 2 -> 3 [ label = "" ] 3 -> 4 [ label = "" ] - 4 -> 5 [ label = "from: 0, to: 0" ] - 4 -> 6 [ label = "from: 1, to: 0" ] - 4 -> 7 [ label = "from: 2, to: 0" ] - 4 -> 8 [ label = "from: 3, to: 0" ] - 5 -> 9 [ label = "" ] + 4 -> 5 [ label = "" ] + 5 -> 6 [ label = "from: 0, to: 0" ] + 5 -> 7 [ label = "from: 1, to: 0" ] + 5 -> 8 [ label = "from: 2, to: 0" ] + 5 -> 9 [ label = "from: 3, to: 0" ] 6 -> 10 [ label = "" ] 7 -> 11 [ label = "" ] 8 -> 12 [ label = "" ] - 9 -> 13 [ label = "from: 0, to: 0" ] - 10 -> 13 [ label = "from: 0, to: 1" ] - 11 -> 13 [ label = "from: 0, to: 2" ] - 12 -> 13 [ label = "from: 0, to: 3" ] - 13 -> 14 [ label = "from: 0, to: 0" ] - 13 -> 15 [ label = "from: 1, to: 0" ] - 13 -> 16 [ label = "from: 2, to: 0" ] - 13 -> 17 [ label = "from: 3, to: 0" ] - 14 -> 18 [ label = "from: 0, to: 0" ] - 15 -> 18 [ label = "from: 0, to: 1" ] - 16 -> 18 [ label = "from: 0, to: 2" ] - 17 -> 18 [ label = "from: 0, to: 3" ] - 18 -> 19 [ label = "" ] + 9 -> 13 [ label = "" ] + 10 -> 14 [ label = "from: 0, to: 0" ] + 11 -> 14 [ label = "from: 0, to: 1" ] + 12 -> 14 [ label = "from: 0, to: 2" ] + 13 -> 14 [ label = "from: 0, to: 3" ] + 14 -> 15 [ label = "from: 0, to: 0" ] + 14 -> 16 [ label = "from: 1, to: 0" ] + 14 -> 17 [ label = "from: 2, to: 0" ] + 14 -> 18 [ label = "from: 3, to: 0" ] + 15 -> 19 [ label = "from: 0, to: 0" ] + 16 -> 19 [ label = "from: 0, to: 1" ] + 17 -> 19 [ label = "from: 0, to: 2" ] + 18 -> 19 [ label = "from: 0, to: 3" ] + 19 -> 20 [ label = "" ] } # Sort spilling @@ -264,50 +274,52 @@ explain pipeline select a + 1, b from t1 order by a + 1; ---- digraph { 0 [ label = "BlockPartitionSource" ] - 1 [ label = "SyncReadParquetDataTransform" ] - 2 [ label = "DeserializeDataTransform" ] - 3 [ label = "CompoundBlockOperator(Map)" ] - 4 [ label = "Resize" ] - 5 [ label = "SortPartialTransform" ] + 1 [ label = "TransformRuntimeFilterWait" ] + 2 [ label = "SyncReadParquetDataTransform" ] + 3 [ label = "DeserializeDataTransform" ] + 4 [ label = "CompoundBlockOperator(Map)" ] + 5 [ label = "Resize" ] 6 [ label = "SortPartialTransform" ] 7 [ label = "SortPartialTransform" ] 8 [ label = "SortPartialTransform" ] - 9 [ label = "TransformSortMerge" ] + 9 [ label = "SortPartialTransform" ] 10 [ label = "TransformSortMerge" ] 11 [ label = "TransformSortMerge" ] 12 [ label = "TransformSortMerge" ] - 13 [ label = "KWayMergePartitioner" ] - 14 [ label = "KWayMergeWorker" ] + 13 [ label = "TransformSortMerge" ] + 14 [ label = "KWayMergePartitioner" ] 15 [ label = "KWayMergeWorker" ] 16 [ label = "KWayMergeWorker" ] 17 [ label = "KWayMergeWorker" ] - 18 [ label = "KWayMergeCombiner" ] - 19 [ label = "CompoundBlockOperator(Project)" ] + 18 [ label = "KWayMergeWorker" ] + 19 [ label = "KWayMergeCombiner" ] + 20 [ label = "CompoundBlockOperator(Project)" ] 0 -> 1 [ label = "" ] 1 -> 2 [ label = "" ] 2 -> 3 [ label = "" ] 3 -> 4 [ label = "" ] - 4 -> 5 [ label = "from: 0, to: 0" ] - 4 -> 6 [ label = "from: 1, to: 0" ] - 4 -> 7 [ label = "from: 2, to: 0" ] - 4 -> 8 [ label = "from: 3, to: 0" ] - 5 -> 9 [ label = "" ] + 4 -> 5 [ label = "" ] + 5 -> 6 [ label = "from: 0, to: 0" ] + 5 -> 7 [ label = "from: 1, to: 0" ] + 5 -> 8 [ label = "from: 2, to: 0" ] + 5 -> 9 [ label = "from: 3, to: 0" ] 6 -> 10 [ label = "" ] 7 -> 11 [ label = "" ] 8 -> 12 [ label = "" ] - 9 -> 13 [ label = "from: 0, to: 0" ] - 10 -> 13 [ label = "from: 0, to: 1" ] - 11 -> 13 [ label = "from: 0, to: 2" ] - 12 -> 13 [ label = "from: 0, to: 3" ] - 13 -> 14 [ label = "from: 0, to: 0" ] - 13 -> 15 [ label = "from: 1, to: 0" ] - 13 -> 16 [ label = "from: 2, to: 0" ] - 13 -> 17 [ label = "from: 3, to: 0" ] - 14 -> 18 [ label = "from: 0, to: 0" ] - 15 -> 18 [ label = "from: 0, to: 1" ] - 16 -> 18 [ label = "from: 0, to: 2" ] - 17 -> 18 [ label = "from: 0, to: 3" ] - 18 -> 19 [ label = "" ] + 9 -> 13 [ label = "" ] + 10 -> 14 [ label = "from: 0, to: 0" ] + 11 -> 14 [ label = "from: 0, to: 1" ] + 12 -> 14 [ label = "from: 0, to: 2" ] + 13 -> 14 [ label = "from: 0, to: 3" ] + 14 -> 15 [ label = "from: 0, to: 0" ] + 14 -> 16 [ label = "from: 1, to: 0" ] + 14 -> 17 [ label = "from: 2, to: 0" ] + 14 -> 18 [ label = "from: 3, to: 0" ] + 15 -> 19 [ label = "from: 0, to: 0" ] + 16 -> 19 [ label = "from: 0, to: 1" ] + 17 -> 19 [ label = "from: 0, to: 2" ] + 18 -> 19 [ label = "from: 0, to: 3" ] + 19 -> 20 [ label = "" ] } statement ok diff --git a/tests/sqllogictests/suites/mode/standalone/explain/union.test b/tests/sqllogictests/suites/mode/standalone/explain/union.test index a79962114934f..416d8d14944f4 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/union.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/union.test @@ -34,6 +34,7 @@ UnionAll │ ├── estimated rows: 0.40 │ └── TableScan │ ├── table: default.default.t1 +│ ├── scan id: 0 │ ├── output columns: [a (#0), b (#1)] │ ├── read rows: 0 │ ├── read size: 0 @@ -48,6 +49,7 @@ UnionAll ├── estimated rows: 0.40 └── TableScan ├── table: default.default.t2 + ├── scan id: 1 ├── output columns: [a (#2), b (#3)] ├── read rows: 0 ├── read size: 0 @@ -69,6 +71,7 @@ UnionAll │ ├── estimated rows: 1.00 │ └── TableScan │ ├── table: default.default.t1 +│ ├── scan id: 0 │ ├── output columns: [a (#0), b (#1)] │ ├── read rows: 2 │ ├── read size: < 1 KiB @@ -83,6 +86,7 @@ UnionAll ├── estimated rows: 1.00 └── TableScan ├── table: default.default.t2 + ├── scan id: 1 ├── output columns: [a (#2), b (#3)] ├── read rows: 2 ├── read size: < 1 KiB @@ -110,6 +114,7 @@ Limit │ ├── estimated rows: 2.00 │ └── TableScan │ ├── table: default.default.t1 + │ ├── scan id: 0 │ ├── output columns: [a (#0), b (#1)] │ ├── read rows: 2 │ ├── read size: < 1 KiB @@ -125,6 +130,7 @@ Limit ├── estimated rows: 2.00 └── TableScan ├── table: default.default.t2 + ├── scan id: 1 ├── output columns: [a (#2), b (#3)] ├── read rows: 2 ├── read size: < 1 KiB @@ -152,6 +158,7 @@ Limit │ ├── estimated rows: 2.00 │ └── TableScan │ ├── table: default.default.t1 + │ ├── scan id: 0 │ ├── output columns: [a (#0), b (#1)] │ ├── read rows: 2 │ ├── read size: < 1 KiB @@ -167,6 +174,7 @@ Limit ├── estimated rows: 2.00 └── TableScan ├── table: default.default.t2 + ├── scan id: 1 ├── output columns: [a (#2), b (#3)] ├── read rows: 2 ├── read size: < 1 KiB @@ -194,6 +202,7 @@ Limit │ ├── estimated rows: 1.00 │ └── TableScan │ ├── table: default.default.t1 + │ ├── scan id: 0 │ ├── output columns: [a (#0), b (#1)] │ ├── read rows: 2 │ ├── read size: < 1 KiB @@ -209,6 +218,7 @@ Limit ├── estimated rows: 1.00 └── TableScan ├── table: default.default.t2 + ├── scan id: 1 ├── output columns: [a (#2), b (#3)] ├── read rows: 2 ├── read size: < 1 KiB @@ -231,6 +241,7 @@ UnionAll │ ├── estimated rows: 1.00 │ └── TableScan │ ├── table: default.default.t1 +│ ├── scan id: 0 │ ├── output columns: [a (#0), b (#1)] │ ├── read rows: 2 │ ├── read size: < 1 KiB @@ -245,6 +256,7 @@ UnionAll ├── estimated rows: 1.00 └── TableScan ├── table: default.default.t2 + ├── scan id: 1 ├── output columns: [b (#3)] ├── read rows: 2 ├── read size: < 1 KiB @@ -286,6 +298,7 @@ UnionAll ├── EmptyResultScan └── TableScan ├── table: default.default.t2 + ├── scan id: 1 ├── output columns: [b (#1)] ├── read rows: 10000 ├── read size: 10.59 KiB @@ -304,6 +317,7 @@ UnionAll ├── estimated rows: 10000.00 ├── TableScan │ ├── table: default.default.t1 +│ ├── scan id: 0 │ ├── output columns: [a (#0)] │ ├── read rows: 10000 │ ├── read size: 10.59 KiB diff --git a/tests/sqllogictests/suites/mode/standalone/explain/update.test b/tests/sqllogictests/suites/mode/standalone/explain/update.test index e0969b18837c1..8fc9c9b0ddab5 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/update.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/update.test @@ -37,6 +37,7 @@ CommitSink ├── estimated rows: 2.00 ├── TableScan(Build) │ ├── table: default.default.t2 + │ ├── scan id: 1 │ ├── output columns: [a (#2)] │ ├── read rows: 2 │ ├── read size: < 1 KiB @@ -47,6 +48,7 @@ CommitSink │ └── estimated rows: 2.00 └── TableScan(Probe) ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), _row_id (#4)] ├── read rows: 4 ├── read size: < 1 KiB @@ -104,6 +106,7 @@ CommitSink ├── estimated rows: 2.00 ├── TableScan(Build) │ ├── table: default.default.t2 + │ ├── scan id: 1 │ ├── output columns: [a (#2)] │ ├── read rows: 2 │ ├── read size: < 1 KiB @@ -118,6 +121,7 @@ CommitSink ├── estimated rows: 3.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1), _row_id (#4)] ├── read rows: 4 ├── read size: < 1 KiB @@ -205,12 +209,13 @@ CommitSink ├── keys is null equal: [false, false, false] ├── filters: [] ├── build join filters: - │ ├── filter id:0, build key:p.c_code (#9), probe key:c.c_code (#1), filter type:inlist,min_max - │ ├── filter id:1, build key:p.id (#8), probe key:c.id (#0), filter type:inlist,min_max - │ └── filter id:2, build key:p.me_id (#10), probe key:c.me_id (#2), filter type:inlist,min_max + │ ├── filter id:0, build key:p.c_code (#9), probe targets:[c.c_code (#1)@scan0], filter type:bloom,inlist,min_max + │ ├── filter id:1, build key:p.id (#8), probe targets:[c.id (#0)@scan0], filter type:bloom,inlist,min_max + │ └── filter id:2, build key:p.me_id (#10), probe targets:[c.me_id (#2)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 3.20 ├── TableScan(Build) │ ├── table: default.default.t2 + │ ├── scan id: 1 │ ├── output columns: [id (#8), c_code (#9), me_id (#10), p_id (#11), m_p_id (#12), lkp_id (#13), l_p_id (#14), l_m_p_id (#15)] │ ├── read rows: 4 │ ├── read size: < 1 KiB @@ -221,6 +226,7 @@ CommitSink │ └── estimated rows: 4.00 └── TableScan(Probe) ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [id (#0), c_code (#1), me_id (#2), _row_id (#16)] ├── read rows: 4 ├── read size: < 1 KiB diff --git a/tests/sqllogictests/suites/mode/standalone/explain/window.test b/tests/sqllogictests/suites/mode/standalone/explain/window.test index 6cca3828bdf90..8c25d3dff7796 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/window.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/window.test @@ -32,6 +32,7 @@ Sort(Single) ├── estimated rows: 0.00 └── TableScan ├── table: default.test_explain_window.empsalary + ├── scan id: 0 ├── output columns: [depname (#0), empno (#1), salary (#2)] ├── read rows: 0 ├── read size: 0 @@ -52,28 +53,29 @@ explain pipeline SELECT depname, empno, salary, sum(salary) OVER (PARTITION BY d ---- digraph { 0 [ label = "BlockPartitionSource" ] - 1 [ label = "SyncReadParquetDataTransform" ] - 2 [ label = "DeserializeDataTransform" ] - 3 [ label = "ShufflePartition(Window)" ] - 4 [ label = "ShuffleMergePartition(Window)" ] - 5 [ label = "TransformWindowPartitionCollect(Sort)" ] - 6 [ label = "Transform Window" ] - 7 [ label = "Resize" ] - 8 [ label = "SortPartialTransform" ] + 1 [ label = "TransformRuntimeFilterWait" ] + 2 [ label = "SyncReadParquetDataTransform" ] + 3 [ label = "DeserializeDataTransform" ] + 4 [ label = "ShufflePartition(Window)" ] + 5 [ label = "ShuffleMergePartition(Window)" ] + 6 [ label = "TransformWindowPartitionCollect(Sort)" ] + 7 [ label = "Transform Window" ] + 8 [ label = "Resize" ] 9 [ label = "SortPartialTransform" ] 10 [ label = "SortPartialTransform" ] 11 [ label = "SortPartialTransform" ] - 12 [ label = "TransformSortMerge" ] + 12 [ label = "SortPartialTransform" ] 13 [ label = "TransformSortMerge" ] 14 [ label = "TransformSortMerge" ] 15 [ label = "TransformSortMerge" ] - 16 [ label = "KWayMergePartitioner" ] - 17 [ label = "KWayMergeWorker" ] + 16 [ label = "TransformSortMerge" ] + 17 [ label = "KWayMergePartitioner" ] 18 [ label = "KWayMergeWorker" ] 19 [ label = "KWayMergeWorker" ] 20 [ label = "KWayMergeWorker" ] - 21 [ label = "KWayMergeCombiner" ] - 22 [ label = "CompoundBlockOperator(Project)" ] + 21 [ label = "KWayMergeWorker" ] + 22 [ label = "KWayMergeCombiner" ] + 23 [ label = "CompoundBlockOperator(Project)" ] 0 -> 1 [ label = "" ] 1 -> 2 [ label = "" ] 2 -> 3 [ label = "" ] @@ -81,27 +83,28 @@ digraph { 4 -> 5 [ label = "" ] 5 -> 6 [ label = "" ] 6 -> 7 [ label = "" ] - 7 -> 8 [ label = "from: 0, to: 0" ] - 7 -> 9 [ label = "from: 1, to: 0" ] - 7 -> 10 [ label = "from: 2, to: 0" ] - 7 -> 11 [ label = "from: 3, to: 0" ] - 8 -> 12 [ label = "" ] + 7 -> 8 [ label = "" ] + 8 -> 9 [ label = "from: 0, to: 0" ] + 8 -> 10 [ label = "from: 1, to: 0" ] + 8 -> 11 [ label = "from: 2, to: 0" ] + 8 -> 12 [ label = "from: 3, to: 0" ] 9 -> 13 [ label = "" ] 10 -> 14 [ label = "" ] 11 -> 15 [ label = "" ] - 12 -> 16 [ label = "from: 0, to: 0" ] - 13 -> 16 [ label = "from: 0, to: 1" ] - 14 -> 16 [ label = "from: 0, to: 2" ] - 15 -> 16 [ label = "from: 0, to: 3" ] - 16 -> 17 [ label = "from: 0, to: 0" ] - 16 -> 18 [ label = "from: 1, to: 0" ] - 16 -> 19 [ label = "from: 2, to: 0" ] - 16 -> 20 [ label = "from: 3, to: 0" ] - 17 -> 21 [ label = "from: 0, to: 0" ] - 18 -> 21 [ label = "from: 0, to: 1" ] - 19 -> 21 [ label = "from: 0, to: 2" ] - 20 -> 21 [ label = "from: 0, to: 3" ] - 21 -> 22 [ label = "" ] + 12 -> 16 [ label = "" ] + 13 -> 17 [ label = "from: 0, to: 0" ] + 14 -> 17 [ label = "from: 0, to: 1" ] + 15 -> 17 [ label = "from: 0, to: 2" ] + 16 -> 17 [ label = "from: 0, to: 3" ] + 17 -> 18 [ label = "from: 0, to: 0" ] + 17 -> 19 [ label = "from: 1, to: 0" ] + 17 -> 20 [ label = "from: 2, to: 0" ] + 17 -> 21 [ label = "from: 3, to: 0" ] + 18 -> 22 [ label = "from: 0, to: 0" ] + 19 -> 22 [ label = "from: 0, to: 1" ] + 20 -> 22 [ label = "from: 0, to: 2" ] + 21 -> 22 [ label = "from: 0, to: 3" ] + 22 -> 23 [ label = "" ] } @@ -114,28 +117,29 @@ explain pipeline SELECT depname, empno, salary, sum(salary) OVER (PARTITION BY d ---- digraph { 0 [ label = "BlockPartitionSource" ] - 1 [ label = "SyncReadParquetDataTransform" ] - 2 [ label = "DeserializeDataTransform" ] - 3 [ label = "ShufflePartition(Window)" ] - 4 [ label = "ShuffleMergePartition(Window)" ] - 5 [ label = "TransformWindowPartitionCollect(Sort)" ] - 6 [ label = "Transform Window" ] - 7 [ label = "Resize" ] - 8 [ label = "SortPartialTransform" ] + 1 [ label = "TransformRuntimeFilterWait" ] + 2 [ label = "SyncReadParquetDataTransform" ] + 3 [ label = "DeserializeDataTransform" ] + 4 [ label = "ShufflePartition(Window)" ] + 5 [ label = "ShuffleMergePartition(Window)" ] + 6 [ label = "TransformWindowPartitionCollect(Sort)" ] + 7 [ label = "Transform Window" ] + 8 [ label = "Resize" ] 9 [ label = "SortPartialTransform" ] 10 [ label = "SortPartialTransform" ] 11 [ label = "SortPartialTransform" ] - 12 [ label = "TransformSortMerge" ] + 12 [ label = "SortPartialTransform" ] 13 [ label = "TransformSortMerge" ] 14 [ label = "TransformSortMerge" ] 15 [ label = "TransformSortMerge" ] - 16 [ label = "KWayMergePartitioner" ] - 17 [ label = "KWayMergeWorker" ] + 16 [ label = "TransformSortMerge" ] + 17 [ label = "KWayMergePartitioner" ] 18 [ label = "KWayMergeWorker" ] 19 [ label = "KWayMergeWorker" ] 20 [ label = "KWayMergeWorker" ] - 21 [ label = "KWayMergeCombiner" ] - 22 [ label = "CompoundBlockOperator(Project)" ] + 21 [ label = "KWayMergeWorker" ] + 22 [ label = "KWayMergeCombiner" ] + 23 [ label = "CompoundBlockOperator(Project)" ] 0 -> 1 [ label = "" ] 1 -> 2 [ label = "" ] 2 -> 3 [ label = "" ] @@ -143,27 +147,28 @@ digraph { 4 -> 5 [ label = "" ] 5 -> 6 [ label = "" ] 6 -> 7 [ label = "" ] - 7 -> 8 [ label = "from: 0, to: 0" ] - 7 -> 9 [ label = "from: 1, to: 0" ] - 7 -> 10 [ label = "from: 2, to: 0" ] - 7 -> 11 [ label = "from: 3, to: 0" ] - 8 -> 12 [ label = "" ] + 7 -> 8 [ label = "" ] + 8 -> 9 [ label = "from: 0, to: 0" ] + 8 -> 10 [ label = "from: 1, to: 0" ] + 8 -> 11 [ label = "from: 2, to: 0" ] + 8 -> 12 [ label = "from: 3, to: 0" ] 9 -> 13 [ label = "" ] 10 -> 14 [ label = "" ] 11 -> 15 [ label = "" ] - 12 -> 16 [ label = "from: 0, to: 0" ] - 13 -> 16 [ label = "from: 0, to: 1" ] - 14 -> 16 [ label = "from: 0, to: 2" ] - 15 -> 16 [ label = "from: 0, to: 3" ] - 16 -> 17 [ label = "from: 0, to: 0" ] - 16 -> 18 [ label = "from: 1, to: 0" ] - 16 -> 19 [ label = "from: 2, to: 0" ] - 16 -> 20 [ label = "from: 3, to: 0" ] - 17 -> 21 [ label = "from: 0, to: 0" ] - 18 -> 21 [ label = "from: 0, to: 1" ] - 19 -> 21 [ label = "from: 0, to: 2" ] - 20 -> 21 [ label = "from: 0, to: 3" ] - 21 -> 22 [ label = "" ] + 12 -> 16 [ label = "" ] + 13 -> 17 [ label = "from: 0, to: 0" ] + 14 -> 17 [ label = "from: 0, to: 1" ] + 15 -> 17 [ label = "from: 0, to: 2" ] + 16 -> 17 [ label = "from: 0, to: 3" ] + 17 -> 18 [ label = "from: 0, to: 0" ] + 17 -> 19 [ label = "from: 1, to: 0" ] + 17 -> 20 [ label = "from: 2, to: 0" ] + 17 -> 21 [ label = "from: 3, to: 0" ] + 18 -> 22 [ label = "from: 0, to: 0" ] + 19 -> 22 [ label = "from: 0, to: 1" ] + 20 -> 22 [ label = "from: 0, to: 2" ] + 21 -> 22 [ label = "from: 0, to: 3" ] + 22 -> 23 [ label = "" ] } @@ -201,6 +206,7 @@ Filter │ ├── estimated rows: 0.00 │ └── TableScan │ ├── table: default.test_explain_window.test + │ ├── scan id: 0 │ ├── output columns: [k (#0), v (#1)] │ ├── read rows: 0 │ ├── read size: 0 @@ -214,6 +220,7 @@ Filter ├── estimated rows: 0.00 └── TableScan ├── table: default.test_explain_window.test + ├── scan id: 1 ├── output columns: [k (#2), v (#3)] ├── read rows: 0 ├── read size: 0 @@ -246,6 +253,7 @@ Filter ├── estimated rows: 0.00 ├── TableScan │ ├── table: default.test_explain_window.test + │ ├── scan id: 0 │ ├── output columns: [k (#0), v (#1)] │ ├── read rows: 0 │ ├── read size: 0 @@ -255,6 +263,7 @@ Filter │ └── estimated rows: 0.00 └── TableScan ├── table: default.test_explain_window.test + ├── scan id: 1 ├── output columns: [k (#2), v (#3)] ├── read rows: 0 ├── read size: 0 @@ -286,6 +295,7 @@ Filter ├── estimated rows: 0.00 ├── TableScan │ ├── table: default.test_explain_window.test + │ ├── scan id: 0 │ ├── output columns: [k (#0), v (#1)] │ ├── read rows: 0 │ ├── read size: 0 @@ -295,6 +305,7 @@ Filter │ └── estimated rows: 0.00 └── TableScan ├── table: default.test_explain_window.test + ├── scan id: 1 ├── output columns: [k (#2), v (#3)] ├── read rows: 0 ├── read size: 0 @@ -328,6 +339,7 @@ Filter ├── estimated rows: 0.00 └── TableScan ├── table: default.test_explain_window.t + ├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 0 ├── read size: 0 @@ -368,6 +380,7 @@ Window ├── estimated rows: 0.00 └── TableScan ├── table: default.test_explain_window.tbpush + ├── scan id: 0 ├── output columns: [b (#0)] ├── read rows: 0 ├── read size: 0 @@ -395,6 +408,7 @@ Filter ├── estimated rows: 0.00 └── TableScan ├── table: default.test_explain_window.tbpush + ├── scan id: 0 ├── output columns: [b (#0)] ├── read rows: 0 ├── read size: 0 @@ -422,6 +436,7 @@ Filter ├── estimated rows: 0.00 └── TableScan ├── table: default.test_explain_window.tbpush + ├── scan id: 0 ├── output columns: [b (#0)] ├── read rows: 0 ├── read size: 0 @@ -448,14 +463,15 @@ explain pipeline select a, sum(a) over (partition by a order by a desc) from t l ---- digraph { 0 [ label = "BlockPartitionSource" ] - 1 [ label = "SyncReadParquetDataTransform" ] - 2 [ label = "DeserializeDataTransform" ] - 3 [ label = "ShufflePartition(Window)" ] - 4 [ label = "ShuffleMergePartition(Window)" ] - 5 [ label = "TransformWindowPartitionCollect(Sort)" ] - 6 [ label = "Transform Window" ] - 7 [ label = "LimitTransform" ] - 8 [ label = "CompoundBlockOperator(Project)" ] + 1 [ label = "TransformRuntimeFilterWait" ] + 2 [ label = "SyncReadParquetDataTransform" ] + 3 [ label = "DeserializeDataTransform" ] + 4 [ label = "ShufflePartition(Window)" ] + 5 [ label = "ShuffleMergePartition(Window)" ] + 6 [ label = "TransformWindowPartitionCollect(Sort)" ] + 7 [ label = "Transform Window" ] + 8 [ label = "LimitTransform" ] + 9 [ label = "CompoundBlockOperator(Project)" ] 0 -> 1 [ label = "" ] 1 -> 2 [ label = "" ] 2 -> 3 [ label = "" ] @@ -464,6 +480,7 @@ digraph { 5 -> 6 [ label = "" ] 6 -> 7 [ label = "" ] 7 -> 8 [ label = "" ] + 8 -> 9 [ label = "" ] } # Enable sort spilling @@ -476,14 +493,15 @@ explain pipeline select a, sum(a) over (partition by a order by a desc) from t l ---- digraph { 0 [ label = "BlockPartitionSource" ] - 1 [ label = "SyncReadParquetDataTransform" ] - 2 [ label = "DeserializeDataTransform" ] - 3 [ label = "ShufflePartition(Window)" ] - 4 [ label = "ShuffleMergePartition(Window)" ] - 5 [ label = "TransformWindowPartitionCollect(Sort)" ] - 6 [ label = "Transform Window" ] - 7 [ label = "LimitTransform" ] - 8 [ label = "CompoundBlockOperator(Project)" ] + 1 [ label = "TransformRuntimeFilterWait" ] + 2 [ label = "SyncReadParquetDataTransform" ] + 3 [ label = "DeserializeDataTransform" ] + 4 [ label = "ShufflePartition(Window)" ] + 5 [ label = "ShuffleMergePartition(Window)" ] + 6 [ label = "TransformWindowPartitionCollect(Sort)" ] + 7 [ label = "Transform Window" ] + 8 [ label = "LimitTransform" ] + 9 [ label = "CompoundBlockOperator(Project)" ] 0 -> 1 [ label = "" ] 1 -> 2 [ label = "" ] 2 -> 3 [ label = "" ] @@ -492,6 +510,7 @@ digraph { 5 -> 6 [ label = "" ] 6 -> 7 [ label = "" ] 7 -> 8 [ label = "" ] + 8 -> 9 [ label = "" ] } @@ -505,14 +524,15 @@ explain pipeline select a, dense_rank() over (partition by a order by a desc) fr ---- digraph { 0 [ label = "BlockPartitionSource" ] - 1 [ label = "SyncReadParquetDataTransform" ] - 2 [ label = "DeserializeDataTransform" ] - 3 [ label = "ShufflePartition(Window)" ] - 4 [ label = "ShuffleMergePartition(Window)" ] - 5 [ label = "TransformWindowPartitionCollect(Sort)" ] - 6 [ label = "Transform Window" ] - 7 [ label = "LimitTransform" ] - 8 [ label = "CompoundBlockOperator(Project)" ] + 1 [ label = "TransformRuntimeFilterWait" ] + 2 [ label = "SyncReadParquetDataTransform" ] + 3 [ label = "DeserializeDataTransform" ] + 4 [ label = "ShufflePartition(Window)" ] + 5 [ label = "ShuffleMergePartition(Window)" ] + 6 [ label = "TransformWindowPartitionCollect(Sort)" ] + 7 [ label = "Transform Window" ] + 8 [ label = "LimitTransform" ] + 9 [ label = "CompoundBlockOperator(Project)" ] 0 -> 1 [ label = "" ] 1 -> 2 [ label = "" ] 2 -> 3 [ label = "" ] @@ -521,6 +541,7 @@ digraph { 5 -> 6 [ label = "" ] 6 -> 7 [ label = "" ] 7 -> 8 [ label = "" ] + 8 -> 9 [ label = "" ] } # rows frame single window (can push down limit) @@ -529,14 +550,15 @@ explain pipeline select a, sum(a) over (partition by a order by a desc rows betw ---- digraph { 0 [ label = "BlockPartitionSource" ] - 1 [ label = "SyncReadParquetDataTransform" ] - 2 [ label = "DeserializeDataTransform" ] - 3 [ label = "ShufflePartition(Window)" ] - 4 [ label = "ShuffleMergePartition(Window)" ] - 5 [ label = "TransformWindowPartitionCollect(Sort)" ] - 6 [ label = "Transform Window" ] - 7 [ label = "LimitTransform" ] - 8 [ label = "CompoundBlockOperator(Project)" ] + 1 [ label = "TransformRuntimeFilterWait" ] + 2 [ label = "SyncReadParquetDataTransform" ] + 3 [ label = "DeserializeDataTransform" ] + 4 [ label = "ShufflePartition(Window)" ] + 5 [ label = "ShuffleMergePartition(Window)" ] + 6 [ label = "TransformWindowPartitionCollect(Sort)" ] + 7 [ label = "Transform Window" ] + 8 [ label = "LimitTransform" ] + 9 [ label = "CompoundBlockOperator(Project)" ] 0 -> 1 [ label = "" ] 1 -> 2 [ label = "" ] 2 -> 3 [ label = "" ] @@ -545,6 +567,7 @@ digraph { 5 -> 6 [ label = "" ] 6 -> 7 [ label = "" ] 7 -> 8 [ label = "" ] + 8 -> 9 [ label = "" ] } # rows frame single window (can not push down limit) @@ -553,14 +576,15 @@ explain pipeline select a, sum(a) over (partition by a order by a desc rows betw ---- digraph { 0 [ label = "BlockPartitionSource" ] - 1 [ label = "SyncReadParquetDataTransform" ] - 2 [ label = "DeserializeDataTransform" ] - 3 [ label = "ShufflePartition(Window)" ] - 4 [ label = "ShuffleMergePartition(Window)" ] - 5 [ label = "TransformWindowPartitionCollect(Sort)" ] - 6 [ label = "Transform Window" ] - 7 [ label = "LimitTransform" ] - 8 [ label = "CompoundBlockOperator(Project)" ] + 1 [ label = "TransformRuntimeFilterWait" ] + 2 [ label = "SyncReadParquetDataTransform" ] + 3 [ label = "DeserializeDataTransform" ] + 4 [ label = "ShufflePartition(Window)" ] + 5 [ label = "ShuffleMergePartition(Window)" ] + 6 [ label = "TransformWindowPartitionCollect(Sort)" ] + 7 [ label = "Transform Window" ] + 8 [ label = "LimitTransform" ] + 9 [ label = "CompoundBlockOperator(Project)" ] 0 -> 1 [ label = "" ] 1 -> 2 [ label = "" ] 2 -> 3 [ label = "" ] @@ -569,6 +593,7 @@ digraph { 5 -> 6 [ label = "" ] 6 -> 7 [ label = "" ] 7 -> 8 [ label = "" ] + 8 -> 9 [ label = "" ] } # rows frame multi window (can not push down limit) @@ -578,30 +603,31 @@ avg(a) over (order by a rows between unbounded preceding and current row) from t ---- digraph { 0 [ label = "BlockPartitionSource" ] - 1 [ label = "SyncReadParquetDataTransform" ] - 2 [ label = "DeserializeDataTransform" ] - 3 [ label = "ShufflePartition(Window)" ] - 4 [ label = "ShuffleMergePartition(Window)" ] - 5 [ label = "TransformWindowPartitionCollect(Sort)" ] - 6 [ label = "Transform Window" ] - 7 [ label = "Resize" ] - 8 [ label = "SortPartialTransform" ] + 1 [ label = "TransformRuntimeFilterWait" ] + 2 [ label = "SyncReadParquetDataTransform" ] + 3 [ label = "DeserializeDataTransform" ] + 4 [ label = "ShufflePartition(Window)" ] + 5 [ label = "ShuffleMergePartition(Window)" ] + 6 [ label = "TransformWindowPartitionCollect(Sort)" ] + 7 [ label = "Transform Window" ] + 8 [ label = "Resize" ] 9 [ label = "SortPartialTransform" ] 10 [ label = "SortPartialTransform" ] 11 [ label = "SortPartialTransform" ] - 12 [ label = "TransformSortMerge" ] + 12 [ label = "SortPartialTransform" ] 13 [ label = "TransformSortMerge" ] 14 [ label = "TransformSortMerge" ] 15 [ label = "TransformSortMerge" ] - 16 [ label = "KWayMergePartitioner" ] - 17 [ label = "KWayMergeWorker" ] + 16 [ label = "TransformSortMerge" ] + 17 [ label = "KWayMergePartitioner" ] 18 [ label = "KWayMergeWorker" ] 19 [ label = "KWayMergeWorker" ] 20 [ label = "KWayMergeWorker" ] - 21 [ label = "KWayMergeCombiner" ] - 22 [ label = "Transform Window" ] - 23 [ label = "LimitTransform" ] - 24 [ label = "CompoundBlockOperator(Project)" ] + 21 [ label = "KWayMergeWorker" ] + 22 [ label = "KWayMergeCombiner" ] + 23 [ label = "Transform Window" ] + 24 [ label = "LimitTransform" ] + 25 [ label = "CompoundBlockOperator(Project)" ] 0 -> 1 [ label = "" ] 1 -> 2 [ label = "" ] 2 -> 3 [ label = "" ] @@ -609,29 +635,30 @@ digraph { 4 -> 5 [ label = "" ] 5 -> 6 [ label = "" ] 6 -> 7 [ label = "" ] - 7 -> 8 [ label = "from: 0, to: 0" ] - 7 -> 9 [ label = "from: 1, to: 0" ] - 7 -> 10 [ label = "from: 2, to: 0" ] - 7 -> 11 [ label = "from: 3, to: 0" ] - 8 -> 12 [ label = "" ] + 7 -> 8 [ label = "" ] + 8 -> 9 [ label = "from: 0, to: 0" ] + 8 -> 10 [ label = "from: 1, to: 0" ] + 8 -> 11 [ label = "from: 2, to: 0" ] + 8 -> 12 [ label = "from: 3, to: 0" ] 9 -> 13 [ label = "" ] 10 -> 14 [ label = "" ] 11 -> 15 [ label = "" ] - 12 -> 16 [ label = "from: 0, to: 0" ] - 13 -> 16 [ label = "from: 0, to: 1" ] - 14 -> 16 [ label = "from: 0, to: 2" ] - 15 -> 16 [ label = "from: 0, to: 3" ] - 16 -> 17 [ label = "from: 0, to: 0" ] - 16 -> 18 [ label = "from: 1, to: 0" ] - 16 -> 19 [ label = "from: 2, to: 0" ] - 16 -> 20 [ label = "from: 3, to: 0" ] - 17 -> 21 [ label = "from: 0, to: 0" ] - 18 -> 21 [ label = "from: 0, to: 1" ] - 19 -> 21 [ label = "from: 0, to: 2" ] - 20 -> 21 [ label = "from: 0, to: 3" ] - 21 -> 22 [ label = "" ] + 12 -> 16 [ label = "" ] + 13 -> 17 [ label = "from: 0, to: 0" ] + 14 -> 17 [ label = "from: 0, to: 1" ] + 15 -> 17 [ label = "from: 0, to: 2" ] + 16 -> 17 [ label = "from: 0, to: 3" ] + 17 -> 18 [ label = "from: 0, to: 0" ] + 17 -> 19 [ label = "from: 1, to: 0" ] + 17 -> 20 [ label = "from: 2, to: 0" ] + 17 -> 21 [ label = "from: 3, to: 0" ] + 18 -> 22 [ label = "from: 0, to: 0" ] + 19 -> 22 [ label = "from: 0, to: 1" ] + 20 -> 22 [ label = "from: 0, to: 2" ] + 21 -> 22 [ label = "from: 0, to: 3" ] 22 -> 23 [ label = "" ] 23 -> 24 [ label = "" ] + 24 -> 25 [ label = "" ] } # row fetch with window function(pipeline explain) @@ -640,32 +667,33 @@ explain pipeline select *, sum(a) over (partition by a order by a desc rows betw ---- digraph { 0 [ label = "BlockPartitionSource" ] - 1 [ label = "SyncReadParquetDataTransform" ] - 2 [ label = "DeserializeDataTransform" ] - 3 [ label = "AddInternalColumnsTransform" ] - 4 [ label = "TransformFilter" ] - 5 [ label = "ShufflePartition(Window)" ] - 6 [ label = "ShuffleMergePartition(Window)" ] - 7 [ label = "TransformWindowPartitionCollect(Sort)" ] - 8 [ label = "Transform Window" ] - 9 [ label = "Resize" ] - 10 [ label = "SortPartialTransform" ] + 1 [ label = "TransformRuntimeFilterWait" ] + 2 [ label = "SyncReadParquetDataTransform" ] + 3 [ label = "DeserializeDataTransform" ] + 4 [ label = "AddInternalColumnsTransform" ] + 5 [ label = "TransformFilter" ] + 6 [ label = "ShufflePartition(Window)" ] + 7 [ label = "ShuffleMergePartition(Window)" ] + 8 [ label = "TransformWindowPartitionCollect(Sort)" ] + 9 [ label = "Transform Window" ] + 10 [ label = "Resize" ] 11 [ label = "SortPartialTransform" ] 12 [ label = "SortPartialTransform" ] 13 [ label = "SortPartialTransform" ] - 14 [ label = "TransformSortMergeLimit" ] + 14 [ label = "SortPartialTransform" ] 15 [ label = "TransformSortMergeLimit" ] 16 [ label = "TransformSortMergeLimit" ] 17 [ label = "TransformSortMergeLimit" ] - 18 [ label = "KWayMergePartitioner" ] - 19 [ label = "KWayMergeWorker" ] + 18 [ label = "TransformSortMergeLimit" ] + 19 [ label = "KWayMergePartitioner" ] 20 [ label = "KWayMergeWorker" ] 21 [ label = "KWayMergeWorker" ] 22 [ label = "KWayMergeWorker" ] - 23 [ label = "KWayMergeCombiner" ] - 24 [ label = "LimitTransform" ] - 25 [ label = "TransformRowsFetcher" ] - 26 [ label = "CompoundBlockOperator(Project)" ] + 23 [ label = "KWayMergeWorker" ] + 24 [ label = "KWayMergeCombiner" ] + 25 [ label = "LimitTransform" ] + 26 [ label = "TransformRowsFetcher" ] + 27 [ label = "CompoundBlockOperator(Project)" ] 0 -> 1 [ label = "" ] 1 -> 2 [ label = "" ] 2 -> 3 [ label = "" ] @@ -675,29 +703,30 @@ digraph { 6 -> 7 [ label = "" ] 7 -> 8 [ label = "" ] 8 -> 9 [ label = "" ] - 9 -> 10 [ label = "from: 0, to: 0" ] - 9 -> 11 [ label = "from: 1, to: 0" ] - 9 -> 12 [ label = "from: 2, to: 0" ] - 9 -> 13 [ label = "from: 3, to: 0" ] - 10 -> 14 [ label = "" ] + 9 -> 10 [ label = "" ] + 10 -> 11 [ label = "from: 0, to: 0" ] + 10 -> 12 [ label = "from: 1, to: 0" ] + 10 -> 13 [ label = "from: 2, to: 0" ] + 10 -> 14 [ label = "from: 3, to: 0" ] 11 -> 15 [ label = "" ] 12 -> 16 [ label = "" ] 13 -> 17 [ label = "" ] - 14 -> 18 [ label = "from: 0, to: 0" ] - 15 -> 18 [ label = "from: 0, to: 1" ] - 16 -> 18 [ label = "from: 0, to: 2" ] - 17 -> 18 [ label = "from: 0, to: 3" ] - 18 -> 19 [ label = "from: 0, to: 0" ] - 18 -> 20 [ label = "from: 1, to: 0" ] - 18 -> 21 [ label = "from: 2, to: 0" ] - 18 -> 22 [ label = "from: 3, to: 0" ] - 19 -> 23 [ label = "from: 0, to: 0" ] - 20 -> 23 [ label = "from: 0, to: 1" ] - 21 -> 23 [ label = "from: 0, to: 2" ] - 22 -> 23 [ label = "from: 0, to: 3" ] - 23 -> 24 [ label = "" ] + 14 -> 18 [ label = "" ] + 15 -> 19 [ label = "from: 0, to: 0" ] + 16 -> 19 [ label = "from: 0, to: 1" ] + 17 -> 19 [ label = "from: 0, to: 2" ] + 18 -> 19 [ label = "from: 0, to: 3" ] + 19 -> 20 [ label = "from: 0, to: 0" ] + 19 -> 21 [ label = "from: 1, to: 0" ] + 19 -> 22 [ label = "from: 2, to: 0" ] + 19 -> 23 [ label = "from: 3, to: 0" ] + 20 -> 24 [ label = "from: 0, to: 0" ] + 21 -> 24 [ label = "from: 0, to: 1" ] + 22 -> 24 [ label = "from: 0, to: 2" ] + 23 -> 24 [ label = "from: 0, to: 3" ] 24 -> 25 [ label = "" ] 25 -> 26 [ label = "" ] + 26 -> 27 [ label = "" ] } # row fetch with window function(plan explain) @@ -733,6 +762,7 @@ RowFetch ├── estimated rows: 0.00 └── TableScan ├── table: default.test_explain_window.t + ├── scan id: 0 ├── output columns: [a (#0), b (#1), _row_id (#7)] ├── read rows: 0 ├── read size: 0 @@ -753,15 +783,16 @@ explain pipeline select time, rowkey from (select *, row_number() over(partition ---- digraph { 0 [ label = "BlockPartitionSource" ] - 1 [ label = "SyncReadParquetDataTransform" ] - 2 [ label = "DeserializeDataTransform" ] - 3 [ label = "ShufflePartition(WindowTopN)" ] - 4 [ label = "ShuffleMergePartition(WindowTopN)" ] - 5 [ label = "TransformWindowPartitionCollect(Sort)" ] - 6 [ label = "Transform Window" ] - 7 [ label = "TransformFilter" ] - 8 [ label = "LimitTransform" ] - 9 [ label = "CompoundBlockOperator(Project)" ] + 1 [ label = "TransformRuntimeFilterWait" ] + 2 [ label = "SyncReadParquetDataTransform" ] + 3 [ label = "DeserializeDataTransform" ] + 4 [ label = "ShufflePartition(WindowTopN)" ] + 5 [ label = "ShuffleMergePartition(WindowTopN)" ] + 6 [ label = "TransformWindowPartitionCollect(Sort)" ] + 7 [ label = "Transform Window" ] + 8 [ label = "TransformFilter" ] + 9 [ label = "LimitTransform" ] + 10 [ label = "CompoundBlockOperator(Project)" ] 0 -> 1 [ label = "" ] 1 -> 2 [ label = "" ] 2 -> 3 [ label = "" ] @@ -771,6 +802,7 @@ digraph { 6 -> 7 [ label = "" ] 7 -> 8 [ label = "" ] 8 -> 9 [ label = "" ] + 9 -> 10 [ label = "" ] } # top n 0 @@ -906,6 +938,7 @@ Window ├── estimated rows: 50.00 └── TableScan ├── table: default.system.numbers + ├── scan id: 0 ├── output columns: [number (#0)] ├── read rows: 50 ├── read size: < 1 KiB @@ -962,6 +995,7 @@ Window ├── estimated rows: 50.00 └── TableScan ├── table: default.system.numbers + ├── scan id: 0 ├── output columns: [number (#0)] ├── read rows: 50 ├── read size: < 1 KiB @@ -1003,6 +1037,7 @@ Window ├── estimated rows: 50.00 └── TableScan ├── table: default.system.numbers + ├── scan id: 0 ├── output columns: [number (#0)] ├── read rows: 50 ├── read size: < 1 KiB @@ -1044,6 +1079,7 @@ Window ├── estimated rows: 50.00 └── TableScan ├── table: default.system.numbers + ├── scan id: 0 ├── output columns: [number (#0)] ├── read rows: 50 ├── read size: < 1 KiB @@ -1079,6 +1115,7 @@ EvalScalar ├── estimated rows: 1000.00 └── TableScan ├── table: default.system.numbers + ├── scan id: 0 ├── output columns: [number (#0)] ├── read rows: 1000 ├── read size: 7.81 KiB @@ -1120,6 +1157,7 @@ AggregateFinal ├── estimated rows: 1000.00 └── TableScan ├── table: default.system.numbers + ├── scan id: 0 ├── output columns: [number (#0)] ├── read rows: 1000 ├── read size: 7.81 KiB diff --git a/tests/sqllogictests/suites/mode/standalone/explain_native/aggregate.test b/tests/sqllogictests/suites/mode/standalone/explain_native/aggregate.test index df9061312611e..ad69141ab02df 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain_native/aggregate.test +++ b/tests/sqllogictests/suites/mode/standalone/explain_native/aggregate.test @@ -12,6 +12,7 @@ AggregateFinal ├── estimated rows: 3.33 └── TableScan ├── table: default.system.numbers + ├── scan id: 0 ├── output columns: [number (#0)] ├── read rows: 10 ├── read size: < 1 KiB @@ -35,6 +36,7 @@ AggregateFinal ├── estimated rows: 3.33 └── TableScan ├── table: default.system.numbers + ├── scan id: 0 ├── output columns: [number (#0)] ├── read rows: 10 ├── read size: < 1 KiB @@ -63,6 +65,7 @@ EvalScalar ├── estimated rows: 3.33 └── TableScan ├── table: default.system.numbers + ├── scan id: 0 ├── output columns: [number (#0)] ├── read rows: 10 ├── read size: < 1 KiB @@ -90,6 +93,7 @@ EvalScalar ├── estimated rows: 1.00 └── TableScan ├── table: default.system.columns + ├── scan id: 0 ├── output columns: [name (#0), type (#3)] ├── read rows: 0 ├── read size: 0 @@ -121,6 +125,7 @@ AggregateFinal ├── estimated rows: 1.00 └── TableScan ├── table: default.default.explain_agg_t1 + ├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 0 ├── read size: 0 @@ -143,6 +148,7 @@ AggregateFinal ├── estimated rows: 1.00 └── TableScan ├── table: default.default.explain_agg_t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -173,6 +179,7 @@ EvalScalar ├── estimated rows: 1.00 └── TableScan ├── table: default.default.explain_agg_t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -203,6 +210,7 @@ EvalScalar ├── estimated rows: 1.00 └── TableScan ├── table: default.default.explain_agg_t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -246,6 +254,7 @@ AggregateFinal ├── estimated rows: 1000.00 ├── TableScan(Left) │ ├── table: default.default.t1 + │ ├── scan id: 0 │ ├── output columns: [a (#0)] │ ├── read rows: 10 │ ├── read size: < 1 KiB @@ -256,6 +265,7 @@ AggregateFinal │ └── estimated rows: 10.00 └── TableScan(Right) ├── table: default.default.t2 + ├── scan id: 1 ├── output columns: [a (#1)] ├── read rows: 100 ├── read size: < 1 KiB @@ -296,6 +306,7 @@ EvalScalar ├── estimated rows: 1.00 └── TableScan ├── table: default.default.t + ├── scan id: 0 ├── output columns: [referer (#0), isrefresh (#1)] ├── read rows: 0 ├── read size: 0 diff --git a/tests/sqllogictests/suites/mode/standalone/explain_native/bloom_filter.test b/tests/sqllogictests/suites/mode/standalone/explain_native/bloom_filter.test index 5e4c2ccc2caf1..e4cffcdd496f7 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain_native/bloom_filter.test +++ b/tests/sqllogictests/suites/mode/standalone/explain_native/bloom_filter.test @@ -19,6 +19,7 @@ explain select * from bloom_test_t where c1 = 5 ---- TableScan ├── table: default.default.bloom_test_t +├── scan id: 0 ├── output columns: [c1 (#0), c2 (#1)] ├── read rows: 3 ├── read size: < 1 KiB @@ -94,6 +95,7 @@ explain select * from bloom_test_t where c2=0; ---- TableScan ├── table: default.default.bloom_test_t +├── scan id: 0 ├── output columns: [c1 (#0), c2 (#1)] ├── read rows: 0 ├── read size: 0 @@ -108,6 +110,7 @@ explain select * from bloom_test_t where c2=3; ---- TableScan ├── table: default.default.bloom_test_t +├── scan id: 0 ├── output columns: [c1 (#0), c2 (#1)] ├── read rows: 0 ├── read size: 0 @@ -137,6 +140,7 @@ explain select * from bloom_test_nullable_t where c1 = 5 and c2 > 1; ---- TableScan ├── table: default.default.bloom_test_nullable_t +├── scan id: 0 ├── output columns: [c1 (#0), c2 (#1)] ├── read rows: 3 ├── read size: < 1 KiB @@ -163,6 +167,7 @@ explain select * from bloom_test_nullable_t2 where ((c2) in ('1') is null); ---- TableScan ├── table: default.default.bloom_test_nullable_t2 +├── scan id: 0 ├── output columns: [c0 (#0), c1 (#1), c2 (#2)] ├── read rows: 2 ├── read size: < 1 KiB @@ -177,6 +182,7 @@ explain select * from bloom_test_nullable_t2 where (not (not c0)) ---- TableScan ├── table: default.default.bloom_test_nullable_t2 +├── scan id: 0 ├── output columns: [c0 (#0), c1 (#1), c2 (#2)] ├── read rows: 2 ├── read size: < 1 KiB diff --git a/tests/sqllogictests/suites/mode/standalone/explain_native/explain.test b/tests/sqllogictests/suites/mode/standalone/explain_native/explain.test index f41bfd9fd9376..925c60aa1b5a6 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain_native/explain.test +++ b/tests/sqllogictests/suites/mode/standalone/explain_native/explain.test @@ -15,6 +15,7 @@ explain select t1.a from t1 where a > 0 ---- TableScan ├── table: default.default.t1 +├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 0 ├── read size: 0 @@ -39,10 +40,11 @@ Filter ├── keys is null equal: [false] ├── filters: [] ├── build join filters: - │ └── filter id:0, build key:t1.a (#0), probe key:t2.a (#2), filter type:bloom,inlist,min_max + │ └── filter id:0, build key:t1.a (#0), probe targets:[t2.a (#2)@scan1], filter type:bloom,inlist,min_max ├── estimated rows: 0.00 ├── TableScan(Build) │ ├── table: default.default.t1 + │ ├── scan id: 0 │ ├── output columns: [a (#0), b (#1)] │ ├── read rows: 0 │ ├── read size: 0 @@ -53,6 +55,7 @@ Filter │ └── estimated rows: 0.00 └── TableScan(Probe) ├── table: default.default.t2 + ├── scan id: 1 ├── output columns: [a (#2), b (#3)] ├── read rows: 5 ├── read size: < 1 KiB @@ -74,10 +77,11 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:0, build key:t1.a (#0), probe key:t2.a (#2), filter type:inlist,min_max +│ └── filter id:0, build key:t1.a (#0), probe targets:[t2.a (#2)@scan1], filter type:bloom,inlist,min_max ├── estimated rows: 1.00 ├── TableScan(Build) │ ├── table: default.default.t1 +│ ├── scan id: 0 │ ├── output columns: [a (#0), b (#1)] │ ├── read rows: 1 │ ├── read size: < 1 KiB @@ -88,6 +92,7 @@ HashJoin │ └── estimated rows: 1.00 └── TableScan(Probe) ├── table: default.default.t2 + ├── scan id: 1 ├── output columns: [a (#2), b (#3)] ├── read rows: 5 ├── read size: < 1 KiB @@ -278,6 +283,7 @@ UnionAll ├── estimated rows: 6.00 ├── TableScan │ ├── table: default.default.t1 +│ ├── scan id: 0 │ ├── output columns: [a (#0)] │ ├── read rows: 1 │ ├── read size: < 1 KiB @@ -288,6 +294,7 @@ UnionAll │ └── estimated rows: 1.00 └── TableScan ├── table: default.default.t2 + ├── scan id: 1 ├── output columns: [a (#2)] ├── read rows: 5 ├── read size: < 1 KiB @@ -314,6 +321,7 @@ Filter ├── estimated rows: 4.40 ├── TableScan(Build) │ ├── table: default.default.t1 + │ ├── scan id: 0 │ ├── output columns: [a (#0), b (#1)] │ ├── read rows: 1 │ ├── read size: < 1 KiB @@ -324,6 +332,7 @@ Filter │ └── estimated rows: 1.00 └── TableScan(Probe) ├── table: default.default.t2 + ├── scan id: 1 ├── output columns: [a (#2), b (#3)] ├── read rows: 5 ├── read size: < 1 KiB @@ -350,6 +359,7 @@ Filter ├── estimated rows: 5.00 ├── TableScan(Build) │ ├── table: default.default.t1 + │ ├── scan id: 0 │ ├── output columns: [a (#0), b (#1)] │ ├── read rows: 1 │ ├── read size: < 1 KiB @@ -360,6 +370,7 @@ Filter │ └── estimated rows: 1.00 └── TableScan(Probe) ├── table: default.default.t2 + ├── scan id: 1 ├── output columns: [a (#2), b (#3)] ├── read rows: 5 ├── read size: < 1 KiB @@ -396,6 +407,7 @@ HashJoin │ ├── estimated rows: 5.00 │ ├── TableScan(Build) │ │ ├── table: default.default.t1 +│ │ ├── scan id: 0 │ │ ├── output columns: [a (#0), b (#1)] │ │ ├── read rows: 1 │ │ ├── read size: < 1 KiB @@ -406,6 +418,7 @@ HashJoin │ │ └── estimated rows: 1.00 │ └── TableScan(Probe) │ ├── table: default.default.t2 +│ ├── scan id: 1 │ ├── output columns: [a (#2), b (#3)] │ ├── read rows: 5 │ ├── read size: < 1 KiB @@ -416,6 +429,7 @@ HashJoin │ └── estimated rows: 5.00 └── TableScan(Probe) ├── table: default.default.t3 + ├── scan id: 2 ├── output columns: [a (#4), b (#5)] ├── read rows: 10 ├── read size: < 1 KiB @@ -450,6 +464,7 @@ HashJoin │ ├── estimated rows: 4.40 │ ├── TableScan(Build) │ │ ├── table: default.default.t1 +│ │ ├── scan id: 0 │ │ ├── output columns: [a (#0), b (#1)] │ │ ├── read rows: 1 │ │ ├── read size: < 1 KiB @@ -460,6 +475,7 @@ HashJoin │ │ └── estimated rows: 1.00 │ └── TableScan(Probe) │ ├── table: default.default.t2 +│ ├── scan id: 1 │ ├── output columns: [a (#2), b (#3)] │ ├── read rows: 5 │ ├── read size: < 1 KiB @@ -470,6 +486,7 @@ HashJoin │ └── estimated rows: 4.40 └── TableScan(Probe) ├── table: default.default.t3 + ├── scan id: 2 ├── output columns: [a (#4), b (#5)] ├── read rows: 10 ├── read size: < 1 KiB @@ -505,6 +522,7 @@ Limit ├── estimated rows: 4.40 ├── TableScan(Build) │ ├── table: default.default.t1 + │ ├── scan id: 0 │ ├── output columns: [a (#0), b (#1)] │ ├── read rows: 1 │ ├── read size: < 1 KiB @@ -515,6 +533,7 @@ Limit │ └── estimated rows: 1.00 └── TableScan(Probe) ├── table: default.default.t2 + ├── scan id: 1 ├── output columns: [a (#2), b (#3)] ├── read rows: 5 ├── read size: < 1 KiB @@ -537,6 +556,7 @@ HashJoin ├── estimated rows: 5.00 ├── TableScan(Build) │ ├── table: default.default.t1 +│ ├── scan id: 0 │ ├── output columns: [a (#0), b (#1)] │ ├── read rows: 1 │ ├── read size: < 1 KiB @@ -547,6 +567,7 @@ HashJoin │ └── estimated rows: 1.00 └── TableScan(Probe) ├── table: default.default.t2 + ├── scan id: 1 ├── output columns: [a (#2), b (#3)] ├── read rows: 5 ├── read size: < 1 KiB @@ -579,6 +600,7 @@ AggregateFinal ├── estimated rows: 1.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 1 ├── read size: < 1 KiB @@ -611,6 +633,7 @@ AggregateFinal ├── estimated rows: 1.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 1 ├── read size: < 1 KiB @@ -665,6 +688,7 @@ explain select * from t4 where a = 1 and try_cast(get(try_parse_json(b),'bb') as ---- TableScan ├── table: default.default.t4 +├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -688,6 +712,7 @@ EvalScalar ├── estimated rows: 0.00 └── TableScan ├── table: default.default.t4 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -705,6 +730,7 @@ EvalScalar ├── estimated rows: 0.00 └── TableScan ├── table: default.default.t4 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -749,6 +775,7 @@ Sort(Single) ├── EmptyResultScan(Build) └── TableScan(Probe) ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [i (#0), j (#1)] ├── read rows: 3 ├── read size: < 1 KiB @@ -775,6 +802,7 @@ EXPLAIN SELECT * FROM t3 WHERE b['k2'] = 'm'; ---- TableScan ├── table: default.default.t3 +├── scan id: 0 ├── output columns: [a (#0), b (#1), c (#2)] ├── read rows: 3 ├── read size: < 1 KiB @@ -789,6 +817,7 @@ EXPLAIN SELECT * FROM t3 WHERE c['k3'] = 'v'; ---- TableScan ├── table: default.default.t3 +├── scan id: 0 ├── output columns: [a (#0), b (#1), c (#2)] ├── read rows: 3 ├── read size: < 1 KiB @@ -803,6 +832,7 @@ EXPLAIN SELECT * FROM t3 WHERE c['k2'] = 100; ---- TableScan ├── table: default.default.t3 +├── scan id: 0 ├── output columns: [a (#0), b (#1), c (#2)] ├── read rows: 6 ├── read size: < 1 KiB @@ -833,6 +863,7 @@ EXPLAIN SELECT * FROM t WHERE a.1 > 0; ---- TableScan ├── table: default.default.t +├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 2 ├── read size: < 1 KiB @@ -847,6 +878,7 @@ EXPLAIN SELECT * FROM t WHERE a.1 > 1; ---- TableScan ├── table: default.default.t +├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 2 ├── read size: < 1 KiB @@ -861,6 +893,7 @@ EXPLAIN SELECT * FROM t WHERE a.2 > 1; ---- TableScan ├── table: default.default.t +├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 2 ├── read size: < 1 KiB diff --git a/tests/sqllogictests/suites/mode/standalone/explain_native/explain_grouping_sets.test b/tests/sqllogictests/suites/mode/standalone/explain_native/explain_grouping_sets.test index 1573e610ea0f1..32596f3fa45ff 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain_native/explain_grouping_sets.test +++ b/tests/sqllogictests/suites/mode/standalone/explain_native/explain_grouping_sets.test @@ -24,6 +24,7 @@ EvalScalar ├── estimated rows: 1.00 └── TableScan ├── table: default.system.numbers + ├── scan id: 0 ├── output columns: [number (#0)] ├── read rows: 1 ├── read size: < 1 KiB @@ -58,6 +59,7 @@ EvalScalar ├── estimated rows: 1.00 └── TableScan ├── table: default.system.numbers + ├── scan id: 0 ├── output columns: [number (#0)] ├── read rows: 1 ├── read size: < 1 KiB diff --git a/tests/sqllogictests/suites/mode/standalone/explain_native/explain_pipeline.test b/tests/sqllogictests/suites/mode/standalone/explain_native/explain_pipeline.test index 709e3b7bca620..ae33d7f56311c 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain_native/explain_pipeline.test +++ b/tests/sqllogictests/suites/mode/standalone/explain_native/explain_pipeline.test @@ -17,12 +17,14 @@ explain pipeline select a from t1 ignore_result ---- digraph { 0 [ label = "BlockPartitionSource" ] - 1 [ label = "SyncReadNativeDataTransform" ] - 2 [ label = "NativeDeserializeDataTransform" ] - 3 [ label = "EmptySink" ] + 1 [ label = "TransformRuntimeFilterWait" ] + 2 [ label = "SyncReadNativeDataTransform" ] + 3 [ label = "NativeDeserializeDataTransform" ] + 4 [ label = "EmptySink" ] 0 -> 1 [ label = "" ] 1 -> 2 [ label = "" ] 2 -> 3 [ label = "" ] + 3 -> 4 [ label = "" ] } statement ok diff --git a/tests/sqllogictests/suites/mode/standalone/explain_native/expression_scan.test b/tests/sqllogictests/suites/mode/standalone/explain_native/expression_scan.test index 26a5024bfcbee..95b14704ccf2d 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain_native/expression_scan.test +++ b/tests/sqllogictests/suites/mode/standalone/explain_native/expression_scan.test @@ -31,6 +31,7 @@ HashJoin ├── estimated rows: 3.00 ├── TableScan(Build) │ ├── table: default.default.t1 +│ ├── scan id: 0 │ ├── output columns: [a (#0), b (#1)] │ ├── read rows: 3 │ ├── read size: < 1 KiB @@ -72,6 +73,7 @@ HashJoin ├── estimated rows: 6.00 ├── TableScan(Build) │ ├── table: default.default.t1 +│ ├── scan id: 0 │ ├── output columns: [a (#0), b (#1), c (#2)] │ ├── read rows: 3 │ ├── read size: < 1 KiB @@ -113,6 +115,7 @@ HashJoin ├── estimated rows: 6.00 ├── TableScan(Build) │ ├── table: default.default.t1 +│ ├── scan id: 0 │ ├── output columns: [a (#0), b (#1), c (#2)] │ ├── read rows: 3 │ ├── read size: < 1 KiB @@ -168,6 +171,7 @@ HashJoin │ ├── estimated rows: 6.00 │ ├── TableScan(Build) │ │ ├── table: default.default.t1 +│ │ ├── scan id: 0 │ │ ├── output columns: [a (#0), b (#1), c (#2)] │ │ ├── read rows: 3 │ │ ├── read size: < 1 KiB @@ -240,6 +244,7 @@ HashJoin │ ├── estimated rows: 6.00 │ ├── TableScan(Build) │ │ ├── table: default.default.t1 +│ │ ├── scan id: 0 │ │ ├── output columns: [a (#0), b (#1), c (#2)] │ │ ├── read rows: 3 │ │ ├── read size: < 1 KiB diff --git a/tests/sqllogictests/suites/mode/standalone/explain_native/filter.test b/tests/sqllogictests/suites/mode/standalone/explain_native/filter.test index 446fe5a84da76..42b9cb298196e 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain_native/filter.test +++ b/tests/sqllogictests/suites/mode/standalone/explain_native/filter.test @@ -15,6 +15,7 @@ explain select * from t1 where (a = 1 and b > 2) or (a = 1 and b < 100) or (a = ---- TableScan ├── table: default.default.t1 +├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -28,6 +29,7 @@ explain select * from t1 where b > 2 or b > 2 or b < 100; ---- TableScan ├── table: default.default.t1 +├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 diff --git a/tests/sqllogictests/suites/mode/standalone/explain_native/fold_count.test b/tests/sqllogictests/suites/mode/standalone/explain_native/fold_count.test index 72a31598959f9..c74d30c7410c6 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain_native/fold_count.test +++ b/tests/sqllogictests/suites/mode/standalone/explain_native/fold_count.test @@ -39,6 +39,7 @@ AggregateFinal ├── estimated rows: 1.00 └── TableScan ├── table: default.default.t + ├── scan id: 0 ├── output columns: [] ├── read rows: 1000 ├── read size: 3.92 KiB diff --git a/tests/sqllogictests/suites/mode/standalone/explain_native/infer_filter.test b/tests/sqllogictests/suites/mode/standalone/explain_native/infer_filter.test index 02211e007a1f4..d57e1b871c5b2 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain_native/infer_filter.test +++ b/tests/sqllogictests/suites/mode/standalone/explain_native/infer_filter.test @@ -28,6 +28,7 @@ explain select * from t1 where a = 1 and a = 1; ---- TableScan ├── table: default.default.t1 +├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -54,6 +55,7 @@ explain select * from t1 where a = 1 and a != 2; ---- TableScan ├── table: default.default.t1 +├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -68,6 +70,7 @@ explain select * from t1 where a = 1 and a < 2; ---- TableScan ├── table: default.default.t1 +├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -88,6 +91,7 @@ explain select * from t1 where a = 1 and a <= 2; ---- TableScan ├── table: default.default.t1 +├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -102,6 +106,7 @@ explain select * from t1 where a = 1 and a <= 1; ---- TableScan ├── table: default.default.t1 +├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -116,6 +121,7 @@ explain select * from t1 where a = 1 and a > 0; ---- TableScan ├── table: default.default.t1 +├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -136,6 +142,7 @@ explain select * from t1 where a = 1 and a >= 0; ---- TableScan ├── table: default.default.t1 +├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -150,6 +157,7 @@ explain select * from t1 where a = 1 and a >= 1; ---- TableScan ├── table: default.default.t1 +├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -171,6 +179,7 @@ explain select * from t1 where a != 1 and a = 2; ---- TableScan ├── table: default.default.t1 +├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -185,6 +194,7 @@ explain select * from t1 where a != 1 and a != 1 ---- TableScan ├── table: default.default.t1 +├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -199,6 +209,7 @@ explain select * from t1 where a != 1 and a != 2; ---- TableScan ├── table: default.default.t1 +├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -213,6 +224,7 @@ explain select * from t1 where a != 1 and a < 1; ---- TableScan ├── table: default.default.t1 +├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -227,6 +239,7 @@ explain select * from t1 where a != 1 and a < 2; ---- TableScan ├── table: default.default.t1 +├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -241,6 +254,7 @@ explain select * from t1 where a != 1 and a <= 1; ---- TableScan ├── table: default.default.t1 +├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -255,6 +269,7 @@ explain select * from t1 where a != 1 and a <= 2; ---- TableScan ├── table: default.default.t1 +├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -269,6 +284,7 @@ explain select * from t1 where a != 1 and a > 1; ---- TableScan ├── table: default.default.t1 +├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -283,6 +299,7 @@ explain select * from t1 where a != 1 and a > 0; ---- TableScan ├── table: default.default.t1 +├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -297,6 +314,7 @@ explain select * from t1 where a != 1 and a >= 1; ---- TableScan ├── table: default.default.t1 +├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -311,6 +329,7 @@ explain select * from t1 where a != 1 and a >= 0; ---- TableScan ├── table: default.default.t1 +├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -331,6 +350,7 @@ explain select * from t1 where a < 5 and a = 2; ---- TableScan ├── table: default.default.t1 +├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -345,6 +365,7 @@ explain select * from t1 where a < 5 and a != 10 ---- TableScan ├── table: default.default.t1 +├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -359,6 +380,7 @@ explain select * from t1 where a < 5 and a != 2; ---- TableScan ├── table: default.default.t1 +├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -373,6 +395,7 @@ explain select * from t1 where a < 5 and a <= 10; ---- TableScan ├── table: default.default.t1 +├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -393,6 +416,7 @@ explain select * from t1 where a < 5 and a > 2; ---- TableScan ├── table: default.default.t1 +├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -407,6 +431,7 @@ explain select * from t1 where 10000 > a and a > 10 and 100 >= a; ---- TableScan ├── table: default.default.t1 +├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -427,10 +452,11 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:0, build key:t2.a (#2), probe key:t1.a (#0), filter type:bloom,inlist,min_max +│ └── filter id:0, build key:t2.a (#2), probe targets:[t1.a (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 0.00 ├── TableScan(Build) │ ├── table: default.default.t2 +│ ├── scan id: 1 │ ├── output columns: [a (#2), b (#3)] │ ├── read rows: 0 │ ├── read size: 0 @@ -440,6 +466,7 @@ HashJoin │ └── estimated rows: 0.00 └── TableScan(Probe) ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -461,7 +488,7 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:1, build key:t3.a (#4), probe key:t1.a (#0), filter type:bloom,inlist,min_max +│ └── filter id:1, build key:t3.a (#4), probe targets:[t1.a (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 0.00 ├── HashJoin(Build) │ ├── output columns: [t2.a (#2), t2.b (#3), t3.b (#5), t3.a (#4)] @@ -471,10 +498,11 @@ HashJoin │ ├── keys is null equal: [false] │ ├── filters: [] │ ├── build join filters: -│ │ └── filter id:0, build key:t3.a (#4), probe key:t2.a (#2), filter type:bloom,inlist,min_max +│ │ └── filter id:0, build key:t3.a (#4), probe targets:[t2.a (#2)@scan1], filter type:bloom,inlist,min_max │ ├── estimated rows: 0.00 │ ├── TableScan(Build) │ │ ├── table: default.default.t3 +│ │ ├── scan id: 2 │ │ ├── output columns: [a (#4), b (#5)] │ │ ├── read rows: 0 │ │ ├── read size: 0 @@ -484,6 +512,7 @@ HashJoin │ │ └── estimated rows: 0.00 │ └── TableScan(Probe) │ ├── table: default.default.t2 +│ ├── scan id: 1 │ ├── output columns: [a (#2), b (#3)] │ ├── read rows: 0 │ ├── read size: 0 @@ -494,6 +523,7 @@ HashJoin │ └── estimated rows: 0.00 └── TableScan(Probe) ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -515,6 +545,7 @@ MergeJoin ├── estimated rows: 0.00 ├── TableScan(Left) │ ├── table: default.default.t2 +│ ├── scan id: 1 │ ├── output columns: [a (#2), b (#3)] │ ├── read rows: 0 │ ├── read size: 0 @@ -524,6 +555,7 @@ MergeJoin │ └── estimated rows: 0.00 └── TableScan(Right) ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -544,6 +576,7 @@ MergeJoin ├── estimated rows: 0.00 ├── TableScan(Left) │ ├── table: default.default.t2 +│ ├── scan id: 1 │ ├── output columns: [a (#2), b (#3)] │ ├── read rows: 0 │ ├── read size: 0 @@ -553,6 +586,7 @@ MergeJoin │ └── estimated rows: 0.00 └── TableScan(Right) ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -585,7 +619,7 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:1, build key:t3.a (#4), probe key:t1.a (#0), filter type:bloom,inlist,min_max +│ └── filter id:1, build key:t3.a (#4), probe targets:[t1.a (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 0.00 ├── HashJoin(Build) │ ├── output columns: [t2.a (#2), t2.b (#3), t3.b (#5), t3.a (#4)] @@ -595,10 +629,11 @@ HashJoin │ ├── keys is null equal: [false] │ ├── filters: [] │ ├── build join filters: -│ │ └── filter id:0, build key:t3.a (#4), probe key:t2.a (#2), filter type:bloom,inlist,min_max +│ │ └── filter id:0, build key:t3.a (#4), probe targets:[t2.a (#2)@scan1], filter type:bloom,inlist,min_max │ ├── estimated rows: 0.00 │ ├── TableScan(Build) │ │ ├── table: default.default.t3 +│ │ ├── scan id: 2 │ │ ├── output columns: [a (#4), b (#5)] │ │ ├── read rows: 0 │ │ ├── read size: 0 @@ -608,6 +643,7 @@ HashJoin │ │ └── estimated rows: 0.00 │ └── TableScan(Probe) │ ├── table: default.default.t2 +│ ├── scan id: 1 │ ├── output columns: [a (#2), b (#3)] │ ├── read rows: 0 │ ├── read size: 0 @@ -618,6 +654,7 @@ HashJoin │ └── estimated rows: 0.00 └── TableScan(Probe) ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -632,6 +669,7 @@ explain select * from t4 where a = b and strip_null_value(b) is not null; ---- TableScan ├── table: default.default.t4 +├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -651,10 +689,11 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:0, build key:CAST(t4.b (#3) AS Int32 NULL), probe key:t3.b (#1), filter type:bloom,inlist,min_max +│ └── filter id:0, build key:CAST(t4.b (#3) AS Int32 NULL), probe targets:[t3.b (#1)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 0.00 ├── TableScan(Build) │ ├── table: default.default.t4 +│ ├── scan id: 1 │ ├── output columns: [a (#2), b (#3)] │ ├── read rows: 0 │ ├── read size: 0 @@ -664,6 +703,7 @@ HashJoin │ └── estimated rows: 0.00 └── TableScan(Probe) ├── table: default.default.t3 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -711,6 +751,7 @@ HashJoin ├── estimated rows: 1.00 ├── TableScan(Build) │ ├── table: default.default.t2 +│ ├── scan id: 1 │ ├── output columns: [id (#1)] │ ├── read rows: 1 │ ├── read size: < 1 KiB @@ -721,6 +762,7 @@ HashJoin │ └── estimated rows: 1.00 └── TableScan(Probe) ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [id (#0)] ├── read rows: 1 ├── read size: < 1 KiB diff --git a/tests/sqllogictests/suites/mode/standalone/explain_native/join.test b/tests/sqllogictests/suites/mode/standalone/explain_native/join.test index 933c16c21c4d2..31f9e5d2d979c 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain_native/join.test +++ b/tests/sqllogictests/suites/mode/standalone/explain_native/join.test @@ -27,10 +27,11 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:0, build key:t.number (#0), probe key:t1.number (#1), filter type:inlist,min_max +│ └── filter id:0, build key:t.number (#0), probe targets:[t1.number (#1)@scan1], filter type:bloom,inlist,min_max ├── estimated rows: 1.00 ├── TableScan(Build) │ ├── table: default.default.t +│ ├── scan id: 0 │ ├── output columns: [number (#0)] │ ├── read rows: 1 │ ├── read size: < 1 KiB @@ -41,6 +42,7 @@ HashJoin │ └── estimated rows: 1.00 └── TableScan(Probe) ├── table: default.default.t1 + ├── scan id: 1 ├── output columns: [number (#1)] ├── read rows: 10 ├── read size: < 1 KiB @@ -62,10 +64,11 @@ HashJoin ├── keys is null equal: [false, false] ├── filters: [] ├── build join filters: -│ └── filter id:0, build key:t.number (#0), probe key:t1.number (#1), filter type:inlist,min_max +│ └── filter id:0, build key:t.number (#0), probe targets:[t1.number (#1)@scan1], filter type:bloom,inlist,min_max ├── estimated rows: 1.00 ├── TableScan(Build) │ ├── table: default.default.t +│ ├── scan id: 0 │ ├── output columns: [number (#0)] │ ├── read rows: 1 │ ├── read size: < 1 KiB @@ -76,6 +79,7 @@ HashJoin │ └── estimated rows: 1.00 └── TableScan(Probe) ├── table: default.default.t1 + ├── scan id: 1 ├── output columns: [number (#1)] ├── read rows: 10 ├── read size: < 1 KiB @@ -99,6 +103,7 @@ HashJoin ├── estimated rows: 0.00 ├── TableScan(Build) │ ├── table: default.default.t +│ ├── scan id: 0 │ ├── output columns: [number (#0)] │ ├── read rows: 0 │ ├── read size: 0 @@ -109,6 +114,7 @@ HashJoin │ └── estimated rows: 0.00 └── TableScan(Probe) ├── table: default.default.t1 + ├── scan id: 1 ├── output columns: [] ├── read rows: 10 ├── read size: < 1 KiB @@ -135,6 +141,7 @@ Filter ├── estimated rows: 10.00 ├── TableScan(Build) │ ├── table: default.default.t + │ ├── scan id: 0 │ ├── output columns: [number (#0)] │ ├── read rows: 1 │ ├── read size: < 1 KiB @@ -145,6 +152,7 @@ Filter │ └── estimated rows: 1.00 └── TableScan(Probe) ├── table: default.default.t1 + ├── scan id: 1 ├── output columns: [number (#1)] ├── read rows: 10 ├── read size: < 1 KiB @@ -166,7 +174,7 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:0, build key:t1.number (#1), probe key:t2.number (#2), filter type:bloom,inlist,min_max +│ └── filter id:0, build key:t1.number (#1), probe targets:[t2.number (#2)@scan2], filter type:bloom,inlist,min_max ├── estimated rows: 0.00 ├── HashJoin(Build) │ ├── output columns: [t1.number (#1), t.number (#0)] @@ -178,6 +186,7 @@ HashJoin │ ├── estimated rows: 0.00 │ ├── TableScan(Build) │ │ ├── table: default.default.t +│ │ ├── scan id: 0 │ │ ├── output columns: [number (#0)] │ │ ├── read rows: 0 │ │ ├── read size: 0 @@ -188,6 +197,7 @@ HashJoin │ │ └── estimated rows: 0.00 │ └── TableScan(Probe) │ ├── table: default.default.t1 +│ ├── scan id: 1 │ ├── output columns: [number (#1)] │ ├── read rows: 10 │ ├── read size: < 1 KiB @@ -198,6 +208,7 @@ HashJoin │ └── estimated rows: 10.00 └── TableScan(Probe) ├── table: default.default.t2 + ├── scan id: 2 ├── output columns: [number (#2)] ├── read rows: 100 ├── read size: < 1 KiB @@ -239,10 +250,11 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:0, build key:a.x (#0), probe key:b.x (#1), filter type:inlist,min_max +│ └── filter id:0, build key:a.x (#0), probe targets:[b.x (#1)@scan1], filter type:bloom,inlist,min_max ├── estimated rows: 2.67 ├── TableScan(Build) │ ├── table: default.default.onecolumn +│ ├── scan id: 0 │ ├── output columns: [x (#0)] │ ├── read rows: 4 │ ├── read size: < 1 KiB @@ -253,6 +265,7 @@ HashJoin │ └── estimated rows: 2.00 └── TableScan(Probe) ├── table: default.default.twocolumn + ├── scan id: 1 ├── output columns: [x (#1), y (#2)] ├── read rows: 4 ├── read size: < 1 KiB @@ -274,10 +287,11 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:0, build key:a.x (#0), probe key:b.x (#1), filter type:inlist,min_max +│ └── filter id:0, build key:a.x (#0), probe targets:[b.x (#1)@scan1], filter type:bloom,inlist,min_max ├── estimated rows: 1.33 ├── TableScan(Build) │ ├── table: default.default.onecolumn +│ ├── scan id: 0 │ ├── output columns: [x (#0)] │ ├── read rows: 4 │ ├── read size: < 1 KiB @@ -288,6 +302,7 @@ HashJoin │ └── estimated rows: 1.33 └── TableScan(Probe) ├── table: default.default.twocolumn + ├── scan id: 1 ├── output columns: [x (#1), y (#2)] ├── read rows: 4 ├── read size: < 1 KiB @@ -309,10 +324,11 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:0, build key:b.x (#1), probe key:a.x (#0), filter type:inlist,min_max +│ └── filter id:0, build key:b.x (#1), probe targets:[a.x (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 4.00 ├── TableScan(Build) │ ├── table: default.default.twocolumn +│ ├── scan id: 1 │ ├── output columns: [x (#1), y (#2)] │ ├── read rows: 4 │ ├── read size: < 1 KiB @@ -323,6 +339,7 @@ HashJoin │ └── estimated rows: 2.00 └── TableScan(Probe) ├── table: default.default.onecolumn + ├── scan id: 0 ├── output columns: [x (#0)] ├── read rows: 4 ├── read size: < 1 KiB @@ -352,6 +369,7 @@ Filter ├── estimated rows: 5.33 ├── TableScan(Build) │ ├── table: default.default.twocolumn + │ ├── scan id: 1 │ ├── output columns: [x (#1), y (#2)] │ ├── read rows: 4 │ ├── read size: < 1 KiB @@ -362,6 +380,7 @@ Filter │ └── estimated rows: 4.00 └── TableScan(Probe) ├── table: default.default.onecolumn + ├── scan id: 0 ├── output columns: [x (#0)] ├── read rows: 4 ├── read size: < 1 KiB @@ -384,6 +403,7 @@ HashJoin ├── estimated rows: 4.00 ├── TableScan(Build) │ ├── table: default.default.onecolumn +│ ├── scan id: 0 │ ├── output columns: [x (#0)] │ ├── read rows: 4 │ ├── read size: < 1 KiB @@ -394,6 +414,7 @@ HashJoin │ └── estimated rows: 2.00 └── TableScan(Probe) ├── table: default.default.twocolumn + ├── scan id: 1 ├── output columns: [x (#1), y (#2)] ├── read rows: 4 ├── read size: < 1 KiB @@ -429,10 +450,11 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:0, build key:t2.a (#2), probe key:t1.a (#0), filter type:bloom,inlist,min_max +│ └── filter id:0, build key:t2.a (#2), probe targets:[t1.a (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 0.00 ├── TableScan(Build) │ ├── table: default.default.t2 +│ ├── scan id: 1 │ ├── output columns: [a (#2), b (#3)] │ ├── read rows: 0 │ ├── read size: 0 @@ -442,6 +464,7 @@ HashJoin │ └── estimated rows: 0.00 └── TableScan(Probe) ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -462,10 +485,11 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:0, build key:t2.a (#2), probe key:t1.a (#0), filter type:bloom,inlist,min_max +│ └── filter id:0, build key:t2.a (#2), probe targets:[t1.a (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 0.00 ├── TableScan(Build) │ ├── table: default.default.t2 +│ ├── scan id: 1 │ ├── output columns: [a (#2), b (#3)] │ ├── read rows: 0 │ ├── read size: 0 @@ -475,6 +499,7 @@ HashJoin │ └── estimated rows: 0.00 └── TableScan(Probe) ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -495,11 +520,12 @@ HashJoin ├── keys is null equal: [false, false] ├── filters: [] ├── build join filters: -│ ├── filter id:0, build key:t2.a (#2), probe key:t1.a (#0), filter type:bloom,inlist,min_max -│ └── filter id:1, build key:t2.b (#3), probe key:t1.b (#1), filter type:bloom,inlist,min_max +│ ├── filter id:0, build key:t2.a (#2), probe targets:[t1.a (#0)@scan0], filter type:bloom,inlist,min_max +│ └── filter id:1, build key:t2.b (#3), probe targets:[t1.b (#1)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 0.00 ├── TableScan(Build) │ ├── table: default.default.t2 +│ ├── scan id: 1 │ ├── output columns: [a (#2), b (#3)] │ ├── read rows: 0 │ ├── read size: 0 @@ -509,6 +535,7 @@ HashJoin │ └── estimated rows: 0.00 └── TableScan(Probe) ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -529,11 +556,12 @@ HashJoin ├── keys is null equal: [false, false] ├── filters: [] ├── build join filters: -│ ├── filter id:0, build key:t2.a (#2), probe key:t1.a (#0), filter type:bloom,inlist,min_max -│ └── filter id:1, build key:t2.b (#3), probe key:t1.b (#1), filter type:bloom,inlist,min_max +│ ├── filter id:0, build key:t2.a (#2), probe targets:[t1.a (#0)@scan0], filter type:bloom,inlist,min_max +│ └── filter id:1, build key:t2.b (#3), probe targets:[t1.b (#1)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 0.00 ├── TableScan(Build) │ ├── table: default.default.t2 +│ ├── scan id: 1 │ ├── output columns: [a (#2), b (#3)] │ ├── read rows: 0 │ ├── read size: 0 @@ -543,6 +571,7 @@ HashJoin │ └── estimated rows: 0.00 └── TableScan(Probe) ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -563,11 +592,12 @@ HashJoin ├── keys is null equal: [false, false] ├── filters: [] ├── build join filters: -│ ├── filter id:0, build key:t2.a (#2), probe key:t1.a (#0), filter type:bloom,inlist,min_max -│ └── filter id:1, build key:t2.b (#3), probe key:t1.b (#1), filter type:bloom,inlist,min_max +│ ├── filter id:0, build key:t2.a (#2), probe targets:[t1.a (#0)@scan0], filter type:bloom,inlist,min_max +│ └── filter id:1, build key:t2.b (#3), probe targets:[t1.b (#1)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 0.00 ├── TableScan(Build) │ ├── table: default.default.t2 +│ ├── scan id: 1 │ ├── output columns: [a (#2), b (#3)] │ ├── read rows: 0 │ ├── read size: 0 @@ -577,6 +607,7 @@ HashJoin │ └── estimated rows: 0.00 └── TableScan(Probe) ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -597,10 +628,11 @@ HashJoin ├── keys is null equal: [false] ├── filters: [t1.b (#1) > t2.b (#3)] ├── build join filters: -│ └── filter id:0, build key:t2.a (#2), probe key:t1.a (#0), filter type:bloom,inlist,min_max +│ └── filter id:0, build key:t2.a (#2), probe targets:[t1.a (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 0.00 ├── TableScan(Build) │ ├── table: default.default.t2 +│ ├── scan id: 1 │ ├── output columns: [a (#2), b (#3)] │ ├── read rows: 0 │ ├── read size: 0 @@ -610,6 +642,7 @@ HashJoin │ └── estimated rows: 0.00 └── TableScan(Probe) ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -727,6 +760,7 @@ HashJoin │ └── DummyTableScan └── TableScan(Probe) ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 4 ├── read size: < 1 KiB @@ -749,6 +783,7 @@ HashJoin ├── estimated rows: 4.00 ├── TableScan(Build) │ ├── table: default.default.t2 +│ ├── scan id: 1 │ ├── output columns: [a (#1)] │ ├── read rows: 1 │ ├── read size: < 1 KiB @@ -759,6 +794,7 @@ HashJoin │ └── estimated rows: 1.00 └── TableScan(Probe) ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 4 ├── read size: < 1 KiB @@ -781,6 +817,7 @@ HashJoin ├── estimated rows: 12.00 ├── TableScan(Build) │ ├── table: default.default.t3 +│ ├── scan id: 1 │ ├── output columns: [a (#1), b (#2)] │ ├── read rows: 3 │ ├── read size: < 1 KiB @@ -791,6 +828,7 @@ HashJoin │ └── estimated rows: 3.00 └── TableScan(Probe) ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 4 ├── read size: < 1 KiB @@ -811,10 +849,11 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:0, build key:t3.b (#2), probe key:t1.a (#0), filter type:inlist,min_max +│ └── filter id:0, build key:t3.b (#2), probe targets:[t1.a (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 3.00 ├── TableScan(Build) │ ├── table: default.default.t3 +│ ├── scan id: 1 │ ├── output columns: [a (#1), b (#2)] │ ├── read rows: 3 │ ├── read size: < 1 KiB @@ -825,6 +864,7 @@ HashJoin │ └── estimated rows: 3.00 └── TableScan(Probe) ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 4 ├── read size: < 1 KiB diff --git a/tests/sqllogictests/suites/mode/standalone/explain_native/join_reorder/chain.test b/tests/sqllogictests/suites/mode/standalone/explain_native/join_reorder/chain.test index c831d6776f0fe..0591e35fc8083 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain_native/join_reorder/chain.test +++ b/tests/sqllogictests/suites/mode/standalone/explain_native/join_reorder/chain.test @@ -36,7 +36,7 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:1, build key:t.a (#0), probe key:t2.a (#2), filter type:inlist,min_max +│ └── filter id:1, build key:t.a (#0), probe targets:[t2.a (#2)@scan2], filter type:bloom,inlist,min_max ├── estimated rows: 1.00 ├── HashJoin(Build) │ ├── output columns: [t1.a (#1), t.a (#0)] @@ -46,10 +46,11 @@ HashJoin │ ├── keys is null equal: [false] │ ├── filters: [] │ ├── build join filters: -│ │ └── filter id:0, build key:t.a (#0), probe key:t1.a (#1), filter type:inlist,min_max +│ │ └── filter id:0, build key:t.a (#0), probe targets:[t1.a (#1)@scan1], filter type:bloom,inlist,min_max │ ├── estimated rows: 1.00 │ ├── TableScan(Build) │ │ ├── table: default.join_reorder.t +│ │ ├── scan id: 0 │ │ ├── output columns: [a (#0)] │ │ ├── read rows: 1 │ │ ├── read size: < 1 KiB @@ -60,6 +61,7 @@ HashJoin │ │ └── estimated rows: 1.00 │ └── TableScan(Probe) │ ├── table: default.join_reorder.t1 +│ ├── scan id: 1 │ ├── output columns: [a (#1)] │ ├── read rows: 10 │ ├── read size: < 1 KiB @@ -71,6 +73,7 @@ HashJoin │ └── estimated rows: 10.00 └── TableScan(Probe) ├── table: default.join_reorder.t2 + ├── scan id: 2 ├── output columns: [a (#2)] ├── read rows: 100 ├── read size: < 1 KiB @@ -92,7 +95,7 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:1, build key:t.a (#0), probe key:t1.a (#2), filter type:inlist,min_max +│ └── filter id:1, build key:t.a (#0), probe targets:[t1.a (#2)@scan2], filter type:bloom,inlist,min_max ├── estimated rows: 1.00 ├── HashJoin(Build) │ ├── output columns: [t2.a (#1), t.a (#0)] @@ -102,10 +105,11 @@ HashJoin │ ├── keys is null equal: [false] │ ├── filters: [] │ ├── build join filters: -│ │ └── filter id:0, build key:t.a (#0), probe key:t2.a (#1), filter type:inlist,min_max +│ │ └── filter id:0, build key:t.a (#0), probe targets:[t2.a (#1)@scan1], filter type:bloom,inlist,min_max │ ├── estimated rows: 1.00 │ ├── TableScan(Build) │ │ ├── table: default.join_reorder.t +│ │ ├── scan id: 0 │ │ ├── output columns: [a (#0)] │ │ ├── read rows: 1 │ │ ├── read size: < 1 KiB @@ -116,6 +120,7 @@ HashJoin │ │ └── estimated rows: 1.00 │ └── TableScan(Probe) │ ├── table: default.join_reorder.t2 +│ ├── scan id: 1 │ ├── output columns: [a (#1)] │ ├── read rows: 100 │ ├── read size: < 1 KiB @@ -127,6 +132,7 @@ HashJoin │ └── estimated rows: 100.00 └── TableScan(Probe) ├── table: default.join_reorder.t1 + ├── scan id: 2 ├── output columns: [a (#2)] ├── read rows: 10 ├── read size: < 1 KiB @@ -148,7 +154,7 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:1, build key:t2.a (#2), probe key:t1.a (#0), filter type:inlist,min_max +│ └── filter id:1, build key:t2.a (#2), probe targets:[t1.a (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 1.00 ├── HashJoin(Build) │ ├── output columns: [t2.a (#2), t.a (#1)] @@ -158,10 +164,11 @@ HashJoin │ ├── keys is null equal: [false] │ ├── filters: [] │ ├── build join filters: -│ │ └── filter id:0, build key:t.a (#1), probe key:t2.a (#2), filter type:inlist,min_max +│ │ └── filter id:0, build key:t.a (#1), probe targets:[t2.a (#2)@scan2], filter type:bloom,inlist,min_max │ ├── estimated rows: 1.00 │ ├── TableScan(Build) │ │ ├── table: default.join_reorder.t +│ │ ├── scan id: 1 │ │ ├── output columns: [a (#1)] │ │ ├── read rows: 1 │ │ ├── read size: < 1 KiB @@ -172,6 +179,7 @@ HashJoin │ │ └── estimated rows: 1.00 │ └── TableScan(Probe) │ ├── table: default.join_reorder.t2 +│ ├── scan id: 2 │ ├── output columns: [a (#2)] │ ├── read rows: 100 │ ├── read size: < 1 KiB @@ -183,6 +191,7 @@ HashJoin │ └── estimated rows: 100.00 └── TableScan(Probe) ├── table: default.join_reorder.t1 + ├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 10 ├── read size: < 1 KiB @@ -204,7 +213,7 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:1, build key:t.a (#2), probe key:t1.a (#0), filter type:inlist,min_max +│ └── filter id:1, build key:t.a (#2), probe targets:[t1.a (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 1.00 ├── HashJoin(Build) │ ├── output columns: [t2.a (#1), t.a (#2)] @@ -214,10 +223,11 @@ HashJoin │ ├── keys is null equal: [false] │ ├── filters: [] │ ├── build join filters: -│ │ └── filter id:0, build key:t.a (#2), probe key:t2.a (#1), filter type:inlist,min_max +│ │ └── filter id:0, build key:t.a (#2), probe targets:[t2.a (#1)@scan1], filter type:bloom,inlist,min_max │ ├── estimated rows: 1.00 │ ├── TableScan(Build) │ │ ├── table: default.join_reorder.t +│ │ ├── scan id: 2 │ │ ├── output columns: [a (#2)] │ │ ├── read rows: 1 │ │ ├── read size: < 1 KiB @@ -228,6 +238,7 @@ HashJoin │ │ └── estimated rows: 1.00 │ └── TableScan(Probe) │ ├── table: default.join_reorder.t2 +│ ├── scan id: 1 │ ├── output columns: [a (#1)] │ ├── read rows: 100 │ ├── read size: < 1 KiB @@ -239,6 +250,7 @@ HashJoin │ └── estimated rows: 100.00 └── TableScan(Probe) ├── table: default.join_reorder.t1 + ├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 10 ├── read size: < 1 KiB @@ -260,7 +272,7 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:1, build key:t.a (#2), probe key:t2.a (#0), filter type:inlist,min_max +│ └── filter id:1, build key:t.a (#2), probe targets:[t2.a (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 1.00 ├── HashJoin(Build) │ ├── output columns: [t1.a (#1), t.a (#2)] @@ -270,10 +282,11 @@ HashJoin │ ├── keys is null equal: [false] │ ├── filters: [] │ ├── build join filters: -│ │ └── filter id:0, build key:t.a (#2), probe key:t1.a (#1), filter type:inlist,min_max +│ │ └── filter id:0, build key:t.a (#2), probe targets:[t1.a (#1)@scan1], filter type:bloom,inlist,min_max │ ├── estimated rows: 1.00 │ ├── TableScan(Build) │ │ ├── table: default.join_reorder.t +│ │ ├── scan id: 2 │ │ ├── output columns: [a (#2)] │ │ ├── read rows: 1 │ │ ├── read size: < 1 KiB @@ -284,6 +297,7 @@ HashJoin │ │ └── estimated rows: 1.00 │ └── TableScan(Probe) │ ├── table: default.join_reorder.t1 +│ ├── scan id: 1 │ ├── output columns: [a (#1)] │ ├── read rows: 10 │ ├── read size: < 1 KiB @@ -295,6 +309,7 @@ HashJoin │ └── estimated rows: 10.00 └── TableScan(Probe) ├── table: default.join_reorder.t2 + ├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 100 ├── read size: < 1 KiB @@ -316,7 +331,7 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:1, build key:t1.a (#2), probe key:t2.a (#0), filter type:inlist,min_max +│ └── filter id:1, build key:t1.a (#2), probe targets:[t2.a (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 1.00 ├── HashJoin(Build) │ ├── output columns: [t1.a (#2), t.a (#1)] @@ -326,10 +341,11 @@ HashJoin │ ├── keys is null equal: [false] │ ├── filters: [] │ ├── build join filters: -│ │ └── filter id:0, build key:t.a (#1), probe key:t1.a (#2), filter type:inlist,min_max +│ │ └── filter id:0, build key:t.a (#1), probe targets:[t1.a (#2)@scan2], filter type:bloom,inlist,min_max │ ├── estimated rows: 1.00 │ ├── TableScan(Build) │ │ ├── table: default.join_reorder.t +│ │ ├── scan id: 1 │ │ ├── output columns: [a (#1)] │ │ ├── read rows: 1 │ │ ├── read size: < 1 KiB @@ -340,6 +356,7 @@ HashJoin │ │ └── estimated rows: 1.00 │ └── TableScan(Probe) │ ├── table: default.join_reorder.t1 +│ ├── scan id: 2 │ ├── output columns: [a (#2)] │ ├── read rows: 10 │ ├── read size: < 1 KiB @@ -351,6 +368,7 @@ HashJoin │ └── estimated rows: 10.00 └── TableScan(Probe) ├── table: default.join_reorder.t2 + ├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 100 ├── read size: < 1 KiB @@ -374,6 +392,7 @@ HashJoin ├── estimated rows: 10.00 ├── TableScan(Build) │ ├── table: default.join_reorder.t +│ ├── scan id: 0 │ ├── output columns: [a (#0)] │ ├── read rows: 1 │ ├── read size: < 1 KiB @@ -388,6 +407,7 @@ HashJoin ├── estimated rows: 10.00 └── TableScan ├── table: default.join_reorder.t1 + ├── scan id: 1 ├── output columns: [a (#1)] ├── read rows: 10 ├── read size: < 1 KiB @@ -414,6 +434,7 @@ HashJoin │ ├── estimated rows: 1.00 │ └── TableScan │ ├── table: default.join_reorder.t +│ ├── scan id: 0 │ ├── output columns: [a (#0)] │ ├── read rows: 1 │ ├── read size: < 1 KiB @@ -424,6 +445,7 @@ HashJoin │ └── estimated rows: 1.00 └── TableScan(Probe) ├── table: default.join_reorder.t1 + ├── scan id: 1 ├── output columns: [a (#1)] ├── read rows: 10 ├── read size: < 1 KiB @@ -444,10 +466,11 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:0, build key:t.a (#0), probe key:t1.a (#1), filter type:inlist,min_max +│ └── filter id:0, build key:t.a (#0), probe targets:[t1.a (#1)@scan1], filter type:bloom,inlist,min_max ├── estimated rows: 1.00 ├── TableScan(Build) │ ├── table: default.join_reorder.t +│ ├── scan id: 0 │ ├── output columns: [a (#0)] │ ├── read rows: 1 │ ├── read size: < 1 KiB @@ -458,6 +481,7 @@ HashJoin │ └── estimated rows: 1.00 └── TableScan(Probe) ├── table: default.join_reorder.t1 + ├── scan id: 1 ├── output columns: [a (#1)] ├── read rows: 10 ├── read size: < 1 KiB @@ -481,6 +505,7 @@ HashJoin ├── estimated rows: 1.00 ├── TableScan(Build) │ ├── table: default.join_reorder.t +│ ├── scan id: 0 │ ├── output columns: [a (#0)] │ ├── read rows: 1 │ ├── read size: < 1 KiB @@ -491,6 +516,7 @@ HashJoin │ └── estimated rows: 1.00 └── TableScan(Probe) ├── table: default.join_reorder.t1 + ├── scan id: 1 ├── output columns: [a (#1)] ├── read rows: 10 ├── read size: < 1 KiB @@ -511,10 +537,11 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:0, build key:t.a (#0), probe key:t1.a (#1), filter type:inlist,min_max +│ └── filter id:0, build key:t.a (#0), probe targets:[t1.a (#1)@scan1], filter type:bloom,inlist,min_max ├── estimated rows: 1.00 ├── TableScan(Build) │ ├── table: default.join_reorder.t +│ ├── scan id: 0 │ ├── output columns: [a (#0)] │ ├── read rows: 1 │ ├── read size: < 1 KiB @@ -525,6 +552,7 @@ HashJoin │ └── estimated rows: 1.00 └── TableScan(Probe) ├── table: default.join_reorder.t1 + ├── scan id: 1 ├── output columns: [a (#1)] ├── read rows: 10 ├── read size: < 1 KiB @@ -548,6 +576,7 @@ HashJoin ├── estimated rows: 10.00 ├── TableScan(Build) │ ├── table: default.join_reorder.t +│ ├── scan id: 0 │ ├── output columns: [a (#0)] │ ├── read rows: 1 │ ├── read size: < 1 KiB @@ -558,6 +587,7 @@ HashJoin │ └── estimated rows: 1.00 └── TableScan(Probe) ├── table: default.join_reorder.t1 + ├── scan id: 1 ├── output columns: [a (#1)] ├── read rows: 10 ├── read size: < 1 KiB diff --git a/tests/sqllogictests/suites/mode/standalone/explain_native/join_reorder/cycles.test b/tests/sqllogictests/suites/mode/standalone/explain_native/join_reorder/cycles.test index e46e983860c78..cbb01953da8bc 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain_native/join_reorder/cycles.test +++ b/tests/sqllogictests/suites/mode/standalone/explain_native/join_reorder/cycles.test @@ -27,7 +27,7 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:1, build key:t.a (#0), probe key:t2.a (#2), filter type:inlist,min_max +│ └── filter id:1, build key:t.a (#0), probe targets:[t2.a (#2)@scan2], filter type:bloom,inlist,min_max ├── estimated rows: 1.00 ├── HashJoin(Build) │ ├── output columns: [t1.a (#1), t.a (#0)] @@ -37,10 +37,11 @@ HashJoin │ ├── keys is null equal: [false] │ ├── filters: [] │ ├── build join filters: -│ │ └── filter id:0, build key:t.a (#0), probe key:t1.a (#1), filter type:inlist,min_max +│ │ └── filter id:0, build key:t.a (#0), probe targets:[t1.a (#1)@scan1], filter type:bloom,inlist,min_max │ ├── estimated rows: 1.00 │ ├── TableScan(Build) │ │ ├── table: default.join_reorder.t +│ │ ├── scan id: 0 │ │ ├── output columns: [a (#0)] │ │ ├── read rows: 1 │ │ ├── read size: < 1 KiB @@ -51,6 +52,7 @@ HashJoin │ │ └── estimated rows: 1.00 │ └── TableScan(Probe) │ ├── table: default.join_reorder.t1 +│ ├── scan id: 1 │ ├── output columns: [a (#1)] │ ├── read rows: 10 │ ├── read size: < 1 KiB @@ -62,6 +64,7 @@ HashJoin │ └── estimated rows: 10.00 └── TableScan(Probe) ├── table: default.join_reorder.t2 + ├── scan id: 2 ├── output columns: [a (#2)] ├── read rows: 100 ├── read size: < 1 KiB @@ -83,7 +86,7 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:1, build key:t.a (#0), probe key:t1.a (#2), filter type:inlist,min_max +│ └── filter id:1, build key:t.a (#0), probe targets:[t1.a (#2)@scan2], filter type:bloom,inlist,min_max ├── estimated rows: 1.00 ├── HashJoin(Build) │ ├── output columns: [t2.a (#1), t.a (#0)] @@ -93,10 +96,11 @@ HashJoin │ ├── keys is null equal: [false] │ ├── filters: [] │ ├── build join filters: -│ │ └── filter id:0, build key:t.a (#0), probe key:t2.a (#1), filter type:inlist,min_max +│ │ └── filter id:0, build key:t.a (#0), probe targets:[t2.a (#1)@scan1], filter type:bloom,inlist,min_max │ ├── estimated rows: 1.00 │ ├── TableScan(Build) │ │ ├── table: default.join_reorder.t +│ │ ├── scan id: 0 │ │ ├── output columns: [a (#0)] │ │ ├── read rows: 1 │ │ ├── read size: < 1 KiB @@ -107,6 +111,7 @@ HashJoin │ │ └── estimated rows: 1.00 │ └── TableScan(Probe) │ ├── table: default.join_reorder.t2 +│ ├── scan id: 1 │ ├── output columns: [a (#1)] │ ├── read rows: 100 │ ├── read size: < 1 KiB @@ -118,6 +123,7 @@ HashJoin │ └── estimated rows: 100.00 └── TableScan(Probe) ├── table: default.join_reorder.t1 + ├── scan id: 2 ├── output columns: [a (#2)] ├── read rows: 10 ├── read size: < 1 KiB @@ -139,7 +145,7 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:1, build key:t2.a (#2), probe key:t1.a (#0), filter type:inlist,min_max +│ └── filter id:1, build key:t2.a (#2), probe targets:[t1.a (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 1.00 ├── HashJoin(Build) │ ├── output columns: [t2.a (#2), t.a (#1)] @@ -149,10 +155,11 @@ HashJoin │ ├── keys is null equal: [false] │ ├── filters: [] │ ├── build join filters: -│ │ └── filter id:0, build key:t.a (#1), probe key:t2.a (#2), filter type:inlist,min_max +│ │ └── filter id:0, build key:t.a (#1), probe targets:[t2.a (#2)@scan2], filter type:bloom,inlist,min_max │ ├── estimated rows: 1.00 │ ├── TableScan(Build) │ │ ├── table: default.join_reorder.t +│ │ ├── scan id: 1 │ │ ├── output columns: [a (#1)] │ │ ├── read rows: 1 │ │ ├── read size: < 1 KiB @@ -163,6 +170,7 @@ HashJoin │ │ └── estimated rows: 1.00 │ └── TableScan(Probe) │ ├── table: default.join_reorder.t2 +│ ├── scan id: 2 │ ├── output columns: [a (#2)] │ ├── read rows: 100 │ ├── read size: < 1 KiB @@ -174,6 +182,7 @@ HashJoin │ └── estimated rows: 100.00 └── TableScan(Probe) ├── table: default.join_reorder.t1 + ├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 10 ├── read size: < 1 KiB @@ -195,7 +204,7 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:1, build key:t.a (#2), probe key:t1.a (#0), filter type:inlist,min_max +│ └── filter id:1, build key:t.a (#2), probe targets:[t1.a (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 1.00 ├── HashJoin(Build) │ ├── output columns: [t2.a (#1), t.a (#2)] @@ -205,10 +214,11 @@ HashJoin │ ├── keys is null equal: [false] │ ├── filters: [] │ ├── build join filters: -│ │ └── filter id:0, build key:t.a (#2), probe key:t2.a (#1), filter type:inlist,min_max +│ │ └── filter id:0, build key:t.a (#2), probe targets:[t2.a (#1)@scan1], filter type:bloom,inlist,min_max │ ├── estimated rows: 1.00 │ ├── TableScan(Build) │ │ ├── table: default.join_reorder.t +│ │ ├── scan id: 2 │ │ ├── output columns: [a (#2)] │ │ ├── read rows: 1 │ │ ├── read size: < 1 KiB @@ -219,6 +229,7 @@ HashJoin │ │ └── estimated rows: 1.00 │ └── TableScan(Probe) │ ├── table: default.join_reorder.t2 +│ ├── scan id: 1 │ ├── output columns: [a (#1)] │ ├── read rows: 100 │ ├── read size: < 1 KiB @@ -230,6 +241,7 @@ HashJoin │ └── estimated rows: 100.00 └── TableScan(Probe) ├── table: default.join_reorder.t1 + ├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 10 ├── read size: < 1 KiB @@ -251,7 +263,7 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:1, build key:t.a (#2), probe key:t2.a (#0), filter type:inlist,min_max +│ └── filter id:1, build key:t.a (#2), probe targets:[t2.a (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 1.00 ├── HashJoin(Build) │ ├── output columns: [t1.a (#1), t.a (#2)] @@ -261,10 +273,11 @@ HashJoin │ ├── keys is null equal: [false] │ ├── filters: [] │ ├── build join filters: -│ │ └── filter id:0, build key:t.a (#2), probe key:t1.a (#1), filter type:inlist,min_max +│ │ └── filter id:0, build key:t.a (#2), probe targets:[t1.a (#1)@scan1], filter type:bloom,inlist,min_max │ ├── estimated rows: 1.00 │ ├── TableScan(Build) │ │ ├── table: default.join_reorder.t +│ │ ├── scan id: 2 │ │ ├── output columns: [a (#2)] │ │ ├── read rows: 1 │ │ ├── read size: < 1 KiB @@ -275,6 +288,7 @@ HashJoin │ │ └── estimated rows: 1.00 │ └── TableScan(Probe) │ ├── table: default.join_reorder.t1 +│ ├── scan id: 1 │ ├── output columns: [a (#1)] │ ├── read rows: 10 │ ├── read size: < 1 KiB @@ -286,6 +300,7 @@ HashJoin │ └── estimated rows: 10.00 └── TableScan(Probe) ├── table: default.join_reorder.t2 + ├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 100 ├── read size: < 1 KiB @@ -307,7 +322,7 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:1, build key:t1.a (#2), probe key:t2.a (#0), filter type:inlist,min_max +│ └── filter id:1, build key:t1.a (#2), probe targets:[t2.a (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 1.00 ├── HashJoin(Build) │ ├── output columns: [t1.a (#2), t.a (#1)] @@ -317,10 +332,11 @@ HashJoin │ ├── keys is null equal: [false] │ ├── filters: [] │ ├── build join filters: -│ │ └── filter id:0, build key:t.a (#1), probe key:t1.a (#2), filter type:inlist,min_max +│ │ └── filter id:0, build key:t.a (#1), probe targets:[t1.a (#2)@scan2], filter type:bloom,inlist,min_max │ ├── estimated rows: 1.00 │ ├── TableScan(Build) │ │ ├── table: default.join_reorder.t +│ │ ├── scan id: 1 │ │ ├── output columns: [a (#1)] │ │ ├── read rows: 1 │ │ ├── read size: < 1 KiB @@ -331,6 +347,7 @@ HashJoin │ │ └── estimated rows: 1.00 │ └── TableScan(Probe) │ ├── table: default.join_reorder.t1 +│ ├── scan id: 2 │ ├── output columns: [a (#2)] │ ├── read rows: 10 │ ├── read size: < 1 KiB @@ -342,6 +359,7 @@ HashJoin │ └── estimated rows: 10.00 └── TableScan(Probe) ├── table: default.join_reorder.t2 + ├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 100 ├── read size: < 1 KiB diff --git a/tests/sqllogictests/suites/mode/standalone/explain_native/join_reorder/mark.test b/tests/sqllogictests/suites/mode/standalone/explain_native/join_reorder/mark.test index 22b4a7468aad3..db37647cf490f 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain_native/join_reorder/mark.test +++ b/tests/sqllogictests/suites/mode/standalone/explain_native/join_reorder/mark.test @@ -11,6 +11,7 @@ HashJoin ├── estimated rows: 10000.00 ├── TableScan(Build) │ ├── table: default.system.numbers +│ ├── scan id: 1 │ ├── output columns: [number (#1)] │ ├── read rows: 1000 │ ├── read size: 7.81 KiB @@ -20,6 +21,7 @@ HashJoin │ └── estimated rows: 1000.00 └── TableScan(Probe) ├── table: default.system.numbers + ├── scan id: 0 ├── output columns: [number (#0)] ├── read rows: 10000 ├── read size: 78.12 KiB @@ -39,10 +41,11 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:0, build key:numbers.number (#0), probe key:subquery_1 (#1), filter type:inlist,min_max +│ └── filter id:0, build key:numbers.number (#0), probe targets:[subquery_1 (#1)@scan1], filter type:bloom,inlist,min_max ├── estimated rows: 1000.00 ├── TableScan(Build) │ ├── table: default.system.numbers +│ ├── scan id: 0 │ ├── output columns: [number (#0)] │ ├── read rows: 1000 │ ├── read size: 7.81 KiB @@ -52,6 +55,7 @@ HashJoin │ └── estimated rows: 1000.00 └── TableScan(Probe) ├── table: default.system.numbers + ├── scan id: 1 ├── output columns: [number (#1)] ├── read rows: 10000 ├── read size: 78.12 KiB diff --git a/tests/sqllogictests/suites/mode/standalone/explain_native/join_reorder/star.test b/tests/sqllogictests/suites/mode/standalone/explain_native/join_reorder/star.test index f02aff118b009..6b10db4a1194b 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain_native/join_reorder/star.test +++ b/tests/sqllogictests/suites/mode/standalone/explain_native/join_reorder/star.test @@ -27,7 +27,7 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:1, build key:t.a (#0), probe key:t2.a (#2), filter type:inlist,min_max +│ └── filter id:1, build key:t.a (#0), probe targets:[t2.a (#2)@scan2], filter type:bloom,inlist,min_max ├── estimated rows: 1.00 ├── HashJoin(Build) │ ├── output columns: [t1.a (#1), t.a (#0)] @@ -37,10 +37,11 @@ HashJoin │ ├── keys is null equal: [false] │ ├── filters: [] │ ├── build join filters: -│ │ └── filter id:0, build key:t.a (#0), probe key:t1.a (#1), filter type:inlist,min_max +│ │ └── filter id:0, build key:t.a (#0), probe targets:[t1.a (#1)@scan1], filter type:bloom,inlist,min_max │ ├── estimated rows: 1.00 │ ├── TableScan(Build) │ │ ├── table: default.join_reorder.t +│ │ ├── scan id: 0 │ │ ├── output columns: [a (#0)] │ │ ├── read rows: 1 │ │ ├── read size: < 1 KiB @@ -51,6 +52,7 @@ HashJoin │ │ └── estimated rows: 1.00 │ └── TableScan(Probe) │ ├── table: default.join_reorder.t1 +│ ├── scan id: 1 │ ├── output columns: [a (#1)] │ ├── read rows: 10 │ ├── read size: < 1 KiB @@ -62,6 +64,7 @@ HashJoin │ └── estimated rows: 10.00 └── TableScan(Probe) ├── table: default.join_reorder.t2 + ├── scan id: 2 ├── output columns: [a (#2)] ├── read rows: 100 ├── read size: < 1 KiB @@ -83,7 +86,7 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:1, build key:t.a (#0), probe key:t1.a (#2), filter type:inlist,min_max +│ └── filter id:1, build key:t.a (#0), probe targets:[t1.a (#2)@scan2], filter type:bloom,inlist,min_max ├── estimated rows: 1.00 ├── HashJoin(Build) │ ├── output columns: [t2.a (#1), t.a (#0)] @@ -93,10 +96,11 @@ HashJoin │ ├── keys is null equal: [false] │ ├── filters: [] │ ├── build join filters: -│ │ └── filter id:0, build key:t.a (#0), probe key:t2.a (#1), filter type:inlist,min_max +│ │ └── filter id:0, build key:t.a (#0), probe targets:[t2.a (#1)@scan1], filter type:bloom,inlist,min_max │ ├── estimated rows: 1.00 │ ├── TableScan(Build) │ │ ├── table: default.join_reorder.t +│ │ ├── scan id: 0 │ │ ├── output columns: [a (#0)] │ │ ├── read rows: 1 │ │ ├── read size: < 1 KiB @@ -107,6 +111,7 @@ HashJoin │ │ └── estimated rows: 1.00 │ └── TableScan(Probe) │ ├── table: default.join_reorder.t2 +│ ├── scan id: 1 │ ├── output columns: [a (#1)] │ ├── read rows: 100 │ ├── read size: < 1 KiB @@ -118,6 +123,7 @@ HashJoin │ └── estimated rows: 100.00 └── TableScan(Probe) ├── table: default.join_reorder.t1 + ├── scan id: 2 ├── output columns: [a (#2)] ├── read rows: 10 ├── read size: < 1 KiB @@ -139,7 +145,7 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:1, build key:t2.a (#2), probe key:t1.a (#0), filter type:inlist,min_max +│ └── filter id:1, build key:t2.a (#2), probe targets:[t1.a (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 1.00 ├── HashJoin(Build) │ ├── output columns: [t2.a (#2), t.a (#1)] @@ -149,10 +155,11 @@ HashJoin │ ├── keys is null equal: [false] │ ├── filters: [] │ ├── build join filters: -│ │ └── filter id:0, build key:t.a (#1), probe key:t2.a (#2), filter type:inlist,min_max +│ │ └── filter id:0, build key:t.a (#1), probe targets:[t2.a (#2)@scan2], filter type:bloom,inlist,min_max │ ├── estimated rows: 1.00 │ ├── TableScan(Build) │ │ ├── table: default.join_reorder.t +│ │ ├── scan id: 1 │ │ ├── output columns: [a (#1)] │ │ ├── read rows: 1 │ │ ├── read size: < 1 KiB @@ -163,6 +170,7 @@ HashJoin │ │ └── estimated rows: 1.00 │ └── TableScan(Probe) │ ├── table: default.join_reorder.t2 +│ ├── scan id: 2 │ ├── output columns: [a (#2)] │ ├── read rows: 100 │ ├── read size: < 1 KiB @@ -174,6 +182,7 @@ HashJoin │ └── estimated rows: 100.00 └── TableScan(Probe) ├── table: default.join_reorder.t1 + ├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 10 ├── read size: < 1 KiB @@ -195,7 +204,7 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:1, build key:t.a (#2), probe key:t1.a (#0), filter type:inlist,min_max +│ └── filter id:1, build key:t.a (#2), probe targets:[t1.a (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 1.00 ├── HashJoin(Build) │ ├── output columns: [t2.a (#1), t.a (#2)] @@ -205,10 +214,11 @@ HashJoin │ ├── keys is null equal: [false] │ ├── filters: [] │ ├── build join filters: -│ │ └── filter id:0, build key:t.a (#2), probe key:t2.a (#1), filter type:inlist,min_max +│ │ └── filter id:0, build key:t.a (#2), probe targets:[t2.a (#1)@scan1], filter type:bloom,inlist,min_max │ ├── estimated rows: 1.00 │ ├── TableScan(Build) │ │ ├── table: default.join_reorder.t +│ │ ├── scan id: 2 │ │ ├── output columns: [a (#2)] │ │ ├── read rows: 1 │ │ ├── read size: < 1 KiB @@ -219,6 +229,7 @@ HashJoin │ │ └── estimated rows: 1.00 │ └── TableScan(Probe) │ ├── table: default.join_reorder.t2 +│ ├── scan id: 1 │ ├── output columns: [a (#1)] │ ├── read rows: 100 │ ├── read size: < 1 KiB @@ -230,6 +241,7 @@ HashJoin │ └── estimated rows: 100.00 └── TableScan(Probe) ├── table: default.join_reorder.t1 + ├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 10 ├── read size: < 1 KiB @@ -251,7 +263,7 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:1, build key:t.a (#2), probe key:t2.a (#0), filter type:inlist,min_max +│ └── filter id:1, build key:t.a (#2), probe targets:[t2.a (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 1.00 ├── HashJoin(Build) │ ├── output columns: [t1.a (#1), t.a (#2)] @@ -261,10 +273,11 @@ HashJoin │ ├── keys is null equal: [false] │ ├── filters: [] │ ├── build join filters: -│ │ └── filter id:0, build key:t.a (#2), probe key:t1.a (#1), filter type:inlist,min_max +│ │ └── filter id:0, build key:t.a (#2), probe targets:[t1.a (#1)@scan1], filter type:bloom,inlist,min_max │ ├── estimated rows: 1.00 │ ├── TableScan(Build) │ │ ├── table: default.join_reorder.t +│ │ ├── scan id: 2 │ │ ├── output columns: [a (#2)] │ │ ├── read rows: 1 │ │ ├── read size: < 1 KiB @@ -275,6 +288,7 @@ HashJoin │ │ └── estimated rows: 1.00 │ └── TableScan(Probe) │ ├── table: default.join_reorder.t1 +│ ├── scan id: 1 │ ├── output columns: [a (#1)] │ ├── read rows: 10 │ ├── read size: < 1 KiB @@ -286,6 +300,7 @@ HashJoin │ └── estimated rows: 10.00 └── TableScan(Probe) ├── table: default.join_reorder.t2 + ├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 100 ├── read size: < 1 KiB @@ -307,7 +322,7 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:1, build key:t1.a (#2), probe key:t2.a (#0), filter type:inlist,min_max +│ └── filter id:1, build key:t1.a (#2), probe targets:[t2.a (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 1.00 ├── HashJoin(Build) │ ├── output columns: [t1.a (#2), t.a (#1)] @@ -317,10 +332,11 @@ HashJoin │ ├── keys is null equal: [false] │ ├── filters: [] │ ├── build join filters: -│ │ └── filter id:0, build key:t.a (#1), probe key:t1.a (#2), filter type:inlist,min_max +│ │ └── filter id:0, build key:t.a (#1), probe targets:[t1.a (#2)@scan2], filter type:bloom,inlist,min_max │ ├── estimated rows: 1.00 │ ├── TableScan(Build) │ │ ├── table: default.join_reorder.t +│ │ ├── scan id: 1 │ │ ├── output columns: [a (#1)] │ │ ├── read rows: 1 │ │ ├── read size: < 1 KiB @@ -331,6 +347,7 @@ HashJoin │ │ └── estimated rows: 1.00 │ └── TableScan(Probe) │ ├── table: default.join_reorder.t1 +│ ├── scan id: 2 │ ├── output columns: [a (#2)] │ ├── read rows: 10 │ ├── read size: < 1 KiB @@ -342,6 +359,7 @@ HashJoin │ └── estimated rows: 10.00 └── TableScan(Probe) ├── table: default.join_reorder.t2 + ├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 100 ├── read size: < 1 KiB diff --git a/tests/sqllogictests/suites/mode/standalone/explain_native/nullable_prune.test b/tests/sqllogictests/suites/mode/standalone/explain_native/nullable_prune.test index e1ffe596de1b4..d295216f27c72 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain_native/nullable_prune.test +++ b/tests/sqllogictests/suites/mode/standalone/explain_native/nullable_prune.test @@ -18,6 +18,7 @@ explain select * from t_nullable_prune ---- TableScan ├── table: default.default.t_nullable_prune +├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 6 ├── read size: < 1 KiB @@ -32,6 +33,7 @@ explain select * from t_nullable_prune where a is not null ---- TableScan ├── table: default.default.t_nullable_prune +├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 3 ├── read size: < 1 KiB @@ -46,6 +48,7 @@ explain select * from t_nullable_prune where a is null ---- TableScan ├── table: default.default.t_nullable_prune +├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 3 ├── read size: < 1 KiB diff --git a/tests/sqllogictests/suites/mode/standalone/explain_native/prewhere_optimization.test b/tests/sqllogictests/suites/mode/standalone/explain_native/prewhere_optimization.test index 4eff696396413..ff144c065e662 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain_native/prewhere_optimization.test +++ b/tests/sqllogictests/suites/mode/standalone/explain_native/prewhere_optimization.test @@ -15,6 +15,7 @@ explain select a from t_where_optimizer where a = 1 ---- TableScan ├── table: default.default.t_where_optimizer +├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 0 ├── read size: 0 @@ -28,6 +29,7 @@ explain select a from t_where_optimizer where isnotempty(a) ---- TableScan ├── table: default.default.t_where_optimizer +├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 0 ├── read size: 0 @@ -42,6 +44,7 @@ explain select * from t_where_optimizer where a = b ---- TableScan ├── table: default.default.t_where_optimizer +├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -55,6 +58,7 @@ explain select * from t_where_optimizer where a = 1 or b > 2 ---- TableScan ├── table: default.default.t_where_optimizer +├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -68,6 +72,7 @@ explain select * from t_where_optimizer where a = 1 and b > 2 ---- TableScan ├── table: default.default.t_where_optimizer +├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -81,6 +86,7 @@ explain select * from t_where_optimizer where b = 1 ---- TableScan ├── table: default.default.t_where_optimizer +├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -94,6 +100,7 @@ explain select a from t_where_optimizer where b = 1 ---- TableScan ├── table: default.default.t_where_optimizer +├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 0 ├── read size: 0 @@ -113,6 +120,7 @@ explain select * from t_where_optimizer where s:a > 0 ---- TableScan ├── table: default.default.t_where_optimizer +├── scan id: 0 ├── output columns: [id (#0), s (#1)] ├── read rows: 0 ├── read size: 0 diff --git a/tests/sqllogictests/suites/mode/standalone/explain_native/project_set.test b/tests/sqllogictests/suites/mode/standalone/explain_native/project_set.test index 7d91f9cdab6b8..2876d51b95884 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain_native/project_set.test +++ b/tests/sqllogictests/suites/mode/standalone/explain_native/project_set.test @@ -28,6 +28,7 @@ AggregateFinal ├── set returning functions: unnest(fold_count.id (#0)) └── TableScan ├── table: default.default.fold_count + ├── scan id: 0 ├── output columns: [id (#0)] ├── read rows: 1 ├── read size: < 1 KiB @@ -58,6 +59,7 @@ ProjectSet ├── set returning functions: unnest([1, 2, 3]) └── TableScan ├── table: default.system.numbers + ├── scan id: 0 ├── output columns: [number (#0)] ├── read rows: 10 ├── read size: < 1 KiB @@ -75,6 +77,7 @@ ProjectSet ├── set returning functions: unnest(CAST(array(1, 2, 3, numbers.number (#0)) AS Array(UInt64 NULL))) └── TableScan ├── table: default.system.numbers + ├── scan id: 0 ├── output columns: [number (#0)] ├── read rows: 10 ├── read size: < 1 KiB @@ -96,6 +99,7 @@ HashJoin ├── estimated rows: 450.00 ├── TableScan(Build) │ ├── table: default.system.numbers +│ ├── scan id: 1 │ ├── output columns: [number (#3)] │ ├── read rows: 15 │ ├── read size: < 1 KiB @@ -113,6 +117,7 @@ HashJoin ├── set returning functions: unnest([1, 2, 3]) └── TableScan ├── table: default.system.numbers + ├── scan id: 0 ├── output columns: [number (#0)] ├── read rows: 10 ├── read size: < 1 KiB diff --git a/tests/sqllogictests/suites/mode/standalone/explain_native/prune_column.test b/tests/sqllogictests/suites/mode/standalone/explain_native/prune_column.test index 2bf210ade54b8..d5e2e7588ce57 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain_native/prune_column.test +++ b/tests/sqllogictests/suites/mode/standalone/explain_native/prune_column.test @@ -3,6 +3,7 @@ explain select * from (select a from (select number as a, number + 1 as b from n ---- TableScan ├── table: default.system.numbers +├── scan id: 0 ├── output columns: [number (#0)] ├── read rows: 1 ├── read size: < 1 KiB @@ -25,6 +26,7 @@ AggregateFinal ├── estimated rows: 1.00 └── TableScan ├── table: default.system.numbers + ├── scan id: 0 ├── output columns: [number (#0)] ├── read rows: 1 ├── read size: < 1 KiB @@ -61,6 +63,7 @@ Limit ├── estimated rows: 0.50 └── TableScan ├── table: default.system.numbers + ├── scan id: 0 ├── output columns: [number (#0)] ├── read rows: 1 ├── read size: < 1 KiB @@ -90,6 +93,7 @@ HashJoin │ ├── estimated rows: 0.20 │ └── TableScan │ ├── table: default.system.numbers +│ ├── scan id: 0 │ ├── output columns: [number (#0)] │ ├── read rows: 1 │ ├── read size: < 1 KiB @@ -103,6 +107,7 @@ HashJoin ├── estimated rows: 1.00 └── TableScan ├── table: default.system.numbers + ├── scan id: 1 ├── output columns: [number (#5)] ├── read rows: 1 ├── read size: < 1 KiB @@ -149,6 +154,7 @@ HashJoin │ │ ├── estimated rows: 0.20 │ │ └── TableScan │ │ ├── table: default.system.numbers +│ │ ├── scan id: 1 │ │ ├── output columns: [number (#3)] │ │ ├── read rows: 1 │ │ ├── read size: < 1 KiB @@ -162,6 +168,7 @@ HashJoin │ ├── estimated rows: 1.00 │ └── TableScan │ ├── table: default.system.numbers +│ ├── scan id: 2 │ ├── output columns: [number (#8)] │ ├── read rows: 1 │ ├── read size: < 1 KiB @@ -175,6 +182,7 @@ HashJoin ├── estimated rows: 2.00 └── TableScan ├── table: default.system.numbers + ├── scan id: 0 ├── output columns: [number (#0)] ├── read rows: 2 ├── read size: < 1 KiB @@ -192,6 +200,7 @@ Sort(Single) ├── estimated rows: 0.00 └── TableScan ├── table: default.system.functions + ├── scan id: 0 ├── output columns: [name (#0), example (#4)] ├── read rows: 0 ├── read size: 0 @@ -218,6 +227,7 @@ HashJoin │ └── DummyTableScan └── TableScan(Probe) ├── table: default.system.numbers + ├── scan id: 0 ├── output columns: [number (#0)] ├── read rows: 10 ├── read size: < 1 KiB @@ -250,6 +260,7 @@ AggregateFinal ├── estimated rows: 1.00 └── TableScan ├── table: default.default.t + ├── scan id: 0 ├── output columns: [] ├── read rows: 2 ├── read size: < 1 KiB diff --git a/tests/sqllogictests/suites/mode/standalone/explain_native/push_down_filter/push_down_filter_join/push_down_filter_join_full_outer.test b/tests/sqllogictests/suites/mode/standalone/explain_native/push_down_filter/push_down_filter_join/push_down_filter_join_full_outer.test index 11250d005a38e..1e1601a1fca85 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain_native/push_down_filter/push_down_filter_join/push_down_filter_join_full_outer.test +++ b/tests/sqllogictests/suites/mode/standalone/explain_native/push_down_filter/push_down_filter_join/push_down_filter_join_full_outer.test @@ -31,6 +31,7 @@ HashJoin ├── estimated rows: 4.00 ├── TableScan(Build) │ ├── table: default.default.t2 +│ ├── scan id: 1 │ ├── output columns: [a (#2), b (#3)] │ ├── read rows: 3 │ ├── read size: < 1 KiB @@ -41,6 +42,7 @@ HashJoin │ └── estimated rows: 3.00 └── TableScan(Probe) ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 4 ├── read size: < 1 KiB @@ -62,10 +64,11 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:0, build key:t2.a (#2), probe key:t1.a (#0), filter type:inlist,min_max +│ └── filter id:0, build key:t2.a (#2), probe targets:[t1.a (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 4.00 ├── TableScan(Build) │ ├── table: default.default.t2 +│ ├── scan id: 1 │ ├── output columns: [a (#2), b (#3)] │ ├── read rows: 3 │ ├── read size: < 1 KiB @@ -76,6 +79,7 @@ HashJoin │ └── estimated rows: 3.00 └── TableScan(Probe) ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 4 ├── read size: < 1 KiB @@ -100,6 +104,7 @@ HashJoin ├── estimated rows: 4.00 ├── TableScan(Build) │ ├── table: default.default.t2 +│ ├── scan id: 1 │ ├── output columns: [a (#2), b (#3)] │ ├── read rows: 3 │ ├── read size: < 1 KiB @@ -110,6 +115,7 @@ HashJoin │ └── estimated rows: 3.00 └── TableScan(Probe) ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 4 ├── read size: < 1 KiB @@ -131,10 +137,11 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:0, build key:t2.a (#2), probe key:t1.a (#0), filter type:inlist,min_max +│ └── filter id:0, build key:t2.a (#2), probe targets:[t1.a (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 4.00 ├── TableScan(Build) │ ├── table: default.default.t2 +│ ├── scan id: 1 │ ├── output columns: [a (#2), b (#3)] │ ├── read rows: 3 │ ├── read size: < 1 KiB @@ -145,6 +152,7 @@ HashJoin │ └── estimated rows: 3.00 └── TableScan(Probe) ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 4 ├── read size: < 1 KiB @@ -167,10 +175,11 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:0, build key:t2.a (#2), probe key:t1.a (#0), filter type:inlist,min_max +│ └── filter id:0, build key:t2.a (#2), probe targets:[t1.a (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 4.00 ├── TableScan(Build) │ ├── table: default.default.t2 +│ ├── scan id: 1 │ ├── output columns: [a (#2), b (#3)] │ ├── read rows: 3 │ ├── read size: < 1 KiB @@ -181,6 +190,7 @@ HashJoin │ └── estimated rows: 3.00 └── TableScan(Probe) ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 4 ├── read size: < 1 KiB @@ -203,10 +213,11 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:0, build key:t2.a (#2), probe key:t1.a (#0), filter type:inlist,min_max +│ └── filter id:0, build key:t2.a (#2), probe targets:[t1.a (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 4.00 ├── TableScan(Build) │ ├── table: default.default.t2 +│ ├── scan id: 1 │ ├── output columns: [a (#2), b (#3)] │ ├── read rows: 3 │ ├── read size: < 1 KiB @@ -217,6 +228,7 @@ HashJoin │ └── estimated rows: 3.00 └── TableScan(Probe) ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 4 ├── read size: < 1 KiB @@ -241,6 +253,7 @@ HashJoin ├── estimated rows: 4.00 ├── TableScan(Build) │ ├── table: default.default.t2 +│ ├── scan id: 1 │ ├── output columns: [a (#2), b (#3)] │ ├── read rows: 3 │ ├── read size: < 1 KiB @@ -251,6 +264,7 @@ HashJoin │ └── estimated rows: 3.00 └── TableScan(Probe) ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 4 ├── read size: < 1 KiB diff --git a/tests/sqllogictests/suites/mode/standalone/explain_native/push_down_filter/push_down_filter_join/push_down_filter_join_inner.test b/tests/sqllogictests/suites/mode/standalone/explain_native/push_down_filter/push_down_filter_join/push_down_filter_join_inner.test index b38695503349f..59dc47792b056 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain_native/push_down_filter/push_down_filter_join/push_down_filter_join_inner.test +++ b/tests/sqllogictests/suites/mode/standalone/explain_native/push_down_filter/push_down_filter_join/push_down_filter_join_inner.test @@ -29,10 +29,11 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:0, build key:t2.a (#2), probe key:t1.a (#0), filter type:bloom,inlist,min_max +│ └── filter id:0, build key:t2.a (#2), probe targets:[t1.a (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 0.00 ├── TableScan(Build) │ ├── table: default.default.t2 +│ ├── scan id: 1 │ ├── output columns: [a (#2), b (#3)] │ ├── read rows: 0 │ ├── read size: 0 @@ -43,6 +44,7 @@ HashJoin │ └── estimated rows: 0.00 └── TableScan(Probe) ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 0 ├── read size: 0 @@ -69,10 +71,11 @@ Filter ├── keys is null equal: [false] ├── filters: [] ├── build join filters: - │ └── filter id:0, build key:t2.a (#2), probe key:t1.a (#0), filter type:inlist,min_max + │ └── filter id:0, build key:t2.a (#2), probe targets:[t1.a (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 3.56 ├── TableScan(Build) │ ├── table: default.default.t2 + │ ├── scan id: 1 │ ├── output columns: [a (#2), b (#3)] │ ├── read rows: 3 │ ├── read size: < 1 KiB @@ -83,6 +86,7 @@ Filter │ └── estimated rows: 3.00 └── TableScan(Probe) ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 4 ├── read size: < 1 KiB diff --git a/tests/sqllogictests/suites/mode/standalone/explain_native/push_down_filter/push_down_filter_join/push_down_filter_join_left_outer.test b/tests/sqllogictests/suites/mode/standalone/explain_native/push_down_filter/push_down_filter_join/push_down_filter_join_left_outer.test index d9243a76c44e3..efc808d91dfe1 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain_native/push_down_filter/push_down_filter_join/push_down_filter_join_left_outer.test +++ b/tests/sqllogictests/suites/mode/standalone/explain_native/push_down_filter/push_down_filter_join/push_down_filter_join_left_outer.test @@ -34,6 +34,7 @@ HashJoin ├── estimated rows: 4.00 ├── TableScan(Build) │ ├── table: default.default.t2 +│ ├── scan id: 1 │ ├── output columns: [a (#2), b (#3)] │ ├── read rows: 3 │ ├── read size: < 1 KiB @@ -44,6 +45,7 @@ HashJoin │ └── estimated rows: 3.00 └── TableScan(Probe) ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 4 ├── read size: < 1 KiB @@ -67,6 +69,7 @@ HashJoin ├── estimated rows: 4.00 ├── TableScan(Build) │ ├── table: default.default.t2 +│ ├── scan id: 1 │ ├── output columns: [a (#2), b (#3)] │ ├── read rows: 3 │ ├── read size: < 1 KiB @@ -77,6 +80,7 @@ HashJoin │ └── estimated rows: 3.00 └── TableScan(Probe) ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 4 ├── read size: < 1 KiB @@ -100,6 +104,7 @@ HashJoin ├── estimated rows: 4.00 ├── TableScan(Build) │ ├── table: default.default.t2 +│ ├── scan id: 1 │ ├── output columns: [a (#2), b (#3)] │ ├── read rows: 3 │ ├── read size: < 1 KiB @@ -110,6 +115,7 @@ HashJoin │ └── estimated rows: 3.00 └── TableScan(Probe) ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 4 ├── read size: < 1 KiB @@ -131,10 +137,11 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:0, build key:t2.a (#2), probe key:t1.a (#0), filter type:inlist,min_max +│ └── filter id:0, build key:t2.a (#2), probe targets:[t1.a (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 4.00 ├── TableScan(Build) │ ├── table: default.default.t2 +│ ├── scan id: 1 │ ├── output columns: [a (#2), b (#3)] │ ├── read rows: 3 │ ├── read size: < 1 KiB @@ -145,6 +152,7 @@ HashJoin │ └── estimated rows: 3.00 └── TableScan(Probe) ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 4 ├── read size: < 1 KiB @@ -167,10 +175,11 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:0, build key:t2.a (#2), probe key:t1.a (#0), filter type:inlist,min_max +│ └── filter id:0, build key:t2.a (#2), probe targets:[t1.a (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 4.00 ├── TableScan(Build) │ ├── table: default.default.t2 +│ ├── scan id: 1 │ ├── output columns: [a (#2), b (#3)] │ ├── read rows: 3 │ ├── read size: < 1 KiB @@ -181,6 +190,7 @@ HashJoin │ └── estimated rows: 3.00 └── TableScan(Probe) ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0), b (#1)] ├── read rows: 4 ├── read size: < 1 KiB @@ -223,6 +233,7 @@ HashJoin ├── estimated rows: 0.00 ├── TableScan(Build) │ ├── table: default.default.j1 +│ ├── scan id: 1 │ ├── output columns: [] │ ├── read rows: 0 │ ├── read size: 0 @@ -232,6 +243,7 @@ HashJoin │ └── estimated rows: 0.00 └── TableScan(Probe) ├── table: default.default.m1 + ├── scan id: 0 ├── output columns: [id (#0)] ├── read rows: 0 ├── read size: 0 @@ -270,7 +282,7 @@ HashJoin ├── keys is null equal: [false] ├── filters: [] ├── build join filters: -│ └── filter id:0, build key:t2.a (#0), probe key:t1.a (#2), filter type:bloom,inlist,min_max +│ └── filter id:0, build key:t2.a (#0), probe targets:[t1.a (#2)@scan1], filter type:bloom,inlist,min_max ├── estimated rows: 0.00 ├── EvalScalar(Build) │ ├── output columns: [t.a (#0), action (#1)] @@ -278,6 +290,7 @@ HashJoin │ ├── estimated rows: 0.00 │ └── TableScan │ ├── table: default.default.t +│ ├── scan id: 0 │ ├── output columns: [a (#0)] │ ├── read rows: 0 │ ├── read size: 0 @@ -287,6 +300,7 @@ HashJoin │ └── estimated rows: 0.00 └── TableScan(Probe) ├── table: default.default.t1 + ├── scan id: 1 ├── output columns: [a (#2)] ├── read rows: 5 ├── read size: < 1 KiB diff --git a/tests/sqllogictests/suites/mode/standalone/explain_native/push_down_filter/push_down_filter_join/push_down_filter_join_semi_anti.test b/tests/sqllogictests/suites/mode/standalone/explain_native/push_down_filter/push_down_filter_join/push_down_filter_join_semi_anti.test index db3ba30d7547d..1890e8e898195 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain_native/push_down_filter/push_down_filter_join/push_down_filter_join_semi_anti.test +++ b/tests/sqllogictests/suites/mode/standalone/explain_native/push_down_filter/push_down_filter_join/push_down_filter_join_semi_anti.test @@ -31,6 +31,7 @@ HashJoin ├── estimated rows: 0.00 ├── TableScan(Build) │ ├── table: default.default.t2 +│ ├── scan id: 1 │ ├── output columns: [a (#2)] │ ├── read rows: 0 │ ├── read size: 0 @@ -41,6 +42,7 @@ HashJoin │ └── estimated rows: 0.00 └── TableScan(Probe) ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 0 ├── read size: 0 @@ -64,6 +66,7 @@ HashJoin ├── estimated rows: 0.00 ├── TableScan(Build) │ ├── table: default.default.t2 +│ ├── scan id: 1 │ ├── output columns: [a (#2)] │ ├── read rows: 0 │ ├── read size: 0 @@ -74,6 +77,7 @@ HashJoin │ └── estimated rows: 0.00 └── TableScan(Probe) ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 0 ├── read size: 0 diff --git a/tests/sqllogictests/suites/mode/standalone/explain_native/push_down_filter/push_down_filter_project_set.test b/tests/sqllogictests/suites/mode/standalone/explain_native/push_down_filter/push_down_filter_project_set.test index 792d5b640a885..c66cb9ebe03e6 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain_native/push_down_filter/push_down_filter_project_set.test +++ b/tests/sqllogictests/suites/mode/standalone/explain_native/push_down_filter/push_down_filter_project_set.test @@ -25,6 +25,7 @@ EvalScalar ├── set returning functions: json_path_query(products.details (#1), '$.features.*') └── TableScan ├── table: default.default.products + ├── scan id: 0 ├── output columns: [name (#0), details (#1)] ├── read rows: 3 ├── read size: < 1 KiB @@ -52,6 +53,7 @@ EvalScalar ├── set returning functions: json_path_query(products.details (#1), '$.features.*') └── TableScan ├── table: default.default.products + ├── scan id: 0 ├── output columns: [name (#0), details (#1)] ├── read rows: 3 ├── read size: < 1 KiB diff --git a/tests/sqllogictests/suites/mode/standalone/explain_native/push_down_filter/push_down_filter_scan.test b/tests/sqllogictests/suites/mode/standalone/explain_native/push_down_filter/push_down_filter_scan.test index 7a26bcef438f8..923e52da12edd 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain_native/push_down_filter/push_down_filter_scan.test +++ b/tests/sqllogictests/suites/mode/standalone/explain_native/push_down_filter/push_down_filter_scan.test @@ -17,6 +17,7 @@ Filter ├── estimated rows: 1.00 └── TableScan ├── table: default.default.t + ├── scan id: 0 ├── output columns: [x (#0)] ├── read rows: 2 ├── read size: < 1 KiB diff --git a/tests/sqllogictests/suites/mode/standalone/explain_native/range_pruner.test b/tests/sqllogictests/suites/mode/standalone/explain_native/range_pruner.test index 65ead88dd450e..0b1b36139fb7b 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain_native/range_pruner.test +++ b/tests/sqllogictests/suites/mode/standalone/explain_native/range_pruner.test @@ -13,6 +13,7 @@ EvalScalar ├── estimated rows: 0.40 └── TableScan ├── table: default.default.range_t + ├── scan id: 0 ├── output columns: [] ├── read rows: 0 ├── read size: 0 @@ -31,6 +32,7 @@ EvalScalar ├── estimated rows: 0.00 └── TableScan ├── table: default.default.range_t + ├── scan id: 0 ├── output columns: [] ├── read rows: 0 ├── read size: 0 diff --git a/tests/sqllogictests/suites/mode/standalone/explain_native/select.test b/tests/sqllogictests/suites/mode/standalone/explain_native/select.test index 6e7a0b0b302bd..e4451c1fe9612 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain_native/select.test +++ b/tests/sqllogictests/suites/mode/standalone/explain_native/select.test @@ -3,6 +3,7 @@ explain select * from numbers(1) ---- TableScan ├── table: default.system.numbers +├── scan id: 0 ├── output columns: [number (#0)] ├── read rows: 1 ├── read size: < 1 KiB @@ -20,6 +21,7 @@ Filter ├── estimated rows: 0.50 └── TableScan ├── table: default.system.numbers + ├── scan id: 0 ├── output columns: [number (#0)] ├── read rows: 1 ├── read size: < 1 KiB @@ -41,6 +43,7 @@ EvalScalar ├── estimated rows: 0.20 └── TableScan ├── table: default.system.numbers + ├── scan id: 0 ├── output columns: [number (#0)] ├── read rows: 0 ├── read size: 0 @@ -62,6 +65,7 @@ EvalScalar ├── estimated rows: 0.50 └── TableScan ├── table: default.system.numbers + ├── scan id: 0 ├── output columns: [number (#0)] ├── read rows: 1 ├── read size: < 1 KiB @@ -79,6 +83,7 @@ Filter ├── estimated rows: 0.50 └── TableScan ├── table: default.system.numbers + ├── scan id: 0 ├── output columns: [number (#0)] ├── read rows: 1 ├── read size: < 1 KiB @@ -92,6 +97,7 @@ explain select * from numbers(1) where TRUE and 1 = 1 ---- TableScan ├── table: default.system.numbers +├── scan id: 0 ├── output columns: [number (#0)] ├── read rows: 1 ├── read size: < 1 KiB @@ -120,6 +126,7 @@ explain select a from (select number as a, number as b from numbers(1)) ---- TableScan ├── table: default.system.numbers +├── scan id: 0 ├── output columns: [number (#0)] ├── read rows: 1 ├── read size: < 1 KiB @@ -133,6 +140,7 @@ explain select a from (select number as a, number+1 as b from numbers(1)) ---- TableScan ├── table: default.system.numbers +├── scan id: 0 ├── output columns: [number (#0)] ├── read rows: 1 ├── read size: < 1 KiB @@ -164,6 +172,7 @@ EvalScalar ├── estimated rows: 1.00 └── TableScan ├── table: default.default.t_json + ├── scan id: 0 ├── output columns: [a (#0), v (#1)] ├── read rows: 1 ├── read size: < 1 KiB @@ -182,6 +191,7 @@ EvalScalar ├── estimated rows: 1.00 └── TableScan ├── table: default.default.t_json + ├── scan id: 0 ├── output columns: [a (#0), v (#1)] ├── read rows: 1 ├── read size: < 1 KiB diff --git a/tests/sqllogictests/suites/mode/standalone/explain_native/select_limit_offset.test b/tests/sqllogictests/suites/mode/standalone/explain_native/select_limit_offset.test index 314a118a96bf4..6127a9d2d9f69 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain_native/select_limit_offset.test +++ b/tests/sqllogictests/suites/mode/standalone/explain_native/select_limit_offset.test @@ -139,7 +139,7 @@ Limit ├── keys is null equal: [false] ├── filters: [] ├── build join filters: - │ └── filter id:0, build key:t.a (#0), probe key:t1.a (#1), filter type:inlist,min_max + │ └── filter id:0, build key:t.a (#0), probe targets:[t1.a (#1)@scan1], filter type:bloom,inlist,min_max ├── estimated rows: 2.00 ├── Limit(Build) │ ├── output columns: [t.a (#0)] @@ -148,6 +148,7 @@ Limit │ ├── estimated rows: 1.00 │ └── TableScan │ ├── table: default.default.t + │ ├── scan id: 0 │ ├── output columns: [a (#0)] │ ├── read rows: 1 │ ├── read size: < 1 KiB @@ -158,6 +159,7 @@ Limit │ └── estimated rows: 1.00 └── TableScan(Probe) ├── table: default.default.t1 + ├── scan id: 1 ├── output columns: [a (#1)] ├── read rows: 2 ├── read size: < 1 KiB @@ -184,7 +186,7 @@ Limit ├── keys is null equal: [false] ├── filters: [] ├── build join filters: - │ └── filter id:0, build key:t.a (#1), probe key:t1.a (#0), filter type:inlist,min_max + │ └── filter id:0, build key:t.a (#1), probe targets:[t1.a (#0)@scan0], filter type:bloom,inlist,min_max ├── estimated rows: 2.00 ├── Limit(Build) │ ├── output columns: [t.a (#1)] @@ -193,6 +195,7 @@ Limit │ ├── estimated rows: 1.00 │ └── TableScan │ ├── table: default.default.t + │ ├── scan id: 1 │ ├── output columns: [a (#1)] │ ├── read rows: 1 │ ├── read size: < 1 KiB @@ -203,6 +206,7 @@ Limit │ └── estimated rows: 1.00 └── TableScan(Probe) ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 2 ├── read size: < 1 KiB diff --git a/tests/sqllogictests/suites/mode/standalone/explain_native/sort.test b/tests/sqllogictests/suites/mode/standalone/explain_native/sort.test index b649d14871a38..246f1780e397e 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain_native/sort.test +++ b/tests/sqllogictests/suites/mode/standalone/explain_native/sort.test @@ -10,6 +10,7 @@ Sort(Single) ├── estimated rows: 0.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 0 ├── read size: 0 @@ -27,6 +28,7 @@ Sort(Single) ├── estimated rows: 0.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 0 ├── read size: 0 @@ -44,6 +46,7 @@ Sort(Single) ├── estimated rows: 0.00 └── TableScan ├── table: default.default.t1 + ├── scan id: 0 ├── output columns: [a (#0)] ├── read rows: 0 ├── read size: 0 diff --git a/tests/sqllogictests/suites/mode/standalone/explain_native/union.test b/tests/sqllogictests/suites/mode/standalone/explain_native/union.test index 307a462090810..03a904a1f703c 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain_native/union.test +++ b/tests/sqllogictests/suites/mode/standalone/explain_native/union.test @@ -30,6 +30,7 @@ UnionAll ├── estimated rows: 0.80 ├── TableScan │ ├── table: default.default.t1 +│ ├── scan id: 0 │ ├── output columns: [a (#0), b (#1)] │ ├── read rows: 0 │ ├── read size: 0 @@ -40,6 +41,7 @@ UnionAll │ └── estimated rows: 0.40 └── TableScan ├── table: default.default.t2 + ├── scan id: 1 ├── output columns: [a (#2), b (#3)] ├── read rows: 0 ├── read size: 0 @@ -57,6 +59,7 @@ UnionAll ├── estimated rows: 2.00 ├── TableScan │ ├── table: default.default.t1 +│ ├── scan id: 0 │ ├── output columns: [a (#0), b (#1)] │ ├── read rows: 2 │ ├── read size: < 1 KiB @@ -67,6 +70,7 @@ UnionAll │ └── estimated rows: 1.00 └── TableScan ├── table: default.default.t2 + ├── scan id: 1 ├── output columns: [a (#2), b (#3)] ├── read rows: 2 ├── read size: < 1 KiB @@ -94,6 +98,7 @@ Limit │ ├── estimated rows: 2.00 │ └── TableScan │ ├── table: default.default.t1 + │ ├── scan id: 0 │ ├── output columns: [a (#0), b (#1)] │ ├── read rows: 2 │ ├── read size: < 1 KiB @@ -109,6 +114,7 @@ Limit ├── estimated rows: 2.00 └── TableScan ├── table: default.default.t2 + ├── scan id: 1 ├── output columns: [a (#2), b (#3)] ├── read rows: 2 ├── read size: < 1 KiB @@ -136,6 +142,7 @@ Limit │ ├── estimated rows: 2.00 │ └── TableScan │ ├── table: default.default.t1 + │ ├── scan id: 0 │ ├── output columns: [a (#0), b (#1)] │ ├── read rows: 2 │ ├── read size: < 1 KiB @@ -151,6 +158,7 @@ Limit ├── estimated rows: 2.00 └── TableScan ├── table: default.default.t2 + ├── scan id: 1 ├── output columns: [a (#2), b (#3)] ├── read rows: 2 ├── read size: < 1 KiB @@ -178,6 +186,7 @@ Limit │ ├── estimated rows: 1.00 │ └── TableScan │ ├── table: default.default.t1 + │ ├── scan id: 0 │ ├── output columns: [a (#0), b (#1)] │ ├── read rows: 2 │ ├── read size: < 1 KiB @@ -193,6 +202,7 @@ Limit ├── estimated rows: 1.00 └── TableScan ├── table: default.default.t2 + ├── scan id: 1 ├── output columns: [a (#2), b (#3)] ├── read rows: 2 ├── read size: < 1 KiB diff --git a/tests/sqllogictests/suites/no_table_meta_cache/explain/auto_rebuild_missing_bloom_index.test b/tests/sqllogictests/suites/no_table_meta_cache/explain/auto_rebuild_missing_bloom_index.test index 04d0d1b59f703..a508f3abf5c8f 100644 --- a/tests/sqllogictests/suites/no_table_meta_cache/explain/auto_rebuild_missing_bloom_index.test +++ b/tests/sqllogictests/suites/no_table_meta_cache/explain/auto_rebuild_missing_bloom_index.test @@ -30,6 +30,7 @@ Filter ├── estimated rows: 1.00 └── TableScan ├── table: default.test_auto_rebuild_missing.t + ├── scan id: 0 ├── output columns: [c (#0)] ├── read rows: 0 ├── read size: 0 @@ -57,6 +58,7 @@ Filter ├── estimated rows: 1.00 └── TableScan ├── table: default.test_auto_rebuild_missing.t + ├── scan id: 0 ├── output columns: [c (#0)] ├── read rows: 5 ├── read size: < 1 KiB @@ -85,6 +87,7 @@ Filter ├── estimated rows: 1.00 └── TableScan ├── table: default.test_auto_rebuild_missing.t + ├── scan id: 0 ├── output columns: [c (#0)] ├── read rows: 0 ├── read size: 0 @@ -125,6 +128,7 @@ Filter ├── estimated rows: 2.67 └── TableScan ├── table: default.test_auto_rebuild_missing.t + ├── scan id: 0 ├── output columns: [c (#0), s (#1)] ├── read rows: 0 ├── read size: 0 @@ -157,6 +161,7 @@ Filter ├── estimated rows: 1.00 └── TableScan ├── table: default.test_auto_rebuild_missing.t + ├── scan id: 0 ├── output columns: [c (#0), s (#1)] ├── read rows: 0 ├── read size: 0 @@ -190,4 +195,4 @@ select is_null(`bloom(0)`) as b1, is_null(`bloom(1)`) as b2 from t_index order # re-generate other bloom index query TT select * from t where c = 12; ----- +---- \ No newline at end of file diff --git a/tests/sqllogictests/suites/no_table_meta_cache/explain_native/auto_rebuild_missing_bloom_index.test b/tests/sqllogictests/suites/no_table_meta_cache/explain_native/auto_rebuild_missing_bloom_index.test index e1c8436c959be..ac93bc7d2a80e 100644 --- a/tests/sqllogictests/suites/no_table_meta_cache/explain_native/auto_rebuild_missing_bloom_index.test +++ b/tests/sqllogictests/suites/no_table_meta_cache/explain_native/auto_rebuild_missing_bloom_index.test @@ -26,6 +26,7 @@ explain select * from t where c = 6; ---- TableScan ├── table: default.test_auto_rebuild_missing.t +├── scan id: 0 ├── output columns: [c (#0)] ├── read rows: 0 ├── read size: 0 @@ -49,6 +50,7 @@ explain select * from t where c = 6; ---- TableScan ├── table: default.test_auto_rebuild_missing.t +├── scan id: 0 ├── output columns: [c (#0)] ├── read rows: 5 ├── read size: < 1 KiB @@ -73,6 +75,7 @@ explain select * from t where c = 6; ---- TableScan ├── table: default.test_auto_rebuild_missing.t +├── scan id: 0 ├── output columns: [c (#0)] ├── read rows: 0 ├── read size: 0 @@ -109,6 +112,7 @@ explain select * from t where s = '12'; ---- TableScan ├── table: default.test_auto_rebuild_missing.t +├── scan id: 0 ├── output columns: [c (#0), s (#1)] ├── read rows: 0 ├── read size: 0 @@ -137,6 +141,7 @@ explain select * from t where c = 6; ---- TableScan ├── table: default.test_auto_rebuild_missing.t +├── scan id: 0 ├── output columns: [c (#0), s (#1)] ├── read rows: 0 ├── read size: 0