Skip to content

Commit 481e179

Browse files
craig[bot]yuzefovich
andcommitted
Merge #154914
154914: rowexec: fix inverted join with prefix columns with DESC direction r=yuzefovich a=yuzefovich This commit fixes a bug that was added long time ago when we added support for inverted joins on multi-column inverted indexes (i.e. such indexes that have a prefix of "forward" columns that we can constrain using an equality filter) in d3f9a8b. In particular, in that change we added tricky logic which prepended the prefix to the encoded inverted key; however, we then didn't change how we use the `span.Builder`. That guy assumes that it receives datums for each index column (both prefix and inverted) whereas we only provided the inverted bytes (with the prepended prefix). This led to us being confused about which key column is being encoded during span construction. Consider an example where we have one prefix key column like `i INT DESC` (followed by the inverted column). In this case, when calling `MakeKeyFromEncDatums` we'd try to encode the single EncDatum using the DESC key direction. The EncDatum internally stores the prefix + inverted key using ASC key direction, so we'd try to decode it and fail - because of the prepended prefix. After this patch, we'll correctly only use a single column in the call to `MakeKeyFromEncDatums`. Note that if prefix columns had only ASC directions, then we would simply get lucky. Namely, since we have EncDatumRow with a single EncDatum AND that EncDatum already has encoded value of ASC direction stored, we'd just reuse it and exit. The code here is quite difficult to follow and perhaps it could be refactored (e.g. the comment on `SpansFromInvertedSpans` says that multi-column inverted indexes are expected to use non-nil constraint, yet in the inverted joiner we always pass nil), but I chose a more targetted fix that seems reasonable. Additionally, to simplify the change a bit, I inlined the code for `rowenc.MakeSpanFromEncDatums` into two call sites (which also allows us to avoid constructing the unnecessary end key for the inverted spans case). Fixes: #153499. Release note (bug fix): Previously, CockroachDB would hit an internal error when performing an inverted join using an inverted index in which the first prefix column had DESC direction. The bug has been present since 21.1 and is now fixed. Co-authored-by: Yahor Yuzefovich <[email protected]>
2 parents 2c4fb19 + 83d950e commit 481e179

File tree

7 files changed

+43
-37
lines changed

7 files changed

+43
-37
lines changed

