forked from pingcap/tidb
-
Notifications
You must be signed in to change notification settings - Fork 0
/
Copy pathtable_reader.go
262 lines (238 loc) · 8.13 KB
/
table_reader.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
// 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 executor
import (
"context"
"fmt"
"github.com/opentracing/opentracing-go"
"github.com/pingcap/parser/model"
"github.com/pingcap/tidb/distsql"
"github.com/pingcap/tidb/kv"
plannercore "github.com/pingcap/tidb/planner/core"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/statistics"
"github.com/pingcap/tidb/table"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/logutil"
"github.com/pingcap/tidb/util/memory"
"github.com/pingcap/tidb/util/ranger"
"github.com/pingcap/tidb/util/stringutil"
"github.com/pingcap/tipb/go-tipb"
)
// make sure `TableReaderExecutor` implements `Executor`.
var _ Executor = &TableReaderExecutor{}
// selectResultHook is used to hack distsql.SelectWithRuntimeStats safely for testing.
type selectResultHook struct {
selectResultFunc func(ctx context.Context, sctx sessionctx.Context, kvReq *kv.Request,
fieldTypes []*types.FieldType, fb *statistics.QueryFeedback, copPlanIDs []fmt.Stringer) (distsql.SelectResult, error)
}
func (sr selectResultHook) SelectResult(ctx context.Context, sctx sessionctx.Context, kvReq *kv.Request,
fieldTypes []*types.FieldType, fb *statistics.QueryFeedback, copPlanIDs []fmt.Stringer, rootPlanID fmt.Stringer) (distsql.SelectResult, error) {
if sr.selectResultFunc == nil {
return distsql.SelectWithRuntimeStats(ctx, sctx, kvReq, fieldTypes, fb, copPlanIDs, rootPlanID)
}
return sr.selectResultFunc(ctx, sctx, kvReq, fieldTypes, fb, copPlanIDs)
}
// TableReaderExecutor sends DAG request and reads table data from kv layer.
type TableReaderExecutor struct {
baseExecutor
table table.Table
ranges []*ranger.Range
// kvRanges are only use for union scan.
kvRanges []kv.KeyRange
dagPB *tipb.DAGRequest
// columns are only required by union scan.
columns []*model.ColumnInfo
// resultHandler handles the order of the result. Since (MAXInt64, MAXUint64] stores before [0, MaxInt64] physically
// for unsigned int.
resultHandler *tableResultHandler
feedback *statistics.QueryFeedback
plans []plannercore.PhysicalPlan
memTracker *memory.Tracker
selectResultHook // for testing
keepOrder bool
desc bool
streaming bool
storeType kv.StoreType
// corColInFilter tells whether there's correlated column in filter.
corColInFilter bool
// corColInAccess tells whether there's correlated column in access conditions.
corColInAccess bool
}
// Open initialzes necessary variables for using this executor.
func (e *TableReaderExecutor) Open(ctx context.Context) error {
if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil {
span1 := span.Tracer().StartSpan("TableReaderExecutor.Open", opentracing.ChildOf(span.Context()))
defer span1.Finish()
ctx = opentracing.ContextWithSpan(ctx, span1)
}
e.memTracker = memory.NewTracker(e.id, e.ctx.GetSessionVars().MemQuotaDistSQL)
e.memTracker.AttachTo(e.ctx.GetSessionVars().StmtCtx.MemTracker)
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
}
if e.corColInAccess {
ts := e.plans[0].(*plannercore.PhysicalTableScan)
access := ts.AccessCondition
pkTP := ts.Table.GetPkColInfo().FieldType
e.ranges, err = ranger.BuildTableRange(access, e.ctx.GetSessionVars().StmtCtx, &pkTP)
if err != nil {
return err
}
}
e.resultHandler = &tableResultHandler{}
if e.feedback != nil && e.feedback.Hist != nil {
// EncodeInt don't need *statement.Context.
var ok bool
e.ranges, ok = e.feedback.Hist.SplitRange(nil, e.ranges, false)
if !ok {
e.feedback.Invalidate()
}
}
firstPartRanges, secondPartRanges := splitRanges(e.ranges, e.keepOrder, e.desc)
firstResult, err := e.buildResp(ctx, firstPartRanges)
if err != nil {
e.feedback.Invalidate()
return err
}
if len(secondPartRanges) == 0 {
e.resultHandler.open(nil, firstResult)
return nil
}
var secondResult distsql.SelectResult
secondResult, err = e.buildResp(ctx, secondPartRanges)
if err != nil {
e.feedback.Invalidate()
return err
}
e.resultHandler.open(firstResult, secondResult)
return nil
}
// Next fills data into the chunk passed by its caller.
// The task was actually done by tableReaderHandler.
func (e *TableReaderExecutor) Next(ctx context.Context, req *chunk.Chunk) error {
logutil.Eventf(ctx, "table scan table: %s, range: %v", stringutil.MemoizeStr(func() string {
var tableName string
if meta := e.table.Meta(); meta != nil {
tableName = meta.Name.L
}
return tableName
}), e.ranges)
if err := e.resultHandler.nextChunk(ctx, req); err != nil {
e.feedback.Invalidate()
return err
}
return nil
}
// Close implements the Executor Close interface.
func (e *TableReaderExecutor) Close() error {
var err error
if e.resultHandler != nil {
err = e.resultHandler.Close()
}
if e.runtimeStats != nil {
copStats := e.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.GetRootStats(e.plans[0].ExplainID().String())
copStats.SetRowNum(e.feedback.Actual())
}
e.ctx.StoreQueryFeedback(e.feedback)
return err
}
// buildResp first builds request and sends it to tikv using distsql.Select. It uses SelectResut returned by the callee
// to fetch all results.
func (e *TableReaderExecutor) buildResp(ctx context.Context, ranges []*ranger.Range) (distsql.SelectResult, error) {
var builder distsql.RequestBuilder
kvReq, err := builder.SetTableRanges(getPhysicalTableID(e.table), ranges, e.feedback).
SetDAGRequest(e.dagPB).
SetDesc(e.desc).
SetKeepOrder(e.keepOrder).
SetStreaming(e.streaming).
SetFromSessionVars(e.ctx.GetSessionVars()).
SetMemTracker(e.memTracker).
SetStoreType(e.storeType).
Build()
if err != nil {
return nil, err
}
e.kvRanges = append(e.kvRanges, kvReq.KeyRanges...)
result, err := e.SelectResult(ctx, e.ctx, kvReq, retTypes(e), e.feedback, getPhysicalPlanIDs(e.plans), e.id)
if err != nil {
return nil, err
}
result.Fetch(ctx)
return result, nil
}
type tableResultHandler struct {
// If the pk is unsigned and we have KeepOrder=true and want ascending order,
// `optionalResult` will handles the request whose range is in signed int range, and
// `result` will handle the request whose range is exceed signed int range.
// If we want descending order, `optionalResult` will handles the request whose range is exceed signed, and
// the `result` will handle the request whose range is in signed.
// Otherwise, we just set `optionalFinished` true and the `result` handles the whole ranges.
optionalResult distsql.SelectResult
result distsql.SelectResult
optionalFinished bool
}
func (tr *tableResultHandler) open(optionalResult, result distsql.SelectResult) {
if optionalResult == nil {
tr.optionalFinished = true
tr.result = result
return
}
tr.optionalResult = optionalResult
tr.result = result
tr.optionalFinished = false
}
func (tr *tableResultHandler) nextChunk(ctx context.Context, chk *chunk.Chunk) error {
if !tr.optionalFinished {
err := tr.optionalResult.Next(ctx, chk)
if err != nil {
return err
}
if chk.NumRows() > 0 {
return nil
}
tr.optionalFinished = true
}
return tr.result.Next(ctx, chk)
}
func (tr *tableResultHandler) nextRaw(ctx context.Context) (data []byte, err error) {
if !tr.optionalFinished {
data, err = tr.optionalResult.NextRaw(ctx)
if err != nil {
return nil, err
}
if data != nil {
return data, nil
}
tr.optionalFinished = true
}
data, err = tr.result.NextRaw(ctx)
if err != nil {
return nil, err
}
return data, nil
}
func (tr *tableResultHandler) Close() error {
err := closeAll(tr.optionalResult, tr.result)
tr.optionalResult, tr.result = nil, nil
return err
}