github.com/Jeffail/benthos/v3@v3.65.0/lib/processor/switch.go (about) 1 package processor 2 3 import ( 4 "encoding/json" 5 "errors" 6 "fmt" 7 "sort" 8 "strconv" 9 "time" 10 11 "github.com/Jeffail/benthos/v3/internal/bloblang/mapping" 12 "github.com/Jeffail/benthos/v3/internal/docs" 13 "github.com/Jeffail/benthos/v3/internal/interop" 14 imessage "github.com/Jeffail/benthos/v3/internal/message" 15 "github.com/Jeffail/benthos/v3/lib/condition" 16 "github.com/Jeffail/benthos/v3/lib/log" 17 "github.com/Jeffail/benthos/v3/lib/message" 18 "github.com/Jeffail/benthos/v3/lib/metrics" 19 "github.com/Jeffail/benthos/v3/lib/response" 20 "github.com/Jeffail/benthos/v3/lib/types" 21 ) 22 23 //------------------------------------------------------------------------------ 24 25 func init() { 26 Constructors[TypeSwitch] = TypeSpec{ 27 constructor: NewSwitch, 28 Categories: []Category{ 29 CategoryComposition, 30 }, 31 Summary: ` 32 Conditionally processes messages based on their contents.`, 33 Description: ` 34 For each switch case a [Bloblang query](/docs/guides/bloblang/about/) is checked and, if the result is true (or the check is empty) the child processors are executed on the message.`, 35 Footnotes: ` 36 ## Batching 37 38 When a switch processor executes on a [batch of messages](/docs/configuration/batching/) they are checked individually and can be matched independently against cases. During processing the messages matched against a case are processed as a batch, although the ordering of messages during case processing cannot be guaranteed to match the order as received. 39 40 At the end of switch processing the resulting batch will follow the same ordering as the batch was received. If any child processors have split or otherwise grouped messages this grouping will be lost as the result of a switch is always a single batch. In order to perform conditional grouping and/or splitting use the [` + "`group_by`" + ` processor](/docs/components/processors/group_by/).`, 41 config: docs.FieldComponent().Array().WithChildren( 42 docs.FieldDeprecated("condition").HasType(docs.FieldTypeCondition).OmitWhen(func(v, _ interface{}) (string, bool) { 43 m, ok := v.(map[string]interface{}) 44 if !ok { 45 return "", false 46 } 47 return "field condition is deprecated in favour of check", m["type"] == "static" && m["static"] == true 48 }), 49 docs.FieldBloblang( 50 "check", 51 "A [Bloblang query](/docs/guides/bloblang/about/) that should return a boolean value indicating whether a message should have the processors of this case executed on it. If left empty the case always passes. If the check mapping throws an error the message will be flagged [as having failed](/docs/configuration/error_handling) and will not be tested against any other cases.", 52 `this.type == "foo"`, 53 `this.contents.urls.contains("https://benthos.dev/")`, 54 ).HasDefault(""), 55 docs.FieldCommon( 56 "processors", 57 "A list of [processors](/docs/components/processors/about/) to execute on a message.", 58 ).HasDefault([]interface{}{}).Array().HasType(docs.FieldTypeProcessor), 59 docs.FieldAdvanced( 60 "fallthrough", 61 "Indicates whether, if this case passes for a message, the next case should also be executed.", 62 ).HasDefault(false).HasType(docs.FieldTypeBool), 63 ), 64 Examples: []docs.AnnotatedExample{ 65 { 66 Title: "I Hate George", 67 Summary: ` 68 We have a system where we're counting a metric for all messages that pass through our system. However, occasionally we get messages from George where he's rambling about dumb stuff we don't care about. 69 70 For Georges messages we want to instead emit a metric that gauges how angry he is about being ignored and then we drop it.`, 71 Config: ` 72 pipeline: 73 processors: 74 - switch: 75 - check: this.user.name.first != "George" 76 processors: 77 - metric: 78 type: counter 79 name: MessagesWeCareAbout 80 81 - processors: 82 - metric: 83 type: gauge 84 name: GeorgesAnger 85 value: ${! json("user.anger") } 86 - bloblang: root = deleted() 87 `, 88 }, 89 }, 90 } 91 } 92 93 //------------------------------------------------------------------------------ 94 95 // SwitchCaseConfig contains a condition, processors and other fields for an 96 // individual case in the Switch processor. 97 type SwitchCaseConfig struct { 98 Condition condition.Config `json:"condition" yaml:"condition"` 99 Check string `json:"check" yaml:"check"` 100 Processors []Config `json:"processors" yaml:"processors"` 101 Fallthrough bool `json:"fallthrough" yaml:"fallthrough"` 102 } 103 104 // NewSwitchCaseConfig returns a new SwitchCaseConfig with default values. 105 func NewSwitchCaseConfig() SwitchCaseConfig { 106 cond := condition.NewConfig() 107 cond.Type = condition.TypeStatic 108 cond.Static = true 109 return SwitchCaseConfig{ 110 Condition: cond, 111 Check: "", 112 Processors: []Config{}, 113 Fallthrough: false, 114 } 115 } 116 117 // UnmarshalJSON ensures that when parsing configs that are in a map or slice 118 // the default values are still applied. 119 func (s *SwitchCaseConfig) UnmarshalJSON(bytes []byte) error { 120 type confAlias SwitchCaseConfig 121 aliased := confAlias(NewSwitchCaseConfig()) 122 123 if err := json.Unmarshal(bytes, &aliased); err != nil { 124 return err 125 } 126 127 *s = SwitchCaseConfig(aliased) 128 return nil 129 } 130 131 // UnmarshalYAML ensures that when parsing configs that are in a map or slice 132 // the default values are still applied. 133 func (s *SwitchCaseConfig) UnmarshalYAML(unmarshal func(interface{}) error) error { 134 type confAlias SwitchCaseConfig 135 aliased := confAlias(NewSwitchCaseConfig()) 136 137 if err := unmarshal(&aliased); err != nil { 138 return err 139 } 140 141 *s = SwitchCaseConfig(aliased) 142 return nil 143 } 144 145 //------------------------------------------------------------------------------ 146 147 // SwitchConfig is a config struct containing fields for the Switch processor. 148 type SwitchConfig []SwitchCaseConfig 149 150 // NewSwitchConfig returns a default SwitchConfig. 151 func NewSwitchConfig() SwitchConfig { 152 return SwitchConfig{} 153 } 154 155 //------------------------------------------------------------------------------ 156 157 // switchCase contains a condition, processors and other fields for an 158 // individual case in the Switch processor. 159 type switchCase struct { 160 check *mapping.Executor 161 processors []types.Processor 162 fallThrough bool 163 } 164 165 // Switch is a processor that only applies child processors under a certain 166 // condition. 167 type Switch struct { 168 cases []switchCase 169 log log.Modular 170 171 mCount metrics.StatCounter 172 mSent metrics.StatCounter 173 } 174 175 func isDefaultCaseCond(cond condition.Config) bool { 176 return cond.Type == condition.TypeStatic && cond.Static 177 } 178 179 // NewSwitch returns a Switch processor. 180 func NewSwitch( 181 conf Config, mgr types.Manager, log log.Modular, stats metrics.Type, 182 ) (Type, error) { 183 deprecated := false 184 for _, caseConf := range conf.Switch { 185 if deprecated || !isDefaultCaseCond(caseConf.Condition) { 186 deprecated = true 187 } 188 if deprecated { 189 if len(caseConf.Check) > 0 { 190 return nil, errors.New("cannot use both deprecated condition field in combination with field check") 191 } 192 } 193 } 194 if deprecated { 195 return newSwitchDeprecated(conf, mgr, log, stats) 196 } 197 198 var cases []switchCase 199 for i, caseConf := range conf.Switch { 200 prefix := strconv.Itoa(i) 201 202 var err error 203 var check *mapping.Executor 204 var procs []types.Processor 205 206 if len(caseConf.Check) > 0 { 207 if check, err = interop.NewBloblangMapping(mgr, caseConf.Check); err != nil { 208 return nil, fmt.Errorf("failed to parse case %v check: %w", i, err) 209 } 210 } 211 212 if len(caseConf.Processors) == 0 { 213 return nil, fmt.Errorf("case [%v] has no processors, in order to have a no-op case use a `noop` processor", i) 214 } 215 216 for j, procConf := range caseConf.Processors { 217 pMgr, pLog, pStats := interop.LabelChild(prefix+"."+strconv.Itoa(j), mgr, log, stats) 218 var proc types.Processor 219 if proc, err = New(procConf, pMgr, pLog, pStats); err != nil { 220 return nil, fmt.Errorf("case [%v] processor [%v]: %w", i, j, err) 221 } 222 procs = append(procs, proc) 223 } 224 225 cases = append(cases, switchCase{ 226 check: check, 227 processors: procs, 228 fallThrough: caseConf.Fallthrough, 229 }) 230 } 231 return &Switch{ 232 cases: cases, 233 log: log, 234 235 mCount: stats.GetCounter("count"), 236 mSent: stats.GetCounter("sent"), 237 }, nil 238 } 239 240 //------------------------------------------------------------------------------ 241 242 func reorderFromGroup(group *imessage.SortGroup, parts []types.Part) { 243 partToIndex := map[types.Part]int{} 244 for _, p := range parts { 245 if i := group.GetIndex(p); i >= 0 { 246 partToIndex[p] = i 247 } 248 } 249 250 sort.SliceStable(parts, func(i, j int) bool { 251 if index, found := partToIndex[parts[i]]; found { 252 i = index 253 } 254 if index, found := partToIndex[parts[j]]; found { 255 j = index 256 } 257 return i < j 258 }) 259 } 260 261 // ProcessMessage applies the processor to a message, either creating >0 262 // resulting messages or a response to be sent back to the message source. 263 func (s *Switch) ProcessMessage(msg types.Message) (msgs []types.Message, res types.Response) { 264 s.mCount.Incr(1) 265 266 var result []types.Part 267 var remaining []types.Part 268 var carryOver []types.Part 269 270 sortGroup, sortMsg := imessage.NewSortGroup(msg) 271 remaining = make([]types.Part, sortMsg.Len()) 272 sortMsg.Iter(func(i int, p types.Part) error { 273 remaining[i] = p 274 return nil 275 }) 276 277 for i, switchCase := range s.cases { 278 passed, failed := carryOver, []types.Part{} 279 280 // Form a message to test against, consisting of fallen through messages 281 // from prior cases plus remaining messages that haven't passed a case 282 // yet. 283 testMsg := message.New(nil) 284 testMsg.Append(remaining...) 285 286 for j, p := range remaining { 287 test := switchCase.check == nil 288 if !test { 289 var err error 290 if test, err = switchCase.check.QueryPart(j, testMsg); err != nil { 291 s.log.Errorf("Failed to test case %v: %v\n", i, err) 292 FlagErr(p, err) 293 result = append(result, p) 294 continue 295 } 296 } 297 if test { 298 passed = append(passed, p) 299 } else { 300 failed = append(failed, p) 301 } 302 } 303 304 carryOver = nil 305 remaining = failed 306 307 if len(passed) > 0 { 308 execMsg := message.New(nil) 309 execMsg.SetAll(passed) 310 311 msgs, res := ExecuteAll(switchCase.processors, execMsg) 312 if res != nil && res.Error() != nil { 313 return nil, res 314 } 315 316 for _, m := range msgs { 317 m.Iter(func(_ int, p types.Part) error { 318 if switchCase.fallThrough { 319 carryOver = append(carryOver, p) 320 } else { 321 result = append(result, p) 322 } 323 return nil 324 }) 325 } 326 } 327 } 328 329 result = append(result, remaining...) 330 if len(result) > 1 { 331 reorderFromGroup(sortGroup, result) 332 } 333 334 resMsg := message.New(nil) 335 resMsg.SetAll(result) 336 337 if resMsg.Len() == 0 { 338 return nil, response.NewAck() 339 } 340 341 s.mSent.Incr(int64(resMsg.Len())) 342 return []types.Message{resMsg}, nil 343 } 344 345 // CloseAsync shuts down the processor and stops processing requests. 346 func (s *Switch) CloseAsync() { 347 for _, s := range s.cases { 348 for _, proc := range s.processors { 349 proc.CloseAsync() 350 } 351 } 352 } 353 354 // WaitForClose blocks until the processor has closed down. 355 func (s *Switch) WaitForClose(timeout time.Duration) error { 356 stopBy := time.Now().Add(timeout) 357 for _, s := range s.cases { 358 for _, proc := range s.processors { 359 if err := proc.WaitForClose(time.Until(stopBy)); err != nil { 360 return err 361 } 362 } 363 } 364 return nil 365 } 366 367 //------------------------------------------------------------------------------