github.com/Jeffail/benthos/v3@v3.65.0/lib/buffer/single_wrapper.go (about) 1 package buffer 2 3 import ( 4 "sync" 5 "sync/atomic" 6 "time" 7 8 "github.com/Jeffail/benthos/v3/internal/tracing" 9 "github.com/Jeffail/benthos/v3/lib/log" 10 "github.com/Jeffail/benthos/v3/lib/metrics" 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 // Single represents a method of buffering sequential messages, supporting 19 // only a single, sequential consumer. 20 type Single interface { 21 // ShiftMessage removes the oldest message from the stack. Returns the 22 // backlog in bytes. 23 ShiftMessage() (int, error) 24 25 // NextMessage reads the oldest message, the message is preserved until 26 // ShiftMessage is called. 27 NextMessage() (types.Message, error) 28 29 // PushMessage adds a new message to the stack. Returns the backlog in 30 // bytes. 31 PushMessage(types.Message) (int, error) 32 33 // CloseOnceEmpty closes the Buffer once the buffer has been emptied, this 34 // is a way for a writer to signal to a reader that it is finished writing 35 // messages, and therefore the reader can close once it is caught up. This 36 // call blocks until the close is completed. 37 CloseOnceEmpty() 38 39 // Close closes the Buffer so that blocked readers or writers become 40 // unblocked. 41 Close() 42 } 43 44 //------------------------------------------------------------------------------ 45 46 // SingleWrapper wraps a buffer with a Producer/Consumer interface. 47 type SingleWrapper struct { 48 stats metrics.Type 49 log log.Modular 50 conf Config 51 52 buffer Single 53 errThrottle *throttle.Type 54 55 running int32 56 consuming int32 57 58 messagesIn <-chan types.Transaction 59 messagesOut chan types.Transaction 60 responsesOut chan types.Response 61 62 closedWG sync.WaitGroup 63 64 stopConsumingChan chan struct{} 65 closeChan chan struct{} 66 closedChan chan struct{} 67 } 68 69 // NewSingleWrapper creates a new Producer/Consumer around a buffer. 70 func NewSingleWrapper( 71 conf Config, 72 buffer Single, 73 log log.Modular, 74 stats metrics.Type, 75 ) Type { 76 m := SingleWrapper{ 77 stats: stats, 78 log: log, 79 conf: conf, 80 buffer: buffer, 81 running: 1, 82 consuming: 1, 83 messagesOut: make(chan types.Transaction), 84 responsesOut: make(chan types.Response), 85 stopConsumingChan: make(chan struct{}), 86 closeChan: make(chan struct{}), 87 closedChan: make(chan struct{}), 88 } 89 90 m.errThrottle = throttle.New(throttle.OptCloseChan(m.closeChan)) 91 return &m 92 } 93 94 //------------------------------------------------------------------------------ 95 96 // inputLoop is an internal loop that brokers incoming messages to the buffer. 97 func (m *SingleWrapper) inputLoop() { 98 defer func() { 99 m.buffer.CloseOnceEmpty() 100 m.closedWG.Done() 101 }() 102 103 var ( 104 mWriteCount = m.stats.GetCounter("write.count") 105 mWriteErr = m.stats.GetCounter("write.error") 106 mWriteBacklog = m.stats.GetGauge("backlog") 107 ) 108 109 for atomic.LoadInt32(&m.consuming) == 1 { 110 var tr types.Transaction 111 var open bool 112 select { 113 case tr, open = <-m.messagesIn: 114 if !open { 115 return 116 } 117 case <-m.stopConsumingChan: 118 return 119 } 120 backlog, err := m.buffer.PushMessage(tracing.WithSiblingSpans("buffer_"+m.conf.Type, tr.Payload)) 121 if err == nil { 122 mWriteCount.Incr(1) 123 mWriteBacklog.Set(int64(backlog)) 124 } else { 125 mWriteErr.Incr(1) 126 } 127 select { 128 case tr.ResponseChan <- response.NewError(err): 129 case <-m.stopConsumingChan: 130 return 131 } 132 } 133 } 134 135 // outputLoop is an internal loop brokers buffer messages to output pipe. 136 func (m *SingleWrapper) outputLoop() { 137 defer func() { 138 m.buffer.Close() 139 close(m.messagesOut) 140 m.closedWG.Done() 141 }() 142 143 var ( 144 mReadCount = m.stats.GetCounter("read.count") 145 mReadErr = m.stats.GetCounter("read.error") 146 mSendSuccess = m.stats.GetCounter("send.success") 147 mSendErr = m.stats.GetCounter("send.error") 148 mLatency = m.stats.GetTimer("latency") 149 mBacklog = m.stats.GetGauge("backlog") 150 ) 151 152 var msg types.Message 153 for atomic.LoadInt32(&m.running) == 1 { 154 if msg == nil { 155 var err error 156 if msg, err = m.buffer.NextMessage(); err != nil { 157 if err != types.ErrTypeClosed { 158 mReadErr.Incr(1) 159 m.log.Errorf("Failed to read buffer: %v\n", err) 160 161 m.errThrottle.Retry() 162 163 // Unconventional errors here should always indicate some 164 // sort of corruption. Hopefully the corruption was message 165 // specific and not the whole buffer, so we can try shifting 166 // and reading again. 167 m.buffer.ShiftMessage() 168 } else { 169 // If our buffer is closed then we exit. 170 return 171 } 172 } else { 173 mReadCount.Incr(1) 174 m.errThrottle.Reset() 175 } 176 } 177 178 if msg != nil { 179 // It's possible that the buffer wiped our previous root span. 180 tracing.InitSpans("buffer_"+m.conf.Type, msg) 181 182 select { 183 case m.messagesOut <- types.NewTransaction(msg, m.responsesOut): 184 case <-m.closeChan: 185 return 186 } 187 res, open := <-m.responsesOut 188 if !open { 189 return 190 } 191 if res.Error() == nil { 192 mLatency.Timing(time.Since(msg.CreatedAt()).Nanoseconds()) 193 tracing.FinishSpans(msg) 194 msg = nil 195 backlog, _ := m.buffer.ShiftMessage() 196 mBacklog.Set(int64(backlog)) 197 mSendSuccess.Incr(1) 198 } else { 199 mSendErr.Incr(1) 200 } 201 } 202 } 203 } 204 205 // Consume assigns a messages channel for the output to read. 206 func (m *SingleWrapper) Consume(msgs <-chan types.Transaction) error { 207 if m.messagesIn != nil { 208 return types.ErrAlreadyStarted 209 } 210 m.messagesIn = msgs 211 212 m.closedWG.Add(2) 213 go m.inputLoop() 214 go m.outputLoop() 215 go func() { 216 m.closedWG.Wait() 217 close(m.closedChan) 218 }() 219 return nil 220 } 221 222 // TransactionChan returns the channel used for consuming messages from this 223 // buffer. 224 func (m *SingleWrapper) TransactionChan() <-chan types.Transaction { 225 return m.messagesOut 226 } 227 228 // CloseAsync shuts down the SingleWrapper and stops processing messages. 229 func (m *SingleWrapper) CloseAsync() { 230 m.StopConsuming() 231 if atomic.CompareAndSwapInt32(&m.running, 1, 0) { 232 close(m.closeChan) 233 } 234 } 235 236 // StopConsuming instructs the buffer to stop consuming messages and close once 237 // the buffer is empty. 238 func (m *SingleWrapper) StopConsuming() { 239 if atomic.CompareAndSwapInt32(&m.consuming, 1, 0) { 240 close(m.stopConsumingChan) 241 } 242 } 243 244 // WaitForClose blocks until the SingleWrapper output has closed down. 245 func (m *SingleWrapper) WaitForClose(timeout time.Duration) error { 246 select { 247 case <-m.closedChan: 248 case <-time.After(timeout): 249 return types.ErrTimeout 250 } 251 return nil 252 } 253 254 //------------------------------------------------------------------------------