forked from pingcap/tidb
-
Notifications
You must be signed in to change notification settings - Fork 3
/
Copy pathselect_result.go
246 lines (221 loc) · 6.42 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
// 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 (
"context"
"fmt"
"time"
"github.com/pingcap/errors"
"github.com/pingcap/parser/terror"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/metrics"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/statistics"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/codec"
"github.com/pingcap/tidb/util/logutil"
"github.com/pingcap/tidb/util/memory"
"github.com/pingcap/tipb/go-tipb"
"go.uber.org/zap"
)
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 resultWithErr struct {
result kv.ResultSubset
err error
}
type selectResult struct {
label string
resp kv.Response
results chan resultWithErr
closed chan struct{}
rowLen int
fieldTypes []*types.FieldType
ctx sessionctx.Context
selectResp *tipb.SelectResponse
respChkIdx int
feedback *statistics.QueryFeedback
partialCount int64 // number of partial results.
sqlType string
// copPlanIDs contains all copTasks' planIDs,
// which help to collect copTasks' runtime stats.
copPlanIDs []fmt.Stringer
memTracker *memory.Tracker
}
func (r *selectResult) Fetch(ctx context.Context) {
go r.fetch(ctx)
}
func (r *selectResult) fetch(ctx context.Context) {
startTime := time.Now()
defer func() {
if c := recover(); c != nil {
err := fmt.Errorf("%v", c)
logutil.Logger(ctx).Error("OOM", zap.Error(err))
r.results <- resultWithErr{err: err}
}
close(r.results)
duration := time.Since(startTime)
metrics.DistSQLQueryHistgram.WithLabelValues(r.label, r.sqlType).Observe(duration.Seconds())
}()
for {
var result resultWithErr
resultSubset, err := r.resp.Next(ctx)
if err != nil {
result.err = err
} else if resultSubset == nil {
return
} else {
result.result = resultSubset
if r.memTracker != nil {
r.memTracker.Consume(int64(resultSubset.MemSize()))
}
}
select {
case r.results <- result:
case <-r.closed:
// If selectResult called Close() already, make fetch goroutine exit.
return
case <-ctx.Done():
return
}
}
}
// NextRaw returns the next raw partial result.
func (r *selectResult) NextRaw(ctx context.Context) ([]byte, error) {
re := <-r.results
r.partialCount++
r.feedback.Invalidate()
if re.result == nil || re.err != nil {
return nil, errors.Trace(re.err)
}
return re.result.GetData(), nil
}
// Next reads data to the chunk.
func (r *selectResult) Next(ctx context.Context, chk *chunk.Chunk) error {
chk.Reset()
for !chk.IsFull() {
if r.selectResp == nil || r.respChkIdx == len(r.selectResp.Chunks) {
err := r.getSelectResp()
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) getSelectResp() error {
r.respChkIdx = 0
for {
re := <-r.results
if re.err != nil {
return errors.Trace(re.err)
}
if r.memTracker != nil && r.selectResp != nil {
r.memTracker.Consume(-int64(r.selectResp.Size()))
}
if re.result == nil {
r.selectResp = nil
return nil
}
if r.memTracker != nil {
r.memTracker.Consume(-int64(re.result.MemSize()))
}
r.selectResp = new(tipb.SelectResponse)
err := r.selectResp.Unmarshal(re.result.GetData())
if err != nil {
return errors.Trace(err)
}
if r.memTracker != nil && r.selectResp != nil {
r.memTracker.Consume(int64(r.selectResp.Size()))
}
if err := r.selectResp.Error; err != nil {
return terror.ClassTiKV.New(terror.ErrCode(err.Code), err.Msg)
}
sc := r.ctx.GetSessionVars().StmtCtx
for _, warning := range r.selectResp.Warnings {
sc.AppendWarning(terror.ClassTiKV.New(terror.ErrCode(warning.Code), warning.Msg))
}
r.updateCopRuntimeStats(re.result.GetExecDetails().CalleeAddress)
r.feedback.Update(re.result.GetStartKey(), r.selectResp.OutputCounts)
r.partialCount++
sc.MergeExecDetails(re.result.GetExecDetails(), nil)
if len(r.selectResp.Chunks) == 0 {
continue
}
return nil
}
}
func (r *selectResult) updateCopRuntimeStats(callee string) {
if r.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl == nil || callee == "" {
return
}
if len(r.selectResp.GetExecutionSummaries()) != len(r.copPlanIDs) {
logutil.Logger(context.Background()).Error("invalid cop task execution summaries length",
zap.Int("expected", len(r.copPlanIDs)),
zap.Int("received", len(r.selectResp.GetExecutionSummaries())))
return
}
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.String(), 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
}
// Close closes selectResult.
func (r *selectResult) Close() error {
// Close this channel tell fetch goroutine to exit.
if r.feedback.Actual() >= 0 {
metrics.DistSQLScanKeysHistogram.Observe(float64(r.feedback.Actual()))
}
metrics.DistSQLPartialCountHistogram.Observe(float64(r.partialCount))
close(r.closed)
return r.resp.Close()
}