Skip to content

Commit 844a17b

Browse files
craig[bot]rafissmw5hwenyihu6
committed
144189: sql: support ROUTINES syntax in GRANT, REVOKE commands r=rafiss a=rafiss fixes #144179 Release note (sql change): The following syntax is now supported: - GRANT ... ON ALL ROUTINES IN SCHEMA ... - REVOKE ... ON ALL ROUTINES IN SCHEMA ... - ALTER DEFAULT PRIVILEGES GRANT ... ON ROUTINES ... - ALTER DEFAULT PRIVIELGES REVOKE ... ON ROUTINES ... The ROUTINES keyword makes the command apply to both functions and stored procedures. Note that ALTER DEFAULT PRIVILEGES ... ON FUNCTIONS already applied to stored procedures (which aligns with the PostgreSQL behavior), and that is not changing. 144328: backfill: handle transaction retry for vector index backfill r=mw5h a=mw5h Previously, when backfilling a vector index, a transaction retry would cause backfill failure because the vector index backfill writer would overwrite the values read by the backfill reader with what it wanted to push down to KV so as to avoid new allocations. This worked well so long as there were no transaction retries but would fail to re-encode the index entry on a retry because the writer lost access to the unquantized vector. The quantized vector cannot be reused on a transaction retry because fixups may cause the target partition to change. This patch creates a scratch rowenc.IndexEntry in the backfill helper to store the input vector entry. Before attempting to write the entry, we copy the input IndexEntry to the scratch entry and use that to re-encode the vector, which is still modified in place to limit new allocations. Additionally, this patch switches the writer from using CPut() to CPutAllowingIfNotExists() so that if the backfill job restarts, we don't see the partially written index and fail due to duplicate keys. Informs: #143107 Release note: None 144386: roachtest: metamorphically enable lead_for_global_reads_auto_tune r=arulajmani a=wenyihu6 This commit metamorphically enables lead_for_global_reads_auto_tune for follower reads roachtests. Release note: none Epic: none Co-authored-by: Rafi Shamim <[email protected]> Co-authored-by: Matt White <[email protected]> Co-authored-by: wenyihu6 <[email protected]>
4 parents c312854 + 335807b + 02738e6 + 883c363 commit 844a17b

22 files changed

