-
Notifications
You must be signed in to change notification settings - Fork 3.8k
/
distsql_running.go
1198 lines (1094 loc) · 40.5 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
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
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
// Copyright 2016 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
package sql
import (
"context"
"fmt"
"math"
"sync"
"sync/atomic"
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/rpc"
"github.com/cockroachdb/cockroach/pkg/rpc/nodedialer"
"github.com/cockroachdb/cockroach/pkg/server/telemetry"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo"
"github.com/cockroachdb/cockroach/pkg/sql/colflow"
"github.com/cockroachdb/cockroach/pkg/sql/distsql"
"github.com/cockroachdb/cockroach/pkg/sql/execinfra"
"github.com/cockroachdb/cockroach/pkg/sql/execinfrapb"
"github.com/cockroachdb/cockroach/pkg/sql/flowinfra"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/physicalplan"
"github.com/cockroachdb/cockroach/pkg/sql/rowcontainer"
"github.com/cockroachdb/cockroach/pkg/sql/rowenc"
"github.com/cockroachdb/cockroach/pkg/sql/rowexec"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
"github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/util/errorutil"
"github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/mon"
"github.com/cockroachdb/cockroach/pkg/util/tracing"
"github.com/cockroachdb/cockroach/pkg/util/uuid"
"github.com/cockroachdb/errors"
opentracing "github.com/opentracing/opentracing-go"
)
// 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
const clientRejectedMsg string = "client rejected when attempting to run DistSQL plan"
// runnerRequest is the request that is sent (via a channel) to a worker.
type runnerRequest struct {
ctx context.Context
nodeDialer *nodedialer.Dialer
flowReq *execinfrapb.SetupFlowRequest
nodeID roachpb.NodeID
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.nodeDialer.Dial(req.ctx, req.nodeID, rpc.DefaultClass)
if err != nil {
res.err = err
} else {
client := execinfrapb.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.ctx)
}
}
req.resultChan <- res
}
func (dsp *DistSQLPlanner) initRunners(ctx context.Context) {
// 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(ctx, func(context.Context) {
runnerChan := dsp.runnerChan
stopChan := dsp.stopper.ShouldStop()
for {
select {
case req := <-runnerChan:
req.run()
case <-stopChan:
return
}
}
})
}
}
// setupFlows sets up all the flows specified in flows using the provided state.
// It will first attempt to set up all remote flows using the dsp workers if
// available or sequentially if not, and then finally set up the gateway flow,
// whose output is the DistSQLReceiver provided. This flow is then returned to
// be run. It also returns a boolean indicating whether the flow is vectorized.
func (dsp *DistSQLPlanner) setupFlows(
ctx context.Context,
evalCtx *extendedEvalContext,
leafInputState *roachpb.LeafTxnInputState,
flows map[roachpb.NodeID]*execinfrapb.FlowSpec,
recv *DistSQLReceiver,
localState distsql.LocalState,
vectorizeThresholdMet bool,
) (context.Context, flowinfra.Flow, error) {
thisNodeID := dsp.gatewayNodeID
_, ok := flows[thisNodeID]
if !ok {
return nil, nil, errors.AssertionFailedf("missing gateway flow")
}
if localState.IsLocal && len(flows) != 1 {
return nil, nil, errors.AssertionFailedf("IsLocal set but there's multiple flows")
}
evalCtxProto := execinfrapb.MakeEvalContext(&evalCtx.EvalContext)
setupReq := execinfrapb.SetupFlowRequest{
LeafTxnInputState: leafInputState,
Version: execinfra.Version,
EvalContext: evalCtxProto,
TraceKV: evalCtx.Tracing.KVTracingEnabled(),
}
// 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)
}
if evalCtx.SessionData.VectorizeMode != sessiondata.VectorizeOff {
if !vectorizeThresholdMet && (evalCtx.SessionData.VectorizeMode == sessiondata.Vectorize201Auto || evalCtx.SessionData.VectorizeMode == sessiondata.VectorizeOn) {
// Vectorization is not justified for this flow because the expected
// amount of data is too small and the overhead of pre-allocating data
// structures needed for the vectorized engine is expected to dominate
// the execution time.
setupReq.EvalContext.Vectorize = int32(sessiondata.VectorizeOff)
} else {
// Now we check to see whether or not to even try vectorizing the flow.
// The goal here is to determine up front whether all of the flows can be
// vectorized. If any of them can't, turn off the setting.
// TODO(yuzefovich): this is a safe but quite inefficient way of setting
// up vectorized flows since the flows will effectively be planned twice.
// TODO (rohany): This is unfortunate that this call to setup vectorize makes
// it's own flow context rather than being able to use the one that is made
// later.
flowCtx := dsp.distSQLSrv.NewFlowContext(ctx, execinfrapb.FlowID{}, &evalCtx.EvalContext, setupReq.TraceKV, localState)
// This flowCtx is only used during the vectorize check, so we need to
// clean up any accessed descriptors after checking.
defer func() {
flowCtx.TypeResolverFactory.CleanupFunc(ctx)
}()
for scheduledOnNodeID, spec := range flows {
scheduledOnRemoteNode := scheduledOnNodeID != thisNodeID
if _, err := colflow.SupportsVectorized(
ctx, &flowCtx, spec.Processors, localState.IsLocal, recv, scheduledOnRemoteNode,
); err != nil {
// Vectorization attempt failed with an error.
returnVectorizationSetupError := false
if evalCtx.SessionData.VectorizeMode == sessiondata.VectorizeExperimentalAlways {
returnVectorizationSetupError = true
// If running with VectorizeExperimentalAlways, this check makes sure
// that we can still run SET statements (mostly to set vectorize to
// off) and the like.
if len(spec.Processors) == 1 &&
spec.Processors[0].Core.LocalPlanNode != nil {
rsidx := spec.Processors[0].Core.LocalPlanNode.RowSourceIdx
if rsidx != nil {
lp := localState.LocalProcs[*rsidx]
if z, ok := lp.(colflow.VectorizeAlwaysException); ok {
if z.IsException() {
returnVectorizationSetupError = false
}
}
}
}
}
log.VEventf(ctx, 1, "failed to vectorize: %s", err)
if returnVectorizationSetupError {
return nil, nil, err
}
// Vectorization is not supported for this flow, so we override the
// setting.
setupReq.EvalContext.Vectorize = int32(sessiondata.VectorizeOff)
break
}
}
}
}
for nodeID, flowSpec := range flows {
if nodeID == thisNodeID {
// Skip this node.
continue
}
if !evalCtx.Codec.ForSystemTenant() {
// A tenant server should never find itself distributing flows.
// NB: we wouldn't hit this in practice but if we did the actual
// error would be opaque.
return nil, nil, errorutil.UnsupportedWithMultiTenancy(47900)
}
req := setupReq
req.Flow = *flowSpec
runReq := runnerRequest{
ctx: ctx,
nodeDialer: dsp.nodeDialer,
flowReq: &req,
nodeID: nodeID,
resultChan: resultChan,
}
defer physicalplan.ReleaseSetupFlowRequest(&req)
// 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 nil, nil, firstErr
}
// Set up the flow on this node.
localReq := setupReq
localReq.Flow = *flows[thisNodeID]
defer physicalplan.ReleaseSetupFlowRequest(&localReq)
ctx, flow, err := dsp.distSQLSrv.SetupLocalSyncFlow(ctx, evalCtx.Mon, &localReq, recv, localState)
if err != nil {
return nil, nil, err
}
return ctx, flow, nil
}
// Run executes a physical plan. The plan should have been finalized using
// FinalizePlan.
//
// All errors encountered are reported to the DistSQLReceiver's resultWriter.
// Additionally, if the error is a "communication error" (an error encountered
// while using that resultWriter), the error is also stored in
// DistSQLReceiver.commErr. That can be tested to see if a client session needs
// to be closed.
//
// Args:
// - txn is the transaction in which the plan will run. If nil, the different
// processors are expected to manage their own internal transactions.
// - evalCtx is the evaluation context in which the plan will run. It might be
// mutated.
// - finishedSetupFn, if non-nil, is called synchronously after all the
// processors have successfully started up.
//
// It returns a non-nil (although it can be a noop when an error is
// encountered) cleanup function that must be called in order to release the
// resources.
func (dsp *DistSQLPlanner) Run(
planCtx *PlanningCtx,
txn *kv.Txn,
plan *PhysicalPlan,
recv *DistSQLReceiver,
evalCtx *extendedEvalContext,
finishedSetupFn func(),
) (cleanup func()) {
ctx := planCtx.ctx
var (
localState distsql.LocalState
leafInputState *roachpb.LeafTxnInputState
)
// NB: putting part of evalCtx in localState means it might be mutated down
// the line.
localState.EvalContext = &evalCtx.EvalContext
localState.Txn = txn
localState.LocalProcs = plan.LocalProcessors
// If we have access to a planner and are currently being used to plan
// statements in a user transaction, then take the descs.Collection to resolve
// types with during flow execution. This is necessary to do in the case of
// a transaction that has already created or updated some types. If we do not
// use the local descs.Collection, we would attempt to acquire a lease on
// modified types when accessing them, which would error out.
if planCtx.planner != nil && !planCtx.planner.isInternalPlanner {
localState.Collection = planCtx.planner.Descriptors()
}
if planCtx.isLocal {
localState.IsLocal = true
} else if txn != nil {
// If the plan is not local, we will have to set up leaf txns using the
// txnCoordMeta.
tis, err := txn.GetLeafTxnInputStateOrRejectClient(ctx)
if err != nil {
log.Infof(ctx, "%s: %s", clientRejectedMsg, err)
recv.SetError(err)
return func() {}
}
leafInputState = &tis
}
flows := plan.GenerateFlowSpecs()
if _, ok := flows[dsp.gatewayNodeID]; !ok {
recv.SetError(errors.Errorf("expected to find gateway flow"))
return func() {}
}
if planCtx.saveDiagram != nil {
// Local flows might not have the UUID field set. We need it to be set to
// distinguish statistics for processors in subqueries vs the main query vs
// postqueries.
if len(flows) == 1 {
for _, f := range flows {
if f.FlowID == (execinfrapb.FlowID{}) {
f.FlowID.UUID = uuid.MakeV4()
}
}
}
log.VEvent(ctx, 1, "creating plan diagram")
var stmtStr string
if planCtx.planner != nil && planCtx.planner.stmt != nil {
stmtStr = planCtx.planner.stmt.String()
}
diagram, err := execinfrapb.GeneratePlanDiagram(
stmtStr, flows, planCtx.saveDiagramShowInputTypes,
)
if err != nil {
recv.SetError(err)
return func() {}
}
planCtx.saveDiagram(diagram)
}
if logPlanDiagram {
log.VEvent(ctx, 1, "creating plan diagram for logging")
var stmtStr string
if planCtx.planner != nil && planCtx.planner.stmt != nil {
stmtStr = planCtx.planner.stmt.String()
}
_, url, err := execinfrapb.GeneratePlanDiagramURL(stmtStr, flows, false /* showInputTypes */)
if err != nil {
log.Infof(ctx, "error generating diagram: %s", err)
} else {
log.Infof(ctx, "plan diagram URL:\n%s", url.String())
}
}
log.VEvent(ctx, 1, "running DistSQL plan")
dsp.distSQLSrv.ServerConfig.Metrics.QueryStart()
defer dsp.distSQLSrv.ServerConfig.Metrics.QueryStop()
recv.outputTypes = plan.ResultTypes
vectorizedThresholdMet := plan.MaxEstimatedRowCount >= evalCtx.SessionData.VectorizeRowCountThreshold
if len(flows) == 1 {
// We ended up planning everything locally, regardless of whether we
// intended to distribute or not.
localState.IsLocal = true
}
ctx, flow, err := dsp.setupFlows(ctx, evalCtx, leafInputState, flows, recv, localState, vectorizedThresholdMet)
if err != nil {
recv.SetError(err)
return func() {}
}
if finishedSetupFn != nil {
finishedSetupFn()
}
if planCtx.planner != nil && flow.IsVectorized() {
planCtx.planner.curPlan.flags.Set(planFlagVectorized)
}
// Check that flows that were forced to be planned locally also have no concurrency.
// This is important, since these flows are forced to use the RootTxn (since
// they might have mutations), and the RootTxn does not permit concurrency.
// For such flows, we were supposed to have fused everything.
if txn != nil && planCtx.isLocal && flow.ConcurrentTxnUse() {
recv.SetError(errors.AssertionFailedf(
"unexpected concurrency for a flow that was forced to be planned locally"))
return func() {}
}
// TODO(radu): this should go through the flow scheduler.
if err := flow.Run(ctx, func() {}); err != nil {
log.Fatalf(ctx, "unexpected error from syncFlow.Start(): %v\n"+
"The error should have gone to the consumer.", err)
}
// TODO(yuzefovich): it feels like this closing should happen after
// PlanAndRun. We should refactor this and get rid off ignoreClose field.
if planCtx.planner != nil && !planCtx.ignoreClose {
// planCtx can change before the cleanup function is executed, so we make
// a copy of the planner and bind it to the function.
curPlan := &planCtx.planner.curPlan
return func() {
// We need to close the planNode tree we translated into a DistSQL plan
// before flow.Cleanup, which closes memory accounts that expect to be
// emptied.
curPlan.execErr = recv.resultWriter.Err()
curPlan.close(ctx)
flow.Cleanup(ctx)
}
}
// ignoreClose is set to true meaning that someone else will handle the
// closing of the current plan, so we simply clean up the flow.
return func() {
flow.Cleanup(ctx)
}
}
// DistSQLReceiver is a RowReceiver that writes results to a rowResultWriter.
// This is where the DistSQL execution meets the SQL Session - the RowContainer
// comes from a client Session.
//
// DistSQLReceiver also update the RangeDescriptorCache in response to DistSQL
// metadata about misplanned ranges.
type DistSQLReceiver struct {
ctx context.Context
// resultWriter is the interface which we send results to.
resultWriter rowResultWriter
stmtType tree.StatementType
// outputTypes are the types of the result columns produced by the plan.
outputTypes []*types.T
// noColsRequired indicates that the caller is only interested in the
// existence of a single row. Used by subqueries in EXISTS mode.
noColsRequired bool
// discardRows is set when we want to discard rows (for testing/benchmarks).
// See EXECUTE .. DISCARD ROWS.
discardRows bool
// commErr keeps track of the error received from interacting with the
// resultWriter. This represents a "communication error" and as such is unlike
// query execution errors: when the DistSQLReceiver is used within a SQL
// session, such errors mean that we have to bail on the session.
// Query execution errors are reported to the resultWriter. For some client's
// convenience, communication errors are also reported to the resultWriter.
//
// Once set, no more rows are accepted.
commErr error
row tree.Datums
status execinfra.ConsumerStatus
alloc rowenc.DatumAlloc
closed bool
rangeCache *kvcoord.RangeDescriptorCache
tracing *SessionTracing
cleanup func()
// The transaction in which the flow producing data for this
// receiver runs. The DistSQLReceiver updates the transaction in
// response to RetryableTxnError's and when distributed processors
// pass back LeafTxnFinalState objects via ProducerMetas. Nil if no
// transaction should be updated on errors (i.e. if the flow overall
// doesn't run in a transaction).
txn *kv.Txn
// A handler for clock signals arriving from remote nodes. This should update
// this node's clock.
updateClock func(observedTs hlc.Timestamp)
stats topLevelQueryStats
expectedRowsRead int64
progressAtomic *uint64
}
// rowResultWriter is a subset of CommandResult to be used with the
// DistSQLReceiver. It's implemented by RowResultWriter.
type rowResultWriter interface {
// AddRow writes a result row.
// Note that the caller owns the row slice and might reuse it.
AddRow(ctx context.Context, row tree.Datums) error
IncrementRowsAffected(n int)
SetError(error)
Err() error
}
// MetadataResultWriter is used to stream metadata rather than row results in a
// DistSQL flow.
type MetadataResultWriter interface {
AddMeta(ctx context.Context, meta *execinfrapb.ProducerMetadata)
}
// MetadataCallbackWriter wraps a rowResultWriter to stream metadata in a
// DistSQL flow. It executes a given callback when metadata is added.
type MetadataCallbackWriter struct {
rowResultWriter
fn func(ctx context.Context, meta *execinfrapb.ProducerMetadata) error
}
// AddMeta implements the MetadataResultWriter interface.
func (w *MetadataCallbackWriter) AddMeta(ctx context.Context, meta *execinfrapb.ProducerMetadata) {
if err := w.fn(ctx, meta); err != nil {
w.SetError(err)
}
}
// NewMetadataCallbackWriter creates a new MetadataCallbackWriter.
func NewMetadataCallbackWriter(
rowResultWriter rowResultWriter,
metaFn func(ctx context.Context, meta *execinfrapb.ProducerMetadata) error,
) *MetadataCallbackWriter {
return &MetadataCallbackWriter{rowResultWriter: rowResultWriter, fn: metaFn}
}
// errOnlyResultWriter is a rowResultWriter that only supports receiving an
// error. All other functions that deal with producing results panic.
type errOnlyResultWriter struct {
err error
}
var _ rowResultWriter = &errOnlyResultWriter{}
func (w *errOnlyResultWriter) SetError(err error) {
w.err = err
}
func (w *errOnlyResultWriter) Err() error {
return w.err
}
func (w *errOnlyResultWriter) AddRow(ctx context.Context, row tree.Datums) error {
panic("AddRow not supported by errOnlyResultWriter")
}
func (w *errOnlyResultWriter) IncrementRowsAffected(n int) {
panic("IncrementRowsAffected not supported by errOnlyResultWriter")
}
var _ execinfra.RowReceiver = &DistSQLReceiver{}
var receiverSyncPool = sync.Pool{
New: func() interface{} {
return &DistSQLReceiver{}
},
}
// MakeDistSQLReceiver creates a DistSQLReceiver.
//
// ctx is the Context that the receiver will use throughout its
// lifetime. resultWriter 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,
resultWriter rowResultWriter,
stmtType tree.StatementType,
rangeCache *kvcoord.RangeDescriptorCache,
txn *kv.Txn,
updateClock func(observedTs hlc.Timestamp),
tracing *SessionTracing,
) *DistSQLReceiver {
consumeCtx, cleanup := tracing.TraceExecConsume(ctx)
r := receiverSyncPool.Get().(*DistSQLReceiver)
*r = DistSQLReceiver{
ctx: consumeCtx,
cleanup: cleanup,
resultWriter: resultWriter,
rangeCache: rangeCache,
txn: txn,
updateClock: updateClock,
stmtType: stmtType,
tracing: tracing,
}
return r
}
// Release releases this DistSQLReceiver back to the pool.
func (r *DistSQLReceiver) Release() {
*r = DistSQLReceiver{}
receiverSyncPool.Put(r)
}
// clone clones the receiver for running subqueries. Not all fields are cloned,
// only those required for running subqueries.
func (r *DistSQLReceiver) clone() *DistSQLReceiver {
ret := receiverSyncPool.Get().(*DistSQLReceiver)
*ret = DistSQLReceiver{
ctx: r.ctx,
cleanup: func() {},
rangeCache: r.rangeCache,
txn: r.txn,
updateClock: r.updateClock,
stmtType: tree.Rows,
tracing: r.tracing,
}
return ret
}
// SetError provides a convenient way for a client to pass in an error, thus
// pretending that a query execution error happened. The error is passed along
// to the resultWriter.
func (r *DistSQLReceiver) SetError(err error) {
r.resultWriter.SetError(err)
}
// Push is part of the RowReceiver interface.
func (r *DistSQLReceiver) Push(
row rowenc.EncDatumRow, meta *execinfrapb.ProducerMetadata,
) execinfra.ConsumerStatus {
if meta != nil {
if meta.LeafTxnFinalState != nil {
if r.txn != nil {
if r.txn.ID() == meta.LeafTxnFinalState.Txn.ID {
if err := r.txn.UpdateRootWithLeafFinalState(r.ctx, meta.LeafTxnFinalState); err != nil {
r.resultWriter.SetError(err)
}
}
} else {
r.resultWriter.SetError(
errors.Errorf("received a leaf final state (%s); but have no root", meta.LeafTxnFinalState))
}
}
if meta.Err != nil {
// Check if the error we just received should take precedence over a
// previous error (if any).
if roachpb.ErrPriority(meta.Err) > roachpb.ErrPriority(r.resultWriter.Err()) {
if r.txn != nil {
if retryErr := (*roachpb.UnhandledRetryableError)(nil); errors.As(meta.Err, &retryErr) {
// Update the txn in response to remote errors. In the non-DistSQL
// world, the TxnCoordSender handles "unhandled" retryable errors,
// but this one is coming from a distributed SQL node, which has
// left the handling up to the root transaction.
meta.Err = 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)
}
}
r.resultWriter.SetError(meta.Err)
}
}
if len(meta.Ranges) > 0 {
r.rangeCache.Insert(r.ctx, meta.Ranges...)
}
if len(meta.TraceData) > 0 {
span := opentracing.SpanFromContext(r.ctx)
if span == nil {
r.resultWriter.SetError(
errors.New("trying to ingest remote spans but there is no recording span set up"))
} else if err := tracing.ImportRemoteSpans(span, meta.TraceData); err != nil {
r.resultWriter.SetError(errors.Errorf("error ingesting remote spans: %s", err))
}
}
if meta.Metrics != nil {
r.stats.bytesRead += meta.Metrics.BytesRead
r.stats.rowsRead += meta.Metrics.RowsRead
if r.progressAtomic != nil && r.expectedRowsRead != 0 {
progress := float64(r.stats.rowsRead) / float64(r.expectedRowsRead)
atomic.StoreUint64(r.progressAtomic, math.Float64bits(progress))
}
meta.Metrics.Release()
meta.Release()
}
if metaWriter, ok := r.resultWriter.(MetadataResultWriter); ok {
metaWriter.AddMeta(r.ctx, meta)
}
return r.status
}
if r.resultWriter.Err() == nil && r.ctx.Err() != nil {
r.resultWriter.SetError(r.ctx.Err())
}
if r.resultWriter.Err() != nil {
// TODO(andrei): We should drain here if we weren't canceled.
return execinfra.ConsumerClosed
}
if r.status != execinfra.NeedMoreRows {
return r.status
}
if r.stmtType != tree.Rows {
// We only need the row count. planNodeToRowSource is set up to handle
// ensuring that the last stage in the pipeline will return a single-column
// row with the row count in it, so just grab that and exit.
r.resultWriter.IncrementRowsAffected(int(tree.MustBeDInt(row[0].Datum)))
return r.status
}
if r.discardRows {
// Discard rows.
return r.status
}
// If no columns are needed by the output, the consumer is only looking for
// whether a single row is pushed or not, so the contents do not matter, and
// planNodeToRowSource is not set up to handle decoding the row.
if r.noColsRequired {
r.row = []tree.Datum{}
r.status = execinfra.ConsumerClosed
} else {
if r.row == nil {
r.row = make(tree.Datums, len(row))
}
for i, encDatum := range row {
err := encDatum.EnsureDecoded(r.outputTypes[i], &r.alloc)
if err != nil {
r.resultWriter.SetError(err)
r.status = execinfra.ConsumerClosed
return r.status
}
r.row[i] = encDatum.Datum
}
}
r.tracing.TraceExecRowsResult(r.ctx, r.row)
// Note that AddRow accounts for the memory used by the Datums.
if commErr := r.resultWriter.AddRow(r.ctx, r.row); commErr != nil {
// ErrLimitedResultClosed is not a real error, it is a
// signal to stop distsql and return success to the client.
if !errors.Is(commErr, ErrLimitedResultClosed) {
// Set the error on the resultWriter too, for the convenience of some of the
// clients. If clients don't care to differentiate between communication
// errors and query execution errors, they can simply inspect
// resultWriter.Err(). Also, this function itself doesn't care about the
// distinction and just uses resultWriter.Err() to see if we're still
// accepting results.
r.resultWriter.SetError(commErr)
// We don't need to shut down the connection
// if there's a portal-related error. This is
// definitely a layering violation, but is part
// of some accepted technical debt (see comments on
// sql/pgwire.limitedCommandResult.moreResultsNeeded).
// Instead of changing the signature of AddRow, we have
// a sentinel error that is handled specially here.
if !errors.Is(commErr, ErrLimitedResultNotSupported) {
r.commErr = commErr
}
}
// 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 = execinfra.ConsumerClosed
}
return r.status
}
var (
// ErrLimitedResultNotSupported is an error produced by pgwire
// indicating an unsupported feature of row count limits was attempted.
ErrLimitedResultNotSupported = unimplemented.NewWithIssue(40195, "multiple active portals not supported")
// ErrLimitedResultClosed is a sentinel error produced by pgwire
// indicating the portal should be closed without error.
ErrLimitedResultClosed = errors.New("row count limit closed")
)
// ProducerDone is part of the RowReceiver interface.
func (r *DistSQLReceiver) ProducerDone() {
if r.closed {
panic("double close")
}
r.closed = true
r.cleanup()
}
// Types is part of the RowReceiver interface.
func (r *DistSQLReceiver) Types() []*types.T {
return r.outputTypes
}
// PlanAndRunSubqueries returns false if an error was encountered and sets that
// error in the provided receiver. Note that if false is returned, then this
// function will have closed all the subquery plans because it assumes that the
// caller will not try to run the main plan given that the subqueries'
// evaluation failed.
func (dsp *DistSQLPlanner) PlanAndRunSubqueries(
ctx context.Context,
planner *planner,
evalCtxFactory func() *extendedEvalContext,
subqueryPlans []subquery,
recv *DistSQLReceiver,
maybeDistribute bool,
) bool {
for planIdx, subqueryPlan := range subqueryPlans {
if err := dsp.planAndRunSubquery(
ctx,
planIdx,
subqueryPlan,
planner,
evalCtxFactory(),
subqueryPlans,
recv,
maybeDistribute,
); err != nil {
recv.SetError(err)
// Usually we leave the closure of subqueries to occur when the
// whole plan is being closed (i.e. planTop.close); however, since
// we've encountered an error, we might never get to the point of
// closing the whole plan, so we choose to defensively close the
// subqueries here.
for i := range subqueryPlans {
subqueryPlans[i].plan.Close(ctx)
}
return false
}
}
return true
}
func (dsp *DistSQLPlanner) planAndRunSubquery(
ctx context.Context,
planIdx int,
subqueryPlan subquery,
planner *planner,
evalCtx *extendedEvalContext,
subqueryPlans []subquery,
recv *DistSQLReceiver,
maybeDistribute bool,
) error {
subqueryMonitor := mon.NewMonitor(
"subquery",
mon.MemoryResource,
dsp.distSQLSrv.Metrics.CurBytesCount,
dsp.distSQLSrv.Metrics.MaxBytesHist,
-1, /* use default block size */
noteworthyMemoryUsageBytes,
dsp.distSQLSrv.Settings,
)
subqueryMonitor.Start(ctx, evalCtx.Mon, mon.BoundAccount{})
defer subqueryMonitor.Stop(ctx)
subqueryMemAccount := subqueryMonitor.MakeBoundAccount()
defer subqueryMemAccount.Close(ctx)
var distributeSubquery bool
if maybeDistribute {
distributeSubquery = getPlanDistribution(
ctx, planner, planner.execCfg.NodeID, planner.SessionData().DistSQLMode, subqueryPlan.plan,
).WillDistribute()
}
subqueryPlanCtx := dsp.NewPlanningCtx(ctx, evalCtx, planner, planner.txn, distributeSubquery)
subqueryPlanCtx.stmtType = tree.Rows
if planner.collectBundle {
subqueryPlanCtx.saveDiagram = func(diagram execinfrapb.FlowDiagram) {
planner.curPlan.distSQLDiagrams = append(planner.curPlan.distSQLDiagrams, diagram)
}
}
// Don't close the top-level plan from subqueries - someone else will handle
// that.
subqueryPlanCtx.ignoreClose = true
subqueryPhysPlan, err := dsp.createPhysPlan(subqueryPlanCtx, subqueryPlan.plan)
if err != nil {
return err
}
dsp.FinalizePlan(subqueryPlanCtx, subqueryPhysPlan)
// TODO(arjun): #28264: We set up a row container, wrap it in a row
// receiver, and use it and serialize the results of the subquery. The type
// of the results stored in the container depends on the type of the subquery.
subqueryRecv := recv.clone()
var typ colinfo.ColTypeInfo
var rows *rowcontainer.RowContainer
if subqueryPlan.execMode == rowexec.SubqueryExecModeExists {
subqueryRecv.noColsRequired = true
typ = colinfo.ColTypeInfoFromColTypes([]*types.T{})
} else {
typ = colinfo.ColTypeInfoFromColTypes(subqueryPhysPlan.ResultTypes)
}
rows = rowcontainer.NewRowContainer(subqueryMemAccount, typ)
defer rows.Close(ctx)
subqueryRowReceiver := NewRowResultWriter(rows)
subqueryRecv.resultWriter = subqueryRowReceiver
subqueryPlans[planIdx].started = true
dsp.Run(subqueryPlanCtx, planner.txn, subqueryPhysPlan, subqueryRecv, evalCtx, nil /* finishedSetupFn */)()
if subqueryRecv.commErr != nil {
return subqueryRecv.commErr
}
if err := subqueryRowReceiver.Err(); err != nil {
return err
}
switch subqueryPlan.execMode {
case rowexec.SubqueryExecModeExists:
// For EXISTS expressions, all we want to know if there is at least one row.
hasRows := rows.Len() != 0
subqueryPlans[planIdx].result = tree.MakeDBool(tree.DBool(hasRows))
case rowexec.SubqueryExecModeAllRows, rowexec.SubqueryExecModeAllRowsNormalized:
var result tree.DTuple
for rows.Len() > 0 {
row := rows.At(0)
rows.PopFirst(ctx)
if row.Len() == 1 {
// This seems hokey, but if we don't do this then the subquery expands
// to a tuple of tuples instead of a tuple of values and an expression
// like "k IN (SELECT foo FROM bar)" will fail because we're comparing
// a single value against a tuple.
result.D = append(result.D, row[0])
} else {
result.D = append(result.D, &tree.DTuple{D: row})
}
}
if subqueryPlan.execMode == rowexec.SubqueryExecModeAllRowsNormalized {
result.Normalize(&evalCtx.EvalContext)
}
subqueryPlans[planIdx].result = &result
case rowexec.SubqueryExecModeOneRow:
switch rows.Len() {
case 0:
subqueryPlans[planIdx].result = tree.DNull
case 1:
row := rows.At(0)
switch row.Len() {
case 1:
subqueryPlans[planIdx].result = row[0]
default:
subqueryPlans[planIdx].result = &tree.DTuple{D: rows.At(0)}
}
default:
return pgerror.Newf(pgcode.CardinalityViolation,
"more than one row returned by a subquery used as an expression")
}
default:
return fmt.Errorf("unexpected subqueryExecMode: %d", subqueryPlan.execMode)
}
return nil
}
// PlanAndRun generates a physical plan from a planNode tree and executes it. It
// assumes that the tree is supported (see CheckSupport).
//
// All errors encountered are reported to the DistSQLReceiver's resultWriter.
// Additionally, if the error is a "communication error" (an error encountered
// while using that resultWriter), the error is also stored in
// DistSQLReceiver.commErr. That can be tested to see if a client session needs
// to be closed.
//
// It returns a non-nil (although it can be a noop when an error is
// encountered) cleanup function that must be called once the planTop AST is no
// longer needed and can be closed. Note that this function also cleans up the
// flow which is unfortunate but is caused by the sharing of memory monitors
// between planning and execution - cleaning up the flow wants to close the
// monitor, but it cannot do so because the AST needs to live longer and still
// uses the same monitor. That's why we end up in a situation that in order to
// clean up the flow, we need to close the AST first, but we can only do that
// after PlanAndRun returns.
func (dsp *DistSQLPlanner) PlanAndRun(
ctx context.Context,
evalCtx *extendedEvalContext,
planCtx *PlanningCtx,
txn *kv.Txn,
plan planMaybePhysical,
recv *DistSQLReceiver,
) (cleanup func()) {
log.VEventf(ctx, 1, "creating DistSQL plan with isLocal=%v", planCtx.isLocal)
physPlan, err := dsp.createPhysPlan(planCtx, plan)
if err != nil {
recv.SetError(err)
return func() {}
}
dsp.FinalizePlan(planCtx, physPlan)