diff --git a/datafusion/optimizer/src/push_down_filter.rs b/datafusion/optimizer/src/push_down_filter.rs index 269ce2910074..acb7ba0fa757 100644 --- a/datafusion/optimizer/src/push_down_filter.rs +++ b/datafusion/optimizer/src/push_down_filter.rs @@ -36,9 +36,7 @@ use datafusion_expr::{ }; use crate::optimizer::ApplyOrder; -use crate::utils::{ - contain_all_columns, has_all_column_refs, is_restrict_null_predicate, -}; +use crate::utils::{has_all_column_refs, is_restrict_null_predicate}; use crate::{OptimizerConfig, OptimizerRule}; /// Optimizer rule for pushing (moving) filter expressions down in a plan so @@ -217,19 +215,19 @@ impl<'a> ColumnChecker<'a> { } /// Return true if the expression references only columns from the left side of the join - fn left_only(&mut self, column_refs: &HashSet<&Column>) -> bool { + fn is_left_only(&mut self, predicate: &Expr) -> bool { if self.left_columns.is_none() { self.left_columns = Some(schema_columns(self.left_schema)); } - contain_all_columns(column_refs, self.left_columns.as_ref().unwrap()) + has_all_column_refs(predicate, self.left_columns.as_ref().unwrap()) } /// Return true if the expression references only columns from the right side of the join - fn right_only(&mut self, column_refs: &HashSet<&Column>) -> bool { + fn is_right_only(&mut self, predicate: &Expr) -> bool { if self.right_columns.is_none() { self.right_columns = Some(schema_columns(self.right_schema)); } - contain_all_columns(column_refs, self.right_columns.as_ref().unwrap()) + has_all_column_refs(predicate, self.right_columns.as_ref().unwrap()) } } @@ -413,13 +411,10 @@ fn extract_or_clause(expr: &Expr, schema_columns: &HashSet) -> Option, - inferred_from_predicates: Vec, + inferred_join_predicates: Vec, mut join: Join, on_filter: Vec, - inferred_from_on_filter: Vec, ) -> Result> { - assert_ne!(join.join_type, JoinType::Full); - let is_inner_join = join.join_type == JoinType::Inner; // Get pushable predicates from current optimizer state let (left_preserved, right_preserved) = lr_is_preserved(join.join_type); @@ -435,24 +430,14 @@ fn push_down_all_join( let mut keep_predicates = vec![]; let mut join_conditions = vec![]; let mut checker = ColumnChecker::new(left_schema, right_schema); - for predicate in predicates { - let columns = predicate.column_refs(); - macro_rules! restrict_null { - () => {{ - let predicate_cloned = predicate.clone(); - let cols = columns.iter().cloned(); - is_restrict_null_predicate(predicate_cloned, cols).unwrap_or(false) - }}; - } - - if checker.left_only(&columns) && (left_preserved || restrict_null!()) { + if left_preserved && checker.is_left_only(&predicate) { left_push.push(predicate); - } else if checker.right_only(&columns) && (right_preserved || restrict_null!()) { + } else if right_preserved && checker.is_right_only(&predicate) { right_push.push(predicate); } else if is_inner_join && can_evaluate_as_join_condition(&predicate)? { - // Here we do not differ it is eq or non-eq predicate, ExtractEquijoinPredicate will - // extract the eq predicate and convert to the join on condition + // Here we do not differ it is eq or non-eq predicate, ExtractEquijoinPredicate will extract the eq predicate + // and convert to the join on condition join_conditions.push(predicate); } else { keep_predicates.push(predicate); @@ -460,13 +445,10 @@ fn push_down_all_join( } // For infer predicates, if they can not push through join, just drop them - // Because we check whether it is_restrict_null in the process of Infer, there is no need to - // check again - for predicate in inferred_from_predicates { - let columns = predicate.column_refs(); - if checker.left_only(&columns) { + for predicate in inferred_join_predicates { + if left_preserved && checker.is_left_only(&predicate) { left_push.push(predicate); - } else if checker.right_only(&columns) { + } else if right_preserved && checker.is_right_only(&predicate) { right_push.push(predicate); } } @@ -474,24 +456,15 @@ fn push_down_all_join( let mut on_filter_join_conditions = vec![]; let (on_left_preserved, on_right_preserved) = on_lr_is_preserved(join.join_type); - for on in on_filter { - let columns = on.column_refs(); - if on_left_preserved && checker.left_only(&columns) { - left_push.push(on) - } else if on_right_preserved && checker.right_only(&columns) { - right_push.push(on) - } else { - on_filter_join_conditions.push(on) - } - } - - // For infer predicates, if they can not push through join, just drop them - for on in inferred_from_on_filter { - let columns = on.column_refs(); - if on_left_preserved && checker.left_only(&columns) { - left_push.push(on) - } else if on_right_preserved && checker.right_only(&columns) { - right_push.push(on) + if !on_filter.is_empty() { + for on in on_filter { + if on_left_preserved && checker.is_left_only(&on) { + left_push.push(on) + } else if on_right_preserved && checker.is_right_only(&on) { + right_push.push(on) + } else { + on_filter_join_conditions.push(on) + } } } @@ -547,17 +520,6 @@ fn push_down_join( join: Join, parent_predicate: Option<&Expr>, ) -> Result> { - if matches!(join.join_type, JoinType::Full) { - let plan = LogicalPlan::Join(join); - return Ok(match parent_predicate { - Some(predicate) => Transformed::yes(LogicalPlan::Filter(Filter::try_new( - predicate.clone(), - Arc::new(plan), - )?)), - None => Transformed::no(plan), - }); - } - // Split the parent predicate into individual conjunctive parts. let predicates = parent_predicate .map_or_else(Vec::new, |pred| split_conjunction_owned(pred.clone())); @@ -569,24 +531,17 @@ fn push_down_join( .map_or_else(Vec::new, |filter| split_conjunction_owned(filter.clone())); // Are there any new join predicates that can be inferred from the filter expressions? - let (inferred_from_predicates, inferred_from_on_filter) = + let inferred_join_predicates = infer_join_predicates(&join, &predicates, &on_filters)?; if on_filters.is_empty() - && inferred_from_on_filter.is_empty() && predicates.is_empty() - && inferred_from_predicates.is_empty() + && inferred_join_predicates.is_empty() { return Ok(Transformed::no(LogicalPlan::Join(join))); } - push_down_all_join( - predicates, - inferred_from_predicates, - join, - on_filters, - inferred_from_on_filter, - ) + push_down_all_join(predicates, inferred_join_predicates, join, on_filters) } /// Extracts any equi-join join predicates from the given filter expressions. @@ -599,13 +554,11 @@ fn push_down_join( /// * `on_filters` filters from the join ON clause that have not already been /// identified as join predicates /// -/// # Return Value -/// A tuple of Expr Vec - (inferred_from_predicates, inferred_from_on_filters). fn infer_join_predicates( join: &Join, predicates: &[Expr], on_filters: &[Expr], -) -> Result<(Vec, Vec)> { +) -> Result> { // Only allow both side key is column. let join_col_keys = join .on @@ -626,7 +579,6 @@ fn infer_join_predicates( predicates, &mut inferred_predicates, )?; - let inferred_from_predicates = inferred_predicates.take_all(); infer_join_predicates_from_on_filters( &join_col_keys, @@ -634,9 +586,8 @@ fn infer_join_predicates( on_filters, &mut inferred_predicates, )?; - let inferred_from_on_filters = inferred_predicates.predicates; - Ok((inferred_from_predicates, inferred_from_on_filters)) + Ok(inferred_predicates.predicates) } /// Inferred predicates collector. @@ -660,12 +611,6 @@ impl InferredPredicates { } } - fn take_all(&mut self) -> Vec { - let mut temp = vec![]; - std::mem::swap(&mut self.predicates, &mut temp); - temp - } - fn try_build_predicate( &mut self, predicate: Expr, @@ -2165,10 +2110,11 @@ mod tests { // filter not duplicated nor pushed down - i.e. noop let expected = "\ - Left Join: Using test.a = test2.a\ - \n TableScan: test, full_filters=[test.a <= Int64(1)]\ - \n Projection: test2.a\ - \n TableScan: test2, full_filters=[test2.a <= Int64(1)]"; + Filter: test2.a <= Int64(1)\ + \n Left Join: Using test.a = test2.a\ + \n TableScan: test, full_filters=[test.a <= Int64(1)]\ + \n Projection: test2.a\ + \n TableScan: test2"; assert_optimized_plan_eq(plan, expected) } @@ -2202,10 +2148,11 @@ mod tests { // filter not duplicated nor pushed down - i.e. noop let expected = "\ - Right Join: Using test.a = test2.a\ - \n TableScan: test, full_filters=[test.a <= Int64(1)]\ - \n Projection: test2.a\ - \n TableScan: test2, full_filters=[test2.a <= Int64(1)]"; + Filter: test.a <= Int64(1)\ + \n Right Join: Using test.a = test2.a\ + \n TableScan: test\ + \n Projection: test2.a\ + \n TableScan: test2, full_filters=[test2.a <= Int64(1)]"; assert_optimized_plan_eq(plan, expected) } @@ -2243,7 +2190,7 @@ mod tests { Left Join: Using test.a = test2.a\ \n TableScan: test, full_filters=[test.a <= Int64(1)]\ \n Projection: test2.a\ - \n TableScan: test2, full_filters=[test2.a <= Int64(1)]"; + \n TableScan: test2"; assert_optimized_plan_eq(plan, expected) } @@ -2279,7 +2226,7 @@ mod tests { // filter sent to right side of join, not duplicated to the left let expected = "\ Right Join: Using test.a = test2.a\ - \n TableScan: test, full_filters=[test.a <= Int64(1)]\ + \n TableScan: test\ \n Projection: test2.a\ \n TableScan: test2, full_filters=[test2.a <= Int64(1)]"; assert_optimized_plan_eq(plan, expected) @@ -2328,47 +2275,6 @@ mod tests { assert_optimized_plan_eq(plan, expected) } - #[test] - fn join_with_non_restrict_null_predicate() -> Result<()> { - let table_scan = test_table_scan()?; - let left = LogicalPlanBuilder::from(table_scan) - .project(vec![col("a"), col("b"), col("c")])? - .build()?; - let right_table_scan = test_table_scan_with_name("test2")?; - let right = LogicalPlanBuilder::from(right_table_scan) - .project(vec![col("a"), col("b"), col("c")])? - .build()?; - let filter = col("test.b").is_null().and(col("test2.b").is_null()); - let plan = LogicalPlanBuilder::from(left) - .join( - right, - JoinType::Inner, - (vec![Column::from_name("a")], vec![Column::from_name("a")]), - None, - )? - .filter(filter)? - .build()?; - - // not part of the test, just good to know: - assert_eq!( - format!("{plan}"), - "Filter: test.b IS NULL AND test2.b IS NULL\ - \n Inner Join: test.a = test2.a\ - \n Projection: test.a, test.b, test.c\ - \n TableScan: test\ - \n Projection: test2.a, test2.b, test2.c\ - \n TableScan: test2" - ); - - let expected = "\ - Inner Join: test.a = test2.a\ - \n Projection: test.a, test.b, test.c\ - \n TableScan: test, full_filters=[test.b IS NULL]\ - \n Projection: test2.a, test2.b, test2.c\ - \n TableScan: test2, full_filters=[test2.b IS NULL]"; - assert_optimized_plan_eq(plan, expected) - } - /// join filter should be completely removed after pushdown #[test] fn join_filter_removed() -> Result<()> { @@ -2490,49 +2396,7 @@ mod tests { \n Projection: test.a, test.b, test.c\ \n TableScan: test\ \n Projection: test2.a, test2.b, test2.c\ - \n TableScan: test2, full_filters=[test2.c > UInt32(4), test2.a > UInt32(1)]"; - assert_optimized_plan_eq(plan, expected) - } - - #[test] - fn left_join_with_non_restrict_null_predicate() -> Result<()> { - let table_scan = test_table_scan()?; - let left = LogicalPlanBuilder::from(table_scan) - .project(vec![col("a"), col("b"), col("c")])? - .build()?; - let right_table_scan = test_table_scan_with_name("test2")?; - let right = LogicalPlanBuilder::from(right_table_scan) - .project(vec![col("a"), col("b"), col("c")])? - .build()?; - let filter = col("test.b").is_null().and(col("test2.b").is_null()); - let plan = LogicalPlanBuilder::from(left) - .join( - right, - JoinType::Left, - (vec![Column::from_name("a")], vec![Column::from_name("a")]), - None, - )? - .filter(filter)? - .build()?; - - // not part of the test, just good to know: - assert_eq!( - format!("{plan}"), - "Filter: test.b IS NULL AND test2.b IS NULL\ - \n Left Join: test.a = test2.a\ - \n Projection: test.a, test.b, test.c\ - \n TableScan: test\ - \n Projection: test2.a, test2.b, test2.c\ - \n TableScan: test2" - ); - - let expected = "\ - Filter: test2.b IS NULL\ - \n Left Join: test.a = test2.a\ - \n Projection: test.a, test.b, test.c\ - \n TableScan: test, full_filters=[test.b IS NULL]\ - \n Projection: test2.a, test2.b, test2.c\ - \n TableScan: test2"; + \n TableScan: test2, full_filters=[test2.c > UInt32(4)]"; assert_optimized_plan_eq(plan, expected) } @@ -2579,87 +2443,6 @@ mod tests { assert_optimized_plan_eq(plan, expected) } - #[test] - fn right_join_with_non_restrict_null_predicate() -> Result<()> { - let table_scan = test_table_scan()?; - let left = LogicalPlanBuilder::from(table_scan) - .project(vec![col("a"), col("b"), col("c")])? - .build()?; - let right_table_scan = test_table_scan_with_name("test2")?; - let right = LogicalPlanBuilder::from(right_table_scan) - .project(vec![col("a"), col("b"), col("c")])? - .build()?; - let filter = col("test.b").is_null().and(col("test2.b").is_null()); - let plan = LogicalPlanBuilder::from(left) - .join( - right, - JoinType::Right, - (vec![Column::from_name("a")], vec![Column::from_name("a")]), - None, - )? - .filter(filter)? - .build()?; - - // not part of the test, just good to know: - assert_eq!( - format!("{plan}"), - "Filter: test.b IS NULL AND test2.b IS NULL\ - \n Right Join: test.a = test2.a\ - \n Projection: test.a, test.b, test.c\ - \n TableScan: test\ - \n Projection: test2.a, test2.b, test2.c\ - \n TableScan: test2" - ); - - let expected = "\ - Filter: test.b IS NULL\ - \n Right Join: test.a = test2.a\ - \n Projection: test.a, test.b, test.c\ - \n TableScan: test\ - \n Projection: test2.a, test2.b, test2.c\ - \n TableScan: test2, full_filters=[test2.b IS NULL]"; - assert_optimized_plan_eq(plan, expected) - } - - #[test] - fn full_join() -> Result<()> { - let table_scan = test_table_scan()?; - let left = LogicalPlanBuilder::from(table_scan) - .project(vec![col("a"), col("b"), col("c")])? - .build()?; - let right_table_scan = test_table_scan_with_name("test2")?; - let right = LogicalPlanBuilder::from(right_table_scan) - .project(vec![col("a"), col("b"), col("c")])? - .build()?; - let filter = col("test.a") - .gt(lit(1u32)) - .and(col("test.b").lt(col("test2.b"))) - .and(col("test2.c").gt(lit(4u32))); - let plan = LogicalPlanBuilder::from(left) - .join( - right, - JoinType::Full, - (vec![Column::from_name("a")], vec![Column::from_name("a")]), - None, - )? - .filter(filter)? - .build()?; - - // not part of the test, just good to know: - assert_eq!( - format!("{plan}"), - "Filter: test.a > UInt32(1) AND test.b < test2.b AND test2.c > UInt32(4)\ - \n Full Join: test.a = test2.a\ - \n Projection: test.a, test.b, test.c\ - \n TableScan: test\ - \n Projection: test2.a, test2.b, test2.c\ - \n TableScan: test2" - ); - - let expected = &format!("{plan}"); - assert_optimized_plan_eq(plan, expected) - } - /// single table predicate parts of ON condition should not be pushed #[test] fn full_join_on_with_filter() -> Result<()> { @@ -3183,10 +2966,11 @@ Projection: a, b // Inferred the predicate `test1.a <= Int64(1)` and push it down to the left side. let expected = "\ - LeftSemi Join: test1.a = test2.a\ - \n TableScan: test1, full_filters=[test1.a <= Int64(1)]\ - \n Projection: test2.a, test2.b\ - \n TableScan: test2, full_filters=[test2.a <= Int64(1)]"; + Filter: test2.a <= Int64(1)\ + \n LeftSemi Join: test1.a = test2.a\ + \n TableScan: test1, full_filters=[test1.a <= Int64(1)]\ + \n Projection: test2.a, test2.b\ + \n TableScan: test2"; assert_optimized_plan_eq(plan, expected) } @@ -3263,10 +3047,11 @@ Projection: a, b // Inferred the predicate `test2.a <= Int64(1)` and push it down to the right side. let expected = "\ - RightSemi Join: test1.a = test2.a\ - \n TableScan: test1, full_filters=[test1.a <= Int64(1)]\ - \n Projection: test2.a, test2.b\ - \n TableScan: test2, full_filters=[test2.a <= Int64(1)]"; + Filter: test1.a <= Int64(1)\ + \n RightSemi Join: test1.a = test2.a\ + \n TableScan: test1\ + \n Projection: test2.a, test2.b\ + \n TableScan: test2, full_filters=[test2.a <= Int64(1)]"; assert_optimized_plan_eq(plan, expected) } @@ -3347,11 +3132,12 @@ Projection: a, b // For left anti, filter of the right side filter can be pushed down. let expected = "\ - LeftAnti Join: test1.a = test2.a\ - \n Projection: test1.a, test1.b\ - \n TableScan: test1, full_filters=[test1.a > UInt32(2)]\ - \n Projection: test2.a, test2.b\ - \n TableScan: test2, full_filters=[test2.a > UInt32(2)]"; + Filter: test2.a > UInt32(2)\ + \n LeftAnti Join: test1.a = test2.a\ + \n Projection: test1.a, test1.b\ + \n TableScan: test1, full_filters=[test1.a > UInt32(2)]\ + \n Projection: test2.a, test2.b\ + \n TableScan: test2"; assert_optimized_plan_eq(plan, expected) } @@ -3437,11 +3223,12 @@ Projection: a, b // For right anti, filter of the left side can be pushed down. let expected = "\ - RightAnti Join: test1.a = test2.a\ - \n Projection: test1.a, test1.b\ - \n TableScan: test1, full_filters=[test1.a > UInt32(2)]\ - \n Projection: test2.a, test2.b\ - \n TableScan: test2, full_filters=[test2.a > UInt32(2)]"; + Filter: test1.a > UInt32(2)\ + \n RightAnti Join: test1.a = test2.a\ + \n Projection: test1.a, test1.b\ + \n TableScan: test1\ + \n Projection: test2.a, test2.b\ + \n TableScan: test2, full_filters=[test2.a > UInt32(2)]"; assert_optimized_plan_eq(plan, expected) } diff --git a/datafusion/optimizer/src/utils.rs b/datafusion/optimizer/src/utils.rs index 9f8d7b7f97a0..9f325bc01b1d 100644 --- a/datafusion/optimizer/src/utils.rs +++ b/datafusion/optimizer/src/utils.rs @@ -79,14 +79,6 @@ pub fn optimize_children( /// Returns true if `expr` contains all columns in `schema_cols` pub(crate) fn has_all_column_refs(expr: &Expr, schema_cols: &HashSet) -> bool { let column_refs = expr.column_refs(); - contain_all_columns(&column_refs, schema_cols) -} - -/// Returns true if `column_refs` contains all columns in `schema_cols` -pub(crate) fn contain_all_columns( - column_refs: &HashSet<&Column>, - schema_cols: &HashSet, -) -> bool { // note can't use HashSet::intersect because of different types (owned vs References) schema_cols .iter() diff --git a/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt b/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt index 44c5ed0d2dab..cf897d628da5 100644 --- a/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt +++ b/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt @@ -89,25 +89,20 @@ logical_plan 02)--Projection: t2.a AS a2, t2.b 03)----RightSemi Join: t1.d = t2.d, t1.c = t2.c 04)------SubqueryAlias: t1 -05)--------Filter: annotated_data.d = Int32(3) -06)----------TableScan: annotated_data projection=[c, d], partial_filters=[annotated_data.d = Int32(3)] -07)------SubqueryAlias: t2 -08)--------Filter: annotated_data.d = Int32(3) -09)----------TableScan: annotated_data projection=[a, b, c, d], partial_filters=[annotated_data.d = Int32(3)] +05)--------TableScan: annotated_data projection=[c, d] +06)------SubqueryAlias: t2 +07)--------Filter: annotated_data.d = Int32(3) +08)----------TableScan: annotated_data projection=[a, b, c, d], partial_filters=[annotated_data.d = Int32(3)] physical_plan 01)SortPreservingMergeExec: [a2@0 ASC NULLS LAST, b@1 ASC NULLS LAST], fetch=10 02)--ProjectionExec: expr=[a@0 as a2, b@1 as b] 03)----CoalesceBatchesExec: target_batch_size=8192, fetch=10 04)------HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(d@1, d@3), (c@0, c@2)], projection=[a@0, b@1] -05)--------CoalescePartitionsExec -06)----------CoalesceBatchesExec: target_batch_size=8192 -07)------------FilterExec: d@1 = 3 -08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], has_header=true -10)--------CoalesceBatchesExec: target_batch_size=8192 -11)----------FilterExec: d@3 = 3 -12)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -13)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true +05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], has_header=true +06)--------CoalesceBatchesExec: target_batch_size=8192 +07)----------FilterExec: d@3 = 3 +08)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +09)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true # preserve_right_semi_join query II nosort diff --git a/datafusion/sqllogictest/test_files/push_down_filter_join.slt b/datafusion/sqllogictest/test_files/push_down_filter_join.slt deleted file mode 100644 index f687c542a683..000000000000 --- a/datafusion/sqllogictest/test_files/push_down_filter_join.slt +++ /dev/null @@ -1,612 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at - -# http://www.apache.org/licenses/LICENSE-2.0 - -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. - -# Contents -# Inner Join -# `WHERE` clause -# `ON` clause -# Left Outer Join -# `WHERE` clause -# `ON` clause -# Right Outer Join -# `WHERE` clause -# `ON` clause -# Full Outer Join -# `WHERE` clause -# `ON` clause - -# Create table t1 -statement ok -CREATE TABLE t1(t1_id INT, t1_name VARCHAR) AS VALUES -(11, 'a'), -(22, 'b'), -(33, 'c'), -(44, 'd'), -(77, 'e'), -(88, NULL), -(99, NULL) - -# Create table t2 -statement ok -CREATE TABLE t2(t2_id INT, t2_name VARCHAR) AS VALUES -(11, 'z'), -(22, NULL), -(44, 'x'), -(55, 'w'), -(99, 'u') - -# Inner Join - -## `WHERE` clause - -query ITIT -SELECT * FROM t1 INNER JOIN t2 ON t1.t1_id = t2.t2_id ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a 11 z -22 b 22 NULL -44 d 44 x -99 NULL 99 u - -query ITIT -SELECT * FROM t1 INNER JOIN t2 ON t1.t1_id = t2.t2_id WHERE t1.t1_name = 'a' ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a 11 z - -query ITIT -SELECT * FROM t1 INNER JOIN t2 ON t1.t1_id = t2.t2_id WHERE t2.t2_name = 'z' ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a 11 z - -query ITIT -SELECT * FROM t1 INNER JOIN t2 ON t1.t1_id = t2.t2_id WHERE t1.t1_name IS NULL ORDER BY t1_id, t1_name, t2_id, t2_name ----- -99 NULL 99 u - -query ITIT -SELECT * FROM t1 INNER JOIN t2 ON t1.t1_id = t2.t2_id WHERE t2.t2_name IS NULL ORDER BY t1_id, t1_name, t2_id, t2_name ----- -22 b 22 NULL - -query ITIT -SELECT * FROM t1 INNER JOIN t2 ON t1.t1_id = t2.t2_id WHERE t1.t1_id <= 22 ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a 11 z -22 b 22 NULL - -query ITIT -SELECT * FROM t1 INNER JOIN t2 ON t1.t1_id = t2.t2_id WHERE t2.t2_id <= 22 ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a 11 z -22 b 22 NULL - -query ITIT -SELECT * FROM t1 INNER JOIN t2 ON t1.t1_id = t2.t2_id WHERE t1.t1_id IS NULL ORDER BY t1_id, t1_name, t2_id, t2_name ----- - -query ITIT -SELECT * FROM t1 INNER JOIN t2 ON t1.t1_id = t2.t2_id WHERE t2.t2_id IS NULL ORDER BY t1_id, t1_name, t2_id, t2_name ----- - -## `ON` clause - -query ITIT -SELECT * FROM t1 INNER JOIN t2 ON t1.t1_id = t2.t2_id AND t1.t1_name = 'a' ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a 11 z - -query ITIT -SELECT * FROM t1 INNER JOIN t2 ON t1.t1_id = t2.t2_id AND t2.t2_name = 'z' ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a 11 z - -query ITIT -SELECT * FROM t1 INNER JOIN t2 ON t1.t1_id = t2.t2_id AND t1.t1_name IS NULL ORDER BY t1_id, t1_name, t2_id, t2_name ----- -99 NULL 99 u - -query ITIT -SELECT * FROM t1 INNER JOIN t2 ON t1.t1_id = t2.t2_id AND t2.t2_name IS NULL ORDER BY t1_id, t1_name, t2_id, t2_name ----- -22 b 22 NULL - -query ITIT -SELECT * FROM t1 INNER JOIN t2 ON t1.t1_id = t2.t2_id AND t1.t1_name IS NULL ORDER BY t1_id, t1_name, t2_id, t2_name ----- -99 NULL 99 u - -query ITIT -SELECT * FROM t1 INNER JOIN t2 ON t1.t1_id = t2.t2_id AND t2.t2_name IS NULL ORDER BY t1_id, t1_name, t2_id, t2_name ----- -22 b 22 NULL - -query ITIT -SELECT * FROM t1 INNER JOIN t2 ON t1.t1_id = t2.t2_id AND t1.t1_id <= 22 ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a 11 z -22 b 22 NULL - -query ITIT -SELECT * FROM t1 INNER JOIN t2 ON t1.t1_id = t2.t2_id AND t2.t2_id <= 22 ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a 11 z -22 b 22 NULL - -# Left Outer Join - -## `WHERE` clause - -query ITIT -SELECT * FROM t1 LEFT JOIN t2 ON t1.t1_id = t2.t2_id ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a 11 z -22 b 22 NULL -33 c NULL NULL -44 d 44 x -77 e NULL NULL -88 NULL NULL NULL -99 NULL 99 u - -query ITIT -SELECT * FROM t1 LEFT JOIN t2 ON t1.t1_id = t2.t2_id WHERE t1.t1_name = 'a' ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a 11 z - -query ITIT -SELECT * FROM t1 LEFT JOIN t2 ON t1.t1_id = t2.t2_id WHERE t2.t2_name = 'z' ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a 11 z - -query ITIT -SELECT * FROM t1 LEFT JOIN t2 ON t1.t1_id = t2.t2_id WHERE t1.t1_name IS NULL ORDER BY t1_id, t1_name, t2_id, t2_name ----- -88 NULL NULL NULL -99 NULL 99 u - -query ITIT -SELECT * FROM t1 LEFT JOIN t2 ON t1.t1_id = t2.t2_id WHERE t2.t2_name IS NULL ORDER BY t1_id, t1_name, t2_id, t2_name ----- -22 b 22 NULL -33 c NULL NULL -77 e NULL NULL -88 NULL NULL NULL - -query ITIT -SELECT * FROM t1 LEFT JOIN t2 ON t1.t1_id = t2.t2_id WHERE t1.t1_id <= 22 ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a 11 z -22 b 22 NULL - -query ITIT -SELECT * FROM t1 LEFT JOIN t2 ON t1.t1_id = t2.t2_id WHERE t2.t2_id <= 22 ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a 11 z -22 b 22 NULL - -query ITIT -SELECT * FROM t1 LEFT JOIN t2 ON t1.t1_id = t2.t2_id WHERE t1.t1_id IS NULL ORDER BY t1_id, t1_name, t2_id, t2_name ----- - -query ITIT -SELECT * FROM t1 LEFT JOIN t2 ON t1.t1_id = t2.t2_id WHERE t2.t2_id IS NULL ORDER BY t1_id, t1_name, t2_id, t2_name ----- -33 c NULL NULL -77 e NULL NULL -88 NULL NULL NULL - -## `ON` clause - -query ITIT -SELECT * FROM t1 LEFT JOIN t2 ON t1.t1_id = t2.t2_id AND t1.t1_name = 'a' ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a 11 z -22 b NULL NULL -33 c NULL NULL -44 d NULL NULL -77 e NULL NULL -88 NULL NULL NULL -99 NULL NULL NULL - -query ITIT -SELECT * FROM t1 LEFT JOIN t2 ON t1.t1_id = t2.t2_id AND t2.t2_name = 'z' ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a 11 z -22 b NULL NULL -33 c NULL NULL -44 d NULL NULL -77 e NULL NULL -88 NULL NULL NULL -99 NULL NULL NULL - -query ITIT -SELECT * FROM t1 LEFT JOIN t2 ON t1.t1_id = t2.t2_id AND t1.t1_name IS NULL ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a NULL NULL -22 b NULL NULL -33 c NULL NULL -44 d NULL NULL -77 e NULL NULL -88 NULL NULL NULL -99 NULL 99 u - -query ITIT -SELECT * FROM t1 LEFT JOIN t2 ON t1.t1_id = t2.t2_id AND t2.t2_name IS NULL ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a NULL NULL -22 b 22 NULL -33 c NULL NULL -44 d NULL NULL -77 e NULL NULL -88 NULL NULL NULL -99 NULL NULL NULL - -query ITIT -SELECT * FROM t1 LEFT JOIN t2 ON t1.t1_id = t2.t2_id AND t1.t1_name IS NULL ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a NULL NULL -22 b NULL NULL -33 c NULL NULL -44 d NULL NULL -77 e NULL NULL -88 NULL NULL NULL -99 NULL 99 u - -query ITIT -SELECT * FROM t1 LEFT JOIN t2 ON t1.t1_id = t2.t2_id AND t2.t2_name IS NULL ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a NULL NULL -22 b 22 NULL -33 c NULL NULL -44 d NULL NULL -77 e NULL NULL -88 NULL NULL NULL -99 NULL NULL NULL - -query ITIT -SELECT * FROM t1 LEFT JOIN t2 ON t1.t1_id = t2.t2_id AND t1.t1_id <= 22 ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a 11 z -22 b 22 NULL -33 c NULL NULL -44 d NULL NULL -77 e NULL NULL -88 NULL NULL NULL -99 NULL NULL NULL - -query ITIT -SELECT * FROM t1 LEFT JOIN t2 ON t1.t1_id = t2.t2_id AND t2.t2_id <= 22 ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a 11 z -22 b 22 NULL -33 c NULL NULL -44 d NULL NULL -77 e NULL NULL -88 NULL NULL NULL -99 NULL NULL NULL - -# Right Outer Join - -## `WHERE` clause - -query ITIT -SELECT * FROM t1 RIGHT JOIN t2 ON t1.t1_id = t2.t2_id ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a 11 z -22 b 22 NULL -44 d 44 x -99 NULL 99 u -NULL NULL 55 w - -query ITIT -SELECT * FROM t1 RIGHT JOIN t2 ON t1.t1_id = t2.t2_id WHERE t1.t1_name = 'a' ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a 11 z - -query ITIT -SELECT * FROM t1 RIGHT JOIN t2 ON t1.t1_id = t2.t2_id WHERE t2.t2_name = 'z' ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a 11 z - -query ITIT -SELECT * FROM t1 RIGHT JOIN t2 ON t1.t1_id = t2.t2_id WHERE t1.t1_name IS NULL ORDER BY t1_id, t1_name, t2_id, t2_name ----- -99 NULL 99 u -NULL NULL 55 w - -query ITIT -SELECT * FROM t1 RIGHT JOIN t2 ON t1.t1_id = t2.t2_id WHERE t2.t2_name IS NULL ORDER BY t1_id, t1_name, t2_id, t2_name ----- -22 b 22 NULL - -query ITIT -SELECT * FROM t1 RIGHT JOIN t2 ON t1.t1_id = t2.t2_id WHERE t1.t1_id <= 22 ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a 11 z -22 b 22 NULL - -query ITIT -SELECT * FROM t1 RIGHT JOIN t2 ON t1.t1_id = t2.t2_id WHERE t2.t2_id <= 22 ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a 11 z -22 b 22 NULL - -query ITIT -SELECT * FROM t1 RIGHT JOIN t2 ON t1.t1_id = t2.t2_id WHERE t1.t1_id IS NULL ORDER BY t1_id, t1_name, t2_id, t2_name ----- -NULL NULL 55 w - -query ITIT -SELECT * FROM t1 RIGHT JOIN t2 ON t1.t1_id = t2.t2_id WHERE t2.t2_id IS NULL ORDER BY t1_id, t1_name, t2_id, t2_name ----- - -## `ON` clause - -query ITIT -SELECT * FROM t1 RIGHT JOIN t2 ON t1.t1_id = t2.t2_id AND t1.t1_name = 'a' ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a 11 z -NULL NULL 22 NULL -NULL NULL 44 x -NULL NULL 55 w -NULL NULL 99 u - -query ITIT -SELECT * FROM t1 RIGHT JOIN t2 ON t1.t1_id = t2.t2_id AND t2.t2_name = 'z' ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a 11 z -NULL NULL 22 NULL -NULL NULL 44 x -NULL NULL 55 w -NULL NULL 99 u - -query ITIT -SELECT * FROM t1 RIGHT JOIN t2 ON t1.t1_id = t2.t2_id AND t1.t1_name IS NULL ORDER BY t1_id, t1_name, t2_id, t2_name ----- -99 NULL 99 u -NULL NULL 11 z -NULL NULL 22 NULL -NULL NULL 44 x -NULL NULL 55 w - -query ITIT -SELECT * FROM t1 RIGHT JOIN t2 ON t1.t1_id = t2.t2_id AND t2.t2_name IS NULL ORDER BY t1_id, t1_name, t2_id, t2_name ----- -22 b 22 NULL -NULL NULL 11 z -NULL NULL 44 x -NULL NULL 55 w -NULL NULL 99 u - -query ITIT -SELECT * FROM t1 RIGHT JOIN t2 ON t1.t1_id = t2.t2_id AND t1.t1_name IS NULL ORDER BY t1_id, t1_name, t2_id, t2_name ----- -99 NULL 99 u -NULL NULL 11 z -NULL NULL 22 NULL -NULL NULL 44 x -NULL NULL 55 w - -query ITIT -SELECT * FROM t1 RIGHT JOIN t2 ON t1.t1_id = t2.t2_id AND t2.t2_name IS NULL ORDER BY t1_id, t1_name, t2_id, t2_name ----- -22 b 22 NULL -NULL NULL 11 z -NULL NULL 44 x -NULL NULL 55 w -NULL NULL 99 u - -query ITIT -SELECT * FROM t1 RIGHT JOIN t2 ON t1.t1_id = t2.t2_id AND t1.t1_id <= 22 ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a 11 z -22 b 22 NULL -NULL NULL 44 x -NULL NULL 55 w -NULL NULL 99 u - -query ITIT -SELECT * FROM t1 RIGHT JOIN t2 ON t1.t1_id = t2.t2_id AND t2.t2_id <= 22 ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a 11 z -22 b 22 NULL -NULL NULL 44 x -NULL NULL 55 w -NULL NULL 99 u - -# Full Outer Join - -## `WHERE` clause - -query ITIT -SELECT * FROM t1 FULL JOIN t2 ON t1.t1_id = t2.t2_id ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a 11 z -22 b 22 NULL -33 c NULL NULL -44 d 44 x -77 e NULL NULL -88 NULL NULL NULL -99 NULL 99 u -NULL NULL 55 w - -query ITIT -SELECT * FROM t1 FULL JOIN t2 ON t1.t1_id = t2.t2_id WHERE t1.t1_name = 'a' ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a 11 z - -query ITIT -SELECT * FROM t1 FULL JOIN t2 ON t1.t1_id = t2.t2_id WHERE t2.t2_name = 'z' ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a 11 z - -query ITIT -SELECT * FROM t1 FULL JOIN t2 ON t1.t1_id = t2.t2_id WHERE t1.t1_name IS NULL ORDER BY t1_id, t1_name, t2_id, t2_name ----- -88 NULL NULL NULL -99 NULL 99 u -NULL NULL 55 w - -query ITIT -SELECT * FROM t1 FULL JOIN t2 ON t1.t1_id = t2.t2_id WHERE t2.t2_name IS NULL ORDER BY t1_id, t1_name, t2_id, t2_name ----- -22 b 22 NULL -33 c NULL NULL -77 e NULL NULL -88 NULL NULL NULL - -query ITIT -SELECT * FROM t1 FULL JOIN t2 ON t1.t1_id = t2.t2_id WHERE t1.t1_id <= 22 ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a 11 z -22 b 22 NULL - -query ITIT -SELECT * FROM t1 FULL JOIN t2 ON t1.t1_id = t2.t2_id WHERE t2.t2_id <= 22 ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a 11 z -22 b 22 NULL - -query ITIT -SELECT * FROM t1 FULL JOIN t2 ON t1.t1_id = t2.t2_id WHERE t1.t1_id IS NULL ORDER BY t1_id, t1_name, t2_id, t2_name ----- -NULL NULL 55 w - -query ITIT -SELECT * FROM t1 FULL JOIN t2 ON t1.t1_id = t2.t2_id WHERE t2.t2_id IS NULL ORDER BY t1_id, t1_name, t2_id, t2_name ----- -33 c NULL NULL -77 e NULL NULL -88 NULL NULL NULL - -## `ON` clause - -query ITIT -SELECT * FROM t1 FULL JOIN t2 ON t1.t1_id = t2.t2_id AND t1.t1_name = 'a' ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a 11 z -22 b NULL NULL -33 c NULL NULL -44 d NULL NULL -77 e NULL NULL -88 NULL NULL NULL -99 NULL NULL NULL -NULL NULL 22 NULL -NULL NULL 44 x -NULL NULL 55 w -NULL NULL 99 u - -query ITIT -SELECT * FROM t1 FULL JOIN t2 ON t1.t1_id = t2.t2_id AND t2.t2_name = 'z' ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a 11 z -22 b NULL NULL -33 c NULL NULL -44 d NULL NULL -77 e NULL NULL -88 NULL NULL NULL -99 NULL NULL NULL -NULL NULL 22 NULL -NULL NULL 44 x -NULL NULL 55 w -NULL NULL 99 u - -query ITIT -SELECT * FROM t1 FULL JOIN t2 ON t1.t1_id = t2.t2_id AND t1.t1_name IS NULL ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a NULL NULL -22 b NULL NULL -33 c NULL NULL -44 d NULL NULL -77 e NULL NULL -88 NULL NULL NULL -99 NULL 99 u -NULL NULL 11 z -NULL NULL 22 NULL -NULL NULL 44 x -NULL NULL 55 w - -query ITIT -SELECT * FROM t1 FULL JOIN t2 ON t1.t1_id = t2.t2_id AND t2.t2_name IS NULL ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a NULL NULL -22 b 22 NULL -33 c NULL NULL -44 d NULL NULL -77 e NULL NULL -88 NULL NULL NULL -99 NULL NULL NULL -NULL NULL 11 z -NULL NULL 44 x -NULL NULL 55 w -NULL NULL 99 u - -query ITIT -SELECT * FROM t1 FULL JOIN t2 ON t1.t1_id = t2.t2_id AND t1.t1_name IS NULL ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a NULL NULL -22 b NULL NULL -33 c NULL NULL -44 d NULL NULL -77 e NULL NULL -88 NULL NULL NULL -99 NULL 99 u -NULL NULL 11 z -NULL NULL 22 NULL -NULL NULL 44 x -NULL NULL 55 w - -query ITIT -SELECT * FROM t1 FULL JOIN t2 ON t1.t1_id = t2.t2_id AND t2.t2_name IS NULL ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a NULL NULL -22 b 22 NULL -33 c NULL NULL -44 d NULL NULL -77 e NULL NULL -88 NULL NULL NULL -99 NULL NULL NULL -NULL NULL 11 z -NULL NULL 44 x -NULL NULL 55 w -NULL NULL 99 u - -query ITIT -SELECT * FROM t1 FULL JOIN t2 ON t1.t1_id = t2.t2_id AND t1.t1_id <= 22 ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a 11 z -22 b 22 NULL -33 c NULL NULL -44 d NULL NULL -77 e NULL NULL -88 NULL NULL NULL -99 NULL NULL NULL -NULL NULL 44 x -NULL NULL 55 w -NULL NULL 99 u - -query ITIT -SELECT * FROM t1 FULL JOIN t2 ON t1.t1_id = t2.t2_id AND t2.t2_id <= 22 ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a 11 z -22 b 22 NULL -33 c NULL NULL -44 d NULL NULL -77 e NULL NULL -88 NULL NULL NULL -99 NULL NULL NULL -NULL NULL 44 x -NULL NULL 55 w -NULL NULL 99 u