github.com/Jeffail/benthos/v3@v3.65.0/lib/stream/type.go (about) 1 package stream 2 3 import ( 4 "bytes" 5 "net/http" 6 "runtime/pprof" 7 "time" 8 9 "github.com/Jeffail/benthos/v3/internal/interop" 10 "github.com/Jeffail/benthos/v3/lib/buffer" 11 "github.com/Jeffail/benthos/v3/lib/input" 12 "github.com/Jeffail/benthos/v3/lib/log" 13 "github.com/Jeffail/benthos/v3/lib/metrics" 14 "github.com/Jeffail/benthos/v3/lib/output" 15 "github.com/Jeffail/benthos/v3/lib/pipeline" 16 "github.com/Jeffail/benthos/v3/lib/types" 17 18 // TODO: V4 Remove this as it's a temporary work around to ensure current 19 // plugin users automatically import all components. 20 _ "github.com/Jeffail/benthos/v3/public/components/legacy" 21 ) 22 23 //------------------------------------------------------------------------------ 24 25 // Type creates and manages the lifetime of a Benthos stream. 26 type Type struct { 27 conf Config 28 29 inputLayer input.Type 30 bufferLayer buffer.Type 31 pipelineLayer pipeline.Type 32 outputLayer output.Type 33 34 complementaryProcs []types.ProcessorConstructorFunc 35 36 manager types.Manager 37 stats metrics.Type 38 logger log.Modular 39 40 onClose func() 41 } 42 43 // New creates a new stream.Type. 44 func New(conf Config, opts ...func(*Type)) (*Type, error) { 45 t := &Type{ 46 conf: conf, 47 stats: metrics.Noop(), 48 logger: log.Noop(), 49 manager: types.NoopMgr(), 50 onClose: func() {}, 51 } 52 for _, opt := range opts { 53 opt(t) 54 } 55 if err := t.start(); err != nil { 56 return nil, err 57 } 58 59 healthCheck := func(w http.ResponseWriter, r *http.Request) { 60 connected := true 61 if !t.inputLayer.Connected() { 62 connected = false 63 w.WriteHeader(http.StatusServiceUnavailable) 64 w.Write([]byte("input not connected\n")) 65 } 66 if !t.outputLayer.Connected() { 67 connected = false 68 w.WriteHeader(http.StatusServiceUnavailable) 69 w.Write([]byte("output not connected\n")) 70 } 71 if connected { 72 w.Write([]byte("OK")) 73 } 74 } 75 t.manager.RegisterEndpoint( 76 "/ready", 77 "Returns 200 OK if all inputs and outputs are connected, otherwise a 503 is returned.", 78 healthCheck, 79 ) 80 return t, nil 81 } 82 83 //------------------------------------------------------------------------------ 84 85 // OptAddProcessors adds additional processors that will be constructed for each 86 // logical thread of the processing pipeline layer of the Benthos stream. 87 func OptAddProcessors(procs ...types.ProcessorConstructorFunc) func(*Type) { 88 return func(t *Type) { 89 t.complementaryProcs = append(t.complementaryProcs, procs...) 90 } 91 } 92 93 // OptSetStats sets the metrics aggregator to be used by all components of the 94 // stream. 95 func OptSetStats(stats metrics.Type) func(*Type) { 96 return func(t *Type) { 97 t.stats = stats 98 } 99 } 100 101 // OptSetLogger sets the logging output to be used by all components of the 102 // stream. To avoid implementing the log.Modular interface with a custom logger 103 // consider using OptSetLogSimple instead. 104 func OptSetLogger(l log.Modular) func(*Type) { 105 return func(t *Type) { 106 t.logger = l 107 } 108 } 109 110 // OptSetLogSimple sets the logging output to a simpler log interface 111 // (implemented by the standard *log.Logger.) 112 func OptSetLogSimple(l log.PrintFormatter) func(*Type) { 113 return func(t *Type) { 114 t.logger = log.Wrap(l) 115 } 116 } 117 118 // OptSetManager sets the service manager to be used by all components of the 119 // stream. 120 func OptSetManager(mgr types.Manager) func(*Type) { 121 return func(t *Type) { 122 t.manager = mgr 123 } 124 } 125 126 // OptOnClose sets a closure to be called when the stream closes. 127 func OptOnClose(onClose func()) func(*Type) { 128 return func(t *Type) { 129 t.onClose = onClose 130 } 131 } 132 133 //------------------------------------------------------------------------------ 134 135 // IsReady returns a boolean indicating whether both the input and output layers 136 // of the stream are connected. 137 func (t *Type) IsReady() bool { 138 return t.inputLayer.Connected() && t.outputLayer.Connected() 139 } 140 141 func (t *Type) start() (err error) { 142 // Constructors 143 iMgr, iLog, iStats := interop.LabelChild("input", t.manager, t.logger, t.stats) 144 if t.inputLayer, err = input.New(t.conf.Input, iMgr, iLog, iStats); err != nil { 145 return 146 } 147 if t.conf.Buffer.Type != buffer.TypeNone { 148 bMgr, bLog, bStats := interop.LabelChild("buffer", t.manager, t.logger, t.stats) 149 if t.bufferLayer, err = buffer.New(t.conf.Buffer, bMgr, bLog, bStats); err != nil { 150 return 151 } 152 } 153 if tLen := len(t.complementaryProcs) + len(t.conf.Pipeline.Processors); tLen > 0 { 154 pMgr, pLog, pStats := interop.LabelChild("pipeline", t.manager, t.logger, t.stats) 155 if t.pipelineLayer, err = pipeline.New(t.conf.Pipeline, pMgr, pLog, pStats, t.complementaryProcs...); err != nil { 156 return 157 } 158 } 159 oMgr, oLog, oStats := interop.LabelChild("output", t.manager, t.logger, t.stats) 160 if t.outputLayer, err = output.New(t.conf.Output, oMgr, oLog, oStats); err != nil { 161 return 162 } 163 164 // Start chaining components 165 var nextTranChan <-chan types.Transaction 166 167 nextTranChan = t.inputLayer.TransactionChan() 168 if t.bufferLayer != nil { 169 if err = t.bufferLayer.Consume(nextTranChan); err != nil { 170 return 171 } 172 nextTranChan = t.bufferLayer.TransactionChan() 173 } 174 if t.pipelineLayer != nil { 175 if err = t.pipelineLayer.Consume(nextTranChan); err != nil { 176 return 177 } 178 nextTranChan = t.pipelineLayer.TransactionChan() 179 } 180 if err = t.outputLayer.Consume(nextTranChan); err != nil { 181 return 182 } 183 184 go func(out output.Type) { 185 for { 186 if err := out.WaitForClose(time.Second); err == nil { 187 t.onClose() 188 return 189 } 190 } 191 }(t.outputLayer) 192 193 return nil 194 } 195 196 // stopGracefully attempts to close the stream in the most graceful way by only 197 // closing the input layer and waiting for all other layers to terminate by 198 // proxy. This should guarantee that all in-flight and buffered data is resolved 199 // before shutting down. 200 func (t *Type) stopGracefully(timeout time.Duration) (err error) { 201 t.inputLayer.CloseAsync() 202 started := time.Now() 203 if err = t.inputLayer.WaitForClose(timeout); err != nil { 204 return 205 } 206 207 var remaining time.Duration 208 209 // If we have a buffer then wait right here. We want to try and allow the 210 // buffer to empty out before prompting the other layers to shut down. 211 if t.bufferLayer != nil { 212 t.bufferLayer.StopConsuming() 213 remaining = timeout - time.Since(started) 214 if remaining < 0 { 215 return types.ErrTimeout 216 } 217 if err = t.bufferLayer.WaitForClose(remaining); err != nil { 218 return 219 } 220 } 221 222 // After this point we can start closing the remaining components. 223 if t.pipelineLayer != nil { 224 t.pipelineLayer.CloseAsync() 225 remaining = timeout - time.Since(started) 226 if remaining < 0 { 227 return types.ErrTimeout 228 } 229 if err = t.pipelineLayer.WaitForClose(remaining); err != nil { 230 return 231 } 232 } 233 234 t.outputLayer.CloseAsync() 235 remaining = timeout - time.Since(started) 236 if remaining < 0 { 237 return types.ErrTimeout 238 } 239 if err = t.outputLayer.WaitForClose(remaining); err != nil { 240 return 241 } 242 243 return nil 244 } 245 246 // stopOrdered attempts to close all components of the stream in the order of 247 // positions within the stream, this allows data to flush all the way through 248 // the pipeline under certain circumstances but is less graceful than 249 // stopGracefully, which should be attempted first. 250 func (t *Type) stopOrdered(timeout time.Duration) (err error) { 251 t.inputLayer.CloseAsync() 252 started := time.Now() 253 if err = t.inputLayer.WaitForClose(timeout); err != nil { 254 return 255 } 256 257 var remaining time.Duration 258 259 if t.bufferLayer != nil { 260 t.bufferLayer.CloseAsync() 261 remaining = timeout - time.Since(started) 262 if remaining < 0 { 263 return types.ErrTimeout 264 } 265 if err = t.bufferLayer.WaitForClose(remaining); err != nil { 266 return 267 } 268 } 269 270 if t.pipelineLayer != nil { 271 t.pipelineLayer.CloseAsync() 272 remaining = timeout - time.Since(started) 273 if remaining < 0 { 274 return types.ErrTimeout 275 } 276 if err = t.pipelineLayer.WaitForClose(remaining); err != nil { 277 return 278 } 279 } 280 281 t.outputLayer.CloseAsync() 282 remaining = timeout - time.Since(started) 283 if remaining < 0 { 284 return types.ErrTimeout 285 } 286 if err = t.outputLayer.WaitForClose(remaining); err != nil { 287 return 288 } 289 290 return nil 291 } 292 293 // stopUnorderd attempts to close all components in parallel without allowing 294 // the stream to gracefully wind down in the order of component layers. This 295 // should only be attempted if both stopGracefully and stopOrdered failed. 296 func (t *Type) stopUnordered(timeout time.Duration) (err error) { 297 t.inputLayer.CloseAsync() 298 if t.bufferLayer != nil { 299 t.bufferLayer.CloseAsync() 300 } 301 if t.pipelineLayer != nil { 302 t.pipelineLayer.CloseAsync() 303 } 304 t.outputLayer.CloseAsync() 305 306 started := time.Now() 307 if err = t.inputLayer.WaitForClose(timeout); err != nil { 308 return 309 } 310 311 var remaining time.Duration 312 313 if t.bufferLayer != nil { 314 remaining = timeout - time.Since(started) 315 if remaining < 0 { 316 return types.ErrTimeout 317 } 318 if err = t.bufferLayer.WaitForClose(remaining); err != nil { 319 return 320 } 321 } 322 323 if t.pipelineLayer != nil { 324 remaining = timeout - time.Since(started) 325 if remaining < 0 { 326 return types.ErrTimeout 327 } 328 if err = t.pipelineLayer.WaitForClose(remaining); err != nil { 329 return 330 } 331 } 332 333 remaining = timeout - time.Since(started) 334 if remaining < 0 { 335 return types.ErrTimeout 336 } 337 if err = t.outputLayer.WaitForClose(remaining); err != nil { 338 return 339 } 340 341 return nil 342 } 343 344 // Stop attempts to close the stream within the specified timeout period. 345 // Initially the attempt is graceful, but as the timeout draws close the attempt 346 // becomes progressively less graceful. 347 func (t *Type) Stop(timeout time.Duration) error { 348 tOutUnordered := timeout / 4 349 tOutGraceful := timeout - tOutUnordered 350 351 err := t.stopGracefully(tOutGraceful) 352 if err == nil { 353 return nil 354 } 355 if err == types.ErrTimeout { 356 t.logger.Infoln("Unable to fully drain buffered messages within target time.") 357 } else { 358 t.logger.Errorf("Encountered error whilst shutting down: %v\n", err) 359 } 360 361 err = t.stopUnordered(tOutUnordered) 362 if err == nil { 363 return nil 364 } 365 if err == types.ErrTimeout { 366 t.logger.Errorln("Failed to stop stream gracefully within target time.") 367 368 dumpBuf := bytes.NewBuffer(nil) 369 pprof.Lookup("goroutine").WriteTo(dumpBuf, 1) 370 371 t.logger.Debugln(dumpBuf.String()) 372 } else { 373 t.logger.Errorf("Encountered error whilst shutting down: %v\n", err) 374 } 375 376 return err 377 } 378 379 //------------------------------------------------------------------------------