forked from redpanda-data/connect
-
Notifications
You must be signed in to change notification settings - Fork 1
/
dynamic_fan_out.go
349 lines (307 loc) · 9.93 KB
/
dynamic_fan_out.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
// 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 broker
import (
"fmt"
"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"
"github.com/Jeffail/benthos/lib/util/throttle"
)
//------------------------------------------------------------------------------
// DynamicOutput is an interface of output types that must be closable.
type DynamicOutput interface {
types.TransactionReceiver
types.Closable
}
// wrappedOutput is a struct that wraps a DynamicOutput with an identifying
// name.
type wrappedOutput struct {
Name string
Output DynamicOutput
Timeout time.Duration
ResChan chan<- error
}
// outputWithResChan is a struct containing both an output and the response chan
// it reads from.
type outputWithResChan struct {
tsChan chan types.Transaction
resChan chan types.Response
output DynamicOutput
}
//------------------------------------------------------------------------------
// DynamicFanOut is a broker that implements types.Consumer and broadcasts each
// message out to a dynamic map of outputs.
type DynamicFanOut struct {
running int32
log log.Modular
stats metrics.Type
throt *throttle.Type
onAdd func(label string)
onRemove func(label string)
transactions <-chan types.Transaction
newOutputChan chan wrappedOutput
outputs map[string]outputWithResChan
closedChan chan struct{}
closeChan chan struct{}
}
// NewDynamicFanOut creates a new DynamicFanOut type by providing outputs.
func NewDynamicFanOut(
outputs map[string]DynamicOutput,
logger log.Modular,
stats metrics.Type,
options ...func(*DynamicFanOut),
) (*DynamicFanOut, error) {
d := &DynamicFanOut{
running: 1,
stats: stats,
log: logger.NewModule(".broker.dynamic_fan_out"),
onAdd: func(l string) {},
onRemove: func(l string) {},
transactions: nil,
newOutputChan: make(chan wrappedOutput),
outputs: make(map[string]outputWithResChan, len(outputs)),
closedChan: make(chan struct{}),
closeChan: make(chan struct{}),
}
for _, opt := range options {
opt(d)
}
d.throt = throttle.New(throttle.OptCloseChan(d.closeChan))
for k, v := range outputs {
if err := d.addOutput(k, v); err != nil {
d.log.Errorf("Failed to initialise dynamic output '%v': %v\n", k, err)
d.stats.Incr("broker.dynamic_fan_out.output.add.error", 1)
} else {
d.onAdd(k)
}
}
return d, nil
}
// SetOutput attempts to add a new output to the dynamic output broker. If an
// output already exists with the same identifier it will be closed and removed.
// If either action takes longer than the timeout period an error will be
// returned.
//
// A nil output argument is safe and will simply remove the previous output
// under the indentifier, if there was one.
func (d *DynamicFanOut) SetOutput(ident string, output DynamicOutput, timeout time.Duration) error {
if atomic.LoadInt32(&d.running) != 1 {
return types.ErrTypeClosed
}
resChan := make(chan error)
select {
case d.newOutputChan <- wrappedOutput{
Name: ident,
Output: output,
ResChan: resChan,
Timeout: timeout,
}:
case <-d.closeChan:
return types.ErrTypeClosed
}
return <-resChan
}
//------------------------------------------------------------------------------
// OptDynamicFanOutSetOnAdd sets the function that is called whenever a dynamic
// output is added.
func OptDynamicFanOutSetOnAdd(onAddFunc func(label string)) func(*DynamicFanOut) {
return func(d *DynamicFanOut) {
d.onAdd = onAddFunc
}
}
// OptDynamicFanOutSetOnRemove sets the function that is called whenever a
// dynamic output is removed.
func OptDynamicFanOutSetOnRemove(onRemoveFunc func(label string)) func(*DynamicFanOut) {
return func(d *DynamicFanOut) {
d.onRemove = onRemoveFunc
}
}
//------------------------------------------------------------------------------
// StartReceiving assigns a new transactions channel for the broker to read.
func (d *DynamicFanOut) StartReceiving(transactions <-chan types.Transaction) error {
if d.transactions != nil {
return types.ErrAlreadyStarted
}
d.transactions = transactions
go d.loop()
return nil
}
//------------------------------------------------------------------------------
func (d *DynamicFanOut) addOutput(ident string, output DynamicOutput) error {
if _, exists := d.outputs[ident]; exists {
return fmt.Errorf("output key '%v' already exists", ident)
}
ow := outputWithResChan{
tsChan: make(chan types.Transaction),
resChan: make(chan types.Response),
output: output,
}
if err := output.StartReceiving(ow.tsChan); err != nil {
output.CloseAsync()
return err
}
d.outputs[ident] = ow
return nil
}
func (d *DynamicFanOut) removeOutput(ident string, timeout time.Duration) error {
ow, exists := d.outputs[ident]
if !exists {
return nil
}
ow.output.CloseAsync()
if err := ow.output.WaitForClose(timeout); err != nil {
return err
}
close(ow.tsChan)
delete(d.outputs, ident)
return nil
}
//------------------------------------------------------------------------------
// loop is an internal loop that brokers incoming messages to many outputs.
func (d *DynamicFanOut) loop() {
defer func() {
for _, ow := range d.outputs {
ow.output.CloseAsync()
close(ow.tsChan)
}
for _, ow := range d.outputs {
if err := ow.output.WaitForClose(time.Second); err != nil {
for err != nil {
err = ow.output.WaitForClose(time.Second)
}
}
}
d.outputs = map[string]outputWithResChan{}
close(d.closedChan)
}()
for atomic.LoadInt32(&d.running) == 1 {
var ts types.Transaction
var open bool
// Only actually consume messages if we have at least one output.
var tsChan <-chan types.Transaction
if len(d.outputs) > 0 {
tsChan = d.transactions
}
// Either attempt to read the next message or listen for changes to our
// outputs and apply them.
select {
case wrappedOutput, open := <-d.newOutputChan:
if !open {
return
}
d.stats.Incr("broker.dynamic_fan_out.output.count", 1)
if _, exists := d.outputs[wrappedOutput.Name]; exists {
if err := d.removeOutput(wrappedOutput.Name, wrappedOutput.Timeout); err != nil {
d.stats.Incr("broker.dynamic_fan_out.output.remove.error", 1)
d.log.Errorf("Failed to stop old copy of dynamic output '%v': %v\n", wrappedOutput.Name, err)
wrappedOutput.ResChan <- err
continue
}
d.stats.Incr("broker.dynamic_fan_out.output.remove.success", 1)
d.onRemove(wrappedOutput.Name)
}
if wrappedOutput.Output == nil {
wrappedOutput.ResChan <- nil
} else {
err := d.addOutput(wrappedOutput.Name, wrappedOutput.Output)
if err != nil {
d.stats.Incr("broker.dynamic_fan_out.output.add.error", 1)
d.log.Errorf("Failed to start new dynamic output '%v': %v\n", wrappedOutput.Name, err)
} else {
d.stats.Incr("broker.dynamic_fan_out.output.add.success", 1)
d.onAdd(wrappedOutput.Name)
}
wrappedOutput.ResChan <- err
}
continue
case ts, open = <-tsChan:
if !open {
return
}
case <-d.closeChan:
return
}
d.stats.Incr("broker.dynamic_fan_out.messages.received", 1)
// If we received a message attempt to send it to each output.
remainingTargets := make(map[string]outputWithResChan, len(d.outputs))
for k, v := range d.outputs {
remainingTargets[k] = v
}
for len(remainingTargets) > 0 {
for _, v := range remainingTargets {
// Perform a copy here as it could be dangerous to release the
// same message to parallel processor pipelines.
msgCopy := ts.Payload.ShallowCopy()
select {
case v.tsChan <- types.NewTransaction(msgCopy, v.resChan):
case <-d.closeChan:
return
}
}
for k, v := range remainingTargets {
var res types.Response
select {
case res, open = <-v.resChan:
if !open {
d.log.Warnf("Dynamic output '%v' has closed\n", k)
d.removeOutput(k, time.Second)
delete(remainingTargets, k)
} else if res.Error() != nil {
d.log.Errorf("Failed to dispatch dynamic fan out message: %v\n", res.Error())
d.stats.Incr("broker.dynamic_fan_out.output.error", 1)
if !d.throt.Retry() {
return
}
} else {
d.throt.Reset()
d.stats.Incr("broker.dynamic_fan_out.messages.sent", 1)
delete(remainingTargets, k)
}
case <-d.closeChan:
return
}
}
}
select {
case ts.ResponseChan <- types.NewSimpleResponse(nil):
case <-d.closeChan:
return
}
}
}
// CloseAsync shuts down the DynamicFanOut broker and stops processing requests.
func (d *DynamicFanOut) CloseAsync() {
if atomic.CompareAndSwapInt32(&d.running, 1, 0) {
close(d.closeChan)
}
}
// WaitForClose blocks until the DynamicFanOut broker has closed down.
func (d *DynamicFanOut) WaitForClose(timeout time.Duration) error {
select {
case <-d.closedChan:
case <-time.After(timeout):
return types.ErrTimeout
}
return nil
}
//------------------------------------------------------------------------------