-
Notifications
You must be signed in to change notification settings - Fork 5.9k
/
coprocessor.go
1690 lines (1564 loc) · 54.7 KB
/
coprocessor.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 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 copr
import (
"context"
"fmt"
"math"
"strconv"
"strings"
"sync"
"sync/atomic"
"time"
"unsafe"
"github.com/gogo/protobuf/proto"
"github.com/pingcap/errors"
"github.com/pingcap/failpoint"
"github.com/pingcap/kvproto/pkg/coprocessor"
"github.com/pingcap/kvproto/pkg/kvrpcpb"
"github.com/pingcap/kvproto/pkg/metapb"
"github.com/pingcap/tidb/domain/infosync"
"github.com/pingcap/tidb/errno"
"github.com/pingcap/tidb/kv"
tidbmetrics "github.com/pingcap/tidb/metrics"
"github.com/pingcap/tidb/parser/terror"
"github.com/pingcap/tidb/store/driver/backoff"
derr "github.com/pingcap/tidb/store/driver/error"
"github.com/pingcap/tidb/store/driver/options"
"github.com/pingcap/tidb/util/execdetails"
"github.com/pingcap/tidb/util/logutil"
"github.com/pingcap/tidb/util/mathutil"
"github.com/pingcap/tidb/util/memory"
"github.com/pingcap/tidb/util/paging"
"github.com/pingcap/tidb/util/trxevents"
"github.com/pingcap/tipb/go-tipb"
"github.com/tikv/client-go/v2/metrics"
"github.com/tikv/client-go/v2/tikv"
"github.com/tikv/client-go/v2/tikvrpc"
"github.com/tikv/client-go/v2/txnkv/txnlock"
"github.com/tikv/client-go/v2/txnkv/txnsnapshot"
"github.com/tikv/client-go/v2/util"
"go.uber.org/zap"
)
var coprCacheCounterEvict = tidbmetrics.DistSQLCoprCacheCounter.WithLabelValues("evict")
var (
coprCacheCounterHit = tidbmetrics.DistSQLCoprCacheCounter.WithLabelValues("hit")
coprCacheCounterMiss = tidbmetrics.DistSQLCoprCacheCounter.WithLabelValues("miss")
)
// Maximum total sleep time(in ms) for kv/cop commands.
const (
copBuildTaskMaxBackoff = 5000
copNextMaxBackoff = 20000
CopSmallTaskRow = 32 // 32 is the initial batch size of TiKV
smallTaskSigma = 0.5
)
// CopClient is coprocessor client.
type CopClient struct {
kv.RequestTypeSupportedChecker
store *Store
replicaReadSeed uint32
}
// Send builds the request and gets the coprocessor iterator response.
func (c *CopClient) Send(ctx context.Context, req *kv.Request, variables interface{}, option *kv.ClientSendOption) kv.Response {
vars, ok := variables.(*tikv.Variables)
if !ok {
return copErrorResponse{errors.Errorf("unsupported variables:%+v", variables)}
}
if req.StoreType == kv.TiFlash && req.BatchCop {
logutil.BgLogger().Debug("send batch requests")
return c.sendBatch(ctx, req, vars, option)
}
ctx = context.WithValue(ctx, tikv.TxnStartKey(), req.StartTs)
ctx = context.WithValue(ctx, util.RequestSourceKey, req.RequestSource)
enabledRateLimitAction := option.EnabledRateLimitAction
sessionMemTracker := option.SessionMemTracker
it, errRes := c.BuildCopIterator(ctx, req, vars, option)
if errRes != nil {
return errRes
}
ctx = context.WithValue(ctx, tikv.RPCCancellerCtxKey{}, it.rpcCancel)
if sessionMemTracker != nil && enabledRateLimitAction {
sessionMemTracker.FallbackOldAndSetNewAction(it.actionOnExceed)
}
it.open(ctx, enabledRateLimitAction, option.EnableCollectExecutionInfo)
return it
}
// BuildCopIterator builds the iterator without calling `open`.
func (c *CopClient) BuildCopIterator(ctx context.Context, req *kv.Request, vars *tikv.Variables, option *kv.ClientSendOption) (*copIterator, kv.Response) {
eventCb := option.EventCb
failpoint.Inject("DisablePaging", func(_ failpoint.Value) {
req.Paging.Enable = false
})
if req.StoreType == kv.TiDB {
// coprocessor on TiDB doesn't support paging
req.Paging.Enable = false
req.FixedRowCountHint = nil
}
if req.Tp != kv.ReqTypeDAG {
// coprocessor request but type is not DAG
req.Paging.Enable = false
}
failpoint.Inject("checkKeyRangeSortedForPaging", func(_ failpoint.Value) {
if req.Paging.Enable {
if !req.KeyRanges.IsFullySorted() {
logutil.BgLogger().Fatal("distsql request key range not sorted!")
}
}
})
if req.RequestSource.RequestSourceInternal || req.Tp != kv.ReqTypeDAG {
// disable extra concurrency for internal tasks.
req.FixedRowCountHint = nil
}
failpoint.Inject("disableFixedRowCountHint", func(_ failpoint.Value) {
req.FixedRowCountHint = nil
})
if req.Tp != kv.ReqTypeDAG || req.StoreType != kv.TiKV {
req.StoreBatchSize = 0
}
// TODO: support keep-order batch
if req.ReplicaRead != kv.ReplicaReadLeader || req.KeepOrder {
// disable batch copr for follower read
req.StoreBatchSize = 0
}
// disable paging for batch copr
if req.Paging.Enable {
req.StoreBatchSize = 0
}
bo := backoff.NewBackofferWithVars(ctx, copBuildTaskMaxBackoff, vars)
var (
tasks []*copTask
err error
)
buildTaskFunc := func(ranges []kv.KeyRange) error {
keyRanges := NewKeyRanges(ranges)
tasksFromRanges, err := buildCopTasks(bo, c.store.GetRegionCache(), keyRanges, req, eventCb)
if err != nil {
return err
}
if len(tasks) == 0 {
tasks = tasksFromRanges
return nil
}
tasks = append(tasks, tasksFromRanges...)
return nil
}
// Here we build the task by partition, not directly by region.
// This is because it's possible that TiDB merge multiple small partition into one region which break some assumption.
// Keep it split by partition would be more safe.
err = req.KeyRanges.ForEachPartitionWithErr(buildTaskFunc)
// only batch store requests in first build.
req.StoreBatchSize = 0
reqType := "null"
if req.ClosestReplicaReadAdjuster != nil {
reqType = "miss"
if req.ClosestReplicaReadAdjuster(req, len(tasks)) {
reqType = "hit"
}
}
tidbmetrics.DistSQLCoprClosestReadCounter.WithLabelValues(reqType).Inc()
if err != nil {
return nil, copErrorResponse{err}
}
it := &copIterator{
store: c.store,
req: req,
concurrency: req.Concurrency,
finishCh: make(chan struct{}),
vars: vars,
memTracker: req.MemTracker,
replicaReadSeed: c.replicaReadSeed,
rpcCancel: tikv.NewRPCanceller(),
}
it.tasks = tasks
if it.concurrency > len(tasks) {
it.concurrency = len(tasks)
}
if req.FixedRowCountHint != nil {
var smallTasks int
smallTasks, it.smallTaskConcurrency = smallTaskConcurrency(tasks)
if len(tasks)-smallTasks < it.concurrency {
it.concurrency = len(tasks) - smallTasks
}
}
if it.concurrency < 1 {
// Make sure that there is at least one worker.
it.concurrency = 1
}
if it.req.KeepOrder {
// Don't set high concurrency for the keep order case. It wastes a lot of memory and gains nothing.
// TL;DR
// Because for a keep order coprocessor request, the cop tasks are handled one by one, if we set a
// higher concurrency, the data is just cached and not consumed for a while, this increase the memory usage.
// Set concurrency to 2 can reduce the memory usage and I've tested that it does not necessarily
// decrease the performance.
if it.concurrency > 2 {
oldConcurrency := it.concurrency
it.concurrency = 2
failpoint.Inject("testRateLimitActionMockConsumeAndAssert", func(val failpoint.Value) {
if val.(bool) {
// When the concurrency is too small, test case tests/realtikvtest/sessiontest.TestCoprocessorOOMAction can't trigger OOM condition
it.concurrency = oldConcurrency
}
})
}
if it.smallTaskConcurrency > 20 {
it.smallTaskConcurrency = 20
}
it.sendRate = util.NewRateLimit(2 * (it.concurrency + it.smallTaskConcurrency))
it.respChan = nil
} else {
it.respChan = make(chan *copResponse)
it.sendRate = util.NewRateLimit(it.concurrency + it.smallTaskConcurrency)
}
it.actionOnExceed = newRateLimitAction(uint(it.sendRate.GetCapacity()))
return it, nil
}
// copTask contains a related Region and KeyRange for a kv.Request.
type copTask struct {
taskID uint64
region tikv.RegionVerID
bucketsVer uint64
ranges *KeyRanges
respChan chan *copResponse
storeAddr string
cmdType tikvrpc.CmdType
storeType kv.StoreType
eventCb trxevents.EventCallback
paging bool
pagingSize uint64
pagingTaskIdx uint32
partitionIndex int64 // used by balanceBatchCopTask in PartitionTableScan
requestSource util.RequestSource
RowCountHint int // used for extra concurrency of small tasks, -1 for unknown row count
batchTaskList map[uint64]*batchedCopTask
}
type batchedCopTask struct {
task *copTask
region coprocessor.RegionInfo
storeID uint64
peer *metapb.Peer
}
func (r *copTask) String() string {
return fmt.Sprintf("region(%d %d %d) ranges(%d) store(%s)",
r.region.GetID(), r.region.GetConfVer(), r.region.GetVer(), r.ranges.Len(), r.storeAddr)
}
func (r *copTask) ToPBBatchTasks() []*coprocessor.StoreBatchTask {
if len(r.batchTaskList) == 0 {
return nil
}
pbTasks := make([]*coprocessor.StoreBatchTask, 0, len(r.batchTaskList))
for _, task := range r.batchTaskList {
pbTasks = append(pbTasks, &coprocessor.StoreBatchTask{
RegionId: task.region.GetRegionId(),
RegionEpoch: task.region.GetRegionEpoch(),
Peer: task.peer,
Ranges: task.region.GetRanges(),
TaskId: task.task.taskID,
})
}
return pbTasks
}
// rangesPerTask limits the length of the ranges slice sent in one copTask.
const rangesPerTask = 25000
func buildCopTasks(bo *Backoffer, cache *RegionCache, ranges *KeyRanges, req *kv.Request, eventCb trxevents.EventCallback) ([]*copTask, error) {
start := time.Now()
cmdType := tikvrpc.CmdCop
if req.StoreType == kv.TiDB {
return buildTiDBMemCopTasks(ranges, req)
}
rangesLen := ranges.Len()
// TODO(youjiali1995): is there any request type that needn't be splitted by buckets?
locs, err := cache.SplitKeyRangesByBuckets(bo, ranges)
if err != nil {
return nil, errors.Trace(err)
}
// Channel buffer is 2 for handling region split.
// In a common case, two region split tasks will not be blocked.
chanSize := 2
// in paging request, a request will be returned in multi batches,
// enlarge the channel size to avoid the request blocked by buffer full.
if req.Paging.Enable {
chanSize = 18
}
tasks := make([]*copTask, 0, len(locs))
origRangeIdx := 0
taskID := uint64(0)
var store2Idx map[uint64]int
if req.StoreBatchSize > 0 {
store2Idx = make(map[uint64]int, 16)
}
for _, loc := range locs {
// TiKV will return gRPC error if the message is too large. So we need to limit the length of the ranges slice
// to make sure the message can be sent successfully.
rLen := loc.Ranges.Len()
// If this is a paging request, we set the paging size to minPagingSize,
// the size will grow every round.
pagingSize := uint64(0)
if req.Paging.Enable {
pagingSize = req.Paging.MinPagingSize
}
for i := 0; i < rLen; {
nextI := mathutil.Min(i+rangesPerTask, rLen)
hint := -1
// calculate the row count hint
if req.FixedRowCountHint != nil {
startKey, endKey := loc.Ranges.At(i).StartKey, loc.Ranges.At(nextI-1).EndKey
// move to the previous range if startKey of current range is lower than endKey of previous location.
// In the following example, task1 will move origRangeIdx to region(i, z).
// When counting the row hint for task2, we need to move origRangeIdx back to region(a, h).
// |<- region(a, h) ->| |<- region(i, z) ->|
// |<- task1 ->| |<- task2 ->| ...
if origRangeIdx > 0 && ranges.At(origRangeIdx-1).EndKey.Cmp(startKey) > 0 {
origRangeIdx--
}
hint = 0
for nextOrigRangeIdx := origRangeIdx; nextOrigRangeIdx < ranges.Len(); nextOrigRangeIdx++ {
rangeStart := ranges.At(nextOrigRangeIdx).StartKey
if rangeStart.Cmp(endKey) > 0 {
origRangeIdx = nextOrigRangeIdx
break
}
hint += req.FixedRowCountHint[nextOrigRangeIdx]
}
}
task := &copTask{
taskID: taskID,
region: loc.Location.Region,
bucketsVer: loc.getBucketVersion(),
ranges: loc.Ranges.Slice(i, nextI),
respChan: make(chan *copResponse, chanSize),
cmdType: cmdType,
storeType: req.StoreType,
eventCb: eventCb,
paging: req.Paging.Enable,
pagingSize: pagingSize,
requestSource: req.RequestSource,
RowCountHint: hint,
}
if req.StoreBatchSize > 0 {
batchedTask, err := cache.BuildBatchTask(bo, task, req.ReplicaRead)
if err != nil {
return nil, err
}
if idx, ok := store2Idx[batchedTask.storeID]; !ok || len(tasks[idx].batchTaskList) >= req.StoreBatchSize {
tasks = append(tasks, batchedTask.task)
store2Idx[batchedTask.storeID] = len(tasks) - 1
} else {
if tasks[idx].batchTaskList == nil {
tasks[idx].batchTaskList = make(map[uint64]*batchedCopTask, req.StoreBatchSize)
// disable paging for batched task.
tasks[idx].paging = false
tasks[idx].pagingSize = 0
}
if task.RowCountHint > 0 {
tasks[idx].RowCountHint += task.RowCountHint
}
tasks[idx].batchTaskList[taskID] = batchedTask
}
} else {
tasks = append(tasks, task)
}
i = nextI
if req.Paging.Enable {
pagingSize = paging.GrowPagingSize(pagingSize, req.Paging.MaxPagingSize)
}
taskID++
}
}
if req.Desc {
reverseTasks(tasks)
}
if elapsed := time.Since(start); elapsed > time.Millisecond*500 {
logutil.BgLogger().Warn("buildCopTasks takes too much time",
zap.Duration("elapsed", elapsed),
zap.Int("range len", rangesLen),
zap.Int("task len", len(tasks)))
}
metrics.TxnRegionsNumHistogramWithCoprocessor.Observe(float64(len(tasks)))
return tasks, nil
}
func buildTiDBMemCopTasks(ranges *KeyRanges, req *kv.Request) ([]*copTask, error) {
servers, err := infosync.GetAllServerInfo(context.Background())
if err != nil {
return nil, err
}
cmdType := tikvrpc.CmdCop
tasks := make([]*copTask, 0, len(servers))
for _, ser := range servers {
if req.TiDBServerID > 0 && req.TiDBServerID != ser.ServerIDGetter() {
continue
}
addr := ser.IP + ":" + strconv.FormatUint(uint64(ser.StatusPort), 10)
tasks = append(tasks, &copTask{
ranges: ranges,
respChan: make(chan *copResponse, 2),
cmdType: cmdType,
storeType: req.StoreType,
storeAddr: addr,
RowCountHint: -1,
})
}
return tasks, nil
}
func reverseTasks(tasks []*copTask) {
for i := 0; i < len(tasks)/2; i++ {
j := len(tasks) - i - 1
tasks[i], tasks[j] = tasks[j], tasks[i]
}
}
func isSmallTask(task *copTask) bool {
// strictly, only RowCountHint == -1 stands for unknown task rows,
// but when RowCountHint == 0, it may be caused by initialized value,
// to avoid the future bugs, let the tasks with RowCountHint == 0 be non-small tasks.
return task.RowCountHint > 0 && task.RowCountHint <= CopSmallTaskRow
}
// smallTaskConcurrency counts the small tasks of tasks,
// then returns the task count and extra concurrency for small tasks.
func smallTaskConcurrency(tasks []*copTask) (int, int) {
res := 0
for _, task := range tasks {
if isSmallTask(task) {
res++
}
}
if res == 0 {
return 0, 0
}
// Calculate the extra concurrency for small tasks
// extra concurrency = tasks / (1 + sigma * sqrt(log(tasks ^ 2)))
extraConc := float64(res) / (1 + smallTaskSigma*math.Sqrt(2*math.Log(float64(res))))
return res, int(extraConc)
}
type copIterator struct {
store *Store
req *kv.Request
concurrency int
smallTaskConcurrency int
finishCh chan struct{}
// If keepOrder, results are stored in copTask.respChan, read them out one by one.
tasks []*copTask
// curr indicates the curr id of the finished copTask
curr int
// sendRate controls the sending rate of copIteratorTaskSender
sendRate *util.RateLimit
// Otherwise, results are stored in respChan.
respChan chan *copResponse
vars *tikv.Variables
memTracker *memory.Tracker
replicaReadSeed uint32
rpcCancel *tikv.RPCCanceller
wg sync.WaitGroup
// closed represents when the Close is called.
// There are two cases we need to close the `finishCh` channel, one is when context is done, the other one is
// when the Close is called. we use atomic.CompareAndSwap `closed` to make sure the channel is not closed twice.
closed uint32
resolvedLocks util.TSSet
committedLocks util.TSSet
actionOnExceed *rateLimitAction
pagingTaskIdx uint32
}
// copIteratorWorker receives tasks from copIteratorTaskSender, handles tasks and sends the copResponse to respChan.
type copIteratorWorker struct {
taskCh <-chan *copTask
wg *sync.WaitGroup
store *Store
req *kv.Request
respChan chan<- *copResponse
finishCh <-chan struct{}
vars *tikv.Variables
kvclient *txnsnapshot.ClientHelper
memTracker *memory.Tracker
replicaReadSeed uint32
enableCollectExecutionInfo bool
pagingTaskIdx *uint32
}
// copIteratorTaskSender sends tasks to taskCh then wait for the workers to exit.
type copIteratorTaskSender struct {
taskCh chan<- *copTask
smallTaskCh chan<- *copTask
wg *sync.WaitGroup
tasks []*copTask
finishCh <-chan struct{}
respChan chan<- *copResponse
sendRate *util.RateLimit
}
type copResponse struct {
pbResp *coprocessor.Response
detail *CopRuntimeStats
startKey kv.Key
err error
respSize int64
respTime time.Duration
}
const sizeofExecDetails = int(unsafe.Sizeof(execdetails.ExecDetails{}))
// GetData implements the kv.ResultSubset GetData interface.
func (rs *copResponse) GetData() []byte {
return rs.pbResp.Data
}
// GetStartKey implements the kv.ResultSubset GetStartKey interface.
func (rs *copResponse) GetStartKey() kv.Key {
return rs.startKey
}
func (rs *copResponse) GetCopRuntimeStats() *CopRuntimeStats {
return rs.detail
}
// MemSize returns how many bytes of memory this response use
func (rs *copResponse) MemSize() int64 {
if rs.respSize != 0 {
return rs.respSize
}
if rs == finCopResp {
return 0
}
// ignore rs.err
rs.respSize += int64(cap(rs.startKey))
if rs.detail != nil {
rs.respSize += int64(sizeofExecDetails)
}
if rs.pbResp != nil {
// Using a approximate size since it's hard to get a accurate value.
rs.respSize += int64(rs.pbResp.Size())
}
return rs.respSize
}
func (rs *copResponse) RespTime() time.Duration {
return rs.respTime
}
const minLogCopTaskTime = 300 * time.Millisecond
// When the worker finished `handleTask`, we need to notify the copIterator that there is one task finished.
// For the non-keep-order case, we send a finCopResp into the respCh after `handleTask`. When copIterator recv
// finCopResp from the respCh, it will be aware that there is one task finished.
var finCopResp *copResponse
func init() {
finCopResp = &copResponse{}
}
// run is a worker function that get a copTask from channel, handle it and
// send the result back.
func (worker *copIteratorWorker) run(ctx context.Context) {
defer func() {
failpoint.Inject("ticase-4169", func(val failpoint.Value) {
if val.(bool) {
worker.memTracker.Consume(10 * MockResponseSizeForTest)
worker.memTracker.Consume(10 * MockResponseSizeForTest)
}
})
worker.wg.Done()
}()
for task := range worker.taskCh {
respCh := worker.respChan
if respCh == nil {
respCh = task.respChan
}
worker.handleTask(ctx, task, respCh)
if worker.respChan != nil {
// When a task is finished by the worker, send a finCopResp into channel to notify the copIterator that
// there is a task finished.
worker.sendToRespCh(finCopResp, worker.respChan, false)
}
close(task.respChan)
if worker.finished() {
return
}
}
}
// open starts workers and sender goroutines.
func (it *copIterator) open(ctx context.Context, enabledRateLimitAction, enableCollectExecutionInfo bool) {
taskCh := make(chan *copTask, 1)
smallTaskCh := make(chan *copTask, 1)
it.wg.Add(it.concurrency + it.smallTaskConcurrency)
// Start it.concurrency number of workers to handle cop requests.
for i := 0; i < it.concurrency+it.smallTaskConcurrency; i++ {
var ch chan *copTask
if i < it.concurrency {
ch = taskCh
} else {
ch = smallTaskCh
}
worker := &copIteratorWorker{
taskCh: ch,
wg: &it.wg,
store: it.store,
req: it.req,
respChan: it.respChan,
finishCh: it.finishCh,
vars: it.vars,
kvclient: txnsnapshot.NewClientHelper(it.store.store, &it.resolvedLocks, &it.committedLocks, false),
memTracker: it.memTracker,
replicaReadSeed: it.replicaReadSeed,
enableCollectExecutionInfo: enableCollectExecutionInfo,
pagingTaskIdx: &it.pagingTaskIdx,
}
go worker.run(ctx)
}
taskSender := &copIteratorTaskSender{
taskCh: taskCh,
smallTaskCh: smallTaskCh,
wg: &it.wg,
tasks: it.tasks,
finishCh: it.finishCh,
sendRate: it.sendRate,
}
taskSender.respChan = it.respChan
it.actionOnExceed.setEnabled(enabledRateLimitAction)
failpoint.Inject("ticase-4171", func(val failpoint.Value) {
if val.(bool) {
it.memTracker.Consume(10 * MockResponseSizeForTest)
it.memTracker.Consume(10 * MockResponseSizeForTest)
}
})
go taskSender.run()
}
func (sender *copIteratorTaskSender) run() {
// Send tasks to feed the worker goroutines.
for _, t := range sender.tasks {
// we control the sending rate to prevent all tasks
// being done (aka. all of the responses are buffered) by copIteratorWorker.
// We keep the number of inflight tasks within the number of 2 * concurrency when Keep Order is true.
// If KeepOrder is false, the number equals the concurrency.
// It sends one more task if a task has been finished in copIterator.Next.
exit := sender.sendRate.GetToken(sender.finishCh)
if exit {
break
}
var sendTo chan<- *copTask
if isSmallTask(t) {
sendTo = sender.smallTaskCh
} else {
sendTo = sender.taskCh
}
exit = sender.sendToTaskCh(t, sendTo)
if exit {
break
}
}
close(sender.taskCh)
close(sender.smallTaskCh)
// Wait for worker goroutines to exit.
sender.wg.Wait()
if sender.respChan != nil {
close(sender.respChan)
}
}
func (it *copIterator) recvFromRespCh(ctx context.Context, respCh <-chan *copResponse) (resp *copResponse, ok bool, exit bool) {
ticker := time.NewTicker(3 * time.Second)
defer ticker.Stop()
for {
select {
case resp, ok = <-respCh:
if it.memTracker != nil && resp != nil {
consumed := resp.MemSize()
failpoint.Inject("testRateLimitActionMockConsumeAndAssert", func(val failpoint.Value) {
if val.(bool) {
if resp != finCopResp {
consumed = MockResponseSizeForTest
}
}
})
it.memTracker.Consume(-consumed)
}
return
case <-it.finishCh:
exit = true
return
case <-ticker.C:
if atomic.LoadUint32(it.vars.Killed) == 1 {
resp = &copResponse{err: derr.ErrQueryInterrupted}
ok = true
return
}
case <-ctx.Done():
// We select the ctx.Done() in the thread of `Next` instead of in the worker to avoid the cost of `WithCancel`.
if atomic.CompareAndSwapUint32(&it.closed, 0, 1) {
close(it.finishCh)
}
exit = true
return
}
}
}
// GetConcurrency returns the concurrency and small task concurrency.
func (it *copIterator) GetConcurrency() (int, int) {
return it.concurrency, it.smallTaskConcurrency
}
// GetSendRate returns the rate-limit object.
func (it *copIterator) GetSendRate() *util.RateLimit {
return it.sendRate
}
// GetTasks returns the built tasks.
func (it *copIterator) GetTasks() []*copTask {
return it.tasks
}
func (sender *copIteratorTaskSender) sendToTaskCh(t *copTask, sendTo chan<- *copTask) (exit bool) {
select {
case sendTo <- t:
case <-sender.finishCh:
exit = true
}
return
}
func (worker *copIteratorWorker) sendToRespCh(resp *copResponse, respCh chan<- *copResponse, checkOOM bool) (exit bool) {
if worker.memTracker != nil && checkOOM {
consumed := resp.MemSize()
failpoint.Inject("testRateLimitActionMockConsumeAndAssert", func(val failpoint.Value) {
if val.(bool) {
if resp != finCopResp {
consumed = MockResponseSizeForTest
}
}
})
failpoint.Inject("ConsumeRandomPanic", nil)
worker.memTracker.Consume(consumed)
}
select {
case respCh <- resp:
case <-worker.finishCh:
exit = true
}
return
}
// MockResponseSizeForTest mock the response size
const MockResponseSizeForTest = 100 * 1024 * 1024
// Next returns next coprocessor result.
// NOTE: Use nil to indicate finish, so if the returned ResultSubset is not nil, reader should continue to call Next().
func (it *copIterator) Next(ctx context.Context) (kv.ResultSubset, error) {
var (
resp *copResponse
ok bool
closed bool
)
defer func() {
if resp == nil {
failpoint.Inject("ticase-4170", func(val failpoint.Value) {
if val.(bool) {
it.memTracker.Consume(10 * MockResponseSizeForTest)
it.memTracker.Consume(10 * MockResponseSizeForTest)
}
})
}
}()
// wait unit at least 5 copResponse received.
failpoint.Inject("testRateLimitActionMockWaitMax", func(val failpoint.Value) {
if val.(bool) {
// we only need to trigger oom at least once.
if len(it.tasks) > 9 {
for it.memTracker.MaxConsumed() < 5*MockResponseSizeForTest {
time.Sleep(10 * time.Millisecond)
}
}
}
})
// If data order matters, response should be returned in the same order as copTask slice.
// Otherwise all responses are returned from a single channel.
if it.respChan != nil {
// Get next fetched resp from chan
resp, ok, closed = it.recvFromRespCh(ctx, it.respChan)
if !ok || closed {
it.actionOnExceed.close()
return nil, nil
}
if resp == finCopResp {
it.actionOnExceed.destroyTokenIfNeeded(func() {
it.sendRate.PutToken()
})
return it.Next(ctx)
}
} else {
for {
if it.curr >= len(it.tasks) {
// Resp will be nil if iterator is finishCh.
it.actionOnExceed.close()
return nil, nil
}
task := it.tasks[it.curr]
resp, ok, closed = it.recvFromRespCh(ctx, task.respChan)
if closed {
// Close() is already called, so Next() is invalid.
return nil, nil
}
if ok {
break
}
it.actionOnExceed.destroyTokenIfNeeded(func() {
it.sendRate.PutToken()
})
// Switch to next task.
it.tasks[it.curr] = nil
it.curr++
}
}
if resp.err != nil {
return nil, errors.Trace(resp.err)
}
err := it.store.CheckVisibility(it.req.StartTs)
if err != nil {
return nil, errors.Trace(err)
}
return resp, nil
}
// Associate each region with an independent backoffer. In this way, when multiple regions are
// unavailable, TiDB can execute very quickly without blocking
func chooseBackoffer(ctx context.Context, backoffermap map[uint64]*Backoffer, task *copTask, worker *copIteratorWorker) *Backoffer {
bo, ok := backoffermap[task.region.GetID()]
if ok {
return bo
}
newbo := backoff.NewBackofferWithVars(ctx, copNextMaxBackoff, worker.vars)
backoffermap[task.region.GetID()] = newbo
return newbo
}
// handleTask handles single copTask, sends the result to channel, retry automatically on error.
func (worker *copIteratorWorker) handleTask(ctx context.Context, task *copTask, respCh chan<- *copResponse) {
defer func() {
r := recover()
if r != nil {
logutil.BgLogger().Error("copIteratorWork meet panic",
zap.Reflect("r", r),
zap.Stack("stack trace"))
resp := &copResponse{err: errors.Errorf("%v", r)}
// if panic has happened, set checkOOM to false to avoid another panic.
worker.sendToRespCh(resp, respCh, false)
}
}()
remainTasks := []*copTask{task}
backoffermap := make(map[uint64]*Backoffer)
for len(remainTasks) > 0 {
curTask := remainTasks[0]
bo := chooseBackoffer(ctx, backoffermap, curTask, worker)
tasks, err := worker.handleTaskOnce(bo, curTask, respCh)
if err != nil {
resp := &copResponse{err: errors.Trace(err)}
worker.sendToRespCh(resp, respCh, true)
return
}
if worker.finished() {
break
}
if len(tasks) > 0 {
remainTasks = append(tasks, remainTasks[1:]...)
} else {
remainTasks = remainTasks[1:]
}
}
if worker.store.coprCache != nil && worker.store.coprCache.cache.Metrics != nil {
coprCacheCounterEvict.Add(float64(worker.store.coprCache.cache.Metrics.KeysEvicted()))
}
}
// handleTaskOnce handles single copTask, successful results are send to channel.
// If error happened, returns error. If region split or meet lock, returns the remain tasks.
func (worker *copIteratorWorker) handleTaskOnce(bo *Backoffer, task *copTask, ch chan<- *copResponse) ([]*copTask, error) {
failpoint.Inject("handleTaskOnceError", func(val failpoint.Value) {
if val.(bool) {
failpoint.Return(nil, errors.New("mock handleTaskOnce error"))
}
})
if task.paging {
task.pagingTaskIdx = atomic.AddUint32(worker.pagingTaskIdx, 1)
}
copReq := coprocessor.Request{
Tp: worker.req.Tp,
StartTs: worker.req.StartTs,
Data: worker.req.Data,
Ranges: task.ranges.ToPBRanges(),
SchemaVer: worker.req.SchemaVar,
PagingSize: task.pagingSize,
Tasks: task.ToPBBatchTasks(),
}
var cacheKey []byte
var cacheValue *coprCacheValue
// If there are many ranges, it is very likely to be a TableLookupRequest. They are not worth to cache since
// computing is not the main cost. Ignore such requests directly to avoid slowly building the cache key.
if task.cmdType == tikvrpc.CmdCop && worker.store.coprCache != nil && worker.req.Cacheable && worker.store.coprCache.CheckRequestAdmission(len(copReq.Ranges)) {
cKey, err := coprCacheBuildKey(&copReq)
if err == nil {
cacheKey = cKey
cValue := worker.store.coprCache.Get(cKey)
copReq.IsCacheEnabled = true
if cValue != nil && cValue.RegionID == task.region.GetID() && cValue.TimeStamp <= worker.req.StartTs {
// Append cache version to the request to skip Coprocessor computation if possible
// when request result is cached
copReq.CacheIfMatchVersion = cValue.RegionDataVersion
cacheValue = cValue
} else {
copReq.CacheIfMatchVersion = 0
}
} else {
logutil.BgLogger().Warn("Failed to build copr cache key", zap.Error(err))
}
}
req := tikvrpc.NewReplicaReadRequest(task.cmdType, &copReq, options.GetTiKVReplicaReadType(worker.req.ReplicaRead), &worker.replicaReadSeed, kvrpcpb.Context{
IsolationLevel: isolationLevelToPB(worker.req.IsolationLevel),
Priority: priorityToPB(worker.req.Priority),
NotFillCache: worker.req.NotFillCache,
RecordTimeStat: true,
RecordScanStat: true,
TaskId: worker.req.TaskID,
RequestSource: task.requestSource.GetRequestSource(),
})
if worker.req.ResourceGroupTagger != nil {
worker.req.ResourceGroupTagger(req)
}
req.StoreTp = getEndPointType(task.storeType)
startTime := time.Now()
if worker.kvclient.Stats == nil {
worker.kvclient.Stats = make(map[tikvrpc.CmdType]*tikv.RPCRuntimeStats)
}
req.ReadReplicaScope = worker.req.ReadReplicaScope
if worker.req.IsStaleness {
req.EnableStaleRead()
}
staleRead := req.GetStaleRead()