Skip to content

Commit fa0bc8e

Browse files
authored
planner: change more conditions that are always false to dual (#59199)
close #51446
1 parent c60c841 commit fa0bc8e

File tree

16 files changed

+143
-87
lines changed

16 files changed

+143
-87
lines changed

pkg/executor/prepared_test.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -55,7 +55,7 @@ func TestPreparedNullParam(t *testing.T) {
5555
ps := []*util.ProcessInfo{tkProcess}
5656
tk.Session().SetSessionManager(&testkit.MockSessionManager{PS: ps})
5757
tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)).Check(testkit.Rows(
58-
"TableDual_5 0.00 root rows:0"))
58+
"TableDual_6 0.00 root rows:0"))
5959
}
6060
}
6161

pkg/expression/util.go

Lines changed: 16 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -2162,3 +2162,19 @@ func binaryDurationWithMS(pos int, paramValues []byte,
21622162
pos += 4
21632163
return pos, fmt.Sprintf("%s.%06d", dur, microSecond)
21642164
}
2165+
2166+
// IsConstNull is used to check whether the expression is a constant null expression.
2167+
// For example, `1 > NULL` is a constant null expression.
2168+
// Now we just assume that the first argrument is a column,
2169+
// the second argument is a constant null.
2170+
func IsConstNull(expr Expression) bool {
2171+
if e, ok := expr.(*ScalarFunction); ok {
2172+
switch e.FuncName.L {
2173+
case ast.LT, ast.LE, ast.GT, ast.GE, ast.EQ, ast.NE:
2174+
if constExpr, ok := e.GetArgs()[1].(*Constant); ok && constExpr.Value.IsNull() && constExpr.DeferredExpr == nil {
2175+
return true
2176+
}
2177+
}
2178+
}
2179+
return false
2180+
}

