diff --git a/pkg/bindinfo/BUILD.bazel b/pkg/bindinfo/BUILD.bazel index d3055013450d4..ecc9c57d51b57 100644 --- a/pkg/bindinfo/BUILD.bazel +++ b/pkg/bindinfo/BUILD.bazel @@ -14,6 +14,7 @@ go_library( importpath = "github.com/pingcap/tidb/pkg/bindinfo", visibility = ["//visibility:public"], deps = [ + "//pkg/bindinfo/internal/logutil", "//pkg/bindinfo/norm", "//pkg/kv", "//pkg/metrics", @@ -33,7 +34,6 @@ go_library( "//pkg/util/hint", "//pkg/util/intest", "//pkg/util/kvcache", - "//pkg/util/logutil", "//pkg/util/mathutil", "//pkg/util/memory", "//pkg/util/parser", diff --git a/pkg/bindinfo/binding_cache.go b/pkg/bindinfo/binding_cache.go index 1a9851862c3d1..649906ee1d2f9 100644 --- a/pkg/bindinfo/binding_cache.go +++ b/pkg/bindinfo/binding_cache.go @@ -20,6 +20,7 @@ import ( "sync/atomic" "time" + "github.com/pingcap/tidb/pkg/bindinfo/internal/logutil" "github.com/pingcap/tidb/pkg/bindinfo/norm" "github.com/pingcap/tidb/pkg/parser" "github.com/pingcap/tidb/pkg/parser/ast" @@ -28,7 +29,6 @@ import ( "github.com/pingcap/tidb/pkg/util/hack" "github.com/pingcap/tidb/pkg/util/intest" "github.com/pingcap/tidb/pkg/util/kvcache" - "github.com/pingcap/tidb/pkg/util/logutil" "github.com/pingcap/tidb/pkg/util/mathutil" "github.com/pingcap/tidb/pkg/util/memory" "github.com/pingcap/tidb/pkg/util/stringutil" @@ -146,7 +146,7 @@ func (fbc *fuzzyBindingCache) loadFromStore(sctx sessionctx.Context, missingSQLD start := time.Now() bindings, err := fbc.loadBindingFromStorageFunc(sctx, sqlDigest) if err != nil { - logutil.BgLogger().Warn("failed to load binding from storage", + logutil.BindLogger().Warn("failed to load binding from storage", zap.String("sqlDigest", sqlDigest), zap.Error(err), zap.Duration("duration", time.Since(start)), @@ -162,7 +162,7 @@ func (fbc *fuzzyBindingCache) loadFromStore(sctx sessionctx.Context, missingSQLD // When the memory capacity of bing_cache is not enough, // there will be some memory-related errors in multiple places. // Only needs to be handled once. - logutil.BgLogger().Warn("BindHandle.Update", zap.String("category", "sql-bind"), zap.Error(err)) + logutil.BindLogger().Warn("BindHandle.Update", zap.Error(err)) } } } diff --git a/pkg/bindinfo/capture.go b/pkg/bindinfo/capture.go index 8e30a295848c4..e1f7fc8dc3681 100644 --- a/pkg/bindinfo/capture.go +++ b/pkg/bindinfo/capture.go @@ -15,15 +15,14 @@ package bindinfo import ( - "context" "strconv" "strings" + "github.com/pingcap/tidb/pkg/bindinfo/internal/logutil" "github.com/pingcap/tidb/pkg/parser" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" - "github.com/pingcap/tidb/pkg/util/logutil" utilparser "github.com/pingcap/tidb/pkg/util/parser" stmtsummaryv2 "github.com/pingcap/tidb/pkg/util/stmtsummary/v2" tablefilter "github.com/pingcap/tidb/pkg/util/table-filter" @@ -95,7 +94,7 @@ func (h *globalBindingHandle) extractCaptureFilterFromStorage() (filter *capture // uses another background session. rows, _, err := execRows(sctx, `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)) + logutil.BindLogger().Warn("failed to load mysql.capture_plan_baselines_blacklist", zap.Error(err)) return err } for _, row := range rows { @@ -105,7 +104,7 @@ func (h *globalBindingHandle) extractCaptureFilterFromStorage() (filter *capture 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)) + logutil.BindLogger().Warn("capture table filter is invalid, ignore it", zap.String("filter_value", valStr)) continue } filter.tables = append(filter.tables, tfilter) @@ -114,18 +113,18 @@ func (h *globalBindingHandle) extractCaptureFilterFromStorage() (filter *capture 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)) + logutil.BindLogger().Warn("failed to parse frequency type value, ignore it", 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)) + logutil.BindLogger().Warn("frequency threshold is less than 1, ignore it", 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)) + logutil.BindLogger().Warn("unknown capture filter type, ignore it", zap.String("filter_type", filterTp)) } } return nil @@ -142,7 +141,7 @@ func (h *globalBindingHandle) CaptureBaselines() { 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)) + logutil.BindLogger().Debug("parse SQL failed in baseline capture", zap.String("SQL", bindableStmt.Query), zap.Error(err)) continue } if insertStmt, ok := stmt.(*ast.InsertStmt); ok && insertStmt.Select == nil { @@ -174,7 +173,7 @@ func (h *globalBindingHandle) CaptureBaselines() { if r := h.getCache().GetBinding(digest.String()); HasAvailableBinding(r) { continue } - bindSQL := GenerateBindingSQL(context.TODO(), stmt, bindableStmt.PlanHint, true, dbName) + bindSQL := GenerateBindingSQL(stmt, bindableStmt.PlanHint, true, dbName) if bindSQL == "" { continue } @@ -197,7 +196,7 @@ func (h *globalBindingHandle) CaptureBaselines() { // We don't need to pass the `sctx` because the BindSQL has been validated already. err = h.CreateGlobalBinding(nil, 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)) + logutil.BindLogger().Debug("create bind record failed in baseline capture", zap.String("SQL", bindableStmt.Query), zap.Error(err)) } } } diff --git a/pkg/bindinfo/global_handle.go b/pkg/bindinfo/global_handle.go index 017797c5e23ce..45789901e7d94 100644 --- a/pkg/bindinfo/global_handle.go +++ b/pkg/bindinfo/global_handle.go @@ -24,6 +24,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" + "github.com/pingcap/tidb/pkg/bindinfo/internal/logutil" "github.com/pingcap/tidb/pkg/parser" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/format" @@ -35,7 +36,6 @@ import ( driver "github.com/pingcap/tidb/pkg/types/parser_driver" "github.com/pingcap/tidb/pkg/util/chunk" "github.com/pingcap/tidb/pkg/util/hint" - "github.com/pingcap/tidb/pkg/util/logutil" utilparser "github.com/pingcap/tidb/pkg/util/parser" "go.uber.org/zap" "golang.org/x/sync/singleflight" @@ -226,7 +226,7 @@ func (h *globalBindingHandle) LoadFromStorageToCache(fullLoad bool) (err error) } if err != nil { - logutil.BgLogger().Warn("failed to generate bind record from data row", zap.String("category", "sql-bind"), zap.Error(err)) + logutil.BindLogger().Warn("failed to generate bind record from data row", zap.Error(err)) continue } @@ -238,7 +238,7 @@ func (h *globalBindingHandle) LoadFromStorageToCache(fullLoad bool) (err error) // When the memory capacity of bing_cache is not enough, // there will be some memory-related errors in multiple places. // Only needs to be handled once. - logutil.BgLogger().Warn("BindHandle.Update", zap.String("category", "sql-bind"), zap.Error(err)) + logutil.BindLogger().Warn("BindHandle.Update", zap.Error(err)) } } else { newCache.RemoveBinding(sqlDigest) @@ -439,7 +439,7 @@ func (c *invalidBindingCache) reset() { func (h *globalBindingHandle) DropInvalidGlobalBinding() { defer func() { if err := h.LoadFromStorageToCache(false); err != nil { - logutil.BgLogger().Warn("drop invalid global binding error", zap.Error(err)) + logutil.BindLogger().Warn("drop invalid global binding error", zap.Error(err)) } }() @@ -447,7 +447,7 @@ func (h *globalBindingHandle) DropInvalidGlobalBinding() { h.invalidBindings.reset() for _, invalidBinding := range invalidBindings { if _, err := h.dropGlobalBinding(invalidBinding.SQLDigest); err != nil { - logutil.BgLogger().Debug("flush bind record failed", zap.String("category", "sql-bind"), zap.Error(err)) + logutil.BindLogger().Debug("flush bind record failed", zap.Error(err)) } } } @@ -545,7 +545,7 @@ func getHintsForSQL(sctx sessionctx.Context, sql string) (string, error) { } // GenerateBindingSQL generates binding sqls from stmt node and plan hints. -func GenerateBindingSQL(ctx context.Context, stmtNode ast.StmtNode, planHint string, skipCheckIfHasParam bool, defaultDB string) string { +func GenerateBindingSQL(stmtNode ast.StmtNode, planHint string, skipCheckIfHasParam bool, defaultDB string) string { // If would be nil for very simple cases such as point get, we do not need to evolve for them. if planHint == "" { return "" @@ -585,7 +585,7 @@ func GenerateBindingSQL(ctx context.Context, stmtNode ast.StmtNode, planHint str restoreCtx := format.NewRestoreCtx(format.RestoreStringSingleQuotes|format.RestoreSpacesAroundBinaryOperation|format.RestoreStringWithoutCharset|format.RestoreNameBackQuotes, &withSb) restoreCtx.DefaultDB = defaultDB if err := n.With.Restore(restoreCtx); err != nil { - logutil.BgLogger().Debug("restore SQL failed", zap.String("category", "sql-bind"), zap.Error(err)) + logutil.BindLogger().Debug("restore SQL failed", zap.Error(err)) return "" } withEnd := withIdx + len(withSb.String()) @@ -607,7 +607,7 @@ func GenerateBindingSQL(ctx context.Context, stmtNode ast.StmtNode, planHint str bindSQL = bindSQL[insertIdx:] return strings.Replace(bindSQL, "SELECT", fmt.Sprintf("SELECT /*+ %s*/", planHint), 1) } - logutil.Logger(ctx).Debug("unexpected statement type when generating bind SQL", zap.String("category", "sql-bind"), zap.Any("statement", stmtNode)) + logutil.BindLogger().Debug("unexpected statement type when generating bind SQL", zap.Any("statement", stmtNode)) return "" } @@ -713,7 +713,6 @@ func (h *globalBindingHandle) loadBindingsFromStorageInternal(sqlDigest string) failpoint.Inject("load_bindings_from_storage_internal_timeout", func() { time.Sleep(time.Second) }) - var bindings Bindings selectStmt := fmt.Sprintf("SELECT original_sql, bind_sql, default_db, status, create_time, update_time, charset, collation, source, sql_digest, plan_digest FROM mysql.bind_info where sql_digest = '%s'", sqlDigest) err := h.callWithSCtx(false, func(sctx sessionctx.Context) error { @@ -729,7 +728,7 @@ func (h *globalBindingHandle) loadBindingsFromStorageInternal(sqlDigest string) } _, binding, err := newBinding(sctx, row) if err != nil { - logutil.BgLogger().Warn("failed to generate bind record from data row", zap.String("category", "sql-bind"), zap.Error(err)) + logutil.BindLogger().Warn("failed to generate bind record from data row", zap.Error(err)) continue } bindings = append(bindings, binding) diff --git a/pkg/bindinfo/internal/logutil/BUILD.bazel b/pkg/bindinfo/internal/logutil/BUILD.bazel new file mode 100644 index 0000000000000..dfbdf07486ed5 --- /dev/null +++ b/pkg/bindinfo/internal/logutil/BUILD.bazel @@ -0,0 +1,12 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library") + +go_library( + name = "logutil", + srcs = ["logutil.go"], + importpath = "github.com/pingcap/tidb/pkg/bindinfo/internal/logutil", + visibility = ["//pkg/bindinfo:__subpackages__"], + deps = [ + "//pkg/util/logutil", + "@org_uber_go_zap//:zap", + ], +) diff --git a/pkg/bindinfo/internal/logutil/logutil.go b/pkg/bindinfo/internal/logutil/logutil.go new file mode 100644 index 0000000000000..fca251c9b3c55 --- /dev/null +++ b/pkg/bindinfo/internal/logutil/logutil.go @@ -0,0 +1,25 @@ +// 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 logutil + +import ( + "github.com/pingcap/tidb/pkg/util/logutil" + "go.uber.org/zap" +) + +// BindLogger with category "sql-bind" is used to log statistic related messages. +func BindLogger() *zap.Logger { + return logutil.BgLogger().With(zap.String("category", "sql-bind")) +} diff --git a/pkg/bindinfo/session_handle.go b/pkg/bindinfo/session_handle.go index f1fabe174e43c..61ee7bbdbc369 100644 --- a/pkg/bindinfo/session_handle.go +++ b/pkg/bindinfo/session_handle.go @@ -21,6 +21,7 @@ import ( "time" "github.com/pingcap/errors" + "github.com/pingcap/tidb/pkg/bindinfo/internal/logutil" "github.com/pingcap/tidb/pkg/parser" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" @@ -28,7 +29,6 @@ import ( "github.com/pingcap/tidb/pkg/sessionctx/sessionstates" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/hack" - "github.com/pingcap/tidb/pkg/util/logutil" "go.uber.org/zap" ) @@ -69,7 +69,7 @@ func NewSessionBindingHandle() SessionBindingHandle { func (h *sessionBindingHandle) appendSessionBinding(sqlDigest string, meta Bindings) { err := h.ch.SetBinding(sqlDigest, meta) if err != nil { - logutil.BgLogger().Warn("SessionHandle.appendSessionBinding", zap.String("category", "sql-bind"), zap.Error(err)) + logutil.BindLogger().Warn("SessionHandle.appendSessionBinding", zap.Error(err)) } } diff --git a/pkg/planner/core/planbuilder.go b/pkg/planner/core/planbuilder.go index 0bedfec4c1dfc..1b5b2b245373a 100644 --- a/pkg/planner/core/planbuilder.go +++ b/pkg/planner/core/planbuilder.go @@ -850,7 +850,7 @@ func (b *PlanBuilder) buildCreateBindPlanFromPlanDigest(v *ast.CreateBindingStmt if err = hint.CheckBindingFromHistoryBindable(originNode, bindableStmt.PlanHint); err != nil { return nil, err } - bindSQL := bindinfo.GenerateBindingSQL(context.TODO(), originNode, bindableStmt.PlanHint, true, bindableStmt.Schema) + bindSQL := bindinfo.GenerateBindingSQL(originNode, bindableStmt.PlanHint, true, bindableStmt.Schema) var hintNode ast.StmtNode hintNode, err = parser4binding.ParseOneStmt(bindSQL, bindableStmt.Charset, bindableStmt.Collation) if err != nil {