Skip to content

Commit 89d6c57

Browse files
committed
sql: support vector indexes with external row data
This commit fixes a few cases where vector-indexing logic was not correctly using the table's `ExternalRowData` if it was set. It also initializes `cspann.VectorIndex` instances in read-only mode if the table with the index has external row data. Fixes #144679 Release note (bug fix): Fixed a bug in alpha versions of 25.2 as well as `v25.2.0-beta.1` that would cause vector indexes to return incorrect or no results from a standby reader in a PCR setup.
1 parent 4f62762 commit 89d6c57

File tree

15 files changed

+229
-62
lines changed

15 files changed

+229
-62
lines changed

pkg/ccl/testccl/sqlccl/BUILD.bazel

Lines changed: 0 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -41,8 +41,6 @@ go_test(
4141
"//pkg/settings/cluster",
4242
"//pkg/spanconfig",
4343
"//pkg/sql",
44-
"//pkg/sql/catalog/lease",
45-
"//pkg/sql/catalog/replication",
4644
"//pkg/sql/gcjob",
4745
"//pkg/sql/isql",
4846
"//pkg/sql/lexbase",

pkg/ccl/testccl/sqlccl/standby_read_test.go

Lines changed: 2 additions & 25 deletions
Original file line numberDiff line numberDiff line change
@@ -10,16 +10,12 @@ import (
1010
"testing"
1111

1212
"github.com/cockroachdb/cockroach/pkg/base"
13-
"github.com/cockroachdb/cockroach/pkg/sql"
14-
"github.com/cockroachdb/cockroach/pkg/sql/catalog/lease"
15-
"github.com/cockroachdb/cockroach/pkg/sql/catalog/replication"
16-
"github.com/cockroachdb/cockroach/pkg/testutils"
1713
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
1814
"github.com/cockroachdb/cockroach/pkg/testutils/skip"
1915
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
16+
"github.com/cockroachdb/cockroach/pkg/testutils/testcluster"
2017
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
2118
"github.com/cockroachdb/cockroach/pkg/util/log"
22-
"github.com/cockroachdb/errors"
2319
"github.com/stretchr/testify/require"
2420
)
2521

@@ -73,33 +69,14 @@ func TestStandbyRead(t *testing.T) {
7369

7470
srcRunner := sqlutils.MakeSQLRunner(srcDB)
7571
dstRunner := sqlutils.MakeSQLRunner(dstDB)
76-
dstInternal := dstTenant.InternalDB().(*sql.InternalDB)
7772

7873
dstRunner.Exec(t, `SET CLUSTER SETTING sql.defaults.distsql = always`)
7974
dstRunner.Exec(t, `SET distsql = always`)
8075

81-
waitForReplication := func() {
82-
now := ts.Clock().Now()
83-
err := replication.SetupOrAdvanceStandbyReaderCatalog(
84-
ctx, serverutils.TestTenantID(), now, dstInternal, dstTenant.ClusterSettings(),
85-
)
86-
if err != nil {
87-
t.Fatal(err)
88-
}
89-
now = ts.Clock().Now()
90-
lm := dstTenant.LeaseManager().(*lease.Manager)
91-
testutils.SucceedsSoon(t, func() error {
92-
if lm.GetSafeReplicationTS().Less(now) {
93-
return errors.AssertionFailedf("waiting for descriptor close timestamp to catch up")
94-
}
95-
return nil
96-
})
97-
}
98-
9976
for _, tc := range testcases {
10077
var runner *sqlutils.SQLRunner
10178
if tc.standby {
102-
waitForReplication()
79+
testcluster.WaitForStandbyTenantReplication(t, ctx, ts.Clock(), dstTenant)
10380
runner = dstRunner
10481
} else {
10582
runner = srcRunner

pkg/sql/opt_exec_factory.go

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1865,7 +1865,9 @@ func (ef *execFactory) ConstructVectorSearch(
18651865

18661866
// Encode the prefix constraint as a list of roachpb.Keys.
18671867
var sb span.Builder
1868-
sb.Init(ef.planner.EvalContext(), ef.planner.ExecCfg().Codec, tabDesc, indexDesc)
1868+
sb.InitAllowingExternalRowData(
1869+
ef.planner.EvalContext(), ef.planner.ExecCfg().Codec, tabDesc, indexDesc,
1870+
)
18691871
prefixKeys, err := sb.KeysFromVectorPrefixConstraint(ef.ctx, prefixConstraint)
18701872
if err != nil {
18711873
return nil, err

pkg/sql/vecindex/BUILD.bazel

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -43,6 +43,7 @@ go_library(
4343
go_test(
4444
name = "vecindex_test",
4545
srcs = [
46+
"main_test.go",
4647
"manager_test.go",
4748
"searcher_test.go",
4849
"vecindex_test.go",
@@ -51,6 +52,8 @@ go_test(
5152
embed = [":vecindex"],
5253
deps = [
5354
"//pkg/base",
55+
"//pkg/ccl",
56+
"//pkg/ccl/storageccl",
5457
"//pkg/keys",
5558
"//pkg/roachpb",
5659
"//pkg/security/securityassets",
@@ -78,7 +81,9 @@ go_test(
7881
"//pkg/sql/vecindex/vecpb",
7982
"//pkg/sql/vecindex/vecstore",
8083
"//pkg/testutils/serverutils",
84+
"//pkg/testutils/skip",
8185
"//pkg/testutils/sqlutils",
86+
"//pkg/testutils/testcluster",
8287
"//pkg/util/encoding",
8388
"//pkg/util/leaktest",
8489
"//pkg/util/log",

pkg/sql/vecindex/main_test.go

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

pkg/sql/vecindex/manager.go

Lines changed: 11 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -171,13 +171,17 @@ func (m *Manager) GetWithDesc(
171171
// TODO(drewk): use the config to populate the index options as well.
172172
config := index.GetVecConfig()
173173
quantizer := quantize.NewRaBitQuantizer(int(config.Dims), config.Seed)
174-
store, err := vecstore.NewWithColumnID(m.db, quantizer, m.codec, desc, index.GetID(), index.VectorColumnID())
174+
store, err := vecstore.NewWithColumnID(
175+
ctx, m.db, quantizer, m.codec, desc, index.GetID(), index.VectorColumnID(),
176+
)
175177
if err != nil {
176178
return nil, err
177179
}
178180

179181
return cspann.NewIndex(
180-
m.ctx, store, quantizer, config.Seed, m.getIndexOptions(config), m.stopper)
182+
m.ctx, store, quantizer, config.Seed,
183+
m.getIndexOptions(config, store.ReadOnly()), m.stopper,
184+
)
181185
},
182186
)
183187
}
@@ -207,12 +211,14 @@ func (m *Manager) Get(
207211
// passed to cspann.NewIndex, and we don't want that to be the context of
208212
// the Get call.
209213
return cspann.NewIndex(
210-
m.ctx, store, quantizer, config.Seed, m.getIndexOptions(config), m.stopper)
214+
m.ctx, store, quantizer, config.Seed,
215+
m.getIndexOptions(config, store.ReadOnly()), m.stopper,
216+
)
211217
},
212218
)
213219
}
214220

215-
func (m *Manager) getIndexOptions(config vecpb.Config) *cspann.IndexOptions {
221+
func (m *Manager) getIndexOptions(config vecpb.Config, readOnly bool) *cspann.IndexOptions {
216222
return &cspann.IndexOptions{
217223
MinPartitionSize: int(config.MinPartitionSize),
218224
MaxPartitionSize: int(config.MaxPartitionSize),
@@ -222,6 +228,7 @@ func (m *Manager) getIndexOptions(config vecpb.Config) *cspann.IndexOptions {
222228
return StalledOpTimeoutSetting.Get(m.sv)
223229
},
224230
IsDeterministic: config.IsDeterministic,
231+
ReadOnly: readOnly,
225232
}
226233
}
227234

pkg/sql/vecindex/manager_test.go

Lines changed: 0 additions & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -7,17 +7,13 @@ package vecindex_test
77

88
import (
99
"context"
10-
"os"
1110
"strconv"
1211
"sync"
1312
"testing"
1413
"time"
1514

1615
"github.com/cockroachdb/cockroach/pkg/base"
17-
"github.com/cockroachdb/cockroach/pkg/security/securityassets"
18-
"github.com/cockroachdb/cockroach/pkg/security/securitytest"
1916
"github.com/cockroachdb/cockroach/pkg/security/username"
20-
"github.com/cockroachdb/cockroach/pkg/server"
2117
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
2218
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catenumpb"
2319
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb"
@@ -34,18 +30,9 @@ import (
3430
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
3531
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
3632
"github.com/cockroachdb/cockroach/pkg/util/log"
37-
"github.com/cockroachdb/cockroach/pkg/util/randutil"
3833
"github.com/stretchr/testify/require"
3934
)
4035

41-
func TestMain(m *testing.M) {
42-
securityassets.SetLoader(securitytest.EmbeddedAssets)
43-
randutil.SeedForTests()
44-
serverutils.InitTestServerFactory(server.TestServerFactory)
45-
46-
os.Exit(m.Run())
47-
}
48-
4936
func buildTestTable(tableID catid.DescID, tableName string) catalog.MutableTableDescriptor {
5037
return tabledesc.NewBuilder(&descpb.TableDescriptor{
5138
ID: tableID,

pkg/sql/vecindex/searcher_test.go

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -65,6 +65,7 @@ func TestSearcher(t *testing.T) {
6565

6666
quantizer := quantize.NewUnQuantizer(2)
6767
store, err := vecstore.NewWithColumnID(
68+
ctx,
6869
internalDB,
6970
quantizer,
7071
codec,

pkg/sql/vecindex/vecindex_test.go

Lines changed: 79 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -25,15 +25,19 @@ import (
2525
"github.com/cockroachdb/cockroach/pkg/sql/vecindex/cspann/testutils"
2626
"github.com/cockroachdb/cockroach/pkg/sql/vecindex/vecencoding"
2727
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
28+
"github.com/cockroachdb/cockroach/pkg/testutils/skip"
2829
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
30+
"github.com/cockroachdb/cockroach/pkg/testutils/testcluster"
2931
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
3032
"github.com/cockroachdb/cockroach/pkg/util/log"
33+
"github.com/cockroachdb/cockroach/pkg/util/randutil"
3134
"github.com/cockroachdb/cockroach/pkg/util/vector"
35+
"github.com/stretchr/testify/require"
3236
)
3337

34-
// TestVecindexConcurrency builds an index on multiple goroutines, with
38+
// TestVecIndexConcurrency builds an index on multiple goroutines, with
3539
// background splits and merges enabled.
36-
func TestVecindexConcurrency(t *testing.T) {
40+
func TestVecIndexConcurrency(t *testing.T) {
3741
defer leaktest.AfterTest(t)()
3842
defer log.Scope(t).Close(t)
3943

@@ -143,6 +147,77 @@ func TestVecindexConcurrency(t *testing.T) {
143147
logProgress()
144148
}
145149

150+
// TestVecIndexStandbyReader builds an index on a source tenant and verifies
151+
// that a PCR standby reader can read the index, but doesn't attempt to initiate
152+
// fixups.
153+
func TestVecIndexStandbyReader(t *testing.T) {
154+
defer leaktest.AfterTest(t)()
155+
defer log.Scope(t).Close(t)
156+
157+
skip.UnderDuress(t, "slow test")
158+
159+
rnd, seed := randutil.NewTestRand()
160+
t.Logf("random seed: %v", seed)
161+
162+
ctx := context.Background()
163+
tc := serverutils.StartCluster(t, 3, /* numNodes */
164+
base.TestClusterArgs{
165+
ReplicationMode: base.ReplicationManual,
166+
ServerArgs: base.TestServerArgs{
167+
DefaultTestTenant: base.TestControlsTenantsExplicitly,
168+
},
169+
})
170+
defer tc.Stopper().Stop(ctx)
171+
ts := tc.Server(0)
172+
173+
_, srcDB, err := ts.TenantController().StartSharedProcessTenant(ctx,
174+
base.TestSharedProcessTenantArgs{
175+
TenantID: serverutils.TestTenantID(),
176+
TenantName: "src",
177+
UseDatabase: "defaultdb",
178+
},
179+
)
180+
require.NoError(t, err)
181+
dstTenant, dstDB, err := ts.TenantController().StartSharedProcessTenant(ctx,
182+
base.TestSharedProcessTenantArgs{
183+
TenantID: serverutils.TestTenantID2(),
184+
TenantName: "dst",
185+
UseDatabase: "defaultdb",
186+
},
187+
)
188+
require.NoError(t, err)
189+
190+
srcRunner := sqlutils.MakeSQLRunner(srcDB)
191+
dstRunner := sqlutils.MakeSQLRunner(dstDB)
192+
193+
// Enable vector indexes.
194+
srcRunner.Exec(t, `SET CLUSTER SETTING feature.vector_index.enabled = true`)
195+
196+
// Construct the table.
197+
srcRunner.Exec(t, "CREATE TABLE t (id INT PRIMARY KEY, v VECTOR(512), VECTOR INDEX foo (v))")
198+
199+
// Load features and build the index.
200+
const batchSize = 10
201+
const numBatches = 100
202+
vectors := testutils.LoadFeatures(t, batchSize*numBatches)
203+
for i := 0; i < numBatches; i++ {
204+
insertVectors(t, srcRunner, i*batchSize, vectors.Slice(i*batchSize, batchSize))
205+
}
206+
207+
// Wait for the standby reader to catch up.
208+
asOf := testcluster.WaitForStandbyTenantReplication(t, ctx, ts.Clock(), dstTenant)
209+
210+
const queryTemplate = `SELECT * FROM t@foo %s ORDER BY v <-> '%s' LIMIT 3`
211+
asOfClause := fmt.Sprintf("AS OF SYSTEM TIME %s", asOf.AsOfSystemTime())
212+
for range 10 {
213+
// Select a random vector from the set and run an ANN query against both
214+
// tenants. The query results should be identical.
215+
vec := vectors.At(rnd.Intn(vectors.Count)).String()
216+
expected := srcRunner.QueryStr(t, fmt.Sprintf(queryTemplate, asOfClause, vec))
217+
dstRunner.CheckQueryResults(t, fmt.Sprintf(queryTemplate, "", vec), expected)
218+
}
219+
}
220+
146221
// Insert block of vectors within the scope of a transaction.
147222
func insertVectors(t *testing.T, runner *sqlutils.SQLRunner, startId int, vectors vector.Set) {
148223
var valuesClause strings.Builder
@@ -161,8 +236,8 @@ func insertVectors(t *testing.T, runner *sqlutils.SQLRunner, startId int, vector
161236
runner.Exec(t, query, args...)
162237
}
163238

164-
// TestVecindexDeletion tests that rows can be properly deleted from a vector index.
165-
func TestVecindexDeletion(t *testing.T) {
239+
// TestVecIndexDeletion tests that rows can be properly deleted from a vector index.
240+
func TestVecIndexDeletion(t *testing.T) {
166241
defer leaktest.AfterTest(t)()
167242
defer log.Scope(t).Close(t)
168243

pkg/sql/vecindex/vecstore/BUILD.bazel

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -28,6 +28,7 @@ go_library(
2828
"//pkg/sql/vecindex/cspann/quantize",
2929
"//pkg/sql/vecindex/cspann/workspace",
3030
"//pkg/sql/vecindex/vecencoding",
31+
"//pkg/util/log",
3132
"//pkg/util/unique",
3233
"//pkg/util/vector",
3334
"@com_github_cockroachdb_errors//:errors",

0 commit comments

Comments
 (0)