github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/engine/internal/pkg/discovery/agent.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 discovery 15 16 import ( 17 "context" 18 "time" 19 20 "github.com/pingcap/log" 21 "github.com/pingcap/tiflow/engine/pkg/client" 22 "github.com/pingcap/tiflow/engine/pkg/notifier" 23 "github.com/pingcap/tiflow/pkg/errors" 24 "go.uber.org/zap" 25 "golang.org/x/exp/maps" 26 "golang.org/x/sync/errgroup" 27 ) 28 29 // EventType describes the type of the event, i.e. Add or Del. 30 type EventType string 31 32 const ( 33 // EventTypeAdd indicates that a node has been added. 34 EventTypeAdd = EventType("add") 35 36 // EventTypeDel indicates that a node has been removed. 37 EventTypeDel = EventType("del") 38 ) 39 40 // NodeType describes the type of the node. 41 type NodeType string 42 43 const ( 44 // NodeTypeMaster indicates that the node is a master. 45 NodeTypeMaster = NodeType("master") 46 // NodeTypeExecutor indicates that the node is an executor. 47 NodeTypeExecutor = NodeType("executor") 48 ) 49 50 // Event is a node membership change event. 51 type Event struct { 52 Tp EventType 53 Node Node 54 } 55 56 // Node describes the information of a node. 57 type Node struct { 58 Tp NodeType 59 ID string 60 Addr string 61 } 62 63 // Snapshot is the current service membership snapshot. 64 type Snapshot map[string]Node 65 66 // Agent registers receives membership changes of all nodes. 67 type Agent interface { 68 Run(ctx context.Context) error 69 Subscribe(ctx context.Context) (Snapshot, *notifier.Receiver[Event], error) 70 } 71 72 type agentImpl struct { 73 discoveryClient client.DiscoveryClient 74 autoSyncInterval time.Duration 75 eventCh chan Event 76 subscribeCh chan *subscribeReq 77 } 78 79 type subscribeReq struct { 80 snap Snapshot 81 receiver *notifier.Receiver[Event] 82 doneCh chan struct{} 83 } 84 85 // NewAgent creates a new Agent that receives membership changes from the discovery service. 86 // autoSyncInterval is the interval to update membership with the latest information and notify subscribers. 87 func NewAgent( 88 discoveryClient client.DiscoveryClient, 89 autoSyncInterval time.Duration, 90 ) Agent { 91 return &agentImpl{ 92 discoveryClient: discoveryClient, 93 eventCh: make(chan Event, 16), 94 subscribeCh: make(chan *subscribeReq, 1), 95 autoSyncInterval: autoSyncInterval, 96 } 97 } 98 99 // Run runs the internal logic of an Agent. It blocks 100 // until an error has occurred, or it has been canceled. 101 func (a *agentImpl) Run(ctx context.Context) error { 102 eventNotifier := notifier.NewNotifier[Event]() 103 defer eventNotifier.Close() 104 105 g, gCtx := errgroup.WithContext(ctx) 106 107 g.Go(func() error { 108 return a.autoSync(gCtx) 109 }) 110 g.Go(func() error { 111 snap := make(Snapshot) 112 for { 113 select { 114 case <-gCtx.Done(): 115 return errors.Trace(gCtx.Err()) 116 case event := <-a.eventCh: 117 eventNotifier.Notify(event) 118 applyEvent(snap, event) 119 case req := <-a.subscribeCh: 120 req.snap = maps.Clone(snap) 121 if err := eventNotifier.Flush(gCtx); err != nil { 122 return errors.Trace(err) 123 } 124 req.receiver = eventNotifier.NewReceiver() 125 close(req.doneCh) 126 } 127 } 128 }) 129 return g.Wait() 130 } 131 132 func (a *agentImpl) autoSync(ctx context.Context) error { 133 snap := make(Snapshot) 134 for { 135 newSnap, err := a.getSnapshot(ctx) 136 if err != nil { 137 if errors.Cause(err) == context.Canceled || errors.Cause(err) == context.DeadlineExceeded { 138 return errors.Trace(err) 139 } 140 log.Warn("failed to get snapshot", zap.Error(err)) 141 } else { 142 events := computeEvents(snap, newSnap) 143 for _, event := range events { 144 select { 145 case a.eventCh <- event: 146 case <-ctx.Done(): 147 return errors.Trace(ctx.Err()) 148 } 149 } 150 snap = newSnap 151 } 152 153 select { 154 case <-ctx.Done(): 155 return errors.Trace(ctx.Err()) 156 case <-time.After(a.autoSyncInterval): 157 } 158 } 159 } 160 161 func (a *agentImpl) getSnapshot(ctx context.Context) (Snapshot, error) { 162 masters, err := a.discoveryClient.ListMasters(ctx) 163 if err != nil { 164 return nil, errors.Trace(err) 165 } 166 executors, err := a.discoveryClient.ListExecutors(ctx) 167 if err != nil { 168 return nil, errors.Trace(err) 169 } 170 snap := make(Snapshot) 171 for _, m := range masters { 172 snap[m.Id] = Node{ 173 Tp: NodeTypeMaster, 174 ID: m.Id, 175 Addr: m.Address, 176 } 177 } 178 for _, e := range executors { 179 snap[e.Id] = Node{ 180 Tp: NodeTypeExecutor, 181 ID: e.Id, 182 Addr: e.Address, 183 } 184 } 185 return snap, nil 186 } 187 188 const ( 189 subscribeTimeout = 10 * time.Second 190 ) 191 192 // Subscribe subscribes to the service membership changes. 193 func (a *agentImpl) Subscribe(ctx context.Context) (Snapshot, *notifier.Receiver[Event], error) { 194 ctx, cancel := context.WithTimeout(ctx, subscribeTimeout) 195 defer cancel() 196 197 req := &subscribeReq{ 198 doneCh: make(chan struct{}), 199 } 200 select { 201 case <-ctx.Done(): 202 return nil, nil, errors.Trace(ctx.Err()) 203 case a.subscribeCh <- req: 204 } 205 206 select { 207 case <-ctx.Done(): 208 return nil, nil, errors.Trace(ctx.Err()) 209 case <-req.doneCh: 210 } 211 212 return req.snap, req.receiver, nil 213 } 214 215 func applyEvent(snap Snapshot, event Event) { 216 switch event.Tp { 217 case EventTypeAdd: 218 snap[event.Node.ID] = event.Node 219 case EventTypeDel: 220 delete(snap, event.Node.ID) 221 } 222 } 223 224 func computeEvents(old, new Snapshot) []Event { 225 var events []Event 226 for id, node := range new { 227 if _, ok := old[id]; !ok { 228 events = append(events, Event{ 229 Tp: EventTypeAdd, 230 Node: node, 231 }) 232 } 233 } 234 for id := range old { 235 if _, ok := new[id]; !ok { 236 events = append(events, Event{ 237 Tp: EventTypeDel, 238 Node: old[id], 239 }) 240 } 241 } 242 return events 243 }