/
batcher.go
203 lines (173 loc) · 4.55 KB
/
batcher.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
package input
import (
"context"
"sync"
"time"
"github.com/Jeffail/benthos/v3/internal/shutdown"
"github.com/Jeffail/benthos/v3/internal/transaction"
"github.com/Jeffail/benthos/v3/lib/log"
"github.com/Jeffail/benthos/v3/lib/message/batch"
"github.com/Jeffail/benthos/v3/lib/metrics"
"github.com/Jeffail/benthos/v3/lib/types"
)
//------------------------------------------------------------------------------
// Batcher wraps an input with a batch policy.
type Batcher struct {
stats metrics.Type
log log.Modular
child Type
batcher *batch.Policy
messagesOut chan types.Transaction
shutSig *shutdown.Signaller
}
// NewBatcher creates a new Batcher around an input.
func NewBatcher(
batcher *batch.Policy,
child Type,
log log.Modular,
stats metrics.Type,
) Type {
b := Batcher{
stats: stats,
log: log,
child: child,
batcher: batcher,
messagesOut: make(chan types.Transaction),
shutSig: shutdown.NewSignaller(),
}
go b.loop()
return &b
}
//------------------------------------------------------------------------------
func (m *Batcher) loop() {
defer func() {
go func() {
select {
case <-m.shutSig.CloseNowChan():
_ = m.child.WaitForClose(0)
_ = m.batcher.WaitForClose(0)
case <-m.shutSig.HasClosedChan():
}
}()
m.child.CloseAsync()
_ = m.child.WaitForClose(shutdown.MaximumShutdownWait())
m.batcher.CloseAsync()
_ = m.batcher.WaitForClose(shutdown.MaximumShutdownWait())
close(m.messagesOut)
m.shutSig.ShutdownComplete()
}()
var nextTimedBatchChan <-chan time.Time
if tNext := m.batcher.UntilNext(); tNext >= 0 {
nextTimedBatchChan = time.After(tNext)
}
pendingTrans := []*transaction.Tracked{}
pendingAcks := sync.WaitGroup{}
flushBatchFn := func() {
sendMsg := m.batcher.Flush()
if sendMsg == nil {
return
}
resChan := make(chan types.Response)
select {
case m.messagesOut <- types.NewTransaction(sendMsg, resChan):
case <-m.shutSig.CloseNowChan():
return
}
pendingAcks.Add(1)
go func(rChan <-chan types.Response, aggregatedTransactions []*transaction.Tracked) {
defer pendingAcks.Done()
select {
case <-m.shutSig.CloseNowChan():
return
case res, open := <-rChan:
if !open {
return
}
closeNowCtx, done := m.shutSig.CloseNowCtx(context.Background())
for _, c := range aggregatedTransactions {
if err := c.Ack(closeNowCtx, res.Error()); err != nil {
done()
return
}
}
done()
}
}(resChan, pendingTrans)
pendingTrans = nil
}
defer func() {
// Final flush of remaining documents.
m.log.Debugln("Flushing remaining messages of batch.")
flushBatchFn()
// Wait for all pending acks to resolve.
m.log.Debugln("Waiting for pending acks to resolve before shutting down.")
pendingAcks.Wait()
m.log.Debugln("Pending acks resolved.")
}()
for {
if nextTimedBatchChan == nil {
if tNext := m.batcher.UntilNext(); tNext >= 0 {
nextTimedBatchChan = time.After(tNext)
}
}
var flushBatch bool
select {
case tran, open := <-m.child.TransactionChan():
if !open {
// If we're waiting for a timed batch then we will respect it.
if nextTimedBatchChan != nil {
select {
case <-nextTimedBatchChan:
case <-m.shutSig.CloseAtLeisureChan():
return
}
}
flushBatchFn()
return
}
trackedTran := transaction.NewTracked(tran.Payload, tran.ResponseChan)
trackedTran.Message().Iter(func(i int, p types.Part) error {
if m.batcher.Add(p) {
flushBatch = true
}
return nil
})
pendingTrans = append(pendingTrans, trackedTran)
case <-nextTimedBatchChan:
flushBatch = true
nextTimedBatchChan = nil
case <-m.shutSig.CloseAtLeisureChan():
return
}
if flushBatch {
flushBatchFn()
}
}
}
// Connected returns true if the underlying input is connected.
func (m *Batcher) Connected() bool {
return m.child.Connected()
}
// TransactionChan returns the channel used for consuming messages from this
// buffer.
func (m *Batcher) TransactionChan() <-chan types.Transaction {
return m.messagesOut
}
// CloseAsync shuts down the Batcher and stops processing messages.
func (m *Batcher) CloseAsync() {
m.shutSig.CloseAtLeisure()
}
// WaitForClose blocks until the Batcher output has closed down.
func (m *Batcher) WaitForClose(timeout time.Duration) error {
go func() {
<-time.After(timeout - time.Second)
m.shutSig.CloseNow()
}()
select {
case <-m.shutSig.HasClosedChan():
case <-time.After(timeout):
return types.ErrTimeout
}
return nil
}
//------------------------------------------------------------------------------