-
Notifications
You must be signed in to change notification settings - Fork 5.9k
/
Copy pathdistsql.go
258 lines (233 loc) · 9.37 KB
/
distsql.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
// Copyright 2017 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 distsql
import (
"context"
"strconv"
"unsafe"
"github.com/pingcap/errors"
"github.com/pingcap/tidb/config"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/metrics"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/sessionctx/stmtctx"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/statistics"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/logutil"
"github.com/pingcap/tidb/util/tracing"
"github.com/pingcap/tidb/util/trxevents"
"github.com/pingcap/tipb/go-tipb"
"github.com/tikv/client-go/v2/tikvrpc/interceptor"
"go.uber.org/zap"
"google.golang.org/grpc/metadata"
)
// GenSelectResultFromResponse generates an iterator from response.
func GenSelectResultFromResponse(sctx sessionctx.Context, fieldTypes []*types.FieldType, planIDs []int, rootID int, resp kv.Response) SelectResult {
// TODO: Add metric label and set open tracing.
return &selectResult{
label: "mpp",
resp: resp,
rowLen: len(fieldTypes),
fieldTypes: fieldTypes,
ctx: sctx,
feedback: statistics.NewQueryFeedback(0, nil, 0, false),
copPlanIDs: planIDs,
rootPlanID: rootID,
storeType: kv.TiFlash,
}
}
// Select sends a DAG request, returns SelectResult.
// In kvReq, KeyRanges is required, Concurrency/KeepOrder/Desc/IsolationLevel/Priority are optional.
func Select(ctx context.Context, sctx sessionctx.Context, kvReq *kv.Request, fieldTypes []*types.FieldType, fb *statistics.QueryFeedback) (SelectResult, error) {
r, ctx := tracing.StartRegionEx(ctx, "distsql.Select")
defer r.End()
// For testing purpose.
if hook := ctx.Value("CheckSelectRequestHook"); hook != nil {
hook.(func(*kv.Request))(kvReq)
}
enabledRateLimitAction := sctx.GetSessionVars().EnabledRateLimitAction
originalSQL := sctx.GetSessionVars().StmtCtx.OriginalSQL
eventCb := func(event trxevents.TransactionEvent) {
// Note: Do not assume this callback will be invoked within the same goroutine.
if copMeetLock := event.GetCopMeetLock(); copMeetLock != nil {
logutil.Logger(ctx).Debug("coprocessor encounters lock",
zap.Uint64("startTS", kvReq.StartTs),
zap.Stringer("lock", copMeetLock.LockInfo),
zap.String("stmt", originalSQL))
}
}
ctx = WithSQLKvExecCounterInterceptor(ctx, sctx.GetSessionVars().StmtCtx)
option := &kv.ClientSendOption{
SessionMemTracker: sctx.GetSessionVars().MemTracker,
EnabledRateLimitAction: enabledRateLimitAction,
EventCb: eventCb,
EnableCollectExecutionInfo: config.GetGlobalConfig().Instance.EnableCollectExecutionInfo.Load(),
}
if kvReq.StoreType == kv.TiFlash {
ctx = SetTiFlashConfVarsInContext(ctx, sctx)
}
resp := sctx.GetClient().Send(ctx, kvReq, sctx.GetSessionVars().KVVars, option)
if resp == nil {
return nil, errors.New("client returns nil response")
}
label := metrics.LblGeneral
if sctx.GetSessionVars().InRestrictedSQL {
label = metrics.LblInternal
}
// kvReq.MemTracker is used to trace and control memory usage in DistSQL layer;
// for selectResult, we just use the kvReq.MemTracker prepared for co-processor
// instead of creating a new one for simplification.
return &selectResult{
label: "dag",
resp: resp,
rowLen: len(fieldTypes),
fieldTypes: fieldTypes,
ctx: sctx,
feedback: fb,
sqlType: label,
memTracker: kvReq.MemTracker,
storeType: kvReq.StoreType,
paging: kvReq.Paging.Enable,
distSQLConcurrency: kvReq.Concurrency,
}, nil
}
// SetTiFlashConfVarsInContext set some TiFlash config variables in context.
func SetTiFlashConfVarsInContext(ctx context.Context, sctx sessionctx.Context) context.Context {
if sctx.GetSessionVars().TiFlashMaxThreads != -1 {
ctx = metadata.AppendToOutgoingContext(ctx, variable.TiDBMaxTiFlashThreads, strconv.FormatInt(sctx.GetSessionVars().TiFlashMaxThreads, 10))
}
if sctx.GetSessionVars().TiFlashMaxBytesBeforeExternalJoin != -1 {
ctx = metadata.AppendToOutgoingContext(ctx, variable.TiDBMaxBytesBeforeTiFlashExternalJoin, strconv.FormatInt(sctx.GetSessionVars().TiFlashMaxBytesBeforeExternalJoin, 10))
}
if sctx.GetSessionVars().TiFlashMaxBytesBeforeExternalGroupBy != -1 {
ctx = metadata.AppendToOutgoingContext(ctx, variable.TiDBMaxBytesBeforeTiFlashExternalGroupBy, strconv.FormatInt(sctx.GetSessionVars().TiFlashMaxBytesBeforeExternalGroupBy, 10))
}
if sctx.GetSessionVars().TiFlashMaxBytesBeforeExternalSort != -1 {
ctx = metadata.AppendToOutgoingContext(ctx, variable.TiDBMaxBytesBeforeTiFlashExternalSort, strconv.FormatInt(sctx.GetSessionVars().TiFlashMaxBytesBeforeExternalSort, 10))
}
return ctx
}
// SelectWithRuntimeStats sends a DAG request, returns SelectResult.
// The difference from Select is that SelectWithRuntimeStats will set copPlanIDs into selectResult,
// which can help selectResult to collect runtime stats.
func SelectWithRuntimeStats(ctx context.Context, sctx sessionctx.Context, kvReq *kv.Request,
fieldTypes []*types.FieldType, fb *statistics.QueryFeedback, copPlanIDs []int, rootPlanID int) (SelectResult, error) {
sr, err := Select(ctx, sctx, kvReq, fieldTypes, fb)
if err != nil {
return nil, err
}
if selectResult, ok := sr.(*selectResult); ok {
selectResult.copPlanIDs = copPlanIDs
selectResult.rootPlanID = rootPlanID
}
return sr, nil
}
// Analyze do a analyze request.
func Analyze(ctx context.Context, client kv.Client, kvReq *kv.Request, vars interface{},
isRestrict bool, stmtCtx *stmtctx.StatementContext) (SelectResult, error) {
ctx = WithSQLKvExecCounterInterceptor(ctx, stmtCtx)
kvReq.RequestSource.RequestSourceInternal = true
kvReq.RequestSource.RequestSourceType = kv.InternalTxnStats
resp := client.Send(ctx, kvReq, vars, &kv.ClientSendOption{})
if resp == nil {
return nil, errors.New("client returns nil response")
}
label := metrics.LblGeneral
if isRestrict {
label = metrics.LblInternal
}
result := &selectResult{
label: "analyze",
resp: resp,
feedback: statistics.NewQueryFeedback(0, nil, 0, false),
sqlType: label,
storeType: kvReq.StoreType,
}
return result, nil
}
// Checksum sends a checksum request.
func Checksum(ctx context.Context, client kv.Client, kvReq *kv.Request, vars interface{}) (SelectResult, error) {
// FIXME: As BR have dependency of `Checksum` and TiDB also introduced BR as dependency, Currently we can't edit
// Checksum function signature. The two-way dependence should be removed in future.
resp := client.Send(ctx, kvReq, vars, &kv.ClientSendOption{})
if resp == nil {
return nil, errors.New("client returns nil response")
}
result := &selectResult{
label: "checksum",
resp: resp,
feedback: statistics.NewQueryFeedback(0, nil, 0, false),
sqlType: metrics.LblGeneral,
storeType: kvReq.StoreType,
}
return result, nil
}
// SetEncodeType sets the encoding method for the DAGRequest. The supported encoding
// methods are:
// 1. TypeChunk: the result is encoded using the Chunk format, refer util/chunk/chunk.go
// 2. TypeDefault: the result is encoded row by row
func SetEncodeType(ctx sessionctx.Context, dagReq *tipb.DAGRequest) {
if canUseChunkRPC(ctx) {
dagReq.EncodeType = tipb.EncodeType_TypeChunk
setChunkMemoryLayout(dagReq)
} else {
dagReq.EncodeType = tipb.EncodeType_TypeDefault
}
}
func canUseChunkRPC(ctx sessionctx.Context) bool {
if !ctx.GetSessionVars().EnableChunkRPC {
return false
}
if !checkAlignment() {
return false
}
return true
}
var supportedAlignment = unsafe.Sizeof(types.MyDecimal{}) == 40
// checkAlignment checks the alignment in current system environment.
// The alignment is influenced by system, machine and Golang version.
// Using this function can guarantee the alignment is we want.
func checkAlignment() bool {
return supportedAlignment
}
var systemEndian tipb.Endian
// setChunkMemoryLayout sets the chunk memory layout for the DAGRequest.
func setChunkMemoryLayout(dagReq *tipb.DAGRequest) {
dagReq.ChunkMemoryLayout = &tipb.ChunkMemoryLayout{Endian: GetSystemEndian()}
}
// GetSystemEndian gets the system endian.
func GetSystemEndian() tipb.Endian {
return systemEndian
}
func init() {
i := 0x0100
ptr := unsafe.Pointer(&i)
if 0x01 == *(*byte)(ptr) {
systemEndian = tipb.Endian_BigEndian
} else {
systemEndian = tipb.Endian_LittleEndian
}
}
// WithSQLKvExecCounterInterceptor binds an interceptor for client-go to count the
// number of SQL executions of each TiKV (if any).
func WithSQLKvExecCounterInterceptor(ctx context.Context, stmtCtx *stmtctx.StatementContext) context.Context {
if stmtCtx.KvExecCounter != nil {
// Unlike calling Transaction or Snapshot interface, in distsql package we directly
// face tikv Request. So we need to manually bind RPCInterceptor to ctx. Instead of
// calling SetRPCInterceptor on Transaction or Snapshot.
return interceptor.WithRPCInterceptor(ctx, stmtCtx.KvExecCounter.RPCInterceptor())
}
return ctx
}