forked from benthosdev/benthos
/
input_sqs.go
484 lines (426 loc) · 12.5 KB
/
input_sqs.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
package aws
import (
"context"
"sync"
"time"
"github.com/aws/aws-sdk-go/aws"
"github.com/aws/aws-sdk-go/aws/awserr"
"github.com/aws/aws-sdk-go/aws/request"
"github.com/aws/aws-sdk-go/aws/session"
"github.com/aws/aws-sdk-go/service/sqs"
"github.com/cenkalti/backoff/v4"
"github.com/benthosdev/benthos/v4/internal/component"
"github.com/benthosdev/benthos/v4/internal/impl/aws/config"
"github.com/benthosdev/benthos/v4/internal/shutdown"
"github.com/benthosdev/benthos/v4/public/service"
)
const (
// SQS Input Fields
sqsiFieldURL = "url"
sqsiFieldWaitTimeSeconds = "wait_time_seconds"
sqsiFieldDeleteMessage = "delete_message"
sqsiFieldResetVisibility = "reset_visibility"
sqsiFieldMaxNumberOfMessages = "max_number_of_messages"
)
type sqsiConfig struct {
URL string
WaitTimeSeconds int
DeleteMessage bool
ResetVisibility bool
MaxNumberOfMessages int
}
func sqsiConfigFromParsed(pConf *service.ParsedConfig) (conf sqsiConfig, err error) {
if conf.URL, err = pConf.FieldString(sqsiFieldURL); err != nil {
return
}
if conf.WaitTimeSeconds, err = pConf.FieldInt(sqsiFieldWaitTimeSeconds); err != nil {
return
}
if conf.DeleteMessage, err = pConf.FieldBool(sqsiFieldDeleteMessage); err != nil {
return
}
if conf.ResetVisibility, err = pConf.FieldBool(sqsiFieldResetVisibility); err != nil {
return
}
if conf.MaxNumberOfMessages, err = pConf.FieldInt(sqsiFieldMaxNumberOfMessages); err != nil {
return
}
return
}
func sqsInputSpec() *service.ConfigSpec {
return service.NewConfigSpec().
Stable().
Categories("Services", "AWS").
Summary(`Consume messages from an AWS SQS URL.`).
Description(`
### Credentials
By default Benthos will use a shared credentials file when connecting to AWS
services. It's also possible to set them explicitly at the component level,
allowing you to transfer data across accounts. You can find out more
[in this document](/docs/guides/cloud/aws).
### Metadata
This input adds the following metadata fields to each message:
`+"```text"+`
- sqs_message_id
- sqs_receipt_handle
- sqs_approximate_receive_count
- All message attributes
`+"```"+`
You can access these metadata fields using
[function interpolation](/docs/configuration/interpolation#bloblang-queries).`).
Fields(
service.NewURLField(sqsiFieldURL).
Description("The SQS URL to consume from."),
service.NewBoolField(sqsiFieldDeleteMessage).
Description("Whether to delete the consumed message once it is acked. Disabling allows you to handle the deletion using a different mechanism.").
Default(true).
Advanced(),
service.NewBoolField(sqsiFieldResetVisibility).
Description("Whether to set the visibility timeout of the consumed message to zero once it is nacked. Disabling honors the preset visibility timeout specified for the queue.").
Version("3.58.0").
Default(true).
Advanced(),
service.NewIntField(sqsiFieldMaxNumberOfMessages).
Description("The maximum number of messages to return on one poll. Valid values: 1 to 10.").
Default(10).
Advanced(),
service.NewIntField("wait_time_seconds").
Description("Whether to set the wait time. Enabling this activates long-polling. Valid values: 0 to 20.").
Default(0).
Advanced(),
).
Fields(config.SessionFields()...)
}
func init() {
err := service.RegisterInput("aws_sqs", sqsInputSpec(),
func(pConf *service.ParsedConfig, mgr *service.Resources) (service.Input, error) {
sess, err := GetSession(pConf)
if err != nil {
return nil, err
}
conf, err := sqsiConfigFromParsed(pConf)
if err != nil {
return nil, err
}
return newAWSSQSReader(conf, sess, mgr.Logger())
})
if err != nil {
panic(err)
}
}
//------------------------------------------------------------------------------
type awsSQSReader struct {
conf sqsiConfig
session *session.Session
sqs *sqs.SQS
messagesChan chan *sqs.Message
ackMessagesChan chan sqsMessageHandle
nackMessagesChan chan sqsMessageHandle
closeSignal *shutdown.Signaller
log *service.Logger
}
func newAWSSQSReader(conf sqsiConfig, sess *session.Session, log *service.Logger) (*awsSQSReader, error) {
return &awsSQSReader{
conf: conf,
session: sess,
log: log,
messagesChan: make(chan *sqs.Message),
ackMessagesChan: make(chan sqsMessageHandle),
nackMessagesChan: make(chan sqsMessageHandle),
closeSignal: shutdown.NewSignaller(),
}, nil
}
// Connect attempts to establish a connection to the target SQS
// queue.
func (a *awsSQSReader) Connect(ctx context.Context) error {
if a.sqs != nil {
return nil
}
a.sqs = sqs.New(a.session)
var wg sync.WaitGroup
wg.Add(2)
go a.readLoop(&wg)
go a.ackLoop(&wg)
go func() {
wg.Wait()
a.closeSignal.ShutdownComplete()
}()
a.log.Infof("Receiving Amazon SQS messages from URL: %v\n", a.conf.URL)
return nil
}
func (a *awsSQSReader) ackLoop(wg *sync.WaitGroup) {
defer wg.Done()
var pendingAcks []sqsMessageHandle
var pendingNacks []sqsMessageHandle
flushAcks := func() {
tmpAcks := pendingAcks
pendingAcks = nil
if len(tmpAcks) == 0 {
return
}
ctx, done := a.closeSignal.CloseNowCtx(context.Background())
defer done()
if err := a.deleteMessages(ctx, tmpAcks...); err != nil {
a.log.Errorf("Failed to delete messages: %v", err)
}
}
flushNacks := func() {
tmpNacks := pendingNacks
pendingNacks = nil
if len(tmpNacks) == 0 {
return
}
ctx, done := a.closeSignal.CloseNowCtx(context.Background())
defer done()
if err := a.resetMessages(ctx, tmpNacks...); err != nil {
a.log.Errorf("Failed to reset the visibility timeout of messages: %v", err)
}
}
flushTimer := time.NewTicker(time.Second)
defer flushTimer.Stop()
ackLoop:
for {
select {
case h := <-a.ackMessagesChan:
pendingAcks = append(pendingAcks, h)
if len(pendingAcks) >= a.conf.MaxNumberOfMessages {
flushAcks()
}
case h := <-a.nackMessagesChan:
pendingNacks = append(pendingNacks, h)
if len(pendingNacks) >= a.conf.MaxNumberOfMessages {
flushNacks()
}
case <-flushTimer.C:
flushAcks()
flushNacks()
case <-a.closeSignal.CloseAtLeisureChan():
break ackLoop
}
}
flushAcks()
flushNacks()
}
func (a *awsSQSReader) readLoop(wg *sync.WaitGroup) {
defer wg.Done()
var pendingMsgs []*sqs.Message
defer func() {
if len(pendingMsgs) > 0 {
tmpNacks := make([]sqsMessageHandle, 0, len(pendingMsgs))
for _, m := range pendingMsgs {
if m.MessageId == nil || m.ReceiptHandle == nil {
continue
}
tmpNacks = append(tmpNacks, sqsMessageHandle{
id: *m.MessageId,
receiptHandle: *m.ReceiptHandle,
})
}
ctx, done := a.closeSignal.CloseNowCtx(context.Background())
defer done()
if err := a.resetMessages(ctx, tmpNacks...); err != nil {
a.log.Errorf("Failed to reset visibility timeout for pending messages: %v", err)
}
}
}()
backoff := backoff.NewExponentialBackOff()
backoff.InitialInterval = 10 * time.Millisecond
backoff.MaxInterval = time.Minute
backoff.MaxElapsedTime = 0
getMsgs := func() {
ctx, done := a.closeSignal.CloseAtLeisureCtx(context.Background())
defer done()
res, err := a.sqs.ReceiveMessageWithContext(ctx, &sqs.ReceiveMessageInput{
QueueUrl: aws.String(a.conf.URL),
MaxNumberOfMessages: aws.Int64(int64(a.conf.MaxNumberOfMessages)),
WaitTimeSeconds: aws.Int64(int64(a.conf.WaitTimeSeconds)),
AttributeNames: []*string{aws.String("All")},
MessageAttributeNames: []*string{aws.String("All")},
})
if err != nil {
if aerr, ok := err.(awserr.Error); !ok || aerr.Code() != request.CanceledErrorCode {
a.log.Errorf("Failed to pull new SQS messages: %v", aerr)
}
return
}
if len(res.Messages) > 0 {
pendingMsgs = append(pendingMsgs, res.Messages...)
}
if len(res.Messages) > 0 || a.conf.WaitTimeSeconds > 0 {
// When long polling we want to reset our back off even if we didn't
// receive messages. However, with long polling disabled we back off
// each time we get an empty response.
backoff.Reset()
}
}
for {
if len(pendingMsgs) == 0 {
getMsgs()
if len(pendingMsgs) == 0 {
select {
case <-time.After(backoff.NextBackOff()):
case <-a.closeSignal.CloseAtLeisureChan():
return
}
continue
}
}
select {
case a.messagesChan <- pendingMsgs[0]:
pendingMsgs = pendingMsgs[1:]
case <-a.closeSignal.CloseAtLeisureChan():
return
}
}
}
type sqsMessageHandle struct {
id, receiptHandle string
}
func (a *awsSQSReader) deleteMessages(ctx context.Context, msgs ...sqsMessageHandle) error {
for len(msgs) > 0 {
input := sqs.DeleteMessageBatchInput{
QueueUrl: aws.String(a.conf.URL),
Entries: []*sqs.DeleteMessageBatchRequestEntry{},
}
for _, msg := range msgs {
input.Entries = append(input.Entries, &sqs.DeleteMessageBatchRequestEntry{
Id: aws.String(msg.id),
ReceiptHandle: aws.String(msg.receiptHandle),
})
if len(input.Entries) == a.conf.MaxNumberOfMessages {
break
}
}
msgs = msgs[len(input.Entries):]
response, err := a.sqs.DeleteMessageBatchWithContext(ctx, &input)
if err != nil {
return err
}
for _, fail := range response.Failed {
a.log.Errorf("Failed to delete consumed SQS message '%v', response code: %v\n", *fail.Id, *fail.Code)
}
}
return nil
}
func (a *awsSQSReader) resetMessages(ctx context.Context, msgs ...sqsMessageHandle) error {
if !a.conf.ResetVisibility {
return nil
}
for len(msgs) > 0 {
input := sqs.ChangeMessageVisibilityBatchInput{
QueueUrl: aws.String(a.conf.URL),
Entries: []*sqs.ChangeMessageVisibilityBatchRequestEntry{},
}
for _, msg := range msgs {
input.Entries = append(input.Entries, &sqs.ChangeMessageVisibilityBatchRequestEntry{
Id: aws.String(msg.id),
ReceiptHandle: aws.String(msg.receiptHandle),
VisibilityTimeout: aws.Int64(0),
})
if len(input.Entries) == a.conf.MaxNumberOfMessages {
break
}
}
msgs = msgs[len(input.Entries):]
response, err := a.sqs.ChangeMessageVisibilityBatchWithContext(ctx, &input)
if err != nil {
return err
}
for _, fail := range response.Failed {
a.log.Errorf("Failed to delete consumed SQS message '%v', response code: %v\n", *fail.Id, *fail.Code)
}
}
return nil
}
func addSQSMetadata(p *service.Message, sqsMsg *sqs.Message) {
p.MetaSetMut("sqs_message_id", *sqsMsg.MessageId)
p.MetaSetMut("sqs_receipt_handle", *sqsMsg.ReceiptHandle)
if rCountStr := sqsMsg.Attributes["ApproximateReceiveCount"]; rCountStr != nil {
p.MetaSetMut("sqs_approximate_receive_count", *rCountStr)
}
for k, v := range sqsMsg.MessageAttributes {
if v.StringValue != nil {
p.MetaSetMut(k, *v.StringValue)
}
}
}
// ReadBatch attempts to read a new message from the target SQS.
func (a *awsSQSReader) Read(ctx context.Context) (*service.Message, service.AckFunc, error) {
if a.session == nil {
return nil, nil, service.ErrNotConnected
}
var next *sqs.Message
var open bool
select {
case next, open = <-a.messagesChan:
if !open {
return nil, nil, component.ErrTypeClosed
}
case <-a.closeSignal.CloseAtLeisureChan():
return nil, nil, component.ErrTypeClosed
case <-ctx.Done():
return nil, nil, ctx.Err()
}
if next.Body == nil {
return nil, nil, component.ErrTimeout
}
msg := service.NewMessage([]byte(*next.Body))
addSQSMetadata(msg, next)
mHandle := sqsMessageHandle{
id: *next.MessageId,
}
if next.ReceiptHandle != nil {
mHandle.receiptHandle = *next.ReceiptHandle
}
return msg, func(rctx context.Context, res error) error {
if mHandle.receiptHandle == "" {
return nil
}
if res == nil {
if !a.conf.DeleteMessage {
return nil
}
select {
case <-rctx.Done():
return rctx.Err()
case <-a.closeSignal.CloseAtLeisureChan():
return a.deleteMessages(rctx, mHandle)
case a.ackMessagesChan <- mHandle:
}
return nil
}
select {
case <-rctx.Done():
return rctx.Err()
case <-a.closeSignal.CloseAtLeisureChan():
return a.resetMessages(rctx, mHandle)
case a.nackMessagesChan <- mHandle:
}
return nil
}, nil
}
func (a *awsSQSReader) Close(ctx context.Context) error {
a.closeSignal.CloseAtLeisure()
var closeNowAt time.Duration
if dline, ok := ctx.Deadline(); ok {
if closeNowAt = time.Until(dline) - time.Second; closeNowAt <= 0 {
a.closeSignal.CloseNow()
}
}
if closeNowAt > 0 {
select {
case <-time.After(closeNowAt):
a.closeSignal.CloseNow()
case <-ctx.Done():
return ctx.Err()
case <-a.closeSignal.HasClosedChan():
return nil
}
}
select {
case <-ctx.Done():
return ctx.Err()
case <-a.closeSignal.HasClosedChan():
}
return nil
}