+512
-315
lines changed
Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -1,5 +1,5 @@
11
alter_default_privileges_stmt ::=
2-
'ALTER' 'DEFAULT' 'PRIVILEGES' ( 'FOR' ( 'ROLE' | 'USER' ) role_spec_list | ) ( 'IN' 'SCHEMA' ( ( qualifiable_schema_name ) ( ( ',' qualifiable_schema_name ) )* ) | ) ( 'GRANT' privileges 'ON' ( 'TABLES' | 'SEQUENCES' | 'TYPES' | 'SCHEMAS' | 'FUNCTIONS' ) 'TO' role_spec_list ( 'WITH' 'GRANT' 'OPTION' | ) )
3-
| 'ALTER' 'DEFAULT' 'PRIVILEGES' ( 'FOR' ( 'ROLE' | 'USER' ) role_spec_list | ) ( 'IN' 'SCHEMA' ( ( qualifiable_schema_name ) ( ( ',' qualifiable_schema_name ) )* ) | ) ( 'REVOKE' privileges 'ON' ( 'TABLES' | 'SEQUENCES' | 'TYPES' | 'SCHEMAS' | 'FUNCTIONS' ) 'FROM' role_spec_list ( 'CASCADE' | 'RESTRICT' | ) | 'REVOKE' 'GRANT' 'OPTION' 'FOR' privileges 'ON' ( 'TABLES' | 'SEQUENCES' | 'TYPES' | 'SCHEMAS' | 'FUNCTIONS' ) 'FROM' role_spec_list ( 'CASCADE' | 'RESTRICT' | ) )
4-
| 'ALTER' 'DEFAULT' 'PRIVILEGES' 'FOR' 'ALL' 'ROLES' ( 'IN' 'SCHEMA' ( ( qualifiable_schema_name ) ( ( ',' qualifiable_schema_name ) )* ) | ) ( 'GRANT' privileges 'ON' ( 'TABLES' | 'SEQUENCES' | 'TYPES' | 'SCHEMAS' | 'FUNCTIONS' ) 'TO' role_spec_list ( 'WITH' 'GRANT' 'OPTION' | ) )
5-
| 'ALTER' 'DEFAULT' 'PRIVILEGES' 'FOR' 'ALL' 'ROLES' ( 'IN' 'SCHEMA' ( ( qualifiable_schema_name ) ( ( ',' qualifiable_schema_name ) )* ) | ) ( 'REVOKE' privileges 'ON' ( 'TABLES' | 'SEQUENCES' | 'TYPES' | 'SCHEMAS' | 'FUNCTIONS' ) 'FROM' role_spec_list ( 'CASCADE' | 'RESTRICT' | ) | 'REVOKE' 'GRANT' 'OPTION' 'FOR' privileges 'ON' ( 'TABLES' | 'SEQUENCES' | 'TYPES' | 'SCHEMAS' | 'FUNCTIONS' ) 'FROM' role_spec_list ( 'CASCADE' | 'RESTRICT' | ) )
2+
'ALTER' 'DEFAULT' 'PRIVILEGES' ( 'FOR' ( 'ROLE' | 'USER' ) role_spec_list | ) ( 'IN' 'SCHEMA' ( ( qualifiable_schema_name ) ( ( ',' qualifiable_schema_name ) )* ) | ) ( 'GRANT' privileges 'ON' ( 'TABLES' | 'SEQUENCES' | 'TYPES' | 'SCHEMAS' | 'FUNCTIONS' | 'ROUTINES' ) 'TO' role_spec_list ( 'WITH' 'GRANT' 'OPTION' | ) )
3+
| 'ALTER' 'DEFAULT' 'PRIVILEGES' ( 'FOR' ( 'ROLE' | 'USER' ) role_spec_list | ) ( 'IN' 'SCHEMA' ( ( qualifiable_schema_name ) ( ( ',' qualifiable_schema_name ) )* ) | ) ( 'REVOKE' privileges 'ON' ( 'TABLES' | 'SEQUENCES' | 'TYPES' | 'SCHEMAS' | 'FUNCTIONS' | 'ROUTINES' ) 'FROM' role_spec_list ( 'CASCADE' | 'RESTRICT' | ) | 'REVOKE' 'GRANT' 'OPTION' 'FOR' privileges 'ON' ( 'TABLES' | 'SEQUENCES' | 'TYPES' | 'SCHEMAS' | 'FUNCTIONS' | 'ROUTINES' ) 'FROM' role_spec_list ( 'CASCADE' | 'RESTRICT' | ) )
4+
| 'ALTER' 'DEFAULT' 'PRIVILEGES' 'FOR' 'ALL' 'ROLES' ( 'IN' 'SCHEMA' ( ( qualifiable_schema_name ) ( ( ',' qualifiable_schema_name ) )* ) | ) ( 'GRANT' privileges 'ON' ( 'TABLES' | 'SEQUENCES' | 'TYPES' | 'SCHEMAS' | 'FUNCTIONS' | 'ROUTINES' ) 'TO' role_spec_list ( 'WITH' 'GRANT' 'OPTION' | ) )
5+
| 'ALTER' 'DEFAULT' 'PRIVILEGES' 'FOR' 'ALL' 'ROLES' ( 'IN' 'SCHEMA' ( ( qualifiable_schema_name ) ( ( ',' qualifiable_schema_name ) )* ) | ) ( 'REVOKE' privileges 'ON' ( 'TABLES' | 'SEQUENCES' | 'TYPES' | 'SCHEMAS' | 'FUNCTIONS' | 'ROUTINES' ) 'FROM' role_spec_list ( 'CASCADE' | 'RESTRICT' | ) | 'REVOKE' 'GRANT' 'OPTION' 'FOR' privileges 'ON' ( 'TABLES' | 'SEQUENCES' | 'TYPES' | 'SCHEMAS' | 'FUNCTIONS' | 'ROUTINES' ) 'FROM' role_spec_list ( 'CASCADE' | 'RESTRICT' | ) )

