github.com/Jeffail/benthos/v3@v3.65.0/lib/output/drop_on.go (about) 1 package output 2 3 import ( 4 "context" 5 "encoding/json" 6 "errors" 7 "fmt" 8 "time" 9 10 "github.com/Jeffail/benthos/v3/internal/component/output" 11 "github.com/Jeffail/benthos/v3/internal/docs" 12 "github.com/Jeffail/benthos/v3/internal/shutdown" 13 "github.com/Jeffail/benthos/v3/lib/log" 14 "github.com/Jeffail/benthos/v3/lib/metrics" 15 "github.com/Jeffail/benthos/v3/lib/response" 16 "github.com/Jeffail/benthos/v3/lib/types" 17 ) 18 19 //------------------------------------------------------------------------------ 20 21 func init() { 22 Constructors[TypeDropOn] = TypeSpec{ 23 constructor: fromSimpleConstructor(func(conf Config, mgr types.Manager, log log.Modular, stats metrics.Type) (Type, error) { 24 if conf.DropOn.Output == nil { 25 return nil, errors.New("cannot create a drop_on output without a child") 26 } 27 wrapped, err := New(*conf.DropOn.Output, mgr, log, stats) 28 if err != nil { 29 return nil, fmt.Errorf("failed to create output '%v': %v", conf.DropOn.Output.Type, err) 30 } 31 return newDropOn(conf.DropOn.DropOnConditions, wrapped, log, stats) 32 }), 33 Summary: ` 34 Attempts to write messages to a child output and if the write fails for one of a list of configurable reasons the message is dropped instead of being reattempted.`, 35 Description: ` 36 Regular Benthos outputs will apply back pressure when downstream services aren't accessible, and Benthos retries (or nacks) all messages that fail to be delivered. However, in some circumstances, or for certain output types, we instead might want to relax these mechanisms, which is when this output becomes useful.`, 37 Categories: []Category{ 38 CategoryUtility, 39 }, 40 FieldSpecs: docs.FieldSpecs{ 41 docs.FieldCommon("error", "Whether messages should be dropped when the child output returns an error. For example, this could be when an http_client output gets a 4XX response code."), 42 docs.FieldCommon("back_pressure", "An optional duration string that determines the maximum length of time to wait for a given message to be accepted by the child output before the message should be dropped instead. The most common reason for an output to block is when waiting for a lost connection to be re-established. Once a message has been dropped due to back pressure all subsequent messages are dropped immediately until the output is ready to process them again. Note that if `error` is set to `false` and this field is specified then messages dropped due to back pressure will return an error response.", "30s", "1m"), 43 docs.FieldCommon("output", "A child output.").HasType(docs.FieldTypeOutput), 44 }, 45 Examples: []docs.AnnotatedExample{ 46 { 47 Title: "Dropping failed HTTP requests", 48 Summary: "In this example we have a fan_out broker, where we guarantee delivery to our Kafka output, but drop messages if they fail our secondary HTTP client output.", 49 Config: ` 50 output: 51 broker: 52 pattern: fan_out 53 outputs: 54 - kafka: 55 addresses: [ foobar:6379 ] 56 topic: foo 57 - drop_on: 58 error: true 59 output: 60 http_client: 61 url: http://example.com/foo/messages 62 verb: POST 63 `, 64 }, 65 { 66 Title: "Dropping from outputs that cannot connect", 67 Summary: "Most outputs that attempt to establish and long-lived connection will apply back-pressure when the connection is lost. The following example has a websocket output where if it takes longer than 10 seconds to establish a connection, or recover a lost one, pending messages are dropped.", 68 Config: ` 69 output: 70 drop_on: 71 back_pressure: 10s 72 output: 73 websocket: 74 url: ws://example.com/foo/messages 75 `, 76 }, 77 }, 78 } 79 } 80 81 //------------------------------------------------------------------------------ 82 83 // DropOnConditions is a config struct representing the different circumstances 84 // under which messages should be dropped. 85 type DropOnConditions struct { 86 Error bool `json:"error" yaml:"error"` 87 BackPressure string `json:"back_pressure" yaml:"back_pressure"` 88 } 89 90 // DropOnConfig contains configuration values for the DropOn output type. 91 type DropOnConfig struct { 92 DropOnConditions `json:",inline" yaml:",inline"` 93 Output *Config `json:"output" yaml:"output"` 94 } 95 96 // NewDropOnConfig creates a new DropOnConfig with default values. 97 func NewDropOnConfig() DropOnConfig { 98 return DropOnConfig{ 99 DropOnConditions: DropOnConditions{ 100 Error: false, 101 BackPressure: "", 102 }, 103 Output: nil, 104 } 105 } 106 107 //------------------------------------------------------------------------------ 108 109 type dummyDropOnConfig struct { 110 DropOnConditions `json:",inline" yaml:",inline"` 111 Output interface{} `json:"output" yaml:"output"` 112 } 113 114 // MarshalJSON prints an empty object instead of nil. 115 func (d DropOnConfig) MarshalJSON() ([]byte, error) { 116 dummy := dummyDropOnConfig{ 117 Output: d.Output, 118 DropOnConditions: d.DropOnConditions, 119 } 120 if d.Output == nil { 121 dummy.Output = struct{}{} 122 } 123 return json.Marshal(dummy) 124 } 125 126 // MarshalYAML prints an empty object instead of nil. 127 func (d DropOnConfig) MarshalYAML() (interface{}, error) { 128 dummy := dummyDropOnConfig{ 129 Output: d.Output, 130 DropOnConditions: d.DropOnConditions, 131 } 132 if d.Output == nil { 133 dummy.Output = struct{}{} 134 } 135 return dummy, nil 136 } 137 138 //------------------------------------------------------------------------------ 139 140 // dropOn attempts to forward messages to a child output, but under certain 141 // conditions will abandon the request and drop the message. 142 type dropOn struct { 143 stats metrics.Type 144 log log.Modular 145 146 onError bool 147 onBackpressure time.Duration 148 wrapped Type 149 150 transactionsIn <-chan types.Transaction 151 transactionsOut chan types.Transaction 152 153 ctx context.Context 154 done func() 155 closedChan chan struct{} 156 } 157 158 func newDropOn(conf DropOnConditions, wrapped Type, log log.Modular, stats metrics.Type) (*dropOn, error) { 159 var backPressure time.Duration 160 if len(conf.BackPressure) > 0 { 161 var err error 162 if backPressure, err = time.ParseDuration(conf.BackPressure); err != nil { 163 return nil, fmt.Errorf("failed to parse back_pressure duration: %w", err) 164 } 165 } 166 167 ctx, done := context.WithCancel(context.Background()) 168 return &dropOn{ 169 log: log, 170 stats: stats, 171 wrapped: wrapped, 172 transactionsOut: make(chan types.Transaction), 173 174 onError: conf.Error, 175 onBackpressure: backPressure, 176 177 ctx: ctx, 178 done: done, 179 closedChan: make(chan struct{}), 180 }, nil 181 } 182 183 //------------------------------------------------------------------------------ 184 185 func (d *dropOn) loop() { 186 // Metrics paths 187 var ( 188 mDropped = d.stats.GetCounter("drop_on.dropped") 189 mDroppedBatch = d.stats.GetCounter("drop_on.batch.dropped") 190 ) 191 192 defer func() { 193 close(d.transactionsOut) 194 d.wrapped.CloseAsync() 195 _ = d.wrapped.WaitForClose(shutdown.MaximumShutdownWait()) 196 close(d.closedChan) 197 }() 198 199 resChan := make(chan types.Response) 200 201 var gotBackPressure bool 202 for { 203 var ts types.Transaction 204 var open bool 205 select { 206 case ts, open = <-d.transactionsIn: 207 if !open { 208 return 209 } 210 case <-d.ctx.Done(): 211 return 212 } 213 214 var res types.Response 215 if d.onBackpressure > 0 { 216 if !func() bool { 217 // Use a ticker here and call Stop explicitly. 218 ticker := time.NewTicker(d.onBackpressure) 219 defer ticker.Stop() 220 221 if gotBackPressure { 222 select { 223 case d.transactionsOut <- types.NewTransaction(ts.Payload, resChan): 224 gotBackPressure = false 225 default: 226 } 227 } else { 228 select { 229 case d.transactionsOut <- types.NewTransaction(ts.Payload, resChan): 230 case <-ticker.C: 231 gotBackPressure = true 232 case <-d.ctx.Done(): 233 return false 234 } 235 } 236 if !gotBackPressure { 237 select { 238 case res = <-resChan: 239 case <-ticker.C: 240 gotBackPressure = true 241 go func() { 242 // We must pull the response that we're due, since 243 // the component isn't being shut down. 244 <-resChan 245 }() 246 case <-d.ctx.Done(): 247 return false 248 } 249 } 250 if gotBackPressure { 251 mDropped.Incr(int64(ts.Payload.Len())) 252 mDroppedBatch.Incr(1) 253 d.log.Warnln("Message dropped due to back pressure.") 254 if d.onError { 255 res = response.NewAck() 256 } else { 257 res = response.NewError(fmt.Errorf("experienced back pressure beyond: %v", d.onBackpressure)) 258 } 259 } 260 return true 261 }() { 262 return 263 } 264 } else { 265 // Push data as usual, if the output blocks due to a disconnect then 266 // we wait as long as it takes. 267 select { 268 case d.transactionsOut <- types.NewTransaction(ts.Payload, resChan): 269 case <-d.ctx.Done(): 270 return 271 } 272 select { 273 case res = <-resChan: 274 case <-d.ctx.Done(): 275 return 276 } 277 } 278 279 if res.Error() != nil && d.onError { 280 mDropped.Incr(int64(ts.Payload.Len())) 281 mDroppedBatch.Incr(1) 282 d.log.Warnf("Message dropped due to: %v\n", res.Error()) 283 res = response.NewAck() 284 } 285 286 select { 287 case ts.ResponseChan <- res: 288 case <-d.ctx.Done(): 289 return 290 } 291 } 292 } 293 294 // Consume assigns a messages channel for the output to read. 295 func (d *dropOn) Consume(ts <-chan types.Transaction) error { 296 if d.transactionsIn != nil { 297 return types.ErrAlreadyStarted 298 } 299 if err := d.wrapped.Consume(d.transactionsOut); err != nil { 300 return err 301 } 302 d.transactionsIn = ts 303 go d.loop() 304 return nil 305 } 306 307 // Connected returns a boolean indicating whether this output is currently 308 // connected to its target. 309 func (d *dropOn) Connected() bool { 310 return d.wrapped.Connected() 311 } 312 313 func (d *dropOn) MaxInFlight() (int, bool) { 314 return output.GetMaxInFlight(d.wrapped) 315 } 316 317 // CloseAsync shuts down the DropOnError input and stops processing requests. 318 func (d *dropOn) CloseAsync() { 319 d.done() 320 } 321 322 // WaitForClose blocks until the DropOnError input has closed down. 323 func (d *dropOn) WaitForClose(timeout time.Duration) error { 324 select { 325 case <-d.closedChan: 326 case <-time.After(timeout): 327 return types.ErrTimeout 328 } 329 return nil 330 } 331 332 //------------------------------------------------------------------------------