github.com/Jeffail/benthos/v3@v3.65.0/lib/pipeline/processor.go (about) 1 package pipeline 2 3 import ( 4 "sync" 5 "sync/atomic" 6 "time" 7 8 "github.com/Jeffail/benthos/v3/lib/log" 9 "github.com/Jeffail/benthos/v3/lib/metrics" 10 "github.com/Jeffail/benthos/v3/lib/processor" 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 ) 15 16 //------------------------------------------------------------------------------ 17 18 // Processor is a pipeline that supports both Consumer and Producer interfaces. 19 // The processor will read from a source, perform some processing, and then 20 // either propagate a new message or drop it. 21 type Processor struct { 22 running int32 23 24 log log.Modular 25 stats metrics.Type 26 27 msgProcessors []types.Processor 28 29 messagesOut chan types.Transaction 30 responsesIn chan types.Response 31 32 messagesIn <-chan types.Transaction 33 34 closeChan chan struct{} 35 closed chan struct{} 36 } 37 38 // NewProcessor returns a new message processing pipeline. 39 func NewProcessor( 40 log log.Modular, 41 stats metrics.Type, 42 msgProcessors ...types.Processor, 43 ) *Processor { 44 return &Processor{ 45 running: 1, 46 msgProcessors: msgProcessors, 47 stats: stats, 48 messagesOut: make(chan types.Transaction), 49 responsesIn: make(chan types.Response), 50 closeChan: make(chan struct{}), 51 closed: make(chan struct{}), 52 } 53 } 54 55 //------------------------------------------------------------------------------ 56 57 // loop is the processing loop of this pipeline. 58 func (p *Processor) loop() { 59 defer func() { 60 // Signal all children to close. 61 for _, c := range p.msgProcessors { 62 c.CloseAsync() 63 } 64 65 close(p.messagesOut) 66 close(p.closed) 67 }() 68 69 var open bool 70 for atomic.LoadInt32(&p.running) == 1 { 71 var tran types.Transaction 72 select { 73 case tran, open = <-p.messagesIn: 74 if !open { 75 return 76 } 77 case <-p.closeChan: 78 return 79 } 80 81 resultMsgs, resultRes := processor.ExecuteAll(p.msgProcessors, tran.Payload) 82 if len(resultMsgs) == 0 { 83 if resultRes == nil { 84 resultRes = response.NewUnack() 85 p.log.Warnln("Nil response returned with zero messages from processors") 86 } 87 select { 88 case tran.ResponseChan <- resultRes: 89 case <-p.closeChan: 90 return 91 } 92 continue 93 } 94 95 if len(resultMsgs) > 1 { 96 p.dispatchMessages(resultMsgs, tran.ResponseChan) 97 } else { 98 select { 99 case p.messagesOut <- types.NewTransaction(resultMsgs[0], tran.ResponseChan): 100 case <-p.closeChan: 101 return 102 } 103 } 104 } 105 } 106 107 // dispatchMessages attempts to send a multiple messages results of processors 108 // over the shared messages channel. This send is retried until success. 109 func (p *Processor) dispatchMessages(msgs []types.Message, ogResChan chan<- types.Response) { 110 throt := throttle.New(throttle.OptCloseChan(p.closeChan)) 111 112 var skipAcks int64 113 sendMsg := func(m types.Message) { 114 resChan := make(chan types.Response) 115 transac := types.NewTransaction(m, resChan) 116 117 for { 118 select { 119 case p.messagesOut <- transac: 120 case <-p.closeChan: 121 return 122 } 123 124 var res types.Response 125 var open bool 126 select { 127 case res, open = <-resChan: 128 if !open { 129 return 130 } 131 case <-p.closeChan: 132 return 133 } 134 135 if skipAck := res.SkipAck(); res.Error() == nil || skipAck { 136 if skipAck { 137 atomic.AddInt64(&skipAcks, 1) 138 } 139 return 140 } 141 if !throt.Retry() { 142 return 143 } 144 } 145 } 146 147 wg := sync.WaitGroup{} 148 wg.Add(len(msgs)) 149 150 for _, msg := range msgs { 151 go func(m types.Message) { 152 sendMsg(m) 153 wg.Done() 154 }(msg) 155 } 156 157 wg.Wait() 158 throt.Reset() 159 160 var res types.Response 161 if skipAcks == int64(len(msgs)) { 162 res = response.NewUnack() 163 } else { 164 res = response.NewAck() 165 } 166 167 select { 168 case ogResChan <- res: 169 case <-p.closeChan: 170 return 171 } 172 } 173 174 //------------------------------------------------------------------------------ 175 176 // Consume assigns a messages channel for the pipeline to read. 177 func (p *Processor) Consume(msgs <-chan types.Transaction) error { 178 if p.messagesIn != nil { 179 return types.ErrAlreadyStarted 180 } 181 p.messagesIn = msgs 182 go p.loop() 183 return nil 184 } 185 186 // TransactionChan returns the channel used for consuming messages from this 187 // pipeline. 188 func (p *Processor) TransactionChan() <-chan types.Transaction { 189 return p.messagesOut 190 } 191 192 // CloseAsync shuts down the pipeline and stops processing messages. 193 func (p *Processor) CloseAsync() { 194 if atomic.CompareAndSwapInt32(&p.running, 1, 0) { 195 close(p.closeChan) 196 197 // Signal all children to close. 198 for _, c := range p.msgProcessors { 199 c.CloseAsync() 200 } 201 } 202 } 203 204 // WaitForClose blocks until the StackBuffer output has closed down. 205 func (p *Processor) WaitForClose(timeout time.Duration) error { 206 stopBy := time.Now().Add(timeout) 207 select { 208 case <-p.closed: 209 case <-time.After(time.Until(stopBy)): 210 return types.ErrTimeout 211 } 212 213 // Wait for all processors to close. 214 for _, c := range p.msgProcessors { 215 if err := c.WaitForClose(time.Until(stopBy)); err != nil { 216 return err 217 } 218 } 219 return nil 220 } 221 222 //------------------------------------------------------------------------------