forked from benthosdev/benthos
/
type.go
273 lines (237 loc) · 7.66 KB
/
type.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
package stream
import (
"bytes"
"context"
"errors"
"net/http"
"runtime/pprof"
"sync/atomic"
"time"
"github.com/dafanshu/benthos/v4/internal/bundle"
"github.com/dafanshu/benthos/v4/internal/component/buffer"
"github.com/dafanshu/benthos/v4/internal/component/input"
"github.com/dafanshu/benthos/v4/internal/component/output"
"github.com/dafanshu/benthos/v4/internal/component/processor"
"github.com/dafanshu/benthos/v4/internal/message"
"github.com/dafanshu/benthos/v4/internal/pipeline"
)
// Type creates and manages the lifetime of a Benthos stream.
type Type struct {
conf Config
inputLayer input.Streamed
bufferLayer buffer.Streamed
pipelineLayer processor.Pipeline
outputLayer output.Streamed
manager bundle.NewManagement
onClose func()
closed uint32
}
// New creates a new stream.Type.
func New(conf Config, mgr bundle.NewManagement, opts ...func(*Type)) (*Type, error) {
t := &Type{
conf: conf,
manager: mgr,
onClose: func() {},
closed: 0,
}
for _, opt := range opts {
opt(t)
}
if err := t.start(); err != nil {
return nil, err
}
healthCheck := func(w http.ResponseWriter, r *http.Request) {
inputConnected := t.inputLayer.Connected()
outputConnected := t.outputLayer.Connected()
if atomic.LoadUint32(&t.closed) == 1 {
http.Error(w, "Stream terminated", http.StatusNotFound)
return
}
if inputConnected && outputConnected {
_, _ = w.Write([]byte("OK"))
return
}
w.WriteHeader(http.StatusServiceUnavailable)
if !inputConnected {
_, _ = w.Write([]byte("input not connected\n"))
}
if !outputConnected {
_, _ = w.Write([]byte("output not connected\n"))
}
}
t.manager.RegisterEndpoint(
"/ready",
"Returns 200 OK if all inputs and outputs are connected, otherwise a 503 is returned.",
healthCheck,
)
return t, nil
}
//------------------------------------------------------------------------------
// OptOnClose sets a closure to be called when the stream closes.
func OptOnClose(onClose func()) func(*Type) {
return func(t *Type) {
t.onClose = onClose
}
}
//------------------------------------------------------------------------------
// IsReady returns a boolean indicating whether both the input and output layers
// of the stream are connected.
func (t *Type) IsReady() bool {
return t.inputLayer.Connected() && t.outputLayer.Connected()
}
func (t *Type) start() (err error) {
// Constructors
iMgr := t.manager.IntoPath("input")
if t.inputLayer, err = iMgr.NewInput(t.conf.Input); err != nil {
return
}
if t.conf.Buffer.Type != "none" {
bMgr := t.manager.IntoPath("buffer")
if t.bufferLayer, err = bMgr.NewBuffer(t.conf.Buffer); err != nil {
return
}
}
if tLen := len(t.conf.Pipeline.Processors); tLen > 0 {
pMgr := t.manager.IntoPath("pipeline")
if t.pipelineLayer, err = pipeline.New(t.conf.Pipeline, pMgr); err != nil {
return
}
}
oMgr := t.manager.IntoPath("output")
if t.outputLayer, err = oMgr.NewOutput(t.conf.Output); err != nil {
return
}
// Start chaining components
var nextTranChan <-chan message.Transaction
nextTranChan = t.inputLayer.TransactionChan()
if t.bufferLayer != nil {
if err = t.bufferLayer.Consume(nextTranChan); err != nil {
return
}
nextTranChan = t.bufferLayer.TransactionChan()
}
if t.pipelineLayer != nil {
if err = t.pipelineLayer.Consume(nextTranChan); err != nil {
return
}
nextTranChan = t.pipelineLayer.TransactionChan()
}
if err = t.outputLayer.Consume(nextTranChan); err != nil {
return
}
go func(out output.Streamed) {
for {
if err := out.WaitForClose(context.Background()); err == nil {
t.onClose()
atomic.StoreUint32(&t.closed, 1)
return
}
}
}(t.outputLayer)
return nil
}
// StopGracefully attempts to close the stream in the most graceful way by only
// closing the input layer and waiting for all other layers to terminate by
// proxy. This should guarantee that all in-flight and buffered data is resolved
// before shutting down.
func (t *Type) StopGracefully(ctx context.Context) (err error) {
t.inputLayer.TriggerStopConsuming()
if err = t.inputLayer.WaitForClose(ctx); err != nil {
return
}
// If we have a buffer then wait right here. We want to try and allow the
// buffer to empty out before prompting the other layers to shut down.
if t.bufferLayer != nil {
t.bufferLayer.TriggerStopConsuming()
if err = t.bufferLayer.WaitForClose(ctx); err != nil {
return
}
}
// After this point we can start closing the remaining components.
if t.pipelineLayer != nil {
if err = t.pipelineLayer.WaitForClose(ctx); err != nil {
return
}
}
if err = t.outputLayer.WaitForClose(ctx); err != nil {
return
}
return nil
}
// StopUnordered attempts to close all components in parallel without allowing
// the stream to gracefully wind down in the order of component layers. This
// should only be attempted if both stopGracefully and stopOrdered failed.
func (t *Type) StopUnordered(ctx context.Context) (err error) {
t.inputLayer.TriggerCloseNow()
if t.bufferLayer != nil {
t.bufferLayer.TriggerCloseNow()
}
if t.pipelineLayer != nil {
t.pipelineLayer.TriggerCloseNow()
}
t.outputLayer.TriggerCloseNow()
if err = t.inputLayer.WaitForClose(ctx); err != nil {
return
}
if t.bufferLayer != nil {
if err = t.bufferLayer.WaitForClose(ctx); err != nil {
return
}
}
if t.pipelineLayer != nil {
if err = t.pipelineLayer.WaitForClose(ctx); err != nil {
return
}
}
if err = t.outputLayer.WaitForClose(ctx); err != nil {
return
}
return nil
}
// Stop attempts to close the stream within the specified timeout period.
// Initially the attempt is graceful, but if the context contains a deadline and
// it draws near the attempt becomes progressively less graceful.
//
// If the context is cancelled an error is returned _after_ asynchronously
// instructing the remaining stream components to terminate ungracefully.
func (t *Type) Stop(ctx context.Context) error {
ctxCloseGraceful := ctx
// If the provided context has a known deadline then we calculate a period
// of time whereby it would be appropriate to abandon graceful termination
// and attempt ungraceful termination within that deadline.
if deadline, ok := ctx.Deadline(); ok {
// The calculated time we're willing to wait for graceful termination is
// three quarters of the overall deadline.
tUntil := time.Until(deadline)
tUntil -= (tUntil / 4)
if tUntil > time.Second {
var gDone func()
ctxCloseGraceful, gDone = context.WithTimeout(ctx, tUntil)
defer gDone()
}
}
// Attempt graceful termination by instructing the input to stop consuming
// and for all downstream components to finish.
err := t.StopGracefully(ctxCloseGraceful)
if err == nil {
return nil
}
if !(errors.Is(err, context.Canceled) && errors.Is(err, context.DeadlineExceeded)) {
t.manager.Logger().Errorf("Encountered error whilst attempting to shut down gracefully: %v\n", err)
}
// If graceful termination failed then call unordered termination, if the
// overall ctx is already cancelled this will still trigger asynchronous
// clean up of resources, which is a best attempt.
if err = t.StopUnordered(ctx); err == nil {
return nil
}
if errors.Is(err, context.Canceled) || errors.Is(err, context.DeadlineExceeded) {
t.manager.Logger().Infoln("Some components prevented forced termination as they were either blocked from delivering data or from acknowledging delivered data within the shutdown timeout. This could potentially cause duplicate messages to be delivered on the next run.")
dumpBuf := bytes.NewBuffer(nil)
_ = pprof.Lookup("goroutine").WriteTo(dumpBuf, 1)
t.manager.Logger().Debugln(dumpBuf.String())
} else {
t.manager.Logger().Errorf("Encountered error whilst forcefully shutting down: %v\n", err)
}
return err
}