From 8a98b5573b1ade6446eaf89f60a0e484da4dd748 Mon Sep 17 00:00:00 2001
From: Yuanjia Zhang <zhangyuanjia@pingcap.com>
Date: Thu, 5 Dec 2024 21:57:47 +0800
Subject: [PATCH 1/3] This is an automated cherry-pick of #58017

Signed-off-by: ti-chi-bot <ti-community-prow-bot@tidb.io>
---
 pkg/bindinfo/BUILD.bazel |  99 ++++++++++++++++
 pkg/bindinfo/binding.go  | 237 +++++++++++++++++++++++++++++++++++++++
 2 files changed, 336 insertions(+)
 create mode 100644 pkg/bindinfo/BUILD.bazel
 create mode 100644 pkg/bindinfo/binding.go

diff --git a/pkg/bindinfo/BUILD.bazel b/pkg/bindinfo/BUILD.bazel
new file mode 100644
index 0000000000000..fc81b0958b6b9
--- /dev/null
+++ b/pkg/bindinfo/BUILD.bazel
@@ -0,0 +1,99 @@
+load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test")
+
+go_library(
+    name = "bindinfo",
+    srcs = [
+        "binding.go",
+        "binding_cache.go",
+        "binding_match.go",
+        "capture.go",
+        "global_handle.go",
+        "session_handle.go",
+        "util.go",
+    ],
+    importpath = "github.com/pingcap/tidb/pkg/bindinfo",
+    visibility = ["//visibility:public"],
+    deps = [
+        "//pkg/bindinfo/internal/logutil",
+        "//pkg/bindinfo/norm",
+        "//pkg/kv",
+        "//pkg/metrics",
+        "//pkg/parser",
+        "//pkg/parser/ast",
+        "//pkg/parser/format",
+        "//pkg/parser/mysql",
+        "//pkg/parser/terror",
+        "//pkg/planner/core/resolve",
+        "//pkg/sessionctx",
+        "//pkg/sessionctx/sessionstates",
+        "//pkg/sessionctx/stmtctx",
+        "//pkg/sessionctx/variable",
+        "//pkg/types",
+        "//pkg/types/parser_driver",
+        "//pkg/util",
+        "//pkg/util/chunk",
+        "//pkg/util/hack",
+        "//pkg/util/hint",
+        "//pkg/util/intest",
+        "//pkg/util/kvcache",
+        "//pkg/util/mathutil",
+        "//pkg/util/memory",
+        "//pkg/util/parser",
+        "//pkg/util/sqlexec",
+        "//pkg/util/stmtsummary/v2:stmtsummary",
+        "//pkg/util/stringutil",
+        "//pkg/util/table-filter",
+        "@com_github_pingcap_errors//:errors",
+        "@com_github_pingcap_failpoint//:failpoint",
+        "@com_github_pkg_errors//:errors",
+        "@org_golang_x_sync//singleflight",
+        "@org_uber_go_zap//:zap",
+    ],
+)
+
+go_test(
+    name = "bindinfo_test",
+    timeout = "moderate",
+    srcs = [
+        "binding_cache_test.go",
+        "binding_match_test.go",
+        "capture_test.go",
+        "fuzzy_binding_test.go",
+        "global_handle_test.go",
+        "main_test.go",
+        "optimize_test.go",
+        "session_handle_test.go",
+    ],
+    embed = [":bindinfo"],
+    flaky = True,
+    race = "on",
+    shard_count = 50,
+    deps = [
+        "//pkg/bindinfo/internal",
+        "//pkg/bindinfo/norm",
+        "//pkg/config",
+        "//pkg/domain",
+        "//pkg/meta/model",
+        "//pkg/parser",
+        "//pkg/parser/ast",
+        "//pkg/parser/auth",
+        "//pkg/parser/format",
+        "//pkg/parser/model",
+        "//pkg/parser/mysql",
+        "//pkg/server",
+        "//pkg/session/types",
+        "//pkg/sessionctx/variable",
+        "//pkg/testkit",
+        "//pkg/testkit/testsetup",
+        "//pkg/types",
+        "//pkg/util",
+        "//pkg/util/hack",
+        "//pkg/util/parser",
+        "//pkg/util/stmtsummary",
+        "@com_github_ngaut_pools//:pools",
+        "@com_github_pingcap_failpoint//:failpoint",
+        "@com_github_stretchr_testify//require",
+        "@io_opencensus_go//stats/view",
+        "@org_uber_go_goleak//:goleak",
+    ],
+)
diff --git a/pkg/bindinfo/binding.go b/pkg/bindinfo/binding.go
new file mode 100644
index 0000000000000..8f3586f676175
--- /dev/null
+++ b/pkg/bindinfo/binding.go
@@ -0,0 +1,237 @@
+// Copyright 2019 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 (
+	"time"
+	"unsafe"
+
+	"github.com/pingcap/tidb/pkg/parser"
+	"github.com/pingcap/tidb/pkg/parser/ast"
+	"github.com/pingcap/tidb/pkg/sessionctx"
+	"github.com/pingcap/tidb/pkg/types"
+	"github.com/pingcap/tidb/pkg/util/hint"
+	"github.com/pkg/errors"
+)
+
+const (
+	// Enabled is the bind info's in enabled status.
+	// It is the same as the previous 'Using' status.
+	// Only use 'Enabled' status in the future, not the 'Using' status.
+	// The 'Using' status is preserved for compatibility.
+	Enabled = "enabled"
+	// Disabled is the bind info's in disabled status.
+	Disabled = "disabled"
+	// Using is the bind info's in use status.
+	// The 'Using' status is preserved for compatibility.
+	Using = "using"
+	// deleted is the bind info's deleted status.
+	deleted = "deleted"
+	// Invalid is the bind info's invalid status.
+	Invalid = "invalid"
+	// Manual indicates the binding is created by SQL like "create binding for ...".
+	Manual = "manual"
+	// Capture indicates the binding is captured by TiDB automatically.
+	Capture = "capture"
+	// Builtin indicates the binding is a builtin record for internal locking purpose. It is also the status for the builtin binding.
+	Builtin = "builtin"
+	// History indicate the binding is created from statement summary by plan digest
+	History = "history"
+)
+
+// Binding stores the basic bind hint info.
+type Binding struct {
+	OriginalSQL string
+	Db          string
+	BindSQL     string
+	// Status represents the status of the binding. It can only be one of the following values:
+	// 1. deleted: Bindings is deleted, can not be used anymore.
+	// 2. enabled, using: Binding is in the normal active mode.
+	Status     string
+	CreateTime types.Time
+	UpdateTime types.Time
+	Source     string
+	Charset    string
+	Collation  string
+	// Hint is the parsed hints, it is used to bind hints to stmt node.
+	Hint *hint.HintsSet `json:"-"`
+	// ID is the string form of Hint. It would be non-empty only when the status is `Using` or `PendingVerify`.
+	ID         string `json:"-"`
+	SQLDigest  string
+	PlanDigest string
+
+	// TableNames records all schema and table names in this binding statement, which are used for fuzzy matching.
+	TableNames []*ast.TableName `json:"-"`
+}
+
+func (b *Binding) isSame(rb *Binding) bool {
+	if b.ID != "" && rb.ID != "" {
+		return b.ID == rb.ID
+	}
+	// Sometimes we cannot construct `ID` because of the changed schema, so we need to compare by bind sql.
+	return b.BindSQL == rb.BindSQL
+}
+
+// IsBindingEnabled returns whether the binding is enabled.
+func (b *Binding) IsBindingEnabled() bool {
+	return b.Status == Enabled || b.Status == Using
+}
+
+// IsBindingAvailable returns whether the binding is available.
+// The available means the binding can be used or can be converted into a usable status.
+// It includes the 'Enabled', 'Using' and 'Disabled' status.
+func (b *Binding) IsBindingAvailable() bool {
+	return b.IsBindingEnabled() || b.Status == Disabled
+}
+
+// SinceUpdateTime returns the duration since last update time. Export for test.
+func (b *Binding) SinceUpdateTime() (time.Duration, error) {
+	updateTime, err := b.UpdateTime.GoTime(time.Local)
+	if err != nil {
+		return 0, err
+	}
+	return time.Since(updateTime), nil
+}
+
+// Bindings represents a sql bind record retrieved from the storage.
+type Bindings []Binding
+
+// Copy get the copy of bindings
+func (br Bindings) Copy() Bindings {
+	nbr := append(make(Bindings, 0, len(br)), br...)
+	return nbr
+}
+
+// HasAvailableBinding checks if there are any available bindings in bind record.
+// The available means the binding can be used or can be converted into a usable status.
+// It includes the 'Enabled', 'Using' and 'Disabled' status.
+func HasAvailableBinding(br Bindings) bool {
+	if br == nil {
+		return false
+	}
+	for _, binding := range br {
+		if binding.IsBindingAvailable() {
+			return true
+		}
+	}
+	return false
+}
+
+// prepareHints builds ID and Hint for Bindings. If sctx is not nil, we check if
+// the BindSQL is still valid.
+func prepareHints(sctx sessionctx.Context, binding *Binding) (rerr error) {
+	defer func() {
+		if r := recover(); r != nil {
+			rerr = errors.Errorf("panic when preparing hints for binding %v, panic: %v", binding.BindSQL, r)
+		}
+	}()
+
+	p := parser.New()
+	if (binding.Hint != nil && binding.ID != "") || binding.Status == deleted {
+		return nil
+	}
+	dbName := binding.Db
+	bindingStmt, err := p.ParseOneStmt(binding.BindSQL, binding.Charset, binding.Collation)
+	if err != nil {
+		return err
+	}
+	tableNames := CollectTableNames(bindingStmt)
+	isFuzzy := isFuzzyBinding(bindingStmt)
+	if isFuzzy {
+		dbName = "*" // ues '*' for universal bindings
+	}
+
+	hintsSet, stmt, warns, err := hint.ParseHintsSet(p, binding.BindSQL, binding.Charset, binding.Collation, dbName)
+	if err != nil {
+		return err
+	}
+	if sctx != nil && !isFuzzy {
+		paramChecker := &paramMarkerChecker{}
+		stmt.Accept(paramChecker)
+		if !paramChecker.hasParamMarker {
+			_, err = getHintsForSQL(sctx, binding.BindSQL)
+			if err != nil {
+				return err
+			}
+		}
+	}
+	hintsStr, err := hintsSet.Restore()
+	if err != nil {
+		return err
+	}
+	// For `create global binding for select * from t using select * from t`, we allow it though hintsStr is empty.
+	// For `create global binding for select * from t using select /*+ non_exist_hint() */ * from t`,
+	// the hint is totally invalid, we escalate warning to error.
+	if hintsStr == "" && len(warns) > 0 {
+		return warns[0]
+	}
+	binding.Hint = hintsSet
+	binding.ID = hintsStr
+	binding.TableNames = tableNames
+	return nil
+}
+
+// `merge` merges two Bindings. It will replace old bindings with new bindings if there are new updates.
+func merge(lBindings, rBindings Bindings) Bindings {
+	if lBindings == nil {
+		return rBindings
+	}
+	if rBindings == nil {
+		return lBindings
+	}
+	result := lBindings.Copy()
+	for i := range rBindings {
+		rbind := rBindings[i]
+		found := false
+		for j, lbind := range lBindings {
+			if lbind.isSame(&rbind) {
+				found = true
+				if rbind.UpdateTime.Compare(lbind.UpdateTime) >= 0 {
+					result[j] = rbind
+				}
+				break
+			}
+		}
+		if !found {
+			result = append(result, rbind)
+		}
+	}
+	return result
+}
+
+func removeDeletedBindings(br Bindings) Bindings {
+	result := make(Bindings, 0, len(br))
+	for _, binding := range br {
+		if binding.Status != deleted {
+			result = append(result, binding)
+		}
+	}
+	return result
+}
+
+// size calculates the memory size of a Bindings.
+func (br Bindings) size() float64 {
+	mem := float64(0)
+	for _, binding := range br {
+		mem += binding.size()
+	}
+	return mem
+}
+
+// size calculates the memory size of a bind info.
+func (b *Binding) size() float64 {
+	res := len(b.OriginalSQL) + len(b.Db) + len(b.BindSQL) + len(b.Status) + 2*int(unsafe.Sizeof(b.CreateTime)) + len(b.Charset) + len(b.Collation) + len(b.ID)
+	return float64(res)
+}

