forked from cockroachdb/cockroach
-
Notifications
You must be signed in to change notification settings - Fork 0
/
distsql_physical_planner.go
2265 lines (2041 loc) · 71.1 KB
/
distsql_physical_planner.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.
//
// 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 sql
import (
"fmt"
"math"
"sort"
"strings"
"time"
"github.com/cockroachdb/cockroach/pkg/gossip"
"github.com/cockroachdb/cockroach/pkg/internal/client"
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/rpc"
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql/distsqlplan"
"github.com/cockroachdb/cockroach/pkg/sql/distsqlrun"
"github.com/cockroachdb/cockroach/pkg/sql/jobs"
"github.com/cockroachdb/cockroach/pkg/sql/parser"
"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/encoding"
"github.com/cockroachdb/cockroach/pkg/util/envutil"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/stop"
"github.com/pkg/errors"
"golang.org/x/net/context"
)
//
// A rough overview of the process:
//
// - the plan is based on a planNode tree (in the future it will be based on an
// intermediate representation tree). Only a subset of the possible trees is
// supported (this can be checked via CheckSupport).
//
// - we generate a physicalPlan for the planNode tree recursively. The
// physicalPlan consists of a network of processors and streams, with a set
// of unconnected "result routers". The physicalPlan also has information on
// ordering and on the mapping planNode columns to columns in the result
// streams (all result routers output streams with the same schema).
//
// The physicalPlan for a scanNode leaf consists of TableReaders, one for each node
// that has one or more ranges.
//
// - for each an internal planNode we start with the plan of the child node(s)
// and add processing stages (connected to the result routers of the children
// node).
type distSQLPlanner struct {
// planVersion is the version of DistSQL targeted by the plan we're building.
// This is currently only assigned to the node's current DistSQL version and
// is used to skip incompatible nodes when mapping spans.
planVersion distsqlrun.DistSQLVersion
st *cluster.Settings
// The node descriptor for the gateway node that initiated this query.
nodeDesc roachpb.NodeDescriptor
rpcContext *rpc.Context
stopper *stop.Stopper
distSQLSrv *distsqlrun.ServerImpl
spanResolver distsqlplan.SpanResolver
testingKnobs DistSQLPlannerTestingKnobs
// runnerChan is used to send out requests (for running SetupFlow RPCs) to a
// pool of workers.
runnerChan chan runnerRequest
// gossip handle use to check node version compatibility
gossip *gossip.Gossip
}
const resolverPolicy = distsqlplan.BinPackingLeaseHolderChoice
// If true, the plan diagram (in JSON) is logged for each plan (used for
// debugging).
var logPlanDiagram = envutil.EnvOrDefaultBool("COCKROACH_DISTSQL_LOG_PLAN", false)
// If true, for index joins we instantiate a join reader on every node that
// has a stream (usually from a table reader). If false, there is a single join
// reader.
var distributeIndexJoin = settings.RegisterBoolSetting(
"sql.distsql.distribute_index_joins",
"if set, for index joins we instantiate a join reader on every node that has a "+
"stream; if not set, we use a single join reader",
true,
)
var planMergeJoins = settings.RegisterBoolSetting(
"sql.distsql.merge_joins.enabled",
"if set, we plan merge joins when possible",
true,
)
func newDistSQLPlanner(
planVersion distsqlrun.DistSQLVersion,
st *cluster.Settings,
nodeDesc roachpb.NodeDescriptor,
rpcCtx *rpc.Context,
distSQLSrv *distsqlrun.ServerImpl,
distSender *kv.DistSender,
gossip *gossip.Gossip,
stopper *stop.Stopper,
testingKnobs DistSQLPlannerTestingKnobs,
) *distSQLPlanner {
dsp := &distSQLPlanner{
planVersion: planVersion,
st: st,
nodeDesc: nodeDesc,
rpcContext: rpcCtx,
stopper: stopper,
distSQLSrv: distSQLSrv,
gossip: gossip,
spanResolver: distsqlplan.NewSpanResolver(distSender, gossip, nodeDesc, resolverPolicy),
testingKnobs: testingKnobs,
}
dsp.initRunners()
return dsp
}
// setSpanResolver switches to a different SpanResolver. It is the caller's
// responsibility to make sure the distSQLPlanner is not in use.
func (dsp *distSQLPlanner) setSpanResolver(spanResolver distsqlplan.SpanResolver) {
dsp.spanResolver = spanResolver
}
// distSQLExprCheckVisitor is a parser.Visitor that checks if expressions
// contain things not supported by distSQL (like subqueries).
type distSQLExprCheckVisitor struct {
err error
}
var _ parser.Visitor = &distSQLExprCheckVisitor{}
func (v *distSQLExprCheckVisitor) VisitPre(expr parser.Expr) (recurse bool, newExpr parser.Expr) {
if v.err != nil {
return false, expr
}
switch t := expr.(type) {
case *subquery, *parser.Subquery:
v.err = newQueryNotSupportedError("subqueries not supported yet")
return false, expr
case *parser.FuncExpr:
if t.IsDistSQLBlacklist() {
v.err = newQueryNotSupportedErrorf("function %s cannot be executed with distsql", t)
return false, expr
}
}
return true, expr
}
func (v *distSQLExprCheckVisitor) VisitPost(expr parser.Expr) parser.Expr { return expr }
// checkExpr verifies that an expression doesn't contain things that are not yet
// supported by distSQL, like subqueries.
func (dsp *distSQLPlanner) checkExpr(expr parser.Expr) error {
if expr == nil {
return nil
}
v := distSQLExprCheckVisitor{}
parser.WalkExprConst(&v, expr)
return v.err
}
// CheckSupport looks at a planNode tree and decides:
// - whether DistSQL is equipped to handle the query (if not, an error is
// returned).
// - whether it is recommended that the query be run with DistSQL.
func (dsp *distSQLPlanner) CheckSupport(node planNode) (bool, error) {
rec, err := dsp.checkSupportForNode(node)
if err != nil {
return false, err
}
return (rec == shouldDistribute), nil
}
type distRecommendation int
const (
// shouldNotDistribute indicates that a plan could suffer if run
// under DistSQL
shouldNotDistribute distRecommendation = iota
// canDistribute indicates that a plan will probably not benefit but will
// probably not suffer if run under DistSQL.
canDistribute
// shouldDistribute indicates that a plan will likely benefit if run under
// DistSQL.
shouldDistribute
)
// compose returns the recommendation for a plan given recommendations for two
// parts of it: if we shouldNotDistribute either part, then we
// shouldNotDistribute the overall plan either.
func (a distRecommendation) compose(b distRecommendation) distRecommendation {
if a == shouldNotDistribute || b == shouldNotDistribute {
return shouldNotDistribute
}
if a == shouldDistribute || b == shouldDistribute {
return shouldDistribute
}
return canDistribute
}
type queryNotSupportedError struct {
msg string
}
func (e *queryNotSupportedError) Error() string {
return e.msg
}
func newQueryNotSupportedError(msg string) error {
return &queryNotSupportedError{msg: msg}
}
func newQueryNotSupportedErrorf(format string, args ...interface{}) error {
return &queryNotSupportedError{msg: fmt.Sprintf(format, args...)}
}
var mutationsNotSupportedError = newQueryNotSupportedError("mutations not supported")
var setNotSupportedError = newQueryNotSupportedError("SET / SET CLUSTER SETTING should never distribute")
// leafType returns the element type if the given type is an array, and the type
// itself otherwise.
func leafType(t parser.Type) parser.Type {
if a, ok := t.(parser.TArray); ok {
return leafType(a.Typ)
}
return t
}
// checkSupportForNode returns a distRecommendation (as described above) or an
// error if the plan subtree is not supported by DistSQL.
// TODO(radu): add tests for this.
func (dsp *distSQLPlanner) checkSupportForNode(node planNode) (distRecommendation, error) {
switch n := node.(type) {
case *filterNode:
if err := dsp.checkExpr(n.filter); err != nil {
return 0, err
}
return dsp.checkSupportForNode(n.source.plan)
case *renderNode:
for i, e := range n.render {
typ := n.columns[i].Typ
if leafType(typ).FamilyEqual(parser.TypeTuple) {
return 0, newQueryNotSupportedErrorf("unsupported render type %s", typ)
}
if err := dsp.checkExpr(e); err != nil {
return 0, err
}
}
return dsp.checkSupportForNode(n.source.plan)
case *sortNode:
rec, err := dsp.checkSupportForNode(n.plan)
if err != nil {
return 0, err
}
// If we have to sort, distribute the query.
if n.needSort {
rec = rec.compose(shouldDistribute)
}
return rec, nil
case *joinNode:
if err := dsp.checkExpr(n.pred.onCond); err != nil {
return 0, err
}
recLeft, err := dsp.checkSupportForNode(n.left.plan)
if err != nil {
return 0, err
}
recRight, err := dsp.checkSupportForNode(n.right.plan)
if err != nil {
return 0, err
}
// If either the left or the right side can benefit from distribution, we
// should distribute.
rec := recLeft.compose(recRight)
// If we can do a hash join, we distribute if possible.
if len(n.pred.leftEqualityIndices) > 0 {
rec = rec.compose(shouldDistribute)
}
return rec, nil
case *scanNode:
rec := canDistribute
if n.hardLimit != 0 || n.softLimit != 0 {
// We don't yet recommend distributing plans where limits propagate
// to scan nodes; we don't have infrastructure to only plan for a few
// ranges at a time.
rec = shouldNotDistribute
}
// We recommend running scans distributed if we have a filtering
// expression or if we have a full table scan.
if n.filter != nil {
if err := dsp.checkExpr(n.filter); err != nil {
return 0, err
}
rec = rec.compose(shouldDistribute)
}
// Check if we are doing a full scan.
if len(n.spans) == 1 && n.spans[0].EqualValue(n.desc.IndexSpan(n.index.ID)) {
rec = rec.compose(shouldDistribute)
}
return rec, nil
case *indexJoinNode:
// n.table doesn't have meaningful spans, but we need to check support (e.g.
// for any filtering expression).
if _, err := dsp.checkSupportForNode(n.table); err != nil {
return 0, err
}
return dsp.checkSupportForNode(n.index)
case *groupNode:
for _, fholder := range n.funcs {
if f, ok := fholder.expr.(*parser.FuncExpr); ok {
if strings.ToUpper(f.Func.FunctionReference.String()) == "ARRAY_AGG" {
return 0, newQueryNotSupportedError("ARRAY_AGG aggregation not supported yet")
}
}
}
rec, err := dsp.checkSupportForNode(n.plan)
if err != nil {
return 0, err
}
// Distribute aggregations if possible.
return rec.compose(shouldDistribute), nil
case *limitNode:
if err := dsp.checkExpr(n.countExpr); err != nil {
return 0, err
}
if err := dsp.checkExpr(n.offsetExpr); err != nil {
return 0, err
}
return dsp.checkSupportForNode(n.plan)
case *distinctNode:
return dsp.checkSupportForNode(n.plan)
case *valuesNode:
if n.n == nil {
return 0, newQueryNotSupportedErrorf("unsupported node %T without SQL VALUES clause", node)
}
for _, tuple := range n.tuples {
for _, expr := range tuple {
if err := dsp.checkExpr(expr); err != nil {
return 0, err
}
}
}
return shouldDistribute, nil
case *insertNode, *updateNode, *deleteNode:
// This is a potential hot path.
return 0, mutationsNotSupportedError
case *setNode, *setClusterSettingNode:
// SET statements are never distributed.
return 0, setNotSupportedError
default:
return 0, newQueryNotSupportedErrorf("unsupported node %T", node)
}
}
// planningCtx contains data used and updated throughout the planning process of
// a single query.
type planningCtx struct {
ctx context.Context
spanIter distsqlplan.SpanResolverIterator
// nodeAddresses contains addresses for all NodeIDs that are referenced by any
// physicalPlan we generate with this context.
// Nodes that fail a health check have empty addresses.
nodeAddresses map[roachpb.NodeID]string
}
// sanityCheckAddresses returns an error if the same address is used by two
// nodes.
func (p *planningCtx) sanityCheckAddresses() error {
inverted := make(map[string]roachpb.NodeID)
for nodeID, addr := range p.nodeAddresses {
if otherNodeID, ok := inverted[addr]; ok {
return util.UnexpectedWithIssueErrorf(
12876,
"different nodes %d and %d with the same address '%s'", nodeID, otherNodeID, addr)
}
inverted[addr] = nodeID
}
return nil
}
// physicalPlan is a partial physical plan which corresponds to a planNode
// (partial in that it can correspond to a planNode subtree and not necessarily
// to the entire planNode for a given query).
//
// It augments distsqlplan.PhysicalPlan with information relating the physical
// plan to a planNode subtree.
//
// These plans are built recursively on a planNode tree.
type physicalPlan struct {
distsqlplan.PhysicalPlan
// planToStreamColMap maps planNode Columns() to columns in the result
// streams. Note that in some cases, not all columns in the result streams
// are referenced in the map (this is due to some processors not being
// configurable to output only certain columns and will be fixed).
//
// Conversely, in some cases not all planNode columns have a corresponding
// result stream column (these map to index -1); this is the case for scanNode
// and indexJoinNode where not all columns in the table are actually used in
// the plan.
planToStreamColMap []int
}
// orderingTerminated is used when streams can be joined without needing to be
// merged with respect to a particular ordering.
var orderingTerminated = distsqlrun.Ordering{}
// makePlanToStreamColMap initializes a new physicalPlan.planToStreamColMap. The
// columns that are present in the result stream(s) should be set in the map.
func makePlanToStreamColMap(numCols int) []int {
m := make([]int, numCols)
for i := 0; i < numCols; i++ {
m[i] = -1
}
return m
}
// indetityMap returns the slice {0, 1, 2, ..., numCols-1}.
// buf can be optionally provided as a buffer.
func identityMap(buf []int, numCols int) []int {
buf = buf[:0]
for i := 0; i < numCols; i++ {
buf = append(buf, i)
}
return buf
}
// spanPartition is the intersection between a set of spans for a certain
// operation (e.g table scan) and the set of ranges owned by a given node.
type spanPartition struct {
node roachpb.NodeID
spans roachpb.Spans
}
// partitionSpans finds out which nodes are owners for ranges touching the given
// spans, and splits the spans according to owning nodes. The result is a set of
// spanPartitions (one for each relevant node), which form a partitioning of the
// spans (i.e. they are non-overlapping and their union is exactly the original
// set of spans).
//
// partitionSpans does its best to not assign ranges on nodes that are known to
// either be unhealthy or running an incompatible version. The ranges owned by
// such nodes are assigned to the gateway.
func (dsp *distSQLPlanner) partitionSpans(
planCtx *planningCtx, spans roachpb.Spans,
) ([]spanPartition, error) {
if len(spans) == 0 {
panic("no spans")
}
ctx := planCtx.ctx
partitions := make([]spanPartition, 0, 1)
// nodeMap maps a nodeID to an index inside the partitions array.
nodeMap := make(map[roachpb.NodeID]int)
// nodeVerCompatMap maintains info about which nodes advertise DistSQL
// versions compatible with this plan and which ones don't.
nodeVerCompatMap := make(map[roachpb.NodeID]bool)
it := planCtx.spanIter
for _, span := range spans {
var rspan roachpb.RSpan
var err error
if rspan.Key, err = keys.Addr(span.Key); err != nil {
return nil, err
}
if rspan.EndKey, err = keys.Addr(span.EndKey); err != nil {
return nil, err
}
var lastNodeID roachpb.NodeID
// lastKey maintains the EndKey of the last piece of `span`.
lastKey := rspan.Key
if log.V(1) {
log.Infof(ctx, "partitioning span %s", span)
}
for it.Seek(ctx, span, kv.Ascending); ; it.Next(ctx) {
if !it.Valid() {
return nil, it.Error()
}
replInfo, err := it.ReplicaInfo(ctx)
if err != nil {
return nil, err
}
desc := it.Desc()
if log.V(1) {
log.Infof(ctx, "lastKey: %s desc: %s", lastKey, desc)
}
if !desc.ContainsKey(lastKey) {
// This range must contain the last range's EndKey.
log.Fatalf(
ctx, "next range %v doesn't cover last end key %v. Partitions: %#v",
desc.RSpan(), lastKey, partitions,
)
}
// Limit the end key to the end of the span we are resolving.
endKey := desc.EndKey
if rspan.EndKey.Less(endKey) {
endKey = rspan.EndKey
}
nodeID := replInfo.NodeDesc.NodeID
partitionIdx, inNodeMap := nodeMap[nodeID]
if !inNodeMap {
// This is the first time we are seeing nodeID for these spans. Check
// its health.
addr, inAddrMap := planCtx.nodeAddresses[nodeID]
if !inAddrMap {
addr = replInfo.NodeDesc.Address.String()
checkNodeHealth := func() error {
// Check if the node is still in gossip - i.e. if it hasn't been
// decommissioned or overridden by another node at the same address.
if _, err := dsp.gossip.GetNodeIDAddress(nodeID); err != nil {
log.VEventf(ctx, 1, "not using n%d because gossip doesn't know about it. "+
"It might have gone away from the cluster. Gossip said: %s.", nodeID, err)
return err
}
var err error
if dsp.testingKnobs.OverrideHealthCheck != nil {
err = dsp.testingKnobs.OverrideHealthCheck(replInfo.NodeDesc.NodeID, addr)
} else {
err = dsp.rpcContext.ConnHealth(addr)
}
if err != nil && err != rpc.ErrNotConnected && err != rpc.ErrNotHeartbeated {
// This host is known to be unhealthy. Don't use it (use the gateway
// instead). Note: this can never happen for our nodeID (which
// always has its address in the nodeMap).
log.VEventf(ctx, 1, "marking n%d as unhealthy for this plan: %v", nodeID, err)
return err
}
return nil
}
if err := checkNodeHealth(); err != nil {
addr = ""
}
if err == nil && addr != "" {
planCtx.nodeAddresses[nodeID] = addr
}
}
compat := true
if addr != "" {
// Check if the node's DistSQL version is compatible with this plan.
// If it isn't, we'll use the gateway.
var ok bool
if compat, ok = nodeVerCompatMap[nodeID]; !ok {
compat = dsp.nodeVersionIsCompatible(nodeID, dsp.planVersion)
nodeVerCompatMap[nodeID] = compat
}
}
// If the node is unhealthy or its DistSQL version is incompatible, use
// the gateway to process this span instead of the unhealthy host.
// An empty address indicates an unhealthy host.
if addr == "" || !compat {
log.Eventf(ctx, "not planning on node %d. unhealthy: %t, incompatible version: %t",
nodeID, addr == "", !compat)
nodeID = dsp.nodeDesc.NodeID
partitionIdx, inNodeMap = nodeMap[nodeID]
}
if !inNodeMap {
partitionIdx = len(partitions)
partitions = append(partitions, spanPartition{node: nodeID})
nodeMap[nodeID] = partitionIdx
}
}
partition := &partitions[partitionIdx]
if lastNodeID == nodeID {
// Two consecutive ranges on the same node, merge the spans.
partition.spans[len(partition.spans)-1].EndKey = endKey.AsRawKey()
} else {
partition.spans = append(partition.spans, roachpb.Span{
Key: lastKey.AsRawKey(),
EndKey: endKey.AsRawKey(),
})
}
if !endKey.Less(rspan.EndKey) {
// Done.
break
}
lastKey = endKey
lastNodeID = nodeID
}
}
return partitions, nil
}
// nodeVersionIsCompatible decides whether a particular node's DistSQL version
// is compatible with planVer. It uses gossip to find out the node's version
// range.
func (dsp *distSQLPlanner) nodeVersionIsCompatible(
nodeID roachpb.NodeID, planVer distsqlrun.DistSQLVersion,
) bool {
var v distsqlrun.DistSQLVersionGossipInfo
if err := dsp.gossip.GetInfoProto(gossip.MakeDistSQLNodeVersionKey(nodeID), &v); err != nil {
return false
}
return distsqlrun.FlowVerIsCompatible(dsp.planVersion, v.MinAcceptedVersion, v.Version)
}
// initTableReaderSpec initializes a TableReaderSpec/PostProcessSpec that
// corresponds to a scanNode, except for the Spans and OutputColumns.
func initTableReaderSpec(
n *scanNode,
) (distsqlrun.TableReaderSpec, distsqlrun.PostProcessSpec, error) {
s := distsqlrun.TableReaderSpec{
Table: *n.desc,
Reverse: n.reverse,
}
if n.index != &n.desc.PrimaryIndex {
for i := range n.desc.Indexes {
if n.index == &n.desc.Indexes[i] {
// IndexIdx is 1 based (0 means primary index).
s.IndexIdx = uint32(i + 1)
break
}
}
if s.IndexIdx == 0 {
err := errors.Errorf("invalid scanNode index %v (table %s)", n.index, n.desc.Name)
return distsqlrun.TableReaderSpec{}, distsqlrun.PostProcessSpec{}, err
}
}
post := distsqlrun.PostProcessSpec{
Filter: distsqlplan.MakeExpression(n.filter, nil),
}
if n.hardLimit != 0 {
post.Limit = uint64(n.hardLimit)
} else if n.softLimit != 0 {
s.LimitHint = n.softLimit
}
return s, post, nil
}
// getOutputColumnsFromScanNode returns the indices of the columns that are
// returned by a scanNode.
func getOutputColumnsFromScanNode(n *scanNode) []uint32 {
num := 0
for i := range n.resultColumns {
if n.valNeededForCol[i] {
num++
}
}
outputColumns := make([]uint32, 0, num)
for i := range n.resultColumns {
// TODO(radu): if we have a scan with a filter, valNeededForCol will include
// the columns needed for the filter, even if they aren't needed for the
// next stage.
if n.valNeededForCol[i] {
outputColumns = append(outputColumns, uint32(i))
}
}
return outputColumns
}
// convert ordering takes sql.orderingInfo
func (dsp *distSQLPlanner) convertOrdering(
planOrdering []orderingColumnGroup, planToStreamColMap []int,
) distsqlrun.Ordering {
if len(planOrdering) == 0 {
return distsqlrun.Ordering{}
}
ordering := distsqlrun.Ordering{
Columns: make([]distsqlrun.Ordering_Column, 0, len(planOrdering)),
}
for _, group := range planOrdering {
// Try to find any column in the group that is part of the processor output.
streamColIdx := -1
for col, ok := group.cols.Next(0); ok; col, ok = group.cols.Next(col + 1) {
streamColIdx = planToStreamColMap[col]
if streamColIdx != -1 {
break
}
}
if streamColIdx == -1 {
panic("column in ordering not part of processor output")
}
oc := distsqlrun.Ordering_Column{
ColIdx: uint32(streamColIdx),
Direction: distsqlrun.Ordering_Column_ASC,
}
if group.dir == encoding.Descending {
oc.Direction = distsqlrun.Ordering_Column_DESC
}
ordering.Columns = append(ordering.Columns, oc)
}
return ordering
}
// createTableReaders generates a plan consisting of table reader processors,
// one for each node that has spans that we are reading.
// overridesResultColumns is optional.
func (dsp *distSQLPlanner) createTableReaders(
planCtx *planningCtx, n *scanNode, overrideResultColumns []uint32,
) (physicalPlan, error) {
spec, post, err := initTableReaderSpec(n)
if err != nil {
return physicalPlan{}, err
}
spanPartitions, err := dsp.partitionSpans(planCtx, n.spans)
if err != nil {
return physicalPlan{}, err
}
var p physicalPlan
stageID := p.NewStageID()
for _, sp := range spanPartitions {
tr := &distsqlrun.TableReaderSpec{}
*tr = spec
tr.Spans = make([]distsqlrun.TableReaderSpan, len(sp.spans))
for i := range sp.spans {
tr.Spans[i].Span = sp.spans[i]
}
proc := distsqlplan.Processor{
Node: sp.node,
Spec: distsqlrun.ProcessorSpec{
Core: distsqlrun.ProcessorCoreUnion{TableReader: tr},
Output: []distsqlrun.OutputRouterSpec{{Type: distsqlrun.OutputRouterSpec_PASS_THROUGH}},
StageID: stageID,
},
}
pIdx := p.AddProcessor(proc)
p.ResultRouters = append(p.ResultRouters, pIdx)
}
planToStreamColMap := make([]int, len(n.resultColumns))
for i := range planToStreamColMap {
planToStreamColMap[i] = i
}
if len(p.ResultRouters) > 1 && len(n.ordering.ordering) > 0 {
// Make a note of the fact that we have to maintain a certain ordering
// between the parallel streams.
//
// This information is taken into account by the AddProjection call below:
// specifically, it will make sure these columns are kept even if they are
// not in the projection (e.g. "SELECT v FROM kv ORDER BY k").
p.SetMergeOrdering(dsp.convertOrdering(n.ordering.ordering, planToStreamColMap))
}
p.SetLastStagePost(post, getTypesForPlanResult(n, planToStreamColMap))
outCols := overrideResultColumns
if outCols == nil {
outCols = getOutputColumnsFromScanNode(n)
}
p.AddProjection(outCols)
post = p.GetLastStagePost()
for i := range planToStreamColMap {
planToStreamColMap[i] = -1
}
for i, col := range post.OutputColumns {
planToStreamColMap[col] = i
}
p.planToStreamColMap = planToStreamColMap
return p, nil
}
func initBackfillerSpec(
backfillType backfillType,
desc sqlbase.TableDescriptor,
duration time.Duration,
chunkSize int64,
otherTables []sqlbase.TableDescriptor,
readAsOf hlc.Timestamp,
) (distsqlrun.BackfillerSpec, error) {
ret := distsqlrun.BackfillerSpec{
Table: desc,
Duration: duration,
ChunkSize: chunkSize,
OtherTables: otherTables,
ReadAsOf: readAsOf,
}
switch backfillType {
case indexBackfill:
ret.Type = distsqlrun.BackfillerSpec_Index
case columnBackfill:
ret.Type = distsqlrun.BackfillerSpec_Column
default:
return distsqlrun.BackfillerSpec{}, errors.Errorf("bad backfill type %d", backfillType)
}
return ret, nil
}
// CreateBackfiller generates a plan consisting of index/column backfiller
// processors, one for each node that has spans that we are reading. The plan is
// finalized.
func (dsp *distSQLPlanner) CreateBackfiller(
planCtx *planningCtx,
backfillType backfillType,
desc sqlbase.TableDescriptor,
duration time.Duration,
chunkSize int64,
spans []roachpb.Span,
otherTables []sqlbase.TableDescriptor,
readAsOf hlc.Timestamp,
) (physicalPlan, error) {
spec, err := initBackfillerSpec(backfillType, desc, duration, chunkSize, otherTables, readAsOf)
if err != nil {
return physicalPlan{}, err
}
spanPartitions, err := dsp.partitionSpans(planCtx, spans)
if err != nil {
return physicalPlan{}, err
}
p := physicalPlan{}
for _, sp := range spanPartitions {
ib := &distsqlrun.BackfillerSpec{}
*ib = spec
ib.Spans = make([]distsqlrun.TableReaderSpan, len(sp.spans))
for i := range sp.spans {
ib.Spans[i].Span = sp.spans[i]
}
proc := distsqlplan.Processor{
Node: sp.node,
Spec: distsqlrun.ProcessorSpec{
Core: distsqlrun.ProcessorCoreUnion{Backfiller: ib},
Output: []distsqlrun.OutputRouterSpec{{Type: distsqlrun.OutputRouterSpec_PASS_THROUGH}},
},
}
pIdx := p.AddProcessor(proc)
p.ResultRouters = append(p.ResultRouters, pIdx)
}
dsp.FinalizePlan(planCtx, &p)
return p, nil
}
// DistLoader uses DistSQL to convert external data formats (csv, etc) into
// sstables of our mvcc-format key values.
type DistLoader struct {
distSQLPlanner *distSQLPlanner
}
// RowResultWriter is a thin wrapper around a RowContainer.
type RowResultWriter struct {
statementType parser.StatementType
rowContainer *sqlbase.RowContainer
rowsAffected int
}
// NewRowResultWriter creates a new RowResultWriter.
func NewRowResultWriter(
statementType parser.StatementType, rowContainer *sqlbase.RowContainer,
) *RowResultWriter {
return &RowResultWriter{statementType: statementType, rowContainer: rowContainer}
}
// StatementType implements the rowResultWriter interface.
func (b *RowResultWriter) StatementType() parser.StatementType {
return b.statementType
}
// IncrementRowsAffected implements the rowResultWriter interface.
func (b *RowResultWriter) IncrementRowsAffected(n int) {
b.rowsAffected += n
}
// AddRow implements the rowResultWriter interface.
func (b *RowResultWriter) AddRow(ctx context.Context, row parser.Datums) error {
_, err := b.rowContainer.AddRow(ctx, row)
return err
}
// LoadCSV performs a distributed transformation of the CSV files at from
// and stores them in enterprise backup format at to.
func (l *DistLoader) LoadCSV(
ctx context.Context,
job *jobs.Job,
db *client.DB,
evalCtx parser.EvalContext,
thisNode roachpb.NodeID,
nodes []roachpb.NodeDescriptor,
resultRows *RowResultWriter,
tableDesc *sqlbase.TableDescriptor,
from []string,
to string,
comma, comment rune,
nullif *string,
walltime int64,
splitSize int64,
) error {
// splitSize is the target number of bytes at which to create SST files. We
// attempt to do this by sampling, which is what the first DistSQL plan of this
// function does. CSV rows are converted into KVs. The total size of the KV is
// used to determine if we should sample it or not. For example, if we had a
// 100 byte KV and a 30MB splitSize, we would sample the KV with probability
// 100/30000000. Over many KVs, this produces samples at approximately the
// correct spacing, but obviously also with some error. We use oversample
// below to decrease the error. We divide the splitSize by oversample to
// produce the actual sampling rate. So in the example above, oversampling by a
// factor of 3 would sample the KV with probability 100/10000000 since we are
// sampling at 3x. Since we're now getting back 3x more samples than needed,
// we only use every 1/(oversample), or 1/3 here, in our final sampling.
const oversample = 3
sampleSize := splitSize / oversample
if sampleSize > math.MaxInt32 {
return errors.Errorf("SST size must fit in an int32: %d", splitSize)
}
var p physicalPlan
colTypeBytes := sqlbase.ColumnType{SemanticType: sqlbase.ColumnType_BYTES}
stageID := p.NewStageID()
// Stage 1: for each input file, assign it to a node
for i, input := range from {
// TODO(mjibson): attempt to intelligently schedule http files to matching cockroach nodes
rcs := distsqlrun.ReadCSVSpec{
SampleSize: int32(sampleSize),
TableDesc: *tableDesc,
Uri: input,
Options: roachpb.CSVOptions{
Comma: comma,
Comment: comment,
Nullif: nullif,
},
}
node := nodes[i%len(nodes)]
proc := distsqlplan.Processor{
Node: node.NodeID,
Spec: distsqlrun.ProcessorSpec{
Core: distsqlrun.ProcessorCoreUnion{ReadCSV: &rcs},
Output: []distsqlrun.OutputRouterSpec{{Type: distsqlrun.OutputRouterSpec_PASS_THROUGH}},
StageID: stageID,
},
}
pIdx := p.AddProcessor(proc)
p.ResultRouters = append(p.ResultRouters, pIdx)
}
// We only need the key during sorting.
p.planToStreamColMap = []int{0}
p.ResultTypes = []sqlbase.ColumnType{colTypeBytes, colTypeBytes}
kvOrdering := distsqlrun.Ordering{
Columns: []distsqlrun.Ordering_Column{{
ColIdx: 0,
Direction: distsqlrun.Ordering_Column_ASC,
}},
}
sorterSpec := distsqlrun.SorterSpec{
OutputOrdering: kvOrdering,
}
p.AddSingleGroupStage(thisNode,
distsqlrun.ProcessorCoreUnion{Sorter: &sorterSpec},
distsqlrun.PostProcessSpec{},
[]sqlbase.ColumnType{colTypeBytes},
)
ci := sqlbase.ColTypeInfoFromColTypes([]sqlbase.ColumnType{colTypeBytes})
rowContainer := sqlbase.NewRowContainer(*evalCtx.ActiveMemAcc, ci, 0)
rowResultWriter := NewRowResultWriter(parser.Rows, rowContainer)