From b25bc46618a7a6fcacfd8800eea4331a21fe6e45 Mon Sep 17 00:00:00 2001 From: metesynnada <100111937+metesynnada@users.noreply.github.com> Date: Wed, 22 Mar 2023 15:58:19 +0300 Subject: [PATCH 01/25] Increase optimizer performance --- datafusion/core/src/execution/context.rs | 3 - .../physical_optimizer/pipeline_checker.rs | 18 +- .../src/physical_optimizer/pipeline_fixer.rs | 53 +- .../physical_plan/joins/hash_join_utils.rs | 31 +- .../joins/symmetric_hash_join.rs | 513 ++++++++++-------- 5 files changed, 344 insertions(+), 274 deletions(-) diff --git a/datafusion/core/src/execution/context.rs b/datafusion/core/src/execution/context.rs index 397f4109b62d..46879dc0a8a0 100644 --- a/datafusion/core/src/execution/context.rs +++ b/datafusion/core/src/execution/context.rs @@ -1294,9 +1294,6 @@ impl SessionState { // repartitioning and local sorting steps to meet distribution and ordering requirements. // Therefore, it should run before EnforceDistribution and EnforceSorting. Arc::new(JoinSelection::new()), - // Enforce sort before PipelineFixer - Arc::new(EnforceDistribution::new()), - Arc::new(EnforceSorting::new()), // If the query is processing infinite inputs, the PipelineFixer rule applies the // necessary transformations to make the query runnable (if it is not already runnable). // If the query can not be made runnable, the rule emits an error with a diagnostic message. diff --git a/datafusion/core/src/physical_optimizer/pipeline_checker.rs b/datafusion/core/src/physical_optimizer/pipeline_checker.rs index f097196cd1fc..cdb180ef8f3d 100644 --- a/datafusion/core/src/physical_optimizer/pipeline_checker.rs +++ b/datafusion/core/src/physical_optimizer/pipeline_checker.rs @@ -22,8 +22,10 @@ use crate::config::ConfigOptions; use crate::error::Result; use crate::physical_optimizer::PhysicalOptimizerRule; +use crate::physical_plan::joins::SymmetricHashJoinExec; use crate::physical_plan::tree_node::TreeNodeRewritable; use crate::physical_plan::{with_new_children_if_necessary, ExecutionPlan}; +use datafusion_common::DataFusionError; use std::sync::Arc; /// The PipelineChecker rule rejects non-runnable query plans that use @@ -45,7 +47,8 @@ impl PhysicalOptimizerRule for PipelineChecker { _config: &ConfigOptions, ) -> Result> { let pipeline = PipelineStatePropagator::new(plan); - let state = pipeline.transform_up(&check_finiteness_requirements)?; + // TODO: Add config to smhj_can_run_unsorted_filter. + let state = pipeline.transform_up(&|p| check_finiteness_requirements(p, true))?; Ok(state.plan) } @@ -111,10 +114,19 @@ impl TreeNodeRewritable for PipelineStatePropagator { /// This function propagates finiteness information and rejects any plan with /// pipeline-breaking operators acting on infinite inputs. +// TODO: Accept config here in next PR. pub fn check_finiteness_requirements( input: PipelineStatePropagator, + smhj_can_run_unsorted_filter: bool, ) -> Result> { let plan = input.plan; + if let Some(smhj) = plan.as_any().downcast_ref::() { + if smhj.sorted_filter_exprs().iter().any(|s| s.is_none()) + && !smhj_can_run_unsorted_filter + { + return Err(DataFusionError::Plan("Join operation cannot operate on stream without changing the configuration".to_owned())); + } + } let children = input.children_unbounded; plan.unbounded_output(&children).map(|value| { Some(PipelineStatePropagator { @@ -305,7 +317,7 @@ mod sql_tests { FROM test LIMIT 5".to_string(), cases: vec![Arc::new(test1), Arc::new(test2)], - error_operator: "Sort Error".to_string() + error_operator: "Window Error".to_string() }; case.run().await?; @@ -328,7 +340,7 @@ mod sql_tests { SUM(c9) OVER(ORDER BY c9 ASC ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING) as sum1 FROM test".to_string(), cases: vec![Arc::new(test1), Arc::new(test2)], - error_operator: "Sort Error".to_string() + error_operator: "Window Error".to_string() }; case.run().await?; Ok(()) diff --git a/datafusion/core/src/physical_optimizer/pipeline_fixer.rs b/datafusion/core/src/physical_optimizer/pipeline_fixer.rs index 7532914c1258..ee97ed8ee166 100644 --- a/datafusion/core/src/physical_optimizer/pipeline_fixer.rs +++ b/datafusion/core/src/physical_optimizer/pipeline_fixer.rs @@ -29,11 +29,7 @@ use crate::physical_optimizer::pipeline_checker::{ check_finiteness_requirements, PipelineStatePropagator, }; use crate::physical_optimizer::PhysicalOptimizerRule; -use crate::physical_plan::joins::utils::JoinSide; -use crate::physical_plan::joins::{ - convert_sort_expr_with_filter_schema, HashJoinExec, PartitionMode, - SymmetricHashJoinExec, -}; +use crate::physical_plan::joins::{HashJoinExec, PartitionMode, SymmetricHashJoinExec}; use crate::physical_plan::tree_node::TreeNodeRewritable; use crate::physical_plan::ExecutionPlan; use datafusion_common::DataFusionError; @@ -113,39 +109,17 @@ fn check_support(expr: &Arc) -> bool { /// [PhysicalExpr]s, [Operator]s and data types need to be supported, /// and order information must cover every column in the filter expression. fn is_suitable_for_symmetric_hash_join(hash_join: &HashJoinExec) -> Result { - if let Some(filter) = hash_join.filter() { - let left = hash_join.left(); - if let Some(left_ordering) = left.output_ordering() { - let right = hash_join.right(); - if let Some(right_ordering) = right.output_ordering() { - let expr_supported = check_support(filter.expression()); - let left_convertible = convert_sort_expr_with_filter_schema( - &JoinSide::Left, - filter, - &left.schema(), - &left_ordering[0], - )? - .is_some(); - let right_convertible = convert_sort_expr_with_filter_schema( - &JoinSide::Right, - filter, - &right.schema(), - &right_ordering[0], - )? - .is_some(); - let fields_supported = filter - .schema() - .fields() - .iter() - .all(|f| is_datatype_supported(f.data_type())); - return Ok(expr_supported - && fields_supported - && left_convertible - && right_convertible); - } - } - } - Ok(false) + Ok(if let Some(filter) = hash_join.filter() { + let expr_supported = check_support(filter.expression()); + let fields_supported = filter + .schema() + .fields() + .iter() + .all(|f| is_datatype_supported(f.data_type())); + expr_supported && fields_supported + } else { + false + }) } /// This subrule checks if one can replace a hash join with a symmetric hash @@ -298,7 +272,8 @@ fn apply_subrules_and_check_finiteness_requirements( input = value; } } - check_finiteness_requirements(input) + // TODO: Accept config here in next PR. + check_finiteness_requirements(input, true) } #[cfg(test)] diff --git a/datafusion/core/src/physical_plan/joins/hash_join_utils.rs b/datafusion/core/src/physical_plan/joins/hash_join_utils.rs index 6bfc8a1fcf19..6ec0d5597c71 100644 --- a/datafusion/core/src/physical_plan/joins/hash_join_utils.rs +++ b/datafusion/core/src/physical_plan/joins/hash_join_utils.rs @@ -24,7 +24,6 @@ use std::usize; use arrow::datatypes::SchemaRef; -use datafusion_common::DataFusionError; use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::intervals::Interval; use datafusion_physical_expr::rewrite::TreeNodeRewritable; @@ -141,15 +140,13 @@ pub fn build_filter_input_order( filter: &JoinFilter, schema: &SchemaRef, order: &PhysicalSortExpr, -) -> Result { +) -> Result> { if let Some(expr) = convert_sort_expr_with_filter_schema(&side, filter, schema, order)? { - Ok(SortedFilterExpr::new(order.clone(), expr)) + Ok(Some(SortedFilterExpr::new(order.clone(), expr))) } else { - Err(DataFusionError::Plan(format!( - "The {side} side of the join does not have an expression sorted." - ))) + Ok(None) } } @@ -358,7 +355,8 @@ pub mod tests { &filter, &Arc::new(left_child_schema), &left_child_sort_expr, - )?; + )? + .unwrap(); assert!(left_child_sort_expr.eq(left_sort_filter_expr.origin_sorted_expr())); let right_sort_filter_expr = build_filter_input_order( @@ -366,7 +364,8 @@ pub mod tests { &filter, &Arc::new(right_child_schema), &right_child_sort_expr, - )?; + )? + .unwrap(); assert!(right_child_sort_expr.eq(right_sort_filter_expr.origin_sorted_expr())); // Assert that adjusted (left) filter expression matches with `left_child_sort_expr`: @@ -489,8 +488,8 @@ pub mod tests { expr: col("la1", left_schema.as_ref())?, options: SortOptions::default(), } - ) - .is_ok()); + )? + .is_some()); assert!(build_filter_input_order( JoinSide::Left, &filter, @@ -499,8 +498,8 @@ pub mod tests { expr: col("lt1", left_schema.as_ref())?, options: SortOptions::default(), } - ) - .is_err()); + )? + .is_none()); assert!(build_filter_input_order( JoinSide::Right, &filter, @@ -509,8 +508,8 @@ pub mod tests { expr: col("ra1", right_schema.as_ref())?, options: SortOptions::default(), } - ) - .is_ok()); + )? + .is_some()); assert!(build_filter_input_order( JoinSide::Right, &filter, @@ -519,8 +518,8 @@ pub mod tests { expr: col("rb1", right_schema.as_ref())?, options: SortOptions::default(), } - ) - .is_err()); + )? + .is_none()); Ok(()) } diff --git a/datafusion/core/src/physical_plan/joins/symmetric_hash_join.rs b/datafusion/core/src/physical_plan/joins/symmetric_hash_join.rs index 3af983d8f06a..6a9654a2f6a0 100644 --- a/datafusion/core/src/physical_plan/joins/symmetric_hash_join.rs +++ b/datafusion/core/src/physical_plan/joins/symmetric_hash_join.rs @@ -164,11 +164,7 @@ pub struct SymmetricHashJoinExec { /// How the join is performed pub(crate) join_type: JoinType, /// Order information of filter expressions - sorted_filter_exprs: Vec, - /// Left required sort - left_required_sort_exprs: Vec, - /// Right required sort - right_required_sort_exprs: Vec, + sorted_filter_exprs: Vec>, /// Expression graph for interval calculations physical_expr_graph: ExprIntervalGraph, /// The schema once the join is applied @@ -254,87 +250,82 @@ impl SymmetricHashJoinExec { let left_schema = left.schema(); let right_schema = right.schema(); - // Error out if no "on" contraints are given: + // Ensure that at least one "on" constraint is provided for the join: if on.is_empty() { return Err(DataFusionError::Plan( "On constraints in SymmetricHashJoinExec should be non-empty".to_string(), )); } - // Check if the join is valid with the given on constraints: + // Validate if the join operation is feasible with the given "on" constraints: check_join_is_valid(&left_schema, &right_schema, &on)?; - // Build the join schema from the left and right schemas: + // Construct the resulting schema of the join operation using the input schemas and join type: let (schema, column_indices) = build_join_schema(&left_schema, &right_schema, join_type); - // Set a random state for the join: + // Initialize the random state for the join operation: let random_state = RandomState::with_seeds(0, 0, 0, 0); - // Create an expression DAG for the join filter: + // Create a directed acyclic graph (DAG) of expressions for the join filter: let mut physical_expr_graph = ExprIntervalGraph::try_new(filter.expression().clone())?; - // Interval calculations require each column to exhibit monotonicity - // independently. However, a `PhysicalSortExpr` object defines a - // lexicographical ordering, so we can only use their first elements. - // when deducing column monotonicities. - // TODO: Extend the `PhysicalSortExpr` mechanism to express independent - // (i.e. simultaneous) ordering properties of columns. - let (left_ordering, right_ordering) = match ( - left.output_ordering(), - right.output_ordering(), - ) { - (Some([left_ordering, ..]), Some([right_ordering, ..])) => { - (left_ordering, right_ordering) - } - _ => { - return Err(DataFusionError::Plan( - "Symmetric hash join requires its children to have an output ordering".to_string(), - )); - } - }; - - // Build the sorted filter expression for the left child: - let left_filter_expression = build_filter_input_order( - JoinSide::Left, - &filter, - &left.schema(), - left_ordering, - )?; - - // Build the sorted filter expression for the right child: - let right_filter_expression = build_filter_input_order( - JoinSide::Right, - &filter, - &right.schema(), - right_ordering, - )?; - - // Store the left and right sorted filter expressions in a vector - let mut sorted_filter_exprs = - vec![left_filter_expression, right_filter_expression]; + let mut sorted_filter_exprs = vec![]; + + // Build sorted filter expressions for the left join side: + sorted_filter_exprs.push( + left.output_ordering() + .and_then(|orders| orders.get(0)) + .and_then(|order| { + build_filter_input_order( + JoinSide::Left, + &filter, + &left.schema(), + order, + ) + .transpose() + }) + .transpose()?, + ); + // Build sorted filter expressions for the right join side: + sorted_filter_exprs.push( + right + .output_ordering() + .and_then(|orders| orders.get(0)) + .and_then(|order| { + build_filter_input_order( + JoinSide::Right, + &filter, + &right.schema(), + order, + ) + .transpose() + }) + .transpose()?, + ); - // Gather node indices of converted filter expressions in `SortedFilterExpr` + // Collect node indices of the converted filter expressions in `SortedFilterExpr` // using the filter columns vector: let child_node_indexes = physical_expr_graph.gather_node_indices( &sorted_filter_exprs .iter() - .map(|sorted_expr| sorted_expr.filter_expr().clone()) + .filter_map(|sorted_expr| { + sorted_expr.as_ref().map(|expr| expr.filter_expr().clone()) + }) .collect::>(), ); - // Inject calculated node indices into SortedFilterExpr: + // Update SortedFilterExpr instances with the corresponding node indices: for (sorted_expr, (_, index)) in sorted_filter_exprs .iter_mut() .zip(child_node_indexes.iter()) { - sorted_expr.set_node_index(*index); + if let Some(expr) = sorted_expr.as_mut() { + expr.set_node_index(*index) + } } - let left_required_sort_exprs = vec![left_ordering.clone()]; - let right_required_sort_exprs = vec![right_ordering.clone()]; - Ok(SymmetricHashJoinExec { left, right, @@ -342,8 +333,6 @@ impl SymmetricHashJoinExec { filter, join_type: *join_type, sorted_filter_exprs, - left_required_sort_exprs, - right_required_sort_exprs, physical_expr_graph, schema: Arc::new(schema), random_state, @@ -382,6 +371,11 @@ impl SymmetricHashJoinExec { pub fn null_equals_null(&self) -> bool { self.null_equals_null } + + /// Get sorted_filter_exprs + pub fn sorted_filter_exprs(&self) -> &Vec> { + &self.sorted_filter_exprs + } } impl Debug for SymmetricHashJoinExec { @@ -399,13 +393,6 @@ impl ExecutionPlan for SymmetricHashJoinExec { self.schema.clone() } - fn required_input_ordering(&self) -> Vec> { - vec![ - Some(&self.left_required_sort_exprs), - Some(&self.right_required_sort_exprs), - ] - } - fn unbounded_output(&self, children: &[bool]) -> Result { Ok(children.iter().any(|u| *u)) } @@ -507,13 +494,13 @@ impl ExecutionPlan for SymmetricHashJoinExec { let on_right = self.on.iter().map(|on| on.1.clone()).collect::>(); let left_side_joiner = OneSideHashJoiner::new( JoinSide::Left, - self.sorted_filter_exprs[0].clone(), + self.sorted_filter_exprs[0].as_ref().cloned(), on_left, self.left.schema(), ); let right_side_joiner = OneSideHashJoiner::new( JoinSide::Right, - self.sorted_filter_exprs[1].clone(), + self.sorted_filter_exprs[1].as_ref().cloned(), on_right, self.right.schema(), ); @@ -684,39 +671,49 @@ fn prune_hash_values( /// ``` fn calculate_filter_expr_intervals( build_input_buffer: &RecordBatch, - build_sorted_filter_expr: &mut SortedFilterExpr, + build_sorted_filter_expr: Option<&mut SortedFilterExpr>, probe_batch: &RecordBatch, - probe_sorted_filter_expr: &mut SortedFilterExpr, + probe_sorted_filter_expr: Option<&mut SortedFilterExpr>, ) -> Result<()> { // If either build or probe side has no data, return early: if build_input_buffer.num_rows() == 0 || probe_batch.num_rows() == 0 { return Ok(()); } - // Evaluate build side filter expression and convert the result to an array - let build_array = build_sorted_filter_expr - .origin_sorted_expr() - .expr - .evaluate(&build_input_buffer.slice(0, 1))? - .into_array(1); - // Evaluate probe side filter expression and convert the result to an array - let probe_array = probe_sorted_filter_expr - .origin_sorted_expr() - .expr - .evaluate(&probe_batch.slice(probe_batch.num_rows() - 1, 1))? - .into_array(1); + // Evaluate build side filter expression and convert the result to an optional array + let build_array = match build_sorted_filter_expr.as_ref() { + Some(expr) => Some( + expr.origin_sorted_expr() + .expr + .evaluate(&build_input_buffer.slice(0, 1))? + .into_array(1), + ), + None => None, + }; + + // Evaluate probe side filter expression and convert the result to an optional array + let probe_array = match probe_sorted_filter_expr.as_ref() { + Some(expr) => Some( + expr.origin_sorted_expr() + .expr + .evaluate(&probe_batch.slice(probe_batch.num_rows() - 1, 1))? + .into_array(1), + ), + None => None, + }; // Update intervals for both build and probe side filter expressions - for (array, sorted_expr) in vec![ - (build_array, build_sorted_filter_expr), - (probe_array, probe_sorted_filter_expr), + for (array_opt, sorted_expr_opt) in [ + (&build_array, build_sorted_filter_expr), + (&probe_array, probe_sorted_filter_expr), ] { - // Convert the array to a ScalarValue: - let value = ScalarValue::try_from_array(&array, 0)?; - // Create a ScalarValue representing positive or negative infinity for the same data type: - let infinite = ScalarValue::try_from(value.get_datatype())?; - // Update the interval with lower and upper bounds based on the sort option - sorted_expr.set_interval( - if sorted_expr.origin_sorted_expr().options.descending { + // Process the array and sorted filter expressions if both are present: + if let (Some(array), Some(sorted_expr)) = (array_opt, sorted_expr_opt) { + // Convert the array to a ScalarValue: + let value = ScalarValue::try_from_array(array, 0)?; + // Create a ScalarValue representing positive or negative infinity for the same data type: + let infinite = ScalarValue::try_from(value.get_datatype())?; + // Update the interval with lower and upper bounds based on the sort option + let interval = if sorted_expr.origin_sorted_expr().options.descending { Interval { lower: infinite, upper: value, @@ -726,8 +723,10 @@ fn calculate_filter_expr_intervals( lower: value, upper: infinite, } - }, - ); + }; + // Set the calculated interval for the sorted filter expression: + sorted_expr.set_interval(interval); + } } Ok(()) } @@ -947,7 +946,7 @@ struct OneSideHashJoiner { /// Build side build_side: JoinSide, /// Build side filter sort information - sorted_filter_expr: SortedFilterExpr, + sorted_filter_expr: Option, /// Input record batch buffer input_buffer: RecordBatch, /// Columns from the side @@ -971,7 +970,7 @@ struct OneSideHashJoiner { impl OneSideHashJoiner { pub fn new( build_side: JoinSide, - sorted_filter_expr: SortedFilterExpr, + sorted_filter_expr: Option, on: Vec, schema: SchemaRef, ) -> Self { @@ -1184,66 +1183,71 @@ impl OneSideHashJoiner { &mut self, schema: &SchemaRef, probe_batch: &RecordBatch, - probe_side_sorted_filter_expr: &mut SortedFilterExpr, + probe_side_sorted_filter_expr: Option<&mut SortedFilterExpr>, join_type: JoinType, column_indices: &[ColumnIndex], physical_expr_graph: &mut ExprIntervalGraph, ) -> Result> { - // Check if the input buffer is empty: + // Return early if the input buffer is empty: if self.input_buffer.num_rows() == 0 { return Ok(None); } - // Convert the sorted filter expressions into a vector of (node_index, interval) - // tuples for use when updating the interval graph. - let mut filter_intervals = vec![ - ( - self.sorted_filter_expr.node_index(), - self.sorted_filter_expr.interval().clone(), - ), - ( - probe_side_sorted_filter_expr.node_index(), - probe_side_sorted_filter_expr.interval().clone(), - ), - ]; - // Use the join filter intervals to update the physical expression graph: - physical_expr_graph.update_ranges(&mut filter_intervals)?; - // Get the new join filter interval for build side: - let calculated_build_side_interval = filter_intervals.remove(0).1; - // Check if the intervals changed, exit early if not: - if calculated_build_side_interval.eq(self.sorted_filter_expr.interval()) { - return Ok(None); - } - // Determine the pruning length if there was a change in the intervals: - self.sorted_filter_expr - .set_interval(calculated_build_side_interval); - let prune_length = - determine_prune_length(&self.input_buffer, &self.sorted_filter_expr)?; - // If we can not prune, exit early: - if prune_length == 0 { - return Ok(None); - } - // Compute the result, and perform pruning if there are rows to prune: - let result = self.build_side_determined_results( - schema, - prune_length, - probe_batch.schema(), - join_type, - column_indices, - ); - prune_hash_values( - prune_length, - &mut self.hashmap, - &mut self.row_hash_values, - self.deleted_offset as u64, - )?; - for row in self.deleted_offset..(self.deleted_offset + prune_length) { - self.visited_rows.remove(&row); + // Process the build and probe side sorted filter expressions if both are present: + match ( + self.sorted_filter_expr.as_mut(), + probe_side_sorted_filter_expr, + ) { + (Some(sorted_filter_expr), Some(probe_side_sorted_filter_expr)) => { + // Collect the sorted filter expressions into a vector of (node_index, interval) tuples: + let mut filter_intervals = vec![]; + for expr in &[&sorted_filter_expr, &probe_side_sorted_filter_expr] { + filter_intervals.push((expr.node_index(), expr.interval().clone())) + } + // Update the physical expression graph using the join filter intervals: + physical_expr_graph.update_ranges(&mut filter_intervals)?; + // Extract the new join filter interval for the build side: + let calculated_build_side_interval = filter_intervals.remove(0).1; + // If the intervals have not changed, return early without pruning: + if calculated_build_side_interval.eq(sorted_filter_expr.interval()) { + return Ok(None); + } + // Update the build side interval and determine the pruning length: + sorted_filter_expr.set_interval(calculated_build_side_interval); + let prune_length = + determine_prune_length(&self.input_buffer, sorted_filter_expr)?; + // If no rows can be pruned, return early without pruning: + if prune_length == 0 { + return Ok(None); + } + // Compute the result, and perform pruning if there are rows to prune: + let result = self.build_side_determined_results( + schema, + prune_length, + probe_batch.schema(), + join_type, + column_indices, + ); + // Prune the hash values: + prune_hash_values( + prune_length, + &mut self.hashmap, + &mut self.row_hash_values, + self.deleted_offset as u64, + )?; + // Remove pruned rows from the visited rows set: + for row in self.deleted_offset..(self.deleted_offset + prune_length) { + self.visited_rows.remove(&row); + } + // Update the input buffer after pruning: + self.input_buffer = self + .input_buffer + .slice(prune_length, self.input_buffer.num_rows() - prune_length); + // Increment the deleted offset: + self.deleted_offset += prune_length; + result + } + (_, _) => Ok(None), } - self.input_buffer = self - .input_buffer - .slice(prune_length, self.input_buffer.num_rows() - prune_length); - self.deleted_offset += prune_length; - result } } @@ -1356,9 +1360,9 @@ impl SymmetricHashJoinStream { // Calculate filter intervals: calculate_filter_expr_intervals( &build_hash_joiner.input_buffer, - &mut build_hash_joiner.sorted_filter_expr, + build_hash_joiner.sorted_filter_expr.as_mut(), &probe_batch, - &mut probe_hash_joiner.sorted_filter_expr, + probe_hash_joiner.sorted_filter_expr.as_mut(), )?; // Join the two sides: let equal_result = build_hash_joiner.join_with_probe_batch( @@ -1380,7 +1384,7 @@ impl SymmetricHashJoinStream { let anti_result = build_hash_joiner.prune_with_probe_batch( &self.schema, &probe_batch, - &mut probe_hash_joiner.sorted_filter_expr, + probe_hash_joiner.sorted_filter_expr.as_mut(), self.join_type, &self.column_indices, &mut self.physical_expr_graph, @@ -1442,12 +1446,12 @@ mod tests { use crate::physical_plan::{ collect, common, memory::MemoryExec, repartition::RepartitionExec, }; - use crate::prelude::{SessionConfig, SessionContext}; + use crate::prelude::{CsvReadOptions, SessionConfig, SessionContext}; use crate::test_util; use super::*; - const TABLE_SIZE: i32 = 1_000; + const TABLE_SIZE: i32 = 100; fn compare_batches(collected_1: &[RecordBatch], collected_2: &[RecordBatch]) { // compare @@ -1736,28 +1740,31 @@ mod tests { fn create_memory_table( left_batch: RecordBatch, right_batch: RecordBatch, - left_sorted: Vec, - right_sorted: Vec, + left_sorted: Option>, + right_sorted: Option>, batch_size: usize, ) -> Result<(Arc, Arc)> { - Ok(( - Arc::new( - MemoryExec::try_new( - &[split_record_batches(&left_batch, batch_size).unwrap()], - left_batch.schema(), - None, - )? - .with_sort_information(left_sorted), - ), - Arc::new( - MemoryExec::try_new( - &[split_record_batches(&right_batch, batch_size).unwrap()], - right_batch.schema(), - None, - )? - .with_sort_information(right_sorted), - ), - )) + let temp_left = MemoryExec::try_new( + &[split_record_batches(&left_batch, batch_size).unwrap()], + left_batch.schema(), + None, + )?; + let left = if let Some(sorted) = left_sorted { + temp_left.with_sort_information(sorted) + } else { + temp_left + }; + let temp_right = MemoryExec::try_new( + &[split_record_batches(&right_batch, batch_size).unwrap()], + right_batch.schema(), + None, + )?; + let right = if let Some(sorted) = right_sorted { + temp_right.with_sort_information(sorted) + } else { + temp_right + }; + Ok((Arc::new(left), Arc::new(right))) } async fn experiment( @@ -1829,8 +1836,13 @@ mod tests { expr: col("ra1", right_schema)?, options: SortOptions::default(), }]; - let (left, right) = - create_memory_table(left_batch, right_batch, left_sorted, right_sorted, 13)?; + let (left, right) = create_memory_table( + left_batch, + right_batch, + Some(left_sorted), + Some(right_sorted), + 13, + )?; let on = vec![( Column::new_with_schema("lc1", left_schema)?, @@ -1901,8 +1913,13 @@ mod tests { expr: col("ra1", right_schema)?, options: SortOptions::default(), }]; - let (left, right) = - create_memory_table(left_batch, right_batch, left_sorted, right_sorted, 13)?; + let (left, right) = create_memory_table( + left_batch, + right_batch, + Some(left_sorted), + Some(right_sorted), + 13, + )?; let on = vec![( Column::new_with_schema("lc1", left_schema)?, @@ -1934,34 +1951,37 @@ mod tests { Ok(()) } + #[rstest] #[tokio::test(flavor = "multi_thread")] - async fn single_test() -> Result<()> { - let case_expr = 1; - let cardinality = (11, 21); - let join_type = JoinType::Full; - let config = SessionConfig::new().with_repartition_joins(false); + async fn join_without_sort_information( + #[values( + JoinType::Inner, + JoinType::Left, + JoinType::Right, + JoinType::RightSemi, + JoinType::LeftSemi, + JoinType::LeftAnti, + JoinType::RightAnti, + JoinType::Full + )] + join_type: JoinType, + #[values( + (4, 5), + (11, 21), + (31, 71), + (99, 12), + )] + cardinality: (i32, i32), + #[values(0, 1, 2, 3, 4)] case_expr: usize, + ) -> Result<()> { + let config = SessionConfig::new(); let session_ctx = SessionContext::with_config(config); let task_ctx = session_ctx.task_ctx(); let (left_batch, right_batch) = build_sides_record_batches(TABLE_SIZE, cardinality)?; let left_schema = &left_batch.schema(); let right_schema = &right_batch.schema(); - let left_sorted = vec![PhysicalSortExpr { - expr: col("la1_des", left_schema)?, - options: SortOptions { - descending: true, - nulls_first: true, - }, - }]; - let right_sorted = vec![PhysicalSortExpr { - expr: col("ra1_des", right_schema)?, - options: SortOptions { - descending: true, - nulls_first: true, - }, - }]; - let (left, right) = - create_memory_table(left_batch, right_batch, left_sorted, right_sorted, 13)?; + let (left, right) = create_memory_table(left_batch, right_batch, None, None, 13)?; let on = vec![( Column::new_with_schema("lc1", left_schema)?, @@ -2037,8 +2057,13 @@ mod tests { nulls_first: true, }, }]; - let (left, right) = - create_memory_table(left_batch, right_batch, left_sorted, right_sorted, 13)?; + let (left, right) = create_memory_table( + left_batch, + right_batch, + Some(left_sorted), + Some(right_sorted), + 13, + )?; let on = vec![( Column::new_with_schema("lc1", left_schema)?, @@ -2103,6 +2128,43 @@ mod tests { Ok(()) } + #[tokio::test(flavor = "multi_thread")] + async fn join_change_in_planner_without_sort() -> Result<()> { + let config = SessionConfig::new().with_target_partitions(1); + let ctx = SessionContext::with_config(config); + let tmp_dir = TempDir::new().unwrap(); + let left_file_path = tmp_dir.path().join("left.csv"); + File::create(left_file_path.clone()).unwrap(); + let schema = Arc::new(Schema::new(vec![ + Field::new("a1", DataType::UInt32, false), + Field::new("a2", DataType::UInt32, false), + ])); + ctx.register_csv( + "left", + left_file_path.as_os_str().to_str().unwrap(), + CsvReadOptions::new().schema(&schema).mark_infinite(true), + ) + .await?; + let right_file_path = tmp_dir.path().join("right.csv"); + File::create(right_file_path.clone()).unwrap(); + ctx.register_csv( + "right", + right_file_path.as_os_str().to_str().unwrap(), + CsvReadOptions::new().schema(&schema).mark_infinite(true), + ) + .await?; + let df = ctx.sql("EXPLAIN SELECT t1.a1, t1.a2, t2.a1, t2.a2 FROM left as t1 FULL JOIN right as t2 ON t1.a2 = t2.a2 AND t1.a1 > t2.a1 + 3 AND t1.a1 < t2.a1 + 10").await?; + let physical_plan = df.create_physical_plan().await?; + let task_ctx = ctx.task_ctx(); + let results = collect(physical_plan.clone(), task_ctx).await.unwrap(); + let formatted = pretty_format_batches(&results).unwrap().to_string(); + let found = formatted + .lines() + .any(|line| line.contains("SymmetricHashJoinExec")); + assert!(found); + Ok(()) + } + #[tokio::test(flavor = "multi_thread")] async fn build_null_columns_first() -> Result<()> { let join_type = JoinType::Full; @@ -2129,8 +2191,13 @@ mod tests { nulls_first: true, }, }]; - let (left, right) = - create_memory_table(left_batch, right_batch, left_sorted, right_sorted, 13)?; + let (left, right) = create_memory_table( + left_batch, + right_batch, + Some(left_sorted), + Some(right_sorted), + 13, + )?; let on = vec![( Column::new_with_schema("lc1", left_schema)?, @@ -2187,8 +2254,13 @@ mod tests { nulls_first: false, }, }]; - let (left, right) = - create_memory_table(left_batch, right_batch, left_sorted, right_sorted, 13)?; + let (left, right) = create_memory_table( + left_batch, + right_batch, + Some(left_sorted), + Some(right_sorted), + 13, + )?; let on = vec![( Column::new_with_schema("lc1", left_schema)?, @@ -2246,8 +2318,13 @@ mod tests { nulls_first: true, }, }]; - let (left, right) = - create_memory_table(left_batch, right_batch, left_sorted, right_sorted, 13)?; + let (left, right) = create_memory_table( + left_batch, + right_batch, + Some(left_sorted), + Some(right_sorted), + 13, + )?; let on = vec![( Column::new_with_schema("lc1", left_schema)?, @@ -2301,8 +2378,13 @@ mod tests { expr: col("ra1", right_schema)?, options: SortOptions::default(), }]; - let (left, right) = - create_memory_table(left_batch, right_batch, left_sorted, right_sorted, 13)?; + let (left, right) = create_memory_table( + left_batch, + right_batch, + Some(left_sorted), + Some(right_sorted), + 13, + )?; let on = vec![( Column::new_with_schema("lc1", left_schema)?, @@ -2378,8 +2460,13 @@ mod tests { options: SortOptions::default(), }]; // Construct MemoryExec - let (left, right) = - create_memory_table(left_batch, right_batch, left_sorted, right_sorted, 10)?; + let (left, right) = create_memory_table( + left_batch, + right_batch, + Some(left_sorted), + Some(right_sorted), + 10, + )?; // Filter columns, ensure first batches will have matching rows. let intermediate_schema = Schema::new(vec![ @@ -2410,13 +2497,13 @@ mod tests { ]; let filter = JoinFilter::new(filter_expr, column_indices, intermediate_schema); - let left_sorted_filter_expr = SortedFilterExpr::new( + let left_sorted_filter_expr = Some(SortedFilterExpr::new( PhysicalSortExpr { expr: col("la1", &left_schema)?, options: SortOptions::default(), }, Arc::new(Column::new("0", 0)), - ); + )); let mut left_side_joiner = OneSideHashJoiner::new( JoinSide::Left, left_sorted_filter_expr, @@ -2424,13 +2511,13 @@ mod tests { left_schema, ); - let right_sorted_filter_expr = SortedFilterExpr::new( + let right_sorted_filter_expr = Some(SortedFilterExpr::new( PhysicalSortExpr { expr: col("ra1", &right_schema)?, options: SortOptions::default(), }, Arc::new(Column::new("1", 0)), - ); + )); let mut right_side_joiner = OneSideHashJoiner::new( JoinSide::Right, right_sorted_filter_expr, From b5eb32b4fedb3725044a86255e95a887d2ed27b2 Mon Sep 17 00:00:00 2001 From: metesynnada <100111937+metesynnada@users.noreply.github.com> Date: Wed, 22 Mar 2023 19:16:25 +0300 Subject: [PATCH 02/25] Config added. --- datafusion/common/src/config.rs | 4 +++ .../physical_optimizer/pipeline_checker.rs | 14 ++++---- .../src/physical_optimizer/pipeline_fixer.rs | 15 +++++--- .../joins/symmetric_hash_join.rs | 35 +++++++++++++++++++ datafusion/execution/src/config.rs | 6 ++++ 5 files changed, 62 insertions(+), 12 deletions(-) diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index ecc2f444ab7f..025191721b23 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -280,6 +280,10 @@ config_namespace! { /// using the provided `target_partitions` level pub repartition_joins: bool, default = true + /// Should DataFusion allow symmetric hash join for unbounded data sources, even if + /// there is no order present in its children + pub allow_unsorted_symmetric_joins: bool, default = true + /// When set to true, file groups will be repartitioned to achieve maximum parallelism. /// Currently supported only for Parquet format in which case /// multiple row groups from the same file may be read concurrently. If false then each diff --git a/datafusion/core/src/physical_optimizer/pipeline_checker.rs b/datafusion/core/src/physical_optimizer/pipeline_checker.rs index cdb180ef8f3d..d486578838cb 100644 --- a/datafusion/core/src/physical_optimizer/pipeline_checker.rs +++ b/datafusion/core/src/physical_optimizer/pipeline_checker.rs @@ -25,6 +25,7 @@ use crate::physical_optimizer::PhysicalOptimizerRule; use crate::physical_plan::joins::SymmetricHashJoinExec; use crate::physical_plan::tree_node::TreeNodeRewritable; use crate::physical_plan::{with_new_children_if_necessary, ExecutionPlan}; +use datafusion_common::config::OptimizerOptions; use datafusion_common::DataFusionError; use std::sync::Arc; @@ -44,11 +45,11 @@ impl PhysicalOptimizerRule for PipelineChecker { fn optimize( &self, plan: Arc, - _config: &ConfigOptions, + config: &ConfigOptions, ) -> Result> { let pipeline = PipelineStatePropagator::new(plan); - // TODO: Add config to smhj_can_run_unsorted_filter. - let state = pipeline.transform_up(&|p| check_finiteness_requirements(p, true))?; + let state = pipeline + .transform_up(&|p| check_finiteness_requirements(p, &config.optimizer))?; Ok(state.plan) } @@ -114,17 +115,16 @@ impl TreeNodeRewritable for PipelineStatePropagator { /// This function propagates finiteness information and rejects any plan with /// pipeline-breaking operators acting on infinite inputs. -// TODO: Accept config here in next PR. pub fn check_finiteness_requirements( input: PipelineStatePropagator, - smhj_can_run_unsorted_filter: bool, + optimizer_options: &OptimizerOptions, ) -> Result> { let plan = input.plan; if let Some(smhj) = plan.as_any().downcast_ref::() { if smhj.sorted_filter_exprs().iter().any(|s| s.is_none()) - && !smhj_can_run_unsorted_filter + && !optimizer_options.allow_unsorted_symmetric_joins { - return Err(DataFusionError::Plan("Join operation cannot operate on stream without changing the configuration".to_owned())); + return Err(DataFusionError::Plan("Join operation cannot operate on stream without changing the 'allow_unsorted_symmetric_joins' configuration".to_owned())); } } let children = input.children_unbounded; diff --git a/datafusion/core/src/physical_optimizer/pipeline_fixer.rs b/datafusion/core/src/physical_optimizer/pipeline_fixer.rs index ee97ed8ee166..91918611f1ef 100644 --- a/datafusion/core/src/physical_optimizer/pipeline_fixer.rs +++ b/datafusion/core/src/physical_optimizer/pipeline_fixer.rs @@ -25,9 +25,7 @@ use crate::config::ConfigOptions; use crate::error::Result; use crate::physical_optimizer::join_selection::swap_hash_join; -use crate::physical_optimizer::pipeline_checker::{ - check_finiteness_requirements, PipelineStatePropagator, -}; +use crate::physical_optimizer::pipeline_checker::PipelineStatePropagator; use crate::physical_optimizer::PhysicalOptimizerRule; use crate::physical_plan::joins::{HashJoinExec, PartitionMode, SymmetricHashJoinExec}; use crate::physical_plan::tree_node::TreeNodeRewritable; @@ -272,8 +270,15 @@ fn apply_subrules_and_check_finiteness_requirements( input = value; } } - // TODO: Accept config here in next PR. - check_finiteness_requirements(input, true) + let plan = input.plan; + let children = input.children_unbounded; + plan.unbounded_output(&children).map(|value| { + Some(PipelineStatePropagator { + plan, + unbounded: value, + children_unbounded: children, + }) + }) } #[cfg(test)] diff --git a/datafusion/core/src/physical_plan/joins/symmetric_hash_join.rs b/datafusion/core/src/physical_plan/joins/symmetric_hash_join.rs index 6a9654a2f6a0..ee3683cc9a9f 100644 --- a/datafusion/core/src/physical_plan/joins/symmetric_hash_join.rs +++ b/datafusion/core/src/physical_plan/joins/symmetric_hash_join.rs @@ -2165,6 +2165,41 @@ mod tests { Ok(()) } + #[tokio::test(flavor = "multi_thread")] + async fn join_change_in_planner_without_sort_not_allowed() -> Result<()> { + let config = SessionConfig::new().with_allow_unsorted_symmetric_joins(false); + let ctx = SessionContext::with_config(config); + let tmp_dir = TempDir::new().unwrap(); + let left_file_path = tmp_dir.path().join("left.csv"); + File::create(left_file_path.clone()).unwrap(); + let schema = Arc::new(Schema::new(vec![ + Field::new("a1", DataType::UInt32, false), + Field::new("a2", DataType::UInt32, false), + ])); + ctx.register_csv( + "left", + left_file_path.as_os_str().to_str().unwrap(), + CsvReadOptions::new().schema(&schema).mark_infinite(true), + ) + .await?; + let right_file_path = tmp_dir.path().join("right.csv"); + File::create(right_file_path.clone()).unwrap(); + ctx.register_csv( + "right", + right_file_path.as_os_str().to_str().unwrap(), + CsvReadOptions::new().schema(&schema).mark_infinite(true), + ) + .await?; + let df = ctx.sql("SELECT t1.a1, t1.a2, t2.a1, t2.a2 FROM left as t1 FULL JOIN right as t2 ON t1.a2 = t2.a2 AND t1.a1 > t2.a1 + 3 AND t1.a1 < t2.a1 + 10").await?; + match df.create_physical_plan().await { + Ok(_) => panic!("Expecting error."), + Err(e) => { + assert_eq!(e.to_string(), "PipelineChecker\ncaused by\nError during planning: Join operation cannot operate on stream without changing the 'allow_unsorted_symmetric_joins' configuration") + } + } + Ok(()) + } + #[tokio::test(flavor = "multi_thread")] async fn build_null_columns_first() -> Result<()> { let join_type = JoinType::Full; diff --git a/datafusion/execution/src/config.rs b/datafusion/execution/src/config.rs index dbaaba304bf4..cdcc0f10a637 100644 --- a/datafusion/execution/src/config.rs +++ b/datafusion/execution/src/config.rs @@ -186,6 +186,12 @@ impl SessionConfig { self } + /// Enables or disables the allowing unordered symmetric hash join + pub fn with_allow_unsorted_symmetric_joins(mut self, enabled: bool) -> Self { + self.options.optimizer.allow_unsorted_symmetric_joins = enabled; + self + } + /// Enables or disables the use of repartitioning for file scans pub fn with_repartition_file_scans(mut self, enabled: bool) -> Self { self.options.optimizer.repartition_file_scans = enabled; From 575a6d74633a0f7d82d731e7a95c8e4aba105fd4 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Wed, 22 Mar 2023 23:09:53 -0500 Subject: [PATCH 03/25] Simplifications and comment improvements --- datafusion/common/src/config.rs | 4 +- .../physical_optimizer/pipeline_checker.rs | 9 +- .../src/physical_optimizer/pipeline_fixer.rs | 14 +- .../physical_plan/joins/hash_join_utils.rs | 37 ++- .../core/src/physical_plan/joins/mod.rs | 1 - .../joins/symmetric_hash_join.rs | 271 +++++++++--------- .../test_files/information_schema.slt | 1 + 7 files changed, 164 insertions(+), 173 deletions(-) diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index 025191721b23..ee430256fcb4 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -280,8 +280,8 @@ config_namespace! { /// using the provided `target_partitions` level pub repartition_joins: bool, default = true - /// Should DataFusion allow symmetric hash join for unbounded data sources, even if - /// there is no order present in its children + /// Should DataFusion allow symmetric hash joins for unbounded data sources even when + /// its inputs do not have any ordering pub allow_unsorted_symmetric_joins: bool, default = true /// When set to true, file groups will be repartitioned to achieve maximum parallelism. diff --git a/datafusion/core/src/physical_optimizer/pipeline_checker.rs b/datafusion/core/src/physical_optimizer/pipeline_checker.rs index d486578838cb..2b879725779c 100644 --- a/datafusion/core/src/physical_optimizer/pipeline_checker.rs +++ b/datafusion/core/src/physical_optimizer/pipeline_checker.rs @@ -120,11 +120,12 @@ pub fn check_finiteness_requirements( optimizer_options: &OptimizerOptions, ) -> Result> { let plan = input.plan; - if let Some(smhj) = plan.as_any().downcast_ref::() { - if smhj.sorted_filter_exprs().iter().any(|s| s.is_none()) - && !optimizer_options.allow_unsorted_symmetric_joins + if let Some(exec) = plan.as_any().downcast_ref::() { + if !optimizer_options.allow_unsorted_symmetric_joins + && exec.sorted_filter_exprs().iter().any(|s| s.is_none()) { - return Err(DataFusionError::Plan("Join operation cannot operate on stream without changing the 'allow_unsorted_symmetric_joins' configuration".to_owned())); + let msg = "Join operation cannot operate on stream without enabling the 'allow_unsorted_symmetric_joins' configuration flag"; + return Err(DataFusionError::Plan(msg.to_owned())); } } let children = input.children_unbounded; diff --git a/datafusion/core/src/physical_optimizer/pipeline_fixer.rs b/datafusion/core/src/physical_optimizer/pipeline_fixer.rs index 91918611f1ef..8f442f1d9b10 100644 --- a/datafusion/core/src/physical_optimizer/pipeline_fixer.rs +++ b/datafusion/core/src/physical_optimizer/pipeline_fixer.rs @@ -270,15 +270,13 @@ fn apply_subrules_and_check_finiteness_requirements( input = value; } } - let plan = input.plan; - let children = input.children_unbounded; - plan.unbounded_output(&children).map(|value| { - Some(PipelineStatePropagator { - plan, - unbounded: value, - children_unbounded: children, + input + .plan + .unbounded_output(&input.children_unbounded) + .map(|value| { + input.unbounded = value; + Some(input) }) - }) } #[cfg(test)] diff --git a/datafusion/core/src/physical_plan/joins/hash_join_utils.rs b/datafusion/core/src/physical_plan/joins/hash_join_utils.rs index 6ec0d5597c71..66a330afd7ee 100644 --- a/datafusion/core/src/physical_plan/joins/hash_join_utils.rs +++ b/datafusion/core/src/physical_plan/joins/hash_join_utils.rs @@ -76,19 +76,21 @@ pub fn map_origin_col_to_filter_col( Ok(col_to_col_map) } -/// This function analyzes [PhysicalSortExpr] graphs with respect to monotonicity +/// This function analyzes [`PhysicalSortExpr`] graphs with respect to monotonicity /// (sorting) properties. This is necessary since monotonically increasing and/or /// decreasing expressions are required when using join filter expressions for /// data pruning purposes. /// /// The method works as follows: -/// 1. Maps the original columns to the filter columns using the `map_origin_col_to_filter_col` function. -/// 2. Collects all columns in the sort expression using the `PhysicalExprColumnCollector` visitor. -/// 3. Checks if all columns are included in the `column_mapping_information` map. -/// 4. If all columns are included, the sort expression is converted into a filter expression using the `transform_up` and `convert_filter_columns` functions. -/// 5. Searches the converted filter expression in the filter expression using the `check_filter_expr_contains_sort_information`. -/// 6. If an exact match is encountered, returns the converted filter expression as `Some(Arc)`. -/// 7. If all columns are not included or the exact match is not encountered, returns `None`. +/// 1. Maps the original columns to the filter columns using the [`map_origin_col_to_filter_col`] function. +/// 2. Collects all columns in the sort expression using the [`collect_columns`] function. +/// 3. Checks if all columns are included in the map we obtain in the first step. +/// 4. If all columns are included, the sort expression is converted into a filter expression using +/// the [`convert_filter_columns`] function. +/// 5. Searches for the converted filter expression in the filter expression using the +/// [`check_filter_expr_contains_sort_information`] function. +/// 6. If an exact match is found, returns the converted filter expression as [`Some(Arc)`]. +/// 7. If all columns are not included or an exact match is not found, returns [`None`]. /// /// Examples: /// Consider the filter expression "a + b > c + 10 AND a + b < c + 100". @@ -96,7 +98,7 @@ pub fn map_origin_col_to_filter_col( /// 2. If the expression "d@" is sorted, it will not be accepted since the "d@" column is not part of the filter. /// 3. If the expression "a@ + b@ + c@" is sorted, all columns are represented in the filter expression. However, /// there is no exact match, so this expression does not indicate pruning. -pub fn convert_sort_expr_with_filter_schema( +fn convert_sort_expr_with_filter_schema( side: &JoinSide, filter: &JoinFilter, schema: &SchemaRef, @@ -128,12 +130,12 @@ pub fn convert_sort_expr_with_filter_schema( /// This function is used to build the filter expression based on the sort order of input columns. /// -/// It first calls the [convert_sort_expr_with_filter_schema] method to determine if the sort -/// order of columns can be used in the filter expression. If it returns a [Some] value, the -/// method wraps the result in a [SortedFilterExpr] instance with the original sort expression and +/// It first calls the [`convert_sort_expr_with_filter_schema`] method to determine if the sort +/// order of columns can be used in the filter expression. If it returns a [`Some`] value, the +/// method wraps the result in a [`SortedFilterExpr`] instance with the original sort expression and /// the converted filter expression. Otherwise, this function returns an error. /// -/// The [SortedFilterExpr] instance contains information about the sort order of columns that can +/// The `SortedFilterExpr` instance contains information about the sort order of columns that can /// be used in the filter expression, which can be used to optimize the query execution process. pub fn build_filter_input_order( side: JoinSide, @@ -141,13 +143,8 @@ pub fn build_filter_input_order( schema: &SchemaRef, order: &PhysicalSortExpr, ) -> Result> { - if let Some(expr) = - convert_sort_expr_with_filter_schema(&side, filter, schema, order)? - { - Ok(Some(SortedFilterExpr::new(order.clone(), expr))) - } else { - Ok(None) - } + let opt_expr = convert_sort_expr_with_filter_schema(&side, filter, schema, order)?; + Ok(opt_expr.map(|filter_expr| SortedFilterExpr::new(order.clone(), filter_expr))) } /// Convert a physical expression into a filter expression using the given diff --git a/datafusion/core/src/physical_plan/joins/mod.rs b/datafusion/core/src/physical_plan/joins/mod.rs index 8ad50514f0b0..0a1bc147b80c 100644 --- a/datafusion/core/src/physical_plan/joins/mod.rs +++ b/datafusion/core/src/physical_plan/joins/mod.rs @@ -19,7 +19,6 @@ pub use cross_join::CrossJoinExec; pub use hash_join::HashJoinExec; -pub use hash_join_utils::convert_sort_expr_with_filter_schema; pub use nested_loop_join::NestedLoopJoinExec; // Note: SortMergeJoin is not used in plans yet pub use sort_merge_join::SortMergeJoinExec; diff --git a/datafusion/core/src/physical_plan/joins/symmetric_hash_join.rs b/datafusion/core/src/physical_plan/joins/symmetric_hash_join.rs index ee3683cc9a9f..78e1bbd70b00 100644 --- a/datafusion/core/src/physical_plan/joins/symmetric_hash_join.rs +++ b/datafusion/core/src/physical_plan/joins/symmetric_hash_join.rs @@ -250,33 +250,38 @@ impl SymmetricHashJoinExec { let left_schema = left.schema(); let right_schema = right.schema(); - // Ensure that at least one "on" constraint is provided for the join: + // Error out if no "on" contraints are given: if on.is_empty() { return Err(DataFusionError::Plan( "On constraints in SymmetricHashJoinExec should be non-empty".to_string(), )); } - // Validate if the join operation is feasible with the given "on" constraints: + // Check if the join is valid with the given on constraints: check_join_is_valid(&left_schema, &right_schema, &on)?; - // Construct the resulting schema of the join operation using the input schemas and join type: + // Build the join schema from the left and right schemas: let (schema, column_indices) = build_join_schema(&left_schema, &right_schema, join_type); // Initialize the random state for the join operation: let random_state = RandomState::with_seeds(0, 0, 0, 0); - // Create a directed acyclic graph (DAG) of expressions for the join filter: + // Create an expression DAG for the join filter: let mut physical_expr_graph = ExprIntervalGraph::try_new(filter.expression().clone())?; - let mut sorted_filter_exprs = vec![]; + // Interval calculations require each column to exhibit monotonicity + // independently. However, a `PhysicalSortExpr` object defines a + // lexicographical ordering, so we can only use their first elements. + // when deducing column monotonicities. + // TODO: Extend the `PhysicalSortExpr` mechanism to express independent + // (i.e. simultaneous) ordering properties of columns. - // Build sorted filter expressions for the left join side: - sorted_filter_exprs.push( + let mut sorted_filter_exprs = vec![ + // Build sorted filter expressions for the left join side: left.output_ordering() - .and_then(|orders| orders.get(0)) + .and_then(|orders| orders.first()) .and_then(|order| { build_filter_input_order( JoinSide::Left, @@ -287,12 +292,10 @@ impl SymmetricHashJoinExec { .transpose() }) .transpose()?, - ); - // Build sorted filter expressions for the right join side: - sorted_filter_exprs.push( + // Build sorted filter expressions for the right join side: right .output_ordering() - .and_then(|orders| orders.get(0)) + .and_then(|orders| orders.first()) .and_then(|order| { build_filter_input_order( JoinSide::Right, @@ -303,11 +306,11 @@ impl SymmetricHashJoinExec { .transpose() }) .transpose()?, - ); + ]; - // Collect node indices of the converted filter expressions in `SortedFilterExpr` + // Gather node indices of converted filter expressions in `SortedFilterExpr` // using the filter columns vector: - let child_node_indexes = physical_expr_graph.gather_node_indices( + let child_node_indices = physical_expr_graph.gather_node_indices( &sorted_filter_exprs .iter() .filter_map(|sorted_expr| { @@ -319,7 +322,7 @@ impl SymmetricHashJoinExec { // Update SortedFilterExpr instances with the corresponding node indices: for (sorted_expr, (_, index)) in sorted_filter_exprs .iter_mut() - .zip(child_node_indexes.iter()) + .zip(child_node_indices.iter()) { if let Some(expr) = sorted_expr.as_mut() { expr.set_node_index(*index) @@ -494,13 +497,13 @@ impl ExecutionPlan for SymmetricHashJoinExec { let on_right = self.on.iter().map(|on| on.1.clone()).collect::>(); let left_side_joiner = OneSideHashJoiner::new( JoinSide::Left, - self.sorted_filter_exprs[0].as_ref().cloned(), + self.sorted_filter_exprs[0].clone(), on_left, self.left.schema(), ); let right_side_joiner = OneSideHashJoiner::new( JoinSide::Right, - self.sorted_filter_exprs[1].as_ref().cloned(), + self.sorted_filter_exprs[1].clone(), on_right, self.right.schema(), ); @@ -679,55 +682,53 @@ fn calculate_filter_expr_intervals( if build_input_buffer.num_rows() == 0 || probe_batch.num_rows() == 0 { return Ok(()); } - // Evaluate build side filter expression and convert the result to an optional array - let build_array = match build_sorted_filter_expr.as_ref() { - Some(expr) => Some( - expr.origin_sorted_expr() - .expr - .evaluate(&build_input_buffer.slice(0, 1))? - .into_array(1), - ), - None => None, - }; - - // Evaluate probe side filter expression and convert the result to an optional array - let probe_array = match probe_sorted_filter_expr.as_ref() { - Some(expr) => Some( - expr.origin_sorted_expr() - .expr - .evaluate(&probe_batch.slice(probe_batch.num_rows() - 1, 1))? - .into_array(1), - ), - None => None, - }; + // Calculate the interval for the build side filter expression (if present): + if let Some(expr) = build_sorted_filter_expr { + // Note that we use the oldest entry from the build side. + update_filter_expr_interval(&build_input_buffer.slice(0, 1), expr)?; + } + // Calculate the interval for the probe side filter expression (if present): + if let Some(expr) = probe_sorted_filter_expr { + // Note that we use the newest entry from the probe side. + update_filter_expr_interval( + &probe_batch.slice(probe_batch.num_rows() - 1, 1), + expr, + )?; + } + Ok(()) +} - // Update intervals for both build and probe side filter expressions - for (array_opt, sorted_expr_opt) in [ - (&build_array, build_sorted_filter_expr), - (&probe_array, probe_sorted_filter_expr), - ] { - // Process the array and sorted filter expressions if both are present: - if let (Some(array), Some(sorted_expr)) = (array_opt, sorted_expr_opt) { - // Convert the array to a ScalarValue: - let value = ScalarValue::try_from_array(array, 0)?; - // Create a ScalarValue representing positive or negative infinity for the same data type: - let infinite = ScalarValue::try_from(value.get_datatype())?; - // Update the interval with lower and upper bounds based on the sort option - let interval = if sorted_expr.origin_sorted_expr().options.descending { - Interval { - lower: infinite, - upper: value, - } - } else { - Interval { - lower: value, - upper: infinite, - } - }; - // Set the calculated interval for the sorted filter expression: - sorted_expr.set_interval(interval); +/// This is a subroutine of the function [`calculate_filter_expr_intervals`]. +/// It constructs the current interval using the given `batch` and updates +/// the filter expression (i.e. `sorted_expr`) with this interval. +fn update_filter_expr_interval( + batch: &RecordBatch, + sorted_expr: &mut SortedFilterExpr, +) -> Result<()> { + // Evaluate the filter expression and convert the result to an array: + let array = sorted_expr + .origin_sorted_expr() + .expr + .evaluate(batch)? + .into_array(1); + // Convert the array to a ScalarValue: + let value = ScalarValue::try_from_array(&array, 0)?; + // Create a ScalarValue representing positive or negative infinity for the same data type: + let infinite = ScalarValue::try_from(value.get_datatype())?; + // Update the interval with lower and upper bounds based on the sort option: + let interval = if sorted_expr.origin_sorted_expr().options.descending { + Interval { + lower: infinite, + upper: value, } - } + } else { + Interval { + lower: value, + upper: infinite, + } + }; + // Set the calculated interval for the sorted filter expression: + sorted_expr.set_interval(interval); Ok(()) } @@ -1193,61 +1194,59 @@ impl OneSideHashJoiner { return Ok(None); } // Process the build and probe side sorted filter expressions if both are present: - match ( + if let (Some(sorted_filter_expr), Some(probe_side_sorted_filter_expr)) = ( self.sorted_filter_expr.as_mut(), probe_side_sorted_filter_expr, ) { - (Some(sorted_filter_expr), Some(probe_side_sorted_filter_expr)) => { - // Collect the sorted filter expressions into a vector of (node_index, interval) tuples: - let mut filter_intervals = vec![]; - for expr in &[&sorted_filter_expr, &probe_side_sorted_filter_expr] { - filter_intervals.push((expr.node_index(), expr.interval().clone())) - } - // Update the physical expression graph using the join filter intervals: - physical_expr_graph.update_ranges(&mut filter_intervals)?; - // Extract the new join filter interval for the build side: - let calculated_build_side_interval = filter_intervals.remove(0).1; - // If the intervals have not changed, return early without pruning: - if calculated_build_side_interval.eq(sorted_filter_expr.interval()) { - return Ok(None); - } - // Update the build side interval and determine the pruning length: - sorted_filter_expr.set_interval(calculated_build_side_interval); - let prune_length = - determine_prune_length(&self.input_buffer, sorted_filter_expr)?; - // If no rows can be pruned, return early without pruning: - if prune_length == 0 { - return Ok(None); - } - // Compute the result, and perform pruning if there are rows to prune: - let result = self.build_side_determined_results( - schema, - prune_length, - probe_batch.schema(), - join_type, - column_indices, - ); - // Prune the hash values: - prune_hash_values( - prune_length, - &mut self.hashmap, - &mut self.row_hash_values, - self.deleted_offset as u64, - )?; - // Remove pruned rows from the visited rows set: - for row in self.deleted_offset..(self.deleted_offset + prune_length) { - self.visited_rows.remove(&row); - } - // Update the input buffer after pruning: - self.input_buffer = self - .input_buffer - .slice(prune_length, self.input_buffer.num_rows() - prune_length); - // Increment the deleted offset: - self.deleted_offset += prune_length; - result + // Collect the sorted filter expressions into a vector of (node_index, interval) tuples: + let mut filter_intervals = vec![]; + for expr in [&sorted_filter_expr, &probe_side_sorted_filter_expr] { + filter_intervals.push((expr.node_index(), expr.interval().clone())) + } + // Update the physical expression graph using the join filter intervals: + physical_expr_graph.update_ranges(&mut filter_intervals)?; + // Extract the new join filter interval for the build side: + let calculated_build_side_interval = filter_intervals.remove(0).1; + // If the intervals have not changed, return early without pruning: + if calculated_build_side_interval.eq(sorted_filter_expr.interval()) { + return Ok(None); } - (_, _) => Ok(None), + // Update the build side interval and determine the pruning length: + sorted_filter_expr.set_interval(calculated_build_side_interval); + let prune_length = + determine_prune_length(&self.input_buffer, sorted_filter_expr)?; + // If no rows can be pruned, return early without pruning: + if prune_length == 0 { + return Ok(None); + } + // Compute the result and perform pruning if there are rows to prune: + let result = self.build_side_determined_results( + schema, + prune_length, + probe_batch.schema(), + join_type, + column_indices, + ); + // Prune the hash values: + prune_hash_values( + prune_length, + &mut self.hashmap, + &mut self.row_hash_values, + self.deleted_offset as u64, + )?; + // Remove pruned rows from the visited rows set: + for row in self.deleted_offset..(self.deleted_offset + prune_length) { + self.visited_rows.remove(&row); + } + // Update the input buffer after pruning: + self.input_buffer = self + .input_buffer + .slice(prune_length, self.input_buffer.num_rows() - prune_length); + // Increment the deleted offset: + self.deleted_offset += prune_length; + return result; } + Ok(None) } } @@ -1744,26 +1743,22 @@ mod tests { right_sorted: Option>, batch_size: usize, ) -> Result<(Arc, Arc)> { - let temp_left = MemoryExec::try_new( - &[split_record_batches(&left_batch, batch_size).unwrap()], + let mut left = MemoryExec::try_new( + &[split_record_batches(&left_batch, batch_size)?], left_batch.schema(), None, )?; - let left = if let Some(sorted) = left_sorted { - temp_left.with_sort_information(sorted) - } else { - temp_left - }; - let temp_right = MemoryExec::try_new( - &[split_record_batches(&right_batch, batch_size).unwrap()], + if let Some(sorted) = left_sorted { + left = left.with_sort_information(sorted); + } + let mut right = MemoryExec::try_new( + &[split_record_batches(&right_batch, batch_size)?], right_batch.schema(), None, )?; - let right = if let Some(sorted) = right_sorted { - temp_right.with_sort_information(sorted) - } else { - temp_right - }; + if let Some(sorted) = right_sorted { + right = right.with_sort_information(sorted); + } Ok((Arc::new(left), Arc::new(right))) } @@ -2132,9 +2127,9 @@ mod tests { async fn join_change_in_planner_without_sort() -> Result<()> { let config = SessionConfig::new().with_target_partitions(1); let ctx = SessionContext::with_config(config); - let tmp_dir = TempDir::new().unwrap(); + let tmp_dir = TempDir::new()?; let left_file_path = tmp_dir.path().join("left.csv"); - File::create(left_file_path.clone()).unwrap(); + File::create(left_file_path.clone())?; let schema = Arc::new(Schema::new(vec![ Field::new("a1", DataType::UInt32, false), Field::new("a2", DataType::UInt32, false), @@ -2146,7 +2141,7 @@ mod tests { ) .await?; let right_file_path = tmp_dir.path().join("right.csv"); - File::create(right_file_path.clone()).unwrap(); + File::create(right_file_path.clone())?; ctx.register_csv( "right", right_file_path.as_os_str().to_str().unwrap(), @@ -2156,8 +2151,8 @@ mod tests { let df = ctx.sql("EXPLAIN SELECT t1.a1, t1.a2, t2.a1, t2.a2 FROM left as t1 FULL JOIN right as t2 ON t1.a2 = t2.a2 AND t1.a1 > t2.a1 + 3 AND t1.a1 < t2.a1 + 10").await?; let physical_plan = df.create_physical_plan().await?; let task_ctx = ctx.task_ctx(); - let results = collect(physical_plan.clone(), task_ctx).await.unwrap(); - let formatted = pretty_format_batches(&results).unwrap().to_string(); + let results = collect(physical_plan.clone(), task_ctx).await?; + let formatted = pretty_format_batches(&results)?.to_string(); let found = formatted .lines() .any(|line| line.contains("SymmetricHashJoinExec")); @@ -2169,9 +2164,9 @@ mod tests { async fn join_change_in_planner_without_sort_not_allowed() -> Result<()> { let config = SessionConfig::new().with_allow_unsorted_symmetric_joins(false); let ctx = SessionContext::with_config(config); - let tmp_dir = TempDir::new().unwrap(); + let tmp_dir = TempDir::new()?; let left_file_path = tmp_dir.path().join("left.csv"); - File::create(left_file_path.clone()).unwrap(); + File::create(left_file_path.clone())?; let schema = Arc::new(Schema::new(vec![ Field::new("a1", DataType::UInt32, false), Field::new("a2", DataType::UInt32, false), @@ -2183,7 +2178,7 @@ mod tests { ) .await?; let right_file_path = tmp_dir.path().join("right.csv"); - File::create(right_file_path.clone()).unwrap(); + File::create(right_file_path.clone())?; ctx.register_csv( "right", right_file_path.as_os_str().to_str().unwrap(), @@ -2194,7 +2189,7 @@ mod tests { match df.create_physical_plan().await { Ok(_) => panic!("Expecting error."), Err(e) => { - assert_eq!(e.to_string(), "PipelineChecker\ncaused by\nError during planning: Join operation cannot operate on stream without changing the 'allow_unsorted_symmetric_joins' configuration") + assert_eq!(e.to_string(), "PipelineChecker\ncaused by\nError during planning: Join operation cannot operate on stream without enabling the 'allow_unsorted_symmetric_joins' configuration flag") } } Ok(()) diff --git a/datafusion/core/tests/sqllogictests/test_files/information_schema.slt b/datafusion/core/tests/sqllogictests/test_files/information_schema.slt index 25e4195fba6c..0f2afbfc057e 100644 --- a/datafusion/core/tests/sqllogictests/test_files/information_schema.slt +++ b/datafusion/core/tests/sqllogictests/test_files/information_schema.slt @@ -149,6 +149,7 @@ datafusion.execution.target_partitions 7 datafusion.execution.time_zone +00:00 datafusion.explain.logical_plan_only false datafusion.explain.physical_plan_only false +datafusion.optimizer.allow_unsorted_symmetric_joins true datafusion.optimizer.enable_round_robin_repartition true datafusion.optimizer.filter_null_join_keys false datafusion.optimizer.hash_join_single_partition_threshold 1048576 From 05f768cb1575250893ca43c7df4e130200779df3 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Wed, 22 Mar 2023 23:33:43 -0500 Subject: [PATCH 04/25] More simplifications --- .../src/physical_optimizer/pipeline_fixer.rs | 56 +++++++++---------- 1 file changed, 26 insertions(+), 30 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/pipeline_fixer.rs b/datafusion/core/src/physical_optimizer/pipeline_fixer.rs index 8f442f1d9b10..0101c40f6ff8 100644 --- a/datafusion/core/src/physical_optimizer/pipeline_fixer.rs +++ b/datafusion/core/src/physical_optimizer/pipeline_fixer.rs @@ -104,19 +104,16 @@ fn check_support(expr: &Arc) -> bool { /// This function returns whether a given hash join is replaceable by a /// symmetric hash join. Basically, the requirement is that involved -/// [PhysicalExpr]s, [Operator]s and data types need to be supported, +/// [`PhysicalExpr`]s, [`Operator`]s and data types need to be supported, /// and order information must cover every column in the filter expression. -fn is_suitable_for_symmetric_hash_join(hash_join: &HashJoinExec) -> Result { - Ok(if let Some(filter) = hash_join.filter() { - let expr_supported = check_support(filter.expression()); - let fields_supported = filter - .schema() - .fields() - .iter() - .all(|f| is_datatype_supported(f.data_type())); - expr_supported && fields_supported - } else { - false +fn is_suitable_for_symmetric_hash_join(hash_join: &HashJoinExec) -> bool { + hash_join.filter().map_or(false, |filter| { + check_support(filter.expression()) + && filter + .schema() + .fields() + .iter() + .all(|f| is_datatype_supported(f.data_type())) }) } @@ -131,24 +128,23 @@ fn hash_join_convert_symmetric_subrule( if let Some(hash_join) = plan.as_any().downcast_ref::() { let ub_flags = input.children_unbounded; let (left_unbounded, right_unbounded) = (ub_flags[0], ub_flags[1]); - let new_plan = if left_unbounded && right_unbounded { - match is_suitable_for_symmetric_hash_join(hash_join) { - Ok(true) => SymmetricHashJoinExec::try_new( - hash_join.left().clone(), - hash_join.right().clone(), - hash_join - .on() - .iter() - .map(|(l, r)| (l.clone(), r.clone())) - .collect(), - hash_join.filter().unwrap().clone(), - hash_join.join_type(), - hash_join.null_equals_null(), - ) - .map(|e| Arc::new(e) as _), - Ok(false) => Ok(plan), - Err(e) => return Some(Err(e)), - } + let new_plan = if left_unbounded + && right_unbounded + && is_suitable_for_symmetric_hash_join(hash_join) + { + SymmetricHashJoinExec::try_new( + hash_join.left().clone(), + hash_join.right().clone(), + hash_join + .on() + .iter() + .map(|(l, r)| (l.clone(), r.clone())) + .collect(), + hash_join.filter().unwrap().clone(), + hash_join.join_type(), + hash_join.null_equals_null(), + ) + .map(|e| Arc::new(e) as _) } else { Ok(plan) }; From 90d82dfd27a527b9511fdbbb2328cff2a463d67d Mon Sep 17 00:00:00 2001 From: metesynnada <100111937+metesynnada@users.noreply.github.com> Date: Thu, 23 Mar 2023 10:48:28 +0300 Subject: [PATCH 05/25] Revamping tests for unbounded-unbounded cases. --- datafusion/common/src/config.rs | 4 +- .../physical_optimizer/pipeline_checker.rs | 59 +- .../src/physical_optimizer/pipeline_fixer.rs | 4 +- .../physical_plan/joins/hash_join_utils.rs | 2 +- .../joins/symmetric_hash_join.rs | 580 ++++++++++-------- .../test_files/information_schema.slt | 2 +- datafusion/execution/src/config.rs | 4 +- docs/source/user-guide/configs.md | 1 + 8 files changed, 361 insertions(+), 295 deletions(-) diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index ee430256fcb4..a4c4abfa53d0 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -281,8 +281,8 @@ config_namespace! { pub repartition_joins: bool, default = true /// Should DataFusion allow symmetric hash joins for unbounded data sources even when - /// its inputs do not have any ordering - pub allow_unsorted_symmetric_joins: bool, default = true + /// its inputs do not have any ordering or filtering + pub allow_unpruning_symmetric_joins: bool, default = true /// When set to true, file groups will be repartitioned to achieve maximum parallelism. /// Currently supported only for Parquet format in which case diff --git a/datafusion/core/src/physical_optimizer/pipeline_checker.rs b/datafusion/core/src/physical_optimizer/pipeline_checker.rs index 2b879725779c..1f236ce16a38 100644 --- a/datafusion/core/src/physical_optimizer/pipeline_checker.rs +++ b/datafusion/core/src/physical_optimizer/pipeline_checker.rs @@ -121,10 +121,10 @@ pub fn check_finiteness_requirements( ) -> Result> { let plan = input.plan; if let Some(exec) = plan.as_any().downcast_ref::() { - if !optimizer_options.allow_unsorted_symmetric_joins - && exec.sorted_filter_exprs().iter().any(|s| s.is_none()) + if !(optimizer_options.allow_unpruning_symmetric_joins + || exec.check_if_order_information_available()?) { - let msg = "Join operation cannot operate on stream without enabling the 'allow_unsorted_symmetric_joins' configuration flag"; + let msg = "Join operation cannot operate on stream without enabling the 'allow_unpruning_symmetric_joins' configuration flag"; return Err(DataFusionError::Plan(msg.to_owned())); } } @@ -151,27 +151,19 @@ mod sql_tests { source_types: (SourceType::Unbounded, SourceType::Bounded), expect_fail: false, }; + let test2 = BinaryTestCase { - source_types: (SourceType::Unbounded, SourceType::Unbounded), - expect_fail: true, - }; - let test3 = BinaryTestCase { source_types: (SourceType::Bounded, SourceType::Unbounded), expect_fail: true, }; - let test4 = BinaryTestCase { + let test3 = BinaryTestCase { source_types: (SourceType::Bounded, SourceType::Bounded), expect_fail: false, }; let case = QueryCase { sql: "SELECT t2.c1 FROM left as t1 LEFT JOIN right as t2 ON t1.c1 = t2.c1" .to_string(), - cases: vec![ - Arc::new(test1), - Arc::new(test2), - Arc::new(test3), - Arc::new(test4), - ], + cases: vec![Arc::new(test1), Arc::new(test2), Arc::new(test3)], error_operator: "Join Error".to_string(), }; @@ -186,26 +178,17 @@ mod sql_tests { expect_fail: true, }; let test2 = BinaryTestCase { - source_types: (SourceType::Unbounded, SourceType::Unbounded), - expect_fail: true, - }; - let test3 = BinaryTestCase { source_types: (SourceType::Bounded, SourceType::Unbounded), expect_fail: false, }; - let test4 = BinaryTestCase { + let test3 = BinaryTestCase { source_types: (SourceType::Bounded, SourceType::Bounded), expect_fail: false, }; let case = QueryCase { sql: "SELECT t2.c1 FROM left as t1 RIGHT JOIN right as t2 ON t1.c1 = t2.c1" .to_string(), - cases: vec![ - Arc::new(test1), - Arc::new(test2), - Arc::new(test3), - Arc::new(test4), - ], + cases: vec![Arc::new(test1), Arc::new(test2), Arc::new(test3)], error_operator: "Join Error".to_string(), }; @@ -220,26 +203,17 @@ mod sql_tests { expect_fail: false, }; let test2 = BinaryTestCase { - source_types: (SourceType::Unbounded, SourceType::Unbounded), - expect_fail: true, - }; - let test3 = BinaryTestCase { source_types: (SourceType::Bounded, SourceType::Unbounded), expect_fail: false, }; - let test4 = BinaryTestCase { + let test3 = BinaryTestCase { source_types: (SourceType::Bounded, SourceType::Bounded), expect_fail: false, }; let case = QueryCase { sql: "SELECT t2.c1 FROM left as t1 JOIN right as t2 ON t1.c1 = t2.c1" .to_string(), - cases: vec![ - Arc::new(test1), - Arc::new(test2), - Arc::new(test3), - Arc::new(test4), - ], + cases: vec![Arc::new(test1), Arc::new(test2), Arc::new(test3)], error_operator: "Join Error".to_string(), }; @@ -254,26 +228,17 @@ mod sql_tests { expect_fail: true, }; let test2 = BinaryTestCase { - source_types: (SourceType::Unbounded, SourceType::Unbounded), - expect_fail: true, - }; - let test3 = BinaryTestCase { source_types: (SourceType::Bounded, SourceType::Unbounded), expect_fail: true, }; - let test4 = BinaryTestCase { + let test3 = BinaryTestCase { source_types: (SourceType::Bounded, SourceType::Bounded), expect_fail: false, }; let case = QueryCase { sql: "SELECT t2.c1 FROM left as t1 FULL JOIN right as t2 ON t1.c1 = t2.c1" .to_string(), - cases: vec![ - Arc::new(test1), - Arc::new(test2), - Arc::new(test3), - Arc::new(test4), - ], + cases: vec![Arc::new(test1), Arc::new(test2), Arc::new(test3)], error_operator: "Join Error".to_string(), }; diff --git a/datafusion/core/src/physical_optimizer/pipeline_fixer.rs b/datafusion/core/src/physical_optimizer/pipeline_fixer.rs index 0101c40f6ff8..cb8aa7eb5e86 100644 --- a/datafusion/core/src/physical_optimizer/pipeline_fixer.rs +++ b/datafusion/core/src/physical_optimizer/pipeline_fixer.rs @@ -107,7 +107,7 @@ fn check_support(expr: &Arc) -> bool { /// [`PhysicalExpr`]s, [`Operator`]s and data types need to be supported, /// and order information must cover every column in the filter expression. fn is_suitable_for_symmetric_hash_join(hash_join: &HashJoinExec) -> bool { - hash_join.filter().map_or(false, |filter| { + hash_join.filter().map_or(true, |filter| { check_support(filter.expression()) && filter .schema() @@ -140,7 +140,7 @@ fn hash_join_convert_symmetric_subrule( .iter() .map(|(l, r)| (l.clone(), r.clone())) .collect(), - hash_join.filter().unwrap().clone(), + hash_join.filter().cloned(), hash_join.join_type(), hash_join.null_equals_null(), ) diff --git a/datafusion/core/src/physical_plan/joins/hash_join_utils.rs b/datafusion/core/src/physical_plan/joins/hash_join_utils.rs index 66a330afd7ee..b0d09c82ac50 100644 --- a/datafusion/core/src/physical_plan/joins/hash_join_utils.rs +++ b/datafusion/core/src/physical_plan/joins/hash_join_utils.rs @@ -98,7 +98,7 @@ pub fn map_origin_col_to_filter_col( /// 2. If the expression "d@" is sorted, it will not be accepted since the "d@" column is not part of the filter. /// 3. If the expression "a@ + b@ + c@" is sorted, all columns are represented in the filter expression. However, /// there is no exact match, so this expression does not indicate pruning. -fn convert_sort_expr_with_filter_schema( +pub fn convert_sort_expr_with_filter_schema( side: &JoinSide, filter: &JoinFilter, schema: &SchemaRef, diff --git a/datafusion/core/src/physical_plan/joins/symmetric_hash_join.rs b/datafusion/core/src/physical_plan/joins/symmetric_hash_join.rs index 78e1bbd70b00..8e55c6edf439 100644 --- a/datafusion/core/src/physical_plan/joins/symmetric_hash_join.rs +++ b/datafusion/core/src/physical_plan/joins/symmetric_hash_join.rs @@ -43,6 +43,7 @@ use arrow::datatypes::{ArrowNativeType, Schema, SchemaRef}; use arrow::record_batch::RecordBatch; use futures::{Stream, StreamExt}; use hashbrown::{raw::RawTable, HashSet}; +use parking_lot::Mutex; use datafusion_common::{utils::bisect, ScalarValue}; use datafusion_physical_expr::intervals::{ExprIntervalGraph, Interval}; @@ -50,6 +51,7 @@ use datafusion_physical_expr::intervals::{ExprIntervalGraph, Interval}; use crate::error::{DataFusionError, Result}; use crate::execution::context::TaskContext; use crate::logical_expr::JoinType; +use crate::physical_plan::joins::hash_join_utils::convert_sort_expr_with_filter_schema; use crate::physical_plan::{ expressions::Column, expressions::PhysicalSortExpr, @@ -152,6 +154,7 @@ use crate::physical_plan::{ /// making the smallest value in 'left_sorted' 1231 and any rows below (since ascending) /// than that can be dropped from the inner buffer. /// ``` +#[derive(Debug)] pub struct SymmetricHashJoinExec { /// Left side stream pub(crate) left: Arc, @@ -160,13 +163,11 @@ pub struct SymmetricHashJoinExec { /// Set of common columns used to join on pub(crate) on: Vec<(Column, Column)>, /// Filters applied when finding matching rows - pub(crate) filter: JoinFilter, + pub(crate) filter: Option, /// How the join is performed pub(crate) join_type: JoinType, - /// Order information of filter expressions - sorted_filter_exprs: Vec>, - /// Expression graph for interval calculations - physical_expr_graph: ExprIntervalGraph, + /// Expression graph and SortedFilterExprs for interval calculations + filter_order_state: Option>>, /// The schema once the join is applied schema: SchemaRef, /// Shares the `RandomState` for the hashing algorithm @@ -179,6 +180,19 @@ pub struct SymmetricHashJoinExec { pub(crate) null_equals_null: bool, } +struct IntervalCalculatorInnerState { + /// Expression graph for interval calculations + physical_expr_graph: Option, + sorted_filter_exprs: Vec>, + calculated: bool, +} + +impl Debug for IntervalCalculatorInnerState { + fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result { + write!(f, "Exprs({:?})", self.sorted_filter_exprs) + } +} + #[derive(Debug)] struct SymmetricHashJoinSideMetrics { /// Number of batches consumed by this operator @@ -243,7 +257,7 @@ impl SymmetricHashJoinExec { left: Arc, right: Arc, on: JoinOn, - filter: JoinFilter, + filter: Option, join_type: &JoinType, null_equals_null: bool, ) -> Result { @@ -267,67 +281,16 @@ impl SymmetricHashJoinExec { // Initialize the random state for the join operation: let random_state = RandomState::with_seeds(0, 0, 0, 0); - // Create an expression DAG for the join filter: - let mut physical_expr_graph = - ExprIntervalGraph::try_new(filter.expression().clone())?; - - // Interval calculations require each column to exhibit monotonicity - // independently. However, a `PhysicalSortExpr` object defines a - // lexicographical ordering, so we can only use their first elements. - // when deducing column monotonicities. - // TODO: Extend the `PhysicalSortExpr` mechanism to express independent - // (i.e. simultaneous) ordering properties of columns. - - let mut sorted_filter_exprs = vec![ - // Build sorted filter expressions for the left join side: - left.output_ordering() - .and_then(|orders| orders.first()) - .and_then(|order| { - build_filter_input_order( - JoinSide::Left, - &filter, - &left.schema(), - order, - ) - .transpose() - }) - .transpose()?, - // Build sorted filter expressions for the right join side: - right - .output_ordering() - .and_then(|orders| orders.first()) - .and_then(|order| { - build_filter_input_order( - JoinSide::Right, - &filter, - &right.schema(), - order, - ) - .transpose() - }) - .transpose()?, - ]; - - // Gather node indices of converted filter expressions in `SortedFilterExpr` - // using the filter columns vector: - let child_node_indices = physical_expr_graph.gather_node_indices( - &sorted_filter_exprs - .iter() - .filter_map(|sorted_expr| { - sorted_expr.as_ref().map(|expr| expr.filter_expr().clone()) - }) - .collect::>(), - ); - - // Update SortedFilterExpr instances with the corresponding node indices: - for (sorted_expr, (_, index)) in sorted_filter_exprs - .iter_mut() - .zip(child_node_indices.iter()) - { - if let Some(expr) = sorted_expr.as_mut() { - expr.set_node_index(*index) - } - } + let filter_order_state = if filter.is_some() { + let inner_state = IntervalCalculatorInnerState { + physical_expr_graph: None, + sorted_filter_exprs: vec![], + calculated: false, + }; + Some(Arc::new(Mutex::new(inner_state))) + } else { + None + }; Ok(SymmetricHashJoinExec { left, @@ -335,8 +298,7 @@ impl SymmetricHashJoinExec { on, filter, join_type: *join_type, - sorted_filter_exprs, - physical_expr_graph, + filter_order_state, schema: Arc::new(schema), random_state, metrics: ExecutionPlanMetricsSet::new(), @@ -361,8 +323,8 @@ impl SymmetricHashJoinExec { } /// Filters applied before join output - pub fn filter(&self) -> &JoinFilter { - &self.filter + pub fn filter(&self) -> Option<&JoinFilter> { + self.filter.as_ref() } /// How the join is performed @@ -375,15 +337,32 @@ impl SymmetricHashJoinExec { self.null_equals_null } - /// Get sorted_filter_exprs - pub fn sorted_filter_exprs(&self) -> &Vec> { - &self.sorted_filter_exprs - } -} - -impl Debug for SymmetricHashJoinExec { - fn fmt(&self, _f: &mut Formatter<'_>) -> fmt::Result { - todo!() + /// Check if order information covers every column in the filter expression. + pub fn check_if_order_information_available(&self) -> Result { + if let Some(filter) = self.filter() { + let left = self.left(); + if let Some(left_ordering) = left.output_ordering() { + let right = self.right(); + if let Some(right_ordering) = right.output_ordering() { + let left_convertible = convert_sort_expr_with_filter_schema( + &JoinSide::Left, + filter, + &left.schema(), + &left_ordering[0], + )? + .is_some(); + let right_convertible = convert_sort_expr_with_filter_schema( + &JoinSide::Right, + filter, + &right.schema(), + &right_ordering[0], + )? + .is_some(); + return Ok(left_convertible && right_convertible); + } + } + } + Ok(false) } } @@ -469,7 +448,10 @@ impl ExecutionPlan for SymmetricHashJoinExec { fn fmt_as(&self, t: DisplayFormatType, f: &mut fmt::Formatter) -> fmt::Result { match t { DisplayFormatType::Default => { - let display_filter = format!(", filter={:?}", self.filter.expression()); + let display_filter = self.filter.as_ref().map_or_else( + || "".to_string(), + |f| format!(", filter={:?}", f.expression()), + ); write!( f, "SymmetricHashJoinExec: join_type={:?}, on={:?}{}", @@ -495,18 +477,103 @@ impl ExecutionPlan for SymmetricHashJoinExec { ) -> Result { let on_left = self.on.iter().map(|on| on.0.clone()).collect::>(); let on_right = self.on.iter().map(|on| on.1.clone()).collect::>(); - let left_side_joiner = OneSideHashJoiner::new( - JoinSide::Left, - self.sorted_filter_exprs[0].clone(), - on_left, - self.left.schema(), - ); - let right_side_joiner = OneSideHashJoiner::new( - JoinSide::Right, - self.sorted_filter_exprs[1].clone(), - on_right, - self.right.schema(), - ); + let (left_sorted_filter_expr, right_sorted_filter_expr, physical_expr_graph) = + match (&self.filter_order_state, &self.filter) { + (Some(interval_state), Some(filter)) => { + // lock mutexes + let mut filter_order_state = interval_state.lock(); + // if this is the first partition to be invoked then we need to set up initial state + if !filter_order_state.calculated { + // Interval calculations require each column to exhibit monotonicity + // independently. However, a `PhysicalSortExpr` object defines a + // lexicographical ordering, so we can only use their first elements. + // when deducing column monotonicities. + // TODO: Extend the `PhysicalSortExpr` mechanism to express independent + // (i.e. simultaneous) ordering properties of columns. + // Build sorted filter expressions for the left join side: + filter_order_state.sorted_filter_exprs.push( + self.left + .output_ordering() + .and_then(|orders| orders.first()) + .and_then(|order| { + build_filter_input_order( + JoinSide::Left, + filter, + &self.left.schema(), + order, + ) + .transpose() + }) + .transpose()?, + ); + + // Build sorted filter expressions for the right join side: + filter_order_state.sorted_filter_exprs.push( + self.right + .output_ordering() + .and_then(|orders| orders.first()) + .and_then(|order| { + build_filter_input_order( + JoinSide::Right, + filter, + &self.right.schema(), + order, + ) + .transpose() + }) + .transpose()?, + ); + // Gather filter expressions + let filter_exprs = filter_order_state + .sorted_filter_exprs + .iter() + .filter_map(|sorted_expr| { + sorted_expr + .as_ref() + .map(|expr| expr.filter_expr().clone()) + }) + .collect::>(); + + // Create the graph if it is convenient + let physical_expr_graph = if filter_exprs.len() == 2 { + let mut physical_expr_graph = + ExprIntervalGraph::try_new(filter.expression().clone())?; + // Gather node indices of converted filter expressions in `SortedFilterExpr` + // using the filter columns vector: + let child_node_indices = + physical_expr_graph.gather_node_indices(&filter_exprs); + + // Update SortedFilterExpr instances with the corresponding node indices: + for (sorted_expr, (_, index)) in filter_order_state + .sorted_filter_exprs + .iter_mut() + .zip(child_node_indices.iter()) + { + if let Some(expr) = sorted_expr.as_mut() { + expr.set_node_index(*index) + } + } + Some(physical_expr_graph) + } else { + None + }; + filter_order_state.physical_expr_graph = physical_expr_graph; + filter_order_state.calculated = true; + } + ( + filter_order_state.sorted_filter_exprs[0].clone(), + filter_order_state.sorted_filter_exprs[1].clone(), + filter_order_state.physical_expr_graph.as_ref().cloned(), + ) + } + (_, _) => (None, None, None), + }; + + let left_side_joiner = + OneSideHashJoiner::new(JoinSide::Left, on_left, self.left.schema()); + let right_side_joiner = + OneSideHashJoiner::new(JoinSide::Right, on_right, self.right.schema()); + let left_stream = self.left.execute(partition, context.clone())?; let right_stream = self.right.execute(partition, context)?; @@ -521,7 +588,9 @@ impl ExecutionPlan for SymmetricHashJoinExec { right: right_side_joiner, column_indices: self.column_indices.clone(), metrics: SymmetricHashJoinMetrics::new(partition, &self.metrics), - physical_expr_graph: self.physical_expr_graph.clone(), + physical_expr_graph, + left_sorted_filter_expr, + right_sorted_filter_expr, null_equals_null: self.null_equals_null, final_result: false, probe_side: JoinSide::Left, @@ -538,7 +607,7 @@ struct SymmetricHashJoinStream { /// Input schema schema: Arc, /// join filter - filter: JoinFilter, + filter: Option, /// type of the join join_type: JoinType, // left hash joiner @@ -548,7 +617,11 @@ struct SymmetricHashJoinStream { /// Information of index and left / right placement of columns column_indices: Vec, // Range pruner. - physical_expr_graph: ExprIntervalGraph, + physical_expr_graph: Option, + // Left globally sorted filter expr + left_sorted_filter_expr: Option, + // Right globally sorted filter expr + right_sorted_filter_expr: Option, /// Random state used for hashing initialization random_state: RandomState, /// If null_equals_null is true, null == null else null != null @@ -674,27 +747,24 @@ fn prune_hash_values( /// ``` fn calculate_filter_expr_intervals( build_input_buffer: &RecordBatch, - build_sorted_filter_expr: Option<&mut SortedFilterExpr>, + build_sorted_filter_expr: &mut SortedFilterExpr, probe_batch: &RecordBatch, - probe_sorted_filter_expr: Option<&mut SortedFilterExpr>, + probe_sorted_filter_expr: &mut SortedFilterExpr, ) -> Result<()> { // If either build or probe side has no data, return early: if build_input_buffer.num_rows() == 0 || probe_batch.num_rows() == 0 { return Ok(()); } // Calculate the interval for the build side filter expression (if present): - if let Some(expr) = build_sorted_filter_expr { - // Note that we use the oldest entry from the build side. - update_filter_expr_interval(&build_input_buffer.slice(0, 1), expr)?; - } + update_filter_expr_interval( + &build_input_buffer.slice(0, 1), + build_sorted_filter_expr, + )?; // Calculate the interval for the probe side filter expression (if present): - if let Some(expr) = probe_sorted_filter_expr { - // Note that we use the newest entry from the probe side. - update_filter_expr_interval( - &probe_batch.slice(probe_batch.num_rows() - 1, 1), - expr, - )?; - } + update_filter_expr_interval( + &probe_batch.slice(probe_batch.num_rows() - 1, 1), + probe_sorted_filter_expr, + )?; Ok(()) } @@ -946,8 +1016,6 @@ where struct OneSideHashJoiner { /// Build side build_side: JoinSide, - /// Build side filter sort information - sorted_filter_expr: Option, /// Input record batch buffer input_buffer: RecordBatch, /// Columns from the side @@ -969,12 +1037,7 @@ struct OneSideHashJoiner { } impl OneSideHashJoiner { - pub fn new( - build_side: JoinSide, - sorted_filter_expr: Option, - on: Vec, - schema: SchemaRef, - ) -> Self { + pub fn new(build_side: JoinSide, on: Vec, schema: SchemaRef) -> Self { Self { build_side, input_buffer: RecordBatch::new_empty(schema), @@ -982,7 +1045,6 @@ impl OneSideHashJoiner { hashmap: JoinHashMap(RawTable::with_capacity(10_000)), row_hash_values: VecDeque::new(), hashes_buffer: vec![], - sorted_filter_expr, visited_rows: HashSet::new(), offset: 0, deleted_offset: 0, @@ -1048,7 +1110,7 @@ impl OneSideHashJoiner { schema: &SchemaRef, join_type: JoinType, on_probe: &[Column], - filter: &JoinFilter, + filter: Option<&JoinFilter>, probe_batch: &RecordBatch, probe_visited: &mut HashSet, probe_offset: usize, @@ -1057,7 +1119,7 @@ impl OneSideHashJoiner { null_equals_null: bool, ) -> Result> { if self.input_buffer.num_rows() == 0 || probe_batch.num_rows() == 0 { - return Ok(Some(RecordBatch::new_empty(schema.clone()))); + return Ok(None); } let (build_indices, probe_indices) = build_join_indices( probe_batch, @@ -1065,7 +1127,7 @@ impl OneSideHashJoiner { &self.input_buffer, &self.on, on_probe, - Some(filter), + filter, random_state, null_equals_null, &mut self.hashes_buffer, @@ -1100,7 +1162,7 @@ impl OneSideHashJoiner { column_indices, self.build_side, ) - .map(Some) + .map(|batch| (batch.num_rows() > 0).then_some(batch)) } } @@ -1152,7 +1214,7 @@ impl OneSideHashJoiner { column_indices, self.build_side, ) - .map(Some) + .map(|batch| (batch.num_rows() > 0).then_some(batch)) } else { // If we don't need to produce a result, return None Ok(None) @@ -1180,73 +1242,73 @@ impl OneSideHashJoiner { /// /// If there are rows to prune, returns the pruned build side record batch wrapped in an `Ok` variant. /// Otherwise, returns `Ok(None)`. - fn prune_with_probe_batch( + fn calculate_prune_length_with_probe_batch( &mut self, - schema: &SchemaRef, - probe_batch: &RecordBatch, - probe_side_sorted_filter_expr: Option<&mut SortedFilterExpr>, - join_type: JoinType, - column_indices: &[ColumnIndex], + build_side_sorted_filter_expr: &mut SortedFilterExpr, + probe_side_sorted_filter_expr: &mut SortedFilterExpr, physical_expr_graph: &mut ExprIntervalGraph, - ) -> Result> { + ) -> Result { // Return early if the input buffer is empty: if self.input_buffer.num_rows() == 0 { - return Ok(None); + return Ok(0); } // Process the build and probe side sorted filter expressions if both are present: - if let (Some(sorted_filter_expr), Some(probe_side_sorted_filter_expr)) = ( - self.sorted_filter_expr.as_mut(), - probe_side_sorted_filter_expr, - ) { - // Collect the sorted filter expressions into a vector of (node_index, interval) tuples: - let mut filter_intervals = vec![]; - for expr in [&sorted_filter_expr, &probe_side_sorted_filter_expr] { - filter_intervals.push((expr.node_index(), expr.interval().clone())) - } - // Update the physical expression graph using the join filter intervals: - physical_expr_graph.update_ranges(&mut filter_intervals)?; - // Extract the new join filter interval for the build side: - let calculated_build_side_interval = filter_intervals.remove(0).1; - // If the intervals have not changed, return early without pruning: - if calculated_build_side_interval.eq(sorted_filter_expr.interval()) { - return Ok(None); - } - // Update the build side interval and determine the pruning length: - sorted_filter_expr.set_interval(calculated_build_side_interval); - let prune_length = - determine_prune_length(&self.input_buffer, sorted_filter_expr)?; - // If no rows can be pruned, return early without pruning: - if prune_length == 0 { - return Ok(None); - } - // Compute the result and perform pruning if there are rows to prune: - let result = self.build_side_determined_results( - schema, - prune_length, - probe_batch.schema(), - join_type, - column_indices, - ); - // Prune the hash values: - prune_hash_values( - prune_length, - &mut self.hashmap, - &mut self.row_hash_values, - self.deleted_offset as u64, - )?; - // Remove pruned rows from the visited rows set: - for row in self.deleted_offset..(self.deleted_offset + prune_length) { - self.visited_rows.remove(&row); - } - // Update the input buffer after pruning: - self.input_buffer = self - .input_buffer - .slice(prune_length, self.input_buffer.num_rows() - prune_length); - // Increment the deleted offset: - self.deleted_offset += prune_length; - return result; + // Collect the sorted filter expressions into a vector of (node_index, interval) tuples: + let mut filter_intervals = vec![]; + for expr in [ + &build_side_sorted_filter_expr, + &probe_side_sorted_filter_expr, + ] { + filter_intervals.push((expr.node_index(), expr.interval().clone())) + } + // Update the physical expression graph using the join filter intervals: + physical_expr_graph.update_ranges(&mut filter_intervals)?; + // Extract the new join filter interval for the build side: + let calculated_build_side_interval = filter_intervals.remove(0).1; + // If the intervals have not changed, return early without pruning: + if calculated_build_side_interval.eq(build_side_sorted_filter_expr.interval()) { + return Ok(0); + } + // Update the build side interval and determine the pruning length: + build_side_sorted_filter_expr.set_interval(calculated_build_side_interval); + + determine_prune_length(&self.input_buffer, build_side_sorted_filter_expr) + } + + fn prune_internal_state_and_build_anti_result( + &mut self, + prune_length: usize, + schema: &SchemaRef, + probe_batch: &RecordBatch, + join_type: JoinType, + column_indices: &[ColumnIndex], + ) -> Result> { + // Compute the result and perform pruning if there are rows to prune: + let result = self.build_side_determined_results( + schema, + prune_length, + probe_batch.schema(), + join_type, + column_indices, + ); + // Prune the hash values: + prune_hash_values( + prune_length, + &mut self.hashmap, + &mut self.row_hash_values, + self.deleted_offset as u64, + )?; + // Remove pruned rows from the visited rows set: + for row in self.deleted_offset..(self.deleted_offset + prune_length) { + self.visited_rows.remove(&row); } - Ok(None) + // Update the input buffer after pruning: + self.input_buffer = self + .input_buffer + .slice(prune_length, self.input_buffer.num_rows() - prune_length); + // Increment the deleted offset: + self.deleted_offset += prune_length; + result } } @@ -1327,6 +1389,8 @@ impl SymmetricHashJoinStream { input_stream, probe_hash_joiner, build_hash_joiner, + probe_side_sorted_filter_expr, + build_side_sorted_filter_expr, build_join_side, probe_side_metrics, ) = if self.probe_side.eq(&JoinSide::Left) { @@ -1334,6 +1398,8 @@ impl SymmetricHashJoinStream { &mut self.left_stream, &mut self.left, &mut self.right, + &mut self.left_sorted_filter_expr, + &mut self.right_sorted_filter_expr, JoinSide::Right, &mut self.metrics.left, ) @@ -1342,6 +1408,8 @@ impl SymmetricHashJoinStream { &mut self.right_stream, &mut self.right, &mut self.left, + &mut self.right_sorted_filter_expr, + &mut self.left_sorted_filter_expr, JoinSide::Left, &mut self.metrics.right, ) @@ -1356,19 +1424,12 @@ impl SymmetricHashJoinStream { // Update the internal state of the hash joiner for the build side: probe_hash_joiner .update_internal_state(&probe_batch, &self.random_state)?; - // Calculate filter intervals: - calculate_filter_expr_intervals( - &build_hash_joiner.input_buffer, - build_hash_joiner.sorted_filter_expr.as_mut(), - &probe_batch, - probe_hash_joiner.sorted_filter_expr.as_mut(), - )?; // Join the two sides: let equal_result = build_hash_joiner.join_with_probe_batch( &self.schema, self.join_type, &probe_hash_joiner.on, - &self.filter, + self.filter.as_ref(), &probe_batch, &mut probe_hash_joiner.visited_rows, probe_hash_joiner.offset, @@ -1378,16 +1439,45 @@ impl SymmetricHashJoinStream { )?; // Increment the offset for the probe hash joiner: probe_hash_joiner.offset += probe_batch.num_rows(); - // Prune the build side input buffer using the expression - // DAG and filter intervals: - let anti_result = build_hash_joiner.prune_with_probe_batch( - &self.schema, - &probe_batch, - probe_hash_joiner.sorted_filter_expr.as_mut(), - self.join_type, - &self.column_indices, - &mut self.physical_expr_graph, - )?; + + let anti_result = if let ( + Some(build_side_sorted_filter_expr), + Some(probe_side_sorted_filter_expr), + Some(physical_expr_graph), + ) = ( + build_side_sorted_filter_expr.as_mut(), + probe_side_sorted_filter_expr.as_mut(), + self.physical_expr_graph.as_mut(), + ) { + // Calculate filter intervals: + calculate_filter_expr_intervals( + &build_hash_joiner.input_buffer, + build_side_sorted_filter_expr, + &probe_batch, + probe_side_sorted_filter_expr, + )?; + let prune_length = build_hash_joiner + .calculate_prune_length_with_probe_batch( + build_side_sorted_filter_expr, + probe_side_sorted_filter_expr, + physical_expr_graph, + )?; + + if prune_length > 0 { + build_hash_joiner.prune_internal_state_and_build_anti_result( + prune_length, + &self.schema, + &probe_batch, + self.join_type, + &self.column_indices, + )? + } else { + None + } + } else { + None + }; + // Combine results: let result = combine_two_batches(&self.schema, equal_result, anti_result)?; @@ -1478,7 +1568,7 @@ mod tests { left: Arc, right: Arc, on: JoinOn, - filter: JoinFilter, + filter: Option, join_type: &JoinType, null_equals_null: bool, context: Arc, @@ -1529,7 +1619,7 @@ mod tests { left: Arc, right: Arc, on: JoinOn, - filter: JoinFilter, + filter: Option, join_type: &JoinType, null_equals_null: bool, context: Arc, @@ -1551,7 +1641,7 @@ mod tests { Partitioning::Hash(right_expr, partition_count), )?), on, - Some(filter), + filter, join_type, PartitionMode::Partitioned, null_equals_null, @@ -1765,7 +1855,7 @@ mod tests { async fn experiment( left: Arc, right: Arc, - filter: JoinFilter, + filter: Option, join_type: JoinType, on: JoinOn, task_ctx: Arc, @@ -1811,8 +1901,7 @@ mod tests { cardinality: (i32, i32), ) -> Result<()> { // a + b > c + 10 AND a + b < c + 100 - let config = SessionConfig::new().with_repartition_joins(false); - let session_ctx = SessionContext::with_config(config); + let session_ctx = SessionContext::new(); let task_ctx = session_ctx.task_ctx(); let (left_batch, right_batch) = build_sides_record_batches(TABLE_SIZE, cardinality)?; @@ -1866,7 +1955,7 @@ mod tests { ]; let filter = JoinFilter::new(filter_expr, column_indices, intermediate_schema); - experiment(left, right, filter, join_type, on, task_ctx).await?; + experiment(left, right, Some(filter), join_type, on, task_ctx).await?; Ok(()) } @@ -1893,8 +1982,7 @@ mod tests { cardinality: (i32, i32), #[values(0, 1, 2, 3, 4)] case_expr: usize, ) -> Result<()> { - let config = SessionConfig::new().with_repartition_joins(false); - let session_ctx = SessionContext::with_config(config); + let session_ctx = SessionContext::new(); let task_ctx = session_ctx.task_ctx(); let (left_batch, right_batch) = build_sides_record_batches(TABLE_SIZE, cardinality)?; @@ -1942,7 +2030,7 @@ mod tests { ]; let filter = JoinFilter::new(filter_expr, column_indices, intermediate_schema); - experiment(left, right, filter, join_type, on, task_ctx).await?; + experiment(left, right, Some(filter), join_type, on, task_ctx).await?; Ok(()) } @@ -1969,8 +2057,7 @@ mod tests { cardinality: (i32, i32), #[values(0, 1, 2, 3, 4)] case_expr: usize, ) -> Result<()> { - let config = SessionConfig::new(); - let session_ctx = SessionContext::with_config(config); + let session_ctx = SessionContext::new(); let task_ctx = session_ctx.task_ctx(); let (left_batch, right_batch) = build_sides_record_batches(TABLE_SIZE, cardinality)?; @@ -2004,7 +2091,37 @@ mod tests { ]; let filter = JoinFilter::new(filter_expr, column_indices, intermediate_schema); - experiment(left, right, filter, join_type, on, task_ctx).await?; + experiment(left, right, Some(filter), join_type, on, task_ctx).await?; + Ok(()) + } + + #[rstest] + #[tokio::test(flavor = "multi_thread")] + async fn join_without_filter( + #[values( + JoinType::Inner, + JoinType::Left, + JoinType::Right, + JoinType::RightSemi, + JoinType::LeftSemi, + JoinType::LeftAnti, + JoinType::RightAnti, + JoinType::Full + )] + join_type: JoinType, + ) -> Result<()> { + let session_ctx = SessionContext::new(); + let task_ctx = session_ctx.task_ctx(); + let (left_batch, right_batch) = build_sides_record_batches(TABLE_SIZE, (11, 21))?; + let left_schema = &left_batch.schema(); + let right_schema = &right_batch.schema(); + let (left, right) = create_memory_table(left_batch, right_batch, None, None, 13)?; + + let on = vec![( + Column::new_with_schema("lc1", left_schema)?, + Column::new_with_schema("rc1", right_schema)?, + )]; + experiment(left, right, None, join_type, on, task_ctx).await?; Ok(()) } @@ -2031,8 +2148,7 @@ mod tests { cardinality: (i32, i32), #[values(0, 1, 2, 3, 4)] case_expr: usize, ) -> Result<()> { - let config = SessionConfig::new().with_repartition_joins(false); - let session_ctx = SessionContext::with_config(config); + let session_ctx = SessionContext::new(); let task_ctx = session_ctx.task_ctx(); let (left_batch, right_batch) = build_sides_record_batches(TABLE_SIZE, cardinality)?; @@ -2086,7 +2202,7 @@ mod tests { ]; let filter = JoinFilter::new(filter_expr, column_indices, intermediate_schema); - experiment(left, right, filter, join_type, on, task_ctx).await?; + experiment(left, right, Some(filter), join_type, on, task_ctx).await?; Ok(()) } @@ -2162,7 +2278,7 @@ mod tests { #[tokio::test(flavor = "multi_thread")] async fn join_change_in_planner_without_sort_not_allowed() -> Result<()> { - let config = SessionConfig::new().with_allow_unsorted_symmetric_joins(false); + let config = SessionConfig::new().with_allow_unpruning_symmetric_joins(false); let ctx = SessionContext::with_config(config); let tmp_dir = TempDir::new()?; let left_file_path = tmp_dir.path().join("left.csv"); @@ -2189,7 +2305,7 @@ mod tests { match df.create_physical_plan().await { Ok(_) => panic!("Expecting error."), Err(e) => { - assert_eq!(e.to_string(), "PipelineChecker\ncaused by\nError during planning: Join operation cannot operate on stream without enabling the 'allow_unsorted_symmetric_joins' configuration flag") + assert_eq!(e.to_string(), "PipelineChecker\ncaused by\nError during planning: Join operation cannot operate on stream without enabling the 'allow_unpruning_symmetric_joins' configuration flag") } } Ok(()) @@ -2254,7 +2370,7 @@ mod tests { }, ]; let filter = JoinFilter::new(filter_expr, column_indices, intermediate_schema); - experiment(left, right, filter, join_type, on, task_ctx).await?; + experiment(left, right, Some(filter), join_type, on, task_ctx).await?; Ok(()) } @@ -2318,7 +2434,7 @@ mod tests { ]; let filter = JoinFilter::new(filter_expr, column_indices, intermediate_schema); - experiment(left, right, filter, join_type, on, task_ctx).await?; + experiment(left, right, Some(filter), join_type, on, task_ctx).await?; Ok(()) } @@ -2382,7 +2498,7 @@ mod tests { ]; let filter = JoinFilter::new(filter_expr, column_indices, intermediate_schema); - experiment(left, right, filter, join_type, on, task_ctx).await?; + experiment(left, right, Some(filter), join_type, on, task_ctx).await?; Ok(()) } @@ -2443,7 +2559,7 @@ mod tests { ]; let filter = JoinFilter::new(filter_expr, column_indices, intermediate_schema); - experiment(left, right, filter, join_type, on, task_ctx).await?; + experiment(left, right, Some(filter), join_type, on, task_ctx).await?; Ok(()) } @@ -2527,30 +2643,14 @@ mod tests { ]; let filter = JoinFilter::new(filter_expr, column_indices, intermediate_schema); - let left_sorted_filter_expr = Some(SortedFilterExpr::new( - PhysicalSortExpr { - expr: col("la1", &left_schema)?, - options: SortOptions::default(), - }, - Arc::new(Column::new("0", 0)), - )); let mut left_side_joiner = OneSideHashJoiner::new( JoinSide::Left, - left_sorted_filter_expr, vec![Column::new_with_schema("lc1", &left_schema)?], left_schema, ); - let right_sorted_filter_expr = Some(SortedFilterExpr::new( - PhysicalSortExpr { - expr: col("ra1", &right_schema)?, - options: SortOptions::default(), - }, - Arc::new(Column::new("1", 0)), - )); let mut right_side_joiner = OneSideHashJoiner::new( JoinSide::Right, - right_sorted_filter_expr, vec![Column::new_with_schema("rc1", &right_schema)?], right_schema, ); @@ -2576,7 +2676,7 @@ mod tests { &join_schema, join_type, &right_side_joiner.on, - &filter, + Some(&filter), &initial_right_batch, &mut right_side_joiner.visited_rows, right_side_joiner.offset, diff --git a/datafusion/core/tests/sqllogictests/test_files/information_schema.slt b/datafusion/core/tests/sqllogictests/test_files/information_schema.slt index 0f2afbfc057e..21593d256230 100644 --- a/datafusion/core/tests/sqllogictests/test_files/information_schema.slt +++ b/datafusion/core/tests/sqllogictests/test_files/information_schema.slt @@ -149,7 +149,7 @@ datafusion.execution.target_partitions 7 datafusion.execution.time_zone +00:00 datafusion.explain.logical_plan_only false datafusion.explain.physical_plan_only false -datafusion.optimizer.allow_unsorted_symmetric_joins true +datafusion.optimizer.allow_unpruning_symmetric_joins true datafusion.optimizer.enable_round_robin_repartition true datafusion.optimizer.filter_null_join_keys false datafusion.optimizer.hash_join_single_partition_threshold 1048576 diff --git a/datafusion/execution/src/config.rs b/datafusion/execution/src/config.rs index cdcc0f10a637..13f50b6e3106 100644 --- a/datafusion/execution/src/config.rs +++ b/datafusion/execution/src/config.rs @@ -187,8 +187,8 @@ impl SessionConfig { } /// Enables or disables the allowing unordered symmetric hash join - pub fn with_allow_unsorted_symmetric_joins(mut self, enabled: bool) -> Self { - self.options.optimizer.allow_unsorted_symmetric_joins = enabled; + pub fn with_allow_unpruning_symmetric_joins(mut self, enabled: bool) -> Self { + self.options.optimizer.allow_unpruning_symmetric_joins = enabled; self } diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 1e457ee9b608..2c7ddae688e9 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -60,6 +60,7 @@ Environment variables are read during `SessionConfig` initialisation so they mus | datafusion.optimizer.repartition_aggregations | true | Should DataFusion repartition data using the aggregate keys to execute aggregates in parallel using the provided `target_partitions` level | | datafusion.optimizer.repartition_file_min_size | 10485760 | Minimum total files size in bytes to perform file scan repartitioning. | | datafusion.optimizer.repartition_joins | true | Should DataFusion repartition data using the join keys to execute joins in parallel using the provided `target_partitions` level | +| datafusion.optimizer.allow_unpruning_symmetric_joins | true | Should DataFusion allow symmetric hash joins for unbounded data sources even when its inputs do not have any ordering or filtering | | datafusion.optimizer.repartition_file_scans | true | When set to true, file groups will be repartitioned to achieve maximum parallelism. Currently supported only for Parquet format in which case multiple row groups from the same file may be read concurrently. If false then each row group is read serially, though different files may be read in parallel. | | datafusion.optimizer.repartition_windows | true | Should DataFusion repartition data using the partitions keys to execute window functions in parallel using the provided `target_partitions` level | | datafusion.optimizer.repartition_sorts | true | Should DataFusion execute sorts in a per-partition fashion and merge afterwards instead of coalescing first and sorting globally. With this flag is enabled, plans in the form below "SortExec: [a@0 ASC]", " CoalescePartitionsExec", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", would turn into the plan below which performs better in multithreaded environments "SortPreservingMergeExec: [a@0 ASC]", " SortExec: [a@0 ASC]", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", | From 8886457ce6f8374bcf4ff77484f347a688372929 Mon Sep 17 00:00:00 2001 From: metesynnada <100111937+metesynnada@users.noreply.github.com> Date: Fri, 24 Mar 2023 11:25:59 +0300 Subject: [PATCH 06/25] Review code --- datafusion/common/src/config.rs | 2 +- .../physical_optimizer/pipeline_checker.rs | 4 +- .../src/physical_optimizer/pipeline_fixer.rs | 3 +- .../joins/symmetric_hash_join.rs | 68 ++++++++--------- .../test_files/information_schema.slt | 2 +- datafusion/execution/src/config.rs | 4 +- docs/source/user-guide/configs.md | 76 +++++++++---------- 7 files changed, 79 insertions(+), 80 deletions(-) diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index a4c4abfa53d0..9daa6f02282b 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -282,7 +282,7 @@ config_namespace! { /// Should DataFusion allow symmetric hash joins for unbounded data sources even when /// its inputs do not have any ordering or filtering - pub allow_unpruning_symmetric_joins: bool, default = true + pub allow_symmetric_joins_without_pruning: bool, default = true /// When set to true, file groups will be repartitioned to achieve maximum parallelism. /// Currently supported only for Parquet format in which case diff --git a/datafusion/core/src/physical_optimizer/pipeline_checker.rs b/datafusion/core/src/physical_optimizer/pipeline_checker.rs index 1f236ce16a38..0ba27338a899 100644 --- a/datafusion/core/src/physical_optimizer/pipeline_checker.rs +++ b/datafusion/core/src/physical_optimizer/pipeline_checker.rs @@ -121,10 +121,10 @@ pub fn check_finiteness_requirements( ) -> Result> { let plan = input.plan; if let Some(exec) = plan.as_any().downcast_ref::() { - if !(optimizer_options.allow_unpruning_symmetric_joins + if !(optimizer_options.allow_symmetric_joins_without_pruning || exec.check_if_order_information_available()?) { - let msg = "Join operation cannot operate on stream without enabling the 'allow_unpruning_symmetric_joins' configuration flag"; + let msg = "Join operation cannot operate on stream without enabling the 'allow_symmetric_joins_without_pruning' configuration flag"; return Err(DataFusionError::Plan(msg.to_owned())); } } diff --git a/datafusion/core/src/physical_optimizer/pipeline_fixer.rs b/datafusion/core/src/physical_optimizer/pipeline_fixer.rs index cb8aa7eb5e86..a7763d90b1da 100644 --- a/datafusion/core/src/physical_optimizer/pipeline_fixer.rs +++ b/datafusion/core/src/physical_optimizer/pipeline_fixer.rs @@ -104,8 +104,7 @@ fn check_support(expr: &Arc) -> bool { /// This function returns whether a given hash join is replaceable by a /// symmetric hash join. Basically, the requirement is that involved -/// [`PhysicalExpr`]s, [`Operator`]s and data types need to be supported, -/// and order information must cover every column in the filter expression. +/// [`PhysicalExpr`]s, [`Operator`]s and data types need to be supported fn is_suitable_for_symmetric_hash_join(hash_join: &HashJoinExec) -> bool { hash_join.filter().map_or(true, |filter| { check_support(filter.expression()) diff --git a/datafusion/core/src/physical_plan/joins/symmetric_hash_join.rs b/datafusion/core/src/physical_plan/joins/symmetric_hash_join.rs index 8e55c6edf439..172aaf09e711 100644 --- a/datafusion/core/src/physical_plan/joins/symmetric_hash_join.rs +++ b/datafusion/core/src/physical_plan/joins/symmetric_hash_join.rs @@ -477,10 +477,12 @@ impl ExecutionPlan for SymmetricHashJoinExec { ) -> Result { let on_left = self.on.iter().map(|on| on.0.clone()).collect::>(); let on_right = self.on.iter().map(|on| on.1.clone()).collect::>(); + // If self.filter_order_state and self.filter are both present, then calculate sorted filter expressions + // for the left and right sides and build a physical expression graph from them let (left_sorted_filter_expr, right_sorted_filter_expr, physical_expr_graph) = match (&self.filter_order_state, &self.filter) { (Some(interval_state), Some(filter)) => { - // lock mutexes + // Lock the mutexes of the interval state let mut filter_order_state = interval_state.lock(); // if this is the first partition to be invoked then we need to set up initial state if !filter_order_state.calculated { @@ -490,39 +492,27 @@ impl ExecutionPlan for SymmetricHashJoinExec { // when deducing column monotonicities. // TODO: Extend the `PhysicalSortExpr` mechanism to express independent // (i.e. simultaneous) ordering properties of columns. - // Build sorted filter expressions for the left join side: - filter_order_state.sorted_filter_exprs.push( - self.left - .output_ordering() - .and_then(|orders| orders.first()) - .and_then(|order| { - build_filter_input_order( - JoinSide::Left, - filter, - &self.left.schema(), - order, - ) - .transpose() - }) - .transpose()?, - ); - // Build sorted filter expressions for the right join side: - filter_order_state.sorted_filter_exprs.push( - self.right + // Build sorted filter expressions for the left and right join side: + let join_sides = [JoinSide::Left, JoinSide::Right]; + let children = [&self.left, &self.right]; + for (join_side, child) in join_sides.iter().zip(children.iter()) { + let sorted_expr = child .output_ordering() .and_then(|orders| orders.first()) .and_then(|order| { build_filter_input_order( - JoinSide::Right, + *join_side, filter, - &self.right.schema(), + &child.schema(), order, ) .transpose() }) - .transpose()?, - ); + .transpose()?; + + filter_order_state.sorted_filter_exprs.push(sorted_expr); + } // Gather filter expressions let filter_exprs = filter_order_state .sorted_filter_exprs @@ -534,8 +524,12 @@ impl ExecutionPlan for SymmetricHashJoinExec { }) .collect::>(); - // Create the graph if it is convenient - let physical_expr_graph = if filter_exprs.len() == 2 { + // Create the physical expression graph if sorted filter expressions can be created for both children: + let physical_expr_graph = if filter_order_state + .sorted_filter_exprs + .iter() + .all(Option::is_some) + { let mut physical_expr_graph = ExprIntervalGraph::try_new(filter.expression().clone())?; // Gather node indices of converted filter expressions in `SortedFilterExpr` @@ -544,28 +538,33 @@ impl ExecutionPlan for SymmetricHashJoinExec { physical_expr_graph.gather_node_indices(&filter_exprs); // Update SortedFilterExpr instances with the corresponding node indices: - for (sorted_expr, (_, index)) in filter_order_state + filter_order_state .sorted_filter_exprs .iter_mut() + .map(Option::as_mut) + .map(Option::unwrap) .zip(child_node_indices.iter()) - { - if let Some(expr) = sorted_expr.as_mut() { - expr.set_node_index(*index) - } - } + .for_each(|(sorted_expr, (_, index))| { + sorted_expr.set_node_index(*index); + }); + Some(physical_expr_graph) } else { None }; + // Store the calculated physical expression graph in the interval state filter_order_state.physical_expr_graph = physical_expr_graph; filter_order_state.calculated = true; } + // Return the sorted filter expressions for the left and right sides, along with the physical + // expression graph ( filter_order_state.sorted_filter_exprs[0].clone(), filter_order_state.sorted_filter_exprs[1].clone(), filter_order_state.physical_expr_graph.as_ref().cloned(), ) } + // If self.filter_order_state or self.filter is None, then return None for all three values (_, _) => (None, None, None), }; @@ -2278,7 +2277,8 @@ mod tests { #[tokio::test(flavor = "multi_thread")] async fn join_change_in_planner_without_sort_not_allowed() -> Result<()> { - let config = SessionConfig::new().with_allow_unpruning_symmetric_joins(false); + let config = + SessionConfig::new().with_allow_symmetric_joins_without_pruning(false); let ctx = SessionContext::with_config(config); let tmp_dir = TempDir::new()?; let left_file_path = tmp_dir.path().join("left.csv"); @@ -2305,7 +2305,7 @@ mod tests { match df.create_physical_plan().await { Ok(_) => panic!("Expecting error."), Err(e) => { - assert_eq!(e.to_string(), "PipelineChecker\ncaused by\nError during planning: Join operation cannot operate on stream without enabling the 'allow_unpruning_symmetric_joins' configuration flag") + assert_eq!(e.to_string(), "PipelineChecker\ncaused by\nError during planning: Join operation cannot operate on stream without enabling the 'allow_symmetric_joins_without_pruning' configuration flag") } } Ok(()) diff --git a/datafusion/core/tests/sqllogictests/test_files/information_schema.slt b/datafusion/core/tests/sqllogictests/test_files/information_schema.slt index 21593d256230..3adf5585d707 100644 --- a/datafusion/core/tests/sqllogictests/test_files/information_schema.slt +++ b/datafusion/core/tests/sqllogictests/test_files/information_schema.slt @@ -149,7 +149,7 @@ datafusion.execution.target_partitions 7 datafusion.execution.time_zone +00:00 datafusion.explain.logical_plan_only false datafusion.explain.physical_plan_only false -datafusion.optimizer.allow_unpruning_symmetric_joins true +datafusion.optimizer.allow_symmetric_joins_without_pruning true datafusion.optimizer.enable_round_robin_repartition true datafusion.optimizer.filter_null_join_keys false datafusion.optimizer.hash_join_single_partition_threshold 1048576 diff --git a/datafusion/execution/src/config.rs b/datafusion/execution/src/config.rs index 13f50b6e3106..8f0094fadf50 100644 --- a/datafusion/execution/src/config.rs +++ b/datafusion/execution/src/config.rs @@ -187,8 +187,8 @@ impl SessionConfig { } /// Enables or disables the allowing unordered symmetric hash join - pub fn with_allow_unpruning_symmetric_joins(mut self, enabled: bool) -> Self { - self.options.optimizer.allow_unpruning_symmetric_joins = enabled; + pub fn with_allow_symmetric_joins_without_pruning(mut self, enabled: bool) -> Self { + self.options.optimizer.allow_symmetric_joins_without_pruning = enabled; self } diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 2c7ddae688e9..f1e5ff114261 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -35,41 +35,41 @@ Values are parsed according to the [same rules used in casts from Utf8](https:// If the value in the environment variable cannot be cast to the type of the configuration option, the default value will be used instead and a warning emitted. Environment variables are read during `SessionConfig` initialisation so they must be set beforehand and will not affect running sessions. -| key | default | description | -| --------------------------------------------------------- | ---------- | ----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | -| datafusion.catalog.create_default_catalog_and_schema | true | Whether the default catalog and schema should be created automatically. | -| datafusion.catalog.default_catalog | datafusion | The default catalog name - this impacts what SQL queries use if not specified | -| datafusion.catalog.default_schema | public | The default schema name - this impacts what SQL queries use if not specified | -| datafusion.catalog.information_schema | false | Should DataFusion provide access to `information_schema` virtual tables for displaying schema information | -| datafusion.catalog.location | NULL | Location scanned to load tables for `default` schema | -| datafusion.catalog.format | NULL | Type of `TableProvider` to use when loading `default` schema | -| datafusion.catalog.has_header | false | If the file has a header | -| datafusion.execution.batch_size | 8192 | Default batch size while creating new batches, it's especially useful for buffer-in-memory batches since creating tiny batches would result in too much metadata memory consumption | -| datafusion.execution.coalesce_batches | true | When set to true, record batches will be examined between each operator and small batches will be coalesced into larger batches. This is helpful when there are highly selective filters or joins that could produce tiny output batches. The target batch size is determined by the configuration setting | -| datafusion.execution.collect_statistics | false | Should DataFusion collect statistics after listing files | -| datafusion.execution.target_partitions | 0 | Number of partitions for query execution. Increasing partitions can increase concurrency. Defaults to the number of CPU cores on the system | -| datafusion.execution.time_zone | +00:00 | The default time zone Some functions, e.g. `EXTRACT(HOUR from SOME_TIME)`, shift the underlying datetime according to this time zone, and then extract the hour | -| datafusion.execution.parquet.enable_page_index | false | If true, uses parquet data page level metadata (Page Index) statistics to reduce the number of rows decoded. | -| datafusion.execution.parquet.pruning | true | If true, the parquet reader attempts to skip entire row groups based on the predicate in the query and the metadata (min/max values) stored in the parquet file | -| datafusion.execution.parquet.skip_metadata | true | If true, the parquet reader skip the optional embedded metadata that may be in the file Schema. This setting can help avoid schema conflicts when querying multiple parquet files with schemas containing compatible types but different metadata | -| datafusion.execution.parquet.metadata_size_hint | NULL | If specified, the parquet reader will try and fetch the last `size_hint` bytes of the parquet file optimistically. If not specified, two reads are required: One read to fetch the 8-byte parquet footer and another to fetch the metadata length encoded in the footer | -| datafusion.execution.parquet.pushdown_filters | false | If true, filter expressions are be applied during the parquet decoding operation to reduce the number of rows decoded | -| datafusion.execution.parquet.reorder_filters | false | If true, filter expressions evaluated during the parquet decoding operation will be reordered heuristically to minimize the cost of evaluation. If false, the filters are applied in the same order as written in the query | -| datafusion.optimizer.enable_round_robin_repartition | true | When set to true, the physical plan optimizer will try to add round robin repartitioning to increase parallelism to leverage more CPU cores | -| datafusion.optimizer.filter_null_join_keys | false | When set to true, the optimizer will insert filters before a join between a nullable and non-nullable column to filter out nulls on the nullable side. This filter can add additional overhead when the file format does not fully support predicate push down. | -| datafusion.optimizer.repartition_aggregations | true | Should DataFusion repartition data using the aggregate keys to execute aggregates in parallel using the provided `target_partitions` level | -| datafusion.optimizer.repartition_file_min_size | 10485760 | Minimum total files size in bytes to perform file scan repartitioning. | -| datafusion.optimizer.repartition_joins | true | Should DataFusion repartition data using the join keys to execute joins in parallel using the provided `target_partitions` level | -| datafusion.optimizer.allow_unpruning_symmetric_joins | true | Should DataFusion allow symmetric hash joins for unbounded data sources even when its inputs do not have any ordering or filtering | -| datafusion.optimizer.repartition_file_scans | true | When set to true, file groups will be repartitioned to achieve maximum parallelism. Currently supported only for Parquet format in which case multiple row groups from the same file may be read concurrently. If false then each row group is read serially, though different files may be read in parallel. | -| datafusion.optimizer.repartition_windows | true | Should DataFusion repartition data using the partitions keys to execute window functions in parallel using the provided `target_partitions` level | -| datafusion.optimizer.repartition_sorts | true | Should DataFusion execute sorts in a per-partition fashion and merge afterwards instead of coalescing first and sorting globally. With this flag is enabled, plans in the form below "SortExec: [a@0 ASC]", " CoalescePartitionsExec", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", would turn into the plan below which performs better in multithreaded environments "SortPreservingMergeExec: [a@0 ASC]", " SortExec: [a@0 ASC]", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", | -| datafusion.optimizer.skip_failed_rules | true | When set to true, the logical plan optimizer will produce warning messages if any optimization rules produce errors and then proceed to the next rule. When set to false, any rules that produce errors will cause the query to fail | -| datafusion.optimizer.max_passes | 3 | Number of times that the optimizer will attempt to optimize the plan | -| datafusion.optimizer.top_down_join_key_reordering | true | When set to true, the physical plan optimizer will run a top down process to reorder the join keys | -| datafusion.optimizer.prefer_hash_join | true | When set to true, the physical plan optimizer will prefer HashJoin over SortMergeJoin. HashJoin can work more efficiently than SortMergeJoin but consumes more memory | -| datafusion.optimizer.hash_join_single_partition_threshold | 1048576 | The maximum estimated size in bytes for one input side of a HashJoin will be collected into a single partition | -| datafusion.explain.logical_plan_only | false | When set to true, the explain statement will only print logical plans | -| datafusion.explain.physical_plan_only | false | When set to true, the explain statement will only print physical plans | -| datafusion.sql_parser.parse_float_as_decimal | false | When set to true, SQL parser will parse float as decimal type | -| datafusion.sql_parser.enable_ident_normalization | true | When set to true, SQL parser will normalize ident (convert ident to lowercase when not quoted) | +| key | default | description | +| ---------------------------------------------------------- | ---------- | ----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | +| datafusion.catalog.create_default_catalog_and_schema | true | Whether the default catalog and schema should be created automatically. | +| datafusion.catalog.default_catalog | datafusion | The default catalog name - this impacts what SQL queries use if not specified | +| datafusion.catalog.default_schema | public | The default schema name - this impacts what SQL queries use if not specified | +| datafusion.catalog.information_schema | false | Should DataFusion provide access to `information_schema` virtual tables for displaying schema information | +| datafusion.catalog.location | NULL | Location scanned to load tables for `default` schema | +| datafusion.catalog.format | NULL | Type of `TableProvider` to use when loading `default` schema | +| datafusion.catalog.has_header | false | If the file has a header | +| datafusion.execution.batch_size | 8192 | Default batch size while creating new batches, it's especially useful for buffer-in-memory batches since creating tiny batches would result in too much metadata memory consumption | +| datafusion.execution.coalesce_batches | true | When set to true, record batches will be examined between each operator and small batches will be coalesced into larger batches. This is helpful when there are highly selective filters or joins that could produce tiny output batches. The target batch size is determined by the configuration setting | +| datafusion.execution.collect_statistics | false | Should DataFusion collect statistics after listing files | +| datafusion.execution.target_partitions | 0 | Number of partitions for query execution. Increasing partitions can increase concurrency. Defaults to the number of CPU cores on the system | +| datafusion.execution.time_zone | +00:00 | The default time zone Some functions, e.g. `EXTRACT(HOUR from SOME_TIME)`, shift the underlying datetime according to this time zone, and then extract the hour | +| datafusion.execution.parquet.enable_page_index | false | If true, uses parquet data page level metadata (Page Index) statistics to reduce the number of rows decoded. | +| datafusion.execution.parquet.pruning | true | If true, the parquet reader attempts to skip entire row groups based on the predicate in the query and the metadata (min/max values) stored in the parquet file | +| datafusion.execution.parquet.skip_metadata | true | If true, the parquet reader skip the optional embedded metadata that may be in the file Schema. This setting can help avoid schema conflicts when querying multiple parquet files with schemas containing compatible types but different metadata | +| datafusion.execution.parquet.metadata_size_hint | NULL | If specified, the parquet reader will try and fetch the last `size_hint` bytes of the parquet file optimistically. If not specified, two reads are required: One read to fetch the 8-byte parquet footer and another to fetch the metadata length encoded in the footer | +| datafusion.execution.parquet.pushdown_filters | false | If true, filter expressions are be applied during the parquet decoding operation to reduce the number of rows decoded | +| datafusion.execution.parquet.reorder_filters | false | If true, filter expressions evaluated during the parquet decoding operation will be reordered heuristically to minimize the cost of evaluation. If false, the filters are applied in the same order as written in the query | +| datafusion.optimizer.enable_round_robin_repartition | true | When set to true, the physical plan optimizer will try to add round robin repartitioning to increase parallelism to leverage more CPU cores | +| datafusion.optimizer.filter_null_join_keys | false | When set to true, the optimizer will insert filters before a join between a nullable and non-nullable column to filter out nulls on the nullable side. This filter can add additional overhead when the file format does not fully support predicate push down. | +| datafusion.optimizer.repartition_aggregations | true | Should DataFusion repartition data using the aggregate keys to execute aggregates in parallel using the provided `target_partitions` level | +| datafusion.optimizer.repartition_file_min_size | 10485760 | Minimum total files size in bytes to perform file scan repartitioning. | +| datafusion.optimizer.repartition_joins | true | Should DataFusion repartition data using the join keys to execute joins in parallel using the provided `target_partitions` level | +| datafusion.optimizer.allow_symmetric_joins_without_pruning | true | Should DataFusion allow symmetric hash joins for unbounded data sources even when its inputs do not have any ordering or filtering | +| datafusion.optimizer.repartition_file_scans | true | When set to true, file groups will be repartitioned to achieve maximum parallelism. Currently supported only for Parquet format in which case multiple row groups from the same file may be read concurrently. If false then each row group is read serially, though different files may be read in parallel. | +| datafusion.optimizer.repartition_windows | true | Should DataFusion repartition data using the partitions keys to execute window functions in parallel using the provided `target_partitions` level | +| datafusion.optimizer.repartition_sorts | true | Should DataFusion execute sorts in a per-partition fashion and merge afterwards instead of coalescing first and sorting globally. With this flag is enabled, plans in the form below "SortExec: [a@0 ASC]", " CoalescePartitionsExec", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", would turn into the plan below which performs better in multithreaded environments "SortPreservingMergeExec: [a@0 ASC]", " SortExec: [a@0 ASC]", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", | +| datafusion.optimizer.skip_failed_rules | true | When set to true, the logical plan optimizer will produce warning messages if any optimization rules produce errors and then proceed to the next rule. When set to false, any rules that produce errors will cause the query to fail | +| datafusion.optimizer.max_passes | 3 | Number of times that the optimizer will attempt to optimize the plan | +| datafusion.optimizer.top_down_join_key_reordering | true | When set to true, the physical plan optimizer will run a top down process to reorder the join keys | +| datafusion.optimizer.prefer_hash_join | true | When set to true, the physical plan optimizer will prefer HashJoin over SortMergeJoin. HashJoin can work more efficiently than SortMergeJoin but consumes more memory | +| datafusion.optimizer.hash_join_single_partition_threshold | 1048576 | The maximum estimated size in bytes for one input side of a HashJoin will be collected into a single partition | +| datafusion.explain.logical_plan_only | false | When set to true, the explain statement will only print logical plans | +| datafusion.explain.physical_plan_only | false | When set to true, the explain statement will only print physical plans | +| datafusion.sql_parser.parse_float_as_decimal | false | When set to true, SQL parser will parse float as decimal type | +| datafusion.sql_parser.enable_ident_normalization | true | When set to true, SQL parser will normalize ident (convert ident to lowercase when not quoted) | From 36d450ee4a9d24a25747747884d26dea1142e108 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Fri, 24 Mar 2023 19:12:27 -0500 Subject: [PATCH 07/25] Move SHJ suitability from PipelineFixer to PipelineChecker, further SHJ code simplifications --- .../physical_optimizer/pipeline_checker.rs | 23 ++- .../src/physical_optimizer/pipeline_fixer.rs | 50 +------ .../joins/symmetric_hash_join.rs | 135 +++++++++--------- .../physical-expr/src/intervals/cp_solver.rs | 22 ++- datafusion/physical-expr/src/intervals/mod.rs | 2 +- 5 files changed, 113 insertions(+), 119 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/pipeline_checker.rs b/datafusion/core/src/physical_optimizer/pipeline_checker.rs index 0ba27338a899..730ae15c6af9 100644 --- a/datafusion/core/src/physical_optimizer/pipeline_checker.rs +++ b/datafusion/core/src/physical_optimizer/pipeline_checker.rs @@ -27,6 +27,7 @@ use crate::physical_plan::tree_node::TreeNodeRewritable; use crate::physical_plan::{with_new_children_if_necessary, ExecutionPlan}; use datafusion_common::config::OptimizerOptions; use datafusion_common::DataFusionError; +use datafusion_physical_expr::intervals::{check_support, is_datatype_supported}; use std::sync::Arc; /// The PipelineChecker rule rejects non-runnable query plans that use @@ -122,10 +123,11 @@ pub fn check_finiteness_requirements( let plan = input.plan; if let Some(exec) = plan.as_any().downcast_ref::() { if !(optimizer_options.allow_symmetric_joins_without_pruning - || exec.check_if_order_information_available()?) + || (exec.check_if_order_information_available()? && is_prunable(exec))) { - let msg = "Join operation cannot operate on stream without enabling the 'allow_symmetric_joins_without_pruning' configuration flag"; - return Err(DataFusionError::Plan(msg.to_owned())); + const MSG: &str = "Join operation cannot operate on a non-prunable stream without enabling \ + the 'allow_symmetric_joins_without_pruning' configuration flag"; + return Err(DataFusionError::Plan(MSG.to_owned())); } } let children = input.children_unbounded; @@ -138,6 +140,21 @@ pub fn check_finiteness_requirements( }) } +/// This function returns whether a given symmetric hash join is amenable to +/// data pruning. For this to be possible, it needs to have a filter where +/// all involved [`PhysicalExpr`]s, [`Operator`]s and data types support +/// interval calculations. +fn is_prunable(join: &SymmetricHashJoinExec) -> bool { + join.filter().map_or(false, |filter| { + check_support(filter.expression()) + && filter + .schema() + .fields() + .iter() + .all(|f| is_datatype_supported(f.data_type())) + }) +} + #[cfg(test)] mod sql_tests { use super::*; diff --git a/datafusion/core/src/physical_optimizer/pipeline_fixer.rs b/datafusion/core/src/physical_optimizer/pipeline_fixer.rs index a7763d90b1da..cd4c3ede7440 100644 --- a/datafusion/core/src/physical_optimizer/pipeline_fixer.rs +++ b/datafusion/core/src/physical_optimizer/pipeline_fixer.rs @@ -32,13 +32,10 @@ use crate::physical_plan::tree_node::TreeNodeRewritable; use crate::physical_plan::ExecutionPlan; use datafusion_common::DataFusionError; use datafusion_expr::logical_plan::JoinType; -use datafusion_physical_expr::expressions::{BinaryExpr, CastExpr, Column, Literal}; -use datafusion_physical_expr::intervals::{is_datatype_supported, is_operator_supported}; -use datafusion_physical_expr::PhysicalExpr; use std::sync::Arc; -/// The [PipelineFixer] rule tries to modify a given plan so that it can +/// The [`PipelineFixer`] rule tries to modify a given plan so that it can /// accommodate its infinite sources, if there are any. If this is not /// possible, the rule emits a diagnostic error message. #[derive(Default)] @@ -50,10 +47,10 @@ impl PipelineFixer { Self {} } } -/// [PipelineFixer] subrules are functions of this type. Such functions take a -/// single [PipelineStatePropagator] argument, which stores state variables -/// indicating the unboundedness status of the current [ExecutionPlan] as -/// the [PipelineFixer] rule traverses the entire plan tree. +/// [`PipelineFixer`] subrules are functions of this type. Such functions take a +/// single [`PipelineStatePropagator`] argument, which stores state variables +/// indicating the unboundedness status of the current [`ExecutionPlan`] as +/// the `PipelineFixer` rule traverses the entire plan tree. type PipelineFixerSubrule = dyn Fn(PipelineStatePropagator) -> Option>; @@ -86,36 +83,6 @@ impl PhysicalOptimizerRule for PipelineFixer { } } -/// Indicates whether interval arithmetic is supported for the given expression. -/// Currently, we do not support all [PhysicalExpr]s for interval calculations. -/// We do not support every type of [Operator]s either. Over time, this check -/// will relax as more types of [PhysicalExpr]s and [Operator]s are supported. -/// Currently, [CastExpr], [BinaryExpr], [Column] and [Literal] is supported. -fn check_support(expr: &Arc) -> bool { - let expr_any = expr.as_any(); - let expr_supported = if let Some(binary_expr) = expr_any.downcast_ref::() - { - is_operator_supported(binary_expr.op()) - } else { - expr_any.is::() || expr_any.is::() || expr_any.is::() - }; - expr_supported && expr.children().iter().all(check_support) -} - -/// This function returns whether a given hash join is replaceable by a -/// symmetric hash join. Basically, the requirement is that involved -/// [`PhysicalExpr`]s, [`Operator`]s and data types need to be supported -fn is_suitable_for_symmetric_hash_join(hash_join: &HashJoinExec) -> bool { - hash_join.filter().map_or(true, |filter| { - check_support(filter.expression()) - && filter - .schema() - .fields() - .iter() - .all(|f| is_datatype_supported(f.data_type())) - }) -} - /// This subrule checks if one can replace a hash join with a symmetric hash /// join so that the pipeline does not break due to the join operation in /// question. If possible, it makes this replacement; otherwise, it has no @@ -127,10 +94,7 @@ fn hash_join_convert_symmetric_subrule( if let Some(hash_join) = plan.as_any().downcast_ref::() { let ub_flags = input.children_unbounded; let (left_unbounded, right_unbounded) = (ub_flags[0], ub_flags[1]); - let new_plan = if left_unbounded - && right_unbounded - && is_suitable_for_symmetric_hash_join(hash_join) - { + let new_plan = if left_unbounded && right_unbounded { SymmetricHashJoinExec::try_new( hash_join.left().clone(), hash_join.right().clone(), @@ -276,9 +240,9 @@ fn apply_subrules_and_check_finiteness_requirements( #[cfg(test)] mod util_tests { - use crate::physical_optimizer::pipeline_fixer::check_support; use datafusion_expr::Operator; use datafusion_physical_expr::expressions::{BinaryExpr, Column, NegativeExpr}; + use datafusion_physical_expr::intervals::check_support; use datafusion_physical_expr::PhysicalExpr; use std::sync::Arc; diff --git a/datafusion/core/src/physical_plan/joins/symmetric_hash_join.rs b/datafusion/core/src/physical_plan/joins/symmetric_hash_join.rs index 172aaf09e711..9d04b5d9a96c 100644 --- a/datafusion/core/src/physical_plan/joins/symmetric_hash_join.rs +++ b/datafusion/core/src/physical_plan/joins/symmetric_hash_join.rs @@ -166,8 +166,8 @@ pub struct SymmetricHashJoinExec { pub(crate) filter: Option, /// How the join is performed pub(crate) join_type: JoinType, - /// Expression graph and SortedFilterExprs for interval calculations - filter_order_state: Option>>, + /// Expression graph and `SortedFilterExpr`s for interval calculations + filter_state: Option>>, /// The schema once the join is applied schema: SchemaRef, /// Shares the `RandomState` for the hashing algorithm @@ -182,14 +182,14 @@ pub struct SymmetricHashJoinExec { struct IntervalCalculatorInnerState { /// Expression graph for interval calculations - physical_expr_graph: Option, - sorted_filter_exprs: Vec>, + graph: Option, + sorted_exprs: Vec>, calculated: bool, } impl Debug for IntervalCalculatorInnerState { fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result { - write!(f, "Exprs({:?})", self.sorted_filter_exprs) + write!(f, "Exprs({:?})", self.sorted_exprs) } } @@ -281,10 +281,10 @@ impl SymmetricHashJoinExec { // Initialize the random state for the join operation: let random_state = RandomState::with_seeds(0, 0, 0, 0); - let filter_order_state = if filter.is_some() { + let filter_state = if filter.is_some() { let inner_state = IntervalCalculatorInnerState { - physical_expr_graph: None, - sorted_filter_exprs: vec![], + graph: None, + sorted_exprs: vec![], calculated: false, }; Some(Arc::new(Mutex::new(inner_state))) @@ -298,7 +298,7 @@ impl SymmetricHashJoinExec { on, filter, join_type: *join_type, - filter_order_state, + filter_state, schema: Arc::new(schema), random_state, metrics: ExecutionPlanMetricsSet::new(), @@ -475,17 +475,16 @@ impl ExecutionPlan for SymmetricHashJoinExec { partition: usize, context: Arc, ) -> Result { - let on_left = self.on.iter().map(|on| on.0.clone()).collect::>(); - let on_right = self.on.iter().map(|on| on.1.clone()).collect::>(); - // If self.filter_order_state and self.filter are both present, then calculate sorted filter expressions - // for the left and right sides and build a physical expression graph from them - let (left_sorted_filter_expr, right_sorted_filter_expr, physical_expr_graph) = - match (&self.filter_order_state, &self.filter) { + // If `filter_state` and `filter` are both present, then calculate sorted filter expressions + // for both sides, and build an expression graph if one is not already built. + let (left_sorted_filter_expr, right_sorted_filter_expr, graph) = + match (&self.filter_state, &self.filter) { (Some(interval_state), Some(filter)) => { - // Lock the mutexes of the interval state - let mut filter_order_state = interval_state.lock(); - // if this is the first partition to be invoked then we need to set up initial state - if !filter_order_state.calculated { + // Lock the mutex of the interval state: + let mut filter_state = interval_state.lock(); + // If this is the first partition to be invoked, then we need to initialize our state + // (the expression graph for pruning, sorted filter expressions etc.) + if !filter_state.calculated { // Interval calculations require each column to exhibit monotonicity // independently. However, a `PhysicalSortExpr` object defines a // lexicographical ordering, so we can only use their first elements. @@ -511,63 +510,60 @@ impl ExecutionPlan for SymmetricHashJoinExec { }) .transpose()?; - filter_order_state.sorted_filter_exprs.push(sorted_expr); + filter_state.sorted_exprs.push(sorted_expr); } - // Gather filter expressions - let filter_exprs = filter_order_state - .sorted_filter_exprs - .iter() - .filter_map(|sorted_expr| { - sorted_expr - .as_ref() - .map(|expr| expr.filter_expr().clone()) - }) + + // Collect available sorted filter expressions: + let sorted_exprs_size = filter_state.sorted_exprs.len(); + let mut sorted_exprs = filter_state + .sorted_exprs + .iter_mut() + .flatten() .collect::>(); - // Create the physical expression graph if sorted filter expressions can be created for both children: - let physical_expr_graph = if filter_order_state - .sorted_filter_exprs - .iter() - .all(Option::is_some) - { - let mut physical_expr_graph = + // Create the expression graph if we can create sorted filter expressions for both children: + filter_state.graph = if sorted_exprs.len() == sorted_exprs_size { + let mut graph = ExprIntervalGraph::try_new(filter.expression().clone())?; - // Gather node indices of converted filter expressions in `SortedFilterExpr` + + // Gather filter expressions: + let filter_exprs = sorted_exprs + .iter() + .map(|sorted_expr| sorted_expr.filter_expr().clone()) + .collect::>(); + + // Gather node indices of converted filter expressions in `SortedFilterExpr`s // using the filter columns vector: let child_node_indices = - physical_expr_graph.gather_node_indices(&filter_exprs); + graph.gather_node_indices(&filter_exprs); // Update SortedFilterExpr instances with the corresponding node indices: - filter_order_state - .sorted_filter_exprs - .iter_mut() - .map(Option::as_mut) - .map(Option::unwrap) - .zip(child_node_indices.iter()) - .for_each(|(sorted_expr, (_, index))| { - sorted_expr.set_node_index(*index); - }); - - Some(physical_expr_graph) + for (sorted_expr, (_, index)) in + sorted_exprs.iter_mut().zip(child_node_indices.iter()) + { + sorted_expr.set_node_index(*index); + } + + Some(graph) } else { None }; - // Store the calculated physical expression graph in the interval state - filter_order_state.physical_expr_graph = physical_expr_graph; - filter_order_state.calculated = true; + filter_state.calculated = true; } - // Return the sorted filter expressions for the left and right sides, along with the physical - // expression graph + // Return the sorted filter expressions for both sides along with the expression graph: ( - filter_order_state.sorted_filter_exprs[0].clone(), - filter_order_state.sorted_filter_exprs[1].clone(), - filter_order_state.physical_expr_graph.as_ref().cloned(), + filter_state.sorted_exprs[0].clone(), + filter_state.sorted_exprs[1].clone(), + filter_state.graph.as_ref().cloned(), ) } - // If self.filter_order_state or self.filter is None, then return None for all three values + // If `filter_state` or `filter` is not present, then return None for all three values: (_, _) => (None, None, None), }; + let on_left = self.on.iter().map(|on| on.0.clone()).collect::>(); + let on_right = self.on.iter().map(|on| on.1.clone()).collect::>(); + let left_side_joiner = OneSideHashJoiner::new(JoinSide::Left, on_left, self.left.schema()); let right_side_joiner = @@ -587,7 +583,7 @@ impl ExecutionPlan for SymmetricHashJoinExec { right: right_side_joiner, column_indices: self.column_indices.clone(), metrics: SymmetricHashJoinMetrics::new(partition, &self.metrics), - physical_expr_graph, + graph, left_sorted_filter_expr, right_sorted_filter_expr, null_equals_null: self.null_equals_null, @@ -615,8 +611,8 @@ struct SymmetricHashJoinStream { right: OneSideHashJoiner, /// Information of index and left / right placement of columns column_indices: Vec, - // Range pruner. - physical_expr_graph: Option, + // Expression graph for range pruning. + graph: Option, // Left globally sorted filter expr left_sorted_filter_expr: Option, // Right globally sorted filter expr @@ -763,8 +759,7 @@ fn calculate_filter_expr_intervals( update_filter_expr_interval( &probe_batch.slice(probe_batch.num_rows() - 1, 1), probe_sorted_filter_expr, - )?; - Ok(()) + ) } /// This is a subroutine of the function [`calculate_filter_expr_intervals`]. @@ -1235,7 +1230,7 @@ impl OneSideHashJoiner { /// * `join_type` - The type of join (e.g. inner, left, right, etc.). /// * `column_indices` - A vector of column indices that specifies which columns from the /// build side should be included in the output. - /// * `physical_expr_graph` - A mutable reference to the physical expression graph. + /// * `graph` - A mutable reference to the physical expression graph. /// /// # Returns /// @@ -1245,7 +1240,7 @@ impl OneSideHashJoiner { &mut self, build_side_sorted_filter_expr: &mut SortedFilterExpr, probe_side_sorted_filter_expr: &mut SortedFilterExpr, - physical_expr_graph: &mut ExprIntervalGraph, + graph: &mut ExprIntervalGraph, ) -> Result { // Return early if the input buffer is empty: if self.input_buffer.num_rows() == 0 { @@ -1261,7 +1256,7 @@ impl OneSideHashJoiner { filter_intervals.push((expr.node_index(), expr.interval().clone())) } // Update the physical expression graph using the join filter intervals: - physical_expr_graph.update_ranges(&mut filter_intervals)?; + graph.update_ranges(&mut filter_intervals)?; // Extract the new join filter interval for the build side: let calculated_build_side_interval = filter_intervals.remove(0).1; // If the intervals have not changed, return early without pruning: @@ -1442,11 +1437,11 @@ impl SymmetricHashJoinStream { let anti_result = if let ( Some(build_side_sorted_filter_expr), Some(probe_side_sorted_filter_expr), - Some(physical_expr_graph), + Some(graph), ) = ( build_side_sorted_filter_expr.as_mut(), probe_side_sorted_filter_expr.as_mut(), - self.physical_expr_graph.as_mut(), + self.graph.as_mut(), ) { // Calculate filter intervals: calculate_filter_expr_intervals( @@ -1459,7 +1454,7 @@ impl SymmetricHashJoinStream { .calculate_prune_length_with_probe_batch( build_side_sorted_filter_expr, probe_side_sorted_filter_expr, - physical_expr_graph, + graph, )?; if prune_length > 0 { @@ -2305,7 +2300,7 @@ mod tests { match df.create_physical_plan().await { Ok(_) => panic!("Expecting error."), Err(e) => { - assert_eq!(e.to_string(), "PipelineChecker\ncaused by\nError during planning: Join operation cannot operate on stream without enabling the 'allow_symmetric_joins_without_pruning' configuration flag") + assert_eq!(e.to_string(), "PipelineChecker\ncaused by\nError during planning: Join operation cannot operate on a non-prunable stream without enabling the 'allow_symmetric_joins_without_pruning' configuration flag") } } Ok(()) diff --git a/datafusion/physical-expr/src/intervals/cp_solver.rs b/datafusion/physical-expr/src/intervals/cp_solver.rs index 66367001c642..a54bcb2f35f0 100644 --- a/datafusion/physical-expr/src/intervals/cp_solver.rs +++ b/datafusion/physical-expr/src/intervals/cp_solver.rs @@ -29,8 +29,10 @@ use petgraph::stable_graph::{DefaultIx, StableGraph}; use petgraph::visit::{Bfs, Dfs, DfsPostOrder, EdgeRef}; use petgraph::Outgoing; -use crate::expressions::Literal; -use crate::intervals::interval_aritmetic::{apply_operator, Interval}; +use crate::expressions::{BinaryExpr, CastExpr, Column, Literal}; +use crate::intervals::interval_aritmetic::{ + apply_operator, is_operator_supported, Interval, +}; use crate::utils::{build_dag, ExprTreeNode}; use crate::PhysicalExpr; @@ -521,6 +523,22 @@ impl ExprIntervalGraph { } } +/// Indicates whether interval arithmetic is supported for the given expression. +/// Currently, we do not support all [`PhysicalExpr`]s for interval calculations. +/// We do not support every type of [`Operator`]s either. Over time, this check +/// will relax as more types of `PhysicalExpr`s and `Operator`s are supported. +/// Currently, [`CastExpr`], [`BinaryExpr`], [`Column`] and [`Literal`] are supported. +pub fn check_support(expr: &Arc) -> bool { + let expr_any = expr.as_any(); + let expr_supported = if let Some(binary_expr) = expr_any.downcast_ref::() + { + is_operator_supported(binary_expr.op()) + } else { + expr_any.is::() || expr_any.is::() || expr_any.is::() + }; + expr_supported && expr.children().iter().all(check_support) +} + #[cfg(test)] mod tests { use super::*; diff --git a/datafusion/physical-expr/src/intervals/mod.rs b/datafusion/physical-expr/src/intervals/mod.rs index 45616534cb17..9883ba15b2e7 100644 --- a/datafusion/physical-expr/src/intervals/mod.rs +++ b/datafusion/physical-expr/src/intervals/mod.rs @@ -22,5 +22,5 @@ pub mod cp_solver; pub mod interval_aritmetic; pub mod test_utils; -pub use cp_solver::ExprIntervalGraph; +pub use cp_solver::{check_support, ExprIntervalGraph}; pub use interval_aritmetic::*; From 119a87031d6236d28b82cab67fd7a2225b951e95 Mon Sep 17 00:00:00 2001 From: metesynnada <100111937+metesynnada@users.noreply.github.com> Date: Tue, 28 Mar 2023 16:15:57 +0300 Subject: [PATCH 08/25] Added logging on tests and ensure timeout --- datafusion/core/tests/fifo.rs | 193 ++++++++++++++++++---------------- 1 file changed, 102 insertions(+), 91 deletions(-) diff --git a/datafusion/core/tests/fifo.rs b/datafusion/core/tests/fifo.rs index d8b66b0a29cd..5bf0bae6afa5 100644 --- a/datafusion/core/tests/fifo.rs +++ b/datafusion/core/tests/fifo.rs @@ -143,11 +143,11 @@ mod unix_test { // This test provides a relatively realistic end-to-end scenario where // we swap join sides to accommodate a FIFO source. #[rstest] - #[timeout(std::time::Duration::from_secs(30))] - #[tokio::test(flavor = "multi_thread", worker_threads = 5)] + #[tokio::test(flavor = "multi_thread", worker_threads = 8)] async fn unbounded_file_with_swapped_join( #[values(true, false)] unbounded_file: bool, ) -> Result<()> { + tokio::time::timeout(Duration::from_secs(45), async { // To make unbounded deterministic let waiting = Arc::new(Mutex::new(unbounded_file)); let waiting_thread = waiting.clone(); @@ -166,8 +166,9 @@ mod unix_test { let fifo_writer = thread::spawn(move || { let first_file = OpenOptions::new() .write(true) - .open(fifo_path_thread) + .open(fifo_path_thread.clone()) .unwrap(); + log::debug!("File at {:?} opened.", fifo_path_thread); // Reference time to use when deciding to fail the test let execution_start = Instant::now(); // Execution can calculated at least one RecordBatch after the number of @@ -201,6 +202,7 @@ mod unix_test { write_to_fifo(&first_file, &line, execution_start, broken_pipe_timeout) .unwrap(); } + log::debug!("File at {:?} finished.", fifo_path_thread); }); // Collects operations from both writer and executor. let result_collector = thread::spawn(move || { @@ -223,6 +225,8 @@ mod unix_test { drop(tx); let result = result_collector.join().unwrap(); assert_eq!(interleave(&result), unbounded_file); + Ok::<(), DataFusionError>(()) + }).await.expect("Timeout test.")?; Ok(()) } @@ -236,101 +240,108 @@ mod unix_test { // This test provides a relatively realistic end-to-end scenario where // we change the join into a [SymmetricHashJoin] to accommodate two // unbounded (FIFO) sources. - #[rstest] - #[timeout(std::time::Duration::from_secs(30))] - #[tokio::test(flavor = "multi_thread")] + #[tokio::test(flavor = "multi_thread", worker_threads = 8)] async fn unbounded_file_with_symmetric_join() -> Result<()> { - // To make unbounded deterministic - let waiting = Arc::new(Mutex::new(true)); - let thread_bools = vec![waiting.clone(), waiting.clone()]; - // Create a new temporary FIFO file - let tmp_dir = TempDir::new()?; - let file_names = vec!["first_fifo.csv", "second_fifo.csv"]; - // The sender endpoint can be copied - let (threads, file_paths): (Vec>, Vec) = file_names - .iter() - .zip(thread_bools.iter()) - .map(|(file_name, lock)| { - let waiting_thread = lock.clone(); - let fifo_path = create_fifo_file(&tmp_dir, file_name).unwrap(); - let return_path = fifo_path.clone(); - // Timeout for a long period of BrokenPipe error - let broken_pipe_timeout = Duration::from_secs(5); - // Spawn a new thread to write to the FIFO file - let fifo_writer = thread::spawn(move || { - let mut rng = StdRng::seed_from_u64(42); - let file = OpenOptions::new() - .write(true) - .open(fifo_path.clone()) - .unwrap(); - // Reference time to use when deciding to fail the test - let execution_start = Instant::now(); - // Join filter - let a1_iter = (0..TEST_DATA_SIZE).map(|x| { - if rng.gen_range(0.0..1.0) < 0.3 { - x - 1 - } else { - x + tokio::time::timeout(Duration::from_secs(45), async { + // To make unbounded deterministic + let waiting = Arc::new(Mutex::new(true)); + let thread_bools = vec![waiting.clone(), waiting.clone()]; + // Create a new temporary FIFO file + let tmp_dir = TempDir::new()?; + let file_names = vec!["first_fifo.csv", "second_fifo.csv"]; + // The sender endpoint can be copied + let (threads, file_paths): (Vec>, Vec) = file_names + .iter() + .zip(thread_bools.iter()) + .map(|(file_name, lock)| { + let waiting_thread = lock.clone(); + let fifo_path = create_fifo_file(&tmp_dir, file_name).unwrap(); + let return_path = fifo_path.clone(); + // Timeout for a long period of BrokenPipe error + let broken_pipe_timeout = Duration::from_secs(5); + // Spawn a new thread to write to the FIFO file + let fifo_writer = thread::spawn(move || { + let mut rng = StdRng::seed_from_u64(42); + let file = OpenOptions::new() + .write(true) + .open(fifo_path.clone()) + .unwrap(); + log::debug!("File at {:?} opened.", fifo_path); + // Reference time to use when deciding to fail the test + let execution_start = Instant::now(); + // Join filter + let a1_iter = (0..TEST_DATA_SIZE).map(|x| { + if rng.gen_range(0.0..1.0) < 0.3 { + x - 1 + } else { + x + } + }); + // Join key + let a2_iter = (0..TEST_DATA_SIZE).map(|x| x % 10); + for (cnt, (a1, a2)) in a1_iter.zip(a2_iter).enumerate() { + // Wait a reading sign for unbounded execution + // After first batch FIFO reading, we will wait for a batch created. + while *waiting_thread.lock().unwrap() && TEST_BATCH_SIZE + 1 < cnt + { + log::debug!("Waiting."); + thread::sleep(Duration::from_millis(200)); + } + let line = format!("{a1},{a2}\n").to_owned(); + write_to_fifo(&file, &line, execution_start, broken_pipe_timeout) + .unwrap(); } + log::debug!("File at {:?} finished.", fifo_path); }); - // Join key - let a2_iter = (0..TEST_DATA_SIZE).map(|x| x % 10); - for (cnt, (a1, a2)) in a1_iter.zip(a2_iter).enumerate() { - // Wait a reading sign for unbounded execution - // After first batch FIFO reading, we will wait for a batch created. - while *waiting_thread.lock().unwrap() && TEST_BATCH_SIZE + 1 < cnt - { - thread::sleep(Duration::from_millis(200)); - } - let line = format!("{a1},{a2}\n").to_owned(); - write_to_fifo(&file, &line, execution_start, broken_pipe_timeout) - .unwrap(); - } - }); - (fifo_writer, return_path) - }) - .unzip(); - let config = SessionConfig::new() - .with_batch_size(TEST_BATCH_SIZE) - .set_bool("datafusion.execution.coalesce_batches", false) - .with_target_partitions(1); - let ctx = SessionContext::with_config(config); - test_create_unbounded_sorted_file(&ctx, file_paths[0].clone(), "left").await?; - test_create_unbounded_sorted_file(&ctx, file_paths[1].clone(), "right").await?; - // Execute the query - let df = ctx.sql("SELECT t1.a1, t1.a2, t2.a1, t2.a2 FROM left as t1 FULL JOIN right as t2 ON t1.a2 = t2.a2 AND t1.a1 > t2.a1 + 3 AND t1.a1 < t2.a1 + 10").await?; - let mut stream = df.execute_stream().await?; - let mut operations = vec![]; - while let Some(Ok(batch)) = stream.next().await { - *waiting.lock().unwrap() = false; - let op = if batch.column(0).null_count() > 0 { - JoinOperation::LeftUnmatched - } else if batch.column(2).null_count() > 0 { - JoinOperation::RightUnmatched - } else { - JoinOperation::Equal - }; - operations.push(op); - } - - // The SymmetricHashJoin executor produces FULL join results at every - // pruning, which happens before it reaches the end of input and more - // than once. In this test, we feed partially joinable data to both - // sides in order to ensure that both left/right unmatched results are - // generated more than once during the test. - assert!( - operations - .iter() - .filter(|&n| JoinOperation::RightUnmatched.eq(n)) - .count() - > 1 - && operations + (fifo_writer, return_path) + }) + .unzip(); + let config = SessionConfig::new() + .with_batch_size(TEST_BATCH_SIZE) + .set_bool("datafusion.execution.coalesce_batches", false) + .with_target_partitions(1); + let ctx = SessionContext::with_config(config); + test_create_unbounded_sorted_file(&ctx, file_paths[0].clone(), "left").await?; + test_create_unbounded_sorted_file(&ctx, file_paths[1].clone(), "right").await?; + // Execute the query + let df = ctx.sql("SELECT t1.a1, t1.a2, t2.a1, t2.a2 FROM left as t1 FULL JOIN right as t2 ON t1.a2 = t2.a2 AND t1.a1 > t2.a1 + 3 AND t1.a1 < t2.a1 + 10").await?; + let mut stream = df.execute_stream().await?; + let mut operations = vec![]; + while let Some(Ok(batch)) = stream.next().await { + log::debug!("Test gets the batch"); + *waiting.lock().unwrap() = false; + let op = if batch.column(0).null_count() > 0 { + log::debug!("Test gets the LeftUnmatched"); + JoinOperation::LeftUnmatched + } else if batch.column(2).null_count() > 0 { + log::debug!("Test gets the RightUnmatched"); + JoinOperation::RightUnmatched + } else { + log::debug!("Test gets the Equal"); + JoinOperation::Equal + }; + operations.push(op); + } + threads.into_iter().for_each(|j| j.join().unwrap()); + // The SymmetricHashJoin executor produces FULL join results at every + // pruning, which happens before it reaches the end of input and more + // than once. In this test, we feed partially joinable data to both + // sides in order to ensure that both left/right unmatched results are + // generated more than once during the test. + assert!( + operations + .iter() + .filter(|&n| JoinOperation::RightUnmatched.eq(n)) + .count() + > 1 + && operations .iter() .filter(|&n| JoinOperation::LeftUnmatched.eq(n)) .count() > 1 - ); - threads.into_iter().for_each(|j| j.join().unwrap()); + ); + Ok::<(), DataFusionError>(()) + }).await.expect("Timeout test.")?; Ok(()) } } From a83a28402a79eddbc8606621d0d2157dc5989f24 Mon Sep 17 00:00:00 2001 From: metesynnada <100111937+metesynnada@users.noreply.github.com> Date: Wed, 29 Mar 2023 10:31:46 +0300 Subject: [PATCH 09/25] Robust fifo writing in case of slow executions --- datafusion/core/tests/fifo.rs | 24 ++++++++---------------- 1 file changed, 8 insertions(+), 16 deletions(-) diff --git a/datafusion/core/tests/fifo.rs b/datafusion/core/tests/fifo.rs index 5bf0bae6afa5..03a7ca37f269 100644 --- a/datafusion/core/tests/fifo.rs +++ b/datafusion/core/tests/fifo.rs @@ -33,8 +33,6 @@ mod unix_test { use itertools::enumerate; use nix::sys::stat; use nix::unistd; - use rand::rngs::StdRng; - use rand::{Rng, SeedableRng}; use rstest::*; use std::fs::{File, OpenOptions}; use std::io::Write; @@ -73,20 +71,21 @@ mod unix_test { line: &str, ref_time: Instant, broken_pipe_timeout: Duration, - ) -> Result { + ) -> Result<()> { // We need to handle broken pipe error until the reader is ready. This // is why we use a timeout to limit the wait duration for the reader. // If the error is different than broken pipe, we fail immediately. - file.write(line.as_bytes()).or_else(|e| { + while let Err(e) = file.write_all(line.as_bytes()) { if e.raw_os_error().unwrap() == 32 { let interval = Instant::now().duration_since(ref_time); if interval < broken_pipe_timeout { thread::sleep(Duration::from_millis(100)); - return Ok(0); + continue; } } - Err(DataFusionError::Execution(e.to_string())) - }) + return Err(DataFusionError::Execution(e.to_string())); + } + Ok(()) } async fn create_ctx( @@ -258,10 +257,9 @@ mod unix_test { let fifo_path = create_fifo_file(&tmp_dir, file_name).unwrap(); let return_path = fifo_path.clone(); // Timeout for a long period of BrokenPipe error - let broken_pipe_timeout = Duration::from_secs(5); + let broken_pipe_timeout = Duration::from_secs(45); // Spawn a new thread to write to the FIFO file let fifo_writer = thread::spawn(move || { - let mut rng = StdRng::seed_from_u64(42); let file = OpenOptions::new() .write(true) .open(fifo_path.clone()) @@ -270,13 +268,7 @@ mod unix_test { // Reference time to use when deciding to fail the test let execution_start = Instant::now(); // Join filter - let a1_iter = (0..TEST_DATA_SIZE).map(|x| { - if rng.gen_range(0.0..1.0) < 0.3 { - x - 1 - } else { - x - } - }); + let a1_iter = 0..TEST_DATA_SIZE; // Join key let a2_iter = (0..TEST_DATA_SIZE).map(|x| x % 10); for (cnt, (a1, a2)) in a1_iter.zip(a2_iter).enumerate() { From 87990007e93a2d0261c0e5caccb897751091e9c8 Mon Sep 17 00:00:00 2001 From: metesynnada <100111937+metesynnada@users.noreply.github.com> Date: Wed, 29 Mar 2023 13:00:26 +0300 Subject: [PATCH 10/25] Update fifo.rs --- datafusion/core/tests/fifo.rs | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/datafusion/core/tests/fifo.rs b/datafusion/core/tests/fifo.rs index 03a7ca37f269..75659864051e 100644 --- a/datafusion/core/tests/fifo.rs +++ b/datafusion/core/tests/fifo.rs @@ -241,7 +241,7 @@ mod unix_test { // unbounded (FIFO) sources. #[tokio::test(flavor = "multi_thread", worker_threads = 8)] async fn unbounded_file_with_symmetric_join() -> Result<()> { - tokio::time::timeout(Duration::from_secs(45), async { + let result = tokio::time::timeout(Duration::from_secs(360), async { // To make unbounded deterministic let waiting = Arc::new(Mutex::new(true)); let thread_bools = vec![waiting.clone(), waiting.clone()]; @@ -283,6 +283,7 @@ mod unix_test { write_to_fifo(&file, &line, execution_start, broken_pipe_timeout) .unwrap(); } + drop(file); log::debug!("File at {:?} finished.", fifo_path); }); (fifo_writer, return_path) @@ -333,7 +334,8 @@ mod unix_test { > 1 ); Ok::<(), DataFusionError>(()) - }).await.expect("Timeout test.")?; - Ok(()) + }).await; + assert!(result.is_ok(), "Test did not timeout as expected."); + result.unwrap() } } From 02bd036d45611e087473d320710425f527be5848 Mon Sep 17 00:00:00 2001 From: metesynnada <100111937+metesynnada@users.noreply.github.com> Date: Wed, 29 Mar 2023 13:51:40 +0300 Subject: [PATCH 11/25] Update fifo.rs --- datafusion/core/tests/fifo.rs | 2 ++ 1 file changed, 2 insertions(+) diff --git a/datafusion/core/tests/fifo.rs b/datafusion/core/tests/fifo.rs index 75659864051e..010dcdef5bb0 100644 --- a/datafusion/core/tests/fifo.rs +++ b/datafusion/core/tests/fifo.rs @@ -44,6 +44,7 @@ mod unix_test { use std::thread; use std::thread::JoinHandle; use std::time::{Duration, Instant}; + use arrow::util::pretty::print_batches; use tempfile::TempDir; // ! For the sake of the test, do not alter the numbers. ! @@ -315,6 +316,7 @@ mod unix_test { }; operations.push(op); } + log::debug!("Stream is finished"); threads.into_iter().for_each(|j| j.join().unwrap()); // The SymmetricHashJoin executor produces FULL join results at every // pruning, which happens before it reaches the end of input and more From 05497ef1f8141da612e4eb1aefa750e20843b5f2 Mon Sep 17 00:00:00 2001 From: metesynnada <100111937+metesynnada@users.noreply.github.com> Date: Wed, 29 Mar 2023 14:36:33 +0300 Subject: [PATCH 12/25] Update fifo.rs --- datafusion/core/tests/fifo.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/datafusion/core/tests/fifo.rs b/datafusion/core/tests/fifo.rs index 010dcdef5bb0..6e8b8e63d57c 100644 --- a/datafusion/core/tests/fifo.rs +++ b/datafusion/core/tests/fifo.rs @@ -44,7 +44,6 @@ mod unix_test { use std::thread; use std::thread::JoinHandle; use std::time::{Duration, Instant}; - use arrow::util::pretty::print_batches; use tempfile::TempDir; // ! For the sake of the test, do not alter the numbers. ! From 311a8910db337576b6584a8b3ec132cb84b70d45 Mon Sep 17 00:00:00 2001 From: metesynnada <100111937+metesynnada@users.noreply.github.com> Date: Wed, 29 Mar 2023 15:34:53 +0300 Subject: [PATCH 13/25] Update fifo.rs --- datafusion/core/tests/fifo.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/core/tests/fifo.rs b/datafusion/core/tests/fifo.rs index 6e8b8e63d57c..81c537920f48 100644 --- a/datafusion/core/tests/fifo.rs +++ b/datafusion/core/tests/fifo.rs @@ -239,7 +239,7 @@ mod unix_test { // This test provides a relatively realistic end-to-end scenario where // we change the join into a [SymmetricHashJoin] to accommodate two // unbounded (FIFO) sources. - #[tokio::test(flavor = "multi_thread", worker_threads = 8)] + #[tokio::test(flavor = "multi_thread", worker_threads = 12)] async fn unbounded_file_with_symmetric_join() -> Result<()> { let result = tokio::time::timeout(Duration::from_secs(360), async { // To make unbounded deterministic From b051efbf030716f3a66537014e4b1ae750568d09 Mon Sep 17 00:00:00 2001 From: metesynnada <100111937+metesynnada@users.noreply.github.com> Date: Wed, 29 Mar 2023 17:14:23 +0300 Subject: [PATCH 14/25] Get rid of locks --- datafusion/core/tests/fifo.rs | 31 ++++++++++++++++--------------- 1 file changed, 16 insertions(+), 15 deletions(-) diff --git a/datafusion/core/tests/fifo.rs b/datafusion/core/tests/fifo.rs index 81c537920f48..65a2c9795996 100644 --- a/datafusion/core/tests/fifo.rs +++ b/datafusion/core/tests/fifo.rs @@ -38,6 +38,7 @@ mod unix_test { use std::io::Write; use std::path::Path; use std::path::PathBuf; + use std::sync::atomic::{AtomicBool, Ordering}; use std::sync::mpsc; use std::sync::mpsc::{Receiver, Sender}; use std::sync::{Arc, Mutex}; @@ -146,7 +147,7 @@ mod unix_test { async fn unbounded_file_with_swapped_join( #[values(true, false)] unbounded_file: bool, ) -> Result<()> { - tokio::time::timeout(Duration::from_secs(45), async { + let result = tokio::time::timeout(Duration::from_secs(45), async { // To make unbounded deterministic let waiting = Arc::new(Mutex::new(unbounded_file)); let waiting_thread = waiting.clone(); @@ -225,8 +226,9 @@ mod unix_test { let result = result_collector.join().unwrap(); assert_eq!(interleave(&result), unbounded_file); Ok::<(), DataFusionError>(()) - }).await.expect("Timeout test.")?; - Ok(()) + }).await; + assert!(result.is_ok(), "Test did not timeout as expected."); + result.unwrap() } #[derive(Debug, PartialEq)] @@ -239,11 +241,11 @@ mod unix_test { // This test provides a relatively realistic end-to-end scenario where // we change the join into a [SymmetricHashJoin] to accommodate two // unbounded (FIFO) sources. - #[tokio::test(flavor = "multi_thread", worker_threads = 12)] + #[tokio::test(flavor = "multi_thread", worker_threads = 8)] async fn unbounded_file_with_symmetric_join() -> Result<()> { let result = tokio::time::timeout(Duration::from_secs(360), async { // To make unbounded deterministic - let waiting = Arc::new(Mutex::new(true)); + let waiting = Arc::new(AtomicBool::new(true)); let thread_bools = vec![waiting.clone(), waiting.clone()]; // Create a new temporary FIFO file let tmp_dir = TempDir::new()?; @@ -252,8 +254,8 @@ mod unix_test { let (threads, file_paths): (Vec>, Vec) = file_names .iter() .zip(thread_bools.iter()) - .map(|(file_name, lock)| { - let waiting_thread = lock.clone(); + .map(|(file_name, atomic_bool)| { + let waiting_thread = atomic_bool.clone(); let fifo_path = create_fifo_file(&tmp_dir, file_name).unwrap(); let return_path = fifo_path.clone(); // Timeout for a long period of BrokenPipe error @@ -271,18 +273,18 @@ mod unix_test { let a1_iter = 0..TEST_DATA_SIZE; // Join key let a2_iter = (0..TEST_DATA_SIZE).map(|x| x % 10); - for (cnt, (a1, a2)) in a1_iter.zip(a2_iter).enumerate() { + for (a1, a2) in a1_iter.zip(a2_iter) { // Wait a reading sign for unbounded execution // After first batch FIFO reading, we will wait for a batch created. - while *waiting_thread.lock().unwrap() && TEST_BATCH_SIZE + 1 < cnt - { - log::debug!("Waiting."); - thread::sleep(Duration::from_millis(200)); - } let line = format!("{a1},{a2}\n").to_owned(); write_to_fifo(&file, &line, execution_start, broken_pipe_timeout) .unwrap(); } + while waiting_thread.load(Ordering::SeqCst) + { + log::debug!("Waiting."); + thread::sleep(Duration::from_millis(200)); + } drop(file); log::debug!("File at {:?} finished.", fifo_path); }); @@ -301,8 +303,7 @@ mod unix_test { let mut stream = df.execute_stream().await?; let mut operations = vec![]; while let Some(Ok(batch)) = stream.next().await { - log::debug!("Test gets the batch"); - *waiting.lock().unwrap() = false; + waiting.store(false, Ordering::SeqCst); let op = if batch.column(0).null_count() > 0 { log::debug!("Test gets the LeftUnmatched"); JoinOperation::LeftUnmatched From c0ecbe4b33b903041cc8d4ecef5c6d677b984930 Mon Sep 17 00:00:00 2001 From: metesynnada <100111937+metesynnada@users.noreply.github.com> Date: Thu, 30 Mar 2023 13:06:58 +0300 Subject: [PATCH 15/25] Try exact one batch size --- datafusion/core/tests/fifo.rs | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/datafusion/core/tests/fifo.rs b/datafusion/core/tests/fifo.rs index 65a2c9795996..855a00bfe32c 100644 --- a/datafusion/core/tests/fifo.rs +++ b/datafusion/core/tests/fifo.rs @@ -82,6 +82,8 @@ mod unix_test { if interval < broken_pipe_timeout { thread::sleep(Duration::from_millis(100)); continue; + } else { + return Err(DataFusionError::Execution(e.to_string())); } } return Err(DataFusionError::Execution(e.to_string())); @@ -273,18 +275,18 @@ mod unix_test { let a1_iter = 0..TEST_DATA_SIZE; // Join key let a2_iter = (0..TEST_DATA_SIZE).map(|x| x % 10); - for (a1, a2) in a1_iter.zip(a2_iter) { + for (cnt, (a1, a2)) in a1_iter.zip(a2_iter).enumerate() { // Wait a reading sign for unbounded execution // After first batch FIFO reading, we will wait for a batch created. + while waiting_thread.load(Ordering::SeqCst) && TEST_BATCH_SIZE + 1 < cnt + { + log::debug!("Waiting."); + thread::sleep(Duration::from_millis(200)); + } let line = format!("{a1},{a2}\n").to_owned(); write_to_fifo(&file, &line, execution_start, broken_pipe_timeout) .unwrap(); } - while waiting_thread.load(Ordering::SeqCst) - { - log::debug!("Waiting."); - thread::sleep(Duration::from_millis(200)); - } drop(file); log::debug!("File at {:?} finished.", fifo_path); }); From e6ab62119e5b6eb48a201b4ded03bc639e3fb17b Mon Sep 17 00:00:00 2001 From: metesynnada <100111937+metesynnada@users.noreply.github.com> Date: Thu, 30 Mar 2023 14:51:58 +0300 Subject: [PATCH 16/25] Update fifo.rs --- datafusion/core/tests/fifo.rs | 21 ++++++++------------- 1 file changed, 8 insertions(+), 13 deletions(-) diff --git a/datafusion/core/tests/fifo.rs b/datafusion/core/tests/fifo.rs index 855a00bfe32c..5527ca087277 100644 --- a/datafusion/core/tests/fifo.rs +++ b/datafusion/core/tests/fifo.rs @@ -268,7 +268,6 @@ mod unix_test { .write(true) .open(fifo_path.clone()) .unwrap(); - log::debug!("File at {:?} opened.", fifo_path); // Reference time to use when deciding to fail the test let execution_start = Instant::now(); // Join filter @@ -277,10 +276,9 @@ mod unix_test { let a2_iter = (0..TEST_DATA_SIZE).map(|x| x % 10); for (cnt, (a1, a2)) in a1_iter.zip(a2_iter).enumerate() { // Wait a reading sign for unbounded execution - // After first batch FIFO reading, we will wait for a batch created. - while waiting_thread.load(Ordering::SeqCst) && TEST_BATCH_SIZE + 1 < cnt + // After first 2 batch FIFO reading in both sides, we will wait for a batch created. + while waiting_thread.load(Ordering::SeqCst) && (TEST_BATCH_SIZE * 2) + 1 < cnt { - log::debug!("Waiting."); thread::sleep(Duration::from_millis(200)); } let line = format!("{a1},{a2}\n").to_owned(); @@ -288,7 +286,6 @@ mod unix_test { .unwrap(); } drop(file); - log::debug!("File at {:?} finished.", fifo_path); }); (fifo_writer, return_path) }) @@ -306,19 +303,17 @@ mod unix_test { let mut operations = vec![]; while let Some(Ok(batch)) = stream.next().await { waiting.store(false, Ordering::SeqCst); - let op = if batch.column(0).null_count() > 0 { - log::debug!("Test gets the LeftUnmatched"); - JoinOperation::LeftUnmatched - } else if batch.column(2).null_count() > 0 { - log::debug!("Test gets the RightUnmatched"); + let left_unmatched = batch.column(2).null_count(); + let right_unmatched = batch.column(0).null_count(); + let op = if left_unmatched == 0 && right_unmatched == 0 { + JoinOperation::Equal + } else if right_unmatched > left_unmatched { JoinOperation::RightUnmatched } else { - log::debug!("Test gets the Equal"); - JoinOperation::Equal + JoinOperation::LeftUnmatched }; operations.push(op); } - log::debug!("Stream is finished"); threads.into_iter().for_each(|j| j.join().unwrap()); // The SymmetricHashJoin executor produces FULL join results at every // pruning, which happens before it reaches the end of input and more From 7cd6b1ee53a8b4390bacc17ccc4fd1bc4de6cc3d Mon Sep 17 00:00:00 2001 From: metesynnada <100111937+metesynnada@users.noreply.github.com> Date: Thu, 30 Mar 2023 17:09:55 +0300 Subject: [PATCH 17/25] Update fifo.rs --- datafusion/core/tests/fifo.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/core/tests/fifo.rs b/datafusion/core/tests/fifo.rs index 5527ca087277..9a2bc0b16fcc 100644 --- a/datafusion/core/tests/fifo.rs +++ b/datafusion/core/tests/fifo.rs @@ -279,7 +279,7 @@ mod unix_test { // After first 2 batch FIFO reading in both sides, we will wait for a batch created. while waiting_thread.load(Ordering::SeqCst) && (TEST_BATCH_SIZE * 2) + 1 < cnt { - thread::sleep(Duration::from_millis(200)); + thread::sleep(Duration::from_millis(100)); } let line = format!("{a1},{a2}\n").to_owned(); write_to_fifo(&file, &line, execution_start, broken_pipe_timeout) From 9fa0c714548f38cdaac96918465394997969bcbf Mon Sep 17 00:00:00 2001 From: metesynnada <100111937+metesynnada@users.noreply.github.com> Date: Thu, 30 Mar 2023 18:11:14 +0300 Subject: [PATCH 18/25] Update fifo.rs --- datafusion/core/tests/fifo.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/core/tests/fifo.rs b/datafusion/core/tests/fifo.rs index 9a2bc0b16fcc..4c3b262a87a2 100644 --- a/datafusion/core/tests/fifo.rs +++ b/datafusion/core/tests/fifo.rs @@ -279,7 +279,7 @@ mod unix_test { // After first 2 batch FIFO reading in both sides, we will wait for a batch created. while waiting_thread.load(Ordering::SeqCst) && (TEST_BATCH_SIZE * 2) + 1 < cnt { - thread::sleep(Duration::from_millis(100)); + thread::sleep(Duration::from_millis(150)); } let line = format!("{a1},{a2}\n").to_owned(); write_to_fifo(&file, &line, execution_start, broken_pipe_timeout) From c9bece5f1c5f7d72b81c69356b7c70499a4c3c5e Mon Sep 17 00:00:00 2001 From: metesynnada <100111937+metesynnada@users.noreply.github.com> Date: Thu, 30 Mar 2023 20:26:31 +0300 Subject: [PATCH 19/25] Ignore FIFO test --- datafusion-cli/Cargo.lock | 38 +- .../joins/symmetric_hash_join.rs | 3 - datafusion/core/tests/fifo.rs | 446 ++++++++++-------- 3 files changed, 256 insertions(+), 231 deletions(-) diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index e8f97824b6c7..d5a54033f623 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -1093,9 +1093,9 @@ dependencies = [ [[package]] name = "futures" -version = "0.3.27" +version = "0.3.28" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "531ac96c6ff5fd7c62263c5e3c67a603af4fcaee2e1a0ae5565ba3a11e69e549" +checksum = "23342abe12aba583913b2e62f22225ff9c950774065e4bfb61a19cd9770fec40" dependencies = [ "futures-channel", "futures-core", @@ -1108,9 +1108,9 @@ dependencies = [ [[package]] name = "futures-channel" -version = "0.3.27" +version = "0.3.28" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "164713a5a0dcc3e7b4b1ed7d3b433cabc18025386f9339346e8daf15963cf7ac" +checksum = "955518d47e09b25bbebc7a18df10b81f0c766eaf4c4f1cccef2fca5f2a4fb5f2" dependencies = [ "futures-core", "futures-sink", @@ -1118,15 +1118,15 @@ dependencies = [ [[package]] name = "futures-core" -version = "0.3.27" +version = "0.3.28" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "86d7a0c1aa76363dac491de0ee99faf6941128376f1cf96f07db7603b7de69dd" +checksum = "4bca583b7e26f571124fe5b7561d49cb2868d79116cfa0eefce955557c6fee8c" [[package]] name = "futures-executor" -version = "0.3.27" +version = "0.3.28" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1997dd9df74cdac935c76252744c1ed5794fac083242ea4fe77ef3ed60ba0f83" +checksum = "ccecee823288125bd88b4d7f565c9e58e41858e47ab72e8ea2d64e93624386e0" dependencies = [ "futures-core", "futures-task", @@ -1135,38 +1135,38 @@ dependencies = [ [[package]] name = "futures-io" -version = "0.3.27" +version = "0.3.28" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "89d422fa3cbe3b40dca574ab087abb5bc98258ea57eea3fd6f1fa7162c778b91" +checksum = "4fff74096e71ed47f8e023204cfd0aa1289cd54ae5430a9523be060cdb849964" [[package]] name = "futures-macro" -version = "0.3.27" +version = "0.3.28" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3eb14ed937631bd8b8b8977f2c198443447a8355b6e3ca599f38c975e5a963b6" +checksum = "89ca545a94061b6365f2c7355b4b32bd20df3ff95f02da9329b34ccc3bd6ee72" dependencies = [ "proc-macro2", "quote", - "syn 1.0.109", + "syn 2.0.11", ] [[package]] name = "futures-sink" -version = "0.3.27" +version = "0.3.28" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ec93083a4aecafb2a80a885c9de1f0ccae9dbd32c2bb54b0c3a65690e0b8d2f2" +checksum = "f43be4fe21a13b9781a69afa4985b0f6ee0e1afab2c6f454a8cf30e2b2237b6e" [[package]] name = "futures-task" -version = "0.3.27" +version = "0.3.28" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fd65540d33b37b16542a0438c12e6aeead10d4ac5d05bd3f805b8f35ab592879" +checksum = "76d3d132be6c0e6aa1534069c705a74a5997a356c0dc2f86a47765e5617c5b65" [[package]] name = "futures-util" -version = "0.3.27" +version = "0.3.28" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3ef6b17e481503ec85211fed8f39d1970f128935ca1f814cd32ac4a6842e84ab" +checksum = "26b01e40b772d54cf6c6d721c1d1abd0647a0106a12ecaa1c186273392a69533" dependencies = [ "futures-channel", "futures-core", diff --git a/datafusion/core/src/physical_plan/joins/symmetric_hash_join.rs b/datafusion/core/src/physical_plan/joins/symmetric_hash_join.rs index a0a527a058ee..9d04b5d9a96c 100644 --- a/datafusion/core/src/physical_plan/joins/symmetric_hash_join.rs +++ b/datafusion/core/src/physical_plan/joins/symmetric_hash_join.rs @@ -47,9 +47,6 @@ use parking_lot::Mutex; use datafusion_common::{utils::bisect, ScalarValue}; use datafusion_physical_expr::intervals::{ExprIntervalGraph, Interval}; -use datafusion_physical_expr::{ - make_sort_requirements_from_exprs, PhysicalSortRequirement, -}; use crate::error::{DataFusionError, Result}; use crate::execution::context::TaskContext; diff --git a/datafusion/core/tests/fifo.rs b/datafusion/core/tests/fifo.rs index 4c3b262a87a2..a3dfe42e21a5 100644 --- a/datafusion/core/tests/fifo.rs +++ b/datafusion/core/tests/fifo.rs @@ -22,13 +22,13 @@ mod unix_test { use arrow::array::Array; use arrow::datatypes::{DataType, Field, Schema}; + use datafusion::execution::options::ReadOptions; use datafusion::{ prelude::{CsvReadOptions, SessionConfig, SessionContext}, - test_util::{ - aggr_test_schema, arrow_test_data, test_create_unbounded_sorted_file, - }, + test_util::{aggr_test_schema, arrow_test_data}, }; use datafusion_common::{DataFusionError, Result}; + use datafusion_expr::Expr; use futures::StreamExt; use itertools::enumerate; use nix::sys::stat; @@ -39,9 +39,7 @@ mod unix_test { use std::path::Path; use std::path::PathBuf; use std::sync::atomic::{AtomicBool, Ordering}; - use std::sync::mpsc; - use std::sync::mpsc::{Receiver, Sender}; - use std::sync::{Arc, Mutex}; + use std::sync::Arc; use std::thread; use std::thread::JoinHandle; use std::time::{Duration, Instant}; @@ -82,8 +80,6 @@ mod unix_test { if interval < broken_pipe_timeout { thread::sleep(Duration::from_millis(100)); continue; - } else { - return Err(DataFusionError::Execution(e.to_string())); } } return Err(DataFusionError::Execution(e.to_string())); @@ -91,24 +87,64 @@ mod unix_test { Ok(()) } - async fn create_ctx( - fifo_path: &Path, - with_unbounded_execution: bool, - ) -> Result { - let config = SessionConfig::new().with_batch_size(TEST_BATCH_SIZE); + // This test provides a relatively realistic end-to-end scenario where + // we swap join sides to accommodate a FIFO source. + #[rstest] + #[timeout(std::time::Duration::from_secs(30))] + #[tokio::test(flavor = "multi_thread", worker_threads = 8)] + async fn unbounded_file_with_swapped_join( + #[values(true, false)] unbounded_file: bool, + ) -> Result<()> { + // Create session context + let config = SessionConfig::new() + .with_batch_size(TEST_BATCH_SIZE) + .with_collect_statistics(false) + .with_target_partitions(1); let ctx = SessionContext::with_config(config); - // Register left table - let left_schema = Arc::new(Schema::new(vec![ + // To make unbounded deterministic + let waiting = Arc::new(AtomicBool::new(unbounded_file)); + // Create a new temporary FIFO file + let tmp_dir = TempDir::new()?; + let fifo_path = + create_fifo_file(&tmp_dir, &format!("fifo_{:?}.csv", unbounded_file))?; + // Execution can calculated at least one RecordBatch after the number of + // "joinable_lines_length" lines are read. + let joinable_lines_length = + (TEST_DATA_SIZE as f64 * TEST_JOIN_RATIO).round() as usize; + // The row including "a" is joinable with aggregate_test_100.c1 + let joinable_iterator = (0..joinable_lines_length).map(|_| "a".to_string()); + let second_joinable_iterator = + (0..joinable_lines_length).map(|_| "a".to_string()); + // The row including "zzz" is not joinable with aggregate_test_100.c1 + let non_joinable_iterator = + (0..(TEST_DATA_SIZE - joinable_lines_length)).map(|_| "zzz".to_string()); + let lines = joinable_iterator + .chain(non_joinable_iterator) + .chain(second_joinable_iterator) + .zip(0..TEST_DATA_SIZE) + .map(|(a1, a2)| format!("{a1},{a2}\n")) + .collect::>(); + // Create writing threads for the left and right FIFO files + let task = create_writing_thread( + fifo_path.clone(), + "a1,a2\n".to_owned(), + lines, + waiting.clone(), + joinable_lines_length, + ); + + // Data Schema + let schema = Arc::new(Schema::new(vec![ Field::new("a1", DataType::Utf8, false), Field::new("a2", DataType::UInt32, false), ])); + // Create a file with bounded or unbounded flag. ctx.register_csv( "left", fifo_path.as_os_str().to_str().unwrap(), CsvReadOptions::new() - .schema(left_schema.as_ref()) - .has_header(false) - .mark_infinite(with_unbounded_execution), + .schema(schema.as_ref()) + .mark_infinite(unbounded_file), ) .await?; // Register right table @@ -120,221 +156,213 @@ mod unix_test { CsvReadOptions::new().schema(schema.as_ref()), ) .await?; - Ok(ctx) + // Execute the query + let df = ctx.sql("SELECT t1.a2, t2.c1, t2.c4, t2.c5 FROM left as t1 JOIN right as t2 ON t1.a1 = t2.c1").await?; + let mut stream = df.execute_stream().await?; + while (stream.next().await).is_some() { + waiting.store(false, Ordering::SeqCst); + } + task.join().unwrap(); + Ok(()) } #[derive(Debug, PartialEq)] - enum Operation { - Read, - Write, - } - - /// Checks if there is a [Operation::Read] between [Operation::Write]s. - /// This indicates we did not wait for the file to finish before processing it. - fn interleave(result: &[Operation]) -> bool { - let first_read = result.iter().position(|op| op == &Operation::Read); - let last_write = result.iter().rev().position(|op| op == &Operation::Write); - match (first_read, last_write) { - (Some(first_read), Some(last_write)) => { - result.len() - 1 - last_write > first_read - } - (_, _) => false, - } + enum JoinOperation { + LeftUnmatched, + RightUnmatched, + Equal, } - // This test provides a relatively realistic end-to-end scenario where - // we swap join sides to accommodate a FIFO source. - #[rstest] - #[tokio::test(flavor = "multi_thread", worker_threads = 8)] - async fn unbounded_file_with_swapped_join( - #[values(true, false)] unbounded_file: bool, - ) -> Result<()> { - let result = tokio::time::timeout(Duration::from_secs(45), async { - // To make unbounded deterministic - let waiting = Arc::new(Mutex::new(unbounded_file)); - let waiting_thread = waiting.clone(); - // Create a channel - let (tx, rx): (Sender, Receiver) = mpsc::channel(); - // Create a new temporary FIFO file - let tmp_dir = TempDir::new()?; - let fifo_path = create_fifo_file(&tmp_dir, "first_fifo.csv")?; - // Prevent move - let fifo_path_thread = fifo_path.clone(); + fn create_writing_thread( + file_path: PathBuf, + header: String, + lines: Vec, + waiting_lock: Arc, + wait_until: usize, + ) -> JoinHandle<()> { // Timeout for a long period of BrokenPipe error - let broken_pipe_timeout = Duration::from_secs(5); - // The sender endpoint can be copied - let thread_tx = tx.clone(); + let broken_pipe_timeout = Duration::from_secs(10); // Spawn a new thread to write to the FIFO file - let fifo_writer = thread::spawn(move || { - let first_file = OpenOptions::new() - .write(true) - .open(fifo_path_thread.clone()) - .unwrap(); - log::debug!("File at {:?} opened.", fifo_path_thread); + thread::spawn(move || { + let file = OpenOptions::new().write(true).open(file_path).unwrap(); // Reference time to use when deciding to fail the test let execution_start = Instant::now(); - // Execution can calculated at least one RecordBatch after the number of - // "joinable_lines_length" lines are read. - let joinable_lines_length = - (TEST_DATA_SIZE as f64 * TEST_JOIN_RATIO).round() as usize; - // The row including "a" is joinable with aggregate_test_100.c1 - let joinable_iterator = (0..joinable_lines_length).map(|_| "a".to_string()); - let second_joinable_iterator = - (0..joinable_lines_length).map(|_| "a".to_string()); - // The row including "zzz" is not joinable with aggregate_test_100.c1 - let non_joinable_iterator = - (0..(TEST_DATA_SIZE - joinable_lines_length)).map(|_| "zzz".to_string()); - let string_array = joinable_iterator - .chain(non_joinable_iterator) - .chain(second_joinable_iterator); - for (cnt, string_col) in enumerate(string_array) { - // Wait a reading sign for unbounded execution - // For unbounded execution: - // After joinable_lines_length FIFO reading, we MUST get a Operation::Read. - // For bounded execution: - // Never goes into while loop since waiting_thread initiated as false. - while *waiting_thread.lock().unwrap() && joinable_lines_length < cnt { - thread::sleep(Duration::from_millis(200)); - } - // Each thread queues a message in the channel - if cnt % TEST_BATCH_SIZE == 0 { - thread_tx.send(Operation::Write).unwrap(); + write_to_fifo(&file, &header, execution_start, broken_pipe_timeout).unwrap(); + for (cnt, line) in enumerate(lines) { + while waiting_lock.load(Ordering::SeqCst) && cnt > wait_until { + thread::sleep(Duration::from_millis(50)); } - let line = format!("{string_col},{cnt}\n").to_owned(); - write_to_fifo(&first_file, &line, execution_start, broken_pipe_timeout) + write_to_fifo(&file, &line, execution_start, broken_pipe_timeout) .unwrap(); } - log::debug!("File at {:?} finished.", fifo_path_thread); - }); - // Collects operations from both writer and executor. - let result_collector = thread::spawn(move || { - let mut results = vec![]; - while let Ok(res) = rx.recv() { - results.push(res); - } - results - }); - // Create an execution case with bounded or unbounded flag. - let ctx = create_ctx(&fifo_path, unbounded_file).await?; - // Execute the query - let df = ctx.sql("SELECT t1.a2, t2.c1, t2.c4, t2.c5 FROM left as t1 JOIN right as t2 ON t1.a1 = t2.c1").await?; - let mut stream = df.execute_stream().await?; - while (stream.next().await).is_some() { - *waiting.lock().unwrap() = false; - tx.send(Operation::Read).unwrap(); - } - fifo_writer.join().unwrap(); - drop(tx); - let result = result_collector.join().unwrap(); - assert_eq!(interleave(&result), unbounded_file); - Ok::<(), DataFusionError>(()) - }).await; - assert!(result.is_ok(), "Test did not timeout as expected."); - result.unwrap() + drop(file); + }) } - #[derive(Debug, PartialEq)] - enum JoinOperation { - LeftUnmatched, - RightUnmatched, - Equal, + /// This function creates an unbounded sorted file for testing purposes. + pub async fn register_unbounded_file_with_ordering( + ctx: &SessionContext, + schema: arrow::datatypes::SchemaRef, + file_path: &Path, + table_name: &str, + file_sort_order: Option>, + with_unbounded_execution: bool, + ) -> Result<()> { + // Mark infinite and provide schema: + let fifo_options = CsvReadOptions::new() + .schema(schema.as_ref()) + .mark_infinite(with_unbounded_execution); + // Get listing options: + let options_sort = fifo_options + .to_listing_options(&ctx.copied_config()) + .with_file_sort_order(file_sort_order); + // Register table: + ctx.register_listing_table( + table_name, + file_path.as_os_str().to_str().unwrap(), + options_sort, + Some(schema), + None, + ) + .await?; + Ok(()) } // This test provides a relatively realistic end-to-end scenario where // we change the join into a [SymmetricHashJoin] to accommodate two // unbounded (FIFO) sources. - #[tokio::test(flavor = "multi_thread", worker_threads = 8)] + #[rstest] + #[timeout(std::time::Duration::from_secs(30))] + #[tokio::test(flavor = "multi_thread")] + #[ignore] async fn unbounded_file_with_symmetric_join() -> Result<()> { - let result = tokio::time::timeout(Duration::from_secs(360), async { - // To make unbounded deterministic - let waiting = Arc::new(AtomicBool::new(true)); - let thread_bools = vec![waiting.clone(), waiting.clone()]; - // Create a new temporary FIFO file - let tmp_dir = TempDir::new()?; - let file_names = vec!["first_fifo.csv", "second_fifo.csv"]; - // The sender endpoint can be copied - let (threads, file_paths): (Vec>, Vec) = file_names - .iter() - .zip(thread_bools.iter()) - .map(|(file_name, atomic_bool)| { - let waiting_thread = atomic_bool.clone(); - let fifo_path = create_fifo_file(&tmp_dir, file_name).unwrap(); - let return_path = fifo_path.clone(); - // Timeout for a long period of BrokenPipe error - let broken_pipe_timeout = Duration::from_secs(45); - // Spawn a new thread to write to the FIFO file - let fifo_writer = thread::spawn(move || { - let file = OpenOptions::new() - .write(true) - .open(fifo_path.clone()) - .unwrap(); - // Reference time to use when deciding to fail the test - let execution_start = Instant::now(); - // Join filter - let a1_iter = 0..TEST_DATA_SIZE; - // Join key - let a2_iter = (0..TEST_DATA_SIZE).map(|x| x % 10); - for (cnt, (a1, a2)) in a1_iter.zip(a2_iter).enumerate() { - // Wait a reading sign for unbounded execution - // After first 2 batch FIFO reading in both sides, we will wait for a batch created. - while waiting_thread.load(Ordering::SeqCst) && (TEST_BATCH_SIZE * 2) + 1 < cnt - { - thread::sleep(Duration::from_millis(150)); - } - let line = format!("{a1},{a2}\n").to_owned(); - write_to_fifo(&file, &line, execution_start, broken_pipe_timeout) - .unwrap(); - } - drop(file); - }); - (fifo_writer, return_path) + // Create session context + let config = SessionConfig::new() + .with_batch_size(TEST_BATCH_SIZE) + .set_bool("datafusion.execution.coalesce_batches", false) + .with_target_partitions(1); + let ctx = SessionContext::with_config(config); + // Tasks + let mut tasks: Vec> = vec![]; + + // Join filter + let a1_iter = 0..TEST_DATA_SIZE; + // Join key + let a2_iter = (0..TEST_DATA_SIZE).map(|x| x % 10); + let lines = a1_iter + .zip(a2_iter) + .map(|(a1, a2)| format!("{a1},{a2}\n")) + .collect::>(); + + // Create a new temporary FIFO file + let tmp_dir = TempDir::new()?; + // Create a FIFO file for the left input source. + let left_fifo = create_fifo_file(&tmp_dir, "left.csv")?; + // Create a FIFO file for the right input source. + let right_fifo = create_fifo_file(&tmp_dir, "right.csv")?; + // Create a mutex for tracking if the right input source is waiting for data. + let waiting = Arc::new(AtomicBool::new(true)); + + // Create writing threads for the left and right FIFO files + tasks.push(create_writing_thread( + left_fifo.clone(), + "a1,a2\n".to_owned(), + lines.clone(), + waiting.clone(), + TEST_BATCH_SIZE, + )); + tasks.push(create_writing_thread( + right_fifo.clone(), + "a1,a2\n".to_owned(), + lines.clone(), + waiting.clone(), + TEST_BATCH_SIZE, + )); + + // Create schema + let schema = Arc::new(Schema::new(vec![ + Field::new("a1", DataType::UInt32, false), + Field::new("a2", DataType::UInt32, false), + ])); + // Specify the ordering: + let file_sort_order = Some( + [datafusion_expr::col("a1")] + .into_iter() + .map(|e| { + let ascending = true; + let nulls_first = false; + e.sort(ascending, nulls_first) }) - .unzip(); - let config = SessionConfig::new() - .with_batch_size(TEST_BATCH_SIZE) - .set_bool("datafusion.execution.coalesce_batches", false) - .with_target_partitions(1); - let ctx = SessionContext::with_config(config); - test_create_unbounded_sorted_file(&ctx, file_paths[0].clone(), "left").await?; - test_create_unbounded_sorted_file(&ctx, file_paths[1].clone(), "right").await?; - // Execute the query - let df = ctx.sql("SELECT t1.a1, t1.a2, t2.a1, t2.a2 FROM left as t1 FULL JOIN right as t2 ON t1.a2 = t2.a2 AND t1.a1 > t2.a1 + 3 AND t1.a1 < t2.a1 + 10").await?; - let mut stream = df.execute_stream().await?; - let mut operations = vec![]; - while let Some(Ok(batch)) = stream.next().await { - waiting.store(false, Ordering::SeqCst); - let left_unmatched = batch.column(2).null_count(); - let right_unmatched = batch.column(0).null_count(); - let op = if left_unmatched == 0 && right_unmatched == 0 { - JoinOperation::Equal - } else if right_unmatched > left_unmatched { - JoinOperation::RightUnmatched - } else { - JoinOperation::LeftUnmatched - }; - operations.push(op); - } - threads.into_iter().for_each(|j| j.join().unwrap()); - // The SymmetricHashJoin executor produces FULL join results at every - // pruning, which happens before it reaches the end of input and more - // than once. In this test, we feed partially joinable data to both - // sides in order to ensure that both left/right unmatched results are - // generated more than once during the test. - assert!( - operations - .iter() - .filter(|&n| JoinOperation::RightUnmatched.eq(n)) - .count() - > 1 - && operations + .collect::>(), + ); + // Set unbounded sorted files read configuration + register_unbounded_file_with_ordering( + &ctx, + schema.clone(), + &left_fifo, + "left", + file_sort_order.clone(), + true, + ) + .await?; + register_unbounded_file_with_ordering( + &ctx, + schema, + &right_fifo, + "right", + file_sort_order, + true, + ) + .await?; + // Execute the query, with no matching rows. (since key is modulus 10) + let df = ctx + .sql( + "SELECT + t1.a1, + t1.a2, + t2.a1, + t2.a2 + FROM + left as t1 FULL + JOIN right as t2 ON t1.a2 = t2.a2 + AND t1.a1 > t2.a1 + 4 + AND t1.a1 < t2.a1 + 9", + ) + .await?; + let mut stream = df.execute_stream().await?; + let mut operations = vec![]; + // Partial. + while let Some(Ok(batch)) = stream.next().await { + waiting.store(false, Ordering::SeqCst); + let left_unmatched = batch.column(2).null_count(); + let right_unmatched = batch.column(0).null_count(); + let op = if left_unmatched == 0 && right_unmatched == 0 { + JoinOperation::Equal + } else if right_unmatched > left_unmatched { + JoinOperation::RightUnmatched + } else { + JoinOperation::LeftUnmatched + }; + operations.push(op); + } + tasks.into_iter().for_each(|jh| jh.join().unwrap()); + // The SymmetricHashJoin executor produces FULL join results at every + // pruning, which happens before it reaches the end of input and more + // than once. In this test, we feed partially joinable data to both + // sides in order to ensure that left or right unmatched results are + // generated more than once during the test. + assert!( + operations + .iter() + .filter(|&n| JoinOperation::RightUnmatched.eq(n)) + .count() + > 1 + && operations .iter() .filter(|&n| JoinOperation::LeftUnmatched.eq(n)) .count() > 1 - ); - Ok::<(), DataFusionError>(()) - }).await; - assert!(result.is_ok(), "Test did not timeout as expected."); - result.unwrap() + ); + Ok(()) } } From 331851b5904652b5ffe7c101e8c101cbd03095d9 Mon Sep 17 00:00:00 2001 From: metesynnada <100111937+metesynnada@users.noreply.github.com> Date: Mon, 3 Apr 2023 18:56:11 +0300 Subject: [PATCH 20/25] Update config.rs --- datafusion/common/src/config.rs | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index 7918fb9500b8..4a55fd1e06f6 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -282,6 +282,19 @@ config_namespace! { /// Should DataFusion allow symmetric hash joins for unbounded data sources even when /// its inputs do not have any ordering or filtering + /// + /// If the flag is not enabled, the SymmetricHashJoin operator will be unable to prune its internal buffers, + /// resulting in certain join types + /// - [JoinType::Full], + /// - [JoinType::Left] + /// - [JoinType::LeftAnti] + /// - [JoinType::LeftSemi] + /// - [JoinType::Right] + /// - [JoinType::RightAnti] + /// - [JoinType::RightSemi] + /// being produced only at the end of the execution. This is not typical in stream processing. + /// Additionally, without proper design for long runner execution, all types of joins may + /// encounter out-of-memory errors. pub allow_symmetric_joins_without_pruning: bool, default = true /// When set to true, file groups will be repartitioned to achieve maximum parallelism. From f96264337b494977a64aa70e3a36fab5da4e8840 Mon Sep 17 00:00:00 2001 From: metesynnada <100111937+metesynnada@users.noreply.github.com> Date: Mon, 3 Apr 2023 20:35:04 +0300 Subject: [PATCH 21/25] Config update --- datafusion/common/src/config.rs | 15 ++---- docs/source/user-guide/configs.md | 76 +++++++++++++++---------------- 2 files changed, 42 insertions(+), 49 deletions(-) diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index 4a55fd1e06f6..75ee4e892b50 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -284,17 +284,10 @@ config_namespace! { /// its inputs do not have any ordering or filtering /// /// If the flag is not enabled, the SymmetricHashJoin operator will be unable to prune its internal buffers, - /// resulting in certain join types - /// - [JoinType::Full], - /// - [JoinType::Left] - /// - [JoinType::LeftAnti] - /// - [JoinType::LeftSemi] - /// - [JoinType::Right] - /// - [JoinType::RightAnti] - /// - [JoinType::RightSemi] - /// being produced only at the end of the execution. This is not typical in stream processing. - /// Additionally, without proper design for long runner execution, all types of joins may - /// encounter out-of-memory errors. + /// resulting in certain join types - such as Full, Left, LeftAnti, LeftSemi, Right, + /// RightAnti, and RightSemi -being produced only at the end of the execution. + /// This is not typical in stream processing. Additionally, without proper design for long + /// runner execution, all types of joins may encounter out-of-memory errors. pub allow_symmetric_joins_without_pruning: bool, default = true /// When set to true, file groups will be repartitioned to achieve maximum parallelism. diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index f1e5ff114261..e5f2941bb0de 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -35,41 +35,41 @@ Values are parsed according to the [same rules used in casts from Utf8](https:// If the value in the environment variable cannot be cast to the type of the configuration option, the default value will be used instead and a warning emitted. Environment variables are read during `SessionConfig` initialisation so they must be set beforehand and will not affect running sessions. -| key | default | description | -| ---------------------------------------------------------- | ---------- | ----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | -| datafusion.catalog.create_default_catalog_and_schema | true | Whether the default catalog and schema should be created automatically. | -| datafusion.catalog.default_catalog | datafusion | The default catalog name - this impacts what SQL queries use if not specified | -| datafusion.catalog.default_schema | public | The default schema name - this impacts what SQL queries use if not specified | -| datafusion.catalog.information_schema | false | Should DataFusion provide access to `information_schema` virtual tables for displaying schema information | -| datafusion.catalog.location | NULL | Location scanned to load tables for `default` schema | -| datafusion.catalog.format | NULL | Type of `TableProvider` to use when loading `default` schema | -| datafusion.catalog.has_header | false | If the file has a header | -| datafusion.execution.batch_size | 8192 | Default batch size while creating new batches, it's especially useful for buffer-in-memory batches since creating tiny batches would result in too much metadata memory consumption | -| datafusion.execution.coalesce_batches | true | When set to true, record batches will be examined between each operator and small batches will be coalesced into larger batches. This is helpful when there are highly selective filters or joins that could produce tiny output batches. The target batch size is determined by the configuration setting | -| datafusion.execution.collect_statistics | false | Should DataFusion collect statistics after listing files | -| datafusion.execution.target_partitions | 0 | Number of partitions for query execution. Increasing partitions can increase concurrency. Defaults to the number of CPU cores on the system | -| datafusion.execution.time_zone | +00:00 | The default time zone Some functions, e.g. `EXTRACT(HOUR from SOME_TIME)`, shift the underlying datetime according to this time zone, and then extract the hour | -| datafusion.execution.parquet.enable_page_index | false | If true, uses parquet data page level metadata (Page Index) statistics to reduce the number of rows decoded. | -| datafusion.execution.parquet.pruning | true | If true, the parquet reader attempts to skip entire row groups based on the predicate in the query and the metadata (min/max values) stored in the parquet file | -| datafusion.execution.parquet.skip_metadata | true | If true, the parquet reader skip the optional embedded metadata that may be in the file Schema. This setting can help avoid schema conflicts when querying multiple parquet files with schemas containing compatible types but different metadata | -| datafusion.execution.parquet.metadata_size_hint | NULL | If specified, the parquet reader will try and fetch the last `size_hint` bytes of the parquet file optimistically. If not specified, two reads are required: One read to fetch the 8-byte parquet footer and another to fetch the metadata length encoded in the footer | -| datafusion.execution.parquet.pushdown_filters | false | If true, filter expressions are be applied during the parquet decoding operation to reduce the number of rows decoded | -| datafusion.execution.parquet.reorder_filters | false | If true, filter expressions evaluated during the parquet decoding operation will be reordered heuristically to minimize the cost of evaluation. If false, the filters are applied in the same order as written in the query | -| datafusion.optimizer.enable_round_robin_repartition | true | When set to true, the physical plan optimizer will try to add round robin repartitioning to increase parallelism to leverage more CPU cores | -| datafusion.optimizer.filter_null_join_keys | false | When set to true, the optimizer will insert filters before a join between a nullable and non-nullable column to filter out nulls on the nullable side. This filter can add additional overhead when the file format does not fully support predicate push down. | -| datafusion.optimizer.repartition_aggregations | true | Should DataFusion repartition data using the aggregate keys to execute aggregates in parallel using the provided `target_partitions` level | -| datafusion.optimizer.repartition_file_min_size | 10485760 | Minimum total files size in bytes to perform file scan repartitioning. | -| datafusion.optimizer.repartition_joins | true | Should DataFusion repartition data using the join keys to execute joins in parallel using the provided `target_partitions` level | -| datafusion.optimizer.allow_symmetric_joins_without_pruning | true | Should DataFusion allow symmetric hash joins for unbounded data sources even when its inputs do not have any ordering or filtering | -| datafusion.optimizer.repartition_file_scans | true | When set to true, file groups will be repartitioned to achieve maximum parallelism. Currently supported only for Parquet format in which case multiple row groups from the same file may be read concurrently. If false then each row group is read serially, though different files may be read in parallel. | -| datafusion.optimizer.repartition_windows | true | Should DataFusion repartition data using the partitions keys to execute window functions in parallel using the provided `target_partitions` level | -| datafusion.optimizer.repartition_sorts | true | Should DataFusion execute sorts in a per-partition fashion and merge afterwards instead of coalescing first and sorting globally. With this flag is enabled, plans in the form below "SortExec: [a@0 ASC]", " CoalescePartitionsExec", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", would turn into the plan below which performs better in multithreaded environments "SortPreservingMergeExec: [a@0 ASC]", " SortExec: [a@0 ASC]", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", | -| datafusion.optimizer.skip_failed_rules | true | When set to true, the logical plan optimizer will produce warning messages if any optimization rules produce errors and then proceed to the next rule. When set to false, any rules that produce errors will cause the query to fail | -| datafusion.optimizer.max_passes | 3 | Number of times that the optimizer will attempt to optimize the plan | -| datafusion.optimizer.top_down_join_key_reordering | true | When set to true, the physical plan optimizer will run a top down process to reorder the join keys | -| datafusion.optimizer.prefer_hash_join | true | When set to true, the physical plan optimizer will prefer HashJoin over SortMergeJoin. HashJoin can work more efficiently than SortMergeJoin but consumes more memory | -| datafusion.optimizer.hash_join_single_partition_threshold | 1048576 | The maximum estimated size in bytes for one input side of a HashJoin will be collected into a single partition | -| datafusion.explain.logical_plan_only | false | When set to true, the explain statement will only print logical plans | -| datafusion.explain.physical_plan_only | false | When set to true, the explain statement will only print physical plans | -| datafusion.sql_parser.parse_float_as_decimal | false | When set to true, SQL parser will parse float as decimal type | -| datafusion.sql_parser.enable_ident_normalization | true | When set to true, SQL parser will normalize ident (convert ident to lowercase when not quoted) | +| key | default | description | +| ---------------------------------------------------------- | ---------- | ------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | +| datafusion.catalog.create_default_catalog_and_schema | true | Whether the default catalog and schema should be created automatically. | +| datafusion.catalog.default_catalog | datafusion | The default catalog name - this impacts what SQL queries use if not specified | +| datafusion.catalog.default_schema | public | The default schema name - this impacts what SQL queries use if not specified | +| datafusion.catalog.information_schema | false | Should DataFusion provide access to `information_schema` virtual tables for displaying schema information | +| datafusion.catalog.location | NULL | Location scanned to load tables for `default` schema | +| datafusion.catalog.format | NULL | Type of `TableProvider` to use when loading `default` schema | +| datafusion.catalog.has_header | false | If the file has a header | +| datafusion.execution.batch_size | 8192 | Default batch size while creating new batches, it's especially useful for buffer-in-memory batches since creating tiny batches would result in too much metadata memory consumption | +| datafusion.execution.coalesce_batches | true | When set to true, record batches will be examined between each operator and small batches will be coalesced into larger batches. This is helpful when there are highly selective filters or joins that could produce tiny output batches. The target batch size is determined by the configuration setting | +| datafusion.execution.collect_statistics | false | Should DataFusion collect statistics after listing files | +| datafusion.execution.target_partitions | 0 | Number of partitions for query execution. Increasing partitions can increase concurrency. Defaults to the number of CPU cores on the system | +| datafusion.execution.time_zone | +00:00 | The default time zone Some functions, e.g. `EXTRACT(HOUR from SOME_TIME)`, shift the underlying datetime according to this time zone, and then extract the hour | +| datafusion.execution.parquet.enable_page_index | false | If true, uses parquet data page level metadata (Page Index) statistics to reduce the number of rows decoded. | +| datafusion.execution.parquet.pruning | true | If true, the parquet reader attempts to skip entire row groups based on the predicate in the query and the metadata (min/max values) stored in the parquet file | +| datafusion.execution.parquet.skip_metadata | true | If true, the parquet reader skip the optional embedded metadata that may be in the file Schema. This setting can help avoid schema conflicts when querying multiple parquet files with schemas containing compatible types but different metadata | +| datafusion.execution.parquet.metadata_size_hint | NULL | If specified, the parquet reader will try and fetch the last `size_hint` bytes of the parquet file optimistically. If not specified, two reads are required: One read to fetch the 8-byte parquet footer and another to fetch the metadata length encoded in the footer | +| datafusion.execution.parquet.pushdown_filters | false | If true, filter expressions are be applied during the parquet decoding operation to reduce the number of rows decoded | +| datafusion.execution.parquet.reorder_filters | false | If true, filter expressions evaluated during the parquet decoding operation will be reordered heuristically to minimize the cost of evaluation. If false, the filters are applied in the same order as written in the query | +| datafusion.optimizer.enable_round_robin_repartition | true | When set to true, the physical plan optimizer will try to add round robin repartitioning to increase parallelism to leverage more CPU cores | +| datafusion.optimizer.filter_null_join_keys | false | When set to true, the optimizer will insert filters before a join between a nullable and non-nullable column to filter out nulls on the nullable side. This filter can add additional overhead when the file format does not fully support predicate push down. | +| datafusion.optimizer.repartition_aggregations | true | Should DataFusion repartition data using the aggregate keys to execute aggregates in parallel using the provided `target_partitions` level | +| datafusion.optimizer.repartition_file_min_size | 10485760 | Minimum total files size in bytes to perform file scan repartitioning. | +| datafusion.optimizer.repartition_joins | true | Should DataFusion repartition data using the join keys to execute joins in parallel using the provided `target_partitions` level | +| datafusion.optimizer.allow_symmetric_joins_without_pruning | true | Should DataFusion allow symmetric hash joins for unbounded data sources even when its inputs do not have any ordering or filtering. If the flag is not enabled, the SymmetricHashJoin operator will be unable to prune its internal buffers, resulting in certain join types - such as Full, Left, LeftAnti, LeftSemi, Right, RightAnti, and RightSemi - being produced only at the end of the execution. This is not typical in stream processing. Additionally, without proper design for long runner execution, all types of joins may encounter out-of-memory errors. | +| datafusion.optimizer.repartition_file_scans | true | When set to true, file groups will be repartitioned to achieve maximum parallelism. Currently supported only for Parquet format in which case multiple row groups from the same file may be read concurrently. If false then each row group is read serially, though different files may be read in parallel. | +| datafusion.optimizer.repartition_windows | true | Should DataFusion repartition data using the partitions keys to execute window functions in parallel using the provided `target_partitions` level | +| datafusion.optimizer.repartition_sorts | true | Should DataFusion execute sorts in a per-partition fashion and merge afterwards instead of coalescing first and sorting globally. With this flag is enabled, plans in the form below "SortExec: [a@0 ASC]", " CoalescePartitionsExec", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", would turn into the plan below which performs better in multithreaded environments "SortPreservingMergeExec: [a@0 ASC]", " SortExec: [a@0 ASC]", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", | +| datafusion.optimizer.skip_failed_rules | true | When set to true, the logical plan optimizer will produce warning messages if any optimization rules produce errors and then proceed to the next rule. When set to false, any rules that produce errors will cause the query to fail | +| datafusion.optimizer.max_passes | 3 | Number of times that the optimizer will attempt to optimize the plan | +| datafusion.optimizer.top_down_join_key_reordering | true | When set to true, the physical plan optimizer will run a top down process to reorder the join keys | +| datafusion.optimizer.prefer_hash_join | true | When set to true, the physical plan optimizer will prefer HashJoin over SortMergeJoin. HashJoin can work more efficiently than SortMergeJoin but consumes more memory | +| datafusion.optimizer.hash_join_single_partition_threshold | 1048576 | The maximum estimated size in bytes for one input side of a HashJoin will be collected into a single partition | +| datafusion.explain.logical_plan_only | false | When set to true, the explain statement will only print logical plans | +| datafusion.explain.physical_plan_only | false | When set to true, the explain statement will only print physical plans | +| datafusion.sql_parser.parse_float_as_decimal | false | When set to true, SQL parser will parse float as decimal type | +| datafusion.sql_parser.enable_ident_normalization | true | When set to true, SQL parser will normalize ident (convert ident to lowercase when not quoted) | From 2c15d4e677a780e4ac0e38ab583cb430ee8c32d5 Mon Sep 17 00:00:00 2001 From: metesynnada <100111937+metesynnada@users.noreply.github.com> Date: Mon, 3 Apr 2023 20:57:31 +0300 Subject: [PATCH 22/25] Update config.rs --- datafusion/common/src/config.rs | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index 75ee4e892b50..c64f2321960c 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -281,13 +281,12 @@ config_namespace! { pub repartition_joins: bool, default = true /// Should DataFusion allow symmetric hash joins for unbounded data sources even when - /// its inputs do not have any ordering or filtering - /// - /// If the flag is not enabled, the SymmetricHashJoin operator will be unable to prune its internal buffers, + /// its inputs do not have any ordering or filtering If the flag is not enabled, + /// the SymmetricHashJoin operator will be unable to prune its internal buffers, /// resulting in certain join types - such as Full, Left, LeftAnti, LeftSemi, Right, /// RightAnti, and RightSemi -being produced only at the end of the execution. - /// This is not typical in stream processing. Additionally, without proper design for long - /// runner execution, all types of joins may encounter out-of-memory errors. + /// This is not typical in stream processing. Additionally, without proper design for + /// long runner execution, all types of joins may encounter out-of-memory errors. pub allow_symmetric_joins_without_pruning: bool, default = true /// When set to true, file groups will be repartitioned to achieve maximum parallelism. From 363960f9d2a66a0725a41f0c9ef4c41cc7a4c6e5 Mon Sep 17 00:00:00 2001 From: metesynnada <100111937+metesynnada@users.noreply.github.com> Date: Mon, 3 Apr 2023 21:26:26 +0300 Subject: [PATCH 23/25] Update configs.md --- docs/source/user-guide/configs.md | 76 +++++++++++++++---------------- 1 file changed, 38 insertions(+), 38 deletions(-) diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index e5f2941bb0de..1f8b0ec6884f 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -35,41 +35,41 @@ Values are parsed according to the [same rules used in casts from Utf8](https:// If the value in the environment variable cannot be cast to the type of the configuration option, the default value will be used instead and a warning emitted. Environment variables are read during `SessionConfig` initialisation so they must be set beforehand and will not affect running sessions. -| key | default | description | -| ---------------------------------------------------------- | ---------- | ------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | -| datafusion.catalog.create_default_catalog_and_schema | true | Whether the default catalog and schema should be created automatically. | -| datafusion.catalog.default_catalog | datafusion | The default catalog name - this impacts what SQL queries use if not specified | -| datafusion.catalog.default_schema | public | The default schema name - this impacts what SQL queries use if not specified | -| datafusion.catalog.information_schema | false | Should DataFusion provide access to `information_schema` virtual tables for displaying schema information | -| datafusion.catalog.location | NULL | Location scanned to load tables for `default` schema | -| datafusion.catalog.format | NULL | Type of `TableProvider` to use when loading `default` schema | -| datafusion.catalog.has_header | false | If the file has a header | -| datafusion.execution.batch_size | 8192 | Default batch size while creating new batches, it's especially useful for buffer-in-memory batches since creating tiny batches would result in too much metadata memory consumption | -| datafusion.execution.coalesce_batches | true | When set to true, record batches will be examined between each operator and small batches will be coalesced into larger batches. This is helpful when there are highly selective filters or joins that could produce tiny output batches. The target batch size is determined by the configuration setting | -| datafusion.execution.collect_statistics | false | Should DataFusion collect statistics after listing files | -| datafusion.execution.target_partitions | 0 | Number of partitions for query execution. Increasing partitions can increase concurrency. Defaults to the number of CPU cores on the system | -| datafusion.execution.time_zone | +00:00 | The default time zone Some functions, e.g. `EXTRACT(HOUR from SOME_TIME)`, shift the underlying datetime according to this time zone, and then extract the hour | -| datafusion.execution.parquet.enable_page_index | false | If true, uses parquet data page level metadata (Page Index) statistics to reduce the number of rows decoded. | -| datafusion.execution.parquet.pruning | true | If true, the parquet reader attempts to skip entire row groups based on the predicate in the query and the metadata (min/max values) stored in the parquet file | -| datafusion.execution.parquet.skip_metadata | true | If true, the parquet reader skip the optional embedded metadata that may be in the file Schema. This setting can help avoid schema conflicts when querying multiple parquet files with schemas containing compatible types but different metadata | -| datafusion.execution.parquet.metadata_size_hint | NULL | If specified, the parquet reader will try and fetch the last `size_hint` bytes of the parquet file optimistically. If not specified, two reads are required: One read to fetch the 8-byte parquet footer and another to fetch the metadata length encoded in the footer | -| datafusion.execution.parquet.pushdown_filters | false | If true, filter expressions are be applied during the parquet decoding operation to reduce the number of rows decoded | -| datafusion.execution.parquet.reorder_filters | false | If true, filter expressions evaluated during the parquet decoding operation will be reordered heuristically to minimize the cost of evaluation. If false, the filters are applied in the same order as written in the query | -| datafusion.optimizer.enable_round_robin_repartition | true | When set to true, the physical plan optimizer will try to add round robin repartitioning to increase parallelism to leverage more CPU cores | -| datafusion.optimizer.filter_null_join_keys | false | When set to true, the optimizer will insert filters before a join between a nullable and non-nullable column to filter out nulls on the nullable side. This filter can add additional overhead when the file format does not fully support predicate push down. | -| datafusion.optimizer.repartition_aggregations | true | Should DataFusion repartition data using the aggregate keys to execute aggregates in parallel using the provided `target_partitions` level | -| datafusion.optimizer.repartition_file_min_size | 10485760 | Minimum total files size in bytes to perform file scan repartitioning. | -| datafusion.optimizer.repartition_joins | true | Should DataFusion repartition data using the join keys to execute joins in parallel using the provided `target_partitions` level | -| datafusion.optimizer.allow_symmetric_joins_without_pruning | true | Should DataFusion allow symmetric hash joins for unbounded data sources even when its inputs do not have any ordering or filtering. If the flag is not enabled, the SymmetricHashJoin operator will be unable to prune its internal buffers, resulting in certain join types - such as Full, Left, LeftAnti, LeftSemi, Right, RightAnti, and RightSemi - being produced only at the end of the execution. This is not typical in stream processing. Additionally, without proper design for long runner execution, all types of joins may encounter out-of-memory errors. | -| datafusion.optimizer.repartition_file_scans | true | When set to true, file groups will be repartitioned to achieve maximum parallelism. Currently supported only for Parquet format in which case multiple row groups from the same file may be read concurrently. If false then each row group is read serially, though different files may be read in parallel. | -| datafusion.optimizer.repartition_windows | true | Should DataFusion repartition data using the partitions keys to execute window functions in parallel using the provided `target_partitions` level | -| datafusion.optimizer.repartition_sorts | true | Should DataFusion execute sorts in a per-partition fashion and merge afterwards instead of coalescing first and sorting globally. With this flag is enabled, plans in the form below "SortExec: [a@0 ASC]", " CoalescePartitionsExec", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", would turn into the plan below which performs better in multithreaded environments "SortPreservingMergeExec: [a@0 ASC]", " SortExec: [a@0 ASC]", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", | -| datafusion.optimizer.skip_failed_rules | true | When set to true, the logical plan optimizer will produce warning messages if any optimization rules produce errors and then proceed to the next rule. When set to false, any rules that produce errors will cause the query to fail | -| datafusion.optimizer.max_passes | 3 | Number of times that the optimizer will attempt to optimize the plan | -| datafusion.optimizer.top_down_join_key_reordering | true | When set to true, the physical plan optimizer will run a top down process to reorder the join keys | -| datafusion.optimizer.prefer_hash_join | true | When set to true, the physical plan optimizer will prefer HashJoin over SortMergeJoin. HashJoin can work more efficiently than SortMergeJoin but consumes more memory | -| datafusion.optimizer.hash_join_single_partition_threshold | 1048576 | The maximum estimated size in bytes for one input side of a HashJoin will be collected into a single partition | -| datafusion.explain.logical_plan_only | false | When set to true, the explain statement will only print logical plans | -| datafusion.explain.physical_plan_only | false | When set to true, the explain statement will only print physical plans | -| datafusion.sql_parser.parse_float_as_decimal | false | When set to true, SQL parser will parse float as decimal type | -| datafusion.sql_parser.enable_ident_normalization | true | When set to true, SQL parser will normalize ident (convert ident to lowercase when not quoted) | +| key | default | description | +| ---------------------------------------------------------- | ---------- | ----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | +| datafusion.catalog.create_default_catalog_and_schema | true | Whether the default catalog and schema should be created automatically. | +| datafusion.catalog.default_catalog | datafusion | The default catalog name - this impacts what SQL queries use if not specified | +| datafusion.catalog.default_schema | public | The default schema name - this impacts what SQL queries use if not specified | +| datafusion.catalog.information_schema | false | Should DataFusion provide access to `information_schema` virtual tables for displaying schema information | +| datafusion.catalog.location | NULL | Location scanned to load tables for `default` schema | +| datafusion.catalog.format | NULL | Type of `TableProvider` to use when loading `default` schema | +| datafusion.catalog.has_header | false | If the file has a header | +| datafusion.execution.batch_size | 8192 | Default batch size while creating new batches, it's especially useful for buffer-in-memory batches since creating tiny batches would result in too much metadata memory consumption | +| datafusion.execution.coalesce_batches | true | When set to true, record batches will be examined between each operator and small batches will be coalesced into larger batches. This is helpful when there are highly selective filters or joins that could produce tiny output batches. The target batch size is determined by the configuration setting | +| datafusion.execution.collect_statistics | false | Should DataFusion collect statistics after listing files | +| datafusion.execution.target_partitions | 0 | Number of partitions for query execution. Increasing partitions can increase concurrency. Defaults to the number of CPU cores on the system | +| datafusion.execution.time_zone | +00:00 | The default time zone Some functions, e.g. `EXTRACT(HOUR from SOME_TIME)`, shift the underlying datetime according to this time zone, and then extract the hour | +| datafusion.execution.parquet.enable_page_index | false | If true, uses parquet data page level metadata (Page Index) statistics to reduce the number of rows decoded. | +| datafusion.execution.parquet.pruning | true | If true, the parquet reader attempts to skip entire row groups based on the predicate in the query and the metadata (min/max values) stored in the parquet file | +| datafusion.execution.parquet.skip_metadata | true | If true, the parquet reader skip the optional embedded metadata that may be in the file Schema. This setting can help avoid schema conflicts when querying multiple parquet files with schemas containing compatible types but different metadata | +| datafusion.execution.parquet.metadata_size_hint | NULL | If specified, the parquet reader will try and fetch the last `size_hint` bytes of the parquet file optimistically. If not specified, two reads are required: One read to fetch the 8-byte parquet footer and another to fetch the metadata length encoded in the footer | +| datafusion.execution.parquet.pushdown_filters | false | If true, filter expressions are be applied during the parquet decoding operation to reduce the number of rows decoded | +| datafusion.execution.parquet.reorder_filters | false | If true, filter expressions evaluated during the parquet decoding operation will be reordered heuristically to minimize the cost of evaluation. If false, the filters are applied in the same order as written in the query | +| datafusion.optimizer.enable_round_robin_repartition | true | When set to true, the physical plan optimizer will try to add round robin repartitioning to increase parallelism to leverage more CPU cores | +| datafusion.optimizer.filter_null_join_keys | false | When set to true, the optimizer will insert filters before a join between a nullable and non-nullable column to filter out nulls on the nullable side. This filter can add additional overhead when the file format does not fully support predicate push down. | +| datafusion.optimizer.repartition_aggregations | true | Should DataFusion repartition data using the aggregate keys to execute aggregates in parallel using the provided `target_partitions` level | +| datafusion.optimizer.repartition_file_min_size | 10485760 | Minimum total files size in bytes to perform file scan repartitioning. | +| datafusion.optimizer.repartition_joins | true | Should DataFusion repartition data using the join keys to execute joins in parallel using the provided `target_partitions` level | +| datafusion.optimizer.allow_symmetric_joins_without_pruning | true | Should DataFusion allow symmetric hash joins for unbounded data sources even when its inputs do not have any ordering or filtering If the flag is not enabled, the SymmetricHashJoin operator will be unable to prune its internal buffers, resulting in certain join types - such as Full, Left, LeftAnti, LeftSemi, Right, RightAnti, and RightSemi -being produced only at the end of the execution. This is not typical in stream processing. Additionally, without proper design for long runner execution, all types of joins may encounter out-of-memory errors. | +| datafusion.optimizer.repartition_file_scans | true | When set to true, file groups will be repartitioned to achieve maximum parallelism. Currently supported only for Parquet format in which case multiple row groups from the same file may be read concurrently. If false then each row group is read serially, though different files may be read in parallel. | +| datafusion.optimizer.repartition_windows | true | Should DataFusion repartition data using the partitions keys to execute window functions in parallel using the provided `target_partitions` level | +| datafusion.optimizer.repartition_sorts | true | Should DataFusion execute sorts in a per-partition fashion and merge afterwards instead of coalescing first and sorting globally. With this flag is enabled, plans in the form below "SortExec: [a@0 ASC]", " CoalescePartitionsExec", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", would turn into the plan below which performs better in multithreaded environments "SortPreservingMergeExec: [a@0 ASC]", " SortExec: [a@0 ASC]", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", | +| datafusion.optimizer.skip_failed_rules | true | When set to true, the logical plan optimizer will produce warning messages if any optimization rules produce errors and then proceed to the next rule. When set to false, any rules that produce errors will cause the query to fail | +| datafusion.optimizer.max_passes | 3 | Number of times that the optimizer will attempt to optimize the plan | +| datafusion.optimizer.top_down_join_key_reordering | true | When set to true, the physical plan optimizer will run a top down process to reorder the join keys | +| datafusion.optimizer.prefer_hash_join | true | When set to true, the physical plan optimizer will prefer HashJoin over SortMergeJoin. HashJoin can work more efficiently than SortMergeJoin but consumes more memory | +| datafusion.optimizer.hash_join_single_partition_threshold | 1048576 | The maximum estimated size in bytes for one input side of a HashJoin will be collected into a single partition | +| datafusion.explain.logical_plan_only | false | When set to true, the explain statement will only print logical plans | +| datafusion.explain.physical_plan_only | false | When set to true, the explain statement will only print physical plans | +| datafusion.sql_parser.parse_float_as_decimal | false | When set to true, SQL parser will parse float as decimal type | +| datafusion.sql_parser.enable_ident_normalization | true | When set to true, SQL parser will normalize ident (convert ident to lowercase when not quoted) | From 61c3434df6612306160ab24e84e53500b417b9aa Mon Sep 17 00:00:00 2001 From: metesynnada <100111937+metesynnada@users.noreply.github.com> Date: Mon, 3 Apr 2023 21:31:27 +0300 Subject: [PATCH 24/25] Update config --- datafusion/common/src/config.rs | 2 +- docs/source/user-guide/configs.md | 76 +++++++++++++++---------------- 2 files changed, 39 insertions(+), 39 deletions(-) diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index c64f2321960c..55cdc36d20a0 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -284,7 +284,7 @@ config_namespace! { /// its inputs do not have any ordering or filtering If the flag is not enabled, /// the SymmetricHashJoin operator will be unable to prune its internal buffers, /// resulting in certain join types - such as Full, Left, LeftAnti, LeftSemi, Right, - /// RightAnti, and RightSemi -being produced only at the end of the execution. + /// RightAnti, and RightSemi - being produced only at the end of the execution. /// This is not typical in stream processing. Additionally, without proper design for /// long runner execution, all types of joins may encounter out-of-memory errors. pub allow_symmetric_joins_without_pruning: bool, default = true diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 1f8b0ec6884f..749a0bcb0636 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -35,41 +35,41 @@ Values are parsed according to the [same rules used in casts from Utf8](https:// If the value in the environment variable cannot be cast to the type of the configuration option, the default value will be used instead and a warning emitted. Environment variables are read during `SessionConfig` initialisation so they must be set beforehand and will not affect running sessions. -| key | default | description | -| ---------------------------------------------------------- | ---------- | ----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | -| datafusion.catalog.create_default_catalog_and_schema | true | Whether the default catalog and schema should be created automatically. | -| datafusion.catalog.default_catalog | datafusion | The default catalog name - this impacts what SQL queries use if not specified | -| datafusion.catalog.default_schema | public | The default schema name - this impacts what SQL queries use if not specified | -| datafusion.catalog.information_schema | false | Should DataFusion provide access to `information_schema` virtual tables for displaying schema information | -| datafusion.catalog.location | NULL | Location scanned to load tables for `default` schema | -| datafusion.catalog.format | NULL | Type of `TableProvider` to use when loading `default` schema | -| datafusion.catalog.has_header | false | If the file has a header | -| datafusion.execution.batch_size | 8192 | Default batch size while creating new batches, it's especially useful for buffer-in-memory batches since creating tiny batches would result in too much metadata memory consumption | -| datafusion.execution.coalesce_batches | true | When set to true, record batches will be examined between each operator and small batches will be coalesced into larger batches. This is helpful when there are highly selective filters or joins that could produce tiny output batches. The target batch size is determined by the configuration setting | -| datafusion.execution.collect_statistics | false | Should DataFusion collect statistics after listing files | -| datafusion.execution.target_partitions | 0 | Number of partitions for query execution. Increasing partitions can increase concurrency. Defaults to the number of CPU cores on the system | -| datafusion.execution.time_zone | +00:00 | The default time zone Some functions, e.g. `EXTRACT(HOUR from SOME_TIME)`, shift the underlying datetime according to this time zone, and then extract the hour | -| datafusion.execution.parquet.enable_page_index | false | If true, uses parquet data page level metadata (Page Index) statistics to reduce the number of rows decoded. | -| datafusion.execution.parquet.pruning | true | If true, the parquet reader attempts to skip entire row groups based on the predicate in the query and the metadata (min/max values) stored in the parquet file | -| datafusion.execution.parquet.skip_metadata | true | If true, the parquet reader skip the optional embedded metadata that may be in the file Schema. This setting can help avoid schema conflicts when querying multiple parquet files with schemas containing compatible types but different metadata | -| datafusion.execution.parquet.metadata_size_hint | NULL | If specified, the parquet reader will try and fetch the last `size_hint` bytes of the parquet file optimistically. If not specified, two reads are required: One read to fetch the 8-byte parquet footer and another to fetch the metadata length encoded in the footer | -| datafusion.execution.parquet.pushdown_filters | false | If true, filter expressions are be applied during the parquet decoding operation to reduce the number of rows decoded | -| datafusion.execution.parquet.reorder_filters | false | If true, filter expressions evaluated during the parquet decoding operation will be reordered heuristically to minimize the cost of evaluation. If false, the filters are applied in the same order as written in the query | -| datafusion.optimizer.enable_round_robin_repartition | true | When set to true, the physical plan optimizer will try to add round robin repartitioning to increase parallelism to leverage more CPU cores | -| datafusion.optimizer.filter_null_join_keys | false | When set to true, the optimizer will insert filters before a join between a nullable and non-nullable column to filter out nulls on the nullable side. This filter can add additional overhead when the file format does not fully support predicate push down. | -| datafusion.optimizer.repartition_aggregations | true | Should DataFusion repartition data using the aggregate keys to execute aggregates in parallel using the provided `target_partitions` level | -| datafusion.optimizer.repartition_file_min_size | 10485760 | Minimum total files size in bytes to perform file scan repartitioning. | -| datafusion.optimizer.repartition_joins | true | Should DataFusion repartition data using the join keys to execute joins in parallel using the provided `target_partitions` level | -| datafusion.optimizer.allow_symmetric_joins_without_pruning | true | Should DataFusion allow symmetric hash joins for unbounded data sources even when its inputs do not have any ordering or filtering If the flag is not enabled, the SymmetricHashJoin operator will be unable to prune its internal buffers, resulting in certain join types - such as Full, Left, LeftAnti, LeftSemi, Right, RightAnti, and RightSemi -being produced only at the end of the execution. This is not typical in stream processing. Additionally, without proper design for long runner execution, all types of joins may encounter out-of-memory errors. | -| datafusion.optimizer.repartition_file_scans | true | When set to true, file groups will be repartitioned to achieve maximum parallelism. Currently supported only for Parquet format in which case multiple row groups from the same file may be read concurrently. If false then each row group is read serially, though different files may be read in parallel. | -| datafusion.optimizer.repartition_windows | true | Should DataFusion repartition data using the partitions keys to execute window functions in parallel using the provided `target_partitions` level | -| datafusion.optimizer.repartition_sorts | true | Should DataFusion execute sorts in a per-partition fashion and merge afterwards instead of coalescing first and sorting globally. With this flag is enabled, plans in the form below "SortExec: [a@0 ASC]", " CoalescePartitionsExec", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", would turn into the plan below which performs better in multithreaded environments "SortPreservingMergeExec: [a@0 ASC]", " SortExec: [a@0 ASC]", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", | -| datafusion.optimizer.skip_failed_rules | true | When set to true, the logical plan optimizer will produce warning messages if any optimization rules produce errors and then proceed to the next rule. When set to false, any rules that produce errors will cause the query to fail | -| datafusion.optimizer.max_passes | 3 | Number of times that the optimizer will attempt to optimize the plan | -| datafusion.optimizer.top_down_join_key_reordering | true | When set to true, the physical plan optimizer will run a top down process to reorder the join keys | -| datafusion.optimizer.prefer_hash_join | true | When set to true, the physical plan optimizer will prefer HashJoin over SortMergeJoin. HashJoin can work more efficiently than SortMergeJoin but consumes more memory | -| datafusion.optimizer.hash_join_single_partition_threshold | 1048576 | The maximum estimated size in bytes for one input side of a HashJoin will be collected into a single partition | -| datafusion.explain.logical_plan_only | false | When set to true, the explain statement will only print logical plans | -| datafusion.explain.physical_plan_only | false | When set to true, the explain statement will only print physical plans | -| datafusion.sql_parser.parse_float_as_decimal | false | When set to true, SQL parser will parse float as decimal type | -| datafusion.sql_parser.enable_ident_normalization | true | When set to true, SQL parser will normalize ident (convert ident to lowercase when not quoted) | +| key | default | description | +| ---------------------------------------------------------- | ---------- | ------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------ | +| datafusion.catalog.create_default_catalog_and_schema | true | Whether the default catalog and schema should be created automatically. | +| datafusion.catalog.default_catalog | datafusion | The default catalog name - this impacts what SQL queries use if not specified | +| datafusion.catalog.default_schema | public | The default schema name - this impacts what SQL queries use if not specified | +| datafusion.catalog.information_schema | false | Should DataFusion provide access to `information_schema` virtual tables for displaying schema information | +| datafusion.catalog.location | NULL | Location scanned to load tables for `default` schema | +| datafusion.catalog.format | NULL | Type of `TableProvider` to use when loading `default` schema | +| datafusion.catalog.has_header | false | If the file has a header | +| datafusion.execution.batch_size | 8192 | Default batch size while creating new batches, it's especially useful for buffer-in-memory batches since creating tiny batches would result in too much metadata memory consumption | +| datafusion.execution.coalesce_batches | true | When set to true, record batches will be examined between each operator and small batches will be coalesced into larger batches. This is helpful when there are highly selective filters or joins that could produce tiny output batches. The target batch size is determined by the configuration setting | +| datafusion.execution.collect_statistics | false | Should DataFusion collect statistics after listing files | +| datafusion.execution.target_partitions | 0 | Number of partitions for query execution. Increasing partitions can increase concurrency. Defaults to the number of CPU cores on the system | +| datafusion.execution.time_zone | +00:00 | The default time zone Some functions, e.g. `EXTRACT(HOUR from SOME_TIME)`, shift the underlying datetime according to this time zone, and then extract the hour | +| datafusion.execution.parquet.enable_page_index | false | If true, uses parquet data page level metadata (Page Index) statistics to reduce the number of rows decoded. | +| datafusion.execution.parquet.pruning | true | If true, the parquet reader attempts to skip entire row groups based on the predicate in the query and the metadata (min/max values) stored in the parquet file | +| datafusion.execution.parquet.skip_metadata | true | If true, the parquet reader skip the optional embedded metadata that may be in the file Schema. This setting can help avoid schema conflicts when querying multiple parquet files with schemas containing compatible types but different metadata | +| datafusion.execution.parquet.metadata_size_hint | NULL | If specified, the parquet reader will try and fetch the last `size_hint` bytes of the parquet file optimistically. If not specified, two reads are required: One read to fetch the 8-byte parquet footer and another to fetch the metadata length encoded in the footer | +| datafusion.execution.parquet.pushdown_filters | false | If true, filter expressions are be applied during the parquet decoding operation to reduce the number of rows decoded | +| datafusion.execution.parquet.reorder_filters | false | If true, filter expressions evaluated during the parquet decoding operation will be reordered heuristically to minimize the cost of evaluation. If false, the filters are applied in the same order as written in the query | +| datafusion.optimizer.enable_round_robin_repartition | true | When set to true, the physical plan optimizer will try to add round robin repartitioning to increase parallelism to leverage more CPU cores | +| datafusion.optimizer.filter_null_join_keys | false | When set to true, the optimizer will insert filters before a join between a nullable and non-nullable column to filter out nulls on the nullable side. This filter can add additional overhead when the file format does not fully support predicate push down. | +| datafusion.optimizer.repartition_aggregations | true | Should DataFusion repartition data using the aggregate keys to execute aggregates in parallel using the provided `target_partitions` level | +| datafusion.optimizer.repartition_file_min_size | 10485760 | Minimum total files size in bytes to perform file scan repartitioning. | +| datafusion.optimizer.repartition_joins | true | Should DataFusion repartition data using the join keys to execute joins in parallel using the provided `target_partitions` level | +| datafusion.optimizer.allow_symmetric_joins_without_pruning | true | Should DataFusion allow symmetric hash joins for unbounded data sources even when its inputs do not have any ordering or filtering If the flag is not enabled, the SymmetricHashJoin operator will be unable to prune its internal buffers, resulting in certain join types - such as Full, Left, LeftAnti, LeftSemi, Right, RightAnti, and RightSemi - being produced only at the end of the execution. This is not typical in stream processing. Additionally, without proper design for long runner execution, all types of joins may encounter out-of-memory errors. | +| datafusion.optimizer.repartition_file_scans | true | When set to true, file groups will be repartitioned to achieve maximum parallelism. Currently supported only for Parquet format in which case multiple row groups from the same file may be read concurrently. If false then each row group is read serially, though different files may be read in parallel. | +| datafusion.optimizer.repartition_windows | true | Should DataFusion repartition data using the partitions keys to execute window functions in parallel using the provided `target_partitions` level | +| datafusion.optimizer.repartition_sorts | true | Should DataFusion execute sorts in a per-partition fashion and merge afterwards instead of coalescing first and sorting globally. With this flag is enabled, plans in the form below "SortExec: [a@0 ASC]", " CoalescePartitionsExec", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", would turn into the plan below which performs better in multithreaded environments "SortPreservingMergeExec: [a@0 ASC]", " SortExec: [a@0 ASC]", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", | +| datafusion.optimizer.skip_failed_rules | true | When set to true, the logical plan optimizer will produce warning messages if any optimization rules produce errors and then proceed to the next rule. When set to false, any rules that produce errors will cause the query to fail | +| datafusion.optimizer.max_passes | 3 | Number of times that the optimizer will attempt to optimize the plan | +| datafusion.optimizer.top_down_join_key_reordering | true | When set to true, the physical plan optimizer will run a top down process to reorder the join keys | +| datafusion.optimizer.prefer_hash_join | true | When set to true, the physical plan optimizer will prefer HashJoin over SortMergeJoin. HashJoin can work more efficiently than SortMergeJoin but consumes more memory | +| datafusion.optimizer.hash_join_single_partition_threshold | 1048576 | The maximum estimated size in bytes for one input side of a HashJoin will be collected into a single partition | +| datafusion.explain.logical_plan_only | false | When set to true, the explain statement will only print logical plans | +| datafusion.explain.physical_plan_only | false | When set to true, the explain statement will only print physical plans | +| datafusion.sql_parser.parse_float_as_decimal | false | When set to true, SQL parser will parse float as decimal type | +| datafusion.sql_parser.enable_ident_normalization | true | When set to true, SQL parser will normalize ident (convert ident to lowercase when not quoted) | From 7d32c1265c1d7ae38014999ff041eb834eb69dd0 Mon Sep 17 00:00:00 2001 From: metesynnada <100111937+metesynnada@users.noreply.github.com> Date: Mon, 3 Apr 2023 21:39:59 +0300 Subject: [PATCH 25/25] Update symmetric_hash_join.rs --- datafusion/core/src/physical_plan/joins/symmetric_hash_join.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/core/src/physical_plan/joins/symmetric_hash_join.rs b/datafusion/core/src/physical_plan/joins/symmetric_hash_join.rs index 9d04b5d9a96c..dafd0bfd4940 100644 --- a/datafusion/core/src/physical_plan/joins/symmetric_hash_join.rs +++ b/datafusion/core/src/physical_plan/joins/symmetric_hash_join.rs @@ -385,7 +385,7 @@ impl ExecutionPlan for SymmetricHashJoinExec { .iter() .map(|(l, r)| (Arc::new(l.clone()) as _, Arc::new(r.clone()) as _)) .unzip(); - // TODO: This will change when we extend collected executions. + // TODO: This will change when we extend collected executions. vec![ if self.left.output_partitioning().partition_count() == 1 { Distribution::SinglePartition