github.com/Jeffail/benthos/v3@v3.65.0/lib/broker/dynamic_fan_out.go (about) 1 package broker 2 3 import ( 4 "context" 5 "fmt" 6 "sync" 7 "time" 8 9 "github.com/Jeffail/benthos/v3/lib/log" 10 "github.com/Jeffail/benthos/v3/lib/metrics" 11 "github.com/Jeffail/benthos/v3/lib/response" 12 "github.com/Jeffail/benthos/v3/lib/types" 13 "github.com/Jeffail/benthos/v3/lib/util/throttle" 14 "golang.org/x/sync/errgroup" 15 ) 16 17 //------------------------------------------------------------------------------ 18 19 // DynamicOutput is an interface of output types that must be closable. 20 type DynamicOutput interface { 21 types.Output 22 } 23 24 // wrappedOutput is a struct that wraps a DynamicOutput with an identifying 25 // name. 26 type wrappedOutput struct { 27 Ctx context.Context 28 Name string 29 Output DynamicOutput 30 ResChan chan<- error 31 } 32 33 // outputWithTSChan is a struct containing both an output and the transaction 34 // chan it reads from. 35 type outputWithTSChan struct { 36 tsChan chan types.Transaction 37 output DynamicOutput 38 ctx context.Context 39 done func() 40 } 41 42 //------------------------------------------------------------------------------ 43 44 // DynamicFanOut is a broker that implements types.Consumer and broadcasts each 45 // message out to a dynamic map of outputs. 46 type DynamicFanOut struct { 47 maxInFlight int 48 49 log log.Modular 50 stats metrics.Type 51 52 onAdd func(label string) 53 onRemove func(label string) 54 55 transactions <-chan types.Transaction 56 57 outputsMut sync.RWMutex 58 newOutputChan chan wrappedOutput 59 outputs map[string]outputWithTSChan 60 61 ctx context.Context 62 close func() 63 closedChan chan struct{} 64 } 65 66 // NewDynamicFanOut creates a new DynamicFanOut type by providing outputs. 67 func NewDynamicFanOut( 68 outputs map[string]DynamicOutput, 69 logger log.Modular, 70 stats metrics.Type, 71 options ...func(*DynamicFanOut), 72 ) (*DynamicFanOut, error) { 73 ctx, done := context.WithCancel(context.Background()) 74 d := &DynamicFanOut{ 75 maxInFlight: 1, 76 stats: stats, 77 log: logger, 78 onAdd: func(l string) {}, 79 onRemove: func(l string) {}, 80 transactions: nil, 81 newOutputChan: make(chan wrappedOutput), 82 outputs: make(map[string]outputWithTSChan, len(outputs)), 83 closedChan: make(chan struct{}), 84 ctx: ctx, 85 close: done, 86 } 87 for _, opt := range options { 88 opt(d) 89 } 90 91 for k, v := range outputs { 92 if err := d.addOutput(k, v); err != nil { 93 return nil, fmt.Errorf("failed to initialise dynamic output '%v': %v", k, err) 94 } 95 d.onAdd(k) 96 } 97 return d, nil 98 } 99 100 // WithMaxInFlight sets the maximum number of in-flight messages this broker 101 // supports. This must be set before calling Consume. 102 func (d *DynamicFanOut) WithMaxInFlight(i int) *DynamicFanOut { 103 if i < 1 { 104 i = 1 105 } 106 d.maxInFlight = i 107 return d 108 } 109 110 // SetOutput attempts to add a new output to the dynamic output broker. If an 111 // output already exists with the same identifier it will be closed and removed. 112 // If either action takes longer than the timeout period an error will be 113 // returned. 114 // 115 // A nil output argument is safe and will simply remove the previous output 116 // under the indentifier, if there was one. 117 // 118 // TODO: V4 use context here instead. 119 func (d *DynamicFanOut) SetOutput(ident string, output DynamicOutput, timeout time.Duration) error { 120 ctx, done := context.WithTimeout(d.ctx, timeout) 121 defer done() 122 resChan := make(chan error, 1) 123 select { 124 case d.newOutputChan <- wrappedOutput{ 125 Name: ident, 126 Output: output, 127 ResChan: resChan, 128 Ctx: ctx, 129 }: 130 case <-ctx.Done(): 131 return types.ErrTimeout 132 } 133 select { 134 case err := <-resChan: 135 return err 136 case <-ctx.Done(): 137 } 138 return types.ErrTimeout 139 } 140 141 //------------------------------------------------------------------------------ 142 143 // OptDynamicFanOutSetOnAdd sets the function that is called whenever a dynamic 144 // output is added. 145 func OptDynamicFanOutSetOnAdd(onAddFunc func(label string)) func(*DynamicFanOut) { 146 return func(d *DynamicFanOut) { 147 d.onAdd = onAddFunc 148 } 149 } 150 151 // OptDynamicFanOutSetOnRemove sets the function that is called whenever a 152 // dynamic output is removed. 153 func OptDynamicFanOutSetOnRemove(onRemoveFunc func(label string)) func(*DynamicFanOut) { 154 return func(d *DynamicFanOut) { 155 d.onRemove = onRemoveFunc 156 } 157 } 158 159 //------------------------------------------------------------------------------ 160 161 // Consume assigns a new transactions channel for the broker to read. 162 func (d *DynamicFanOut) Consume(transactions <-chan types.Transaction) error { 163 if d.transactions != nil { 164 return types.ErrAlreadyStarted 165 } 166 d.transactions = transactions 167 168 go d.loop() 169 return nil 170 } 171 172 //------------------------------------------------------------------------------ 173 174 func (d *DynamicFanOut) addOutput(ident string, output DynamicOutput) error { 175 if _, exists := d.outputs[ident]; exists { 176 return fmt.Errorf("output key '%v' already exists", ident) 177 } 178 179 ow := outputWithTSChan{ 180 tsChan: make(chan types.Transaction), 181 output: output, 182 } 183 184 if err := output.Consume(ow.tsChan); err != nil { 185 output.CloseAsync() 186 return err 187 } 188 ow.ctx, ow.done = context.WithCancel(context.Background()) 189 190 d.outputs[ident] = ow 191 return nil 192 } 193 194 func (d *DynamicFanOut) removeOutput(ctx context.Context, ident string) error { 195 ow, exists := d.outputs[ident] 196 if !exists { 197 return nil 198 } 199 200 timeout := time.Second * 5 201 if deadline, ok := ctx.Deadline(); ok { 202 timeout = time.Until(deadline) 203 } 204 205 ow.output.CloseAsync() 206 err := ow.output.WaitForClose(timeout) 207 208 ow.done() 209 close(ow.tsChan) 210 delete(d.outputs, ident) 211 212 return err 213 } 214 215 //------------------------------------------------------------------------------ 216 217 // loop is an internal loop that brokers incoming messages to many outputs. 218 func (d *DynamicFanOut) loop() { 219 var ( 220 wg = sync.WaitGroup{} 221 mCount = d.stats.GetCounter("count") 222 mRemoveErr = d.stats.GetCounter("output.remove.error") 223 mRemoveSucc = d.stats.GetCounter("output.remove.success") 224 mAddErr = d.stats.GetCounter("output.add.error") 225 mAddSucc = d.stats.GetCounter("output.add.success") 226 mMsgsRcd = d.stats.GetCounter("messages.received") 227 mOutputErr = d.stats.GetCounter("output.error") 228 mMsgsSnt = d.stats.GetCounter("messages.sent") 229 ) 230 231 defer func() { 232 wg.Wait() 233 for _, ow := range d.outputs { 234 ow.output.CloseAsync() 235 close(ow.tsChan) 236 } 237 for _, ow := range d.outputs { 238 if err := ow.output.WaitForClose(time.Second); err != nil { 239 for err != nil { 240 err = ow.output.WaitForClose(time.Second) 241 } 242 } 243 } 244 d.outputs = map[string]outputWithTSChan{} 245 close(d.closedChan) 246 }() 247 248 wg.Add(1) 249 go func() { 250 defer wg.Done() 251 252 for { 253 select { 254 case wrappedOutput, open := <-d.newOutputChan: 255 if !open { 256 return 257 } 258 func() { 259 d.outputsMut.Lock() 260 defer d.outputsMut.Unlock() 261 262 // First, always remove the previous output if it exists. 263 if _, exists := d.outputs[wrappedOutput.Name]; exists { 264 if err := d.removeOutput(wrappedOutput.Ctx, wrappedOutput.Name); err != nil { 265 mRemoveErr.Incr(1) 266 d.log.Errorf("Failed to stop old copy of dynamic output '%v' in time: %v, the output will continue to shut down in the background.\n", wrappedOutput.Name, err) 267 } else { 268 mRemoveSucc.Incr(1) 269 } 270 d.onRemove(wrappedOutput.Name) 271 } 272 273 // Next, attempt to create a new output (if specified). 274 if wrappedOutput.Output == nil { 275 wrappedOutput.ResChan <- nil 276 } else { 277 err := d.addOutput(wrappedOutput.Name, wrappedOutput.Output) 278 if err != nil { 279 mAddErr.Incr(1) 280 d.log.Errorf("Failed to start new dynamic output '%v': %v\n", wrappedOutput.Name, err) 281 } else { 282 mAddSucc.Incr(1) 283 d.onAdd(wrappedOutput.Name) 284 } 285 wrappedOutput.ResChan <- err 286 } 287 }() 288 case <-d.ctx.Done(): 289 return 290 } 291 } 292 }() 293 294 sendLoop := func() { 295 defer wg.Done() 296 297 for { 298 var ts types.Transaction 299 var open bool 300 select { 301 case ts, open = <-d.transactions: 302 if !open { 303 d.close() 304 return 305 } 306 mCount.Incr(1) 307 case <-d.ctx.Done(): 308 return 309 } 310 mMsgsRcd.Incr(1) 311 312 d.outputsMut.RLock() 313 for len(d.outputs) == 0 { 314 // Assuming this isn't a common enough occurrence that it 315 // won't be busy enough to require a sync.Cond, looping with 316 // a sleep is fine for now. 317 d.outputsMut.RUnlock() 318 select { 319 case <-time.After(time.Millisecond * 10): 320 case <-d.ctx.Done(): 321 return 322 } 323 d.outputsMut.RLock() 324 } 325 326 var owg errgroup.Group 327 for name := range d.outputs { 328 msgCopy, name := ts.Payload.Copy(), name 329 owg.Go(func() error { 330 throt := throttle.New(throttle.OptCloseChan(d.ctx.Done())) 331 resChan := make(chan types.Response) 332 333 // Try until success, shutdown, or the output was removed. 334 for { 335 d.outputsMut.RLock() 336 output, exists := d.outputs[name] 337 if !exists { 338 d.outputsMut.RUnlock() 339 return nil 340 } 341 342 select { 343 case output.tsChan <- types.NewTransaction(msgCopy, resChan): 344 case <-d.ctx.Done(): 345 d.outputsMut.RUnlock() 346 return types.ErrTypeClosed 347 } 348 349 // Allow outputs to be mutated at this stage in case the 350 // response is slow. 351 d.outputsMut.RUnlock() 352 353 select { 354 case res := <-resChan: 355 if res.Error() != nil { 356 d.log.Errorf("Failed to dispatch dynamic fan out message to '%v': %v\n", name, res.Error()) 357 mOutputErr.Incr(1) 358 if cont := throt.Retry(); !cont { 359 return types.ErrTypeClosed 360 } 361 } else { 362 mMsgsSnt.Incr(1) 363 return nil 364 } 365 case <-output.ctx.Done(): 366 return nil 367 case <-d.ctx.Done(): 368 return types.ErrTypeClosed 369 } 370 } 371 }) 372 } 373 d.outputsMut.RUnlock() 374 375 if owg.Wait() == nil { 376 select { 377 case ts.ResponseChan <- response.NewAck(): 378 case <-d.ctx.Done(): 379 return 380 } 381 } 382 } 383 } 384 385 // Max in flight 386 for i := 0; i < d.maxInFlight; i++ { 387 wg.Add(1) 388 go sendLoop() 389 } 390 } 391 392 // Connected returns a boolean indicating whether this output is currently 393 // connected to its target. 394 func (d *DynamicFanOut) Connected() bool { 395 d.outputsMut.RLock() 396 defer d.outputsMut.RUnlock() 397 for _, out := range d.outputs { 398 if !out.output.Connected() { 399 return false 400 } 401 } 402 return true 403 } 404 405 // MaxInFlight returns the maximum number of in flight messages permitted by the 406 // output. This value can be used to determine a sensible value for parent 407 // outputs, but should not be relied upon as part of dispatcher logic. 408 func (d *DynamicFanOut) MaxInFlight() (int, bool) { 409 return d.maxInFlight, true 410 } 411 412 // CloseAsync shuts down the DynamicFanOut broker and stops processing requests. 413 func (d *DynamicFanOut) CloseAsync() { 414 d.close() 415 } 416 417 // WaitForClose blocks until the DynamicFanOut broker has closed down. 418 func (d *DynamicFanOut) WaitForClose(timeout time.Duration) error { 419 select { 420 case <-d.closedChan: 421 case <-time.After(timeout): 422 return types.ErrTimeout 423 } 424 return nil 425 } 426 427 //------------------------------------------------------------------------------