Skip to content

Commit d78b3ad

Browse files
committed
hints: add injection hint and refactor usage of hint union
This commit adds the protobuf struct for injecting "internal" hints like (index or join hints) into a statement's AST. This can be serialized and inserted into the `system.statement_hints` table. This commit also refactors the way hints are represented in memory. Informs #153633 Release note: None
1 parent 0c9b934 commit d78b3ad

File tree

7 files changed

+94
-26
lines changed

7 files changed

+94
-26
lines changed

pkg/BUILD.bazel

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -505,6 +505,7 @@ ALL_TESTS = [
505505
"//pkg/sql/gcjob/gcjobnotifier:gcjobnotifier_test",
506506
"//pkg/sql/gcjob:gcjob_test",
507507
"//pkg/sql/gcjob_test:gcjob_test_test",
508+
"//pkg/sql/hintpb:hintpb_test",
508509
"//pkg/sql/hints:hints_test",
509510
"//pkg/sql/idxrecommendations:idxrecommendations_test",
510511
"//pkg/sql/idxusage:idxusage_test",
@@ -2071,6 +2072,7 @@ GO_TARGETS = [
20712072
"//pkg/sql/gcjob:gcjob_test",
20722073
"//pkg/sql/gcjob_test:gcjob_test_test",
20732074
"//pkg/sql/hintpb:hintpb",
2075+
"//pkg/sql/hintpb:hintpb_test",
20742076
"//pkg/sql/hints:hints",
20752077
"//pkg/sql/hints:hints_test",
20762078
"//pkg/sql/idxrecommendations:idxrecommendations",

pkg/sql/hintpb/BUILD.bazel

Lines changed: 12 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -1,5 +1,5 @@
11
load("@rules_proto//proto:defs.bzl", "proto_library")
2-
load("@io_bazel_rules_go//go:def.bzl", "go_library")
2+
load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test")
33
load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library")
44

55
proto_library(
@@ -25,5 +25,15 @@ go_library(
2525
embed = [":hintpb_go_proto"],
2626
importpath = "github.com/cockroachdb/cockroach/pkg/sql/hintpb",
2727
visibility = ["//visibility:public"],
28-
deps = ["//pkg/util/protoutil"],
28+
deps = [
29+
"//pkg/util/protoutil",
30+
"@com_github_cockroachdb_errors//:errors",
31+
],
32+
)
33+
34+
go_test(
35+
name = "hintpb_test",
36+
srcs = ["statement_hint_test.go"],
37+
embed = [":hintpb"],
38+
deps = ["@com_github_stretchr_testify//require"],
2939
)

pkg/sql/hintpb/statement_hint.go

Lines changed: 17 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -5,24 +5,29 @@
55

66
package hintpb
77

8-
import "github.com/cockroachdb/cockroach/pkg/util/protoutil"
8+
import (
9+
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
10+
"github.com/cockroachdb/errors"
11+
)
912

10-
type StatementHint interface {
11-
protoutil.Message
12-
}
13-
14-
// NewStatementHint converts the raw bytes from system.statement_hints into a
13+
// FromBytes converts the raw bytes from system.statement_hints into a
1514
// StatementHintUnion object.
16-
func NewStatementHint(bytes []byte) (*StatementHintUnion, error) {
17-
res := &StatementHintUnion{}
18-
if err := protoutil.Unmarshal(bytes, res); err != nil {
19-
return nil, err
15+
func FromBytes(bytes []byte) (StatementHintUnion, error) {
16+
res := StatementHintUnion{}
17+
if err := protoutil.Unmarshal(bytes, &res); err != nil {
18+
return StatementHintUnion{}, err
19+
}
20+
if res.GetValue() == nil {
21+
return StatementHintUnion{}, errors.New("invalid hint bytes: no value set")
2022
}
2123
return res, nil
2224
}
2325

2426
// ToBytes converts the StatementHintUnion to a raw bytes representation that
2527
// can be inserted into the system.statement_hints table.
26-
func (hint *StatementHintUnion) ToBytes() ([]byte, error) {
27-
return protoutil.Marshal(hint)
28+
func ToBytes(hint StatementHintUnion) ([]byte, error) {
29+
if hint.GetValue() == nil {
30+
return nil, errors.New("cannot convert empty hint to bytes")
31+
}
32+
return protoutil.Marshal(&hint)
2833
}

pkg/sql/hintpb/statement_hint.proto

Lines changed: 11 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -12,6 +12,15 @@ import "gogoproto/gogo.proto";
1212
// that can be applied to statements with a given fingerprint using the
1313
// system.statement_hints table.
1414
message StatementHintUnion {
15-
// TODO: add a field for each type of statement hint.
16-
// oneof value {}
15+
option (gogoproto.onlyone) = true;
16+
17+
InjectHints inject_hints = 1;
18+
}
19+
20+
// InjectHints applies inline query plan hints (join and index hints) from the
21+
// DonorSQL field to the hinted statement. The DonorSQL string must have
22+
// the same syntactical structure as the hinted statement, with only the
23+
// addition of inline hints.
24+
message InjectHints {
25+
string donor_sql = 1 [(gogoproto.customname) = "DonorSQL"];
1726
}
Lines changed: 42 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,42 @@
1+
// Copyright 2025 The Cockroach Authors.
2+
//
3+
// Use of this software is governed by the CockroachDB Software License
4+
// included in the /LICENSE file.
5+
6+
package hintpb
7+
8+
import (
9+
"testing"
10+
11+
"github.com/stretchr/testify/require"
12+
)
13+
14+
func TestFromToBytes(t *testing.T) {
15+
// Test writing empty hint.
16+
_, err := ToBytes(StatementHintUnion{})
17+
require.EqualError(t, err, "cannot convert empty hint to bytes")
18+
19+
// Test reading empty bytes.
20+
_, err = FromBytes(nil)
21+
require.EqualError(t, err, "invalid hint bytes: no value set")
22+
_, err = FromBytes([]byte{})
23+
require.EqualError(t, err, "invalid hint bytes: no value set")
24+
25+
// Test reading invalid bytes.
26+
_, err = FromBytes([]byte{0xFF, 0xFF, 0xFF})
27+
require.Error(t, err)
28+
29+
// Test that a valid hint round trips.
30+
testRT := func(hint interface{}) {
31+
var hintUnion StatementHintUnion
32+
hintUnion.SetValue(hint)
33+
bytes, err := ToBytes(hintUnion)
34+
require.NoError(t, err)
35+
require.NotEmpty(t, bytes)
36+
decodedHintUnion, err := FromBytes(bytes)
37+
require.NoError(t, err)
38+
require.Equal(t, hint, decodedHintUnion.GetValue())
39+
}
40+
testRT(&InjectHints{})
41+
testRT(&InjectHints{DonorSQL: "SELECT * FROM t"})
42+
}

pkg/sql/hints/hint_cache.go

Lines changed: 5 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -147,8 +147,6 @@ func NewStatementHintsCache(
147147
func (c *StatementHintsCache) Start(
148148
ctx context.Context, sysTableResolver catalog.SystemTableIDResolver,
149149
) error {
150-
// TODO(drewk): in a follow-up commit, block until the initial scan is done
151-
// so that statements without hints don't have to check the LRU cache.
152150
return c.startRangefeedInternal(ctx, sysTableResolver)
153151
}
154152

@@ -393,7 +391,7 @@ func (c *StatementHintsCache) GetGeneration() int64 {
393391
// retrieving them.
394392
func (c *StatementHintsCache) MaybeGetStatementHints(
395393
ctx context.Context, statementFingerprint string,
396-
) (hints []hintpb.StatementHint, ids []int64) {
394+
) (hints []hintpb.StatementHintUnion, ids []int64) {
397395
hash := fnv.New64()
398396
_, err := hash.Write([]byte(statementFingerprint))
399397
if err != nil {
@@ -451,7 +449,7 @@ func (c *StatementHintsCache) maybeWaitForRefreshLocked(
451449
// released while reading from the db, and then reacquired.
452450
func (c *StatementHintsCache) addCacheEntryLocked(
453451
ctx context.Context, statementHash int64, statementFingerprint string,
454-
) (hints []hintpb.StatementHint, ids []int64) {
452+
) (hints []hintpb.StatementHintUnion, ids []int64) {
455453
c.mu.AssertHeld()
456454

457455
// Add a cache entry that other queries can find and wait on until we have the
@@ -516,7 +514,7 @@ func (c *StatementHintsCache) getStatementHintsFromDB(
516514
datums := it.Cur()
517515
rowID := int64(tree.MustBeDInt(datums[0]))
518516
fingerprint := string(tree.MustBeDString(datums[1]))
519-
hint, err := hintpb.NewStatementHint([]byte(tree.MustBeDBytes(datums[2])))
517+
hint, err := hintpb.FromBytes([]byte(tree.MustBeDBytes(datums[2])))
520518
if err != nil {
521519
return err
522520
}
@@ -543,7 +541,7 @@ type cacheEntry struct {
543541
// be duplicate entries in the fingerprints slice.
544542
// TODO(drewk): consider de-duplicating the fingerprint strings to reduce
545543
// memory usage.
546-
hints []hintpb.StatementHint
544+
hints []hintpb.StatementHintUnion
547545
fingerprints []string
548546
ids []int64
549547
}
@@ -552,7 +550,7 @@ type cacheEntry struct {
552550
// fingerprint, or nil if they don't exist. The results are in order of row ID.
553551
func (entry *cacheEntry) getMatchingHints(
554552
statementFingerprint string,
555-
) (hints []hintpb.StatementHint, ids []int64) {
553+
) (hints []hintpb.StatementHintUnion, ids []int64) {
556554
for i := range entry.hints {
557555
if entry.fingerprints[i] == statementFingerprint {
558556
hints = append(hints, entry.hints[i])

pkg/sql/hints/hint_cache_test.go

Lines changed: 5 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -539,11 +539,13 @@ func checkIDOrder(t *testing.T, ids []int64) {
539539
}
540540
}
541541

542-
// insertStatementHint inserts an empty statement hint into the
542+
// insertStatementHint inserts a random statement hint into the
543543
// system.statement_hints table.
544544
func insertStatementHint(t *testing.T, r *sqlutils.SQLRunner, fingerprint string) {
545-
emptyHint := &hintpb.StatementHintUnion{}
546-
hintBytes, err := emptyHint.ToBytes()
545+
// TODO(drewk,michae2): randomly choose the hint type once we support others.
546+
var hint hintpb.StatementHintUnion
547+
hint.SetValue(&hintpb.InjectHints{})
548+
hintBytes, err := hintpb.ToBytes(hint)
547549
require.NoError(t, err)
548550
const insertStmt = `INSERT INTO system.statement_hints ("fingerprint", "hint") VALUES ($1, $2)`
549551
r.Exec(t, insertStmt, fingerprint, hintBytes)

0 commit comments

Comments
 (0)