github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/engine/pkg/notifier/notifier.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 notifier 15 16 import ( 17 "context" 18 "sync" 19 "time" 20 21 "github.com/pingcap/tiflow/engine/pkg/containers" 22 "github.com/pingcap/tiflow/pkg/errors" 23 "go.uber.org/atomic" 24 ) 25 26 type receiverID = int64 27 28 // Notifier is the sending endpoint of an event 29 // notification mechanism. It broadcasts a stream of 30 // events to a number of receivers. 31 type Notifier[T any] struct { 32 receivers sync.Map // receiverID -> *Receiver[T] 33 nextID atomic.Int64 34 35 // queue is unbounded. 36 queue *containers.SliceQueue[T] 37 38 closed atomic.Bool 39 closeCh chan struct{} 40 synchronizeCh chan struct{} 41 42 wg sync.WaitGroup 43 } 44 45 // Receiver is the receiving endpoint of a single-producer-multiple-consumer 46 // notification mechanism. 47 type Receiver[T any] struct { 48 // C is a channel to read the events from. 49 // Note that it is part of the public interface of this package. 50 C chan T 51 52 id receiverID 53 54 closeOnce sync.Once 55 56 // closed MUST be closed before closing `C`. 57 closed chan struct{} 58 59 notifier *Notifier[T] 60 } 61 62 // Close closes the receiver 63 func (r *Receiver[T]) Close() { 64 r.closeOnce.Do( 65 func() { 66 close(r.closed) 67 // Waits for the synchronization barrier, which 68 // means that run() has finished the last iteration, 69 // and since we have set `closed` to true, the `C` channel, 70 // will not be written to anymore. So it is safe to close it now. 71 <-r.notifier.synchronizeCh 72 close(r.C) 73 r.notifier.receivers.Delete(r.id) 74 }) 75 } 76 77 // NewNotifier creates a new Notifier. 78 func NewNotifier[T any]() *Notifier[T] { 79 ret := &Notifier[T]{ 80 receivers: sync.Map{}, 81 queue: containers.NewSliceQueue[T](), 82 closeCh: make(chan struct{}), 83 synchronizeCh: make(chan struct{}), 84 } 85 86 ret.wg.Add(1) 87 go func() { 88 defer ret.wg.Done() 89 ret.run() 90 }() 91 return ret 92 } 93 94 // NewReceiver creates a new Receiver associated with 95 // the given Notifier. 96 func (n *Notifier[T]) NewReceiver() *Receiver[T] { 97 ch := make(chan T, 16) 98 receiver := &Receiver[T]{ 99 id: n.nextID.Add(1), 100 C: ch, 101 closed: make(chan struct{}), 102 notifier: n, 103 } 104 105 n.receivers.Store(receiver.id, receiver) 106 return receiver 107 } 108 109 // Notify sends a new notification event. 110 func (n *Notifier[T]) Notify(event T) { 111 n.queue.Push(event) 112 } 113 114 // Close closes the notifier. 115 func (n *Notifier[T]) Close() { 116 if n.closed.Swap(true) { 117 // Ensures idempotency of closing once. 118 return 119 } 120 121 close(n.closeCh) 122 n.wg.Wait() 123 124 n.receivers.Range(func(_, value any) bool { 125 receiver := value.(*Receiver[T]) 126 receiver.Close() 127 return true 128 }) 129 } 130 131 // Flush flushes all pending notifications. 132 // Note that for Flush to work as expected, a 133 // quiescent period is required, i.e. you should 134 // not send more events until Flush returns. 135 func (n *Notifier[T]) Flush(ctx context.Context) error { 136 for { 137 select { 138 case <-ctx.Done(): 139 return errors.Trace(ctx.Err()) 140 case <-n.closeCh: 141 return nil 142 case <-n.synchronizeCh: 143 // Checks the queue size after each iteration 144 // of run(). 145 } 146 147 if n.queue.Size() == 0 { 148 return nil 149 } 150 } 151 } 152 153 func (n *Notifier[T]) run() { 154 ticker := time.NewTicker(1 * time.Second) 155 defer ticker.Stop() 156 157 defer func() { 158 close(n.synchronizeCh) 159 }() 160 161 for { 162 select { 163 case <-n.closeCh: 164 return 165 case n.synchronizeCh <- struct{}{}: 166 // no-op here. Just a synchronization barrier. 167 case <-n.queue.C: 168 Inner: 169 for { 170 event, ok := n.queue.Pop() 171 if !ok { 172 break Inner 173 } 174 175 // TODO In the current implementation, congestion 176 // in once receiver will prevent all other receivers 177 // from receiving events. 178 n.receivers.Range(func(_, value any) bool { 179 receiver := value.(*Receiver[T]) 180 181 select { 182 case <-n.closeCh: 183 return false 184 case <-receiver.closed: 185 // Receiver has been closed. 186 case receiver.C <- event: 187 // send the event to the receiver. 188 } 189 return true 190 }) 191 192 select { 193 case <-n.closeCh: 194 return 195 default: 196 } 197 } 198 } 199 } 200 }