github.com/Jeffail/benthos/v3@v3.65.0/lib/buffer/parallel_batcher.go (about) 1 package buffer 2 3 import ( 4 "sync/atomic" 5 "time" 6 7 "github.com/Jeffail/benthos/v3/internal/shutdown" 8 "github.com/Jeffail/benthos/v3/lib/log" 9 "github.com/Jeffail/benthos/v3/lib/message/batch" 10 "github.com/Jeffail/benthos/v3/lib/metrics" 11 "github.com/Jeffail/benthos/v3/lib/types" 12 ) 13 14 //------------------------------------------------------------------------------ 15 16 // ParallelBatcher wraps a buffer with a Producer/Consumer interface. 17 type ParallelBatcher struct { 18 stats metrics.Type 19 log log.Modular 20 21 child Type 22 batcher *batch.Policy 23 24 messagesOut chan types.Transaction 25 26 running int32 27 closeChan chan struct{} 28 closedChan chan struct{} 29 } 30 31 // NewParallelBatcher creates a new Producer/Consumer around a buffer. 32 func NewParallelBatcher( 33 batcher *batch.Policy, 34 child Type, 35 log log.Modular, 36 stats metrics.Type, 37 ) Type { 38 m := ParallelBatcher{ 39 stats: stats, 40 log: log, 41 child: child, 42 batcher: batcher, 43 messagesOut: make(chan types.Transaction), 44 running: 1, 45 closeChan: make(chan struct{}), 46 closedChan: make(chan struct{}), 47 } 48 return &m 49 } 50 51 //------------------------------------------------------------------------------ 52 53 // outputLoop is an internal loop brokers buffer messages to output pipe. 54 func (m *ParallelBatcher) outputLoop() { 55 defer func() { 56 m.child.CloseAsync() 57 _ = m.child.WaitForClose(shutdown.MaximumShutdownWait()) 58 59 m.batcher.CloseAsync() 60 _ = m.batcher.WaitForClose(shutdown.MaximumShutdownWait()) 61 62 close(m.messagesOut) 63 close(m.closedChan) 64 }() 65 66 var nextTimedBatchChan <-chan time.Time 67 if tNext := m.batcher.UntilNext(); tNext >= 0 { 68 nextTimedBatchChan = time.After(tNext) 69 } 70 71 var pendingResChans []chan<- types.Response 72 for atomic.LoadInt32(&m.running) == 1 { 73 if nextTimedBatchChan == nil { 74 if tNext := m.batcher.UntilNext(); tNext >= 0 { 75 nextTimedBatchChan = time.After(tNext) 76 } 77 } 78 79 var flushBatch bool 80 select { 81 case tran, open := <-m.child.TransactionChan(): 82 if !open { 83 // Final flush of remaining documents. 84 atomic.StoreInt32(&m.running, 0) 85 flushBatch = true 86 // If we're waiting for a timed batch then we will respect it. 87 if nextTimedBatchChan != nil { 88 select { 89 case <-nextTimedBatchChan: 90 case <-m.closeChan: 91 return 92 } 93 } 94 } else { 95 tran.Payload.Iter(func(i int, p types.Part) error { 96 if m.batcher.Add(p) { 97 flushBatch = true 98 } 99 return nil 100 }) 101 pendingResChans = append(pendingResChans, tran.ResponseChan) 102 } 103 case <-nextTimedBatchChan: 104 flushBatch = true 105 nextTimedBatchChan = nil 106 case <-m.closeChan: 107 return 108 } 109 110 if !flushBatch { 111 continue 112 } 113 114 sendMsg := m.batcher.Flush() 115 if sendMsg == nil { 116 continue 117 } 118 119 resChan := make(chan types.Response) 120 select { 121 case m.messagesOut <- types.NewTransaction(sendMsg, resChan): 122 case <-m.closeChan: 123 return 124 } 125 126 go func(rChan chan types.Response, upstreamResChans []chan<- types.Response) { 127 select { 128 case <-m.closeChan: 129 return 130 case res, open := <-rChan: 131 if !open { 132 return 133 } 134 for _, c := range upstreamResChans { 135 select { 136 case <-m.closeChan: 137 return 138 case c <- res: 139 } 140 } 141 } 142 }(resChan, pendingResChans) 143 pendingResChans = nil 144 } 145 } 146 147 // Consume assigns a messages channel for the output to read. 148 func (m *ParallelBatcher) Consume(msgs <-chan types.Transaction) error { 149 if err := m.child.Consume(msgs); err != nil { 150 return err 151 } 152 go m.outputLoop() 153 return nil 154 } 155 156 // TransactionChan returns the channel used for consuming messages from this 157 // buffer. 158 func (m *ParallelBatcher) TransactionChan() <-chan types.Transaction { 159 return m.messagesOut 160 } 161 162 // CloseAsync shuts down the ParallelBatcher and stops processing messages. 163 func (m *ParallelBatcher) CloseAsync() { 164 m.child.CloseAsync() 165 if atomic.CompareAndSwapInt32(&m.running, 1, 0) { 166 close(m.closeChan) 167 } 168 } 169 170 // StopConsuming instructs the buffer to stop consuming messages and close once 171 // the buffer is empty. 172 func (m *ParallelBatcher) StopConsuming() { 173 m.child.StopConsuming() 174 } 175 176 // WaitForClose blocks until the ParallelBatcher output has closed down. 177 func (m *ParallelBatcher) WaitForClose(timeout time.Duration) error { 178 select { 179 case <-m.closedChan: 180 case <-time.After(timeout): 181 return types.ErrTimeout 182 } 183 return nil 184 } 185 186 //------------------------------------------------------------------------------