github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/cdc/controller/controller.go (about) 1 // Copyright 2023 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 controller 15 16 import ( 17 "context" 18 "sync" 19 "sync/atomic" 20 "time" 21 22 "github.com/pingcap/errors" 23 "github.com/pingcap/log" 24 "github.com/pingcap/tiflow/cdc/model" 25 cerror "github.com/pingcap/tiflow/pkg/errors" 26 "github.com/pingcap/tiflow/pkg/etcd" 27 "github.com/pingcap/tiflow/pkg/orchestrator" 28 "github.com/pingcap/tiflow/pkg/upstream" 29 "github.com/pingcap/tiflow/pkg/version" 30 "github.com/tikv/client-go/v2/oracle" 31 "go.uber.org/zap" 32 "golang.org/x/time/rate" 33 ) 34 35 type controllerJobType int 36 37 // All ControllerJob types 38 const ( 39 controllerJobTypeQuery controllerJobType = iota 40 ) 41 42 // versionInconsistentLogRate represents the rate of log output when there are 43 // captures with versions different from that of the controller 44 const versionInconsistentLogRate = 1 45 46 // Controller is a manager to schedule changefeeds 47 type Controller interface { 48 AsyncStop() 49 GetChangefeedOwnerCaptureInfo(id model.ChangeFeedID) *model.CaptureInfo 50 GetAllChangeFeedInfo(ctx context.Context) ( 51 map[model.ChangeFeedID]*model.ChangeFeedInfo, error, 52 ) 53 GetAllChangeFeedCheckpointTs(ctx context.Context) ( 54 map[model.ChangeFeedID]uint64, error, 55 ) 56 GetCaptures(ctx context.Context) ([]*model.CaptureInfo, error) 57 GetProcessors(ctx context.Context) ([]*model.ProcInfoSnap, error) 58 IsChangefeedExists(ctx context.Context, id model.ChangeFeedID) (bool, error) 59 CreateChangefeed(context.Context, 60 *model.UpstreamInfo, 61 *model.ChangeFeedInfo, 62 ) error 63 } 64 65 var ( 66 _ orchestrator.Reactor = &controllerImpl{} 67 _ Controller = &controllerImpl{} 68 ) 69 70 type controllerImpl struct { 71 changefeeds map[model.ChangeFeedID]*orchestrator.ChangefeedReactorState 72 captures map[model.CaptureID]*model.CaptureInfo 73 upstreamManager *upstream.Manager 74 75 // logLimiter controls cluster version check log output rate 76 logLimiter *rate.Limiter 77 lastTickTime time.Time 78 // bootstrapped specifies whether the controller has been initialized. 79 // This will only be done when the controller starts the first Tick. 80 // NOTICE: Do not use it in a method other than tick unexpectedly, 81 // as it is not a thread-safe value. 82 bootstrapped bool 83 84 closed int32 85 86 controllerJobQueue struct { 87 sync.Mutex 88 queue []*controllerJob 89 } 90 etcdClient etcd.CDCEtcdClient 91 92 captureInfo *model.CaptureInfo 93 } 94 95 // NewController creates a new Controller 96 func NewController( 97 upstreamManager *upstream.Manager, 98 captureInfo *model.CaptureInfo, 99 etcdClient etcd.CDCEtcdClient, 100 ) Controller { 101 return &controllerImpl{ 102 upstreamManager: upstreamManager, 103 changefeeds: make(map[model.ChangeFeedID]*orchestrator.ChangefeedReactorState), 104 lastTickTime: time.Now(), 105 logLimiter: rate.NewLimiter(versionInconsistentLogRate, versionInconsistentLogRate), 106 captureInfo: captureInfo, 107 etcdClient: etcdClient, 108 } 109 } 110 111 // Tick implements the Reactor interface 112 func (o *controllerImpl) Tick(stdCtx context.Context, rawState orchestrator.ReactorState) (nextState orchestrator.ReactorState, err error) { 113 state := rawState.(*orchestrator.GlobalReactorState) 114 o.captures = state.Captures 115 116 // At the first Tick, we need to do a bootstrap operation. 117 // Fix incompatible or incorrect meta information. 118 if !o.bootstrapped { 119 o.Bootstrap(state) 120 o.bootstrapped = true 121 return state, nil 122 } 123 // handleJobs() should be called before clusterVersionConsistent(), because 124 // when there are different versions of cdc nodes in the cluster, 125 // the admin job may not be processed all the time. And http api relies on 126 // admin job, which will cause all http api unavailable. 127 o.handleJobs(stdCtx) 128 129 if !o.clusterVersionConsistent(state.Captures) { 130 return state, nil 131 } 132 // controller should update GC safepoint before initializing changefeed, so 133 // changefeed can remove its "ticdc-creating" service GC safepoint during 134 // initializing. 135 // 136 // See more gc doc. 137 if err = o.updateGCSafepoint(stdCtx, state); err != nil { 138 return nil, errors.Trace(err) 139 } 140 141 // Tick all changefeeds. 142 for _, changefeed := range state.Changefeeds { 143 o.changefeeds[changefeed.ID] = changefeed 144 } 145 146 // Cleanup changefeeds that are not in the state. 147 if len(o.changefeeds) != len(state.Changefeeds) { 148 for changefeedID := range o.changefeeds { 149 if _, exist := state.Changefeeds[changefeedID]; exist { 150 continue 151 } 152 delete(o.changefeeds, changefeedID) 153 } 154 } 155 156 // if closed, exit the etcd worker loop 157 if atomic.LoadInt32(&o.closed) != 0 { 158 return state, cerror.ErrReactorFinished.GenWithStackByArgs() 159 } 160 161 return state, nil 162 } 163 164 // Bootstrap checks if the state contains incompatible or incorrect information and tries to fix it. 165 func (o *controllerImpl) Bootstrap(state *orchestrator.GlobalReactorState) { 166 log.Info("Start bootstrapping") 167 fixChangefeedInfos(state) 168 } 169 170 // fixChangefeedInfos attempts to fix incompatible or incorrect meta information in changefeed state. 171 func fixChangefeedInfos(state *orchestrator.GlobalReactorState) { 172 for _, changefeedState := range state.Changefeeds { 173 if changefeedState != nil { 174 changefeedState.PatchInfo(func(info *model.ChangeFeedInfo) (*model.ChangeFeedInfo, bool, error) { 175 if info == nil { 176 return nil, false, nil 177 } 178 info.FixIncompatible() 179 return info, true, nil 180 }) 181 } 182 } 183 } 184 185 func (o *controllerImpl) clusterVersionConsistent(captures map[model.CaptureID]*model.CaptureInfo) bool { 186 versions := make(map[string]struct{}, len(captures)) 187 for _, capture := range captures { 188 versions[capture.Version] = struct{}{} 189 } 190 191 if err := version.CheckTiCDCVersion(versions); err != nil { 192 if o.logLimiter.Allow() { 193 log.Warn("TiCDC cluster versions not allowed", 194 zap.String("controllerVersion", version.ReleaseVersion), 195 zap.Any("captures", captures), zap.Error(err)) 196 } 197 return false 198 } 199 return true 200 } 201 202 func (o *controllerImpl) updateGCSafepoint( 203 ctx context.Context, state *orchestrator.GlobalReactorState, 204 ) error { 205 minChekpoinTsMap, forceUpdateMap := o.calculateGCSafepoint(state) 206 207 for upstreamID, minCheckpointTs := range minChekpoinTsMap { 208 up, ok := o.upstreamManager.Get(upstreamID) 209 if !ok { 210 upstreamInfo := state.Upstreams[upstreamID] 211 up = o.upstreamManager.AddUpstream(upstreamInfo) 212 } 213 if !up.IsNormal() { 214 log.Warn("upstream is not ready, skip", 215 zap.Uint64("id", up.ID), 216 zap.Strings("pd", up.PdEndpoints)) 217 continue 218 } 219 220 // When the changefeed starts up, CDC will do a snapshot read at 221 // (checkpointTs - 1) from TiKV, so (checkpointTs - 1) should be an upper 222 // bound for the GC safepoint. 223 gcSafepointUpperBound := minCheckpointTs - 1 224 225 var forceUpdate bool 226 if _, exist := forceUpdateMap[upstreamID]; exist { 227 forceUpdate = true 228 } 229 230 err := up.GCManager.TryUpdateGCSafePoint(ctx, gcSafepointUpperBound, forceUpdate) 231 if err != nil { 232 return errors.Trace(err) 233 } 234 } 235 return nil 236 } 237 238 // calculateGCSafepoint calculates GCSafepoint for different upstream. 239 // Note: we need to maintain a TiCDC service GC safepoint for each upstream TiDB cluster 240 // to prevent upstream TiDB GC from removing data that is still needed by TiCDC. 241 // GcSafepoint is the minimum checkpointTs of all changefeeds that replicating a same upstream TiDB cluster. 242 func (o *controllerImpl) calculateGCSafepoint(state *orchestrator.GlobalReactorState) ( 243 map[uint64]uint64, map[uint64]interface{}, 244 ) { 245 minCheckpointTsMap := make(map[uint64]uint64) 246 forceUpdateMap := make(map[uint64]interface{}) 247 248 for changefeedID, changefeedState := range state.Changefeeds { 249 if changefeedState.Info == nil || !changefeedState.Info.NeedBlockGC() { 250 continue 251 } 252 253 checkpointTs := changefeedState.Info.GetCheckpointTs(changefeedState.Status) 254 upstreamID := changefeedState.Info.UpstreamID 255 256 if _, exist := minCheckpointTsMap[upstreamID]; !exist { 257 minCheckpointTsMap[upstreamID] = checkpointTs 258 } 259 260 minCpts := minCheckpointTsMap[upstreamID] 261 262 if minCpts > checkpointTs { 263 minCpts = checkpointTs 264 minCheckpointTsMap[upstreamID] = minCpts 265 } 266 // Force update when adding a new changefeed. 267 _, exist := o.changefeeds[changefeedID] 268 if !exist { 269 forceUpdateMap[upstreamID] = nil 270 } 271 } 272 273 // check if the upstream has a changefeed, if not we should update the gc safepoint 274 _ = o.upstreamManager.Visit(func(up *upstream.Upstream) error { 275 if _, exist := minCheckpointTsMap[up.ID]; !exist { 276 ts := up.PDClock.CurrentTime() 277 minCheckpointTsMap[up.ID] = oracle.GoTimeToTS(ts) 278 } 279 return nil 280 }) 281 return minCheckpointTsMap, forceUpdateMap 282 } 283 284 // AsyncStop stops the server manager asynchronously 285 func (o *controllerImpl) AsyncStop() { 286 atomic.StoreInt32(&o.closed, 1) 287 } 288 289 // GetChangefeedOwnerCaptureInfo returns the capture info of the owner of the changefeed 290 func (o *controllerImpl) GetChangefeedOwnerCaptureInfo(id model.ChangeFeedID) *model.CaptureInfo { 291 // todo: schedule changefeed owner to other capture 292 return o.captureInfo 293 } 294 295 func (o *controllerImpl) CreateChangefeed(ctx context.Context, 296 upstreamInfo *model.UpstreamInfo, 297 cfInfo *model.ChangeFeedInfo, 298 ) error { 299 return o.etcdClient.CreateChangefeedInfo(ctx, upstreamInfo, cfInfo) 300 } 301 302 // Export field names for pretty printing. 303 type controllerJob struct { 304 Tp controllerJobType 305 ChangefeedID model.ChangeFeedID 306 307 // for status provider 308 query *Query 309 310 done chan<- error 311 }