-
Notifications
You must be signed in to change notification settings - Fork 0
/
distsql_running.go
408 lines (370 loc) · 12.3 KB
/
distsql_running.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
// Copyright 2016 The Cockroach Authors.
//
// 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.
//
// Author: Radu Berinde (radu@cockroachlabs.com)
package sql
import (
"golang.org/x/net/context"
"github.com/cockroachdb/cockroach/pkg/internal/client"
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/rpc"
"github.com/cockroachdb/cockroach/pkg/sql/distsqlrun"
"github.com/cockroachdb/cockroach/pkg/sql/parser"
"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/log"
)
// To allow queries to send out flow RPCs in parallel, we use a pool of workers
// that can issue the RPCs on behalf of the running code. The pool is shared by
// multiple queries.
const numRunners = 16
// runnerRequest is the request that is sent (via a channel) to a worker.
type runnerRequest struct {
ctx context.Context
rpcContext *rpc.Context
flowReq *distsqlrun.SetupFlowRequest
nodeID roachpb.NodeID
nodeAddress string
resultChan chan<- runnerResult
}
// runnerResult is returned by a worker (via a channel) for each received
// request.
type runnerResult struct {
nodeID roachpb.NodeID
err error
}
func (req runnerRequest) run() {
res := runnerResult{nodeID: req.nodeID}
conn, err := req.rpcContext.GRPCDial(req.nodeAddress)
if err != nil {
res.err = err
} else {
client := distsqlrun.NewDistSQLClient(conn)
// TODO(radu): do we want a timeout here?
resp, err := client.SetupFlow(req.ctx, req.flowReq)
if err != nil {
res.err = err
} else {
res.err = resp.Error.ErrorDetail()
}
}
req.resultChan <- res
}
func (dsp *distSQLPlanner) initRunners() {
// This channel has to be unbuffered because we want to only be able to send
// requests if a worker is actually there to receive them.
dsp.runnerChan = make(chan runnerRequest)
for i := 0; i < numRunners; i++ {
dsp.stopper.RunWorker(context.TODO(), func(context.Context) {
runnerChan := dsp.runnerChan
stopChan := dsp.stopper.ShouldStop()
for {
select {
case req := <-runnerChan:
req.run()
case <-stopChan:
return
}
}
})
}
}
// Run executes a physical plan. The plan should have been finalized using
// FinalizePlan.
//
// Note that errors that happen while actually running the flow are reported to
// recv, not returned by this function.
func (dsp *distSQLPlanner) Run(
planCtx *planningCtx,
txn *client.Txn,
plan *physicalPlan,
recv *distSQLReceiver,
evalCtx parser.EvalContext,
) error {
ctx := planCtx.ctx
flows := plan.GenerateFlowSpecs()
if logPlanDiagram {
log.VEvent(ctx, 1, "creating plan diagram")
json, url, err := distsqlrun.GeneratePlanDiagramWithURL(flows)
if err != nil {
log.Infof(ctx, "Error generating diagram: %s", err)
} else {
log.Infof(ctx, "Plan diagram JSON:\n%s", json)
log.Infof(ctx, "Plan diagram URL:\n%s", url.String())
}
}
log.VEvent(ctx, 1, "running DistSQL plan")
recv.resultToStreamColMap = plan.planToStreamColMap
thisNodeID := dsp.nodeDesc.NodeID
// DistSQL needs to initialize the Transaction proto before we put it in the
// FlowRequest's below. This is because we might not have used the txn do to
// anything else (we might not have sent any requests through the client.Txn,
// which normally does this init).
txn.EnsureProto()
evalCtxProto := distsqlrun.MakeEvalContext(evalCtx)
for _, s := range evalCtx.SearchPath {
evalCtxProto.SearchPath = append(evalCtxProto.SearchPath, s)
}
// Start all the flows except the flow on this node (there is always a flow on
// this node).
var resultChan chan runnerResult
if len(flows) > 1 {
resultChan = make(chan runnerResult, len(flows)-1)
}
for nodeID, flowSpec := range flows {
if nodeID == thisNodeID {
// Skip this node.
continue
}
req := &distsqlrun.SetupFlowRequest{
Version: distsqlrun.Version,
Txn: *txn.Proto(),
Flow: flowSpec,
EvalContext: evalCtxProto,
}
if err := distsqlrun.SetFlowRequestTrace(ctx, req); err != nil {
return err
}
runReq := runnerRequest{
ctx: ctx,
rpcContext: dsp.rpcContext,
flowReq: req,
nodeID: nodeID,
nodeAddress: planCtx.nodeAddresses[nodeID],
resultChan: resultChan,
}
// Send out a request to the workers; if no worker is available, run
// directly.
select {
case dsp.runnerChan <- runReq:
default:
runReq.run()
}
}
var firstErr error
// Now wait for all the flows to be scheduled on remote nodes. Note that we
// are not waiting for the flows themselves to complete.
for i := 0; i < len(flows)-1; i++ {
res := <-resultChan
if firstErr == nil {
firstErr = res.err
}
// TODO(radu): accumulate the flows that we failed to set up and move them
// into the local flow.
}
if firstErr != nil {
return firstErr
}
// Set up the flow on this node.
localReq := distsqlrun.SetupFlowRequest{
Version: distsqlrun.Version,
Txn: *txn.Proto(),
Flow: flows[thisNodeID],
EvalContext: evalCtxProto,
}
if err := distsqlrun.SetFlowRequestTrace(ctx, &localReq); err != nil {
return err
}
ctx, flow, err := dsp.distSQLSrv.SetupSyncFlow(ctx, &localReq, recv)
if err != nil {
return err
}
// TODO(radu): this should go through the flow scheduler.
flow.Start(ctx, func() {})
flow.Wait()
flow.Cleanup(ctx)
return nil
}
// distSQLReceiver is a RowReceiver that stores incoming rows in a RowContainer.
// This is where the DistSQL execution meets the SQL Session - the RowContainer
// comes from a client Session.
//
// distSQLReceiver also update the RangeDescriptorCache and the LeaseholderCache
// in response to DistSQL metadata about misplanned ranges.
type distSQLReceiver struct {
ctx context.Context
// rows is the container where we store the results; if we only need the count
// of the rows, it is nil.
rows *sqlbase.RowContainer
// resultToStreamColMap maps result columns to columns in the distsqlrun results
// stream.
resultToStreamColMap []int
// numRows counts the number of rows we received when rows is nil.
numRows int64
// err represents the error that we received either from a producer or
// internally in the operation of the distSQLReceiver. If set, this will
// ultimately be returned as the error for the SQL query.
//
// Once set, no more rows are accepted.
err error
row parser.Datums
status distsqlrun.ConsumerStatus
alloc sqlbase.DatumAlloc
closed bool
rangeCache *kv.RangeDescriptorCache
leaseCache *kv.LeaseHolderCache
// The transaction in which the flow producing data for this receiver runs.
// The distSQLReceiver updates the TransactionProto in response to
// RetryableTxnError's. Nil if no transaction should be updated on errors
// (i.e. if the flow overall doesn't run in a transaction).
txn *client.Txn
// A handler for clock signals arriving from remote nodes. This should update
// this node's clock.
updateClock func(observedTs hlc.Timestamp)
}
var _ distsqlrun.RowReceiver = &distSQLReceiver{}
// makeDistSQLReceiver creates a distSQLReceiver.
//
// ctx is the Context that the receiver will use throughput its lifetime.
// sink is the container where the results will be stored. If only the row count
// is needed, this can be nil.
//
// txn is the transaction in which the producer flow runs; it will be updated
// on errors. Nil if the flow overall doesn't run in a transaction.
func makeDistSQLReceiver(
ctx context.Context,
sink *sqlbase.RowContainer,
rangeCache *kv.RangeDescriptorCache,
leaseCache *kv.LeaseHolderCache,
txn *client.Txn,
updateClock func(observedTs hlc.Timestamp),
) (distSQLReceiver, error) {
return distSQLReceiver{
ctx: ctx,
rows: sink,
rangeCache: rangeCache,
leaseCache: leaseCache,
txn: txn,
updateClock: updateClock,
}, nil
}
// Push is part of the RowReceiver interface.
func (r *distSQLReceiver) Push(
row sqlbase.EncDatumRow, meta distsqlrun.ProducerMetadata,
) distsqlrun.ConsumerStatus {
if !meta.Empty() {
if meta.Err != nil && r.err == nil {
if r.txn != nil {
if retryErr, ok := meta.Err.(*roachpb.UnhandledRetryableError); ok {
// Update the txn in response to remote errors. In the non-DistSQL
// world, the TxnCoordSender does this, and the client.Txn updates
// itself in non-error cases. Those updates are not necessary if we're
// just doing reads. Once DistSQL starts performing writes, we'll need
// to perform such updates too.
r.txn.UpdateStateOnRemoteRetryableErr(r.ctx, retryErr.PErr)
// Update the clock with information from the error. On non-DistSQL
// code paths, the DistSender does this.
// TODO(andrei): We don't propagate clock signals on success cases
// through DistSQL; we should. We also don't propagate them through
// non-retryable errors; we also should.
r.updateClock(retryErr.PErr.Now)
meta.Err = roachpb.NewHandledRetryableTxnError(
meta.Err.Error(), r.txn.Proto().ID, *r.txn.Proto())
}
}
r.err = meta.Err
}
if len(meta.Ranges) > 0 {
if err := r.updateCaches(r.ctx, meta.Ranges); err != nil && r.err == nil {
r.err = err
}
}
return r.status
}
if r.err != nil {
// TODO(andrei): We should drain here.
return distsqlrun.ConsumerClosed
}
if r.status != distsqlrun.NeedMoreRows {
return r.status
}
if r.rows == nil {
// We only need the row count.
r.numRows++
return r.status
}
if r.row == nil {
r.row = make(parser.Datums, len(r.resultToStreamColMap))
}
for i, resIdx := range r.resultToStreamColMap {
err := row[resIdx].EnsureDecoded(&r.alloc)
if err != nil {
r.err = err
r.status = distsqlrun.ConsumerClosed
return r.status
}
r.row[i] = row[resIdx].Datum
}
// Note that AddRow accounts for the memory used by the Datums.
if _, err := r.rows.AddRow(r.ctx, r.row); err != nil {
r.err = err
// TODO(andrei): We should drain here. Metadata from this query would be
// useful, particularly as it was likely a large query (since AddRow()
// above failed, presumably with an out-of-memory error).
r.status = distsqlrun.ConsumerClosed
return r.status
}
return r.status
}
// ProducerDone is part of the RowReceiver interface.
func (r *distSQLReceiver) ProducerDone() {
if r.closed {
panic("double close")
}
r.closed = true
}
// updateCaches takes information about some ranges that were mis-planned and
// updates the range descriptor and lease-holder caches accordingly.
//
// TODO(andrei): updating these caches is not perfect: we can clobber newer
// information that someone else has populated because there's no timing info
// anywhere. We also may fail to remove stale info from the LeaseHolderCache if
// the ids of the ranges that we get are different than the ids in that cache.
func (r *distSQLReceiver) updateCaches(ctx context.Context, ranges []roachpb.RangeInfo) error {
// Update the RangeDescriptorCache.
rngDescs := make([]roachpb.RangeDescriptor, len(ranges))
for i, ri := range ranges {
rngDescs[i] = ri.Desc
}
if err := r.rangeCache.InsertRangeDescriptors(ctx, rngDescs...); err != nil {
return err
}
// Update the LeaseHolderCache.
for _, ri := range ranges {
r.leaseCache.Update(ctx, ri.Desc.RangeID, ri.Lease.Replica)
}
return nil
}
// PlanAndRun generates a physical plan from a planNode tree and executes it. It
// assumes that the tree is supported (see CheckSupport).
//
// Note that errors that happen while actually running the flow are reported to
// recv, not returned by this function.
func (dsp *distSQLPlanner) PlanAndRun(
ctx context.Context,
txn *client.Txn,
tree planNode,
recv *distSQLReceiver,
evalCtx parser.EvalContext,
) error {
planCtx := dsp.NewPlanningCtx(ctx, txn)
log.VEvent(ctx, 1, "creating DistSQL plan")
plan, err := dsp.createPlanForNode(&planCtx, tree)
if err != nil {
return err
}
dsp.FinalizePlan(&planCtx, &plan)
return dsp.Run(&planCtx, txn, &plan, recv, evalCtx)
}