pkg/sql/distsql_spec_exec_factory.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -282,7 +282,7 @@ func (e *distSQLSpecExecFactory) ConstructScan(
282282
var spans roachpb.Spans
283283
var err error
284284
if params.InvertedConstraint != nil {
285-
spans, err = sb.SpansFromInvertedSpans(e.ctx, params.InvertedConstraint, params.IndexConstraint, nil /* scratch */)
285+
spans, err = sb.SpansFromInvertedSpans(e.ctx, params.InvertedConstraint, params.IndexConstraint, false /* prefixIncludedInKeys */, nil /* scratch */)
286286
} else {
287287
var splitter span.Splitter
288288
if params.Locking.MustLockAllRequestedColumnFamilies() {

pkg/sql/logictest/testdata/logic_test/inverted_join_multi_column

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -61,7 +61,7 @@ CREATE TABLE j2 (
6161
i INT,
6262
j JSON,
6363
s STRING,
64-
INVERTED INDEX ij_idx (i, j),
64+
INVERTED INDEX ij_idx (i DESC, j),
6565
INVERTED INDEX isj_idx (i, s, j)
6666
)
6767

pkg/sql/opt/exec/execbuilder/testdata/inverted_index_multi_column

Lines changed: 5 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -7,15 +7,15 @@ CREATE TABLE t (
77
s STRING,
88
j JSON,
99
FAMILY (k, i, s, j),
10-
INVERTED INDEX (i, j),
10+
INVERTED INDEX (i DESC, j),
1111
INVERTED INDEX (i, s, j)
1212
)
1313

1414
query T kvtrace
1515
SELECT k FROM t WHERE i = 10 AND j @> '1'
1616
----
1717
Scan /Table/20/1/10{6-7}
18-
Scan /Table/106/2/10/1{-/PrefixEnd}, /Table/106/2/10/Arr/1{-/PrefixEnd}
18+
Scan /Table/106/2/-11/1{-/PrefixEnd}, /Table/106/2/-11/Arr/1{-/PrefixEnd}
1919

2020
query T kvtrace
2121
SELECT k FROM t WHERE i = 10 AND s = 'foo' AND j @> '1'
@@ -39,7 +39,7 @@ Put /Table/106/3/333/"foo"/"a"/"b"/1/0 -> /BYTES/
3939
query T kvtrace
4040
SELECT * FROM t WHERE i = 333 AND j @> '{"a": "b"}'
4141
----
42-
Scan /Table/106/2/333/"a"/"b"{-/PrefixEnd}
42+
Scan /Table/106/2/-334/"a"/"b"{-/PrefixEnd}
4343
Scan /Table/106/1/1/0
4444

4545
# Don't insert duplicate values.
@@ -103,7 +103,7 @@ UPDATE t SET j = NULL WHERE k = 4
103103
----
104104
Scan /Table/106/1/4/0 lock Exclusive (Block, Unreplicated)
105105
Put /Table/106/1/4/0 -> /TUPLE/2:2:Int/333/1:3:Bytes/foo
106-
Del /Table/106/2/333/Arr/1/4/0
106+
Del /Table/106/2/-334/Arr/1/4/0
107107
Del /Table/106/3/333/"foo"/Arr/1/4/0
108108

109109
# Deleting a NULL shouldn't remove anything from the inv idx.
@@ -138,7 +138,7 @@ UPDATE t SET i = NULL WHERE k = 5
138138
----
139139
Scan /Table/106/1/5/0 lock Exclusive (Block, Unreplicated)
140140
Put /Table/106/1/5/0 -> /TUPLE/3:3:Bytes/foo/1:4:SentinelType/{"a": "b"}
141-
Del /Table/106/2/333/"a"/"b"/5/0
141+
Del /Table/106/2/-334/"a"/"b"/5/0
142142
Put /Table/106/2/NULL/"a"/"b"/5/0 -> /BYTES/
143143
Del /Table/106/3/333/"foo"/"a"/"b"/5/0
144144
Put /Table/106/3/NULL/"foo"/"a"/"b"/5/0 -> /BYTES/

pkg/sql/opt_exec_factory.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -179,7 +179,7 @@ func generateScanSpans(
179179
var sb span.Builder
180180
sb.InitAllowingExternalRowData(evalCtx, codec, tabDesc, index)
181181
if params.InvertedConstraint != nil {
182-
return sb.SpansFromInvertedSpans(ctx, params.InvertedConstraint, params.IndexConstraint, nil /* scratch */)
182+
return sb.SpansFromInvertedSpans(ctx, params.InvertedConstraint, params.IndexConstraint, false /* prefixIncludedInKeys */, nil /* scratch */)
183183
}
184184
var splitter span.Splitter
185185
if params.Locking.MustLockAllRequestedColumnFamilies() {

pkg/sql/rowenc/index_encoding.go

Lines changed: 0 additions & 20 deletions
Original file line numberDiff line numberDiff line change
@@ -141,26 +141,6 @@ func (d Directions) Get(i int) (encoding.Direction, error) {
141141
return encoding.Ascending, nil
142142
}
143143

144-
// MakeSpanFromEncDatums creates a minimal index key span on the input
145-
// values. A minimal index key span is a span that includes the fewest possible
146-
// keys after the start key generated by the input values.
147-
//
148-
// The start key is generated by concatenating keyPrefix with the encodings of
149-
// the given EncDatum values. The values, types, and dirs parameters should be
150-
// specified in the same order as the index key columns and may be a prefix.
151-
func MakeSpanFromEncDatums(
152-
values EncDatumRow,
153-
keyCols []fetchpb.IndexFetchSpec_KeyColumn,
154-
alloc *tree.DatumAlloc,
155-
keyPrefix []byte,
156-
) (_ roachpb.Span, containsNull bool, _ error) {
157-
startKey, containsNull, err := MakeKeyFromEncDatums(values, keyCols, alloc, keyPrefix)
158-
if err != nil {
159-
return roachpb.Span{}, false, err
160-
}
161-
return roachpb.Span{Key: startKey, EndKey: startKey.PrefixEnd()}, containsNull, nil
162-
}
163-
164144
// NeededColumnFamilyIDs returns the minimal set of column families required to
165145
// retrieve neededCols for the specified table and index. The returned
166146
// descpb.FamilyIDs are in sorted order. If forSideEffect is true, column

pkg/sql/rowexec/inverted_joiner.go

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -499,7 +499,8 @@ func (ij *invertedJoiner) readInput() (invertedJoinerState, *execinfrapb.Produce
499499
}
500500
// NB: spans is already sorted, and that sorting is preserved when
501501
// generating ij.indexSpans.
502-
ij.indexSpans, err = ij.spanBuilder.SpansFromInvertedSpans(ij.Ctx(), spans, nil /* constraint */, ij.indexSpans)
502+
prefixIncludedInKeys := len(ij.prefixEqualityCols) > 0
503+
ij.indexSpans, err = ij.spanBuilder.SpansFromInvertedSpans(ij.Ctx(), spans, nil /* constraint */, prefixIncludedInKeys, ij.indexSpans)
503504
if err != nil {
504505
ij.MoveToDraining(err)
505506
return ijStateUnknown, ij.DrainHelper()

pkg/sql/span/span_builder.go

Lines changed: 33 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -109,7 +109,11 @@ func (s *Builder) IsPreEncoded(values rowenc.EncDatumRow) bool {
109109
func (s *Builder) SpanFromEncDatums(
110110
values rowenc.EncDatumRow,
111111
) (_ roachpb.Span, containsNull bool, _ error) {
112-
return rowenc.MakeSpanFromEncDatums(values, s.keyAndPrefixCols, &s.alloc, s.KeyPrefix)
112+
startKey, containsNull, err := rowenc.MakeKeyFromEncDatums(values, s.keyAndPrefixCols, &s.alloc, s.KeyPrefix)
113+
if err != nil {
114+
return roachpb.Span{}, false, err
115+
}
116+
return roachpb.Span{Key: startKey, EndKey: startKey.PrefixEnd()}, containsNull, nil
113117
}
114118

115119
// SpanFromEncDatumsWithRange encodes a range span. The inequality is assumed to
@@ -408,16 +412,31 @@ var _ InvertedSpans = inverted.SpanExpressionProtoSpans{}
408412
// If the index is a multi-column inverted index, c should constrain the
409413
// non-inverted prefix columns of the index. Each span in c must have a single
410414
// key. The resulting roachpb.Spans are created by performing a cross product of
411-
// keys in c and the invertedSpan keys.
415+
// keys in c and the invertedSpan keys. Cannot be used in combination with
416+
// prefixIncludedInKeys.
417+
//
418+
// prefixIncludedInKeys, if set, indicates that we have a multi-column inverted
419+
// index AND the non-inverted prefix columns have already been prepended into
420+
// the inverted spans. Cannot be used in combination with non-nil constraint.
421+
// TODO(yuzefovich): consider splitting this method into two: one which accepts
422+
// the constraint and constructs a cross-product of spans, and another which
423+
// already has the inverted spans fully encoded.
412424
//
413425
// scratch can be an optional roachpb.Spans slice that will be reused to
414426
// populate the result.
415427
func (s *Builder) SpansFromInvertedSpans(
416-
ctx context.Context, invertedSpans InvertedSpans, c *constraint.Constraint, scratch roachpb.Spans,
428+
ctx context.Context,
429+
invertedSpans InvertedSpans,
430+
c *constraint.Constraint,
431+
prefixIncludedInKeys bool,
432+
scratch roachpb.Spans,
417433
) (roachpb.Spans, error) {
418434
if invertedSpans == nil {
419435
return nil, errors.AssertionFailedf("invertedSpans cannot be nil")
420436
}
437+
if c != nil && prefixIncludedInKeys {
438+
return nil, errors.AssertionFailedf("constraint and prefixIncludedInKeys cannot be used at the same time")
439+
}
421440

422441
var scratchRows []rowenc.EncDatumRow
423442
if c != nil {
@@ -453,10 +472,10 @@ func (s *Builder) SpansFromInvertedSpans(
453472
for j, n := 0, invertedSpans.Len(); j < n; j++ {
454473
var indexSpan roachpb.Span
455474
var err error
456-
if indexSpan.Key, err = s.generateInvertedSpanKey(invertedSpans.Start(j), scratchRows[i]); err != nil {
475+
if indexSpan.Key, err = s.generateInvertedSpanKey(invertedSpans.Start(j), scratchRows[i], prefixIncludedInKeys); err != nil {
457476
return nil, err
458477
}
459-
if indexSpan.EndKey, err = s.generateInvertedSpanKey(invertedSpans.End(j), scratchRows[i]); err != nil {
478+
if indexSpan.EndKey, err = s.generateInvertedSpanKey(invertedSpans.End(j), scratchRows[i], prefixIncludedInKeys); err != nil {
460479
return nil, err
461480
}
462481
scratch = append(scratch, indexSpan)
@@ -471,7 +490,7 @@ func (s *Builder) SpansFromInvertedSpans(
471490
// of scratchRow is greater than one, the EncDatums that precede the last slot
472491
// are encoded as prefix keys of enc.
473492
func (s *Builder) generateInvertedSpanKey(
474-
enc []byte, scratchRow rowenc.EncDatumRow,
493+
enc []byte, scratchRow rowenc.EncDatumRow, prefixIncludedInKeys bool,
475494
) (roachpb.Key, error) {
476495
keyLen := len(scratchRow) - 1
477496
scratchRow = scratchRow[:keyLen]
@@ -489,8 +508,14 @@ func (s *Builder) generateInvertedSpanKey(
489508
// generate a span, of which we will only use Span.Key. Span.EndKey is
490509
// generated by the caller in the second call, with RKeyMax.
491510

492-
span, _, err := s.SpanFromEncDatums(scratchRow[:keyLen])
493-
return span.Key, err
511+
keyCols := s.keyAndPrefixCols
512+
if prefixIncludedInKeys {
513+
// When prefix is already included in keys, then we only have a single
514+
// "value" which is prefix + encoded inverted value.
515+
keyCols = keyCols[len(keyCols)-1:]
516+
}
517+
startKey, _, err := rowenc.MakeKeyFromEncDatums(scratchRow[:keyLen], keyCols, &s.alloc, s.KeyPrefix)
518+
return startKey, err
494519
}
495520

496521
// KeysFromVectorPrefixConstraint extracts the encoded prefix keys from a

0 commit comments

Comments
 (0)