github.com/Jeffail/benthos/v3@v3.65.0/lib/output/switch.go (about) 1 package output 2 3 import ( 4 "context" 5 "errors" 6 "fmt" 7 "sync" 8 "time" 9 10 "github.com/Jeffail/benthos/v3/internal/batch" 11 "github.com/Jeffail/benthos/v3/internal/bloblang/mapping" 12 "github.com/Jeffail/benthos/v3/internal/component/output" 13 "github.com/Jeffail/benthos/v3/internal/docs" 14 "github.com/Jeffail/benthos/v3/internal/interop" 15 imessage "github.com/Jeffail/benthos/v3/internal/message" 16 "github.com/Jeffail/benthos/v3/internal/shutdown" 17 "github.com/Jeffail/benthos/v3/lib/condition" 18 "github.com/Jeffail/benthos/v3/lib/log" 19 "github.com/Jeffail/benthos/v3/lib/message" 20 "github.com/Jeffail/benthos/v3/lib/metrics" 21 "github.com/Jeffail/benthos/v3/lib/response" 22 "github.com/Jeffail/benthos/v3/lib/types" 23 "github.com/Jeffail/benthos/v3/lib/util/throttle" 24 "github.com/Jeffail/gabs/v2" 25 "golang.org/x/sync/errgroup" 26 ) 27 28 //------------------------------------------------------------------------------ 29 30 var ( 31 // ErrSwitchNoConditionMet is returned when a message does not match any 32 // output conditions. 33 ErrSwitchNoConditionMet = errors.New("no switch output conditions were met by message") 34 // ErrSwitchNoCasesMatched is returned when a message does not match any 35 // output cases. 36 ErrSwitchNoCasesMatched = errors.New("no switch cases were matched by message") 37 // ErrSwitchNoOutputs is returned when creating a Switch type with less than 38 // 2 outputs. 39 ErrSwitchNoOutputs = errors.New("attempting to create switch with fewer than 2 cases") 40 ) 41 42 //------------------------------------------------------------------------------ 43 44 func init() { 45 Constructors[TypeSwitch] = TypeSpec{ 46 constructor: fromSimpleConstructor(NewSwitch), 47 Summary: ` 48 The switch output type allows you to route messages to different outputs based on their contents.`, 49 Description: ` 50 Messages must successfully route to one or more outputs, otherwise this is considered an error and the message is reprocessed. In order to explicitly drop messages that do not match your cases add one final case with a [drop output](/docs/components/outputs/drop).`, 51 config: docs.FieldComponent().WithChildren( 52 docs.FieldCommon( 53 "retry_until_success", ` 54 If a selected output fails to send a message this field determines whether it is 55 reattempted indefinitely. If set to false the error is instead propagated back 56 to the input level. 57 58 If a message can be routed to >1 outputs it is usually best to set this to true 59 in order to avoid duplicate messages being routed to an output.`, 60 ), 61 docs.FieldAdvanced( 62 "strict_mode", ` 63 This field determines whether an error should be reported if no condition is met. 64 If set to true, an error is propagated back to the input level. The default 65 behavior is false, which will drop the message.`, 66 ), 67 docs.FieldAdvanced( 68 "max_in_flight", "The maximum number of parallel message batches to have in flight at any given time. Note that if a child output has a higher `max_in_flight` then the switch output will automatically match it, therefore this value is the minimum `max_in_flight` to set in cases where the child values can't be inferred (such as when using resource outputs as children).", 69 ), 70 docs.FieldCommon( 71 "cases", 72 "A list of switch cases, outlining outputs that can be routed to.", 73 []interface{}{ 74 map[string]interface{}{ 75 "check": `this.urls.contains("http://benthos.dev")`, 76 "output": map[string]interface{}{ 77 "cache": map[string]interface{}{ 78 "target": "foo", 79 "key": "${!json(\"id\")}", 80 }, 81 }, 82 "continue": true, 83 }, 84 map[string]interface{}{ 85 "output": map[string]interface{}{ 86 "s3": map[string]interface{}{ 87 "bucket": "bar", 88 "path": "${!json(\"id\")}", 89 }, 90 }, 91 }, 92 }, 93 ).Array().WithChildren( 94 docs.FieldBloblang( 95 "check", 96 "A [Bloblang query](/docs/guides/bloblang/about/) that should return a boolean value indicating whether a message should be routed to the case output. If left empty the case always passes.", 97 `this.type == "foo"`, 98 `this.contents.urls.contains("https://benthos.dev/")`, 99 ).HasDefault(""), 100 docs.FieldCommon( 101 "output", "An [output](/docs/components/outputs/about/) for messages that pass the check to be routed to.", 102 ).HasDefault(map[string]interface{}{}).HasType(docs.FieldTypeOutput), 103 docs.FieldAdvanced( 104 "continue", 105 "Indicates whether, if this case passes for a message, the next case should also be tested.", 106 ).HasDefault(false).HasType(docs.FieldTypeBool), 107 ), 108 docs.FieldDeprecated("outputs").Array().WithChildren( 109 docs.FieldDeprecated("condition").HasType(docs.FieldTypeCondition), 110 docs.FieldDeprecated("fallthrough"), 111 docs.FieldDeprecated("output").HasType(docs.FieldTypeOutput), 112 ).OmitWhen(func(v, _ interface{}) (string, bool) { 113 arr, ok := v.([]interface{}) 114 return "field outputs is deprecated in favour of cases", ok && len(arr) == 0 115 }), 116 ).Linter(func(ctx docs.LintContext, line, col int, value interface{}) []docs.Lint { 117 if _, ok := value.(map[string]interface{}); !ok { 118 return nil 119 } 120 gObj := gabs.Wrap(value) 121 retry, exists := gObj.S("retry_until_success").Data().(bool) 122 // TODO: V4 Is retry_until_success going to be false by default now? 123 if exists && !retry { 124 return nil 125 } 126 for _, cObj := range gObj.S("cases").Children() { 127 typeStr, _ := cObj.S("output", "type").Data().(string) 128 isReject := cObj.Exists("output", "reject") 129 if typeStr == "reject" || isReject { 130 return []docs.Lint{ 131 docs.NewLintError(line, "a `switch` output with a `reject` case output must have the field `switch.retry_until_success` set to `false` (defaults to `true`), otherwise the `reject` child output will result in infinite retries"), 132 } 133 } 134 } 135 return nil 136 }), 137 Categories: []Category{ 138 CategoryUtility, 139 }, 140 Examples: []docs.AnnotatedExample{ 141 { 142 Title: "Basic Multiplexing", 143 Summary: ` 144 The most common use for a switch output is to multiplex messages across a range of output destinations. The following config checks the contents of the field ` + "`type` of messages and sends `foo` type messages to an `amqp_1` output, `bar` type messages to a `gcp_pubsub` output, and everything else to a `redis_streams` output" + `. 145 146 Outputs can have their own processors associated with them, and in this example the ` + "`redis_streams`" + ` output has a processor that enforces the presence of a type field before sending it.`, 147 Config: ` 148 output: 149 switch: 150 cases: 151 - check: this.type == "foo" 152 output: 153 amqp_1: 154 url: amqps://guest:guest@localhost:5672/ 155 target_address: queue:/the_foos 156 157 - check: this.type == "bar" 158 output: 159 gcp_pubsub: 160 project: dealing_with_mike 161 topic: mikes_bars 162 163 - output: 164 redis_streams: 165 url: tcp://localhost:6379 166 stream: everything_else 167 processors: 168 - bloblang: | 169 root = this 170 root.type = this.type | "unknown" 171 `, 172 }, 173 { 174 Title: "Control Flow", 175 Summary: ` 176 The ` + "`continue`" + ` field allows messages that have passed a case to be tested against the next one also. This can be useful when combining non-mutually-exclusive case checks. 177 178 In the following example a message that passes both the check of the first case as well as the second will be routed to both.`, 179 Config: ` 180 output: 181 switch: 182 cases: 183 - check: 'this.user.interests.contains("walks").catch(false)' 184 output: 185 amqp_1: 186 url: amqps://guest:guest@localhost:5672/ 187 target_address: queue:/people_what_think_good 188 continue: true 189 190 - check: 'this.user.dislikes.contains("videogames").catch(false)' 191 output: 192 gcp_pubsub: 193 project: people 194 topic: that_i_dont_want_to_hang_with 195 `, 196 }, 197 }, 198 } 199 } 200 201 //------------------------------------------------------------------------------ 202 203 // SwitchConfig contains configuration fields for the Switch output type. 204 type SwitchConfig struct { 205 RetryUntilSuccess bool `json:"retry_until_success" yaml:"retry_until_success"` 206 StrictMode bool `json:"strict_mode" yaml:"strict_mode"` 207 MaxInFlight int `json:"max_in_flight" yaml:"max_in_flight"` 208 Cases []SwitchConfigCase `json:"cases" yaml:"cases"` 209 Outputs []SwitchConfigOutput `json:"outputs" yaml:"outputs"` 210 } 211 212 // NewSwitchConfig creates a new SwitchConfig with default values. 213 func NewSwitchConfig() SwitchConfig { 214 return SwitchConfig{ 215 RetryUntilSuccess: true, 216 // TODO: V4 consider making this true by default. 217 StrictMode: false, 218 MaxInFlight: 1, 219 Cases: []SwitchConfigCase{}, 220 Outputs: []SwitchConfigOutput{}, 221 } 222 } 223 224 // SwitchConfigCase contains configuration fields per output of a switch type. 225 type SwitchConfigCase struct { 226 Check string `json:"check" yaml:"check"` 227 Continue bool `json:"continue" yaml:"continue"` 228 Output Config `json:"output" yaml:"output"` 229 } 230 231 // NewSwitchConfigCase creates a new switch output config with default values. 232 func NewSwitchConfigCase() SwitchConfigCase { 233 return SwitchConfigCase{ 234 Check: "", 235 Continue: false, 236 Output: NewConfig(), 237 } 238 } 239 240 //------------------------------------------------------------------------------ 241 242 // Switch is a broker that implements types.Consumer and broadcasts each message 243 // out to an array of outputs. 244 type Switch struct { 245 logger log.Modular 246 stats metrics.Type 247 mMsgRcvd metrics.StatCounter 248 mMsgSnt metrics.StatCounter 249 mOutputErr metrics.StatCounter 250 251 maxInFlight int 252 transactions <-chan types.Transaction 253 254 retryUntilSuccess bool 255 strictMode bool 256 outputTSChans []chan types.Transaction 257 outputs []types.Output 258 checks []*mapping.Executor 259 conditions []types.Condition 260 continues []bool 261 fallthroughs []bool 262 263 ctx context.Context 264 close func() 265 closedChan chan struct{} 266 } 267 268 // NewSwitch creates a new Switch type by providing outputs. Messages will be 269 // sent to a subset of outputs according to condition and fallthrough settings. 270 func NewSwitch( 271 conf Config, 272 mgr types.Manager, 273 logger log.Modular, 274 stats metrics.Type, 275 ) (Type, error) { 276 ctx, done := context.WithCancel(context.Background()) 277 o := &Switch{ 278 stats: stats, 279 logger: logger, 280 maxInFlight: conf.Switch.MaxInFlight, 281 transactions: nil, 282 retryUntilSuccess: conf.Switch.RetryUntilSuccess, 283 strictMode: conf.Switch.StrictMode, 284 closedChan: make(chan struct{}), 285 ctx: ctx, 286 close: done, 287 mMsgRcvd: stats.GetCounter("switch.messages.received"), 288 mMsgSnt: stats.GetCounter("switch.messages.sent"), 289 mOutputErr: stats.GetCounter("switch.output.error"), 290 } 291 292 lCases := len(conf.Switch.Cases) 293 lOutputs := len(conf.Switch.Outputs) 294 if lCases < 2 && lOutputs < 2 { 295 return nil, ErrSwitchNoOutputs 296 } 297 if lCases > 0 { 298 if lOutputs > 0 { 299 return nil, errors.New("combining switch cases with deprecated outputs is not supported") 300 } 301 o.outputs = make([]types.Output, lCases) 302 o.checks = make([]*mapping.Executor, lCases) 303 o.continues = make([]bool, lCases) 304 o.fallthroughs = make([]bool, lCases) 305 } else { 306 o.outputs = make([]types.Output, lOutputs) 307 o.conditions = make([]types.Condition, lOutputs) 308 o.fallthroughs = make([]bool, lOutputs) 309 } 310 311 var err error 312 for i, oConf := range conf.Switch.Outputs { 313 ns := fmt.Sprintf("switch.%v", i) 314 oMgr, oLog, oStats := interop.LabelChild(ns+".output", mgr, logger, stats) 315 oStats = metrics.Combine(stats, oStats) 316 if o.outputs[i], err = New(oConf.Output, oMgr, oLog, oStats); err != nil { 317 return nil, fmt.Errorf("failed to create output '%v' type '%v': %v", i, oConf.Output.Type, err) 318 } 319 cMgr, cLog, cStats := interop.LabelChild(ns+".condition", mgr, logger, stats) 320 if o.conditions[i], err = condition.New(oConf.Condition, cMgr, cLog, cStats); err != nil { 321 return nil, fmt.Errorf("failed to create output '%v' condition '%v': %v", i, oConf.Condition.Type, err) 322 } 323 o.fallthroughs[i] = oConf.Fallthrough 324 } 325 326 for i, cConf := range conf.Switch.Cases { 327 oMgr, oLog, oStats := interop.LabelChild(fmt.Sprintf("switch.%v.output", i), mgr, logger, stats) 328 oStats = metrics.Combine(stats, oStats) 329 if o.outputs[i], err = New(cConf.Output, oMgr, oLog, oStats); err != nil { 330 return nil, fmt.Errorf("failed to create case '%v' output type '%v': %v", i, cConf.Output.Type, err) 331 } 332 if len(cConf.Check) > 0 { 333 if o.checks[i], err = interop.NewBloblangMapping(mgr, cConf.Check); err != nil { 334 return nil, fmt.Errorf("failed to parse case '%v' check mapping: %v", i, err) 335 } 336 } 337 o.continues[i] = cConf.Continue 338 } 339 340 o.outputTSChans = make([]chan types.Transaction, len(o.outputs)) 341 for i := range o.outputTSChans { 342 if mif, ok := output.GetMaxInFlight(o.outputs[i]); ok && mif > o.maxInFlight { 343 o.maxInFlight = mif 344 } 345 o.outputTSChans[i] = make(chan types.Transaction) 346 if err := o.outputs[i].Consume(o.outputTSChans[i]); err != nil { 347 return nil, err 348 } 349 } 350 return o, nil 351 } 352 353 //------------------------------------------------------------------------------ 354 355 // Consume assigns a new transactions channel for the broker to read. 356 func (o *Switch) Consume(transactions <-chan types.Transaction) error { 357 if o.transactions != nil { 358 return types.ErrAlreadyStarted 359 } 360 o.transactions = transactions 361 362 if len(o.conditions) > 0 { 363 o.logger.Warnf("Using deprecated field `outputs` which will be removed in the next major release of Benthos. For more information check out the docs at https://www.benthos.dev/docs/components/outputs/switch.") 364 go o.loopDeprecated() 365 } else { 366 go o.loop() 367 } 368 return nil 369 } 370 371 // MaxInFlight returns the maximum number of in flight messages permitted by the 372 // output. This value can be used to determine a sensible value for parent 373 // outputs, but should not be relied upon as part of dispatcher logic. 374 func (o *Switch) MaxInFlight() (int, bool) { 375 return o.maxInFlight, true 376 } 377 378 // Connected returns a boolean indicating whether this output is currently 379 // connected to its target. 380 func (o *Switch) Connected() bool { 381 for _, out := range o.outputs { 382 if !out.Connected() { 383 return false 384 } 385 } 386 return true 387 } 388 389 //------------------------------------------------------------------------------ 390 391 func (o *Switch) dispatchRetryOnErr(outputTargets [][]types.Part) error { 392 var owg errgroup.Group 393 for target, parts := range outputTargets { 394 if len(parts) == 0 { 395 continue 396 } 397 msgCopy, i := message.New(nil), target 398 msgCopy.SetAll(parts) 399 owg.Go(func() error { 400 throt := throttle.New(throttle.OptCloseChan(o.ctx.Done())) 401 resChan := make(chan types.Response) 402 403 // Try until success or shutdown. 404 for { 405 select { 406 case o.outputTSChans[i] <- types.NewTransaction(msgCopy, resChan): 407 case <-o.ctx.Done(): 408 return types.ErrTypeClosed 409 } 410 select { 411 case res := <-resChan: 412 if res.Error() != nil { 413 o.logger.Errorf("Failed to dispatch switch message: %v\n", res.Error()) 414 o.mOutputErr.Incr(1) 415 if !throt.Retry() { 416 return types.ErrTypeClosed 417 } 418 } else { 419 o.mMsgSnt.Incr(1) 420 return nil 421 } 422 case <-o.ctx.Done(): 423 return types.ErrTypeClosed 424 } 425 } 426 }) 427 } 428 return owg.Wait() 429 } 430 431 func (o *Switch) dispatchNoRetries(group *imessage.SortGroup, sourceMessage types.Message, outputTargets [][]types.Part) error { 432 var wg sync.WaitGroup 433 434 var setErr func(error) 435 var setErrForPart func(types.Part, error) 436 var getErr func() error 437 { 438 var generalErr error 439 var batchErr *batch.Error 440 var errLock sync.Mutex 441 442 setErr = func(err error) { 443 if err == nil { 444 return 445 } 446 errLock.Lock() 447 generalErr = err 448 errLock.Unlock() 449 } 450 setErrForPart = func(part types.Part, err error) { 451 if err == nil { 452 return 453 } 454 errLock.Lock() 455 defer errLock.Unlock() 456 457 index := group.GetIndex(part) 458 if index == -1 { 459 generalErr = err 460 return 461 } 462 463 if batchErr == nil { 464 batchErr = batch.NewError(sourceMessage, err) 465 } 466 batchErr.Failed(index, err) 467 } 468 getErr = func() error { 469 if batchErr != nil { 470 return batchErr 471 } 472 return generalErr 473 } 474 } 475 476 for target, parts := range outputTargets { 477 if len(parts) == 0 { 478 continue 479 } 480 wg.Add(1) 481 msgCopy, i := message.New(nil), target 482 msgCopy.SetAll(parts) 483 484 go func() { 485 defer wg.Done() 486 487 resChan := make(chan types.Response) 488 select { 489 case o.outputTSChans[i] <- types.NewTransaction(msgCopy, resChan): 490 case <-o.ctx.Done(): 491 setErr(types.ErrTypeClosed) 492 return 493 } 494 select { 495 case res := <-resChan: 496 if res.Error() != nil { 497 o.mOutputErr.Incr(1) 498 if bErr, ok := res.Error().(*batch.Error); ok { 499 bErr.WalkParts(func(i int, p types.Part, e error) bool { 500 if e != nil { 501 setErrForPart(p, e) 502 } 503 return true 504 }) 505 } else { 506 msgCopy.Iter(func(i int, p types.Part) error { 507 setErrForPart(p, res.Error()) 508 return nil 509 }) 510 } 511 } else { 512 o.mMsgSnt.Incr(1) 513 } 514 case <-o.ctx.Done(): 515 setErr(types.ErrTypeClosed) 516 } 517 }() 518 } 519 520 wg.Wait() 521 return getErr() 522 } 523 524 // loop is an internal loop that brokers incoming messages to many outputs. 525 func (o *Switch) loop() { 526 var wg sync.WaitGroup 527 528 defer func() { 529 wg.Wait() 530 for i, output := range o.outputs { 531 output.CloseAsync() 532 close(o.outputTSChans[i]) 533 } 534 for _, output := range o.outputs { 535 _ = output.WaitForClose(shutdown.MaximumShutdownWait()) 536 } 537 close(o.closedChan) 538 }() 539 540 sendLoop := func() { 541 defer wg.Done() 542 for { 543 var ts types.Transaction 544 var open bool 545 546 select { 547 case ts, open = <-o.transactions: 548 if !open { 549 return 550 } 551 case <-o.ctx.Done(): 552 return 553 } 554 o.mMsgRcvd.Incr(1) 555 556 group, trackedMsg := imessage.NewSortGroup(ts.Payload) 557 558 outputTargets := make([][]types.Part, len(o.checks)) 559 if checksErr := trackedMsg.Iter(func(i int, p types.Part) error { 560 routedAtLeastOnce := false 561 for j, exe := range o.checks { 562 test := true 563 if exe != nil { 564 var err error 565 if test, err = exe.QueryPart(i, trackedMsg); err != nil { 566 test = false 567 o.logger.Errorf("Failed to test case %v: %v\n", j, err) 568 } 569 } 570 if test { 571 routedAtLeastOnce = true 572 outputTargets[j] = append(outputTargets[j], p.Copy()) 573 if !o.continues[j] { 574 return nil 575 } 576 } 577 } 578 if !routedAtLeastOnce && o.strictMode { 579 return ErrSwitchNoConditionMet 580 } 581 return nil 582 }); checksErr != nil { 583 select { 584 case ts.ResponseChan <- response.NewError(checksErr): 585 case <-o.ctx.Done(): 586 return 587 } 588 continue 589 } 590 591 var resErr error 592 if o.retryUntilSuccess { 593 resErr = o.dispatchRetryOnErr(outputTargets) 594 } else { 595 resErr = o.dispatchNoRetries(group, trackedMsg, outputTargets) 596 } 597 598 var oResponse types.Response = response.NewAck() 599 if resErr != nil { 600 oResponse = response.NewError(resErr) 601 } 602 select { 603 case ts.ResponseChan <- oResponse: 604 case <-o.ctx.Done(): 605 return 606 } 607 } 608 } 609 610 // Max in flight 611 for i := 0; i < o.maxInFlight; i++ { 612 wg.Add(1) 613 go sendLoop() 614 } 615 } 616 617 // CloseAsync shuts down the Switch broker and stops processing requests. 618 func (o *Switch) CloseAsync() { 619 o.close() 620 } 621 622 // WaitForClose blocks until the Switch broker has closed down. 623 func (o *Switch) WaitForClose(timeout time.Duration) error { 624 select { 625 case <-o.closedChan: 626 case <-time.After(timeout): 627 return types.ErrTimeout 628 } 629 return nil 630 } 631 632 //------------------------------------------------------------------------------