github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/cdc/redo/writer/memory/file_worker.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 memory 15 16 import ( 17 "bytes" 18 "context" 19 "fmt" 20 "io" 21 "sync" 22 "time" 23 24 "github.com/pierrec/lz4/v4" 25 "github.com/pingcap/log" 26 "github.com/pingcap/tidb/br/pkg/storage" 27 "github.com/pingcap/tiflow/cdc/model" 28 "github.com/pingcap/tiflow/cdc/redo/common" 29 "github.com/pingcap/tiflow/cdc/redo/writer" 30 "github.com/pingcap/tiflow/pkg/compression" 31 "github.com/pingcap/tiflow/pkg/errors" 32 "github.com/pingcap/tiflow/pkg/redo" 33 "github.com/pingcap/tiflow/pkg/uuid" 34 "github.com/prometheus/client_golang/prometheus" 35 "go.uber.org/zap" 36 "golang.org/x/sync/errgroup" 37 ) 38 39 type fileCache struct { 40 data []byte 41 fileSize int64 42 maxCommitTs model.Ts 43 // After memoryWriter become stable, this field would be used to 44 // avoid traversing log files. 45 minCommitTs model.Ts 46 47 filename string 48 flushed chan struct{} 49 writer *dataWriter 50 } 51 52 type dataWriter struct { 53 buf *bytes.Buffer 54 writer io.Writer 55 closer io.Closer 56 } 57 58 func (w *dataWriter) Write(p []byte) (n int, err error) { 59 return w.writer.Write(p) 60 } 61 62 func (w *dataWriter) Close() error { 63 if w.closer != nil { 64 return w.closer.Close() 65 } 66 return nil 67 } 68 69 func (f *fileCache) waitFlushed(ctx context.Context) error { 70 if f.flushed != nil { 71 select { 72 case <-ctx.Done(): 73 return ctx.Err() 74 case <-f.flushed: 75 } 76 } 77 return nil 78 } 79 80 func (f *fileCache) markFlushed() { 81 if f.flushed != nil { 82 close(f.flushed) 83 } 84 } 85 86 func (f *fileCache) appendData(event *polymorphicRedoEvent) error { 87 _, err := f.writer.Write(event.data.Bytes()) 88 if err != nil { 89 return err 90 } 91 f.fileSize += int64(event.data.Len()) 92 if event.commitTs > f.maxCommitTs { 93 f.maxCommitTs = event.commitTs 94 } 95 if event.commitTs < f.minCommitTs { 96 f.minCommitTs = event.commitTs 97 } 98 return nil 99 } 100 101 type fileWorkerGroup struct { 102 cfg *writer.LogWriterConfig 103 op *writer.LogWriterOptions 104 workerNum int 105 106 extStorage storage.ExternalStorage 107 uuidGenerator uuid.Generator 108 109 pool sync.Pool 110 files []*fileCache 111 flushCh chan *fileCache 112 113 metricWriteBytes prometheus.Gauge 114 metricFlushAllDuration prometheus.Observer 115 } 116 117 func newFileWorkerGroup( 118 cfg *writer.LogWriterConfig, workerNum int, 119 extStorage storage.ExternalStorage, 120 opts ...writer.Option, 121 ) *fileWorkerGroup { 122 if workerNum <= 0 { 123 workerNum = redo.DefaultFlushWorkerNum 124 } 125 126 op := &writer.LogWriterOptions{} 127 for _, opt := range opts { 128 opt(op) 129 } 130 131 return &fileWorkerGroup{ 132 cfg: cfg, 133 op: op, 134 workerNum: workerNum, 135 extStorage: extStorage, 136 uuidGenerator: uuid.NewGenerator(), 137 pool: sync.Pool{ 138 New: func() interface{} { 139 // Use pointer here to prevent static checkers from reporting errors. 140 // Ref: https://github.com/dominikh/go-tools/issues/1336. 141 buf := make([]byte, 0, cfg.MaxLogSizeInBytes) 142 return &buf 143 }, 144 }, 145 flushCh: make(chan *fileCache), 146 metricWriteBytes: common.RedoWriteBytesGauge. 147 WithLabelValues(cfg.ChangeFeedID.Namespace, cfg.ChangeFeedID.ID), 148 metricFlushAllDuration: common.RedoFlushAllDurationHistogram. 149 WithLabelValues(cfg.ChangeFeedID.Namespace, cfg.ChangeFeedID.ID), 150 } 151 } 152 153 func (f *fileWorkerGroup) Run( 154 ctx context.Context, inputCh <-chan *polymorphicRedoEvent, 155 ) (err error) { 156 defer func() { 157 f.close() 158 log.Warn("redo file workers closed", 159 zap.String("namespace", f.cfg.ChangeFeedID.Namespace), 160 zap.String("changefeed", f.cfg.ChangeFeedID.ID), 161 zap.Error(err)) 162 }() 163 164 eg, egCtx := errgroup.WithContext(ctx) 165 eg.Go(func() error { 166 return f.bgWriteLogs(egCtx, inputCh) 167 }) 168 for i := 0; i < f.workerNum; i++ { 169 eg.Go(func() error { 170 return f.bgFlushFileCache(egCtx) 171 }) 172 } 173 log.Info("redo file workers started", 174 zap.String("namespace", f.cfg.ChangeFeedID.Namespace), 175 zap.String("changefeed", f.cfg.ChangeFeedID.ID), 176 zap.Int("workerNum", f.workerNum)) 177 return eg.Wait() 178 } 179 180 func (f *fileWorkerGroup) close() { 181 common.RedoFlushAllDurationHistogram. 182 DeleteLabelValues(f.cfg.ChangeFeedID.Namespace, f.cfg.ChangeFeedID.ID) 183 common.RedoWriteBytesGauge. 184 DeleteLabelValues(f.cfg.ChangeFeedID.Namespace, f.cfg.ChangeFeedID.ID) 185 } 186 187 func (f *fileWorkerGroup) bgFlushFileCache(egCtx context.Context) error { 188 for { 189 select { 190 case <-egCtx.Done(): 191 return errors.Trace(egCtx.Err()) 192 case file := <-f.flushCh: 193 start := time.Now() 194 if err := file.writer.Close(); err != nil { 195 return errors.Trace(err) 196 } 197 var err error 198 if f.cfg.FlushConcurrency <= 1 { 199 err = f.extStorage.WriteFile(egCtx, file.filename, file.writer.buf.Bytes()) 200 } else { 201 err = f.multiPartUpload(egCtx, file) 202 } 203 f.metricFlushAllDuration.Observe(time.Since(start).Seconds()) 204 if err != nil { 205 return errors.Trace(err) 206 } 207 file.markFlushed() 208 209 bufPtr := &file.data 210 file.data = nil 211 f.pool.Put(bufPtr) 212 } 213 } 214 } 215 216 func (f *fileWorkerGroup) multiPartUpload(ctx context.Context, file *fileCache) error { 217 multipartWrite, err := f.extStorage.Create(ctx, file.filename, &storage.WriterOption{ 218 Concurrency: f.cfg.FlushConcurrency, 219 }) 220 if err != nil { 221 return errors.Trace(err) 222 } 223 if _, err = multipartWrite.Write(ctx, file.writer.buf.Bytes()); err != nil { 224 return errors.Trace(err) 225 } 226 return errors.Trace(multipartWrite.Close(ctx)) 227 } 228 229 func (f *fileWorkerGroup) bgWriteLogs( 230 egCtx context.Context, inputCh <-chan *polymorphicRedoEvent, 231 ) (err error) { 232 for { 233 select { 234 case <-egCtx.Done(): 235 return errors.Trace(egCtx.Err()) 236 case event := <-inputCh: 237 if event == nil { 238 log.Error("inputCh of redo file worker is closed unexpectedly") 239 return errors.ErrUnexpected.FastGenByArgs("inputCh of redo file worker is closed unexpectedly") 240 } 241 242 if event.data != nil { 243 err = f.writeToCache(egCtx, event) 244 event.reset() 245 eventPool.Put(event) 246 } else if event.flushCallback != nil { 247 err = f.flushAll(egCtx) 248 event.flushCallback() 249 } 250 251 if err != nil { 252 return errors.Trace(err) 253 } 254 } 255 } 256 } 257 258 // newFileCache write event to a new file cache. 259 func (f *fileWorkerGroup) newFileCache(event *polymorphicRedoEvent) error { 260 bufPtr := f.pool.Get().(*[]byte) 261 buf := *bufPtr 262 buf = buf[:0] 263 var ( 264 wr io.Writer 265 closer io.Closer 266 ) 267 bufferWriter := bytes.NewBuffer(buf) 268 wr = bufferWriter 269 if f.cfg.Compression == compression.LZ4 { 270 wr = lz4.NewWriter(bufferWriter) 271 closer = wr.(io.Closer) 272 } 273 _, err := wr.Write(event.data.Bytes()) 274 if err != nil { 275 return errors.Trace(err) 276 } 277 278 dw := &dataWriter{ 279 buf: bufferWriter, 280 writer: wr, 281 closer: closer, 282 } 283 file := &fileCache{ 284 data: buf, 285 fileSize: int64(len(event.data.Bytes())), 286 maxCommitTs: event.commitTs, 287 minCommitTs: event.commitTs, 288 flushed: make(chan struct{}), 289 writer: dw, 290 } 291 f.files = append(f.files, file) 292 return nil 293 } 294 295 func (f *fileWorkerGroup) writeToCache( 296 egCtx context.Context, event *polymorphicRedoEvent, 297 ) error { 298 writeLen := int64(event.data.Len()) 299 if writeLen > f.cfg.MaxLogSizeInBytes { 300 // TODO: maybe we need to deal with the oversized event. 301 return errors.ErrFileSizeExceed.GenWithStackByArgs(writeLen, f.cfg.MaxLogSizeInBytes) 302 } 303 defer f.metricWriteBytes.Add(float64(writeLen)) 304 305 if len(f.files) == 0 { 306 return f.newFileCache(event) 307 } 308 309 file := f.files[len(f.files)-1] 310 if file.fileSize+writeLen > f.cfg.MaxLogSizeInBytes { 311 file.filename = f.getLogFileName(file.maxCommitTs) 312 select { 313 case <-egCtx.Done(): 314 return errors.Trace(egCtx.Err()) 315 case f.flushCh <- file: 316 } 317 318 return f.newFileCache(event) 319 } 320 321 return file.appendData(event) 322 } 323 324 func (f *fileWorkerGroup) flushAll(egCtx context.Context) error { 325 if len(f.files) == 0 { 326 return nil 327 } 328 329 file := f.files[len(f.files)-1] 330 file.filename = f.getLogFileName(file.maxCommitTs) 331 select { 332 case <-egCtx.Done(): 333 return errors.Trace(egCtx.Err()) 334 case f.flushCh <- file: 335 } 336 337 // wait all files flushed 338 for _, file := range f.files { 339 err := file.waitFlushed(egCtx) 340 if err != nil { 341 return errors.Trace(err) 342 } 343 } 344 f.files = f.files[:0] 345 return nil 346 } 347 348 func (f *fileWorkerGroup) getLogFileName(maxCommitTS model.Ts) string { 349 if f.op != nil && f.op.GetLogFileName != nil { 350 return f.op.GetLogFileName() 351 } 352 uid := f.uuidGenerator.NewString() 353 if model.DefaultNamespace == f.cfg.ChangeFeedID.Namespace { 354 return fmt.Sprintf(redo.RedoLogFileFormatV1, 355 f.cfg.CaptureID, f.cfg.ChangeFeedID.ID, f.cfg.LogType, 356 maxCommitTS, uid, redo.LogEXT) 357 } 358 return fmt.Sprintf(redo.RedoLogFileFormatV2, 359 f.cfg.CaptureID, f.cfg.ChangeFeedID.Namespace, f.cfg.ChangeFeedID.ID, 360 f.cfg.LogType, maxCommitTS, uid, redo.LogEXT) 361 }