Skip to content

Commit 808258a

Browse files
committed
sql: add execution support for vector search with multiple prefix keys
This commit changes the vector-search operator to propagate a constraint set rather than a single list of values to be used in constraining index prefix columns. Execution has been modified as well to handle multiple prefixes. This change will allow the vector-search optimizer rule to handle cases where prefix columns can take on multiple possible values. Note that modifying the vector search rule to take advantage of this change is left for a following commit. Epic: CRDB-42943 Release note: None
1 parent 796ede8 commit 808258a

File tree

16 files changed

+120
-73
lines changed

16 files changed

+120
-73
lines changed

pkg/sql/distsql_physical_planner.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -4569,7 +4569,7 @@ func (dsp *DistSQLPlanner) planVectorSearch(
45694569

45704570
colTypes := getTypesFromResultColumns(planInfo.columns)
45714571
spec := &execinfrapb.VectorSearchSpec{
4572-
PrefixKey: planInfo.prefixKey,
4572+
PrefixKeys: planInfo.prefixKeys,
45734573
QueryVector: queryVector,
45744574
TargetNeighborCount: planInfo.targetNeighborCount,
45754575
}

pkg/sql/distsql_spec_exec_factory.go

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -1452,7 +1452,7 @@ func (e *distSQLSpecExecFactory) ConstructVectorSearch(
14521452
table cat.Table,
14531453
index cat.Index,
14541454
outCols exec.TableColumnOrdinalSet,
1455-
prefixKey constraint.Key,
1455+
prefixConstraint *constraint.Constraint,
14561456
queryVector tree.TypedExpr,
14571457
targetNeighborCount uint64,
14581458
) (exec.Node, error) {
@@ -1461,17 +1461,17 @@ func (e *distSQLSpecExecFactory) ConstructVectorSearch(
14611461
cols := makeColList(table, outCols)
14621462
resultCols := colinfo.ResultColumnsFromColumns(tabDesc.GetID(), cols)
14631463

1464-
// Encode the prefix values as a roachpb.Key.
1464+
// Encode the prefix constraint as a list of roachpb.Keys.
14651465
var sb span.Builder
14661466
sb.Init(e.planner.EvalContext(), e.planner.ExecCfg().Codec, tabDesc, indexDesc)
1467-
encPrefixKey, _, err := sb.EncodeConstraintKey(prefixKey)
1467+
prefixKeys, err := sb.KeysFromVectorPrefixConstraint(e.ctx, prefixConstraint)
14681468
if err != nil {
14691469
return nil, err
14701470
}
14711471
planInfo := &vectorSearchPlanningInfo{
14721472
table: tabDesc,
14731473
index: indexDesc,
1474-
prefixKey: encPrefixKey,
1474+
prefixKeys: prefixKeys,
14751475
queryVector: queryVector,
14761476
targetNeighborCount: targetNeighborCount,
14771477
cols: cols,

pkg/sql/execinfrapb/flow_diagram.go

Lines changed: 9 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -103,9 +103,15 @@ func (v *VectorSearchSpec) summary() (string, []string) {
103103
fmt.Sprintf("Nearest Neighbor Target Count: %d", v.TargetNeighborCount),
104104
fmt.Sprintf("Query Vector: %s", vector.T(v.QueryVector).String()),
105105
}
106-
if len(v.PrefixKey) > 0 {
107-
vals, _ := encoding.PrettyPrintValuesWithTypes(nil /* valDirs */, v.PrefixKey)
108-
details = append(details, fmt.Sprintf("Prefix Vals: %s", strings.Join(vals, "/")))
106+
if len(v.PrefixKeys) > 0 {
107+
// Only show the first prefix key.
108+
var spanStr strings.Builder
109+
vals, _ := encoding.PrettyPrintValuesWithTypes(nil /* valDirs */, v.PrefixKeys[0])
110+
spanStr.WriteString(fmt.Sprintf("Prefix Vals: %s", strings.Join(vals, "/")))
111+
if len(v.PrefixKeys) > 1 {
112+
spanStr.WriteString(fmt.Sprintf(" and %d more", len(v.PrefixKeys)-1))
113+
}
114+
details = append(details, spanStr.String())
109115
}
110116
return "VectorSearch", details
111117
}

pkg/sql/execinfrapb/processors_sql.proto

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -1102,9 +1102,9 @@ message InsertSpec {
11021102
message VectorSearchSpec {
11031103
optional sqlbase.IndexFetchSpec fetch_spec = 1 [(gogoproto.nullable) = false];
11041104

1105-
// PrefixKey constrains the prefix index columns to a single value. It is
1106-
// empty for an index without prefix columns.
1107-
optional bytes prefix_key = 2 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.Key"];
1105+
// PrefixKeys, if set, contains keys that each constrain every index prefix
1106+
// column to a single value. It is set IFF the index has prefix columns.
1107+
repeated bytes prefix_keys = 2 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.Key"];
11081108

11091109
// QueryVector is the vector to search for.
11101110
repeated float query_vector = 3;

pkg/sql/opt/exec/execbuilder/relational.go

Lines changed: 1 addition & 20 deletions
Original file line numberDiff line numberDiff line change
@@ -3949,25 +3949,6 @@ func (b *Builder) buildVectorSearch(
39493949
"vector search output column %d is not a primary key column", col)
39503950
}
39513951
}
3952-
// Evaluate the prefix expressions.
3953-
var prefixKey constraint.Key
3954-
if len(search.PrefixVals) > 0 {
3955-
values := make([]tree.Datum, len(search.PrefixVals))
3956-
for i, expr := range search.PrefixVals {
3957-
// The expression is either a placeholder or a constant.
3958-
if p, ok := expr.(*memo.PlaceholderExpr); ok {
3959-
val, err := eval.Expr(b.ctx, b.evalCtx, p.Value)
3960-
if err != nil {
3961-
return execPlan{}, colOrdMap{}, err
3962-
}
3963-
values[i] = val
3964-
} else {
3965-
values[i] = memo.ExtractConstDatum(expr)
3966-
}
3967-
}
3968-
prefixKey = constraint.MakeCompositeKey(values...)
3969-
}
3970-
39713952
outColOrds, outColMap := b.getColumns(search.Cols, search.Table)
39723953
ctx := buildScalarCtx{}
39733954
queryVector, err := b.buildScalar(&ctx, search.QueryVector)
@@ -3978,7 +3959,7 @@ func (b *Builder) buildVectorSearch(
39783959

39793960
var res execPlan
39803961
res.root, err = b.factory.ConstructVectorSearch(
3981-
table, index, outColOrds, prefixKey, queryVector, targetNeighborCount,
3962+
table, index, outColOrds, search.PrefixConstraint, queryVector, targetNeighborCount,
39823963
)
39833964
if err != nil {
39843965
return execPlan{}, colOrdMap{}, err

pkg/sql/opt/exec/explain/emit.go

Lines changed: 8 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -942,10 +942,15 @@ func (e *emitter) emitNodeAttributes(ctx context.Context, evalCtx *eval.Context,
942942
a := n.args.(*vectorSearchArgs)
943943
e.emitTableAndIndex("table", a.Table, a.Index, "" /* suffix */)
944944
ob.Attr("target count", a.TargetNeighborCount)
945-
if ob.flags.Verbose {
946-
if !a.PrefixKey.IsEmpty() {
947-
ob.Attr("prefix key", a.PrefixKey)
945+
if a.PrefixConstraint != nil {
946+
params := exec.ScanParams{
947+
NeededCols: a.OutCols,
948+
IndexConstraint: a.PrefixConstraint,
948949
}
950+
e.emitSpans("prefix spans", a.Table, a.Index, params)
951+
}
952+
if ob.flags.Verbose {
953+
// Vectors can have many dimensions, so don't print them unless verbose.
949954
ob.Expr("query vector", a.QueryVector, nil /* varColumns */)
950955
}
951956

pkg/sql/opt/exec/factory.opt

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -860,7 +860,7 @@ define VectorSearch {
860860
Table cat.Table
861861
Index cat.Index
862862
OutCols exec.TableColumnOrdinalSet
863-
PrefixKey constraint.Key
863+
PrefixConstraint *constraint.Constraint
864864
QueryVector tree.TypedExpr
865865
TargetNeighborCount uint64
866866
}

pkg/sql/opt/memo/expr_format.go

Lines changed: 7 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -747,6 +747,13 @@ func (f *ExprFmtCtx) formatRelational(e RelExpr, tp treeprinter.Node) {
747747

748748
case *VectorSearchExpr:
749749
tp.Childf("target nearest neighbors: %d", t.TargetNeighborCount)
750+
if t.PrefixConstraint != nil {
751+
n := tp.Childf("prefix constraint: %s", t.PrefixConstraint.Columns.String())
752+
for i := 0; i < t.PrefixConstraint.Spans.Count(); i++ {
753+
spanString := t.PrefixConstraint.Spans.Get(i).String()
754+
n.Child(cat.MaybeMarkRedactable(spanString, f.RedactableValues))
755+
}
756+
}
750757

751758
case *VectorMutationSearchExpr:
752759
if t.IsIndexPut {

pkg/sql/opt/ops/relational.opt

Lines changed: 4 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -1439,11 +1439,6 @@ define RecursiveCTEPrivate {
14391439
# distances to the query vector, and applies the NN limit.
14401440
[Relational, Telemetry]
14411441
define VectorSearch {
1442-
# PrefixVals is set iff the vector index has a prefix of non-vector columns.
1443-
# It contains a list of constant values or placeholders that map 1-1 to the
1444-
# prefix columns.
1445-
PrefixVals ScalarListExpr
1446-
14471442
# QueryVector is the scalar query vector. It is either a constant or a
14481443
# placeholder.
14491444
QueryVector ScalarExpr
@@ -1460,6 +1455,10 @@ define VectorSearchPrivate {
14601455
# cat.Table.Index() method in order to fetch the cat.Index metadata.
14611456
Index IndexOrdinal
14621457

1458+
# PrefixConstraint, if set, contains a set of spans that each constrain
1459+
# every index prefix column to a single value.
1460+
PrefixConstraint Constraint
1461+
14631462
# Cols is the set of columns produced by the vector search operator. This
14641463
# is currently always the set of primary key columns.
14651464
Cols ColSet

pkg/sql/opt/xform/limit_funcs.go

Lines changed: 1 addition & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -469,7 +469,6 @@ func (c *CustomFuncs) TryGenerateVectorSearch(
469469
// This index is for a different vector column.
470470
return
471471
}
472-
var prefixVals memo.ScalarListExpr
473472
if index.PrefixColumnCount() > 0 {
474473
// TODO(drewk, mw5h): support multi-column vector indexes.
475474
return
@@ -479,7 +478,7 @@ func (c *CustomFuncs) TryGenerateVectorSearch(
479478
limitInt := int64(*limit.(*tree.DInt))
480479
indexCols := c.PrimaryKeyCols(sp.Table)
481480
vectorSearch := c.e.f.ConstructVectorSearch(
482-
prefixVals, queryVector,
481+
queryVector,
483482
&memo.VectorSearchPrivate{
484483
Table: sp.Table,
485484
Index: index.Ordinal(),

0 commit comments

Comments
 (0)