docs/generated/sql/bnf/grant_stmt.bnf

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -43,6 +43,12 @@ grant_stmt ::=
4343
| 'GRANT' 'ALL' 'ON' 'ALL' 'PROCEDURES' 'IN' 'SCHEMA' schema_name_list 'TO' role_spec_list
4444
| 'GRANT' privilege_list 'ON' 'ALL' 'PROCEDURES' 'IN' 'SCHEMA' schema_name_list 'TO' role_spec_list 'WITH' 'GRANT' 'OPTION'
4545
| 'GRANT' privilege_list 'ON' 'ALL' 'PROCEDURES' 'IN' 'SCHEMA' schema_name_list 'TO' role_spec_list
46+
| 'GRANT' 'ALL' 'PRIVILEGES' 'ON' 'ALL' 'ROUTINES' 'IN' 'SCHEMA' schema_name_list 'TO' role_spec_list 'WITH' 'GRANT' 'OPTION'
47+
| 'GRANT' 'ALL' 'PRIVILEGES' 'ON' 'ALL' 'ROUTINES' 'IN' 'SCHEMA' schema_name_list 'TO' role_spec_list
48+
| 'GRANT' 'ALL' 'ON' 'ALL' 'ROUTINES' 'IN' 'SCHEMA' schema_name_list 'TO' role_spec_list 'WITH' 'GRANT' 'OPTION'
49+
| 'GRANT' 'ALL' 'ON' 'ALL' 'ROUTINES' 'IN' 'SCHEMA' schema_name_list 'TO' role_spec_list
50+
| 'GRANT' privilege_list 'ON' 'ALL' 'ROUTINES' 'IN' 'SCHEMA' schema_name_list 'TO' role_spec_list 'WITH' 'GRANT' 'OPTION'
51+
| 'GRANT' privilege_list 'ON' 'ALL' 'ROUTINES' 'IN' 'SCHEMA' schema_name_list 'TO' role_spec_list
4652
| 'GRANT' 'SYSTEM' 'ALL' 'PRIVILEGES' 'TO' role_spec_list 'WITH' 'GRANT' 'OPTION'
4753
| 'GRANT' 'SYSTEM' 'ALL' 'PRIVILEGES' 'TO' role_spec_list
4854
| 'GRANT' 'SYSTEM' 'ALL' 'TO' role_spec_list 'WITH' 'GRANT' 'OPTION'

docs/generated/sql/bnf/revoke_stmt.bnf

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -40,6 +40,12 @@ revoke_stmt ::=
4040
| 'REVOKE' 'GRANT' 'OPTION' 'FOR' 'ALL' 'PRIVILEGES' 'ON' 'ALL' 'PROCEDURES' 'IN' 'SCHEMA' schema_name_list 'FROM' role_spec_list
4141
| 'REVOKE' 'GRANT' 'OPTION' 'FOR' 'ALL' 'ON' 'ALL' 'PROCEDURES' 'IN' 'SCHEMA' schema_name_list 'FROM' role_spec_list
4242
| 'REVOKE' 'GRANT' 'OPTION' 'FOR' privilege_list 'ON' 'ALL' 'PROCEDURES' 'IN' 'SCHEMA' schema_name_list 'FROM' role_spec_list
43+
| 'REVOKE' 'ALL' 'PRIVILEGES' 'ON' 'ALL' 'ROUTINES' 'IN' 'SCHEMA' schema_name_list 'FROM' role_spec_list
44+
| 'REVOKE' 'ALL' 'ON' 'ALL' 'ROUTINES' 'IN' 'SCHEMA' schema_name_list 'FROM' role_spec_list
45+
| 'REVOKE' privilege_list 'ON' 'ALL' 'ROUTINES' 'IN' 'SCHEMA' schema_name_list 'FROM' role_spec_list
46+
| 'REVOKE' 'GRANT' 'OPTION' 'FOR' 'ALL' 'PRIVILEGES' 'ON' 'ALL' 'ROUTINES' 'IN' 'SCHEMA' schema_name_list 'FROM' role_spec_list
47+
| 'REVOKE' 'GRANT' 'OPTION' 'FOR' 'ALL' 'ON' 'ALL' 'ROUTINES' 'IN' 'SCHEMA' schema_name_list 'FROM' role_spec_list
48+
| 'REVOKE' 'GRANT' 'OPTION' 'FOR' privilege_list 'ON' 'ALL' 'ROUTINES' 'IN' 'SCHEMA' schema_name_list 'FROM' role_spec_list
4349
| 'REVOKE' 'SYSTEM' 'ALL' 'PRIVILEGES' 'FROM' role_spec_list
4450
| 'REVOKE' 'SYSTEM' 'ALL' 'FROM' role_spec_list
4551
| 'REVOKE' 'SYSTEM' privilege_list 'FROM' role_spec_list

