github.com/pingcap/ticdc@v0.0.0-20220526033649-485a10ef2652/cdc/sink/cdclog/s3.go (about) 1 // Copyright 2020 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 cdclog 15 16 import ( 17 "context" 18 "net/url" 19 "strings" 20 "time" 21 22 "github.com/pingcap/br/pkg/storage" 23 "github.com/pingcap/errors" 24 "github.com/pingcap/kvproto/pkg/backup" 25 "github.com/pingcap/log" 26 parsemodel "github.com/pingcap/parser/model" 27 "github.com/pingcap/ticdc/cdc/model" 28 "github.com/pingcap/ticdc/cdc/sink/codec" 29 cerror "github.com/pingcap/ticdc/pkg/errors" 30 "github.com/pingcap/ticdc/pkg/quotes" 31 "github.com/uber-go/atomic" 32 "go.uber.org/zap" 33 ) 34 35 const ( 36 maxPartFlushSize = 5 << 20 // The minimal multipart upload size is 5Mb. 37 maxCompletePartSize = 100 << 20 // rotate row changed event file if one complete file larger than 100Mb 38 maxDDLFlushSize = 10 << 20 // rotate ddl event file if one complete file larger than 10Mb 39 40 defaultBufferChanSize = 20480 41 defaultFlushRowChangedEventDuration = 5 * time.Second // TODO make it as a config 42 ) 43 44 type tableBuffer struct { 45 // for log 46 tableID int64 47 dataCh chan *model.RowChangedEvent 48 sendSize *atomic.Int64 49 sendEvents *atomic.Int64 50 51 encoder codec.EventBatchEncoder 52 53 uploadParts struct { 54 writer storage.ExternalFileWriter 55 uploadNum int 56 byteSize int64 57 } 58 } 59 60 func (tb *tableBuffer) dataChan() chan *model.RowChangedEvent { 61 return tb.dataCh 62 } 63 64 func (tb *tableBuffer) TableID() int64 { 65 return tb.tableID 66 } 67 68 func (tb *tableBuffer) Events() *atomic.Int64 { 69 return tb.sendEvents 70 } 71 72 func (tb *tableBuffer) Size() *atomic.Int64 { 73 return tb.sendSize 74 } 75 76 func (tb *tableBuffer) isEmpty() bool { 77 return tb.sendEvents.Load() == 0 && tb.uploadParts.uploadNum == 0 78 } 79 80 func (tb *tableBuffer) shouldFlush() bool { 81 // if sendSize > 5 MB or data chennal is full, flush it 82 return tb.sendSize.Load() > maxPartFlushSize || tb.sendEvents.Load() == defaultBufferChanSize 83 } 84 85 func (tb *tableBuffer) flush(ctx context.Context, sink *logSink) error { 86 hashPart := tb.uploadParts 87 sendEvents := tb.sendEvents.Load() 88 if sendEvents == 0 && hashPart.uploadNum == 0 { 89 log.Info("nothing to flush", zap.Int64("tableID", tb.tableID)) 90 return nil 91 } 92 93 firstCreated := false 94 if tb.encoder == nil { 95 // create encoder for each file 96 tb.encoder = sink.encoder() 97 firstCreated = true 98 } 99 100 var newFileName string 101 flushedSize := int64(0) 102 for event := int64(0); event < sendEvents; event++ { 103 row := <-tb.dataCh 104 flushedSize += row.ApproximateSize 105 if event == sendEvents-1 { 106 // if last event, we record ts as new rotate file name 107 newFileName = makeTableFileObject(row.Table.TableID, row.CommitTs) 108 } 109 _, err := tb.encoder.AppendRowChangedEvent(row) 110 if err != nil { 111 return err 112 } 113 } 114 rowDatas := tb.encoder.MixedBuild(firstCreated) 115 // reset encoder buf for next round append 116 defer func() { 117 if tb.encoder != nil { 118 tb.encoder.Reset() 119 } 120 }() 121 122 log.Debug("[FlushRowChangedEvents[Debug]] flush table buffer", 123 zap.Int64("table", tb.tableID), 124 zap.Int64("event size", sendEvents), 125 zap.Int("row data size", len(rowDatas)), 126 zap.Int("upload num", hashPart.uploadNum), 127 zap.Int64("upload byte size", hashPart.byteSize), 128 // zap.ByteString("rowDatas", rowDatas), 129 ) 130 131 if len(rowDatas) > maxPartFlushSize || hashPart.uploadNum > 0 { 132 // S3 multi-upload need every chunk(except the last one) is greater than 5Mb 133 // so, if this batch data size is greater than 5Mb or it has uploadPart already 134 // we will use multi-upload this batch data 135 if len(rowDatas) > 0 { 136 if hashPart.writer == nil { 137 fileWriter, err := sink.storage().Create(ctx, newFileName) 138 if err != nil { 139 return cerror.WrapError(cerror.ErrS3SinkStorageAPI, err) 140 } 141 hashPart.writer = fileWriter 142 } 143 144 _, err := hashPart.writer.Write(ctx, rowDatas) 145 if err != nil { 146 return cerror.WrapError(cerror.ErrS3SinkStorageAPI, err) 147 } 148 149 hashPart.byteSize += int64(len(rowDatas)) 150 hashPart.uploadNum++ 151 } 152 153 if hashPart.byteSize > maxCompletePartSize || len(rowDatas) <= maxPartFlushSize { 154 // we need do complete when total upload size is greater than 100Mb 155 // or this part data is less than 5Mb to avoid meet EntityTooSmall error 156 log.Info("[FlushRowChangedEvents] complete file", zap.Int64("tableID", tb.tableID)) 157 err := hashPart.writer.Close(ctx) 158 if err != nil { 159 return cerror.WrapError(cerror.ErrS3SinkStorageAPI, err) 160 } 161 hashPart.byteSize = 0 162 hashPart.uploadNum = 0 163 hashPart.writer = nil 164 tb.encoder = nil 165 } 166 } else { 167 // generate normal file because S3 multi-upload need every part at least 5Mb. 168 log.Info("[FlushRowChangedEvents] normal upload file", zap.Int64("tableID", tb.tableID)) 169 err := sink.storage().WriteFile(ctx, newFileName, rowDatas) 170 if err != nil { 171 return cerror.WrapError(cerror.ErrS3SinkStorageAPI, err) 172 } 173 tb.encoder = nil 174 } 175 176 tb.sendEvents.Sub(sendEvents) 177 tb.sendSize.Sub(flushedSize) 178 tb.uploadParts = hashPart 179 return nil 180 } 181 182 func newTableBuffer(tableID int64) logUnit { 183 return &tableBuffer{ 184 tableID: tableID, 185 dataCh: make(chan *model.RowChangedEvent, defaultBufferChanSize), 186 sendSize: atomic.NewInt64(0), 187 sendEvents: atomic.NewInt64(0), 188 uploadParts: struct { 189 writer storage.ExternalFileWriter 190 uploadNum int 191 byteSize int64 192 }{ 193 writer: nil, 194 uploadNum: 0, 195 byteSize: 0, 196 }, 197 } 198 } 199 200 type s3Sink struct { 201 *logSink 202 203 prefix string 204 205 storage storage.ExternalStorage 206 207 logMeta *logMeta 208 209 // hold encoder for ddl event log 210 ddlEncoder codec.EventBatchEncoder 211 } 212 213 func (s *s3Sink) EmitRowChangedEvents(ctx context.Context, rows ...*model.RowChangedEvent) error { 214 return s.emitRowChangedEvents(ctx, newTableBuffer, rows...) 215 } 216 217 func (s *s3Sink) flushLogMeta(ctx context.Context) error { 218 data, err := s.logMeta.Marshal() 219 if err != nil { 220 return cerror.WrapError(cerror.ErrMarshalFailed, err) 221 } 222 return cerror.WrapError(cerror.ErrS3SinkWriteStorage, s.storage.WriteFile(ctx, logMetaFile, data)) 223 } 224 225 func (s *s3Sink) FlushRowChangedEvents(ctx context.Context, resolvedTs uint64) (uint64, error) { 226 // we should flush all events before resolvedTs, there are two kind of flush policy 227 // 1. flush row events to a s3 chunk: if the event size is not enough, 228 // TODO: when cdc crashed, we should repair these chunks to a complete file 229 // 2. flush row events to a complete s3 file: if the event size is enough 230 return s.flushRowChangedEvents(ctx, resolvedTs) 231 } 232 233 // EmitCheckpointTs update the global resolved ts in log meta 234 // sleep 5 seconds to avoid update too frequently 235 func (s *s3Sink) EmitCheckpointTs(ctx context.Context, ts uint64) error { 236 s.logMeta.GlobalResolvedTS = ts 237 return s.flushLogMeta(ctx) 238 } 239 240 // EmitDDLEvent write ddl event to S3 directory, all events split by '\n' 241 // Because S3 doesn't support append-like write. 242 // we choose a hack way to read origin file then write in place. 243 func (s *s3Sink) EmitDDLEvent(ctx context.Context, ddl *model.DDLEvent) error { 244 switch ddl.Type { 245 case parsemodel.ActionCreateTable: 246 s.logMeta.Names[ddl.TableInfo.TableID] = quotes.QuoteSchema(ddl.TableInfo.Schema, ddl.TableInfo.Table) 247 err := s.flushLogMeta(ctx) 248 if err != nil { 249 return err 250 } 251 case parsemodel.ActionRenameTable: 252 delete(s.logMeta.Names, ddl.PreTableInfo.TableID) 253 s.logMeta.Names[ddl.TableInfo.TableID] = quotes.QuoteSchema(ddl.TableInfo.Schema, ddl.TableInfo.Table) 254 err := s.flushLogMeta(ctx) 255 if err != nil { 256 return err 257 } 258 } 259 firstCreated := false 260 if s.ddlEncoder == nil { 261 s.ddlEncoder = s.encoder() 262 firstCreated = true 263 } 264 // reset encoder buf for next round append 265 defer s.ddlEncoder.Reset() 266 267 var ( 268 name string 269 size int64 270 fileData []byte 271 ) 272 opt := &storage.WalkOption{ 273 SubDir: ddlEventsDir, 274 ListCount: 1, 275 } 276 err := s.storage.WalkDir(ctx, opt, func(key string, fileSize int64) error { 277 log.Debug("[EmitDDLEvent] list content from s3", 278 zap.String("key", key), 279 zap.Int64("size", size), 280 zap.Any("ddl", ddl)) 281 name = strings.ReplaceAll(key, s.prefix, "") 282 size = fileSize 283 return nil 284 }) 285 if err != nil { 286 return cerror.WrapError(cerror.ErrS3SinkStorageAPI, err) 287 } 288 289 // only reboot and (size = 0 or size >= maxRowFileSize) should we add version to s3 290 withVersion := firstCreated && (size == 0 || size >= maxDDLFlushSize) 291 292 // clean ddlEncoder version part 293 // if we reboot cdc and size between (0, maxDDLFlushSize), we should skip version part in 294 // JSONEventBatchEncoder.keyBuf, JSONEventBatchEncoder consturctor func has 295 // alreay filled with version part, see NewJSONEventBatchEncoder and 296 // JSONEventBatchEncoder.MixedBuild 297 if firstCreated && size > 0 && size < maxDDLFlushSize { 298 s.ddlEncoder.Reset() 299 } 300 301 _, er := s.ddlEncoder.EncodeDDLEvent(ddl) 302 if er != nil { 303 return er 304 } 305 306 data := s.ddlEncoder.MixedBuild(withVersion) 307 308 if size == 0 || size >= maxDDLFlushSize { 309 // no ddl file exists or 310 // exists file is oversized. we should generate a new file 311 fileData = data 312 name = makeDDLFileObject(ddl.CommitTs) 313 log.Debug("[EmitDDLEvent] create first or rotate ddl log", 314 zap.String("name", name), zap.Any("ddl", ddl)) 315 if size > maxDDLFlushSize { 316 // reset ddl encoder for new file 317 s.ddlEncoder = nil 318 } 319 } else { 320 // hack way: append data to old file 321 log.Debug("[EmitDDLEvent] append ddl to origin log", 322 zap.String("name", name), zap.Any("ddl", ddl)) 323 fileData, err = s.storage.ReadFile(ctx, name) 324 if err != nil { 325 return cerror.WrapError(cerror.ErrS3SinkStorageAPI, err) 326 } 327 fileData = append(fileData, data...) 328 } 329 return s.storage.WriteFile(ctx, name, fileData) 330 } 331 332 func (s *s3Sink) Initialize(ctx context.Context, tableInfo []*model.SimpleTableInfo) error { 333 if tableInfo != nil { 334 // update log meta to record the relationship about tableName and tableID 335 s.logMeta = makeLogMetaContent(tableInfo) 336 337 data, err := s.logMeta.Marshal() 338 if err != nil { 339 return cerror.WrapError(cerror.ErrMarshalFailed, err) 340 } 341 return s.storage.WriteFile(ctx, logMetaFile, data) 342 } 343 return nil 344 } 345 346 func (s *s3Sink) Close(ctx context.Context) error { 347 return nil 348 } 349 350 func (s *s3Sink) Barrier(ctx context.Context) error { 351 // Barrier does nothing because FlushRowChangedEvents in s3 sink has flushed 352 // all buffered events forcedlly. 353 return nil 354 } 355 356 // NewS3Sink creates new sink support log data to s3 directly 357 func NewS3Sink(ctx context.Context, sinkURI *url.URL, errCh chan error) (*s3Sink, error) { 358 if len(sinkURI.Host) == 0 { 359 return nil, errors.Errorf("please specify the bucket for s3 in %s", sinkURI) 360 } 361 prefix := strings.Trim(sinkURI.Path, "/") 362 s3 := &backup.S3{Bucket: sinkURI.Host, Prefix: prefix} 363 options := &storage.BackendOptions{} 364 storage.ExtractQueryParameters(sinkURI, &options.S3) 365 if err := options.S3.Apply(s3); err != nil { 366 return nil, cerror.WrapError(cerror.ErrS3SinkInitialize, err) 367 } 368 // we should set this to true, since br set it by default in parseBackend 369 s3.ForcePathStyle = true 370 backend := &backup.StorageBackend{ 371 Backend: &backup.StorageBackend_S3{S3: s3}, 372 } 373 s3storage, err := storage.New(ctx, backend, &storage.ExternalStorageOptions{ 374 SendCredentials: false, 375 SkipCheckPath: true, 376 HTTPClient: nil, 377 }) 378 if err != nil { 379 return nil, cerror.WrapError(cerror.ErrS3SinkInitialize, err) 380 } 381 382 s := &s3Sink{ 383 prefix: prefix, 384 storage: s3storage, 385 logMeta: newLogMeta(), 386 logSink: newLogSink("", s3storage), 387 } 388 389 // important! we should flush asynchronously in another goroutine 390 go func() { 391 if err := s.startFlush(ctx); err != nil && errors.Cause(err) != context.Canceled { 392 select { 393 case <-ctx.Done(): 394 return 395 case errCh <- err: 396 default: 397 log.Error("error channel is full", zap.Error(err)) 398 } 399 } 400 }() 401 402 return s, nil 403 }