-
Notifications
You must be signed in to change notification settings - Fork 3.8k
/
joinreader.go
927 lines (836 loc) · 30.4 KB
/
joinreader.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
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
// Copyright 2016 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"
"sort"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo"
"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/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/log"
"github.com/cockroachdb/cockroach/pkg/util/mon"
"github.com/cockroachdb/cockroach/pkg/util/tracing"
"github.com/cockroachdb/errors"
)
// joinReaderState represents the state of the processor.
type joinReaderState int
const (
jrStateUnknown joinReaderState = iota
// jrReadingInput means that a batch of rows is being read from the input.
jrReadingInput
// jrPerformingLookup means we are performing an index lookup for the current
// input row batch.
jrPerformingLookup
// jrEmittingRows means we are emitting the results of the index lookup.
jrEmittingRows
// jrReadyToDrain means we are done but have not yet started draining.
jrReadyToDrain
)
// joinReaderType represents the type of join being used.
type joinReaderType int
const (
// lookupJoinReaderType means we are performing a lookup join.
lookupJoinReaderType joinReaderType = iota
// indexJoinReaderType means we are performing an index join.
indexJoinReaderType
)
// joinReader performs a lookup join between `input` and the specified `index`.
// `lookupCols` specifies the input columns which will be used for the index
// lookup.
type joinReader struct {
joinerBase
strategy joinReaderStrategy
// runningState represents the state of the joinReader. This is in addition to
// ProcessorBase.State - the runningState is only relevant when
// ProcessorBase.State == StateRunning.
runningState joinReaderState
diskMonitor *mon.BytesMonitor
desc tabledesc.Immutable
index *descpb.IndexDescriptor
colIdxMap map[descpb.ColumnID]int
maintainOrdering bool
// fetcher wraps the row.Fetcher used to perform lookups. This enables the
// joinReader to wrap the fetcher with a stat collector when necessary.
fetcher rowFetcher
alloc rowenc.DatumAlloc
rowAlloc rowenc.EncDatumRowAlloc
shouldLimitBatches bool
readerType joinReaderType
input execinfra.RowSource
inputTypes []*types.T
// Column indexes in the input stream specifying the columns which match with
// the index columns. These are the equality columns of the join.
lookupCols []uint32
// Batch size for fetches. Not a constant so we can lower for testing.
batchSizeBytes int64
curBatchSizeBytes int64
// rowsRead is the total number of rows that this fetcher read from
// disk.
rowsRead int64
// State variables for each batch of input rows.
scratchInputRows rowenc.EncDatumRows
// Fields used when this is the second join in a pair of joins that are
// together implementing left {outer,semi,anti} joins where the first join
// produces false positives because it cannot evaluate the whole expression
// (or evaluate it accurately, as is sometimes the case with inverted
// indexes). The first join is running a left outer or inner join, and each
// group of rows seen by the second join correspond to one left row.
// The input rows in the current batch belong to groups which are tracked in
// groupingState. The last row from the last batch is in
// lastInputRowFromLastBatch -- it is tracked because we don't know if it
// was the last row in a group until we get to the next batch. NB:
// groupingState is used even when there is no grouping -- we simply have
// groups of one. The no grouping cases include the case of this join being
// the first join in the paired joins.
groupingState *inputBatchGroupingState
lastBatchState struct {
lastInputRow rowenc.EncDatumRow
lastGroupMatched bool
lastGroupContinued bool
}
// Set to true when this is the first join in the paired-joins (see the
// detailed comment in the spec). This can never be true for index joins,
// and requires that the spec has MaintainOrdering set to true.
outputGroupContinuationForLeftRow bool
}
var _ execinfra.Processor = &joinReader{}
var _ execinfra.RowSource = &joinReader{}
var _ execinfrapb.MetadataSource = &joinReader{}
var _ execinfra.OpNode = &joinReader{}
var _ execinfra.IOReader = &joinReader{}
const joinReaderProcName = "join reader"
// newJoinReader returns a new joinReader.
func newJoinReader(
flowCtx *execinfra.FlowCtx,
processorID int32,
spec *execinfrapb.JoinReaderSpec,
input execinfra.RowSource,
post *execinfrapb.PostProcessSpec,
output execinfra.RowReceiver,
readerType joinReaderType,
) (execinfra.RowSourcedProcessor, error) {
if spec.IndexIdx != 0 && readerType == indexJoinReaderType {
return nil, errors.AssertionFailedf("index join must be against primary index")
}
if spec.OutputGroupContinuationForLeftRow && !spec.MaintainOrdering {
return nil, errors.AssertionFailedf(
"lookup join must maintain ordering since it is first join in paired-joins")
}
var lookupCols []uint32
switch readerType {
case indexJoinReaderType:
pkIDs := spec.Table.PrimaryIndex.ColumnIDs
lookupCols = make([]uint32, len(pkIDs))
for i := range pkIDs {
lookupCols[i] = uint32(i)
}
case lookupJoinReaderType:
lookupCols = spec.LookupColumns
default:
return nil, errors.Errorf("unsupported joinReaderType")
}
jr := &joinReader{
desc: tabledesc.MakeImmutable(spec.Table),
maintainOrdering: spec.MaintainOrdering,
input: input,
inputTypes: input.OutputTypes(),
lookupCols: lookupCols,
outputGroupContinuationForLeftRow: spec.OutputGroupContinuationForLeftRow,
}
if readerType != indexJoinReaderType {
jr.groupingState = &inputBatchGroupingState{doGrouping: spec.LeftJoinWithPairedJoiner}
}
var err error
var isSecondary bool
jr.index, isSecondary, err = jr.desc.FindIndexByIndexIdx(int(spec.IndexIdx))
if err != nil {
return nil, err
}
returnMutations := spec.Visibility == execinfra.ScanVisibilityPublicAndNotPublic
jr.colIdxMap = jr.desc.ColumnIdxMapWithMutations(returnMutations)
columnIDs, _ := jr.index.FullColumnIDs()
indexCols := make([]uint32, len(columnIDs))
columnTypes := jr.desc.ColumnTypesWithMutations(returnMutations)
for i, columnID := range columnIDs {
indexCols[i] = uint32(columnID)
}
// If the lookup columns form a key, there is only one result per lookup, so the fetcher
// should parallelize the key lookups it performs.
jr.shouldLimitBatches = !spec.LookupColumnsAreKey && readerType == lookupJoinReaderType
jr.readerType = readerType
// Add all requested system columns to the output.
var sysColDescs []descpb.ColumnDescriptor
if spec.HasSystemColumns {
sysColDescs = colinfo.AllSystemColumnDescs
}
for i := range sysColDescs {
columnTypes = append(columnTypes, sysColDescs[i].Type)
jr.colIdxMap[sysColDescs[i].ID] = len(jr.colIdxMap)
}
var leftTypes []*types.T
var leftEqCols []uint32
switch readerType {
case indexJoinReaderType:
// Index join performs a join between a secondary index, the `input`,
// and the primary index of the same table, `desc`, to retrieve columns
// which are not stored in the secondary index. It outputs the looked
// up rows as is (meaning that the output rows before post-processing
// will contain all columns from the table) whereas the columns that
// came from the secondary index (input rows) are ignored. As a result,
// we leave leftTypes as empty.
leftEqCols = indexCols
case lookupJoinReaderType:
leftTypes = input.OutputTypes()
leftEqCols = jr.lookupCols
default:
return nil, errors.Errorf("unsupported joinReaderType")
}
if err := jr.joinerBase.init(
jr,
flowCtx,
processorID,
leftTypes,
columnTypes,
spec.Type,
spec.OnExpr,
leftEqCols,
indexCols,
spec.OutputGroupContinuationForLeftRow,
post,
output,
execinfra.ProcStateOpts{
InputsToDrain: []execinfra.RowSource{jr.input},
TrailingMetaCallback: func(ctx context.Context) []execinfrapb.ProducerMetadata {
jr.close()
return jr.generateMeta(ctx)
},
},
); err != nil {
return nil, err
}
collectingStats := false
if sp := tracing.SpanFromContext(flowCtx.EvalCtx.Ctx()); sp != nil && sp.IsRecording() {
collectingStats = true
}
neededRightCols := jr.neededRightCols()
if isSecondary && !neededRightCols.SubsetOf(getIndexColSet(jr.index, jr.colIdxMap)) {
return nil, errors.Errorf("joinreader index does not cover all columns")
}
var fetcher row.Fetcher
var rightCols util.FastIntSet
switch readerType {
case indexJoinReaderType:
rightCols = jr.Out.NeededColumns()
case lookupJoinReaderType:
rightCols = neededRightCols
default:
return nil, errors.Errorf("unsupported joinReaderType")
}
_, _, err = initRowFetcher(
flowCtx, &fetcher, &jr.desc, int(spec.IndexIdx), jr.colIdxMap, false, /* reverse */
rightCols, false /* isCheck */, jr.EvalCtx.Mon, &jr.alloc, spec.Visibility, spec.LockingStrength,
spec.LockingWaitPolicy, sysColDescs,
)
if err != nil {
return nil, err
}
if collectingStats {
jr.input = newInputStatCollector(jr.input)
jr.fetcher = newRowFetcherStatCollector(&fetcher)
jr.FinishTrace = jr.outputStatsToTrace
} else {
jr.fetcher = &fetcher
}
jr.initJoinReaderStrategy(flowCtx, columnTypes, len(columnIDs), rightCols, readerType)
jr.batchSizeBytes = jr.strategy.getLookupRowsBatchSizeHint()
// TODO(radu): verify the input types match the index key types
return jr, nil
}
func (jr *joinReader) initJoinReaderStrategy(
flowCtx *execinfra.FlowCtx,
typs []*types.T,
numKeyCols int,
neededRightCols util.FastIntSet,
readerType joinReaderType,
) {
spanBuilder := span.MakeBuilder(flowCtx.Codec(), &jr.desc, jr.index)
spanBuilder.SetNeededColumns(neededRightCols)
var keyToInputRowIndices map[string][]int
if readerType != indexJoinReaderType {
keyToInputRowIndices = make(map[string][]int)
}
// Else: see the comment in defaultSpanGenerator on why we don't need
// this map for index joins.
spanGenerator := defaultSpanGenerator{
spanBuilder: spanBuilder,
keyToInputRowIndices: keyToInputRowIndices,
numKeyCols: numKeyCols,
lookupCols: jr.lookupCols,
}
if readerType == indexJoinReaderType {
jr.strategy = &joinReaderIndexJoinStrategy{
joinerBase: &jr.joinerBase,
defaultSpanGenerator: spanGenerator,
}
return
}
if !jr.maintainOrdering {
jr.strategy = &joinReaderNoOrderingStrategy{
joinerBase: &jr.joinerBase,
defaultSpanGenerator: spanGenerator,
isPartialJoin: jr.joinType == descpb.LeftSemiJoin || jr.joinType == descpb.LeftAntiJoin,
groupingState: jr.groupingState,
}
return
}
ctx := flowCtx.EvalCtx.Ctx()
// Limit the memory use by creating a child monitor with a hard limit.
// joinReader will overflow to disk if this limit is not enough.
limit := execinfra.GetWorkMemLimit(flowCtx.Cfg)
if flowCtx.Cfg.TestingKnobs.ForceDiskSpill {
limit = 1
}
// Initialize memory monitors and row container for looked up rows.
jr.MemMonitor = execinfra.NewLimitedMonitor(ctx, flowCtx.EvalCtx.Mon, flowCtx.Cfg, "joinreader-limited")
jr.diskMonitor = execinfra.NewMonitor(ctx, flowCtx.Cfg.DiskMonitor, "joinreader-disk")
drc := rowcontainer.NewDiskBackedNumberedRowContainer(
false, /* deDup */
typs,
jr.EvalCtx,
jr.FlowCtx.Cfg.TempStorage,
jr.MemMonitor,
jr.diskMonitor,
)
if limit < mon.DefaultPoolAllocationSize {
// The memory limit is too low for caching, most likely to force disk
// spilling for testing.
drc.DisableCache = true
}
jr.strategy = &joinReaderOrderingStrategy{
joinerBase: &jr.joinerBase,
defaultSpanGenerator: spanGenerator,
isPartialJoin: jr.joinType == descpb.LeftSemiJoin || jr.joinType == descpb.LeftAntiJoin,
lookedUpRows: drc,
groupingState: jr.groupingState,
outputGroupContinuationForLeftRow: jr.outputGroupContinuationForLeftRow,
}
}
// getIndexColSet returns a set of all column indices for the given index.
func getIndexColSet(
index *descpb.IndexDescriptor, colIdxMap map[descpb.ColumnID]int,
) util.FastIntSet {
cols := util.MakeFastIntSet()
err := index.RunOverAllColumns(func(id descpb.ColumnID) error {
cols.Add(colIdxMap[id])
return nil
})
if err != nil {
// This path should never be hit since the column function never returns an
// error.
panic(err)
}
return cols
}
// SetBatchSizeBytes sets the desired batch size. It should only be used in tests.
func (jr *joinReader) SetBatchSizeBytes(batchSize int64) {
jr.batchSizeBytes = batchSize
}
// Spilled returns whether the joinReader spilled to disk.
func (jr *joinReader) Spilled() bool {
return jr.strategy.spilled()
}
// neededRightCols returns the set of column indices which need to be fetched
// from the right side of the join (jr.desc).
func (jr *joinReader) neededRightCols() util.FastIntSet {
neededCols := jr.Out.NeededColumns()
// Get the columns from the right side of the join and shift them over by
// the size of the left side so the right side starts at 0.
neededRightCols := util.MakeFastIntSet()
var lastCol int
for i, ok := neededCols.Next(len(jr.inputTypes)); ok; i, ok = neededCols.Next(i + 1) {
lastCol = i - len(jr.inputTypes)
neededRightCols.Add(lastCol)
}
if jr.outputGroupContinuationForLeftRow {
// The lastCol is the bool continuation column and not a right column.
neededRightCols.Remove(lastCol)
}
// Add columns needed by OnExpr.
for _, v := range jr.onCond.Vars.GetIndexedVars() {
rightIdx := v.Idx - len(jr.inputTypes)
if rightIdx >= 0 {
neededRightCols.Add(rightIdx)
}
}
return neededRightCols
}
// Next is part of the RowSource interface.
func (jr *joinReader) Next() (rowenc.EncDatumRow, *execinfrapb.ProducerMetadata) {
// The lookup join is implemented as follows:
// - Read the input rows in batches.
// - For each batch, map the rows onto index keys and perform an index
// lookup for those keys. Note that multiple rows may map to the same key.
// - Retrieve the index lookup results in batches, since the index scan may
// return more rows than the input batch size.
// - Join the index rows with the corresponding input rows and buffer the
// results in jr.toEmit.
for jr.State == execinfra.StateRunning {
var row rowenc.EncDatumRow
var meta *execinfrapb.ProducerMetadata
switch jr.runningState {
case jrReadingInput:
jr.runningState, row, meta = jr.readInput()
case jrPerformingLookup:
jr.runningState, meta = jr.performLookup()
case jrEmittingRows:
jr.runningState, row, meta = jr.emitRow()
case jrReadyToDrain:
jr.MoveToDraining(nil)
meta = jr.DrainHelper()
jr.runningState = jrStateUnknown
default:
log.Fatalf(jr.Ctx, "unsupported state: %d", jr.runningState)
}
if row == nil && meta == nil {
continue
}
if meta != nil {
return nil, meta
}
if outRow := jr.ProcessRowHelper(row); outRow != nil {
return outRow, nil
}
}
return nil, jr.DrainHelper()
}
// readInput reads the next batch of input rows and starts an index scan.
// It can sometimes emit a single row on behalf of the previous batch.
func (jr *joinReader) readInput() (
joinReaderState,
rowenc.EncDatumRow,
*execinfrapb.ProducerMetadata,
) {
if jr.groupingState != nil {
// Lookup join.
if jr.groupingState.initialized {
// State is from last batch.
jr.lastBatchState.lastGroupMatched = jr.groupingState.lastGroupMatched()
jr.groupingState.reset()
jr.lastBatchState.lastGroupContinued = false
}
// Else, returning meta interrupted reading the input batch, so we already
// did the reset for this batch.
}
// Read the next batch of input rows.
for jr.curBatchSizeBytes < jr.batchSizeBytes {
row, meta := jr.input.Next()
if meta != nil {
if meta.Err != nil {
jr.MoveToDraining(nil /* err */)
return jrStateUnknown, nil, meta
}
return jrReadingInput, nil, meta
}
if row == nil {
break
}
jr.curBatchSizeBytes += int64(row.Size())
if jr.groupingState != nil {
// Lookup Join.
if err := jr.processContinuationValForRow(row); err != nil {
jr.MoveToDraining(err)
return jrStateUnknown, nil, jr.DrainHelper()
}
}
jr.scratchInputRows = append(jr.scratchInputRows, jr.rowAlloc.CopyRow(row))
}
var outRow rowenc.EncDatumRow
// Finished reading the input batch.
if jr.groupingState != nil {
// Lookup join.
outRow = jr.allContinuationValsProcessed()
}
if len(jr.scratchInputRows) == 0 {
log.VEventf(jr.Ctx, 1, "no more input rows")
if outRow != nil {
return jrReadyToDrain, outRow, nil
}
// We're done.
jr.MoveToDraining(nil)
return jrStateUnknown, nil, jr.DrainHelper()
}
log.VEventf(jr.Ctx, 1, "read %d input rows", len(jr.scratchInputRows))
if jr.groupingState != nil && len(jr.scratchInputRows) > 0 {
jr.updateGroupingStateForNonEmptyBatch()
}
spans, err := jr.strategy.processLookupRows(jr.scratchInputRows)
if err != nil {
jr.MoveToDraining(err)
return jrStateUnknown, nil, jr.DrainHelper()
}
jr.scratchInputRows = jr.scratchInputRows[:0]
jr.curBatchSizeBytes = 0
if len(spans) == 0 {
// All of the input rows were filtered out. Skip the index lookup.
return jrEmittingRows, outRow, nil
}
// Sort the spans for the following cases:
// - For lookupJoinReaderType: this is so that we can rely upon the fetcher
// to limit the number of results per batch. It's safe to reorder the
// spans here because we already restore the original order of the output
// during the output collection phase.
// - For indexJoinReaderType when !maintainOrdering: this allows lower
// layers to optimize iteration over the data. Note that the looked up
// rows are output unchanged, in the retrieval order, so it is not safe to
// do this when maintainOrdering is true (the ordering to be maintained
// may be different than the ordering in the index).
if jr.readerType == lookupJoinReaderType ||
(jr.readerType == indexJoinReaderType && !jr.maintainOrdering) {
sort.Sort(spans)
}
log.VEventf(jr.Ctx, 1, "scanning %d spans", len(spans))
if err := jr.fetcher.StartScan(
jr.Ctx, jr.FlowCtx.Txn, spans, jr.shouldLimitBatches, 0, /* limitHint */
jr.FlowCtx.TraceKV); err != nil {
jr.MoveToDraining(err)
return jrStateUnknown, nil, jr.DrainHelper()
}
return jrPerformingLookup, outRow, nil
}
// performLookup reads the next batch of index rows.
func (jr *joinReader) performLookup() (joinReaderState, *execinfrapb.ProducerMetadata) {
nCols := len(jr.lookupCols)
for {
// Construct a "partial key" of nCols, so we can match the key format that
// was stored in our keyToInputRowIndices map. This matches the format that
// is output in jr.generateSpan.
var key roachpb.Key
// Index joins do not look at this key parameter so don't bother populating
// it, since it is not cheap for long keys.
if jr.readerType != indexJoinReaderType {
var err error
key, err = jr.fetcher.PartialKey(nCols)
if err != nil {
jr.MoveToDraining(err)
return jrStateUnknown, jr.DrainHelper()
}
}
// Fetch the next row and tell the strategy to process it.
lookedUpRow, _, _, err := jr.fetcher.NextRow(jr.Ctx)
if err != nil {
jr.MoveToDraining(scrub.UnwrapScrubError(err))
return jrStateUnknown, jr.DrainHelper()
}
if lookedUpRow == nil {
// Done with this input batch.
break
}
jr.rowsRead++
if nextState, err := jr.strategy.processLookedUpRow(jr.Ctx, lookedUpRow, key); err != nil {
jr.MoveToDraining(err)
return jrStateUnknown, jr.DrainHelper()
} else if nextState != jrPerformingLookup {
return nextState, nil
}
}
log.VEvent(jr.Ctx, 1, "done joining rows")
jr.strategy.prepareToEmit(jr.Ctx)
return jrEmittingRows, nil
}
// emitRow returns the next row from jr.toEmit, if present. Otherwise it
// prepares for another input batch.
func (jr *joinReader) emitRow() (
joinReaderState,
rowenc.EncDatumRow,
*execinfrapb.ProducerMetadata,
) {
rowToEmit, nextState, err := jr.strategy.nextRowToEmit(jr.Ctx)
if err != nil {
jr.MoveToDraining(err)
return jrStateUnknown, nil, jr.DrainHelper()
}
return nextState, rowToEmit, nil
}
// Start is part of the RowSource interface.
func (jr *joinReader) Start(ctx context.Context) context.Context {
jr.input.Start(ctx)
ctx = jr.StartInternal(ctx, joinReaderProcName)
jr.runningState = jrReadingInput
return ctx
}
// ConsumerClosed is part of the RowSource interface.
func (jr *joinReader) ConsumerClosed() {
// The consumer is done, Next() will not be called again.
jr.close()
}
func (jr *joinReader) close() {
if jr.InternalClose() {
if jr.fetcher != nil {
jr.fetcher.Close(jr.Ctx)
}
jr.strategy.close(jr.Ctx)
if jr.MemMonitor != nil {
jr.MemMonitor.Stop(jr.Ctx)
}
if jr.diskMonitor != nil {
jr.diskMonitor.Stop(jr.Ctx)
}
}
}
var _ execinfrapb.DistSQLSpanStats = &JoinReaderStats{}
const joinReaderTagPrefix = "joinreader."
// Stats implements the SpanStats interface.
func (jrs *JoinReaderStats) Stats() map[string]string {
statsMap := jrs.InputStats.Stats(joinReaderTagPrefix)
toMerge := jrs.IndexLookupStats.Stats(joinReaderTagPrefix + "index.")
for k, v := range toMerge {
statsMap[k] = v
}
return statsMap
}
// StatsForQueryPlan implements the DistSQLSpanStats interface.
func (jrs *JoinReaderStats) StatsForQueryPlan() []string {
is := append(
jrs.InputStats.StatsForQueryPlan(""),
jrs.IndexLookupStats.StatsForQueryPlan("index ")...,
)
return is
}
// outputStatsToTrace outputs the collected joinReader stats to the trace. Will
// fail silently if the joinReader is not collecting stats.
func (jr *joinReader) outputStatsToTrace() {
is, ok := getInputStats(jr.FlowCtx, jr.input)
if !ok {
return
}
ils, ok := getFetcherInputStats(jr.FlowCtx, jr.fetcher)
if !ok {
return
}
// TODO(asubiotto): Add memory and disk usage to EXPLAIN ANALYZE.
jrs := &JoinReaderStats{
InputStats: is,
IndexLookupStats: ils,
}
if sp := tracing.SpanFromContext(jr.Ctx); sp != nil {
sp.SetSpanStats(jrs)
}
}
// GetBytesRead is part of the execinfra.IOReader interface.
func (jr *joinReader) GetBytesRead() int64 {
return jr.fetcher.GetBytesRead()
}
// GetRowsRead is part of the execinfra.IOReader interface.
func (jr *joinReader) GetRowsRead() int64 {
return jr.rowsRead
}
func (jr *joinReader) generateMeta(ctx context.Context) []execinfrapb.ProducerMetadata {
trailingMeta := make([]execinfrapb.ProducerMetadata, 1)
meta := &trailingMeta[0]
meta.Metrics = execinfrapb.GetMetricsMeta()
meta.Metrics.RowsRead = jr.GetRowsRead()
meta.Metrics.BytesRead = jr.GetBytesRead()
if tfs := execinfra.GetLeafTxnFinalState(ctx, jr.FlowCtx.Txn); tfs != nil {
trailingMeta = append(trailingMeta,
execinfrapb.ProducerMetadata{LeafTxnFinalState: tfs},
)
}
return trailingMeta
}
// DrainMeta is part of the MetadataSource interface.
func (jr *joinReader) DrainMeta(ctx context.Context) []execinfrapb.ProducerMetadata {
return jr.generateMeta(ctx)
}
// ChildCount is part of the execinfra.OpNode interface.
func (jr *joinReader) ChildCount(verbose bool) int {
if _, ok := jr.input.(execinfra.OpNode); ok {
return 1
}
return 0
}
// Child is part of the execinfra.OpNode interface.
func (jr *joinReader) Child(nth int, verbose bool) execinfra.OpNode {
if nth == 0 {
if n, ok := jr.input.(execinfra.OpNode); ok {
return n
}
panic("input to joinReader is not an execinfra.OpNode")
}
panic(errors.AssertionFailedf("invalid index %d", nth))
}
// processContinuationValForRow is called for each row in a batch which has a
// continuation column.
func (jr *joinReader) processContinuationValForRow(row rowenc.EncDatumRow) error {
if !jr.groupingState.doGrouping {
// Lookup join with no continuation column.
jr.groupingState.addContinuationValForRow(false)
} else {
continuationEncDatum := row[len(row)-1]
if err := continuationEncDatum.EnsureDecoded(types.Bool, &jr.alloc); err != nil {
return err
}
continuationVal := bool(*continuationEncDatum.Datum.(*tree.DBool))
jr.groupingState.addContinuationValForRow(continuationVal)
if len(jr.scratchInputRows) == 0 && continuationVal {
// First row in batch is a continuation of last group.
jr.lastBatchState.lastGroupContinued = true
}
}
return nil
}
// allContinuationValsProcessed is called after all the rows in the batch have
// been read, or the batch is empty, and processContinuationValForRow has been
// called. It returns a non-nil row if one needs to output a row from the
// batch previous to the current batch.
func (jr *joinReader) allContinuationValsProcessed() rowenc.EncDatumRow {
var outRow rowenc.EncDatumRow
jr.groupingState.initialized = true
if jr.lastBatchState.lastInputRow != nil && !jr.lastBatchState.lastGroupContinued {
// Group ended in previous batch and this is a lookup join with a
// continuation column.
if !jr.lastBatchState.lastGroupMatched {
// Handle the cases where we need to emit the left row when there is no
// match.
switch jr.joinType {
case descpb.LeftOuterJoin:
outRow = jr.renderUnmatchedRow(jr.lastBatchState.lastInputRow, leftSide)
case descpb.LeftAntiJoin:
outRow = jr.lastBatchState.lastInputRow
}
}
// Else the last group matched, so already emitted 1+ row for left outer
// join, 1 row for semi join, and no need to emit for anti join.
}
// Else, last group continued, or this is the first ever batch, or all
// groups are of length 1. Either way, we don't need to do anything
// special for the last group from the last batch.
jr.lastBatchState.lastInputRow = nil
return outRow
}
// updateGroupingStateForNonEmptyBatch is called once the batch has been read
// and found to be non-empty.
func (jr *joinReader) updateGroupingStateForNonEmptyBatch() {
if jr.groupingState.doGrouping {
// Groups can continue from one batch to another.
// Remember state from the last group in this batch.
jr.lastBatchState.lastInputRow = jr.scratchInputRows[len(jr.scratchInputRows)-1]
// Initialize matching state for the first group in this batch.
if jr.lastBatchState.lastGroupMatched && jr.lastBatchState.lastGroupContinued {
jr.groupingState.setFirstGroupMatched()
}
}
}
// inputBatchGroupingState encapsulates the state needed for all the
// groups in an input batch, for lookup joins (not used for index
// joins).
// It functions in one of two modes:
// - doGrouping is false: It is expected that for each input row in
// a batch, addContinuationValForRow(false) will be called.
// - doGrouping is true: The join is functioning in a manner where
// the continuation column in the input indicates the parameter
// value of addContinuationValForRow calls.
//
// The initialization and resetting of state for a batch is
// handled by joinReader. Updates to this state based on row
// matching is done by the appropriate joinReaderStrategy
// implementation. The joinReaderStrategy implementations
// also lookup the state to decide when to output.
type inputBatchGroupingState struct {
doGrouping bool
initialized bool
// Row index in batch to the group index. Only used when doGrouping = true.
batchRowToGroupIndex []int
// State per group.
groupState []groupState
}
type groupState struct {
// Whether the group matched.
matched bool
// The last row index in the group. Only valid when doGrouping = true.
lastRow int
}
func (ib *inputBatchGroupingState) reset() {
ib.batchRowToGroupIndex = ib.batchRowToGroupIndex[:0]
ib.groupState = ib.groupState[:0]
ib.initialized = false
}
// addContinuationValForRow is called with each row in an input batch, with
// the cont parameter indicating whether or not it is a continuation of the
// group from the previous row.
func (ib *inputBatchGroupingState) addContinuationValForRow(cont bool) {
if len(ib.groupState) == 0 || !cont {
// First row in input batch or the start of a new group. We need to
// add entries in the group indexed slices.
ib.groupState = append(ib.groupState,
groupState{matched: false, lastRow: len(ib.batchRowToGroupIndex)})
}
if ib.doGrouping {
groupIndex := len(ib.groupState) - 1
ib.groupState[groupIndex].lastRow = len(ib.batchRowToGroupIndex)
ib.batchRowToGroupIndex = append(ib.batchRowToGroupIndex, groupIndex)
}
}
func (ib *inputBatchGroupingState) setFirstGroupMatched() {
ib.groupState[0].matched = true
}
// setMatched returns the previous value of the matched field.
func (ib *inputBatchGroupingState) setMatched(rowIndex int) bool {
groupIndex := rowIndex
if ib.doGrouping {
groupIndex = ib.batchRowToGroupIndex[rowIndex]
}
rv := ib.groupState[groupIndex].matched
ib.groupState[groupIndex].matched = true
return rv
}
func (ib *inputBatchGroupingState) getMatched(rowIndex int) bool {
groupIndex := rowIndex
if ib.doGrouping {
groupIndex = ib.batchRowToGroupIndex[rowIndex]
}
return ib.groupState[groupIndex].matched
}
func (ib *inputBatchGroupingState) lastGroupMatched() bool {
if !ib.doGrouping || len(ib.groupState) == 0 {
return false
}
return ib.groupState[len(ib.groupState)-1].matched
}
func (ib *inputBatchGroupingState) isUnmatched(rowIndex int) bool {
if !ib.doGrouping {
// The rowIndex is also the groupIndex.
return !ib.groupState[rowIndex].matched
}
groupIndex := ib.batchRowToGroupIndex[rowIndex]
if groupIndex == len(ib.groupState)-1 {
// Return false for last group since it is not necessarily complete yet --
// the next batch may continue the group.
return false
}
// Group is complete -- return true for the last row index in a group that
// is unmatched. Note that there are join reader strategies that on a
// row-by-row basis (a) evaluate the match condition, (b) decide whether to
// output (including when there is no match). It is necessary to delay
// saying that there is no match for the group until the last row in the
// group since for earlier rows, when at step (b), one does not know the
// match state of later rows in the group.
return !ib.groupState[groupIndex].matched && ib.groupState[groupIndex].lastRow == rowIndex
}