From 68271e99eee87c3e648388214f3b9c79659a1be0 Mon Sep 17 00:00:00 2001 From: Yuanjia Zhang Date: Wed, 29 Nov 2023 00:20:49 +0800 Subject: [PATCH] planner: refactor some code of the binding package (#48959) ref pingcap/tidb#48875 --- pkg/bindinfo/BUILD.bazel | 4 +- pkg/bindinfo/capture.go | 197 +++++++++++++++++ pkg/bindinfo/capture_test.go | 38 ---- pkg/bindinfo/handle.go | 376 -------------------------------- pkg/bindinfo/handle_test.go | 43 ---- pkg/bindinfo/tests/BUILD.bazel | 3 +- pkg/bindinfo/tests/bind_test.go | 260 ---------------------- pkg/domain/domain.go | 27 +-- pkg/executor/bind.go | 6 +- pkg/planner/optimize.go | 30 +-- 10 files changed, 203 insertions(+), 781 deletions(-) create mode 100644 pkg/bindinfo/capture.go diff --git a/pkg/bindinfo/BUILD.bazel b/pkg/bindinfo/BUILD.bazel index fc3ba7b3b04bc..8170e1c701194 100644 --- a/pkg/bindinfo/BUILD.bazel +++ b/pkg/bindinfo/BUILD.bazel @@ -5,6 +5,7 @@ go_library( srcs = [ "bind_cache.go", "bind_record.go", + "capture.go", "handle.go", "session_handle.go", "stat.go", @@ -37,7 +38,6 @@ go_library( "//pkg/util/sqlexec", "//pkg/util/stmtsummary/v2:stmtsummary", "//pkg/util/table-filter", - "//pkg/util/timeutil", "@org_golang_x_exp//maps", "@org_uber_go_zap//:zap", ], @@ -57,7 +57,7 @@ go_test( embed = [":bindinfo"], flaky = True, race = "on", - shard_count = 41, + shard_count = 40, deps = [ "//pkg/bindinfo/internal", "//pkg/config", diff --git a/pkg/bindinfo/capture.go b/pkg/bindinfo/capture.go new file mode 100644 index 0000000000000..ba5751f3e4d35 --- /dev/null +++ b/pkg/bindinfo/capture.go @@ -0,0 +1,197 @@ +// 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 bindinfo + +import ( + "context" + "strconv" + "strings" + + "github.com/pingcap/tidb/pkg/kv" + "github.com/pingcap/tidb/pkg/parser" + "github.com/pingcap/tidb/pkg/parser/ast" + "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" + "github.com/pingcap/tidb/pkg/util/logutil" + utilparser "github.com/pingcap/tidb/pkg/util/parser" + "github.com/pingcap/tidb/pkg/util/sqlexec" + stmtsummaryv2 "github.com/pingcap/tidb/pkg/util/stmtsummary/v2" + tablefilter "github.com/pingcap/tidb/pkg/util/table-filter" + "go.uber.org/zap" +) + +type captureFilter struct { + frequency int64 + tables []tablefilter.Filter // `schema.table` + users map[string]struct{} + + fail bool + currentDB string +} + +func (cf *captureFilter) Enter(in ast.Node) (out ast.Node, skipChildren bool) { + if x, ok := in.(*ast.TableName); ok { + tblEntry := stmtctx.TableEntry{ + DB: x.Schema.L, + Table: x.Name.L, + } + if x.Schema.L == "" { + tblEntry.DB = cf.currentDB + } + for _, tableFilter := range cf.tables { + if tableFilter.MatchTable(tblEntry.DB, tblEntry.Table) { + cf.fail = true // some filter is matched + } + } + } + return in, cf.fail +} + +func (*captureFilter) Leave(in ast.Node) (out ast.Node, ok bool) { + return in, true +} + +func (cf *captureFilter) isEmpty() bool { + return len(cf.tables) == 0 && len(cf.users) == 0 +} + +// ParseCaptureTableFilter checks whether this filter is valid and parses it. +func ParseCaptureTableFilter(tableFilter string) (f tablefilter.Filter, valid bool) { + // forbid wildcards '!' and '@' for safety, + // please see https://github.com/pingcap/tidb-tools/tree/master/pkg/table-filter for more details. + tableFilter = strings.TrimLeft(tableFilter, " \t") + if tableFilter == "" { + return nil, false + } + if tableFilter[0] == '!' || tableFilter[0] == '@' { + return nil, false + } + var err error + f, err = tablefilter.Parse([]string{tableFilter}) + if err != nil { + return nil, false + } + return f, true +} + +func (h *BindHandle) extractCaptureFilterFromStorage() (filter *captureFilter) { + filter = &captureFilter{ + frequency: 1, + users: make(map[string]struct{}), + } + exec := h.sctx.Context.(sqlexec.RestrictedSQLExecutor) + ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnBindInfo) + // No need to acquire the session context lock for ExecRestrictedSQL, it + // uses another background session. + rows, _, err := exec.ExecRestrictedSQL(ctx, nil, `SELECT filter_type, filter_value FROM mysql.capture_plan_baselines_blacklist order by filter_type`) + if err != nil { + logutil.BgLogger().Warn("failed to load mysql.capture_plan_baselines_blacklist", zap.String("category", "sql-bind"), zap.Error(err)) + return + } + for _, row := range rows { + filterTp := strings.ToLower(row.GetString(0)) + valStr := strings.ToLower(row.GetString(1)) + switch filterTp { + case "table": + tfilter, valid := ParseCaptureTableFilter(valStr) + if !valid { + logutil.BgLogger().Warn("capture table filter is invalid, ignore it", zap.String("category", "sql-bind"), zap.String("filter_value", valStr)) + continue + } + filter.tables = append(filter.tables, tfilter) + case "user": + filter.users[valStr] = struct{}{} + case "frequency": + f, err := strconv.ParseInt(valStr, 10, 64) + if err != nil { + logutil.BgLogger().Warn("failed to parse frequency type value, ignore it", zap.String("category", "sql-bind"), zap.String("filter_value", valStr), zap.Error(err)) + continue + } + if f < 1 { + logutil.BgLogger().Warn("frequency threshold is less than 1, ignore it", zap.String("category", "sql-bind"), zap.Int64("frequency", f)) + continue + } + if f > filter.frequency { + filter.frequency = f + } + default: + logutil.BgLogger().Warn("unknown capture filter type, ignore it", zap.String("category", "sql-bind"), zap.String("filter_type", filterTp)) + } + } + return +} + +// CaptureBaselines is used to automatically capture plan baselines. +func (h *BindHandle) CaptureBaselines() { + parser4Capture := parser.New() + captureFilter := h.extractCaptureFilterFromStorage() + emptyCaptureFilter := captureFilter.isEmpty() + bindableStmts := stmtsummaryv2.GetMoreThanCntBindableStmt(captureFilter.frequency) + for _, bindableStmt := range bindableStmts { + stmt, err := parser4Capture.ParseOneStmt(bindableStmt.Query, bindableStmt.Charset, bindableStmt.Collation) + if err != nil { + logutil.BgLogger().Debug("parse SQL failed in baseline capture", zap.String("category", "sql-bind"), zap.String("SQL", bindableStmt.Query), zap.Error(err)) + continue + } + if insertStmt, ok := stmt.(*ast.InsertStmt); ok && insertStmt.Select == nil { + continue + } + if !emptyCaptureFilter { + captureFilter.fail = false + captureFilter.currentDB = bindableStmt.Schema + stmt.Accept(captureFilter) + if captureFilter.fail { + continue + } + + if len(captureFilter.users) > 0 { + filteredByUser := true + for user := range bindableStmt.Users { + if _, ok := captureFilter.users[user]; !ok { + filteredByUser = false // some user not in the black-list has processed this stmt + break + } + } + if filteredByUser { + continue + } + } + } + dbName := utilparser.GetDefaultDB(stmt, bindableStmt.Schema) + normalizedSQL, digest := parser.NormalizeDigest(utilparser.RestoreWithDefaultDB(stmt, dbName, bindableStmt.Query)) + if r := h.GetGlobalBinding(digest.String(), normalizedSQL, dbName); r != nil && r.HasAvailableBinding() { + continue + } + bindSQL := GenerateBindSQL(context.TODO(), stmt, bindableStmt.PlanHint, true, dbName) + if bindSQL == "" { + continue + } + h.sctx.Lock() + charset, collation := h.sctx.GetSessionVars().GetCharsetInfo() + h.sctx.Unlock() + binding := Binding{ + BindSQL: bindSQL, + Status: Enabled, + Charset: charset, + Collation: collation, + Source: Capture, + SQLDigest: digest.String(), + } + // We don't need to pass the `sctx` because the BindSQL has been validated already. + err = h.CreateGlobalBinding(nil, &BindRecord{OriginalSQL: normalizedSQL, Db: dbName, Bindings: []Binding{binding}}) + if err != nil { + logutil.BgLogger().Debug("create bind record failed in baseline capture", zap.String("category", "sql-bind"), zap.String("SQL", bindableStmt.Query), zap.Error(err)) + } + } +} diff --git a/pkg/bindinfo/capture_test.go b/pkg/bindinfo/capture_test.go index 9d4abe3d39822..2c75899d12f01 100644 --- a/pkg/bindinfo/capture_test.go +++ b/pkg/bindinfo/capture_test.go @@ -336,19 +336,6 @@ func TestBindingSource(t *testing.T) { bind := bindData.Bindings[0] require.Equal(t, bindinfo.Manual, bind.Source) - // Test Source for evolved sql - tk.MustExec("set @@tidb_evolve_plan_baselines=1") - tk.MustQuery("select * from t where a > 10") - bindHandle.SaveEvolveTasksToStore() - sql, sqlDigest = internal.UtilNormalizeWithDefaultDB(t, "select * from t where a > ?") - bindData = bindHandle.GetGlobalBinding(sqlDigest, sql, "test") - require.NotNil(t, bindData) - require.Equal(t, "select * from `test` . `t` where `a` > ?", bindData.OriginalSQL) - require.Len(t, bindData.Bindings, 2) - bind = bindData.Bindings[1] - require.Equal(t, bindinfo.Evolve, bind.Source) - tk.MustExec("set @@tidb_evolve_plan_baselines=0") - // Test Source for captured sqls stmtsummary.StmtSummaryByDigestMap.Clear() tk.MustExec("SET GLOBAL tidb_capture_plan_baselines = on") @@ -492,31 +479,6 @@ func TestIssue20417(t *testing.T) { require.Equal(t, "select * from `test` . `t` where `b` = ? and `c` = ?", rows[0][0]) require.Equal(t, "SELECT /*+ use_index(@`sel_1` `test`.`t` `idxb`), no_order_index(@`sel_1` `test`.`t` `idxb`)*/ * FROM `test`.`t` WHERE `b` = 2 AND `c` = 213124", rows[0][1]) tk.MustExec("SET GLOBAL tidb_capture_plan_baselines = off") - - // Test for evolve baseline - internal.UtilCleanBindingEnv(tk, dom) - tk.MustExec("set @@tidb_evolve_plan_baselines=1") - tk.MustExec("create global binding for select * from t WHERE c=3924541 using select /*+ use_index(@sel_1 test.t idxb) */ * from t WHERE c=3924541") - rows = tk.MustQuery("show global bindings").Rows() - require.Len(t, rows, 1) - require.Equal(t, "select * from `test` . `t` where `c` = ?", rows[0][0]) - require.Equal(t, "SELECT /*+ use_index(@`sel_1` `test`.`t` `idxb`)*/ * FROM `test`.`t` WHERE `c` = 3924541", rows[0][1]) - tk.MustExec("select /*+ use_index(t idxc)*/ * from t where c=3924541") - require.Equal(t, "t:idxb", tk.Session().GetSessionVars().StmtCtx.IndexNames[0]) - tk.MustExec("admin flush bindings") - rows = tk.MustQuery("show global bindings").Rows() - require.Len(t, rows, 2) - require.Equal(t, "select * from `test` . `t` where `c` = ?", rows[0][0]) - require.Equal(t, "SELECT /*+ use_index(@`sel_1` `test`.`t` `idxc`), no_order_index(@`sel_1` `test`.`t` `idxc`)*/ * FROM `test`.`t` WHERE `c` = 3924541", rows[0][1]) - require.Equal(t, "pending verify", rows[0][3]) - tk.MustExec("admin evolve bindings") - rows = tk.MustQuery("show global bindings").Rows() - require.Len(t, rows, 2) - require.Equal(t, "select * from `test` . `t` where `c` = ?", rows[0][0]) - require.Equal(t, "SELECT /*+ use_index(@`sel_1` `test`.`t` `idxc`), no_order_index(@`sel_1` `test`.`t` `idxc`)*/ * FROM `test`.`t` WHERE `c` = 3924541", rows[0][1]) - status := rows[0][3].(string) - require.True(t, status == bindinfo.Enabled || status == bindinfo.Rejected) - tk.MustExec("set @@tidb_evolve_plan_baselines=0") } func TestCaptureWithZeroSlowLogThreshold(t *testing.T) { diff --git a/pkg/bindinfo/handle.go b/pkg/bindinfo/handle.go index 5932ad93a9fab..57612962d9986 100644 --- a/pkg/bindinfo/handle.go +++ b/pkg/bindinfo/handle.go @@ -17,8 +17,6 @@ package bindinfo import ( "context" "fmt" - "runtime" - "strconv" "strings" "sync" "sync/atomic" @@ -32,7 +30,6 @@ import ( "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/sessionctx" - "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/types" driver "github.com/pingcap/tidb/pkg/types/parser_driver" @@ -42,9 +39,6 @@ import ( utilparser "github.com/pingcap/tidb/pkg/util/parser" "github.com/pingcap/tidb/pkg/util/sqlescape" "github.com/pingcap/tidb/pkg/util/sqlexec" - stmtsummaryv2 "github.com/pingcap/tidb/pkg/util/stmtsummary/v2" - tablefilter "github.com/pingcap/tidb/pkg/util/table-filter" - "github.com/pingcap/tidb/pkg/util/timeutil" "go.uber.org/zap" "golang.org/x/exp/maps" ) @@ -83,9 +77,6 @@ type BindHandle struct { // invalidBindRecordMap indicates the invalid bind records found during querying. // A record will be deleted from this map, after 2 bind-lease, after it is dropped from the kv. invalidBindRecordMap tmpBindRecordMap - - // pendingVerifyBindRecordMap indicates the pending verify bind records that found during query. - pendingVerifyBindRecordMap tmpBindRecordMap } // Lease influences the duration of loading bind info and handling invalid bind. @@ -131,11 +122,6 @@ func (h *BindHandle) Reset(ctx sessionctx.Context) { _, err := h.DropGlobalBinding(record.OriginalSQL, record.Db, &record.Bindings[0]) return err } - h.pendingVerifyBindRecordMap.Value.Store(make(map[string]*bindRecordUpdate)) - h.pendingVerifyBindRecordMap.flushFunc = func(record *BindRecord) error { - // BindSQL has already been validated when coming here, so we use nil sctx parameter. - return h.AddGlobalBinding(nil, record) - } variable.RegisterStatistics(h) } @@ -794,171 +780,6 @@ func copyBindRecordUpdateMap(oldMap map[string]*bindRecordUpdate) map[string]*bi return newMap } -type captureFilter struct { - frequency int64 - tables []tablefilter.Filter // `schema.table` - users map[string]struct{} - - fail bool - currentDB string -} - -func (cf *captureFilter) Enter(in ast.Node) (out ast.Node, skipChildren bool) { - if x, ok := in.(*ast.TableName); ok { - tblEntry := stmtctx.TableEntry{ - DB: x.Schema.L, - Table: x.Name.L, - } - if x.Schema.L == "" { - tblEntry.DB = cf.currentDB - } - for _, tableFilter := range cf.tables { - if tableFilter.MatchTable(tblEntry.DB, tblEntry.Table) { - cf.fail = true // some filter is matched - } - } - } - return in, cf.fail -} - -func (*captureFilter) Leave(in ast.Node) (out ast.Node, ok bool) { - return in, true -} - -func (cf *captureFilter) isEmpty() bool { - return len(cf.tables) == 0 && len(cf.users) == 0 -} - -// ParseCaptureTableFilter checks whether this filter is valid and parses it. -func ParseCaptureTableFilter(tableFilter string) (f tablefilter.Filter, valid bool) { - // forbid wildcards '!' and '@' for safety, - // please see https://github.com/pingcap/tidb-tools/tree/master/pkg/table-filter for more details. - tableFilter = strings.TrimLeft(tableFilter, " \t") - if tableFilter == "" { - return nil, false - } - if tableFilter[0] == '!' || tableFilter[0] == '@' { - return nil, false - } - var err error - f, err = tablefilter.Parse([]string{tableFilter}) - if err != nil { - return nil, false - } - return f, true -} - -func (h *BindHandle) extractCaptureFilterFromStorage() (filter *captureFilter) { - filter = &captureFilter{ - frequency: 1, - users: make(map[string]struct{}), - } - exec := h.sctx.Context.(sqlexec.RestrictedSQLExecutor) - ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnBindInfo) - // No need to acquire the session context lock for ExecRestrictedSQL, it - // uses another background session. - rows, _, err := exec.ExecRestrictedSQL(ctx, nil, `SELECT filter_type, filter_value FROM mysql.capture_plan_baselines_blacklist order by filter_type`) - if err != nil { - logutil.BgLogger().Warn("failed to load mysql.capture_plan_baselines_blacklist", zap.String("category", "sql-bind"), zap.Error(err)) - return - } - for _, row := range rows { - filterTp := strings.ToLower(row.GetString(0)) - valStr := strings.ToLower(row.GetString(1)) - switch filterTp { - case "table": - tfilter, valid := ParseCaptureTableFilter(valStr) - if !valid { - logutil.BgLogger().Warn("capture table filter is invalid, ignore it", zap.String("category", "sql-bind"), zap.String("filter_value", valStr)) - continue - } - filter.tables = append(filter.tables, tfilter) - case "user": - filter.users[valStr] = struct{}{} - case "frequency": - f, err := strconv.ParseInt(valStr, 10, 64) - if err != nil { - logutil.BgLogger().Warn("failed to parse frequency type value, ignore it", zap.String("category", "sql-bind"), zap.String("filter_value", valStr), zap.Error(err)) - continue - } - if f < 1 { - logutil.BgLogger().Warn("frequency threshold is less than 1, ignore it", zap.String("category", "sql-bind"), zap.Int64("frequency", f)) - continue - } - if f > filter.frequency { - filter.frequency = f - } - default: - logutil.BgLogger().Warn("unknown capture filter type, ignore it", zap.String("category", "sql-bind"), zap.String("filter_type", filterTp)) - } - } - return -} - -// CaptureBaselines is used to automatically capture plan baselines. -func (h *BindHandle) CaptureBaselines() { - parser4Capture := parser.New() - captureFilter := h.extractCaptureFilterFromStorage() - emptyCaptureFilter := captureFilter.isEmpty() - bindableStmts := stmtsummaryv2.GetMoreThanCntBindableStmt(captureFilter.frequency) - for _, bindableStmt := range bindableStmts { - stmt, err := parser4Capture.ParseOneStmt(bindableStmt.Query, bindableStmt.Charset, bindableStmt.Collation) - if err != nil { - logutil.BgLogger().Debug("parse SQL failed in baseline capture", zap.String("category", "sql-bind"), zap.String("SQL", bindableStmt.Query), zap.Error(err)) - continue - } - if insertStmt, ok := stmt.(*ast.InsertStmt); ok && insertStmt.Select == nil { - continue - } - if !emptyCaptureFilter { - captureFilter.fail = false - captureFilter.currentDB = bindableStmt.Schema - stmt.Accept(captureFilter) - if captureFilter.fail { - continue - } - - if len(captureFilter.users) > 0 { - filteredByUser := true - for user := range bindableStmt.Users { - if _, ok := captureFilter.users[user]; !ok { - filteredByUser = false // some user not in the black-list has processed this stmt - break - } - } - if filteredByUser { - continue - } - } - } - dbName := utilparser.GetDefaultDB(stmt, bindableStmt.Schema) - normalizedSQL, digest := parser.NormalizeDigest(utilparser.RestoreWithDefaultDB(stmt, dbName, bindableStmt.Query)) - if r := h.GetGlobalBinding(digest.String(), normalizedSQL, dbName); r != nil && r.HasAvailableBinding() { - continue - } - bindSQL := GenerateBindSQL(context.TODO(), stmt, bindableStmt.PlanHint, true, dbName) - if bindSQL == "" { - continue - } - h.sctx.Lock() - charset, collation := h.sctx.GetSessionVars().GetCharsetInfo() - h.sctx.Unlock() - binding := Binding{ - BindSQL: bindSQL, - Status: Enabled, - Charset: charset, - Collation: collation, - Source: Capture, - SQLDigest: digest.String(), - } - // We don't need to pass the `sctx` because the BindSQL has been validated already. - err = h.CreateGlobalBinding(nil, &BindRecord{OriginalSQL: normalizedSQL, Db: dbName, Bindings: []Binding{binding}}) - if err != nil { - logutil.BgLogger().Debug("create bind record failed in baseline capture", zap.String("category", "sql-bind"), zap.String("SQL", bindableStmt.Query), zap.Error(err)) - } - } -} - func getHintsForSQL(sctx sessionctx.Context, sql string) (string, error) { origVals := sctx.GetSessionVars().UsePlanBaselines sctx.GetSessionVars().UsePlanBaselines = false @@ -1072,201 +893,6 @@ func (*paramMarkerChecker) Leave(in ast.Node) (ast.Node, bool) { return in, true } -// AddEvolvePlanTask adds the evolve plan task into memory cache. It would be flushed to store periodically. -func (h *BindHandle) AddEvolvePlanTask(originalSQL, db string, binding Binding) { - br := &BindRecord{ - OriginalSQL: originalSQL, - Db: db, - Bindings: []Binding{binding}, - } - h.pendingVerifyBindRecordMap.Add(br) -} - -// SaveEvolveTasksToStore saves the evolve task into store. -func (h *BindHandle) SaveEvolveTasksToStore() { - h.pendingVerifyBindRecordMap.flushToStore() -} - -func getEvolveParameters(sctx sessionctx.Context) (time.Duration, time.Time, time.Time, error) { - ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnBindInfo) - rows, _, err := sctx.(sqlexec.RestrictedSQLExecutor).ExecRestrictedSQL( - ctx, - nil, - "SELECT variable_name, variable_value FROM mysql.global_variables WHERE variable_name IN (%?, %?, %?)", - variable.TiDBEvolvePlanTaskMaxTime, - variable.TiDBEvolvePlanTaskStartTime, - variable.TiDBEvolvePlanTaskEndTime, - ) - if err != nil { - return 0, time.Time{}, time.Time{}, err - } - maxTime, startTimeStr, endTimeStr := int64(variable.DefTiDBEvolvePlanTaskMaxTime), variable.DefTiDBEvolvePlanTaskStartTime, variable.DefAutoAnalyzeEndTime - for _, row := range rows { - switch row.GetString(0) { - case variable.TiDBEvolvePlanTaskMaxTime: - maxTime, err = strconv.ParseInt(row.GetString(1), 10, 64) - if err != nil { - return 0, time.Time{}, time.Time{}, err - } - case variable.TiDBEvolvePlanTaskStartTime: - startTimeStr = row.GetString(1) - case variable.TiDBEvolvePlanTaskEndTime: - endTimeStr = row.GetString(1) - } - } - startTime, err := time.ParseInLocation(variable.FullDayTimeFormat, startTimeStr, time.UTC) - if err != nil { - return 0, time.Time{}, time.Time{}, err - } - endTime, err := time.ParseInLocation(variable.FullDayTimeFormat, endTimeStr, time.UTC) - if err != nil { - return 0, time.Time{}, time.Time{}, err - } - return time.Duration(maxTime) * time.Second, startTime, endTime, nil -} - -const ( - // acceptFactor is the factor to decide should we accept the pending verified plan. - // A pending verified plan will be accepted if it performs at least `acceptFactor` times better than the accepted plans. - acceptFactor = 1.5 - // verifyTimeoutFactor is how long to wait to verify the pending plan. - // For debugging purposes it is useful to wait a few times longer than the current execution time so that - // an informative error can be written to the log. - verifyTimeoutFactor = 2.0 - // nextVerifyDuration is the duration that we will retry the rejected plans. - nextVerifyDuration = 7 * 24 * time.Hour -) - -func (h *BindHandle) getOnePendingVerifyJob() (originalSQL, db string, binding Binding) { - cache := h.bindInfo.Value.Load().(*bindCache) - for _, bindRecord := range cache.GetAllBindings() { - for _, bind := range bindRecord.Bindings { - if bind.Status == PendingVerify { - return bindRecord.OriginalSQL, bindRecord.Db, bind - } - if bind.Status != Rejected { - continue - } - dur, err := bind.SinceUpdateTime() - // Should not happen. - if err != nil { - continue - } - // Rejected and retry it now. - if dur > nextVerifyDuration { - return bindRecord.OriginalSQL, bindRecord.Db, bind - } - } - } - return "", "", Binding{} -} - -func (*BindHandle) getRunningDuration(sctx sessionctx.Context, db, sql string, maxTime time.Duration) (time.Duration, error) { - ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnBindInfo) - if db != "" { - _, err := sctx.(sqlexec.SQLExecutor).ExecuteInternal(ctx, "use %n", db) - if err != nil { - return 0, err - } - } - ctx, cancelFunc := context.WithCancel(ctx) - timer := time.NewTimer(maxTime) - defer timer.Stop() - resultChan := make(chan error) - startTime := time.Now() - go runSQL(ctx, sctx, sql, resultChan) - select { - case err := <-resultChan: - cancelFunc() - if err != nil { - return 0, err - } - return time.Since(startTime), nil - case <-timer.C: - cancelFunc() - logutil.BgLogger().Debug("plan verification timed out", zap.String("category", "sql-bind"), zap.Duration("timeElapsed", time.Since(startTime)), zap.String("query", sql)) - } - <-resultChan - return -1, nil -} - -func runSQL(ctx context.Context, sctx sessionctx.Context, sql string, resultChan chan<- error) { - defer func() { - if r := recover(); r != nil { - buf := make([]byte, 4096) - stackSize := runtime.Stack(buf, false) - buf = buf[:stackSize] - resultChan <- fmt.Errorf("run sql panicked: %v", string(buf)) - } - }() - rs, err := sctx.(sqlexec.SQLExecutor).ExecuteInternal(ctx, sql) - if err != nil { - if rs != nil { - terror.Call(rs.Close) - } - resultChan <- err - return - } - chk := rs.NewChunk(nil) - for { - err = rs.Next(ctx, chk) - if err != nil || chk.NumRows() == 0 { - break - } - } - terror.Call(rs.Close) - resultChan <- err -} - -// HandleEvolvePlanTask tries to evolve one plan task. -// It only processes one task at a time because we want each task to use the latest parameters. -func (h *BindHandle) HandleEvolvePlanTask(sctx sessionctx.Context, adminEvolve bool) error { - originalSQL, db, binding := h.getOnePendingVerifyJob() - if originalSQL == "" { - return nil - } - maxTime, startTime, endTime, err := getEvolveParameters(sctx) - if err != nil { - return err - } - if maxTime == 0 || (!timeutil.WithinDayTimePeriod(startTime, endTime, time.Now()) && !adminEvolve) { - return nil - } - sctx.GetSessionVars().UsePlanBaselines = true - currentPlanTime, err := h.getRunningDuration(sctx, db, binding.BindSQL, maxTime) - // If we just return the error to the caller, this job will be retried again and again and cause endless logs, - // since it is still in the bind record. Now we just drop it and if it is actually retryable, - // we will hope for that we can capture this evolve task again. - if err != nil { - _, err = h.DropGlobalBinding(originalSQL, db, &binding) - return err - } - // If the accepted plan timeouts, it is hard to decide the timeout for verify plan. - // Currently we simply mark the verify plan as `using` if it could run successfully within maxTime. - if currentPlanTime > 0 { - maxTime = time.Duration(float64(currentPlanTime) * verifyTimeoutFactor) - } - sctx.GetSessionVars().UsePlanBaselines = false - verifyPlanTime, err := h.getRunningDuration(sctx, db, binding.BindSQL, maxTime) - if err != nil { - _, err = h.DropGlobalBinding(originalSQL, db, &binding) - return err - } - if verifyPlanTime == -1 || (float64(verifyPlanTime)*acceptFactor > float64(currentPlanTime)) { - binding.Status = Rejected - digestText, _ := parser.NormalizeDigest(binding.BindSQL) // for log desensitization - logutil.BgLogger().Debug("new plan rejected", zap.String("category", "sql-bind"), - zap.Duration("currentPlanTime", currentPlanTime), - zap.Duration("verifyPlanTime", verifyPlanTime), - zap.String("digestText", digestText), - ) - } else { - binding.Status = Enabled - } - // We don't need to pass the `sctx` because the BindSQL has been validated already. - return h.AddGlobalBinding(nil, &BindRecord{OriginalSQL: originalSQL, Db: db, Bindings: []Binding{binding}}) -} - // Clear resets the bind handle. It is only used for test. func (h *BindHandle) Clear() { h.bindInfo.Lock() @@ -1274,13 +900,11 @@ func (h *BindHandle) Clear() { h.bindInfo.lastUpdateTime = types.ZeroTimestamp h.bindInfo.Unlock() h.invalidBindRecordMap.Store(make(map[string]*bindRecordUpdate)) - h.pendingVerifyBindRecordMap.Store(make(map[string]*bindRecordUpdate)) } // FlushGlobalBindings flushes the BindRecord in temp maps to storage and loads them into cache. func (h *BindHandle) FlushGlobalBindings() error { h.DropInvalidGlobalBinding() - h.SaveEvolveTasksToStore() return h.Update(false) } diff --git a/pkg/bindinfo/handle_test.go b/pkg/bindinfo/handle_test.go index 263e501445968..3be6f37728620 100644 --- a/pkg/bindinfo/handle_test.go +++ b/pkg/bindinfo/handle_test.go @@ -21,7 +21,6 @@ import ( "github.com/pingcap/tidb/pkg/bindinfo" "github.com/pingcap/tidb/pkg/bindinfo/internal" - "github.com/pingcap/tidb/pkg/config" "github.com/pingcap/tidb/pkg/metrics" "github.com/pingcap/tidb/pkg/parser" "github.com/pingcap/tidb/pkg/testkit" @@ -191,48 +190,6 @@ func TestBindParse(t *testing.T) { require.NotNil(t, err, "err %v", err) } -func TestEvolveInvalidBindings(t *testing.T) { - originalVal := config.CheckTableBeforeDrop - config.CheckTableBeforeDrop = true - defer func() { - config.CheckTableBeforeDrop = originalVal - }() - - store, dom := testkit.CreateMockStoreAndDomain(t) - - tk := testkit.NewTestKit(t, store) - - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(a int, b int, index idx_a(a))") - tk.MustExec("create global binding for select * from t where a > 10 using select /*+ USE_INDEX(t) */ * from t where a > 10") - // Manufacture a rejected binding by hacking mysql.bind_info. - tk.MustExec("insert into mysql.bind_info values('select * from test . t where a > ?', 'SELECT /*+ USE_INDEX(t,idx_a) */ * FROM test.t WHERE a > 10', 'test', 'rejected', '2000-01-01 09:00:00', '2000-01-01 09:00:00', '', '','" + - bindinfo.Manual + "', '', '')") - tk.MustQuery("select bind_sql, status from mysql.bind_info where source != 'builtin'").Sort().Check(testkit.Rows( - "SELECT /*+ USE_INDEX(`t` )*/ * FROM `test`.`t` WHERE `a` > 10 enabled", - "SELECT /*+ USE_INDEX(t,idx_a) */ * FROM test.t WHERE a > 10 rejected", - )) - // Reload cache from mysql.bind_info. - dom.BindHandle().Clear() - require.Nil(t, dom.BindHandle().Update(true)) - - tk.MustExec("alter table t drop index idx_a") - tk.MustExec("admin evolve bindings") - require.Nil(t, dom.BindHandle().Update(false)) - rows := tk.MustQuery("show global bindings").Sort().Rows() - require.Equal(t, 2, len(rows)) - // Make sure this "enabled" binding is not overrided. - require.Equal(t, "SELECT /*+ USE_INDEX(`t` )*/ * FROM `test`.`t` WHERE `a` > 10", rows[0][1]) - status := rows[0][3].(string) - require.True(t, status == bindinfo.Enabled) - require.Equal(t, "SELECT /*+ USE_INDEX(t,idx_a) */ * FROM test.t WHERE a > 10", rows[1][1]) - status = rows[1][3].(string) - require.True(t, status == bindinfo.Enabled || status == bindinfo.Rejected) - _, sqlDigestWithDB := parser.NormalizeDigest("select * from test.t where a > 10") // test sqlDigest if exists after add columns to mysql.bind_info - require.Equal(t, rows[0][9], sqlDigestWithDB.String()) -} - func TestSetBindingStatus(t *testing.T) { store := testkit.CreateMockStore(t) diff --git a/pkg/bindinfo/tests/BUILD.bazel b/pkg/bindinfo/tests/BUILD.bazel index baac332bd5fd0..113fa0f6bd8d2 100644 --- a/pkg/bindinfo/tests/BUILD.bazel +++ b/pkg/bindinfo/tests/BUILD.bazel @@ -9,11 +9,10 @@ go_test( ], flaky = True, race = "on", - shard_count = 20, + shard_count = 13, deps = [ "//pkg/bindinfo", "//pkg/bindinfo/internal", - "//pkg/config", "//pkg/domain", "//pkg/parser", "//pkg/parser/model", diff --git a/pkg/bindinfo/tests/bind_test.go b/pkg/bindinfo/tests/bind_test.go index 8e5e2d5ad6022..1e6e48c43970e 100644 --- a/pkg/bindinfo/tests/bind_test.go +++ b/pkg/bindinfo/tests/bind_test.go @@ -22,7 +22,6 @@ import ( "github.com/pingcap/tidb/pkg/bindinfo" "github.com/pingcap/tidb/pkg/bindinfo/internal" - "github.com/pingcap/tidb/pkg/config" "github.com/pingcap/tidb/pkg/domain" "github.com/pingcap/tidb/pkg/parser" "github.com/pingcap/tidb/pkg/parser/model" @@ -459,129 +458,6 @@ func TestErrorBind(t *testing.T) { require.Equal(t, 0, chk.NumRows()) } -func TestDMLEvolveBaselines(t *testing.T) { - originalVal := config.CheckTableBeforeDrop - config.CheckTableBeforeDrop = true - defer func() { - config.CheckTableBeforeDrop = originalVal - }() - - store := testkit.CreateMockStore(t) - - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(a int, b int, c int, index idx_b(b), index idx_c(c))") - tk.MustExec("insert into t values (1,1,1), (2,2,2), (3,3,3), (4,4,4), (5,5,5)") - tk.MustExec("analyze table t") - tk.MustExec("set @@tidb_evolve_plan_baselines=1") - - tk.MustExec("create global binding for delete from t where b = 1 and c > 1 using delete /*+ use_index(t,idx_c) */ from t where b = 1 and c > 1") - rows := tk.MustQuery("show global bindings").Rows() - require.Len(t, rows, 1) - tk.MustExec("delete /*+ use_index(t,idx_b) */ from t where b = 2 and c > 1") - require.Equal(t, "t:idx_c", tk.Session().GetSessionVars().StmtCtx.IndexNames[0]) - tk.MustExec("admin flush bindings") - rows = tk.MustQuery("show global bindings").Rows() - require.Len(t, rows, 1) - tk.MustExec("admin evolve bindings") - rows = tk.MustQuery("show global bindings").Rows() - require.Len(t, rows, 1) - - tk.MustExec("create global binding for update t set a = 1 where b = 1 and c > 1 using update /*+ use_index(t,idx_c) */ t set a = 1 where b = 1 and c > 1") - rows = tk.MustQuery("show global bindings").Rows() - require.Len(t, rows, 2) - tk.MustExec("update /*+ use_index(t,idx_b) */ t set a = 2 where b = 2 and c > 1") - require.Equal(t, "t:idx_c", tk.Session().GetSessionVars().StmtCtx.IndexNames[0]) - tk.MustExec("admin flush bindings") - rows = tk.MustQuery("show global bindings").Rows() - require.Len(t, rows, 2) - tk.MustExec("admin evolve bindings") - rows = tk.MustQuery("show global bindings").Rows() - require.Len(t, rows, 2) - - tk.MustExec("create table t1 like t") - tk.MustExec("create global binding for insert into t1 select * from t where t.b = 1 and t.c > 1 using insert into t1 select /*+ use_index(t,idx_c) */ * from t where t.b = 1 and t.c > 1") - rows = tk.MustQuery("show global bindings").Rows() - require.Len(t, rows, 3) - tk.MustExec("insert into t1 select /*+ use_index(t,idx_b) */ * from t where t.b = 2 and t.c > 2") - require.Equal(t, "t:idx_c", tk.Session().GetSessionVars().StmtCtx.IndexNames[0]) - tk.MustExec("admin flush bindings") - rows = tk.MustQuery("show global bindings").Rows() - require.Len(t, rows, 3) - tk.MustExec("admin evolve bindings") - rows = tk.MustQuery("show global bindings").Rows() - require.Len(t, rows, 3) - - tk.MustExec("create global binding for replace into t1 select * from t where t.b = 1 and t.c > 1 using replace into t1 select /*+ use_index(t,idx_c) */ * from t where t.b = 1 and t.c > 1") - rows = tk.MustQuery("show global bindings").Rows() - require.Len(t, rows, 4) - tk.MustExec("replace into t1 select /*+ use_index(t,idx_b) */ * from t where t.b = 2 and t.c > 2") - require.Equal(t, "t:idx_c", tk.Session().GetSessionVars().StmtCtx.IndexNames[0]) - tk.MustExec("admin flush bindings") - rows = tk.MustQuery("show global bindings").Rows() - require.Len(t, rows, 4) - tk.MustExec("admin evolve bindings") - rows = tk.MustQuery("show global bindings").Rows() - require.Len(t, rows, 4) -} - -func TestAddEvolveTasks(t *testing.T) { - originalVal := config.CheckTableBeforeDrop - config.CheckTableBeforeDrop = true - defer func() { - config.CheckTableBeforeDrop = originalVal - }() - - store := testkit.CreateMockStore(t) - - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(a int, b int, c int, index idx_a(a), index idx_b(b), index idx_c(c))") - tk.MustExec("insert into t values (1,1,1), (2,2,2), (3,3,3), (4,4,4), (5,5,5)") - tk.MustExec("analyze table t") - tk.MustExec("create global binding for select * from t where a >= 1 and b >= 1 and c = 0 using select * from t use index(idx_a) where a >= 1 and b >= 1 and c = 0") - tk.MustExec("set @@tidb_evolve_plan_baselines=1") - // It cannot choose table path although it has lowest cost. - tk.MustQuery("select * from t where a >= 4 and b >= 1 and c = 0") - require.Equal(t, "t:idx_a", tk.Session().GetSessionVars().StmtCtx.IndexNames[0]) - tk.MustExec("admin flush bindings") - rows := tk.MustQuery("show global bindings").Rows() - require.Len(t, rows, 2) - require.Equal(t, "SELECT /*+ use_index(@`sel_1` `test`.`t` )*/ * FROM `test`.`t` WHERE `a` >= 4 AND `b` >= 1 AND `c` = 0", rows[0][1]) - require.Equal(t, "pending verify", rows[0][3]) - tk.MustExec("admin evolve bindings") - rows = tk.MustQuery("show global bindings").Rows() - require.Len(t, rows, 2) - require.Equal(t, "SELECT /*+ use_index(@`sel_1` `test`.`t` )*/ * FROM `test`.`t` WHERE `a` >= 4 AND `b` >= 1 AND `c` = 0", rows[0][1]) - status := rows[0][3].(string) - require.True(t, status == bindinfo.Enabled || status == bindinfo.Rejected) -} - -func TestRuntimeHintsInEvolveTasks(t *testing.T) { - originalVal := config.CheckTableBeforeDrop - config.CheckTableBeforeDrop = true - defer func() { - config.CheckTableBeforeDrop = originalVal - }() - - store := testkit.CreateMockStore(t) - - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.MustExec("set @@tidb_evolve_plan_baselines=1") - tk.MustExec("create table t(a int, b int, c int, index idx_a(a), index idx_b(b), index idx_c(c))") - - tk.MustExec("create global binding for select * from t where a >= 1 and b >= 1 and c = 0 using select * from t use index(idx_a) where a >= 1 and b >= 1 and c = 0") - tk.MustQuery("select /*+ MAX_EXECUTION_TIME(5000), SET_VAR(TIKV_CLIENT_READ_TIMEOUT=20) */ * from t where a >= 4 and b >= 1 and c = 0") - tk.MustExec("admin flush bindings") - rows := tk.MustQuery("show global bindings").Rows() - require.Len(t, rows, 2) - require.Equal(t, "SELECT /*+ use_index(@`sel_1` `test`.`t` `idx_c`), no_order_index(@`sel_1` `test`.`t` `idx_c`), max_execution_time(5000), set_var(tikv_client_read_timeout = 20)*/ * FROM `test`.`t` WHERE `a` >= 4 AND `b` >= 1 AND `c` = 0", rows[0][1]) -} - func TestStmtHints(t *testing.T) { store := testkit.CreateMockStore(t) @@ -601,36 +477,6 @@ func TestStmtHints(t *testing.T) { //require.Equal(t, uint64(0), tk.Session().GetSessionVars().GetTiKVClientReadTimeout()) } -func TestHintsSetEvolveTask(t *testing.T) { - originalVal := config.CheckTableBeforeDrop - config.CheckTableBeforeDrop = true - defer func() { - config.CheckTableBeforeDrop = originalVal - }() - - store, dom := testkit.CreateMockStoreAndDomain(t) - - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(a int, index idx_a(a))") - tk.MustExec("create global binding for select * from t where a > 10 using select * from t ignore index(idx_a) where a > 10") - tk.MustExec("set @@tidb_evolve_plan_baselines=1") - tk.MustQuery("select * from t use index(idx_a) where a > 0") - bindHandle := dom.BindHandle() - bindHandle.SaveEvolveTasksToStore() - // Verify the added Binding for evolution contains valid ID and Hint, otherwise, panic may happen. - sql, hash := internal.UtilNormalizeWithDefaultDB(t, "select * from t where a > ?") - bindData := bindHandle.GetGlobalBinding(hash, sql, "test") - require.NotNil(t, bindData) - require.Equal(t, "select * from `test` . `t` where `a` > ?", bindData.OriginalSQL) - require.Len(t, bindData.Bindings, 2) - bind := bindData.Bindings[1] - require.Equal(t, bindinfo.PendingVerify, bind.Status) - require.NotEqual(t, "", bind.ID) - require.NotNil(t, bind.Hint) -} - func TestHintsSetID(t *testing.T) { store, dom := testkit.CreateMockStoreAndDomain(t) @@ -697,57 +543,6 @@ func TestHintsSetID(t *testing.T) { require.Equal(t, "", bind.ID) } -func TestNotEvolvePlanForReadStorageHint(t *testing.T) { - originalVal := config.CheckTableBeforeDrop - config.CheckTableBeforeDrop = true - defer func() { - config.CheckTableBeforeDrop = originalVal - }() - - store := testkit.CreateMockStore(t) - - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("set tidb_cost_model_version=2") - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(a int, b int, index idx_a(a), index idx_b(b))") - tk.MustExec("insert into t values (1,1), (2,2), (3,3), (4,4), (5,5), (6,6), (7,7), (8,8), (9,9), (10,10)") - tk.MustExec("analyze table t") - // Create virtual tiflash replica info. - dom := domain.GetDomain(tk.Session()) - is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tblInfo := range db.Tables { - if tblInfo.Name.L == "t" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } - - // Make sure the best plan of the SQL is use TiKV index. - tk.MustExec("set @@session.tidb_executor_concurrency = 4; set @@tidb_allow_mpp=0;") - rows := tk.MustQuery("explain select * from t where a >= 11 and b >= 11").Rows() - require.Equal(t, "cop[tikv]", fmt.Sprintf("%v", rows[len(rows)-1][2])) - tk.MustExec("set @@tidb_allow_mpp=1") - - tk.MustExec("create global binding for select * from t where a >= 1 and b >= 1 using select /*+ read_from_storage(tiflash[t]) */ * from t where a >= 1 and b >= 1") - tk.MustExec("set @@tidb_evolve_plan_baselines=1") - - // Even if index of TiKV has lower cost, it chooses TiFlash. - rows = tk.MustQuery("explain select * from t where a >= 11 and b >= 11").Rows() - require.Equal(t, "mpp[tiflash]", fmt.Sprintf("%v", rows[len(rows)-1][2])) - - tk.MustExec("admin flush bindings") - rows = tk.MustQuery("show global bindings").Rows() - // None evolve task, because of the origin binding is a read_from_storage binding. - require.Len(t, rows, 1) - require.Equal(t, "SELECT /*+ read_from_storage(tiflash[`t`])*/ * FROM `test`.`t` WHERE `a` >= 1 AND `b` >= 1", rows[0][1]) - require.Equal(t, bindinfo.Enabled, rows[0][3]) -} - func TestBindingWithIsolationRead(t *testing.T) { store := testkit.CreateMockStore(t) @@ -781,41 +576,6 @@ func TestBindingWithIsolationRead(t *testing.T) { require.Equal(t, "mpp[tiflash]", rows[len(rows)-1][2]) } -func TestReCreateBindAfterEvolvePlan(t *testing.T) { - originalVal := config.CheckTableBeforeDrop - config.CheckTableBeforeDrop = true - defer func() { - config.CheckTableBeforeDrop = originalVal - }() - - store := testkit.CreateMockStore(t) - - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(a int, b int, c int, index idx_a(a), index idx_b(b), index idx_c(c))") - tk.MustExec("insert into t values (1,1,1), (2,2,2), (3,3,3), (4,4,4), (5,5,5)") - tk.MustExec("analyze table t") - tk.MustExec("create global binding for select * from t where a >= 1 and b >= 1 using select * from t use index(idx_a) where a >= 1 and b >= 1") - tk.MustExec("set @@tidb_evolve_plan_baselines=1") - - // It cannot choose table path although it has lowest cost. - tk.MustQuery("select * from t where a >= 0 and b >= 0") - require.Equal(t, "t:idx_a", tk.Session().GetSessionVars().StmtCtx.IndexNames[0]) - - tk.MustExec("admin flush bindings") - rows := tk.MustQuery("show global bindings").Rows() - require.Len(t, rows, 2) - require.Equal(t, "SELECT /*+ use_index(@`sel_1` `test`.`t` )*/ * FROM `test`.`t` WHERE `a` >= 0 AND `b` >= 0", rows[0][1]) - require.Equal(t, "pending verify", rows[0][3]) - - tk.MustExec("create global binding for select * from t where a >= 1 and b >= 1 using select * from t use index(idx_b) where a >= 1 and b >= 1") - rows = tk.MustQuery("show global bindings").Rows() - require.Len(t, rows, 1) - tk.MustQuery("select * from t where a >= 4 and b >= 1") - require.Equal(t, "t:idx_b", tk.Session().GetSessionVars().StmtCtx.IndexNames[0]) -} - func TestInvisibleIndex(t *testing.T) { store := testkit.CreateMockStore(t) @@ -850,26 +610,6 @@ func TestInvisibleIndex(t *testing.T) { tk.MustExec("drop binding for select * from t") } -func TestForbidEvolvePlanBaseLinesBeforeGA(t *testing.T) { - originalVal := config.CheckTableBeforeDrop - config.CheckTableBeforeDrop = false - defer func() { - config.CheckTableBeforeDrop = originalVal - }() - - store := testkit.CreateMockStore(t) - - tk := testkit.NewTestKit(t, store) - err := tk.ExecToErr("set @@tidb_evolve_plan_baselines=0") - require.Equal(t, nil, err) - err = tk.ExecToErr("set @@TiDB_Evolve_pLan_baselines=1") - require.EqualError(t, err, "Cannot enable baseline evolution feature, it is not generally available now") - err = tk.ExecToErr("set @@TiDB_Evolve_pLan_baselines=oN") - require.EqualError(t, err, "Cannot enable baseline evolution feature, it is not generally available now") - err = tk.ExecToErr("admin evolve bindings") - require.EqualError(t, err, "Cannot enable baseline evolution feature, it is not generally available now") -} - func TestGCBindRecord(t *testing.T) { // set lease for gc tests originLease := bindinfo.Lease diff --git a/pkg/domain/domain.go b/pkg/domain/domain.go index 6af0a74e2161a..445997c1b8fb9 100644 --- a/pkg/domain/domain.go +++ b/pkg/domain/domain.go @@ -1852,7 +1852,6 @@ func (do *Domain) LoadBindInfoLoop(ctxForHandle sessionctx.Context, ctxForEvolve owner := do.newOwnerManager(bindinfo.Prompt, bindinfo.OwnerKey) do.globalBindHandleWorkerLoop(owner) - do.handleEvolvePlanTasksLoop(ctxForEvolve, owner) return nil } @@ -1872,6 +1871,7 @@ func (do *Domain) globalBindHandleWorkerLoop(owner owner.Manager) { for { select { case <-do.exit: + owner.Cancel() return case <-bindWorkerTicker.C: bindHandle := do.bindHandle.Load() @@ -1885,7 +1885,6 @@ func (do *Domain) globalBindHandleWorkerLoop(owner owner.Manager) { if err == nil && variable.TiDBOptOn(optVal) { bindHandle.CaptureBaselines() } - bindHandle.SaveEvolveTasksToStore() case <-gcBindTicker.C: if !owner.IsOwner() { continue @@ -1899,30 +1898,6 @@ func (do *Domain) globalBindHandleWorkerLoop(owner owner.Manager) { }, "globalBindHandleWorkerLoop") } -func (do *Domain) handleEvolvePlanTasksLoop(ctx sessionctx.Context, owner owner.Manager) { - do.wg.Run(func() { - defer func() { - logutil.BgLogger().Info("handleEvolvePlanTasksLoop exited.") - }() - defer util.Recover(metrics.LabelDomain, "handleEvolvePlanTasksLoop", nil, false) - - for { - select { - case <-do.exit: - owner.Cancel() - return - case <-time.After(bindinfo.Lease): - } - if owner.IsOwner() { - err := do.bindHandle.Load().HandleEvolvePlanTask(ctx, false) - if err != nil { - logutil.BgLogger().Info("evolve plan failed", zap.Error(err)) - } - } - } - }, "handleEvolvePlanTasksLoop") -} - // TelemetryReportLoop create a goroutine that reports usage data in a loop, it should be called only once // in BootstrapSession. func (do *Domain) TelemetryReportLoop(ctx sessionctx.Context) { diff --git a/pkg/executor/bind.go b/pkg/executor/bind.go index 97c222ff2714d..ba19d50f48a7c 100644 --- a/pkg/executor/bind.go +++ b/pkg/executor/bind.go @@ -59,7 +59,7 @@ func (e *SQLBindExec) Next(_ context.Context, req *chunk.Chunk) error { case plannercore.OpCaptureBindings: e.captureBindings() case plannercore.OpEvolveBindings: - return e.evolveBindings() + return nil // not support yet case plannercore.OpReloadBindings: return e.reloadBindings() case plannercore.OpSetBindingStatus: @@ -172,10 +172,6 @@ func (e *SQLBindExec) captureBindings() { domain.GetDomain(e.Ctx()).BindHandle().CaptureBaselines() } -func (e *SQLBindExec) evolveBindings() error { - return domain.GetDomain(e.Ctx()).BindHandle().HandleEvolvePlanTask(e.Ctx(), true) -} - func (e *SQLBindExec) reloadBindings() error { return domain.GetDomain(e.Ctx()).BindHandle().ReloadGlobalBindings() } diff --git a/pkg/planner/optimize.go b/pkg/planner/optimize.go index 849de12b966a6..4e2a34bd19467 100644 --- a/pkg/planner/optimize.go +++ b/pkg/planner/optimize.go @@ -181,7 +181,7 @@ func Optimize(ctx context.Context, sctx sessionctx.Context, node ast.Node, is in } tableHints := hint.ExtractTableHintsFromStmtNode(node, sctx) - originStmtHints, originStmtHintsOffs, warns := handleStmtHints(tableHints) + originStmtHints, _, warns := handleStmtHints(tableHints) sessVars.StmtCtx.StmtHints = originStmtHints for _, warn := range warns { sessVars.StmtCtx.AppendWarning(warn) @@ -381,15 +381,6 @@ func Optimize(ctx context.Context, sctx sessionctx.Context, node ast.Node, is in return bestPlan, names, nil } } - // The hints generated from the plan do not contain the statement hints of the query, add them back. - for _, off := range originStmtHintsOffs { - defPlanHints = append(defPlanHints, tableHints[off]) - } - defPlanHintsStr := hint.RestoreOptimizerHints(defPlanHints) - binding := bindRecord.FindBinding(defPlanHintsStr) - if binding == nil { - handleEvolveTasks(ctx, sctx, bindRecord, stmtNode, defPlanHintsStr) - } } } @@ -704,25 +695,6 @@ func handleInvalidBinding(ctx context.Context, sctx sessionctx.Context, level st globalHandle.AddInvalidGlobalBinding(&bindRecord) } -func handleEvolveTasks(ctx context.Context, sctx sessionctx.Context, br *bindinfo.BindRecord, stmtNode ast.StmtNode, planHint string) { - bindSQL := bindinfo.GenerateBindSQL(ctx, stmtNode, planHint, false, br.Db) - if bindSQL == "" { - return - } - charset, collation := sctx.GetSessionVars().GetCharsetInfo() - _, sqlDigestWithDB := parser.NormalizeDigest(utilparser.RestoreWithDefaultDB(stmtNode, br.Db, br.OriginalSQL)) - binding := bindinfo.Binding{ - BindSQL: bindSQL, - Status: bindinfo.PendingVerify, - Charset: charset, - Collation: collation, - Source: bindinfo.Evolve, - SQLDigest: sqlDigestWithDB.String(), - } - globalHandle := domain.GetDomain(sctx).BindHandle() - globalHandle.AddEvolvePlanTask(br.OriginalSQL, br.Db, binding) -} - func handleStmtHints(hints []*ast.TableOptimizerHint) (stmtHints stmtctx.StmtHints, offs []int, warns []error) { if len(hints) == 0 { return