forked from pingcap/tidb
-
Notifications
You must be signed in to change notification settings - Fork 0
/
Copy pathselect_result.go
491 lines (448 loc) · 14.4 KB
/
select_result.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
// Copyright 2018 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 distsql
import (
"bytes"
"context"
"fmt"
"sort"
"strconv"
"sync/atomic"
"time"
"github.com/pingcap/errors"
"github.com/pingcap/parser/terror"
"github.com/pingcap/tidb/config"
"github.com/pingcap/tidb/errno"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/metrics"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/statistics"
"github.com/pingcap/tidb/store/copr"
"github.com/pingcap/tidb/store/tikv"
"github.com/pingcap/tidb/store/tikv/tikvrpc"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/codec"
"github.com/pingcap/tidb/util/dbterror"
"github.com/pingcap/tidb/util/execdetails"
"github.com/pingcap/tidb/util/logutil"
"github.com/pingcap/tidb/util/memory"
"github.com/pingcap/tipb/go-tipb"
"go.uber.org/zap"
)
var (
errQueryInterrupted = dbterror.ClassExecutor.NewStd(errno.ErrQueryInterrupted)
)
var (
coprCacheHistogramHit = metrics.DistSQLCoprCacheHistogram.WithLabelValues("hit")
coprCacheHistogramMiss = metrics.DistSQLCoprCacheHistogram.WithLabelValues("miss")
)
var (
_ SelectResult = (*selectResult)(nil)
_ SelectResult = (*streamResult)(nil)
)
// SelectResult is an iterator of coprocessor partial results.
type SelectResult interface {
// Fetch fetches partial results from client.
Fetch(context.Context)
// NextRaw gets the next raw result.
NextRaw(context.Context) ([]byte, error)
// Next reads the data into chunk.
Next(context.Context, *chunk.Chunk) error
// Close closes the iterator.
Close() error
}
type selectResult struct {
label string
resp kv.Response
rowLen int
fieldTypes []*types.FieldType
ctx sessionctx.Context
selectResp *tipb.SelectResponse
selectRespSize int64 // record the selectResp.Size() when it is initialized.
respChkIdx int
respChunkDecoder *chunk.Decoder
feedback *statistics.QueryFeedback
partialCount int64 // number of partial results.
sqlType string
encodeType tipb.EncodeType
// copPlanIDs contains all copTasks' planIDs,
// which help to collect copTasks' runtime stats.
copPlanIDs []int
rootPlanID int
storeType kv.StoreType
fetchDuration time.Duration
durationReported bool
memTracker *memory.Tracker
stats *selectResultRuntimeStats
}
func (r *selectResult) Fetch(ctx context.Context) {
}
func (r *selectResult) fetchResp(ctx context.Context) error {
for {
r.respChkIdx = 0
startTime := time.Now()
resultSubset, err := r.resp.Next(ctx)
duration := time.Since(startTime)
r.fetchDuration += duration
if err != nil {
return errors.Trace(err)
}
if r.selectResp != nil {
r.memConsume(-atomic.LoadInt64(&r.selectRespSize))
}
if resultSubset == nil {
r.selectResp = nil
atomic.StoreInt64(&r.selectRespSize, 0)
if !r.durationReported {
// final round of fetch
// TODO: Add a label to distinguish between success or failure.
// https://github.com/pingcap/tidb/issues/11397
metrics.DistSQLQueryHistogram.WithLabelValues(r.label, r.sqlType).Observe(r.fetchDuration.Seconds())
r.durationReported = true
}
return nil
}
r.selectResp = new(tipb.SelectResponse)
err = r.selectResp.Unmarshal(resultSubset.GetData())
if err != nil {
return errors.Trace(err)
}
respSize := int64(r.selectResp.Size())
atomic.StoreInt64(&r.selectRespSize, respSize)
r.memConsume(respSize)
if err := r.selectResp.Error; err != nil {
return dbterror.ClassTiKV.Synthesize(terror.ErrCode(err.Code), err.Msg)
}
sessVars := r.ctx.GetSessionVars()
if atomic.LoadUint32(&sessVars.Killed) == 1 {
return errors.Trace(errQueryInterrupted)
}
sc := sessVars.StmtCtx
for _, warning := range r.selectResp.Warnings {
sc.AppendWarning(dbterror.ClassTiKV.Synthesize(terror.ErrCode(warning.Code), warning.Msg))
}
if r.feedback != nil {
r.feedback.Update(resultSubset.GetStartKey(), r.selectResp.OutputCounts, r.selectResp.Ndvs)
}
r.partialCount++
hasStats, ok := resultSubset.(CopRuntimeStats)
if ok {
copStats := hasStats.GetCopRuntimeStats()
if copStats != nil {
r.updateCopRuntimeStats(ctx, copStats, resultSubset.RespTime())
copStats.CopTime = duration
sc.MergeExecDetails(&copStats.ExecDetails, nil)
}
}
if len(r.selectResp.Chunks) != 0 {
break
}
}
if r.stats != nil {
coprCacheHistogramHit.Observe(float64(r.stats.CoprCacheHitNum))
coprCacheHistogramMiss.Observe(float64(len(r.stats.copRespTime) - int(r.stats.CoprCacheHitNum)))
}
return nil
}
func (r *selectResult) Next(ctx context.Context, chk *chunk.Chunk) error {
chk.Reset()
if r.selectResp == nil || r.respChkIdx == len(r.selectResp.Chunks) {
err := r.fetchResp(ctx)
if err != nil {
return err
}
if r.selectResp == nil {
return nil
}
}
// TODO(Shenghui Wu): add metrics
switch r.selectResp.GetEncodeType() {
case tipb.EncodeType_TypeDefault:
return r.readFromDefault(ctx, chk)
case tipb.EncodeType_TypeChunk:
return r.readFromChunk(ctx, chk)
}
return errors.Errorf("unsupported encode type:%v", r.encodeType)
}
// NextRaw returns the next raw partial result.
func (r *selectResult) NextRaw(ctx context.Context) (data []byte, err error) {
resultSubset, err := r.resp.Next(ctx)
r.partialCount++
r.feedback.Invalidate()
if resultSubset != nil && err == nil {
data = resultSubset.GetData()
}
return data, err
}
func (r *selectResult) readFromDefault(ctx context.Context, chk *chunk.Chunk) error {
for !chk.IsFull() {
if r.respChkIdx == len(r.selectResp.Chunks) {
err := r.fetchResp(ctx)
if err != nil || r.selectResp == nil {
return err
}
}
err := r.readRowsData(chk)
if err != nil {
return err
}
if len(r.selectResp.Chunks[r.respChkIdx].RowsData) == 0 {
r.respChkIdx++
}
}
return nil
}
func (r *selectResult) readFromChunk(ctx context.Context, chk *chunk.Chunk) error {
if r.respChunkDecoder == nil {
r.respChunkDecoder = chunk.NewDecoder(
chunk.NewChunkWithCapacity(r.fieldTypes, 0),
r.fieldTypes,
)
}
for !chk.IsFull() {
if r.respChkIdx == len(r.selectResp.Chunks) {
err := r.fetchResp(ctx)
if err != nil || r.selectResp == nil {
return err
}
}
if r.respChunkDecoder.IsFinished() {
r.respChunkDecoder.Reset(r.selectResp.Chunks[r.respChkIdx].RowsData)
}
// If the next chunk size is greater than required rows * 0.8, reuse the memory of the next chunk and return
// immediately. Otherwise, splice the data to one chunk and wait the next chunk.
if r.respChunkDecoder.RemainedRows() > int(float64(chk.RequiredRows())*0.8) {
if chk.NumRows() > 0 {
return nil
}
r.respChunkDecoder.ReuseIntermChk(chk)
r.respChkIdx++
return nil
}
r.respChunkDecoder.Decode(chk)
if r.respChunkDecoder.IsFinished() {
r.respChkIdx++
}
}
return nil
}
func (r *selectResult) updateCopRuntimeStats(ctx context.Context, copStats *copr.CopRuntimeStats, respTime time.Duration) {
callee := copStats.CalleeAddress
if r.rootPlanID <= 0 || r.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl == nil || callee == "" {
return
}
if len(r.selectResp.GetExecutionSummaries()) != len(r.copPlanIDs) {
logutil.Logger(ctx).Error("invalid cop task execution summaries length",
zap.Int("expected", len(r.copPlanIDs)),
zap.Int("received", len(r.selectResp.GetExecutionSummaries())))
return
}
if r.stats == nil {
id := r.rootPlanID
r.stats = &selectResultRuntimeStats{
backoffSleep: make(map[string]time.Duration),
rpcStat: tikv.NewRegionRequestRuntimeStats(),
}
r.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.RegisterStats(id, r.stats)
}
r.stats.mergeCopRuntimeStats(copStats, respTime)
if copStats.ScanDetail != nil && len(r.copPlanIDs) > 0 {
r.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.RecordScanDetail(r.copPlanIDs[len(r.copPlanIDs)-1], r.storeType.Name(), copStats.ScanDetail)
}
for i, detail := range r.selectResp.GetExecutionSummaries() {
if detail != nil && detail.TimeProcessedNs != nil &&
detail.NumProducedRows != nil && detail.NumIterations != nil {
planID := r.copPlanIDs[i]
r.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.
RecordOneCopTask(planID, r.storeType.Name(), callee, detail)
}
}
}
func (r *selectResult) readRowsData(chk *chunk.Chunk) (err error) {
rowsData := r.selectResp.Chunks[r.respChkIdx].RowsData
decoder := codec.NewDecoder(chk, r.ctx.GetSessionVars().Location())
for !chk.IsFull() && len(rowsData) > 0 {
for i := 0; i < r.rowLen; i++ {
rowsData, err = decoder.DecodeOne(rowsData, i, r.fieldTypes[i])
if err != nil {
return err
}
}
}
r.selectResp.Chunks[r.respChkIdx].RowsData = rowsData
return nil
}
func (r *selectResult) memConsume(bytes int64) {
if r.memTracker != nil {
r.memTracker.Consume(bytes)
}
}
// Close closes selectResult.
func (r *selectResult) Close() error {
if r.feedback.Actual() >= 0 {
metrics.DistSQLScanKeysHistogram.Observe(float64(r.feedback.Actual()))
}
metrics.DistSQLPartialCountHistogram.Observe(float64(r.partialCount))
respSize := atomic.SwapInt64(&r.selectRespSize, 0)
if respSize > 0 {
r.memConsume(-respSize)
}
return r.resp.Close()
}
// CopRuntimeStats is a interface uses to check whether the result has cop runtime stats.
type CopRuntimeStats interface {
// GetCopRuntimeStats gets the cop runtime stats information.
GetCopRuntimeStats() *copr.CopRuntimeStats
}
type selectResultRuntimeStats struct {
copRespTime []time.Duration
procKeys []int64
backoffSleep map[string]time.Duration
totalProcessTime time.Duration
totalWaitTime time.Duration
rpcStat tikv.RegionRequestRuntimeStats
CoprCacheHitNum int64
}
func (s *selectResultRuntimeStats) mergeCopRuntimeStats(copStats *copr.CopRuntimeStats, respTime time.Duration) {
s.copRespTime = append(s.copRespTime, respTime)
if copStats.ScanDetail != nil {
s.procKeys = append(s.procKeys, copStats.ScanDetail.ProcessedKeys)
} else {
s.procKeys = append(s.procKeys, 0)
}
for k, v := range copStats.BackoffSleep {
s.backoffSleep[k] += v
}
s.totalProcessTime += copStats.TimeDetail.ProcessTime
s.totalWaitTime += copStats.TimeDetail.WaitTime
s.rpcStat.Merge(copStats.RegionRequestRuntimeStats)
if copStats.CoprCacheHit {
s.CoprCacheHitNum++
}
}
func (s *selectResultRuntimeStats) Clone() execdetails.RuntimeStats {
newRs := selectResultRuntimeStats{
copRespTime: make([]time.Duration, 0, len(s.copRespTime)),
procKeys: make([]int64, 0, len(s.procKeys)),
backoffSleep: make(map[string]time.Duration, len(s.backoffSleep)),
rpcStat: tikv.NewRegionRequestRuntimeStats(),
}
newRs.copRespTime = append(newRs.copRespTime, s.copRespTime...)
newRs.procKeys = append(newRs.procKeys, s.procKeys...)
for k, v := range s.backoffSleep {
newRs.backoffSleep[k] += v
}
newRs.totalProcessTime += s.totalProcessTime
newRs.totalWaitTime += s.totalWaitTime
for k, v := range s.rpcStat.Stats {
newRs.rpcStat.Stats[k] = v
}
return &newRs
}
func (s *selectResultRuntimeStats) Merge(rs execdetails.RuntimeStats) {
other, ok := rs.(*selectResultRuntimeStats)
if !ok {
return
}
s.copRespTime = append(s.copRespTime, other.copRespTime...)
s.procKeys = append(s.procKeys, other.procKeys...)
for k, v := range other.backoffSleep {
s.backoffSleep[k] += v
}
s.totalProcessTime += other.totalProcessTime
s.totalWaitTime += other.totalWaitTime
s.rpcStat.Merge(other.rpcStat)
s.CoprCacheHitNum += other.CoprCacheHitNum
}
func (s *selectResultRuntimeStats) String() string {
buf := bytes.NewBuffer(nil)
rpcStat := s.rpcStat
if len(s.copRespTime) > 0 {
size := len(s.copRespTime)
if size == 1 {
buf.WriteString(fmt.Sprintf("cop_task: {num: 1, max: %v, proc_keys: %v", execdetails.FormatDuration(s.copRespTime[0]), s.procKeys[0]))
} else {
sort.Slice(s.copRespTime, func(i, j int) bool {
return s.copRespTime[i] < s.copRespTime[j]
})
vMax, vMin := s.copRespTime[size-1], s.copRespTime[0]
vP95 := s.copRespTime[size*19/20]
sum := 0.0
for _, t := range s.copRespTime {
sum += float64(t)
}
vAvg := time.Duration(sum / float64(size))
sort.Slice(s.procKeys, func(i, j int) bool {
return s.procKeys[i] < s.procKeys[j]
})
keyMax := s.procKeys[size-1]
keyP95 := s.procKeys[size*19/20]
buf.WriteString(fmt.Sprintf("cop_task: {num: %v, max: %v, min: %v, avg: %v, p95: %v", size,
execdetails.FormatDuration(vMax), execdetails.FormatDuration(vMin),
execdetails.FormatDuration(vAvg), execdetails.FormatDuration(vP95)))
if keyMax > 0 {
buf.WriteString(", max_proc_keys: ")
buf.WriteString(strconv.FormatInt(keyMax, 10))
buf.WriteString(", p95_proc_keys: ")
buf.WriteString(strconv.FormatInt(keyP95, 10))
}
}
if s.totalProcessTime > 0 {
buf.WriteString(", tot_proc: ")
buf.WriteString(execdetails.FormatDuration(s.totalProcessTime))
if s.totalWaitTime > 0 {
buf.WriteString(", tot_wait: ")
buf.WriteString(execdetails.FormatDuration(s.totalWaitTime))
}
}
copRPC := rpcStat.Stats[tikvrpc.CmdCop]
if copRPC != nil && copRPC.Count > 0 {
rpcStat = rpcStat.Clone()
delete(rpcStat.Stats, tikvrpc.CmdCop)
buf.WriteString(", rpc_num: ")
buf.WriteString(strconv.FormatInt(copRPC.Count, 10))
buf.WriteString(", rpc_time: ")
buf.WriteString(execdetails.FormatDuration(time.Duration(copRPC.Consume)))
}
if config.GetGlobalConfig().TiKVClient.CoprCache.CapacityMB > 0 {
buf.WriteString(fmt.Sprintf(", copr_cache_hit_ratio: %v",
strconv.FormatFloat(float64(s.CoprCacheHitNum)/float64(len(s.copRespTime)), 'f', 2, 64)))
} else {
buf.WriteString(", copr_cache: disabled")
}
buf.WriteString("}")
}
rpcStatsStr := rpcStat.String()
if len(rpcStatsStr) > 0 {
buf.WriteString(", ")
buf.WriteString(rpcStatsStr)
}
if len(s.backoffSleep) > 0 {
buf.WriteString(", backoff{")
idx := 0
for k, d := range s.backoffSleep {
if idx > 0 {
buf.WriteString(", ")
}
idx++
buf.WriteString(fmt.Sprintf("%s: %s", k, execdetails.FormatDuration(d)))
}
buf.WriteString("}")
}
return buf.String()
}
// Tp implements the RuntimeStats interface.
func (s *selectResultRuntimeStats) Tp() int {
return execdetails.TpSelectResultRuntimeStats
}