From 6a37e3e236659df926f49b00ad6287f9524497eb Mon Sep 17 00:00:00 2001
From: qw4990 <zhangyuanjia@pingcap.com>
Date: Wed, 5 Feb 2025 19:37:15 +0800
Subject: [PATCH 2/3] fixup

---
 bindinfo/bind_record.go  |   9 +-
 pkg/bindinfo/BUILD.bazel |  99 ----------------
 pkg/bindinfo/binding.go  | 237 ---------------------------------------
 3 files changed, 8 insertions(+), 337 deletions(-)
 delete mode 100644 pkg/bindinfo/BUILD.bazel
 delete mode 100644 pkg/bindinfo/binding.go

diff --git a/bindinfo/bind_record.go b/bindinfo/bind_record.go
index 6395bbaa278ba..6a6dc817b7628 100644
--- a/bindinfo/bind_record.go
+++ b/bindinfo/bind_record.go
@@ -18,6 +18,7 @@ import (
 	"time"
 	"unsafe"
 
+	"github.com/pingcap/errors"
 	"github.com/pingcap/tidb/metrics"
 	"github.com/pingcap/tidb/parser"
 	"github.com/pingcap/tidb/sessionctx"
@@ -161,7 +162,13 @@ func (br *BindRecord) FindBinding(hint string) *Binding {
 
 // prepareHints builds ID and Hint for BindRecord. If sctx is not nil, we check if
 // the BindSQL is still valid.
-func (br *BindRecord) prepareHints(sctx sessionctx.Context) error {
+func (br *BindRecord) prepareHints(sctx sessionctx.Context) (rerr error) {
+	defer func() {
+		if r := recover(); r != nil {
+			rerr = errors.Errorf("panic when preparing hints for binding panic: %v", r)
+		}
+	}()
+
 	p := parser.New()
 	for i, bind := range br.Bindings {
 		if (bind.Hint != nil && bind.ID != "") || bind.Status == deleted {
diff --git a/pkg/bindinfo/BUILD.bazel b/pkg/bindinfo/BUILD.bazel
deleted file mode 100644
index fc81b0958b6b9..0000000000000
--- a/pkg/bindinfo/BUILD.bazel
+++ /dev/null
@@ -1,99 +0,0 @@
-load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test")
-
-go_library(
-    name = "bindinfo",
-    srcs = [
-        "binding.go",
-        "binding_cache.go",
-        "binding_match.go",
-        "capture.go",
-        "global_handle.go",
-        "session_handle.go",
-        "util.go",
-    ],
-    importpath = "github.com/pingcap/tidb/pkg/bindinfo",
-    visibility = ["//visibility:public"],
-    deps = [
-        "//pkg/bindinfo/internal/logutil",
-        "//pkg/bindinfo/norm",
-        "//pkg/kv",
-        "//pkg/metrics",
-        "//pkg/parser",
-        "//pkg/parser/ast",
-        "//pkg/parser/format",
-        "//pkg/parser/mysql",
-        "//pkg/parser/terror",
-        "//pkg/planner/core/resolve",
-        "//pkg/sessionctx",
-        "//pkg/sessionctx/sessionstates",
-        "//pkg/sessionctx/stmtctx",
-        "//pkg/sessionctx/variable",
-        "//pkg/types",
-        "//pkg/types/parser_driver",
-        "//pkg/util",
-        "//pkg/util/chunk",
-        "//pkg/util/hack",
-        "//pkg/util/hint",
-        "//pkg/util/intest",
-        "//pkg/util/kvcache",
-        "//pkg/util/mathutil",
-        "//pkg/util/memory",
-        "//pkg/util/parser",
-        "//pkg/util/sqlexec",
-        "//pkg/util/stmtsummary/v2:stmtsummary",
-        "//pkg/util/stringutil",
-        "//pkg/util/table-filter",
-        "@com_github_pingcap_errors//:errors",
-        "@com_github_pingcap_failpoint//:failpoint",
-        "@com_github_pkg_errors//:errors",
-        "@org_golang_x_sync//singleflight",
-        "@org_uber_go_zap//:zap",
-    ],
-)
-
-go_test(
-    name = "bindinfo_test",
-    timeout = "moderate",
-    srcs = [
-        "binding_cache_test.go",
-        "binding_match_test.go",
-        "capture_test.go",
-        "fuzzy_binding_test.go",
-        "global_handle_test.go",
-        "main_test.go",
-        "optimize_test.go",
-        "session_handle_test.go",
-    ],
-    embed = [":bindinfo"],
-    flaky = True,
-    race = "on",
-    shard_count = 50,
-    deps = [
-        "//pkg/bindinfo/internal",
-        "//pkg/bindinfo/norm",
-        "//pkg/config",
-        "//pkg/domain",
-        "//pkg/meta/model",
-        "//pkg/parser",
-        "//pkg/parser/ast",
-        "//pkg/parser/auth",
-        "//pkg/parser/format",
-        "//pkg/parser/model",
-        "//pkg/parser/mysql",
-        "//pkg/server",
-        "//pkg/session/types",
-        "//pkg/sessionctx/variable",
-        "//pkg/testkit",
-        "//pkg/testkit/testsetup",
-        "//pkg/types",
-        "//pkg/util",
-        "//pkg/util/hack",
-        "//pkg/util/parser",
-        "//pkg/util/stmtsummary",
-        "@com_github_ngaut_pools//:pools",
-        "@com_github_pingcap_failpoint//:failpoint",
-        "@com_github_stretchr_testify//require",
-        "@io_opencensus_go//stats/view",
-        "@org_uber_go_goleak//:goleak",
-    ],
-)
diff --git a/pkg/bindinfo/binding.go b/pkg/bindinfo/binding.go
deleted file mode 100644
index 8f3586f676175..0000000000000
--- a/pkg/bindinfo/binding.go
+++ /dev/null
@@ -1,237 +0,0 @@
-// Copyright 2019 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 (
-	"time"
-	"unsafe"
-
-	"github.com/pingcap/tidb/pkg/parser"
-	"github.com/pingcap/tidb/pkg/parser/ast"
-	"github.com/pingcap/tidb/pkg/sessionctx"
-	"github.com/pingcap/tidb/pkg/types"
-	"github.com/pingcap/tidb/pkg/util/hint"
-	"github.com/pkg/errors"
-)
-
-const (
-	// Enabled is the bind info's in enabled status.
-	// It is the same as the previous 'Using' status.
-	// Only use 'Enabled' status in the future, not the 'Using' status.
-	// The 'Using' status is preserved for compatibility.
-	Enabled = "enabled"
-	// Disabled is the bind info's in disabled status.
-	Disabled = "disabled"
-	// Using is the bind info's in use status.
-	// The 'Using' status is preserved for compatibility.
-	Using = "using"
-	// deleted is the bind info's deleted status.
-	deleted = "deleted"
-	// Invalid is the bind info's invalid status.
-	Invalid = "invalid"
-	// Manual indicates the binding is created by SQL like "create binding for ...".
-	Manual = "manual"
-	// Capture indicates the binding is captured by TiDB automatically.
-	Capture = "capture"
-	// Builtin indicates the binding is a builtin record for internal locking purpose. It is also the status for the builtin binding.
-	Builtin = "builtin"
-	// History indicate the binding is created from statement summary by plan digest
-	History = "history"
-)
-
-// Binding stores the basic bind hint info.
-type Binding struct {
-	OriginalSQL string
-	Db          string
-	BindSQL     string
-	// Status represents the status of the binding. It can only be one of the following values:
-	// 1. deleted: Bindings is deleted, can not be used anymore.
-	// 2. enabled, using: Binding is in the normal active mode.
-	Status     string
-	CreateTime types.Time
-	UpdateTime types.Time
-	Source     string
-	Charset    string
-	Collation  string
-	// Hint is the parsed hints, it is used to bind hints to stmt node.
-	Hint *hint.HintsSet `json:"-"`
-	// ID is the string form of Hint. It would be non-empty only when the status is `Using` or `PendingVerify`.
-	ID         string `json:"-"`
-	SQLDigest  string
-	PlanDigest string
-
-	// TableNames records all schema and table names in this binding statement, which are used for fuzzy matching.
-	TableNames []*ast.TableName `json:"-"`
-}
-
-func (b *Binding) isSame(rb *Binding) bool {
-	if b.ID != "" && rb.ID != "" {
-		return b.ID == rb.ID
-	}
-	// Sometimes we cannot construct `ID` because of the changed schema, so we need to compare by bind sql.
-	return b.BindSQL == rb.BindSQL
-}
-
-// IsBindingEnabled returns whether the binding is enabled.
-func (b *Binding) IsBindingEnabled() bool {
-	return b.Status == Enabled || b.Status == Using
-}
-
-// IsBindingAvailable returns whether the binding is available.
-// The available means the binding can be used or can be converted into a usable status.
-// It includes the 'Enabled', 'Using' and 'Disabled' status.
-func (b *Binding) IsBindingAvailable() bool {
-	return b.IsBindingEnabled() || b.Status == Disabled
-}
-
-// SinceUpdateTime returns the duration since last update time. Export for test.
-func (b *Binding) SinceUpdateTime() (time.Duration, error) {
-	updateTime, err := b.UpdateTime.GoTime(time.Local)
-	if err != nil {
-		return 0, err
-	}
-	return time.Since(updateTime), nil
-}
-
-// Bindings represents a sql bind record retrieved from the storage.
-type Bindings []Binding
-
-// Copy get the copy of bindings
-func (br Bindings) Copy() Bindings {
-	nbr := append(make(Bindings, 0, len(br)), br...)
-	return nbr
-}
-
-// HasAvailableBinding checks if there are any available bindings in bind record.
-// The available means the binding can be used or can be converted into a usable status.
-// It includes the 'Enabled', 'Using' and 'Disabled' status.
-func HasAvailableBinding(br Bindings) bool {
-	if br == nil {
-		return false
-	}
-	for _, binding := range br {
-		if binding.IsBindingAvailable() {
-			return true
-		}
-	}
-	return false
-}
-
-// prepareHints builds ID and Hint for Bindings. If sctx is not nil, we check if
-// the BindSQL is still valid.
-func prepareHints(sctx sessionctx.Context, binding *Binding) (rerr error) {
-	defer func() {
-		if r := recover(); r != nil {
-			rerr = errors.Errorf("panic when preparing hints for binding %v, panic: %v", binding.BindSQL, r)
-		}
-	}()
-
-	p := parser.New()
-	if (binding.Hint != nil && binding.ID != "") || binding.Status == deleted {
-		return nil
-	}
-	dbName := binding.Db
-	bindingStmt, err := p.ParseOneStmt(binding.BindSQL, binding.Charset, binding.Collation)
-	if err != nil {
-		return err
-	}
-	tableNames := CollectTableNames(bindingStmt)
-	isFuzzy := isFuzzyBinding(bindingStmt)
-	if isFuzzy {
-		dbName = "*" // ues '*' for universal bindings
-	}
-
-	hintsSet, stmt, warns, err := hint.ParseHintsSet(p, binding.BindSQL, binding.Charset, binding.Collation, dbName)
-	if err != nil {
-		return err
-	}
-	if sctx != nil && !isFuzzy {
-		paramChecker := &paramMarkerChecker{}
-		stmt.Accept(paramChecker)
-		if !paramChecker.hasParamMarker {
-			_, err = getHintsForSQL(sctx, binding.BindSQL)
-			if err != nil {
-				return err
-			}
-		}
-	}
-	hintsStr, err := hintsSet.Restore()
-	if err != nil {
-		return err
-	}
-	// For `create global binding for select * from t using select * from t`, we allow it though hintsStr is empty.
-	// For `create global binding for select * from t using select /*+ non_exist_hint() */ * from t`,
-	// the hint is totally invalid, we escalate warning to error.
-	if hintsStr == "" && len(warns) > 0 {
-		return warns[0]
-	}
-	binding.Hint = hintsSet
-	binding.ID = hintsStr
-	binding.TableNames = tableNames
-	return nil
-}
-
-// `merge` merges two Bindings. It will replace old bindings with new bindings if there are new updates.
-func merge(lBindings, rBindings Bindings) Bindings {
-	if lBindings == nil {
-		return rBindings
-	}
-	if rBindings == nil {
-		return lBindings
-	}
-	result := lBindings.Copy()
-	for i := range rBindings {
-		rbind := rBindings[i]
-		found := false
-		for j, lbind := range lBindings {
-			if lbind.isSame(&rbind) {
-				found = true
-				if rbind.UpdateTime.Compare(lbind.UpdateTime) >= 0 {
-					result[j] = rbind
-				}
-				break
-			}
-		}
-		if !found {
-			result = append(result, rbind)
-		}
-	}
-	return result
-}
-
-func removeDeletedBindings(br Bindings) Bindings {
-	result := make(Bindings, 0, len(br))
-	for _, binding := range br {
-		if binding.Status != deleted {
-			result = append(result, binding)
-		}
-	}
-	return result
-}
-
-// size calculates the memory size of a Bindings.
-func (br Bindings) size() float64 {
-	mem := float64(0)
-	for _, binding := range br {
-		mem += binding.size()
-	}
-	return mem
-}
-
-// size calculates the memory size of a bind info.
-func (b *Binding) size() float64 {
-	res := len(b.OriginalSQL) + len(b.Db) + len(b.BindSQL) + len(b.Status) + 2*int(unsafe.Sizeof(b.CreateTime)) + len(b.Charset) + len(b.Collation) + len(b.ID)
-	return float64(res)
-}

From f67411dd84765a74853a48547ae49a2b11b15196 Mon Sep 17 00:00:00 2001
From: qw4990 <zhangyuanjia@pingcap.com>
Date: Sat, 8 Feb 2025 15:10:15 +0800
Subject: [PATCH 3/3] fixup

---
 .bazelversion        | 1 +
 bindinfo/BUILD.bazel | 1 +
 2 files changed, 2 insertions(+)
 create mode 100644 .bazelversion

diff --git a/.bazelversion b/.bazelversion
new file mode 100644
index 0000000000000..815da58b7a9ed
--- /dev/null
+++ b/.bazelversion
@@ -0,0 +1 @@
+7.4.1
diff --git a/bindinfo/BUILD.bazel b/bindinfo/BUILD.bazel
index 93b55bfab21c2..ad177460afc50 100644
--- a/bindinfo/BUILD.bazel
+++ b/bindinfo/BUILD.bazel
@@ -37,6 +37,7 @@ go_library(
         "//util/stmtsummary",
         "//util/table-filter",
         "//util/timeutil",
+        "@com_github_pingcap_errors//:errors",
         "@org_golang_x_exp//maps",
         "@org_uber_go_zap//:zap",
     ],