github.com/Jeffail/benthos/v3@v3.65.0/internal/checkpoint/capped.go (about) 1 package checkpoint 2 3 import ( 4 "context" 5 "sync" 6 7 "github.com/Jeffail/benthos/v3/lib/types" 8 ) 9 10 // Capped receives an ordered feed of integer based offsets being tracked, and 11 // an unordered feed of integer based offsets that are resolved, and is able to 12 // return the highest offset currently able to be committed such that an 13 // unresolved offset is never committed. 14 // 15 // If the number of unresolved tracked values meets a given cap the next attempt 16 // to track a value will be blocked until the next value is resolved. 17 // 18 // This component is safe to use concurrently across goroutines. 19 type Capped struct { 20 t *Type 21 cap int64 22 cond *sync.Cond 23 } 24 25 // NewCapped returns a new capped checkpointer. 26 func NewCapped(capacity int64) *Capped { 27 return &Capped{ 28 t: New(), 29 cap: capacity, 30 cond: sync.NewCond(&sync.Mutex{}), 31 } 32 } 33 34 // Highest returns the current highest checkpoint. 35 func (c *Capped) Highest() interface{} { 36 c.cond.L.Lock() 37 defer c.cond.L.Unlock() 38 return c.t.Highest() 39 } 40 41 // Track a new unresolved integer offset. This offset will be cached until it is 42 // marked as resolved. While it is cached no higher valued offset will ever be 43 // committed. If the provided value is lower than an already provided value an 44 // error is returned. 45 func (c *Capped) Track(ctx context.Context, payload interface{}, batchSize int64) (func() interface{}, error) { 46 c.cond.L.Lock() 47 defer c.cond.L.Unlock() 48 49 var cancel func() 50 ctx, cancel = context.WithCancel(ctx) 51 defer cancel() 52 go func() { 53 <-ctx.Done() 54 c.cond.L.Lock() 55 c.cond.Broadcast() 56 c.cond.L.Unlock() 57 }() 58 59 pending := c.t.Pending() 60 for pending > 0 && pending+batchSize > c.cap { 61 c.cond.Wait() 62 select { 63 case <-ctx.Done(): 64 return nil, types.ErrTimeout 65 default: 66 } 67 pending = c.t.Pending() 68 } 69 70 resolveFn := c.t.Track(payload, batchSize) 71 72 return func() interface{} { 73 c.cond.L.Lock() 74 defer c.cond.L.Unlock() 75 76 highest := resolveFn() 77 c.cond.Broadcast() 78 return highest 79 }, nil 80 }