pkg/planner/core/casetest/rule/BUILD.bazel

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -4,6 +4,7 @@ go_test(
44
name = "rule_test",
55
timeout = "short",
66
srcs = [
7+
"dual_test.go",
78
"main_test.go",
89
"rule_derive_topn_from_window_test.go",
910
"rule_inject_extra_projection_test.go",
@@ -13,7 +14,7 @@ go_test(
1314
],
1415
data = glob(["testdata/**"]),
1516
flaky = True,
16-
shard_count = 8,
17+
shard_count = 9,
1718
deps = [
1819
"//pkg/domain",
1920
"//pkg/expression",
Lines changed: 47 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,47 @@
1+
// Copyright 2023 PingCAP, Inc.
2+
//
3+
// Licensed under the Apache License, Version 2.0 (the "License");
4+
// you may not use this file except in compliance with the License.
5+
// You may obtain a copy of the License at
6+
//
7+
// http://www.apache.org/licenses/LICENSE-2.0
8+
//
9+
// Unless required by applicable law or agreed to in writing, software
10+
// distributed under the License is distributed on an "AS IS" BASIS,
11+
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12+
// See the License for the specific language governing permissions and
13+
// limitations under the License.
14+
15+
package rule
16+
17+
import (
18+
"testing"
19+
20+
"github.com/pingcap/tidb/pkg/testkit"
21+
)
22+
23+
func TestDual(t *testing.T) {
24+
store := testkit.CreateMockStore(t)
25+
tk := testkit.NewTestKit(t, store)
26+
tk.MustExec("use test")
27+
tk.MustExec("CREATE TABLE t (id INT PRIMARY KEY AUTO_INCREMENT,d INT);")
28+
tk.MustQuery("explain select a from (select d as a from t where d = 0) k where k.a = 5").Check(testkit.Rows(
29+
"TableDual_8 0.00 root rows:0"))
30+
tk.MustQuery("select a from (select d as a from t where d = 0) k where k.a = 5").Check(testkit.Rows())
31+
tk.MustQuery("explain select a from (select 1+2 as a from t where d = 0) k where k.a = 5").Check(testkit.Rows(
32+
"Projection_8 0.00 root 3->Column#3",
33+
"└─TableDual_9 0.00 root rows:0"))
34+
tk.MustQuery("select a from (select 1+2 as a from t where d = 0) k where k.a = 5").Check(testkit.Rows())
35+
tk.MustQuery("explain select * from t where d != null;").Check(testkit.Rows(
36+
"TableDual_6 0.00 root rows:0"))
37+
tk.MustQuery("explain select * from t where d > null;").Check(testkit.Rows(
38+
"TableDual_6 0.00 root rows:0"))
39+
tk.MustQuery("explain select * from t where d >= null;").Check(testkit.Rows(
40+
"TableDual_6 0.00 root rows:0"))
41+
tk.MustQuery("explain select * from t where d < null;").Check(testkit.Rows(
42+
"TableDual_6 0.00 root rows:0"))
43+
tk.MustQuery("explain select * from t where d <= null;").Check(testkit.Rows(
44+
"TableDual_6 0.00 root rows:0"))
45+
tk.MustQuery("explain select * from t where d = null;").Check(testkit.Rows(
46+
"TableDual_6 0.00 root rows:0"))
47+
}

pkg/planner/core/casetest/rule/testdata/outer2inner_out.json

Lines changed: 2 additions & 15 deletions
Original file line numberDiff line numberDiff line change
@@ -161,14 +161,7 @@
161161
{
162162
"SQL": "select * from t3 as t1 left join t3 as t2 on t1.c3 = t2.c3 where t2.b3 != NULL; -- self join",
163163
"Plan": [
164-
"Projection 0.00 root test.t3.a3, test.t3.b3, test.t3.c3, test.t3.a3, test.t3.b3, test.t3.c3",
165-
"└─HashJoin 0.00 root inner join, equal:[eq(test.t3.c3, test.t3.c3)]",
166-
" ├─TableReader(Build) 0.00 root data:Selection",
167-
" │ └─Selection 0.00 cop[tikv] ne(test.t3.b3, NULL), not(isnull(test.t3.c3))",
168-
" │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo",
169-
" └─TableReader(Probe) 9990.00 root data:Selection",
170-
" └─Selection 9990.00 cop[tikv] not(isnull(test.t3.c3))",
171-
" └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo"
164+
"TableDual 0.00 root rows:0"
172165
]
173166
},
174167
{
@@ -434,13 +427,7 @@
434427
{
435428
"SQL": "select * from t3 as t1 left join t3 as t2 on t1.c3 = t2.c3 where t1.b3 != NULL -- negative case with self join",
436429
"Plan": [
437-
"HashJoin 0.00 root left outer join, left side:TableReader, equal:[eq(test.t3.c3, test.t3.c3)]",
438-
"├─TableReader(Build) 0.00 root data:Selection",
439-
"│ └─Selection 0.00 cop[tikv] ne(test.t3.b3, NULL)",
440-
"│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo",
441-
"└─TableReader(Probe) 9990.00 root data:Selection",
442-
" └─Selection 9990.00 cop[tikv] not(isnull(test.t3.c3))",
443-
" └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo"
430+
"TableDual 0.00 root rows:0"
444431
]
445432
},
446433
{

pkg/planner/core/issuetest/main_test.go

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -30,6 +30,7 @@ func TestMain(m *testing.M) {
3030
testsetup.SetupForCommonTest()
3131
flag.Parse()
3232
opts := []goleak.Option{
33+
goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"),
3334
goleak.IgnoreTopFunction("github.com/golang/glog.(*fileSink).flushDaemon"),
3435
goleak.IgnoreTopFunction("github.com/bazelbuild/rules_go/go/tools/bzltestutil.RegisterTimeoutHandler.func1"),
3536
goleak.IgnoreTopFunction("github.com/lestrrat-go/httprc.runFetchWorker"),

pkg/planner/core/logical_plans_test.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -330,7 +330,7 @@ func TestAntiSemiJoinConstFalse(t *testing.T) {
330330
}{
331331
{
332332
sql: "select a from t t1 where not exists (select a from t t2 where t1.a = t2.a and t2.b = 1 and t2.b = 2)",
333-
best: "Join{DataScan(t1)->DataScan(t2)}(test.t.a,test.t.a)->Projection",
333+
best: "Join{DataScan(t1)->Dual}(test.t.a,test.t.a)->Projection",
334334
joinType: "anti semi join",
335335
},
336336
}

pkg/planner/core/operator/logicalop/BUILD.bazel

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -4,6 +4,7 @@ go_library(
44
name = "logicalop",
55
srcs = [
66
"base_logical_plan.go",
7+
"expression_util.go",
78
"hash64_equals_generated.go",
89
"logical_aggregation.go",
910
"logical_apply.go",
Lines changed: 52 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,52 @@
1+
// Copyright 2025 PingCAP, Inc.
2+
//
3+
// Licensed under the Apache License, Version 2.0 (the "License");
4+
// you may not use this file except in compliance with the License.
5+
// You may obtain a copy of the License at
6+
//
7+
// http://www.apache.org/licenses/LICENSE-2.0
8+
//
9+
// Unless required by applicable law or agreed to in writing, software
10+
// distributed under the License is distributed on an "AS IS" BASIS,
11+
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12+
// See the License for the specific language governing permissions and
13+
// limitations under the License.
14+
15+
package logicalop
16+
17+
import (
18+
"github.com/pingcap/tidb/pkg/expression"
19+
"github.com/pingcap/tidb/pkg/planner/core/base"
20+
)
21+
22+
// Conds2TableDual builds a LogicalTableDual if cond is constant false or null.
23+
func Conds2TableDual(p base.LogicalPlan, conds []expression.Expression) base.LogicalPlan {
24+
for _, cond := range conds {
25+
if expression.IsConstNull(cond) {
26+
if expression.MaybeOverOptimized4PlanCache(p.SCtx().GetExprCtx(), conds) {
27+
return nil
28+
}
29+
dual := LogicalTableDual{}.Init(p.SCtx(), p.QueryBlockOffset())
30+
dual.SetSchema(p.Schema())
31+
return dual
32+
}
33+
}
34+
if len(conds) != 1 {
35+
return nil
36+
}
37+
38+
con, ok := conds[0].(*expression.Constant)
39+
if !ok {
40+
return nil
41+
}
42+
sc := p.SCtx().GetSessionVars().StmtCtx
43+
if expression.MaybeOverOptimized4PlanCache(p.SCtx().GetExprCtx(), []expression.Expression{con}) {
44+
return nil
45+
}
46+
if isTrue, err := con.Value.ToBool(sc.TypeCtxOrDefault()); (err == nil && isTrue == 0) || con.Value.IsNull() {
47+
dual := LogicalTableDual{}.Init(p.SCtx(), p.QueryBlockOffset())
48+
dual.SetSchema(p.Schema())
49+
return dual
50+
}
51+
return nil
52+
}

pkg/planner/core/operator/logicalop/logical_datasource.go

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -155,6 +155,11 @@ func (ds *DataSource) PredicatePushDown(predicates []expression.Expression, opt
155155
// TODO: remove it to the place building logical plan
156156
predicates = utilfuncp.AddPrefix4ShardIndexes(ds, ds.SCtx(), predicates)
157157
ds.AllConds = predicates
158+
dual := Conds2TableDual(ds, ds.AllConds)
159+
if dual != nil {
160+
AppendTableDualTraceStep(ds, dual, predicates, opt)
161+
return nil, dual
162+
}
158163
ds.PushedDownConds, predicates = expression.PushDownExprs(util.GetPushDownCtx(ds.SCtx()), predicates, kv.UnSpecified)
159164
appendDataSourcePredicatePushDownTraceStep(ds, opt)
160165
return predicates, ds

0 commit comments

Comments
 (0)