Skip to content

Commit

Permalink
Optimizer: Refactor and simplify outer to inner join conversion rule p…
Browse files Browse the repository at this point in the history
  • Loading branch information
ghazalfamilyusa authored and terry1purcell committed May 17, 2024
1 parent e401396 commit 324cdb8
Show file tree
Hide file tree
Showing 15 changed files with 895 additions and 54 deletions.
18 changes: 18 additions & 0 deletions pkg/expression/schema.go
Original file line number Diff line number Diff line change
Expand Up @@ -80,6 +80,24 @@ func (s *Schema) Clone() *Schema {
return schema
}

// ExprReferenceSchema checks if any column of this expression are from the schema.
func ExprReferenceSchema(expr Expression, schema *Schema) bool {
switch v := expr.(type) {
case *Column:
return schema.Contains(v)
case *ScalarFunction:
for _, arg := range v.GetArgs() {
if ExprReferenceSchema(arg, schema) {
return true
}
}
return false
case *CorrelatedColumn, *Constant:
return false
}
return false
}

// ExprFromSchema checks if all columns of this expression are from the same schema.
func ExprFromSchema(expr Expression, schema *Schema) bool {
switch v := expr.(type) {
Expand Down
1 change: 1 addition & 0 deletions pkg/planner/core/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -63,6 +63,7 @@ go_library(
"rule_join_reorder_dp.go",
"rule_join_reorder_greedy.go",
"rule_max_min_eliminate.go",
"rule_outer_to_inner_join.go",
"rule_partition_processor.go",
"rule_predicate_push_down.go",
"rule_predicate_simplification.go",
Expand Down
3 changes: 3 additions & 0 deletions pkg/planner/core/base/plan_base.go
Original file line number Diff line number Diff line change
Expand Up @@ -217,6 +217,9 @@ type LogicalPlan interface {
// interface definition should depend on concrete implementation type.
PushDownTopN(topN LogicalPlan, opt *optimizetrace.LogicalOptimizeOp) LogicalPlan

// ConvertOuterToInnerJoin converts outer joins if the unmatching rows are filtered.
ConvertOuterToInnerJoin(predicates []expression.Expression) LogicalPlan

// DeriveTopN derives an implicit TopN from a filter on row_number window function...
DeriveTopN(opt *optimizetrace.LogicalOptimizeOp) LogicalPlan

Expand Down
3 changes: 2 additions & 1 deletion pkg/planner/core/casetest/rule/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -8,10 +8,11 @@ go_test(
"rule_derive_topn_from_window_test.go",
"rule_inject_extra_projection_test.go",
"rule_join_reorder_test.go",
"rule_outer2inner_test.go",
],
data = glob(["testdata/**"]),
flaky = True,
shard_count = 5,
shard_count = 6,
deps = [
"//pkg/domain",
"//pkg/expression",
Expand Down
5 changes: 5 additions & 0 deletions pkg/planner/core/casetest/rule/main_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@ var testDataMap = make(testdata.BookKeeper)
func TestMain(m *testing.M) {
testsetup.SetupForCommonTest()
flag.Parse()
testDataMap.LoadTestSuiteData("testdata", "outer2inner")
testDataMap.LoadTestSuiteData("testdata", "derive_topn_from_window")
testDataMap.LoadTestSuiteData("testdata", "join_reorder_suite")
opts := []goleak.Option{
Expand All @@ -49,6 +50,10 @@ func TestMain(m *testing.M) {
goleak.VerifyTestMain(testmain.WrapTestingM(m, callback), opts...)
}

func GetOuter2InnerSuiteData() testdata.TestData {
return testDataMap["outer2inner"]
}

func GetDerivedTopNSuiteData() testdata.TestData {
return testDataMap["derive_topn_from_window"]
}
Expand Down
55 changes: 55 additions & 0 deletions pkg/planner/core/casetest/rule/rule_outer2inner_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,55 @@
// Copyright 2024 PingCAP, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

package rule

import (
"testing"

"github.com/pingcap/tidb/pkg/testkit"
"github.com/pingcap/tidb/pkg/testkit/testdata"
)

func TestOuter2Inner(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)

tk.MustExec("use test")
tk.MustExec("drop table if exists t")
tk.MustExec("create table t1(a1 int, b1 int, c1 int)")
tk.MustExec("create table t2(a2 int, b2 int, c2 int)")
tk.MustExec("create table t3(a3 int, b3 int, c3 int)")
tk.MustExec("create table t4(a4 int, b4 int, c4 int)")
tk.MustExec("create table ti(i int)")
tk.MustExec("CREATE TABLE lineitem (L_PARTKEY INTEGER ,L_QUANTITY DECIMAL(15,2),L_EXTENDEDPRICE DECIMAL(15,2))")
tk.MustExec("CREATE TABLE part(P_PARTKEY INTEGER,P_BRAND CHAR(10),P_CONTAINER CHAR(10))")
tk.MustExec("CREATE TABLE d (pk int, col_blob blob, col_blob_key blob, col_varchar_key varchar(1) , col_date date, col_int_key int)")
tk.MustExec("CREATE TABLE dd (pk int, col_blob blob, col_blob_key blob, col_date date, col_int_key int)")

var input Input
var output []struct {
SQL string
Plan []string
}
suiteData := GetOuter2InnerSuiteData()
suiteData.LoadTestCases(t, &input, &output)
for i, sql := range input {
plan := tk.MustQuery("explain format = 'brief' " + sql)
testdata.OnRecord(func() {
output[i].SQL = sql
output[i].Plan = testdata.ConvertRowsToStrings(plan.Rows())
})
plan.Check(testkit.Rows(output[i].Plan...))
}
}
43 changes: 43 additions & 0 deletions pkg/planner/core/casetest/rule/testdata/outer2inner_in.json
Original file line number Diff line number Diff line change
@@ -0,0 +1,43 @@
[
{
"name": "TestOuter2Inner",
"cases": [
"select * from t1 left outer join t2 on a1=a2 where b2 < 1 -- basic case of outer to inner join conversion",
"select * from t1 left outer join t2 on a1=a2 where b2 is not null -- basic case of not null",
"select * from t1 left outer join t2 on a1=a2 where not(b2 is null) -- another form of basic case of not null",
"select * from t1 left outer join t2 on a1=a2 where c2 = 5 OR b2 < 55 -- case with A OR B (Both A and B are null filtering)",
"select * from t1 left outer join t2 on a1=a2 where c2 = 5 AND b2 is null -- case with A AND B (A is null filtering and B is not)",
"select * from t2 left outer join t1 on a1=a2 where b1+b1 > 2; -- expression evaluates to UNKNOWN/FALSE",
"select * from t2 left outer join t1 on a1=a2 where coalesce(b1,2) > 2; -- false condition for b1=NULL",
"select * from t2 left outer join t1 on a1=a2 where true and b1 = 5; -- AND with one branch is null filtering",
"select * from t2 left outer join t1 on a1=a2 where false OR b1 = 5; -- OR with both branches are null filtering",
"select * from t3 as t1 left join t3 as t2 on t1.c3 = t2.c3 where t2.b3 != NULL; -- self join",
"select * from t1 ta left outer join (t1 tb left outer join t1 tc on tb.b1 = tc.b1) on ta.a1=tc.a1; -- nested join. On clause is null filtering on tc.",
"select * from t1 ta left outer join (t1 tb left outer join t1 tc on tb.b1 = tc.b1) on ta.a1=tc.a1 where tb.a1 > 5; -- nested join. On clause and WHERE clause are filters",
"select * from (t2 left join t1 on a1=a2) join t3 on b1=b3 -- on clause applied nested join",
"select * from ((t1 left join t2 on a1=a2) left join t3 on b2=b3) join t4 on b3=b4 -- nested and propagation of null filtering",
"select * from t1 right join t2 on a1=a2 where exists (select 1 from t3 where b1=b3) -- semi join is null filtering on the outer join",
"select sum(l_extendedprice) / 7.0 as avg_yearly from lineitem, part where p_partkey = l_partkey and p_brand = 'Brand#44' and p_container = 'WRAP PKG' and l_quantity < ( select 0.2 * avg(l_quantity) from lineitem where l_partkey = p_partkey) -- Q17 in TPCH. null filter on derived outer join",
"WITH cte AS ( SELECT alias1.col_date AS field1 FROM d AS alias1 LEFT JOIN dd AS alias2 ON alias1.col_blob_key=alias2.col_blob_key WHERE alias1.col_varchar_key IS NULL OR alias1.col_blob_key >= 'a') DELETE FROM outr1.*, outr2.* USING d AS outr1 LEFT OUTER JOIN dd AS outr2 ON (outr1.col_date=outr2.col_date) JOIN cte AS outrcte ON outr2.col_blob_key=outrcte.field1 -- nested complex case",
"with cte as (select count(a2) as cnt,b2-5 as b3 from t1 left outer join t2 on a1=a2 group by b3) select * from cte where b3 > 1 -- aggregate case.",
"select * from dd as outr1 WHERE outr1.col_blob IN (SELECT DISTINCT innr1.col_blob_key AS y FROM d AS innrcte left outer join dd AS innr1 ON innr1.pk = innrcte.col_date WHERE outr1.col_int_key > 6)",
"select * from t1 left outer join t2 on a1=a2 where b2 is null -- negative case with single predicate which is not null filtering",
"select * from t1 left outer join t2 on a1=a2 where c2 = 5 OR b2 is null -- negative case with A OR B (A is null filtering and B is not)",
"select * from t1 left outer join t2 on a1=a2 where not(b2 is not null) -- nested 'not' negative case",
"select * from t1 left outer join t2 on a1=a2 where not(not(b2 is null)) -- nested 'not' negative case",
"select * from t1 left outer join t2 on a1=a2 where b1 is not null -- negative case with condition on outer table.",
"select * from t2 left outer join t1 on a1=a2 where coalesce(b1,2) = 2; -- true condition for b1=NULL",
"select * from t2 left outer join t1 on a1=a2 where true OR b1 = 5; -- negative case with OR and one branch is TRUE",
"select * from t3 as t1 left join t3 as t2 on t1.c3 = t2.c3 where t1.b3 != NULL -- negative case with self join",
"select * from (t1 left outer join t2 on a1=a2) left outer join t3 on a2=a3 and b2 = 5 -- negative case. inner side is not a join",
"select * from t1 ta right outer join (t1 tb right outer join t1 tc on tb.b1 = tc.b1) on ta.a1=tc.a1; -- negative case. inner side is not a join",
"select * from t1 ta right outer join (t1 tb right outer join t1 tc on tb.b1 = tc.b1) on ta.a1=tc.a1 where tc.a1 > 5; -- negative case. inner side is not a join and WHERE clause on outer table",
"select * from (t2 left join t1 on a1=a2) join t3 on b2=b3 -- negative case, on clause on outer table in nested join",
"select t1.c1 in (select count(s.b1) from t1 s where s.a1 = t1.a1) from t1 -- subquery test that generates outer join and not converted",
"SELECT * FROM ti LEFT JOIN (SELECT i FROM ti WHERE FALSE) AS d1 ON ti.i = d1.i WHERE NOT EXISTS (SELECT 1 FROM ti AS inner_t1 WHERE i = d1.i) -- anti semi join",
"select count(*) from t1 where t1.a1+100 > ( select count(*) from t2 where t1.a1=t2.a2 and t1.b1=t2.b2) group by t1.b1 -- filter not filtering over derived outer join",
"with cte as (select count(a2) as cnt,ifnull(b2,5) as b2 from t1 left outer join t2 on a1=a2 group by b2) select * from cte where b2 > 1 -- non null filter on group by",
"with cte as (select count(a2) as cnt,ifnull(b2,5) as b2 from t1 left outer join t2 on a1=a2 group by b2) select * from cte where cnt > 1 -- filter on aggregates not applicable"
]
}
]
Loading

0 comments on commit 324cdb8

Please sign in to comment.