docs/generated/sql/bnf/stmt_block.bnf

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -111,6 +111,7 @@ grant_stmt ::=
111111
| 'GRANT' privileges 'ON' 'ALL' 'TABLES' 'IN' 'SCHEMA' schema_name_list 'TO' role_spec_list opt_with_grant_option
112112
| 'GRANT' privileges 'ON' 'ALL' 'FUNCTIONS' 'IN' 'SCHEMA' schema_name_list 'TO' role_spec_list opt_with_grant_option
113113
| 'GRANT' privileges 'ON' 'ALL' 'PROCEDURES' 'IN' 'SCHEMA' schema_name_list 'TO' role_spec_list opt_with_grant_option
114+
| 'GRANT' privileges 'ON' 'ALL' 'ROUTINES' 'IN' 'SCHEMA' schema_name_list 'TO' role_spec_list opt_with_grant_option
114115
| 'GRANT' 'SYSTEM' privileges 'TO' role_spec_list opt_with_grant_option
115116

116117
prepare_stmt ::=
@@ -132,6 +133,8 @@ revoke_stmt ::=
132133
| 'REVOKE' 'GRANT' 'OPTION' 'FOR' privileges 'ON' 'ALL' 'FUNCTIONS' 'IN' 'SCHEMA' schema_name_list 'FROM' role_spec_list
133134
| 'REVOKE' privileges 'ON' 'ALL' 'PROCEDURES' 'IN' 'SCHEMA' schema_name_list 'FROM' role_spec_list
134135
| 'REVOKE' 'GRANT' 'OPTION' 'FOR' privileges 'ON' 'ALL' 'PROCEDURES' 'IN' 'SCHEMA' schema_name_list 'FROM' role_spec_list
136+
| 'REVOKE' privileges 'ON' 'ALL' 'ROUTINES' 'IN' 'SCHEMA' schema_name_list 'FROM' role_spec_list
137+
| 'REVOKE' 'GRANT' 'OPTION' 'FOR' privileges 'ON' 'ALL' 'ROUTINES' 'IN' 'SCHEMA' schema_name_list 'FROM' role_spec_list
135138
| 'REVOKE' 'SYSTEM' privileges 'FROM' role_spec_list
136139
| 'REVOKE' 'GRANT' 'OPTION' 'FOR' 'SYSTEM' privileges 'FROM' role_spec_list
137140

@@ -3147,6 +3150,7 @@ target_object_type ::=
31473150
| 'TYPES'
31483151
| 'SCHEMAS'
31493152
| 'FUNCTIONS'
3153+
| 'ROUTINES'
31503154

31513155
alter_changefeed_cmd ::=
31523156
'ADD' changefeed_targets opt_with_options

pkg/cmd/roachtest/tests/follower_reads.go

