From d413fe7686bf65ad5f2c0e90b96bebb288f0e772 Mon Sep 17 00:00:00 2001 From: DrewKimball Date: Mon, 8 Aug 2022 03:08:10 -0700 Subject: [PATCH 1/7] rowexec: allow ordered joinReader to stream matches to the first row Currently the `joinReaderOrderingStrategy` implementation buffers all looked up rows before matching them with input rows and emitting them. This is necessary because the looked up rows may not be received in input order (which must be maintained). However, rows that match the first input row can be emitted immediately. In the case when there are many rows that match the first input row, this can decrease overhead of the buffer. Additionally, this change can allow a limit to be satisfied earlier, which can significantly decrease latency. This is especially advantageous in the case when there is only one input row, since all lookups can then be rendered and returned in streaming fashion. Release note (performance improvement): The execution engine can now short-circuit execution of lookup joins in more cases, which can decrease latency for queries with limits. --- .../execbuilder/testdata/lookup_join_limit | 92 +++++++++++++++++++ pkg/sql/rowexec/joinreader_strategies.go | 73 +++++++++++++-- pkg/sql/rowexec/joinreader_test.go | 6 +- 3 files changed, 160 insertions(+), 11 deletions(-) diff --git a/pkg/sql/opt/exec/execbuilder/testdata/lookup_join_limit b/pkg/sql/opt/exec/execbuilder/testdata/lookup_join_limit index 67c60309a01b..d847ceb125da 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/lookup_join_limit +++ b/pkg/sql/opt/exec/execbuilder/testdata/lookup_join_limit @@ -8,6 +8,8 @@ CREATE TABLE a (x INT PRIMARY KEY, y INT, z INT, INDEX (y)); CREATE TABLE b (x INT PRIMARY KEY); INSERT INTO a VALUES (1, 1, 1), (2, 1, 1), (3, 2, 2), (4, 2, 2); INSERT INTO b VALUES (1), (2), (3), (4); +CREATE TABLE xy (x INT, y INT, PRIMARY KEY(x, y)); +INSERT INTO xy VALUES (1, 1), (1, 2), (1, 3), (2, 1); # Query with an index join and a limit hint. query T @@ -344,3 +346,93 @@ regions: estimated row count: 1 (100% of the table; stats collected ago) table: a@a_y_idx spans: FULL SCAN (SOFT LIMIT) + +# Query with a lookup join and a limit. The lookup join has to preserve the +# input ordering. +query T +EXPLAIN (OPT, VERBOSE) SELECT a.x, a.y, xy.x, xy.y FROM a INNER LOOKUP JOIN xy ON xy.x = a.x ORDER BY a.y, a.x LIMIT 2 +---- +limit + ├── columns: x:1 y:2 x:6 y:7 + ├── internal-ordering: +2,+(1|6) + ├── cardinality: [0 - 2] + ├── stats: [rows=2] + ├── cost: 55.99 + ├── key: (6,7) + ├── fd: (1)-->(2), (1)==(6), (6)==(1) + ├── ordering: +2,+(1|6) [actual: +2,+1] + ├── distribution: test + ├── prune: (7) + ├── interesting orderings: (+2,+1) + ├── inner-join (lookup xy) + │ ├── columns: a.x:1 a.y:2 xy.x:6 xy.y:7 + │ ├── flags: force lookup join (into right side) + │ ├── key columns: [1] = [6] + │ ├── stats: [rows=10, distinct(1)=1, null(1)=0, avgsize(1)=1, distinct(6)=1, null(6)=0, avgsize(6)=4] + │ ├── cost: 55.96 + │ ├── key: (6,7) + │ ├── fd: (1)-->(2), (1)==(6), (6)==(1) + │ ├── ordering: +2,+(1|6) [actual: +2,+1] + │ ├── limit hint: 2.00 + │ ├── distribution: test + │ ├── prune: (2,7) + │ ├── interesting orderings: (+1) (+2,+1) (+6,+7) + │ ├── scan a@a_y_idx + │ │ ├── columns: a.x:1 a.y:2 + │ │ ├── stats: [rows=1, distinct(1)=1, null(1)=0, avgsize(1)=1] + │ │ ├── cost: 15.04 + │ │ ├── key: (1) + │ │ ├── fd: (1)-->(2) + │ │ ├── ordering: +2,+1 + │ │ ├── limit hint: 1.00 + │ │ ├── distribution: test + │ │ ├── prune: (1,2) + │ │ ├── interesting orderings: (+1) (+2,+1) + │ │ └── unfiltered-cols: (1-5) + │ └── filters (true) + └── 2 + +# Perform a lookup join that preserves its input ordering. Make sure that only +# two rows are read from kv. +query T +EXPLAIN ANALYZE SELECT a.x, a.y, xy.x, xy.y FROM a INNER LOOKUP JOIN xy ON xy.x = a.x ORDER BY a.y, a.x LIMIT 2 +---- +planning time: 10µs +execution time: 100µs +distribution: +vectorized: +rows read from KV: 5 (40 B, 5 gRPC calls) +maximum memory usage: +network usage: +regions: +· +• limit +│ count: 2 +│ +└── • lookup join + │ nodes: + │ regions: + │ actual row count: 2 + │ KV time: 0µs + │ KV contention time: 0µs + │ KV rows read: 2 + │ KV bytes read: 16 B + │ KV gRPC calls: 2 + │ estimated max memory allocated: 0 B + │ estimated max sql temp disk usage: 0 B + │ table: xy@xy_pkey + │ equality: (x) = (x) + │ + └── • scan + nodes: + regions: + actual row count: 3 + KV time: 0µs + KV contention time: 0µs + KV rows read: 3 + KV bytes read: 24 B + KV gRPC calls: 3 + estimated max memory allocated: 0 B + estimated row count: 1 (100% of the table; stats collected ago) + table: a@a_y_idx + spans: FULL SCAN (SOFT LIMIT) diff --git a/pkg/sql/rowexec/joinreader_strategies.go b/pkg/sql/rowexec/joinreader_strategies.go index 36fde25c1455..bb55eddc9762 100644 --- a/pkg/sql/rowexec/joinreader_strategies.go +++ b/pkg/sql/rowexec/joinreader_strategies.go @@ -504,6 +504,16 @@ type joinReaderOrderingStrategy struct { // outputRowIdx contains the index into the inputRowIdx'th row of // inputRowIdxToLookedUpRowIndices that we're about to emit. outputRowIdx int + // notBufferedRow, if non-nil, contains a looked-up row that matches the + // first input row of the batch. Since joinReaderOrderingStrategy returns + // results in input order, it is safe to return looked-up rows that match + // the first input row immediately. + // TODO(drewk): If we had a way of knowing when no more lookups will be + // performed for a given span ID, it would be possible to start immediately + // returning results for the second row once the first was finished, and so + // on. This could significantly decrease the overhead of buffering looked + // up rows. + notBufferedRow rowenc.EncDatumRow } groupingState *inputBatchGroupingState @@ -527,6 +537,10 @@ type joinReaderOrderingStrategy struct { // inputRowIdxToLookedUpRowIndices is a 1:1 mapping with the multimap // with the same name, where each int64 indicates the memory usage of // the corresponding []int that is currently registered with memAcc. + // + // Note that inputRowIdxToLookedUpRowIndices does not contain entries for + // the first input row, because matches to the first row are emitted + // immediately. inputRowIdxToLookedUpRowIndices []int64 } @@ -619,8 +633,11 @@ func (s *joinReaderOrderingStrategy) processLookedUpRow( ctx context.Context, row rowenc.EncDatumRow, spanID int, ) (joinReaderState, error) { matchingInputRowIndices := s.getMatchingRowIndices(spanID) + + // Avoid adding to the buffer if only the first input row was matched, since + // in this case we can just output the row immediately. var containerIdx int - if !s.isPartialJoin { + if !s.isPartialJoin && (len(matchingInputRowIndices) != 1 || matchingInputRowIndices[0] != 0) { // Replace missing values with nulls to appease the row container. for i := range row { if row[i].IsUnset() { @@ -637,6 +654,12 @@ func (s *joinReaderOrderingStrategy) processLookedUpRow( // Update our map from input rows to looked up rows. for _, inputRowIdx := range matchingInputRowIndices { if !s.isPartialJoin { + if inputRowIdx == 0 { + // Don't add to inputRowIdxToLookedUpRowIndices in order to avoid + // emitting more than once. + s.emitCursor.notBufferedRow = row + continue + } s.inputRowIdxToLookedUpRowIndices[inputRowIdx] = append( s.inputRowIdxToLookedUpRowIndices[inputRowIdx], containerIdx) continue @@ -656,7 +679,14 @@ func (s *joinReaderOrderingStrategy) processLookedUpRow( // We failed our on-condition. continue } - s.groupingState.setMatched(inputRowIdx) + wasMatched := s.groupingState.setMatched(inputRowIdx) + if !wasMatched && inputRowIdx == 0 { + // This looked up row matches the first row, and we haven't seen a match + // for the first row yet. Don't add to inputRowIdxToLookedUpRowIndices + // in order to avoid emitting more than once. + s.emitCursor.notBufferedRow = row + continue + } s.inputRowIdxToLookedUpRowIndices[inputRowIdx] = partialJoinSentinel } } @@ -673,6 +703,12 @@ func (s *joinReaderOrderingStrategy) processLookedUpRow( return jrStateUnknown, err } + if s.emitCursor.notBufferedRow != nil { + // The looked up row matched the first input row. Render and emit them + // immediately, then return to performing lookups. + return jrEmittingRows, nil + } + return jrPerformingLookup, nil } @@ -699,7 +735,7 @@ func (s *joinReaderOrderingStrategy) nextRowToEmit( inputRow := s.inputRows[s.emitCursor.inputRowIdx] lookedUpRows := s.inputRowIdxToLookedUpRowIndices[s.emitCursor.inputRowIdx] - if s.emitCursor.outputRowIdx >= len(lookedUpRows) { + if s.emitCursor.notBufferedRow == nil && s.emitCursor.outputRowIdx >= len(lookedUpRows) { // We have no more rows for the current input row. Emit an outer or anti // row if we didn't see a match, and bump to the next input row. inputRowIdx := s.emitCursor.inputRowIdx @@ -725,20 +761,39 @@ func (s *joinReaderOrderingStrategy) nextRowToEmit( return nil, jrEmittingRows, nil } - lookedUpRowIdx := lookedUpRows[s.emitCursor.outputRowIdx] - s.emitCursor.outputRowIdx++ + var nextState joinReaderState + if s.emitCursor.notBufferedRow != nil { + // Make sure we return to looking up rows after outputting one that matches + // the first input row. + nextState = jrPerformingLookup + defer func() { s.emitCursor.notBufferedRow = nil }() + } else { + // All lookups have finished, and we are currently iterating through the + // input rows and emitting them. + nextState = jrEmittingRows + defer func() { s.emitCursor.outputRowIdx++ }() + } + switch s.joinType { case descpb.LeftSemiJoin: // A semi-join match means we emit our input row. This is the case where // we used the partialJoinSentinel. - return inputRow, jrEmittingRows, nil + return inputRow, nextState, nil case descpb.LeftAntiJoin: // An anti-join match means we emit nothing. This is the case where // we used the partialJoinSentinel. - return nil, jrEmittingRows, nil + return nil, nextState, nil } - lookedUpRow, err := s.lookedUpRows.GetRow(s.Ctx, lookedUpRowIdx, false /* skip */) + var err error + var lookedUpRow rowenc.EncDatumRow + if s.emitCursor.notBufferedRow != nil { + lookedUpRow = s.emitCursor.notBufferedRow + } else { + lookedUpRow, err = s.lookedUpRows.GetRow( + s.Ctx, lookedUpRows[s.emitCursor.outputRowIdx], false, /* skip */ + ) + } if err != nil { return nil, jrStateUnknown, err } @@ -758,7 +813,7 @@ func (s *joinReaderOrderingStrategy) nextRowToEmit( } } } - return outputRow, jrEmittingRows, nil + return outputRow, nextState, nil } func (s *joinReaderOrderingStrategy) spilled() bool { diff --git a/pkg/sql/rowexec/joinreader_test.go b/pkg/sql/rowexec/joinreader_test.go index 325fa7ff6593..28ca3214cdc7 100644 --- a/pkg/sql/rowexec/joinreader_test.go +++ b/pkg/sql/rowexec/joinreader_test.go @@ -1288,9 +1288,11 @@ CREATE TABLE test.t (a INT, s STRING, INDEX (a, s))`); err != nil { // DiskBackedIndexedRowContainer. flowCtx.Cfg.TestingKnobs.MemoryLimitBytes = mon.DefaultPoolAllocationSize - // Input row is just a single 0. + // The two input rows are just a single 0 each. We use two input rows because + // matches to the first input row are never buffered. inputRows := rowenc.EncDatumRows{ rowenc.EncDatumRow{rowenc.EncDatum{Datum: tree.NewDInt(tree.DInt(key))}}, + rowenc.EncDatumRow{rowenc.EncDatum{Datum: tree.NewDInt(tree.DInt(key))}}, } var fetchSpec descpb.IndexFetchSpec if err := rowenc.InitIndexFetchSpec( @@ -1341,7 +1343,7 @@ CREATE TABLE test.t (a INT, s STRING, INDEX (a, s))`); err != nil { require.Equal(t, expected, actual) count++ } - require.Equal(t, numRows, count) + require.Equal(t, numRows*len(inputRows), count) require.True(t, jr.(*joinReader).Spilled()) } From 0835fc28bb111488f25482909c19ad251e644870 Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Fri, 5 Aug 2022 15:57:10 -0700 Subject: [PATCH 2/7] colexec: clean up NOT LIKE operator generation MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This commit cleans up the way we generate operators for NOT LIKE. Previously, they would get their own copy which was exactly the same as for LIKE with a difference in a single line, and now the same underlying operator will handle both LIKE and NOT LIKE - the result of comparison just needs to be negated. The performance hit of this extra boolean comparison is negligible yet we can remove some of the duplicated generated code. ``` name old time/op new time/op delta LikeOps/selPrefixBytesBytesConstOp-24 17.8µs ± 1% 16.9µs ± 0% -4.93% (p=0.000 n=10+10) LikeOps/selSuffixBytesBytesConstOp-24 18.5µs ± 0% 18.7µs ± 0% +1.37% (p=0.000 n=10+10) LikeOps/selContainsBytesBytesConstOp-24 27.8µs ± 0% 28.0µs ± 0% +1.02% (p=0.000 n=9+10) LikeOps/selRegexpBytesBytesConstOp-24 479µs ± 1% 484µs ± 0% +1.10% (p=0.000 n=10+10) LikeOps/selSkeletonBytesBytesConstOp-24 39.9µs ± 0% 40.3µs ± 0% +0.85% (p=0.000 n=10+10) LikeOps/selRegexpSkeleton-24 871µs ± 2% 871µs ± 0% ~ (p=1.000 n=10+10) ``` Release note: None --- pkg/sql/colexec/colexeccmp/like_ops.go | 47 +- pkg/sql/colexec/colexecprojconst/like_ops.go | 78 +-- .../colexecprojconst/proj_const_ops_tmpl.go | 11 + .../colexecprojconst/proj_like_ops.eg.go | 428 +-------------- pkg/sql/colexec/colexecsel/like_ops.go | 78 +-- pkg/sql/colexec/colexecsel/sel_like_ops.eg.go | 513 +----------------- .../colexec/colexecsel/selection_ops_tmpl.go | 11 + .../execgen/cmd/execgen/like_ops_gen.go | 32 +- .../execgen/cmd/execgen/overloads_base.go | 5 + .../opt/exec/execbuilder/testdata/tpch_vec | 4 +- 10 files changed, 149 insertions(+), 1058 deletions(-) diff --git a/pkg/sql/colexec/colexeccmp/like_ops.go b/pkg/sql/colexec/colexeccmp/like_ops.go index cd18ab42f6fc..db75dca09b36 100644 --- a/pkg/sql/colexec/colexeccmp/like_ops.go +++ b/pkg/sql/colexec/colexeccmp/like_ops.go @@ -25,56 +25,29 @@ const ( // LikeConstant is used when comparing against a constant string with no // wildcards. LikeConstant - // LikeConstantNegate is used when comparing against a constant string with - // no wildcards, and the result is negated. - LikeConstantNegate // LikeContains is used when comparing against a constant substring. LikeContains - // LikeContainsNegate is used when comparing against a constant substring, - // and the result is negated. - LikeContainsNegate - // LikeNeverMatch doesn't match anything. - LikeNeverMatch // LikePrefix is used when comparing against a constant prefix. LikePrefix - // LikePrefixNegate is used when comparing against a constant prefix, and - // the result is negated. - LikePrefixNegate // LikeRegexp is the default slow case when we need to fallback to RegExp // matching. LikeRegexp - // LikeRegexpNegate is the default slow case when we need to fallback to - // RegExp matching, but the result is negated. - LikeRegexpNegate // LikeSkeleton is used when comparing against a "skeleton" string (of the // form '%foo%bar%' with any number of "skeleton words"). LikeSkeleton - // LikeSkeletonNegate is used when comparing against a "skeleton" string (of - // the form '%foo%bar%' with any number of "skeleton words"), and the result - // is negated. - LikeSkeletonNegate // LikeSuffix is used when comparing against a constant suffix. LikeSuffix - // LikeSuffixNegate is used when comparing against a constant suffix, and - // the result is negated. - LikeSuffixNegate ) // GetLikeOperatorType returns LikeOpType corresponding to the inputs. // // The second return parameter always contains a single []byte, unless // "skeleton" LikeOpType is returned. -func GetLikeOperatorType(pattern string, negate bool) (LikeOpType, [][]byte, error) { +func GetLikeOperatorType(pattern string) (LikeOpType, [][]byte, error) { if pattern == "" { - if negate { - return LikeConstantNegate, [][]byte{{}}, nil - } return LikeConstant, [][]byte{{}}, nil } if pattern == "%" { - if negate { - return LikeNeverMatch, [][]byte{{}}, nil - } return LikeAlwaysMatch, [][]byte{{}}, nil } hasEscape := strings.Contains(pattern, `\`) @@ -94,30 +67,18 @@ func GetLikeOperatorType(pattern string, negate bool) (LikeOpType, [][]byte, err lastChar := pattern[len(pattern)-1] if !isWildcard(firstChar) && !isWildcard(lastChar) { // No wildcards, so this is just an exact string match. - if negate { - return LikeConstantNegate, [][]byte{[]byte(pattern)}, nil - } return LikeConstant, [][]byte{[]byte(pattern)}, nil } if firstChar == '%' && !isWildcard(lastChar) { suffix := pattern[1:] - if negate { - return LikeSuffixNegate, [][]byte{[]byte(suffix)}, nil - } return LikeSuffix, [][]byte{[]byte(suffix)}, nil } if lastChar == '%' && !isWildcard(firstChar) { prefix := pattern[:len(pattern)-1] - if negate { - return LikePrefixNegate, [][]byte{[]byte(prefix)}, nil - } return LikePrefix, [][]byte{[]byte(prefix)}, nil } if firstChar == '%' && lastChar == '%' { contains := pattern[1 : len(pattern)-1] - if negate { - return LikeContainsNegate, [][]byte{[]byte(contains)}, nil - } return LikeContains, [][]byte{[]byte(contains)}, nil } } @@ -136,15 +97,9 @@ func GetLikeOperatorType(pattern string, negate bool) (LikeOpType, [][]byte, err skeleton = append(skeleton, pat[:idx]) pat = pat[idx+1:] } - if negate { - return LikeSkeletonNegate, skeleton, nil - } return LikeSkeleton, skeleton, nil } // Default (slow) case: execute as a regular expression match. - if negate { - return LikeRegexpNegate, [][]byte{[]byte(pattern)}, nil - } return LikeRegexp, [][]byte{[]byte(pattern)}, nil } diff --git a/pkg/sql/colexec/colexecprojconst/like_ops.go b/pkg/sql/colexec/colexecprojconst/like_ops.go index 285bec75367d..b079160e6b1d 100644 --- a/pkg/sql/colexec/colexecprojconst/like_ops.go +++ b/pkg/sql/colexec/colexecprojconst/like_ops.go @@ -32,7 +32,7 @@ func GetLikeProjectionOperator( pattern string, negate bool, ) (colexecop.Operator, error) { - likeOpType, patterns, err := colexeccmp.GetLikeOperatorType(pattern, negate) + likeOpType, patterns, err := colexeccmp.GetLikeOperatorType(pattern) if err != nil { return nil, err } @@ -45,45 +45,21 @@ func GetLikeProjectionOperator( outputIdx: resultIdx, } switch likeOpType { - case colexeccmp.LikeConstant: - return &projEQBytesBytesConstOp{ - projConstOpBase: base, - constArg: pat, - }, nil - case colexeccmp.LikeConstantNegate: - return &projNEBytesBytesConstOp{ - projConstOpBase: base, - constArg: pat, - }, nil - case colexeccmp.LikeNeverMatch: - // Use an empty not-prefix operator to get correct NULL behavior. - return &projNotPrefixBytesBytesConstOp{ - projConstOpBase: base, - constArg: []byte{}, - }, nil case colexeccmp.LikeAlwaysMatch: // Use an empty prefix operator to get correct NULL behavior. return &projPrefixBytesBytesConstOp{ projConstOpBase: base, constArg: []byte{}, + negate: negate, }, nil - case colexeccmp.LikeSuffix: - return &projSuffixBytesBytesConstOp{ - projConstOpBase: base, - constArg: pat, - }, nil - case colexeccmp.LikeSuffixNegate: - return &projNotSuffixBytesBytesConstOp{ - projConstOpBase: base, - constArg: pat, - }, nil - case colexeccmp.LikePrefix: - return &projPrefixBytesBytesConstOp{ - projConstOpBase: base, - constArg: pat, - }, nil - case colexeccmp.LikePrefixNegate: - return &projNotPrefixBytesBytesConstOp{ + case colexeccmp.LikeConstant: + if negate { + return &projNEBytesBytesConstOp{ + projConstOpBase: base, + constArg: pat, + }, nil + } + return &projEQBytesBytesConstOp{ projConstOpBase: base, constArg: pat, }, nil @@ -91,21 +67,13 @@ func GetLikeProjectionOperator( return &projContainsBytesBytesConstOp{ projConstOpBase: base, constArg: pat, + negate: negate, }, nil - case colexeccmp.LikeContainsNegate: - return &projNotContainsBytesBytesConstOp{ + case colexeccmp.LikePrefix: + return &projPrefixBytesBytesConstOp{ projConstOpBase: base, constArg: pat, - }, nil - case colexeccmp.LikeSkeleton: - return &projSkeletonBytesBytesConstOp{ - projConstOpBase: base, - constArg: patterns, - }, nil - case colexeccmp.LikeSkeletonNegate: - return &projNotSkeletonBytesBytesConstOp{ - projConstOpBase: base, - constArg: patterns, + negate: negate, }, nil case colexeccmp.LikeRegexp: re, err := eval.ConvertLikeToRegexp(ctx, string(patterns[0]), false, '\\') @@ -115,15 +83,19 @@ func GetLikeProjectionOperator( return &projRegexpBytesBytesConstOp{ projConstOpBase: base, constArg: re, + negate: negate, }, nil - case colexeccmp.LikeRegexpNegate: - re, err := eval.ConvertLikeToRegexp(ctx, string(patterns[0]), false, '\\') - if err != nil { - return nil, err - } - return &projNotRegexpBytesBytesConstOp{ + case colexeccmp.LikeSkeleton: + return &projSkeletonBytesBytesConstOp{ projConstOpBase: base, - constArg: re, + constArg: patterns, + negate: negate, + }, nil + case colexeccmp.LikeSuffix: + return &projSuffixBytesBytesConstOp{ + projConstOpBase: base, + constArg: pat, + negate: negate, }, nil default: return nil, errors.AssertionFailedf("unsupported like op type %d", likeOpType) diff --git a/pkg/sql/colexec/colexecprojconst/proj_const_ops_tmpl.go b/pkg/sql/colexec/colexecprojconst/proj_const_ops_tmpl.go index 85689c373de8..3581f8ec835e 100644 --- a/pkg/sql/colexec/colexecprojconst/proj_const_ops_tmpl.go +++ b/pkg/sql/colexec/colexecprojconst/proj_const_ops_tmpl.go @@ -93,6 +93,9 @@ type _OP_CONST_NAME struct { // {{else}} constArg _R_GO_TYPE // {{end}} + // {{if .Negatable}} + negate bool + // {{end}} } func (p _OP_CONST_NAME) Next() coldata.Batch { @@ -104,6 +107,14 @@ func (p _OP_CONST_NAME) Next() coldata.Batch { // */}} _overloadHelper := p.BinaryOverloadHelper // {{end}} + // {{if .Negatable}} + // {{/* + // In order to inline the templated code of the LIKE overloads, we need + // to have a `_negate` local variable indicating whether the assignment + // should be negated. + // */}} + _negate := p.negate + // {{end}} batch := p.Input.Next() n := batch.Length() if n == 0 { diff --git a/pkg/sql/colexec/colexecprojconst/proj_like_ops.eg.go b/pkg/sql/colexec/colexecprojconst/proj_like_ops.eg.go index e6e0d9a3081b..f1a797db8bd2 100644 --- a/pkg/sql/colexec/colexecprojconst/proj_like_ops.eg.go +++ b/pkg/sql/colexec/colexecprojconst/proj_like_ops.eg.go @@ -20,9 +20,11 @@ import ( type projPrefixBytesBytesConstOp struct { projConstOpBase constArg []byte + negate bool } func (p projPrefixBytesBytesConstOp) Next() coldata.Batch { + _negate := p.negate batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -48,7 +50,7 @@ func (p projPrefixBytesBytesConstOp) Next() coldata.Batch { if !colNulls.NullAt(i) { // We only want to perform the projection operation if the value is not null. arg := col.Get(i) - projCol[i] = bytes.HasPrefix(arg, p.constArg) + projCol[i] = bytes.HasPrefix(arg, p.constArg) != _negate } } } else { @@ -58,7 +60,7 @@ func (p projPrefixBytesBytesConstOp) Next() coldata.Batch { if !colNulls.NullAt(i) { // We only want to perform the projection operation if the value is not null. arg := col.Get(i) - projCol[i] = bytes.HasPrefix(arg, p.constArg) + projCol[i] = bytes.HasPrefix(arg, p.constArg) != _negate } } } @@ -68,14 +70,14 @@ func (p projPrefixBytesBytesConstOp) Next() coldata.Batch { sel = sel[:n] for _, i := range sel { arg := col.Get(i) - projCol[i] = bytes.HasPrefix(arg, p.constArg) + projCol[i] = bytes.HasPrefix(arg, p.constArg) != _negate } } else { _ = projCol.Get(n - 1) _ = col.Get(n - 1) for i := 0; i < n; i++ { arg := col.Get(i) - projCol[i] = bytes.HasPrefix(arg, p.constArg) + projCol[i] = bytes.HasPrefix(arg, p.constArg) != _negate } } } @@ -86,9 +88,11 @@ func (p projPrefixBytesBytesConstOp) Next() coldata.Batch { type projSuffixBytesBytesConstOp struct { projConstOpBase constArg []byte + negate bool } func (p projSuffixBytesBytesConstOp) Next() coldata.Batch { + _negate := p.negate batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -114,7 +118,7 @@ func (p projSuffixBytesBytesConstOp) Next() coldata.Batch { if !colNulls.NullAt(i) { // We only want to perform the projection operation if the value is not null. arg := col.Get(i) - projCol[i] = bytes.HasSuffix(arg, p.constArg) + projCol[i] = bytes.HasSuffix(arg, p.constArg) != _negate } } } else { @@ -124,7 +128,7 @@ func (p projSuffixBytesBytesConstOp) Next() coldata.Batch { if !colNulls.NullAt(i) { // We only want to perform the projection operation if the value is not null. arg := col.Get(i) - projCol[i] = bytes.HasSuffix(arg, p.constArg) + projCol[i] = bytes.HasSuffix(arg, p.constArg) != _negate } } } @@ -134,14 +138,14 @@ func (p projSuffixBytesBytesConstOp) Next() coldata.Batch { sel = sel[:n] for _, i := range sel { arg := col.Get(i) - projCol[i] = bytes.HasSuffix(arg, p.constArg) + projCol[i] = bytes.HasSuffix(arg, p.constArg) != _negate } } else { _ = projCol.Get(n - 1) _ = col.Get(n - 1) for i := 0; i < n; i++ { arg := col.Get(i) - projCol[i] = bytes.HasSuffix(arg, p.constArg) + projCol[i] = bytes.HasSuffix(arg, p.constArg) != _negate } } } @@ -152,9 +156,11 @@ func (p projSuffixBytesBytesConstOp) Next() coldata.Batch { type projContainsBytesBytesConstOp struct { projConstOpBase constArg []byte + negate bool } func (p projContainsBytesBytesConstOp) Next() coldata.Batch { + _negate := p.negate batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -180,7 +186,7 @@ func (p projContainsBytesBytesConstOp) Next() coldata.Batch { if !colNulls.NullAt(i) { // We only want to perform the projection operation if the value is not null. arg := col.Get(i) - projCol[i] = bytes.Contains(arg, p.constArg) + projCol[i] = bytes.Contains(arg, p.constArg) != _negate } } } else { @@ -190,7 +196,7 @@ func (p projContainsBytesBytesConstOp) Next() coldata.Batch { if !colNulls.NullAt(i) { // We only want to perform the projection operation if the value is not null. arg := col.Get(i) - projCol[i] = bytes.Contains(arg, p.constArg) + projCol[i] = bytes.Contains(arg, p.constArg) != _negate } } } @@ -200,14 +206,14 @@ func (p projContainsBytesBytesConstOp) Next() coldata.Batch { sel = sel[:n] for _, i := range sel { arg := col.Get(i) - projCol[i] = bytes.Contains(arg, p.constArg) + projCol[i] = bytes.Contains(arg, p.constArg) != _negate } } else { _ = projCol.Get(n - 1) _ = col.Get(n - 1) for i := 0; i < n; i++ { arg := col.Get(i) - projCol[i] = bytes.Contains(arg, p.constArg) + projCol[i] = bytes.Contains(arg, p.constArg) != _negate } } } @@ -218,9 +224,11 @@ func (p projContainsBytesBytesConstOp) Next() coldata.Batch { type projSkeletonBytesBytesConstOp struct { projConstOpBase constArg [][]byte + negate bool } func (p projSkeletonBytesBytesConstOp) Next() coldata.Batch { + _negate := p.negate batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -257,7 +265,7 @@ func (p projSkeletonBytesBytesConstOp) Next() coldata.Batch { arg = arg[idx+len(p.constArg[skeletonIdx]):] skeletonIdx++ } - projCol[i] = skeletonIdx == len(p.constArg) + projCol[i] = skeletonIdx == len(p.constArg) != _negate } } } @@ -279,7 +287,7 @@ func (p projSkeletonBytesBytesConstOp) Next() coldata.Batch { arg = arg[idx+len(p.constArg[skeletonIdx]):] skeletonIdx++ } - projCol[i] = skeletonIdx == len(p.constArg) + projCol[i] = skeletonIdx == len(p.constArg) != _negate } } } @@ -301,7 +309,7 @@ func (p projSkeletonBytesBytesConstOp) Next() coldata.Batch { arg = arg[idx+len(p.constArg[skeletonIdx]):] skeletonIdx++ } - projCol[i] = skeletonIdx == len(p.constArg) + projCol[i] = skeletonIdx == len(p.constArg) != _negate } } } else { @@ -320,7 +328,7 @@ func (p projSkeletonBytesBytesConstOp) Next() coldata.Batch { arg = arg[idx+len(p.constArg[skeletonIdx]):] skeletonIdx++ } - projCol[i] = skeletonIdx == len(p.constArg) + projCol[i] = skeletonIdx == len(p.constArg) != _negate } } } @@ -332,9 +340,11 @@ func (p projSkeletonBytesBytesConstOp) Next() coldata.Batch { type projRegexpBytesBytesConstOp struct { projConstOpBase constArg *regexp.Regexp + negate bool } func (p projRegexpBytesBytesConstOp) Next() coldata.Batch { + _negate := p.negate batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -360,7 +370,7 @@ func (p projRegexpBytesBytesConstOp) Next() coldata.Batch { if !colNulls.NullAt(i) { // We only want to perform the projection operation if the value is not null. arg := col.Get(i) - projCol[i] = p.constArg.Match(arg) + projCol[i] = p.constArg.Match(arg) != _negate } } } else { @@ -370,7 +380,7 @@ func (p projRegexpBytesBytesConstOp) Next() coldata.Batch { if !colNulls.NullAt(i) { // We only want to perform the projection operation if the value is not null. arg := col.Get(i) - projCol[i] = p.constArg.Match(arg) + projCol[i] = p.constArg.Match(arg) != _negate } } } @@ -380,392 +390,14 @@ func (p projRegexpBytesBytesConstOp) Next() coldata.Batch { sel = sel[:n] for _, i := range sel { arg := col.Get(i) - projCol[i] = p.constArg.Match(arg) + projCol[i] = p.constArg.Match(arg) != _negate } } else { _ = projCol.Get(n - 1) _ = col.Get(n - 1) for i := 0; i < n; i++ { arg := col.Get(i) - projCol[i] = p.constArg.Match(arg) - } - } - } - }) - return batch -} - -type projNotPrefixBytesBytesConstOp struct { - projConstOpBase - constArg []byte -} - -func (p projNotPrefixBytesBytesConstOp) Next() coldata.Batch { - batch := p.Input.Next() - n := batch.Length() - if n == 0 { - return coldata.ZeroBatch - } - vec := batch.ColVec(p.colIdx) - var col *coldata.Bytes - col = vec.Bytes() - projVec := batch.ColVec(p.outputIdx) - p.allocator.PerformOperation([]coldata.Vec{projVec}, func() { - // Capture col to force bounds check to work. See - // https://github.com/golang/go/issues/39756 - col := col - projCol := projVec.Bool() - _outNulls := projVec.Nulls() - - hasNullsAndNotCalledOnNullInput := vec.Nulls().MaybeHasNulls() - if hasNullsAndNotCalledOnNullInput { - colNulls := vec.Nulls() - if sel := batch.Selection(); sel != nil { - sel = sel[:n] - for _, i := range sel { - if !colNulls.NullAt(i) { - // We only want to perform the projection operation if the value is not null. - arg := col.Get(i) - projCol[i] = !bytes.HasPrefix(arg, p.constArg) - } - } - } else { - _ = projCol.Get(n - 1) - _ = col.Get(n - 1) - for i := 0; i < n; i++ { - if !colNulls.NullAt(i) { - // We only want to perform the projection operation if the value is not null. - arg := col.Get(i) - projCol[i] = !bytes.HasPrefix(arg, p.constArg) - } - } - } - projVec.SetNulls(_outNulls.Or(*colNulls)) - } else { - if sel := batch.Selection(); sel != nil { - sel = sel[:n] - for _, i := range sel { - arg := col.Get(i) - projCol[i] = !bytes.HasPrefix(arg, p.constArg) - } - } else { - _ = projCol.Get(n - 1) - _ = col.Get(n - 1) - for i := 0; i < n; i++ { - arg := col.Get(i) - projCol[i] = !bytes.HasPrefix(arg, p.constArg) - } - } - } - }) - return batch -} - -type projNotSuffixBytesBytesConstOp struct { - projConstOpBase - constArg []byte -} - -func (p projNotSuffixBytesBytesConstOp) Next() coldata.Batch { - batch := p.Input.Next() - n := batch.Length() - if n == 0 { - return coldata.ZeroBatch - } - vec := batch.ColVec(p.colIdx) - var col *coldata.Bytes - col = vec.Bytes() - projVec := batch.ColVec(p.outputIdx) - p.allocator.PerformOperation([]coldata.Vec{projVec}, func() { - // Capture col to force bounds check to work. See - // https://github.com/golang/go/issues/39756 - col := col - projCol := projVec.Bool() - _outNulls := projVec.Nulls() - - hasNullsAndNotCalledOnNullInput := vec.Nulls().MaybeHasNulls() - if hasNullsAndNotCalledOnNullInput { - colNulls := vec.Nulls() - if sel := batch.Selection(); sel != nil { - sel = sel[:n] - for _, i := range sel { - if !colNulls.NullAt(i) { - // We only want to perform the projection operation if the value is not null. - arg := col.Get(i) - projCol[i] = !bytes.HasSuffix(arg, p.constArg) - } - } - } else { - _ = projCol.Get(n - 1) - _ = col.Get(n - 1) - for i := 0; i < n; i++ { - if !colNulls.NullAt(i) { - // We only want to perform the projection operation if the value is not null. - arg := col.Get(i) - projCol[i] = !bytes.HasSuffix(arg, p.constArg) - } - } - } - projVec.SetNulls(_outNulls.Or(*colNulls)) - } else { - if sel := batch.Selection(); sel != nil { - sel = sel[:n] - for _, i := range sel { - arg := col.Get(i) - projCol[i] = !bytes.HasSuffix(arg, p.constArg) - } - } else { - _ = projCol.Get(n - 1) - _ = col.Get(n - 1) - for i := 0; i < n; i++ { - arg := col.Get(i) - projCol[i] = !bytes.HasSuffix(arg, p.constArg) - } - } - } - }) - return batch -} - -type projNotContainsBytesBytesConstOp struct { - projConstOpBase - constArg []byte -} - -func (p projNotContainsBytesBytesConstOp) Next() coldata.Batch { - batch := p.Input.Next() - n := batch.Length() - if n == 0 { - return coldata.ZeroBatch - } - vec := batch.ColVec(p.colIdx) - var col *coldata.Bytes - col = vec.Bytes() - projVec := batch.ColVec(p.outputIdx) - p.allocator.PerformOperation([]coldata.Vec{projVec}, func() { - // Capture col to force bounds check to work. See - // https://github.com/golang/go/issues/39756 - col := col - projCol := projVec.Bool() - _outNulls := projVec.Nulls() - - hasNullsAndNotCalledOnNullInput := vec.Nulls().MaybeHasNulls() - if hasNullsAndNotCalledOnNullInput { - colNulls := vec.Nulls() - if sel := batch.Selection(); sel != nil { - sel = sel[:n] - for _, i := range sel { - if !colNulls.NullAt(i) { - // We only want to perform the projection operation if the value is not null. - arg := col.Get(i) - projCol[i] = !bytes.Contains(arg, p.constArg) - } - } - } else { - _ = projCol.Get(n - 1) - _ = col.Get(n - 1) - for i := 0; i < n; i++ { - if !colNulls.NullAt(i) { - // We only want to perform the projection operation if the value is not null. - arg := col.Get(i) - projCol[i] = !bytes.Contains(arg, p.constArg) - } - } - } - projVec.SetNulls(_outNulls.Or(*colNulls)) - } else { - if sel := batch.Selection(); sel != nil { - sel = sel[:n] - for _, i := range sel { - arg := col.Get(i) - projCol[i] = !bytes.Contains(arg, p.constArg) - } - } else { - _ = projCol.Get(n - 1) - _ = col.Get(n - 1) - for i := 0; i < n; i++ { - arg := col.Get(i) - projCol[i] = !bytes.Contains(arg, p.constArg) - } - } - } - }) - return batch -} - -type projNotSkeletonBytesBytesConstOp struct { - projConstOpBase - constArg [][]byte -} - -func (p projNotSkeletonBytesBytesConstOp) Next() coldata.Batch { - batch := p.Input.Next() - n := batch.Length() - if n == 0 { - return coldata.ZeroBatch - } - vec := batch.ColVec(p.colIdx) - var col *coldata.Bytes - col = vec.Bytes() - projVec := batch.ColVec(p.outputIdx) - p.allocator.PerformOperation([]coldata.Vec{projVec}, func() { - // Capture col to force bounds check to work. See - // https://github.com/golang/go/issues/39756 - col := col - projCol := projVec.Bool() - _outNulls := projVec.Nulls() - - hasNullsAndNotCalledOnNullInput := vec.Nulls().MaybeHasNulls() - if hasNullsAndNotCalledOnNullInput { - colNulls := vec.Nulls() - if sel := batch.Selection(); sel != nil { - sel = sel[:n] - for _, i := range sel { - if !colNulls.NullAt(i) { - // We only want to perform the projection operation if the value is not null. - arg := col.Get(i) - - { - var idx, skeletonIdx int - for skeletonIdx < len(p.constArg) { - idx = bytes.Index(arg, p.constArg[skeletonIdx]) - if idx < 0 { - break - } - arg = arg[idx+len(p.constArg[skeletonIdx]):] - skeletonIdx++ - } - projCol[i] = skeletonIdx != len(p.constArg) - } - } - } - } else { - _ = projCol.Get(n - 1) - _ = col.Get(n - 1) - for i := 0; i < n; i++ { - if !colNulls.NullAt(i) { - // We only want to perform the projection operation if the value is not null. - arg := col.Get(i) - - { - var idx, skeletonIdx int - for skeletonIdx < len(p.constArg) { - idx = bytes.Index(arg, p.constArg[skeletonIdx]) - if idx < 0 { - break - } - arg = arg[idx+len(p.constArg[skeletonIdx]):] - skeletonIdx++ - } - projCol[i] = skeletonIdx != len(p.constArg) - } - } - } - } - projVec.SetNulls(_outNulls.Or(*colNulls)) - } else { - if sel := batch.Selection(); sel != nil { - sel = sel[:n] - for _, i := range sel { - arg := col.Get(i) - - { - var idx, skeletonIdx int - for skeletonIdx < len(p.constArg) { - idx = bytes.Index(arg, p.constArg[skeletonIdx]) - if idx < 0 { - break - } - arg = arg[idx+len(p.constArg[skeletonIdx]):] - skeletonIdx++ - } - projCol[i] = skeletonIdx != len(p.constArg) - } - } - } else { - _ = projCol.Get(n - 1) - _ = col.Get(n - 1) - for i := 0; i < n; i++ { - arg := col.Get(i) - - { - var idx, skeletonIdx int - for skeletonIdx < len(p.constArg) { - idx = bytes.Index(arg, p.constArg[skeletonIdx]) - if idx < 0 { - break - } - arg = arg[idx+len(p.constArg[skeletonIdx]):] - skeletonIdx++ - } - projCol[i] = skeletonIdx != len(p.constArg) - } - } - } - } - }) - return batch -} - -type projNotRegexpBytesBytesConstOp struct { - projConstOpBase - constArg *regexp.Regexp -} - -func (p projNotRegexpBytesBytesConstOp) Next() coldata.Batch { - batch := p.Input.Next() - n := batch.Length() - if n == 0 { - return coldata.ZeroBatch - } - vec := batch.ColVec(p.colIdx) - var col *coldata.Bytes - col = vec.Bytes() - projVec := batch.ColVec(p.outputIdx) - p.allocator.PerformOperation([]coldata.Vec{projVec}, func() { - // Capture col to force bounds check to work. See - // https://github.com/golang/go/issues/39756 - col := col - projCol := projVec.Bool() - _outNulls := projVec.Nulls() - - hasNullsAndNotCalledOnNullInput := vec.Nulls().MaybeHasNulls() - if hasNullsAndNotCalledOnNullInput { - colNulls := vec.Nulls() - if sel := batch.Selection(); sel != nil { - sel = sel[:n] - for _, i := range sel { - if !colNulls.NullAt(i) { - // We only want to perform the projection operation if the value is not null. - arg := col.Get(i) - projCol[i] = !p.constArg.Match(arg) - } - } - } else { - _ = projCol.Get(n - 1) - _ = col.Get(n - 1) - for i := 0; i < n; i++ { - if !colNulls.NullAt(i) { - // We only want to perform the projection operation if the value is not null. - arg := col.Get(i) - projCol[i] = !p.constArg.Match(arg) - } - } - } - projVec.SetNulls(_outNulls.Or(*colNulls)) - } else { - if sel := batch.Selection(); sel != nil { - sel = sel[:n] - for _, i := range sel { - arg := col.Get(i) - projCol[i] = !p.constArg.Match(arg) - } - } else { - _ = projCol.Get(n - 1) - _ = col.Get(n - 1) - for i := 0; i < n; i++ { - arg := col.Get(i) - projCol[i] = !p.constArg.Match(arg) + projCol[i] = p.constArg.Match(arg) != _negate } } } diff --git a/pkg/sql/colexec/colexecsel/like_ops.go b/pkg/sql/colexec/colexecsel/like_ops.go index a94a9531af0a..8d20b43bd9ab 100644 --- a/pkg/sql/colexec/colexecsel/like_ops.go +++ b/pkg/sql/colexec/colexecsel/like_ops.go @@ -23,7 +23,7 @@ import ( func GetLikeOperator( ctx *eval.Context, input colexecop.Operator, colIdx int, pattern string, negate bool, ) (colexecop.Operator, error) { - likeOpType, patterns, err := colexeccmp.GetLikeOperatorType(pattern, negate) + likeOpType, patterns, err := colexeccmp.GetLikeOperatorType(pattern) if err != nil { return nil, err } @@ -33,45 +33,21 @@ func GetLikeOperator( colIdx: colIdx, } switch likeOpType { - case colexeccmp.LikeConstant: - return &selEQBytesBytesConstOp{ - selConstOpBase: base, - constArg: pat, - }, nil - case colexeccmp.LikeConstantNegate: - return &selNEBytesBytesConstOp{ - selConstOpBase: base, - constArg: pat, - }, nil - case colexeccmp.LikeNeverMatch: - // Use an empty not-prefix operator to get correct NULL behavior. - return &selNotPrefixBytesBytesConstOp{ - selConstOpBase: base, - constArg: []byte{}, - }, nil case colexeccmp.LikeAlwaysMatch: // Use an empty prefix operator to get correct NULL behavior. return &selPrefixBytesBytesConstOp{ selConstOpBase: base, constArg: []byte{}, + negate: negate, }, nil - case colexeccmp.LikeSuffix: - return &selSuffixBytesBytesConstOp{ - selConstOpBase: base, - constArg: pat, - }, nil - case colexeccmp.LikeSuffixNegate: - return &selNotSuffixBytesBytesConstOp{ - selConstOpBase: base, - constArg: pat, - }, nil - case colexeccmp.LikePrefix: - return &selPrefixBytesBytesConstOp{ - selConstOpBase: base, - constArg: pat, - }, nil - case colexeccmp.LikePrefixNegate: - return &selNotPrefixBytesBytesConstOp{ + case colexeccmp.LikeConstant: + if negate { + return &selNEBytesBytesConstOp{ + selConstOpBase: base, + constArg: pat, + }, nil + } + return &selEQBytesBytesConstOp{ selConstOpBase: base, constArg: pat, }, nil @@ -79,21 +55,13 @@ func GetLikeOperator( return &selContainsBytesBytesConstOp{ selConstOpBase: base, constArg: pat, + negate: negate, }, nil - case colexeccmp.LikeContainsNegate: - return &selNotContainsBytesBytesConstOp{ + case colexeccmp.LikePrefix: + return &selPrefixBytesBytesConstOp{ selConstOpBase: base, constArg: pat, - }, nil - case colexeccmp.LikeSkeleton: - return &selSkeletonBytesBytesConstOp{ - selConstOpBase: base, - constArg: patterns, - }, nil - case colexeccmp.LikeSkeletonNegate: - return &selNotSkeletonBytesBytesConstOp{ - selConstOpBase: base, - constArg: patterns, + negate: negate, }, nil case colexeccmp.LikeRegexp: re, err := eval.ConvertLikeToRegexp(ctx, string(patterns[0]), false, '\\') @@ -103,15 +71,19 @@ func GetLikeOperator( return &selRegexpBytesBytesConstOp{ selConstOpBase: base, constArg: re, + negate: negate, }, nil - case colexeccmp.LikeRegexpNegate: - re, err := eval.ConvertLikeToRegexp(ctx, string(patterns[0]), false, '\\') - if err != nil { - return nil, err - } - return &selNotRegexpBytesBytesConstOp{ + case colexeccmp.LikeSkeleton: + return &selSkeletonBytesBytesConstOp{ selConstOpBase: base, - constArg: re, + constArg: patterns, + negate: negate, + }, nil + case colexeccmp.LikeSuffix: + return &selSuffixBytesBytesConstOp{ + selConstOpBase: base, + constArg: pat, + negate: negate, }, nil default: return nil, errors.AssertionFailedf("unsupported like op type %d", likeOpType) diff --git a/pkg/sql/colexec/colexecsel/sel_like_ops.eg.go b/pkg/sql/colexec/colexecsel/sel_like_ops.eg.go index 13d7f41c0efc..01e4dc0484d4 100644 --- a/pkg/sql/colexec/colexecsel/sel_like_ops.eg.go +++ b/pkg/sql/colexec/colexecsel/sel_like_ops.eg.go @@ -20,9 +20,11 @@ import ( type selPrefixBytesBytesConstOp struct { selConstOpBase constArg []byte + negate bool } func (p *selPrefixBytesBytesConstOp) Next() coldata.Batch { + _negate := p.negate for { batch := p.Input.Next() if batch.Length() == 0 { @@ -43,7 +45,7 @@ func (p *selPrefixBytesBytesConstOp) Next() coldata.Batch { } var cmp bool arg := col.Get(i) - cmp = bytes.HasPrefix(arg, p.constArg) + cmp = bytes.HasPrefix(arg, p.constArg) != _negate if cmp { sel[idx] = i idx++ @@ -59,7 +61,7 @@ func (p *selPrefixBytesBytesConstOp) Next() coldata.Batch { } var cmp bool arg := col.Get(i) - cmp = bytes.HasPrefix(arg, p.constArg) + cmp = bytes.HasPrefix(arg, p.constArg) != _negate if cmp { sel[idx] = i idx++ @@ -72,7 +74,7 @@ func (p *selPrefixBytesBytesConstOp) Next() coldata.Batch { for _, i := range sel { var cmp bool arg := col.Get(i) - cmp = bytes.HasPrefix(arg, p.constArg) + cmp = bytes.HasPrefix(arg, p.constArg) != _negate if cmp { sel[idx] = i idx++ @@ -85,7 +87,7 @@ func (p *selPrefixBytesBytesConstOp) Next() coldata.Batch { for i := 0; i < n; i++ { var cmp bool arg := col.Get(i) - cmp = bytes.HasPrefix(arg, p.constArg) + cmp = bytes.HasPrefix(arg, p.constArg) != _negate if cmp { sel[idx] = i idx++ @@ -103,9 +105,11 @@ func (p *selPrefixBytesBytesConstOp) Next() coldata.Batch { type selSuffixBytesBytesConstOp struct { selConstOpBase constArg []byte + negate bool } func (p *selSuffixBytesBytesConstOp) Next() coldata.Batch { + _negate := p.negate for { batch := p.Input.Next() if batch.Length() == 0 { @@ -126,7 +130,7 @@ func (p *selSuffixBytesBytesConstOp) Next() coldata.Batch { } var cmp bool arg := col.Get(i) - cmp = bytes.HasSuffix(arg, p.constArg) + cmp = bytes.HasSuffix(arg, p.constArg) != _negate if cmp { sel[idx] = i idx++ @@ -142,7 +146,7 @@ func (p *selSuffixBytesBytesConstOp) Next() coldata.Batch { } var cmp bool arg := col.Get(i) - cmp = bytes.HasSuffix(arg, p.constArg) + cmp = bytes.HasSuffix(arg, p.constArg) != _negate if cmp { sel[idx] = i idx++ @@ -155,7 +159,7 @@ func (p *selSuffixBytesBytesConstOp) Next() coldata.Batch { for _, i := range sel { var cmp bool arg := col.Get(i) - cmp = bytes.HasSuffix(arg, p.constArg) + cmp = bytes.HasSuffix(arg, p.constArg) != _negate if cmp { sel[idx] = i idx++ @@ -168,7 +172,7 @@ func (p *selSuffixBytesBytesConstOp) Next() coldata.Batch { for i := 0; i < n; i++ { var cmp bool arg := col.Get(i) - cmp = bytes.HasSuffix(arg, p.constArg) + cmp = bytes.HasSuffix(arg, p.constArg) != _negate if cmp { sel[idx] = i idx++ @@ -186,9 +190,11 @@ func (p *selSuffixBytesBytesConstOp) Next() coldata.Batch { type selContainsBytesBytesConstOp struct { selConstOpBase constArg []byte + negate bool } func (p *selContainsBytesBytesConstOp) Next() coldata.Batch { + _negate := p.negate for { batch := p.Input.Next() if batch.Length() == 0 { @@ -209,7 +215,7 @@ func (p *selContainsBytesBytesConstOp) Next() coldata.Batch { } var cmp bool arg := col.Get(i) - cmp = bytes.Contains(arg, p.constArg) + cmp = bytes.Contains(arg, p.constArg) != _negate if cmp { sel[idx] = i idx++ @@ -225,7 +231,7 @@ func (p *selContainsBytesBytesConstOp) Next() coldata.Batch { } var cmp bool arg := col.Get(i) - cmp = bytes.Contains(arg, p.constArg) + cmp = bytes.Contains(arg, p.constArg) != _negate if cmp { sel[idx] = i idx++ @@ -238,7 +244,7 @@ func (p *selContainsBytesBytesConstOp) Next() coldata.Batch { for _, i := range sel { var cmp bool arg := col.Get(i) - cmp = bytes.Contains(arg, p.constArg) + cmp = bytes.Contains(arg, p.constArg) != _negate if cmp { sel[idx] = i idx++ @@ -251,7 +257,7 @@ func (p *selContainsBytesBytesConstOp) Next() coldata.Batch { for i := 0; i < n; i++ { var cmp bool arg := col.Get(i) - cmp = bytes.Contains(arg, p.constArg) + cmp = bytes.Contains(arg, p.constArg) != _negate if cmp { sel[idx] = i idx++ @@ -269,9 +275,11 @@ func (p *selContainsBytesBytesConstOp) Next() coldata.Batch { type selSkeletonBytesBytesConstOp struct { selConstOpBase constArg [][]byte + negate bool } func (p *selSkeletonBytesBytesConstOp) Next() coldata.Batch { + _negate := p.negate for { batch := p.Input.Next() if batch.Length() == 0 { @@ -303,7 +311,7 @@ func (p *selSkeletonBytesBytesConstOp) Next() coldata.Batch { arg = arg[idx+len(p.constArg[skeletonIdx]):] skeletonIdx++ } - cmp = skeletonIdx == len(p.constArg) + cmp = skeletonIdx == len(p.constArg) != _negate } if cmp { sel[idx] = i @@ -331,7 +339,7 @@ func (p *selSkeletonBytesBytesConstOp) Next() coldata.Batch { arg = arg[idx+len(p.constArg[skeletonIdx]):] skeletonIdx++ } - cmp = skeletonIdx == len(p.constArg) + cmp = skeletonIdx == len(p.constArg) != _negate } if cmp { sel[idx] = i @@ -356,7 +364,7 @@ func (p *selSkeletonBytesBytesConstOp) Next() coldata.Batch { arg = arg[idx+len(p.constArg[skeletonIdx]):] skeletonIdx++ } - cmp = skeletonIdx == len(p.constArg) + cmp = skeletonIdx == len(p.constArg) != _negate } if cmp { sel[idx] = i @@ -381,7 +389,7 @@ func (p *selSkeletonBytesBytesConstOp) Next() coldata.Batch { arg = arg[idx+len(p.constArg[skeletonIdx]):] skeletonIdx++ } - cmp = skeletonIdx == len(p.constArg) + cmp = skeletonIdx == len(p.constArg) != _negate } if cmp { sel[idx] = i @@ -400,9 +408,11 @@ func (p *selSkeletonBytesBytesConstOp) Next() coldata.Batch { type selRegexpBytesBytesConstOp struct { selConstOpBase constArg *regexp.Regexp + negate bool } func (p *selRegexpBytesBytesConstOp) Next() coldata.Batch { + _negate := p.negate for { batch := p.Input.Next() if batch.Length() == 0 { @@ -423,7 +433,7 @@ func (p *selRegexpBytesBytesConstOp) Next() coldata.Batch { } var cmp bool arg := col.Get(i) - cmp = p.constArg.Match(arg) + cmp = p.constArg.Match(arg) != _negate if cmp { sel[idx] = i idx++ @@ -439,7 +449,7 @@ func (p *selRegexpBytesBytesConstOp) Next() coldata.Batch { } var cmp bool arg := col.Get(i) - cmp = p.constArg.Match(arg) + cmp = p.constArg.Match(arg) != _negate if cmp { sel[idx] = i idx++ @@ -452,7 +462,7 @@ func (p *selRegexpBytesBytesConstOp) Next() coldata.Batch { for _, i := range sel { var cmp bool arg := col.Get(i) - cmp = p.constArg.Match(arg) + cmp = p.constArg.Match(arg) != _negate if cmp { sel[idx] = i idx++ @@ -465,470 +475,7 @@ func (p *selRegexpBytesBytesConstOp) Next() coldata.Batch { for i := 0; i < n; i++ { var cmp bool arg := col.Get(i) - cmp = p.constArg.Match(arg) - if cmp { - sel[idx] = i - idx++ - } - } - } - } - if idx > 0 { - batch.SetLength(idx) - return batch - } - } -} - -type selNotPrefixBytesBytesConstOp struct { - selConstOpBase - constArg []byte -} - -func (p *selNotPrefixBytesBytesConstOp) Next() coldata.Batch { - for { - batch := p.Input.Next() - if batch.Length() == 0 { - return batch - } - - vec := batch.ColVec(p.colIdx) - col := vec.Bytes() - var idx int - n := batch.Length() - if vec.MaybeHasNulls() { - nulls := vec.Nulls() - if sel := batch.Selection(); sel != nil { - sel = sel[:n] - for _, i := range sel { - if nulls.NullAt(i) { - continue - } - var cmp bool - arg := col.Get(i) - cmp = !bytes.HasPrefix(arg, p.constArg) - if cmp { - sel[idx] = i - idx++ - } - } - } else { - batch.SetSelection(true) - sel := batch.Selection() - _ = col.Get(n - 1) - for i := 0; i < n; i++ { - if nulls.NullAt(i) { - continue - } - var cmp bool - arg := col.Get(i) - cmp = !bytes.HasPrefix(arg, p.constArg) - if cmp { - sel[idx] = i - idx++ - } - } - } - } else { - if sel := batch.Selection(); sel != nil { - sel = sel[:n] - for _, i := range sel { - var cmp bool - arg := col.Get(i) - cmp = !bytes.HasPrefix(arg, p.constArg) - if cmp { - sel[idx] = i - idx++ - } - } - } else { - batch.SetSelection(true) - sel := batch.Selection() - _ = col.Get(n - 1) - for i := 0; i < n; i++ { - var cmp bool - arg := col.Get(i) - cmp = !bytes.HasPrefix(arg, p.constArg) - if cmp { - sel[idx] = i - idx++ - } - } - } - } - if idx > 0 { - batch.SetLength(idx) - return batch - } - } -} - -type selNotSuffixBytesBytesConstOp struct { - selConstOpBase - constArg []byte -} - -func (p *selNotSuffixBytesBytesConstOp) Next() coldata.Batch { - for { - batch := p.Input.Next() - if batch.Length() == 0 { - return batch - } - - vec := batch.ColVec(p.colIdx) - col := vec.Bytes() - var idx int - n := batch.Length() - if vec.MaybeHasNulls() { - nulls := vec.Nulls() - if sel := batch.Selection(); sel != nil { - sel = sel[:n] - for _, i := range sel { - if nulls.NullAt(i) { - continue - } - var cmp bool - arg := col.Get(i) - cmp = !bytes.HasSuffix(arg, p.constArg) - if cmp { - sel[idx] = i - idx++ - } - } - } else { - batch.SetSelection(true) - sel := batch.Selection() - _ = col.Get(n - 1) - for i := 0; i < n; i++ { - if nulls.NullAt(i) { - continue - } - var cmp bool - arg := col.Get(i) - cmp = !bytes.HasSuffix(arg, p.constArg) - if cmp { - sel[idx] = i - idx++ - } - } - } - } else { - if sel := batch.Selection(); sel != nil { - sel = sel[:n] - for _, i := range sel { - var cmp bool - arg := col.Get(i) - cmp = !bytes.HasSuffix(arg, p.constArg) - if cmp { - sel[idx] = i - idx++ - } - } - } else { - batch.SetSelection(true) - sel := batch.Selection() - _ = col.Get(n - 1) - for i := 0; i < n; i++ { - var cmp bool - arg := col.Get(i) - cmp = !bytes.HasSuffix(arg, p.constArg) - if cmp { - sel[idx] = i - idx++ - } - } - } - } - if idx > 0 { - batch.SetLength(idx) - return batch - } - } -} - -type selNotContainsBytesBytesConstOp struct { - selConstOpBase - constArg []byte -} - -func (p *selNotContainsBytesBytesConstOp) Next() coldata.Batch { - for { - batch := p.Input.Next() - if batch.Length() == 0 { - return batch - } - - vec := batch.ColVec(p.colIdx) - col := vec.Bytes() - var idx int - n := batch.Length() - if vec.MaybeHasNulls() { - nulls := vec.Nulls() - if sel := batch.Selection(); sel != nil { - sel = sel[:n] - for _, i := range sel { - if nulls.NullAt(i) { - continue - } - var cmp bool - arg := col.Get(i) - cmp = !bytes.Contains(arg, p.constArg) - if cmp { - sel[idx] = i - idx++ - } - } - } else { - batch.SetSelection(true) - sel := batch.Selection() - _ = col.Get(n - 1) - for i := 0; i < n; i++ { - if nulls.NullAt(i) { - continue - } - var cmp bool - arg := col.Get(i) - cmp = !bytes.Contains(arg, p.constArg) - if cmp { - sel[idx] = i - idx++ - } - } - } - } else { - if sel := batch.Selection(); sel != nil { - sel = sel[:n] - for _, i := range sel { - var cmp bool - arg := col.Get(i) - cmp = !bytes.Contains(arg, p.constArg) - if cmp { - sel[idx] = i - idx++ - } - } - } else { - batch.SetSelection(true) - sel := batch.Selection() - _ = col.Get(n - 1) - for i := 0; i < n; i++ { - var cmp bool - arg := col.Get(i) - cmp = !bytes.Contains(arg, p.constArg) - if cmp { - sel[idx] = i - idx++ - } - } - } - } - if idx > 0 { - batch.SetLength(idx) - return batch - } - } -} - -type selNotSkeletonBytesBytesConstOp struct { - selConstOpBase - constArg [][]byte -} - -func (p *selNotSkeletonBytesBytesConstOp) Next() coldata.Batch { - for { - batch := p.Input.Next() - if batch.Length() == 0 { - return batch - } - - vec := batch.ColVec(p.colIdx) - col := vec.Bytes() - var idx int - n := batch.Length() - if vec.MaybeHasNulls() { - nulls := vec.Nulls() - if sel := batch.Selection(); sel != nil { - sel = sel[:n] - for _, i := range sel { - if nulls.NullAt(i) { - continue - } - var cmp bool - arg := col.Get(i) - - { - var idx, skeletonIdx int - for skeletonIdx < len(p.constArg) { - idx = bytes.Index(arg, p.constArg[skeletonIdx]) - if idx < 0 { - break - } - arg = arg[idx+len(p.constArg[skeletonIdx]):] - skeletonIdx++ - } - cmp = skeletonIdx != len(p.constArg) - } - if cmp { - sel[idx] = i - idx++ - } - } - } else { - batch.SetSelection(true) - sel := batch.Selection() - _ = col.Get(n - 1) - for i := 0; i < n; i++ { - if nulls.NullAt(i) { - continue - } - var cmp bool - arg := col.Get(i) - - { - var idx, skeletonIdx int - for skeletonIdx < len(p.constArg) { - idx = bytes.Index(arg, p.constArg[skeletonIdx]) - if idx < 0 { - break - } - arg = arg[idx+len(p.constArg[skeletonIdx]):] - skeletonIdx++ - } - cmp = skeletonIdx != len(p.constArg) - } - if cmp { - sel[idx] = i - idx++ - } - } - } - } else { - if sel := batch.Selection(); sel != nil { - sel = sel[:n] - for _, i := range sel { - var cmp bool - arg := col.Get(i) - - { - var idx, skeletonIdx int - for skeletonIdx < len(p.constArg) { - idx = bytes.Index(arg, p.constArg[skeletonIdx]) - if idx < 0 { - break - } - arg = arg[idx+len(p.constArg[skeletonIdx]):] - skeletonIdx++ - } - cmp = skeletonIdx != len(p.constArg) - } - if cmp { - sel[idx] = i - idx++ - } - } - } else { - batch.SetSelection(true) - sel := batch.Selection() - _ = col.Get(n - 1) - for i := 0; i < n; i++ { - var cmp bool - arg := col.Get(i) - - { - var idx, skeletonIdx int - for skeletonIdx < len(p.constArg) { - idx = bytes.Index(arg, p.constArg[skeletonIdx]) - if idx < 0 { - break - } - arg = arg[idx+len(p.constArg[skeletonIdx]):] - skeletonIdx++ - } - cmp = skeletonIdx != len(p.constArg) - } - if cmp { - sel[idx] = i - idx++ - } - } - } - } - if idx > 0 { - batch.SetLength(idx) - return batch - } - } -} - -type selNotRegexpBytesBytesConstOp struct { - selConstOpBase - constArg *regexp.Regexp -} - -func (p *selNotRegexpBytesBytesConstOp) Next() coldata.Batch { - for { - batch := p.Input.Next() - if batch.Length() == 0 { - return batch - } - - vec := batch.ColVec(p.colIdx) - col := vec.Bytes() - var idx int - n := batch.Length() - if vec.MaybeHasNulls() { - nulls := vec.Nulls() - if sel := batch.Selection(); sel != nil { - sel = sel[:n] - for _, i := range sel { - if nulls.NullAt(i) { - continue - } - var cmp bool - arg := col.Get(i) - cmp = !p.constArg.Match(arg) - if cmp { - sel[idx] = i - idx++ - } - } - } else { - batch.SetSelection(true) - sel := batch.Selection() - _ = col.Get(n - 1) - for i := 0; i < n; i++ { - if nulls.NullAt(i) { - continue - } - var cmp bool - arg := col.Get(i) - cmp = !p.constArg.Match(arg) - if cmp { - sel[idx] = i - idx++ - } - } - } - } else { - if sel := batch.Selection(); sel != nil { - sel = sel[:n] - for _, i := range sel { - var cmp bool - arg := col.Get(i) - cmp = !p.constArg.Match(arg) - if cmp { - sel[idx] = i - idx++ - } - } - } else { - batch.SetSelection(true) - sel := batch.Selection() - _ = col.Get(n - 1) - for i := 0; i < n; i++ { - var cmp bool - arg := col.Get(i) - cmp = !p.constArg.Match(arg) + cmp = p.constArg.Match(arg) != _negate if cmp { sel[idx] = i idx++ diff --git a/pkg/sql/colexec/colexecsel/selection_ops_tmpl.go b/pkg/sql/colexec/colexecsel/selection_ops_tmpl.go index f8416ce07ba6..21cdbb23ae0d 100644 --- a/pkg/sql/colexec/colexecsel/selection_ops_tmpl.go +++ b/pkg/sql/colexec/colexecsel/selection_ops_tmpl.go @@ -191,9 +191,20 @@ type selOpBase struct { type _OP_CONST_NAME struct { selConstOpBase constArg _R_GO_TYPE + // {{if .Negatable}} + negate bool + // {{end}} } func (p *_OP_CONST_NAME) Next() coldata.Batch { + // {{if .Negatable}} + // {{/* + // In order to inline the templated code of the LIKE overloads, we need + // to have a `_negate` local variable indicating whether the assignment + // should be negated. + // */}} + _negate := p.negate + // {{end}} for { batch := p.Input.Next() if batch.Length() == 0 { diff --git a/pkg/sql/colexec/execgen/cmd/execgen/like_ops_gen.go b/pkg/sql/colexec/execgen/cmd/execgen/like_ops_gen.go index a848e3c7abea..c049508c512c 100644 --- a/pkg/sql/colexec/execgen/cmd/execgen/like_ops_gen.go +++ b/pkg/sql/colexec/execgen/cmd/execgen/like_ops_gen.go @@ -115,6 +115,7 @@ func genLikeOps( overloadBase: base, Left: leftWidthOverload, Right: rightWidthOverload, + Negatable: true, } } // makeSkeletonAssignFunc returns a string that assigns 'targetElem' to @@ -126,7 +127,7 @@ func genLikeOps( // find its first occurrence in the unprocessed part of 'leftElem'. If // it is not found, then 'leftElem' doesn't match the pattern, if it is // found, then we advance 'leftElem' right past that first occurrence. - makeSkeletonAssignFunc := func(targetElem, leftElem, rightElem, comparison string) string { + makeSkeletonAssignFunc := func(targetElem, leftElem, rightElem string) string { return fmt.Sprintf(` { var idx, skeletonIdx int @@ -138,39 +139,24 @@ func genLikeOps( %[2]s = %[2]s[idx+len(%[3]s[skeletonIdx]):] skeletonIdx++ } - %[1]s = skeletonIdx %[4]s len(%[3]s) - }`, targetElem, leftElem, rightElem, comparison) + %[1]s = skeletonIdx == len(%[3]s) != _negate + }`, targetElem, leftElem, rightElem) } overloads := []*twoArgsResolvedOverload{ makeOverload("Prefix", bytesRepresentation, func(targetElem, leftElem, rightElem string) string { - return fmt.Sprintf("%s = bytes.HasPrefix(%s, %s)", targetElem, leftElem, rightElem) + return fmt.Sprintf("%s = bytes.HasPrefix(%s, %s) != _negate", targetElem, leftElem, rightElem) }), makeOverload("Suffix", bytesRepresentation, func(targetElem, leftElem, rightElem string) string { - return fmt.Sprintf("%s = bytes.HasSuffix(%s, %s)", targetElem, leftElem, rightElem) + return fmt.Sprintf("%s = bytes.HasSuffix(%s, %s) != _negate", targetElem, leftElem, rightElem) }), makeOverload("Contains", bytesRepresentation, func(targetElem, leftElem, rightElem string) string { - return fmt.Sprintf("%s = bytes.Contains(%s, %s)", targetElem, leftElem, rightElem) + return fmt.Sprintf("%s = bytes.Contains(%s, %s) != _negate", targetElem, leftElem, rightElem) }), makeOverload("Skeleton", "[][]byte", func(targetElem, leftElem, rightElem string) string { - return makeSkeletonAssignFunc(targetElem, leftElem, rightElem, "==") + return makeSkeletonAssignFunc(targetElem, leftElem, rightElem) }), makeOverload("Regexp", "*regexp.Regexp", func(targetElem, leftElem, rightElem string) string { - return fmt.Sprintf("%s = %s.Match(%s)", targetElem, rightElem, leftElem) - }), - makeOverload("NotPrefix", bytesRepresentation, func(targetElem, leftElem, rightElem string) string { - return fmt.Sprintf("%s = !bytes.HasPrefix(%s, %s)", targetElem, leftElem, rightElem) - }), - makeOverload("NotSuffix", bytesRepresentation, func(targetElem, leftElem, rightElem string) string { - return fmt.Sprintf("%s = !bytes.HasSuffix(%s, %s)", targetElem, leftElem, rightElem) - }), - makeOverload("NotContains", bytesRepresentation, func(targetElem, leftElem, rightElem string) string { - return fmt.Sprintf("%s = !bytes.Contains(%s, %s)", targetElem, leftElem, rightElem) - }), - makeOverload("NotSkeleton", "[][]byte", func(targetElem, leftElem, rightElem string) string { - return makeSkeletonAssignFunc(targetElem, leftElem, rightElem, "!=") - }), - makeOverload("NotRegexp", "*regexp.Regexp", func(targetElem, leftElem, rightElem string) string { - return fmt.Sprintf("%s = !%s.Match(%s)", targetElem, rightElem, leftElem) + return fmt.Sprintf("%s = %s.Match(%s) != _negate", targetElem, rightElem, leftElem) }), } return tmpl.Execute(wr, overloads) diff --git a/pkg/sql/colexec/execgen/cmd/execgen/overloads_base.go b/pkg/sql/colexec/execgen/cmd/execgen/overloads_base.go index 74438060106b..303abee77cd3 100644 --- a/pkg/sql/colexec/execgen/cmd/execgen/overloads_base.go +++ b/pkg/sql/colexec/execgen/cmd/execgen/overloads_base.go @@ -312,6 +312,11 @@ type twoArgsResolvedOverload struct { *overloadBase Left *argWidthOverload Right *lastArgWidthOverload + + // Negatable is only used by the LIKE overloads. We cannot easily extract + // out a separate struct for those since we're reusing the same templates as + // all of the selection / projection operators. + Negatable bool } // NeedsBinaryOverloadHelper returns true iff the overload is such that it needs diff --git a/pkg/sql/opt/exec/execbuilder/testdata/tpch_vec b/pkg/sql/opt/exec/execbuilder/testdata/tpch_vec index b2d1699c8400..7f02dd3744d5 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/tpch_vec +++ b/pkg/sql/opt/exec/execbuilder/testdata/tpch_vec @@ -20812,7 +20812,7 @@ EXPLAIN (VEC) SELECT c_count, count(*) AS custdist FROM ( SELECT c_custkey, coun └ *colexec.hashAggregator └ *colexec.hashAggregator └ *colexecjoin.hashJoiner - ├ *colexecsel.selNotSkeletonBytesBytesConstOp + ├ *colexecsel.selSkeletonBytesBytesConstOp │ └ *colfetcher.ColBatchScan └ *colfetcher.ColBatchScan @@ -20876,7 +20876,7 @@ EXPLAIN (VEC) SELECT p_brand, p_type, p_size, count(distinct ps_suppkey) AS supp └ *colexecjoin.hashJoiner ├ *rowexec.joinReader │ └ *colexec.selectInOpInt64 - │ └ *colexecsel.selNotPrefixBytesBytesConstOp + │ └ *colexecsel.selPrefixBytesBytesConstOp │ └ *colexecsel.selNEBytesBytesConstOp │ └ *colfetcher.ColBatchScan └ *colexecsel.selSkeletonBytesBytesConstOp From 02b32a635015bc205f38535b5721544b54ffd0e5 Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Fri, 5 Aug 2022 17:41:46 -0700 Subject: [PATCH 3/7] colexec: add support for ILIKE and NOT ILIKE MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This commit adds the native vectorized support for ILIKE and NOT ILIKE comparisons. The idea is simple - convert both the argument and the pattern to capital letters. This required minor changes to the templates to add a "prelude" step of that conversion as well as conversion of the pattern to the upper case during planning. Initially, I generated separate operators for case-insensitive cases, but the benchmarks shown that the performance impact of a single conditional inside of the `for` loop is barely noticeable given that the branch prediction will always be right, so I refactored the existing operators to support case insensitivity. ``` name old time/op new time/op delta LikeOps/selPrefixBytesBytesConstOp-24 16.8µs ± 0% 17.7µs ± 0% +5.30% (p=0.000 n=10+10) LikeOps/selSuffixBytesBytesConstOp-24 18.7µs ± 0% 19.2µs ± 0% +2.99% (p=0.000 n=10+10) LikeOps/selContainsBytesBytesConstOp-24 28.0µs ± 0% 27.8µs ± 0% -0.73% (p=0.000 n=10+10) LikeOps/selRegexpBytesBytesConstOp-24 479µs ± 0% 480µs ± 0% +0.33% (p=0.008 n=9+10) LikeOps/selSkeletonBytesBytesConstOp-24 40.2µs ± 0% 41.4µs ± 0% +3.20% (p=0.000 n=9+10) LikeOps/selRegexpSkeleton-24 860µs ± 0% 857µs ± 0% -0.36% (p=0.023 n=10+10) ``` Release note (performance improvement): ILIKE and NOT ILIKE filters can now be evaluated more efficiently in some cases. --- pkg/sql/colexec/colbuilder/execplan.go | 22 +- pkg/sql/colexec/colexeccmp/like_ops.go | 5 +- pkg/sql/colexec/colexecprojconst/like_ops.go | 19 +- .../colexecprojconst/proj_const_ops_tmpl.go | 11 + .../colexecprojconst/proj_like_ops.eg.go | 72 ++++- pkg/sql/colexec/colexecsel/like_ops.go | 49 ++-- pkg/sql/colexec/colexecsel/like_ops_test.go | 262 ++++++++++-------- pkg/sql/colexec/colexecsel/sel_like_ops.eg.go | 72 ++++- .../colexec/colexecsel/selection_ops_tmpl.go | 11 + .../execgen/cmd/execgen/like_ops_gen.go | 70 +++-- .../execgen/cmd/execgen/overloads_base.go | 9 +- 11 files changed, 416 insertions(+), 186 deletions(-) diff --git a/pkg/sql/colexec/colbuilder/execplan.go b/pkg/sql/colexec/colbuilder/execplan.go index 23233989c752..6149ca280925 100644 --- a/pkg/sql/colexec/colbuilder/execplan.go +++ b/pkg/sql/colexec/colbuilder/execplan.go @@ -1795,6 +1795,12 @@ func addProjection( return colexecbase.NewSimpleProjectOp(op, len(typs), projection), newTypes } +func examineLikeOp(op treecmp.ComparisonOperator) (negate bool, caseInsensitive bool) { + negate = op.Symbol == treecmp.NotLike || op.Symbol == treecmp.NotILike + caseInsensitive = op.Symbol == treecmp.ILike || op.Symbol == treecmp.NotILike + return negate, caseInsensitive +} + func planSelectionOperators( ctx context.Context, evalCtx *eval.Context, @@ -1842,10 +1848,11 @@ func planSelectionOperators( lTyp := ct[leftIdx] if constArg, ok := t.Right.(tree.Datum); ok { switch cmpOp.Symbol { - case treecmp.Like, treecmp.NotLike: - negate := cmpOp.Symbol == treecmp.NotLike + case treecmp.Like, treecmp.NotLike, treecmp.ILike, treecmp.NotILike: + negate, caseInsensitive := examineLikeOp(cmpOp) op, err = colexecsel.GetLikeOperator( - evalCtx, leftOp, leftIdx, string(tree.MustBeDString(constArg)), negate, + evalCtx, leftOp, leftIdx, string(tree.MustBeDString(constArg)), + negate, caseInsensitive, ) case treecmp.In, treecmp.NotIn: negate := cmpOp.Symbol == treecmp.NotIn @@ -2370,7 +2377,8 @@ func planProjectionExpr( var hasOptimizedOp bool if isCmpProjOp { switch cmpProjOp.Symbol { - case treecmp.Like, treecmp.NotLike, treecmp.In, treecmp.NotIn, treecmp.IsDistinctFrom, treecmp.IsNotDistinctFrom: + case treecmp.Like, treecmp.NotLike, treecmp.ILike, treecmp.NotILike, + treecmp.In, treecmp.NotIn, treecmp.IsDistinctFrom, treecmp.IsNotDistinctFrom: hasOptimizedOp = true } } @@ -2437,11 +2445,11 @@ func planProjectionExpr( resultIdx = len(typs) if isCmpProjOp { switch cmpProjOp.Symbol { - case treecmp.Like, treecmp.NotLike: - negate := cmpProjOp.Symbol == treecmp.NotLike + case treecmp.Like, treecmp.NotLike, treecmp.ILike, treecmp.NotILike: + negate, caseInsensitive := examineLikeOp(cmpProjOp) op, err = colexecprojconst.GetLikeProjectionOperator( allocator, evalCtx, input, leftIdx, resultIdx, - string(tree.MustBeDString(rConstArg)), negate, + string(tree.MustBeDString(rConstArg)), negate, caseInsensitive, ) case treecmp.In, treecmp.NotIn: negate := cmpProjOp.Symbol == treecmp.NotIn diff --git a/pkg/sql/colexec/colexeccmp/like_ops.go b/pkg/sql/colexec/colexeccmp/like_ops.go index db75dca09b36..e01eef27ba7d 100644 --- a/pkg/sql/colexec/colexeccmp/like_ops.go +++ b/pkg/sql/colexec/colexeccmp/like_ops.go @@ -43,13 +43,16 @@ const ( // // The second return parameter always contains a single []byte, unless // "skeleton" LikeOpType is returned. -func GetLikeOperatorType(pattern string) (LikeOpType, [][]byte, error) { +func GetLikeOperatorType(pattern string, caseInsensitive bool) (LikeOpType, [][]byte, error) { if pattern == "" { return LikeConstant, [][]byte{{}}, nil } if pattern == "%" { return LikeAlwaysMatch, [][]byte{{}}, nil } + if caseInsensitive { + pattern = strings.ToUpper(pattern) + } hasEscape := strings.Contains(pattern, `\`) if !hasEscape && len(pattern) > 1 && !strings.ContainsAny(pattern[1:len(pattern)-1], "_%") { // There are no wildcards in the middle of the string as well as no diff --git a/pkg/sql/colexec/colexecprojconst/like_ops.go b/pkg/sql/colexec/colexecprojconst/like_ops.go index b079160e6b1d..b7d6a7bd966a 100644 --- a/pkg/sql/colexec/colexecprojconst/like_ops.go +++ b/pkg/sql/colexec/colexecprojconst/like_ops.go @@ -31,8 +31,9 @@ func GetLikeProjectionOperator( resultIdx int, pattern string, negate bool, + caseInsensitive bool, ) (colexecop.Operator, error) { - likeOpType, patterns, err := colexeccmp.GetLikeOperatorType(pattern) + likeOpType, patterns, err := colexeccmp.GetLikeOperatorType(pattern, caseInsensitive) if err != nil { return nil, err } @@ -46,13 +47,21 @@ func GetLikeProjectionOperator( } switch likeOpType { case colexeccmp.LikeAlwaysMatch: - // Use an empty prefix operator to get correct NULL behavior. + // Use an empty prefix operator to get correct NULL behavior. We don't + // need to pay attention to the case sensitivity here since the pattern + // will always match anyway. return &projPrefixBytesBytesConstOp{ projConstOpBase: base, constArg: []byte{}, negate: negate, }, nil case colexeccmp.LikeConstant: + if caseInsensitive { + // We don't have an equivalent projection operator that would + // convert the argument to capital letters, so for now we fall back + // to the default comparison operator. + return nil, errors.New("ILIKE and NOT ILIKE aren't supported with a constant pattern") + } if negate { return &projNEBytesBytesConstOp{ projConstOpBase: base, @@ -68,15 +77,17 @@ func GetLikeProjectionOperator( projConstOpBase: base, constArg: pat, negate: negate, + caseInsensitive: caseInsensitive, }, nil case colexeccmp.LikePrefix: return &projPrefixBytesBytesConstOp{ projConstOpBase: base, constArg: pat, negate: negate, + caseInsensitive: caseInsensitive, }, nil case colexeccmp.LikeRegexp: - re, err := eval.ConvertLikeToRegexp(ctx, string(patterns[0]), false, '\\') + re, err := eval.ConvertLikeToRegexp(ctx, string(patterns[0]), caseInsensitive, '\\') if err != nil { return nil, err } @@ -90,12 +101,14 @@ func GetLikeProjectionOperator( projConstOpBase: base, constArg: patterns, negate: negate, + caseInsensitive: caseInsensitive, }, nil case colexeccmp.LikeSuffix: return &projSuffixBytesBytesConstOp{ projConstOpBase: base, constArg: pat, negate: negate, + caseInsensitive: caseInsensitive, }, nil default: return nil, errors.AssertionFailedf("unsupported like op type %d", likeOpType) diff --git a/pkg/sql/colexec/colexecprojconst/proj_const_ops_tmpl.go b/pkg/sql/colexec/colexecprojconst/proj_const_ops_tmpl.go index 3581f8ec835e..1cd473ecaa2a 100644 --- a/pkg/sql/colexec/colexecprojconst/proj_const_ops_tmpl.go +++ b/pkg/sql/colexec/colexecprojconst/proj_const_ops_tmpl.go @@ -96,6 +96,9 @@ type _OP_CONST_NAME struct { // {{if .Negatable}} negate bool // {{end}} + // {{if .CaseInsensitive}} + caseInsensitive bool + // {{end}} } func (p _OP_CONST_NAME) Next() coldata.Batch { @@ -114,6 +117,14 @@ func (p _OP_CONST_NAME) Next() coldata.Batch { // should be negated. // */}} _negate := p.negate + // {{ end }} + // {{if .CaseInsensitive}} + // {{/* + // In order to inline the templated code of the LIKE overloads, we need + // to have a `_caseInsensitive` local variable indicating whether the + // operator is case insensitive. + // */}} + _caseInsensitive := p.caseInsensitive // {{end}} batch := p.Input.Next() n := batch.Length() diff --git a/pkg/sql/colexec/colexecprojconst/proj_like_ops.eg.go b/pkg/sql/colexec/colexecprojconst/proj_like_ops.eg.go index f1a797db8bd2..71eccb7a7f53 100644 --- a/pkg/sql/colexec/colexecprojconst/proj_like_ops.eg.go +++ b/pkg/sql/colexec/colexecprojconst/proj_like_ops.eg.go @@ -19,12 +19,14 @@ import ( type projPrefixBytesBytesConstOp struct { projConstOpBase - constArg []byte - negate bool + constArg []byte + negate bool + caseInsensitive bool } func (p projPrefixBytesBytesConstOp) Next() coldata.Batch { _negate := p.negate + _caseInsensitive := p.caseInsensitive batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -50,6 +52,9 @@ func (p projPrefixBytesBytesConstOp) Next() coldata.Batch { if !colNulls.NullAt(i) { // We only want to perform the projection operation if the value is not null. arg := col.Get(i) + if _caseInsensitive { + arg = bytes.ToUpper(arg) + } projCol[i] = bytes.HasPrefix(arg, p.constArg) != _negate } } @@ -60,6 +65,9 @@ func (p projPrefixBytesBytesConstOp) Next() coldata.Batch { if !colNulls.NullAt(i) { // We only want to perform the projection operation if the value is not null. arg := col.Get(i) + if _caseInsensitive { + arg = bytes.ToUpper(arg) + } projCol[i] = bytes.HasPrefix(arg, p.constArg) != _negate } } @@ -70,6 +78,9 @@ func (p projPrefixBytesBytesConstOp) Next() coldata.Batch { sel = sel[:n] for _, i := range sel { arg := col.Get(i) + if _caseInsensitive { + arg = bytes.ToUpper(arg) + } projCol[i] = bytes.HasPrefix(arg, p.constArg) != _negate } } else { @@ -77,6 +88,9 @@ func (p projPrefixBytesBytesConstOp) Next() coldata.Batch { _ = col.Get(n - 1) for i := 0; i < n; i++ { arg := col.Get(i) + if _caseInsensitive { + arg = bytes.ToUpper(arg) + } projCol[i] = bytes.HasPrefix(arg, p.constArg) != _negate } } @@ -87,12 +101,14 @@ func (p projPrefixBytesBytesConstOp) Next() coldata.Batch { type projSuffixBytesBytesConstOp struct { projConstOpBase - constArg []byte - negate bool + constArg []byte + negate bool + caseInsensitive bool } func (p projSuffixBytesBytesConstOp) Next() coldata.Batch { _negate := p.negate + _caseInsensitive := p.caseInsensitive batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -118,6 +134,9 @@ func (p projSuffixBytesBytesConstOp) Next() coldata.Batch { if !colNulls.NullAt(i) { // We only want to perform the projection operation if the value is not null. arg := col.Get(i) + if _caseInsensitive { + arg = bytes.ToUpper(arg) + } projCol[i] = bytes.HasSuffix(arg, p.constArg) != _negate } } @@ -128,6 +147,9 @@ func (p projSuffixBytesBytesConstOp) Next() coldata.Batch { if !colNulls.NullAt(i) { // We only want to perform the projection operation if the value is not null. arg := col.Get(i) + if _caseInsensitive { + arg = bytes.ToUpper(arg) + } projCol[i] = bytes.HasSuffix(arg, p.constArg) != _negate } } @@ -138,6 +160,9 @@ func (p projSuffixBytesBytesConstOp) Next() coldata.Batch { sel = sel[:n] for _, i := range sel { arg := col.Get(i) + if _caseInsensitive { + arg = bytes.ToUpper(arg) + } projCol[i] = bytes.HasSuffix(arg, p.constArg) != _negate } } else { @@ -145,6 +170,9 @@ func (p projSuffixBytesBytesConstOp) Next() coldata.Batch { _ = col.Get(n - 1) for i := 0; i < n; i++ { arg := col.Get(i) + if _caseInsensitive { + arg = bytes.ToUpper(arg) + } projCol[i] = bytes.HasSuffix(arg, p.constArg) != _negate } } @@ -155,12 +183,14 @@ func (p projSuffixBytesBytesConstOp) Next() coldata.Batch { type projContainsBytesBytesConstOp struct { projConstOpBase - constArg []byte - negate bool + constArg []byte + negate bool + caseInsensitive bool } func (p projContainsBytesBytesConstOp) Next() coldata.Batch { _negate := p.negate + _caseInsensitive := p.caseInsensitive batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -186,6 +216,9 @@ func (p projContainsBytesBytesConstOp) Next() coldata.Batch { if !colNulls.NullAt(i) { // We only want to perform the projection operation if the value is not null. arg := col.Get(i) + if _caseInsensitive { + arg = bytes.ToUpper(arg) + } projCol[i] = bytes.Contains(arg, p.constArg) != _negate } } @@ -196,6 +229,9 @@ func (p projContainsBytesBytesConstOp) Next() coldata.Batch { if !colNulls.NullAt(i) { // We only want to perform the projection operation if the value is not null. arg := col.Get(i) + if _caseInsensitive { + arg = bytes.ToUpper(arg) + } projCol[i] = bytes.Contains(arg, p.constArg) != _negate } } @@ -206,6 +242,9 @@ func (p projContainsBytesBytesConstOp) Next() coldata.Batch { sel = sel[:n] for _, i := range sel { arg := col.Get(i) + if _caseInsensitive { + arg = bytes.ToUpper(arg) + } projCol[i] = bytes.Contains(arg, p.constArg) != _negate } } else { @@ -213,6 +252,9 @@ func (p projContainsBytesBytesConstOp) Next() coldata.Batch { _ = col.Get(n - 1) for i := 0; i < n; i++ { arg := col.Get(i) + if _caseInsensitive { + arg = bytes.ToUpper(arg) + } projCol[i] = bytes.Contains(arg, p.constArg) != _negate } } @@ -223,12 +265,14 @@ func (p projContainsBytesBytesConstOp) Next() coldata.Batch { type projSkeletonBytesBytesConstOp struct { projConstOpBase - constArg [][]byte - negate bool + constArg [][]byte + negate bool + caseInsensitive bool } func (p projSkeletonBytesBytesConstOp) Next() coldata.Batch { _negate := p.negate + _caseInsensitive := p.caseInsensitive batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -256,6 +300,9 @@ func (p projSkeletonBytesBytesConstOp) Next() coldata.Batch { arg := col.Get(i) { + if _caseInsensitive { + arg = bytes.ToUpper(arg) + } var idx, skeletonIdx int for skeletonIdx < len(p.constArg) { idx = bytes.Index(arg, p.constArg[skeletonIdx]) @@ -278,6 +325,9 @@ func (p projSkeletonBytesBytesConstOp) Next() coldata.Batch { arg := col.Get(i) { + if _caseInsensitive { + arg = bytes.ToUpper(arg) + } var idx, skeletonIdx int for skeletonIdx < len(p.constArg) { idx = bytes.Index(arg, p.constArg[skeletonIdx]) @@ -300,6 +350,9 @@ func (p projSkeletonBytesBytesConstOp) Next() coldata.Batch { arg := col.Get(i) { + if _caseInsensitive { + arg = bytes.ToUpper(arg) + } var idx, skeletonIdx int for skeletonIdx < len(p.constArg) { idx = bytes.Index(arg, p.constArg[skeletonIdx]) @@ -319,6 +372,9 @@ func (p projSkeletonBytesBytesConstOp) Next() coldata.Batch { arg := col.Get(i) { + if _caseInsensitive { + arg = bytes.ToUpper(arg) + } var idx, skeletonIdx int for skeletonIdx < len(p.constArg) { idx = bytes.Index(arg, p.constArg[skeletonIdx]) diff --git a/pkg/sql/colexec/colexecsel/like_ops.go b/pkg/sql/colexec/colexecsel/like_ops.go index 8d20b43bd9ab..4470625baa52 100644 --- a/pkg/sql/colexec/colexecsel/like_ops.go +++ b/pkg/sql/colexec/colexecsel/like_ops.go @@ -21,9 +21,14 @@ import ( // pattern, or NOT LIKE if the negate argument is true. The implementation // varies depending on the complexity of the pattern. func GetLikeOperator( - ctx *eval.Context, input colexecop.Operator, colIdx int, pattern string, negate bool, + ctx *eval.Context, + input colexecop.Operator, + colIdx int, + pattern string, + negate bool, + caseInsensitive bool, ) (colexecop.Operator, error) { - likeOpType, patterns, err := colexeccmp.GetLikeOperatorType(pattern) + likeOpType, patterns, err := colexeccmp.GetLikeOperatorType(pattern, caseInsensitive) if err != nil { return nil, err } @@ -34,13 +39,21 @@ func GetLikeOperator( } switch likeOpType { case colexeccmp.LikeAlwaysMatch: - // Use an empty prefix operator to get correct NULL behavior. + // Use an empty prefix operator to get correct NULL behavior. We don't + // need to pay attention to the case sensitivity here since the pattern + // will always match anyway. return &selPrefixBytesBytesConstOp{ selConstOpBase: base, constArg: []byte{}, negate: negate, }, nil case colexeccmp.LikeConstant: + if caseInsensitive { + // We don't have an equivalent projection operator that would + // convert the argument to capital letters, so for now we fall back + // to the default comparison operator. + return nil, errors.New("ILIKE and NOT ILIKE aren't supported with a constant pattern") + } if negate { return &selNEBytesBytesConstOp{ selConstOpBase: base, @@ -53,18 +66,20 @@ func GetLikeOperator( }, nil case colexeccmp.LikeContains: return &selContainsBytesBytesConstOp{ - selConstOpBase: base, - constArg: pat, - negate: negate, + selConstOpBase: base, + constArg: pat, + negate: negate, + caseInsensitive: caseInsensitive, }, nil case colexeccmp.LikePrefix: return &selPrefixBytesBytesConstOp{ - selConstOpBase: base, - constArg: pat, - negate: negate, + selConstOpBase: base, + constArg: pat, + negate: negate, + caseInsensitive: caseInsensitive, }, nil case colexeccmp.LikeRegexp: - re, err := eval.ConvertLikeToRegexp(ctx, string(patterns[0]), false, '\\') + re, err := eval.ConvertLikeToRegexp(ctx, string(patterns[0]), caseInsensitive, '\\') if err != nil { return nil, err } @@ -75,15 +90,17 @@ func GetLikeOperator( }, nil case colexeccmp.LikeSkeleton: return &selSkeletonBytesBytesConstOp{ - selConstOpBase: base, - constArg: patterns, - negate: negate, + selConstOpBase: base, + constArg: patterns, + negate: negate, + caseInsensitive: caseInsensitive, }, nil case colexeccmp.LikeSuffix: return &selSuffixBytesBytesConstOp{ - selConstOpBase: base, - constArg: pat, - negate: negate, + selConstOpBase: base, + constArg: pat, + negate: negate, + caseInsensitive: caseInsensitive, }, nil default: return nil, errors.AssertionFailedf("unsupported like op type %d", likeOpType) diff --git a/pkg/sql/colexec/colexecsel/like_ops_test.go b/pkg/sql/colexec/colexecsel/like_ops_test.go index b1a29166eae0..ed7d4defbd95 100644 --- a/pkg/sql/colexec/colexecsel/like_ops_test.go +++ b/pkg/sql/colexec/colexecsel/like_ops_test.go @@ -14,6 +14,7 @@ import ( "context" "fmt" "regexp" + "strings" "testing" "github.com/cockroachdb/cockroach/pkg/col/coldata" @@ -30,121 +31,152 @@ import ( func TestLikeOperators(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - for _, tc := range []struct { - pattern string - negate bool - tups colexectestutils.Tuples - expected colexectestutils.Tuples - }{ - { - pattern: "def", - tups: colexectestutils.Tuples{{"abc"}, {"def"}, {"ghi"}}, - expected: colexectestutils.Tuples{{"def"}}, - }, - { - pattern: "def", - negate: true, - tups: colexectestutils.Tuples{{"abc"}, {"def"}, {"ghi"}}, - expected: colexectestutils.Tuples{{"abc"}, {"ghi"}}, - }, - { - pattern: "de%", - tups: colexectestutils.Tuples{{"abc"}, {"def"}, {"ghi"}}, - expected: colexectestutils.Tuples{{"def"}}, - }, - { - pattern: "de%", - negate: true, - tups: colexectestutils.Tuples{{"abc"}, {"def"}, {"ghi"}}, - expected: colexectestutils.Tuples{{"abc"}, {"ghi"}}, - }, - { - pattern: "%ef", - tups: colexectestutils.Tuples{{"abc"}, {"def"}, {"ghi"}}, - expected: colexectestutils.Tuples{{"def"}}, - }, - { - pattern: "%ef", - negate: true, - tups: colexectestutils.Tuples{{"abc"}, {"def"}, {"ghi"}}, - expected: colexectestutils.Tuples{{"abc"}, {"ghi"}}, - }, - { - pattern: "_e_", - tups: colexectestutils.Tuples{{"abc"}, {"def"}, {"ghi"}}, - expected: colexectestutils.Tuples{{"def"}}, - }, - { - pattern: "_e_", - negate: true, - tups: colexectestutils.Tuples{{"abc"}, {"def"}, {"ghi"}}, - expected: colexectestutils.Tuples{{"abc"}, {"ghi"}}, - }, - { - pattern: "%e%", - tups: colexectestutils.Tuples{{"abc"}, {"def"}, {"ghi"}}, - expected: colexectestutils.Tuples{{"def"}}, - }, - { - pattern: "%e%", - negate: true, - tups: colexectestutils.Tuples{{"abc"}, {"def"}, {"ghi"}}, - expected: colexectestutils.Tuples{{"abc"}, {"ghi"}}, - }, - // These two cases are equivalent to the two previous ones, but the - // pattern is not normalized, so the slow regex matcher will be used. - { - pattern: "%%e%", - tups: colexectestutils.Tuples{{"abc"}, {"def"}, {"ghi"}}, - expected: colexectestutils.Tuples{{"def"}}, - }, - { - pattern: "%%e%", - negate: true, - tups: colexectestutils.Tuples{{"abc"}, {"def"}, {"ghi"}}, - expected: colexectestutils.Tuples{{"abc"}, {"ghi"}}, - }, - { - pattern: "%a%e%", - tups: colexectestutils.Tuples{{"abc"}, {"adef"}, {"gahie"}, {"beb"}, {"ae"}}, - expected: colexectestutils.Tuples{{"adef"}, {"gahie"}, {"ae"}}, - }, - { - pattern: "%a%e%", - negate: true, - tups: colexectestutils.Tuples{{"abc"}, {"adef"}, {"gahie"}, {"beb"}, {"ae"}}, - expected: colexectestutils.Tuples{{"abc"}, {"beb"}}, - }, - { - pattern: "%1%22%333%", - tups: colexectestutils.Tuples{ - {"a1bc22def333fghi"}, - {"abc22def333fghi"}, // 1 is missing. - {"a1bc2def333fghi"}, // 2 is missing. - {"a1bc22def33fghi"}, // 3 is missing. - {"122333"}, - }, - expected: colexectestutils.Tuples{{"a1bc22def333fghi"}, {"122333"}}, - }, - { - pattern: "%1%22%333%", - negate: true, - tups: colexectestutils.Tuples{ - {"a1bc22def333fghi"}, - {"abc22def333fghi"}, // 1 is missing. - {"a1bc2def333fghi"}, // 2 is missing. - {"a1bc22def33fghi"}, // 3 is missing. - {"122333"}, - }, - expected: colexectestutils.Tuples{{"abc22def333fghi"}, {"a1bc2def333fghi"}, {"a1bc22def33fghi"}}, - }, - } { - colexectestutils.RunTests( - t, testAllocator, []colexectestutils.Tuples{tc.tups}, tc.expected, colexectestutils.OrderedVerifier, - func(input []colexecop.Operator) (colexecop.Operator, error) { - ctx := eval.MakeTestingEvalContext(cluster.MakeTestingClusterSettings()) - return GetLikeOperator(&ctx, input[0], 0, tc.pattern, tc.negate) - }) + rng, _ := randutil.NewTestRand() + for _, caseInsensitive := range []bool{false, true} { + for _, tc := range []struct { + pattern string + isConstant bool + negate bool + tups colexectestutils.Tuples + expected colexectestutils.Tuples + }{ + { + pattern: "def", + isConstant: true, + tups: colexectestutils.Tuples{{"abc"}, {"def"}, {"ghi"}}, + expected: colexectestutils.Tuples{{"def"}}, + }, + { + pattern: "def", + isConstant: true, + negate: true, + tups: colexectestutils.Tuples{{"abc"}, {"def"}, {"ghi"}}, + expected: colexectestutils.Tuples{{"abc"}, {"ghi"}}, + }, + { + pattern: "de%", + tups: colexectestutils.Tuples{{"abc"}, {"def"}, {"ghi"}}, + expected: colexectestutils.Tuples{{"def"}}, + }, + { + pattern: "de%", + negate: true, + tups: colexectestutils.Tuples{{"abc"}, {"def"}, {"ghi"}}, + expected: colexectestutils.Tuples{{"abc"}, {"ghi"}}, + }, + { + pattern: "%ef", + tups: colexectestutils.Tuples{{"abc"}, {"def"}, {"ghi"}}, + expected: colexectestutils.Tuples{{"def"}}, + }, + { + pattern: "%ef", + negate: true, + tups: colexectestutils.Tuples{{"abc"}, {"def"}, {"ghi"}}, + expected: colexectestutils.Tuples{{"abc"}, {"ghi"}}, + }, + { + pattern: "_e_", + tups: colexectestutils.Tuples{{"abc"}, {"def"}, {"ghi"}}, + expected: colexectestutils.Tuples{{"def"}}, + }, + { + pattern: "_e_", + negate: true, + tups: colexectestutils.Tuples{{"abc"}, {"def"}, {"ghi"}}, + expected: colexectestutils.Tuples{{"abc"}, {"ghi"}}, + }, + { + pattern: "%e%", + tups: colexectestutils.Tuples{{"abc"}, {"def"}, {"ghi"}}, + expected: colexectestutils.Tuples{{"def"}}, + }, + { + pattern: "%e%", + negate: true, + tups: colexectestutils.Tuples{{"abc"}, {"def"}, {"ghi"}}, + expected: colexectestutils.Tuples{{"abc"}, {"ghi"}}, + }, + // These two cases are equivalent to the two previous ones, but the + // pattern is not normalized, so the slow regex matcher will be used. + { + pattern: "%%e%", + tups: colexectestutils.Tuples{{"abc"}, {"def"}, {"ghi"}}, + expected: colexectestutils.Tuples{{"def"}}, + }, + { + pattern: "%%e%", + negate: true, + tups: colexectestutils.Tuples{{"abc"}, {"def"}, {"ghi"}}, + expected: colexectestutils.Tuples{{"abc"}, {"ghi"}}, + }, + { + pattern: "%a%e%", + tups: colexectestutils.Tuples{{"abc"}, {"adef"}, {"gahie"}, {"beb"}, {"ae"}}, + expected: colexectestutils.Tuples{{"adef"}, {"gahie"}, {"ae"}}, + }, + { + pattern: "%a%e%", + negate: true, + tups: colexectestutils.Tuples{{"abc"}, {"adef"}, {"gahie"}, {"beb"}, {"ae"}}, + expected: colexectestutils.Tuples{{"abc"}, {"beb"}}, + }, + { + pattern: "%1%22%333%", + tups: colexectestutils.Tuples{ + {"a1bc22def333fghi"}, + {"abc22def333fghi"}, // 1 is missing. + {"a1bc2def333fghi"}, // 2 is missing. + {"a1bc22def33fghi"}, // 3 is missing. + {"122333"}, + }, + expected: colexectestutils.Tuples{{"a1bc22def333fghi"}, {"122333"}}, + }, + { + pattern: "%1%22%333%", + negate: true, + tups: colexectestutils.Tuples{ + {"a1bc22def333fghi"}, + {"abc22def333fghi"}, // 1 is missing. + {"a1bc2def333fghi"}, // 2 is missing. + {"a1bc22def33fghi"}, // 3 is missing. + {"122333"}, + }, + expected: colexectestutils.Tuples{{"abc22def333fghi"}, {"a1bc2def333fghi"}, {"a1bc22def33fghi"}}, + }, + } { + if caseInsensitive && tc.isConstant { + // This case is currently not supported. + continue + } + pattern, tups, expected := tc.pattern, tc.tups, tc.expected + if caseInsensitive { + // If we don't care about the case sensitivity, then randomly + // modify the strings and / or the pattern to add some capital + // letters. + if rng.Float64() < 0.5 { + pattern = strings.ToUpper(pattern) + } + var newExpected colexectestutils.Tuples + for i := range tups { + expectedIdx := len(newExpected) + matched := expectedIdx < len(expected) && tups[i][0].(string) == expected[expectedIdx][0].(string) + if rng.Float64() < 0.5 { + tups[i][0] = strings.ToUpper(tups[i][0].(string)) + } + if matched { + newExpected = append(newExpected, tups[i]) + } + } + expected = newExpected + } + colexectestutils.RunTests( + t, testAllocator, []colexectestutils.Tuples{tups}, expected, colexectestutils.OrderedVerifier, + func(input []colexecop.Operator) (colexecop.Operator, error) { + ctx := eval.MakeTestingEvalContext(cluster.MakeTestingClusterSettings()) + return GetLikeOperator(&ctx, input[0], 0, pattern, tc.negate, caseInsensitive) + }) + } } } diff --git a/pkg/sql/colexec/colexecsel/sel_like_ops.eg.go b/pkg/sql/colexec/colexecsel/sel_like_ops.eg.go index 01e4dc0484d4..365ade5957b7 100644 --- a/pkg/sql/colexec/colexecsel/sel_like_ops.eg.go +++ b/pkg/sql/colexec/colexecsel/sel_like_ops.eg.go @@ -19,12 +19,14 @@ import ( type selPrefixBytesBytesConstOp struct { selConstOpBase - constArg []byte - negate bool + constArg []byte + negate bool + caseInsensitive bool } func (p *selPrefixBytesBytesConstOp) Next() coldata.Batch { _negate := p.negate + _caseInsensitive := p.caseInsensitive for { batch := p.Input.Next() if batch.Length() == 0 { @@ -45,6 +47,9 @@ func (p *selPrefixBytesBytesConstOp) Next() coldata.Batch { } var cmp bool arg := col.Get(i) + if _caseInsensitive { + arg = bytes.ToUpper(arg) + } cmp = bytes.HasPrefix(arg, p.constArg) != _negate if cmp { sel[idx] = i @@ -61,6 +66,9 @@ func (p *selPrefixBytesBytesConstOp) Next() coldata.Batch { } var cmp bool arg := col.Get(i) + if _caseInsensitive { + arg = bytes.ToUpper(arg) + } cmp = bytes.HasPrefix(arg, p.constArg) != _negate if cmp { sel[idx] = i @@ -74,6 +82,9 @@ func (p *selPrefixBytesBytesConstOp) Next() coldata.Batch { for _, i := range sel { var cmp bool arg := col.Get(i) + if _caseInsensitive { + arg = bytes.ToUpper(arg) + } cmp = bytes.HasPrefix(arg, p.constArg) != _negate if cmp { sel[idx] = i @@ -87,6 +98,9 @@ func (p *selPrefixBytesBytesConstOp) Next() coldata.Batch { for i := 0; i < n; i++ { var cmp bool arg := col.Get(i) + if _caseInsensitive { + arg = bytes.ToUpper(arg) + } cmp = bytes.HasPrefix(arg, p.constArg) != _negate if cmp { sel[idx] = i @@ -104,12 +118,14 @@ func (p *selPrefixBytesBytesConstOp) Next() coldata.Batch { type selSuffixBytesBytesConstOp struct { selConstOpBase - constArg []byte - negate bool + constArg []byte + negate bool + caseInsensitive bool } func (p *selSuffixBytesBytesConstOp) Next() coldata.Batch { _negate := p.negate + _caseInsensitive := p.caseInsensitive for { batch := p.Input.Next() if batch.Length() == 0 { @@ -130,6 +146,9 @@ func (p *selSuffixBytesBytesConstOp) Next() coldata.Batch { } var cmp bool arg := col.Get(i) + if _caseInsensitive { + arg = bytes.ToUpper(arg) + } cmp = bytes.HasSuffix(arg, p.constArg) != _negate if cmp { sel[idx] = i @@ -146,6 +165,9 @@ func (p *selSuffixBytesBytesConstOp) Next() coldata.Batch { } var cmp bool arg := col.Get(i) + if _caseInsensitive { + arg = bytes.ToUpper(arg) + } cmp = bytes.HasSuffix(arg, p.constArg) != _negate if cmp { sel[idx] = i @@ -159,6 +181,9 @@ func (p *selSuffixBytesBytesConstOp) Next() coldata.Batch { for _, i := range sel { var cmp bool arg := col.Get(i) + if _caseInsensitive { + arg = bytes.ToUpper(arg) + } cmp = bytes.HasSuffix(arg, p.constArg) != _negate if cmp { sel[idx] = i @@ -172,6 +197,9 @@ func (p *selSuffixBytesBytesConstOp) Next() coldata.Batch { for i := 0; i < n; i++ { var cmp bool arg := col.Get(i) + if _caseInsensitive { + arg = bytes.ToUpper(arg) + } cmp = bytes.HasSuffix(arg, p.constArg) != _negate if cmp { sel[idx] = i @@ -189,12 +217,14 @@ func (p *selSuffixBytesBytesConstOp) Next() coldata.Batch { type selContainsBytesBytesConstOp struct { selConstOpBase - constArg []byte - negate bool + constArg []byte + negate bool + caseInsensitive bool } func (p *selContainsBytesBytesConstOp) Next() coldata.Batch { _negate := p.negate + _caseInsensitive := p.caseInsensitive for { batch := p.Input.Next() if batch.Length() == 0 { @@ -215,6 +245,9 @@ func (p *selContainsBytesBytesConstOp) Next() coldata.Batch { } var cmp bool arg := col.Get(i) + if _caseInsensitive { + arg = bytes.ToUpper(arg) + } cmp = bytes.Contains(arg, p.constArg) != _negate if cmp { sel[idx] = i @@ -231,6 +264,9 @@ func (p *selContainsBytesBytesConstOp) Next() coldata.Batch { } var cmp bool arg := col.Get(i) + if _caseInsensitive { + arg = bytes.ToUpper(arg) + } cmp = bytes.Contains(arg, p.constArg) != _negate if cmp { sel[idx] = i @@ -244,6 +280,9 @@ func (p *selContainsBytesBytesConstOp) Next() coldata.Batch { for _, i := range sel { var cmp bool arg := col.Get(i) + if _caseInsensitive { + arg = bytes.ToUpper(arg) + } cmp = bytes.Contains(arg, p.constArg) != _negate if cmp { sel[idx] = i @@ -257,6 +296,9 @@ func (p *selContainsBytesBytesConstOp) Next() coldata.Batch { for i := 0; i < n; i++ { var cmp bool arg := col.Get(i) + if _caseInsensitive { + arg = bytes.ToUpper(arg) + } cmp = bytes.Contains(arg, p.constArg) != _negate if cmp { sel[idx] = i @@ -274,12 +316,14 @@ func (p *selContainsBytesBytesConstOp) Next() coldata.Batch { type selSkeletonBytesBytesConstOp struct { selConstOpBase - constArg [][]byte - negate bool + constArg [][]byte + negate bool + caseInsensitive bool } func (p *selSkeletonBytesBytesConstOp) Next() coldata.Batch { _negate := p.negate + _caseInsensitive := p.caseInsensitive for { batch := p.Input.Next() if batch.Length() == 0 { @@ -302,6 +346,9 @@ func (p *selSkeletonBytesBytesConstOp) Next() coldata.Batch { arg := col.Get(i) { + if _caseInsensitive { + arg = bytes.ToUpper(arg) + } var idx, skeletonIdx int for skeletonIdx < len(p.constArg) { idx = bytes.Index(arg, p.constArg[skeletonIdx]) @@ -330,6 +377,9 @@ func (p *selSkeletonBytesBytesConstOp) Next() coldata.Batch { arg := col.Get(i) { + if _caseInsensitive { + arg = bytes.ToUpper(arg) + } var idx, skeletonIdx int for skeletonIdx < len(p.constArg) { idx = bytes.Index(arg, p.constArg[skeletonIdx]) @@ -355,6 +405,9 @@ func (p *selSkeletonBytesBytesConstOp) Next() coldata.Batch { arg := col.Get(i) { + if _caseInsensitive { + arg = bytes.ToUpper(arg) + } var idx, skeletonIdx int for skeletonIdx < len(p.constArg) { idx = bytes.Index(arg, p.constArg[skeletonIdx]) @@ -380,6 +433,9 @@ func (p *selSkeletonBytesBytesConstOp) Next() coldata.Batch { arg := col.Get(i) { + if _caseInsensitive { + arg = bytes.ToUpper(arg) + } var idx, skeletonIdx int for skeletonIdx < len(p.constArg) { idx = bytes.Index(arg, p.constArg[skeletonIdx]) diff --git a/pkg/sql/colexec/colexecsel/selection_ops_tmpl.go b/pkg/sql/colexec/colexecsel/selection_ops_tmpl.go index 21cdbb23ae0d..a1ffe17a9882 100644 --- a/pkg/sql/colexec/colexecsel/selection_ops_tmpl.go +++ b/pkg/sql/colexec/colexecsel/selection_ops_tmpl.go @@ -194,6 +194,9 @@ type _OP_CONST_NAME struct { // {{if .Negatable}} negate bool // {{end}} + // {{if .CaseInsensitive}} + caseInsensitive bool + // {{end}} } func (p *_OP_CONST_NAME) Next() coldata.Batch { @@ -205,6 +208,14 @@ func (p *_OP_CONST_NAME) Next() coldata.Batch { // */}} _negate := p.negate // {{end}} + // {{if .CaseInsensitive}} + // {{/* + // In order to inline the templated code of the LIKE overloads, we need + // to have a `_caseInsensitive` local variable indicating whether the + // operator is case insensitive. + // */}} + _caseInsensitive := p.caseInsensitive + // {{end}} for { batch := p.Input.Next() if batch.Length() == 0 { diff --git a/pkg/sql/colexec/execgen/cmd/execgen/like_ops_gen.go b/pkg/sql/colexec/execgen/cmd/execgen/like_ops_gen.go index c049508c512c..74a4fabc7c67 100644 --- a/pkg/sql/colexec/execgen/cmd/execgen/like_ops_gen.go +++ b/pkg/sql/colexec/execgen/cmd/execgen/like_ops_gen.go @@ -73,7 +73,10 @@ func genLikeOps( return err } bytesRepresentation := toPhysicalRepresentation(types.BytesFamily, anyWidth) - makeOverload := func(name string, rightGoType string, assignFunc func(targetElem, leftElem, rightElem string) string) *twoArgsResolvedOverload { + makeOverload := func( + name string, rightGoType string, caseInsensitive bool, + assignFunc func(targetElem, leftElem, rightElem string) string, + ) *twoArgsResolvedOverload { base := &overloadBase{ Name: name, } @@ -112,16 +115,34 @@ func genLikeOps( } rightTypeOverload.WidthOverloads[0] = rightWidthOverload return &twoArgsResolvedOverload{ - overloadBase: base, - Left: leftWidthOverload, - Right: rightWidthOverload, - Negatable: true, + overloadBase: base, + Left: leftWidthOverload, + Right: rightWidthOverload, + Negatable: true, + CaseInsensitive: caseInsensitive, } } + caseInsensitivePrelude := func(leftElem string) string { + return fmt.Sprintf(`if _caseInsensitive { + %[1]s = bytes.ToUpper(%[1]s) + }`, leftElem) + } + // makeSimpleOverload returns the overload for "simple" patterns which + // call a single method from 'bytes' package for comparison. + makeSimpleOverload := func(name string, bytesFunc string) *twoArgsResolvedOverload { + return makeOverload(name, bytesRepresentation, true, /* caseInsensitive */ + func(targetElem, leftElem, rightElem string) string { + return fmt.Sprintf( + `%[4]s + %[1]s = bytes.%[5]s(%[2]s, %[3]s) != _negate`, + targetElem, leftElem, rightElem, caseInsensitivePrelude(leftElem), bytesFunc) + }) + } // makeSkeletonAssignFunc returns a string that assigns 'targetElem' to - // the result of evaluation 'leftElem' (LIKE | NOT LIKE) pattern where - // pattern is of the form '%word1%word2%...%' where "words" come from - // 'rightElem' (which is [][]byte). + // the result of evaluation + // 'leftElem' (LIKE | NOT LIKE | ILIKE | NOT ILIKE) pattern + // where pattern is of the form '%word1%word2%...%' where "words" come + // from 'rightElem' (which is [][]byte). // // The logic for evaluating such expression is that for each word we // find its first occurrence in the unprocessed part of 'leftElem'. If @@ -130,6 +151,7 @@ func genLikeOps( makeSkeletonAssignFunc := func(targetElem, leftElem, rightElem string) string { return fmt.Sprintf(` { + %[4]s var idx, skeletonIdx int for skeletonIdx < len(%[3]s) { idx = bytes.Index(%[2]s, %[3]s[skeletonIdx]) @@ -140,24 +162,24 @@ func genLikeOps( skeletonIdx++ } %[1]s = skeletonIdx == len(%[3]s) != _negate - }`, targetElem, leftElem, rightElem) + }`, targetElem, leftElem, rightElem, caseInsensitivePrelude(leftElem)) } overloads := []*twoArgsResolvedOverload{ - makeOverload("Prefix", bytesRepresentation, func(targetElem, leftElem, rightElem string) string { - return fmt.Sprintf("%s = bytes.HasPrefix(%s, %s) != _negate", targetElem, leftElem, rightElem) - }), - makeOverload("Suffix", bytesRepresentation, func(targetElem, leftElem, rightElem string) string { - return fmt.Sprintf("%s = bytes.HasSuffix(%s, %s) != _negate", targetElem, leftElem, rightElem) - }), - makeOverload("Contains", bytesRepresentation, func(targetElem, leftElem, rightElem string) string { - return fmt.Sprintf("%s = bytes.Contains(%s, %s) != _negate", targetElem, leftElem, rightElem) - }), - makeOverload("Skeleton", "[][]byte", func(targetElem, leftElem, rightElem string) string { - return makeSkeletonAssignFunc(targetElem, leftElem, rightElem) - }), - makeOverload("Regexp", "*regexp.Regexp", func(targetElem, leftElem, rightElem string) string { - return fmt.Sprintf("%s = %s.Match(%s) != _negate", targetElem, rightElem, leftElem) - }), + makeSimpleOverload("Prefix", "HasPrefix"), + makeSimpleOverload("Suffix", "HasSuffix"), + makeSimpleOverload("Contains", "Contains"), + makeOverload("Skeleton", "[][]byte", true, /* caseInsensitive */ + func(targetElem, leftElem, rightElem string) string { + return makeSkeletonAssignFunc( + targetElem, leftElem, rightElem, + ) + }), + // Note that the Regexp overload handles the case sensitivity + // itself. + makeOverload("Regexp", "*regexp.Regexp", false, /* caseInsensitive */ + func(targetElem, leftElem, rightElem string) string { + return fmt.Sprintf("%s = %s.Match(%s) != _negate", targetElem, rightElem, leftElem) + }), } return tmpl.Execute(wr, overloads) } diff --git a/pkg/sql/colexec/execgen/cmd/execgen/overloads_base.go b/pkg/sql/colexec/execgen/cmd/execgen/overloads_base.go index 303abee77cd3..87a16dccadc2 100644 --- a/pkg/sql/colexec/execgen/cmd/execgen/overloads_base.go +++ b/pkg/sql/colexec/execgen/cmd/execgen/overloads_base.go @@ -313,10 +313,11 @@ type twoArgsResolvedOverload struct { Left *argWidthOverload Right *lastArgWidthOverload - // Negatable is only used by the LIKE overloads. We cannot easily extract - // out a separate struct for those since we're reusing the same templates as - // all of the selection / projection operators. - Negatable bool + // Negatable and CaseInsensitive are only used by the LIKE overloads. We + // cannot easily extract out a separate struct for those since we're reusing + // the same templates as all of the selection / projection operators. + Negatable bool + CaseInsensitive bool } // NeedsBinaryOverloadHelper returns true iff the overload is such that it needs From b8117398c7cd48c51e53b67936217d8f6a050a3a Mon Sep 17 00:00:00 2001 From: Michael Butler Date: Tue, 2 Aug 2022 19:33:35 -0400 Subject: [PATCH 4/7] backupccl: add RESTORE with schema_only Fixes #83470 Release note (sql change): This pr adds the schema_only flag to RESTORE, allowing a user to run a normal RESTORE, without restoring any user table data. This can be used to quickly validate that a given backup is restorable. A schema_only restore runtime is O(# of descriptors) which is a fraction of a regular restore's runtime O(# of table rows). Note that during a cluster level, schema_only restore, the system tables are read from S3 and written to disk, as this provides important validation coverage without much runtime cost (system tables should not be large). After running a successful schema_only RESTORE, the user can revert the cluster to its pre-restore state by simply dropping the descriptors the schema_only restore added (e.g. if the user restored a database, they can drop the database after the restore completes). Note that in the cluster level case, the restored system data cannot be reverted, this shouldn't matter, as the cluster was empty before hand. For the Backup validation use case, RESTORE with schema_only provides near total validation coverage. In other words, if a user's schema_only RESTORE works, they can be quite confident that a real RESTORE will work. There's one notable place schema_only RESTORE lacks coverage: It doesn't read (or write) from any of the SSTs that store backed up user table data. To ensure a Backup's SSTs are where the RESTORE cmd would expect them to be, a user should run SHOW BACKUP ... with check_files. Further, in an upcoming patch, another flag for RESTORE validation will be introduced -- the verify_backup_table_data flag -- which extends schema_only functionality to read the table data from S3 and conduct checksums on it. Like with the schema_only flag, no table data will be ingested into the cluster. --- docs/generated/sql/bnf/stmt_block.bnf | 2 + pkg/ccl/backupccl/backup_telemetry.go | 4 + pkg/ccl/backupccl/backup_test.go | 168 ++++++++------- .../backupccl/backuprand/backup_rand_test.go | 36 +++- pkg/ccl/backupccl/restore_data_processor.go | 5 - pkg/ccl/backupccl/restore_job.go | 29 ++- .../restore_mid_schema_change_test.go | 109 ++++++---- .../backupccl/restore_old_sequences_test.go | 37 +++- pkg/ccl/backupccl/restore_planning.go | 9 +- .../backupccl/restore_processor_planning.go | 2 - .../backupccl/split_and_scatter_processor.go | 5 - .../split_and_scatter_processor_test.go | 2 - .../backup-restore/restore-schema-only | 144 +++++++++++++ .../restore-schema-only-multiregion | 193 ++++++++++++++++++ pkg/jobs/jobspb/jobs.proto | 11 +- pkg/sql/execinfrapb/processors_bulk_io.proto | 10 +- pkg/sql/parser/sql.y | 9 +- pkg/sql/parser/testdata/backup_restore | 8 + pkg/sql/schemachanger/sctest/cumulative.go | 48 +++-- pkg/sql/sem/tree/backup.go | 15 +- 20 files changed, 646 insertions(+), 200 deletions(-) create mode 100644 pkg/ccl/backupccl/testdata/backup-restore/restore-schema-only create mode 100644 pkg/ccl/backupccl/testdata/backup-restore/restore-schema-only-multiregion diff --git a/docs/generated/sql/bnf/stmt_block.bnf b/docs/generated/sql/bnf/stmt_block.bnf index c87b9ae53b63..55b4f9082be7 100644 --- a/docs/generated/sql/bnf/stmt_block.bnf +++ b/docs/generated/sql/bnf/stmt_block.bnf @@ -1266,6 +1266,7 @@ unreserved_keyword ::= | 'RUNNING' | 'SCHEDULE' | 'SCHEDULES' + | 'SCHEMA_ONLY' | 'SCROLL' | 'SETTING' | 'SETTINGS' @@ -2492,6 +2493,7 @@ restore_options ::= | 'NEW_DB_NAME' '=' string_or_placeholder | 'INCREMENTAL_LOCATION' '=' string_or_placeholder_opt_list | 'TENANT' '=' string_or_placeholder + | 'SCHEMA_ONLY' scrub_option_list ::= ( scrub_option ) ( ( ',' scrub_option ) )* diff --git a/pkg/ccl/backupccl/backup_telemetry.go b/pkg/ccl/backupccl/backup_telemetry.go index 2e80fde6c729..9134f54eb4ec 100644 --- a/pkg/ccl/backupccl/backup_telemetry.go +++ b/pkg/ccl/backupccl/backup_telemetry.go @@ -65,6 +65,7 @@ const ( telemetryOptionSkipMissingSequenceOwners = "skip_missing_sequence_owners" telemetryOptionSkipMissingViews = "skip_missing_views" telemetryOptionSkipLocalitiesCheck = "skip_localities_check" + telemetryOptionSchemaOnly = "schema_only" ) // logBackupTelemetry publishes an eventpb.RecoveryEvent about a manually @@ -397,6 +398,9 @@ func logRestoreTelemetry( if opts.Detached { options = append(options, telemetryOptionDetached) } + if opts.SchemaOnly { + options = append(options, telemetryOptionSchemaOnly) + } sort.Strings(options) event := &eventpb.RecoveryEvent{ diff --git a/pkg/ccl/backupccl/backup_test.go b/pkg/ccl/backupccl/backup_test.go index ce401347ecb5..9685c2a15c44 100644 --- a/pkg/ccl/backupccl/backup_test.go +++ b/pkg/ccl/backupccl/backup_test.go @@ -2674,99 +2674,109 @@ func TestBackupRestoreDuringUserDefinedTypeChange(t *testing.T) { } for _, tc := range testCases { - t.Run(tc.name, func(t *testing.T) { - // Protects numTypeChangesStarted and numTypeChangesFinished. - var mu syncutil.Mutex - numTypeChangesStarted := 0 - numTypeChangesFinished := 0 - typeChangesStarted := make(chan struct{}) - waitForBackup := make(chan struct{}) - typeChangesFinished := make(chan struct{}) - _, sqlDB, _, cleanupFn := backupRestoreTestSetupWithParams(t, singleNode, 0, InitManualReplication, base.TestClusterArgs{ - ServerArgs: base.TestServerArgs{ - Knobs: base.TestingKnobs{ - SQLTypeSchemaChanger: &sql.TypeSchemaChangerTestingKnobs{ - RunBeforeEnumMemberPromotion: func(context.Context) error { - mu.Lock() - if numTypeChangesStarted < len(tc.queries) { - numTypeChangesStarted++ - if numTypeChangesStarted == len(tc.queries) { - close(typeChangesStarted) + for _, isSchemaOnly := range []bool{true, false} { + suffix := "" + if isSchemaOnly { + suffix = "-schema-only" + } + t.Run(tc.name+suffix, func(t *testing.T) { + // Protects numTypeChangesStarted and numTypeChangesFinished. + var mu syncutil.Mutex + numTypeChangesStarted := 0 + numTypeChangesFinished := 0 + typeChangesStarted := make(chan struct{}) + waitForBackup := make(chan struct{}) + typeChangesFinished := make(chan struct{}) + _, sqlDB, _, cleanupFn := backupRestoreTestSetupWithParams(t, singleNode, 0, InitManualReplication, base.TestClusterArgs{ + ServerArgs: base.TestServerArgs{ + Knobs: base.TestingKnobs{ + SQLTypeSchemaChanger: &sql.TypeSchemaChangerTestingKnobs{ + RunBeforeEnumMemberPromotion: func(context.Context) error { + mu.Lock() + if numTypeChangesStarted < len(tc.queries) { + numTypeChangesStarted++ + if numTypeChangesStarted == len(tc.queries) { + close(typeChangesStarted) + } + mu.Unlock() + <-waitForBackup + } else { + mu.Unlock() } - mu.Unlock() - <-waitForBackup - } else { - mu.Unlock() - } - return nil + return nil + }, }, }, }, - }, - }) - defer cleanupFn() + }) + defer cleanupFn() - // Create a database with a type. - sqlDB.Exec(t, ` + // Create a database with a type. + sqlDB.Exec(t, ` CREATE DATABASE d; CREATE TYPE d.greeting AS ENUM ('hello', 'howdy', 'hi'); `) - // Start ALTER TYPE statement(s) that will block. - for _, query := range tc.queries { - go func(query string, totalQueries int) { - // Note we don't use sqlDB.Exec here because we can't Fatal from within a goroutine. - if _, err := sqlDB.DB.ExecContext(context.Background(), query); err != nil { - t.Error(err) - } - mu.Lock() - numTypeChangesFinished++ - if numTypeChangesFinished == totalQueries { - close(typeChangesFinished) - } - mu.Unlock() - }(query, len(tc.queries)) - } - - // Wait on the type changes to start. - <-typeChangesStarted + // Start ALTER TYPE statement(s) that will block. + for _, query := range tc.queries { + go func(query string, totalQueries int) { + // Note we don't use sqlDB.Exec here because we can't Fatal from within a goroutine. + if _, err := sqlDB.DB.ExecContext(context.Background(), query); err != nil { + t.Error(err) + } + mu.Lock() + numTypeChangesFinished++ + if numTypeChangesFinished == totalQueries { + close(typeChangesFinished) + } + mu.Unlock() + }(query, len(tc.queries)) + } - // Now create a backup while the type change job is blocked so that - // greeting is backed up with some enum members in READ_ONLY state. - sqlDB.Exec(t, `BACKUP DATABASE d TO 'nodelocal://0/test/'`) + // Wait on the type changes to start. + <-typeChangesStarted - // Let the type change finish. - close(waitForBackup) - <-typeChangesFinished + // Now create a backup while the type change job is blocked so that + // greeting is backed up with some enum members in READ_ONLY state. + sqlDB.Exec(t, `BACKUP DATABASE d TO 'nodelocal://0/test/'`) - // Now drop the database and restore. - sqlDB.Exec(t, `DROP DATABASE d`) - sqlDB.Exec(t, `RESTORE DATABASE d FROM 'nodelocal://0/test/'`) + // Let the type change finish. + close(waitForBackup) + <-typeChangesFinished - // The type change job should be scheduled and finish. Note that we can't use - // sqlDB.CheckQueryResultsRetry as it Fatal's upon an error. The case below - // will error until the job completes. - for i, query := range tc.succeedAfter { - testutils.SucceedsSoon(t, func() error { - _, err := sqlDB.DB.ExecContext(context.Background(), query) - return err - }) - sqlDB.CheckQueryResults(t, query, [][]string{{tc.expectedSuccess[i]}}) - } + // Now drop the database and restore. + sqlDB.Exec(t, `DROP DATABASE d`) + restoreQuery := `RESTORE DATABASE d FROM 'nodelocal://0/test/'` + if isSchemaOnly { + restoreQuery = restoreQuery + " with schema_only" + } + sqlDB.Exec(t, restoreQuery) + + // The type change job should be scheduled and finish. Note that we can't use + // sqlDB.CheckQueryResultsRetry as it Fatal's upon an error. The case below + // will error until the job completes. + for i, query := range tc.succeedAfter { + testutils.SucceedsSoon(t, func() error { + _, err := sqlDB.DB.ExecContext(context.Background(), query) + return err + }) + sqlDB.CheckQueryResults(t, query, [][]string{{tc.expectedSuccess[i]}}) + } - for i, query := range tc.errorAfter { - testutils.SucceedsSoon(t, func() error { - _, err := sqlDB.DB.ExecContext(context.Background(), query) - if err == nil { - return errors.New("expected error, found none") - } - if !testutils.IsError(err, tc.expectedError[i]) { - return errors.Newf("expected error %q, found %v", tc.expectedError[i], pgerror.FullError(err)) - } - return nil - }) - } - }) + for i, query := range tc.errorAfter { + testutils.SucceedsSoon(t, func() error { + _, err := sqlDB.DB.ExecContext(context.Background(), query) + if err == nil { + return errors.New("expected error, found none") + } + if !testutils.IsError(err, tc.expectedError[i]) { + return errors.Newf("expected error %q, found %v", tc.expectedError[i], pgerror.FullError(err)) + } + return nil + }) + } + }) + } } } diff --git a/pkg/ccl/backupccl/backuprand/backup_rand_test.go b/pkg/ccl/backupccl/backuprand/backup_rand_test.go index 08b5fce1fbdf..7af84d8a56f8 100644 --- a/pkg/ccl/backupccl/backuprand/backup_rand_test.go +++ b/pkg/ccl/backupccl/backuprand/backup_rand_test.go @@ -32,7 +32,8 @@ import ( // TestBackupRestoreRandomDataRoundtrips conducts backup/restore roundtrips on // randomly generated tables and verifies their data and schema are preserved. // It tests that full database backup as well as all subsets of per-table backup -// roundtrip properly. +// roundtrip properly. 50% of the time, the test runs the restore with the +// schema_only parameter, which does not restore any rows from user tables. func TestBackupRestoreRandomDataRoundtrips(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) @@ -66,6 +67,11 @@ func TestBackupRestoreRandomDataRoundtrips(t *testing.T) { } numInserts := 20 + runSchemaOnlyExtension := "" + if rng.Intn(10)%2 == 0 { + runSchemaOnlyExtension = ", schema_only" + } + tables := sqlDB.Query(t, `SELECT name FROM crdb_internal.tables WHERE database_name = 'rand' AND schema_name = 'public'`) var tableNames []string @@ -87,7 +93,9 @@ database_name = 'rand' AND schema_name = 'public'`) expectedData := make(map[string][][]string) for _, tableName := range tableNames { expectedCreateTableStmt[tableName] = sqlDB.QueryStr(t, fmt.Sprintf(`SELECT create_statement FROM [SHOW CREATE TABLE %s]`, tableName))[0][0] - expectedData[tableName] = sqlDB.QueryStr(t, fmt.Sprintf(`SELECT * FROM %s`, tableName)) + if runSchemaOnlyExtension == "" { + expectedData[tableName] = sqlDB.QueryStr(t, fmt.Sprintf(`SELECT * FROM %s`, tableName)) + } } // Now that we've created our random tables, backup and restore the whole DB @@ -97,12 +105,12 @@ database_name = 'rand' AND schema_name = 'public'`) tablesBackup := localFoo + "alltables" dbBackups := []string{dbBackup, tablesBackup} if err := backuputils.VerifyBackupRestoreStatementResult( - t, sqlDB, "BACKUP DATABASE rand TO $1", dbBackup, + t, sqlDB, "BACKUP DATABASE rand INTO $1", dbBackup, ); err != nil { t.Fatal(err) } if err := backuputils.VerifyBackupRestoreStatementResult( - t, sqlDB, "BACKUP TABLE rand.* TO $1", tablesBackup, + t, sqlDB, "BACKUP TABLE rand.* INTO $1", tablesBackup, ); err != nil { t.Fatal(err) } @@ -118,7 +126,12 @@ database_name = 'rand' AND schema_name = 'public'`) fmt.Sprintf(`SELECT create_statement FROM [SHOW CREATE TABLE %s]`, restoreTable))[0][0] assert.Equal(t, expectedCreateTableStmt[tableName], createStmt, "SHOW CREATE %s not equal after RESTORE", tableName) - sqlDB.CheckQueryResults(t, fmt.Sprintf(`SELECT * FROM %s`, tableName), expectedData[tableName]) + if runSchemaOnlyExtension == "" { + sqlDB.CheckQueryResults(t, fmt.Sprintf(`SELECT * FROM %s`, restoreTable), expectedData[tableName]) + } else { + sqlDB.CheckQueryResults(t, fmt.Sprintf(`SELECT count(*) FROM %s`, restoreTable), + [][]string{{"0"}}) + } } } @@ -128,17 +141,17 @@ database_name = 'rand' AND schema_name = 'public'`) for _, backup := range dbBackups { sqlDB.Exec(t, "DROP DATABASE IF EXISTS restoredb") sqlDB.Exec(t, "CREATE DATABASE restoredb") + tableQuery := fmt.Sprintf("RESTORE rand.* FROM LATEST IN $1 WITH OPTIONS (into_db='restoredb'%s)", runSchemaOnlyExtension) if err := backuputils.VerifyBackupRestoreStatementResult( - t, sqlDB, "RESTORE rand.* FROM $1 WITH OPTIONS (into_db='restoredb')", backup, + t, sqlDB, tableQuery, backup, ); err != nil { t.Fatal(err) } verifyTables(t, tableNames) sqlDB.Exec(t, "DROP DATABASE IF EXISTS restoredb") - if err := backuputils.VerifyBackupRestoreStatementResult( - t, sqlDB, "RESTORE DATABASE rand FROM $1 WITH OPTIONS (new_db_name='restoredb')", backup, - ); err != nil { + dbQuery := fmt.Sprintf("RESTORE DATABASE rand FROM LATEST IN $1 WITH OPTIONS (new_db_name='restoredb'%s)", runSchemaOnlyExtension) + if err := backuputils.VerifyBackupRestoreStatementResult(t, sqlDB, dbQuery, backup); err != nil { t.Fatal(err) } verifyTables(t, tableNames) @@ -155,8 +168,9 @@ database_name = 'rand' AND schema_name = 'public'`) } tables := strings.Join(combo, ", ") t.Logf("Testing subset backup/restore %s", tables) - sqlDB.Exec(t, fmt.Sprintf(`BACKUP TABLE %s TO $1`, tables), backupTarget) - _, err := tc.Conns[0].Exec(fmt.Sprintf("RESTORE TABLE %s FROM $1 WITH OPTIONS (into_db='restoredb')", tables), + sqlDB.Exec(t, fmt.Sprintf(`BACKUP TABLE %s INTO $1`, tables), backupTarget) + _, err := tc.Conns[0].Exec( + fmt.Sprintf("RESTORE TABLE %s FROM LATEST IN $1 WITH OPTIONS (into_db='restoredb' %s)", tables, runSchemaOnlyExtension), backupTarget) if err != nil { if strings.Contains(err.Error(), "skip_missing_foreign_keys") { diff --git a/pkg/ccl/backupccl/restore_data_processor.go b/pkg/ccl/backupccl/restore_data_processor.go index c7d902e86b5c..0cc824448778 100644 --- a/pkg/ccl/backupccl/restore_data_processor.go +++ b/pkg/ccl/backupccl/restore_data_processor.go @@ -17,7 +17,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backuppb" "github.com/cockroachdb/cockroach/pkg/ccl/storageccl" "github.com/cockroachdb/cockroach/pkg/cloud" - "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv/bulk" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -138,10 +137,6 @@ func newRestoreDataProcessor( ) (execinfra.Processor, error) { sv := &flowCtx.Cfg.Settings.SV - if spec.Validation != jobspb.RestoreValidation_DefaultRestore { - return nil, errors.New("Restore Data Processor does not support validation yet") - } - rd := &restoreDataProcessor{ flowCtx: flowCtx, input: input, diff --git a/pkg/ccl/backupccl/restore_job.go b/pkg/ccl/backupccl/restore_job.go index ecf9607b99f3..8bca248b09c3 100644 --- a/pkg/ccl/backupccl/restore_job.go +++ b/pkg/ccl/backupccl/restore_job.go @@ -134,6 +134,7 @@ func restoreWithRetry( encryption *jobspb.BackupEncryptionOptions, kmsEnv cloud.KMSEnv, ) (roachpb.RowCount, error) { + // We retry on pretty generic failures -- any rpc error. If a worker node were // to restart, it would produce this kind of error, but there may be other // errors that are also rpc errors. Don't retry to aggressively. @@ -579,13 +580,26 @@ func spansForAllRestoreTableIndexes( codec keys.SQLCodec, tables []catalog.TableDescriptor, revs []backuppb.BackupManifest_DescriptorRevision, + schemaOnly bool, ) []roachpb.Span { + skipTableData := func(table catalog.TableDescriptor) bool { + // The only table data restored during a schemaOnly restore are from system tables, + // which only get covered during a cluster restore. + if table.GetParentID() != keys.SystemDatabaseID && schemaOnly { + return true + } + // We only import spans for physical tables. + if !table.IsPhysicalTable() { + return true + } + return false + } + added := make(map[tableAndIndex]bool, len(tables)) sstIntervalTree := interval.NewTree(interval.ExclusiveOverlapper) for _, table := range tables { - // We only import spans for physical tables. - if !table.IsPhysicalTable() { + if skipTableData(table) { continue } for _, index := range table.ActiveIndexes() { @@ -608,8 +622,7 @@ func spansForAllRestoreTableIndexes( rawTbl, _, _, _, _ := descpb.FromDescriptor(rev.Desc) if rawTbl != nil && !rawTbl.Dropped() { tbl := tabledesc.NewBuilder(rawTbl).BuildImmutableTable() - // We only import spans for physical tables. - if !tbl.IsPhysicalTable() { + if skipTableData(tbl) { continue } for _, idx := range tbl.ActiveIndexes() { @@ -714,8 +727,8 @@ func createImportingDescriptors( // We get the spans of the restoring tables _as they appear in the backup_, // that is, in the 'old' keyspace, before we reassign the table IDs. - preRestoreSpans := spansForAllRestoreTableIndexes(backupCodec, preRestoreTables, nil) - postRestoreSpans := spansForAllRestoreTableIndexes(backupCodec, postRestoreTables, nil) + preRestoreSpans := spansForAllRestoreTableIndexes(backupCodec, preRestoreTables, nil, details.SchemaOnly) + postRestoreSpans := spansForAllRestoreTableIndexes(backupCodec, postRestoreTables, nil, details.SchemaOnly) log.Eventf(ctx, "starting restore for %d tables", len(mutableTables)) @@ -1232,10 +1245,6 @@ func (r *restoreResumer) doResume(ctx context.Context, execCtx interface{}) erro p := execCtx.(sql.JobExecContext) r.execCfg = p.ExecCfg() - if details.Validation != jobspb.RestoreValidation_DefaultRestore { - return errors.Errorf("No restore validation tools are supported") - } - mem := p.ExecCfg().RootMemoryMonitor.MakeBoundAccount() defer mem.Close(ctx) diff --git a/pkg/ccl/backupccl/restore_mid_schema_change_test.go b/pkg/ccl/backupccl/restore_mid_schema_change_test.go index 84d0edd6e4c5..63463abfe0e0 100644 --- a/pkg/ccl/backupccl/restore_mid_schema_change_test.go +++ b/pkg/ccl/backupccl/restore_mid_schema_change_test.go @@ -62,46 +62,53 @@ func TestRestoreMidSchemaChange(t *testing.T) { testdataBase = testutils.TestDataPath(t, "restore_mid_schema_change") exportDirs = testdataBase + "/exports" ) - for _, isClusterRestore := range []bool{true, false} { - name := "table" - if isClusterRestore { - name = "cluster" + for _, isSchemaOnly := range []bool{true, false} { + name := "regular-" + if isSchemaOnly { + name = "schema-only-" } - t.Run(name, func(t *testing.T) { - // blockLocations indicates whether the backup taken was blocked before or - // after the backfill portion of the schema change. - for _, blockLocation := range []string{"before", "after"} { - t.Run(blockLocation, func(t *testing.T) { - versionDirs, err := ioutil.ReadDir(filepath.Join(exportDirs, blockLocation)) - require.NoError(t, err) - - for _, clusterVersionDir := range versionDirs { - if clusterVersionDir.Name() == "19.2" && isClusterRestore { - // 19.2 does not support cluster backups. - continue - } - - t.Run(clusterVersionDir.Name(), func(t *testing.T) { - require.True(t, clusterVersionDir.IsDir()) - fullClusterVersionDir, err := filepath.Abs( - filepath.Join(exportDirs, blockLocation, clusterVersionDir.Name())) - require.NoError(t, err) + for _, isClusterRestore := range []bool{true, false} { + name = name + "table" + if isClusterRestore { + name = name + "cluster" + } + t.Run(name, func(t *testing.T) { + // blockLocations indicates whether the backup taken was blocked before or + // after the backfill portion of the schema change. + for _, blockLocation := range []string{"before", "after"} { + t.Run(blockLocation, func(t *testing.T) { + versionDirs, err := ioutil.ReadDir(filepath.Join(exportDirs, blockLocation)) + require.NoError(t, err) + + for _, clusterVersionDir := range versionDirs { + if clusterVersionDir.Name() == "19.2" && isClusterRestore { + // 19.2 does not support cluster backups. + continue + } - // In each version folder (e.g. "19.2", "20.1"), there is a backup for - // each schema change. - backupDirs, err := ioutil.ReadDir(fullClusterVersionDir) - require.NoError(t, err) + t.Run(clusterVersionDir.Name(), func(t *testing.T) { + require.True(t, clusterVersionDir.IsDir()) + fullClusterVersionDir, err := filepath.Abs( + filepath.Join(exportDirs, blockLocation, clusterVersionDir.Name())) + require.NoError(t, err) - for _, backupDir := range backupDirs { - fullBackupDir, err := filepath.Abs(filepath.Join(fullClusterVersionDir, backupDir.Name())) + // In each version folder (e.g. "19.2", "20.1"), there is a backup for + // each schema change. + backupDirs, err := ioutil.ReadDir(fullClusterVersionDir) require.NoError(t, err) - t.Run(backupDir.Name(), restoreMidSchemaChange(fullBackupDir, backupDir.Name(), isClusterRestore)) - } - }) - } - }) - } - }) + + for _, backupDir := range backupDirs { + fullBackupDir, err := filepath.Abs(filepath.Join(fullClusterVersionDir, backupDir.Name())) + require.NoError(t, err) + t.Run(backupDir.Name(), restoreMidSchemaChange(fullBackupDir, backupDir.Name(), + isClusterRestore, isSchemaOnly)) + } + }) + } + }) + } + }) + } } } @@ -132,7 +139,12 @@ func expectedSCJobCount(scName string) int { } func validateTable( - t *testing.T, kvDB *kv.DB, sqlDB *sqlutils.SQLRunner, dbName string, tableName string, + t *testing.T, + kvDB *kv.DB, + sqlDB *sqlutils.SQLRunner, + dbName string, + tableName string, + isSchemaOnly bool, ) { desc := desctestutils.TestingGetPublicTableDescriptor(kvDB, keys.SystemSQLCodec, dbName, tableName) // There should be no mutations on these table descriptors at this point. @@ -140,7 +152,11 @@ func validateTable( var rowCount int sqlDB.QueryRow(t, fmt.Sprintf(`SELECT count(*) FROM %s.%s`, dbName, tableName)).Scan(&rowCount) - require.Greater(t, rowCount, 0, "expected table to have some rows") + if isSchemaOnly { + require.Equal(t, rowCount, 0, "expected table to have no rows") + } else { + require.Greater(t, rowCount, 0, "expected table to have some rows") + } // The number of entries in all indexes should be the same. for _, index := range desc.AllIndexes() { var indexCount int @@ -162,7 +178,9 @@ func getTablesInTest(scName string) (tableNames []string) { return } -func verifyMidSchemaChange(t *testing.T, scName string, kvDB *kv.DB, sqlDB *sqlutils.SQLRunner) { +func verifyMidSchemaChange( + t *testing.T, scName string, kvDB *kv.DB, sqlDB *sqlutils.SQLRunner, isSchemaOnly bool, +) { tables := getTablesInTest(scName) // Check that we are left with the expected number of schema change jobs. @@ -174,7 +192,7 @@ func verifyMidSchemaChange(t *testing.T, scName string, kvDB *kv.DB, sqlDB *sqlu "Expected %d schema change jobs but found %v", expNumSchemaChangeJobs, synthesizedSchemaChangeJobs) for _, tableName := range tables { - validateTable(t, kvDB, sqlDB, "defaultdb", tableName) + validateTable(t, kvDB, sqlDB, "defaultdb", tableName, isSchemaOnly) // Ensure that a schema change can complete on the restored table. schemaChangeQuery := fmt.Sprintf("ALTER TABLE defaultdb.%s ADD CONSTRAINT post_restore_const CHECK (a > 0)", tableName) sqlDB.Exec(t, schemaChangeQuery) @@ -183,7 +201,7 @@ func verifyMidSchemaChange(t *testing.T, scName string, kvDB *kv.DB, sqlDB *sqlu } func restoreMidSchemaChange( - backupDir, schemaChangeName string, isClusterRestore bool, + backupDir, schemaChangeName string, isClusterRestore bool, isSchemaOnly bool, ) func(t *testing.T) { return func(t *testing.T) { ctx := context.Background() @@ -213,16 +231,19 @@ func restoreMidSchemaChange( require.NoError(t, err) sqlDB.Exec(t, "USE defaultdb") - restoreQuery := "RESTORE defaultdb.* from $1" + restoreQuery := "RESTORE defaultdb.* FROM $1" if isClusterRestore { - restoreQuery = "RESTORE from $1" + restoreQuery = "RESTORE FROM $1" + } + if isSchemaOnly { + restoreQuery = restoreQuery + "with schema_only" } log.Infof(context.Background(), "%+v", sqlDB.QueryStr(t, "SHOW BACKUP $1", localFoo)) sqlDB.Exec(t, restoreQuery, localFoo) // Wait for all jobs to terminate. Some may fail since we don't restore // adding spans. sqlDB.CheckQueryResultsRetry(t, "SELECT * FROM crdb_internal.jobs WHERE job_type = 'SCHEMA CHANGE' AND NOT (status = 'succeeded' OR status = 'failed')", [][]string{}) - verifyMidSchemaChange(t, schemaChangeName, kvDB, sqlDB) + verifyMidSchemaChange(t, schemaChangeName, kvDB, sqlDB, isSchemaOnly) // Because crdb_internal.invalid_objects is a virtual table, by default, the // query will take a lease on the database sqlDB is connected to and only run diff --git a/pkg/ccl/backupccl/restore_old_sequences_test.go b/pkg/ccl/backupccl/restore_old_sequences_test.go index 783f97767702..3e2fb96122b8 100644 --- a/pkg/ccl/backupccl/restore_old_sequences_test.go +++ b/pkg/ccl/backupccl/restore_old_sequences_test.go @@ -31,7 +31,7 @@ import ( // // VERSION=... // roachprod create local -// roachprod wipe local +// roachprod wipe localÅ // roachprod stage local release ${VERSION} // roachprod start local // roachprod sql local:1 -- -e "$(cat pkg/ccl/backupccl/testdata/restore_old_sequences/create.sql)" @@ -50,16 +50,22 @@ func TestRestoreOldSequences(t *testing.T) { t.Run("sequences-restore", func(t *testing.T) { dirs, err := ioutil.ReadDir(exportDirs) require.NoError(t, err) - for _, dir := range dirs { - require.True(t, dir.IsDir()) - exportDir, err := filepath.Abs(filepath.Join(exportDirs, dir.Name())) - require.NoError(t, err) - t.Run(dir.Name(), restoreOldSequencesTest(exportDir)) + for _, isSchemaOnly := range []bool{true, false} { + suffix := "" + if isSchemaOnly { + suffix = "-schema-only" + } + for _, dir := range dirs { + require.True(t, dir.IsDir()) + exportDir, err := filepath.Abs(filepath.Join(exportDirs, dir.Name())) + require.NoError(t, err) + t.Run(dir.Name()+suffix, restoreOldSequencesTest(exportDir, isSchemaOnly)) + } } }) } -func restoreOldSequencesTest(exportDir string) func(t *testing.T) { +func restoreOldSequencesTest(exportDir string, isSchemaOnly bool) func(t *testing.T) { return func(t *testing.T) { params := base.TestServerArgs{} const numAccounts = 1000 @@ -71,10 +77,17 @@ func restoreOldSequencesTest(exportDir string) func(t *testing.T) { sqlDB.Exec(t, `CREATE DATABASE test`) var unused string var importedRows int - sqlDB.QueryRow(t, `RESTORE test.* FROM $1`, localFoo).Scan( + restoreQuery := `RESTORE test.* FROM $1` + if isSchemaOnly { + restoreQuery = restoreQuery + " with schema_only" + } + sqlDB.QueryRow(t, restoreQuery, localFoo).Scan( &unused, &unused, &unused, &importedRows, &unused, &unused, ) - const totalRows = 4 + totalRows := 4 + if isSchemaOnly { + totalRows = 0 + } if importedRows != totalRows { t.Fatalf("expected %d rows, got %d", totalRows, importedRows) } @@ -100,6 +113,12 @@ func restoreOldSequencesTest(exportDir string) func(t *testing.T) { {"1", "1"}, {"2", "2"}, } + if isSchemaOnly { + // In a schema_only RESTORE, the restored sequence will be empty + expectedRows = [][]string{ + {"1", "1"}, + } + } sqlDB.CheckQueryResults(t, `SELECT * FROM test.t1 ORDER BY i`, expectedRows) sqlDB.CheckQueryResults(t, `SELECT * FROM test.v`, [][]string{{"1"}}) sqlDB.CheckQueryResults(t, `SELECT * FROM test.v2`, [][]string{{"2"}}) diff --git a/pkg/ccl/backupccl/restore_planning.go b/pkg/ccl/backupccl/restore_planning.go index 4c1be2a01a78..6796d34fbe4e 100644 --- a/pkg/ccl/backupccl/restore_planning.go +++ b/pkg/ccl/backupccl/restore_planning.go @@ -1138,6 +1138,7 @@ func resolveOptionsForRestoreJobDescription( SkipMissingSequenceOwners: opts.SkipMissingSequenceOwners, SkipMissingViews: opts.SkipMissingViews, Detached: opts.Detached, + SchemaOnly: opts.SchemaOnly, } if opts.EncryptionPassphrase != nil { @@ -1226,6 +1227,12 @@ func restorePlanHook( return nil, nil, nil, false, err } + if restoreStmt.Options.SchemaOnly && + !p.ExecCfg().Settings.Version.IsActive(ctx, clusterversion.Start22_2) { + return nil, nil, nil, false, + errors.New("cannot run RESTORE with schema_only until cluster has fully upgraded to 22.2") + } + fromFns := make([]func() ([]string, error), len(restoreStmt.From)) for i := range restoreStmt.From { fromFn, err := p.TypeAsStringArray(ctx, tree.Exprs(restoreStmt.From[i]), "RESTORE") @@ -2048,7 +2055,7 @@ func doRestorePlan( // TODO(msbutler): Delete in 23.1 RestoreSystemUsers: restoreStmt.DescriptorCoverage == tree.SystemUsers, PreRewriteTenantId: oldTenantID, - Validation: jobspb.RestoreValidation_DefaultRestore, + SchemaOnly: restoreStmt.Options.SchemaOnly, } jr := jobs.Record{ diff --git a/pkg/ccl/backupccl/restore_processor_planning.go b/pkg/ccl/backupccl/restore_processor_planning.go index e9388cc05ca3..860b4bab7400 100644 --- a/pkg/ccl/backupccl/restore_processor_planning.go +++ b/pkg/ccl/backupccl/restore_processor_planning.go @@ -118,7 +118,6 @@ func distRestore( TableRekeys: tableRekeys, TenantRekeys: tenantRekeys, PKIDs: pkIDs, - Validation: jobspb.RestoreValidation_DefaultRestore, } if len(splitAndScatterSpecs) == 0 { @@ -308,7 +307,6 @@ func makeSplitAndScatterSpecs( }}, TableRekeys: tableRekeys, TenantRekeys: tenantRekeys, - Validation: jobspb.RestoreValidation_DefaultRestore, } } } diff --git a/pkg/ccl/backupccl/split_and_scatter_processor.go b/pkg/ccl/backupccl/split_and_scatter_processor.go index 9ccbd6ed835b..676231f6b3a8 100644 --- a/pkg/ccl/backupccl/split_and_scatter_processor.go +++ b/pkg/ccl/backupccl/split_and_scatter_processor.go @@ -15,7 +15,6 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -213,10 +212,6 @@ func newSplitAndScatterProcessor( output execinfra.RowReceiver, ) (execinfra.Processor, error) { - if spec.Validation != jobspb.RestoreValidation_DefaultRestore { - return nil, errors.New("Split and Scatter Processor does not support validation yet") - } - numEntries := 0 for _, chunk := range spec.Chunks { numEntries += len(chunk.Entries) diff --git a/pkg/ccl/backupccl/split_and_scatter_processor_test.go b/pkg/ccl/backupccl/split_and_scatter_processor_test.go index 8b586e095590..3c13b491a202 100644 --- a/pkg/ccl/backupccl/split_and_scatter_processor_test.go +++ b/pkg/ccl/backupccl/split_and_scatter_processor_test.go @@ -15,7 +15,6 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" @@ -247,7 +246,6 @@ func TestSplitAndScatterProcessor(t *testing.T) { require.NoError(t, err) post := execinfrapb.PostProcessSpec{} - c.procSpec.Validation = jobspb.RestoreValidation_DefaultRestore proc, err := newSplitAndScatterProcessor(&flowCtx, 0 /* processorID */, c.procSpec, &post, out) require.NoError(t, err) ssp, ok := proc.(*splitAndScatterProcessor) diff --git a/pkg/ccl/backupccl/testdata/backup-restore/restore-schema-only b/pkg/ccl/backupccl/testdata/backup-restore/restore-schema-only new file mode 100644 index 000000000000..53542234456f --- /dev/null +++ b/pkg/ccl/backupccl/testdata/backup-restore/restore-schema-only @@ -0,0 +1,144 @@ +# Test schema_only restore + +new-server name=s1 allow-implicit-access +---- + +exec-sql +CREATE DATABASE d; +CREATE TYPE d.greeting AS ENUM ('hello', 'howdy', 'hi'); +CREATE TABLE d.t1 (x INT); +INSERT INTO d.t1 VALUES (1), (2), (3); +CREATE TABLE d.t2 (x d.greeting); +INSERT INTO d.t2 VALUES ('hello'), ('howdy'); +COMMENT ON TABLE d.t1 IS 'This comment better get restored from the backed up system table!'; +---- + +query-sql +SHOW CREATE TABLE d.t1; +---- +d.public.t1 CREATE TABLE public.t1 ( + x INT8 NULL, + rowid INT8 NOT VISIBLE NOT NULL DEFAULT unique_rowid(), + CONSTRAINT t1_pkey PRIMARY KEY (rowid ASC) +); +COMMENT ON TABLE public.t1 IS 'This comment better get restored from the backed up system table!' + +# drop and create defaultDB to ensure it has a higher ID than by default. We will check that when +# this cluster is restored, the default db with the higher id was also restored +# by default, default db has an id of 100. +query-sql +SELECT id FROM system.namespace WHERE name = 'defaultdb' +---- +100 + +exec-sql +DROP DATABASE defaultdb; +CREATE DATABASE defaultdb; +---- + +query-sql +SELECT count(*) FROM system.namespace WHERE name = 'defaultdb' AND id > 100 +---- +1 + +exec-sql +BACKUP INTO 'nodelocal://1/full_cluster_backup/'; +---- + +exec-sql +BACKUP Database d INTO 'nodelocal://1/full_database_backup/'; +---- + + +# A new cluster in prep for a cluster level schema_only restore. +new-server name=s2 share-io-dir=s1 allow-implicit-access +---- + +# First, ensure cluster level schema_only restore fails fast in same ways as a cluster level restore. +# +# Fail fast if the user passes new_db_name. +exec-sql +RESTORE FROM LATEST IN 'nodelocal://0/full_cluster_backup/' with schema_only, new_db_name='d2'; +---- +pq: new_db_name can only be used for RESTORE DATABASE with a single target database + + +exec-sql server=s2 +CREATE USER testuser +---- + +# Non admins cannot run schema_only cluster restore +exec-sql user=testuser +RESTORE FROM LATEST IN 'nodelocal://0/full_cluster_backup/' with schema_only +---- +pq: only users with the admin role are allowed to restore full cluster backups + +# Fail fast using a database backup +exec-sql +RESTORE FROM LATEST IN 'nodelocal://0/full_database_backup/' with schema_only; +---- +pq: full cluster RESTORE can only be used on full cluster BACKUP files + +exec-sql +RESTORE FROM LATEST IN 'nodelocal://0/full_cluster_backup/' with schema_only; +---- + +# there should be no data in the restored tables +query-sql +SELECT * FROM d.t1; +---- + +query-sql +SELECT * FROM d.t2; +---- + +# The backed up cluster was initiated with bank. Ensure it's now empty. +query-sql +SELECT * FROM data.bank; +---- + +# The backed table d.t1 had a comment stored in a system table. This should have been restored. +query-sql +SHOW CREATE TABLE d.t1; +---- +d.public.t1 CREATE TABLE public.t1 ( + x INT8 NULL, + rowid INT8 NOT VISIBLE NOT NULL DEFAULT unique_rowid(), + CONSTRAINT t1_pkey PRIMARY KEY (rowid ASC) +); +COMMENT ON TABLE public.t1 IS 'This comment better get restored from the backed up system table!' + +# Ensure the defaultdb from the backed up cluster was restored. +query-sql +SELECT count(*) FROM system.namespace WHERE name = 'defaultdb' AND id > 100 +---- +1 + +############################################################ +# Ensure Database Level schema_only restore logic is sound +############################################################ + +exec-sql +RESTORE DATABASE d FROM LATEST IN 'nodelocal://0/full_database_backup/' with schema_only, new_db_name='d2'; +---- + +# There should be no data in the user tables. +query-sql +SELECT * FROM d2.t1; +---- + +query-sql +SELECT * FROM d2.t2; +---- + +# Each of the restored types should have namespace entries. Test this by +# trying to create types that would cause namespace conflicts. +exec-sql +CREATE TYPE d2.greeting AS ENUM ('hello', 'hiya') +---- +pq: type "d2.public.greeting" already exists + +# We should be able to resolve each restored type. Test this by inserting +# into each of the restored tables. +exec-sql +INSERT INTO d2.t2 VALUES ('hi'); diff --git a/pkg/ccl/backupccl/testdata/backup-restore/restore-schema-only-multiregion b/pkg/ccl/backupccl/testdata/backup-restore/restore-schema-only-multiregion new file mode 100644 index 000000000000..a8852688402e --- /dev/null +++ b/pkg/ccl/backupccl/testdata/backup-restore/restore-schema-only-multiregion @@ -0,0 +1,193 @@ +# Test schema only multi region restore this test is exactly the same as the 'multiregion' datadriven +# test, except schema_only RESTORE is used + +new-server name=s1 allow-implicit-access localities=us-east-1,us-west-1,eu-central-1 +---- + +exec-sql +CREATE DATABASE d PRIMARY REGION "us-east-1" REGIONS "us-west-1", "eu-central-1"; +CREATE TABLE d.t (x INT); +INSERT INTO d.t VALUES (1), (2), (3); +---- + +query-sql +SELECT region FROM [SHOW REGIONS FROM DATABASE d] ORDER BY 1; +---- +eu-central-1 +us-east-1 +us-west-1 + +exec-sql +BACKUP DATABASE d INTO 'nodelocal://1/database_backup/'; +---- + +exec-sql +BACKUP INTO 'nodelocal://1/full_cluster_backup/'; +---- + +# A new cluster with the same locality settings. +new-server name=s2 share-io-dir=s1 allow-implicit-access localities=us-east-1,us-west-1,eu-central-1 +---- + +exec-sql +RESTORE FROM LATEST IN 'nodelocal://0/full_cluster_backup/' with schema_only; +---- + +exec-sql +DROP DATABASE d; +---- + +exec-sql +RESTORE DATABASE d FROM LATEST IN 'nodelocal://0/database_backup/' with schema_only; +---- + +query-sql +SHOW DATABASES; +---- +d root us-east-1 {eu-central-1,us-east-1,us-west-1} zone +data root {} +defaultdb root {} +postgres root {} +system node {} + +# A new cluster with different localities settings. +new-server name=s3 share-io-dir=s1 allow-implicit-access localities=eu-central-1,eu-north-1 +---- + +exec-sql +RESTORE DATABASE d FROM LATEST IN 'nodelocal://0/database_backup/' with schema_only; +---- +pq: detected a mismatch in regions between the restore cluster and the backup cluster, missing regions detected: us-east-1, us-west-1. +HINT: there are two ways you can resolve this issue: 1) update the cluster to which you're restoring to ensure that the regions present on the nodes' --locality flags match those present in the backup image, or 2) restore with the "skip_localities_check" option + +exec-sql +RESTORE FROM LATEST IN 'nodelocal://0/full_cluster_backup/' with schema_only; +---- +pq: detected a mismatch in regions between the restore cluster and the backup cluster, missing regions detected: us-east-1, us-west-1. +HINT: there are two ways you can resolve this issue: 1) update the cluster to which you're restoring to ensure that the regions present on the nodes' --locality flags match those present in the backup image, or 2) restore with the "skip_localities_check" option + +# Create a database with no regions to check default primary regions. +exec-sql +CREATE DATABASE no_region_db; +---- + +exec-sql +CREATE TABLE no_region_db.t (x INT); +INSERT INTO no_region_db.t VALUES (1), (2), (3); +CREATE DATABASE no_region_db_2; +CREATE TABLE no_region_db_2.t (x INT); +INSERT INTO no_region_db_2.t VALUES (1), (2), (3); +---- + +exec-sql +BACKUP DATABASE no_region_db INTO 'nodelocal://1/no_region_database_backup/'; +---- + +exec-sql +BACKUP INTO 'nodelocal://1/no_region_cluster_backup/'; +---- + +exec-sql +DROP DATABASE no_region_db; +---- + +exec-sql +DROP DATABASE no_region_db_2; +---- + +exec-sql ignore-notice +SET CLUSTER SETTING sql.defaults.primary_region = 'non-existent-region'; +---- + +exec-sql +RESTORE DATABASE no_region_db FROM LATEST IN 'nodelocal://1/no_region_database_backup/' with schema_only; +---- +pq: region "non-existent-region" does not exist +HINT: valid regions: eu-central-1, eu-north-1 +-- +set the default PRIMARY REGION to a region that exists (see SHOW REGIONS FROM CLUSTER) then using SET CLUSTER SETTING sql.defaults.primary_region = 'region' + +exec-sql ignore-notice +SET CLUSTER SETTING sql.defaults.primary_region = 'eu-central-1'; +---- + +exec-sql +RESTORE DATABASE no_region_db FROM LATEST IN 'nodelocal://1/no_region_database_backup/' with schema_only; +---- +NOTICE: setting the PRIMARY REGION as eu-central-1 on database no_region_db +HINT: to change the default primary region, use SET CLUSTER SETTING sql.defaults.primary_region = 'region' or use RESET CLUSTER SETTING sql.defaults.primary_region to disable this behavior + +query-sql +SHOW DATABASES; +---- +defaultdb root {} +no_region_db root eu-central-1 {eu-central-1} zone +postgres root {} +system node {} + +query-sql +USE no_region_db; +SELECT schema_name, table_name, type, owner, locality FROM [SHOW TABLES]; +---- +public t table root REGIONAL BY TABLE IN PRIMARY REGION + +exec-sql +CREATE DATABASE eu_central_db; +CREATE TABLE eu_central_db.t (x INT); +INSERT INTO eu_central_db.t VALUES (1), (2), (3); +---- +NOTICE: setting eu-central-1 as the PRIMARY REGION as no PRIMARY REGION was specified + +exec-sql +BACKUP DATABASE eu_central_db INTO 'nodelocal://1/eu_central_database_backup/'; +---- + +# New cluster for a cluster backup. +new-server name=s4 share-io-dir=s1 allow-implicit-access localities=eu-central-1,eu-north-1 +---- + +exec-sql ignore-notice +SET CLUSTER SETTING sql.defaults.primary_region = 'eu-north-1'; +---- + +exec-sql +RESTORE FROM LATEST IN 'nodelocal://1/no_region_cluster_backup/' with schema_only; +---- +NOTICE: setting the PRIMARY REGION as eu-north-1 on database defaultdb +HINT: to change the default primary region, use SET CLUSTER SETTING sql.defaults.primary_region = 'region' or use RESET CLUSTER SETTING sql.defaults.primary_region to disable this behavior +NOTICE: setting the PRIMARY REGION as eu-north-1 on database postgres +HINT: to change the default primary region, use SET CLUSTER SETTING sql.defaults.primary_region = 'region' or use RESET CLUSTER SETTING sql.defaults.primary_region to disable this behavior +NOTICE: setting the PRIMARY REGION as eu-north-1 on database no_region_db +HINT: to change the default primary region, use SET CLUSTER SETTING sql.defaults.primary_region = 'region' or use RESET CLUSTER SETTING sql.defaults.primary_region to disable this behavior +NOTICE: setting the PRIMARY REGION as eu-north-1 on database no_region_db_2 +HINT: to change the default primary region, use SET CLUSTER SETTING sql.defaults.primary_region = 'region' or use RESET CLUSTER SETTING sql.defaults.primary_region to disable this behavior + +query-sql +SHOW DATABASES; +---- +defaultdb root eu-north-1 {eu-north-1} zone +no_region_db root eu-north-1 {eu-north-1} zone +no_region_db_2 root eu-north-1 {eu-north-1} zone +postgres root eu-north-1 {eu-north-1} zone +system node {} + +query-sql +USE no_region_db; +SELECT schema_name, table_name, type, owner, locality FROM [SHOW TABLES]; +---- +public t table root REGIONAL BY TABLE IN PRIMARY REGION + +# Check we can restore without triggering the default primary region. +exec-sql +RESTORE DATABASE eu_central_db FROM LATEST IN 'nodelocal://1/eu_central_database_backup/' with schema_only; +---- + +query-sql +SHOW DATABASES; +---- +defaultdb root eu-north-1 {eu-north-1} zone +eu_central_db root eu-central-1 {eu-central-1} zone +no_region_db root eu-north-1 {eu-north-1} zone +no_region_db_2 root eu-north-1 {eu-north-1} zone +postgres root eu-north-1 {eu-north-1} zone +system node {} diff --git a/pkg/jobs/jobspb/jobs.proto b/pkg/jobs/jobspb/jobs.proto index b7a9c82dd876..483fe501e632 100644 --- a/pkg/jobs/jobspb/jobs.proto +++ b/pkg/jobs/jobspb/jobs.proto @@ -378,13 +378,12 @@ message RestoreDetails { // it is only valid to set this if len(tenants) == 1. roachpb.TenantID pre_rewrite_tenant_id = 23; - // RestoreValidation determines whether to skip certain parts of the restore - // job if its only purpose is to validate the user's restore command. - RestoreValidation validation = 24; -} + reserved 24; + + // SchemaOnly determines whether to only restore the schema in the backup. + bool schema_only = 25; -enum RestoreValidation { - DefaultRestore = 0; + // NEXT ID: 26. } diff --git a/pkg/sql/execinfrapb/processors_bulk_io.proto b/pkg/sql/execinfrapb/processors_bulk_io.proto index 6b18e8216a5d..18e29b3cd6d5 100644 --- a/pkg/sql/execinfrapb/processors_bulk_io.proto +++ b/pkg/sql/execinfrapb/processors_bulk_io.proto @@ -302,9 +302,10 @@ message RestoreDataSpec { // PKIDs is used to convert result from an ExportRequest into row count // information passed back to track progress in the backup job. map pk_ids = 4 [(gogoproto.customname) = "PKIDs"]; - optional jobs.jobspb.RestoreValidation validation = 7 [(gogoproto.nullable) = false]; + reserved 7; + optional bool schema_only = 8 [(gogoproto.nullable) = false]; - // NEXT ID: 8. + // NEXT ID: 9. } message SplitAndScatterSpec { @@ -316,9 +317,10 @@ message SplitAndScatterSpec { repeated RestoreEntryChunk chunks = 1 [(gogoproto.nullable) = false]; repeated TableRekey table_rekeys = 2 [(gogoproto.nullable) = false]; repeated TenantRekey tenant_rekeys = 3 [(gogoproto.nullable) = false]; - optional jobs.jobspb.RestoreValidation validation = 5 [(gogoproto.nullable) = false]; + reserved 5; + optional bool schema_only = 6 [(gogoproto.nullable) = false]; - // NEXTID: 6. + // NEXTID: 7. } // ExporterSpec is the specification for a processor that consumes rows and diff --git a/pkg/sql/parser/sql.y b/pkg/sql/parser/sql.y index f4facdca74c2..ecfc1251c602 100644 --- a/pkg/sql/parser/sql.y +++ b/pkg/sql/parser/sql.y @@ -914,7 +914,8 @@ func (u *sqlSymUnion) functionObjs() tree.FuncObjs { %token RELEASE RESET RESTART RESTORE RESTRICT RESTRICTED RESUME RETURNING RETURN RETURNS RETRY REVISION_HISTORY %token REVOKE RIGHT ROLE ROLES ROLLBACK ROLLUP ROUTINES ROW ROWS RSHIFT RULE RUNNING -%token SAVEPOINT SCANS SCATTER SCHEDULE SCHEDULES SCROLL SCHEMA SCHEMAS SCRUB SEARCH SECOND SECONDARY SECURITY SELECT SEQUENCE SEQUENCES +%token SAVEPOINT SCANS SCATTER SCHEDULE SCHEDULES SCROLL SCHEMA SCHEMA_ONLY SCHEMAS SCRUB +%token SEARCH SECOND SECONDARY SECURITY SELECT SEQUENCE SEQUENCES %token SERIALIZABLE SERVER SESSION SESSIONS SESSION_USER SET SETOF SETS SETTING SETTINGS %token SHARE SHOW SIMILAR SIMPLE SKIP SKIP_LOCALITIES_CHECK SKIP_MISSING_FOREIGN_KEYS %token SKIP_MISSING_SEQUENCES SKIP_MISSING_SEQUENCE_OWNERS SKIP_MISSING_VIEWS SMALLINT SMALLSERIAL SNAPSHOT SOME SPLIT SQL @@ -3530,7 +3531,10 @@ restore_options: { $$.val = &tree.RestoreOptions{AsTenant: $3.expr()} } - +| SCHEMA_ONLY + { + $$.val = &tree.RestoreOptions{SchemaOnly: true} + } import_format: name { @@ -15105,6 +15109,7 @@ unreserved_keyword: | RUNNING | SCHEDULE | SCHEDULES +| SCHEMA_ONLY | SCROLL | SETTING | SETTINGS diff --git a/pkg/sql/parser/testdata/backup_restore b/pkg/sql/parser/testdata/backup_restore index a9409e1a45ef..c56976f2fd6c 100644 --- a/pkg/sql/parser/testdata/backup_restore +++ b/pkg/sql/parser/testdata/backup_restore @@ -518,6 +518,14 @@ RESTORE DATABASE foo FROM ('bar') WITH new_db_name = ('baz') -- fully parenthesi RESTORE DATABASE foo FROM '_' WITH new_db_name = '_' -- literals removed RESTORE DATABASE _ FROM 'bar' WITH new_db_name = 'baz' -- identifiers removed +parse +RESTORE DATABASE foo FROM 'bar' WITH schema_only +---- +RESTORE DATABASE foo FROM 'bar' WITH schema_only +RESTORE DATABASE foo FROM ('bar') WITH schema_only -- fully parenthesized +RESTORE DATABASE foo FROM '_' WITH schema_only -- literals removed +RESTORE DATABASE _ FROM 'bar' WITH schema_only -- identifiers removed + parse RESTORE DATABASE foo FROM 'bar' IN LATEST WITH incremental_location = 'baz' ---- diff --git a/pkg/sql/schemachanger/sctest/cumulative.go b/pkg/sql/schemachanger/sctest/cumulative.go index 079a48ac5cb0..70c5ca788c63 100644 --- a/pkg/sql/schemachanger/sctest/cumulative.go +++ b/pkg/sql/schemachanger/sctest/cumulative.go @@ -472,28 +472,38 @@ func Backup(t *testing.T, dir string, newCluster NewClusterFunc) { t.Logf("finished") for i, b := range backups { - t.Run("", func(t *testing.T) { - t.Logf("testing backup %d %v", i, b.isRollback) - tdb.Exec(t, fmt.Sprintf("DROP DATABASE IF EXISTS %q CASCADE", dbName)) - tdb.Exec(t, "SET use_declarative_schema_changer = 'off'") - tdb.Exec(t, fmt.Sprintf("RESTORE DATABASE %s FROM LATEST IN '%s'", dbName, b.url)) - tdb.Exec(t, fmt.Sprintf("USE %q", dbName)) - waitForSchemaChangesToFinish(t, tdb) - afterRestore := tdb.QueryStr(t, fetchDescriptorStateQuery) - if b.isRollback { - require.Equal(t, before, afterRestore) - } else { - require.Equal(t, after, afterRestore) + for _, isSchemaOnly := range []bool{true, false} { + name := "" + if isSchemaOnly { + name = "schema-only" } - // Hack to deal with corrupt userfiles tables due to #76764. - const validateQuery = ` + t.Run(name, func(t *testing.T) { + t.Logf("testing backup %d %v", i, b.isRollback) + tdb.Exec(t, fmt.Sprintf("DROP DATABASE IF EXISTS %q CASCADE", dbName)) + tdb.Exec(t, "SET use_declarative_schema_changer = 'off'") + restoreQuery := fmt.Sprintf("RESTORE DATABASE %s FROM LATEST IN '%s'", dbName, b.url) + if isSchemaOnly { + restoreQuery = restoreQuery + " with schema_only" + } + tdb.Exec(t, restoreQuery) + tdb.Exec(t, fmt.Sprintf("USE %q", dbName)) + waitForSchemaChangesToFinish(t, tdb) + afterRestore := tdb.QueryStr(t, fetchDescriptorStateQuery) + if b.isRollback { + require.Equal(t, before, afterRestore) + } else { + require.Equal(t, after, afterRestore) + } + // Hack to deal with corrupt userfiles tables due to #76764. + const validateQuery = ` SELECT * FROM crdb_internal.invalid_objects WHERE database_name != 'backups' ` - tdb.CheckQueryResults(t, validateQuery, [][]string{}) - tdb.Exec(t, fmt.Sprintf("DROP DATABASE %q CASCADE", dbName)) - tdb.Exec(t, "USE backups") - tdb.CheckQueryResults(t, validateQuery, [][]string{}) - }) + tdb.CheckQueryResults(t, validateQuery, [][]string{}) + tdb.Exec(t, fmt.Sprintf("DROP DATABASE %q CASCADE", dbName)) + tdb.Exec(t, "USE backups") + tdb.CheckQueryResults(t, validateQuery, [][]string{}) + }) + } } } cumulativeTest(t, dir, testFunc) diff --git a/pkg/sql/sem/tree/backup.go b/pkg/sql/sem/tree/backup.go index 6a68a755148a..c82787c7532c 100644 --- a/pkg/sql/sem/tree/backup.go +++ b/pkg/sql/sem/tree/backup.go @@ -137,6 +137,7 @@ type RestoreOptions struct { NewDBName Expr IncrementalStorage StringOrPlaceholderOptList AsTenant Expr + SchemaOnly bool } var _ NodeFormatter = &RestoreOptions{} @@ -407,6 +408,10 @@ func (o *RestoreOptions) Format(ctx *FmtCtx) { ctx.WriteString("tenant = ") ctx.FormatNode(o.AsTenant) } + if o.SchemaOnly { + maybeAddSep() + ctx.WriteString("schema_only") + } } // CombineWith merges other backup options into this backup options struct. @@ -502,6 +507,13 @@ func (o *RestoreOptions) CombineWith(other *RestoreOptions) error { return errors.New("tenant option specified multiple times") } + if o.SchemaOnly { + if other.SchemaOnly { + return errors.New("schema_only option specified multiple times") + } + } else { + o.SchemaOnly = other.SchemaOnly + } return nil } @@ -520,7 +532,8 @@ func (o RestoreOptions) IsDefault() bool { o.DebugPauseOn == options.DebugPauseOn && o.NewDBName == options.NewDBName && cmp.Equal(o.IncrementalStorage, options.IncrementalStorage) && - o.AsTenant == options.AsTenant + o.AsTenant == options.AsTenant && + o.SchemaOnly == options.SchemaOnly } // BackupTargetList represents a list of targets. From d46ba25addd9c1d85076b0796b2d2e553e6e84d5 Mon Sep 17 00:00:00 2001 From: Andrii Vorobiov Date: Mon, 8 Aug 2022 20:00:19 +0300 Subject: [PATCH 5/7] ui: fix time window selection with mouse on Metrics charts This patch fixes an issue that prevents proper time selection with mouse on Metrics charts. The root cause of it is updated time scale object didn't include correct value of `windowSize` that's used to calculate `start` position of time range. Release note (ui change): fix issue with incorrect start time position of selected time range on Metrics page. --- .../src/views/cluster/components/linegraph/index.tsx | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/pkg/ui/workspaces/db-console/src/views/cluster/components/linegraph/index.tsx b/pkg/ui/workspaces/db-console/src/views/cluster/components/linegraph/index.tsx index 6c8da288be35..770ccb89372a 100644 --- a/pkg/ui/workspaces/db-console/src/views/cluster/components/linegraph/index.tsx +++ b/pkg/ui/workspaces/db-console/src/views/cluster/components/linegraph/index.tsx @@ -207,9 +207,11 @@ export class LineGraph extends React.Component { start: moment.unix(start), end: moment.unix(end), }; + const seconds = moment.duration(moment.utc(end).diff(start)).asSeconds(); let newTimeScale: TimeScale = { - ...findClosestTimeScale(defaultTimeScaleOptions, end - start, start), + ...findClosestTimeScale(defaultTimeScaleOptions, seconds), key: "Custom", + windowSize: moment.duration(moment.unix(end).diff(moment.unix(start))), fixedWindowEnd: moment.unix(end), }; if (this.props.adjustTimeScaleOnChange) { From 49bcc0962a54c4780fa5ca9e1a2e0cea797e7793 Mon Sep 17 00:00:00 2001 From: Aayush Shah Date: Wed, 22 Jun 2022 14:38:10 -0400 Subject: [PATCH 6/7] kvserver: make MVCC GC less disruptive to foreground traffic MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This commit changes GC requests to no longer declare exclusive latches at their BatchRequest's timestamp. This was already incorrect as explained in https://github.com/cockroachdb/cockroach/issues/55293. >The first use is broken because we acquire write latches at the batch header's timestamp, which is set to time.Now(), so we're only serializing with reads in the future and all other writes [1]. So we're disruptive to everyone except who we want to serialize with – reads in the past! This commit makes GC requests only declare a non-mvcc exclusive latch over the `RangeGCThresholdKey`. This is correct because: ``` // 1. We define "correctness" to be the property that a reader reading at / // around the GC threshold will either see the correct results or receive an // error. // 2. Readers perform their command evaluation over a stable snapshot of the // storage engine. This means that the reader will not see the effects of a // subsequent GC run as long as it created a Pebble iterator before the GC // request. // 3. A reader checks the in-memory GC threshold of a Replica after it has // created this snapshot (i.e. after a Pebble iterator has been created). // 4. If the in-memory GC threshold is above the timestamp of the read, the // reader receives an error. Otherwise, the reader is guaranteed to see a // state of the storage engine that hasn't been affected by the GC request [5]. // 5. GC requests bump the in-memory GC threshold of a Replica as a pre-apply // side effect. This means that if a reader checks the in-memory GC threshold // after it has created a Pebble iterator, it is impossible for the iterator // to point to a storage engine state that has been affected by the GC // request. ``` As a result, GC requests should now be much less disruptive to foreground traffic since they're no longer redundantly declaring exclusive latches over global keys. Release note (performance improvement): MVCC garbage collection should now be much less disruptive to foreground traffic than before. --- pkg/kv/kvserver/batcheval/cmd_gc.go | 71 +++++---- .../kvserver/batcheval/cmd_recompute_stats.go | 6 +- pkg/kv/kvserver/replica_test.go | 142 ++++++++++++------ pkg/kv/kvserver/spanset/spanset.go | 19 ++- 4 files changed, 155 insertions(+), 83 deletions(-) diff --git a/pkg/kv/kvserver/batcheval/cmd_gc.go b/pkg/kv/kvserver/batcheval/cmd_gc.go index 2f9290bf2015..95491d3ee48a 100644 --- a/pkg/kv/kvserver/batcheval/cmd_gc.go +++ b/pkg/kv/kvserver/batcheval/cmd_gc.go @@ -21,6 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" + "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/errors" ) @@ -35,35 +36,51 @@ func declareKeysGC( latchSpans, _ *spanset.SpanSet, _ time.Duration, ) { - // Intentionally don't call DefaultDeclareKeys: the key range in the header - // is usually the whole range (pending resolution of #7880). gcr := req.(*roachpb.GCRequest) - for _, key := range gcr.Keys { - if keys.IsLocal(key.Key) { - latchSpans.AddNonMVCC(spanset.SpanReadWrite, roachpb.Span{Key: key.Key}) - } else { - latchSpans.AddMVCC(spanset.SpanReadWrite, roachpb.Span{Key: key.Key}, header.Timestamp) - } - } - // Extend the range key latches by feather to ensure MVCC stats - // computations correctly account for adjacent range keys tombstones if they - // need to be split. - // TODO(oleg): These latches are very broad and will be disruptive to read and - // write operations despite only accessing "stale" data. We should think of - // better integrating it with latchless GC approach. - for _, span := range mergeAdjacentSpans(makeLookupBoundariesForGCRanges(rs.GetStartKey().AsRawKey(), - nil, gcr.RangeKeys)) { - latchSpans.AddMVCC(spanset.SpanReadWrite, span, - header.Timestamp) - } - // Be smart here about blocking on the threshold keys. The MVCC GC queue can - // send an empty request first to bump the thresholds, and then another one - // that actually does work but can avoid declaring these keys below. - if !gcr.Threshold.IsEmpty() { - latchSpans.AddNonMVCC(spanset.SpanReadWrite, roachpb.Span{Key: keys.RangeGCThresholdKey(rs.GetRangeID())}) + // When GC-ing MVCC range key tombstones or individual range keys, we need to + // serialize with all writes that overlap the MVCC range tombstone, as well as + // the immediate left/right neighboring keys. This is because a range key + // write has non-local effects, i.e. it can fragment or merge other range keys + // at other timestamps and at its boundaries, and this has a non-commutative + // effect on MVCC stats -- if someone writes a new range key while we're GCing + // one below, the stats would come out wrong. + // Note that we only need to serialize with writers (including other GC + // processes) and not with readers (that are guaranteed to be above the GC + // threshold). To achieve this, we declare read-write access at + // hlc.MaxTimestamp which will not block any readers. + for _, span := range mergeAdjacentSpans(makeLookupBoundariesForGCRanges( + rs.GetStartKey().AsRawKey(), nil, gcr.RangeKeys, + )) { + latchSpans.AddMVCC(spanset.SpanReadWrite, span, hlc.MaxTimestamp) } + // The RangeGCThresholdKey is only written to if the + // req.(*GCRequest).Threshold is set. However, we always declare an exclusive + // access over this key in order to serialize with other GC requests. + // + // Correctness: + // It is correct for a GC request to not declare exclusive access over the + // keys being GCed because of the following: + // 1. We define "correctness" to be the property that a reader reading at / + // around the GC threshold will either see the correct results or receive an + // error. + // 2. Readers perform their command evaluation over a stable snapshot of the + // storage engine. This means that the reader will not see the effects of a + // subsequent GC run as long as it created a Pebble iterator before the GC + // request. + // 3. A reader checks the in-memory GC threshold of a Replica after it has + // created this snapshot (i.e. after a Pebble iterator has been created). + // 4. If the in-memory GC threshold is above the timestamp of the read, the + // reader receives an error. Otherwise, the reader is guaranteed to see a + // state of the storage engine that hasn't been affected by the GC request [5]. + // 5. GC requests bump the in-memory GC threshold of a Replica as a pre-apply + // side effect. This means that if a reader checks the in-memory GC threshold + // after it has created a Pebble iterator, it is impossible for the iterator + // to point to a storage engine state that has been affected by the GC + // request. + latchSpans.AddNonMVCC(spanset.SpanReadWrite, roachpb.Span{Key: keys.RangeGCThresholdKey(rs.GetRangeID())}) // Needed for Range bounds checks in calls to EvalContext.ContainsKey. latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{Key: keys.RangeDescriptorKey(rs.GetStartKey())}) + latchSpans.DisableUndeclaredAccessAssertions() } // Create latches and merge adjacent. @@ -171,9 +188,7 @@ func GC( newThreshold := oldThreshold updated := newThreshold.Forward(args.Threshold) - // Don't write the GC threshold key unless we have to. We also don't - // declare the key unless we have to (to allow the MVCC GC queue to - // batch requests more efficiently), and we must honor what we declare. + // Don't write the GC threshold key unless we have to. if updated { if err := MakeStateLoader(cArgs.EvalCtx).SetGCThreshold( ctx, readWriter, cArgs.Stats, &newThreshold, diff --git a/pkg/kv/kvserver/batcheval/cmd_recompute_stats.go b/pkg/kv/kvserver/batcheval/cmd_recompute_stats.go index 9a457decb2a6..5146476b67ab 100644 --- a/pkg/kv/kvserver/batcheval/cmd_recompute_stats.go +++ b/pkg/kv/kvserver/batcheval/cmd_recompute_stats.go @@ -53,6 +53,8 @@ func declareKeysRecomputeStats( rdKey := keys.RangeDescriptorKey(rs.GetStartKey()) latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{Key: rdKey}) latchSpans.AddNonMVCC(spanset.SpanReadWrite, roachpb.Span{Key: keys.TransactionKey(rdKey, uuid.Nil)}) + // Disable the assertions which check that all reads were previously declared. + latchSpans.DisableUndeclaredAccessAssertions() } // RecomputeStats recomputes the MVCCStats stored for this range and adjust them accordingly, @@ -69,10 +71,6 @@ func RecomputeStats( args = nil // avoid accidental use below - // Disable the assertions which check that all reads were previously declared. - // See the comment in `declareKeysRecomputeStats` for details on this. - reader = spanset.DisableReaderAssertions(reader) - actualMS, err := rditer.ComputeStatsForRange(desc, reader, cArgs.Header.Timestamp.WallTime) if err != nil { return result.Result{}, err diff --git a/pkg/kv/kvserver/replica_test.go b/pkg/kv/kvserver/replica_test.go index 4e725ce19c97..bba05052b0f0 100644 --- a/pkg/kv/kvserver/replica_test.go +++ b/pkg/kv/kvserver/replica_test.go @@ -1721,6 +1721,14 @@ func gcKey(key roachpb.Key, timestamp hlc.Timestamp) roachpb.GCRequest_GCKey { } } +func recomputeStatsArgs(key roachpb.Key) roachpb.RecomputeStatsRequest { + return roachpb.RecomputeStatsRequest{ + RequestHeader: roachpb.RequestHeader{ + Key: key, + }, + } +} + func gcArgs(startKey []byte, endKey []byte, keys ...roachpb.GCRequest_GCKey) roachpb.GCRequest { return roachpb.GCRequest{ RequestHeader: roachpb.RequestHeader{ @@ -8392,56 +8400,6 @@ func TestReplicaReproposalWithNewLeaseIndexError(t *testing.T) { } } -// TestGCWithoutThreshold validates that GCRequest only declares the threshold -// key if it is subject to change, and that it does not access this key if it -// does not declare them. -func TestGCWithoutThreshold(t *testing.T) { - defer leaktest.AfterTest(t)() - defer log.Scope(t).Close(t) - - ctx := context.Background() - tc := &testContext{} - stopper := stop.NewStopper() - defer stopper.Stop(ctx) - tc.Start(ctx, t, stopper) - - for _, keyThresh := range []hlc.Timestamp{{}, {Logical: 1}} { - t.Run(fmt.Sprintf("thresh=%s", keyThresh), func(t *testing.T) { - var gc roachpb.GCRequest - var spans spanset.SpanSet - - gc.Threshold = keyThresh - cmd, _ := batcheval.LookupCommand(roachpb.GC) - cmd.DeclareKeys(tc.repl.Desc(), &roachpb.Header{RangeID: tc.repl.RangeID}, &gc, &spans, nil, 0) - - expSpans := 1 - if !keyThresh.IsEmpty() { - expSpans++ - } - if numSpans := spans.Len(); numSpans != expSpans { - t.Fatalf("expected %d declared keys, found %d", expSpans, numSpans) - } - - eng := storage.NewDefaultInMemForTesting() - defer eng.Close() - - batch := eng.NewBatch() - defer batch.Close() - rw := spanset.NewBatch(batch, &spans) - - var resp roachpb.GCResponse - if _, err := batcheval.GC(ctx, rw, batcheval.CommandArgs{ - Args: &gc, - EvalCtx: NewReplicaEvalContext( - ctx, tc.repl, &spans, false, /* requiresClosedTSOlderThanStorageSnap */ - ), - }, &resp); err != nil { - t.Fatal(err) - } - }) - } -} - // Test that, if the Raft command resulting from EndTxn request fails to be // processed/apply, then the LocalResult associated with that command is // cleared. @@ -8512,6 +8470,78 @@ func TestFailureToProcessCommandClearsLocalResult(t *testing.T) { } } +// TestMVCCStatsGCCommutesWithWrites tests that the MVCCStats updates +// corresponding to writes and GCs are commutative. +// +// This test does so by: +// 1. Initially writing N versions of a key. +// 2. Concurrently GC-ing the N-1 versions written in step 1 while writing N-1 +// new versions of the key. +// 3. Concurrently recomputing MVCC stats (via RecomputeStatsRequests) in the +// background and ensuring that the stats are always consistent at all times. +func TestMVCCStatsGCCommutesWithWrites(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + ctx := context.Background() + tc := serverutils.StartNewTestCluster(t, 1, base.TestClusterArgs{}) + defer tc.Stopper().Stop(ctx) + key := tc.ScratchRange(t) + store, err := tc.Server(0).GetStores().(*Stores).GetStore(tc.Server(0).GetFirstStoreID()) + require.NoError(t, err) + + write := func() hlc.Timestamp { + var ba roachpb.BatchRequest + put := putArgs(key, []byte("0")) + ba.Add(&put) + resp, pErr := store.TestSender().Send(ctx, ba) + require.Nil(t, pErr) + return resp.Timestamp + } + + // Write `numIterations` versions for a key. + const numIterations = 100 + writeTimestamps := make([]hlc.Timestamp, 0, numIterations) + for i := 0; i < numIterations; i++ { + writeTimestamps = append(writeTimestamps, write()) + } + + // Now, we GC the first `numIterations-1` versions we wrote above while + // concurrently writing `numIterations-1` new versions. + var wg sync.WaitGroup + wg.Add(3) + go func() { + defer wg.Done() + for _, ts := range writeTimestamps[:numIterations-1] { + gcReq := gcArgs(key, key.Next(), gcKey(key, ts)) + _, pErr := kv.SendWrapped(ctx, store.TestSender(), &gcReq) + require.Nil(t, pErr) + } + }() + go func() { + defer wg.Done() + for i := 0; i < numIterations-1; i++ { + write() + } + }() + // Also concurrently recompute stats and ensure that they're consistent at all + // times. + go func() { + defer wg.Done() + expDelta := enginepb.MVCCStats{} + for i := 0; i < numIterations; i++ { + recomputeReq := recomputeStatsArgs(key) + resp, pErr := kv.SendWrapped(ctx, store.TestSender(), &recomputeReq) + require.Nil(t, pErr) + delta := enginepb.MVCCStats(resp.(*roachpb.RecomputeStatsResponse).AddedDelta) + delta.AgeTo(expDelta.LastUpdateNanos) + require.Equal(t, expDelta, delta) + } + }() + + wg.Wait() +} + // TestBatchTimestampBelowGCThreshold verifies that commands below the replica // GC threshold fail. func TestBatchTimestampBelowGCThreshold(t *testing.T) { @@ -8778,6 +8808,18 @@ func TestGCThresholdRacesWithRead(t *testing.T) { require.Nil(t, err) require.Equal(t, va, b) + // Since the GC request does not acquire latches on the keys being GC'ed, + // they're not guaranteed to wait for these above Puts to get applied on + // the leaseholder. See AckCommittedEntriesBeforeApplication() and the the + // comment above it for more details. So we separately ensure both these + // Puts have been applied by just trying to read the latest value @ ts2. + // These Get requests do indeed declare latches on the keys being read, so + // by the time they return, subsequent GC requests are guaranteed to see + // the latest keys. + gArgs = getArgs(key) + _, pErr = kv.SendWrappedWith(ctx, reader, h2, &gArgs) + require.Nil(t, pErr) + // Perform two actions concurrently: // 1. GC up to ts2. This should remove the k@ts1 version. // 2. Read @ ts1. diff --git a/pkg/kv/kvserver/spanset/spanset.go b/pkg/kv/kvserver/spanset/spanset.go index fdefe74434e8..ff98215d4bc3 100644 --- a/pkg/kv/kvserver/spanset/spanset.go +++ b/pkg/kv/kvserver/spanset/spanset.go @@ -82,7 +82,8 @@ type Span struct { // The Span slice for a particular access and scope contains non-overlapping // spans in increasing key order after calls to SortAndDedup. type SpanSet struct { - spans [NumSpanAccess][NumSpanScope][]Span + spans [NumSpanAccess][NumSpanScope][]Span + allowUndeclared bool } var spanSetPool = sync.Pool{ @@ -111,6 +112,7 @@ func (s *SpanSet) Release() { s.spans[sa][ss] = recycle } } + s.allowUndeclared = false spanSetPool.Put(s) } @@ -152,6 +154,7 @@ func (s *SpanSet) Copy() *SpanSet { n.spans[sa][ss] = append(n.spans[sa][ss], s.spans[sa][ss]...) } } + n.allowUndeclared = s.allowUndeclared return n } @@ -204,6 +207,7 @@ func (s *SpanSet) Merge(s2 *SpanSet) { s.spans[sa][ss] = append(s.spans[sa][ss], s2.spans[sa][ss]...) } } + s.allowUndeclared = s2.allowUndeclared s.SortAndDedup() } @@ -335,6 +339,12 @@ func (s *SpanSet) CheckAllowedAt( func (s *SpanSet) checkAllowed( access SpanAccess, span roachpb.Span, check func(SpanAccess, Span) bool, ) error { + if s.allowUndeclared { + // If the request has specified that undeclared spans are allowed, do + // nothing. + return nil + } + scope := SpanGlobal if (span.Key != nil && keys.IsLocal(span.Key)) || (span.EndKey != nil && keys.IsLocal(span.EndKey)) { @@ -387,3 +397,10 @@ func (s *SpanSet) Validate() error { return nil } + +// DisableUndeclaredAccessAssertions disables the assertions that prevent +// undeclared access to spans. This is generally set by requests that rely on +// other forms of synchronization for correctness (e.g. GCRequest). +func (s *SpanSet) DisableUndeclaredAccessAssertions() { + s.allowUndeclared = true +} From 3e0270a6ccda4ddc1c63c04f3ebb58c19e7068b1 Mon Sep 17 00:00:00 2001 From: Aayush Shah Date: Thu, 21 Jul 2022 12:42:41 -0400 Subject: [PATCH 7/7] kvserver: add microbenchmark demonstrating latchless MVCC GC improvement MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This commit adds a couple simple microbenchmarks that demonstrate the improvement in the level of foreground traffic disruption caused by MVCC GC after https://github.com/cockroachdb/cockroach/pull/83213. Results: ``` > benchdiff --old=beforeLatchlessGC --new=latchlessMVCCGC --post-checkout='make buildshort' ./pkg/kv/kvserver --run=MVCCGCWithForegroundTraffic --count=20; pkg=1/1 iter=10/10 cockroachdb/cockroach/pkg/kv/kvserver - name old time/op new time/op delta MVCCGCWithForegroundTraffic/gc_with_reads-24 1.67ms ±18% 0.02ms ± 6% -98.74% (p=0.000 n=10+10) MVCCGCWithForegroundTraffic/gc_with_reads_and_writes-24 1.10ms ±19% 0.57ms ± 4% -48.45% (p=0.000 n=10+10) MVCCGCWithForegroundTraffic/noop_gc_with_reads-24 12.3µs ± 2% 7.3µs ± 1% -40.33% (p=0.000 n=10+9) name old alloc/op new alloc/op delta MVCCGCWithForegroundTraffic/gc_with_reads-24 17.3kB ±11% 1.7kB ± 4% -90.44% (p=0.000 n=10+10) MVCCGCWithForegroundTraffic/gc_with_reads_and_writes-24 37.5kB ± 5% 24.4kB ± 3% -34.92% (p=0.000 n=10+10) MVCCGCWithForegroundTraffic/noop_gc_with_reads-24 1.26kB ± 0% 1.25kB ± 1% -0.43% (p=0.001 n=10+10) name old allocs/op new allocs/op delta MVCCGCWithForegroundTraffic/gc_with_reads-24 111 ± 1% 32 ± 0% -71.08% (p=0.000 n=9+10) MVCCGCWithForegroundTraffic/gc_with_reads_and_writes-24 190 ± 7% 128 ± 2% -32.37% (p=0.000 n=10+9) MVCCGCWithForegroundTraffic/noop_gc_with_reads-24 15.0 ± 0% 15.0 ± 0% ~ (all equal) ``` Release note: None --- pkg/kv/kvserver/replica_test.go | 150 ++++++++++++++++++++++++++++++++ 1 file changed, 150 insertions(+) diff --git a/pkg/kv/kvserver/replica_test.go b/pkg/kv/kvserver/replica_test.go index bba05052b0f0..99d7057dab44 100644 --- a/pkg/kv/kvserver/replica_test.go +++ b/pkg/kv/kvserver/replica_test.go @@ -8870,6 +8870,156 @@ func TestGCThresholdRacesWithRead(t *testing.T) { }) } +// BenchmarkMVCCGCWithForegroundTraffic benchmarks performing GC of a key +// concurrently with reads on that key. +func BenchmarkMVCCGCWithForegroundTraffic(b *testing.B) { + defer leaktest.AfterTest(b)() + defer log.Scope(b).Close(b) + + ctx := context.Background() + tc := testContext{} + stopper := stop.NewStopper() + defer stopper.Stop(ctx) + tc.Start(ctx, b, stopper) + + key := roachpb.Key("test") + now := func() hlc.Timestamp { return hlc.Timestamp{WallTime: timeutil.Now().UnixNano()} } + + // send sends the Request with a present-time batch timestamp. + send := func(args roachpb.Request) *roachpb.BatchResponse { + var header roachpb.Header + header.Timestamp = now() + ba := roachpb.BatchRequest{} + ba.Header = header + ba.Add(args) + resp, err := tc.Sender().Send(ctx, ba) + require.Nil(b, err) + return resp + } + + // gc issues a GC request to garbage collect `key` at `timestamp` with a + // present-time batch header timestamp. + gc := func(key roachpb.Key, timestamp hlc.Timestamp) { + // Note that we're not bumping the GC threshold, just GC'ing the keys. + gcReq := gcArgs(key, key.Next(), gcKey(key, timestamp)) + send(&gcReq) + } + + // read issues a present time read over `key`. + read := func() { + send(scanArgs(key, key.Next())) + } + + // put issues a present time put over `key`. + put := func(key roachpb.Key) (writeTS hlc.Timestamp) { + putReq := putArgs(key, []byte("00")) + resp := send(&putReq) + return resp.Timestamp + } + + // Issue no-op GC requests every 10 microseconds while reads are being + // benchmarked. + b.Run("noop gc with reads", func(b *testing.B) { + var wg sync.WaitGroup + wg.Add(2) + doneCh := make(chan struct{}, 1) + + b.ResetTimer() + go func() { + defer wg.Done() + for { + gc(key, now()) // NB: These are no-op GC requests. + time.Sleep(10 * time.Microsecond) + + select { + case <-doneCh: + return + default: + } + } + }() + + go func() { + for i := 0; i < b.N; i++ { + read() + } + close(doneCh) + wg.Done() + }() + wg.Wait() + }) + + // Write and GC the same key indefinitely while benchmarking read performance. + b.Run("gc with reads and writes", func(b *testing.B) { + var wg sync.WaitGroup + wg.Add(2) + doneCh := make(chan struct{}, 1) + lastWriteTS := put(key) + + b.ResetTimer() + go func() { + defer wg.Done() + for { + // Write a new version and immediately GC the previous version. + writeTS := put(key) + gc(key, lastWriteTS) + lastWriteTS = writeTS + + select { + case <-doneCh: + return + default: + } + } + }() + go func() { + defer wg.Done() + for i := 0; i < b.N; i++ { + read() + } + close(doneCh) + }() + wg.Wait() + }) + + // Write a bunch of versions of a key. Then, GC them while concurrently + // reading those keys. + b.Run("gc with reads", func(b *testing.B) { + var wg sync.WaitGroup + wg.Add(2) + doneCh := make(chan struct{}, 1) + + writeTimestamps := make([]hlc.Timestamp, 0, b.N) + for i := 0; i < b.N; i++ { + writeTimestamps = append(writeTimestamps, put(key)) + } + put(key) + + b.ResetTimer() + go func() { + defer wg.Done() + for _, ts := range writeTimestamps { + gc(key, ts) + + // Stop GC-ing once the reads are done and we're shutting down. + select { + case <-doneCh: + return + default: + } + } + }() + go func() { + defer wg.Done() + for i := 0; i < b.N; i++ { + read() + } + close(doneCh) + }() + wg.Wait() + }) +} + func TestReplicaTimestampCacheBumpNotLost(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t)