github.com/Jeffail/benthos/v3@v3.65.0/lib/output/async_writer.go (about) 1 package output 2 3 import ( 4 "context" 5 "sync" 6 "sync/atomic" 7 "time" 8 9 "github.com/Jeffail/benthos/v3/internal/batch" 10 "github.com/Jeffail/benthos/v3/internal/bloblang/mapping" 11 "github.com/Jeffail/benthos/v3/internal/interop" 12 "github.com/Jeffail/benthos/v3/internal/shutdown" 13 "github.com/Jeffail/benthos/v3/internal/tracing" 14 "github.com/Jeffail/benthos/v3/lib/log" 15 "github.com/Jeffail/benthos/v3/lib/message" 16 "github.com/Jeffail/benthos/v3/lib/metrics" 17 "github.com/Jeffail/benthos/v3/lib/response" 18 "github.com/Jeffail/benthos/v3/lib/types" 19 "github.com/cenkalti/backoff/v4" 20 ) 21 22 // AsyncSink is a type that writes Benthos messages to a third party sink. If 23 // the protocol supports a form of acknowledgement then it will be returned by 24 // the call to Write. 25 type AsyncSink interface { 26 // ConnectWithContext attempts to establish a connection to the sink, if 27 // unsuccessful returns an error. If the attempt is successful (or not 28 // necessary) returns nil. 29 ConnectWithContext(ctx context.Context) error 30 31 // WriteWithContext should block until either the message is sent (and 32 // acknowledged) to a sink, or a transport specific error has occurred, or 33 // the Type is closed. 34 WriteWithContext(ctx context.Context, msg types.Message) error 35 36 types.Closable 37 } 38 39 // AsyncWriter is an output type that writes messages to a writer.Type. 40 type AsyncWriter struct { 41 isConnected int32 42 43 typeStr string 44 maxInflight int 45 noCancel bool 46 writer AsyncSink 47 48 injectTracingMap *mapping.Executor 49 50 mgr types.Manager 51 log log.Modular 52 stats metrics.Type 53 54 transactions <-chan types.Transaction 55 56 shutSig *shutdown.Signaller 57 } 58 59 // NewAsyncWriter creates a new AsyncWriter output type. 60 // Deprecated 61 func NewAsyncWriter( 62 typeStr string, 63 maxInflight int, 64 w AsyncSink, 65 log log.Modular, 66 stats metrics.Type, 67 ) (Type, error) { 68 return newAsyncWriter(typeStr, maxInflight, w, types.NoopMgr(), log, stats) 69 } 70 71 func newAsyncWriter( 72 typeStr string, 73 maxInflight int, 74 w AsyncSink, 75 mgr types.Manager, 76 log log.Modular, 77 stats metrics.Type, 78 ) (Type, error) { 79 aWriter := &AsyncWriter{ 80 typeStr: typeStr, 81 maxInflight: maxInflight, 82 writer: w, 83 mgr: mgr, 84 log: log, 85 stats: stats, 86 transactions: nil, 87 shutSig: shutdown.NewSignaller(), 88 } 89 return aWriter, nil 90 } 91 92 // SetInjectTracingMap sets a mapping to be used for injecting tracing events 93 // into messages. 94 func (w *AsyncWriter) SetInjectTracingMap(mapping string) error { 95 var err error 96 w.injectTracingMap, err = interop.NewBloblangMapping(w.mgr, mapping) 97 return err 98 } 99 100 // SetNoCancel configures the async writer so that write calls do not use a 101 // context that gets cancelled on shutdown. This is much more efficient as it 102 // reduces allocations, goroutines and defers for each write call, but also 103 // means the write can block graceful termination. Therefore this setting should 104 // be reserved for outputs that are exceptionally fast. 105 func (w *AsyncWriter) SetNoCancel() { 106 w.noCancel = true 107 } 108 109 //------------------------------------------------------------------------------ 110 111 func (w *AsyncWriter) latencyMeasuringWrite(msg types.Message) (latencyNs int64, err error) { 112 t0 := time.Now() 113 var ctx context.Context 114 if w.noCancel { 115 ctx = context.Background() 116 } else { 117 var done func() 118 ctx, done = w.shutSig.CloseAtLeisureCtx(context.Background()) 119 defer done() 120 } 121 err = w.writer.WriteWithContext(ctx, msg) 122 latencyNs = time.Since(t0).Nanoseconds() 123 return latencyNs, err 124 } 125 126 func (w *AsyncWriter) injectSpans(msg types.Message, spans []*tracing.Span) types.Message { 127 if w.injectTracingMap == nil || msg.Len() > len(spans) { 128 return msg 129 } 130 131 parts := make([]types.Part, msg.Len()) 132 133 for i := 0; i < msg.Len(); i++ { 134 parts[i] = msg.Get(i).Copy() 135 136 spanMapGeneric, err := spans[i].TextMap() 137 if err != nil { 138 w.log.Warnf("Failed to inject span: %v", err) 139 continue 140 } 141 142 spanPart := message.NewPart(nil) 143 if err = spanPart.SetJSON(spanMapGeneric); err != nil { 144 w.log.Warnf("Failed to inject span: %v", err) 145 continue 146 } 147 148 spanMsg := message.New(nil) 149 spanMsg.Append(spanPart) 150 151 if parts[i], err = w.injectTracingMap.MapOnto(parts[i], i, spanMsg); err != nil { 152 w.log.Warnf("Failed to inject span: %v", err) 153 parts[i] = msg.Get(i) 154 } 155 } 156 157 newMsg := message.New(nil) 158 newMsg.SetAll(parts) 159 return newMsg 160 } 161 162 // loop is an internal loop that brokers incoming messages to output pipe. 163 func (w *AsyncWriter) loop() { 164 // Metrics paths 165 var ( 166 mCount = w.stats.GetCounter("count") 167 mPartsSent = w.stats.GetCounter("sent") 168 mSent = w.stats.GetCounter("batch.sent") 169 mBytesSent = w.stats.GetCounter("batch.bytes") 170 mLatency = w.stats.GetTimer("batch.latency") 171 mConn = w.stats.GetCounter("connection.up") 172 mFailedConn = w.stats.GetCounter("connection.failed") 173 mLostConn = w.stats.GetCounter("connection.lost") 174 ) 175 176 defer func() { 177 w.writer.CloseAsync() 178 _ = w.writer.WaitForClose(shutdown.MaximumShutdownWait()) 179 180 atomic.StoreInt32(&w.isConnected, 0) 181 w.shutSig.ShutdownComplete() 182 }() 183 184 connBackoff := backoff.NewExponentialBackOff() 185 connBackoff.InitialInterval = time.Millisecond * 500 186 connBackoff.MaxInterval = time.Second 187 connBackoff.MaxElapsedTime = 0 188 189 initConnection := func() bool { 190 initConnCtx, initConnDone := w.shutSig.CloseAtLeisureCtx(context.Background()) 191 defer initConnDone() 192 for { 193 if err := w.writer.ConnectWithContext(initConnCtx); err != nil { 194 if w.shutSig.ShouldCloseAtLeisure() || err == types.ErrTypeClosed { 195 return false 196 } 197 w.log.Errorf("Failed to connect to %v: %v\n", w.typeStr, err) 198 mFailedConn.Incr(1) 199 select { 200 case <-time.After(connBackoff.NextBackOff()): 201 case <-initConnCtx.Done(): 202 return false 203 } 204 } else { 205 connBackoff.Reset() 206 return true 207 } 208 } 209 } 210 if !initConnection() { 211 return 212 } 213 mConn.Incr(1) 214 atomic.StoreInt32(&w.isConnected, 1) 215 216 wg := sync.WaitGroup{} 217 wg.Add(w.maxInflight) 218 219 connectMut := sync.Mutex{} 220 connectLoop := func(msg types.Message) (latency int64, err error) { 221 atomic.StoreInt32(&w.isConnected, 0) 222 223 connectMut.Lock() 224 defer connectMut.Unlock() 225 226 // If another goroutine got here first and we're able to send over the 227 // connection, then we gracefully accept defeat. 228 if atomic.LoadInt32(&w.isConnected) == 1 { 229 if latency, err = w.latencyMeasuringWrite(msg); err != types.ErrNotConnected { 230 return 231 } 232 } 233 mLostConn.Incr(1) 234 235 // Continue to try to reconnect while still active. 236 for { 237 if !initConnection() { 238 err = types.ErrTypeClosed 239 return 240 } 241 if latency, err = w.latencyMeasuringWrite(msg); err != types.ErrNotConnected { 242 atomic.StoreInt32(&w.isConnected, 1) 243 mConn.Incr(1) 244 return 245 } 246 } 247 } 248 249 writerLoop := func() { 250 defer wg.Done() 251 252 for { 253 var ts types.Transaction 254 var open bool 255 select { 256 case ts, open = <-w.transactions: 257 if !open { 258 return 259 } 260 mCount.Incr(1) 261 case <-w.shutSig.CloseAtLeisureChan(): 262 return 263 } 264 265 w.log.Tracef("Attempting to write %v messages to '%v'.\n", ts.Payload.Len(), w.typeStr) 266 spans := tracing.CreateChildSpans("output_"+w.typeStr, ts.Payload) 267 ts.Payload = w.injectSpans(ts.Payload, spans) 268 269 latency, err := w.latencyMeasuringWrite(ts.Payload) 270 271 // If our writer says it is not connected. 272 if err == types.ErrNotConnected { 273 latency, err = connectLoop(ts.Payload) 274 } 275 276 // Close immediately if our writer is closed. 277 if err == types.ErrTypeClosed { 278 return 279 } 280 281 if err != nil { 282 if w.typeStr != TypeReject { 283 // TODO: Maybe reintroduce a sleep here if we encounter a 284 // busy retry loop. 285 w.log.Errorf("Failed to send message to %v: %v\n", w.typeStr, err) 286 } else { 287 w.log.Debugf("Rejecting message: %v\n", err) 288 } 289 } else { 290 mSent.Incr(1) 291 mPartsSent.Incr(int64(batch.MessageCollapsedCount(ts.Payload))) 292 mBytesSent.Incr(int64(message.GetAllBytesLen(ts.Payload))) 293 mLatency.Timing(latency) 294 w.log.Tracef("Successfully wrote %v messages to '%v'.\n", ts.Payload.Len(), w.typeStr) 295 } 296 297 for _, s := range spans { 298 s.Finish() 299 } 300 301 select { 302 case ts.ResponseChan <- response.NewError(err): 303 case <-w.shutSig.CloseAtLeisureChan(): 304 return 305 } 306 } 307 } 308 309 for i := 0; i < w.maxInflight; i++ { 310 go writerLoop() 311 } 312 wg.Wait() 313 } 314 315 // Consume assigns a messages channel for the output to read. 316 func (w *AsyncWriter) Consume(ts <-chan types.Transaction) error { 317 if w.transactions != nil { 318 return types.ErrAlreadyStarted 319 } 320 w.transactions = ts 321 go w.loop() 322 return nil 323 } 324 325 // Connected returns a boolean indicating whether this output is currently 326 // connected to its target. 327 func (w *AsyncWriter) Connected() bool { 328 return atomic.LoadInt32(&w.isConnected) == 1 329 } 330 331 // MaxInFlight returns the maximum number of in flight messages permitted by the 332 // output. This value can be used to determine a sensible value for parent 333 // outputs, but should not be relied upon as part of dispatcher logic. 334 func (w *AsyncWriter) MaxInFlight() (int, bool) { 335 return w.maxInflight, true 336 } 337 338 // CloseAsync shuts down the File output and stops processing messages. 339 func (w *AsyncWriter) CloseAsync() { 340 w.shutSig.CloseAtLeisure() 341 } 342 343 // WaitForClose blocks until the File output has closed down. 344 func (w *AsyncWriter) WaitForClose(timeout time.Duration) error { 345 select { 346 case <-w.shutSig.HasClosedChan(): 347 case <-time.After(timeout): 348 return types.ErrTimeout 349 } 350 return nil 351 }