forked from influxdata/kapacitor
-
Notifications
You must be signed in to change notification settings - Fork 0
/
batch.go
392 lines (345 loc) · 7.56 KB
/
batch.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
package kapacitor
import (
"errors"
"fmt"
"log"
"sync"
"time"
"github.com/gorhill/cronexpr"
"github.com/influxdata/kapacitor/models"
"github.com/influxdata/kapacitor/pipeline"
"github.com/influxdb/influxdb/client"
"github.com/influxdb/influxdb/influxql"
)
type SourceBatchNode struct {
node
s *pipeline.SourceBatchNode
idx int
}
func newSourceBatchNode(et *ExecutingTask, n *pipeline.SourceBatchNode, l *log.Logger) (*SourceBatchNode, error) {
sn := &SourceBatchNode{
node: node{Node: n, et: et, logger: l},
s: n,
}
return sn, nil
}
func (s *SourceBatchNode) linkChild(c Node) error {
// add child
if s.Provides() != c.Wants() {
return fmt.Errorf("cannot add child mismatched edges: %s -> %s", s.Provides(), c.Wants())
}
s.children = append(s.children, c)
// add parent
c.addParent(s)
return nil
}
func (s *SourceBatchNode) addParentEdge(in *Edge) {
// Pass edges down to children
s.children[s.idx].addParentEdge(in)
s.idx++
}
func (s *SourceBatchNode) start([]byte) {
}
func (s *SourceBatchNode) Err() error {
return nil
}
// Return list of databases and retention policies
// the batcher will query.
func (s *SourceBatchNode) DBRPs() ([]DBRP, error) {
var dbrps []DBRP
for _, b := range s.children {
d, err := b.(*BatchNode).DBRPs()
if err != nil {
return nil, err
}
dbrps = append(dbrps, d...)
}
return dbrps, nil
}
func (s *SourceBatchNode) Count() int {
return len(s.children)
}
func (s *SourceBatchNode) Start() {
for _, b := range s.children {
b.(*BatchNode).Start()
}
}
func (s *SourceBatchNode) Queries(start, stop time.Time) [][]string {
queries := make([][]string, len(s.children))
for i, b := range s.children {
queries[i] = b.(*BatchNode).Queries(start, stop)
}
return queries
}
type BatchNode struct {
node
b *pipeline.BatchNode
query *Query
ticker ticker
queryMu sync.Mutex
queryErr chan error
closing chan struct{}
}
func newBatchNode(et *ExecutingTask, n *pipeline.BatchNode, l *log.Logger) (*BatchNode, error) {
bn := &BatchNode{
node: node{Node: n, et: et, logger: l},
b: n,
closing: make(chan struct{}),
}
bn.node.runF = bn.runBatch
bn.node.stopF = bn.stopBatch
// Create query
q, err := NewQuery(n.QueryStr)
if err != nil {
return nil, err
}
bn.query = q
// Add in dimensions
err = bn.query.Dimensions(n.Dimensions)
if err != nil {
return nil, err
}
// Set fill
switch fill := n.Fill.(type) {
case string:
switch fill {
case "null":
bn.query.Fill(influxql.NullFill, nil)
case "none":
bn.query.Fill(influxql.NoFill, nil)
case "previous":
bn.query.Fill(influxql.PreviousFill, nil)
default:
return nil, fmt.Errorf("unexpected fill option %s", fill)
}
case int64, float64:
bn.query.Fill(influxql.NumberFill, fill)
}
// Determine schedule
if n.Every != 0 && n.Cron != "" {
return nil, errors.New("must not set both 'every' and 'cron' properties")
}
switch {
case n.Every != 0:
bn.ticker = newTimeTicker(n.Every)
case n.Cron != "":
var err error
bn.ticker, err = newCronTicker(n.Cron)
if err != nil {
return nil, err
}
default:
return nil, errors.New("must define one of 'every' or 'cron'")
}
if n.Every != 0 && n.Cron != "" {
return nil, errors.New("must not set both 'every' and 'cron' properties")
}
switch {
case n.Every != 0:
bn.ticker = newTimeTicker(n.Every)
case n.Cron != "":
var err error
bn.ticker, err = newCronTicker(n.Cron)
if err != nil {
return nil, err
}
default:
return nil, errors.New("must define one of 'every' or 'cron'")
}
return bn, nil
}
// Return list of databases and retention policies
// the batcher will query.
func (b *BatchNode) DBRPs() ([]DBRP, error) {
return b.query.DBRPs()
}
func (b *BatchNode) Start() {
b.queryMu.Lock()
defer b.queryMu.Unlock()
b.queryErr = make(chan error, 1)
go func() {
b.queryErr <- b.doQuery()
}()
}
func (b *BatchNode) Queries(start, stop time.Time) []string {
now := time.Now()
if stop.IsZero() {
stop = now
}
// Crons are sensitive to timezones.
// Make sure we are using local time.
start = start.Local()
queries := make([]string, 0)
for {
start = b.ticker.Next(start)
if start.IsZero() || start.After(stop) {
break
}
b.query.Start(start)
qstop := start.Add(b.b.Period)
if qstop.After(now) {
break
}
b.query.Stop(qstop)
queries = append(queries, b.query.String())
}
return queries
}
// Query InfluxDB and collect batches on batch collector.
func (b *BatchNode) doQuery() error {
defer b.ins[0].Close()
if b.et.tm.InfluxDBService == nil {
return errors.New("InfluxDB not configured, cannot query InfluxDB for batch query")
}
tickC := b.ticker.Start()
for {
select {
case <-b.closing:
return nil
case now := <-tickC:
// Update times for query
stop := now.Add(-1 * b.b.Offset)
b.query.Start(stop.Add(-1 * b.b.Period))
b.query.Stop(stop)
b.logger.Println("D! starting next batch query:", b.query.String())
// Connect
con, err := b.et.tm.InfluxDBService.NewClient()
if err != nil {
return err
}
q := client.Query{
Command: b.query.String(),
}
// Execute query
resp, err := con.Query(q)
if err != nil {
return err
}
if resp.Err != nil {
return resp.Err
}
// Collect batches
for _, res := range resp.Results {
batches, err := models.ResultToBatches(res)
if err != nil {
return err
}
for _, bch := range batches {
b.ins[0].CollectBatch(bch)
}
}
}
}
}
func (b *BatchNode) runBatch([]byte) error {
errC := make(chan error, 1)
go func() {
defer func() {
err := recover()
if err != nil {
errC <- fmt.Errorf("%v", err)
}
}()
for bt, ok := b.ins[0].NextBatch(); ok; bt, ok = b.ins[0].NextBatch() {
for _, child := range b.outs {
err := child.CollectBatch(bt)
if err != nil {
errC <- err
return
}
}
}
errC <- nil
}()
var queryErr error
b.queryMu.Lock()
if b.queryErr != nil {
b.queryMu.Unlock()
queryErr = <-b.queryErr
} else {
b.queryMu.Unlock()
}
err := <-errC
if queryErr != nil {
return queryErr
}
return err
}
func (b *BatchNode) stopBatch() {
if b.ticker != nil {
b.ticker.Stop()
}
close(b.closing)
}
type ticker interface {
Start() <-chan time.Time
Stop()
// Return the next time the ticker will tick after now.
Next(now time.Time) time.Time
}
type timeTicker struct {
every time.Duration
ticker *time.Ticker
mu sync.Mutex
}
func newTimeTicker(every time.Duration) *timeTicker {
return &timeTicker{every: every}
}
func (t *timeTicker) Start() <-chan time.Time {
t.mu.Lock()
defer t.mu.Unlock()
t.ticker = time.NewTicker(t.every)
return t.ticker.C
}
func (t *timeTicker) Stop() {
t.mu.Lock()
defer t.mu.Unlock()
if t.ticker != nil {
t.ticker.Stop()
}
}
func (t *timeTicker) Next(now time.Time) time.Time {
return now.Add(t.every)
}
type cronTicker struct {
expr *cronexpr.Expression
ticker chan time.Time
closing chan struct{}
wg sync.WaitGroup
}
func newCronTicker(cronExpr string) (*cronTicker, error) {
expr, err := cronexpr.Parse(cronExpr)
if err != nil {
return nil, err
}
return &cronTicker{
expr: expr,
ticker: make(chan time.Time),
closing: make(chan struct{}),
}, nil
}
func (c *cronTicker) Start() <-chan time.Time {
c.wg.Add(1)
go func() {
defer c.wg.Done()
for {
now := time.Now()
next := c.expr.Next(now)
diff := next.Sub(now)
select {
case <-time.After(diff):
c.ticker <- next
case <-c.closing:
return
}
}
}()
return c.ticker
}
func (c *cronTicker) Stop() {
close(c.closing)
c.wg.Wait()
}
func (c *cronTicker) Next(now time.Time) time.Time {
return c.expr.Next(now)
}