Skip to content

Commit c117e93

Browse files
committed
sql: add fetch row count assertions in vectorized index joiner
Assertions have been added to the vectorized index joiner that ensure that an index join fetches the expected number of rows. If an index join has a locking wait policy of `SKIP LOCKED`, it should fetch no more than the number of input rows. Otherwise, an index join should fetch exactly one row for each input row. If these assertions fail, the query results may be incorrect, e.g., due to index corruption, and an internal error is preferred over a successful result. Informs #135696 Release note: None
1 parent 4fa3974 commit c117e93

File tree

1 file changed

+57
-11
lines changed

1 file changed

+57
-11
lines changed

pkg/sql/colfetcher/index_join.go

Lines changed: 57 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -17,6 +17,7 @@ import (
1717
"github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvstreamer"
1818
"github.com/cockroachdb/cockroach/pkg/roachpb"
1919
"github.com/cockroachdb/cockroach/pkg/settings"
20+
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
2021
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descs"
2122
"github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecspan"
2223
"github.com/cockroachdb/cockroach/pkg/sql/colexecerror"
@@ -63,6 +64,16 @@ type ColIndexJoin struct {
6364
// and may not correspond to batch boundaries.
6465
startIdx int
6566

67+
// scanRowCounts contains the expected and actual number of rows fetched for
68+
// the current lookup scan. The expected row count is equal to the number of
69+
// input rows that have been consumed to create the spans. These counts are
70+
// used to make assertions that prevent returning incorrect results due to
71+
// index corruption. See assertScanRowCounts() for more details.
72+
scanRowCounts struct {
73+
expected int64
74+
actual int64
75+
}
76+
6677
// limitHintHelper is used in limiting batches of input rows in the presence
6778
// of hard and soft limits.
6879
limitHintHelper execinfra.LimitHintHelper
@@ -120,6 +131,10 @@ type ColIndexJoin struct {
120131
// table because the scan might synthesize additional implicit system columns.
121132
ResultTypes []*types.T
122133

134+
// lockingWaitPolicy is the wait policy for the cFetcher's underlying
135+
// row.KVFetcher.
136+
lockingWaitPolicy descpb.ScanLockingWaitPolicy
137+
123138
// maintainOrdering is true when the index join is required to maintain its
124139
// input ordering, in which case the ordering of the spans cannot be changed.
125140
maintainOrdering bool
@@ -204,8 +219,11 @@ func (s *ColIndexJoin) Next() coldata.Batch {
204219
sort.Sort(spans)
205220
}
206221

207-
// Index joins will always return exactly one output row per input row.
222+
// For memory accounting, we assume the index join will return
223+
// exactly one output row per input row. This is true most of the
224+
// time, except when the locking wait policy is SKIP LOCKED.
208225
s.cf.setEstimatedRowCount(uint64(rowCount))
226+
s.scanRowCounts.expected = rowCount
209227
// Note that the fetcher takes ownership of the spans slice - it
210228
// will modify it and perform the memory accounting. We don't care
211229
// about the modification here, but we want to be conscious about
@@ -239,8 +257,11 @@ func (s *ColIndexJoin) Next() coldata.Batch {
239257
// still has the references to it.
240258
s.spanAssembler.AccountForSpans()
241259
s.state = indexJoinConstructingSpans
260+
s.assertScanRowCounts()
261+
s.scanRowCounts.actual = 0
242262
continue
243263
}
264+
s.scanRowCounts.actual += int64(n)
244265
s.mu.Lock()
245266
s.mu.rowsRead += int64(n)
246267
s.mu.Unlock()
@@ -312,6 +333,30 @@ func (s *ColIndexJoin) getRowSize(idx int) int64 {
312333
return rowSize
313334
}
314335

336+
// assertScanRowCounts performs assertions to prevent silently returning
337+
// incorrect results, e.g., if an index is corrupt. In the common case, the
338+
// number of fetched rows in an index join should be equal to the number of
339+
// input rows. The only exception is when the locking wait policy is
340+
// SKIP LOCKED, in which case the number of fetched rows may be less than
341+
// the number of input rows, but never greater.
342+
func (s *ColIndexJoin) assertScanRowCounts() {
343+
if s.lockingWaitPolicy == descpb.ScanLockingWaitPolicy_SKIP_LOCKED {
344+
if s.scanRowCounts.actual > s.scanRowCounts.expected {
345+
colexecerror.InternalError(errors.AssertionFailedf(
346+
"expected to fetch no more than %d rows, found %d",
347+
s.scanRowCounts.expected, s.scanRowCounts.actual,
348+
))
349+
}
350+
} else {
351+
if s.scanRowCounts.actual != s.scanRowCounts.expected {
352+
colexecerror.InternalError(errors.AssertionFailedf(
353+
"expected to fetch %d rows, found %d",
354+
s.scanRowCounts.expected, s.scanRowCounts.actual,
355+
))
356+
}
357+
}
358+
}
359+
315360
// getBatchSize calculates the size of the entire current batch. Note that it
316361
// accounts only for the size of the data itself, and ignores extra overhead
317362
// such as selection vectors or byte offsets. getBatchSize is not exactly
@@ -616,16 +661,17 @@ func NewColIndexJoin(
616661
)
617662

618663
op := &ColIndexJoin{
619-
OneInputNode: colexecop.NewOneInputNode(input),
620-
flowCtx: flowCtx,
621-
processorID: processorID,
622-
cf: fetcher,
623-
spanAssembler: spanAssembler,
624-
ResultTypes: tableArgs.typs,
625-
maintainOrdering: spec.MaintainOrdering,
626-
txn: txn,
627-
usesStreamer: useStreamer,
628-
limitHintHelper: execinfra.MakeLimitHintHelper(spec.LimitHint, post),
664+
OneInputNode: colexecop.NewOneInputNode(input),
665+
flowCtx: flowCtx,
666+
processorID: processorID,
667+
cf: fetcher,
668+
spanAssembler: spanAssembler,
669+
ResultTypes: tableArgs.typs,
670+
maintainOrdering: spec.MaintainOrdering,
671+
txn: txn,
672+
usesStreamer: useStreamer,
673+
limitHintHelper: execinfra.MakeLimitHintHelper(spec.LimitHint, post),
674+
lockingWaitPolicy: spec.LockingWaitPolicy,
629675
}
630676
op.mem.inputBatchSizeLimit = getIndexJoinBatchSize(
631677
useStreamer, flowCtx.EvalCtx.TestingKnobs.ForceProductionValues, flowCtx.EvalCtx.SessionData(),

0 commit comments

Comments
 (0)