forked from benthosdev/benthos
/
async_preserver.go
128 lines (111 loc) · 4.13 KB
/
async_preserver.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
// Copyright (c) 2018 Ashley Jeffs
//
// Permission is hereby granted, free of charge, to any person obtaining a copy
// of this software and associated documentation files (the "Software"), to deal
// in the Software without restriction, including without limitation the rights
// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
// copies of the Software, and to permit persons to whom the Software is
// furnished to do so, subject to the following conditions:
//
// The above copyright notice and this permission notice shall be included in
// all copies or substantial portions of the Software.
//
// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
// THE SOFTWARE.
package reader
import (
"context"
"sync"
"time"
"github.com/ollystephens/benthos/v3/lib/types"
)
//------------------------------------------------------------------------------
type asyncPreserverResend struct {
msg types.Message
ackFn AsyncAckFn
}
// AsyncPreserver is a wrapper for reader.Async implementations that keeps a
// buffer of sent messages until they are acknowledged. If an error occurs
// during message propagation the contents of the buffer will be resent instead
// of reading new messages until it is depleted. AsyncPreserver implements
// reader.Async.
//
// Wrapping an input with this type is useful when your source of messages
// doesn't have a concept of a NoAck (like Kafka), and instead of "rejecting"
// messages we always intend to simply retry them until success.
type AsyncPreserver struct {
resendMessages []asyncPreserverResend
resendInterrupt func()
msgsMut sync.Mutex
r Async
}
// NewAsyncPreserver returns a new AsyncPreserver wrapper around a reader.Async.
func NewAsyncPreserver(r Async) *AsyncPreserver {
return &AsyncPreserver{
r: r,
resendInterrupt: func() {},
}
}
//------------------------------------------------------------------------------
// ConnectWithContext attempts to establish a connection to the source, if
// unsuccessful returns an error. If the attempt is successful (or not
// necessary) returns nil.
func (p *AsyncPreserver) ConnectWithContext(ctx context.Context) error {
return p.r.ConnectWithContext(ctx)
}
func (p *AsyncPreserver) wrapAckFn(msg types.Message, ackFn AsyncAckFn) AsyncAckFn {
return func(ctx context.Context, res types.Response) error {
if res.Error() != nil {
p.msgsMut.Lock()
p.resendMessages = append(p.resendMessages, asyncPreserverResend{
msg: msg,
ackFn: ackFn,
})
p.resendInterrupt()
p.msgsMut.Unlock()
return nil
}
return ackFn(ctx, res)
}
}
// ReadWithContext attempts to read a new message from the source.
func (p *AsyncPreserver) ReadWithContext(ctx context.Context) (types.Message, AsyncAckFn, error) {
var cancel func()
ctx, cancel = context.WithCancel(ctx)
defer cancel()
// If we have messages queued to be resent we prioritise them over reading
// new messages.
p.msgsMut.Lock()
if lMsgs := len(p.resendMessages); lMsgs > 0 {
resend := p.resendMessages[0]
if lMsgs > 1 {
p.resendMessages = p.resendMessages[1:]
} else {
p.resendMessages = nil
}
p.msgsMut.Unlock()
return resend.msg, p.wrapAckFn(resend.msg, resend.ackFn), nil
}
p.resendInterrupt = cancel
p.msgsMut.Unlock()
msg, aFn, err := p.r.ReadWithContext(ctx)
if err != nil {
return nil, nil, err
}
return msg, p.wrapAckFn(msg, aFn), nil
}
// CloseAsync triggers the asynchronous closing of the reader.
func (p *AsyncPreserver) CloseAsync() {
p.r.CloseAsync()
}
// WaitForClose blocks until either the reader is finished closing or a timeout
// occurs.
func (p *AsyncPreserver) WaitForClose(tout time.Duration) error {
return p.r.WaitForClose(tout)
}
//------------------------------------------------------------------------------