Skip to content

Commit 7b459a4

Browse files
committed
sql: record vector index usage of index_usage_stats
This commit records the vector index reads for `crdb_internal.index_usage_stats`. It also removes some duplicated code in favor of using the shared helper. It also fixes the omission of vector index usage for purposes of IndexesUsed which we include into statement statistics. Also the omission of the index for PlaceholderScan. Release note: None
1 parent 41c2e63 commit 7b459a4

File tree

3 files changed

+42
-55
lines changed

3 files changed

+42
-55
lines changed

pkg/sql/distsql_spec_exec_factory.go

Lines changed: 7 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -317,6 +317,7 @@ func (e *distSQLSpecExecFactory) ConstructScan(
317317
Reverse: params.Reverse,
318318
TableDescriptorModificationTime: tabDesc.GetModificationTime(),
319319
}
320+
// TODO(yuzefovich): record the index usage when applicable.
320321
if err := rowenc.InitIndexFetchSpec(&trSpec.FetchSpec, e.planner.ExecCfg().Codec, tabDesc, idx, columnIDs); err != nil {
321322
return nil, err
322323
}
@@ -871,7 +872,7 @@ func (e *distSQLSpecExecFactory) ConstructIndexJoin(
871872
fetch.lockingWaitPolicy = descpb.ToScanLockingWaitPolicy(locking.WaitPolicy)
872873
fetch.lockingDurability = descpb.ToScanLockingDurability(locking.Durability)
873874

874-
// TODO(drewk): in an EXPLAIN context, record the index usage.
875+
// TODO(drewk): record the index usage when applicable.
875876
planInfo := &indexJoinPlanningInfo{
876877
fetch: fetch,
877878
keyCols: keyCols,
@@ -954,7 +955,7 @@ func (e *distSQLSpecExecFactory) ConstructLookupJoin(
954955
fetch.lockingWaitPolicy = descpb.ToScanLockingWaitPolicy(locking.WaitPolicy)
955956
fetch.lockingDurability = descpb.ToScanLockingDurability(locking.Durability)
956957

957-
// TODO(drewk): if in an EXPLAIN context, record the index usage.
958+
// TODO(drewk): record the index usage when applicable.
958959
planInfo := &lookupJoinPlanningInfo{
959960
fetch: fetch,
960961
joinType: joinType,
@@ -1031,7 +1032,7 @@ func (e *distSQLSpecExecFactory) ConstructInvertedJoin(
10311032
fetch.lockingWaitPolicy = descpb.ToScanLockingWaitPolicy(locking.WaitPolicy)
10321033
fetch.lockingDurability = descpb.ToScanLockingDurability(locking.Durability)
10331034

1034-
// TODO(drewk): if we're in an EXPLAIN context, record the index usage.
1035+
// TODO(drewk): record the index usage when applicable.
10351036
planInfo := &invertedJoinPlanningInfo{
10361037
fetch: fetch,
10371038
joinType: joinType,
@@ -1093,8 +1094,7 @@ func (e *distSQLSpecExecFactory) constructZigzagJoinSide(
10931094
return zigzagPlanningSide{}, err
10941095
}
10951096

1096-
// TODO (cucaroach): update indexUsageStats.
1097-
1097+
// TODO(yuzefovich): record the index usage when applicable.
10981098
return zigzagPlanningSide{
10991099
desc: desc,
11001100
index: index.(*optIndex).idx,
@@ -1524,6 +1524,7 @@ func (e *distSQLSpecExecFactory) ConstructVectorSearch(
15241524
if err != nil {
15251525
return nil, err
15261526
}
1527+
// TODO(yuzefovich): record the index usage when applicable.
15271528
planInfo := &vectorSearchPlanningInfo{
15281529
table: tabDesc,
15291530
index: indexDesc,
@@ -1562,6 +1563,7 @@ func (e *distSQLSpecExecFactory) ConstructVectorMutationSearch(
15621563
if isIndexPut {
15631564
cols = append(cols, colinfo.ResultColumn{Name: "quantized-vector", Typ: types.Bytes})
15641565
}
1566+
// TODO(yuzefovich): record the index usage when applicable.
15651567
planInfo := &vectorMutationSearchPlanningInfo{
15661568
table: table.(*optTable).desc,
15671569
index: index.(*optIndex).idx,

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

Lines changed: 5 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -944,7 +944,7 @@ func (b *Builder) buildScan(scan *memo.ScanExpr) (_ execPlan, outputCols colOrdM
944944
}
945945
root, err := b.factory.ConstructScan(
946946
tab,
947-
tab.Index(scan.Index),
947+
idx,
948948
params,
949949
reqOrdering,
950950
)
@@ -1002,6 +1002,7 @@ func (b *Builder) buildPlaceholderScan(
10021002
md := b.mem.Metadata()
10031003
tab := md.Table(scan.Table)
10041004
idx := tab.Index(scan.Index)
1005+
b.IndexesUsed.add(tab.ID(), idx.ID())
10051006

10061007
// Build the index constraint.
10071008
spanColumns := make([]opt.OrderingColumn, len(scan.Span))
@@ -1039,7 +1040,7 @@ func (b *Builder) buildPlaceholderScan(
10391040
}
10401041
root, err := b.factory.ConstructScan(
10411042
tab,
1042-
tab.Index(scan.Index),
1043+
idx,
10431044
params,
10441045
reqOrdering,
10451046
)
@@ -4115,6 +4116,7 @@ func (b *Builder) buildVectorSearch(
41154116
return execPlan{}, colOrdMap{}, errors.AssertionFailedf(
41164117
"vector search is only supported on vector indexes")
41174118
}
4119+
b.IndexesUsed.add(table.ID(), index.ID())
41184120
primaryKeyCols := md.TableMeta(search.Table).IndexKeyColumns(cat.PrimaryIndex)
41194121
for col, ok := search.Cols.Next(0); ok; col, ok = search.Cols.Next(col + 1) {
41204122
if !primaryKeyCols.Contains(col) {
@@ -4161,6 +4163,7 @@ func (b *Builder) buildVectorMutationSearch(
41614163
return execPlan{}, colOrdMap{}, errors.AssertionFailedf(
41624164
"vector mutation search is only supported on vector indexes")
41634165
}
4166+
b.IndexesUsed.add(table.ID(), index.ID())
41644167

41654168
input, inputCols, err := b.buildRelational(search.Input)
41664169
if err != nil {

pkg/sql/opt_exec_factory.go

Lines changed: 30 additions & 48 deletions
Original file line numberDiff line numberDiff line change
@@ -111,6 +111,18 @@ func (ef *execFactory) ConstructLiteralValues(
111111
}
112112
}
113113

114+
// recordIndexRead - if applicable - records the fact that the given index has
115+
// been used for reading.
116+
func (ef *execFactory) recordIndexRead(tabDesc catalog.TableDescriptor, idx catalog.Index) {
117+
if !ef.isExplain && !ef.planner.SessionData().Internal {
118+
idxUsageKey := roachpb.IndexUsageKey{
119+
TableID: roachpb.TableID(tabDesc.GetID()),
120+
IndexID: roachpb.IndexID(idx.GetID()),
121+
}
122+
ef.planner.extendedEvalCtx.indexUsageStats.RecordRead(idxUsageKey)
123+
}
124+
}
125+
114126
// ConstructScan is part of the exec.Factory interface.
115127
func (ef *execFactory) ConstructScan(
116128
table cat.Table, index cat.Index, params exec.ScanParams, reqOrdering exec.OutputOrdering,
@@ -158,13 +170,7 @@ func (ef *execFactory) ConstructScan(
158170
scan.lockingWaitPolicy = descpb.ToScanLockingWaitPolicy(params.Locking.WaitPolicy)
159171
scan.lockingDurability = descpb.ToScanLockingDurability(params.Locking.Durability)
160172
scan.localityOptimized = params.LocalityOptimized
161-
if !ef.isExplain && !ef.planner.SessionData().Internal {
162-
idxUsageKey := roachpb.IndexUsageKey{
163-
TableID: roachpb.TableID(tabDesc.GetID()),
164-
IndexID: roachpb.IndexID(idx.GetID()),
165-
}
166-
ef.planner.extendedEvalCtx.indexUsageStats.RecordRead(idxUsageKey)
167-
}
173+
ef.recordIndexRead(tabDesc, idx)
168174

169175
return scan, nil
170176
}
@@ -681,19 +687,12 @@ func (ef *execFactory) ConstructIndexJoin(
681687
}
682688

683689
idx := tabDesc.GetPrimaryIndex()
690+
ef.recordIndexRead(tabDesc, idx)
684691
fetch.index = idx
685692
fetch.lockingStrength = descpb.ToScanLockingStrength(locking.Strength)
686693
fetch.lockingWaitPolicy = descpb.ToScanLockingWaitPolicy(locking.WaitPolicy)
687694
fetch.lockingDurability = descpb.ToScanLockingDurability(locking.Durability)
688695

689-
if !ef.isExplain && !ef.planner.SessionData().Internal {
690-
idxUsageKey := roachpb.IndexUsageKey{
691-
TableID: roachpb.TableID(tabDesc.GetID()),
692-
IndexID: roachpb.IndexID(idx.GetID()),
693-
}
694-
ef.planner.extendedEvalCtx.indexUsageStats.RecordRead(idxUsageKey)
695-
}
696-
697696
n := &indexJoinNode{
698697
singleInputPlanNode: singleInputPlanNode{input.(planNode)},
699698
columns: fetch.columns,
@@ -744,19 +743,12 @@ func (ef *execFactory) ConstructLookupJoin(
744743
return nil, err
745744
}
746745

746+
ef.recordIndexRead(tabDesc, idx)
747747
fetch.index = idx
748748
fetch.lockingStrength = descpb.ToScanLockingStrength(locking.Strength)
749749
fetch.lockingWaitPolicy = descpb.ToScanLockingWaitPolicy(locking.WaitPolicy)
750750
fetch.lockingDurability = descpb.ToScanLockingDurability(locking.Durability)
751751

752-
if !ef.isExplain && !ef.planner.SessionData().Internal {
753-
idxUsageKey := roachpb.IndexUsageKey{
754-
TableID: roachpb.TableID(tabDesc.GetID()),
755-
IndexID: roachpb.IndexID(idx.GetID()),
756-
}
757-
ef.planner.extendedEvalCtx.indexUsageStats.RecordRead(idxUsageKey)
758-
}
759-
760752
n := &lookupJoinNode{
761753
singleInputPlanNode: singleInputPlanNode{input.(planNode)},
762754
lookupJoinPlanningInfo: lookupJoinPlanningInfo{
@@ -881,19 +873,12 @@ func (ef *execFactory) ConstructInvertedJoin(
881873
if err := fetch.initDescDefaults(tabDesc, colCfg); err != nil {
882874
return nil, err
883875
}
876+
ef.recordIndexRead(tabDesc, idx)
884877
fetch.index = idx
885878
fetch.lockingStrength = descpb.ToScanLockingStrength(locking.Strength)
886879
fetch.lockingWaitPolicy = descpb.ToScanLockingWaitPolicy(locking.WaitPolicy)
887880
fetch.lockingDurability = descpb.ToScanLockingDurability(locking.Durability)
888881

889-
if !ef.isExplain && !ef.planner.SessionData().Internal {
890-
idxUsageKey := roachpb.IndexUsageKey{
891-
TableID: roachpb.TableID(tabDesc.GetID()),
892-
IndexID: roachpb.IndexID(idx.GetID()),
893-
}
894-
ef.planner.extendedEvalCtx.indexUsageStats.RecordRead(idxUsageKey)
895-
}
896-
897882
n := &invertedJoinNode{
898883
singleInputPlanNode: singleInputPlanNode{input.(planNode)},
899884
invertedJoinPlanningInfo: invertedJoinPlanningInfo{
@@ -954,22 +939,15 @@ func (ef *execFactory) constructFetchForZigzag(
954939
return fetchPlanningInfo{}, nil, err
955940
}
956941

957-
tableDesc := table.(*optTable).desc
958-
idxDesc := index.(*optIndex).idx
942+
tabDesc := table.(*optTable).desc
943+
idx := index.(*optIndex).idx
959944
var fetch fetchPlanningInfo
960-
if err := fetch.initDescDefaults(tableDesc, colCfg); err != nil {
945+
if err := fetch.initDescDefaults(tabDesc, colCfg); err != nil {
961946
return fetchPlanningInfo{}, nil, err
962947
}
963948

964-
if !ef.isExplain && !ef.planner.SessionData().Internal {
965-
idxUsageKey := roachpb.IndexUsageKey{
966-
TableID: roachpb.TableID(tableDesc.GetID()),
967-
IndexID: roachpb.IndexID(idxDesc.GetID()),
968-
}
969-
ef.planner.extendedEvalCtx.indexUsageStats.RecordRead(idxUsageKey)
970-
}
971-
972-
fetch.index = idxDesc
949+
ef.recordIndexRead(tabDesc, idx)
950+
fetch.index = idx
973951
fetch.lockingStrength = descpb.ToScanLockingStrength(locking.Strength)
974952
fetch.lockingWaitPolicy = descpb.ToScanLockingWaitPolicy(locking.WaitPolicy)
975953
fetch.lockingDurability = descpb.ToScanLockingDurability(locking.Durability)
@@ -1962,23 +1940,24 @@ func (ef *execFactory) ConstructVectorSearch(
19621940
targetNeighborCount uint64,
19631941
) (exec.Node, error) {
19641942
tabDesc := table.(*optTable).desc
1965-
indexDesc := index.(*optIndex).idx
1943+
idx := index.(*optIndex).idx
19661944
cols := makeColList(table, outCols)
19671945
resultCols := colinfo.ResultColumnsFromColumns(tabDesc.GetID(), cols)
19681946

19691947
// Encode the prefix constraint as a list of roachpb.Keys.
19701948
var sb span.Builder
19711949
sb.InitAllowingExternalRowData(
1972-
ef.planner.EvalContext(), ef.planner.ExecCfg().Codec, tabDesc, indexDesc,
1950+
ef.planner.EvalContext(), ef.planner.ExecCfg().Codec, tabDesc, idx,
19731951
)
19741952
prefixKeys, err := sb.KeysFromVectorPrefixConstraint(ef.ctx, prefixConstraint)
19751953
if err != nil {
19761954
return nil, err
19771955
}
1956+
ef.recordIndexRead(tabDesc, idx)
19781957
return &vectorSearchNode{
19791958
vectorSearchPlanningInfo: vectorSearchPlanningInfo{
19801959
table: tabDesc,
1981-
index: indexDesc,
1960+
index: idx,
19821961
prefixKeys: prefixKeys,
19831962
queryVector: queryVector,
19841963
targetNeighborCount: targetNeighborCount,
@@ -2009,11 +1988,14 @@ func (ef *execFactory) ConstructVectorMutationSearch(
20091988
cols = append(cols, colinfo.ResultColumn{Name: "quantized-vector", Typ: types.Bytes})
20101989
}
20111990

1991+
tabDesc := table.(*optTable).desc
1992+
idx := index.(*optIndex).idx
1993+
ef.recordIndexRead(tabDesc, idx)
20121994
return &vectorMutationSearchNode{
20131995
singleInputPlanNode: singleInputPlanNode{input: inputPlan},
20141996
vectorMutationSearchPlanningInfo: vectorMutationSearchPlanningInfo{
2015-
table: table.(*optTable).desc,
2016-
index: index.(*optIndex).idx,
1997+
table: tabDesc,
1998+
index: idx,
20171999
prefixKeyCols: prefixKeyCols,
20182000
queryVectorCol: queryVectorCol,
20192001
suffixKeyCols: suffixKeyCols,

0 commit comments

Comments
 (0)