/
merge_join.go
571 lines (510 loc) · 14.3 KB
/
merge_join.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
// Copyright 2022 Dolthub, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package rowexec
import (
"errors"
"io"
"github.com/dolthub/go-mysql-server/sql/plan"
"github.com/dolthub/go-mysql-server/sql"
"github.com/dolthub/go-mysql-server/sql/expression"
)
var ErrMergeJoinExpectsComparerFilters = errors.New("merge join expects expression.Comparer filters, found: %T")
// NewMergeJoin returns a node that performs a presorted merge join on
// two relations. We require 1) the join filter is an equality with disjoint
// join attributes, 2) the free attributes for a relation are a prefix for
// an index that will be used to return sorted rows.
func NewMergeJoin(left, right sql.Node, cond sql.Expression) *plan.JoinNode {
return plan.NewJoin(left, right, plan.JoinTypeMerge, cond)
}
func NewLeftMergeJoin(left, right sql.Node, cond sql.Expression) *plan.JoinNode {
return plan.NewJoin(left, right, plan.JoinTypeLeftOuterMerge, cond)
}
func newMergeJoinIter(ctx *sql.Context, b sql.NodeExecBuilder, j *plan.JoinNode, row sql.Row) (sql.RowIter, error) {
l, err := b.Build(ctx, j.Left(), row)
if err != nil {
return nil, err
}
r, err := b.Build(ctx, j.Right(), row)
if err != nil {
return nil, err
}
fullRow := make(sql.Row, len(row)+len(j.Left().Schema())+len(j.Right().Schema()))
fullRow[0] = row
if len(row) > 0 {
copy(fullRow[0:], row[:])
}
// a merge join's first filter provides direction information
// for which iter to update next
filters := expression.SplitConjunction(j.Filter)
cmp, ok := filters[0].(expression.Comparer)
if !ok {
return nil, sql.ErrMergeJoinExpectsComparerFilters.New(filters[0])
}
if len(filters) == 0 {
return nil, sql.ErrNoJoinFilters.New()
}
var iter sql.RowIter = &mergeJoinIter{
left: l,
right: r,
filters: filters[1:],
cmp: cmp,
typ: j.Op,
fullRow: fullRow,
scopeLen: j.ScopeLen,
parentLen: len(row) - j.ScopeLen,
leftRowLen: len(j.Left().Schema()),
rightRowLen: len(j.Right().Schema()),
}
return iter, nil
}
// mergeJoinIter alternates incrementing two RowIters, assuming
// rows will be provided in a sorted order given the join |expr|
// (see sortedIndexScanForTableCol). Extra join |filters| that do
// not provide a directional ordering signal for index iteration
// are evaluated separately.
type mergeJoinIter struct {
// cmp is a directional indicator for row iter increments
cmp expression.Comparer
// filters is the remaining set of join conditions
filters []sql.Expression
left sql.RowIter
right sql.RowIter
fullRow sql.Row
// match lookahead buffers and state tracking (private to match)
rightBuf []sql.Row
bufI int
rightPeek sql.Row
leftPeek sql.Row
rightDone bool
leftDone bool
// matchIncLeft indicates whether the most recent |i.incMatch|
// call incremented the left row.
matchIncLeft bool
// leftMatched indicates whether the current left in |i.fullRow|
// has satisfied the join condition.
leftMatched bool
// lifecycle maintenance
init bool
leftExhausted bool
rightExhausted bool
typ plan.JoinType
scopeLen int
leftRowLen int
rightRowLen int
parentLen int
}
var _ sql.RowIter = (*mergeJoinIter)(nil)
func (i *mergeJoinIter) sel(ctx *sql.Context, row sql.Row) (bool, error) {
for _, f := range i.filters {
res, err := sql.EvaluateCondition(ctx, f, row)
if err != nil {
return false, err
}
if !sql.IsTrue(res) {
return false, nil
}
}
return true, nil
}
type mergeState uint8
const (
msInit mergeState = iota
msExhaustCheck
msCompare
msIncLeft
msIncRight
msSelect
msRet
msRetLeft
msRejectNull
)
func (i *mergeJoinIter) Next(ctx *sql.Context) (sql.Row, error) {
var err error
var ret sql.Row
var res int
// The common inner join match flow:
// 1) check for io.EOF
// 2) evaluate compare filter
// 3) evaluate select filters
// 4) initialize match state
// 5) drain match state
// 6) repeat
//
// Left-join matching is unique. At any given time, we need to know whether
// a unique left row: 1) has already matched, 2) has more right rows
// available for matching before we can return a nullified-row. Otherwise
// we may accidentally return nullified rows that have matches (before or
// after the current row), or fail to return a nullified row that has no
// matches.
//
// We use two variables to manage the lookahead state management.
// |matchedleft| is a forward-looking indicator of whether the current left
// row has satisfied a join condition. It is reset to false when we
// increment left. |matchincleft| is true when the most recent call to
// |incmatch| incremented the left row. The two vars combined let us
// lookahead during msSelect to 1) identify proper nullified row matches,
// and 2) maintain forward-looking state for the next |i.fullrow|.
//
nextState := msInit
for {
switch nextState {
case msInit:
if !i.init {
err = i.initIters(ctx)
if err != nil {
return nil, err
}
}
nextState = msExhaustCheck
case msExhaustCheck:
if i.lojFinalize() {
ret = i.copyReturnRow()
nextState = msRetLeft
} else if i.exhausted() {
return nil, io.EOF
} else {
nextState = msCompare
}
case msCompare:
res, err = i.cmp.Compare(ctx, i.fullRow)
if expression.ErrNilOperand.Is(err) {
nextState = msRejectNull
break
} else if err != nil {
return nil, err
}
switch {
case res < 0:
if i.typ.IsLeftOuter() {
if i.leftMatched {
nextState = msIncLeft
} else {
ret = i.copyReturnRow()
nextState = msRetLeft
}
} else {
nextState = msIncLeft
}
case res > 0:
nextState = msIncRight
case res == 0:
nextState = msSelect
}
case msRejectNull:
left, _ := i.cmp.Left().Eval(ctx, i.fullRow)
if left == nil {
if i.typ.IsLeftOuter() && !i.leftMatched {
ret = i.copyReturnRow()
nextState = msRetLeft
} else {
nextState = msIncLeft
}
} else {
nextState = msIncRight
}
case msIncLeft:
err = i.incLeft(ctx)
nextState = msExhaustCheck
case msIncRight:
err = i.incRight(ctx)
nextState = msExhaustCheck
case msSelect:
ret = i.copyReturnRow()
currLeftMatched := i.leftMatched
ok, err := i.sel(ctx, ret)
if err != nil {
return nil, err
}
err = i.incMatch(ctx)
if err != nil {
return nil, err
}
if ok {
if !i.matchIncLeft {
// |leftMatched| is forward-looking, sets state for
// current |i.fullRow| (next |ret|)
i.leftMatched = true
}
nextState = msRet
break
}
if !i.typ.IsLeftOuter() {
nextState = msExhaustCheck
break
}
if i.matchIncLeft && !currLeftMatched {
// |i.matchIncLeft| indicates whether the most recent
// |i.incMatch| call incremented the left row.
// |currLeftMatched| indicates whether |ret| has already
// successfully met a join condition.
return i.removeParentRow(i.nullifyRightRow(ret)), nil
} else {
nextState = msExhaustCheck
}
case msRet:
return i.removeParentRow(ret), nil
case msRetLeft:
ret = i.removeParentRow(i.nullifyRightRow(ret))
err = i.incLeft(ctx)
if err != nil {
return nil, err
}
return ret, nil
}
}
}
func (i *mergeJoinIter) copyReturnRow() sql.Row {
ret := make(sql.Row, len(i.fullRow))
copy(ret, i.fullRow)
return ret
}
// incMatch uses two phases to find all left and right rows that match their
// companion rows for the given match stats:
// 1. collect all right rows that match the current left row into a buffer;
// 2. for every left row that matches the original right row, match every
// right row.
//
// We maintain lookaheads for the first non-matching row in each iter. If
// there is no next non-matching row (io.EOF), we trigger |i.exhausted| at
// the appropriate time depending on whether we are left-joining.
func (i *mergeJoinIter) incMatch(ctx *sql.Context) error {
i.matchIncLeft = false
if !i.rightDone {
// initialize right matches buffer
right := make(sql.Row, i.rightRowLen)
copy(right, i.fullRow[i.scopeLen+i.parentLen+i.leftRowLen:])
i.rightBuf = append(i.rightBuf, right)
match := true
var err error
var peek sql.Row
for match {
match, peek, err = i.peekMatch(ctx, i.right)
if err != nil {
return err
} else if match {
i.rightBuf = append(i.rightBuf, peek)
} else {
i.rightPeek = peek
i.rightDone = true
}
}
// left row 1 and right row 1 is a duplicate of the first match
// captured in outer closure, slough one iteration
err = i.incMatch(ctx)
if err != nil {
return err
}
}
if i.bufI > len(i.rightBuf)-1 {
// matched entire right buffer to the current left row, reset
i.matchIncLeft = true
i.bufI = 0
match, peek, err := i.peekMatch(ctx, i.left)
if err != nil {
return err
} else if !match {
i.leftPeek = peek
i.leftDone = true
}
i.leftMatched = false
}
if !i.leftDone {
// rightBuf has already been validated, we don't need compare
copySubslice(i.fullRow, i.rightBuf[i.bufI], i.scopeLen+i.parentLen+i.leftRowLen)
i.bufI++
return nil
}
defer i.resetMatchState()
if i.leftPeek == nil {
i.leftExhausted = true
}
if i.rightPeek == nil {
i.rightExhausted = true
}
if i.exhausted() {
if i.lojFinalize() {
// left joins expect the left row in |i.fullRow| as long
// as the left iter is not exhausted.
copySubslice(i.fullRow, i.leftPeek, i.scopeLen+i.parentLen)
}
return nil
}
// both lookaheads fail the join condition. Drain
// lookahead rows / increment both iterators.
i.matchIncLeft = true
copySubslice(i.fullRow, i.leftPeek, i.scopeLen+i.parentLen)
copySubslice(i.fullRow, i.rightPeek, i.scopeLen+i.parentLen+i.leftRowLen)
return nil
}
// lojFinalize is a unique state where we have exhausted the outer iterator,
// but not the inner iterator we are outer joining against.
func (i *mergeJoinIter) lojFinalize() bool {
return i.rightExhausted && !i.leftExhausted && i.typ.IsLeftOuter()
}
// nullifyRightRow sets the values corresponding to the right row to nil
func (i *mergeJoinIter) nullifyRightRow(r sql.Row) sql.Row {
for j := i.scopeLen + i.parentLen + i.leftRowLen; j < len(r); j++ {
r[j] = nil
}
return r
}
// initIters populates i.fullRow and clears the match state
func (i *mergeJoinIter) initIters(ctx *sql.Context) error {
err := i.incLeft(ctx)
if err != nil {
return err
}
err = i.incRight(ctx)
if err != nil {
return err
}
i.init = true
i.resetMatchState()
return nil
}
// resetMatchState clears the match state variables to zero values
func (i *mergeJoinIter) resetMatchState() {
i.leftPeek = nil
i.rightPeek = nil
i.leftDone = false
i.rightDone = false
i.rightBuf = i.rightBuf[:0]
i.bufI = 0
}
// peekMatch reads the next row from an iterator, attempts to update i.fullRow
// to find a matching condition, rewinding the change in the case of failure.
// We return whether a successful match was found, the lookahead row for saving
// in the case of failure, and an error or nil. If the iterator io.EOFs, we return
// no match, no lookahead row, and no error.
func (i *mergeJoinIter) peekMatch(ctx *sql.Context, iter sql.RowIter) (bool, sql.Row, error) {
var off int
var restore sql.Row
switch iter {
case i.left:
off = i.scopeLen + i.parentLen
restore = make(sql.Row, i.leftRowLen)
copy(restore, i.fullRow[off:off+i.leftRowLen])
case i.right:
off = i.scopeLen + i.parentLen + i.leftRowLen
restore = make(sql.Row, i.rightRowLen)
copy(restore, i.fullRow[off:off+i.rightRowLen])
default:
}
// peek lookahead
peek, err := iter.Next(ctx)
if errors.Is(err, io.EOF) {
// io.EOF is the only nil row nil err return
return false, nil, nil
} else if err != nil {
return false, nil, err
}
// check if lookahead valid
copySubslice(i.fullRow, peek, off)
res, err := i.cmp.Compare(ctx, i.fullRow)
if expression.ErrNilOperand.Is(err) {
// revert change to output row if no match
copySubslice(i.fullRow, restore, off)
} else if err != nil {
return false, nil, err
}
if res != 0 {
// revert change to output row if no match
copySubslice(i.fullRow, restore, off)
}
return res == 0, peek, nil
}
// exhausted returns true if either iterator has io.EOF'd
func (i *mergeJoinIter) exhausted() bool {
return i.leftExhausted || i.rightExhausted
}
// copySubslice copies |src| into |dst| starting at index |off|
func copySubslice(dst, src sql.Row, off int) {
for i, v := range src {
dst[off+i] = v
}
}
// incLeft updates |i.fullRow|'s left row
func (i *mergeJoinIter) incLeft(ctx *sql.Context) error {
i.leftMatched = false
var row sql.Row
var err error
if i.leftPeek != nil {
row = i.leftPeek
i.leftPeek = nil
} else {
row, err = i.left.Next(ctx)
if errors.Is(err, io.EOF) {
i.leftExhausted = true
return nil
} else if err != nil {
return err
}
}
off := i.scopeLen + i.parentLen
for j, v := range row {
i.fullRow[off+j] = v
}
return nil
}
// incRight updates |i.fullRow|'s right row
func (i *mergeJoinIter) incRight(ctx *sql.Context) error {
var row sql.Row
var err error
if i.rightPeek != nil {
row = i.rightPeek
i.rightPeek = nil
} else {
row, err = i.right.Next(ctx)
if errors.Is(err, io.EOF) {
i.rightExhausted = true
return nil
} else if err != nil {
return err
}
}
off := i.scopeLen + i.parentLen + i.leftRowLen
for j, v := range row {
i.fullRow[off+j] = v
}
return nil
}
// incLeft updates |i.fullRow|'s |inRow|
func (i *mergeJoinIter) incIter(ctx *sql.Context, iter sql.RowIter, off int) error {
row, err := iter.Next(ctx)
if err != nil {
return err
}
for j, v := range row {
i.fullRow[off+j] = v
}
return nil
}
func (i *mergeJoinIter) removeParentRow(r sql.Row) sql.Row {
copy(r[i.scopeLen:], r[i.scopeLen+i.parentLen:])
r = r[:len(r)-i.parentLen]
return r
}
func (i *mergeJoinIter) Close(ctx *sql.Context) (err error) {
if i.left != nil {
err = i.left.Close(ctx)
}
if i.right != nil {
if err == nil {
err = i.right.Close(ctx)
} else {
i.right.Close(ctx)
}
}
return err
}