-
Notifications
You must be signed in to change notification settings - Fork 3.9k
/
Copy pathjoinreader.go
1397 lines (1279 loc) · 50.2 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
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
// 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"
"math"
"sort"
"unsafe"
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvstreamer"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/typedesc"
"github.com/cockroachdb/cockroach/pkg/sql/execinfra"
"github.com/cockroachdb/cockroach/pkg/sql/execinfra/execopnode"
"github.com/cockroachdb/cockroach/pkg/sql/execinfrapb"
"github.com/cockroachdb/cockroach/pkg/sql/execstats"
"github.com/cockroachdb/cockroach/pkg/sql/memsize"
"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/rowinfra"
"github.com/cockroachdb/cockroach/pkg/sql/scrub"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"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/optional"
"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
// memAcc is used to account for the memory used by the in-memory data
// structures used directly by the joinReader. Note that the joinReader
// strategies and span generators have separate accounts.
memAcc mon.BoundAccount
// accountedFor tracks the memory usage of scratchInputRows and
// groupingState that is currently registered with memAcc.
accountedFor struct {
// scratchInputRows accounts only for the slice of scratchInputRows, not
// the actual rows.
scratchInputRows int64
groupingState int64
}
// limitedMemMonitor is a limited memory monitor to account for the memory
// used by buffered rows in joinReaderOrderingStrategy. If the memory limit is
// exceeded, the joinReader will spill to disk. diskMonitor is used to monitor
// the disk utilization in this case.
limitedMemMonitor *mon.BytesMonitor
diskMonitor *mon.BytesMonitor
fetchSpec descpb.IndexFetchSpec
splitFamilyIDs []descpb.FamilyID
// Indicates that the join reader should maintain the ordering of the input
// stream. This is applicable to both lookup joins and index joins. For lookup
// joins, maintaining order is expensive because it requires buffering. For
// index joins buffering is not required, but still, if ordering is not
// required, we'll change the output order to allow for some Pebble
// optimizations.
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 tree.DatumAlloc
rowAlloc rowenc.EncDatumRowAlloc
shouldLimitBatches bool
readerType joinReaderType
// txn is the transaction used by the join reader.
txn *kv.Txn
// usesStreamer indicates whether the joinReader performs the lookups using
// the kvstreamer.Streamer API.
usesStreamer bool
streamerInfo struct {
unlimitedMemMonitor *mon.BytesMonitor
budgetAcc mon.BoundAccount
// maintainOrdering indicates whether the ordering of the input stream
// needs to be maintained AND that we rely on the streamer for that.
//
// Currently this is only the case when joinReader.maintainOrdering is
// true and we are performing an index join. Due to implementation
// details, we don't rely on the streamer for maintaining the ordering
// for lookup joins at the moment (since we still buffer all looked up
// rows and restore the ordering explicitly via the
// joinReaderOrderingStrategy).
maintainOrdering bool
diskMonitor *mon.BytesMonitor
txnKVStreamerMemAcc mon.BoundAccount
}
input execinfra.RowSource
// lookupCols and lookupExpr (and optionally remoteLookupExpr) represent the
// part of the join condition used to perform the lookup into the index.
// Exactly one of lookupCols or lookupExpr must be non-empty.
//
// lookupCols is used when the lookup condition is just a simple equality
// between input columns and index columns. In this case, lookupCols contains
// the column indexes in the input stream specifying the columns which match
// with the index columns. These are the equality columns of the join.
//
// lookupExpr is used when the lookup condition is more complicated than a
// simple equality between input columns and index columns. In this case,
// lookupExpr specifies the expression that will be used to construct the
// spans for each lookup. See comments in the spec for details about the
// supported expressions.
//
// If remoteLookupExpr is set, this is a locality optimized lookup join. In
// this case, lookupExpr contains the lookup join conditions targeting ranges
// located on local nodes (relative to the gateway region), and
// remoteLookupExpr contains the lookup join conditions targeting remote
// nodes. See comments in the spec for more details.
lookupCols []uint32
lookupExpr execinfrapb.ExprHelper
remoteLookupExpr execinfrapb.ExprHelper
// Batch size for fetches. Not a constant so we can lower for testing.
batchSizeBytes int64
curBatchSizeBytes int64
// pendingRow tracks the row that has already been read from the input but
// was not included into the lookup batch because it would make the batch
// exceed batchSizeBytes.
pendingRow rowenc.EncDatumRow
// rowsRead is the total number of rows that this fetcher read from
// disk.
rowsRead int64
// curBatchRowsRead is the number of rows that this fetcher read from disk for
// the current batch.
curBatchRowsRead int64
// curBatchInputRowCount is the number of input rows in the current batch.
curBatchInputRowCount int64
// State variables for each batch of input rows.
scratchInputRows rowenc.EncDatumRows
// resetScratchWhenReadingInput tracks whether scratchInputRows needs to be
// reset the next time the joinReader is in the jrReadingInput state.
resetScratchWhenReadingInput bool
// 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
// lookupBatchBytesLimit controls the TargetBytes of lookup requests. If 0, a
// default will be used. Regardless of this value, bytes limits aren't always
// used.
lookupBatchBytesLimit rowinfra.BytesLimit
// limitHintHelper is used in limiting batches of input rows in the presence
// of hard and soft limits.
limitHintHelper execinfra.LimitHintHelper
// scanStats is collected from the trace after we finish doing work for this
// join.
scanStats execstats.ScanStats
}
var _ execinfra.Processor = &joinReader{}
var _ execinfra.RowSource = &joinReader{}
var _ execopnode.OpNode = &joinReader{}
const joinReaderProcName = "join reader"
// ParallelizeMultiKeyLookupJoinsEnabled determines whether the joinReader
// parallelizes KV batches in all cases.
var ParallelizeMultiKeyLookupJoinsEnabled = settings.RegisterBoolSetting(
settings.TenantWritable,
"sql.distsql.parallelize_multi_key_lookup_joins.enabled",
"determines whether KV batches are executed in parallel for lookup joins in all cases. "+
"Enabling this will increase the speed of lookup joins when each input row might get "+
"multiple looked up rows at the cost of increased memory usage.",
false,
)
// 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.OutputGroupContinuationForLeftRow && !spec.MaintainOrdering {
return nil, errors.AssertionFailedf(
"lookup join must maintain ordering since it is first join in paired-joins")
}
switch readerType {
case lookupJoinReaderType:
switch spec.Type {
case descpb.InnerJoin, descpb.LeftOuterJoin, descpb.LeftSemiJoin, descpb.LeftAntiJoin:
default:
return nil, errors.AssertionFailedf("only inner and left {outer, semi, anti} lookup joins are supported, %s requested", spec.Type)
}
case indexJoinReaderType:
if spec.Type != descpb.InnerJoin {
return nil, errors.AssertionFailedf("only inner index joins are supported, %s requested", spec.Type)
}
if !spec.LookupExpr.Empty() {
return nil, errors.AssertionFailedf("non-empty lookup expressions are not supported for index joins")
}
if !spec.RemoteLookupExpr.Empty() {
return nil, errors.AssertionFailedf("non-empty remote lookup expressions are not supported for index joins")
}
if !spec.OnExpr.Empty() {
return nil, errors.AssertionFailedf("non-empty ON expressions are not supported for index joins")
}
}
var lookupCols []uint32
switch readerType {
case indexJoinReaderType:
lookupCols = make([]uint32, len(spec.FetchSpec.KeyColumns()))
for i := range lookupCols {
lookupCols[i] = uint32(i)
}
case lookupJoinReaderType:
lookupCols = spec.LookupColumns
default:
return nil, errors.Errorf("unsupported joinReaderType")
}
// The joiner has a choice to make between getting DistSender-level
// parallelism for its lookup batches and setting row and memory limits (due
// to implementation limitations, you can't have both at the same time). We
// choose parallelism when we know that each lookup returns at most one row:
// in case of indexJoinReaderType, we know that there's exactly one lookup
// row for each input row. Similarly, in case of spec.LookupColumnsAreKey,
// we know that there's at most one lookup row per input row. In other
// cases, we use limits.
shouldLimitBatches := !spec.LookupColumnsAreKey && readerType == lookupJoinReaderType
if flowCtx.EvalCtx.SessionData().ParallelizeMultiKeyLookupJoinsEnabled {
shouldLimitBatches = false
}
if spec.MaintainLookupOrdering {
// MaintainLookupOrdering indicates the output of the lookup joiner should
// be sorted by <inputCols>, <lookupCols>. It doesn't make sense for
// MaintainLookupOrdering to be true when MaintainOrdering is not.
// Additionally, we need to disable parallelism for the traditional fetcher
// in order to ensure the lookups are ordered, so set shouldLimitBatches.
spec.MaintainOrdering, shouldLimitBatches = true, true
}
useStreamer, txn, err := flowCtx.UseStreamer()
if err != nil {
return nil, err
}
jr := &joinReader{
fetchSpec: spec.FetchSpec,
splitFamilyIDs: spec.SplitFamilyIDs,
maintainOrdering: spec.MaintainOrdering,
input: input,
lookupCols: lookupCols,
outputGroupContinuationForLeftRow: spec.OutputGroupContinuationForLeftRow,
shouldLimitBatches: shouldLimitBatches,
readerType: readerType,
txn: txn,
usesStreamer: useStreamer,
lookupBatchBytesLimit: rowinfra.BytesLimit(spec.LookupBatchBytesLimit),
limitHintHelper: execinfra.MakeLimitHintHelper(spec.LimitHint, post),
}
if readerType != indexJoinReaderType {
jr.groupingState = &inputBatchGroupingState{doGrouping: spec.LeftJoinWithPairedJoiner}
}
// Make sure the key column types are hydrated. The fetched column types will
// be hydrated in ProcessorBase.Init (via joinerBase.init).
resolver := flowCtx.NewTypeResolver(jr.txn)
for i := range spec.FetchSpec.KeyAndSuffixColumns {
if err := typedesc.EnsureTypeIsHydrated(
flowCtx.EvalCtx.Ctx(), spec.FetchSpec.KeyAndSuffixColumns[i].Type, &resolver,
); err != nil {
return nil, err
}
}
var leftTypes []*types.T
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.
case lookupJoinReaderType:
leftTypes = input.OutputTypes()
default:
return nil, errors.Errorf("unsupported joinReaderType")
}
rightTypes := spec.FetchSpec.FetchedColumnTypes()
if err := jr.joinerBase.init(
jr,
flowCtx,
processorID,
leftTypes,
rightTypes,
spec.Type,
spec.OnExpr,
spec.OutputGroupContinuationForLeftRow,
post,
output,
execinfra.ProcStateOpts{
InputsToDrain: []execinfra.RowSource{jr.input},
TrailingMetaCallback: func() []execinfrapb.ProducerMetadata {
// We need to generate metadata before closing the processor
// because InternalClose() updates jr.Ctx to the "original"
// context.
trailingMeta := jr.generateMeta()
jr.close()
return trailingMeta
},
},
); err != nil {
return nil, err
}
if !spec.LookupExpr.Empty() {
lookupExprTypes := make([]*types.T, 0, len(leftTypes)+len(rightTypes))
lookupExprTypes = append(lookupExprTypes, leftTypes...)
lookupExprTypes = append(lookupExprTypes, rightTypes...)
semaCtx := flowCtx.NewSemaContext(jr.txn)
if err := jr.lookupExpr.Init(spec.LookupExpr, lookupExprTypes, semaCtx, jr.EvalCtx); err != nil {
return nil, err
}
if !spec.RemoteLookupExpr.Empty() {
if err := jr.remoteLookupExpr.Init(
spec.RemoteLookupExpr, lookupExprTypes, semaCtx, jr.EvalCtx,
); err != nil {
return nil, err
}
}
}
// We will create a memory monitor with a hard memory limit since the join
// reader doesn't know how to spill its in-memory state to disk (separate
// from the buffered rows). It is most likely that if the target limit is
// really low then we're in a test scenario and we don't want to error out.
minMemoryLimit := int64(8 << 20)
// Streamer can handle lower memory limit and doing so makes testing at
// the limits more efficient.
if jr.usesStreamer {
minMemoryLimit = 100 << 10
}
memoryLimit := execinfra.GetWorkMemLimit(flowCtx)
if memoryLimit < minMemoryLimit {
memoryLimit = minMemoryLimit
}
// Initialize memory monitors and bound account for data structures in the joinReader.
jr.MemMonitor = mon.NewMonitorInheritWithLimit(
"joinreader-mem" /* name */, memoryLimit, flowCtx.EvalCtx.Mon,
)
jr.MemMonitor.StartNoReserved(flowCtx.EvalCtx.Ctx(), flowCtx.EvalCtx.Mon)
jr.memAcc = jr.MemMonitor.MakeBoundAccount()
if err := jr.initJoinReaderStrategy(flowCtx, rightTypes, readerType); err != nil {
return nil, err
}
jr.batchSizeBytes = jr.strategy.getLookupRowsBatchSizeHint(flowCtx.EvalCtx.SessionData())
var streamingKVFetcher *row.KVFetcher
if jr.usesStreamer {
// NOTE: this comment should only be considered in a case of low workmem
// limit (which is a testing scenario).
//
// When using the Streamer API, we want to limit the memory usage of the
// join reader itself to be at most a quarter of the workmem limit. That
// memory usage is comprised of several parts:
// - the input batch (i.e. buffered input rows) which is limited by the
// batch size hint;
// - some in-memory state of the join reader strategy;
// - some in-memory state of the span generator.
// We don't have any way of limiting the last two parts, so we apply a
// simple heuristic that each of those parts takes up on the order of
// the batch size hint.
//
// Thus, we arrive at the following setup:
// - make the batch size hint to be at most 1/12 of workmem
// - then reserve 3/12 of workmem for the memory usage of those three
// parts.
//
// Note that it is ok if the batch size hint is set to zero since the
// joinReader will always include at least one row into the lookup
// batch.
if jr.batchSizeBytes > memoryLimit/12 {
jr.batchSizeBytes = memoryLimit / 12
}
// See the comment above for how we arrived at this calculation.
//
// That comment is made in the context of low workmem limit. However, in
// production we expect workmem limit to be on the order of 64MiB
// whereas the batch size hint is at most 4MiB, so the streamer will get
// at least (depending on the hint) on the order of 52MiB which is
// plenty enough.
streamerBudgetLimit := memoryLimit - 3*jr.batchSizeBytes
// We need to use an unlimited monitor for the streamer's budget since
// the streamer itself is responsible for staying under the limit.
jr.streamerInfo.unlimitedMemMonitor = mon.NewMonitorInheritWithLimit(
"joinreader-streamer-unlimited" /* name */, math.MaxInt64, flowCtx.EvalCtx.Mon,
)
jr.streamerInfo.unlimitedMemMonitor.StartNoReserved(flowCtx.EvalCtx.Ctx(), flowCtx.EvalCtx.Mon)
jr.streamerInfo.budgetAcc = jr.streamerInfo.unlimitedMemMonitor.MakeBoundAccount()
jr.streamerInfo.txnKVStreamerMemAcc = jr.streamerInfo.unlimitedMemMonitor.MakeBoundAccount()
jr.streamerInfo.maintainOrdering = jr.maintainOrdering && readerType == indexJoinReaderType
var diskBuffer kvstreamer.ResultDiskBuffer
if jr.streamerInfo.maintainOrdering {
jr.streamerInfo.diskMonitor = execinfra.NewMonitor(
flowCtx.EvalCtx.Ctx(), jr.FlowCtx.DiskMonitor, "streamer-disk", /* name */
)
diskBuffer = rowcontainer.NewKVStreamerResultDiskBuffer(
jr.FlowCtx.Cfg.TempStorage, jr.streamerInfo.diskMonitor,
)
}
singleRowLookup := readerType == indexJoinReaderType || spec.LookupColumnsAreKey
streamingKVFetcher = row.NewStreamingKVFetcher(
flowCtx.Cfg.DistSender,
flowCtx.Stopper(),
jr.txn,
flowCtx.EvalCtx.Settings,
spec.LockingWaitPolicy,
spec.LockingStrength,
streamerBudgetLimit,
&jr.streamerInfo.budgetAcc,
jr.streamerInfo.maintainOrdering,
singleRowLookup,
int(spec.FetchSpec.MaxKeysPerRow),
diskBuffer,
&jr.streamerInfo.txnKVStreamerMemAcc,
)
} else {
// When not using the Streamer API, we want to limit the batch size hint
// to at most half of the workmem limit. Note that it is ok if it is set
// to zero since the joinReader will always include at least one row
// into the lookup batch.
if jr.batchSizeBytes > memoryLimit/2 {
jr.batchSizeBytes = memoryLimit / 2
}
}
var fetcher row.Fetcher
if err := fetcher.Init(
flowCtx.EvalCtx.Context,
row.FetcherInitArgs{
StreamingKVFetcher: streamingKVFetcher,
Txn: jr.txn,
LockStrength: spec.LockingStrength,
LockWaitPolicy: spec.LockingWaitPolicy,
LockTimeout: flowCtx.EvalCtx.SessionData().LockTimeout,
Alloc: &jr.alloc,
MemMonitor: flowCtx.EvalCtx.Mon,
Spec: &spec.FetchSpec,
TraceKV: flowCtx.TraceKV,
ForceProductionKVBatchSize: flowCtx.EvalCtx.TestingKnobs.ForceProductionValues,
},
); err != nil {
return nil, err
}
if execstats.ShouldCollectStats(flowCtx.EvalCtx.Ctx(), flowCtx.CollectStats) {
jr.input = newInputStatCollector(jr.input)
jr.fetcher = newRowFetcherStatCollector(&fetcher)
jr.ExecStatsForTrace = jr.execStatsForTrace
} else {
jr.fetcher = &fetcher
}
// TODO(radu): verify the input types match the index key types
return jr, nil
}
func (jr *joinReader) initJoinReaderStrategy(
flowCtx *execinfra.FlowCtx, typs []*types.T, readerType joinReaderType,
) error {
strategyMemAcc := jr.MemMonitor.MakeBoundAccount()
spanGeneratorMemAcc := jr.MemMonitor.MakeBoundAccount()
var generator joinReaderSpanGenerator
if jr.lookupExpr.Expr == nil {
defGen := &defaultSpanGenerator{}
if err := defGen.init(
flowCtx.EvalCtx,
flowCtx.Codec(),
&jr.fetchSpec,
jr.splitFamilyIDs,
readerType == indexJoinReaderType, /* uniqueRows */
jr.lookupCols,
&spanGeneratorMemAcc,
); err != nil {
return err
}
generator = defGen
} else {
// Since jr.lookupExpr is set, we need to use either multiSpanGenerator or
// localityOptimizedSpanGenerator, which support looking up multiple spans
// per input row.
// Map fetched columns to index key columns.
var fetchedOrdToIndexKeyOrd util.FastIntMap
fullColumns := jr.fetchSpec.KeyFullColumns()
for keyOrdinal := range fullColumns {
keyColID := fullColumns[keyOrdinal].ColumnID
for fetchedOrdinal := range jr.fetchSpec.FetchedColumns {
if jr.fetchSpec.FetchedColumns[fetchedOrdinal].ColumnID == keyColID {
fetchedOrdToIndexKeyOrd.Set(fetchedOrdinal, keyOrdinal)
break
}
}
}
// If jr.remoteLookupExpr is set, this is a locality optimized lookup join
// and we need to use localityOptimizedSpanGenerator.
if jr.remoteLookupExpr.Expr == nil {
multiSpanGen := &multiSpanGenerator{}
if err := multiSpanGen.init(
flowCtx.EvalCtx,
flowCtx.Codec(),
&jr.fetchSpec,
jr.splitFamilyIDs,
len(jr.input.OutputTypes()),
&jr.lookupExpr,
fetchedOrdToIndexKeyOrd,
&spanGeneratorMemAcc,
); err != nil {
return err
}
generator = multiSpanGen
} else {
localityOptSpanGen := &localityOptimizedSpanGenerator{}
remoteSpanGenMemAcc := jr.MemMonitor.MakeBoundAccount()
if err := localityOptSpanGen.init(
flowCtx.EvalCtx,
flowCtx.Codec(),
&jr.fetchSpec,
jr.splitFamilyIDs,
len(jr.input.OutputTypes()),
&jr.lookupExpr,
&jr.remoteLookupExpr,
fetchedOrdToIndexKeyOrd,
&spanGeneratorMemAcc,
&remoteSpanGenMemAcc,
); err != nil {
return err
}
generator = localityOptSpanGen
}
}
defer func() {
generator.setResizeMemoryAccountFunc(jr.strategy.resizeMemoryAccount)
}()
if readerType == indexJoinReaderType {
jr.strategy = &joinReaderIndexJoinStrategy{
joinerBase: &jr.joinerBase,
joinReaderSpanGenerator: generator,
strategyMemAcc: &strategyMemAcc,
}
return nil
}
if !jr.maintainOrdering {
jr.strategy = &joinReaderNoOrderingStrategy{
joinerBase: &jr.joinerBase,
joinReaderSpanGenerator: generator,
isPartialJoin: jr.joinType == descpb.LeftSemiJoin || jr.joinType == descpb.LeftAntiJoin,
groupingState: jr.groupingState,
strategyMemAcc: &strategyMemAcc,
}
return nil
}
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)
// Initialize memory monitors and row container for looked up rows.
jr.limitedMemMonitor = execinfra.NewLimitedMonitor(ctx, jr.MemMonitor, flowCtx, "joinreader-limited")
// We want to make sure that if the disk-backed container is spilled to
// disk, it releases all of the memory reservations, so we make the
// corresponding memory monitor not hold on to any bytes.
jr.limitedMemMonitor.RelinquishAllOnReleaseBytes()
jr.diskMonitor = execinfra.NewMonitor(ctx, flowCtx.DiskMonitor, "joinreader-disk")
drc := rowcontainer.NewDiskBackedNumberedRowContainer(
false, /* deDup */
typs,
jr.EvalCtx,
jr.FlowCtx.Cfg.TempStorage,
jr.limitedMemMonitor,
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,
joinReaderSpanGenerator: generator,
isPartialJoin: jr.joinType == descpb.LeftSemiJoin || jr.joinType == descpb.LeftAntiJoin,
lookedUpRows: drc,
groupingState: jr.groupingState,
outputGroupContinuationForLeftRow: jr.outputGroupContinuationForLeftRow,
strategyMemAcc: &strategyMemAcc,
}
return nil
}
// 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()
}
// 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()
}
// addWorkmemHint checks whether err is non-nil, and if so, wraps it with a hint
// to increase workmem limit. It is expected that err was returned by the memory
// accounting system.
func addWorkmemHint(err error) error {
if err == nil {
return nil
}
return errors.WithHint(
err, "consider increasing sql.distsql.temp_storage.workmem cluster"+
" setting or distsql_workmem session variable",
)
}
type spansWithSpanIDs struct {
spans roachpb.Spans
spanIDs []int
}
var _ sort.Interface = &spansWithSpanIDs{}
func (s spansWithSpanIDs) Len() int {
return len(s.spans)
}
func (s spansWithSpanIDs) Less(i, j int) bool {
return s.spans[i].Key.Compare(s.spans[j].Key) < 0
}
func (s spansWithSpanIDs) Swap(i, j int) {
s.spans[i], s.spans[j] = s.spans[j], s.spans[i]
s.spanIDs[i], s.spanIDs[j] = s.spanIDs[j], s.spanIDs[i]
}
// sortSpans sorts the given spans while maintaining the spanIDs mapping (if it
// is non-nil).
func sortSpans(spans roachpb.Spans, spanIDs []int) {
if spanIDs != nil {
s := spansWithSpanIDs{
spans: spans,
spanIDs: spanIDs,
}
sort.Sort(&s)
} else {
sort.Sort(spans)
}
}
// 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.
}
if jr.resetScratchWhenReadingInput {
// Deeply reset the rows from the previous input batch.
for i := range jr.scratchInputRows {
jr.scratchInputRows[i] = nil
}
// We've just discarded the old rows, so we have to update the memory
// accounting accordingly.
newSz := jr.accountedFor.scratchInputRows + jr.accountedFor.groupingState
if err := jr.strategy.resizeMemoryAccount(&jr.memAcc, jr.memAcc.Used(), newSz); err != nil {
jr.MoveToDraining(err)
return jrStateUnknown, nil, jr.DrainHelper()
}
jr.scratchInputRows = jr.scratchInputRows[:0]
jr.resetScratchWhenReadingInput = false
}
// Read the next batch of input rows.
for {
var encDatumRow rowenc.EncDatumRow
var rowSize int64
if jr.pendingRow == nil {
// There is no pending row, so we have to get the next one from the
// input.
var meta *execinfrapb.ProducerMetadata
encDatumRow, meta = jr.input.Next()
if meta != nil {
if meta.Err != nil {
jr.MoveToDraining(nil /* err */)
return jrStateUnknown, nil, meta
}
if err := jr.performMemoryAccounting(); err != nil {
jr.MoveToDraining(err)
return jrStateUnknown, nil, meta
}
return jrReadingInput, nil, meta
}
if encDatumRow == nil {
break
}
rowSize = int64(encDatumRow.Size())
if jr.curBatchSizeBytes > 0 && jr.curBatchSizeBytes+rowSize > jr.batchSizeBytes {
// Adding this row to the current batch will make the batch
// exceed jr.batchSizeBytes. Additionally, the batch is not
// empty, so we'll store this row as "pending" and will include
// it into the next batch.
//
// The batch being non-empty is important because in case it was
// empty and we decided to not include this (first) row into it,
// then we'd be stalled - we'd generate no spans, so we'd not
// perform the lookup of anything.
jr.pendingRow = encDatumRow
break
}
} else {
encDatumRow = jr.pendingRow
jr.pendingRow = nil
rowSize = int64(encDatumRow.Size())
}
jr.curBatchSizeBytes += rowSize
if jr.groupingState != nil {
// Lookup Join.
if err := jr.processContinuationValForRow(encDatumRow); err != nil {
jr.MoveToDraining(err)
return jrStateUnknown, nil, jr.DrainHelper()
}
}
// Keep the copy of the row after accounting for its memory usage.
//
// We need to subtract the EncDatumRowOverhead because that is already
// tracked in jr.accountedFor.scratchInputRows.
if err := jr.strategy.growMemoryAccount(&jr.memAcc, rowSize-int64(rowenc.EncDatumRowOverhead)); err != nil {
jr.MoveToDraining(err)
return jrStateUnknown, nil, jr.DrainHelper()
}
jr.scratchInputRows = append(jr.scratchInputRows, jr.rowAlloc.CopyRow(encDatumRow))
if l := jr.limitHintHelper.LimitHint(); l != 0 && l == int64(len(jr.scratchInputRows)) {
break
}
}
if err := jr.performMemoryAccounting(); err != nil {
jr.MoveToDraining(err)
return jrStateUnknown, nil, jr.DrainHelper()
}
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()
}
if err := jr.limitHintHelper.ReadSomeRows(int64(len(jr.scratchInputRows))); err != nil {
jr.MoveToDraining(err)
return jrStateUnknown, nil, jr.DrainHelper()
}
// Figure out what key spans we need to lookup.
spans, spanIDs, err := jr.strategy.processLookupRows(jr.scratchInputRows)
if err != nil {
jr.MoveToDraining(err)
return jrStateUnknown, nil, jr.DrainHelper()
}
jr.curBatchInputRowCount = int64(len(jr.scratchInputRows))
jr.resetScratchWhenReadingInput = true
jr.curBatchSizeBytes = 0
jr.curBatchRowsRead = 0
if len(spans) == 0 {
// All of the input rows were filtered out. Skip the index lookup.
return jrEmittingRows, outRow, nil
}
// Sort the spans by key order, except for a special case: an index-join with
// maintainOrdering. That case can be executed efficiently if we don't sort:
// we know that, for an index-join, each input row corresponds to exactly one
// lookup row, and vice-versa. So, `spans` has one span per input/lookup-row,
// in the right order. joinReaderIndexJoinStrategy.processLookedUpRow()
// immediately emits each looked up row (it never buffers or reorders rows)
// so, if ordering matters, we cannot sort the spans here.
//
// In every other case than the one discussed above, we sort the spans because
// a) if we sort, we can then configure the fetcher below with a limit (the
// fetcher only accepts a limit if the spans are sorted), and
// b) Pebble has various optimizations for Seeks in sorted order.
if jr.readerType == indexJoinReaderType && jr.maintainOrdering {
// Assert that the index join doesn't have shouldLimitBatches set. Since we
// didn't sort above, the fetcher doesn't support a limit.
if jr.shouldLimitBatches {
err := errors.AssertionFailedf("index join configured with both maintainOrdering and " +
"shouldLimitBatched; this shouldn't have happened as the implementation doesn't support it")
jr.MoveToDraining(err)
return jrStateUnknown, nil, jr.DrainHelper()
}
} else {
if !jr.usesStreamer || jr.streamerInfo.maintainOrdering {
// We don't want to sort the spans here if we're using the Streamer,
// and it will perform the sort on its own - currently, this is the
// case with OutOfOrder mode.
sortSpans(spans, spanIDs)
}
}
log.VEventf(jr.Ctx, 1, "scanning %d spans", len(spans))
// Note that the fetcher takes ownership of the spans slice - it will modify
// it and perform the memory accounting. We don't care about the
// modification here, but we want to be conscious about the memory
// accounting - we don't double count for any memory of spans because the
// joinReaderStrategy doesn't account for any memory used by the spans.
var bytesLimit rowinfra.BytesLimit
if !jr.usesStreamer {
if !jr.shouldLimitBatches {
bytesLimit = rowinfra.NoBytesLimit
} else {
bytesLimit = jr.lookupBatchBytesLimit
if jr.lookupBatchBytesLimit == 0 {
bytesLimit = rowinfra.GetDefaultBatchBytesLimit(jr.EvalCtx.TestingKnobs.ForceProductionValues)
}
}
}
if err = jr.fetcher.StartScan(
jr.Ctx, spans, spanIDs, bytesLimit, rowinfra.NoRowLimit,
); 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) {
for {
// Fetch the next row and tell the strategy to process it.
lookedUpRow, spanID, err := jr.fetcher.NextRow(jr.Ctx)
if err != nil {