github.com/Jeffail/benthos/v3@v3.65.0/lib/input/reader/async_preserver.go (about) 1 package reader 2 3 import ( 4 "context" 5 "sync" 6 "sync/atomic" 7 "time" 8 9 "github.com/Jeffail/benthos/v3/internal/batch" 10 imessage "github.com/Jeffail/benthos/v3/internal/message" 11 "github.com/Jeffail/benthos/v3/lib/message" 12 "github.com/Jeffail/benthos/v3/lib/types" 13 "github.com/cenkalti/backoff/v4" 14 ) 15 16 //------------------------------------------------------------------------------ 17 18 type asyncPreserverResend struct { 19 boff backoff.BackOff 20 attempts int 21 msg types.Message 22 ackFn AsyncAckFn 23 } 24 25 func newResendMsg(msg types.Message, ackFn AsyncAckFn) asyncPreserverResend { 26 boff := backoff.NewExponentialBackOff() 27 boff.InitialInterval = time.Millisecond 28 boff.MaxInterval = time.Second 29 boff.Multiplier = 1.1 30 boff.MaxElapsedTime = 0 31 return asyncPreserverResend{boff, 0, msg, ackFn} 32 } 33 34 // AsyncPreserver is a wrapper for reader.Async implementations that keeps a 35 // buffer of sent messages until they are acknowledged. If an error occurs 36 // during message propagation the contents of the buffer will be resent instead 37 // of reading new messages until it is depleted. AsyncPreserver implements 38 // reader.Async. 39 // 40 // Wrapping an input with this type is useful when your source of messages 41 // doesn't have a concept of a NoAck (like Kafka), and instead of "rejecting" 42 // messages we always intend to simply retry them until success. 43 type AsyncPreserver struct { 44 resendMessages []asyncPreserverResend 45 resendInterrupt func() 46 msgsMut sync.Mutex 47 pendingMessages int64 48 49 inputClosed int32 50 r Async 51 } 52 53 // NewAsyncPreserver returns a new AsyncPreserver wrapper around a reader.Async. 54 func NewAsyncPreserver(r Async) *AsyncPreserver { 55 return &AsyncPreserver{ 56 r: r, 57 resendInterrupt: func() {}, 58 } 59 } 60 61 //------------------------------------------------------------------------------ 62 63 // ConnectWithContext attempts to establish a connection to the source, if 64 // unsuccessful returns an error. If the attempt is successful (or not 65 // necessary) returns nil. 66 func (p *AsyncPreserver) ConnectWithContext(ctx context.Context) error { 67 err := p.r.ConnectWithContext(ctx) 68 // If our source has finished but we still have messages in flight then 69 // we act like we're still open. Read will be called and we can either 70 // return the pending messages or wait for them. 71 if err == types.ErrTypeClosed && atomic.LoadInt64(&p.pendingMessages) > 0 { 72 atomic.StoreInt32(&p.inputClosed, 1) 73 err = nil 74 } 75 return err 76 } 77 78 func (p *AsyncPreserver) wrapAckFn(m asyncPreserverResend) (types.Message, AsyncAckFn) { 79 if m.msg.Len() == 1 { 80 return p.wrapSingleAckFn(m) 81 } 82 return p.wrapBatchAckFn(m) 83 } 84 85 func (p *AsyncPreserver) wrapBatchAckFn(m asyncPreserverResend) (types.Message, AsyncAckFn) { 86 sortGroup, trackedMsg := imessage.NewSortGroup(m.msg) 87 88 return trackedMsg, func(ctx context.Context, res types.Response) error { 89 if res.Error() != nil { 90 resendMsg := m.msg 91 if walkable, ok := res.Error().(batch.WalkableError); ok && walkable.IndexedErrors() < m.msg.Len() { 92 resendMsg = message.New(nil) 93 walkable.WalkParts(func(i int, p types.Part, e error) bool { 94 if e == nil { 95 return true 96 } 97 if tagIndex := sortGroup.GetIndex(p); tagIndex >= 0 { 98 resendMsg.Append(m.msg.Get(tagIndex)) 99 return true 100 } 101 102 // If we couldn't link the errored part back to an original 103 // message then we need to retry all of them. 104 resendMsg = m.msg 105 return false 106 }) 107 if resendMsg.Len() == 0 { 108 resendMsg = m.msg 109 } 110 } 111 m.msg = resendMsg 112 113 p.msgsMut.Lock() 114 p.resendMessages = append(p.resendMessages, m) 115 p.resendInterrupt() 116 p.msgsMut.Unlock() 117 return nil 118 } 119 atomic.AddInt64(&p.pendingMessages, -1) 120 return m.ackFn(ctx, res) 121 } 122 } 123 124 func (p *AsyncPreserver) wrapSingleAckFn(m asyncPreserverResend) (types.Message, AsyncAckFn) { 125 return m.msg, func(ctx context.Context, res types.Response) error { 126 if res.Error() != nil { 127 p.msgsMut.Lock() 128 p.resendMessages = append(p.resendMessages, m) 129 p.resendInterrupt() 130 p.msgsMut.Unlock() 131 return nil 132 } 133 atomic.AddInt64(&p.pendingMessages, -1) 134 return m.ackFn(ctx, res) 135 } 136 } 137 138 // ReadWithContext attempts to read a new message from the source. 139 func (p *AsyncPreserver) ReadWithContext(ctx context.Context) (types.Message, AsyncAckFn, error) { 140 var cancel func() 141 ctx, cancel = context.WithCancel(ctx) 142 defer cancel() 143 144 // If we have messages queued to be resent we prioritise them over reading 145 // new messages. 146 p.msgsMut.Lock() 147 if lMsgs := len(p.resendMessages); lMsgs > 0 { 148 resend := p.resendMessages[0] 149 if lMsgs > 1 { 150 p.resendMessages = p.resendMessages[1:] 151 } else { 152 p.resendMessages = nil 153 } 154 p.msgsMut.Unlock() 155 156 resend.attempts++ 157 if resend.attempts > 2 { 158 // This sleep prevents a busy loop on permanently failed messages. 159 if tout := resend.boff.NextBackOff(); tout > 0 { 160 select { 161 case <-time.After(tout): 162 case <-ctx.Done(): 163 return nil, nil, ctx.Err() 164 } 165 } 166 } 167 sendMsg, ackFn := p.wrapAckFn(resend) 168 return sendMsg, ackFn, nil 169 } 170 p.resendInterrupt = cancel 171 p.msgsMut.Unlock() 172 173 var ( 174 msg types.Message 175 aFn AsyncAckFn 176 err error 177 ) 178 if atomic.LoadInt32(&p.inputClosed) > 0 { 179 err = types.ErrTypeClosed 180 } else { 181 msg, aFn, err = p.r.ReadWithContext(ctx) 182 } 183 if err != nil { 184 // If our source has finished but we still have messages in flight then 185 // we block, ideally until the messages are acked. 186 if err == types.ErrTypeClosed && atomic.LoadInt64(&p.pendingMessages) > 0 { 187 // The context is cancelled either when new pending messages are 188 // ready, or when the upstream component cancels. If the former 189 // occurs then we still return the cancelled error and let Read get 190 // called to gobble up the new pending messages. 191 for { 192 select { 193 case <-ctx.Done(): 194 return nil, nil, types.ErrTimeout 195 case <-time.After(time.Millisecond * 10): 196 if atomic.LoadInt64(&p.pendingMessages) <= 0 { 197 return nil, nil, types.ErrTypeClosed 198 } 199 } 200 } 201 } 202 return nil, nil, err 203 } 204 atomic.AddInt64(&p.pendingMessages, 1) 205 sendMsg, ackFn := p.wrapAckFn(newResendMsg(msg, aFn)) 206 return sendMsg, ackFn, nil 207 } 208 209 // CloseAsync triggers the asynchronous closing of the reader. 210 func (p *AsyncPreserver) CloseAsync() { 211 p.r.CloseAsync() 212 } 213 214 // WaitForClose blocks until either the reader is finished closing or a timeout 215 // occurs. 216 func (p *AsyncPreserver) WaitForClose(tout time.Duration) error { 217 return p.r.WaitForClose(tout) 218 } 219 220 //------------------------------------------------------------------------------