github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/pkg/chdelay/channel_delayer.go (about) 1 // Copyright 2022 PingCAP, Inc. 2 // 3 // Licensed under the Apache License, Version 2.0 (the "License"); 4 // you may not use this file except in compliance with the License. 5 // You may obtain a copy of the License at 6 // 7 // http://www.apache.org/licenses/LICENSE-2.0 8 // 9 // Unless required by applicable law or agreed to in writing, software 10 // distributed under the License is distributed on an "AS IS" BASIS, 11 // See the License for the specific language governing permissions and 12 // limitations under the License. 13 14 package chdelay 15 16 import ( 17 "sync" 18 "time" 19 ) 20 21 // ChannelDelayer provides a mechanism to inject delay in a channel. 22 // 23 // Algorithm sketch: 24 // When an element arrives from the input channel, 25 // attach the current timestamp and put it in a queue. 26 // The top of the queue is checked, if the element 27 // has stayed in the queue for more than `delayBy`, 28 // then the element in popped and sent to the output channel. 29 type ChannelDelayer[T any] struct { 30 inCh <-chan T 31 outCh chan T 32 33 queue []entry[T] 34 size int 35 36 closeCh chan struct{} 37 wg sync.WaitGroup 38 39 delayBy time.Duration 40 } 41 42 type entry[T any] struct { 43 elem T 44 inTime time.Time 45 } 46 47 // NewChannelDelayer creates a new ChannelDelayer. 48 func NewChannelDelayer[T any]( 49 delayBy time.Duration, 50 in <-chan T, 51 queueSize int, 52 outChSize int, 53 ) *ChannelDelayer[T] { 54 ret := &ChannelDelayer[T]{ 55 inCh: in, 56 outCh: make(chan T, outChSize), 57 queue: make([]entry[T], 0, queueSize), 58 size: queueSize, 59 closeCh: make(chan struct{}), 60 delayBy: delayBy, 61 } 62 63 ret.wg.Add(1) 64 go func() { 65 ret.run() 66 }() 67 68 return ret 69 } 70 71 // Out returns the delayed channel. The downstream logic 72 // should read from Out(). 73 func (d *ChannelDelayer[T]) Out() <-chan T { 74 return d.outCh 75 } 76 77 // Close closes the ChannelDelayer. 78 func (d *ChannelDelayer[T]) Close() { 79 close(d.closeCh) 80 d.wg.Wait() 81 } 82 83 func (d *ChannelDelayer[T]) run() { 84 defer d.wg.Done() 85 defer close(d.outCh) 86 87 ticker := time.NewTicker(time.Millisecond) 88 defer ticker.Stop() 89 90 // currentTime is a timestamp cache to 91 // avoid having to read the system's 92 // clock frequently. 93 currentTime := time.Now() 94 95 // Returns the output channel if 96 // the first element in the queue 97 // is ready to be popped. 98 // Otherwise, nil is returned. 99 // Note that nil channels are ignored in 100 // a select statement, so it would disable 101 // a case block. 102 outChIfReady := func() chan<- T { 103 if len(d.queue) == 0 { 104 return nil 105 } 106 if currentTime.Sub(d.queue[0].inTime) >= d.delayBy { 107 return d.outCh 108 } 109 return nil 110 } 111 112 // dummyEntry provides a zero value entry. 113 var dummyEntry entry[T] 114 115 for { 116 var firstElem *T 117 if len(d.queue) > 0 { 118 firstElem = &d.queue[0].elem 119 } else { 120 // Must provide a valid pointer. 121 firstElem = &dummyEntry.elem 122 } 123 124 select { 125 case <-d.closeCh: 126 return 127 case <-ticker.C: 128 currentTime = time.Now() 129 case inElem, ok := <-d.inChIfSizeOk(): 130 if !ok { 131 if len(d.queue) == 0 { 132 return 133 } 134 continue 135 } 136 d.queue = append(d.queue, entry[T]{ 137 elem: inElem, 138 inTime: time.Now(), 139 }) 140 case outChIfReady() <- *firstElem: 141 // Cleans any reference to *T if T is a pointer, 142 // to prompt a timely GC. 143 d.queue[0] = dummyEntry 144 d.queue = d.queue[1:] 145 146 LOOP: 147 // Drain the queue as much as possible. 148 for { 149 if len(d.queue) == 0 { 150 break LOOP 151 } 152 if currentTime.Sub(d.queue[0].inTime) < d.delayBy { 153 break 154 } 155 156 select { 157 case d.outCh <- d.queue[0].elem: 158 // Cleans any reference to *T if T is a pointer 159 d.queue[0] = dummyEntry 160 d.queue = d.queue[1:] 161 default: 162 break LOOP 163 } 164 } 165 } 166 } 167 } 168 169 func (d *ChannelDelayer[T]) inChIfSizeOk() <-chan T { 170 if len(d.queue) < d.size { 171 return d.inCh 172 } 173 return nil 174 }