github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/dm/relay/purger.go (about) 1 // Copyright 2019 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 relay 15 16 import ( 17 "context" 18 "path/filepath" 19 "sync" 20 "time" 21 22 "github.com/pingcap/tiflow/dm/config" 23 "github.com/pingcap/tiflow/dm/pb" 24 "github.com/pingcap/tiflow/dm/pkg/log" 25 "github.com/pingcap/tiflow/dm/pkg/streamer" 26 "github.com/pingcap/tiflow/dm/pkg/terror" 27 "github.com/pingcap/tiflow/dm/pkg/utils" 28 "go.uber.org/atomic" 29 "go.uber.org/zap" 30 ) 31 32 // Operator represents an operator for relay log files, like writer, reader. 33 type Operator interface { 34 // EarliestActiveRelayLog returns the earliest active relay log info in this operator 35 EarliestActiveRelayLog() *streamer.RelayLogInfo 36 } 37 38 // PurgeInterceptor represents an interceptor may forbid the purge process. 39 type PurgeInterceptor interface { 40 // ForbidPurge returns whether forbidding purge currently and an optional message 41 ForbidPurge() (bool, string) 42 } 43 44 const ( 45 stageNew int32 = iota 46 stageRunning 47 stageClosed 48 ) 49 50 // Purger purges relay log according to some strategies. 51 type Purger interface { 52 // Start starts strategies by config 53 Start() 54 // Close stops the started strategies 55 Close() 56 // Purging returns whether the purger is purging 57 Purging() bool 58 // Do does the purge process one time 59 Do(ctx context.Context, req *pb.PurgeRelayRequest) error 60 } 61 62 // NewPurger creates a new purger. 63 var NewPurger = NewRelayPurger 64 65 // relayPurger purges relay log according to some strategies. 66 type relayPurger struct { 67 lock sync.RWMutex 68 wg sync.WaitGroup 69 cancel context.CancelFunc 70 running atomic.Int32 71 purgingStrategy atomic.Uint32 72 73 cfg config.PurgeConfig 74 baseRelayDir string 75 indexPath string // server-uuid.index file path 76 operators []Operator 77 interceptors []PurgeInterceptor 78 strategies map[strategyType]PurgeStrategy 79 80 logger log.Logger 81 } 82 83 // NewRelayPurger creates a new purger. 84 func NewRelayPurger(cfg config.PurgeConfig, baseRelayDir string, operators []Operator, interceptors []PurgeInterceptor) Purger { 85 p := &relayPurger{ 86 cfg: cfg, 87 baseRelayDir: baseRelayDir, 88 indexPath: filepath.Join(baseRelayDir, utils.UUIDIndexFilename), 89 operators: operators, 90 interceptors: interceptors, 91 strategies: make(map[strategyType]PurgeStrategy), 92 logger: log.With(zap.String("component", "relay purger")), 93 } 94 95 // add strategies 96 p.strategies[strategyInactive] = newInactiveStrategy() 97 p.strategies[strategyFilename] = newFilenameStrategy() 98 p.strategies[strategyTime] = newTimeStrategy() 99 p.strategies[strategySpace] = newSpaceStrategy() 100 101 return p 102 } 103 104 // Start starts strategies by config. 105 func (p *relayPurger) Start() { 106 if !p.running.CAS(stageNew, stageRunning) { 107 return 108 } 109 110 if p.cfg.Interval <= 0 || (p.cfg.Expires <= 0 && p.cfg.RemainSpace <= 0) { 111 return // no need do purge in the background 112 } 113 114 p.logger.Info("starting relay log purger", zap.Reflect("config", p.cfg)) 115 116 // Close will wait process to return 117 p.wg.Add(1) 118 go func() { 119 defer p.wg.Done() 120 p.run() 121 }() 122 } 123 124 // run starts running the process 125 // NOTE: ensure run is called at most once of a Purger. 126 func (p *relayPurger) run() { 127 ticker := time.NewTicker(time.Duration(p.cfg.Interval) * time.Second) 128 defer ticker.Stop() 129 130 var ctx context.Context 131 p.lock.Lock() 132 ctx, p.cancel = context.WithCancel(context.Background()) // run until cancel in `Close`. 133 p.lock.Unlock() 134 for { 135 select { 136 case <-ctx.Done(): 137 return 138 case <-ticker.C: 139 p.tryPurge() 140 } 141 } 142 } 143 144 // Close stops the started strategies. 145 func (p *relayPurger) Close() { 146 if !p.running.CAS(stageRunning, stageClosed) { 147 return 148 } 149 150 p.logger.Info("closing relay log purger") 151 152 p.lock.RLock() 153 if p.cancel != nil { 154 p.cancel() 155 } 156 p.lock.RUnlock() 157 p.wg.Wait() 158 } 159 160 // Purging returns whether the purger is purging. 161 func (p *relayPurger) Purging() bool { 162 return p.purgingStrategy.Load() != uint32(strategyNone) 163 } 164 165 // Do does the purge process one time. 166 func (p *relayPurger) Do(ctx context.Context, req *pb.PurgeRelayRequest) error { 167 uuids, err := utils.ParseUUIDIndex(p.indexPath) 168 if err != nil { 169 return terror.Annotatef(err, "parse UUID index file %s", p.indexPath) 170 } 171 172 switch { 173 case req.Inactive: 174 ps := p.strategies[strategyInactive] 175 args := &inactiveArgs{ 176 relayBaseDir: p.baseRelayDir, 177 uuids: uuids, 178 } 179 return p.doPurge(ps, args) 180 case req.Time > 0: 181 ps := p.strategies[strategyTime] 182 args := &timeArgs{ 183 relayBaseDir: p.baseRelayDir, 184 safeTime: time.Unix(req.Time, 0), 185 uuids: uuids, 186 } 187 return p.doPurge(ps, args) 188 case len(req.Filename) > 0: 189 ps := p.strategies[strategyFilename] 190 args := &filenameArgs{ 191 relayBaseDir: p.baseRelayDir, 192 filename: req.Filename, 193 subDir: req.SubDir, 194 uuids: uuids, 195 } 196 return p.doPurge(ps, args) 197 default: 198 return terror.ErrRelayPurgeRequestNotValid.Generate(req) 199 } 200 } 201 202 // tryPurge tries to do purge by check condition first. 203 func (p *relayPurger) tryPurge() { 204 strategy, args, err := p.check() 205 if err != nil { 206 p.logger.Error("check whether need to purge relay log files in background", zap.Error(err)) 207 return 208 } 209 if strategy == nil { 210 return 211 } 212 err = p.doPurge(strategy, args) 213 if err != nil { 214 p.logger.Error("do purge", zap.Stringer("strategy", strategy.Type()), zap.Error(err)) 215 } 216 } 217 218 // doPurge does the purging operation. 219 func (p *relayPurger) doPurge(ps PurgeStrategy, args StrategyArgs) error { 220 if !p.purgingStrategy.CAS(uint32(strategyNone), uint32(ps.Type())) { 221 return terror.ErrRelayOtherStrategyIsPurging.Generate(ps.Type()) 222 } 223 defer p.purgingStrategy.Store(uint32(strategyNone)) 224 225 for _, inter := range p.interceptors { 226 forbidden, msg := inter.ForbidPurge() 227 if forbidden { 228 return terror.ErrRelayPurgeIsForbidden.Generate(msg) 229 } 230 } 231 232 // set ActiveRelayLog lazily to make it can be protected by purgingStrategy 233 earliest := p.earliestActiveRelayLog() 234 if earliest == nil { 235 return terror.ErrRelayNoActiveRelayLog.Generate() 236 } 237 args.SetActiveRelayLog(earliest) 238 239 p.logger.Info("start purging relay log files", zap.Stringer("type", ps.Type()), zap.Any("args", args)) 240 return ps.Do(args) 241 } 242 243 func (p *relayPurger) check() (PurgeStrategy, StrategyArgs, error) { 244 p.logger.Info("checking whether needing to purge relay log files") 245 246 uuids, err := utils.ParseUUIDIndex(p.indexPath) 247 if err != nil { 248 return nil, nil, terror.Annotatef(err, "parse UUID index file %s", p.indexPath) 249 } 250 251 // NOTE: no priority supported yet 252 // 1. strategyInactive only used by dmctl manually 253 // 2. strategyFilename only used by dmctl manually 254 255 // 3. strategySpace should be started if set RemainSpace 256 if p.cfg.RemainSpace > 0 { 257 args := &spaceArgs{ 258 relayBaseDir: p.baseRelayDir, 259 remainSpace: p.cfg.RemainSpace, 260 uuids: uuids, 261 } 262 ps := p.strategies[strategySpace] 263 need, err := ps.Check(args) 264 if err != nil { 265 return nil, nil, terror.Annotatef(err, "check with %s with args %+v", ps.Type(), args) 266 } 267 if need { 268 return ps, args, nil 269 } 270 } 271 272 // 4. strategyTime should be started if set Expires 273 if p.cfg.Expires > 0 { 274 safeTime := time.Now().Add(time.Duration(-p.cfg.Expires) * time.Hour) 275 args := &timeArgs{ 276 relayBaseDir: p.baseRelayDir, 277 safeTime: safeTime, 278 uuids: uuids, 279 } 280 ps := p.strategies[strategyTime] 281 need, err := ps.Check(args) 282 if err != nil { 283 return nil, nil, terror.Annotatef(err, "check with %s with args %+v", ps.Type(), args) 284 } 285 if need { 286 return ps, args, nil 287 } 288 } 289 290 return nil, nil, nil 291 } 292 293 // earliestActiveRelayLog returns the current earliest active relay log info. 294 func (p *relayPurger) earliestActiveRelayLog() *streamer.RelayLogInfo { 295 var earliest *streamer.RelayLogInfo 296 for _, op := range p.operators { 297 info := op.EarliestActiveRelayLog() 298 if info == nil { 299 continue 300 } else if earliest == nil || info.Earlier(earliest) { 301 earliest = info 302 } 303 } 304 return earliest 305 } 306 307 /************ dummy purger *************.*/ 308 type dummyPurger struct{} 309 310 // NewDummyPurger returns a dummy purger. 311 func NewDummyPurger(cfg config.PurgeConfig, baseRelayDir string, operators []Operator, interceptors []PurgeInterceptor) Purger { 312 return &dummyPurger{} 313 } 314 315 // Start implements interface of Purger. 316 func (d *dummyPurger) Start() {} 317 318 // Close implements interface of Purger. 319 func (d *dummyPurger) Close() {} 320 321 // Purging implements interface of Purger. 322 func (d *dummyPurger) Purging() bool { 323 return false 324 } 325 326 // Do implements interface of Purger. 327 func (d *dummyPurger) Do(ctx context.Context, req *pb.PurgeRelayRequest) error { 328 return nil 329 }