-
Notifications
You must be signed in to change notification settings - Fork 3.9k
/
Copy pathjoin_order_builder.go
1836 lines (1670 loc) · 69.9 KB
/
join_order_builder.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 2020 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 xform
import (
"math"
"math/bits"
"github.com/cockroachdb/cockroach/pkg/sql/opt"
"github.com/cockroachdb/cockroach/pkg/sql/opt/memo"
"github.com/cockroachdb/cockroach/pkg/sql/opt/norm"
"github.com/cockroachdb/cockroach/pkg/sql/opt/props"
"github.com/cockroachdb/cockroach/pkg/sql/sem/eval"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/errors"
)
// OnReorderFunc defines the callback function for the NotifyOnReorder
// event supported by the optimizer and factory. OnReorderFunc is called once
// the join graph has been assembled. The callback parameters give the root join
// as well as the vertexes and edges of the join graph.
type OnReorderFunc func(
join memo.RelExpr,
vertexes []memo.RelExpr,
edges []OnReorderEdgeParam,
)
// OnReorderEdgeParam is a struct representing an edge in the join graph. This
// type is only used for the OnReorderFunc during testing and debugging. See the
// more efficient edge type that is used in the join reordering algorithm.
type OnReorderEdgeParam struct {
// Op is the original join operator from which the edge was constructed.
Op opt.Operator
// Filters is the edge's set of join filters
Filters memo.FiltersExpr
// SES is the edge's syntactic eligibility set.
SES []memo.RelExpr
// TES is the edge's total eligibility set.
TES []memo.RelExpr
// Rules is the set of conflict rules of the edge.
Rules []OnReorderRuleParam
}
// OnReorderRuleParam is a struct representing a conflict rule. This type is
// only used for the OnReorderFunc during testing and debugging. See the more
// efficient conflictRule type that is used in the join reordering algorithm.
type OnReorderRuleParam struct {
From, To []memo.RelExpr
}
// OnAddJoinFunc defines the callback function for the NotifyOnAddJoin event
// supported by JoinOrderBuilder. OnAddJoinFunc is called when JoinOrderBuilder
// attempts to add a join to the memo via addJoin. The callback parameters give
// the base relations of the left and right inputs of the join, the set of all
// base relations currently being considered, the base relations referenced by
// the join's ON condition, and the type of join.
type OnAddJoinFunc func(left, right, all, joinRefs, selectRefs []memo.RelExpr, op opt.Operator)
// JoinOrderBuilder is used to add valid orderings of a given join tree to the
// memo during exploration.
//
// Motivation
// ----------
//
// For any given join tree, it is possible to enumerate all possible orderings
// through exhaustive application of transformation rules. However, enumerating
// join orderings this way leads to a large number of duplicates, which in turn
// significantly impacts planning time. Ideally, each unique join ordering would
// only be enumerated once.
//
// In addition, in the vast majority of cases, the optimal plan for a query does
// not involve introducing cross joins that were not present in the original
// version of the plan. This is because cross joins produce |m| x |n| output
// tuples, where |m| is left input cardinality and |n| is right input
// cardinality. With a query like this:
//
// SELECT *
// FROM (SELECT * FROM xy INNER JOIN ab ON x = a)
// INNER JOIN uv ON x = u
//
// An ordering like the following is valid but not desirable, since the cross
// join will likely be very expensive compared to a join with a predicate:
//
// SELECT *
// FROM (SELECT * FROM uv INNER JOIN ab ON True)
// INNER JOIN xy ON x = a AND x = u
//
// Avoiding cross joins significantly decreases the search space (and therefore
// planning time) without preventing the best plan from being found in most
// cases.
//
// Finally, join trees that incorporate non-inner joins should be handled as
// well as inner join trees. This is non-trivial, because non-inner joins are
// much more restrictive than inner joins. For example, the associative property
// does not apply to an inner join in the right input of a left join. Any join
// enumeration method must take care not to introduce invalid orderings.
//
// How JoinOrderBuilder works
// --------------------------
//
// First, the initial join tree is traversed and encoded as a hypergraph (which
// is a graph in which any edge can reference two or more vertexes). The
// 'leaves' of the join tree (base relations) become the vertexes of the graph,
// and the edges are built using the ON conditions of the joins.
//
// Taking this query as an example:
//
// SELECT *
// FROM (SELECT * FROM xy LEFT JOIN ab ON x = a)
// INNER JOIN uv ON x = u AND (y = b OR b IS NULL)
//
// The vertexes of the graph would represent the base relations xy, ab and uv.
// The three edges would be:
//
// x = a [left]
// x = u [inner]
// y = b OR b IS NULL [inner]
//
// Then, the DPSube algorithm is executed (see citations: [8]). DPSube
// enumerates all disjoint pairs of subsets of base relations such as
// ({xy}, {uv}) or ({xy, uv}, {cd}). This is done efficiently using bit sets.
// For each pair of relation subsets, the edges are iterated over. For each
// edge, if certain edge conditions (to be described in the next section) are
// satisfied, a new join is created using the filters from that edge and the
// relation subsets as inputs.
//
// Avoiding invalid orderings
// --------------------------
//
// Earlier, it was mentioned that edges are restricted in their ability to form
// new joins. How is this accomplished?
//
// The paper 'On the correct and complete enumeration of the core search space'
// introduces the concept of a total eligibility set (TES) (citations: [8]). The
// TES is an extension of the syntactic eligibility set (SES). The SES models
// the producer-consumer relationship of joins and base relations; any relations
// contained in the SES of a join must be present in the join's input. For
// example, take the following query:
//
// SELECT *
// FROM xy
// LEFT JOIN (SELECT * FROM ab INNER JOIN uv ON a = u)
// ON x = u
//
// The SES for the left join will contain relations xy and uv because both are
// referenced by the join's predicate. Therefore, both must be in the input of
// this join for any ordering to be valid.
//
// The TES of an edge is initialized with the SES, and then expanded during
// execution of the CD-C algorithm (see citations: [8] section 5.4). For each
// 'child' join under the current join, associative, left-asscom and
// right-asscom properties are provided by lookup tables (the asscom properties
// are derived from a combination of association and commutation). Depending on
// these properties, the TES will be expanded to take into account whether
// certain transformations of the join tree are valid. During execution of the
// DPSube algorithm, the TES is used to decide whether a given edge can be used
// to construct a new join operator.
//
// Consider the following (invalid) reordering of the above example):
//
// SELECT *
// FROM ab
// INNER JOIN (SELECT * FROM xy LEFT JOIN uv ON x = u)
// ON a = u
//
// The left join's TES will include relations xy and uv because they are in the
// SES. The TES will also contain ab because the right-asscom property does not
// hold for a left join and an inner join. Violation of the right-asscom
// property in this context means that the xy and ab relations cannot switch
// places. Because all relations in the join's TES must be a part of its inputs,
// ab cannot be pulled out of the left join. This prevents the invalid plan from
// being considered.
//
// In addition to the TES, 'conflict rules' are also required to detect invalid
// plans. For details, see the methods: calculateTES, addJoins,
// checkNonInnerJoin, and checkInnerJoin.
//
// Special handling of inner joins
// -------------------------------
//
// In general, a join's ON condition must be treated as a single entity, because
// join filter conjuncts cannot usually be pulled out of (or pushed down from)
// the ON condition. However, this restriction can be relaxed for inner joins
// because inner join trees have a unique property: they can be modeled as a
// series of cross joins followed by a series of selects with the inner join
// conjuncts. This allows inner join conjuncts to be treated as 'detached' from
// their original operator, free to be combined with conjuncts from other inner
// joins. For example, take this query:
//
// SELECT *
// FROM (SELECT * FROM xy INNER JOIN ab ON x = a)
// INNER JOIN uv ON x = u AND a = u
//
// Treating the ON conditions of these joins as a conglomerate (as we do with
// non-inner joins), a join between base relations xy and uv would not be
// possible, because the a = u conjunct requires that the ab base relation also
// be under that edge. However, creating separate edges for each inner join
// conjunct solves this problem, allowing a reordering like the following
// (the ab and uv relations are switched, along with the filters):
//
// SELECT *
// FROM (SELECT * FROM xy INNER JOIN uv ON x = u)
// INNER JOIN ab ON x = a AND a = u
//
// In fact, this idea can be taken even further. Take this query as an example:
//
// SELECT *
// FROM xy
// INNER JOIN (SELECT * FROM ab LEFT JOIN uv ON b = v)
// ON x = a AND (y = u OR u IS NULL)
//
// The following is a valid reformulation:
//
// SELECT *
// FROM (SELECT * FROM xy INNER JOIN ab ON x = a)
// LEFT JOIN uv ON b = v
// WHERE y = u OR u IS NULL
//
// Notice the new Select operation that now carries the inner join conjunct that
// references the right side of the left join. We can model the process that
// leads to this reformulation as follows:
// 1. The inner join is rewritten as a cross join and two selects, each
// carrying a conjunct: (x = a) for one and (y = u OR u IS NULL) for the
// other.
// 2. The Select operators are pulled above the inner join.
// 3. The left join and inner join are reordered according to the associative
// property (see citations: [8] table 2).
// 4. Finally, the inner join conjuncts are pushed back down the reordered
// join tree as far as possible. The x = a conjunct can be pushed to the
// inner join, but the (y = u OR u IS NULL) conjunct must remain on the
// Select.
//
// JoinOrderBuilder is able to effect this transformation (though it is not
// accomplished in so many steps).
//
// Note that it would be correct to handle inner joins in the same way as
// non-inner joins, by never splitting up predicates. However, this would
// diminish the search space for plans involving inner joins, and in many cases
// prevent the best plan from being found. It is for this reason that inner
// joins are handled separately.
//
// Also note that this approach to handling inner joins is not discussed in [8].
// Rather, it is an extension of the ideas of [8] motivated by the fact that the
// best plans for many real-world queries require inner joins to be handled in
// this way.
//
// Transitive closure
// ------------------
//
// Treating inner join conjuncts as separate edges allows yet another addition:
// we can add new edges that are implied by the transitive closure of the inner
// join edges. For example, take this query:
//
// SELECT * FROM xy
// INNER JOIN ab ON x = a
// INNER JOIN uv ON a = u
//
// The two edges x = a and a = u are explicit in this join tree. However, there
// is the additional implicit edge x = u which can be added to the join graph.
// Adding this edge allows xy to be joined to uv without introducing a cross
// join. This step of ensuring transitive closure is often crucial to finding
// the best plan; for example, the plan for TPC-H query 9 is much slower without
// it (try commenting out the call to ensureClosure()).
//
// Citations: [8]
type JoinOrderBuilder struct {
f *norm.Factory
evalCtx *eval.Context
// vertexes is the set of base relations that form the vertexes of the join
// graph. Any RelExpr can be a vertex, including a join (e.g. in case where
// join has flags that prevent reordering).
vertexes []memo.RelExpr
// edges is the set of all edges of the join graph, including both inner and
// non-inner join edges. As noted in the struct comment, each conjunct in an
// inner join predicate becomes its own edge. By contrast, the entire
// predicate becomes a single edge in the non-inner join case.
edges []edge
// innerEdges is the set of edges which were constructed using the ON
// condition of an inner join, represented as indexes into the 'edges' slice.
// It is useful to keep them separate because inner edges are permissive of
// more orderings than non-inner edges. One or more edges can be formed from
// any given inner join.
innerEdges edgeSet
// nonInnerEdges is the set of edges which were constructed using the ON
// condition of a non-inner join, represented as indexes into the 'edges'
// slice. There is a one-to-one correspondence between non-inner joins and
// their respective edges.
nonInnerEdges edgeSet
// plans maps from a set of base relations to the memo group for the join tree
// that contains those relations (and only those relations). As an example,
// the group for [xy, ab, uv] might contain the join trees (xy, (ab, uv)),
// ((xy, ab), uv), (ab, (xy, uv)), etc.
//
// The group for a single base relation is simply the base relation itself.
plans map[vertexSet]memo.RelExpr
// applicableEdges maps from each (sub)set of vertexes to the set of edges
// that must be used when building join trees for the set. See
// checkAppliedEdges for more information.
applicableEdges map[vertexSet]edgeSet
// joinCount counts the number of joins that have been added to the join
// graph. It is used to ensure that the number of joins that are reordered at
// once does not exceed the session limit.
joinCount int
// equivs is an EquivSet used to keep track of equivalence relations when
// assembling filters.
equivs props.EquivSet
// rebuildAllJoins is true when the filters in the original matched join tree
// were not pushed down as far as possible. When this is true, all joins
// except the root join need to be re-built, possibly with additional filters
// pushed down. While technically it is sufficient to only do this for the
// joins that would be changed by a successful push-down, it is simpler to
// handle things this way (and the problem is rare).
rebuildAllJoins bool
onReorderFunc OnReorderFunc
onAddJoinFunc OnAddJoinFunc
}
// Init initializes a new JoinOrderBuilder with the given factory. The join
// graph is reset, so a JoinOrderBuilder can be reused. Callback functions are
// not reset.
func (jb *JoinOrderBuilder) Init(f *norm.Factory, evalCtx *eval.Context) {
// This initialization pattern ensures that fields are not unwittingly
// reused. Field reuse must be explicit.
*jb = JoinOrderBuilder{
f: f,
evalCtx: evalCtx,
plans: make(map[vertexSet]memo.RelExpr),
applicableEdges: make(map[vertexSet]edgeSet),
onReorderFunc: jb.onReorderFunc,
onAddJoinFunc: jb.onAddJoinFunc,
equivs: props.NewEquivSet(),
}
}
// Reorder adds all valid orderings of the given join to the memo.
func (jb *JoinOrderBuilder) Reorder(join memo.RelExpr) {
switch t := join.(type) {
case *memo.InnerJoinExpr, *memo.SemiJoinExpr, *memo.AntiJoinExpr,
*memo.LeftJoinExpr, *memo.FullJoinExpr:
flags := join.Private().(*memo.JoinPrivate).Flags
if !flags.Empty() {
panic(errors.AssertionFailedf("join with hints cannot be reordered"))
}
// Populate the vertexes and edges of the join hypergraph.
jb.populateGraph(join)
// Ensure equivalence closure for the edges. This can be crucial for finding
// the best plan.
jb.ensureClosure(join)
// Ensure that the JoinOrderBuilder will not add reordered joins to the
// original memo groups (apart from the root) in the case when doing so
// would add filters that weren't present in the original joins. See the
// validateEdges comment for more information.
jb.validateEdges()
if jb.onReorderFunc != nil {
// Hook for testing purposes.
jb.callOnReorderFunc(join)
}
// Execute the DPSube algorithm. Enumerate all join orderings and add any
// valid ones to the memo.
jb.dpSube()
default:
panic(errors.AssertionFailedf("%v cannot be reordered", t.Op()))
}
}
// populateGraph traverses the given subtree up to ReorderJoinsLimit and
// initializes the vertexes and edges of the join hypergraph. populateGraph
// returns the sets of vertexes and edges that were added to the graph during
// traversal of the subtree.
func (jb *JoinOrderBuilder) populateGraph(rel memo.RelExpr) (vertexSet, edgeSet) {
// Remember starting set of vertexes and edges so that the vertexes and
// edges added during the traversal of the tree rooted at this node can be
// determined by checking the set difference.
startVertexes := jb.allVertexes()
startEdges := jb.allEdges()
switch t := rel.(type) {
case *memo.InnerJoinExpr, *memo.SemiJoinExpr, *memo.AntiJoinExpr,
*memo.LeftJoinExpr, *memo.FullJoinExpr:
jb.joinCount++
flags := t.Private().(*memo.JoinPrivate).Flags
if !flags.Empty() || jb.joinCount > int(jb.evalCtx.SessionData().ReorderJoinsLimit) {
// If the join has flags or the join limit has been reached, we can't
// reorder. Simply treat the join as a base relation.
jb.addBaseRelation(t)
break
}
left := t.Child(0).(memo.RelExpr)
right := t.Child(1).(memo.RelExpr)
on := *t.Child(2).(*memo.FiltersExpr)
// Traverse the left input of the join, initializing the join graph in the
// process.
leftVertexes, leftEdges := jb.populateGraph(left)
// Traverse the right input of the join, initializing the join graph in the
// process.
rightVertexes, rightEdges := jb.populateGraph(right)
// Construct an operator that will be referred to during TES calculation by
// any joins higher up in the join tree. A pointer to the operator will be
// stored on the edge(s) that are constructed with it.
op := &operator{
joinType: t.Op(),
leftVertexes: leftVertexes,
rightVertexes: rightVertexes,
leftEdges: leftEdges,
rightEdges: rightEdges,
}
// Create hyperedges for the join graph using this join's ON condition.
if t.Op() == opt.InnerJoinOp {
jb.makeInnerEdge(op, on)
} else {
jb.makeNonInnerEdge(op, on)
}
// Initialize the plan for this join. This allows any new joins with the
// same set of input relations to be added to the same memo group.
jb.plans[leftVertexes.union(rightVertexes)] = t
default:
jb.addBaseRelation(t)
}
// Use set difference operations to return all vertexes and edges added to the
// graph during traversal of this subtree.
return jb.allVertexes().difference(startVertexes), jb.allEdges().Difference(startEdges)
}
// ensureClosure ensures that the edges considered during join reordering
// reflect the transitive closure of all equality filters between columns.
// As an example, take a query like the following:
//
// SELECT * FROM xy INNER JOIN ab ON x = a INNER JOIN uv ON u = a
//
// Contains the explicit edges x = a and u = a, and the implicit edge x = u.
// This implicit edge will be added by ensureClosure.
func (jb *JoinOrderBuilder) ensureClosure(join memo.RelExpr) {
// Use the equivalencies of the root join to ensure transitive closure.
equivFDs := &join.Relational().FuncDeps
// Enumerate all distinct pairs of equivalent columns.
reps := equivFDs.EquivReps()
for col, ok := reps.Next(0); ok; col, ok = reps.Next(col + 1) {
// Get all columns which are known to be equal to this column.
equivGroup := equivFDs.ComputeEquivGroup(col)
// Ensure that there exists an edge for each distinct pair of equivalent
// columns.
for col1, ok1 := equivGroup.Next(0); ok1; col1, ok1 = equivGroup.Next(col1 + 1) {
for col2, ok2 := equivGroup.Next(col1 + 1); ok2; col2, ok2 = equivGroup.Next(col2 + 1) {
if !jb.hasEqEdge(col1, col2) {
// This equality is not yet represented among the edges.
jb.makeTransitiveEdge(col1, col2)
}
}
}
}
}
// validateEdges checks whether each edge applies to its original join. If any
// do not, normalization rules failed to synthesize and push a filter down as
// far as possible, and it is not valid to add new reordered joins to the
// original memo groups. When this is the case, all joins except for the root
// join need to be removed from the plans map. This prevents cases where a join
// is added to a memo group that isn't logically equivalent.
//
// This is necessary because the JoinOrderBuilder expects each join tree for a
// given set of relations to contain all filters that apply to those relations.
// When a new join is constructed, it doesn't contain "degenerate" filters -
// filters that only refer to one side of the join. So if the original join tree
// had an implicit filter that could have been synthesized and pushed down the
// tree, but wasn't, using the original join group that *should* have that
// filter when building a new join would cause a filter to be dropped.
//
// Take the following (simplified) example of a join tree where filter push-down
// rules have failed:
//
// (xy join ab on true) join uv on x = u and a = u
//
// Here, the JoinOrderBuilder will synthesize an 'x = a' filter that will be
// used to join xy and ab. If it was added to the original group, we would have
// a memo group that looks like this:
//
// group: (xy join ab on true), (xy join ab on x = a)
//
// Later joins that are constructed using this group would expect the 'x = a'
// filter to be present, and would avoid adding redundant filters. Therefore,
// a join tree like the following would be added to the memo.
//
// (xy join ab on true) join uv on x = u
//
// Notice how the 'a = u' filter has been dropped because it would be redundant
// when 'x = u' and 'x = a' are already present. We prevent this from happening
// by not reusing the original memo groups in the case when the JoinOrderBuilder
// is able to synthesize and/or push down filters that weren't in the original
// join tree.
func (jb *JoinOrderBuilder) validateEdges() {
for i := range jb.edges {
if jb.rebuildAllJoins {
break
}
e := &jb.edges[i]
if e.op.joinType == opt.InnerJoinOp {
jb.rebuildAllJoins = !e.checkInnerJoin(e.op.leftVertexes, e.op.rightVertexes)
} else {
jb.rebuildAllJoins = !e.checkNonInnerJoin(e.op.leftVertexes, e.op.rightVertexes)
}
}
if jb.rebuildAllJoins {
for vertexes := range jb.plans {
if vertexes.isSingleton() || vertexes == jb.allVertexes() {
// Do not remove the plan if it is for a base relation (not a join) or
// it is the root join. Adding to the root join group is correct because
// the JoinOrderBuilder will only consider filters that were present
// (even if only implicitly) in the root join tree. It is also necessary
// because the purpose of the JoinOrderBuilder is to add equivalent join
// plans to the root join group - otherwise, any new joins would be
// disconnected from the main query plan.
continue
}
delete(jb.plans, vertexes)
}
}
}
// dpSube carries out the DPSube algorithm (citations: [8] figure 4). All
// disjoint pairs of subsets of base relations are enumerated and checked for
// validity. If valid, the pair of subsets is used along with the edges
// connecting them to create a new join operator, which is added to the memo.
// TODO(drewk): implement DPHyp (or a similar algorithm).
func (jb *JoinOrderBuilder) dpSube() {
subsets := jb.allVertexes()
for subset := vertexSet(1); subset <= subsets; subset++ {
if subset.isSingleton() {
// This subset has only one set bit, which means it only represents one
// relation. We need at least two relations in order to create a new join.
continue
}
jb.setApplicableEdges(subset)
// Enumerate all possible pairwise-disjoint binary partitions of the subset,
// s1 AND s2. These represent sets of relations that may be joined together.
//
// Only iterate s1 up to subset/2 to avoid enumerating duplicate partitions.
// This works because s1 and s2 are always disjoint, and subset will always
// be equal to s1 + s2. Therefore, any pair of subsets where s1 > s2 will
// already have been handled when s2 < s1. Also note that for subset = 1111
// (in binary), subset / 2 = 0111 (integer division).
for s1 := vertexSet(1); s1 <= subset/2; s1++ {
if !s1.isSubsetOf(subset) {
continue
}
s2 := subset.difference(s1)
jb.addJoins(s1, s2)
}
}
}
// setApplicableEdges initializes applicableEdges with all edges that must show
// up in any join tree that is constructed for the given set of vertexes. See
// checkAppliedEdges for how this information is used.
func (jb *JoinOrderBuilder) setApplicableEdges(s vertexSet) {
applicableEdges := edgeSet{}
for i := range jb.edges {
if jb.edges[i].tes.isSubsetOf(s) {
applicableEdges.Add(i)
}
}
jb.applicableEdges[s] = applicableEdges
}
// addJoins iterates through the edges of the join graph and checks whether any
// joins can be constructed between the memo groups for the two given sets of
// base relations without creating an invalid plan or introducing cross joins.
// If any valid joins are found, they are added to the memo.
func (jb *JoinOrderBuilder) addJoins(s1, s2 vertexSet) {
if jb.plans[s1] == nil || jb.plans[s2] == nil {
// Both inputs must have plans.
return
}
// Keep track of which edges are applicable to this join.
var appliedEdges edgeSet
jb.equivs.Reset()
jb.equivs.AddFromFDs(&jb.plans[s1].Relational().FuncDeps)
jb.equivs.AddFromFDs(&jb.plans[s2].Relational().FuncDeps)
// Gather all inner edges that connect the left and right relation sets.
var innerJoinFilters memo.FiltersExpr
for i, ok := jb.innerEdges.Next(0); ok; i, ok = jb.innerEdges.Next(i + 1) {
e := &jb.edges[i]
// Ensure that this edge forms a valid connection between the two sets. See
// the checkNonInnerJoin and checkInnerJoin comments for more information.
if e.checkInnerJoin(s1, s2) {
// Record this edge as applied even if it's redundant, since redundant
// edges are trivially applied.
appliedEdges.Add(i)
if areFiltersRedundant(&jb.equivs, e.filters) {
// Avoid adding redundant filters.
continue
}
for j := range e.filters {
jb.equivs.AddFromFDs(&e.filters[j].ScalarProps().FuncDeps)
}
innerJoinFilters = append(innerJoinFilters, e.filters...)
}
}
// Iterate through the non-inner edges and attempt to construct joins from
// them.
for i, ok := jb.nonInnerEdges.Next(0); ok; i, ok = jb.nonInnerEdges.Next(i + 1) {
e := &jb.edges[i]
// Ensure that this edge forms a valid connection between the two sets. See
// the checkNonInnerJoin and checkInnerJoin comments for more information.
if e.checkNonInnerJoin(s1, s2) {
appliedEdges.Add(i)
// Construct a non-inner join. If any inner join filters also apply to the
// pair of relationSets, construct a select on top of the join with the
// inner join filters.
jb.addJoin(e.op.joinType, s1, s2, e.filters, innerJoinFilters, appliedEdges)
return
}
if e.checkNonInnerJoin(s2, s1) {
appliedEdges.Add(i)
// If joining s1, s2 is not valid, try s2, s1. We only do this if the
// s1, s2 join fails, because commutation is handled by the addJoin
// function. This is necessary because we only iterate s1 up to subset / 2
// in DPSube(). Take this transformation as an example:
//
// SELECT *
// FROM (SELECT * FROM xy LEFT JOIN ab ON x = a)
// INNER JOIN uv ON x = u
// =>
// SELECT *
// FROM (SELECT * FROM xy INNER JOIN uv ON x = u)
// LEFT JOIN ab ON x = a
//
// Bitset encodings for the base relations:
// xy: 001
// ab: 010
// uv: 100
//
// The left join in the new plan is between set 101 on the left, and set
// 010 on the right. 101 is larger than 111 / 2, so we will not enumerate
// this plan unless we consider a join with s2 on the left and s1 on the
// right.
jb.addJoin(e.op.joinType, s2, s1, e.filters, innerJoinFilters, appliedEdges)
return
}
}
if !appliedEdges.Empty() {
// Construct an inner join. Don't add in the case when a non-inner join has
// already been constructed, because doing so can lead to a case where a
// non-inner join operator 'disappears' because an inner join has replaced
// it.
jb.addJoin(opt.InnerJoinOp, s1, s2, innerJoinFilters, nil /* selectFilters */, appliedEdges)
}
}
// makeInnerEdge constructs edges from the ON condition of an inner join. If the
// inner join has one or more conjuncts, an edge is created for each conjunct.
// If the inner join has no filters, an edge with an empty SES is created.
func (jb *JoinOrderBuilder) makeInnerEdge(op *operator, filters memo.FiltersExpr) {
if len(filters) == 0 {
// This is a cross join. Create a single edge for the empty FiltersExpr.
jb.edges = append(jb.edges, *jb.makeEdge(op, filters))
jb.innerEdges.Add(len(jb.edges) - 1)
return
}
for i := range filters {
// Create an edge for each conjunct.
jb.edges = append(jb.edges, *jb.makeEdge(op, filters[i:i+1]))
jb.innerEdges.Add(len(jb.edges) - 1)
}
}
// makeNonInnerEdge constructs an edge from the ON condition of a non-inner
// join. For any given non-inner join, exactly one edge is constructed.
func (jb *JoinOrderBuilder) makeNonInnerEdge(op *operator, filters memo.FiltersExpr) {
// Always create a single edge from a non-inner join.
jb.edges = append(jb.edges, *jb.makeEdge(op, filters))
jb.nonInnerEdges.Add(len(jb.edges) - 1)
}
// makeTransitiveEdge constructs an edge using an equality between two columns
// that results from calculating transitive closure over the edges of the join
// graph.
func (jb *JoinOrderBuilder) makeTransitiveEdge(col1, col2 opt.ColumnID) {
var op *operator
// Find the inner join down to which this filter would be pushed if it had
// been made explicit in the original query. This operator will be used to
// construct the TES for this edge. Note that this is possible because each
// edge contains a record of all base relations that were in the left and
// right inputs of the original join operators from which they were formed.
relations := jb.getRelations(opt.MakeColSet(col1, col2))
for i, ok := jb.innerEdges.Next(0); ok; i, ok = jb.innerEdges.Next(i + 1) {
currEdge := &jb.edges[i]
if relations.isSubsetOf(currEdge.op.leftVertexes.union(currEdge.op.rightVertexes)) &&
relations.intersects(currEdge.op.leftVertexes) &&
relations.intersects(currEdge.op.rightVertexes) {
op = currEdge.op
break
}
}
if op == nil {
// No valid operator was found. This can happen (for example) when an
// equivalence comes from within a base relation, in which case filters
// would have been pushed down beyond the original join tree.
return
}
originalJoin, ok := jb.plans[op.leftVertexes.union(op.rightVertexes)]
if !ok {
panic(errors.AssertionFailedf("failed to find expected join plan"))
}
if !originalJoin.Relational().FuncDeps.AreColsEquiv(col1, col2) {
// This inferred filter was not pushed down as far as possible. All joins
// apart from the root will have to be rebuilt. We have to do this check
// here because we set the op for this edge to the join to which the filter
// *would* have been pushed down if it existed, so the applicable check will
// always succeed for that join.
jb.rebuildAllJoins = true
}
// Construct the edge.
var1 := jb.f.ConstructVariable(col1)
var2 := jb.f.ConstructVariable(col2)
condition := jb.f.ConstructEq(var1, var2)
filters := memo.FiltersExpr{jb.f.ConstructFiltersItem(condition)}
// Add the edge to the join graph.
jb.edges = append(jb.edges, *jb.makeEdge(op, filters))
jb.innerEdges.Add(len(jb.edges) - 1)
}
// makeEdge returns a new edge given an operator and set of filters.
func (jb *JoinOrderBuilder) makeEdge(op *operator, filters memo.FiltersExpr) (e *edge) {
e = &edge{op: op, filters: filters}
e.calcNullRejectedRels(jb)
e.calcSES(jb)
e.calcTES(jb.edges)
return e
}
// getFreeVars returns the set of columns referenced by the given predicate.
func (jb *JoinOrderBuilder) getFreeVars(predicate memo.FiltersExpr) opt.ColSet {
return predicate.OuterCols()
}
// hasEqEdge returns true if the inner edges include a direct equality between
// the two given columns (e.g. x = a).
func (jb *JoinOrderBuilder) hasEqEdge(leftCol, rightCol opt.ColumnID) bool {
for idx, ok := jb.innerEdges.Next(0); ok; idx, ok = jb.innerEdges.Next(idx + 1) {
for i := range jb.edges[idx].filters {
if jb.edges[idx].filters[i].ScalarProps().FuncDeps.AreColsEquiv(leftCol, rightCol) {
return true
}
}
}
return false
}
// checkAppliedEdges checks that each join plan includes every edge for which
// the TES is a subset of the relations that are joined together by the plan.
// This is necessary to recover a property which the original algorithm relies
// on - namely that if any edge cannot be applied in a given plan, that plan
// must be invalid. Consider the following three points:
//
// 1. The join reordering algorithm never includes a cross-product in an
// enumerated plan unless it was part of the original join tree. This
// means that a join between two sub-plans is only considered if there is
// an applicable edge that can be used to construct the join.
//
// 2. The original paper associates each join in the original join tree with
// exactly one edge in the join hypergraph.
//
// 3. The JoinOrderBuilder departs from the paper by associating each inner
// join conjunct with an edge. This means that each join can be associated
// with one or more edges. See the section in the JoinOrderBuilder comment
// titled "Special handling of inner joins" for details.
//
// (1) and (2) together imply that a reordered join tree is only considered if
// every edge in the hypergraph could be applied to construct a join for every
// subtree. This allows the original algorithm to prevent invalid orderings by
// making a single edge inapplicable. However, because of (3) the same is no
// longer true for the `JoinOrderBuilder`. checkAppliedEdges corrects for this
// by explicitly checking that all applicable edges have been applied when a
// join plan is considered.
func (jb *JoinOrderBuilder) checkAppliedEdges(s1, s2 vertexSet, appliedEdges edgeSet) bool {
leftApplied, rightApplied := jb.applicableEdges[s1], jb.applicableEdges[s2]
allAppliedEdges := appliedEdges.Union(leftApplied).Union(rightApplied)
expectedAppliedEdges := jb.applicableEdges[s1.union(s2)]
return allAppliedEdges.Equals(expectedAppliedEdges)
}
// addJoin adds a join between the given left and right subsets of relations on
// the given set of edges. If the group containing joins between this set of
// relations is already contained in the plans field, the new join is added to
// the memo group. Otherwise, the join is memoized (possibly constructing a new
// group). If the join being considered existed in the originally matched join
// tree, no join is added (though its commuted version may be).
func (jb *JoinOrderBuilder) addJoin(
op opt.Operator,
s1, s2 vertexSet,
joinFilters, selectFilters memo.FiltersExpr,
appliedEdges edgeSet,
) {
if s1.intersects(s2) {
panic(errors.AssertionFailedf("sets are not disjoint"))
}
if !jb.checkAppliedEdges(s1, s2, appliedEdges) {
return
}
if jb.onAddJoinFunc != nil {
// Hook for testing purposes.
jb.callOnAddJoinFunc(s1, s2, joinFilters, selectFilters, op)
}
left := jb.plans[s1]
right := jb.plans[s2]
union := s1.union(s2)
if !jb.joinIsRedundant(s1, s2, appliedEdges) {
if jb.plans[union] != nil {
jb.addToGroup(op, left, right, joinFilters, selectFilters, jb.plans[union])
} else {
jb.plans[union] = jb.memoize(op, left, right, joinFilters, selectFilters)
}
}
if commute(op) {
// Also add the commuted version of the join to the memo. Note that if the
// join is redundant (a join between base relation sets s1 and s2 existed in
// the matched join tree) then jb.plans[union] will already have the
// original join group.
if jb.plans[union] == nil {
panic(errors.AssertionFailedf("expected existing join plan"))
}
jb.addToGroup(op, right, left, joinFilters, selectFilters, jb.plans[union])
if jb.onAddJoinFunc != nil {
// Hook for testing purposes.
jb.callOnAddJoinFunc(s2, s1, joinFilters, selectFilters, op)
}
}
}
// areFiltersRedundant returns true if the given FiltersExpr contains a single
// equality filter that is already represented by the given FuncDepSet.
func areFiltersRedundant(equivs *props.EquivSet, filters memo.FiltersExpr) bool {
if len(filters) != 1 {
return false
}
eq, ok := filters[0].Condition.(*memo.EqExpr)
if !ok {
return false
}
var1, ok1 := eq.Left.(*memo.VariableExpr)
var2, ok2 := eq.Right.(*memo.VariableExpr)
if !ok1 || !ok2 {
return false
}
return equivs.AreColsEquiv(var1.Col, var2.Col)
}
// addToGroup adds a join of the given type and with the given inputs to the
// given memo group. If selectFilters is not empty, the join is memoized instead
// and used as input to a select, which is added to the join group.
func (jb *JoinOrderBuilder) addToGroup(
op opt.Operator, left, right memo.RelExpr, on, selectFilters memo.FiltersExpr, grp memo.RelExpr,
) {
if len(selectFilters) > 0 {
joinExpr := jb.memoize(op, left, right, on, nil)
selectExpr := &memo.SelectExpr{
Input: joinExpr,
Filters: selectFilters,
}
jb.f.Memo().AddSelectToGroup(selectExpr, grp)
return
}
// Set SkipReorderJoins to true in order to avoid duplicate reordering on this
// join.
newJoinPrivate := memo.JoinPrivate{SkipReorderJoins: true}
switch op {
case opt.InnerJoinOp:
newJoin := &memo.InnerJoinExpr{
Left: left,
Right: right,
On: on,
JoinPrivate: newJoinPrivate,
}
jb.f.Memo().AddInnerJoinToGroup(newJoin, grp)
case opt.SemiJoinOp:
newJoin := &memo.SemiJoinExpr{
Left: left,
Right: right,
On: on,
JoinPrivate: newJoinPrivate,
}
jb.f.Memo().AddSemiJoinToGroup(newJoin, grp)
case opt.AntiJoinOp:
newJoin := &memo.AntiJoinExpr{
Left: left,
Right: right,
On: on,
JoinPrivate: newJoinPrivate,
}
jb.f.Memo().AddAntiJoinToGroup(newJoin, grp)
case opt.LeftJoinOp:
newJoin := &memo.LeftJoinExpr{
Left: left,
Right: right,
On: on,
JoinPrivate: newJoinPrivate,
}
jb.f.Memo().AddLeftJoinToGroup(newJoin, grp)
case opt.FullJoinOp:
newJoin := &memo.FullJoinExpr{
Left: left,
Right: right,
On: on,
JoinPrivate: newJoinPrivate,
}
jb.f.Memo().AddFullJoinToGroup(newJoin, grp)
default:
panic(errors.AssertionFailedf("invalid operator: %v", op))
}
}
// memoize adds a join of the given type and with the given inputs to the memo
// and returns it. If selectFilters is not empty, the join becomes the input of
// a select with those filters, which is also added to the memo and returned.
func (jb *JoinOrderBuilder) memoize(
op opt.Operator, left, right memo.RelExpr, on, selectFilters memo.FiltersExpr,
) memo.RelExpr {
var join memo.RelExpr
// Set SkipReorderJoins to true in order to avoid duplicate reordering on this
// join.
newJoinPrivate := &memo.JoinPrivate{SkipReorderJoins: true}
switch op {
case opt.InnerJoinOp:
join = jb.f.Memo().MemoizeInnerJoin(left, right, on, newJoinPrivate)
case opt.SemiJoinOp:
join = jb.f.Memo().MemoizeSemiJoin(left, right, on, newJoinPrivate)
case opt.AntiJoinOp:
join = jb.f.Memo().MemoizeAntiJoin(left, right, on, newJoinPrivate)
case opt.LeftJoinOp:
join = jb.f.Memo().MemoizeLeftJoin(left, right, on, newJoinPrivate)
case opt.FullJoinOp:
join = jb.f.Memo().MemoizeFullJoin(left, right, on, newJoinPrivate)
default:
panic(errors.AssertionFailedf("invalid operator: %v", op))
}
if len(selectFilters) > 0 {