forked from redpanda-data/connect
-
Notifications
You must be signed in to change notification settings - Fork 1
/
pool.go
277 lines (232 loc) · 6.71 KB
/
pool.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
package pipeline
import (
"sync"
"sync/atomic"
"time"
"github.com/Jeffail/benthos/lib/types"
"github.com/Jeffail/benthos/lib/util/service/log"
"github.com/Jeffail/benthos/lib/util/service/metrics"
)
//------------------------------------------------------------------------------
// Pool is a pool of pipelines. It reads from a single source and writes to a
// single source. The input is decoupled which means failed delivery
// notification cannot be propagated back up to the original input.
//
// If delivery acknowledgements to the input is required this pool should not be
// used. Instead, you should configure multiple inputs each with their own
// pipeline e.g. configure 8 kafka_balanced inputs each with a single processor
// rather than a single kafka_balanced with a pool of 8 workers.
type Pool struct {
running uint32
workers []Type
remainingWorkers int32
constructor ConstructorFunc
log log.Modular
stats metrics.Type
workChan chan types.Message
messagesOut chan types.Message
responsesOut chan types.Response
messagesIn <-chan types.Message
responsesIn <-chan types.Response
closeChan chan struct{}
closed chan struct{}
}
// NewPool returns a new pipeline pool that utilized multiple processor threads.
func NewPool(
constructor ConstructorFunc,
workers int,
log log.Modular,
stats metrics.Type,
) (*Pool, error) {
p := &Pool{
running: 1,
workers: make([]Type, workers),
remainingWorkers: 0,
constructor: constructor,
log: log,
stats: stats,
workChan: make(chan types.Message),
messagesOut: make(chan types.Message),
responsesOut: make(chan types.Response),
closeChan: make(chan struct{}),
closed: make(chan struct{}),
}
for i := range p.workers {
var err error
if p.workers[i], err = p.constructor(); err != nil {
return nil, err
}
}
return p, nil
}
//------------------------------------------------------------------------------
// workerLoop is the processing loop of a pool worker.
func (p *Pool) workerLoop(worker Type, wg *sync.WaitGroup) {
sendChan := make(chan types.Message)
resChan := make(chan types.Response)
defer func() {
close(sendChan)
close(resChan)
atomic.AddInt32(&p.remainingWorkers, -1)
wg.Done()
}()
if err := worker.StartReceiving(sendChan); err != nil {
p.log.Errorf("Failed to start pool worker: %v\n", err)
return
}
if err := worker.StartListening(resChan); err != nil {
p.log.Errorf("Failed to start pool worker: %v\n", err)
return
}
var msgOut types.Message
for {
var open bool
if len(msgOut.Parts) == 0 {
var msgIn types.Message
// Read new work from pool.
if msgIn, open = <-p.workChan; !open {
return
}
p.stats.Incr("pipeline.pool.worker.message.received", 1)
// Send work to processing pipeline.
sendChan <- msgIn
p.stats.Incr("pipeline.pool.worker.message.sent", 1)
// Receive result from processing pipeline or response.
select {
case msgOut, open = <-worker.MessageChan():
if !open {
return
}
p.stats.Incr("pipeline.pool.worker.result.received", 1)
// Send decoupled response to processing pipeline
resChan <- types.NewSimpleResponse(nil)
if _, open = <-worker.ResponseChan(); !open {
return
}
case _, open = <-worker.ResponseChan():
if !open {
return
}
// Message was dropped, move onto next.
}
}
if len(msgOut.Parts) > 0 {
// Send result to shared output channel.
p.messagesOut <- msgOut
p.stats.Incr("pipeline.pool.worker.result.sent", 1)
// Receive output response from shared response channel.
var res types.Response
if res, open = <-p.responsesIn; !open {
// TODO: LOST MESSAGE
p.stats.Incr("pipeline.pool.worker.response.lost.shut_down", 1)
return
} else if err := res.Error(); err != nil {
p.log.Errorf("Failed to send message: %v\n", err)
} else {
msgOut = types.Message{}
p.stats.Incr("pipeline.pool.worker.response.received", 1)
}
p.stats.Incr("pipeline.worker.response.sent", 1)
}
}
}
// loop is the processing loop of this pipeline.
func (p *Pool) loop() {
workerGroup := sync.WaitGroup{}
defer func() {
atomic.StoreUint32(&p.running, 0)
// Signal all workers to close.
close(p.workChan)
for _, worker := range p.workers {
worker.CloseAsync()
}
// Wait for all workers to be closed before closing our response and
// messages channels as the workers may still have access to them.
for _, worker := range p.workers {
err := worker.WaitForClose(time.Second)
for err != nil {
err = worker.WaitForClose(time.Second)
}
}
workerGroup.Wait()
close(p.responsesOut)
close(p.messagesOut)
close(p.closed)
}()
workerGroup.Add(len(p.workers))
for _, worker := range p.workers {
atomic.AddInt32(&p.remainingWorkers, 1)
go p.workerLoop(worker, &workerGroup)
}
var open bool
for atomic.LoadUint32(&p.running) == 1 && atomic.LoadInt32(&p.remainingWorkers) > 0 {
var msg types.Message
select {
case msg, open = <-p.messagesIn:
if !open {
return
}
case <-p.closeChan:
return
}
p.stats.Incr("pipeline.pool.message.received", 1)
select {
case p.workChan <- msg:
case <-p.closeChan:
return
}
select {
case p.responsesOut <- types.NewSimpleResponse(nil):
case <-p.closeChan:
return
}
}
}
//------------------------------------------------------------------------------
// StartReceiving assigns a messages channel for the pipeline to read.
func (p *Pool) StartReceiving(msgs <-chan types.Message) error {
if p.messagesIn != nil {
return types.ErrAlreadyStarted
}
p.messagesIn = msgs
if p.responsesIn != nil {
go p.loop()
}
return nil
}
// MessageChan returns the channel used for consuming messages from this
// pipeline.
func (p *Pool) MessageChan() <-chan types.Message {
return p.messagesOut
}
// StartListening sets the channel that this pipeline will read responses from.
func (p *Pool) StartListening(responses <-chan types.Response) error {
if p.responsesIn != nil {
return types.ErrAlreadyStarted
}
p.responsesIn = responses
if p.messagesIn != nil {
go p.loop()
}
return nil
}
// ResponseChan returns the response channel from this pipeline.
func (p *Pool) ResponseChan() <-chan types.Response {
return p.responsesOut
}
// CloseAsync shuts down the pipeline and stops processing messages.
func (p *Pool) CloseAsync() {
if atomic.CompareAndSwapUint32(&p.running, 1, 0) {
close(p.closeChan)
}
}
// WaitForClose - Blocks until the StackBuffer output has closed down.
func (p *Pool) WaitForClose(timeout time.Duration) error {
select {
case <-p.closed:
case <-time.After(timeout):
return types.ErrTimeout
}
return nil
}
//------------------------------------------------------------------------------