forked from redpanda-data/connect
-
Notifications
You must be signed in to change notification settings - Fork 0
/
read_until.go
424 lines (372 loc) · 12.1 KB
/
read_until.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
package input
import (
"encoding/json"
"errors"
"fmt"
"sync/atomic"
"time"
"github.com/cenkalti/backoff/v4"
"github.com/dafanshu/benthos/v3/internal/bloblang/mapping"
"github.com/dafanshu/benthos/v3/internal/docs"
"github.com/dafanshu/benthos/v3/internal/interop"
"github.com/dafanshu/benthos/v3/lib/condition"
"github.com/dafanshu/benthos/v3/lib/log"
"github.com/dafanshu/benthos/v3/lib/metrics"
"github.com/dafanshu/benthos/v3/lib/types"
"github.com/google/go-cmp/cmp"
"gopkg.in/yaml.v3"
)
//------------------------------------------------------------------------------
func init() {
Constructors[TypeReadUntil] = TypeSpec{
constructor: fromSimpleConstructor(NewReadUntil),
Summary: `
Reads messages from a child input until a consumed message passes a [Bloblang query](/docs/guides/bloblang/about/), at which point the input closes.`,
Description: `
Messages are read continuously while the query check returns false, when the query returns true the message that triggered the check is sent out and the input is closed. Use this to define inputs where the stream should end once a certain message appears.
Sometimes inputs close themselves. For example, when the ` + "`file`" + ` input type reaches the end of a file it will shut down. By default this type will also shut down. If you wish for the input type to be restarted every time it shuts down until the query check is met then set ` + "`restart_input` to `true`." + `
### Metadata
A metadata key ` + "`benthos_read_until` containing the value `final`" + ` is added to the first part of the message that triggers the input to stop.`,
Examples: []docs.AnnotatedExample{
{
Title: "Consume N Messages",
Summary: "A common reason to use this input is to consume only N messages from an input and then stop. This can easily be done with the [`count` function](/docs/guides/bloblang/functions/#count):",
Config: `
# Only read 100 messages, and then exit.
input:
read_until:
check: count("messages") >= 100
input:
kafka:
addresses: [ TODO ]
topics: [ foo, bar ]
consumer_group: foogroup
`,
},
},
FieldSpecs: docs.FieldSpecs{
docs.FieldCommon("input", "The child input to consume from.").HasType(docs.FieldTypeInput),
docs.FieldBloblang(
"check",
"A [Bloblang query](/docs/guides/bloblang/about/) that should return a boolean value indicating whether the input should now be closed.",
`this.type == "foo"`,
`count("messages") >= 100`,
).HasDefault(""),
docs.FieldDeprecated("condition").HasType(docs.FieldTypeCondition).OmitWhen(func(field, _ interface{}) (string, bool) {
defaultBytes, err := yaml.Marshal(condition.NewConfig())
if err != nil {
return "", false
}
var iDefault interface{}
if err = yaml.Unmarshal(defaultBytes, &iDefault); err != nil {
return "", false
}
return "the condition field is deprecated in favour of check", cmp.Equal(field, iDefault)
}),
docs.FieldCommon("restart_input", "Whether the input should be reopened if it closes itself before the condition has resolved to true."),
docs.FieldCommon("loop_forever", "Whether the input should be stop when check is true."),
docs.FieldCommon("loop_interval", "When check is true, how often to check again.").HasDefault("60s"),
},
Categories: []Category{
CategoryUtility,
},
}
}
func isDefaultCond(cond condition.Config) bool {
return cmp.Equal(cond, condition.NewConfig())
}
//------------------------------------------------------------------------------
// ReadUntilConfig contains configuration values for the ReadUntil input type.
type ReadUntilConfig struct {
Input *Config `json:"input" yaml:"input"`
Restart bool `json:"restart_input" yaml:"restart_input"`
LoopForever bool `json:"loop_forever" yaml:"loop_forever"`
LoopInterval string `json:"loop_interval" yaml:"loop_interval"`
Condition condition.Config `json:"condition" yaml:"condition"`
Check string `json:"check" yaml:"check"`
}
// NewReadUntilConfig creates a new ReadUntilConfig with default values.
func NewReadUntilConfig() ReadUntilConfig {
return ReadUntilConfig{
Input: nil,
Restart: false,
LoopForever: false,
LoopInterval: "60s",
Condition: condition.NewConfig(),
Check: "",
}
}
//------------------------------------------------------------------------------
type dummyReadUntilConfig struct {
Input interface{} `json:"input" yaml:"input"`
Restart bool `json:"restart_input" yaml:"restart_input"`
LoopForever bool `json:"loop_forever" yaml:"loop_forever"`
LoopInterval string `json:"loop_interval" yaml:"loop_interval"`
Condition condition.Config `json:"condition" yaml:"condition"`
Check string `json:"check" yaml:"check"`
}
// MarshalJSON prints an empty object instead of nil.
func (r ReadUntilConfig) MarshalJSON() ([]byte, error) {
dummy := dummyReadUntilConfig{
Input: r.Input,
Restart: r.Restart,
Condition: r.Condition,
Check: r.Check,
LoopForever: r.LoopForever,
LoopInterval: r.LoopInterval,
}
if r.Input == nil {
dummy.Input = struct{}{}
}
return json.Marshal(dummy)
}
// MarshalYAML prints an empty object instead of nil.
func (r ReadUntilConfig) MarshalYAML() (interface{}, error) {
dummy := dummyReadUntilConfig{
Input: r.Input,
Restart: r.Restart,
Condition: r.Condition,
Check: r.Check,
LoopForever: r.LoopForever,
LoopInterval: r.LoopInterval,
}
if r.Input == nil {
dummy.Input = struct{}{}
}
return dummy, nil
}
//------------------------------------------------------------------------------
// ReadUntil is an input type that continuously reads another input type until a
// condition returns true on a message consumed.
type ReadUntil struct {
running int32
conf ReadUntilConfig
wrapped Type
cond condition.Type
check *mapping.Executor
wrapperMgr types.Manager
wrapperLog log.Modular
wrapperStats metrics.Type
stats metrics.Type
log log.Modular
transactions chan types.Transaction
closeChan chan struct{}
closedChan chan struct{}
}
// NewReadUntil creates a new ReadUntil input type.
func NewReadUntil(
conf Config,
mgr types.Manager,
log log.Modular,
stats metrics.Type,
) (Type, error) {
if conf.ReadUntil.Input == nil {
return nil, errors.New("cannot create read_until input without a child")
}
wrapped, err := New(
*conf.ReadUntil.Input, mgr, log, stats,
)
if err != nil {
return nil, fmt.Errorf("failed to create input '%v': %v", conf.ReadUntil.Input.Type, err)
}
var cond condition.Type
if !isDefaultCond(conf.ReadUntil.Condition) {
cMgr, cLog, cStats := interop.LabelChild("read_until.condition", mgr, log, stats)
if cond, err = condition.New(conf.ReadUntil.Condition, cMgr, cLog, cStats); err != nil {
return nil, fmt.Errorf("failed to create condition '%v': %v", conf.ReadUntil.Condition.Type, err)
}
}
var check *mapping.Executor
if len(conf.ReadUntil.Check) > 0 {
if check, err = interop.NewBloblangMapping(mgr, conf.ReadUntil.Check); err != nil {
return nil, fmt.Errorf("failed to parse check query: %w", err)
}
}
if cond == nil && check == nil {
return nil, errors.New("a check query is required")
}
if cond != nil && check != nil {
return nil, errors.New("cannot specify both a condition and a check query")
}
_, rLog, rStats := interop.LabelChild("read_until", mgr, log, stats)
rdr := &ReadUntil{
running: 1,
conf: conf.ReadUntil,
wrapperLog: log,
wrapperStats: stats,
wrapperMgr: mgr,
log: rLog,
stats: rStats,
wrapped: wrapped,
cond: cond,
check: check,
transactions: make(chan types.Transaction),
closeChan: make(chan struct{}),
closedChan: make(chan struct{}),
}
go rdr.loop()
return rdr, nil
}
//------------------------------------------------------------------------------
func (r *ReadUntil) loop() {
var (
mRunning = r.stats.GetGauge("running")
mRestartErr = r.stats.GetCounter("restart.error")
mRestartSucc = r.stats.GetCounter("restart.success")
mInputClosed = r.stats.GetCounter("input.closed")
mCount = r.stats.GetCounter("count")
mPropagated = r.stats.GetCounter("propagated")
mFinalPropagated = r.stats.GetCounter("final.propagated")
mFinalResSent = r.stats.GetCounter("final.response.sent")
mFinalResSucc = r.stats.GetCounter("final.response.success")
mFinalResErr = r.stats.GetCounter("final.response.error")
)
defer func() {
if r.wrapped != nil {
r.wrapped.CloseAsync()
// TODO: This triggers a tier 2 shutdown in the child.
err := r.wrapped.WaitForClose(time.Second)
for ; err != nil; err = r.wrapped.WaitForClose(time.Second) {
}
}
mRunning.Decr(1)
close(r.transactions)
close(r.closedChan)
}()
mRunning.Incr(1)
// Prevents busy loop when an input never yields messages.
restartBackoff := backoff.NewExponentialBackOff()
restartBackoff.InitialInterval = time.Millisecond
restartBackoff.MaxInterval = time.Millisecond * 100
restartBackoff.MaxElapsedTime = 0
var open bool
runLoop:
for atomic.LoadInt32(&r.running) == 1 {
if r.wrapped == nil {
if r.conf.Restart {
select {
case <-time.After(restartBackoff.NextBackOff()):
case <-r.closeChan:
return
}
var err error
if r.wrapped, err = New(
*r.conf.Input, r.wrapperMgr, r.wrapperLog, r.wrapperStats,
); err != nil {
mRestartErr.Incr(1)
r.log.Errorf("Failed to create input '%v': %v\n", r.conf.Input.Type, err)
return
}
mRestartSucc.Incr(1)
} else {
return
}
}
var tran types.Transaction
select {
case tran, open = <-r.wrapped.TransactionChan():
if !open {
mInputClosed.Incr(1)
r.wrapped = nil
continue runLoop
}
restartBackoff.Reset()
case <-r.closeChan:
return
}
mCount.Incr(1)
var check bool
if r.cond != nil {
check = r.cond.Check(tran.Payload)
} else {
var err error
if check, err = r.check.QueryPart(0, tran.Payload); err != nil {
check = false
r.log.Errorf("Failed to execute check query: %v\n", err)
}
}
if !check {
select {
case r.transactions <- tran:
mPropagated.Incr(1)
case <-r.closeChan:
return
}
continue
}
tran.Payload.Get(0).Metadata().Set("benthos_read_until", "final")
// If this transaction succeeds we shut down.
tmpRes := make(chan types.Response)
select {
case r.transactions <- types.NewTransaction(tran.Payload, tmpRes):
mFinalPropagated.Incr(1)
case <-r.closeChan:
break
}
var res types.Response
select {
case res, open = <-tmpRes:
if !open {
break
}
streamEnds := res.Error() == nil
select {
case tran.ResponseChan <- res:
mFinalResSent.Incr(1)
case <-r.closeChan:
break
}
if streamEnds {
mFinalResSucc.Incr(1)
break
}
mFinalResErr.Incr(1)
case <-r.closeChan:
break
}
if r.conf.LoopForever {
for {
var err error
duration, err := time.ParseDuration(r.conf.LoopInterval)
if err != nil {
duration = 60 * time.Second
}
time.Sleep(duration)
if check, err = r.check.QueryPart(0, tran.Payload); err != nil {
check = false
r.log.Errorf("Failed to execute check query: %v\n", err)
}
if !check {
break
}
}
continue
}
}
}
// TransactionChan returns a transactions channel for consuming messages from
// this input type.
func (r *ReadUntil) TransactionChan() <-chan types.Transaction {
return r.transactions
}
// Connected returns a boolean indicating whether this input is currently
// connected to its target.
func (r *ReadUntil) Connected() bool {
return r.wrapped.Connected()
}
// CloseAsync shuts down the ReadUntil input and stops processing requests.
func (r *ReadUntil) CloseAsync() {
if atomic.CompareAndSwapInt32(&r.running, 1, 0) {
close(r.closeChan)
}
}
// WaitForClose blocks until the ReadUntil input has closed down.
func (r *ReadUntil) WaitForClose(timeout time.Duration) error {
select {
case <-r.closedChan:
case <-time.After(timeout):
return types.ErrTimeout
}
return nil
}
//------------------------------------------------------------------------------