github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/dm/relay/relay_writer.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 "path/filepath" 18 "time" 19 20 gmysql "github.com/go-mysql-org/go-mysql/mysql" 21 "github.com/go-mysql-org/go-mysql/replication" 22 "github.com/pingcap/failpoint" 23 "github.com/pingcap/tiflow/dm/pkg/binlog/event" 24 "github.com/pingcap/tiflow/dm/pkg/log" 25 "github.com/pingcap/tiflow/dm/pkg/terror" 26 "github.com/pingcap/tiflow/dm/pkg/utils" 27 "go.uber.org/atomic" 28 "go.uber.org/zap" 29 ) 30 31 const ( 32 ignoreReasonAlreadyExists = "already exists" 33 ignoreReasonFakeRotate = "fake rotate event" 34 ) 35 36 // WResult represents a write result. 37 type WResult struct { 38 Ignore bool // whether the event ignored by the writer 39 IgnoreReason string // why the writer ignore the event 40 } 41 42 // Writer writes binlog events into disk or any other memory structure. 43 // The writer should support: 44 // 1. write binlog events and report the operation result 45 // 2. skip any obsolete binlog events 46 // 3. generate dummy events to fill the gap if needed 47 // 4. rotate binlog(relay) file if needed 48 // 5. rollback/discard unfinished binlog entries(events or transactions) 49 type Writer interface { 50 // Init inits the writer, should be called before any other method 51 Init(uuid, filename string) 52 // Close closes the writer and release the resource. 53 Close() error 54 55 // WriteEvent writes an binlog event's data into disk or any other places. 56 // It is not safe for concurrent use by multiple goroutines. 57 WriteEvent(ev *replication.BinlogEvent) (WResult, error) 58 // IsActive check whether given uuid+filename is active binlog file, if true return current file offset 59 IsActive(uuid, filename string) (bool, int64) 60 // Flush flushes the binlog writer. 61 Flush() error 62 } 63 64 // FileWriter implements Writer interface. 65 type FileWriter struct { 66 // underlying binlog writer, 67 // it will be created/started until needed. 68 out *BinlogWriter 69 70 relayDir string // base directory of relay files, without UUID part 71 uuid string // with suffix, like 3ccc475b-2343-11e7-be21-6c0b84d59f30.000001 72 filename atomic.String // current binlog filename 73 74 logger log.Logger 75 } 76 77 // NewFileWriter creates a FileWriter instances. 78 func NewFileWriter(logger log.Logger, relayDir string) Writer { 79 w := &FileWriter{ 80 relayDir: relayDir, 81 logger: logger.WithFields(zap.String("sub component", "relay writer")), 82 } 83 w.out = NewBinlogWriter(w.logger, relayDir) 84 return w 85 } 86 87 func (w *FileWriter) Init(uuid, filename string) { 88 w.uuid = uuid 89 w.filename.Store(filename) 90 } 91 92 // Close implements Writer.Close. 93 func (w *FileWriter) Close() error { 94 return w.out.Close() 95 } 96 97 // WriteEvent implements Writer.WriteEvent. 98 func (w *FileWriter) WriteEvent(ev *replication.BinlogEvent) (WResult, error) { 99 switch ev.Event.(type) { 100 case *replication.FormatDescriptionEvent: 101 return w.handleFormatDescriptionEvent(ev) 102 case *replication.RotateEvent: 103 return w.handleRotateEvent(ev) 104 default: 105 return w.handleEventDefault(ev) 106 } 107 } 108 109 // Flush implements Writer.Flush. 110 func (w *FileWriter) Flush() error { 111 return w.out.Flush() 112 } 113 114 // offset returns the current offset of the binlog file. 115 // it is only used for testing now. 116 func (w *FileWriter) offset() int64 { 117 return w.out.Offset() 118 } 119 120 // handle FormatDescriptionEvent: 121 // 1. close the previous binlog file 122 // 2. open/create a new binlog file 123 // 3. write the binlog file header if not exists 124 // 4. write the FormatDescriptionEvent if not exists one 125 func (w *FileWriter) handleFormatDescriptionEvent(ev *replication.BinlogEvent) (WResult, error) { 126 // close the previous binlog file 127 w.logger.Info("closing previous underlying binlog writer", zap.Reflect("status", w.out.Status())) 128 err := w.out.Close() 129 if err != nil { 130 return WResult{}, terror.Annotate(err, "close previous underlying binlog writer") 131 } 132 133 // verify filename 134 if !utils.VerifyFilename(w.filename.Load()) { 135 return WResult{}, terror.ErrRelayBinlogNameNotValid.Generatef("binlog filename %s not valid", w.filename.Load()) 136 } 137 138 // open/create a new binlog file 139 fullName := filepath.Join(w.relayDir, w.uuid, w.filename.Load()) 140 err = w.out.Open(w.uuid, w.filename.Load()) 141 if err != nil { 142 return WResult{}, terror.Annotatef(err, "start underlying binlog writer for %s", fullName) 143 } 144 w.logger.Info("open underlying binlog writer", zap.Reflect("status", w.out.Status())) 145 146 // write the binlog file header if not exists 147 exist, err := checkBinlogHeaderExist(fullName) 148 if err != nil { 149 return WResult{}, terror.Annotatef(err, "check binlog file header for %s", fullName) 150 } else if !exist { 151 err = w.out.Write(replication.BinLogFileHeader) 152 if err != nil { 153 return WResult{}, terror.Annotatef(err, "write binlog file header for %s", fullName) 154 } 155 err = w.Flush() 156 if err != nil { 157 return WResult{}, terror.Annotatef(err, "flush binlog file for %s", fullName) 158 } 159 } 160 161 // write the FormatDescriptionEvent if not exists one 162 exist, err = checkFormatDescriptionEventExist(fullName) 163 if err != nil { 164 return WResult{}, terror.Annotatef(err, "check FormatDescriptionEvent for %s", fullName) 165 } else if !exist { 166 err = w.out.Write(ev.RawData) 167 if err != nil { 168 return WResult{}, terror.Annotatef(err, "write FormatDescriptionEvent %+v for %s", ev.Header, fullName) 169 } 170 } 171 var reason string 172 if exist { 173 reason = ignoreReasonAlreadyExists 174 } 175 176 return WResult{ 177 Ignore: exist, // ignore if exists 178 IgnoreReason: reason, 179 }, nil 180 } 181 182 // handle RotateEvent: 183 // 1. update binlog filename if needed 184 // 2. write the RotateEvent if not fake 185 // 186 // NOTE: we only see fake event for RotateEvent in MySQL source code, 187 // 188 // if see fake event for other event type, then handle them. 189 // 190 // NOTE: we do not create a new binlog file when received a RotateEvent, 191 // 192 // instead, we create a new binlog file when received a FormatDescriptionEvent. 193 // because a binlog file without any events has no meaning. 194 func (w *FileWriter) handleRotateEvent(ev *replication.BinlogEvent) (result WResult, err error) { 195 rotateEv, ok := ev.Event.(*replication.RotateEvent) 196 if !ok { 197 return result, terror.ErrRelayWriterExpectRotateEv.Generate(ev.Header) 198 } 199 200 currFile := w.filename.Load() 201 defer func() { 202 if err == nil { 203 // update binlog filename if needed 204 nextFile := string(rotateEv.NextLogName) 205 if gmysql.CompareBinlogFileName(nextFile, currFile) == 1 { 206 // record the next filename, but not create it. 207 // even it's a fake RotateEvent, we still need to record it, 208 // because if we do not specify the filename when creating the writer (like Auto-Position), 209 // we can only receive a fake RotateEvent before the FormatDescriptionEvent. 210 w.filename.Store(nextFile) 211 } 212 } 213 }() 214 215 // write the RotateEvent if not fake 216 if utils.IsFakeRotateEvent(ev.Header) { 217 // skip fake rotate event 218 return WResult{ 219 Ignore: true, 220 IgnoreReason: ignoreReasonFakeRotate, 221 }, nil 222 } 223 224 result, err = w.handlePotentialHoleOrDuplicate(ev) 225 if err != nil { 226 return result, err 227 } else if result.Ignore { 228 return result, nil 229 } 230 231 err = w.out.Write(ev.RawData) 232 if err != nil { 233 return result, terror.Annotatef(err, "write RotateEvent %+v for %s", ev.Header, filepath.Join(w.relayDir, w.uuid, currFile)) 234 } 235 236 return WResult{ 237 Ignore: false, 238 }, nil 239 } 240 241 // handle non-special event: 242 // 1. handle a potential hole if exists 243 // 2. handle any duplicate events if exist 244 // 3. write the non-duplicate event 245 func (w *FileWriter) handleEventDefault(ev *replication.BinlogEvent) (WResult, error) { 246 result, err := w.handlePotentialHoleOrDuplicate(ev) 247 if err != nil { 248 return WResult{}, err 249 } else if result.Ignore { 250 return result, nil 251 } 252 253 // write the non-duplicate event 254 failpoint.Inject("SlowDownWriteDMLRelayLog", func(_ failpoint.Value) { 255 w.logger.Debug("enter SlowDownWriteDMLRelayLog") 256 switch ev.Header.EventType { 257 case replication.WRITE_ROWS_EVENTv0, replication.WRITE_ROWS_EVENTv1, replication.WRITE_ROWS_EVENTv2, 258 replication.DELETE_ROWS_EVENTv0, replication.DELETE_ROWS_EVENTv1, replication.DELETE_ROWS_EVENTv2, 259 replication.UPDATE_ROWS_EVENTv0, replication.UPDATE_ROWS_EVENTv1, replication.UPDATE_ROWS_EVENTv2: 260 mid := len(ev.RawData) / 2 261 first, second := ev.RawData[:mid], ev.RawData[mid:] 262 err2 := w.out.Write(first) 263 if err2 != nil { 264 w.logger.DPanic("error in failpoint SlowDownWriteDMLRelayLog", zap.Error(err2)) 265 } 266 time.Sleep(time.Second) 267 err = w.out.Write(second) 268 failpoint.Goto("afterWrite") 269 } 270 }) 271 272 err = w.out.Write(ev.RawData) 273 274 failpoint.Label("afterWrite") 275 276 return WResult{ 277 Ignore: false, 278 }, terror.Annotatef(err, "write event %+v", ev.Header) 279 } 280 281 // handlePotentialHoleOrDuplicate combines handleFileHoleExist and handleDuplicateEventsExist. 282 func (w *FileWriter) handlePotentialHoleOrDuplicate(ev *replication.BinlogEvent) (WResult, error) { 283 // handle a potential hole 284 mayDuplicate, err := w.handleFileHoleExist(ev) 285 if err != nil { 286 return WResult{}, terror.Annotatef(err, "handle a potential hole in %s before %+v", 287 w.filename.Load(), ev.Header) 288 } 289 290 if mayDuplicate { 291 if err := w.Flush(); err != nil { 292 return WResult{}, terror.Annotatef(err, "flush before handle duplicate event %v in %s", ev.Header, w.filename.Load()) 293 } 294 // handle any duplicate events if exist 295 result, err2 := w.handleDuplicateEventsExist(ev) 296 if err2 != nil { 297 return WResult{}, terror.Annotatef(err2, "handle a potential duplicate event %+v in %s", 298 ev.Header, w.filename.Load()) 299 } 300 if result.Ignore { 301 // duplicate, and can ignore it. now, we assume duplicate events can all be ignored 302 return result, nil 303 } 304 } 305 306 return WResult{ 307 Ignore: false, 308 }, nil 309 } 310 311 // handleFileHoleExist tries to handle a potential hole after this event wrote. 312 // A hole exists often because some binlog events not sent by the master. 313 // If no hole exists, then ev may be a duplicate event. 314 // NOTE: handle cases when file size > 4GB. 315 func (w *FileWriter) handleFileHoleExist(ev *replication.BinlogEvent) (bool, error) { 316 // 1. detect whether a hole exists 317 evStartPos := int64(ev.Header.LogPos - ev.Header.EventSize) 318 fileOffset := w.out.Offset() 319 holeSize := evStartPos - fileOffset 320 if holeSize <= 0 { 321 // no hole exists, but duplicate events may exists, this should be handled in another place. 322 return holeSize < 0, nil 323 } 324 w.logger.Info("hole exist from pos1 to pos2", zap.Int64("pos1", fileOffset), zap.Int64("pos2", evStartPos), zap.String("file", w.filename.Load())) 325 326 // 2. generate dummy event 327 var ( 328 header = &replication.EventHeader{ 329 Timestamp: uint32(time.Now().Unix()), 330 ServerID: ev.Header.ServerID, 331 } 332 latestPos = uint32(fileOffset) 333 eventSize = uint32(holeSize) 334 ) 335 dummyEv, err := event.GenDummyEvent(header, latestPos, eventSize) 336 if err != nil { 337 return false, terror.Annotatef(err, "generate dummy event at %d with size %d", latestPos, eventSize) 338 } 339 340 // 3. write the dummy event 341 err = w.out.Write(dummyEv.RawData) 342 return false, terror.Annotatef(err, "write dummy event %+v to fill the hole", dummyEv.Header) 343 } 344 345 // handleDuplicateEventsExist tries to handle a potential duplicate event in the binlog file. 346 func (w *FileWriter) handleDuplicateEventsExist(ev *replication.BinlogEvent) (WResult, error) { 347 filename := filepath.Join(w.relayDir, w.uuid, w.filename.Load()) 348 duplicate, err := checkIsDuplicateEvent(filename, ev) 349 if err != nil { 350 return WResult{}, terror.Annotatef(err, "check event %+v whether duplicate in %s", ev.Header, filename) 351 } else if duplicate { 352 w.logger.Info("event is duplicate", zap.Reflect("header", ev.Header), zap.String("file", w.filename.Load())) 353 } 354 355 var reason string 356 if duplicate { 357 reason = ignoreReasonAlreadyExists 358 } 359 360 return WResult{ 361 Ignore: duplicate, 362 IgnoreReason: reason, 363 }, nil 364 } 365 366 func (w *FileWriter) IsActive(uuid, filename string) (bool, int64) { 367 return w.out.isActive(uuid, filename) 368 }