github.com/Jeffail/benthos/v3@v3.65.0/lib/processor/parallel.go (about) 1 package processor 2 3 import ( 4 "fmt" 5 "sync" 6 "sync/atomic" 7 "time" 8 9 "github.com/Jeffail/benthos/v3/internal/docs" 10 "github.com/Jeffail/benthos/v3/internal/interop" 11 "github.com/Jeffail/benthos/v3/lib/log" 12 "github.com/Jeffail/benthos/v3/lib/message" 13 "github.com/Jeffail/benthos/v3/lib/metrics" 14 "github.com/Jeffail/benthos/v3/lib/response" 15 "github.com/Jeffail/benthos/v3/lib/types" 16 ) 17 18 //------------------------------------------------------------------------------ 19 20 func init() { 21 Constructors[TypeParallel] = TypeSpec{ 22 constructor: NewParallel, 23 Categories: []Category{ 24 CategoryComposition, 25 }, 26 Summary: ` 27 A processor that applies a list of child processors to messages of a batch as 28 though they were each a batch of one message (similar to the 29 ` + "[`for_each`](/docs/components/processors/for_each)" + ` processor), but where each message is 30 processed in parallel.`, 31 Description: ` 32 The field ` + "`cap`" + `, if greater than zero, caps the maximum number of 33 parallel processing threads.`, 34 UsesBatches: true, 35 FieldSpecs: docs.FieldSpecs{ 36 docs.FieldCommon("cap", "The maximum number of messages to have processing at a given time."), 37 docs.FieldCommon("processors", "A list of child processors to apply.").Array().HasType(docs.FieldTypeProcessor), 38 }, 39 } 40 } 41 42 //------------------------------------------------------------------------------ 43 44 // ParallelConfig is a config struct containing fields for the Parallel 45 // processor. 46 type ParallelConfig struct { 47 Cap int `json:"cap" yaml:"cap"` 48 Processors []Config `json:"processors" yaml:"processors"` 49 } 50 51 // NewParallelConfig returns a default ParallelConfig. 52 func NewParallelConfig() ParallelConfig { 53 return ParallelConfig{ 54 Cap: 0, 55 Processors: []Config{}, 56 } 57 } 58 59 //------------------------------------------------------------------------------ 60 61 // Parallel is a processor that applies a list of child processors to each 62 // message of a batch individually. 63 type Parallel struct { 64 children []types.Processor 65 cap int 66 67 log log.Modular 68 69 mCount metrics.StatCounter 70 mErr metrics.StatCounter 71 mSent metrics.StatCounter 72 mBatchSent metrics.StatCounter 73 } 74 75 // NewParallel returns a Parallel processor. 76 func NewParallel( 77 conf Config, mgr types.Manager, log log.Modular, stats metrics.Type, 78 ) (Type, error) { 79 var children []types.Processor 80 for i, pconf := range conf.Parallel.Processors { 81 pMgr, pLog, pStats := interop.LabelChild(fmt.Sprintf("%v", i), mgr, log, stats) 82 proc, err := New(pconf, pMgr, pLog, pStats) 83 if err != nil { 84 return nil, err 85 } 86 children = append(children, proc) 87 } 88 return &Parallel{ 89 children: children, 90 cap: conf.Parallel.Cap, 91 log: log, 92 93 mCount: stats.GetCounter("count"), 94 mErr: stats.GetCounter("error"), 95 mSent: stats.GetCounter("sent"), 96 mBatchSent: stats.GetCounter("batch.sent"), 97 }, nil 98 } 99 100 //------------------------------------------------------------------------------ 101 102 // ProcessMessage applies the processor to a message, either creating >0 103 // resulting messages or a response to be sent back to the message source. 104 func (p *Parallel) ProcessMessage(msg types.Message) ([]types.Message, types.Response) { 105 p.mCount.Incr(1) 106 107 resultMsgs := make([]types.Message, msg.Len()) 108 msg.Iter(func(i int, p types.Part) error { 109 tmpMsg := message.New(nil) 110 tmpMsg.SetAll([]types.Part{p}) 111 resultMsgs[i] = tmpMsg 112 return nil 113 }) 114 115 max := p.cap 116 if max == 0 || msg.Len() < max { 117 max = msg.Len() 118 } 119 120 reqChan := make(chan int) 121 wg := sync.WaitGroup{} 122 wg.Add(max) 123 124 var unAcks int32 125 for i := 0; i < max; i++ { 126 go func() { 127 for index := range reqChan { 128 resMsgs, res := ExecuteAll(p.children, resultMsgs[index]) 129 if res != nil && res.SkipAck() { 130 atomic.AddInt32(&unAcks, 1) 131 } 132 resultParts := []types.Part{} 133 for _, m := range resMsgs { 134 m.Iter(func(i int, p types.Part) error { 135 resultParts = append(resultParts, p) 136 return nil 137 }) 138 } 139 resultMsgs[index].SetAll(resultParts) 140 } 141 wg.Done() 142 }() 143 } 144 for i := 0; i < msg.Len(); i++ { 145 reqChan <- i 146 } 147 close(reqChan) 148 wg.Wait() 149 150 resMsg := message.New(nil) 151 for _, m := range resultMsgs { 152 m.Iter(func(i int, p types.Part) error { 153 resMsg.Append(p) 154 return nil 155 }) 156 } 157 if resMsg.Len() == 0 { 158 if msg.Len() > 0 && unAcks == int32(msg.Len()) { 159 return nil, response.NewUnack() 160 } 161 return nil, response.NewAck() 162 } 163 164 p.mBatchSent.Incr(1) 165 p.mSent.Incr(int64(resMsg.Len())) 166 167 return []types.Message{resMsg}, nil 168 } 169 170 // CloseAsync shuts down the processor and stops processing requests. 171 func (p *Parallel) CloseAsync() { 172 for _, c := range p.children { 173 c.CloseAsync() 174 } 175 } 176 177 // WaitForClose blocks until the processor has closed down. 178 func (p *Parallel) WaitForClose(timeout time.Duration) error { 179 stopBy := time.Now().Add(timeout) 180 for _, c := range p.children { 181 if err := c.WaitForClose(time.Until(stopBy)); err != nil { 182 return err 183 } 184 } 185 return nil 186 } 187 188 //------------------------------------------------------------------------------