/
output_broker_fan_out_sequential.go
138 lines (118 loc) · 3.02 KB
/
output_broker_fan_out_sequential.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
package pure
import (
"context"
"errors"
"sync/atomic"
"time"
"github.com/Jeffail/shutdown"
"github.com/benthosdev/benthos/v4/internal/component"
"github.com/benthosdev/benthos/v4/internal/component/output"
"github.com/benthosdev/benthos/v4/internal/message"
)
type fanOutSequentialOutputBroker struct {
transactions <-chan message.Transaction
outputTSChans []chan message.Transaction
outputs []output.Streamed
shutSig *shutdown.Signaller
}
func newFanOutSequentialOutputBroker(outputs []output.Streamed) (*fanOutSequentialOutputBroker, error) {
o := &fanOutSequentialOutputBroker{
transactions: nil,
outputs: outputs,
shutSig: shutdown.NewSignaller(),
}
o.outputTSChans = make([]chan message.Transaction, len(o.outputs))
for i := range o.outputTSChans {
o.outputTSChans[i] = make(chan message.Transaction)
if err := o.outputs[i].Consume(o.outputTSChans[i]); err != nil {
return nil, err
}
}
return o, nil
}
func (o *fanOutSequentialOutputBroker) Consume(transactions <-chan message.Transaction) error {
if o.transactions != nil {
return component.ErrAlreadyStarted
}
o.transactions = transactions
go o.loop()
return nil
}
func (o *fanOutSequentialOutputBroker) Connected() bool {
for _, out := range o.outputs {
if !out.Connected() {
return false
}
}
return true
}
func (o *fanOutSequentialOutputBroker) loop() {
ackInterruptChan := make(chan struct{})
var ackPending int64
defer func() {
// Wait for pending acks to be resolved, or forceful termination
for atomic.LoadInt64(&ackPending) > 0 {
select {
case <-ackInterruptChan:
case <-time.After(time.Millisecond * 100):
// Just incase an interrupt doesn't arrive.
}
}
for _, c := range o.outputTSChans {
close(c)
}
_ = closeAllOutputs(context.Background(), o.outputs)
o.shutSig.TriggerHasStopped()
}()
for {
var ts message.Transaction
var open bool
select {
case ts, open = <-o.transactions:
if !open {
return
}
case <-o.shutSig.HardStopChan():
return
}
_ = atomic.AddInt64(&ackPending, 1)
i := 0
var ackFn func(ctx context.Context, err error) error
ackFn = func(ctx context.Context, err error) error {
i++
if err != nil || len(o.outputTSChans) <= i {
ackErr := ts.Ack(ctx, err)
_ = atomic.AddInt64(&ackPending, -1)
select {
case ackInterruptChan <- struct{}{}:
default:
}
return ackErr
}
select {
case o.outputTSChans[i] <- message.NewTransactionFunc(ts.Payload, ackFn):
case <-o.shutSig.HardStopChan():
return errors.New("component is shutting down")
case <-ctx.Done():
return ctx.Err()
}
return nil
}
select {
case o.outputTSChans[i] <- message.NewTransactionFunc(ts.Payload, ackFn):
case <-o.shutSig.HardStopChan():
return
}
}
}
func (o *fanOutSequentialOutputBroker) TriggerCloseNow() {
o.shutSig.TriggerHardStop()
}
func (o *fanOutSequentialOutputBroker) WaitForClose(ctx context.Context) error {
select {
case <-o.shutSig.HasStoppedChan():
case <-ctx.Done():
return ctx.Err()
}
return nil
}