github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/cdc/redo/writer/file/file.go (about) 1 // Copyright 2023 PingCAP, Inc. 2 // Copyright 2015 CoreOS, Inc. 3 // 4 // Licensed under the Apache License, Version 2.0 (the "License"); 5 // you may not use this file except in compliance with the License. 6 // You may obtain a copy of the License at 7 // 8 // http://www.apache.org/licenses/LICENSE-2.0 9 // 10 // Unless required by applicable law or agreed to in writing, software 11 // distributed under the License is distributed on an "AS IS" BASIS, 12 // See the License for the specific language governing permissions and 13 // limitations under the License. 14 15 package file 16 17 import ( 18 "context" 19 "encoding/binary" 20 "fmt" 21 "io" 22 "os" 23 "path/filepath" 24 "sync" 25 "time" 26 27 "github.com/pingcap/tidb/br/pkg/storage" 28 "github.com/pingcap/tiflow/cdc/model" 29 "github.com/pingcap/tiflow/cdc/redo/common" 30 "github.com/pingcap/tiflow/cdc/redo/writer" 31 "github.com/pingcap/tiflow/pkg/errors" 32 "github.com/pingcap/tiflow/pkg/fsutil" 33 "github.com/pingcap/tiflow/pkg/redo" 34 "github.com/pingcap/tiflow/pkg/uuid" 35 "github.com/prometheus/client_golang/prometheus" 36 "github.com/uber-go/atomic" 37 pioutil "go.etcd.io/etcd/pkg/v3/ioutil" 38 ) 39 40 //go:generate mockery --name=fileWriter --inpackage --quiet 41 type fileWriter interface { 42 io.WriteCloser 43 Flush() error 44 // AdvanceTs receive the commitTs in the event from caller 45 AdvanceTs(commitTs uint64) 46 // IsRunning check the fileWriter status 47 IsRunning() bool 48 } 49 50 // Writer is a redo log event Writer which writes redo log events to a file. 51 type Writer struct { 52 cfg *writer.LogWriterConfig 53 op *writer.LogWriterOptions 54 // maxCommitTS is the max commitTS among the events in one log file 55 maxCommitTS atomic.Uint64 56 // the ts used in file name 57 commitTS atomic.Uint64 58 // the ts send with the event 59 eventCommitTS atomic.Uint64 60 running atomic.Bool 61 size int64 62 file *os.File 63 // record the filepath that is being written, and has not been flushed 64 ongoingFilePath string 65 bw *pioutil.PageWriter 66 uint64buf []byte 67 storage storage.ExternalStorage 68 sync.RWMutex 69 uuidGenerator uuid.Generator 70 allocator *fsutil.FileAllocator 71 72 metricFsyncDuration prometheus.Observer 73 metricFlushAllDuration prometheus.Observer 74 metricWriteBytes prometheus.Gauge 75 } 76 77 // NewFileWriter return a file rotated writer, TODO: extract to a common rotate Writer 78 func NewFileWriter( 79 ctx context.Context, cfg *writer.LogWriterConfig, opts ...writer.Option, 80 ) (*Writer, error) { 81 if cfg == nil { 82 err := errors.New("FileWriterConfig can not be nil") 83 return nil, errors.WrapError(errors.ErrRedoConfigInvalid, err) 84 } 85 86 var extStorage storage.ExternalStorage 87 if cfg.UseExternalStorage { 88 var err error 89 extStorage, err = redo.InitExternalStorage(ctx, *cfg.URI) 90 if err != nil { 91 return nil, err 92 } 93 } 94 95 op := &writer.LogWriterOptions{} 96 for _, opt := range opts { 97 opt(op) 98 } 99 100 w := &Writer{ 101 cfg: cfg, 102 op: op, 103 uint64buf: make([]byte, 8), 104 storage: extStorage, 105 106 metricFsyncDuration: common.RedoFsyncDurationHistogram. 107 WithLabelValues(cfg.ChangeFeedID.Namespace, cfg.ChangeFeedID.ID), 108 metricFlushAllDuration: common.RedoFlushAllDurationHistogram. 109 WithLabelValues(cfg.ChangeFeedID.Namespace, cfg.ChangeFeedID.ID), 110 metricWriteBytes: common.RedoWriteBytesGauge. 111 WithLabelValues(cfg.ChangeFeedID.Namespace, cfg.ChangeFeedID.ID), 112 } 113 if w.op.GetUUIDGenerator != nil { 114 w.uuidGenerator = w.op.GetUUIDGenerator() 115 } else { 116 w.uuidGenerator = uuid.NewGenerator() 117 } 118 119 if len(cfg.Dir) == 0 { 120 return nil, errors.WrapError(errors.ErrRedoFileOp, errors.New("invalid redo dir path")) 121 } 122 123 err := os.MkdirAll(cfg.Dir, redo.DefaultDirMode) 124 if err != nil { 125 return nil, errors.WrapError(errors.ErrRedoFileOp, 126 errors.Annotatef(err, "can't make dir: %s for redo writing", cfg.Dir)) 127 } 128 129 // if we use S3 as the remote storage, a file allocator can be leveraged to 130 // pre-allocate files for us. 131 // TODO: test whether this improvement can also be applied to NFS. 132 if w.cfg.UseExternalStorage { 133 w.allocator = fsutil.NewFileAllocator(cfg.Dir, cfg.LogType, cfg.MaxLogSizeInBytes) 134 } 135 136 w.running.Store(true) 137 return w, nil 138 } 139 140 // Write implement write interface 141 // TODO: more general api with fileName generated by caller 142 func (w *Writer) Write(rawData []byte) (int, error) { 143 w.Lock() 144 defer w.Unlock() 145 146 writeLen := int64(len(rawData)) 147 if writeLen > w.cfg.MaxLogSizeInBytes { 148 return 0, errors.ErrFileSizeExceed.GenWithStackByArgs(writeLen, w.cfg.MaxLogSizeInBytes) 149 } 150 151 if w.file == nil { 152 if err := w.openNew(); err != nil { 153 return 0, err 154 } 155 } 156 157 if w.size+writeLen > w.cfg.MaxLogSizeInBytes { 158 if err := w.rotate(); err != nil { 159 return 0, err 160 } 161 } 162 163 if w.maxCommitTS.Load() < w.eventCommitTS.Load() { 164 w.maxCommitTS.Store(w.eventCommitTS.Load()) 165 } 166 // ref: https://github.com/etcd-io/etcd/pull/5250 167 lenField, padBytes := writer.EncodeFrameSize(len(rawData)) 168 if err := w.writeUint64(lenField, w.uint64buf); err != nil { 169 return 0, err 170 } 171 172 if padBytes != 0 { 173 rawData = append(rawData, make([]byte, padBytes)...) 174 } 175 176 n, err := w.bw.Write(rawData) 177 if err != nil { 178 return 0, err 179 } 180 w.metricWriteBytes.Add(float64(n)) 181 w.size += int64(n) 182 183 return n, err 184 } 185 186 // AdvanceTs implement Advance interface 187 func (w *Writer) AdvanceTs(commitTs uint64) { 188 w.eventCommitTS.Store(commitTs) 189 } 190 191 func (w *Writer) writeUint64(n uint64, buf []byte) error { 192 binary.LittleEndian.PutUint64(buf, n) 193 v, err := w.bw.Write(buf) 194 w.metricWriteBytes.Add(float64(v)) 195 196 return err 197 } 198 199 // Close implements fileWriter.Close. 200 func (w *Writer) Close() error { 201 w.Lock() 202 defer w.Unlock() 203 // always set to false when closed, since if having err may not get fixed just by retry 204 defer w.running.Store(false) 205 206 if w.allocator != nil { 207 w.allocator.Close() 208 w.allocator = nil 209 } 210 211 if !w.IsRunning() { 212 return nil 213 } 214 215 common.RedoFlushAllDurationHistogram. 216 DeleteLabelValues(w.cfg.ChangeFeedID.Namespace, w.cfg.ChangeFeedID.ID) 217 common.RedoFsyncDurationHistogram. 218 DeleteLabelValues(w.cfg.ChangeFeedID.Namespace, w.cfg.ChangeFeedID.ID) 219 common.RedoWriteBytesGauge. 220 DeleteLabelValues(w.cfg.ChangeFeedID.Namespace, w.cfg.ChangeFeedID.ID) 221 222 ctx, cancel := context.WithTimeout(context.Background(), redo.CloseTimeout) 223 defer cancel() 224 return w.close(ctx) 225 } 226 227 // IsRunning implement IsRunning interface 228 func (w *Writer) IsRunning() bool { 229 return w.running.Load() 230 } 231 232 func (w *Writer) close(ctx context.Context) error { 233 if w.file == nil { 234 return nil 235 } 236 237 if err := w.flush(); err != nil { 238 return err 239 } 240 241 if w.cfg.UseExternalStorage { 242 off, err := w.file.Seek(0, io.SeekCurrent) 243 if err != nil { 244 return err 245 } 246 // offset equals to 0 means that no written happened for current file, 247 // we can simply return 248 if off == 0 { 249 return nil 250 } 251 // a file created by a file allocator needs to be truncated 252 // to save disk space and network bandwidth. 253 if err := w.file.Truncate(off); err != nil { 254 return err 255 } 256 } 257 258 // rename the file name from commitTs.log.tmp to maxCommitTS.log if closed safely 259 // after rename, the file name could be used for search, since the ts is the max ts for all events in the file. 260 w.commitTS.Store(w.maxCommitTS.Load()) 261 err := os.Rename(w.file.Name(), w.filePath()) 262 if err != nil { 263 return errors.WrapError(errors.ErrRedoFileOp, err) 264 } 265 266 dirFile, err := os.Open(w.cfg.Dir) 267 if err != nil { 268 return errors.WrapError(errors.ErrRedoFileOp, err) 269 } 270 defer dirFile.Close() 271 // sync the dir to guarantee the renamed file is persisted to disk. 272 err = dirFile.Sync() 273 if err != nil { 274 return errors.WrapError(errors.ErrRedoFileOp, err) 275 } 276 277 // We only write content to S3 before closing the local file. 278 // By this way, we no longer need renaming object in S3. 279 if w.cfg.UseExternalStorage { 280 err = w.writeToS3(ctx, w.ongoingFilePath) 281 if err != nil { 282 w.file.Close() 283 w.file = nil 284 return errors.WrapError(errors.ErrExternalStorageAPI, err) 285 } 286 } 287 288 err = w.file.Close() 289 w.file = nil 290 return errors.WrapError(errors.ErrRedoFileOp, err) 291 } 292 293 func (w *Writer) getLogFileName() string { 294 if w.op != nil && w.op.GetLogFileName != nil { 295 return w.op.GetLogFileName() 296 } 297 uid := w.uuidGenerator.NewString() 298 if model.DefaultNamespace == w.cfg.ChangeFeedID.Namespace { 299 return fmt.Sprintf(redo.RedoLogFileFormatV1, 300 w.cfg.CaptureID, w.cfg.ChangeFeedID.ID, w.cfg.LogType, 301 w.commitTS.Load(), uid, redo.LogEXT) 302 } 303 return fmt.Sprintf(redo.RedoLogFileFormatV2, 304 w.cfg.CaptureID, w.cfg.ChangeFeedID.Namespace, w.cfg.ChangeFeedID.ID, 305 w.cfg.LogType, w.commitTS.Load(), uid, redo.LogEXT) 306 } 307 308 // filePath always creates a new, unique file path, note this function is not 309 // thread-safe, writer needs to ensure lock is acquired when calling it. 310 func (w *Writer) filePath() string { 311 fp := filepath.Join(w.cfg.Dir, w.getLogFileName()) 312 w.ongoingFilePath = fp 313 return fp 314 } 315 316 func openTruncFile(name string) (*os.File, error) { 317 return os.OpenFile(name, os.O_CREATE|os.O_WRONLY|os.O_TRUNC, redo.DefaultFileMode) 318 } 319 320 func (w *Writer) openNew() error { 321 err := os.MkdirAll(w.cfg.Dir, redo.DefaultDirMode) 322 if err != nil { 323 return errors.WrapError(errors.ErrRedoFileOp, 324 errors.Annotatef(err, "can't make dir: %s for new redo logfile", w.cfg.Dir)) 325 } 326 327 // reset ts used in file name when new file 328 var f *os.File 329 if w.allocator == nil { 330 w.commitTS.Store(w.eventCommitTS.Load()) 331 w.maxCommitTS.Store(w.eventCommitTS.Load()) 332 path := w.filePath() + redo.TmpEXT 333 f, err = openTruncFile(path) 334 if err != nil { 335 return errors.WrapError(errors.ErrRedoFileOp, 336 errors.Annotate(err, "can't open new redolog file")) 337 } 338 } else { 339 // if there is a file allocator, we use the pre-created file 340 // supplied by the allocator to boost performance 341 f, err = w.allocator.Open() 342 if err != nil { 343 return errors.WrapError(errors.ErrRedoFileOp, 344 errors.Annotate(err, "can't open new redolog file with file allocator")) 345 } 346 } 347 w.file = f 348 w.size = 0 349 err = w.newPageWriter() 350 if err != nil { 351 return err 352 } 353 return nil 354 } 355 356 func (w *Writer) newPageWriter() error { 357 offset, err := w.file.Seek(0, io.SeekCurrent) 358 if err != nil { 359 return errors.WrapError(errors.ErrRedoFileOp, err) 360 } 361 w.bw = pioutil.NewPageWriter(w.file, redo.PageBytes, int(offset)) 362 363 return nil 364 } 365 366 func (w *Writer) rotate() error { 367 ctx, cancel := context.WithTimeout(context.Background(), redo.DefaultTimeout) 368 defer cancel() 369 if err := w.close(ctx); err != nil { 370 return err 371 } 372 return w.openNew() 373 } 374 375 // flushAndRotateFile flushes the file to disk and rotate it if S3 storage is used. 376 func (w *Writer) flushAndRotateFile() error { 377 if w.file == nil { 378 return nil 379 } 380 381 start := time.Now() 382 err := w.flush() 383 if err != nil { 384 return err 385 } 386 387 if !w.cfg.UseExternalStorage { 388 return nil 389 } 390 391 if w.size == 0 { 392 return nil 393 } 394 395 // for s3 storage, when the file is flushed to disk, we need an immediate 396 // file rotate. Otherwise, the existing file content would be repeatedly written to S3, 397 // which could cause considerable network bandwidth waste. 398 err = w.rotate() 399 if err != nil { 400 return nil 401 } 402 403 w.metricFlushAllDuration.Observe(time.Since(start).Seconds()) 404 405 return err 406 } 407 408 // Flush implement Flush interface 409 func (w *Writer) Flush() error { 410 w.Lock() 411 defer w.Unlock() 412 413 return w.flushAndRotateFile() 414 } 415 416 func (w *Writer) flush() error { 417 if w.file == nil { 418 return nil 419 } 420 421 n, err := w.bw.FlushN() 422 w.metricWriteBytes.Add(float64(n)) 423 if err != nil { 424 return errors.WrapError(errors.ErrRedoFileOp, err) 425 } 426 427 start := time.Now() 428 err = w.file.Sync() 429 w.metricFsyncDuration.Observe(time.Since(start).Seconds()) 430 431 return errors.WrapError(errors.ErrRedoFileOp, err) 432 } 433 434 func (w *Writer) writeToS3(ctx context.Context, name string) error { 435 fileData, err := os.ReadFile(name) 436 if err != nil { 437 return errors.WrapError(errors.ErrRedoFileOp, err) 438 } 439 440 // Key in s3: aws.String(rs.options.Prefix + name), prefix should be changefeed name 441 err = w.storage.WriteFile(ctx, filepath.Base(name), fileData) 442 if err != nil { 443 return errors.WrapError(errors.ErrExternalStorageAPI, err) 444 } 445 446 // in case the page cache piling up triggered the OS memory reclaming which may cause 447 // I/O latency spike, we mandatorily drop the page cache of the file when it is successfully 448 // written to S3. 449 err = fsutil.DropPageCache(name) 450 if err != nil { 451 return errors.WrapError(errors.ErrRedoFileOp, err) 452 } 453 454 return nil 455 }