forked from pingcap/tidb
-
Notifications
You must be signed in to change notification settings - Fork 0
/
Copy pathdistsql.go
1458 lines (1336 loc) · 43.8 KB
/
distsql.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 2017 PingCAP, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package executor
import (
"bytes"
"context"
"fmt"
"runtime/trace"
"sort"
"strings"
"sync"
"sync/atomic"
"time"
"unsafe"
"github.com/pingcap/errors"
"github.com/pingcap/tidb/distsql"
"github.com/pingcap/tidb/expression"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/parser/charset"
"github.com/pingcap/tidb/parser/model"
"github.com/pingcap/tidb/parser/mysql"
"github.com/pingcap/tidb/parser/terror"
plannercore "github.com/pingcap/tidb/planner/core"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/sessionctx/stmtctx"
"github.com/pingcap/tidb/statistics"
"github.com/pingcap/tidb/table"
"github.com/pingcap/tidb/table/tables"
"github.com/pingcap/tidb/tablecodec"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util"
"github.com/pingcap/tidb/util/channel"
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/codec"
"github.com/pingcap/tidb/util/collate"
"github.com/pingcap/tidb/util/execdetails"
"github.com/pingcap/tidb/util/logutil"
"github.com/pingcap/tidb/util/logutil/consistency"
"github.com/pingcap/tidb/util/mathutil"
"github.com/pingcap/tidb/util/memory"
"github.com/pingcap/tidb/util/ranger"
"github.com/pingcap/tipb/go-tipb"
"go.uber.org/zap"
"golang.org/x/exp/slices"
)
var (
_ Executor = &TableReaderExecutor{}
_ Executor = &IndexReaderExecutor{}
_ Executor = &IndexLookUpExecutor{}
)
// LookupTableTaskChannelSize represents the channel size of the index double read taskChan.
var LookupTableTaskChannelSize int32 = 50
// lookupTableTask is created from a partial result of an index request which
// contains the handles in those index keys.
type lookupTableTask struct {
handles []kv.Handle
rowIdx []int // rowIdx represents the handle index for every row. Only used when keep order.
rows []chunk.Row
idxRows *chunk.Chunk
cursor int
// after the cop task is built, buildDone will be set to the current instant, for Next wait duration statistic.
buildDoneTime time.Time
doneCh chan error
// indexOrder map is used to save the original index order for the handles.
// Without this map, the original index order might be lost.
// The handles fetched from index is originally ordered by index, but we need handles to be ordered by itself
// to do table request.
indexOrder *kv.HandleMap
// duplicatedIndexOrder map likes indexOrder. But it's used when checkIndexValue isn't nil and
// the same handle of index has multiple values.
duplicatedIndexOrder *kv.HandleMap
// partitionTable indicates whether this task belongs to a partition table and which partition table it is.
partitionTable table.PhysicalTable
// memUsage records the memory usage of this task calculated by table worker.
// memTracker is used to release memUsage after task is done and unused.
//
// The sequence of function calls are:
// 1. calculate task.memUsage.
// 2. task.memTracker = tableWorker.memTracker
// 3. task.memTracker.Consume(task.memUsage)
// 4. task.memTracker.Consume(-task.memUsage)
//
// Step 1~3 are completed in "tableWorker.executeTask".
// Step 4 is completed in "IndexLookUpExecutor.Next".
memUsage int64
memTracker *memory.Tracker
}
func (task *lookupTableTask) Len() int {
return len(task.rows)
}
func (task *lookupTableTask) Less(i, j int) bool {
return task.rowIdx[i] < task.rowIdx[j]
}
func (task *lookupTableTask) Swap(i, j int) {
task.rowIdx[i], task.rowIdx[j] = task.rowIdx[j], task.rowIdx[i]
task.rows[i], task.rows[j] = task.rows[j], task.rows[i]
}
// Closeable is a interface for closeable structures.
type Closeable interface {
// Close closes the object.
Close() error
}
// closeAll closes all objects even if an object returns an error.
// If multiple objects returns error, the first error will be returned.
func closeAll(objs ...Closeable) error {
var err error
for _, obj := range objs {
if obj != nil {
err1 := obj.Close()
if err == nil && err1 != nil {
err = err1
}
}
}
if err != nil {
return errors.Trace(err)
}
return nil
}
// rebuildIndexRanges will be called if there's correlated column in access conditions. We will rebuild the range
// by substituting correlated column with the constant.
func rebuildIndexRanges(ctx sessionctx.Context, is *plannercore.PhysicalIndexScan, idxCols []*expression.Column, colLens []int) (ranges []*ranger.Range, err error) {
access := make([]expression.Expression, 0, len(is.AccessCondition))
for _, cond := range is.AccessCondition {
newCond, err1 := expression.SubstituteCorCol2Constant(cond)
if err1 != nil {
return nil, err1
}
access = append(access, newCond)
}
// All of access conditions must be used to build ranges, so we don't limit range memory usage.
ranges, _, err = ranger.DetachSimpleCondAndBuildRangeForIndex(ctx, access, idxCols, colLens, 0)
return ranges, err
}
// IndexReaderExecutor sends dag request and reads index data from kv layer.
type IndexReaderExecutor struct {
baseExecutor
// For a partitioned table, the IndexReaderExecutor works on a partition, so
// the type of this table field is actually `table.PhysicalTable`.
table table.Table
index *model.IndexInfo
physicalTableID int64
ranges []*ranger.Range
partitions []table.PhysicalTable
partRangeMap map[int64][]*ranger.Range // each partition may have different ranges
// kvRanges are only used for union scan.
kvRanges []kv.KeyRange
dagPB *tipb.DAGRequest
startTS uint64
txnScope string
readReplicaScope string
isStaleness bool
netDataSize float64
// result returns one or more distsql.PartialResult and each PartialResult is returned by one region.
result distsql.SelectResult
// columns are only required by union scan.
columns []*model.ColumnInfo
// outputColumns are only required by union scan.
outputColumns []*expression.Column
feedback *statistics.QueryFeedback
paging bool
keepOrder bool
desc bool
corColInFilter bool
corColInAccess bool
idxCols []*expression.Column
colLens []int
plans []plannercore.PhysicalPlan
memTracker *memory.Tracker
selectResultHook // for testing
// If dummy flag is set, this is not a real IndexReader, it just provides the KV ranges for UnionScan.
// Used by the temporary table, cached table.
dummy bool
}
// Table implements the dataSourceExecutor interface.
func (e *IndexReaderExecutor) Table() table.Table {
return e.table
}
func (e *IndexReaderExecutor) setDummy() {
e.dummy = true
}
// Close clears all resources hold by current object.
func (e *IndexReaderExecutor) Close() (err error) {
if e.result != nil {
err = e.result.Close()
}
e.result = nil
e.kvRanges = e.kvRanges[:0]
if e.dummy {
return nil
}
e.ctx.StoreQueryFeedback(e.feedback)
return err
}
// Next implements the Executor Next interface.
func (e *IndexReaderExecutor) Next(ctx context.Context, req *chunk.Chunk) error {
if e.dummy {
req.Reset()
return nil
}
err := e.result.Next(ctx, req)
if err != nil {
e.feedback.Invalidate()
}
return err
}
// TODO: cleanup this method.
func (e *IndexReaderExecutor) buildKeyRanges(sc *stmtctx.StatementContext, ranges []*ranger.Range, physicalID int64) ([]kv.KeyRange, error) {
var (
rRanges *kv.KeyRanges
err error
)
if e.index.ID == -1 {
rRanges, err = distsql.CommonHandleRangesToKVRanges(sc, []int64{physicalID}, ranges)
} else {
rRanges, err = distsql.IndexRangesToKVRanges(sc, physicalID, e.index.ID, ranges, e.feedback)
}
return rRanges.FirstPartitionRange(), err
}
// Open implements the Executor Open interface.
func (e *IndexReaderExecutor) Open(ctx context.Context) error {
var err error
if e.corColInAccess {
e.ranges, err = rebuildIndexRanges(e.ctx, e.plans[0].(*plannercore.PhysicalIndexScan), e.idxCols, e.colLens)
if err != nil {
return err
}
}
sc := e.ctx.GetSessionVars().StmtCtx
var kvRanges []kv.KeyRange
if len(e.partitions) > 0 {
for _, p := range e.partitions {
partRange := e.ranges
if pRange, ok := e.partRangeMap[p.GetPhysicalID()]; ok {
partRange = pRange
}
kvRange, err := e.buildKeyRanges(sc, partRange, p.GetPhysicalID())
if err != nil {
return err
}
kvRanges = append(kvRanges, kvRange...)
}
} else {
kvRanges, err = e.buildKeyRanges(sc, e.ranges, e.physicalTableID)
}
if err != nil {
return err
}
return e.open(ctx, kvRanges)
}
func (e *IndexReaderExecutor) open(ctx context.Context, kvRanges []kv.KeyRange) error {
var err error
if e.corColInFilter {
e.dagPB.Executors, err = constructDistExec(e.ctx, e.plans)
if err != nil {
return err
}
}
if e.runtimeStats != nil {
collExec := true
e.dagPB.CollectExecutionSummaries = &collExec
}
e.kvRanges = kvRanges
// Treat temporary table as dummy table, avoid sending distsql request to TiKV.
// In a test case IndexReaderExecutor is mocked and e.table is nil.
// Avoid sending distsql request to TIKV.
if e.dummy {
return nil
}
e.memTracker = memory.NewTracker(e.id, -1)
e.memTracker.AttachTo(e.ctx.GetSessionVars().StmtCtx.MemTracker)
slices.SortFunc(kvRanges, func(i, j kv.KeyRange) bool {
return bytes.Compare(i.StartKey, j.StartKey) < 0
})
var builder distsql.RequestBuilder
builder.SetKeyRanges(kvRanges).
SetDAGRequest(e.dagPB).
SetStartTS(e.startTS).
SetDesc(e.desc).
SetKeepOrder(e.keepOrder).
SetTxnScope(e.txnScope).
SetReadReplicaScope(e.readReplicaScope).
SetIsStaleness(e.isStaleness).
SetFromSessionVars(e.ctx.GetSessionVars()).
SetFromInfoSchema(e.ctx.GetInfoSchema()).
SetMemTracker(e.memTracker).
SetClosestReplicaReadAdjuster(newClosestReadAdjuster(e.ctx, &builder.Request, e.netDataSize))
kvReq, err := builder.Build()
if err != nil {
e.feedback.Invalidate()
return err
}
e.result, err = e.SelectResult(ctx, e.ctx, kvReq, retTypes(e), e.feedback, getPhysicalPlanIDs(e.plans), e.id)
if err != nil {
e.feedback.Invalidate()
return err
}
return nil
}
// IndexLookUpExecutor implements double read for index scan.
type IndexLookUpExecutor struct {
baseExecutor
table table.Table
index *model.IndexInfo
ranges []*ranger.Range
dagPB *tipb.DAGRequest
startTS uint64
// handleIdx is the index of handle, which is only used for case of keeping order.
handleIdx []int
handleCols []*expression.Column
primaryKeyIndex *model.IndexInfo
tableRequest *tipb.DAGRequest
// columns are only required by union scan.
columns []*model.ColumnInfo
*dataReaderBuilder
idxNetDataSize float64
avgRowSize float64
// fields about accessing partition tables
partitionTableMode bool // if this executor is accessing a partition table
prunedPartitions []table.PhysicalTable // partition tables need to access
partitionRangeMap map[int64][]*ranger.Range
partitionKVRanges [][]kv.KeyRange // kvRanges of each prunedPartitions
// All fields above are immutable.
idxWorkerWg sync.WaitGroup
tblWorkerWg sync.WaitGroup
finished chan struct{}
resultCh chan *lookupTableTask
resultCurr *lookupTableTask
feedback *statistics.QueryFeedback
// memTracker is used to track the memory usage of this executor.
memTracker *memory.Tracker
// checkIndexValue is used to check the consistency of the index data.
*checkIndexValue
kvRanges []kv.KeyRange
workerStarted bool
keepOrder bool
desc bool
indexPaging bool
corColInIdxSide bool
corColInTblSide bool
corColInAccess bool
idxPlans []plannercore.PhysicalPlan
tblPlans []plannercore.PhysicalPlan
idxCols []*expression.Column
colLens []int
// PushedLimit is used to skip the preceding and tailing handles when Limit is sunk into IndexLookUpReader.
PushedLimit *plannercore.PushedDownLimit
stats *IndexLookUpRunTimeStats
// cancelFunc is called when close the executor
cancelFunc context.CancelFunc
// If dummy flag is set, this is not a real IndexLookUpReader, it just provides the KV ranges for UnionScan.
// Used by the temporary table, cached table.
dummy bool
}
type getHandleType int8
const (
getHandleFromIndex getHandleType = iota
getHandleFromTable
)
// nolint:structcheck
type checkIndexValue struct {
idxColTps []*types.FieldType
idxTblCols []*table.Column
}
// Table implements the dataSourceExecutor interface.
func (e *IndexLookUpExecutor) Table() table.Table {
return e.table
}
func (e *IndexLookUpExecutor) setDummy() {
e.dummy = true
}
// Open implements the Executor Open interface.
func (e *IndexLookUpExecutor) Open(ctx context.Context) error {
var err error
if e.corColInAccess {
e.ranges, err = rebuildIndexRanges(e.ctx, e.idxPlans[0].(*plannercore.PhysicalIndexScan), e.idxCols, e.colLens)
if err != nil {
return err
}
}
err = e.buildTableKeyRanges()
if err != nil {
e.feedback.Invalidate()
return err
}
// Treat temporary table as dummy table, avoid sending distsql request to TiKV.
if e.dummy {
return nil
}
err = e.open(ctx)
if err != nil {
e.feedback.Invalidate()
}
return err
}
func (e *IndexLookUpExecutor) buildTableKeyRanges() (err error) {
sc := e.ctx.GetSessionVars().StmtCtx
if e.partitionTableMode {
e.feedback.Invalidate() // feedback for partition tables is not ready
e.partitionKVRanges = make([][]kv.KeyRange, 0, len(e.prunedPartitions))
for _, p := range e.prunedPartitions {
// TODO: prune and adjust e.ranges for each partition again, since not all e.ranges are suitable for all e.prunedPartitions.
// For example, a table partitioned by range(a), and p0=(1, 10), p1=(11, 20), for the condition "(a>1 and a<10) or (a>11 and a<20)",
// the first range is only suitable to p0 and the second is to p1, but now we'll also build kvRange for range0+p1 and range1+p0.
physicalID := p.GetPhysicalID()
ranges := e.ranges
if e.partitionRangeMap != nil && e.partitionRangeMap[physicalID] != nil {
ranges = e.partitionRangeMap[physicalID]
}
var kvRange *kv.KeyRanges
if e.index.ID == -1 {
kvRange, err = distsql.CommonHandleRangesToKVRanges(sc, []int64{physicalID}, ranges)
} else {
kvRange, err = distsql.IndexRangesToKVRanges(sc, physicalID, e.index.ID, ranges, e.feedback)
}
if err != nil {
return err
}
e.partitionKVRanges = append(e.partitionKVRanges, kvRange.FirstPartitionRange())
}
} else {
physicalID := getPhysicalTableID(e.table)
var kvRanges *kv.KeyRanges
if e.index.ID == -1 {
kvRanges, err = distsql.CommonHandleRangesToKVRanges(sc, []int64{physicalID}, e.ranges)
} else {
kvRanges, err = distsql.IndexRangesToKVRanges(sc, physicalID, e.index.ID, e.ranges, e.feedback)
}
e.kvRanges = kvRanges.FirstPartitionRange()
}
return err
}
func (e *IndexLookUpExecutor) open(ctx context.Context) error {
// We have to initialize "memTracker" and other execution resources in here
// instead of in function "Open", because this "IndexLookUpExecutor" may be
// constructed by a "IndexLookUpJoin" and "Open" will not be called in that
// situation.
e.initRuntimeStats()
e.memTracker = memory.NewTracker(e.id, -1)
e.memTracker.AttachTo(e.ctx.GetSessionVars().StmtCtx.MemTracker)
e.finished = make(chan struct{})
e.resultCh = make(chan *lookupTableTask, atomic.LoadInt32(&LookupTableTaskChannelSize))
var err error
if e.corColInIdxSide {
e.dagPB.Executors, err = constructDistExec(e.ctx, e.idxPlans)
if err != nil {
return err
}
}
if e.corColInTblSide {
e.tableRequest.Executors, err = constructDistExec(e.ctx, e.tblPlans)
if err != nil {
return err
}
}
return nil
}
func (e *IndexLookUpExecutor) startWorkers(ctx context.Context, initBatchSize int) error {
// indexWorker will write to workCh and tableWorker will read from workCh,
// so fetching index and getting table data can run concurrently.
ctx, cancel := context.WithCancel(ctx)
e.cancelFunc = cancel
workCh := make(chan *lookupTableTask, 1)
if err := e.startIndexWorker(ctx, workCh, initBatchSize); err != nil {
return err
}
e.startTableWorker(ctx, workCh)
e.workerStarted = true
return nil
}
func (e *IndexLookUpExecutor) isCommonHandle() bool {
return !(len(e.handleCols) == 1 && e.handleCols[0].ID == model.ExtraHandleID) && e.table.Meta() != nil && e.table.Meta().IsCommonHandle
}
func (e *IndexLookUpExecutor) getRetTpsByHandle() []*types.FieldType {
var tps []*types.FieldType
if e.isCommonHandle() {
for _, handleCol := range e.handleCols {
tps = append(tps, handleCol.RetType)
}
} else {
tps = []*types.FieldType{types.NewFieldType(mysql.TypeLonglong)}
}
if e.index.Global {
tps = append(tps, types.NewFieldType(mysql.TypeLonglong))
}
if e.checkIndexValue != nil {
tps = e.idxColTps
}
return tps
}
// startIndexWorker launch a background goroutine to fetch handles, send the results to workCh.
func (e *IndexLookUpExecutor) startIndexWorker(ctx context.Context, workCh chan<- *lookupTableTask, initBatchSize int) error {
if e.runtimeStats != nil {
collExec := true
e.dagPB.CollectExecutionSummaries = &collExec
}
tracker := memory.NewTracker(memory.LabelForIndexWorker, -1)
tracker.AttachTo(e.memTracker)
kvRanges := [][]kv.KeyRange{e.kvRanges}
if e.partitionTableMode {
kvRanges = e.partitionKVRanges
}
tps := e.getRetTpsByHandle()
idxID := e.getIndexPlanRootID()
e.idxWorkerWg.Add(1)
go func() {
defer trace.StartRegion(ctx, "IndexLookUpIndexWorker").End()
worker := &indexWorker{
idxLookup: e,
workCh: workCh,
finished: e.finished,
resultCh: e.resultCh,
keepOrder: e.keepOrder,
checkIndexValue: e.checkIndexValue,
maxBatchSize: e.ctx.GetSessionVars().IndexLookupSize,
maxChunkSize: e.maxChunkSize,
PushedLimit: e.PushedLimit,
}
var builder distsql.RequestBuilder
builder.SetDAGRequest(e.dagPB).
SetStartTS(e.startTS).
SetDesc(e.desc).
SetKeepOrder(e.keepOrder).
SetPaging(e.indexPaging).
SetTxnScope(e.txnScope).
SetReadReplicaScope(e.readReplicaScope).
SetIsStaleness(e.isStaleness).
SetFromSessionVars(e.ctx.GetSessionVars()).
SetFromInfoSchema(e.ctx.GetInfoSchema()).
SetClosestReplicaReadAdjuster(newClosestReadAdjuster(e.ctx, &builder.Request, e.idxNetDataSize/float64(len(kvRanges)))).
SetMemTracker(tracker)
for partTblIdx, kvRange := range kvRanges {
// check if executor is closed
finished := false
select {
case <-e.finished:
finished = true
default:
}
if finished {
break
}
if worker.PushedLimit != nil && worker.scannedKeys >= worker.PushedLimit.Count+worker.PushedLimit.Offset {
break
}
// init kvReq, result and worker for this partition
// The key ranges should be ordered.
slices.SortFunc(kvRange, func(i, j kv.KeyRange) bool {
return bytes.Compare(i.StartKey, j.StartKey) < 0
})
kvReq, err := builder.SetKeyRanges(kvRange).Build()
if err != nil {
worker.syncErr(err)
break
}
result, err := distsql.SelectWithRuntimeStats(ctx, e.ctx, kvReq, tps, e.feedback, getPhysicalPlanIDs(e.idxPlans), idxID)
if err != nil {
worker.syncErr(err)
break
}
worker.batchSize = initBatchSize
if worker.batchSize > worker.maxBatchSize {
worker.batchSize = worker.maxBatchSize
}
if e.partitionTableMode {
worker.partitionTable = e.prunedPartitions[partTblIdx]
}
// fetch data from this partition
ctx1, cancel := context.WithCancel(ctx)
fetchErr := worker.fetchHandles(ctx1, result)
if fetchErr != nil { // this error is synced in fetchHandles(), don't sync it again
e.feedback.Invalidate()
}
cancel()
if err := result.Close(); err != nil {
logutil.Logger(ctx).Error("close Select result failed", zap.Error(err))
}
e.ctx.StoreQueryFeedback(e.feedback)
if fetchErr != nil {
break // if any error occurs, exit after releasing all resources
}
}
close(workCh)
close(e.resultCh)
e.idxWorkerWg.Done()
}()
return nil
}
// startTableWorker launchs some background goroutines which pick tasks from workCh and execute the task.
func (e *IndexLookUpExecutor) startTableWorker(ctx context.Context, workCh <-chan *lookupTableTask) {
lookupConcurrencyLimit := e.ctx.GetSessionVars().IndexLookupConcurrency()
e.tblWorkerWg.Add(lookupConcurrencyLimit)
for i := 0; i < lookupConcurrencyLimit; i++ {
workerID := i
worker := &tableWorker{
idxLookup: e,
workCh: workCh,
finished: e.finished,
keepOrder: e.keepOrder,
handleIdx: e.handleIdx,
checkIndexValue: e.checkIndexValue,
memTracker: memory.NewTracker(workerID, -1),
}
worker.memTracker.AttachTo(e.memTracker)
ctx1, cancel := context.WithCancel(ctx)
go func() {
defer trace.StartRegion(ctx1, "IndexLookUpTableWorker").End()
worker.pickAndExecTask(ctx1)
cancel()
e.tblWorkerWg.Done()
}()
}
}
func (e *IndexLookUpExecutor) buildTableReader(ctx context.Context, task *lookupTableTask) (Executor, error) {
table := e.table
if e.partitionTableMode && task.partitionTable != nil {
table = task.partitionTable
}
tableReaderExec := &TableReaderExecutor{
baseExecutor: newBaseExecutor(e.ctx, e.schema, e.getTableRootPlanID()),
table: table,
dagPB: e.tableRequest,
startTS: e.startTS,
txnScope: e.txnScope,
readReplicaScope: e.readReplicaScope,
isStaleness: e.isStaleness,
columns: e.columns,
feedback: statistics.NewQueryFeedback(0, nil, 0, false),
corColInFilter: e.corColInTblSide,
plans: e.tblPlans,
netDataSize: e.avgRowSize * float64(len(task.handles)),
}
tableReaderExec.buildVirtualColumnInfo()
tableReader, err := e.dataReaderBuilder.buildTableReaderFromHandles(ctx, tableReaderExec, task.handles, true)
if err != nil {
logutil.Logger(ctx).Error("build table reader from handles failed", zap.Error(err))
return nil, err
}
return tableReader, nil
}
// Close implements Exec Close interface.
func (e *IndexLookUpExecutor) Close() error {
if e.stats != nil {
defer e.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.RegisterStats(e.id, e.stats)
}
e.kvRanges = e.kvRanges[:0]
if e.dummy {
return nil
}
if !e.workerStarted || e.finished == nil {
return nil
}
if e.cancelFunc != nil {
e.cancelFunc()
e.cancelFunc = nil
}
close(e.finished)
// Drain the resultCh and discard the result, in case that Next() doesn't fully
// consume the data, background worker still writing to resultCh and block forever.
channel.Clear(e.resultCh)
e.idxWorkerWg.Wait()
e.tblWorkerWg.Wait()
e.finished = nil
e.workerStarted = false
e.memTracker = nil
e.resultCurr = nil
return nil
}
// Next implements Exec Next interface.
func (e *IndexLookUpExecutor) Next(ctx context.Context, req *chunk.Chunk) error {
if e.dummy {
req.Reset()
return nil
}
if !e.workerStarted {
if err := e.startWorkers(ctx, req.RequiredRows()); err != nil {
return err
}
}
req.Reset()
for {
resultTask, err := e.getResultTask()
if err != nil {
return err
}
if resultTask == nil {
return nil
}
if resultTask.cursor < len(resultTask.rows) {
numToAppend := mathutil.Min(len(resultTask.rows)-resultTask.cursor, req.RequiredRows()-req.NumRows())
req.AppendRows(resultTask.rows[resultTask.cursor : resultTask.cursor+numToAppend])
resultTask.cursor += numToAppend
if req.IsFull() {
return nil
}
}
}
}
func (e *IndexLookUpExecutor) getResultTask() (*lookupTableTask, error) {
if e.resultCurr != nil && e.resultCurr.cursor < len(e.resultCurr.rows) {
return e.resultCurr, nil
}
var (
enableStats = e.stats != nil
start time.Time
indexFetchedInstant time.Time
)
if enableStats {
start = time.Now()
}
task, ok := <-e.resultCh
if !ok {
return nil, nil
}
if enableStats {
indexFetchedInstant = time.Now()
}
if err := <-task.doneCh; err != nil {
return nil, err
}
if enableStats {
e.stats.NextWaitIndexScan += indexFetchedInstant.Sub(start)
if task.buildDoneTime.After(indexFetchedInstant) {
e.stats.NextWaitTableLookUpBuild += task.buildDoneTime.Sub(indexFetchedInstant)
indexFetchedInstant = task.buildDoneTime
}
e.stats.NextWaitTableLookUpResp += time.Since(indexFetchedInstant)
}
// Release the memory usage of last task before we handle a new task.
if e.resultCurr != nil {
e.resultCurr.memTracker.Consume(-e.resultCurr.memUsage)
}
e.resultCurr = task
return e.resultCurr, nil
}
func (e *IndexLookUpExecutor) initRuntimeStats() {
if e.runtimeStats != nil {
e.stats = &IndexLookUpRunTimeStats{
indexScanBasicStats: &execdetails.BasicRuntimeStats{},
Concurrency: e.ctx.GetSessionVars().IndexLookupConcurrency(),
}
}
}
func (e *IndexLookUpExecutor) getIndexPlanRootID() int {
if len(e.idxPlans) > 0 {
return e.idxPlans[len(e.idxPlans)-1].ID()
}
return e.id
}
func (e *IndexLookUpExecutor) getTableRootPlanID() int {
if len(e.tblPlans) > 0 {
return e.tblPlans[len(e.tblPlans)-1].ID()
}
return e.id
}
// indexWorker is used by IndexLookUpExecutor to maintain index lookup background goroutines.
type indexWorker struct {
idxLookup *IndexLookUpExecutor
workCh chan<- *lookupTableTask
finished <-chan struct{}
resultCh chan<- *lookupTableTask
keepOrder bool
// batchSize is for lightweight startup. It will be increased exponentially until reaches the max batch size value.
batchSize int
maxBatchSize int
maxChunkSize int
// checkIndexValue is used to check the consistency of the index data.
*checkIndexValue
// PushedLimit is used to skip the preceding and tailing handles when Limit is sunk into IndexLookUpReader.
PushedLimit *plannercore.PushedDownLimit
// scannedKeys indicates how many keys be scanned
scannedKeys uint64
// partitionTable indicates if this worker is accessing a particular partition table.
partitionTable table.PhysicalTable
}
func (w *indexWorker) syncErr(err error) {
doneCh := make(chan error, 1)
doneCh <- err
w.resultCh <- &lookupTableTask{
doneCh: doneCh,
}
}
// fetchHandles fetches a batch of handles from index data and builds the index lookup tasks.
// The tasks are sent to workCh to be further processed by tableWorker, and sent to e.resultCh
// at the same time to keep data ordered.
func (w *indexWorker) fetchHandles(ctx context.Context, result distsql.SelectResult) (err error) {
defer func() {
if r := recover(); r != nil {
logutil.Logger(ctx).Error("indexWorker in IndexLookupExecutor panicked", zap.Any("recover", r), zap.Stack("stack"))
err4Panic := errors.Errorf("%v", r)
w.syncErr(err4Panic)
if err != nil {
err = errors.Trace(err4Panic)
}
}
}()
retTps := w.idxLookup.getRetTpsByHandle()
chk := w.idxLookup.ctx.GetSessionVars().GetNewChunkWithCapacity(retTps, w.idxLookup.maxChunkSize, w.idxLookup.maxChunkSize, w.idxLookup.AllocPool)
idxID := w.idxLookup.getIndexPlanRootID()
if w.idxLookup.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl != nil {
if idxID != w.idxLookup.id && w.idxLookup.stats != nil {
w.idxLookup.stats.indexScanBasicStats = w.idxLookup.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.GetBasicRuntimeStats(idxID)
}
}
for {
startTime := time.Now()
handles, retChunk, err := w.extractTaskHandles(ctx, chk, result)
finishFetch := time.Now()
if err != nil {
w.syncErr(err)
return err
}
if len(handles) == 0 {
return nil
}
task := w.buildTableTask(handles, retChunk)
finishBuild := time.Now()
select {
case <-ctx.Done():
return nil
case <-w.finished:
return nil
case w.workCh <- task:
w.resultCh <- task
}
if w.idxLookup.stats != nil {
atomic.AddInt64(&w.idxLookup.stats.FetchHandle, int64(finishFetch.Sub(startTime)))
atomic.AddInt64(&w.idxLookup.stats.TaskWait, int64(time.Since(finishBuild)))
atomic.AddInt64(&w.idxLookup.stats.FetchHandleTotal, int64(time.Since(startTime)))
}
}
}
func (w *indexWorker) extractTaskHandles(ctx context.Context, chk *chunk.Chunk, idxResult distsql.SelectResult) (
handles []kv.Handle, retChk *chunk.Chunk, err error) {
numColsWithoutPid := chk.NumCols()
if w.idxLookup.index.Global {
numColsWithoutPid = numColsWithoutPid - 1
}
handleOffset := make([]int, 0, len(w.idxLookup.handleCols))
for i := range w.idxLookup.handleCols {
handleOffset = append(handleOffset, numColsWithoutPid-len(w.idxLookup.handleCols)+i)
}
if len(handleOffset) == 0 {
handleOffset = []int{numColsWithoutPid - 1}
}
handles = make([]kv.Handle, 0, w.batchSize)
// PushedLimit would always be nil for CheckIndex or CheckTable, we add this check just for insurance.
checkLimit := (w.PushedLimit != nil) && (w.checkIndexValue == nil)
for len(handles) < w.batchSize {
requiredRows := w.batchSize - len(handles)
if checkLimit {
if w.PushedLimit.Offset+w.PushedLimit.Count <= w.scannedKeys {
return handles, nil, nil
}
leftCnt := w.PushedLimit.Offset + w.PushedLimit.Count - w.scannedKeys
if uint64(requiredRows) > leftCnt {
requiredRows = int(leftCnt)
}
}
chk.SetRequiredRows(requiredRows, w.maxChunkSize)
startTime := time.Now()
err = errors.Trace(idxResult.Next(ctx, chk))
if err != nil {
return handles, nil, err
}
if w.idxLookup.stats != nil {
w.idxLookup.stats.indexScanBasicStats.Record(time.Since(startTime), chk.NumRows())
}
if chk.NumRows() == 0 {
return handles, retChk, nil
}
for i := 0; i < chk.NumRows(); i++ {
w.scannedKeys++
if checkLimit {
if w.scannedKeys <= w.PushedLimit.Offset {
continue
}
if w.scannedKeys > (w.PushedLimit.Offset + w.PushedLimit.Count) {
// Skip the handles after Offset+Count.
return handles, nil, nil
}
}
h, err := w.idxLookup.getHandle(chk.GetRow(i), handleOffset, w.idxLookup.isCommonHandle(), getHandleFromIndex)
if err != nil {
return handles, retChk, err
}
handles = append(handles, h)
}
if w.checkIndexValue != nil {
if retChk == nil {
retChk = chunk.NewChunkWithCapacity(w.idxColTps, w.batchSize)
}
retChk.Append(chk, 0, chk.NumRows())
}
}
w.batchSize *= 2
if w.batchSize > w.maxBatchSize {
w.batchSize = w.maxBatchSize