-
Notifications
You must be signed in to change notification settings - Fork 791
/
processor.go
222 lines (188 loc) · 4.88 KB
/
processor.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
package pipeline
import (
"sync"
"sync/atomic"
"time"
"github.com/Jeffail/benthos/v3/lib/log"
"github.com/Jeffail/benthos/v3/lib/metrics"
"github.com/Jeffail/benthos/v3/lib/processor"
"github.com/Jeffail/benthos/v3/lib/response"
"github.com/Jeffail/benthos/v3/lib/types"
"github.com/Jeffail/benthos/v3/lib/util/throttle"
)
//------------------------------------------------------------------------------
// Processor is a pipeline that supports both Consumer and Producer interfaces.
// The processor will read from a source, perform some processing, and then
// either propagate a new message or drop it.
type Processor struct {
running int32
log log.Modular
stats metrics.Type
msgProcessors []types.Processor
messagesOut chan types.Transaction
responsesIn chan types.Response
messagesIn <-chan types.Transaction
closeChan chan struct{}
closed chan struct{}
}
// NewProcessor returns a new message processing pipeline.
func NewProcessor(
log log.Modular,
stats metrics.Type,
msgProcessors ...types.Processor,
) *Processor {
return &Processor{
running: 1,
msgProcessors: msgProcessors,
stats: stats,
messagesOut: make(chan types.Transaction),
responsesIn: make(chan types.Response),
closeChan: make(chan struct{}),
closed: make(chan struct{}),
}
}
//------------------------------------------------------------------------------
// loop is the processing loop of this pipeline.
func (p *Processor) loop() {
defer func() {
// Signal all children to close.
for _, c := range p.msgProcessors {
c.CloseAsync()
}
close(p.messagesOut)
close(p.closed)
}()
var open bool
for atomic.LoadInt32(&p.running) == 1 {
var tran types.Transaction
select {
case tran, open = <-p.messagesIn:
if !open {
return
}
case <-p.closeChan:
return
}
resultMsgs, resultRes := processor.ExecuteAll(p.msgProcessors, tran.Payload)
if len(resultMsgs) == 0 {
if resultRes == nil {
resultRes = response.NewUnack()
p.log.Warnln("Nil response returned with zero messages from processors")
}
select {
case tran.ResponseChan <- resultRes:
case <-p.closeChan:
return
}
continue
}
if len(resultMsgs) > 1 {
p.dispatchMessages(resultMsgs, tran.ResponseChan)
} else {
select {
case p.messagesOut <- types.NewTransaction(resultMsgs[0], tran.ResponseChan):
case <-p.closeChan:
return
}
}
}
}
// dispatchMessages attempts to send a multiple messages results of processors
// over the shared messages channel. This send is retried until success.
func (p *Processor) dispatchMessages(msgs []types.Message, ogResChan chan<- types.Response) {
throt := throttle.New(throttle.OptCloseChan(p.closeChan))
var skipAcks int64
sendMsg := func(m types.Message) {
resChan := make(chan types.Response)
transac := types.NewTransaction(m, resChan)
for {
select {
case p.messagesOut <- transac:
case <-p.closeChan:
return
}
var res types.Response
var open bool
select {
case res, open = <-resChan:
if !open {
return
}
case <-p.closeChan:
return
}
if skipAck := res.SkipAck(); res.Error() == nil || skipAck {
if skipAck {
atomic.AddInt64(&skipAcks, 1)
}
return
}
if !throt.Retry() {
return
}
}
}
wg := sync.WaitGroup{}
wg.Add(len(msgs))
for _, msg := range msgs {
go func(m types.Message) {
defer wg.Done()
sendMsg(m)
}(msg)
}
wg.Wait()
throt.Reset()
var res types.Response
if skipAcks == int64(len(msgs)) {
res = response.NewUnack()
} else {
res = response.NewAck()
}
select {
case ogResChan <- res:
case <-p.closeChan:
return
}
}
//------------------------------------------------------------------------------
// Consume assigns a messages channel for the pipeline to read.
func (p *Processor) Consume(msgs <-chan types.Transaction) error {
if p.messagesIn != nil {
return types.ErrAlreadyStarted
}
p.messagesIn = msgs
go p.loop()
return nil
}
// TransactionChan returns the channel used for consuming messages from this
// pipeline.
func (p *Processor) TransactionChan() <-chan types.Transaction {
return p.messagesOut
}
// CloseAsync shuts down the pipeline and stops processing messages.
func (p *Processor) CloseAsync() {
if atomic.CompareAndSwapInt32(&p.running, 1, 0) {
close(p.closeChan)
// Signal all children to close.
for _, c := range p.msgProcessors {
c.CloseAsync()
}
}
}
// WaitForClose blocks until the StackBuffer output has closed down.
func (p *Processor) WaitForClose(timeout time.Duration) error {
stopBy := time.Now().Add(timeout)
select {
case <-p.closed:
case <-time.After(time.Until(stopBy)):
return types.ErrTimeout
}
// Wait for all processors to close.
for _, c := range p.msgProcessors {
if err := c.WaitForClose(time.Until(stopBy)); err != nil {
return err
}
}
return nil
}
//------------------------------------------------------------------------------