Lines changed: 30 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -219,7 +219,19 @@ func runFollowerReadsTest(
219219
require.NoError(t, func() error {
220220
db := c.Conn(ctx, l, 1)
221221
defer db.Close()
222-
err := enableIsolationLevels(ctx, t, db)
222+
err := enableClosedTsAutoTune(ctx, rng, db, t)
223+
if err != nil && strings.Contains(err.Error(), "unknown cluster setting") {
224+
// Versions v25.1 and earlier do not support these cluster settings and
225+
// should ignore them. The cluster will continue operating normally, with
226+
// old-version nodes ignoring the settings and newer-version nodes
227+
// continuing to run. Auto-tuning of closed timestamps should only start
228+
// taking effect when the entire cluster has been upgraded to v25.2.
229+
err = nil
230+
}
231+
if err != nil {
232+
return err
233+
}
234+
err = enableIsolationLevels(ctx, t, db)
223235
if err != nil && strings.Contains(err.Error(), "unknown cluster setting") {
224236
// v23.1 and below does not have these cluster settings. That's fine, as
225237
// all isolation levels will be transparently promoted to "serializable".
@@ -1080,3 +1092,20 @@ func enableTenantMultiRegion(l *logger.Logger, r *rand.Rand, h *mixedversion.Hel
10801092
err := setTenantSetting(l, r, h, setting, true)
10811093
return errors.Wrapf(err, "setting %s", setting)
10821094
}
1095+
1096+
// enableClosedTsAutoTune metamorphically enables closed timestamp auto-tuning.
1097+
func enableClosedTsAutoTune(ctx context.Context, rng *rand.Rand, db *gosql.DB, t test.Test) error {
1098+
if rng.Intn(2) == 0 {
1099+
for _, cmd := range []string{
1100+
`SET CLUSTER SETTING kv.closed_timestamp.lead_for_global_reads_auto_tune.enabled = 'true';`,
1101+
`SET CLUSTER SETTING kv.closed_timestamp.policy_refresh_interval = '5s';`,
1102+
`SET CLUSTER SETTING kv.closed_timestamp.policy_latency_refresh_interval = '4s';`,
1103+
} {
1104+
if _, err := db.ExecContext(ctx, cmd); err != nil {
1105+
return err
1106+
}
1107+
}
1108+
t.L().Printf("metamorphically enabled closed timestamp auto-tuning")
1109+
}
1110+
return nil
1111+
}

pkg/sql/backfill/BUILD.bazel

Lines changed: 14 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -60,19 +60,33 @@ go_library(
6060
go_test(
6161
name = "backfill_test",
6262
srcs = [
63+
"backfill_test.go",
6364
"index_backfiller_cols_test.go",
65+
"main_test.go",
6466
"mvcc_index_merger_test.go",
6567
],
6668
embed = [":backfill"],
6769
deps = [
70+
"//pkg/base",
6871
"//pkg/keys",
6972
"//pkg/kv",
7073
"//pkg/roachpb",
74+
"//pkg/security/securityassets",
75+
"//pkg/security/securitytest",
76+
"//pkg/server",
7177
"//pkg/sql/catalog",
7278
"//pkg/sql/catalog/catenumpb",
7379
"//pkg/sql/catalog/descpb",
7480
"//pkg/sql/catalog/tabledesc",
81+
"//pkg/sql/execinfra",
7582
"//pkg/sql/sem/catid",
83+
"//pkg/testutils/serverutils",
84+
"//pkg/testutils/sqlutils",
85+
"//pkg/testutils/testcluster",
86+
"//pkg/util/leaktest",
87+
"//pkg/util/log",
88+
"//pkg/util/randutil",
89+
"//pkg/util/syncutil",
7690
"@com_github_cockroachdb_errors//:errors",
7791
"@com_github_stretchr_testify//require",
7892
],

pkg/sql/backfill/backfill.go

Lines changed: 11 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -485,17 +485,19 @@ type VectorIndexHelper struct {
485485
// context of the provided transaction. This lookup then gives the leaf partition
486486
// where the index entry is to be inserted and the unquantized vector can then be
487487
// re-encoded to get the properly quantized vector with the new partition's
488-
// centroid.
488+
// centroid. The new key is then returned in the outputEntry. The inputEntry is
489+
// not overwritten in case the transaction has to be retried.
489490
func (vih *VectorIndexHelper) ReEncodeVector(
490-
ctx context.Context, txn *kv.Txn, keyBytes []byte, entry *rowenc.IndexEntry,
491+
ctx context.Context, txn *kv.Txn, inputEntry rowenc.IndexEntry, outputEntry *rowenc.IndexEntry,
491492
) (*rowenc.IndexEntry, error) {
493+
keyBytes := inputEntry.Key[len(vih.indexPrefix):]
492494
key, err := vecencoding.DecodeVectorKey(keyBytes, vih.numPrefixCols)
493495
if err != nil {
494496
return &rowenc.IndexEntry{}, err
495497
}
496498

497499
// Decode vector and suffix bytes from the entry value.
498-
valueBytes, err := entry.Value.GetBytes()
500+
valueBytes, err := inputEntry.Value.GetBytes()
499501
if err != nil {
500502
return &rowenc.IndexEntry{}, err
501503
}
@@ -517,16 +519,16 @@ func (vih *VectorIndexHelper) ReEncodeVector(
517519
panic("expected encoded vector to be of type DBytes")
518520
}
519521

520-
entry.Key = entry.Key[:0]
521-
entry.Key = append(entry.Key, vih.indexPrefix...)
522-
entry.Key = key.Encode(entry.Key)
522+
outputEntry.Key = append(outputEntry.Key[:0], vih.indexPrefix...)
523+
outputEntry.Key = key.Encode(outputEntry.Key)
524+
523525
entryValueLen := vecencoding.EncodedVectorValueLen(quantizedVector.UnsafeBytes(), suffix)
524-
buf := entry.Value.AllocBytes(entryValueLen)[:0]
526+
buf := outputEntry.Value.AllocBytes(entryValueLen)[:0]
525527
vecencoding.EncodeVectorValue(buf, quantizedVector.UnsafeBytes(), suffix)
526528

527-
// entry.Family is left unchanged from the entry we received, originally encoded by rowenc.EncodeSecondaryIndexes()
529+
outputEntry.Family = inputEntry.Family
528530

529-
return entry, nil
531+
return outputEntry, nil
530532
}
531533

532534
// IndexBackfiller is capable of backfilling all the added index.

pkg/sql/backfill/backfill_test.go

Lines changed: 90 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,90 @@
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 backfill_test
7+
8+
import (
9+
"context"
10+
"testing"
11+
12+
"github.com/cockroachdb/cockroach/pkg/base"
13+
"github.com/cockroachdb/cockroach/pkg/kv"
14+
"github.com/cockroachdb/cockroach/pkg/sql/execinfra"
15+
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
16+
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
17+
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
18+
"github.com/cockroachdb/cockroach/pkg/util/log"
19+
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
20+
"github.com/stretchr/testify/require"
21+
)
22+
23+
func TestVectorColumnAndIndexBackfill(t *testing.T) {
24+
defer leaktest.AfterTest(t)()
25+
defer log.Scope(t).Close(t)
26+
27+
// Track whether we've injected an error
28+
var errorState struct {
29+
mu syncutil.Mutex
30+
hasErrored bool
31+
}
32+
33+
ctx := context.Background()
34+
srv, db, _ := serverutils.StartServer(t, base.TestServerArgs{
35+
Knobs: base.TestingKnobs{
36+
DistSQL: &execinfra.TestingKnobs{
37+
// Inject a retriable error on the first call to the vector index backfiller.
38+
RunDuringReencodeVectorIndexEntry: func(txn *kv.Txn) error {
39+
errorState.mu.Lock()
40+
defer errorState.mu.Unlock()
41+
if !errorState.hasErrored {
42+
errorState.hasErrored = true
43+
return txn.GenerateForcedRetryableErr(ctx, "forcing a retry error")
44+
}
45+
return nil
46+
},
47+
},
48+
},
49+
})
50+
defer srv.Stopper().Stop(ctx)
51+
sqlDB := sqlutils.MakeSQLRunner(db)
52+
53+
// Create a table with a vector column
54+
sqlDB.Exec(t, `
55+
CREATE TABLE vectors (
56+
id INT PRIMARY KEY,
57+
vec VECTOR(3)
58+
)
59+
`)
60+
61+
// Insert 200 rows with random vector data
62+
sqlDB.Exec(t, `
63+
INSERT INTO vectors (id, vec)
64+
SELECT
65+
generate_series(1, 200) as id,
66+
ARRAY[random(), random(), random()]::vector(3) as vec
67+
`)
68+
69+
// Create a vector index on the vector column
70+
sqlDB.Exec(t, `
71+
CREATE VECTOR INDEX vec_idx ON vectors (vec)
72+
`)
73+
74+
// Test vector similarity search and see that the backfiller got at
75+
// least some of the vectors in there.
76+
var matchCount int
77+
sqlDB.QueryRow(t, `
78+
SELECT count(*)
79+
FROM (
80+
SELECT id
81+
FROM vectors@vec_idx
82+
ORDER BY vec <-> ARRAY[0.5, 0.5, 0.5]::vector(3)
83+
LIMIT 200
84+
)
85+
`).Scan(&matchCount)
86+
// There's some non-determinism here where we may not find all 200 vectors.
87+
// I chose 190 as a low water mark to prevent test flakes, but it should really
88+
// be 200 in most cases.
89+
require.Greater(t, matchCount, 190, "Expected to find at least 190 similar vectors")
90+
}

pkg/sql/backfill/index_backfiller_cols_test.go

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -15,6 +15,7 @@ import (
1515
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
1616
"github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc"
1717
"github.com/cockroachdb/cockroach/pkg/sql/sem/catid"
18+
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
1819
"github.com/stretchr/testify/require"
1920
)
2021

@@ -30,6 +31,8 @@ type (
3031
// to ensure that it properly classifies columns needed for evaluation in an
3132
// index backfill.
3233
func TestIndexBackfillerColumns(t *testing.T) {
34+
defer leaktest.AfterTest(t)()
35+
3336
asIndexSlice := func(in indexes) (out []catalog.Index) {
3437
for _, idx := range in {
3538
out = append(out, idx)
@@ -368,6 +371,7 @@ func TestIndexBackfillerColumns(t *testing.T) {
368371
// TestInitIndexesAllowList tests that initIndexes works correctly with
369372
// "allowList" to populate the "added" field of the index backfiller.
370373
func TestInitIndexesAllowList(t *testing.T) {
374+
defer leaktest.AfterTest(t)()
371375
desc := &tabledesc.Mutable{}
372376
desc.TableDescriptor = descpb.TableDescriptor{
373377
Mutations: []descpb.DescriptorMutation{

pkg/sql/backfill/main_test.go

Lines changed: 28 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,28 @@
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 backfill_test
7+
8+
import (
9+
"os"
10+
"testing"
11+
12+
"github.com/cockroachdb/cockroach/pkg/security/securityassets"
13+
"github.com/cockroachdb/cockroach/pkg/security/securitytest"
14+
"github.com/cockroachdb/cockroach/pkg/server"
15+
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
16+
"github.com/cockroachdb/cockroach/pkg/testutils/testcluster"
17+
"github.com/cockroachdb/cockroach/pkg/util/randutil"
18+
)
19+
20+
//go:generate ../../util/leaktest/add-leaktest.sh *_test.go
21+
22+
func TestMain(m *testing.M) {
23+
securityassets.SetLoader(securitytest.EmbeddedAssets)
24+
randutil.SeedForTests()
25+
serverutils.InitTestServerFactory(server.TestServerFactory)
26+
serverutils.InitTestClusterFactory(testcluster.TestClusterFactory)
27+
os.Exit(m.Run())
28+
}

0 commit comments

Comments
 (0)