-
Notifications
You must be signed in to change notification settings - Fork 3.8k
/
inverted_joiner.go
762 lines (698 loc) · 26.1 KB
/
inverted_joiner.go
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
// Copyright 2020 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
package rowexec
import (
"context"
"fmt"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc"
"github.com/cockroachdb/cockroach/pkg/sql/execinfra"
"github.com/cockroachdb/cockroach/pkg/sql/execinfrapb"
"github.com/cockroachdb/cockroach/pkg/sql/opt/invertedexpr"
"github.com/cockroachdb/cockroach/pkg/sql/opt/invertedidx"
"github.com/cockroachdb/cockroach/pkg/sql/row"
"github.com/cockroachdb/cockroach/pkg/sql/rowcontainer"
"github.com/cockroachdb/cockroach/pkg/sql/rowenc"
"github.com/cockroachdb/cockroach/pkg/sql/scrub"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/span"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/humanizeutil"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/mon"
"github.com/cockroachdb/cockroach/pkg/util/tracing"
"github.com/cockroachdb/errors"
"github.com/opentracing/opentracing-go"
)
// TODO(sumeer): adjust this batch size dynamically to balance between the
// higher scan throughput of larger batches and the cost of spilling the
// scanned rows to disk. The spilling cost will probably be dominated by
// the de-duping cost, since it incurs a read.
const invertedJoinerBatchSize = 100
// invertedJoinerState represents the state of the processor.
type invertedJoinerState int
const (
ijStateUnknown invertedJoinerState = iota
// ijReadingInput means that a batch of rows is being read from the input.
ijReadingInput
// ijPerformingIndexScan means it is performing an inverted index scan
// for the current input row batch.
ijPerformingIndexScan
// ijEmittingRows means it is emitting the results of the inverted join.
ijEmittingRows
)
type invertedJoiner struct {
execinfra.ProcessorBase
runningState invertedJoinerState
diskMonitor *mon.BytesMonitor
desc tabledesc.Immutable
// The map from ColumnIDs in the table to the column position.
colIdxMap map[descpb.ColumnID]int
index *descpb.IndexDescriptor
// The ColumnID of the inverted column. Confusingly, this is also the id of
// the table column that was indexed.
invertedColID descpb.ColumnID
onExprHelper execinfrapb.ExprHelper
combinedRow rowenc.EncDatumRow
joinType descpb.JoinType
// fetcher wraps the row.Fetcher used to perform scans. This enables the
// invertedJoiner to wrap the fetcher with a stat collector when necessary.
fetcher rowFetcher
alloc rowenc.DatumAlloc
rowAlloc rowenc.EncDatumRowAlloc
// The row retrieved from the index represents the columns of the table
// with the datums corresponding to the columns in the index populated.
// The inverted column is in the position colIdxMap[invertedColID] and
// the []byte stored there is used as the first parameter in
// batchedExprEvaluator.addIndexRow(enc, keyIndex).
//
// The remaining columns in the index represent the primary key of the
// table. They are at positions described by the keys in the
// tableRowToKeyRowMap. The map is used to transform the retrieved table row
// to the keyRow, and add to the row container, which de-duplicates the
// primary keys. The index assigned by the container is the keyIndex in the
// addIndexRow() call mentioned earlier.
keyRow rowenc.EncDatumRow
keyTypes []*types.T
tableRowToKeyRowMap map[int]int
// The reverse transformation, from a key row to a table row, is done
// before evaluating the onExpr.
tableRow rowenc.EncDatumRow
keyRowToTableRowMap []int
// The input being joined using the index.
input execinfra.RowSource
inputTypes []*types.T
datumsToInvertedExpr invertedexpr.DatumsToInvertedExpr
canPreFilter bool
// Batch size for fetches. Not a constant so we can lower for testing.
batchSize int
// State variables for each batch of input rows.
inputRows rowenc.EncDatumRows
batchedExprEval batchedInvertedExprEvaluator
// The row indexes that are the result of the inverted expression evaluation
// of the join. These will be further filtered using the onExpr.
joinedRowIdx [][]KeyIndex
// The container for the primary key rows retrieved from the index. For
// evaluating each inverted expression, which involved set unions and
// intersections, it is necessary to de-duplicate the primary key rows
// retrieved from the inverted index. Instead of doing such de-duplication
// for each expression in the batch of expressions, it is done once when
// adding to keyRows -- this is more efficient since multiple expressions
// may be using the same spans from the index.
keyRows *rowcontainer.DiskBackedNumberedRowContainer
// emitCursor contains information about where the next row to emit is within
// joinedRowIdx.
emitCursor struct {
// inputRowIdx corresponds to joinedRowIdx[inputRowIdx].
inputRowIdx int
// outputRowIdx corresponds to joinedRowIdx[inputRowIdx][outputRowIdx].
outputRowIdx int
// seenMatch is true if there was a match at the current inputRowIdx.
seenMatch bool
}
spanBuilder *span.Builder
// A row with one element, corresponding to an encoded inverted column
// value. Used to construct the span of the index for that value.
invertedColRow rowenc.EncDatumRow
outputContinuationCol bool
trueEncDatum rowenc.EncDatum
falseEncDatum rowenc.EncDatum
}
var _ execinfra.Processor = &invertedJoiner{}
var _ execinfra.RowSource = &invertedJoiner{}
var _ execinfrapb.MetadataSource = &invertedJoiner{}
var _ execinfra.OpNode = &invertedJoiner{}
const invertedJoinerProcName = "inverted joiner"
// newInvertedJoiner constructs an invertedJoiner. The datumsToInvertedExpr
// argument is non-nil only for tests. When nil, the invertedJoiner uses
// the spec to construct an implementation of DatumsToInvertedExpr.
func newInvertedJoiner(
flowCtx *execinfra.FlowCtx,
processorID int32,
spec *execinfrapb.InvertedJoinerSpec,
datumsToInvertedExpr invertedexpr.DatumsToInvertedExpr,
input execinfra.RowSource,
post *execinfrapb.PostProcessSpec,
output execinfra.RowReceiver,
) (execinfra.RowSourcedProcessor, error) {
ij := &invertedJoiner{
desc: tabledesc.MakeImmutable(spec.Table),
input: input,
inputTypes: input.OutputTypes(),
datumsToInvertedExpr: datumsToInvertedExpr,
joinType: spec.Type,
batchSize: invertedJoinerBatchSize,
}
ij.colIdxMap = ij.desc.ColumnIdxMap()
var err error
ij.index, _, err = ij.desc.FindIndexByIndexIdx(int(spec.IndexIdx))
if err != nil {
return nil, err
}
ij.invertedColID = ij.index.ColumnIDs[0]
indexColumnIDs, _ := ij.index.FullColumnIDs()
// Inverted joins are not used for mutations.
tableColumns := ij.desc.ColumnsWithMutations(false /* mutations */)
ij.keyRow = make(rowenc.EncDatumRow, len(indexColumnIDs)-1)
ij.keyTypes = make([]*types.T, len(ij.keyRow))
ij.tableRow = make(rowenc.EncDatumRow, len(tableColumns))
ij.tableRowToKeyRowMap = make(map[int]int)
ij.keyRowToTableRowMap = make([]int, len(indexColumnIDs)-1)
for i := 1; i < len(indexColumnIDs); i++ {
keyRowIdx := i - 1
tableRowIdx := ij.colIdxMap[indexColumnIDs[i]]
ij.tableRowToKeyRowMap[tableRowIdx] = keyRowIdx
ij.keyRowToTableRowMap[keyRowIdx] = tableRowIdx
ij.keyTypes[keyRowIdx] = ij.desc.Columns[tableRowIdx].Type
}
outputColCount := len(ij.inputTypes)
// Inverted joins are not used for mutations.
rightColTypes := ij.desc.ColumnTypesWithMutations(false /* mutations */)
var includeRightCols bool
if ij.joinType == descpb.InnerJoin || ij.joinType == descpb.LeftOuterJoin {
outputColCount += len(rightColTypes)
includeRightCols = true
}
outputColTypes := make([]*types.T, 0, outputColCount)
outputColTypes = append(outputColTypes, ij.inputTypes...)
if includeRightCols {
outputColTypes = append(outputColTypes, rightColTypes...)
}
if err := ij.ProcessorBase.Init(
ij, post, outputColTypes, flowCtx, processorID, output, nil, /* memMonitor */
execinfra.ProcStateOpts{
InputsToDrain: []execinfra.RowSource{ij.input},
TrailingMetaCallback: func(ctx context.Context) []execinfrapb.ProducerMetadata {
ij.close()
return ij.generateMeta(ctx)
},
},
); err != nil {
return nil, err
}
semaCtx := flowCtx.TypeResolverFactory.NewSemaContext(flowCtx.EvalCtx.Txn)
onExprColTypes := make([]*types.T, 0, len(ij.inputTypes)+len(rightColTypes))
onExprColTypes = append(onExprColTypes, ij.inputTypes...)
onExprColTypes = append(onExprColTypes, rightColTypes...)
if err := ij.onExprHelper.Init(spec.OnExpr, onExprColTypes, semaCtx, ij.EvalCtx); err != nil {
return nil, err
}
ij.combinedRow = make(rowenc.EncDatumRow, 0, len(onExprColTypes))
if ij.datumsToInvertedExpr == nil {
var invertedExprHelper execinfrapb.ExprHelper
if err := invertedExprHelper.Init(spec.InvertedExpr, onExprColTypes, semaCtx, ij.EvalCtx); err != nil {
return nil, err
}
ij.datumsToInvertedExpr, err = invertedidx.NewDatumsToInvertedExpr(
ij.EvalCtx, onExprColTypes, invertedExprHelper.Expr, ij.index,
)
if err != nil {
return nil, err
}
}
ij.canPreFilter = ij.datumsToInvertedExpr.CanPreFilter()
if ij.canPreFilter {
ij.batchedExprEval.filterer = ij.datumsToInvertedExpr
}
var fetcher row.Fetcher
// In general we need all the columns in the index to compute the set
// expression. There may be InvertedJoinerSpec.InvertedExpr that are known
// to generate only set union expressions, which together with LEFT_SEMI and
// LEFT_ANTI, and knowledge of the columns needed by
// InvertedJoinerSpec.OnExpr, could be used to prune the columns needed
// here. For now, we do the simple thing, since we have no idea whether
// such workloads actually occur in practice.
allIndexCols := util.MakeFastIntSet()
for _, colID := range indexColumnIDs {
allIndexCols.Add(ij.colIdxMap[colID])
}
// We use ScanVisibilityPublic since inverted joins are not used for mutations,
// and so do not need to see in-progress schema changes.
_, _, err = initRowFetcher(
flowCtx, &fetcher, &ij.desc, int(spec.IndexIdx), ij.colIdxMap, false, /* reverse */
allIndexCols, false /* isCheck */, flowCtx.EvalCtx.Mon, &ij.alloc, execinfra.ScanVisibilityPublic,
descpb.ScanLockingStrength_FOR_NONE, descpb.ScanLockingWaitPolicy_BLOCK,
nil, /* systemColumns */
)
if err != nil {
return nil, err
}
collectingStats := false
if sp := opentracing.SpanFromContext(flowCtx.EvalCtx.Ctx()); sp != nil && tracing.IsRecording(sp) {
collectingStats = true
}
if collectingStats {
ij.input = newInputStatCollector(ij.input)
ij.fetcher = newRowFetcherStatCollector(&fetcher)
ij.FinishTrace = ij.outputStatsToTrace
} else {
ij.fetcher = &fetcher
}
ij.spanBuilder = span.MakeBuilder(flowCtx.Codec(), &ij.desc, ij.index)
ij.spanBuilder.SetNeededColumns(allIndexCols)
// Initialize memory monitors and row container for key rows.
ctx := flowCtx.EvalCtx.Ctx()
ij.MemMonitor = execinfra.NewLimitedMonitor(ctx, flowCtx.EvalCtx.Mon, flowCtx.Cfg, "invertedjoiner-limited")
ij.diskMonitor = execinfra.NewMonitor(ctx, flowCtx.Cfg.DiskMonitor, "invertedjoiner-disk")
ij.keyRows = rowcontainer.NewDiskBackedNumberedRowContainer(
true, /* deDup */
ij.keyTypes,
ij.EvalCtx,
ij.FlowCtx.Cfg.TempStorage,
ij.MemMonitor,
ij.diskMonitor,
)
if spec.OutputContinuationForLeftJoin {
ij.outputContinuationCol = true
ij.trueEncDatum = rowenc.DatumToEncDatum(types.Bool, tree.DBoolTrue)
ij.falseEncDatum = rowenc.DatumToEncDatum(types.Bool, tree.DBoolFalse)
}
return ij, nil
}
// SetBatchSize sets the desired batch size. It should only be used in tests.
func (ij *invertedJoiner) SetBatchSize(batchSize int) {
ij.batchSize = batchSize
}
func (ij *invertedJoiner) generateSpan(enc []byte) (roachpb.Span, error) {
// Pretend that the encoded inverted val is an EncDatum. This isn't always
// true, since JSON inverted columns use a custom encoding. But since we
// are providing an already encoded Datum, the following will eventually
// fall through to EncDatum.Encode() which will reuse the encoded bytes.
encDatum := rowenc.EncDatumFromEncoded(descpb.DatumEncoding_ASCENDING_KEY, enc)
ij.invertedColRow = append(ij.invertedColRow[:0], encDatum)
span, _, err := ij.spanBuilder.SpanFromEncDatums(ij.invertedColRow, 1 /* prefixLen */)
return span, err
}
func (ij *invertedJoiner) generateSpans(invertedSpans []invertedSpan) ([]roachpb.Span, error) {
spans := make([]roachpb.Span, len(invertedSpans))
for i, span := range invertedSpans {
startSpan, err := ij.generateSpan(span.Start)
if err != nil {
return nil, err
}
endSpan, err := ij.generateSpan(span.End)
if err != nil {
return nil, err
}
startSpan.EndKey = endSpan.Key
spans[i] = startSpan
}
return spans, nil
}
// Next is part of the RowSource interface.
func (ij *invertedJoiner) Next() (rowenc.EncDatumRow, *execinfrapb.ProducerMetadata) {
// The join is implemented as follows:
// - Read the input rows in batches.
// - For each batch, map the rows to SpanExpressionProtos and initialize
// a batchedInvertedExprEvaluator. Use that evaluator to generate spans
// to read from the inverted index.
// - Retrieve the index rows and add the primary keys in these rows to the
// row container, that de-duplicates, and pass the de-duplicated keys to
// the batch evaluator.
// - Retrieve the results from the batch evaluator and buffer in joinedRowIdx,
// and use the emitCursor to emit rows.
for ij.State == execinfra.StateRunning {
var row rowenc.EncDatumRow
var meta *execinfrapb.ProducerMetadata
switch ij.runningState {
case ijReadingInput:
ij.runningState, meta = ij.readInput()
case ijPerformingIndexScan:
ij.runningState, meta = ij.performScan()
case ijEmittingRows:
ij.runningState, row, meta = ij.emitRow()
default:
log.Fatalf(ij.Ctx, "unsupported state: %d", ij.runningState)
}
if row == nil && meta == nil {
continue
}
if meta != nil {
return nil, meta
}
if outRow := ij.ProcessRowHelper(row); outRow != nil {
return outRow, nil
}
}
return nil, ij.DrainHelper()
}
// readInput reads the next batch of input rows and starts an index scan.
func (ij *invertedJoiner) readInput() (invertedJoinerState, *execinfrapb.ProducerMetadata) {
// Read the next batch of input rows.
for len(ij.inputRows) < ij.batchSize {
row, meta := ij.input.Next()
if meta != nil {
if meta.Err != nil {
ij.MoveToDraining(nil /* err */)
return ijStateUnknown, meta
}
return ijReadingInput, meta
}
if row == nil {
break
}
expr, preFilterState, err := ij.datumsToInvertedExpr.Convert(ij.Ctx, row)
if err != nil {
ij.MoveToDraining(err)
return ijStateUnknown, ij.DrainHelper()
}
if expr == nil &&
(ij.joinType != descpb.LeftOuterJoin && ij.joinType != descpb.LeftAntiJoin) {
// One of the input columns was NULL, resulting in a nil expression.
// The join type will emit no row since the evaluation result will be
// an empty set, so don't bother copying the input row.
ij.inputRows = append(ij.inputRows, nil)
} else {
ij.inputRows = append(ij.inputRows, ij.rowAlloc.CopyRow(row))
}
if expr == nil {
// One of the input columns was NULL, resulting in a nil expression.
// The nil serves as a marker that will result in an empty set as the
// evaluation result.
ij.batchedExprEval.exprs = append(ij.batchedExprEval.exprs, nil)
if ij.canPreFilter {
ij.batchedExprEval.preFilterState = append(ij.batchedExprEval.preFilterState, nil)
}
} else {
ij.batchedExprEval.exprs = append(ij.batchedExprEval.exprs, expr)
if ij.canPreFilter {
ij.batchedExprEval.preFilterState = append(ij.batchedExprEval.preFilterState, preFilterState)
}
}
}
if len(ij.inputRows) == 0 {
log.VEventf(ij.Ctx, 1, "no more input rows")
// We're done.
ij.MoveToDraining(nil)
return ijStateUnknown, ij.DrainHelper()
}
log.VEventf(ij.Ctx, 1, "read %d input rows", len(ij.inputRows))
spans := ij.batchedExprEval.init()
if len(spans) == 0 {
// Nothing to scan. For each input row, place a nil slice in the joined
// rows, for emitRow() to process.
ij.joinedRowIdx = ij.joinedRowIdx[:0]
for range ij.inputRows {
ij.joinedRowIdx = append(ij.joinedRowIdx, nil)
}
return ijEmittingRows, nil
}
// NB: spans is already sorted, and that sorting is preserved when
// generating indexSpans.
indexSpans, err := ij.generateSpans(spans)
if err != nil {
ij.MoveToDraining(err)
return ijStateUnknown, ij.DrainHelper()
}
log.VEventf(ij.Ctx, 1, "scanning %d spans", len(indexSpans))
if err = ij.fetcher.StartScan(
ij.Ctx, ij.FlowCtx.Txn, indexSpans, false /* limitBatches */, 0, /* limitHint */
ij.FlowCtx.TraceKV); err != nil {
ij.MoveToDraining(err)
return ijStateUnknown, ij.DrainHelper()
}
return ijPerformingIndexScan, nil
}
func (ij *invertedJoiner) performScan() (invertedJoinerState, *execinfrapb.ProducerMetadata) {
log.VEventf(ij.Ctx, 1, "joining rows")
// Read the entire set of rows that are part of the scan.
for {
// Fetch the next row and copy it into the row container.
scannedRow, _, _, err := ij.fetcher.NextRow(ij.Ctx)
if err != nil {
ij.MoveToDraining(scrub.UnwrapScrubError(err))
return ijStateUnknown, ij.DrainHelper()
}
if scannedRow == nil {
// Done with this input batch.
break
}
encInvertedVal := scannedRow[ij.colIdxMap[ij.invertedColID]].EncodedBytes()
shouldAdd, err := ij.batchedExprEval.prepareAddIndexRow(encInvertedVal)
if err != nil {
ij.MoveToDraining(err)
return ijStateUnknown, ij.DrainHelper()
}
if shouldAdd {
ij.transformToKeyRow(scannedRow)
rowIdx, err := ij.keyRows.AddRow(ij.Ctx, ij.keyRow)
if err != nil {
ij.MoveToDraining(err)
return ijStateUnknown, ij.DrainHelper()
}
if err = ij.batchedExprEval.addIndexRow(rowIdx); err != nil {
ij.MoveToDraining(err)
return ijStateUnknown, ij.DrainHelper()
}
}
}
ij.joinedRowIdx = ij.batchedExprEval.evaluate()
ij.keyRows.SetupForRead(ij.Ctx, ij.joinedRowIdx)
log.VEventf(ij.Ctx, 1, "done evaluating expressions")
return ijEmittingRows, nil
}
// emitRow returns the next row from ij.emitCursor, if present. Otherwise it
// prepares for another input batch.
func (ij *invertedJoiner) emitRow() (
invertedJoinerState,
rowenc.EncDatumRow,
*execinfrapb.ProducerMetadata,
) {
// Finished processing the batch.
if ij.emitCursor.inputRowIdx >= len(ij.joinedRowIdx) {
log.VEventf(ij.Ctx, 1, "done emitting rows")
// Ready for another input batch. Reset state.
ij.inputRows = ij.inputRows[:0]
ij.batchedExprEval.reset()
ij.joinedRowIdx = nil
ij.emitCursor.outputRowIdx = 0
ij.emitCursor.inputRowIdx = 0
ij.emitCursor.seenMatch = false
if err := ij.keyRows.UnsafeReset(ij.Ctx); err != nil {
ij.MoveToDraining(err)
return ijStateUnknown, nil, ij.DrainHelper()
}
return ijReadingInput, nil, nil
}
// Reached the end of the matches for an input row. May need to emit for
// LeftOuterJoin and LeftAntiJoin.
if ij.emitCursor.outputRowIdx >= len(ij.joinedRowIdx[ij.emitCursor.inputRowIdx]) {
inputRowIdx := ij.emitCursor.inputRowIdx
seenMatch := ij.emitCursor.seenMatch
ij.emitCursor.inputRowIdx++
ij.emitCursor.outputRowIdx = 0
ij.emitCursor.seenMatch = false
if !seenMatch {
switch ij.joinType {
case descpb.LeftOuterJoin:
ij.renderUnmatchedRow(ij.inputRows[inputRowIdx])
if ij.outputContinuationCol {
ij.combinedRow = append(ij.combinedRow, ij.falseEncDatum)
}
return ijEmittingRows, ij.combinedRow, nil
case descpb.LeftAntiJoin:
return ijEmittingRows, ij.inputRows[inputRowIdx], nil
}
}
return ijEmittingRows, nil, nil
}
inputRow := ij.inputRows[ij.emitCursor.inputRowIdx]
joinedRowIdx := ij.joinedRowIdx[ij.emitCursor.inputRowIdx][ij.emitCursor.outputRowIdx]
indexedRow, err := ij.keyRows.GetRow(ij.Ctx, joinedRowIdx, false /* skip */)
if err != nil {
ij.MoveToDraining(err)
return ijStateUnknown, nil, ij.DrainHelper()
}
ij.emitCursor.outputRowIdx++
ij.transformToTableRow(indexedRow)
renderedRow, err := ij.render(inputRow, ij.tableRow)
if err != nil {
ij.MoveToDraining(err)
return ijStateUnknown, nil, ij.DrainHelper()
}
skipRemaining := func() error {
for ; ij.emitCursor.outputRowIdx < len(ij.joinedRowIdx[ij.emitCursor.inputRowIdx]); ij.emitCursor.outputRowIdx++ {
idx := ij.joinedRowIdx[ij.emitCursor.inputRowIdx][ij.emitCursor.outputRowIdx]
if _, err := ij.keyRows.GetRow(ij.Ctx, idx, true /* skip */); err != nil {
return err
}
}
return nil
}
if renderedRow != nil {
firstRowForInputIdx := ij.emitCursor.seenMatch
ij.emitCursor.seenMatch = true
switch ij.joinType {
case descpb.InnerJoin, descpb.LeftOuterJoin:
if ij.outputContinuationCol {
if firstRowForInputIdx {
ij.combinedRow = append(ij.combinedRow, ij.falseEncDatum)
} else {
ij.combinedRow = append(ij.combinedRow, ij.trueEncDatum)
}
renderedRow = ij.combinedRow
}
return ijEmittingRows, renderedRow, nil
case descpb.LeftSemiJoin:
// Skip the rest of the joined rows.
if err := skipRemaining(); err != nil {
ij.MoveToDraining(err)
return ijStateUnknown, nil, ij.DrainHelper()
}
return ijEmittingRows, inputRow, nil
case descpb.LeftAntiJoin:
// Skip the rest of the joined rows.
if err := skipRemaining(); err != nil {
ij.MoveToDraining(err)
return ijStateUnknown, nil, ij.DrainHelper()
}
ij.emitCursor.outputRowIdx = len(ij.joinedRowIdx[ij.emitCursor.inputRowIdx])
}
}
return ijEmittingRows, nil, nil
}
// render constructs a row with columns from both sides. The ON condition is
// evaluated; if it fails, returns nil. When it returns a non-nil row, it is
// identical to ij.combinedRow.
func (ij *invertedJoiner) render(lrow, rrow rowenc.EncDatumRow) (rowenc.EncDatumRow, error) {
ij.combinedRow = append(ij.combinedRow[:0], lrow...)
ij.combinedRow = append(ij.combinedRow, rrow...)
if ij.onExprHelper.Expr != nil {
res, err := ij.onExprHelper.EvalFilter(ij.combinedRow)
if !res || err != nil {
return nil, err
}
}
return ij.combinedRow, nil
}
// renderUnmatchedRow creates a result row given an unmatched row.
func (ij *invertedJoiner) renderUnmatchedRow(row rowenc.EncDatumRow) {
ij.combinedRow = append(ij.combinedRow[:0], row...)
ij.combinedRow = ij.combinedRow[:cap(ij.combinedRow)]
for i := len(row); i < len(ij.combinedRow); i++ {
ij.combinedRow[i].Datum = tree.DNull
}
}
func (ij *invertedJoiner) transformToKeyRow(row rowenc.EncDatumRow) {
for i, rowIdx := range ij.keyRowToTableRowMap {
ij.keyRow[i] = row[rowIdx]
}
}
func (ij *invertedJoiner) transformToTableRow(keyRow rowenc.EncDatumRow) {
for r, k := range ij.tableRowToKeyRowMap {
ij.tableRow[r] = keyRow[k]
}
}
// Start is part of the RowSource interface.
func (ij *invertedJoiner) Start(ctx context.Context) context.Context {
ij.input.Start(ctx)
ctx = ij.StartInternal(ctx, invertedJoinerProcName)
ij.runningState = ijReadingInput
return ctx
}
// ConsumerClosed is part of the RowSource interface.
func (ij *invertedJoiner) ConsumerClosed() {
// The consumer is done, Next() will not be called again.
ij.close()
}
func (ij *invertedJoiner) close() {
if ij.InternalClose() {
if ij.fetcher != nil {
ij.fetcher.Close(ij.Ctx)
}
if ij.keyRows != nil {
ij.keyRows.Close(ij.Ctx)
}
ij.MemMonitor.Stop(ij.Ctx)
if ij.diskMonitor != nil {
ij.diskMonitor.Stop(ij.Ctx)
}
}
}
var _ execinfrapb.DistSQLSpanStats = &InvertedJoinerStats{}
const invertedJoinerTagPrefix = "invertedjoiner."
// Stats implements the SpanStats interface.
func (ijs *InvertedJoinerStats) Stats() map[string]string {
statsMap := ijs.InputStats.Stats(invertedJoinerTagPrefix)
toMerge := ijs.IndexScanStats.Stats(invertedJoinerTagPrefix + "index.")
for k, v := range toMerge {
statsMap[k] = v
}
statsMap[invertedJoinerTagPrefix+MaxMemoryTagSuffix] = humanizeutil.IBytes(ijs.MaxAllocatedMem)
statsMap[invertedJoinerTagPrefix+MaxDiskTagSuffix] = humanizeutil.IBytes(ijs.MaxAllocatedDisk)
return statsMap
}
// StatsForQueryPlan implements the DistSQLSpanStats interface.
func (ijs *InvertedJoinerStats) StatsForQueryPlan() []string {
stats := append(
ijs.InputStats.StatsForQueryPlan(""),
ijs.IndexScanStats.StatsForQueryPlan("index ")...,
)
if ijs.MaxAllocatedMem != 0 {
stats = append(stats,
fmt.Sprintf("%s: %s", MaxMemoryQueryPlanSuffix, humanizeutil.IBytes(ijs.MaxAllocatedMem)))
}
if ijs.MaxAllocatedDisk != 0 {
stats = append(stats,
fmt.Sprintf("%s: %s", MaxDiskQueryPlanSuffix, humanizeutil.IBytes(ijs.MaxAllocatedDisk)))
}
return stats
}
// outputStatsToTrace outputs the collected stats to the trace. Will
// fail silently if the invertedJoiner is not collecting stats.
func (ij *invertedJoiner) outputStatsToTrace() {
is, ok := getInputStats(ij.FlowCtx, ij.input)
if !ok {
return
}
fis, ok := getFetcherInputStats(ij.FlowCtx, ij.fetcher)
if !ok {
return
}
if sp := opentracing.SpanFromContext(ij.Ctx); sp != nil {
tracing.SetSpanStats(
sp,
&InvertedJoinerStats{
InputStats: is,
IndexScanStats: fis,
MaxAllocatedMem: ij.MemMonitor.MaximumBytes(),
MaxAllocatedDisk: ij.diskMonitor.MaximumBytes(),
})
}
}
func (ij *invertedJoiner) generateMeta(ctx context.Context) []execinfrapb.ProducerMetadata {
if tfs := execinfra.GetLeafTxnFinalState(ctx, ij.FlowCtx.Txn); tfs != nil {
return []execinfrapb.ProducerMetadata{{LeafTxnFinalState: tfs}}
}
return nil
}
// DrainMeta is part of the MetadataSource interface.
func (ij *invertedJoiner) DrainMeta(ctx context.Context) []execinfrapb.ProducerMetadata {
return ij.generateMeta(ctx)
}
// ChildCount is part of the execinfra.OpNode interface.
func (ij *invertedJoiner) ChildCount(verbose bool) int {
if _, ok := ij.input.(execinfra.OpNode); ok {
return 1
}
return 0
}
// Child is part of the execinfra.OpNode interface.
func (ij *invertedJoiner) Child(nth int, verbose bool) execinfra.OpNode {
if nth == 0 {
if n, ok := ij.input.(execinfra.OpNode); ok {
return n
}
panic("input to invertedJoiner is not an execinfra.OpNode")
}
panic(errors.AssertionFailedf("invalid index %d", nth))
}