Skip to content

Commit 561b230

Browse files
craig[bot]mw5h
andcommitted
Merge #147307
147307: fix pulling full vectors from the primary key when column families are in use r=mw5h a=mw5h #### vecindex: delete Manager.GetWithDesc() method This method was added because backfilling indexes are not considered deletable. In retrospect, there's no reason why we need to restrict an internal interface this way and deleting it removes a fair bit of complexity from upcoming changes to the vector store. Informs: #146046 Release note: None #### fetcher: return spanID from NextRowDecoded() Previously, Fetcher.NextRowDecoded() discarded the spanID it got from Fetcher.NextRow(). This information is needed for a future use of NextRowDecoded(), so this patch passes it through. Existing users ignore this return value. Informs: #146046 Release note: None #### vecstore: use span.Builder in implementation of GetFullVectors Previously, the vector store stored the table and index descriptors for the vector index it was associated with. Since the store itself did not have a lease on this descriptor and would outlive the transaction that caused the store to be created, we could end up using a stale table descriptor for vector operations. This patch moves table information into the vector transaction in the form of an index fetch spec created by a leaseholder on the table. The vecindex transaction is expected to have a lifetime shorter than the lease held by the caller, so the descriptors that generate this fetch spec will not go stale. Addtionally, we generate the needed family IDs for a span.Splitter(), to avoid reading families that we don't require. This is also included with the fetch spec. This patch also: * Introduces a proto buffer to hold the index fetch spec and family IDs. * Introduces a function to initialize this proto buffer. * Introduces a testing API into the vecstore so that tests can provide fake family descriptors to vecstore. Fixes: #146046 Release note (bug fix): A bug where using column families on tables with vector indexes would cause the index to fail to return results has been fixed. #### vecstore: correctly handle composite pk columns Previously, the vector store would blindly use primary key column bytes encoded in a vector index key to rebuild the primary key when retrieving full vector values. SpanBuilder will attempt to reused already encoded datums when building a span, but it can't do so if the encoded datum was encoded in a different direction than required for the span being built. For non-composite columns this was not an issue because they have all the data needed to re-encode in the key value itself. For composite column, the data encoded in the key is a key analogue and can't be used to reconstruct the original column value, which is stored as a suffix column. For vector search, the search processor has access to the ValueBytes array as well as the key, so it's able to rebuild column values as needed. However, the fixup processor only has access to the key bytes, which requires a bit of trickery. In this patch, we disallow assigning directionality to vector index key columns. The mechanics of similarity search preclude scannign prefixes, so this was always a nonsensical thing to do. We now disallow it entirely and take the further step to mechanically switch the directionality of vector index prefix columns to match the primary key, ensuring that the encoded value of these shared columns is always directly usable. We also move the logic for decoding and rebuilding the primary key into the vector store so that it can be shared between fixups and the vector search processor. Note that there is an additional composite key decoder available for the vector search processor because it does indeed need to decode the primary key bytes back to column values. Along the way, we: * Remove ASC/DESC from vector index columns. * Introduce a new index type method to describe indexes with non-directional prefixes. * Alter SHOW commands to no longer show vector index column directionality. * Introduce a bunch of test-gated code to ensure everything I said above is in fact true and remains so in the future. Informs: #146046 Release note (sql change): It is no longer permitted to assign directionality to any vector index column. Prefix columns (those columns before the vector column in a compound vector index key) are not scannable in a vector index, so directionality isn't relevant to them. Co-authored-by: Matt White <[email protected]>
2 parents ab3f969 + 3b6e988 commit 561b230

Some content is hidden

Large Commits have some content hidden by default. Use the searchbox below for content that may be hidden.

48 files changed

+953
-360
lines changed

