github.com/Jeffail/benthos/v3@v3.65.0/lib/input/reader/async_batcher.go (about) 1 package reader 2 3 import ( 4 "context" 5 "fmt" 6 "time" 7 8 "github.com/Jeffail/benthos/v3/internal/interop" 9 "github.com/Jeffail/benthos/v3/lib/log" 10 "github.com/Jeffail/benthos/v3/lib/message/batch" 11 "github.com/Jeffail/benthos/v3/lib/metrics" 12 "github.com/Jeffail/benthos/v3/lib/types" 13 ) 14 15 //------------------------------------------------------------------------------ 16 17 // AsyncBatcher is a wrapper for reader.Async implementations that applies a 18 // batching policy to incoming payloads. Once a batch is created and sent the 19 // provided ack function ensures all messages within the batch are acked. 20 type AsyncBatcher struct { 21 pendingAcks []AsyncAckFn 22 batcher *batch.Policy 23 r Async 24 } 25 26 // NewAsyncBatcher returns a new AsyncBatcher wrapper around a reader.Async. 27 func NewAsyncBatcher( 28 batchConfig batch.PolicyConfig, 29 r Async, 30 mgr types.Manager, 31 log log.Modular, 32 stats metrics.Type, 33 ) (Async, error) { 34 if batchConfig.IsNoop() { 35 return r, nil 36 } 37 mgr, log, stats = interop.LabelChild("batching", mgr, log, stats) 38 policy, err := batch.NewPolicy(batchConfig, mgr, log, stats) 39 if err != nil { 40 return nil, fmt.Errorf("failed to construct batch policy: %v", err) 41 } 42 return &AsyncBatcher{ 43 batcher: policy, 44 r: r, 45 }, nil 46 } 47 48 //------------------------------------------------------------------------------ 49 50 // ConnectWithContext attempts to establish a connection to the source, if 51 // unsuccessful returns an error. If the attempt is successful (or not 52 // necessary) returns nil. 53 func (p *AsyncBatcher) ConnectWithContext(ctx context.Context) error { 54 return p.r.ConnectWithContext(ctx) 55 } 56 57 func (p *AsyncBatcher) wrapAckFns() AsyncAckFn { 58 ackFns := p.pendingAcks 59 p.pendingAcks = nil 60 return func(ctx context.Context, res types.Response) error { 61 errs := []error{} 62 errChan := make(chan error) 63 for _, fn := range ackFns { 64 go func(f AsyncAckFn) { 65 errChan <- f(ctx, res) 66 }(fn) 67 } 68 for range ackFns { 69 if err := <-errChan; err != nil { 70 errs = append(errs, err) 71 } 72 } 73 if len(errs) == 0 { 74 return nil 75 } 76 if len(errs) == 1 { 77 return errs[0] 78 } 79 return fmt.Errorf("multiple messages failed to ack: %v", errs) 80 } 81 } 82 83 // ReadWithContext attempts to read a new message from the source. 84 func (p *AsyncBatcher) ReadWithContext(ctx context.Context) (types.Message, AsyncAckFn, error) { 85 var forcedBatchDeadline time.Time 86 if tout := p.batcher.UntilNext(); tout >= 0 { 87 forcedBatchDeadline = time.Now().Add(tout) 88 var cancel context.CancelFunc 89 ctx, cancel = context.WithDeadline(ctx, forcedBatchDeadline) 90 defer cancel() 91 } 92 93 flushBatch := false 94 for !flushBatch { 95 msg, ackFn, err := p.r.ReadWithContext(ctx) 96 if err != nil { 97 if !forcedBatchDeadline.IsZero() && !time.Now().Before(forcedBatchDeadline) { 98 if batch := p.batcher.Flush(); batch != nil && batch.Len() > 0 { 99 return batch, p.wrapAckFns(), nil 100 } 101 } 102 if err == types.ErrTimeout { 103 // If the call "timed out" it could either mean that the context 104 // was cancelled, in which case we want to return right now, or 105 // that the underlying mechanism timed out, in which case we 106 // simply want to try again. 107 select { 108 case <-ctx.Done(): 109 if batch := p.batcher.Flush(); batch != nil && batch.Len() > 0 { 110 return batch, p.wrapAckFns(), nil 111 } 112 return nil, nil, types.ErrTimeout 113 default: 114 } 115 continue 116 } 117 if err == types.ErrTypeClosed { 118 if batch := p.batcher.Flush(); batch != nil && batch.Len() > 0 { 119 return batch, p.wrapAckFns(), nil 120 } 121 } 122 return nil, nil, err 123 } 124 125 p.pendingAcks = append(p.pendingAcks, ackFn) 126 msg.Iter(func(i int, part types.Part) error { 127 flushBatch = p.batcher.Add(part) || flushBatch 128 return nil 129 }) 130 } 131 return p.batcher.Flush(), p.wrapAckFns(), nil 132 } 133 134 // CloseAsync triggers the asynchronous closing of the reader. 135 func (p *AsyncBatcher) CloseAsync() { 136 p.r.CloseAsync() 137 } 138 139 // WaitForClose blocks until either the reader is finished closing or a timeout 140 // occurs. 141 func (p *AsyncBatcher) WaitForClose(tout time.Duration) error { 142 tstop := time.Now().Add(tout) 143 err := p.r.WaitForClose(time.Until(tstop)) 144 p.batcher.CloseAsync() 145 if err != nil { 146 return err 147 } 148 return p.batcher.WaitForClose(time.Until(tstop)) 149 } 150 151 //------------------------------------------------------------------------------