Skip to content

Commit

Permalink
This is an automated cherry-pick of pingcap#47550
Browse files Browse the repository at this point in the history
Signed-off-by: ti-chi-bot <ti-community-prow-bot@tidb.io>
  • Loading branch information
King-Dylan authored and ti-chi-bot committed Jun 6, 2024
1 parent 739ff55 commit 2ec7023
Show file tree
Hide file tree
Showing 12 changed files with 3,208 additions and 39 deletions.
4 changes: 2 additions & 2 deletions executor/executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -4034,7 +4034,7 @@ func TestApplyCache(t *testing.T) {
tk.MustExec("create table t(a int);")
tk.MustExec("insert into t values (1),(1),(1),(1),(1),(1),(1),(1),(1);")
tk.MustExec("analyze table t;")
result := tk.MustQuery("explain analyze SELECT count(1) FROM (SELECT (SELECT min(a) FROM t as t2 WHERE t2.a > t1.a) AS a from t as t1) t;")
result := tk.MustQuery("explain analyze SELECT count(a) FROM (SELECT (SELECT min(a) FROM t as t2 WHERE t2.a > t1.a) AS a from t as t1) t;")
require.Contains(t, result.Rows()[1][0], "Apply")
var (
ind int
Expand All @@ -4054,7 +4054,7 @@ func TestApplyCache(t *testing.T) {
tk.MustExec("create table t(a int);")
tk.MustExec("insert into t values (1),(2),(3),(4),(5),(6),(7),(8),(9);")
tk.MustExec("analyze table t;")
result = tk.MustQuery("explain analyze SELECT count(1) FROM (SELECT (SELECT min(a) FROM t as t2 WHERE t2.a > t1.a) AS a from t as t1) t;")
result = tk.MustQuery("explain analyze SELECT count(a) FROM (SELECT (SELECT min(a) FROM t as t2 WHERE t2.a > t1.a) AS a from t as t1) t;")
require.Contains(t, result.Rows()[1][0], "Apply")
flag = false
value = (result.Rows()[1][5]).(string)
Expand Down
1,413 changes: 1,413 additions & 0 deletions pkg/planner/core/casetest/testdata/integration_suite_out.json

Large diffs are not rendered by default.

137 changes: 137 additions & 0 deletions pkg/planner/util/fixcontrol/get.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,137 @@
// Copyright 2023 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 fixcontrol

import (
"strconv"
"strings"
)

const (
// Fix44262 controls whether to allow to use dynamic-mode to access partitioning tables without global-stats (#44262).
Fix44262 uint64 = 44262
// Fix44389 controls whether to consider non-point ranges of some CNF item when building ranges.
Fix44389 uint64 = 44389
// Fix44830 controls whether to allow to cache Batch/PointGet from some complex scenarios.
// See #44830 for more details.
Fix44830 uint64 = 44830
// Fix44823 controls the maximum number of parameters for a query that can be cached in the Plan Cache.
Fix44823 uint64 = 44823
// Fix44855 controls whether to use a more accurate upper bound when estimating row count of index
// range scan under inner side of index join.
Fix44855 uint64 = 44855
// Fix45132 controls whether to use access range row count to determine access path on the Skyline pruning.
Fix45132 uint64 = 45132
// Fix45822 controls whether to eliminate apply operator.
Fix45822 uint64 = 45822
// Fix45798 controls whether to cache plans that access generated columns.
Fix45798 uint64 = 45798
// Fix46177 controls whether to explore enforced plans for DataSource if it has already found an unenforced plan.
Fix46177 uint64 = 46177
)

// GetStr fetches the given key from the fix control map as a string type.
func GetStr(fixControlMap map[uint64]string, key uint64) (value string, exists bool) {
if fixControlMap == nil {
return "", false
}
rawValue, ok := fixControlMap[key]
if !ok {
return "", false
}
return rawValue, true
}

// GetStrWithDefault fetches the given key from the fix control map as a string type,
// and a default value would be returned when fail to fetch the expected key.
func GetStrWithDefault(fixControlMap map[uint64]string, key uint64, defaultVal string) string {
value, exists := GetStr(fixControlMap, key)
if !exists {
return defaultVal
}
return value
}

// GetBool fetches the given key from the fix control map as a boolean type.
func GetBool(fixControlMap map[uint64]string, key uint64) (value bool, exists bool) {
if fixControlMap == nil {
return false, false
}
rawValue, ok := fixControlMap[key]
if !ok {
return false, false
}
// The same as TiDBOptOn in sessionctx/variable.
value = strings.EqualFold(rawValue, "ON") || rawValue == "1"
return value, true
}

// GetBoolWithDefault fetches the given key from the fix control map as a boolean type,
// and a default value would be returned when fail to fetch the expected key.
func GetBoolWithDefault(fixControlMap map[uint64]string, key uint64, defaultVal bool) bool {
value, exists := GetBool(fixControlMap, key)
if !exists {
return defaultVal
}
return value
}

// GetInt fetches the given key from the fix control map as an uint64 type.
func GetInt(fixControlMap map[uint64]string, key uint64) (value int64, exists bool, parseErr error) {
if fixControlMap == nil {
return 0, false, nil
}
rawValue, ok := fixControlMap[key]
if !ok {
return 0, false, nil
}
// The same as TidbOptInt64 in sessionctx/variable.
value, parseErr = strconv.ParseInt(rawValue, 10, 64)
return value, true, parseErr
}

// GetIntWithDefault fetches the given key from the fix control map as an uint64 type,
// // and a default value would be returned when fail to fetch the expected key.
func GetIntWithDefault(fixControlMap map[uint64]string, key uint64, defaultVal int64) int64 {
value, exists, err := GetInt(fixControlMap, key)
if !exists || err != nil {
return defaultVal
}
return value
}

// GetFloat fetches the given key from the fix control map as a float64 type.
func GetFloat(fixControlMap map[uint64]string, key uint64) (value float64, exists bool, parseErr error) {
if fixControlMap == nil {
return 0, false, nil
}
rawValue, ok := fixControlMap[key]
if !ok {
return 0, false, nil
}
// The same as tidbOptFloat64 in sessionctx/variable.
value, parseErr = strconv.ParseFloat(rawValue, 64)
return value, true, parseErr
}

// GetFloatWithDefault fetches the given key from the fix control map as a float64 type,
// // and a default value would be returned when fail to fetch the expected key.
func GetFloatWithDefault(fixControlMap map[uint64]string, key uint64, defaultVal float64) float64 {
value, exists, err := GetFloat(fixControlMap, key)
if !exists || err != nil {
return defaultVal
}
return value
}
2 changes: 1 addition & 1 deletion planner/cascades/optimize.go
Original file line number Diff line number Diff line change
Expand Up @@ -116,7 +116,7 @@ func (opt *Optimizer) FindBestPlan(sctx sessionctx.Context, logical plannercore.
}

func (*Optimizer) onPhasePreprocessing(_ sessionctx.Context, plan plannercore.LogicalPlan) (plannercore.LogicalPlan, error) {
err := plan.PruneColumns(plan.Schema().Columns, nil)
err := plan.PruneColumns(plan.Schema().Columns, nil, plan)
if err != nil {
return nil, err
}
Expand Down
29 changes: 29 additions & 0 deletions planner/core/physical_plan_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -506,6 +506,35 @@ func TestAggEliminator(t *testing.T) {
}
}

// Fix Issue #45822
func TestRuleColumnPruningLogicalApply(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
var input []string
var output []struct {
SQL string
Best string
}
planSuiteData := GetPlanSuiteData()
planSuiteData.LoadTestCases(t, &input, &output)
p := parser.New()
is := infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable()})
tk.MustExec("use test")
tk.MustExec("set @@tidb_opt_fix_control = '45822:ON';")
for i, tt := range input {
comment := fmt.Sprintf("input: %s", tt)
stmt, err := p.ParseOneStmt(tt, "", "")
require.NoError(t, err, comment)
p, _, err := planner.Optimize(context.TODO(), tk.Session(), stmt, is)
require.NoError(t, err)
testdata.OnRecord(func() {
output[i].SQL = tt
output[i].Best = core.ToString(p)
})
require.Equal(t, output[i].Best, core.ToString(p), fmt.Sprintf("input: %s", tt))
}
}

func TestINMJHint(t *testing.T) {
var (
input []string
Expand Down
6 changes: 3 additions & 3 deletions planner/core/plan.go
Original file line number Diff line number Diff line change
Expand Up @@ -258,7 +258,7 @@ type LogicalPlan interface {
PredicatePushDown([]expression.Expression, *logicalOptimizeOp) ([]expression.Expression, LogicalPlan)

// PruneColumns prunes the unused columns.
PruneColumns([]*expression.Column, *logicalOptimizeOp) error
PruneColumns([]*expression.Column, *logicalOptimizeOp, LogicalPlan) error

// findBestTask converts the logical plan to the physical plan. It's a new interface.
// It is called recursively from the parent to the children to create the result physical plan.
Expand Down Expand Up @@ -753,11 +753,11 @@ func (*baseLogicalPlan) ExtractCorrelatedCols() []*expression.CorrelatedColumn {
}

// PruneColumns implements LogicalPlan interface.
func (p *baseLogicalPlan) PruneColumns(parentUsedCols []*expression.Column, opt *logicalOptimizeOp) error {
func (p *baseLogicalPlan) PruneColumns(parentUsedCols []*expression.Column, opt *logicalOptimizeOp, _ LogicalPlan) error {
if len(p.children) == 0 {
return nil
}
return p.children[0].PruneColumns(parentUsedCols, opt)
return p.children[0].PruneColumns(parentUsedCols, opt, p)
}

// basePlan implements base Plan interface.
Expand Down
Loading

0 comments on commit 2ec7023

Please sign in to comment.