github.com/Jeffail/benthos/v3@v3.65.0/lib/broker/try.go (about) 1 package broker 2 3 import ( 4 "context" 5 "errors" 6 "fmt" 7 "sync" 8 "time" 9 10 "github.com/Jeffail/benthos/v3/internal/component/output" 11 "github.com/Jeffail/benthos/v3/lib/metrics" 12 "github.com/Jeffail/benthos/v3/lib/types" 13 ) 14 15 //------------------------------------------------------------------------------ 16 17 // Try is a broker that implements types.Consumer and attempts to send each 18 // message to a single output, but on failure will attempt the next output in 19 // the list. 20 type Try struct { 21 stats metrics.Type 22 outputsPrefix string 23 24 maxInFlight int 25 transactions <-chan types.Transaction 26 27 outputTSChans []chan types.Transaction 28 outputs []types.Output 29 30 ctx context.Context 31 close func() 32 closedChan chan struct{} 33 } 34 35 // NewTry creates a new Try type by providing consumers. 36 func NewTry(outputs []types.Output, stats metrics.Type) (*Try, error) { 37 ctx, done := context.WithCancel(context.Background()) 38 t := &Try{ 39 maxInFlight: 1, 40 stats: stats, 41 outputsPrefix: "broker.outputs", 42 transactions: nil, 43 outputs: outputs, 44 closedChan: make(chan struct{}), 45 ctx: ctx, 46 close: done, 47 } 48 if len(outputs) == 0 { 49 return nil, errors.New("missing outputs") 50 } 51 t.outputTSChans = make([]chan types.Transaction, len(t.outputs)) 52 for i := range t.outputTSChans { 53 t.outputTSChans[i] = make(chan types.Transaction) 54 if err := t.outputs[i].Consume(t.outputTSChans[i]); err != nil { 55 return nil, err 56 } 57 if mif, ok := output.GetMaxInFlight(t.outputs[i]); ok && mif > t.maxInFlight { 58 t.maxInFlight = mif 59 } 60 } 61 return t, nil 62 } 63 64 //------------------------------------------------------------------------------ 65 66 // WithMaxInFlight sets the maximum number of in-flight messages this broker 67 // supports. This must be set before calling Consume. 68 func (t *Try) WithMaxInFlight(i int) *Try { 69 if i < 1 { 70 i = 1 71 } 72 t.maxInFlight = i 73 return t 74 } 75 76 // WithOutputMetricsPrefix changes the prefix used for counter metrics showing 77 // errors of an output. 78 func (t *Try) WithOutputMetricsPrefix(prefix string) *Try { 79 t.outputsPrefix = prefix 80 return t 81 } 82 83 // Consume assigns a new messages channel for the broker to read. 84 func (t *Try) Consume(ts <-chan types.Transaction) error { 85 if t.transactions != nil { 86 return types.ErrAlreadyStarted 87 } 88 t.transactions = ts 89 90 go t.loop() 91 return nil 92 } 93 94 // Connected returns a boolean indicating whether this output is currently 95 // connected to its target. 96 func (t *Try) Connected() bool { 97 for _, out := range t.outputs { 98 if !out.Connected() { 99 return false 100 } 101 } 102 return true 103 } 104 105 // MaxInFlight returns the maximum number of in flight messages permitted by the 106 // output. This value can be used to determine a sensible value for parent 107 // outputs, but should not be relied upon as part of dispatcher logic. 108 func (t *Try) MaxInFlight() (int, bool) { 109 return t.maxInFlight, true 110 } 111 112 //------------------------------------------------------------------------------ 113 114 // loop is an internal loop that brokers incoming messages to many outputs. 115 func (t *Try) loop() { 116 var ( 117 wg = sync.WaitGroup{} 118 mMsgsRcvd = t.stats.GetCounter("count") 119 mErrs = []metrics.StatCounter{} 120 ) 121 122 defer func() { 123 wg.Wait() 124 for _, c := range t.outputTSChans { 125 close(c) 126 } 127 closeAllOutputs(t.outputs) 128 close(t.closedChan) 129 }() 130 131 for i := range t.outputs { 132 mErrs = append(mErrs, t.stats.GetCounter(fmt.Sprintf("%v.%v.failed", t.outputsPrefix, i))) 133 } 134 135 sendLoop := func() { 136 defer wg.Done() 137 for { 138 var open bool 139 var tran types.Transaction 140 141 select { 142 case tran, open = <-t.transactions: 143 if !open { 144 return 145 } 146 case <-t.ctx.Done(): 147 return 148 } 149 mMsgsRcvd.Incr(1) 150 151 rChan := make(chan types.Response) 152 select { 153 case t.outputTSChans[0] <- types.NewTransaction(tran.Payload, rChan): 154 case <-t.ctx.Done(): 155 return 156 } 157 158 var res types.Response 159 var lOpen bool 160 161 triesLoop: 162 for i := 1; i <= len(t.outputTSChans); i++ { 163 select { 164 case res, lOpen = <-rChan: 165 if !lOpen { 166 return 167 } 168 if res.Error() != nil { 169 mErrs[i-1].Incr(1) 170 } else { 171 break triesLoop 172 } 173 case <-t.ctx.Done(): 174 return 175 } 176 177 if i < len(t.outputTSChans) { 178 select { 179 case t.outputTSChans[i] <- types.NewTransaction(tran.Payload, rChan): 180 case <-t.ctx.Done(): 181 return 182 } 183 } 184 } 185 select { 186 case tran.ResponseChan <- res: 187 case <-t.ctx.Done(): 188 return 189 } 190 } 191 } 192 193 // Max in flight 194 for i := 0; i < t.maxInFlight; i++ { 195 wg.Add(1) 196 go sendLoop() 197 } 198 } 199 200 // CloseAsync shuts down the Try broker and stops processing requests. 201 func (t *Try) CloseAsync() { 202 t.close() 203 } 204 205 // WaitForClose blocks until the Try broker has closed down. 206 func (t *Try) WaitForClose(timeout time.Duration) error { 207 select { 208 case <-t.closedChan: 209 case <-time.After(timeout): 210 return types.ErrTimeout 211 } 212 return nil 213 } 214 215 //------------------------------------------------------------------------------