diff --git a/pkg/BUILD.bazel b/pkg/BUILD.bazel index 65a4420cfd08..c231f90d804c 100644 --- a/pkg/BUILD.bazel +++ b/pkg/BUILD.bazel @@ -505,6 +505,7 @@ ALL_TESTS = [ "//pkg/sql/gcjob/gcjobnotifier:gcjobnotifier_test", "//pkg/sql/gcjob:gcjob_test", "//pkg/sql/gcjob_test:gcjob_test_test", + "//pkg/sql/hintpb:hintpb_test", "//pkg/sql/hints:hints_test", "//pkg/sql/idxrecommendations:idxrecommendations_test", "//pkg/sql/idxusage:idxusage_test", @@ -2070,6 +2071,8 @@ GO_TARGETS = [ "//pkg/sql/gcjob:gcjob", "//pkg/sql/gcjob:gcjob_test", "//pkg/sql/gcjob_test:gcjob_test_test", + "//pkg/sql/hintpb:hintpb", + "//pkg/sql/hintpb:hintpb_test", "//pkg/sql/hints:hints", "//pkg/sql/hints:hints_test", "//pkg/sql/idxrecommendations:idxrecommendations", diff --git a/pkg/ccl/logictestccl/tests/3node-tenant/generated_test.go b/pkg/ccl/logictestccl/tests/3node-tenant/generated_test.go index 7ed5ccfed330..945d9bc2c86f 100644 --- a/pkg/ccl/logictestccl/tests/3node-tenant/generated_test.go +++ b/pkg/ccl/logictestccl/tests/3node-tenant/generated_test.go @@ -2108,6 +2108,13 @@ func TestTenantLogic_srfs( runLogicTest(t, "srfs") } +func TestTenantLogic_statement_hint_builtins( + t *testing.T, +) { + defer leaktest.AfterTest(t)() + runLogicTest(t, "statement_hint_builtins") +} + func TestTenantLogic_statement_source( t *testing.T, ) { diff --git a/pkg/ccl/logictestccl/tests/local-read-committed/generated_test.go b/pkg/ccl/logictestccl/tests/local-read-committed/generated_test.go index 08805cbcd888..5db2fd6d26f2 100644 --- a/pkg/ccl/logictestccl/tests/local-read-committed/generated_test.go +++ b/pkg/ccl/logictestccl/tests/local-read-committed/generated_test.go @@ -2120,6 +2120,13 @@ func TestReadCommittedLogic_srfs( runLogicTest(t, "srfs") } +func TestReadCommittedLogic_statement_hint_builtins( + t *testing.T, +) { + defer leaktest.AfterTest(t)() + runLogicTest(t, "statement_hint_builtins") +} + func TestReadCommittedLogic_statement_source( t *testing.T, ) { diff --git a/pkg/ccl/logictestccl/tests/local-repeatable-read/generated_test.go b/pkg/ccl/logictestccl/tests/local-repeatable-read/generated_test.go index 53cc8b776ca4..24626e0e9fe8 100644 --- a/pkg/ccl/logictestccl/tests/local-repeatable-read/generated_test.go +++ b/pkg/ccl/logictestccl/tests/local-repeatable-read/generated_test.go @@ -2106,6 +2106,13 @@ func TestRepeatableReadLogic_srfs( runLogicTest(t, "srfs") } +func TestRepeatableReadLogic_statement_hint_builtins( + t *testing.T, +) { + defer leaktest.AfterTest(t)() + runLogicTest(t, "statement_hint_builtins") +} + func TestRepeatableReadLogic_statement_source( t *testing.T, ) { diff --git a/pkg/gen/protobuf.bzl b/pkg/gen/protobuf.bzl index 7312e3ef74bd..4d05026a067e 100644 --- a/pkg/gen/protobuf.bzl +++ b/pkg/gen/protobuf.bzl @@ -63,7 +63,7 @@ PROTOBUF_SRCS = [ "//pkg/sql/catalog/schematelemetry/schematelemetrycontroller:schematelemetrycontroller_go_proto", "//pkg/sql/contentionpb:contentionpb_go_proto", "//pkg/sql/execinfrapb:execinfrapb_go_proto", - "//pkg/sql/hints:hints_go_proto", + "//pkg/sql/hintpb:hintpb_go_proto", "//pkg/sql/inverted:inverted_go_proto", "//pkg/sql/lex:lex_go_proto", "//pkg/sql/pgwire/pgerror:pgerror_go_proto", diff --git a/pkg/sql/BUILD.bazel b/pkg/sql/BUILD.bazel index 855d22741c62..ecd35bb7a7ae 100644 --- a/pkg/sql/BUILD.bazel +++ b/pkg/sql/BUILD.bazel @@ -442,6 +442,7 @@ go_library( "//pkg/sql/faketreeeval", "//pkg/sql/flowinfra", "//pkg/sql/gcjob/gcjobnotifier", + "//pkg/sql/hintpb", "//pkg/sql/hints", "//pkg/sql/idxrecommendations", "//pkg/sql/idxusage", diff --git a/pkg/sql/faketreeeval/BUILD.bazel b/pkg/sql/faketreeeval/BUILD.bazel index 4e464ac02fb9..39fa9f0d7e28 100644 --- a/pkg/sql/faketreeeval/BUILD.bazel +++ b/pkg/sql/faketreeeval/BUILD.bazel @@ -11,6 +11,7 @@ go_library( "//pkg/roachpb", "//pkg/security/username", "//pkg/sql/catalog/descpb", + "//pkg/sql/hintpb", "//pkg/sql/pgwire/pgcode", "//pkg/sql/pgwire/pgerror", "//pkg/sql/pgwire/pgnotice", diff --git a/pkg/sql/faketreeeval/evalctx.go b/pkg/sql/faketreeeval/evalctx.go index b28cbcbc7d47..41951239373f 100644 --- a/pkg/sql/faketreeeval/evalctx.go +++ b/pkg/sql/faketreeeval/evalctx.go @@ -15,6 +15,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/hintpb" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgnotice" @@ -589,6 +590,13 @@ func (ep *DummyEvalPlanner) ProcessVectorIndexFixups( return nil } +// InsertStatementHint is part of the eval.Planner interface. +func (ep *DummyEvalPlanner) InsertStatementHint( + ctx context.Context, statementFingerprint string, hint hintpb.StatementHintUnion, +) (int64, error) { + return 0, nil +} + // DummyPrivilegedAccessor implements the tree.PrivilegedAccessor interface by returning errors. type DummyPrivilegedAccessor struct{} diff --git a/pkg/sql/hintpb/BUILD.bazel b/pkg/sql/hintpb/BUILD.bazel new file mode 100644 index 000000000000..b47afd0bd7fe --- /dev/null +++ b/pkg/sql/hintpb/BUILD.bazel @@ -0,0 +1,39 @@ +load("@rules_proto//proto:defs.bzl", "proto_library") +load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") + +proto_library( + name = "hintpb_proto", + srcs = ["statement_hint.proto"], + strip_import_prefix = "/pkg", + visibility = ["//visibility:public"], + deps = ["@com_github_gogo_protobuf//gogoproto:gogo_proto"], +) + +go_proto_library( + name = "hintpb_go_proto", + compilers = ["//pkg/cmd/protoc-gen-gogoroach:protoc-gen-gogoroach_compiler"], + importpath = "github.com/cockroachdb/cockroach/pkg/sql/hintpb", + proto = ":hintpb_proto", + visibility = ["//visibility:public"], + deps = ["@com_github_gogo_protobuf//gogoproto"], +) + +go_library( + name = "hintpb", + srcs = ["statement_hint.go"], + embed = [":hintpb_go_proto"], + importpath = "github.com/cockroachdb/cockroach/pkg/sql/hintpb", + visibility = ["//visibility:public"], + deps = [ + "//pkg/util/protoutil", + "@com_github_cockroachdb_errors//:errors", + ], +) + +go_test( + name = "hintpb_test", + srcs = ["statement_hint_test.go"], + embed = [":hintpb"], + deps = ["@com_github_stretchr_testify//require"], +) diff --git a/pkg/sql/hintpb/statement_hint.go b/pkg/sql/hintpb/statement_hint.go new file mode 100644 index 000000000000..96bc000347a7 --- /dev/null +++ b/pkg/sql/hintpb/statement_hint.go @@ -0,0 +1,33 @@ +// Copyright 2025 The Cockroach Authors. +// +// Use of this software is governed by the CockroachDB Software License +// included in the /LICENSE file. + +package hintpb + +import ( + "github.com/cockroachdb/cockroach/pkg/util/protoutil" + "github.com/cockroachdb/errors" +) + +// FromBytes converts the raw bytes from system.statement_hints into a +// StatementHintUnion object. +func FromBytes(bytes []byte) (StatementHintUnion, error) { + res := StatementHintUnion{} + if err := protoutil.Unmarshal(bytes, &res); err != nil { + return StatementHintUnion{}, err + } + if res.GetValue() == nil { + return StatementHintUnion{}, errors.New("invalid hint bytes: no value set") + } + return res, nil +} + +// ToBytes converts the StatementHintUnion to a raw bytes representation that +// can be inserted into the system.statement_hints table. +func ToBytes(hint StatementHintUnion) ([]byte, error) { + if hint.GetValue() == nil { + return nil, errors.New("cannot convert empty hint to bytes") + } + return protoutil.Marshal(&hint) +} diff --git a/pkg/sql/hintpb/statement_hint.proto b/pkg/sql/hintpb/statement_hint.proto new file mode 100644 index 000000000000..a35659db2ccb --- /dev/null +++ b/pkg/sql/hintpb/statement_hint.proto @@ -0,0 +1,26 @@ +// Copyright 2025 The Cockroach Authors. +// +// Use of this software is governed by the CockroachDB Software License +// included in the /LICENSE file. + +syntax = "proto3"; +package hintpb; + +import "gogoproto/gogo.proto"; + +// StatementHintsUnion contains exactly one type of "external" statement hint +// that can be applied to statements with a given fingerprint using the +// system.statement_hints table. +message StatementHintUnion { + option (gogoproto.onlyone) = true; + + InjectHints inject_hints = 1; +} + +// InjectHints applies inline query plan hints (join and index hints) from the +// DonorSQL field to the hinted statement. The DonorSQL string must have +// the same syntactical structure as the hinted statement, with only the +// addition of inline hints. +message InjectHints { + string donor_sql = 1 [(gogoproto.customname) = "DonorSQL"]; +} diff --git a/pkg/sql/hintpb/statement_hint_test.go b/pkg/sql/hintpb/statement_hint_test.go new file mode 100644 index 000000000000..e0cac8bd826c --- /dev/null +++ b/pkg/sql/hintpb/statement_hint_test.go @@ -0,0 +1,42 @@ +// Copyright 2025 The Cockroach Authors. +// +// Use of this software is governed by the CockroachDB Software License +// included in the /LICENSE file. + +package hintpb + +import ( + "testing" + + "github.com/stretchr/testify/require" +) + +func TestFromToBytes(t *testing.T) { + // Test writing empty hint. + _, err := ToBytes(StatementHintUnion{}) + require.EqualError(t, err, "cannot convert empty hint to bytes") + + // Test reading empty bytes. + _, err = FromBytes(nil) + require.EqualError(t, err, "invalid hint bytes: no value set") + _, err = FromBytes([]byte{}) + require.EqualError(t, err, "invalid hint bytes: no value set") + + // Test reading invalid bytes. + _, err = FromBytes([]byte{0xFF, 0xFF, 0xFF}) + require.Error(t, err) + + // Test that a valid hint round trips. + testRT := func(hint interface{}) { + var hintUnion StatementHintUnion + hintUnion.SetValue(hint) + bytes, err := ToBytes(hintUnion) + require.NoError(t, err) + require.NotEmpty(t, bytes) + decodedHintUnion, err := FromBytes(bytes) + require.NoError(t, err) + require.Equal(t, hint, decodedHintUnion.GetValue()) + } + testRT(&InjectHints{}) + testRT(&InjectHints{DonorSQL: "SELECT * FROM t"}) +} diff --git a/pkg/sql/hints/BUILD.bazel b/pkg/sql/hints/BUILD.bazel index 302bb56119cc..2f2cddc3798b 100644 --- a/pkg/sql/hints/BUILD.bazel +++ b/pkg/sql/hints/BUILD.bazel @@ -1,14 +1,11 @@ -load("@rules_proto//proto:defs.bzl", "proto_library") -load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") go_library( name = "hints", srcs = [ "hint_cache.go", - "statement_hint.go", + "hint_table.go", ], - embed = [":hints_go_proto"], importpath = "github.com/cockroachdb/cockroach/pkg/sql/hints", visibility = ["//visibility:public"], deps = [ @@ -25,6 +22,8 @@ go_library( "//pkg/sql/catalog/descpb", "//pkg/sql/catalog/descs", "//pkg/sql/catalog/systemschema", + "//pkg/sql/hintpb", + "//pkg/sql/isql", "//pkg/sql/rowenc", "//pkg/sql/sem/tree", "//pkg/sql/sessiondata", @@ -34,7 +33,6 @@ go_library( "//pkg/util/hlc", "//pkg/util/log", "//pkg/util/metamorphic", - "//pkg/util/protoutil", "//pkg/util/retry", "//pkg/util/startup", "//pkg/util/stop", @@ -43,28 +41,12 @@ go_library( ], ) -go_proto_library( - name = "hints_go_proto", - compilers = ["//pkg/cmd/protoc-gen-gogoroach:protoc-gen-gogoroach_compiler"], - importpath = "github.com/cockroachdb/cockroach/pkg/sql/hints", - proto = ":hints_proto", - visibility = ["//visibility:public"], - deps = ["@com_github_gogo_protobuf//gogoproto"], -) - -proto_library( - name = "hints_proto", - srcs = ["statement_hint.proto"], - strip_import_prefix = "/pkg", - visibility = ["//visibility:public"], - deps = ["@com_github_gogo_protobuf//gogoproto:gogo_proto"], -) - go_test( name = "hints_test", size = "medium", srcs = [ "hint_cache_test.go", + "hint_table_test.go", "main_test.go", ], exec_properties = select({ @@ -80,6 +62,8 @@ go_test( "//pkg/server", "//pkg/sql/catalog", "//pkg/sql/catalog/descs", + "//pkg/sql/hintpb", + "//pkg/sql/isql", "//pkg/sql/randgen", "//pkg/sql/stats", "//pkg/testutils", diff --git a/pkg/sql/hints/hint_cache.go b/pkg/sql/hints/hint_cache.go index a61bae3b3d4d..1d4e61a44666 100644 --- a/pkg/sql/hints/hint_cache.go +++ b/pkg/sql/hints/hint_cache.go @@ -25,9 +25,9 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" "github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema" + "github.com/cockroachdb/cockroach/pkg/sql/hintpb" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" - "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/buildutil" "github.com/cockroachdb/cockroach/pkg/util/cache" @@ -146,8 +146,6 @@ func NewStatementHintsCache( func (c *StatementHintsCache) Start( ctx context.Context, sysTableResolver catalog.SystemTableIDResolver, ) error { - // TODO(drewk): in a follow-up commit, block until the initial scan is done - // so that statements without hints don't have to check the LRU cache. return c.startRangefeedInternal(ctx, sysTableResolver) } @@ -292,7 +290,7 @@ func (c *StatementHintsCache) checkHashHasHintsAsync( }) for { log.VEventf(ctx, 1, "checking hints for fingerprint hash %d @ %v", hash, refreshTS) - hasHints, err := c.checkForStatementHintsInDB(ctx, hash) + hasHints, err := CheckForStatementHintsInDB(ctx, c.db.Executor(), hash) if err != nil { log.Dev.Warningf(ctx, "failed to check hints for hash %d: %v", hash, err) if retryOnError.Next() { @@ -324,27 +322,6 @@ func (c *StatementHintsCache) checkHashHasHintsAsync( }) } -// checkForStatementHintsInDB queries the system.statement_hints table to -// determine if there are any hints for the given fingerprint hash. The caller -// must be able to retry if an error is returned. -func (c *StatementHintsCache) checkForStatementHintsInDB( - ctx context.Context, statementHash int64, -) (hasHints bool, retErr error) { - const opName = "get-plan-hints" - const getHintsStmt = `SELECT hash FROM system.statement_hints WHERE "hash" = $1 LIMIT 1` - it, err := c.db.Executor().QueryIteratorEx( - ctx, opName, nil /* txn */, sessiondata.NodeUserSessionDataOverride, - getHintsStmt, statementHash, - ) - if err != nil { - return false, err - } - defer func() { - retErr = errors.CombineErrors(retErr, it.Close()) - }() - return it.Next(ctx) -} - // decodeHashFromStatementHintsKey decodes the query hash from a range feed // event on system.statement_hints. func decodeHashFromStatementHintsKey( @@ -392,7 +369,7 @@ func (c *StatementHintsCache) GetGeneration() int64 { // retrieving them. func (c *StatementHintsCache) MaybeGetStatementHints( ctx context.Context, statementFingerprint string, -) (hints []StatementHint, ids []int64) { +) (hints []hintpb.StatementHintUnion, ids []int64) { hash := fnv.New64() _, err := hash.Write([]byte(statementFingerprint)) if err != nil { @@ -450,7 +427,7 @@ func (c *StatementHintsCache) maybeWaitForRefreshLocked( // released while reading from the db, and then reacquired. func (c *StatementHintsCache) addCacheEntryLocked( ctx context.Context, statementHash int64, statementFingerprint string, -) (hints []StatementHint, ids []int64) { +) (hints []hintpb.StatementHintUnion, ids []int64) { c.mu.AssertHeld() // Add a cache entry that other queries can find and wait on until we have the @@ -467,7 +444,8 @@ func (c *StatementHintsCache) addCacheEntryLocked( c.mu.Unlock() defer c.mu.Lock() log.VEventf(ctx, 1, "reading hints for query %s", statementFingerprint) - err = c.getStatementHintsFromDB(ctx, statementHash, entry) + entry.ids, entry.fingerprints, entry.hints, err = + GetStatementHintsFromDB(ctx, c.db.Executor(), statementHash) log.VEventf(ctx, 1, "finished reading hints for query %s", statementFingerprint) }() @@ -484,47 +462,6 @@ func (c *StatementHintsCache) addCacheEntryLocked( return entry.getMatchingHints(statementFingerprint) } -// getStatementHintsFromDB queries the system.statement_hints table for hints -// matching the given fingerprint hash. It is able to handle the case when -// multiple fingerprints match the hash, as well as the case when there are no -// hints for the fingerprint. Results are ordered by row ID. -func (c *StatementHintsCache) getStatementHintsFromDB( - ctx context.Context, statementHash int64, entry *cacheEntry, -) (retErr error) { - const opName = "get-plan-hints" - const getHintsStmt = ` - SELECT "row_id", "fingerprint", "hint" - FROM system.statement_hints - WHERE "hash" = $1 - ORDER BY "row_id" ASC` - it, err := c.db.Executor().QueryIteratorEx( - ctx, opName, nil /* txn */, sessiondata.NodeUserSessionDataOverride, - getHintsStmt, statementHash, - ) - if err != nil { - return err - } - defer func() { - retErr = errors.CombineErrors(retErr, it.Close()) - }() - for { - ok, err := it.Next(ctx) - if !ok || err != nil { - return err - } - datums := it.Cur() - rowID := int64(tree.MustBeDInt(datums[0])) - fingerprint := string(tree.MustBeDString(datums[1])) - hint, err := NewStatementHint([]byte(tree.MustBeDBytes(datums[2]))) - if err != nil { - return err - } - entry.ids = append(entry.ids, rowID) - entry.fingerprints = append(entry.fingerprints, fingerprint) - entry.hints = append(entry.hints, hint) - } -} - type cacheEntry struct { // If mustWait is true, we do not have any hints for this hash, and we are in // the process of fetching them from the database. Other callers can wait on @@ -542,7 +479,7 @@ type cacheEntry struct { // be duplicate entries in the fingerprints slice. // TODO(drewk): consider de-duplicating the fingerprint strings to reduce // memory usage. - hints []StatementHint + hints []hintpb.StatementHintUnion fingerprints []string ids []int64 } @@ -551,7 +488,7 @@ type cacheEntry struct { // fingerprint, or nil if they don't exist. The results are in order of row ID. func (entry *cacheEntry) getMatchingHints( statementFingerprint string, -) (hints []StatementHint, ids []int64) { +) (hints []hintpb.StatementHintUnion, ids []int64) { for i := range entry.hints { if entry.fingerprints[i] == statementFingerprint { hints = append(hints, entry.hints[i]) diff --git a/pkg/sql/hints/hint_cache_test.go b/pkg/sql/hints/hint_cache_test.go index ffbedd59679e..fdc81f3b6719 100644 --- a/pkg/sql/hints/hint_cache_test.go +++ b/pkg/sql/hints/hint_cache_test.go @@ -15,6 +15,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" + "github.com/cockroachdb/cockroach/pkg/sql/hintpb" "github.com/cockroachdb/cockroach/pkg/sql/hints" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" @@ -538,11 +539,13 @@ func checkIDOrder(t *testing.T, ids []int64) { } } -// insertStatementHint inserts an empty statement hint into the +// insertStatementHint inserts a random statement hint into the // system.statement_hints table. func insertStatementHint(t *testing.T, r *sqlutils.SQLRunner, fingerprint string) { - emptyHint := &hints.StatementHintUnion{} - hintBytes, err := emptyHint.ToBytes() + // TODO(drewk,michae2): randomly choose the hint type once we support others. + var hint hintpb.StatementHintUnion + hint.SetValue(&hintpb.InjectHints{}) + hintBytes, err := hintpb.ToBytes(hint) require.NoError(t, err) const insertStmt = `INSERT INTO system.statement_hints ("fingerprint", "hint") VALUES ($1, $2)` r.Exec(t, insertStmt, fingerprint, hintBytes) diff --git a/pkg/sql/hints/hint_table.go b/pkg/sql/hints/hint_table.go new file mode 100644 index 000000000000..eacf5f5de349 --- /dev/null +++ b/pkg/sql/hints/hint_table.go @@ -0,0 +1,106 @@ +// Copyright 2025 The Cockroach Authors. +// +// Use of this software is governed by the CockroachDB Software License +// included in the /LICENSE file. + +package hints + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/sql/hintpb" + "github.com/cockroachdb/cockroach/pkg/sql/isql" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" + "github.com/cockroachdb/errors" +) + +// CheckForStatementHintsInDB queries the system.statement_hints table to +// determine if there are any hints for the given fingerprint hash. The caller +// must be able to retry if an error is returned. +func CheckForStatementHintsInDB( + ctx context.Context, ex isql.Executor, statementHash int64, +) (hasHints bool, retErr error) { + const opName = "get-plan-hints" + const getHintsStmt = `SELECT hash FROM system.statement_hints WHERE "hash" = $1 LIMIT 1` + it, err := ex.QueryIteratorEx( + ctx, opName, nil /* txn */, sessiondata.NodeUserSessionDataOverride, + getHintsStmt, statementHash, + ) + if err != nil { + return false, err + } + defer func() { + retErr = errors.CombineErrors(retErr, it.Close()) + }() + return it.Next(ctx) +} + +// GetStatementHintsFromDB queries the system.statement_hints table for hints +// matching the given fingerprint hash. It is able to handle the case when +// multiple fingerprints match the hash, as well as the case when there are no +// hints for the fingerprint. +// +// The returned slices (hints, fingerprints, and hintIDs) have the same length. +// fingerprints[i] is the statement fingerprint to which hints[i] applies, while +// hintIDs[i] uniquely identifies a hint in the system table. The results are in +// order of hint ID. +func GetStatementHintsFromDB( + ctx context.Context, ex isql.Executor, statementHash int64, +) (hintIDs []int64, fingerprints []string, hints []hintpb.StatementHintUnion, retErr error) { + const opName = "get-plan-hints" + const getHintsStmt = ` + SELECT "row_id", "fingerprint", "hint" + FROM system.statement_hints + WHERE "hash" = $1 + ORDER BY "row_id" ASC` + it, err := ex.QueryIteratorEx( + ctx, opName, nil /* txn */, sessiondata.NodeUserSessionDataOverride, + getHintsStmt, statementHash, + ) + if err != nil { + return nil, nil, nil, err + } + defer func() { + retErr = errors.CombineErrors(retErr, it.Close()) + }() + for { + ok, err := it.Next(ctx) + if err != nil { + return nil, nil, nil, err + } + if !ok { + break + } + datums := it.Cur() + hintIDs = append(hintIDs, int64(tree.MustBeDInt(datums[0]))) + fingerprints = append(fingerprints, string(tree.MustBeDString(datums[1]))) + hint, err := hintpb.FromBytes([]byte(tree.MustBeDBytes(datums[2]))) + if err != nil { + return nil, nil, nil, err + } + hints = append(hints, hint) + } + return hintIDs, fingerprints, hints, nil +} + +// InsertHintIntoDB inserts a statement hint into the system.statement_hints +// table. It returns the hint ID of the newly inserted hint if successful. +func InsertHintIntoDB( + ctx context.Context, txn isql.Txn, fingerprint string, hint hintpb.StatementHintUnion, +) (int64, error) { + const opName = "insert-statement-hint" + hintBytes, err := hintpb.ToBytes(hint) + if err != nil { + return 0, err + } + const insertStmt = `INSERT INTO system.statement_hints ("fingerprint", "hint") VALUES ($1, $2) RETURNING "row_id"` + row, err := txn.QueryRowEx( + ctx, opName, txn.KV(), sessiondata.NodeUserSessionDataOverride, + insertStmt, fingerprint, hintBytes, + ) + if err != nil { + return 0, err + } + return int64(tree.MustBeDInt(row[0])), nil +} diff --git a/pkg/sql/hints/hint_table_test.go b/pkg/sql/hints/hint_table_test.go new file mode 100644 index 000000000000..1961ed423189 --- /dev/null +++ b/pkg/sql/hints/hint_table_test.go @@ -0,0 +1,130 @@ +// Copyright 2025 The Cockroach Authors. +// +// Use of this software is governed by the CockroachDB Software License +// included in the /LICENSE file. + +package hints_test + +import ( + "context" + "testing" + + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" + "github.com/cockroachdb/cockroach/pkg/sql/hintpb" + "github.com/cockroachdb/cockroach/pkg/sql/hints" + "github.com/cockroachdb/cockroach/pkg/sql/isql" + "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/stretchr/testify/require" +) + +// TestHintTableOperations tests the DB-interfacing functions in hint_table.go: +// CheckForStatementHintsInDB, GetStatementHintsFromDB, and InsertHintIntoDB. +func TestHintTableOperations(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + ctx := context.Background() + srv := serverutils.StartServerOnly(t, base.TestServerArgs{}) + defer srv.Stopper().Stop(ctx) + ts := srv.ApplicationLayer() + db := ts.InternalDB().(descs.DB) + ex := db.Executor() + + // Create test hints. + fingerprint1 := "SELECT a FROM t WHERE b = $1" + fingerprint2 := "SELECT c FROM t WHERE d = $2" + hash1 := computeHash(t, fingerprint1) + hash2 := computeHash(t, fingerprint2) + + var hint1, hint2 hintpb.StatementHintUnion + hint1.SetValue(&hintpb.InjectHints{DonorSQL: "SELECT a FROM t@t_b_idx WHERE b = $1"}) + hint2.SetValue(&hintpb.InjectHints{DonorSQL: "SELECT c FROM t@{NO_FULL_SCAN} WHERE d = $2"}) + + // Check for nonexistent hints. + hasHints, err := hints.CheckForStatementHintsInDB(ctx, ex, hash1) + require.NoError(t, err) + require.False(t, hasHints) + + // Retrieve nonexistent hints. + hintIDs, fingerprints, hintsFromDB, err := hints.GetStatementHintsFromDB(ctx, ex, hash1) + require.NoError(t, err) + require.Empty(t, hintIDs) + require.Empty(t, fingerprints) + require.Empty(t, hintsFromDB) + + // Insert a hint. + var insertedHintID1 int64 + err = db.Txn(ctx, func(ctx context.Context, txn isql.Txn) error { + insertedHintID1, err = hints.InsertHintIntoDB(ctx, txn, fingerprint1, hint1) + return err + }) + require.NoError(t, err) + require.Greater(t, insertedHintID1, int64(0)) // Should return a valid ID. + + // Check for the inserted hint. + hasHints, err = hints.CheckForStatementHintsInDB(ctx, ex, hash1) + require.NoError(t, err) + require.True(t, hasHints) + + // Fetch the inserted hint. + hintIDs, fingerprints, hintsFromDB, err = hints.GetStatementHintsFromDB(ctx, ex, hash1) + require.NoError(t, err) + require.Len(t, hintIDs, 1) + require.Len(t, fingerprints, 1) + require.Len(t, hintsFromDB, 1) + require.Equal(t, fingerprint1, fingerprints[0]) + require.Equal(t, hint1, hintsFromDB[0]) + require.Equal(t, insertedHintID1, hintIDs[0]) + + // Insert multiple hints for the same fingerprint. + var insertedHintID2 int64 + err = db.Txn(ctx, func(ctx context.Context, txn isql.Txn) error { + insertedHintID2, err = hints.InsertHintIntoDB(ctx, txn, fingerprint1, hint1) + return err + }) + require.NoError(t, err) + require.Greater(t, insertedHintID2, int64(0)) + require.NotEqual(t, insertedHintID1, insertedHintID2) + + // Fetch all hints for the fingerprint. + hintIDs, fingerprints, hintsFromDB, err = hints.GetStatementHintsFromDB(ctx, ex, hash1) + require.NoError(t, err) + require.Len(t, hintIDs, 2) + require.Len(t, fingerprints, 2) + require.Len(t, hintsFromDB, 2) + require.Equal(t, fingerprint1, fingerprints[0]) + require.Equal(t, fingerprint1, fingerprints[1]) + require.Less(t, hintIDs[0], hintIDs[1]) + + // Insert hint for different fingerprint. + var insertedHintID3 int64 + err = db.Txn(ctx, func(ctx context.Context, txn isql.Txn) error { + insertedHintID3, err = hints.InsertHintIntoDB(ctx, txn, fingerprint2, hint2) + return err + }) + require.NoError(t, err) + require.Greater(t, insertedHintID3, int64(0)) + + // Retrieve hint for the new fingerprint. + hintIDs2, fingerprints2, hintsFromDB2, err := hints.GetStatementHintsFromDB(ctx, ex, hash2) + require.NoError(t, err) + require.Len(t, hintIDs2, 1) + require.Len(t, fingerprints2, 1) + require.Len(t, hintsFromDB2, 1) + require.Equal(t, fingerprint2, fingerprints2[0]) + require.Equal(t, insertedHintID3, hintIDs2[0]) + + // Test InsertHintIntoDB with empty fingerprint and hint. + var emptyFingerprintHintID int64 + var hintEmpty hintpb.StatementHintUnion + hintEmpty.SetValue(&hintpb.InjectHints{}) + err = db.Txn(ctx, func(ctx context.Context, txn isql.Txn) error { + emptyFingerprintHintID, err = hints.InsertHintIntoDB(ctx, txn, "", hintEmpty) + return err + }) + require.NoError(t, err) + require.Greater(t, emptyFingerprintHintID, int64(0)) +} diff --git a/pkg/sql/hints/statement_hint.go b/pkg/sql/hints/statement_hint.go deleted file mode 100644 index 8d70342fe0cb..000000000000 --- a/pkg/sql/hints/statement_hint.go +++ /dev/null @@ -1,28 +0,0 @@ -// Copyright 2025 The Cockroach Authors. -// -// Use of this software is governed by the CockroachDB Software License -// included in the /LICENSE file. - -package hints - -import "github.com/cockroachdb/cockroach/pkg/util/protoutil" - -type StatementHint interface { - protoutil.Message -} - -// NewStatementHint converts the raw bytes from system.statement_hints into a -// StatementHintUnion object. -func NewStatementHint(bytes []byte) (*StatementHintUnion, error) { - res := &StatementHintUnion{} - if err := protoutil.Unmarshal(bytes, res); err != nil { - return nil, err - } - return res, nil -} - -// ToBytes converts the StatementHintUnion to a raw bytes representation that -// can be inserted into the system.statement_hints table. -func (hint *StatementHintUnion) ToBytes() ([]byte, error) { - return protoutil.Marshal(hint) -} diff --git a/pkg/sql/hints/statement_hint.proto b/pkg/sql/hints/statement_hint.proto deleted file mode 100644 index b057aa54430f..000000000000 --- a/pkg/sql/hints/statement_hint.proto +++ /dev/null @@ -1,17 +0,0 @@ -// Copyright 2025 The Cockroach Authors. -// -// Use of this software is governed by the CockroachDB Software License -// included in the /LICENSE file. - -syntax = "proto3"; -package hints; - -import "gogoproto/gogo.proto"; - -// StatementHintsUnion contains exactly one type of "external" statement hint -// that can be applied to statements with a given fingerprint using the -// system.statement_hints table. -message StatementHintUnion { - // TODO: add a field for each type of statement hint. - // oneof value {} -} diff --git a/pkg/sql/logictest/testdata/logic_test/statement_hint_builtins b/pkg/sql/logictest/testdata/logic_test/statement_hint_builtins new file mode 100644 index 000000000000..0a92a462d1bc --- /dev/null +++ b/pkg/sql/logictest/testdata/logic_test/statement_hint_builtins @@ -0,0 +1,94 @@ +# LogicTest: !local-mixed-25.2 !local-mixed-25.3 + +statement ok +CREATE TABLE xy (x INT PRIMARY KEY, y INT, INDEX (y)); + +statement ok +CREATE TABLE ab (a INT PRIMARY KEY, b INT, INDEX (b)); + +query I +SELECT count(*) FROM system.statement_hints; +---- +0 + +let $hint1 +SELECT crdb_internal.inject_hint( + 'SELECT * FROM xy WHERE y = 10', + 'SELECT * FROM xy@primary WHERE y = 10' +); + +# Verify that the returned hint ID is in the table. +query T +SELECT fingerprint FROM system.statement_hints WHERE row_id = $hint1; +---- +SELECT * FROM xy WHERE y = 10 + +let $hint2 +SELECT crdb_internal.inject_hint( + 'SELECT * FROM xy WHERE y = 10', + 'SELECT * FROM xy@xy_y_idx WHERE y = 10' +); + +# Verify that the returned hint ID is in the table. +query T +SELECT fingerprint FROM system.statement_hints WHERE row_id = $hint2; +---- +SELECT * FROM xy WHERE y = 10 + +let $hint3 +SELECT crdb_internal.inject_hint( + 'SELECT * FROM xy INNER JOIN ab ON xy.x = ab.b', + 'SELECT * FROM xy INNER JOIN ab ON xy.x = ab.b' +); + +# Verify that the returned hint ID is in the table. +query T +SELECT fingerprint FROM system.statement_hints WHERE row_id = $hint3; +---- +SELECT * FROM xy INNER JOIN ab ON xy.x = ab.b + +query I +SELECT count(*) FROM system.statement_hints; +---- +3 + +query I +SELECT count(*) FROM system.statement_hints WHERE fingerprint = 'SELECT * FROM xy WHERE y = 10'; +---- +2 + +query I +SELECT count(*) FROM system.statement_hints WHERE fingerprint = 'SELECT * FROM xy INNER JOIN ab ON xy.x = ab.b'; +---- +1 + +statement ok +DELETE FROM system.statement_hints WHERE true; + +query I +SELECT count(*) FROM system.statement_hints; +---- +0 + +# If the surrounding transaction is rolled back, no hint should be inserted. +statement ok +BEGIN; + +let $hint4 +SELECT crdb_internal.inject_hint( + 'SELECT * FROM xy WHERE y = 10', + 'SELECT * FROM xy@{NO_FULL_SCAN} WHERE y = 10' +); + +query T +SELECT fingerprint FROM system.statement_hints WHERE row_id = $hint4; +---- +SELECT * FROM xy WHERE y = 10 + +statement ok +ROLLBACK; + +query I +SELECT count(*) FROM system.statement_hints; +---- +0 diff --git a/pkg/sql/logictest/tests/fakedist-disk/generated_test.go b/pkg/sql/logictest/tests/fakedist-disk/generated_test.go index fe4b77025435..6046d0a34272 100644 --- a/pkg/sql/logictest/tests/fakedist-disk/generated_test.go +++ b/pkg/sql/logictest/tests/fakedist-disk/generated_test.go @@ -2082,6 +2082,13 @@ func TestLogic_srfs( runLogicTest(t, "srfs") } +func TestLogic_statement_hint_builtins( + t *testing.T, +) { + defer leaktest.AfterTest(t)() + runLogicTest(t, "statement_hint_builtins") +} + func TestLogic_statement_source( t *testing.T, ) { diff --git a/pkg/sql/logictest/tests/fakedist-vec-off/generated_test.go b/pkg/sql/logictest/tests/fakedist-vec-off/generated_test.go index 09e1a8afaf61..d78d467320f6 100644 --- a/pkg/sql/logictest/tests/fakedist-vec-off/generated_test.go +++ b/pkg/sql/logictest/tests/fakedist-vec-off/generated_test.go @@ -2082,6 +2082,13 @@ func TestLogic_srfs( runLogicTest(t, "srfs") } +func TestLogic_statement_hint_builtins( + t *testing.T, +) { + defer leaktest.AfterTest(t)() + runLogicTest(t, "statement_hint_builtins") +} + func TestLogic_statement_source( t *testing.T, ) { diff --git a/pkg/sql/logictest/tests/fakedist/generated_test.go b/pkg/sql/logictest/tests/fakedist/generated_test.go index 220104121321..6b5361b1a63c 100644 --- a/pkg/sql/logictest/tests/fakedist/generated_test.go +++ b/pkg/sql/logictest/tests/fakedist/generated_test.go @@ -2103,6 +2103,13 @@ func TestLogic_srfs( runLogicTest(t, "srfs") } +func TestLogic_statement_hint_builtins( + t *testing.T, +) { + defer leaktest.AfterTest(t)() + runLogicTest(t, "statement_hint_builtins") +} + func TestLogic_statement_source( t *testing.T, ) { diff --git a/pkg/sql/logictest/tests/local-legacy-schema-changer/generated_test.go b/pkg/sql/logictest/tests/local-legacy-schema-changer/generated_test.go index fe1e0f7fc587..d7964924bc84 100644 --- a/pkg/sql/logictest/tests/local-legacy-schema-changer/generated_test.go +++ b/pkg/sql/logictest/tests/local-legacy-schema-changer/generated_test.go @@ -2061,6 +2061,13 @@ func TestLogic_srfs( runLogicTest(t, "srfs") } +func TestLogic_statement_hint_builtins( + t *testing.T, +) { + defer leaktest.AfterTest(t)() + runLogicTest(t, "statement_hint_builtins") +} + func TestLogic_statement_source( t *testing.T, ) { diff --git a/pkg/sql/logictest/tests/local-prepared/generated_test.go b/pkg/sql/logictest/tests/local-prepared/generated_test.go index a27fd5fb3d06..d1f4869bc2bc 100644 --- a/pkg/sql/logictest/tests/local-prepared/generated_test.go +++ b/pkg/sql/logictest/tests/local-prepared/generated_test.go @@ -1368,6 +1368,13 @@ func TestLogic_sqlsmith( runLogicTest(t, "sqlsmith") } +func TestLogic_statement_hint_builtins( + t *testing.T, +) { + defer leaktest.AfterTest(t)() + runLogicTest(t, "statement_hint_builtins") +} + func TestLogic_statement_statistics_errors( t *testing.T, ) { diff --git a/pkg/sql/logictest/tests/local-vec-off/generated_test.go b/pkg/sql/logictest/tests/local-vec-off/generated_test.go index 33c8ea9e3d4a..826152030d97 100644 --- a/pkg/sql/logictest/tests/local-vec-off/generated_test.go +++ b/pkg/sql/logictest/tests/local-vec-off/generated_test.go @@ -2110,6 +2110,13 @@ func TestLogic_srfs( runLogicTest(t, "srfs") } +func TestLogic_statement_hint_builtins( + t *testing.T, +) { + defer leaktest.AfterTest(t)() + runLogicTest(t, "statement_hint_builtins") +} + func TestLogic_statement_source( t *testing.T, ) { diff --git a/pkg/sql/logictest/tests/local/generated_test.go b/pkg/sql/logictest/tests/local/generated_test.go index cdf226600493..783f24dca4ab 100644 --- a/pkg/sql/logictest/tests/local/generated_test.go +++ b/pkg/sql/logictest/tests/local/generated_test.go @@ -2334,6 +2334,13 @@ func TestLogic_srfs( runLogicTest(t, "srfs") } +func TestLogic_statement_hint_builtins( + t *testing.T, +) { + defer leaktest.AfterTest(t)() + runLogicTest(t, "statement_hint_builtins") +} + func TestLogic_statement_source( t *testing.T, ) { diff --git a/pkg/sql/planner.go b/pkg/sql/planner.go index 608c32cdbc08..bd7c88bf9610 100644 --- a/pkg/sql/planner.go +++ b/pkg/sql/planner.go @@ -34,6 +34,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/clusterunique" "github.com/cockroachdb/cockroach/pkg/sql/evalcatalog" "github.com/cockroachdb/cockroach/pkg/sql/exprutil" + "github.com/cockroachdb/cockroach/pkg/sql/hintpb" + "github.com/cockroachdb/cockroach/pkg/sql/hints" "github.com/cockroachdb/cockroach/pkg/sql/idxusage" "github.com/cockroachdb/cockroach/pkg/sql/parser" "github.com/cockroachdb/cockroach/pkg/sql/prep" @@ -1097,3 +1099,10 @@ func (p *planner) ProcessVectorIndexFixups( } return vi.ProcessFixups(ctx) } + +// InsertStatementHint is part of the eval.Planner interface. +func (p *planner) InsertStatementHint( + ctx context.Context, statementFingerprint string, hint hintpb.StatementHintUnion, +) (int64, error) { + return hints.InsertHintIntoDB(ctx, p.InternalSQLTxn(), statementFingerprint, hint) +} diff --git a/pkg/sql/sem/builtins/BUILD.bazel b/pkg/sql/sem/builtins/BUILD.bazel index e97f1875b227..a55dd6973f14 100644 --- a/pkg/sql/sem/builtins/BUILD.bazel +++ b/pkg/sql/sem/builtins/BUILD.bazel @@ -66,6 +66,7 @@ go_library( "//pkg/sql/catalog/descpb", "//pkg/sql/catalog/randgen/randgencfg", "//pkg/sql/colexecerror", + "//pkg/sql/hintpb", "//pkg/sql/lex", "//pkg/sql/lexbase", "//pkg/sql/memsize", diff --git a/pkg/sql/sem/builtins/builtins.go b/pkg/sql/sem/builtins/builtins.go index 19668df88a22..6ee651983c8d 100644 --- a/pkg/sql/sem/builtins/builtins.go +++ b/pkg/sql/sem/builtins/builtins.go @@ -46,6 +46,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/randgen/randgencfg" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/hintpb" "github.com/cockroachdb/cockroach/pkg/sql/lex" "github.com/cockroachdb/cockroach/pkg/sql/lexbase" "github.com/cockroachdb/cockroach/pkg/sql/parserutils" @@ -9685,6 +9686,34 @@ WHERE object_id = table_descriptor_id }, }, ), + + "crdb_internal.inject_hint": makeBuiltin( + tree.FunctionProperties{ + Category: builtinconstants.CategorySystemInfo, + DistsqlBlocklist: true, + }, + tree.Overload{ + Types: tree.ParamTypes{ + {Name: "statement_fingerprint", Typ: types.String}, + {Name: "donor_sql", Typ: types.String}, + }, + ReturnType: tree.FixedReturnType(types.Int), + Fn: func(ctx context.Context, evalCtx *eval.Context, args tree.Datums) (tree.Datum, error) { + stmtFingerprint := string(tree.MustBeDString(args[0])) + donorSQL := string(tree.MustBeDString(args[1])) + var hint hintpb.StatementHintUnion + hint.SetValue(&hintpb.InjectHints{DonorSQL: donorSQL}) + hintID, err := evalCtx.Planner.InsertStatementHint(ctx, stmtFingerprint, hint) + if err != nil { + return nil, err + } + return tree.NewDInt(tree.DInt(hintID)), nil + }, + Info: "This function is used to build a serialized statement hint to be inserted into" + + " the system.statement_hints table. It returns the hint ID of the newly created hint.", + Volatility: volatility.Volatile, + }, + ), } var lengthImpls = func(incBitOverload bool) builtinDefinition { diff --git a/pkg/sql/sem/builtins/fixed_oids.go b/pkg/sql/sem/builtins/fixed_oids.go index c3564a2bf9ef..7a516ab707ed 100644 --- a/pkg/sql/sem/builtins/fixed_oids.go +++ b/pkg/sql/sem/builtins/fixed_oids.go @@ -2860,6 +2860,7 @@ var builtinOidsArray = []string{ 2905: `levenshtein_less_equal(source: string, target: string, max_d: int) -> int`, 2906: `levenshtein_less_equal(source: string, target: string, ins_cost: int, del_cost: int, sub_cost: int, max_d: int) -> int`, 2907: `crdb_internal.request_transaction_bundle(transaction_fingerprint_id: string, sampling_probability: float, min_execution_latency: interval, expires_after: interval, redacted: bool) -> tuple{int AS request_id, bool AS created}`, + 2908: `crdb_internal.inject_hint(statement_fingerprint: string, donor_sql: string) -> int`, } var builtinOidsBySignature map[string]oid.Oid diff --git a/pkg/sql/sem/eval/BUILD.bazel b/pkg/sql/sem/eval/BUILD.bazel index d73a5e991ddf..a802b51e9331 100644 --- a/pkg/sql/sem/eval/BUILD.bazel +++ b/pkg/sql/sem/eval/BUILD.bazel @@ -55,6 +55,7 @@ go_library( "//pkg/settings/cluster", "//pkg/sql/catalog/catpb", "//pkg/sql/catalog/descpb", + "//pkg/sql/hintpb", "//pkg/sql/lex", "//pkg/sql/oidext", "//pkg/sql/parserutils", diff --git a/pkg/sql/sem/eval/deps.go b/pkg/sql/sem/eval/deps.go index 4143eb46e088..fae97980d496 100644 --- a/pkg/sql/sem/eval/deps.go +++ b/pkg/sql/sem/eval/deps.go @@ -14,6 +14,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/hintpb" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgnotice" "github.com/cockroachdb/cockroach/pkg/sql/privilege" "github.com/cockroachdb/cockroach/pkg/sql/sem/catid" @@ -462,6 +463,11 @@ type Planner interface { // ProcessVectorIndexFixups waits until all outstanding fixups for the vector // index with the given ID have been processed. ProcessVectorIndexFixups(ctx context.Context, tableID descpb.ID, indexID descpb.IndexID) error + + // InsertStatementHint adds a new hint for the given statement fingerprint to + // the system.statement_hints table. It returns the hint ID of the newly + // created hint. + InsertStatementHint(ctx context.Context, statementFingerprint string, hint hintpb.StatementHintUnion) (int64, error) } // InternalRows is an iterator interface that's exposed by the internal