github.com/Jeffail/benthos/v3@v3.65.0/lib/output/batcher.go (about) 1 package output 2 3 import ( 4 "context" 5 "fmt" 6 "time" 7 8 "github.com/Jeffail/benthos/v3/internal/component/output" 9 "github.com/Jeffail/benthos/v3/internal/interop" 10 "github.com/Jeffail/benthos/v3/internal/shutdown" 11 "github.com/Jeffail/benthos/v3/internal/transaction" 12 "github.com/Jeffail/benthos/v3/lib/log" 13 "github.com/Jeffail/benthos/v3/lib/message/batch" 14 "github.com/Jeffail/benthos/v3/lib/metrics" 15 "github.com/Jeffail/benthos/v3/lib/types" 16 ) 17 18 // Batcher wraps an output with a batching policy. 19 type Batcher struct { 20 stats metrics.Type 21 log log.Modular 22 23 child Type 24 batcher *batch.Policy 25 26 messagesIn <-chan types.Transaction 27 messagesOut chan types.Transaction 28 29 shutSig *shutdown.Signaller 30 } 31 32 // NewBatcherFromConfig creates a new output preceded by a batching mechanism 33 // that enforces a given batching policy configuration. 34 func NewBatcherFromConfig( 35 conf batch.PolicyConfig, 36 child Type, 37 mgr types.Manager, 38 log log.Modular, 39 stats metrics.Type, 40 ) (Type, error) { 41 if !conf.IsNoop() { 42 bMgr, bLog, bStats := interop.LabelChild("batching", mgr, log, stats) 43 policy, err := batch.NewPolicy(conf, bMgr, bLog, bStats) 44 if err != nil { 45 return nil, fmt.Errorf("failed to construct batch policy: %v", err) 46 } 47 child = NewBatcher(policy, child, log, stats) 48 } 49 return child, nil 50 } 51 52 // NewBatcher creates a new output preceded by a batching mechanism that 53 // enforces a given batching policy. 54 func NewBatcher( 55 batcher *batch.Policy, 56 child Type, 57 log log.Modular, 58 stats metrics.Type, 59 ) Type { 60 m := Batcher{ 61 stats: stats, 62 log: log, 63 child: child, 64 batcher: batcher, 65 messagesOut: make(chan types.Transaction), 66 shutSig: shutdown.NewSignaller(), 67 } 68 return &m 69 } 70 71 //------------------------------------------------------------------------------ 72 73 func (m *Batcher) loop() { 74 defer func() { 75 close(m.messagesOut) 76 m.child.CloseAsync() 77 _ = m.child.WaitForClose(shutdown.MaximumShutdownWait()) 78 79 m.batcher.CloseAsync() 80 _ = m.batcher.WaitForClose(shutdown.MaximumShutdownWait()) 81 82 m.shutSig.ShutdownComplete() 83 }() 84 85 var nextTimedBatchChan <-chan time.Time 86 if tNext := m.batcher.UntilNext(); tNext >= 0 { 87 nextTimedBatchChan = time.After(tNext) 88 } 89 90 var pendingTrans []*transaction.Tracked 91 for !m.shutSig.ShouldCloseAtLeisure() { 92 if nextTimedBatchChan == nil { 93 if tNext := m.batcher.UntilNext(); tNext >= 0 { 94 nextTimedBatchChan = time.After(tNext) 95 } 96 } 97 98 var flushBatch bool 99 select { 100 case tran, open := <-m.messagesIn: 101 if !open { 102 if flushBatch = m.batcher.Count() > 0; !flushBatch { 103 return 104 } 105 106 // If we're waiting for a timed batch then we will respect it. 107 if nextTimedBatchChan != nil { 108 select { 109 case <-nextTimedBatchChan: 110 case <-m.shutSig.CloseAtLeisureChan(): 111 } 112 } 113 } else { 114 trackedTran := transaction.NewTracked(tran.Payload, tran.ResponseChan) 115 trackedTran.Message().Iter(func(i int, p types.Part) error { 116 if m.batcher.Add(p) { 117 flushBatch = true 118 } 119 return nil 120 }) 121 pendingTrans = append(pendingTrans, trackedTran) 122 } 123 case <-nextTimedBatchChan: 124 flushBatch = true 125 nextTimedBatchChan = nil 126 case <-m.shutSig.CloseAtLeisureChan(): 127 flushBatch = true 128 } 129 130 if !flushBatch { 131 continue 132 } 133 134 sendMsg := m.batcher.Flush() 135 if sendMsg == nil { 136 continue 137 } 138 139 resChan := make(chan types.Response) 140 select { 141 case m.messagesOut <- types.NewTransaction(sendMsg, resChan): 142 case <-m.shutSig.CloseAtLeisureChan(): 143 return 144 } 145 146 go func(rChan chan types.Response, upstreamTrans []*transaction.Tracked) { 147 select { 148 case <-m.shutSig.CloseAtLeisureChan(): 149 return 150 case res, open := <-rChan: 151 if !open { 152 return 153 } 154 closeAtLeisureCtx, done := m.shutSig.CloseAtLeisureCtx(context.Background()) 155 for _, t := range upstreamTrans { 156 if err := t.Ack(closeAtLeisureCtx, res.Error()); err != nil { 157 done() 158 return 159 } 160 } 161 done() 162 } 163 }(resChan, pendingTrans) 164 pendingTrans = nil 165 } 166 } 167 168 // Connected returns a boolean indicating whether this output is currently 169 // connected to its target. 170 func (m *Batcher) Connected() bool { 171 return m.child.Connected() 172 } 173 174 // MaxInFlight returns the maximum number of in flight messages permitted by the 175 // output. This value can be used to determine a sensible value for parent 176 // outputs, but should not be relied upon as part of dispatcher logic. 177 func (m *Batcher) MaxInFlight() (int, bool) { 178 return output.GetMaxInFlight(m.child) 179 } 180 181 // Consume assigns a messages channel for the output to read. 182 func (m *Batcher) Consume(msgs <-chan types.Transaction) error { 183 if m.messagesIn != nil { 184 return types.ErrAlreadyStarted 185 } 186 if err := m.child.Consume(m.messagesOut); err != nil { 187 return err 188 } 189 m.messagesIn = msgs 190 go m.loop() 191 return nil 192 } 193 194 // CloseAsync shuts down the Batcher and stops processing messages. 195 func (m *Batcher) CloseAsync() { 196 m.shutSig.CloseAtLeisure() 197 } 198 199 // WaitForClose blocks until the Batcher output has closed down. 200 func (m *Batcher) WaitForClose(timeout time.Duration) error { 201 select { 202 case <-m.shutSig.HasClosedChan(): 203 case <-time.After(timeout): 204 return types.ErrTimeout 205 } 206 return nil 207 }