forked from influxdata/kapacitor
-
Notifications
You must be signed in to change notification settings - Fork 0
/
join.go
706 lines (654 loc) · 16.1 KB
/
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
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
package kapacitor
import (
"fmt"
"log"
"sync"
"time"
"github.com/influxdata/influxdb/influxql"
"github.com/influxdata/kapacitor/expvar"
"github.com/influxdata/kapacitor/models"
"github.com/influxdata/kapacitor/pipeline"
"github.com/influxdata/kapacitor/timer"
)
type JoinNode struct {
node
j *pipeline.JoinNode
fill influxql.FillOption
fillValue interface{}
groups map[models.GroupID]*group
mu sync.RWMutex
runningGroups sync.WaitGroup
// Buffer for caching points that need to be matched with specific points.
matchGroupsBuffer map[models.GroupID][]srcPoint
// Buffer for caching specific points until their match arrivces.
specificGroupsBuffer map[models.GroupID][]srcPoint
// Represents the lower bound of times per group per parent
lowMarks map[srcGroup]time.Time
groupsMu sync.RWMutex
reported map[int]bool
allReported bool
}
// Create a new JoinNode, which takes pairs from parent streams combines them into a single point.
func newJoinNode(et *ExecutingTask, n *pipeline.JoinNode, l *log.Logger) (*JoinNode, error) {
jn := &JoinNode{
j: n,
node: node{Node: n, et: et, logger: l},
matchGroupsBuffer: make(map[models.GroupID][]srcPoint),
specificGroupsBuffer: make(map[models.GroupID][]srcPoint),
lowMarks: make(map[srcGroup]time.Time),
reported: make(map[int]bool),
}
// Set fill
switch fill := n.Fill.(type) {
case string:
switch fill {
case "null":
jn.fill = influxql.NullFill
case "none":
jn.fill = influxql.NoFill
default:
return nil, fmt.Errorf("unexpected fill option %s", fill)
}
case int64, float64:
jn.fill = influxql.NumberFill
jn.fillValue = fill
default:
jn.fill = influxql.NoFill
}
jn.node.runF = jn.runJoin
return jn, nil
}
func (j *JoinNode) runJoin([]byte) error {
j.groups = make(map[models.GroupID]*group)
valueF := func() int64 {
j.groupsMu.RLock()
l := len(j.groups)
j.groupsMu.RUnlock()
return int64(l)
}
j.statMap.Set(statCardinalityGauge, expvar.NewIntFuncGauge(valueF))
groupErrs := make(chan error, 1)
done := make(chan struct{}, len(j.ins))
for i := range j.ins {
// Start gorouting per parent so we do not deadlock.
// This way independent of the order that parents receive data
// we can handle it.
t := j.et.tm.TimingService.NewTimer(j.statMap.Get(statAverageExecTime).(timer.Setter))
go func(i int, t timer.Timer) {
defer func() {
done <- struct{}{}
}()
in := j.ins[i]
for p, ok := in.Next(); ok; p, ok = in.Next() {
t.Start()
srcP := srcPoint{src: i, p: p}
if len(j.j.Dimensions) > 0 {
// Match points with their group based on join dimensions.
j.matchPoints(srcP, groupErrs)
} else {
// Just send point on to group, we are not joining on specific dimensions.
func() {
j.mu.Lock()
defer j.mu.Unlock()
group := j.getGroup(p, groupErrs)
// Send current point
group.points <- srcP
}()
}
t.Stop()
}
}(i, t)
}
for range j.ins {
select {
case <-done:
case err := <-groupErrs:
return err
}
}
// No more points are coming signal all groups to finish up.
j.groupsMu.RLock()
for _, group := range j.groups {
close(group.points)
}
j.groupsMu.RUnlock()
j.runningGroups.Wait()
j.groupsMu.RLock()
for _, group := range j.groups {
err := group.emitAll()
if err != nil {
return err
}
}
j.groupsMu.RUnlock()
return nil
}
// The purpose of this method is to match more specific points
// with the less specific points as they arrive.
//
// Where 'more specific' means, that a point has more dimensions than the join.on dimensions.
func (j *JoinNode) matchPoints(p srcPoint, groupErrs chan<- error) {
// Specific points may be sent to the joinset without a matching point, but not the other way around.
// This is because the specific points have the needed specific tag data.
// The joinset will later handle the fill inner/outer join operations.
j.mu.Lock()
defer j.mu.Unlock()
if !j.allReported {
j.reported[p.src] = true
j.allReported = len(j.reported) == len(j.ins)
}
t := p.p.PointTime().Round(j.j.Tolerance)
groupId := models.ToGroupID(
p.p.PointName(),
p.p.PointTags(),
models.Dimensions{
ByName: p.p.PointDimensions().ByName,
TagNames: j.j.Dimensions,
},
)
// Update current srcGroup lowMark
srcG := srcGroup{src: p.src, groupId: groupId}
j.lowMarks[srcG] = t
// Determine lowMark, the oldest time per parent per group.
var lowMark time.Time
if j.allReported {
for s := 0; s < len(j.ins); s++ {
sg := srcGroup{src: s, groupId: groupId}
if lm := j.lowMarks[sg]; lowMark.IsZero() || lm.Before(lowMark) {
lowMark = lm
}
}
}
// Check for cached specific points that can now be sent alone.
if j.allReported {
// Send all cached specific point that won't match anymore.
var i int
buf := j.specificGroupsBuffer[groupId]
l := len(buf)
for i = 0; i < l; i++ {
st := buf[i].p.PointTime().Round(j.j.Tolerance)
if st.Before(lowMark) {
// Send point by itself since it won't get a match.
j.sendSpecificPoint(buf[i], groupErrs)
} else {
break
}
}
// Remove all sent points.
j.specificGroupsBuffer[groupId] = buf[i:]
}
if len(p.p.PointDimensions().TagNames) > len(j.j.Dimensions) {
// We have a specific point and three options:
// 1. Find the cached match point and send both to group.
// 2. Cache the specific point for later.
// 3. Send the specific point alone if it is no longer possible that a match will arrive.
// Search for a match.
// Also purge any old match points.
matches := j.matchGroupsBuffer[groupId]
matched := false
var i int
l := len(matches)
for i = 0; i < l; i++ {
match := matches[i]
pt := match.p.PointTime().Round(j.j.Tolerance)
if pt.Equal(t) {
// Option 1, send both points
j.sendMatchPoint(p, match, groupErrs)
matched = true
}
if !pt.Before(lowMark) {
break
}
}
if j.allReported {
// Can't trust lowMark until all parents have reported.
// Remove any unneeded match points.
j.matchGroupsBuffer[groupId] = matches[i:]
}
// If the point didn't match that leaves us with options 2 and 3.
if !matched {
if j.allReported && t.Before(lowMark) {
// Option 3
// Send this specific point by itself since it won't get a match.
j.sendSpecificPoint(p, groupErrs)
} else {
// Option 2
// Cache this point for when its match arrives.
j.specificGroupsBuffer[groupId] = append(j.specificGroupsBuffer[groupId], p)
}
}
} else {
// Cache match point.
j.matchGroupsBuffer[groupId] = append(j.matchGroupsBuffer[groupId], p)
// Send all specific points that match, to the group.
var i int
buf := j.specificGroupsBuffer[groupId]
l := len(buf)
for i = 0; i < l; i++ {
st := buf[i].p.PointTime().Round(j.j.Tolerance)
if st.Equal(t) {
j.sendMatchPoint(buf[i], p, groupErrs)
} else {
break
}
}
// Remove all sent points
j.specificGroupsBuffer[groupId] = buf[i:]
}
}
// Add the specific tags from the specific point to the matched point
// and then send both on to the group.
func (j *JoinNode) sendMatchPoint(specific, matched srcPoint, groupErrs chan<- error) {
np := matched.p.Copy().Setter()
for key, value := range specific.p.PointTags() {
np.SetNewDimTag(key, value)
}
np.UpdateGroup()
group := j.getGroup(specific.p, groupErrs)
// Send current point
group.points <- specific
// Send new matched point
matched.p = np.Interface()
group.points <- matched
}
// Send only the specific point to the group
func (j *JoinNode) sendSpecificPoint(specific srcPoint, groupErrs chan<- error) {
group := j.getGroup(specific.p, groupErrs)
// Send current point
group.points <- specific
}
// safely get the group for the point or create one if it doesn't exist.
func (j *JoinNode) getGroup(p models.PointInterface, groupErrs chan<- error) *group {
j.groupsMu.RLock()
group := j.groups[p.PointGroup()]
j.groupsMu.RUnlock()
if group == nil {
group = newGroup(len(j.ins), j)
j.groupsMu.Lock()
j.groups[p.PointGroup()] = group
j.runningGroups.Add(1)
j.groupsMu.Unlock()
go func() {
err := group.run()
if err != nil {
j.incrementErrorCount()
j.logger.Println("E! join group error:", err)
select {
case groupErrs <- err:
default:
}
}
}()
}
return group
}
// A groupId and its parent
type srcGroup struct {
src int
groupId models.GroupID
}
// represents an incoming data point and which parent it came from
type srcPoint struct {
src int
p models.PointInterface
}
// handles emitting joined sets once enough data has arrived from parents.
type group struct {
sets map[time.Time][]*joinset
head []time.Time
oldestTime time.Time
j *JoinNode
points chan srcPoint
}
func newGroup(i int, j *JoinNode) *group {
return &group{
sets: make(map[time.Time][]*joinset),
head: make([]time.Time, i),
j: j,
points: make(chan srcPoint),
}
}
// start consuming incoming points
func (g *group) run() error {
defer g.j.runningGroups.Done()
for sp := range g.points {
err := g.collect(sp.src, sp.p)
if err != nil {
return err
}
}
return nil
}
// collect a point from a given parent.
// emit the oldest set if we have collected enough data.
func (g *group) collect(i int, p models.PointInterface) error {
t := p.PointTime().Round(g.j.j.Tolerance)
if t.Before(g.oldestTime) || g.oldestTime.IsZero() {
g.oldestTime = t
}
var set *joinset
sets := g.sets[t]
if len(sets) == 0 {
set = newJoinset(
g.j,
g.j.j.StreamName,
g.j.fill,
g.j.fillValue,
g.j.j.Names,
g.j.j.Delimiter,
g.j.j.Tolerance,
t,
g.j.logger,
)
sets = append(sets, set)
g.sets[t] = sets
}
for j := 0; j < len(sets); j++ {
if !sets[j].Has(i) {
set = sets[j]
break
}
}
if set == nil {
set = newJoinset(
g.j,
g.j.j.StreamName,
g.j.fill,
g.j.fillValue,
g.j.j.Names,
g.j.j.Delimiter,
g.j.j.Tolerance,
t,
g.j.logger,
)
sets = append(sets, set)
g.sets[t] = sets
}
set.Set(i, p)
// Update head
g.head[i] = t
onlyReadySets := false
for _, t := range g.head {
if !t.After(g.oldestTime) {
onlyReadySets = true
break
}
}
err := g.emit(onlyReadySets)
if err != nil {
return err
}
return nil
}
// emit a set and update the oldestTime.
func (g *group) emit(onlyReadySets bool) error {
sets := g.sets[g.oldestTime]
i := 0
for ; i < len(sets); i++ {
if sets[i].Ready() || !onlyReadySets {
err := g.emitJoinedSet(sets[i])
if err != nil {
return err
}
} else {
break
}
}
if i == len(sets) {
delete(g.sets, g.oldestTime)
} else {
g.sets[g.oldestTime] = sets[i:]
}
g.oldestTime = time.Time{}
for t := range g.sets {
if g.oldestTime.IsZero() || t.Before(g.oldestTime) {
g.oldestTime = t
}
}
return nil
}
// emit sets until we have none left.
func (g *group) emitAll() error {
var lastErr error
for len(g.sets) > 0 {
err := g.emit(false)
if err != nil {
lastErr = err
}
}
return lastErr
}
// emit a single joined set
func (g *group) emitJoinedSet(set *joinset) error {
if set.name == "" {
set.name = set.First().PointName()
}
switch g.j.Wants() {
case pipeline.StreamEdge:
p, ok := set.JoinIntoPoint()
if ok {
for _, out := range g.j.outs {
err := out.CollectPoint(p)
if err != nil {
return err
}
}
}
case pipeline.BatchEdge:
b, ok := set.JoinIntoBatch()
if ok {
for _, out := range g.j.outs {
err := out.CollectBatch(b)
if err != nil {
return err
}
}
}
}
return nil
}
// represents a set of points or batches from the same joined time
type joinset struct {
j *JoinNode
name string
fill influxql.FillOption
fillValue interface{}
prefixes []string
delimiter string
time time.Time
tolerance time.Duration
values []models.PointInterface
expected int
size int
finished int
first int
logger *log.Logger
}
func newJoinset(
n *JoinNode,
name string,
fill influxql.FillOption,
fillValue interface{},
prefixes []string,
delimiter string,
tolerance time.Duration,
time time.Time,
l *log.Logger,
) *joinset {
expected := len(prefixes)
return &joinset{
j: n,
name: name,
fill: fill,
fillValue: fillValue,
prefixes: prefixes,
delimiter: delimiter,
expected: expected,
values: make([]models.PointInterface, expected),
first: expected,
time: time,
tolerance: tolerance,
logger: l,
}
}
func (js *joinset) Ready() bool {
return js.size == js.expected
}
func (js *joinset) Has(i int) bool {
return js.values[i] != nil
}
// add a point to the set from a given parent index.
func (js *joinset) Set(i int, v models.PointInterface) {
if i < js.first {
js.first = i
}
js.values[i] = v
js.size++
}
// a valid point in the set
func (js *joinset) First() models.PointInterface {
return js.values[js.first]
}
// join all points into a single point
func (js *joinset) JoinIntoPoint() (models.Point, bool) {
fields := make(models.Fields, js.size*len(js.First().PointFields()))
for i, p := range js.values {
if p == nil {
switch js.fill {
case influxql.NullFill:
for k := range js.First().PointFields() {
fields[js.prefixes[i]+js.delimiter+k] = nil
}
case influxql.NumberFill:
for k := range js.First().PointFields() {
fields[js.prefixes[i]+js.delimiter+k] = js.fillValue
}
default:
// inner join no valid point possible
return models.Point{}, false
}
} else {
for k, v := range p.PointFields() {
fields[js.prefixes[i]+js.delimiter+k] = v
}
}
}
p := models.Point{
Name: js.name,
Group: js.First().PointGroup(),
Tags: js.First().PointTags(),
Dimensions: js.First().PointDimensions(),
Time: js.time,
Fields: fields,
}
return p, true
}
// join all batches the set into a single batch
func (js *joinset) JoinIntoBatch() (models.Batch, bool) {
newBatch := models.Batch{
Name: js.name,
Group: js.First().PointGroup(),
Tags: js.First().PointTags(),
ByName: js.First().PointDimensions().ByName,
TMax: js.time,
}
empty := make([]bool, js.expected)
emptyCount := 0
indexes := make([]int, js.expected)
var fieldNames []string
BATCH_POINT:
for emptyCount < js.expected {
set := make([]*models.BatchPoint, js.expected)
setTime := time.Time{}
count := 0
for i, batch := range js.values {
if empty[i] {
continue
}
if batch == nil {
emptyCount++
empty[i] = true
continue
}
b, ok := batch.(models.Batch)
if !ok {
js.j.incrementErrorCount()
js.logger.Printf("E! invalid join data got %T expected models.Batch", batch)
return models.Batch{}, false
}
if indexes[i] == len(b.Points) {
emptyCount++
empty[i] = true
continue
}
bp := b.Points[indexes[i]]
t := bp.Time.Round(js.tolerance)
if setTime.IsZero() {
setTime = t
}
if t.Before(setTime) {
// We need to backup
setTime = t
for j := range set {
if set[j] != nil {
indexes[j]--
}
set[j] = nil
}
set[i] = &bp
indexes[i]++
count = 1
} else if t.Equal(setTime) {
if fieldNames == nil {
for k := range bp.Fields {
fieldNames = append(fieldNames, k)
}
}
set[i] = &bp
indexes[i]++
count++
}
}
// we didn't get any points from any group we must be empty
// skip this set
if count == 0 {
continue
}
// Join all batch points in set
fields := make(models.Fields, js.expected*len(fieldNames))
for i, bp := range set {
if bp == nil {
switch js.fill {
case influxql.NullFill:
for _, k := range fieldNames {
fields[js.prefixes[i]+js.delimiter+k] = nil
}
case influxql.NumberFill:
for _, k := range fieldNames {
fields[js.prefixes[i]+js.delimiter+k] = js.fillValue
}
default:
// inner join no valid point possible
continue BATCH_POINT
}
} else {
for k, v := range bp.Fields {
fields[js.prefixes[i]+js.delimiter+k] = v
}
}
}
bp := models.BatchPoint{
Tags: newBatch.Tags,
Time: setTime,
Fields: fields,
}
newBatch.Points = append(newBatch.Points, bp)
}
return newBatch, true
}
type durationVar struct {
expvar.Int
}
func (d *durationVar) String() string {
return time.Duration(d.IntValue()).String()
}