pkg/BUILD.bazel

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -2379,6 +2379,7 @@ GO_TARGETS = [
23792379
"//pkg/sql/vecindex/vecencoding:vecencoding_test",
23802380
"//pkg/sql/vecindex/vecpb:vecpb",
23812381
"//pkg/sql/vecindex/vecpb:vecpb_test",
2382+
"//pkg/sql/vecindex/vecstore/vecstorepb:vecstorepb",
23822383
"//pkg/sql/vecindex/vecstore:vecstore",
23832384
"//pkg/sql/vecindex/vecstore:vecstore_test",
23842385
"//pkg/sql/vecindex:vecindex",

pkg/ccl/logictestccl/testdata/logic_test/partitioning_index

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -211,11 +211,11 @@ SHOW CREATE TABLE vector
211211
b INT8 NULL,
212212
v VECTOR(3) NULL,
213213
CONSTRAINT vector_pkey PRIMARY KEY (a ASC),
214-
VECTOR INDEX vector_b_v_idx (b ASC, v vector_l2_ops) PARTITION BY LIST (b) (
214+
VECTOR INDEX vector_b_v_idx (b, v vector_l2_ops) PARTITION BY LIST (b) (
215215
PARTITION p1 VALUES IN ((1)),
216216
PARTITION pu VALUES IN ((NULL))
217217
),
218-
VECTOR INDEX vector_idx (a ASC, b ASC, v vector_l2_ops) PARTITION BY LIST (a, b) (
218+
VECTOR INDEX vector_idx (a, b, v vector_l2_ops) PARTITION BY LIST (a, b) (
219219
PARTITION p1 VALUES IN ((1, 1), (2, 2))
220220
)
221221
) WITH (schema_locked = true)

pkg/gen/protobuf.bzl

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -79,6 +79,7 @@ PROTOBUF_SRCS = [
7979
"//pkg/sql/vecindex/cspann/quantize:quantize_go_proto",
8080
"//pkg/sql/vecindex/cspann:cspann_go_proto",
8181
"//pkg/sql/vecindex/vecpb:vecpb_go_proto",
82+
"//pkg/sql/vecindex/vecstore/vecstorepb:vecstorepb_go_proto",
8283
"//pkg/storage/enginepb:enginepb_go_proto",
8384
"//pkg/testutils/grpcutils:grpcutils_go_proto",
8485
"//pkg/ts/catalog:catalog_go_proto",

pkg/sql/BUILD.bazel

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -544,6 +544,7 @@ go_library(
544544
"//pkg/sql/types",
545545
"//pkg/sql/vecindex",
546546
"//pkg/sql/vecindex/vecpb",
547+
"//pkg/sql/vecindex/vecstore",
547548
"//pkg/sql/vtable",
548549
"//pkg/storage",
549550
"//pkg/storage/enginepb",

pkg/sql/alter_primary_key.go

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -851,6 +851,11 @@ func setKeySuffixAndStoredColumnIDsFromPrimary(
851851
"indexed vector column cannot be part of the primary key")
852852
}
853853
}
854+
855+
if vecIdx {
856+
tabledesc.UpdateVectorIndexPrefixColDirections(toAdd, primary)
857+
}
858+
854859
// Finally, add all the stored columns if it is not already a key or key suffix column.
855860
toAddOldStoredColumnIDs := toAdd.StoreColumnIDs
856861
toAddOldStoredColumnNames := toAdd.StoreColumnNames

pkg/sql/backfill/BUILD.bazel

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -45,6 +45,8 @@ go_library(
4545
"//pkg/sql/vecindex",
4646
"//pkg/sql/vecindex/cspann",
4747
"//pkg/sql/vecindex/vecencoding",
48+
"//pkg/sql/vecindex/vecstore",
49+
"//pkg/sql/vecindex/vecstore/vecstorepb",
4850
"//pkg/util/admission/admissionpb",
4951
"//pkg/util/ctxgroup",
5052
"//pkg/util/hlc",
@@ -81,6 +83,7 @@ go_test(
8183
"//pkg/sql/catalog/tabledesc",
8284
"//pkg/sql/execinfra",
8385
"//pkg/sql/sem/catid",
86+
"//pkg/sql/sem/eval",
8487
"//pkg/testutils/serverutils",
8588
"//pkg/testutils/sqlutils",
8689
"//pkg/testutils/testcluster",

pkg/sql/backfill/backfill.go

Lines changed: 33 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -12,7 +12,6 @@ import (
1212
"time"
1313
"unsafe"
1414

15-
"github.com/cockroachdb/cockroach/pkg/keys"
1615
"github.com/cockroachdb/cockroach/pkg/kv"
1716
"github.com/cockroachdb/cockroach/pkg/roachpb"
1817
"github.com/cockroachdb/cockroach/pkg/settings"
@@ -39,6 +38,8 @@ import (
3938
"github.com/cockroachdb/cockroach/pkg/sql/vecindex"
4039
"github.com/cockroachdb/cockroach/pkg/sql/vecindex/cspann"
4140
"github.com/cockroachdb/cockroach/pkg/sql/vecindex/vecencoding"
41+
"github.com/cockroachdb/cockroach/pkg/sql/vecindex/vecstore"
42+
"github.com/cockroachdb/cockroach/pkg/sql/vecindex/vecstore/vecstorepb"
4243
"github.com/cockroachdb/cockroach/pkg/util/log"
4344
"github.com/cockroachdb/cockroach/pkg/util/mon"
4445
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
@@ -477,8 +478,12 @@ type VectorIndexHelper struct {
477478
numPrefixCols int
478479
// vecIndex is the vector index retrieved from the vector index manager.
479480
vecIndex *cspann.Index
481+
// fullVecFetchSpec is the fetch spec for the full vector.
482+
fullVecFetchSpec vecstorepb.GetFullVectorsFetchSpec
480483
// indexPrefix are the prefix bytes for this index (/Tenant/Table/Index).
481484
indexPrefix []byte
485+
// evalCtx is the evaluation context used for vector operations.
486+
evalCtx *eval.Context
482487
}
483488

484489
// ReEncodeVector takes a rowenc.indexEntry, extracts the key values, unquantized
@@ -509,7 +514,8 @@ func (vih *VectorIndexHelper) ReEncodeVector(
509514

510515
// Locate a new partition for the key and re-encode the vector.
511516
var searcher vecindex.MutationSearcher
512-
searcher.Init(vih.vecIndex, txn)
517+
searcher.Init(vih.evalCtx, vih.vecIndex, txn, &vih.fullVecFetchSpec)
518+
513519
if err := searcher.SearchForInsert(ctx, roachpb.Key(key.Prefix), vec); err != nil {
514520
return &rowenc.IndexEntry{}, err
515521
}
@@ -598,7 +604,7 @@ func (ib *IndexBackfiller) InitForLocalUse(
598604
) error {
599605

600606
// Initialize ib.added.
601-
if err := ib.initIndexes(ctx, evalCtx.Codec, desc, nil /* allowList */, 0 /*sourceIndex*/, nil); err != nil {
607+
if err := ib.initIndexes(ctx, evalCtx, desc, nil /* allowList */, 0 /*sourceIndex*/, nil); err != nil {
602608
return err
603609
}
604610

@@ -744,7 +750,14 @@ func (ib *IndexBackfiller) InitForDistributedUse(
744750
evalCtx := flowCtx.NewEvalCtx()
745751

746752
// Initialize ib.added.
747-
if err := ib.initIndexes(ctx, evalCtx.Codec, desc, allowList, sourceIndexID, flowCtx.Cfg.VecIndexManager.(*vecindex.Manager)); err != nil {
753+
if err := ib.initIndexes(
754+
ctx,
755+
evalCtx,
756+
desc,
757+
allowList,
758+
sourceIndexID,
759+
flowCtx.Cfg.VecIndexManager.(*vecindex.Manager),
760+
); err != nil {
748761
return err
749762
}
750763

@@ -838,7 +851,7 @@ func (ib *IndexBackfiller) initCols(desc catalog.TableDescriptor) (err error) {
838851
// If `allowList` is nil, we add all adding index mutations.
839852
func (ib *IndexBackfiller) initIndexes(
840853
ctx context.Context,
841-
codec keys.SQLCodec,
854+
evalCtx *eval.Context,
842855
desc catalog.TableDescriptor,
843856
allowList []catid.IndexID,
844857
sourceIndexID catid.IndexID,
@@ -867,7 +880,7 @@ func (ib *IndexBackfiller) initIndexes(
867880
(allowListAsSet.Empty() || allowListAsSet.Contains(m.AsIndex().GetID())) {
868881
idx := m.AsIndex()
869882
ib.added = append(ib.added, idx)
870-
keyPrefix := rowenc.MakeIndexKeyPrefix(codec, desc.GetID(), idx.GetID())
883+
keyPrefix := rowenc.MakeIndexKeyPrefix(evalCtx.Codec, desc.GetID(), idx.GetID())
871884
ib.keyPrefixes = append(ib.keyPrefixes, keyPrefix)
872885
}
873886
}
@@ -891,18 +904,29 @@ func (ib *IndexBackfiller) initIndexes(
891904
return err
892905
}
893906

894-
vecIndex, err := vecIndexManager.GetWithDesc(ctx, desc, idx)
907+
vecIndex, err := vecIndexManager.Get(ctx, desc.GetID(), idx.GetID())
895908
if err != nil {
896909
return err
897910
}
898911

899-
ib.VectorIndexes[idx.GetID()] = VectorIndexHelper{
912+
helper := VectorIndexHelper{
900913
vectorOrd: vectorCol.Ordinal(),
901914
centroid: make(vector.T, idx.GetVecConfig().Dims),
902915
numPrefixCols: idx.NumKeyColumns() - 1,
903916
vecIndex: vecIndex,
904-
indexPrefix: rowenc.MakeIndexKeyPrefix(codec, desc.GetID(), idx.GetID()),
917+
indexPrefix: rowenc.MakeIndexKeyPrefix(evalCtx.Codec, desc.GetID(), idx.GetID()),
918+
evalCtx: ib.evalCtx,
919+
}
920+
if err := vecstore.InitGetFullVectorsFetchSpec(
921+
&helper.fullVecFetchSpec,
922+
evalCtx,
923+
desc,
924+
idx,
925+
ib.sourceIndex,
926+
); err != nil {
927+
return err
905928
}
929+
ib.VectorIndexes[idx.GetID()] = helper
906930
}
907931

908932
return nil

pkg/sql/backfill/index_backfiller_cols_test.go

Lines changed: 3 additions & 2 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/sql/sem/eval"
1819
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
1920
"github.com/stretchr/testify/require"
2021
)
@@ -418,7 +419,7 @@ func TestInitIndexesAllowList(t *testing.T) {
418419
ib := &IndexBackfiller{}
419420
err := ib.initIndexes(
420421
context.Background(),
421-
keys.SystemSQLCodec,
422+
&eval.Context{Codec: keys.SystemSQLCodec},
422423
desc,
423424
nil, /* allowList */
424425
0, /* sourceIndexID */
@@ -434,7 +435,7 @@ func TestInitIndexesAllowList(t *testing.T) {
434435
ib := &IndexBackfiller{}
435436
err := ib.initIndexes(
436437
context.Background(),
437-
keys.SystemSQLCodec,
438+
&eval.Context{Codec: keys.SystemSQLCodec},
438439
desc,
439440
[]catid.IndexID{3}, /* allowList */
440441
0, /* sourceIndexID */

pkg/sql/catalog/catformat/index.go

Lines changed: 8 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -274,10 +274,14 @@ func FormatIndexElements(
274274
}
275275
}
276276
}
277-
// The last column of an inverted or vector index cannot have a DESC
278-
// direction because it does not have a linear ordering. Since the default
279-
// direction is ASC, we omit the direction entirely for inverted/vector
280-
// index columns.
277+
// Vector indexes do not support ASC/DESC modifiers.
278+
if !index.Type.HasScannablePrefix() {
279+
continue
280+
}
281+
// The last column of an inverted index cannot have a DESC direction
282+
// because it does not have a linear ordering. Since the default
283+
// direction is ASC, we omit the direction entirely for inverted index
284+
// columns.
281285
if i < n-1 || index.Type.HasLinearOrdering() {
282286
f.WriteByte(' ')
283287
f.WriteString(index.KeyColumnDirections[i].String())

pkg/sql/catalog/descpb/index.go

Lines changed: 7 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -9,6 +9,8 @@ import (
99
"fmt"
1010

1111
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catenumpb"
12+
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
13+
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
1214
"github.com/cockroachdb/cockroach/pkg/sql/sem/idxtype"
1315
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
1416
"github.com/cockroachdb/cockroach/pkg/sql/types"
@@ -47,6 +49,11 @@ func (desc *IndexDescriptor) FillColumns(elems tree.IndexElemList) error {
4749
if c.Expr != nil {
4850
return errors.AssertionFailedf("index elem expression should have been replaced with a column")
4951
}
52+
// Vector index prefix columns don't have a direction, nor do the vector columns themselves.
53+
if !desc.Type.HasScannablePrefix() && c.Direction != tree.DefaultDirection {
54+
return pgerror.Newf(pgcode.FeatureNotSupported,
55+
"%s does not support the %s option", idxtype.ErrorText(desc.Type), c.Direction)
56+
}
5057
desc.KeyColumnNames = append(desc.KeyColumnNames, string(c.Column))
5158
switch c.Direction {
5259
case tree.Ascending, tree.DefaultDirection:

0 commit comments

Comments
 (0)