forked from pingcap/tidb
-
Notifications
You must be signed in to change notification settings - Fork 0
/
Copy pathrange_task.go
291 lines (250 loc) · 8.31 KB
/
range_task.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
// Copyright 2019 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,
// See the License for the specific language governing permissions and
// limitations under the License.
package tikv
import (
"bytes"
"context"
"sync"
"sync/atomic"
"time"
"github.com/pingcap/errors"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/store/tikv/logutil"
"github.com/pingcap/tidb/store/tikv/metrics"
"go.uber.org/zap"
)
const (
rangeTaskDefaultStatLogInterval = time.Minute * 10
defaultRegionsPerTask = 128
lblCompletedRegions = "completed-regions"
lblFailedRegions = "failed-regions"
)
// RangeTaskRunner splits a range into many ranges to process concurrently, and convenient to send requests to all
// regions in the range. Because of merging and splitting, it's possible that multiple requests for disjoint ranges are
// sent to the same region.
type RangeTaskRunner struct {
name string
store Storage
concurrency int
handler RangeTaskHandler
statLogInterval time.Duration
regionsPerTask int
completedRegions int32
failedRegions int32
}
// RangeTaskStat is used to count Regions that completed or failed to do the task.
type RangeTaskStat struct {
CompletedRegions int
FailedRegions int
}
// RangeTaskHandler is the type of functions that processes a task of a key range.
// The function should calculate Regions that succeeded or failed to the task.
// Returning error from the handler means the error caused the whole task should be stopped.
type RangeTaskHandler = func(ctx context.Context, r kv.KeyRange) (RangeTaskStat, error)
// NewRangeTaskRunner creates a RangeTaskRunner.
//
// `requestCreator` is the function used to create RPC request according to the given range.
// `responseHandler` is the function to process responses of errors. If `responseHandler` returns error, the whole job
// will be canceled.
func NewRangeTaskRunner(
name string,
store Storage,
concurrency int,
handler RangeTaskHandler,
) *RangeTaskRunner {
return &RangeTaskRunner{
name: name,
store: store,
concurrency: concurrency,
handler: handler,
statLogInterval: rangeTaskDefaultStatLogInterval,
regionsPerTask: defaultRegionsPerTask,
}
}
// SetRegionsPerTask sets how many regions is in a divided task. Since regions may split and merge, it's possible that
// a sub task contains not exactly specified number of regions.
func (s *RangeTaskRunner) SetRegionsPerTask(regionsPerTask int) {
if regionsPerTask < 1 {
panic("RangeTaskRunner: regionsPerTask should be at least 1")
}
s.regionsPerTask = regionsPerTask
}
// RunOnRange runs the task on the given range.
// Empty startKey or endKey means unbounded.
func (s *RangeTaskRunner) RunOnRange(ctx context.Context, startKey, endKey kv.Key) error {
s.completedRegions = 0
metrics.TiKVRangeTaskStats.WithLabelValues(s.name, lblCompletedRegions).Set(0)
if len(endKey) != 0 && bytes.Compare(startKey, endKey) >= 0 {
logutil.Logger(ctx).Info("empty range task executed. ignored",
zap.String("name", s.name),
zap.Stringer("startKey", startKey),
zap.Stringer("endKey", endKey))
return nil
}
logutil.Logger(ctx).Info("range task started",
zap.String("name", s.name),
zap.Stringer("startKey", startKey),
zap.Stringer("endKey", endKey),
zap.Int("concurrency", s.concurrency))
// Periodically log the progress
statLogTicker := time.NewTicker(s.statLogInterval)
ctx, cancel := context.WithCancel(ctx)
taskCh := make(chan *kv.KeyRange, s.concurrency)
var wg sync.WaitGroup
// Create workers that concurrently process the whole range.
workers := make([]*rangeTaskWorker, 0, s.concurrency)
for i := 0; i < s.concurrency; i++ {
w := s.createWorker(taskCh, &wg)
workers = append(workers, w)
wg.Add(1)
go w.run(ctx, cancel)
}
startTime := time.Now()
// Make sure taskCh is closed exactly once
isClosed := false
defer func() {
if !isClosed {
close(taskCh)
wg.Wait()
}
statLogTicker.Stop()
cancel()
metrics.TiKVRangeTaskStats.WithLabelValues(s.name, lblCompletedRegions).Set(0)
}()
// Iterate all regions and send each region's range as a task to the workers.
key := startKey
Loop:
for {
select {
case <-statLogTicker.C:
logutil.Logger(ctx).Info("range task in progress",
zap.String("name", s.name),
zap.Stringer("startKey", startKey),
zap.Stringer("endKey", endKey),
zap.Int("concurrency", s.concurrency),
zap.Duration("cost time", time.Since(startTime)),
zap.Int("completed regions", s.CompletedRegions()))
default:
}
bo := NewBackofferWithVars(ctx, locateRegionMaxBackoff, nil)
rangeEndKey, err := s.store.GetRegionCache().BatchLoadRegionsFromKey(bo, key, s.regionsPerTask)
if err != nil {
logutil.Logger(ctx).Info("range task failed",
zap.String("name", s.name),
zap.Stringer("startKey", startKey),
zap.Stringer("endKey", endKey),
zap.Duration("cost time", time.Since(startTime)),
zap.Error(err))
return errors.Trace(err)
}
task := &kv.KeyRange{
StartKey: key,
EndKey: rangeEndKey,
}
isLast := len(task.EndKey) == 0 || (len(endKey) > 0 && bytes.Compare(task.EndKey, endKey) >= 0)
// Let task.EndKey = min(endKey, loc.EndKey)
if isLast {
task.EndKey = endKey
}
pushTaskStartTime := time.Now()
select {
case taskCh <- task:
case <-ctx.Done():
break Loop
}
metrics.TiKVRangeTaskPushDuration.WithLabelValues(s.name).Observe(time.Since(pushTaskStartTime).Seconds())
if isLast {
break
}
key = task.EndKey
}
isClosed = true
close(taskCh)
wg.Wait()
for _, w := range workers {
if w.err != nil {
logutil.Logger(ctx).Info("range task failed",
zap.String("name", s.name),
zap.Stringer("startKey", startKey),
zap.Stringer("endKey", endKey),
zap.Duration("cost time", time.Since(startTime)),
zap.Error(w.err))
return errors.Trace(w.err)
}
}
logutil.Logger(ctx).Info("range task finished",
zap.String("name", s.name),
zap.Stringer("startKey", startKey),
zap.Stringer("endKey", endKey),
zap.Duration("cost time", time.Since(startTime)),
zap.Int("completed regions", s.CompletedRegions()))
return nil
}
// createWorker creates a worker that can process tasks from the given channel.
func (s *RangeTaskRunner) createWorker(taskCh chan *kv.KeyRange, wg *sync.WaitGroup) *rangeTaskWorker {
return &rangeTaskWorker{
name: s.name,
store: s.store,
handler: s.handler,
taskCh: taskCh,
wg: wg,
completedRegions: &s.completedRegions,
failedRegions: &s.failedRegions,
}
}
// CompletedRegions returns how many regions has been sent requests.
func (s *RangeTaskRunner) CompletedRegions() int {
return int(atomic.LoadInt32(&s.completedRegions))
}
// FailedRegions returns how many regions has failed to do the task.
func (s *RangeTaskRunner) FailedRegions() int {
return int(atomic.LoadInt32(&s.failedRegions))
}
// rangeTaskWorker is used by RangeTaskRunner to process tasks concurrently.
type rangeTaskWorker struct {
name string
store Storage
handler RangeTaskHandler
taskCh chan *kv.KeyRange
wg *sync.WaitGroup
err error
completedRegions *int32
failedRegions *int32
}
// run starts the worker. It collects all objects from `w.taskCh` and process them one by one.
func (w *rangeTaskWorker) run(ctx context.Context, cancel context.CancelFunc) {
defer w.wg.Done()
for r := range w.taskCh {
select {
case <-ctx.Done():
w.err = ctx.Err()
return
default:
}
stat, err := w.handler(ctx, *r)
atomic.AddInt32(w.completedRegions, int32(stat.CompletedRegions))
atomic.AddInt32(w.failedRegions, int32(stat.FailedRegions))
metrics.TiKVRangeTaskStats.WithLabelValues(w.name, lblCompletedRegions).Add(float64(stat.CompletedRegions))
metrics.TiKVRangeTaskStats.WithLabelValues(w.name, lblFailedRegions).Add(float64(stat.FailedRegions))
if err != nil {
logutil.Logger(ctx).Info("canceling range task because of error",
zap.String("name", w.name),
zap.Stringer("failed startKey", r.StartKey),
zap.Stringer("failed endKey", r.EndKey),
zap.Error(err))
w.err = err
cancel()
break
}
}
}