diff --git a/cmd/storage-consumer/consumer.go b/cmd/storage-consumer/consumer.go index 0cb07997ee..3df3302983 100644 --- a/cmd/storage-consumer/consumer.go +++ b/cmd/storage-consumer/consumer.go @@ -15,10 +15,8 @@ package main import ( "context" - "encoding/json" "fmt" "sort" - "strings" "time" "github.com/pingcap/errors" @@ -47,8 +45,8 @@ const ( ) type ( - fileIndexRange map[cloudstorage.FileIndexKey]indexRange - fileIndexKeyMap map[cloudstorage.FileIndexKey]uint64 + fileIndexRange map[FileIndexKey]indexRange + fileIndexKeyMap map[FileIndexKey]uint64 ) // indexRange defines a range of files. eg. CDC000002.csv ~ CDC000005.csv @@ -64,7 +62,7 @@ type consumer struct { fileExtension string sink sink.Sink // tableDMLIdxMap maintains a map of - tableDMLIdxMap map[cloudstorage.DmlPathKey]fileIndexKeyMap + tableDMLIdxMap map[DmlPathKey]fileIndexKeyMap eventsGroup map[int64]*util.EventsGroup // tableDefMap maintains a map of <`schema`.`table`, tableDef slice sorted by TableVersion> tableDefMap map[string]map[uint64]*cloudstorage.TableDefinition @@ -147,7 +145,7 @@ func newConsumer(ctx context.Context) (*consumer, error) { fileExtension: extension, sink: sink, errCh: errCh, - tableDMLIdxMap: make(map[cloudstorage.DmlPathKey]fileIndexKeyMap), + tableDMLIdxMap: make(map[DmlPathKey]fileIndexKeyMap), eventsGroup: make(map[int64]*util.EventsGroup), tableDefMap: make(map[string]map[uint64]*cloudstorage.TableDefinition), tableIDGenerator: &fakeTableIDGenerator{ @@ -156,84 +154,6 @@ func newConsumer(ctx context.Context) (*consumer, error) { }, nil } -// map1 - map2 -func diffDMLMaps( - map1, map2 map[cloudstorage.DmlPathKey]fileIndexKeyMap, -) map[cloudstorage.DmlPathKey]fileIndexRange { - resMap := make(map[cloudstorage.DmlPathKey]fileIndexRange) // DmlPathKey -> FileIndexKey -> indexRange - for dmlPathKey1, fileIndexKeyMap1 := range map1 { - dmlPathKey2, ok := map2[dmlPathKey1] - if !ok { - resMap[dmlPathKey1] = make(fileIndexRange) - for indexKey, val1 := range fileIndexKeyMap1 { - resMap[dmlPathKey1][indexKey] = indexRange{ - start: 1, - end: val1, - } - } - continue - } - for fileIndexKey, val1 := range fileIndexKeyMap1 { - val2 := dmlPathKey2[fileIndexKey] - if val1 > val2 { - if _, ok := resMap[dmlPathKey1]; !ok { - resMap[dmlPathKey1] = make(fileIndexRange) - } - resMap[dmlPathKey1][fileIndexKey] = indexRange{ - start: val2 + 1, - end: val1, - } - } - } - } - - return resMap -} - -// getNewFiles returns newly created dml files in specific ranges -func (c *consumer) getNewFiles( - ctx context.Context, -) (map[cloudstorage.DmlPathKey]fileIndexRange, error) { - tableDMLMap := make(map[cloudstorage.DmlPathKey]fileIndexRange) - opt := &storage.WalkOption{SubDir: ""} - - origDMLIdxMap := make(map[cloudstorage.DmlPathKey]fileIndexKeyMap, len(c.tableDMLIdxMap)) - for k, v := range c.tableDMLIdxMap { - m := make(fileIndexKeyMap) - for fileIndexKey, val := range v { - m[fileIndexKey] = val - } - origDMLIdxMap[k] = m - } - - err := c.externalStorage.WalkDir(ctx, opt, func(path string, size int64) error { - if cloudstorage.IsSchemaFile(path) { - err := c.parseSchemaFilePath(ctx, path) - if err != nil { - log.Error("failed to parse schema file path", zap.Error(err)) - // skip handling this file - return nil - } - } else if strings.HasSuffix(path, ".index") { - err := c.parseDMLFilePath(ctx, path) - if err != nil { - log.Error("failed to parse dml file path", zap.Error(err)) - // skip handling this file - return nil - } - } else { - log.Debug("ignore handling file", zap.String("path", path)) - } - return nil - }) - if err != nil { - return tableDMLMap, err - } - - tableDMLMap = diffDMLMaps(c.tableDMLIdxMap, origDMLIdxMap) - return tableDMLMap, err -} - func (c *consumer) appendRow2Group(dml *event.DMLEvent, enableTableAcrossNodes bool) { var ( tableID = dml.GetTableID() @@ -274,8 +194,8 @@ func (c *consumer) appendDMLEvents( ctx context.Context, tableID int64, tableDetail cloudstorage.TableDefinition, - pathKey cloudstorage.DmlPathKey, - fileIdx *cloudstorage.FileIndex, + pathKey DmlPathKey, + fileIdx *FileIndex, ) error { filePath := pathKey.GenerateDMLFilePath(fileIdx, c.fileExtension, fileIndexWidth) log.Debug("read from dml file path", zap.String("path", filePath)) @@ -378,117 +298,7 @@ func (c *consumer) flushDMLEvents(ctx context.Context, tableID int64) error { } } -func (c *consumer) parseDMLFilePath(ctx context.Context, path string) error { - var dmlkey cloudstorage.DmlPathKey - dispatcherID, err := dmlkey.ParseIndexFilePath( - putil.GetOrZero(c.replicationCfg.Sink.DateSeparator), - path, - ) - if err != nil { - return errors.Trace(err) - } - data, err := c.externalStorage.ReadFile(ctx, path) - if err != nil { - return errors.Trace(err) - } - fileName := strings.TrimSuffix(string(data), "\n") - fileIdx, err := cloudstorage.FetchIndexFromFileName(fileName, c.fileExtension) - if err != nil { - return err - } - fileIndex := &cloudstorage.FileIndex{ - FileIndexKey: cloudstorage.FileIndexKey{ - DispatcherID: dispatcherID, - EnableTableAcrossNodes: dispatcherID != "", - }, - Idx: fileIdx, - } - - m, ok := c.tableDMLIdxMap[dmlkey] - if !ok { - c.tableDMLIdxMap[dmlkey] = fileIndexKeyMap{ - fileIndex.FileIndexKey: fileIndex.Idx, - } - } else if fileIndex.Idx >= m[fileIndex.FileIndexKey] { - c.tableDMLIdxMap[dmlkey][fileIndex.FileIndexKey] = fileIndex.Idx - } - return nil -} - -func (c *consumer) parseSchemaFilePath(ctx context.Context, path string) error { - var schemaKey cloudstorage.SchemaPathKey - checksumInFile, err := schemaKey.ParseSchemaFilePath(path) - if err != nil { - return errors.Trace(err) - } - key := schemaKey.GetKey() - if tableDefs, ok := c.tableDefMap[key]; ok { - if _, ok := tableDefs[schemaKey.TableVersion]; ok { - // Skip if tableDef already exists. - return nil - } - } else { - c.tableDefMap[key] = make(map[uint64]*cloudstorage.TableDefinition) - } - - // Read tableDef from schema file and check checksum. - var tableDef cloudstorage.TableDefinition - schemaContent, err := c.externalStorage.ReadFile(ctx, path) - if err != nil { - return errors.Trace(err) - } - err = json.Unmarshal(schemaContent, &tableDef) - if err != nil { - return errors.Trace(err) - } - checksumInMem, err := tableDef.Sum32(nil) - if err != nil { - return errors.Trace(err) - } - if checksumInMem != checksumInFile || schemaKey.TableVersion != tableDef.TableVersion { - log.Panic("checksum mismatch", - zap.Uint32("checksumInMem", checksumInMem), - zap.Uint32("checksumInFile", checksumInFile), - zap.Uint64("tableversionInMem", schemaKey.TableVersion), - zap.Uint64("tableversionInFile", tableDef.TableVersion), - zap.String("path", path)) - } - - // Update tableDefMap. - c.tableDefMap[key][tableDef.TableVersion] = &tableDef - - // Fake a dml key for schema.json file, which is useful for putting DDL - // in front of the DML files when sorting. - // e.g, for the partitioned table: - // - // test/test1/439972354120482843/schema.json (partitionNum = -1) - // test/test1/439972354120482843/55/2023-03-09/CDC000001.csv (partitionNum = 55) - // test/test1/439972354120482843/66/2023-03-09/CDC000001.csv (partitionNum = 66) - // - // and for the non-partitioned table: - // test/test2/439972354120482843/schema.json (partitionNum = -1) - // test/test2/439972354120482843/2023-03-09/CDC000001.csv (partitionNum = 0) - // test/test2/439972354120482843/2023-03-09/CDC000002.csv (partitionNum = 0) - // - // the DDL event recorded in schema.json should be executed first, then the DML events - // in csv files can be executed. - dmlkey := cloudstorage.DmlPathKey{ - SchemaPathKey: schemaKey, - PartitionNum: fakePartitionNumForSchemaFile, - Date: "", - } - if _, ok := c.tableDMLIdxMap[dmlkey]; !ok { - c.tableDMLIdxMap[dmlkey] = fileIndexKeyMap{} - } else { - // duplicate table schema file found, this should not happen. - log.Panic("duplicate schema file found", - zap.String("path", path), zap.Any("tableDef", tableDef), - zap.Any("schemaKey", schemaKey), zap.Any("dmlkey", dmlkey)) - } - return nil -} - -func (c *consumer) mustGetTableDef(key cloudstorage.SchemaPathKey) cloudstorage.TableDefinition { +func (c *consumer) mustGetTableDef(key SchemaPathKey) cloudstorage.TableDefinition { var tableDef *cloudstorage.TableDefinition if tableDefs, ok := c.tableDefMap[key.GetKey()]; ok { tableDef = tableDefs[key.TableVersion] @@ -501,9 +311,9 @@ func (c *consumer) mustGetTableDef(key cloudstorage.SchemaPathKey) cloudstorage. func (c *consumer) handleNewFiles( ctx context.Context, - dmlFileMap map[cloudstorage.DmlPathKey]fileIndexRange, + dmlFileMap map[DmlPathKey]fileIndexRange, ) error { - keys := make([]cloudstorage.DmlPathKey, 0, len(dmlFileMap)) + keys := make([]DmlPathKey, 0, len(dmlFileMap)) for k := range dmlFileMap { keys = append(keys, k) } @@ -550,7 +360,7 @@ func (c *consumer) handleNewFiles( fileRange := dmlFileMap[key] for indexKey, indexRange := range fileRange { for i := indexRange.start; i <= indexRange.end; i++ { - if err := c.appendDMLEvents(ctx, tableID, tableDef, key, &cloudstorage.FileIndex{ + if err := c.appendDMLEvents(ctx, tableID, tableDef, key, &FileIndex{ FileIndexKey: indexKey, Idx: i, }); err != nil { @@ -558,7 +368,9 @@ func (c *consumer) handleNewFiles( } } } - c.flushDMLEvents(ctx, tableID) + if err := c.flushDMLEvents(ctx, tableID); err != nil { + return errors.Trace(err) + } } return nil diff --git a/cmd/storage-consumer/discovery.go b/cmd/storage-consumer/discovery.go new file mode 100644 index 0000000000..32cf808ca3 --- /dev/null +++ b/cmd/storage-consumer/discovery.go @@ -0,0 +1,215 @@ +// Copyright 2025 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package main + +import ( + "context" + "encoding/json" + "strings" + + "github.com/pingcap/errors" + "github.com/pingcap/log" + "github.com/pingcap/ticdc/pkg/sink/cloudstorage" + putil "github.com/pingcap/ticdc/pkg/util" + "github.com/pingcap/tidb/br/pkg/storage" + "go.uber.org/zap" +) + +// map1 - map2 +func diffDMLMaps( + map1, map2 map[DmlPathKey]fileIndexKeyMap, +) map[DmlPathKey]fileIndexRange { + resMap := make(map[DmlPathKey]fileIndexRange) // DmlPathKey -> FileIndexKey -> indexRange + for dmlPathKey1, fileIndexKeyMap1 := range map1 { + dmlPathKey2, ok := map2[dmlPathKey1] + if !ok { + resMap[dmlPathKey1] = make(fileIndexRange) + for indexKey, val1 := range fileIndexKeyMap1 { + resMap[dmlPathKey1][indexKey] = indexRange{ + start: 1, + end: val1, + } + } + continue + } + for fileIndexKey, val1 := range fileIndexKeyMap1 { + val2 := dmlPathKey2[fileIndexKey] + if val1 > val2 { + if _, ok := resMap[dmlPathKey1]; !ok { + resMap[dmlPathKey1] = make(fileIndexRange) + } + resMap[dmlPathKey1][fileIndexKey] = indexRange{ + start: val2 + 1, + end: val1, + } + } + } + } + + return resMap +} + +// getNewFiles returns newly created dml files in specific ranges +func (c *consumer) getNewFiles( + ctx context.Context, +) (map[DmlPathKey]fileIndexRange, error) { + tableDMLMap := make(map[DmlPathKey]fileIndexRange) + opt := &storage.WalkOption{SubDir: ""} + + origDMLIdxMap := make(map[DmlPathKey]fileIndexKeyMap, len(c.tableDMLIdxMap)) + for k, v := range c.tableDMLIdxMap { + m := make(fileIndexKeyMap) + for fileIndexKey, val := range v { + m[fileIndexKey] = val + } + origDMLIdxMap[k] = m + } + + err := c.externalStorage.WalkDir(ctx, opt, func(path string, size int64) error { + if IsSchemaFile(path) { + err := c.parseSchemaFilePath(ctx, path) + if err != nil { + log.Error("failed to parse schema file path", zap.Error(err)) + // skip handling this file + return nil + } + } else if strings.HasSuffix(path, ".index") { + err := c.parseDMLFilePath(ctx, path) + if err != nil { + log.Error("failed to parse dml file path", zap.Error(err)) + // skip handling this file + return nil + } + } else { + log.Debug("ignore handling file", zap.String("path", path)) + } + return nil + }) + if err != nil { + return tableDMLMap, err + } + + tableDMLMap = diffDMLMaps(c.tableDMLIdxMap, origDMLIdxMap) + return tableDMLMap, err +} + +func (c *consumer) parseDMLFilePath(ctx context.Context, path string) error { + var dmlkey DmlPathKey + dispatcherID, err := dmlkey.ParseIndexFilePath( + putil.GetOrZero(c.replicationCfg.Sink.DateSeparator), + path, + ) + if err != nil { + return errors.Trace(err) + } + data, err := c.externalStorage.ReadFile(ctx, path) + if err != nil { + return errors.Trace(err) + } + fileName := strings.TrimSuffix(string(data), "\n") + fileIdx, err := FetchIndexFromFileName(fileName, c.fileExtension) + if err != nil { + return err + } + fileIndex := &FileIndex{ + FileIndexKey: FileIndexKey{ + DispatcherID: dispatcherID, + EnableTableAcrossNodes: dispatcherID != "", + }, + Idx: fileIdx, + } + + m, ok := c.tableDMLIdxMap[dmlkey] + if !ok { + c.tableDMLIdxMap[dmlkey] = fileIndexKeyMap{ + fileIndex.FileIndexKey: fileIndex.Idx, + } + } else if fileIndex.Idx >= m[fileIndex.FileIndexKey] { + c.tableDMLIdxMap[dmlkey][fileIndex.FileIndexKey] = fileIndex.Idx + } + return nil +} + +func (c *consumer) parseSchemaFilePath(ctx context.Context, path string) error { + var schemaKey SchemaPathKey + checksumInFile, err := schemaKey.ParseSchemaFilePath(path) + if err != nil { + return errors.Trace(err) + } + key := schemaKey.GetKey() + if tableDefs, ok := c.tableDefMap[key]; ok { + if _, ok := tableDefs[schemaKey.TableVersion]; ok { + // Skip if tableDef already exists. + return nil + } + } else { + c.tableDefMap[key] = make(map[uint64]*cloudstorage.TableDefinition) + } + + // Read tableDef from schema file and check checksum. + var tableDef cloudstorage.TableDefinition + schemaContent, err := c.externalStorage.ReadFile(ctx, path) + if err != nil { + return errors.Trace(err) + } + err = json.Unmarshal(schemaContent, &tableDef) + if err != nil { + return errors.Trace(err) + } + checksumInMem, err := tableDef.Sum32(nil) + if err != nil { + return errors.Trace(err) + } + if checksumInMem != checksumInFile || schemaKey.TableVersion != tableDef.TableVersion { + log.Panic("checksum mismatch", + zap.Uint32("checksumInMem", checksumInMem), + zap.Uint32("checksumInFile", checksumInFile), + zap.Uint64("tableversionInMem", schemaKey.TableVersion), + zap.Uint64("tableversionInFile", tableDef.TableVersion), + zap.String("path", path)) + } + + // Update tableDefMap. + c.tableDefMap[key][tableDef.TableVersion] = &tableDef + + // Fake a dml key for schema.json file, which is useful for putting DDL + // in front of the DML files when sorting. + // e.g, for the partitioned table: + // + // test/test1/439972354120482843/schema.json (partitionNum = -1) + // test/test1/439972354120482843/55/2023-03-09/CDC000001.csv (partitionNum = 55) + // test/test1/439972354120482843/66/2023-03-09/CDC000001.csv (partitionNum = 66) + // + // and for the non-partitioned table: + // test/test2/439972354120482843/schema.json (partitionNum = -1) + // test/test2/439972354120482843/2023-03-09/CDC000001.csv (partitionNum = 0) + // test/test2/439972354120482843/2023-03-09/CDC000002.csv (partitionNum = 0) + // + // the DDL event recorded in schema.json should be executed first, then the DML events + // in csv files can be executed. + dmlkey := DmlPathKey{ + SchemaPathKey: schemaKey, + PartitionNum: fakePartitionNumForSchemaFile, + Date: "", + } + if _, ok := c.tableDMLIdxMap[dmlkey]; !ok { + c.tableDMLIdxMap[dmlkey] = fileIndexKeyMap{} + } else { + // duplicate table schema file found, this should not happen. + log.Panic("duplicate schema file found", + zap.String("path", path), zap.Any("tableDef", tableDef), + zap.Any("schemaKey", schemaKey), zap.Any("dmlkey", dmlkey)) + } + return nil +} diff --git a/cmd/storage-consumer/main.go b/cmd/storage-consumer/main.go index 67653b73b7..e040d89a3f 100644 --- a/cmd/storage-consumer/main.go +++ b/cmd/storage-consumer/main.go @@ -44,7 +44,22 @@ var ( timezone string ) +func isTestingProcess() bool { + if strings.HasSuffix(os.Args[0], ".test") { + return true + } + for _, arg := range os.Args[1:] { + if strings.HasPrefix(arg, "-test.") { + return true + } + } + return false +} + func init() { + if isTestingProcess() { + return + } version.LogVersionInfo("storage consumer") flag.StringVar(&upstreamURIStr, "upstream-uri", "", "storage uri") flag.StringVar(&downstreamURIStr, "downstream-uri", "", "downstream sink uri") diff --git a/cmd/storage-consumer/path_key.go b/cmd/storage-consumer/path_key.go new file mode 100644 index 0000000000..2e30877a19 --- /dev/null +++ b/cmd/storage-consumer/path_key.go @@ -0,0 +1,308 @@ +// Copyright 2026 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package main + +import ( + "fmt" + "regexp" + "strconv" + "strings" + + "github.com/pingcap/ticdc/pkg/common" + "github.com/pingcap/ticdc/pkg/config" +) + +const minFileNamePrefixLen = 3 + config.MinFileIndexWidth + +var schemaRE = regexp.MustCompile(`meta/schema_\d+_\d{10}\.json$`) + +// SchemaPathKey is the key of schema path for storage consumer. +type SchemaPathKey struct { + Schema string + Table string + TableVersion uint64 +} + +// GetKey returns the key of schema path. +func (s *SchemaPathKey) GetKey() string { + return common.QuoteSchema(s.Schema, s.Table) +} + +// ParseSchemaFilePath parses the schema file path and returns the checksum. +func (s *SchemaPathKey) ParseSchemaFilePath(filePath string) (uint32, error) { + segments := strings.Split(filePath, "/") + if len(segments) != 3 && len(segments) != 4 { + return 0, fmt.Errorf("cannot match schema path pattern for %s", filePath) + } + if segments[len(segments)-2] != "meta" { + return 0, fmt.Errorf("cannot match schema path pattern for %s", filePath) + } + + tableVersion, checksum, err := parseSchemaFileName(segments[len(segments)-1]) + if err != nil { + return 0, err + } + + schema := segments[0] + table := "" + if len(segments) == 4 { + table = segments[1] + } + + *s = SchemaPathKey{ + Schema: schema, + Table: table, + TableVersion: tableVersion, + } + return checksum, nil +} + +type FileIndexKey struct { + DispatcherID string + EnableTableAcrossNodes bool +} + +type FileIndex struct { + FileIndexKey + Idx uint64 +} + +// DmlPathKey is the key of dml path for storage consumer. +type DmlPathKey struct { + SchemaPathKey + PartitionNum int64 + Date string +} + +// GenerateDMLFilePath generates the dml file path. +func (d *DmlPathKey) GenerateDMLFilePath( + fileIndex *FileIndex, extension string, fileIndexWidth int, +) string { + var elems []string + + elems = append(elems, d.Schema) + elems = append(elems, d.Table) + elems = append(elems, fmt.Sprintf("%d", d.TableVersion)) + + if d.PartitionNum != 0 { + elems = append(elems, fmt.Sprintf("%d", d.PartitionNum)) + } + if len(d.Date) != 0 { + elems = append(elems, d.Date) + } + elems = append(elems, generateDataFileName(fileIndex.EnableTableAcrossNodes, fileIndex.DispatcherID, fileIndex.Idx, extension, fileIndexWidth)) + + return strings.Join(elems, "/") +} + +// ParseIndexFilePath parses the index file path and returns dispatcher ID. +func (d *DmlPathKey) ParseIndexFilePath(dateSeparator, filePath string) (string, error) { + segments := strings.Split(filePath, "/") + if len(segments) < 5 || len(segments) > 7 { + return "", fmt.Errorf("cannot match dml path pattern for %s", filePath) + } + if segments[len(segments)-2] != "meta" { + return "", fmt.Errorf("cannot match dml path pattern for %s", filePath) + } + + version, err := strconv.ParseUint(segments[2], 10, 64) + if err != nil { + return "", err + } + + dispatcherID, err := parseDispatcherIDFromIndexFileName(segments[len(segments)-1]) + if err != nil { + return "", err + } + + extraSegments := segments[3 : len(segments)-2] + partitionNum, date, err := parseDMLPathExtraSegments(dateSeparator, extraSegments, filePath) + if err != nil { + return "", err + } + + *d = DmlPathKey{ + SchemaPathKey: SchemaPathKey{ + Schema: segments[0], + Table: segments[1], + TableVersion: version, + }, + PartitionNum: partitionNum, + Date: date, + } + + return dispatcherID, nil +} + +func parseSchemaFileName(fileName string) (uint64, uint32, error) { + parts := strings.Split(fileName, "_") + if len(parts) < 3 { + return 0, 0, fmt.Errorf("cannot match schema file pattern for %s", fileName) + } + + checksumString := strings.TrimSuffix(parts[len(parts)-1], ".json") + checksum, err := strconv.ParseUint(checksumString, 10, 32) + if err != nil { + return 0, 0, err + } + + tableVersionString := parts[len(parts)-2] + tableVersion, err := strconv.ParseUint(tableVersionString, 10, 64) + if err != nil { + return 0, 0, err + } + return tableVersion, uint32(checksum), nil +} + +func parseDMLPathExtraSegments( + dateSeparator string, + extraSegments []string, + path string, +) (int64, string, error) { + parsePartition := func(value string) (int64, error) { + partitionNum, err := strconv.ParseInt(value, 10, 64) + if err != nil { + return 0, fmt.Errorf("cannot match dml path pattern for %s", path) + } + return partitionNum, nil + } + + validateDate := func(value string) error { + if !isValidDateSegment(dateSeparator, value) { + return fmt.Errorf("cannot match dml path pattern for %s", path) + } + return nil + } + + switch len(extraSegments) { + case 0: + if dateSeparator != config.DateSeparatorNone.String() { + return 0, "", fmt.Errorf("cannot match dml path pattern for %s", path) + } + return 0, "", nil + case 1: + segment := extraSegments[0] + if dateSeparator != config.DateSeparatorNone.String() && isValidDateSegment(dateSeparator, segment) { + return 0, segment, nil + } + partitionNum, err := parsePartition(segment) + if err != nil { + return 0, "", err + } + return partitionNum, "", nil + case 2: + partitionNum, err := parsePartition(extraSegments[0]) + if err != nil { + return 0, "", err + } + if err := validateDate(extraSegments[1]); err != nil { + return 0, "", err + } + return partitionNum, extraSegments[1], nil + default: + return 0, "", fmt.Errorf("cannot match dml path pattern for %s", path) + } +} + +func parseDispatcherIDFromIndexFileName(fileName string) (string, error) { + if !strings.HasPrefix(fileName, "CDC") || !strings.HasSuffix(fileName, ".index") { + return "", fmt.Errorf("cannot match dml path pattern for %s", fileName) + } + + fileName = strings.TrimSuffix(fileName, ".index") + if fileName == "CDC" { + return "", nil + } + + if !strings.HasPrefix(fileName, "CDC_") { + return "", fmt.Errorf("cannot match dml path pattern for %s", fileName) + } + dispatcherID := strings.TrimPrefix(fileName, "CDC_") + if dispatcherID == "" { + return "", fmt.Errorf("cannot match dml path pattern for %s", fileName) + } + return dispatcherID, nil +} + +func isValidDateSegment(dateSeparator, value string) bool { + switch dateSeparator { + case config.DateSeparatorYear.String(): + return len(value) == 4 && isNumberString(value) + case config.DateSeparatorMonth.String(): + return len(value) == 7 && + value[4] == '-' && + isNumberString(value[:4]) && + isNumberString(value[5:]) + case config.DateSeparatorDay.String(): + return len(value) == 10 && + value[4] == '-' && + value[7] == '-' && + isNumberString(value[:4]) && + isNumberString(value[5:7]) && + isNumberString(value[8:]) + default: + return false + } +} + +func isNumberString(value string) bool { + if value == "" { + return false + } + for _, ch := range value { + if ch < '0' || ch > '9' { + return false + } + } + return true +} + +// IsSchemaFile checks whether the file is a schema file. +func IsSchemaFile(filePath string) bool { + return schemaRE.MatchString(filePath) +} + +func generateDataFileName(enableTableAcrossNodes bool, dispatcherID string, index uint64, extension string, fileIndexWidth int) string { + indexFmt := "%0" + strconv.Itoa(fileIndexWidth) + "d" + if enableTableAcrossNodes { + return fmt.Sprintf("CDC_%s_"+indexFmt+"%s", dispatcherID, index, extension) + } + return fmt.Sprintf("CDC"+indexFmt+"%s", index, extension) +} + +func FetchIndexFromFileName(fileName string, extension string) (uint64, error) { + if len(fileName) < minFileNamePrefixLen+len(extension) || + !strings.HasPrefix(fileName, "CDC") || + !strings.HasSuffix(fileName, extension) { + return 0, fmt.Errorf("filename in storage sink is invalid: %s", fileName) + } + + fileName = strings.TrimSuffix(fileName, extension) + indexPart := strings.TrimPrefix(fileName, "CDC") + + if strings.HasPrefix(indexPart, "_") { + trimmed := strings.TrimPrefix(indexPart, "_") + separatorIndex := strings.LastIndex(trimmed, "_") + if separatorIndex <= 0 || separatorIndex >= len(trimmed)-1 { + return 0, fmt.Errorf("filename in storage sink is invalid: %s", fileName) + } + indexPart = trimmed[separatorIndex+1:] + } + + if len(indexPart) < config.MinFileIndexWidth || !isNumberString(indexPart) { + return 0, fmt.Errorf("filename in storage sink is invalid: %s", fileName) + } + + return strconv.ParseUint(indexPart, 10, 64) +} diff --git a/cmd/storage-consumer/path_key_test.go b/cmd/storage-consumer/path_key_test.go new file mode 100644 index 0000000000..e7b237a3b5 --- /dev/null +++ b/cmd/storage-consumer/path_key_test.go @@ -0,0 +1,242 @@ +// Copyright 2026 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package main + +import ( + "fmt" + "testing" + + "github.com/pingcap/ticdc/pkg/common" + "github.com/stretchr/testify/require" +) + +func TestSchemaPathKey(t *testing.T) { + t.Parallel() + + testCases := []struct { + path string + schemaKey SchemaPathKey + checksum uint32 + }{ + { + path: "test_schema/meta/schema_1_0000000002.json", + schemaKey: SchemaPathKey{ + Schema: "test_schema", + Table: "", + TableVersion: 1, + }, + checksum: 2, + }, + { + path: "test_schema/test_table/meta/schema_11_0000000022.json", + schemaKey: SchemaPathKey{ + Schema: "test_schema", + Table: "test_table", + TableVersion: 11, + }, + checksum: 22, + }, + } + + for _, tc := range testCases { + var schemaKey SchemaPathKey + checksum, err := schemaKey.ParseSchemaFilePath(tc.path) + require.NoError(t, err) + require.Equal(t, tc.schemaKey, schemaKey) + require.Equal(t, tc.checksum, checksum) + } +} + +func TestDmlPathKey(t *testing.T) { + t.Parallel() + + dispatcherID := common.NewDispatcherID() + testCases := []struct { + name string + dateSeparator string + dispatcherID string + index uint64 + fileIndexWidth int + extension string + path string + indexPath string + dmlKey DmlPathKey + }{ + { + name: "day-with-dispatcher", + dateSeparator: "day", + dispatcherID: dispatcherID.String(), + index: 10, + fileIndexWidth: 20, + extension: ".csv", + path: fmt.Sprintf("schema1/table1/123456/2023-05-09/CDC_%s_00000000000000000010.csv", dispatcherID.String()), + indexPath: fmt.Sprintf("schema1/table1/123456/2023-05-09/meta/CDC_%s.index", dispatcherID.String()), + dmlKey: DmlPathKey{ + SchemaPathKey: SchemaPathKey{ + Schema: "schema1", + Table: "table1", + TableVersion: 123456, + }, + PartitionNum: 0, + Date: "2023-05-09", + }, + }, + { + name: "none-without-dispatcher", + dateSeparator: "none", + dispatcherID: "", + index: 12, + fileIndexWidth: 6, + extension: ".json", + path: "schema1/table1/123456/CDC000012.json", + indexPath: "schema1/table1/123456/meta/CDC.index", + dmlKey: DmlPathKey{ + SchemaPathKey: SchemaPathKey{ + Schema: "schema1", + Table: "table1", + TableVersion: 123456, + }, + PartitionNum: 0, + Date: "", + }, + }, + { + name: "none-with-partition-and-hyphen-dispatcher", + dateSeparator: "none", + dispatcherID: "dispatcher-1", + index: 3, + fileIndexWidth: 6, + extension: ".json", + path: "schema2/table2/88/6/CDC_dispatcher-1_000003.json", + indexPath: "schema2/table2/88/6/meta/CDC_dispatcher-1.index", + dmlKey: DmlPathKey{ + SchemaPathKey: SchemaPathKey{ + Schema: "schema2", + Table: "table2", + TableVersion: 88, + }, + PartitionNum: 6, + Date: "", + }, + }, + { + name: "year-with-partition", + dateSeparator: "year", + dispatcherID: "", + index: 9, + fileIndexWidth: 6, + extension: ".json", + path: "schema3/table3/66/4/2025/CDC000009.json", + indexPath: "schema3/table3/66/4/2025/meta/CDC.index", + dmlKey: DmlPathKey{ + SchemaPathKey: SchemaPathKey{ + Schema: "schema3", + Table: "table3", + TableVersion: 66, + }, + PartitionNum: 4, + Date: "2025", + }, + }, + } + + for _, tc := range testCases { + tc := tc + t.Run(tc.name, func(t *testing.T) { + var dmlKey DmlPathKey + id, err := dmlKey.ParseIndexFilePath(tc.dateSeparator, tc.indexPath) + require.NoError(t, err) + require.Equal(t, tc.dmlKey, dmlKey) + require.Equal(t, tc.dispatcherID, id) + + fileIndex := &FileIndex{ + FileIndexKey: FileIndexKey{ + DispatcherID: id, + EnableTableAcrossNodes: id != "", + }, + Idx: tc.index, + } + fileName := dmlKey.GenerateDMLFilePath(fileIndex, tc.extension, tc.fileIndexWidth) + require.Equal(t, tc.path, fileName) + }) + } +} + +func TestDmlPathKeyInvalidPath(t *testing.T) { + t.Parallel() + + testCases := []struct { + name string + dateSeparator string + indexPath string + }{ + { + name: "invalid-date-format", + dateSeparator: "day", + indexPath: "schema1/table1/123/2025-02/meta/CDC.index", + }, + { + name: "invalid-index-file-name", + dateSeparator: "day", + indexPath: "schema1/table1/123/2025-02-14/meta/cdc.index", + }, + { + name: "unexpected-extra-segment", + dateSeparator: "none", + indexPath: "schema1/table1/123/1/2025/meta/CDC.index", + }, + } + + for _, tc := range testCases { + var dmlKey DmlPathKey + _, err := dmlKey.ParseIndexFilePath(tc.dateSeparator, tc.indexPath) + require.Error(t, err, tc.name) + } +} + +func TestFetchIndexFromFileName(t *testing.T) { + t.Parallel() + + testCases := []struct { + fileName string + index uint64 + wantErr bool + }{ + {fileName: "CDC000011.json", index: 11}, + {fileName: "CDC1000000.json", index: 1000000}, + {fileName: "CDC_dispatcher-1_000007.json", index: 7}, + {fileName: "CDC1.json", wantErr: true}, + {fileName: "cdc000001.json", wantErr: true}, + {fileName: "CDC000005.xxx", wantErr: true}, + {fileName: "CDChello.json", wantErr: true}, + {fileName: "CDC_dispatcher_1.json", wantErr: true}, + } + + for _, tc := range testCases { + index, err := FetchIndexFromFileName(tc.fileName, ".json") + if tc.wantErr { + require.Error(t, err) + continue + } + require.NoError(t, err) + require.Equal(t, tc.index, index) + } +} + +func TestIsSchemaFile(t *testing.T) { + t.Parallel() + + require.True(t, IsSchemaFile("test/table/meta/schema_12_0000000123.json")) + require.False(t, IsSchemaFile("test/table/meta/CDC000001.json")) +} diff --git a/docs/plans/2026-02-04-cloudstorage-sink-ga-task-breakdown.md b/docs/plans/2026-02-04-cloudstorage-sink-ga-task-breakdown.md new file mode 100644 index 0000000000..800577f4e7 --- /dev/null +++ b/docs/plans/2026-02-04-cloudstorage-sink-ga-task-breakdown.md @@ -0,0 +1,241 @@ +# Cloud Storage Sink GA - Task Breakdown(任务拆分) + +> 文档定位(已整合) +> - 类型:任务清单与执行进度(唯一打勾清单) +> - 总览文档:`storage-sink.md` +> - 说明:任务状态仅在本文维护;`storage-sink.md` 负责总体架构与语义说明。 + +> **范围与口径** +> +> - 本文用于把 `storage-sink.md` 的方案落成可执行工程任务(建议按 PR/里程碑推进)。 +> - **只覆盖 cloud storage sink + 最小 dispatcher glue**。不改其他 sink 语义。 +> - 本文不包含任何与本 GA 无关的方案关键词(例如其他历史方案的术语),避免造成阅读者误解。 + +## Source of truth(唯一基准) + +- 总体方案与约束:`storage-sink.md` +- 任务清单与完成状态:本文 +- 评审与风险清单:`docs/plans/2026-02-14-cloudstorage-sink-spool-review-recommendations.md` + +## 核心不变量(写在任务拆分最前,避免实现偏航) + +1) **DML 仍 await=true(dispatcher 语义不变)** +- dynstream 的 path 仍然会 blocking;区别仅在于 wake 的触发点从“flush 完成”前移到“入 spool 成功(且水位允许)”。 + +2) **two-stage ack(必须)** +- enqueue ack:只用于 wake(解除 dynstream path 阻塞)。 +- flush ack:远端写成功后才 `PostFlush()`,用于推进 table progress / checkpoint。 + +3) **DDL 顺序正确性(必须)** +- 对 DDL(commitTs=T):任何 dispatcher 上 ` 说明:一旦 DML wake 前移,DDL 顺序正确性不再能依赖旧的隐含顺序,因此本任务需要同时落地 `PassBlockEvent + DrainMarker`(否则 DDL 可能错序)。 + +**建议拆成 4 个 PR(按顺序合并;前 2 个 PR 不引入行为变化,后 2 个 PR 才让 early-wake 在 cloudstorage 上真正生效):** + +### PR/Task 1.1:DML two-stage ack glue(新增 enqueue ack hook,不改行为) + +**目标:** +- 不新增新的 DML sink interface(仍使用 `sink.AddDMLEvent(event)`)。 +- 把 wakeCallback 从“flush ack(PostFlush)”解耦出来,迁移到“enqueue ack(PostEnqueue)”上。 +- 为了不影响其他 sink:提供兼容兜底,使得即使 sink 侧暂时不主动触发 `PostEnqueue()`,也不会造成 dynstream path 永久阻塞。 + +**Files(预期):** +- Modify: `pkg/common/event/dml_event.go`(新增 enqueue ack hook:`PostEnqueue()` / `AddPostEnqueueFunc`,名称以实现为准) +- Modify: `downstreamadapter/dispatcher/basic_dispatcher.go`(保持 await=true 不变;wake 从 PostFlush 改为 PostEnqueue) +- Test: `pkg/common/event/dml_event_test.go` +- Test: `downstreamadapter/dispatcher/*_test.go` + +**步骤:** +1) `DMLEvent` 新增 enqueue ack hook(只用于 wake,不用于 checkpoint): + - `AddPostEnqueueFunc(func())` + - `PostEnqueue()` +2) 兼容兜底(关键点):在 `DMLEvent.PostFlush()` 的实现中**保证会触发一次** `PostEnqueue()`(若尚未触发),从而保证: + - 其他 sink 仍只需要在远端成功后调用 `PostFlush()`,wake 时机仍与历史一致(无行为变化); + - cloudstorage sink 后续可以在“入 spool 成功且水位允许”时提前调用 `PostEnqueue()` 来真正实现 early-wake。 +3) dispatcher 改造(DML): + - DML 仍照常 `TableProgress.Add(event)`(flush ack / checkpoint 语义不变); + - wakeCallback(带 `sync.Once` 去重)注册到 `AddPostEnqueueFunc`,不再依赖 `PostFlush()`。 + +**验证:** +- Run: `make unit_test_pkg PKG=./pkg/common/event/...` +- Run: `make unit_test_pkg PKG=./downstreamadapter/dispatcher/...` + +### PR/Task 1.2:DDL glue(新增 PassBlockEvent 接口 + dispatcher 调用点,不改行为) + +**目标:** +- sink interface 增加 `PassBlockEvent(event BlockEvent) error`,让 sink 能拿到 DDL 信息并执行 per-dispatcher drain。 +- dispatcher 增加调用点与调用时机(语义见设计文档),但先允许 sink 侧默认 no-op,因此本 PR 不引入行为变化。 + +**Files(预期):** +- Modify: `downstreamadapter/sink/sink.go`(新增 `PassBlockEvent` 方法) +- Modify: `downstreamadapter/sink/*`(为所有 sink 增加默认实现:no-op return nil;cloudstorage 在 1.3 实现真实逻辑) +- Modify: `downstreamadapter/dispatcher/basic_dispatcher.go`(DDL 路径在 barrier 前/WriteBlockEvent 前调用 `PassBlockEvent` 并等待返回) +- Test: `downstreamadapter/dispatcher/*_test.go` + +**dispatcher 调用时机(必须写进实现与测试里):** +- barrier DDL:在 `reportBlockedEventToMaintainer()` **之前**调用并等待返回; +- 非 barrier DDL:在 `WriteBlockEvent()` **之前**调用并等待返回; +- 语义:返回成功表示该 dispatcher 的 ` 文档定位(已整合) +> - 类型:代码评审清单(优先级/风险/测试建议) +> - 总览文档:`storage-sink.md` +> - 说明:本文件只维护 review 视角,不重复维护实现方案与任务状态。 + +## 目标与范围 + +- 目标:给出一份可执行、可验收的 review 清单,覆盖 cloud storage sink 的 spool + task pipeline + DDL drain + 观测性改造。 +- 范围: + - `downstreamadapter/sink/cloudstorage/*` + - `downstreamadapter/sink/cloudstorage/spool/*` + - `downstreamadapter/dispatcher/basic_dispatcher.go` + - `pkg/common/event/dml_event.go` + - `pkg/sink/cloudstorage/path.go` + - `downstreamadapter/sink/metrics/cloudstorage.go` + +## 优先级与关键点 + +### P0(必须优先通过) + +1. **语义锚点:checkpoint 不提前** + - 关键点:`PostEnqueue` 仅用于 wake;`PostFlush` 仍只在远端写成功后触发。 + - 结论标准:压测下 checkpoint 推进速率不快于远端真正落盘速率。 + +2. **DDL 顺序正确性** + - 关键点:`PassBlockEvent` + `DrainMarker` 必须覆盖 encoding queue → spool → writer flush 全链路。 + - 结论标准:任何 DDL(commitTs=T) 写 schema 前,` output shard 路由缓存 | 锁定“同 dispatcher 同 shard”与 cache 行为 | 保持为快速单测,防止后续回归为重复 hash | +| `TestTaskIndexerRouteOutputShardStable` | 验证 route 范围与稳定性 | 保证路由边界正确 | 可补充随机 dispatcher 批量覆盖 | +| `TestEncodingGroupRouteByDispatcher` | 验证 encoding 后按 output shard 消费且保序 | 保障 defragmenter 移除后的核心语义 | 增加多 shard 并发 case | +| `TestWriterDrainMarker` | 验证 DDL drain 会先 flush 再 ack marker | 直接覆盖 DDL 顺序关键路径 | 增加 marker 前后混合多 table case | +| `TestPassBlockEventRecordDrainDurationMetric` | 验证 DDL drain duration 指标打点 | 保障线上可观测性基础 | 可补充慢路径阈值告警回归 | +| `TestPathStateCleanup` | 验证 path state TTL 回收与状态复用 | 防止长期运行 map 膨胀 | 增加多 dispatcher + 多版本 churn case | +| `spool/manager` 现有单测 | 验证 spill/load/release/wake 抑制恢复 | 锁定 spool 正确性 | 建议补多 segment rotate + 大 payload case | + +## 建议评审顺序 + +1. 先看语义锚点:`pkg/common/event/dml_event.go`、`downstreamadapter/dispatcher/basic_dispatcher.go`。 +2. 再看主链路:`downstreamadapter/sink/cloudstorage/dml_writers.go` → `encoding_group.go` → `writer.go`。 +3. 再看资源与状态:`downstreamadapter/sink/cloudstorage/spool/manager.go`、`pkg/sink/cloudstorage/path.go`。 +4. 最后看观测性:`downstreamadapter/sink/metrics/cloudstorage.go` 与对应打点调用点。 diff --git a/downstreamadapter/dispatcher/basic_dispatcher.go b/downstreamadapter/dispatcher/basic_dispatcher.go index 66a94e5e04..eaaa26ad6c 100644 --- a/downstreamadapter/dispatcher/basic_dispatcher.go +++ b/downstreamadapter/dispatcher/basic_dispatcher.go @@ -479,13 +479,8 @@ func (d *BasicDispatcher) handleEvents(dispatcherEvents []DispatcherEvent, wakeC block = true dml.ReplicatingTs = d.creationPDTs - dml.AddPostFlushFunc(func() { - // Considering dml event in sink may be written to downstream not in order, - // thus, we use tableProgress.Empty() to ensure these events are flushed to downstream completely - // and wake dynamic stream to handle the next events. - if d.tableProgress.Empty() { - dmlWakeOnce.Do(wakeCallback) - } + dml.AddPostEnqueueFunc(func() { + dmlWakeOnce.Do(wakeCallback) }) dmlEvents = append(dmlEvents, dml) case commonEvent.TypeDDLEvent: @@ -736,7 +731,15 @@ func (d *BasicDispatcher) DealWithBlockEvent(event commonEvent.BlockEvent) { // we track it as a pending schedule-related event until the maintainer ACKs it. d.pendingACKCount.Add(1) } - err := d.AddBlockEventToSink(event) + err := d.sink.PassBlockEvent(event) + if err != nil { + if needsScheduleACKTracking { + d.pendingACKCount.Add(-1) + } + d.HandleError(err) + return + } + err = d.AddBlockEventToSink(event) if err != nil { if needsScheduleACKTracking { d.pendingACKCount.Add(-1) @@ -804,8 +807,14 @@ func (d *BasicDispatcher) DealWithBlockEvent(event commonEvent.BlockEvent) { d.holdBlockEvent(event) return } - - d.reportBlockedEventToMaintainer(event) + d.sharedInfo.GetBlockEventExecutor().Submit(d, func() { + err := d.sink.PassBlockEvent(event) + if err != nil { + d.HandleError(err) + return + } + d.reportBlockedEventToMaintainer(event) + }) } // dealing with events which update schema ids diff --git a/downstreamadapter/sink/blackhole/sink.go b/downstreamadapter/sink/blackhole/sink.go index c07e886419..a21f95a158 100644 --- a/downstreamadapter/sink/blackhole/sink.go +++ b/downstreamadapter/sink/blackhole/sink.go @@ -69,6 +69,10 @@ func (s *sink) WriteBlockEvent(event commonEvent.BlockEvent) error { return nil } +func (s *sink) PassBlockEvent(_ commonEvent.BlockEvent) error { + return nil +} + func (s *sink) AddCheckpointTs(ts uint64) { log.Debug("BlackHoleSink: Checkpoint Ts Event", zap.Uint64("ts", ts)) } diff --git a/downstreamadapter/sink/cloudstorage/defragmenter.go b/downstreamadapter/sink/cloudstorage/defragmenter.go deleted file mode 100644 index 6f14990e71..0000000000 --- a/downstreamadapter/sink/cloudstorage/defragmenter.go +++ /dev/null @@ -1,132 +0,0 @@ -// Copyright 2025 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package cloudstorage - -import ( - "context" - - commonEvent "github.com/pingcap/ticdc/pkg/common/event" - "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/hash" - "github.com/pingcap/ticdc/pkg/sink/cloudstorage" - "github.com/pingcap/ticdc/pkg/sink/codec/common" - "github.com/pingcap/ticdc/utils/chann" -) - -// eventFragment is used to attach a sequence number to TxnCallbackableEvent. -type eventFragment struct { - event *commonEvent.DMLEvent - versionedTable cloudstorage.VersionedTableName - - // The sequence number is mainly useful for TxnCallbackableEvent defragmentation. - // e.g. TxnCallbackableEvent 1~5 are dispatched to a group of encoding workers, but the - // encoding completion time varies. Let's say the final completion sequence are 1,3,2,5,4, - // we can use the sequence numbers to do defragmentation so that the events can arrive - // at dmlWriters sequentially. - seqNumber uint64 - // encodedMsgs denote the encoded messages after the event is handled in encodingWorker. - encodedMsgs []*common.Message -} - -func newEventFragment(seq uint64, version cloudstorage.VersionedTableName, event *commonEvent.DMLEvent) eventFragment { - return eventFragment{ - seqNumber: seq, - versionedTable: version, - event: event, - } -} - -// defragmenter is used to handle event fragments which can be registered -// out of order. -type defragmenter struct { - lastDispatchedSeq uint64 - future map[uint64]eventFragment - inputCh <-chan eventFragment - outputChs []*chann.DrainableChann[eventFragment] - hasher *hash.PositionInertia -} - -func newDefragmenter( - inputCh <-chan eventFragment, - outputChs []*chann.DrainableChann[eventFragment], -) *defragmenter { - return &defragmenter{ - future: make(map[uint64]eventFragment), - inputCh: inputCh, - outputChs: outputChs, - hasher: hash.NewPositionInertia(), - } -} - -func (d *defragmenter) Run(ctx context.Context) error { - defer d.close() - for { - select { - case <-ctx.Done(): - d.future = nil - return errors.Trace(ctx.Err()) - case frag, ok := <-d.inputCh: - if !ok { - return nil - } - // check whether to write messages to output channel right now - next := d.lastDispatchedSeq + 1 - if frag.seqNumber == next { - d.writeMsgsConsecutive(ctx, frag) - } else if frag.seqNumber > next { - d.future[frag.seqNumber] = frag - } else { - return nil - } - } - } -} - -func (d *defragmenter) writeMsgsConsecutive( - ctx context.Context, - start eventFragment, -) { - d.dispatchFragToDMLWorker(start) - - // try to dispatch more fragments to DML workers - for { - select { - case <-ctx.Done(): - return - default: - } - next := d.lastDispatchedSeq + 1 - if frag, ok := d.future[next]; ok { - delete(d.future, next) - d.dispatchFragToDMLWorker(frag) - } else { - return - } - } -} - -func (d *defragmenter) dispatchFragToDMLWorker(frag eventFragment) { - tableName := frag.versionedTable.TableNameWithPhysicTableID - d.hasher.Reset() - d.hasher.Write([]byte(tableName.Schema), []byte(tableName.Table)) - workerID := d.hasher.Sum32() % uint32(len(d.outputChs)) - d.outputChs[workerID].In() <- frag - d.lastDispatchedSeq = frag.seqNumber -} - -func (d *defragmenter) close() { - for _, ch := range d.outputChs { - ch.CloseAndDrain() - } -} diff --git a/downstreamadapter/sink/cloudstorage/defragmenter_test.go b/downstreamadapter/sink/cloudstorage/defragmenter_test.go deleted file mode 100644 index 4659b978b9..0000000000 --- a/downstreamadapter/sink/cloudstorage/defragmenter_test.go +++ /dev/null @@ -1,136 +0,0 @@ -// Copyright 2022 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package cloudstorage - -import ( - "context" - "math/rand" - "net/url" - "strconv" - "testing" - "time" - - "github.com/pingcap/ticdc/downstreamadapter/sink/helper" - "github.com/pingcap/ticdc/pkg/common" - commonEvent "github.com/pingcap/ticdc/pkg/common/event" - "github.com/pingcap/ticdc/pkg/config" - "github.com/pingcap/ticdc/pkg/sink/cloudstorage" - "github.com/pingcap/ticdc/pkg/sink/codec" - "github.com/pingcap/ticdc/utils/chann" - timodel "github.com/pingcap/tidb/pkg/meta/model" - "github.com/pingcap/tidb/pkg/parser/ast" - "github.com/pingcap/tidb/pkg/parser/mysql" - "github.com/pingcap/tidb/pkg/parser/types" - "github.com/pingcap/tidb/pkg/util/chunk" - "github.com/stretchr/testify/require" - "golang.org/x/sync/errgroup" -) - -func TestDeframenter(t *testing.T) { - t.Parallel() - - ctx, cancel := context.WithCancel(context.Background()) - eg, egCtx := errgroup.WithContext(ctx) - - inputCh := make(chan eventFragment) - outputCh := chann.NewAutoDrainChann[eventFragment]() - defrag := newDefragmenter(inputCh, []*chann.DrainableChann[eventFragment]{outputCh}) - eg.Go(func() error { - return defrag.Run(egCtx) - }) - - uri := "file:///tmp/test" - txnCnt := 50 - sinkURI, err := url.Parse(uri) - require.Nil(t, err) - - replicaConfig := config.GetDefaultReplicaConfig() - changefeedID := common.NewChangefeedID4Test("test", "table1") - encoderConfig, err := helper.GetEncoderConfig(changefeedID, sinkURI, config.ProtocolCsv, - replicaConfig.Sink, config.DefaultMaxMessageBytes) - require.NoError(t, err) - - var seqNumbers []uint64 - for i := 0; i < txnCnt; i++ { - seqNumbers = append(seqNumbers, uint64(i+1)) - } - rand.New(rand.NewSource(time.Now().UnixNano())) - rand.Shuffle(len(seqNumbers), func(i, j int) { - seqNumbers[i], seqNumbers[j] = seqNumbers[j], seqNumbers[i] - }) - - tidbTableInfo := &timodel.TableInfo{ - ID: 100, - Name: ast.NewCIStr("table1"), - Columns: []*timodel.ColumnInfo{ - {ID: 1, Name: ast.NewCIStr("c1"), FieldType: *types.NewFieldType(mysql.TypeLong)}, - {ID: 2, Name: ast.NewCIStr("c2"), FieldType: *types.NewFieldType(mysql.TypeVarchar)}, - }, - } - tableInfo := common.WrapTableInfo("test", tidbTableInfo) - for i := 0; i < txnCnt; i++ { - go func(seq uint64) { - frag := eventFragment{ - versionedTable: cloudstorage.VersionedTableName{ - TableNameWithPhysicTableID: common.TableName{ - Schema: "test", - Table: "table1", - TableID: 100, - }, - }, - seqNumber: seq, - } - rand.New(rand.NewSource(time.Now().UnixNano())) - n := 1 + rand.Intn(1000) - vals := make([]interface{}, 0, n) - for j := 0; j < n; j++ { - vals = append(vals, j+1, "hello world") - } - frag.event = &commonEvent.DMLEvent{ - PhysicalTableID: 100, - TableInfo: tableInfo, - TableInfoVersion: 1, - Rows: chunk.MutRowFromValues(vals...).ToRow().Chunk(), - } - encoder, err := codec.NewTxnEventEncoder(encoderConfig) - require.Nil(t, err) - err = encoder.AppendTxnEvent(frag.event) - require.NoError(t, err) - frag.encodedMsgs = encoder.Build() - - for _, msg := range frag.encodedMsgs { - msg.Key = []byte(strconv.Itoa(int(seq))) - } - inputCh <- frag - }(uint64(i + 1)) - } - - prevSeq := 0 -LOOP: - for { - select { - case frag := <-outputCh.Out(): - for _, msg := range frag.encodedMsgs { - curSeq, err := strconv.Atoi(string(msg.Key)) - require.Nil(t, err) - require.GreaterOrEqual(t, curSeq, prevSeq) - prevSeq = curSeq - } - case <-time.After(5 * time.Second): - break LOOP - } - } - cancel() - require.ErrorIs(t, eg.Wait(), context.Canceled) -} diff --git a/downstreamadapter/sink/cloudstorage/dml_writers.go b/downstreamadapter/sink/cloudstorage/dml_writers.go index bdc442bda2..f14951f69e 100644 --- a/downstreamadapter/sink/cloudstorage/dml_writers.go +++ b/downstreamadapter/sink/cloudstorage/dml_writers.go @@ -15,94 +15,151 @@ package cloudstorage import ( "context" - "sync/atomic" + "time" + "github.com/pingcap/ticdc/downstreamadapter/sink/cloudstorage/spool" + sinkmetrics "github.com/pingcap/ticdc/downstreamadapter/sink/metrics" commonType "github.com/pingcap/ticdc/pkg/common" commonEvent "github.com/pingcap/ticdc/pkg/common/event" + "github.com/pingcap/ticdc/pkg/errors" "github.com/pingcap/ticdc/pkg/metrics" - "github.com/pingcap/ticdc/pkg/sink/cloudstorage" + pkgcloudstorage "github.com/pingcap/ticdc/pkg/sink/cloudstorage" "github.com/pingcap/ticdc/pkg/sink/codec/common" "github.com/pingcap/ticdc/utils/chann" "github.com/pingcap/tidb/br/pkg/storage" "golang.org/x/sync/errgroup" ) -// dmlWriters denotes a worker responsible for writing messages to cloud storage. +// dmlWriters coordinates encoding and output shard writers. type dmlWriters struct { + ctx context.Context changefeedID commonType.ChangeFeedID statistics *metrics.Statistics - // msgCh is a channel to hold eventFragment. - // The caller of WriteEvents will write eventFragment to msgCh and - // the encodingWorkers will read eventFragment from msgCh to encode events. - msgCh *chann.UnlimitedChannel[eventFragment, any] - encodeGroup *encodingGroup + // msgCh is a channel to hold task. + // The caller of WriteEvents will write tasks to msgCh and + // encoding pipelines will read tasks from msgCh to encode events. + msgCh *chann.UnlimitedChannel[*task, any] - // defragmenter is used to defragment the out-of-order encoded messages and - // sends encoded messages to individual dmlWorkers. - defragmenter *defragmenter + encodeGroup *encodingGroup writers []*writer - - // last sequence number - lastSeqNum uint64 + spool *spool.Manager } func newDMLWriters( + ctx context.Context, changefeedID commonType.ChangeFeedID, storage storage.ExternalStorage, - config *cloudstorage.Config, + config *pkgcloudstorage.Config, encoderConfig *common.Config, extension string, statistics *metrics.Statistics, -) *dmlWriters { - messageCh := chann.NewUnlimitedChannelDefault[eventFragment]() - encodedOutCh := make(chan eventFragment, defaultChannelSize) - encoderGroup := newEncodingGroup(changefeedID, encoderConfig, defaultEncodingConcurrency, messageCh, encodedOutCh) +) (*dmlWriters, error) { + spoolManager, err := spool.New(changefeedID, config.SpoolDiskQuota, nil) + if err != nil { + return nil, errors.Trace(err) + } + + messageCh := chann.NewUnlimitedChannelDefault[*task]() + encoderGroup := newEncodingGroup( + changefeedID, + encoderConfig, + defaultEncodingConcurrency, + config.WorkerCount, + ) writers := make([]*writer, config.WorkerCount) - writerInputChs := make([]*chann.DrainableChann[eventFragment], config.WorkerCount) for i := 0; i < config.WorkerCount; i++ { - inputCh := chann.NewAutoDrainChann[eventFragment]() - writerInputChs[i] = inputCh - writers[i] = newWriter(i, changefeedID, storage, config, extension, inputCh, statistics) + writers[i] = newWriter(i, changefeedID, storage, config, extension, statistics, spoolManager) } return &dmlWriters{ + ctx: ctx, changefeedID: changefeedID, statistics: statistics, msgCh: messageCh, - encodeGroup: encoderGroup, - defragmenter: newDefragmenter(encodedOutCh, writerInputChs), writers: writers, - } + spool: spoolManager, + }, nil } func (d *dmlWriters) Run(ctx context.Context) error { eg, ctx := errgroup.WithContext(ctx) eg.Go(func() error { - return d.encodeGroup.Run(ctx) + <-ctx.Done() + d.msgCh.Close() + return nil }) eg.Go(func() error { - return d.defragmenter.Run(ctx) + return d.submitTaskToEncoder(ctx) }) - for i := 0; i < len(d.writers); i++ { + eg.Go(func() error { + return d.encodeGroup.Run(ctx) + }) + + // One dispatcher goroutine per output shard. + // Invariant: each output shard has a single consumer to keep shard ordering. + for i := range d.writers { + index := i eg.Go(func() error { - // UnlimitedChannel will block when there is no event, they cannot dirrectly find ctx.Done() - // Thus, we need to close the channel when the context is done - defer d.encodeGroup.inputCh.Close() - return d.writers[i].Run(ctx) + return d.dispatchTaskToWriter(ctx, index) + }) + } + + for i := range d.writers { + writer := d.writers[i] + eg.Go(func() error { + return writer.Run(ctx) }) } return eg.Wait() } +func (d *dmlWriters) submitTaskToEncoder(ctx context.Context) error { + for { + taskValue, ok := d.msgCh.Get() + if !ok { + return nil + } + if err := d.encodeGroup.Add(ctx, taskValue); err != nil { + return err + } + } +} + +func (d *dmlWriters) dispatchTaskToWriter(ctx context.Context, outputIndex int) error { + writerShard := d.writers[outputIndex] + defer writerShard.closeInput() + + return d.encodeGroup.ConsumeOutputShard(ctx, outputIndex, func(future *taskFuture) error { + // Principle: downstream must observe encode completion before consuming task payload. + if err := future.Ready(ctx); err != nil { + return err + } + + taskValue := future.task + if d.spool != nil && !taskValue.isDrainTask() { + // Invariant: after task is materialized in spool, encodedMsgs can be released + // to cap memory footprint in writer path. + entry, err := d.spool.Enqueue(taskValue.encodedMsgs, taskValue.event.PostEnqueue) + if err != nil { + return err + } + taskValue.spoolEntry = entry + taskValue.encodedMsgs = nil + } + + return writerShard.enqueueTask(ctx, taskValue) + }) +} + func (d *dmlWriters) AddDMLEvent(event *commonEvent.DMLEvent) { - tbl := cloudstorage.VersionedTableName{ + table := pkgcloudstorage.VersionedTableName{ TableNameWithPhysicTableID: commonType.TableName{ Schema: event.TableInfo.GetSchemaName(), Table: event.TableInfo.GetTableName(), @@ -112,18 +169,44 @@ func (d *dmlWriters) AddDMLEvent(event *commonEvent.DMLEvent) { TableInfoVersion: event.TableInfoVersion, DispatcherID: event.GetDispatcherID(), } - seq := atomic.AddUint64(&d.lastSeqNum, 1) + _ = d.statistics.RecordBatchExecution(func() (int, int64, error) { - // emit a TxnCallbackableEvent encoupled with a sequence number starting from one. - d.msgCh.Push(newEventFragment(seq, tbl, event)) + d.msgCh.Push(newDMLTask(table, event)) return int(event.Len()), event.GetSize(), nil }) } +func (d *dmlWriters) PassBlockEvent(event commonEvent.BlockEvent) error { + if event == nil { + return nil + } + + start := time.Now() + defer sinkmetrics.CloudStorageDDLDrainDurationHistogram.WithLabelValues( + d.changefeedID.Keyspace(), + d.changefeedID.ID().String(), + ).Observe(time.Since(start).Seconds()) + + doneCh := make(chan error, 1) + // Invariant for DDL ordering: + // marker follows the same dispatcher route and is acked only after prior tasks + // in that route are fully drained by writer. + d.msgCh.Push(newDrainTask(event.GetDispatcherID(), event.GetCommitTs(), doneCh)) + + select { + case err := <-doneCh: + return err + case <-d.ctx.Done(): + return errors.Trace(d.ctx.Err()) + } +} + func (d *dmlWriters) close() { d.msgCh.Close() - d.encodeGroup.close() for _, w := range d.writers { w.close() } + if d.spool != nil { + d.spool.Close() + } } diff --git a/downstreamadapter/sink/cloudstorage/encoding_group.go b/downstreamadapter/sink/cloudstorage/encoding_group.go index bcc37663e7..5d9bf9162f 100644 --- a/downstreamadapter/sink/cloudstorage/encoding_group.go +++ b/downstreamadapter/sink/cloudstorage/encoding_group.go @@ -20,8 +20,6 @@ import ( "github.com/pingcap/ticdc/pkg/errors" "github.com/pingcap/ticdc/pkg/sink/codec" "github.com/pingcap/ticdc/pkg/sink/codec/common" - "github.com/pingcap/ticdc/utils/chann" - "go.uber.org/atomic" "golang.org/x/sync/errgroup" ) @@ -34,72 +32,177 @@ type encodingGroup struct { changeFeedID commonType.ChangeFeedID codecConfig *common.Config - concurrency int + concurrency int + outputShards int - inputCh *chann.UnlimitedChannel[eventFragment, any] - outputCh chan<- eventFragment + indexer *taskIndexer - closed *atomic.Bool + inputCh []chan *taskFuture + outputCh []chan *taskFuture } +// newEncodingGroup creates an internal two-queue model: +// 1. inputCh: consumed by encoder shards. +// 2. outputCh: consumed by downstream writer shards. +// +// Invariant: the same taskFuture is inserted into both queues, and writer side +// must call future.Ready() before using encoded payload. func newEncodingGroup( changefeedID commonType.ChangeFeedID, codecConfig *common.Config, concurrency int, - inputCh *chann.UnlimitedChannel[eventFragment, any], - outputCh chan<- eventFragment, + outputShards int, ) *encodingGroup { + if concurrency <= 0 { + concurrency = 1 + } + if outputShards <= 0 { + outputShards = 1 + } + + inputCh := make([]chan *taskFuture, concurrency) + for i := 0; i < concurrency; i++ { + inputCh[i] = make(chan *taskFuture, defaultChannelSize) + } + + outputCh := make([]chan *taskFuture, outputShards) + for i := 0; i < outputShards; i++ { + outputCh[i] = make(chan *taskFuture, defaultChannelSize) + } + return &encodingGroup{ changeFeedID: changefeedID, codecConfig: codecConfig, concurrency: concurrency, + outputShards: outputShards, + indexer: newTaskIndexer(concurrency, outputShards), inputCh: inputCh, outputCh: outputCh, - - closed: atomic.NewBool(false), } } func (eg *encodingGroup) Run(ctx context.Context) error { g, ctx := errgroup.WithContext(ctx) for i := 0; i < eg.concurrency; i++ { + idx := i g.Go(func() error { - return eg.runEncoder(ctx) + return eg.runEncoder(ctx, idx) }) } - return g.Wait() + + err := g.Wait() + for _, outCh := range eg.outputCh { + close(outCh) + } + return err } -func (eg *encodingGroup) runEncoder(ctx context.Context) error { +// runEncoder is the only place that mutates task.encodedMsgs. +// Invariant: each task is encoded at most once. +func (eg *encodingGroup) runEncoder(ctx context.Context, index int) error { encoder, err := codec.NewTxnEventEncoder(eg.codecConfig) if err != nil { - return err + return errors.Trace(err) } - defer eg.closed.Store(true) + + inputCh := eg.inputCh[index] for { select { case <-ctx.Done(): return errors.Trace(ctx.Err()) - default: - frag, ok := eg.inputCh.Get() - if !ok || eg.closed.Load() { + case future, ok := <-inputCh: + if !ok { return nil } - err = encoder.AppendTxnEvent(frag.event) + task := future.task + if task.isDrainTask() { + future.finish(nil) + continue + } + + err = encoder.AppendTxnEvent(task.event) if err != nil { - return err + wrappedErr := errors.Trace(err) + future.finish(wrappedErr) + return wrappedErr } - frag.encodedMsgs = encoder.Build() + task.encodedMsgs = encoder.Build() + future.finish(nil) + } + } +} - select { - case <-ctx.Done(): - return errors.Trace(ctx.Err()) - case eg.outputCh <- frag: +func (eg *encodingGroup) Add(ctx context.Context, task *task) error { + if task == nil { + return errors.New("nil task") + } + + future := newTaskFuture(task) + inputIndex, outputIndex := eg.indexer.next(task.dispatcherID) + // Principle: encoder parallelism and writer ordering are decoupled. + // Input shard can be round-robin; output shard must be dispatcher-stable. + select { + case <-ctx.Done(): + return errors.Trace(ctx.Err()) + case eg.inputCh[inputIndex] <- future: + } + + select { + case <-ctx.Done(): + return errors.Trace(ctx.Err()) + case eg.outputCh[outputIndex] <- future: + } + return nil +} + +func (eg *encodingGroup) ConsumeOutputShard( + ctx context.Context, + index int, + handle func(*taskFuture) error, +) error { + if index < 0 || index >= len(eg.outputCh) { + return errors.Errorf("output index out of range: %d", index) + } + outputCh := eg.outputCh[index] + for { + select { + case <-ctx.Done(): + return errors.Trace(ctx.Err()) + case future, ok := <-outputCh: + if !ok { + return nil + } + if err := handle(future); err != nil { + return err } } } } -func (eg *encodingGroup) close() { - eg.closed.Store(true) +type taskFuture struct { + task *task + done chan struct{} + err error +} + +func newTaskFuture(task *task) *taskFuture { + return &taskFuture{ + task: task, + done: make(chan struct{}), + } +} + +func (f *taskFuture) finish(err error) { + f.err = err + close(f.done) +} + +func (f *taskFuture) Ready(ctx context.Context) error { + select { + case <-ctx.Done(): + return errors.Trace(ctx.Err()) + case <-f.done: + } + // f.err is already wrapped at the origin. + return f.err } diff --git a/downstreamadapter/sink/cloudstorage/encoding_group_test.go b/downstreamadapter/sink/cloudstorage/encoding_group_test.go new file mode 100644 index 0000000000..f51bcd07da --- /dev/null +++ b/downstreamadapter/sink/cloudstorage/encoding_group_test.go @@ -0,0 +1,233 @@ +// Copyright 2025 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package cloudstorage + +import ( + "context" + "net/url" + "testing" + "time" + + "github.com/pingcap/ticdc/downstreamadapter/sink/helper" + commonType "github.com/pingcap/ticdc/pkg/common" + commonEvent "github.com/pingcap/ticdc/pkg/common/event" + "github.com/pingcap/ticdc/pkg/config" + pkgcloudstorage "github.com/pingcap/ticdc/pkg/sink/cloudstorage" + "github.com/pingcap/ticdc/pkg/sink/codec/common" + timodel "github.com/pingcap/tidb/pkg/meta/model" + "github.com/pingcap/tidb/pkg/parser/ast" + "github.com/pingcap/tidb/pkg/parser/mysql" + "github.com/pingcap/tidb/pkg/parser/types" + "github.com/pingcap/tidb/pkg/util/chunk" + "github.com/stretchr/testify/require" + "golang.org/x/sync/errgroup" +) + +func TestTaskIndexerRouteOutputShardCached(t *testing.T) { + t.Parallel() + + indexer := newTaskIndexer(2, 4) + dispatcherID := commonType.NewDispatcherID() + first := indexer.routeOutputIndex(dispatcherID) + for i := 0; i < 32; i++ { + require.Equal(t, first, indexer.routeOutputIndex(dispatcherID)) + } + require.Equal(t, 1, indexer.cachedOutputCount()) +} + +func TestTaskIndexerRouteOutputShardStable(t *testing.T) { + t.Parallel() + + dispatcherA := commonType.NewDispatcherID() + dispatcherB := commonType.NewDispatcherID() + + indexer := newTaskIndexer(2, 4) + shardA := indexer.routeOutputIndex(dispatcherA) + shardB := indexer.routeOutputIndex(dispatcherB) + for shardA == shardB { + dispatcherB = commonType.NewDispatcherID() + shardB = indexer.routeOutputIndex(dispatcherB) + } + require.True(t, shardA >= 0 && shardA < 4) + require.True(t, shardB >= 0 && shardB < 4) +} + +func TestEncodingGroupRouteByDispatcher(t *testing.T) { + t.Parallel() + + ctx, cancel := context.WithCancel(context.Background()) + eg, egCtx := errgroup.WithContext(ctx) + + encoderConfig := newTestTxnEncoderConfig(t) + group := newEncodingGroup(commonType.NewChangefeedID4Test("test", "encoder-group"), encoderConfig, 2, 4) + eg.Go(func() error { + return group.Run(egCtx) + }) + + dispatcherA := commonType.NewDispatcherID() + dispatcherB := commonType.NewDispatcherID() + shardA := group.indexer.routeOutputIndex(dispatcherA) + shardB := group.indexer.routeOutputIndex(dispatcherB) + for shardA == shardB { + dispatcherB = commonType.NewDispatcherID() + shardB = group.indexer.routeOutputIndex(dispatcherB) + } + + receivedA := make(chan []uint64, 1) + receivedB := make(chan []uint64, 1) + eg.Go(func() error { + values := make([]uint64, 0, 5) + err := group.ConsumeOutputShard(ctx, shardA, func(future *taskFuture) error { + if err := future.Ready(ctx); err != nil { + return err + } + if future.task.dispatcherID != dispatcherA { + return nil + } + values = append(values, future.task.marker.commitTs) + if len(values) == 5 { + receivedA <- values + } + return nil + }) + return err + }) + eg.Go(func() error { + values := make([]uint64, 0, 5) + err := group.ConsumeOutputShard(ctx, shardB, func(future *taskFuture) error { + if err := future.Ready(ctx); err != nil { + return err + } + if future.task.dispatcherID != dispatcherB { + return nil + } + values = append(values, future.task.marker.commitTs) + if len(values) == 5 { + receivedB <- values + } + return nil + }) + return err + }) + + for i := uint64(1); i <= 5; i++ { + require.NoError(t, group.Add(ctx, newDrainTask(dispatcherA, i, nil))) + require.NoError(t, group.Add(ctx, newDrainTask(dispatcherB, i, nil))) + } + + var resultA []uint64 + var resultB []uint64 + select { + case resultA = <-receivedA: + case <-time.After(5 * time.Second): + t.Fatal("wait dispatcher a timeout") + } + select { + case resultB = <-receivedB: + case <-time.After(5 * time.Second): + t.Fatal("wait dispatcher b timeout") + } + + require.Equal(t, []uint64{1, 2, 3, 4, 5}, resultA) + require.Equal(t, []uint64{1, 2, 3, 4, 5}, resultB) + + cancel() + require.ErrorIs(t, eg.Wait(), context.Canceled) +} + +func TestEncodingGroupEncodeDMLTask(t *testing.T) { + t.Parallel() + + ctx, cancel := context.WithCancel(context.Background()) + eg, egCtx := errgroup.WithContext(ctx) + + encoderConfig := newTestTxnEncoderConfig(t) + changefeedID := commonType.NewChangefeedID4Test("test", "encoder-group") + group := newEncodingGroup(changefeedID, encoderConfig, 2, 1) + eg.Go(func() error { + return group.Run(egCtx) + }) + + dispatcherID := commonType.NewDispatcherID() + taskValue := newDMLTask( + pkgcloudstorage.VersionedTableName{ + TableNameWithPhysicTableID: commonType.TableName{ + Schema: "test", + Table: "table1", + TableID: 100, + }, + TableInfoVersion: 1, + DispatcherID: dispatcherID, + }, + newTestDMLEvent(dispatcherID, 100), + ) + require.NoError(t, group.Add(ctx, taskValue)) + + done := make(chan struct{}, 1) + eg.Go(func() error { + return group.ConsumeOutputShard(ctx, 0, func(future *taskFuture) error { + if err := future.Ready(ctx); err != nil { + return err + } + require.Equal(t, taskValue, future.task) + done <- struct{}{} + return nil + }) + }) + select { + case <-done: + case <-time.After(5 * time.Second): + t.Fatal("wait dml encode timeout") + } + + cancel() + require.ErrorIs(t, eg.Wait(), context.Canceled) +} + +func newTestTxnEncoderConfig(t *testing.T) *common.Config { + uri := "file:///tmp/test" + sinkURI, err := url.Parse(uri) + require.NoError(t, err) + + replicaConfig := config.GetDefaultReplicaConfig() + changefeedID := commonType.NewChangefeedID4Test("test", "encoder-config") + encoderConfig, err := helper.GetEncoderConfig( + changefeedID, + sinkURI, + config.ProtocolCsv, + replicaConfig.Sink, + config.DefaultMaxMessageBytes, + ) + require.NoError(t, err) + return encoderConfig +} + +func newTestDMLEvent(dispatcherID commonType.DispatcherID, tableID int64) *commonEvent.DMLEvent { + tidbTableInfo := &timodel.TableInfo{ + ID: tableID, + Name: ast.NewCIStr("table1"), + Columns: []*timodel.ColumnInfo{ + {ID: 1, Name: ast.NewCIStr("c1"), FieldType: *types.NewFieldType(mysql.TypeLong)}, + {ID: 2, Name: ast.NewCIStr("c2"), FieldType: *types.NewFieldType(mysql.TypeVarchar)}, + }, + } + tableInfo := commonType.WrapTableInfo("test", tidbTableInfo) + return &commonEvent.DMLEvent{ + DispatcherID: dispatcherID, + PhysicalTableID: tableID, + TableInfo: tableInfo, + TableInfoVersion: 1, + Rows: chunk.MutRowFromValues(1, "hello world").ToRow().Chunk(), + } +} diff --git a/downstreamadapter/sink/cloudstorage/sink.go b/downstreamadapter/sink/cloudstorage/sink.go index cb59ce6953..71d29ac5d0 100644 --- a/downstreamadapter/sink/cloudstorage/sink.go +++ b/downstreamadapter/sink/cloudstorage/sink.go @@ -38,11 +38,9 @@ import ( ) // It will send the events to cloud storage systems. -// Messages are encoded in the specific protocol and then sent to the defragmenter. -// The data flow is as follows: **data** -> encodingWorkers -> defragmenter -> dmlWorkers -> external storage -// The defragmenter will defragment the out-of-order encoded messages and sends encoded -// messages to individual dmlWorkers. -// The dmlWorkers will write the encoded messages to external storage in parallel between different tables. +// Messages are encoded in the specific protocol and routed by dispatcher to shard pipelines. +// The data flow is as follows: **data** -> encoding pipeline -> dispatcher routers -> writer shards -> external storage. +// The writer shards write encoded messages to external storage in parallel between different tables. type sink struct { changefeedID common.ChangeFeedID cfg *cloudstorage.Config @@ -120,13 +118,17 @@ func New( return nil, err } statistics := metrics.NewStatistics(changefeedID, "cloudstorage") + dmlWriters, err := newDMLWriters(ctx, changefeedID, storage, cfg, encoderConfig, ext, statistics) + if err != nil { + return nil, errors.Trace(err) + } return &sink{ changefeedID: changefeedID, sinkURI: sinkURI, cfg: cfg, cleanupJobs: cleanupJobs, storage: storage, - dmlWriters: newDMLWriters(changefeedID, storage, cfg, encoderConfig, ext, statistics), + dmlWriters: dmlWriters, checkpointChan: make(chan uint64, 16), lastSendCheckpointTsTime: time.Now(), outputRawChangeEvent: sinkConfig.CloudStorageConfig.GetOutputRawChangeEvent(), @@ -170,6 +172,13 @@ func (s *sink) AddDMLEvent(event *commonEvent.DMLEvent) { s.dmlWriters.AddDMLEvent(event) } +func (s *sink) PassBlockEvent(event commonEvent.BlockEvent) error { + if event == nil { + return nil + } + return s.dmlWriters.PassBlockEvent(event) +} + func (s *sink) WriteBlockEvent(event commonEvent.BlockEvent) error { var err error switch e := event.(type) { diff --git a/downstreamadapter/sink/cloudstorage/spool/codec.go b/downstreamadapter/sink/cloudstorage/spool/codec.go new file mode 100644 index 0000000000..7be2f09fd6 --- /dev/null +++ b/downstreamadapter/sink/cloudstorage/spool/codec.go @@ -0,0 +1,87 @@ +// Copyright 2025 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package spool + +import ( + "bytes" + "encoding/binary" + + "github.com/pingcap/ticdc/pkg/errors" + codeccommon "github.com/pingcap/ticdc/pkg/sink/codec/common" +) + +func serializeMessages(msgs []*codeccommon.Message) ([]byte, error) { + buf := bytes.NewBuffer(make([]byte, 0, 128)) + if err := binary.Write(buf, binary.LittleEndian, uint32(len(msgs))); err != nil { + return nil, errors.Trace(err) + } + for _, msg := range msgs { + keyLen := uint32(len(msg.Key)) + valueLen := uint32(len(msg.Value)) + rows := uint32(msg.GetRowsCount()) + if err := binary.Write(buf, binary.LittleEndian, keyLen); err != nil { + return nil, errors.Trace(err) + } + if err := binary.Write(buf, binary.LittleEndian, valueLen); err != nil { + return nil, errors.Trace(err) + } + if err := binary.Write(buf, binary.LittleEndian, rows); err != nil { + return nil, errors.Trace(err) + } + if _, err := buf.Write(msg.Key); err != nil { + return nil, errors.Trace(err) + } + if _, err := buf.Write(msg.Value); err != nil { + return nil, errors.Trace(err) + } + } + return buf.Bytes(), nil +} + +func deserializeMessages(data []byte) ([]*codeccommon.Message, error) { + reader := bytes.NewReader(data) + var count uint32 + if err := binary.Read(reader, binary.LittleEndian, &count); err != nil { + return nil, errors.Trace(err) + } + + result := make([]*codeccommon.Message, 0, count) + for i := uint32(0); i < count; i++ { + var keyLen uint32 + var valueLen uint32 + var rows uint32 + if err := binary.Read(reader, binary.LittleEndian, &keyLen); err != nil { + return nil, errors.Trace(err) + } + if err := binary.Read(reader, binary.LittleEndian, &valueLen); err != nil { + return nil, errors.Trace(err) + } + if err := binary.Read(reader, binary.LittleEndian, &rows); err != nil { + return nil, errors.Trace(err) + } + + key := make([]byte, keyLen) + if _, err := reader.Read(key); err != nil { + return nil, errors.Trace(err) + } + value := make([]byte, valueLen) + if _, err := reader.Read(value); err != nil { + return nil, errors.Trace(err) + } + msg := codeccommon.NewMsg(key, value) + msg.SetRowsCount(int(rows)) + result = append(result, msg) + } + return result, nil +} diff --git a/downstreamadapter/sink/cloudstorage/spool/manager.go b/downstreamadapter/sink/cloudstorage/spool/manager.go new file mode 100644 index 0000000000..f1706e1ade --- /dev/null +++ b/downstreamadapter/sink/cloudstorage/spool/manager.go @@ -0,0 +1,492 @@ +// Copyright 2025 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package spool + +import ( + "fmt" + "os" + "path/filepath" + "sync" + + "github.com/pingcap/log" + "github.com/pingcap/ticdc/downstreamadapter/sink/metrics" + commonType "github.com/pingcap/ticdc/pkg/common" + "github.com/pingcap/ticdc/pkg/config" + "github.com/pingcap/ticdc/pkg/errors" + codeccommon "github.com/pingcap/ticdc/pkg/sink/codec/common" + "go.uber.org/zap" +) + +const ( + defaultDirectoryName = "cloudstorage-sink-spool" + + defaultSegmentBytes = int64(64 * 1024 * 1024) + + defaultMemoryRatio = 0.2 + defaultHighWatermarkRatio = 0.8 + defaultLowWatermarkRatio = 0.6 +) + +// Options controls spool behavior. +type Options struct { + RootDir string + + SegmentBytes int64 + + MemoryRatio float64 + HighWatermarkRatio float64 + LowWatermarkRatio float64 +} + +// Manager stores encoded messages in memory first and spills to disk when needed. +type Manager struct { + changefeedID commonType.ChangeFeedID + rootDir string + + quotaBytes int64 + + memoryQuotaBytes int64 + highWatermarkBytes int64 + lowWatermarkBytes int64 + segmentBytes int64 + + metricMemoryBytes interface{ Set(float64) } + metricDiskBytes interface{ Set(float64) } + metricTotalBytes interface{ Set(float64) } + metricWakeSuppressed interface{ Inc() } + metricKeyspace string + metricChangefeedLabel string + + mu sync.Mutex + + memoryBytes int64 + diskBytes int64 + + wakeSuppressed bool + pendingWake []func() + + closed bool + + nextSegmentID uint64 + activeSegment *segment + segments map[uint64]*segment +} + +type segment struct { + id uint64 + path string + file *os.File + size int64 + refCnt int64 + written int64 +} + +type pointer struct { + segmentID uint64 + offset int64 + length int64 +} + +// Entry records where encoded messages are stored and how to release them. +type Entry struct { + memoryMsgs []*codeccommon.Message + pointer *pointer + + callbacks []func() + + accountingBytes int64 + fileBytes uint64 +} + +// FileBytes returns the payload bytes counted towards writer task sizing. +func (e *Entry) FileBytes() uint64 { + if e == nil { + return 0 + } + return e.fileBytes +} + +// IsSpilled returns whether this entry is persisted in local segment files. +func (e *Entry) IsSpilled() bool { + return e != nil && e.pointer != nil +} + +// InMemory returns whether this entry is still held in memory. +func (e *Entry) InMemory() bool { + return e != nil && e.memoryMsgs != nil +} + +// New creates a per-changefeed spool manager. +func New( + changefeedID commonType.ChangeFeedID, + quotaBytes int64, + opts *Options, +) (*Manager, error) { + options := withDefaultOptions(opts) + if quotaBytes <= 0 { + return nil, errors.Errorf("invalid spool quota %d", quotaBytes) + } + if options.SegmentBytes <= 0 { + return nil, errors.Errorf("invalid spool segment size %d", options.SegmentBytes) + } + if options.MemoryRatio <= 0 || options.MemoryRatio >= 1 { + return nil, errors.Errorf("invalid spool memory ratio %f", options.MemoryRatio) + } + if options.LowWatermarkRatio <= 0 || options.LowWatermarkRatio >= 1 { + return nil, errors.Errorf("invalid spool low watermark ratio %f", options.LowWatermarkRatio) + } + if options.HighWatermarkRatio <= 0 || options.HighWatermarkRatio >= 1 { + return nil, errors.Errorf("invalid spool high watermark ratio %f", options.HighWatermarkRatio) + } + if options.LowWatermarkRatio >= options.HighWatermarkRatio { + return nil, errors.Errorf( + "invalid spool watermark ratio, low: %f high: %f", + options.LowWatermarkRatio, + options.HighWatermarkRatio, + ) + } + + rootDir := options.RootDir + if rootDir == "" { + dataDir := config.GetGlobalServerConfig().DataDir + if dataDir == "" { + dataDir = os.TempDir() + } + rootDir = filepath.Join( + dataDir, + defaultDirectoryName, + changefeedID.Keyspace(), + fmt.Sprintf("%s-%s", changefeedID.Name(), changefeedID.ID().String()), + ) + } + + if err := os.RemoveAll(rootDir); err != nil { + return nil, errors.Trace(err) + } + if err := os.MkdirAll(rootDir, 0o755); err != nil { + return nil, errors.Trace(err) + } + + changefeedLabel := changefeedID.ID().String() + manager := &Manager{ + changefeedID: changefeedID, + rootDir: rootDir, + quotaBytes: quotaBytes, + + memoryQuotaBytes: int64(float64(quotaBytes) * options.MemoryRatio), + highWatermarkBytes: int64(float64(quotaBytes) * options.HighWatermarkRatio), + lowWatermarkBytes: int64(float64(quotaBytes) * options.LowWatermarkRatio), + segmentBytes: options.SegmentBytes, + + metricMemoryBytes: metrics.CloudStorageSpoolMemoryBytesGauge.WithLabelValues( + changefeedID.Keyspace(), changefeedLabel), + metricDiskBytes: metrics.CloudStorageSpoolDiskBytesGauge.WithLabelValues( + changefeedID.Keyspace(), changefeedLabel), + metricTotalBytes: metrics.CloudStorageSpoolTotalBytesGauge.WithLabelValues( + changefeedID.Keyspace(), changefeedLabel), + metricWakeSuppressed: metrics.CloudStorageWakeSuppressedCounter.WithLabelValues( + changefeedID.Keyspace(), changefeedLabel), + metricKeyspace: changefeedID.Keyspace(), + metricChangefeedLabel: changefeedLabel, + + segments: make(map[uint64]*segment), + } + manager.updateMetricsLocked() + return manager, nil +} + +func withDefaultOptions(opts *Options) *Options { + result := &Options{ + SegmentBytes: defaultSegmentBytes, + MemoryRatio: defaultMemoryRatio, + HighWatermarkRatio: defaultHighWatermarkRatio, + LowWatermarkRatio: defaultLowWatermarkRatio, + } + if opts == nil { + return result + } + if opts.RootDir != "" { + result.RootDir = opts.RootDir + } + if opts.SegmentBytes > 0 { + result.SegmentBytes = opts.SegmentBytes + } + if opts.MemoryRatio > 0 { + result.MemoryRatio = opts.MemoryRatio + } + if opts.HighWatermarkRatio > 0 { + result.HighWatermarkRatio = opts.HighWatermarkRatio + } + if opts.LowWatermarkRatio > 0 { + result.LowWatermarkRatio = opts.LowWatermarkRatio + } + return result +} + +// Enqueue appends encoded messages to spool. +func (s *Manager) Enqueue( + msgs []*codeccommon.Message, + onEnqueued func(), +) (*Entry, error) { + if len(msgs) == 0 { + return &Entry{}, nil + } + + entry := &Entry{ + callbacks: make([]func(), 0, len(msgs)), + } + for _, msg := range msgs { + entry.accountingBytes += int64(len(msg.Key) + len(msg.Value)) + entry.fileBytes += uint64(len(msg.Value)) + entry.callbacks = append(entry.callbacks, msg.Callback) + msg.Callback = nil + } + + callbacksToRun := make([]func(), 0, 1) + needSuppressedMetric := false + + s.mu.Lock() + defer func() { + s.mu.Unlock() + for _, callback := range callbacksToRun { + if callback != nil { + callback() + } + } + if needSuppressedMetric { + s.metricWakeSuppressed.Inc() + } + }() + + if s.closed { + return nil, errors.New("spool is closed") + } + + shouldSpill := s.memoryBytes+entry.accountingBytes > s.memoryQuotaBytes + if shouldSpill { + blob, err := serializeMessages(msgs) + if err != nil { + return nil, errors.Trace(err) + } + pointerValue, err := s.appendBlobLocked(blob) + if err != nil { + return nil, err + } + entry.pointer = pointerValue + s.diskBytes += entry.accountingBytes + } else { + entry.memoryMsgs = msgs + s.memoryBytes += entry.accountingBytes + } + + totalBytes := s.memoryBytes + s.diskBytes + if totalBytes > s.highWatermarkBytes { + s.wakeSuppressed = true + } + if s.wakeSuppressed { + if onEnqueued != nil { + s.pendingWake = append(s.pendingWake, onEnqueued) + needSuppressedMetric = true + } + } else if onEnqueued != nil { + callbacksToRun = append(callbacksToRun, onEnqueued) + } + s.updateMetricsLocked() + + return entry, nil +} + +// Load fetches messages from memory or spilled segments. +func (s *Manager) Load(entry *Entry) ([]*codeccommon.Message, []func(), error) { + if entry == nil { + return nil, nil, nil + } + if entry.memoryMsgs != nil { + return entry.memoryMsgs, entry.callbacks, nil + } + if entry.pointer == nil { + return nil, entry.callbacks, nil + } + + s.mu.Lock() + spoolSegment := s.segments[entry.pointer.segmentID] + if spoolSegment == nil { + s.mu.Unlock() + return nil, nil, errors.Errorf("spool segment %d not found", entry.pointer.segmentID) + } + file := spoolSegment.file + offset := entry.pointer.offset + length := entry.pointer.length + s.mu.Unlock() + + buf := make([]byte, length) + if _, err := file.ReadAt(buf, offset); err != nil { + return nil, nil, errors.Trace(err) + } + msgs, err := deserializeMessages(buf) + if err != nil { + return nil, nil, errors.Trace(err) + } + return msgs, entry.callbacks, nil +} + +// Release releases memory or spilled bytes held by an entry. +func (s *Manager) Release(entry *Entry) { + if entry == nil { + return + } + + callbacksToRun := make([]func(), 0, len(s.pendingWake)) + s.mu.Lock() + if s.closed { + s.mu.Unlock() + return + } + + if entry.pointer != nil { + spoolSegment := s.segments[entry.pointer.segmentID] + if spoolSegment != nil { + spoolSegment.refCnt-- + if spoolSegment.refCnt == 0 && s.activeSegment != spoolSegment { + if err := spoolSegment.file.Close(); err != nil { + log.Warn("close spool segment file failed", zap.Error(err)) + } + if err := os.Remove(spoolSegment.path); err != nil { + log.Warn( + "remove spool segment file failed", + zap.Error(err), + zap.String("path", spoolSegment.path), + ) + } + delete(s.segments, spoolSegment.id) + } + } + s.diskBytes -= entry.accountingBytes + } else { + s.memoryBytes -= entry.accountingBytes + } + if s.memoryBytes < 0 { + s.memoryBytes = 0 + } + if s.diskBytes < 0 { + s.diskBytes = 0 + } + + totalBytes := s.memoryBytes + s.diskBytes + if s.wakeSuppressed && totalBytes <= s.lowWatermarkBytes { + s.wakeSuppressed = false + callbacksToRun = append(callbacksToRun, s.pendingWake...) + s.pendingWake = s.pendingWake[:0] + } + s.updateMetricsLocked() + s.mu.Unlock() + + for _, callback := range callbacksToRun { + if callback != nil { + callback() + } + } +} + +// Close closes spool and removes temporary files. +func (s *Manager) Close() { + s.mu.Lock() + if s.closed { + s.mu.Unlock() + return + } + s.closed = true + for _, spoolSegment := range s.segments { + if spoolSegment.file != nil { + if err := spoolSegment.file.Close(); err != nil { + log.Warn("close spool segment failed", + zap.String("path", spoolSegment.path), + zap.Error(err)) + } + } + } + s.memoryBytes = 0 + s.diskBytes = 0 + s.pendingWake = nil + s.updateMetricsLocked() + s.mu.Unlock() + + if err := os.RemoveAll(s.rootDir); err != nil { + log.Warn("remove spool directory failed", + zap.String("path", s.rootDir), + zap.Error(err)) + } + + metrics.CloudStorageSpoolMemoryBytesGauge.DeleteLabelValues(s.metricKeyspace, s.metricChangefeedLabel) + metrics.CloudStorageSpoolDiskBytesGauge.DeleteLabelValues(s.metricKeyspace, s.metricChangefeedLabel) + metrics.CloudStorageSpoolTotalBytesGauge.DeleteLabelValues(s.metricKeyspace, s.metricChangefeedLabel) + metrics.CloudStorageWakeSuppressedCounter.DeleteLabelValues(s.metricKeyspace, s.metricChangefeedLabel) +} + +func (s *Manager) appendBlobLocked(blob []byte) (*pointer, error) { + spoolSegment, err := s.getWritableSegmentLocked(int64(len(blob))) + if err != nil { + return nil, err + } + offset := spoolSegment.size + n, err := spoolSegment.file.Write(blob) + if err != nil { + return nil, errors.Trace(err) + } + if n != len(blob) { + return nil, errors.Errorf("short write to spool segment, expected %d got %d", len(blob), n) + } + spoolSegment.size += int64(n) + spoolSegment.written += int64(n) + spoolSegment.refCnt++ + + return &pointer{ + segmentID: spoolSegment.id, + offset: offset, + length: int64(n), + }, nil +} + +func (s *Manager) getWritableSegmentLocked(needBytes int64) (*segment, error) { + if s.activeSegment == nil || s.activeSegment.size+needBytes > s.segmentBytes { + if err := s.rotateLocked(); err != nil { + return nil, err + } + } + return s.activeSegment, nil +} + +func (s *Manager) rotateLocked() error { + s.nextSegmentID++ + segmentID := s.nextSegmentID + path := filepath.Join(s.rootDir, fmt.Sprintf("segment-%06d.log", segmentID)) + file, err := os.OpenFile(path, os.O_CREATE|os.O_RDWR|os.O_APPEND, 0o644) + if err != nil { + return errors.Trace(err) + } + spoolSegment := &segment{ + id: segmentID, + path: path, + file: file, + } + s.segments[segmentID] = spoolSegment + s.activeSegment = spoolSegment + return nil +} + +func (s *Manager) updateMetricsLocked() { + s.metricMemoryBytes.Set(float64(s.memoryBytes)) + s.metricDiskBytes.Set(float64(s.diskBytes)) + s.metricTotalBytes.Set(float64(s.memoryBytes + s.diskBytes)) +} diff --git a/downstreamadapter/sink/cloudstorage/spool/manager_test.go b/downstreamadapter/sink/cloudstorage/spool/manager_test.go new file mode 100644 index 0000000000..c90b25e024 --- /dev/null +++ b/downstreamadapter/sink/cloudstorage/spool/manager_test.go @@ -0,0 +1,100 @@ +// Copyright 2025 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package spool + +import ( + "sync/atomic" + "testing" + + commonType "github.com/pingcap/ticdc/pkg/common" + codeccommon "github.com/pingcap/ticdc/pkg/sink/codec/common" + "github.com/stretchr/testify/require" +) + +func newTestMessage(value string, rows int) *codeccommon.Message { + msg := codeccommon.NewMsg(nil, []byte(value)) + msg.SetRowsCount(rows) + return msg +} + +func TestSuppressAndResumeWake(t *testing.T) { + t.Parallel() + + changefeedID := commonType.NewChangefeedID4Test("test", "spool") + options := &Options{ + RootDir: t.TempDir(), + SegmentBytes: 1 << 20, + MemoryRatio: 0.99, + HighWatermarkRatio: 0.6, + LowWatermarkRatio: 0.3, + } + manager, err := New(changefeedID, 1000, options) + require.NoError(t, err) + defer manager.Close() + + var wakeCount int64 + incWake := func() { + atomic.AddInt64(&wakeCount, 1) + } + + entry1, err := manager.Enqueue([]*codeccommon.Message{ + newTestMessage(string(make([]byte, 350)), 1), + }, incWake) + require.NoError(t, err) + require.Equal(t, int64(1), atomic.LoadInt64(&wakeCount)) + + entry2, err := manager.Enqueue([]*codeccommon.Message{ + newTestMessage(string(make([]byte, 450)), 1), + }, incWake) + require.NoError(t, err) + require.Equal(t, int64(1), atomic.LoadInt64(&wakeCount)) + + manager.Release(entry1) + require.Equal(t, int64(1), atomic.LoadInt64(&wakeCount)) + + manager.Release(entry2) + require.Equal(t, int64(2), atomic.LoadInt64(&wakeCount)) +} + +func TestSpillAndReadBack(t *testing.T) { + t.Parallel() + + changefeedID := commonType.NewChangefeedID4Test("test", "spool") + options := &Options{ + RootDir: t.TempDir(), + SegmentBytes: 1 << 20, + MemoryRatio: 0.01, + HighWatermarkRatio: 0.95, + LowWatermarkRatio: 0.7, + } + manager, err := New(changefeedID, 64, options) + require.NoError(t, err) + defer manager.Close() + + msg := newTestMessage("hello-spool", 3) + entry, err := manager.Enqueue([]*codeccommon.Message{msg}, nil) + require.NoError(t, err) + require.True(t, entry.IsSpilled()) + require.False(t, entry.InMemory()) + require.Equal(t, uint64(len(msg.Value)), entry.FileBytes()) + + msgs, callbacks, err := manager.Load(entry) + require.NoError(t, err) + require.Len(t, callbacks, 1) + require.Len(t, msgs, 1) + require.Equal(t, []byte("hello-spool"), msgs[0].Value) + require.Equal(t, 3, msgs[0].GetRowsCount()) + + manager.Release(entry) +} diff --git a/downstreamadapter/sink/cloudstorage/task.go b/downstreamadapter/sink/cloudstorage/task.go new file mode 100644 index 0000000000..ad9ffc60f1 --- /dev/null +++ b/downstreamadapter/sink/cloudstorage/task.go @@ -0,0 +1,85 @@ +// Copyright 2025 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package cloudstorage + +import ( + "github.com/pingcap/ticdc/downstreamadapter/sink/cloudstorage/spool" + commonType "github.com/pingcap/ticdc/pkg/common" + commonEvent "github.com/pingcap/ticdc/pkg/common/event" + pkgcloudstorage "github.com/pingcap/ticdc/pkg/sink/cloudstorage" + "github.com/pingcap/ticdc/pkg/sink/codec/common" +) + +type taskKind uint8 + +const ( + taskKindDML taskKind = iota + taskKindDrain +) + +type task struct { + kind taskKind + event *commonEvent.DMLEvent + versionedTable pkgcloudstorage.VersionedTableName + dispatcherID commonType.DispatcherID + + encodedMsgs []*common.Message + spoolEntry *spool.Entry + marker *drainMarker +} + +func newDMLTask( + version pkgcloudstorage.VersionedTableName, + event *commonEvent.DMLEvent, +) *task { + return &task{ + kind: taskKindDML, + event: event, + versionedTable: version, + dispatcherID: event.GetDispatcherID(), + } +} + +func newDrainTask( + dispatcherID commonType.DispatcherID, + commitTs uint64, + doneCh chan error, +) *task { + return &task{ + kind: taskKindDrain, + dispatcherID: dispatcherID, + marker: &drainMarker{ + dispatcherID: dispatcherID, + commitTs: commitTs, + doneCh: doneCh, + }, + } +} + +func (t *task) isDrainTask() bool { + return t != nil && t.kind == taskKindDrain +} + +type drainMarker struct { + dispatcherID commonType.DispatcherID + commitTs uint64 + doneCh chan error +} + +func (m *drainMarker) done(err error) { + select { + case m.doneCh <- err: + default: + } +} diff --git a/downstreamadapter/sink/cloudstorage/task_indexer.go b/downstreamadapter/sink/cloudstorage/task_indexer.go new file mode 100644 index 0000000000..c4ba29e765 --- /dev/null +++ b/downstreamadapter/sink/cloudstorage/task_indexer.go @@ -0,0 +1,96 @@ +// Copyright 2025 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package cloudstorage + +import ( + "sync" + "sync/atomic" + + commonType "github.com/pingcap/ticdc/pkg/common" +) + +type taskIndexer struct { + inputShards int + outputShards int + + nextInput uint64 + + mu sync.RWMutex + dispatcherToShard map[commonType.DispatcherID]int +} + +// newTaskIndexer builds the routing policy used by storage sink task pipeline. +// +// Invariants: +// 1. Input index only affects encoder parallelism (round-robin). +// 2. Output index is stable per dispatcher to preserve per-dispatcher ordering. +func newTaskIndexer(inputShards, outputShards int) *taskIndexer { + if inputShards <= 0 { + inputShards = 1 + } + if outputShards <= 0 { + outputShards = 1 + } + + return &taskIndexer{ + inputShards: inputShards, + outputShards: outputShards, + dispatcherToShard: make(map[commonType.DispatcherID]int), + } +} + +func (r *taskIndexer) next(dispatcherID commonType.DispatcherID) (int, int) { + return r.nextInputIndex(), r.routeOutputIndex(dispatcherID) +} + +// nextInputIndex uses round-robin so hot dispatchers do not pin a single encoder shard. +func (r *taskIndexer) nextInputIndex() int { + if r.inputShards <= 1 { + return 0 + } + next := atomic.AddUint64(&r.nextInput, 1) + return int((next - 1) % uint64(r.inputShards)) +} + +func (r *taskIndexer) routeOutputIndex(dispatcherID commonType.DispatcherID) int { + if r.outputShards <= 1 { + return 0 + } + + r.mu.RLock() + index, ok := r.dispatcherToShard[dispatcherID] + r.mu.RUnlock() + if ok { + return index + } + + // We compute hash once and then cache it. + // Principle: stable routing + low per-task overhead. + index = commonType.GID(dispatcherID).Hash(uint64(r.outputShards)) + + r.mu.Lock() + if cached, exists := r.dispatcherToShard[dispatcherID]; exists { + r.mu.Unlock() + return cached + } + r.dispatcherToShard[dispatcherID] = index + r.mu.Unlock() + return index +} + +func (r *taskIndexer) cachedOutputCount() int { + r.mu.RLock() + defer r.mu.RUnlock() + return len(r.dispatcherToShard) +} diff --git a/downstreamadapter/sink/cloudstorage/writer.go b/downstreamadapter/sink/cloudstorage/writer.go index 35eee16791..5d002509bc 100644 --- a/downstreamadapter/sink/cloudstorage/writer.go +++ b/downstreamadapter/sink/cloudstorage/writer.go @@ -23,6 +23,7 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/log" + "github.com/pingcap/ticdc/downstreamadapter/sink/cloudstorage/spool" "github.com/pingcap/ticdc/downstreamadapter/sink/metrics" commonType "github.com/pingcap/ticdc/pkg/common" "github.com/pingcap/ticdc/pkg/errors" @@ -37,24 +38,42 @@ import ( "golang.org/x/sync/errgroup" ) -// writer denotes a worker responsible for writing messages to cloud storage. type writer struct { - // worker id - id int + shardID int changeFeedID commonType.ChangeFeedID storage storage.ExternalStorage config *cloudstorage.Config - // toBeFlushedCh contains a set of batchedTask waiting to be flushed to cloud storage. - toBeFlushedCh chan batchedTask - inputCh *chann.DrainableChann[eventFragment] - isClosed uint64 - statistics *pmetrics.Statistics - filePathGenerator *cloudstorage.FilePathGenerator + spool *spool.Manager + + toBeFlushedCh chan writerTask + inputCh *chann.DrainableChann[*task] + isClosed uint64 + + statistics *pmetrics.Statistics + filePathGenerator *cloudstorage.FilePathGenerator + metricWriteBytes prometheus.Gauge metricFileCount prometheus.Gauge metricWriteDuration prometheus.Observer metricFlushDuration prometheus.Observer - metricsWorkerBusyRatio prometheus.Counter + metricShardBusySeconds prometheus.Counter +} + +type flushReason string + +const ( + flushReasonInterval flushReason = "interval" + flushReasonSize flushReason = "size" + flushReasonDDL flushReason = "ddl" + flushReasonClose flushReason = "close" +) + +// writerTask is internal and never crosses component boundary. +// marker task and data batch are mutually exclusive in normal flow. +type writerTask struct { + batch batchedTask + marker *drainMarker + reason flushReason } func newWriter( @@ -63,18 +82,21 @@ func newWriter( storage storage.ExternalStorage, config *cloudstorage.Config, extension string, - inputCh *chann.DrainableChann[eventFragment], statistics *pmetrics.Statistics, + spoolManager *spool.Manager, ) *writer { - d := &writer{ - id: id, - changeFeedID: changefeedID, - storage: storage, - config: config, - inputCh: inputCh, - toBeFlushedCh: make(chan batchedTask, 64), - statistics: statistics, - filePathGenerator: cloudstorage.NewFilePathGenerator(changefeedID, config, storage, extension), + return &writer{ + shardID: id, + changeFeedID: changefeedID, + storage: storage, + config: config, + spool: spoolManager, + inputCh: chann.NewAutoDrainChann[*task](), + toBeFlushedCh: make(chan writerTask, 64), + statistics: statistics, + filePathGenerator: cloudstorage.NewFilePathGenerator( + changefeedID, config, storage, extension, + ), metricWriteBytes: metrics.CloudStorageWriteBytesGauge. WithLabelValues(changefeedID.Keyspace(), changefeedID.ID().String()), metricFileCount: metrics.CloudStorageFileCountGauge. @@ -83,72 +105,72 @@ func newWriter( WithLabelValues(changefeedID.Keyspace(), changefeedID.ID().String()), metricFlushDuration: metrics.CloudStorageFlushDurationHistogram. WithLabelValues(changefeedID.Keyspace(), changefeedID.ID().String()), - metricsWorkerBusyRatio: metrics.CloudStorageWorkerBusyRatio. + metricShardBusySeconds: metrics.CloudStorageShardBusySeconds. WithLabelValues(changefeedID.Keyspace(), changefeedID.ID().String(), strconv.Itoa(id)), } - - return d } -// Run creates a set of background goroutines. func (d *writer) Run(ctx context.Context) error { eg, ctx := errgroup.WithContext(ctx) eg.Go(func() error { return d.flushMessages(ctx) }) - eg.Go(func() error { - return d.genAndDispatchTask(ctx, d.inputCh) + return d.genAndDispatchTask(ctx) }) - return eg.Wait() } -// SetClock is used for unit test func (d *writer) SetClock(pdClock pdutil.Clock) { d.filePathGenerator.SetClock(pdClock) } -// flushMessages flushed messages of active tables to cloud storage. -// active tables are those tables that have received events after the last flush. func (d *writer) flushMessages(ctx context.Context) error { var flushTimeSlice time.Duration overseerDuration := d.config.FlushInterval * 2 overseerTicker := time.NewTicker(overseerDuration) defer overseerTicker.Stop() + for { select { case <-ctx.Done(): return errors.Trace(ctx.Err()) case <-overseerTicker.C: - d.metricsWorkerBusyRatio.Add(flushTimeSlice.Seconds()) + d.metricShardBusySeconds.Add(flushTimeSlice.Seconds()) flushTimeSlice = 0 - case batchedTask := <-d.toBeFlushedCh: - if atomic.LoadUint64(&d.isClosed) == 1 { + case task, ok := <-d.toBeFlushedCh: + if !ok || atomic.LoadUint64(&d.isClosed) == 1 { return nil } + if task.marker != nil { + // Drain marker ack point: + // marker is emitted only after pending batch is flushed in genAndDispatchTask. + task.marker.done(nil) + continue + } + if len(task.batch.batch) == 0 { + continue + } + start := time.Now() - for table, task := range batchedTask.batch { - if len(task.msgs) == 0 { + for table, singleTask := range task.batch.batch { + if len(singleTask.msgs) == 0 && len(singleTask.entries) == 0 { continue } - // generate scheme.json file before generating the first data file if necessary - hasNewerSchemaVersion, err := d.filePathGenerator.CheckOrWriteSchema(ctx, table, task.tableInfo) + hasNewerSchemaVersion, err := d.filePathGenerator.CheckOrWriteSchema(ctx, table, singleTask.tableInfo) if err != nil { log.Error("failed to write schema file to external storage", - zap.Int("workerID", d.id), + zap.Int("shardID", d.shardID), zap.String("keyspace", d.changeFeedID.Keyspace()), zap.Stringer("changefeed", d.changeFeedID.ID()), zap.Error(err)) - return errors.Trace(err) + return err } - // It is possible that a DML event is sent after a DDL event during dispatcher scheduling. - // We need to ignore such DML events, as they belong to a stale schema version. if hasNewerSchemaVersion { - d.ignoreTableTask(task) + d.ignoreTableTask(singleTask) log.Warn("ignore messages belonging to an old schema version", - zap.Int("workerID", d.id), + zap.Int("shardID", d.shardID), zap.String("keyspace", d.changeFeedID.Keyspace()), zap.Stringer("changefeed", d.changeFeedID.ID()), zap.String("schema", table.TableNameWithPhysicTableID.Schema), @@ -157,52 +179,71 @@ func (d *writer) flushMessages(ctx context.Context) error { continue } - // make sure that `generateDateStr()` is invoked ONLY once before - // generating data file path and index file path. Because we don't expect the index - // file is written to a different dir if date change happens between - // generating data and index file. date := d.filePathGenerator.GenerateDateStr() dataFilePath, err := d.filePathGenerator.GenerateDataFilePath(ctx, table, date) if err != nil { log.Error("failed to generate data file path", - zap.Int("workerID", d.id), + zap.Int("shardID", d.shardID), zap.String("keyspace", d.changeFeedID.Keyspace()), zap.Stringer("changefeed", d.changeFeedID.ID()), zap.Error(err)) - return errors.Trace(err) + return err } indexFilePath := d.filePathGenerator.GenerateIndexFilePath(table, date) - // first write the data file to external storage. - err = d.writeDataFile(ctx, dataFilePath, indexFilePath, task) - if err != nil { + if err := d.writeDataFile(ctx, dataFilePath, indexFilePath, singleTask); err != nil { log.Error("failed to write data file to external storage", - zap.Int("workerID", d.id), + zap.Int("shardID", d.shardID), zap.String("keyspace", d.changeFeedID.Keyspace()), zap.Stringer("changefeed", d.changeFeedID.ID()), zap.String("path", dataFilePath), zap.Error(err)) - return errors.Trace(err) + return err } - log.Debug("write file to storage success", zap.Int("workerID", d.id), + log.Debug("write file to storage success", + zap.Int("shardID", d.shardID), zap.String("keyspace", d.changeFeedID.Keyspace()), zap.Stringer("changefeed", d.changeFeedID.ID()), zap.String("schema", table.TableNameWithPhysicTableID.Schema), zap.String("table", table.TableNameWithPhysicTableID.Table), - zap.String("path", dataFilePath), - ) + zap.String("path", dataFilePath)) } - flushTimeSlice += time.Since(start) + + flushDuration := time.Since(start) + flushTimeSlice += flushDuration + d.observeFlushTask(task.reason, task.batch.totalSize(), flushDuration) } } } +func (d *writer) observeFlushTask(reason flushReason, size uint64, duration time.Duration) { + reasonLabel := string(reason) + metrics.CloudStorageFlushTaskCounter.WithLabelValues( + d.changeFeedID.Keyspace(), + d.changeFeedID.ID().String(), + reasonLabel, + ).Inc() + metrics.CloudStorageFlushDurationByReasonHistogram.WithLabelValues( + d.changeFeedID.Keyspace(), + d.changeFeedID.ID().String(), + reasonLabel, + ).Observe(duration.Seconds()) + metrics.CloudStorageFlushFileSizeHistogram.WithLabelValues( + d.changeFeedID.Keyspace(), + d.changeFeedID.ID().String(), + reasonLabel, + ).Observe(float64(size)) +} + func (d *writer) writeIndexFile(ctx context.Context, path, content string) error { start := time.Now() err := d.storage.WriteFile(ctx, path, []byte(content)) d.metricFlushDuration.Observe(time.Since(start).Seconds()) - return err + if err != nil { + return errors.Trace(err) + } + return nil } func (d *writer) ignoreTableTask(task *singleTableTask) { @@ -211,6 +252,27 @@ func (d *writer) ignoreTableTask(task *singleTableTask) { msg.Callback() } } + for _, entry := range task.entries { + if d.spool == nil { + continue + } + _, callbacks, err := d.spool.Load(entry) + if err != nil { + log.Warn("load spool entry failed when ignoring table task", + zap.Int("shardID", d.shardID), + zap.String("keyspace", d.changeFeedID.Keyspace()), + zap.Stringer("changefeed", d.changeFeedID.ID()), + zap.Error(err)) + d.spool.Release(entry) + continue + } + for _, cb := range callbacks { + if cb != nil { + cb() + } + } + d.spool.Release(entry) + } } func (d *writer) writeDataFile(ctx context.Context, dataFilePath, indexFilePath string, task *singleTableTask) error { @@ -218,7 +280,15 @@ func (d *writer) writeDataFile(ctx context.Context, dataFilePath, indexFilePath buf := bytes.NewBuffer(make([]byte, 0, task.size)) rowsCnt := 0 bytesCnt := int64(0) - // There is always only one message here in task.msgs + entriesToRelease := make([]*spool.Entry, 0, len(task.entries)) + defer func() { + for _, entry := range entriesToRelease { + if d.spool != nil { + d.spool.Release(entry) + } + } + }() + for _, msg := range task.msgs { if msg.Key != nil && rowsCnt == 0 { buf.Write(msg.Key) @@ -230,31 +300,60 @@ func (d *writer) writeDataFile(ctx context.Context, dataFilePath, indexFilePath callbacks = append(callbacks, msg.Callback) } + for _, entry := range task.entries { + if d.spool == nil { + continue + } + msgs, entryCallbacks, err := d.spool.Load(entry) + if err != nil { + return err + } + entriesToRelease = append(entriesToRelease, entry) + for i, msg := range msgs { + if msg.Key != nil && rowsCnt == 0 { + buf.Write(msg.Key) + bytesCnt += int64(len(msg.Key)) + } + bytesCnt += int64(len(msg.Value)) + rowsCnt += msg.GetRowsCount() + buf.Write(msg.Value) + if i < len(entryCallbacks) { + callbacks = append(callbacks, entryCallbacks[i]) + } else { + callbacks = append(callbacks, msg.Callback) + } + } + } + if err := d.statistics.RecordBatchExecution(func() (int, int64, error) { start := time.Now() if d.config.FlushConcurrency <= 1 { - return rowsCnt, bytesCnt, d.storage.WriteFile(ctx, dataFilePath, buf.Bytes()) + err := d.storage.WriteFile(ctx, dataFilePath, buf.Bytes()) + if err != nil { + return 0, 0, errors.Trace(err) + } + d.metricWriteDuration.Observe(time.Since(start).Seconds()) + return rowsCnt, bytesCnt, nil } writer, inErr := d.storage.Create(ctx, dataFilePath, &storage.WriterOption{ Concurrency: d.config.FlushConcurrency, }) if inErr != nil { - return 0, 0, inErr + return 0, 0, errors.Trace(inErr) } if _, inErr = writer.Write(ctx, buf.Bytes()); inErr != nil { - return 0, 0, inErr + return 0, 0, errors.Trace(inErr) } - // We have to wait the writer to close to complete the upload - // If failed to close writer, some DMLs may not be upload successfully if inErr = writer.Close(ctx); inErr != nil { - log.Error("failed to close writer", zap.Error(inErr), - zap.Int("workerID", d.id), + log.Error("failed to close writer", + zap.Error(inErr), + zap.Int("shardID", d.shardID), zap.Any("table", task.tableInfo.TableName), zap.String("keyspace", d.changeFeedID.Keyspace()), zap.Stringer("changefeed", d.changeFeedID.ID())) - return 0, 0, inErr + return 0, 0, errors.Trace(inErr) } d.metricFlushDuration.Observe(time.Since(start).Seconds()) @@ -266,17 +365,14 @@ func (d *writer) writeDataFile(ctx context.Context, dataFilePath, indexFilePath d.metricWriteBytes.Add(float64(bytesCnt)) d.metricFileCount.Add(1) - // then write the index file to external storage in the end. - // the file content is simply the last data file path - err := d.writeIndexFile(ctx, indexFilePath, path.Base(dataFilePath)+"\n") - if err != nil { + if err := d.writeIndexFile(ctx, indexFilePath, path.Base(dataFilePath)+"\n"); err != nil { log.Error("failed to write index file to external storage", - zap.Int("workerID", d.id), + zap.Int("shardID", d.shardID), zap.String("keyspace", d.changeFeedID.Keyspace()), zap.Stringer("changefeed", d.changeFeedID.ID()), zap.String("path", indexFilePath), zap.Error(err)) - return errors.Trace(err) + return err } for _, cb := range callbacks { @@ -284,22 +380,20 @@ func (d *writer) writeDataFile(ctx context.Context, dataFilePath, indexFilePath cb() } } - return nil } -// genAndDispatchTask dispatches flush tasks in two conditions: -// 1. the flush interval exceeds the upper limit. -// 2. the file size exceeds the upper limit. -func (d *writer) genAndDispatchTask(ctx context.Context, - ch *chann.DrainableChann[eventFragment], -) error { +// genAndDispatchTask builds table batches and emits flush tasks with explicit reasons. +// Invariants: +// 1. DDL marker will flush current batch first, then emit marker task. +// 2. Writer close path flushes remaining batch with reason=close. +// 3. Size-triggered flush only flushes target table shard batch. +func (d *writer) genAndDispatchTask(ctx context.Context) error { batchedTask := newBatchedTask() ticker := time.NewTicker(d.config.FlushInterval) defer ticker.Stop() + for { - // this failpoint is use to pass this ticker once - // to make writeEvent in the test case can write into the same file failpoint.Inject("passTickerOnce", func() { <-ticker.C }) @@ -311,55 +405,93 @@ func (d *writer) genAndDispatchTask(ctx context.Context, if atomic.LoadUint64(&d.isClosed) == 1 { return nil } + if len(batchedTask.batch) == 0 { + continue + } select { case <-ctx.Done(): return errors.Trace(ctx.Err()) - case d.toBeFlushedCh <- batchedTask: + case d.toBeFlushedCh <- writerTask{batch: batchedTask, reason: flushReasonInterval}: log.Debug("flush task is emitted successfully when flush interval exceeds", zap.Int("tablesLength", len(batchedTask.batch))) batchedTask = newBatchedTask() default: } - case frag, ok := <-ch.Out(): + case tableTask, ok := <-d.inputCh.Out(): if !ok || atomic.LoadUint64(&d.isClosed) == 1 { - return nil + if len(batchedTask.batch) == 0 { + return nil + } + select { + case <-ctx.Done(): + return errors.Trace(ctx.Err()) + case d.toBeFlushedCh <- writerTask{batch: batchedTask, reason: flushReasonClose}: + return nil + } } - batchedTask.handleSingleTableEvent(frag) - // if the file size exceeds the upper limit, emit the flush task containing the table - // as soon as possible. - table := frag.versionedTable + + if tableTask.isDrainTask() { + if len(batchedTask.batch) > 0 { + select { + case <-ctx.Done(): + return errors.Trace(ctx.Err()) + case d.toBeFlushedCh <- writerTask{batch: batchedTask, reason: flushReasonDDL}: + batchedTask = newBatchedTask() + } + } + select { + case <-ctx.Done(): + return errors.Trace(ctx.Err()) + case d.toBeFlushedCh <- writerTask{marker: tableTask.marker, reason: flushReasonDDL}: + } + continue + } + + batchedTask.handleSingleTableEvent(tableTask) + table := tableTask.versionedTable if batchedTask.batch[table].size >= uint64(d.config.FileSize) { - task := batchedTask.generateTaskByTable(table) + taskByTable := batchedTask.generateTaskByTable(table) select { case <-ctx.Done(): return errors.Trace(ctx.Err()) - case d.toBeFlushedCh <- task: + case d.toBeFlushedCh <- writerTask{batch: taskByTable, reason: flushReasonSize}: log.Debug("flush task is emitted successfully when file size exceeds", zap.Any("table", table), - zap.Int("eventsLenth", len(task.batch[table].msgs))) + zap.Int("eventsLength", len(taskByTable.batch[table].msgs))) } } } } } +func (d *writer) enqueueTask(ctx context.Context, taskValue *task) error { + select { + case <-ctx.Done(): + return errors.Trace(ctx.Err()) + case d.inputCh.In() <- taskValue: + return nil + } +} + +func (d *writer) closeInput() { + d.inputCh.CloseAndDrain() +} + func (d *writer) close() { if !atomic.CompareAndSwapUint64(&d.isClosed, 0, 1) { return } } -// batchedTask contains a set of singleTableTask. -// We batch message of different tables together to reduce the overhead of calling external storage API. type batchedTask struct { batch map[cloudstorage.VersionedTableName]*singleTableTask } -// singleTableTask contains a set of messages belonging to the same table. type singleTableTask struct { size uint64 tableInfo *commonType.TableInfo msgs []*common.Message + entries []*spool.Entry } func newBatchedTask() batchedTask { @@ -368,7 +500,7 @@ func newBatchedTask() batchedTask { } } -func (t *batchedTask) handleSingleTableEvent(event eventFragment) { +func (t *batchedTask) handleSingleTableEvent(event *task) { table := event.versionedTable if _, ok := t.batch[table]; !ok { t.batch[table] = &singleTableTask{ @@ -377,21 +509,33 @@ func (t *batchedTask) handleSingleTableEvent(event eventFragment) { } } - v := t.batch[table] + tableTask := t.batch[table] + if event.spoolEntry != nil { + tableTask.size += event.spoolEntry.FileBytes() + tableTask.entries = append(tableTask.entries, event.spoolEntry) + } for _, msg := range event.encodedMsgs { - v.size += uint64(len(msg.Value)) + tableTask.size += uint64(len(msg.Value)) } - v.msgs = append(v.msgs, event.encodedMsgs...) + tableTask.msgs = append(tableTask.msgs, event.encodedMsgs...) } func (t *batchedTask) generateTaskByTable(table cloudstorage.VersionedTableName) batchedTask { - v := t.batch[table] - if v == nil { + tableTask := t.batch[table] + if tableTask == nil { log.Panic("table not found in dml task", zap.Any("table", table), zap.Any("task", t)) } delete(t.batch, table) return batchedTask{ - batch: map[cloudstorage.VersionedTableName]*singleTableTask{table: v}, + batch: map[cloudstorage.VersionedTableName]*singleTableTask{table: tableTask}, + } +} + +func (t batchedTask) totalSize() uint64 { + var total uint64 + for _, singleTask := range t.batch { + total += singleTask.size } + return total } diff --git a/downstreamadapter/sink/cloudstorage/writer_test.go b/downstreamadapter/sink/cloudstorage/writer_test.go index 0be8a78640..3519899def 100644 --- a/downstreamadapter/sink/cloudstorage/writer_test.go +++ b/downstreamadapter/sink/cloudstorage/writer_test.go @@ -19,6 +19,7 @@ import ( "net/url" "path" "sync" + "sync/atomic" "testing" "time" @@ -32,7 +33,6 @@ import ( "github.com/pingcap/ticdc/pkg/sink/cloudstorage" "github.com/pingcap/ticdc/pkg/sink/codec/common" "github.com/pingcap/ticdc/pkg/util" - "github.com/pingcap/ticdc/utils/chann" timodel "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" @@ -54,14 +54,14 @@ func testWriter(ctx context.Context, t *testing.T, dir string) *writer { cfg.FileIndexWidth = 6 require.Nil(t, err) - changefeedID := commonType.NewChangefeedID4Test("test", "dml-worker-test") - statistics := metrics.NewStatistics(changefeedID, "dml-worker-test") + changefeedID := commonType.NewChangefeedID4Test("test", t.Name()) + statistics := metrics.NewStatistics(changefeedID, t.Name()) pdlock := pdutil.NewMonotonicClock(pclock.New()) appcontext.SetService(appcontext.DefaultPDClock, pdlock) mockPDClock := pdutil.NewClock4Test() appcontext.SetService(appcontext.DefaultPDClock, mockPDClock) d := newWriter(1, changefeedID, storage, - cfg, ".json", chann.NewAutoDrainChann[eventFragment](), statistics) + cfg, ".json", statistics, nil) return d } @@ -71,7 +71,6 @@ func TestWriterRun(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) parentDir := t.TempDir() d := testWriter(ctx, t, parentDir) - fragCh := d.inputCh table1Dir := path.Join(parentDir, "test/table1/99") tidbTableInfo := &timodel.TableInfo{ @@ -86,31 +85,29 @@ func TestWriterRun(t *testing.T) { dispatcherID := commonType.NewDispatcherID() for i := 0; i < 5; i++ { - frag := eventFragment{ - seqNumber: uint64(i), - versionedTable: cloudstorage.VersionedTableName{ - TableNameWithPhysicTableID: commonType.TableName{ - Schema: "test", - Table: "table1", - TableID: 100, - }, - TableInfoVersion: 99, - DispatcherID: dispatcherID, + tableName := cloudstorage.VersionedTableName{ + TableNameWithPhysicTableID: commonType.TableName{ + Schema: "test", + Table: "table1", + TableID: 100, }, - event: &commonEvent.DMLEvent{ - PhysicalTableID: 100, - TableInfo: tableInfo, - Rows: chunk.MutRowFromValues(100, "hello world").ToRow().Chunk(), - }, - encodedMsgs: []*common.Message{ - { - Value: []byte(fmt.Sprintf(`{"id":%d,"database":"test","table":"table1","pkNames":[],"isDdl":false,`+ - `"type":"INSERT","es":0,"ts":1663572946034,"sql":"","sqlType":{"c1":12,"c2":12},`+ - `"data":[{"c1":"100","c2":"hello world"}],"old":null}`, i)), - }, + TableInfoVersion: 99, + DispatcherID: dispatcherID, + } + dmlEvent := &commonEvent.DMLEvent{ + PhysicalTableID: 100, + TableInfo: tableInfo, + Rows: chunk.MutRowFromValues(100, "hello world").ToRow().Chunk(), + } + tableTask := newDMLTask(tableName, dmlEvent) + tableTask.encodedMsgs = []*common.Message{ + { + Value: []byte(fmt.Sprintf(`{"id":%d,"database":"test","table":"table1","pkNames":[],"isDdl":false,`+ + `"type":"INSERT","es":0,"ts":1663572946034,"sql":"","sqlType":{"c1":12,"c2":12},`+ + `"data":[{"c1":"100","c2":"hello world"}],"old":null}`, i)), }, } - fragCh.In() <- frag + require.NoError(t, d.enqueueTask(ctx, tableTask)) } var wg sync.WaitGroup @@ -125,8 +122,77 @@ func TestWriterRun(t *testing.T) { fileNames := getTableFiles(t, table1Dir) require.Len(t, fileNames, 2) require.ElementsMatch(t, []string{fmt.Sprintf("CDC_%s_000001.json", dispatcherID.String()), fmt.Sprintf("CDC_%s.index", dispatcherID.String())}, fileNames) - fragCh.CloseAndDrain() + d.closeInput() cancel() d.close() wg.Wait() } + +func TestWriterDrainMarker(t *testing.T) { + t.Parallel() + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + parentDir := t.TempDir() + d := testWriter(ctx, t, parentDir) + + tidbTableInfo := &timodel.TableInfo{ + ID: 100, + Name: ast.NewCIStr("table1"), + Columns: []*timodel.ColumnInfo{ + {ID: 1, Name: ast.NewCIStr("c1"), FieldType: *types.NewFieldType(mysql.TypeLong)}, + }, + } + tableInfo := commonType.WrapTableInfo("test", tidbTableInfo) + dispatcherID := commonType.NewDispatcherID() + + var callbackCnt atomic.Int64 + msg := common.NewMsg(nil, []byte(`{"id":1}`)) + msg.SetRowsCount(1) + msg.Callback = func() { + callbackCnt.Add(1) + } + + tableTask := newDMLTask( + cloudstorage.VersionedTableName{ + TableNameWithPhysicTableID: commonType.TableName{ + Schema: "test", + Table: "table1", + TableID: 100, + }, + TableInfoVersion: 99, + DispatcherID: dispatcherID, + }, + &commonEvent.DMLEvent{ + PhysicalTableID: 100, + TableInfo: tableInfo, + }, + ) + tableTask.encodedMsgs = []*common.Message{msg} + require.NoError(t, d.enqueueTask(ctx, tableTask)) + + doneCh := make(chan error, 1) + require.NoError(t, d.enqueueTask(ctx, newDrainTask(dispatcherID, 100, doneCh))) + + var wg sync.WaitGroup + wg.Add(1) + go func() { + defer wg.Done() + _ = d.Run(ctx) + }() + + select { + case err := <-doneCh: + require.NoError(t, err) + case <-time.After(10 * time.Second): + t.Fatal("wait drain marker timeout") + } + require.Eventually(t, func() bool { + return callbackCnt.Load() == 1 + }, 5*time.Second, 100*time.Millisecond) + + d.closeInput() + d.close() + cancel() + wg.Wait() +} diff --git a/downstreamadapter/sink/kafka/sink.go b/downstreamadapter/sink/kafka/sink.go index 1c9910b0b0..04207c58af 100644 --- a/downstreamadapter/sink/kafka/sink.go +++ b/downstreamadapter/sink/kafka/sink.go @@ -164,6 +164,10 @@ func (s *sink) WriteBlockEvent(event commonEvent.BlockEvent) error { return nil } +func (s *sink) PassBlockEvent(_ commonEvent.BlockEvent) error { + return nil +} + func (s *sink) close() { s.eventChan.Close() s.rowChan.Close() diff --git a/downstreamadapter/sink/metrics/cloudstorage.go b/downstreamadapter/sink/metrics/cloudstorage.go index 7fc8dae975..92bb84eb1d 100644 --- a/downstreamadapter/sink/metrics/cloudstorage.go +++ b/downstreamadapter/sink/metrics/cloudstorage.go @@ -55,14 +55,75 @@ var ( Buckets: prometheus.ExponentialBuckets(0.001, 2.0, 13), }, []string{"namespace", "changefeed"}) - // CloudStorageWorkerBusyRatio records the busy ratio of CloudStorage bgUpdateLog worker. - CloudStorageWorkerBusyRatio = prometheus.NewCounterVec( + CloudStorageFlushTaskCounter = prometheus.NewCounterVec(prometheus.CounterOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "cloud_storage_flush_tasks_total", + Help: "Total number of cloud storage flush tasks by reason", + }, []string{"namespace", "changefeed", "reason"}) + + CloudStorageFlushDurationByReasonHistogram = prometheus.NewHistogramVec(prometheus.HistogramOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "cloud_storage_flush_duration_by_reason_seconds", + Help: "Flush duration distributions by trigger reason", + Buckets: prometheus.ExponentialBuckets(0.001, 2.0, 13), + }, []string{"namespace", "changefeed", "reason"}) + + CloudStorageFlushFileSizeHistogram = prometheus.NewHistogramVec(prometheus.HistogramOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "cloud_storage_flush_file_size_bytes", + Help: "Flushed payload bytes per task by reason", + Buckets: prometheus.ExponentialBuckets(1024, 2.0, 16), + }, []string{"namespace", "changefeed", "reason"}) + + CloudStorageDDLDrainDurationHistogram = prometheus.NewHistogramVec(prometheus.HistogramOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "cloud_storage_ddl_drain_duration_seconds", + Help: "DDL drain duration for cloud storage sink", + Buckets: prometheus.ExponentialBuckets(0.001, 2.0, 13), + }, []string{"namespace", "changefeed"}) + + // CloudStorageShardBusySeconds records the busy seconds of each cloud storage output shard. + CloudStorageShardBusySeconds = prometheus.NewCounterVec( prometheus.CounterOpts{ Namespace: namespace, Subsystem: subsystem, - Name: "cloud_storage_worker_busy_ratio", - Help: "Busy ratio for cloud storage sink dml worker.", + Name: "cloud_storage_shard_busy_seconds_total", + Help: "Busy seconds for cloud storage sink output shard.", }, []string{"namespace", "changefeed", "id"}) + + CloudStorageWorkerBusyRatio = CloudStorageShardBusySeconds + + CloudStorageSpoolMemoryBytesGauge = prometheus.NewGaugeVec(prometheus.GaugeOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "cloud_storage_spool_memory_bytes", + Help: "Current memory bytes used by cloud storage spool", + }, []string{"namespace", "changefeed"}) + + CloudStorageSpoolDiskBytesGauge = prometheus.NewGaugeVec(prometheus.GaugeOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "cloud_storage_spool_disk_bytes", + Help: "Current disk bytes used by cloud storage spool", + }, []string{"namespace", "changefeed"}) + + CloudStorageSpoolTotalBytesGauge = prometheus.NewGaugeVec(prometheus.GaugeOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "cloud_storage_spool_total_bytes", + Help: "Current total bytes used by cloud storage spool", + }, []string{"namespace", "changefeed"}) + + CloudStorageWakeSuppressedCounter = prometheus.NewCounterVec(prometheus.CounterOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "cloud_storage_wake_suppressed_total", + Help: "Total number of suppressed wake callbacks in cloud storage sink", + }, []string{"namespace", "changefeed"}) ) // InitCloudStorageMetrics registers all metrics in this file. @@ -71,5 +132,13 @@ func InitCloudStorageMetrics(registry *prometheus.Registry) { registry.MustRegister(CloudStorageFileCountGauge) registry.MustRegister(CloudStorageWriteDurationHistogram) registry.MustRegister(CloudStorageFlushDurationHistogram) - registry.MustRegister(CloudStorageWorkerBusyRatio) + registry.MustRegister(CloudStorageFlushTaskCounter) + registry.MustRegister(CloudStorageFlushDurationByReasonHistogram) + registry.MustRegister(CloudStorageFlushFileSizeHistogram) + registry.MustRegister(CloudStorageDDLDrainDurationHistogram) + registry.MustRegister(CloudStorageShardBusySeconds) + registry.MustRegister(CloudStorageSpoolMemoryBytesGauge) + registry.MustRegister(CloudStorageSpoolDiskBytesGauge) + registry.MustRegister(CloudStorageSpoolTotalBytesGauge) + registry.MustRegister(CloudStorageWakeSuppressedCounter) } diff --git a/downstreamadapter/sink/mock_sink.go b/downstreamadapter/sink/mock_sink.go index 37b1cfe986..e0ef9750b3 100644 --- a/downstreamadapter/sink/mock_sink.go +++ b/downstreamadapter/sink/mock_sink.go @@ -39,6 +39,10 @@ func (s *mockSink) WriteBlockEvent(event commonEvent.BlockEvent) error { return nil } +func (s *mockSink) PassBlockEvent(_ commonEvent.BlockEvent) error { + return nil +} + func (s *mockSink) AddCheckpointTs(_ uint64) { } diff --git a/downstreamadapter/sink/mysql/sink.go b/downstreamadapter/sink/mysql/sink.go index 4b54451cc1..0da5230913 100644 --- a/downstreamadapter/sink/mysql/sink.go +++ b/downstreamadapter/sink/mysql/sink.go @@ -257,6 +257,10 @@ func (s *Sink) WriteBlockEvent(event commonEvent.BlockEvent) error { return nil } +func (s *Sink) PassBlockEvent(_ commonEvent.BlockEvent) error { + return nil +} + func (s *Sink) AddCheckpointTs(_ uint64) {} // GetTableRecoveryInfo queries DDL crash recovery information for the given tables. diff --git a/downstreamadapter/sink/pulsar/sink.go b/downstreamadapter/sink/pulsar/sink.go index 87b0f18200..004e56e154 100644 --- a/downstreamadapter/sink/pulsar/sink.go +++ b/downstreamadapter/sink/pulsar/sink.go @@ -148,6 +148,10 @@ func (s *sink) WriteBlockEvent(event commonEvent.BlockEvent) error { return nil } +func (s *sink) PassBlockEvent(_ commonEvent.BlockEvent) error { + return nil +} + func (s *sink) sendDDLEvent(event *commonEvent.DDLEvent) error { for _, e := range event.GetEvents() { message, err := s.comp.encoder.EncodeDDLEvent(e) diff --git a/downstreamadapter/sink/redo/sink.go b/downstreamadapter/sink/redo/sink.go index 70ce2c12a2..1964cdc2b8 100644 --- a/downstreamadapter/sink/redo/sink.go +++ b/downstreamadapter/sink/redo/sink.go @@ -126,6 +126,10 @@ func (s *Sink) WriteBlockEvent(event commonEvent.BlockEvent) error { return nil } +func (s *Sink) PassBlockEvent(_ commonEvent.BlockEvent) error { + return nil +} + func (s *Sink) AddDMLEvent(event *commonEvent.DMLEvent) { _ = s.statistics.RecordBatchExecution(func() (int, int64, error) { toRowCallback := func(postTxnFlushed []func(), totalCount uint64) func() { diff --git a/downstreamadapter/sink/sink.go b/downstreamadapter/sink/sink.go index 77a5b43fc4..e89b41b318 100644 --- a/downstreamadapter/sink/sink.go +++ b/downstreamadapter/sink/sink.go @@ -33,6 +33,7 @@ type Sink interface { IsNormal() bool AddDMLEvent(event *commonEvent.DMLEvent) + PassBlockEvent(event commonEvent.BlockEvent) error WriteBlockEvent(event commonEvent.BlockEvent) error AddCheckpointTs(ts uint64) diff --git a/pkg/common/event/dml_event.go b/pkg/common/event/dml_event.go index 1bd926db94..09fc915c92 100644 --- a/pkg/common/event/dml_event.go +++ b/pkg/common/event/dml_event.go @@ -394,9 +394,14 @@ type DMLEvent struct { // PostTxnFlushed is the functions to be executed after the transaction is flushed. // It is set and used by dispatcher. PostTxnFlushed []func() `json:"-"` + // PostTxnEnqueued is the functions to be executed after the transaction is enqueued to sink. + // It is set by dispatcher and called by sink. + PostTxnEnqueued []func() `json:"-"` // eventSize is the size of the event in bytes. It is set when it's unmarshaled. eventSize int64 `json:"-"` + // postEnqueueCalled indicates whether PostEnqueue has been triggered. + postEnqueueCalled bool `json:"-"` // offset is the offset of the current row in the transaction. // It is internal field, not exported. So it doesn't need to be marshalled. offset int `json:"-"` @@ -630,11 +635,22 @@ func (t *DMLEvent) GetStartTs() common.Ts { } func (t *DMLEvent) PostFlush() { + t.PostEnqueue() for _, f := range t.PostTxnFlushed { f() } } +func (t *DMLEvent) PostEnqueue() { + if t.postEnqueueCalled { + return + } + t.postEnqueueCalled = true + for _, f := range t.PostTxnEnqueued { + f() + } +} + func (t *DMLEvent) GetSeq() uint64 { return t.Seq } @@ -655,6 +671,14 @@ func (t *DMLEvent) AddPostFlushFunc(f func()) { t.PostTxnFlushed = append(t.PostTxnFlushed, f) } +func (t *DMLEvent) ClearPostEnqueueFunc() { + t.PostTxnEnqueued = t.PostTxnEnqueued[:0] +} + +func (t *DMLEvent) AddPostEnqueueFunc(f func()) { + t.PostTxnEnqueued = append(t.PostTxnEnqueued, f) +} + // Rewind reset the offset to 0, So that the next GetNextRow will return the first row func (t *DMLEvent) Rewind() { t.offset = 0 diff --git a/pkg/common/event/dml_event_test.go b/pkg/common/event/dml_event_test.go index 555f1587af..ea229ea683 100644 --- a/pkg/common/event/dml_event_test.go +++ b/pkg/common/event/dml_event_test.go @@ -278,6 +278,46 @@ func TestDMLEventHeaderValidation(t *testing.T) { require.Contains(t, err.Error(), "incomplete data") } +func TestDMLEventPostEnqueueFuncs(t *testing.T) { + event := NewDMLEvent(common.NewDispatcherID(), 1, 1, 2, nil) + + called := make([]int, 0, 2) + event.AddPostEnqueueFunc(func() { + called = append(called, 1) + }) + event.AddPostEnqueueFunc(func() { + called = append(called, 2) + }) + + event.PostEnqueue() + require.Equal(t, []int{1, 2}, called) + + event.ClearPostEnqueueFunc() + called = called[:0] + event.PostEnqueue() + require.Empty(t, called) +} + +func TestDMLEventPostFlushTriggersPostEnqueueOnce(t *testing.T) { + event := NewDMLEvent(common.NewDispatcherID(), 1, 1, 2, nil) + + enqueueCalled := 0 + flushCalled := 0 + event.AddPostEnqueueFunc(func() { + enqueueCalled++ + }) + event.AddPostFlushFunc(func() { + flushCalled++ + }) + + event.PostFlush() + require.Equal(t, 1, enqueueCalled) + require.Equal(t, 1, flushCalled) + + event.PostEnqueue() + require.Equal(t, 1, enqueueCalled) +} + func TestBatchDMLEventHeaderValidation(t *testing.T) { helper := NewEventTestHelper(t) defer helper.Close() diff --git a/pkg/config/sink.go b/pkg/config/sink.go index b2db3d6a16..38cefc2d50 100644 --- a/pkg/config/sink.go +++ b/pkg/config/sink.go @@ -690,9 +690,10 @@ type MySQLConfig struct { // CloudStorageConfig represents a cloud storage sink configuration type CloudStorageConfig struct { - WorkerCount *int `toml:"worker-count" json:"worker-count,omitempty"` - FlushInterval *string `toml:"flush-interval" json:"flush-interval,omitempty"` - FileSize *int `toml:"file-size" json:"file-size,omitempty"` + WorkerCount *int `toml:"worker-count" json:"worker-count,omitempty"` + FlushInterval *string `toml:"flush-interval" json:"flush-interval,omitempty"` + FileSize *int `toml:"file-size" json:"file-size,omitempty"` + SpoolDiskQuota *int64 `toml:"spool-disk-quota" json:"spool-disk-quota,omitempty"` OutputColumnID *bool `toml:"output-column-id" json:"output-column-id,omitempty"` FileExpirationDays *int `toml:"file-expiration-days" json:"file-expiration-days,omitempty"` diff --git a/pkg/sink/cloudstorage/config.go b/pkg/sink/cloudstorage/config.go index 4b817dd394..482fdda633 100644 --- a/pkg/sink/cloudstorage/config.go +++ b/pkg/sink/cloudstorage/config.go @@ -49,6 +49,8 @@ const ( maxFlushConcurrency = 512 // defaultFileSize is the default value of file-size. defaultFileSize = 64 * 1024 * 1024 + // defaultSpoolDiskQuota is the default value of spool-disk-quota. + defaultSpoolDiskQuota = int64(10 * 1024 * 1024 * 1024) // the lower limit of file size minFileSize = 1 * 1024 // the upper limit of file size @@ -60,13 +62,14 @@ const ( // `0 0 2 * * ?` means 2:00:00 AM every day defaultFileCleanupCronSpec = "0 0 2 * * *" - defaultEnableTableAcrossNodes = true + defaultEnableTableAcrossNodes = false ) type urlConfig struct { - WorkerCount *int `form:"worker-count"` - FlushInterval *string `form:"flush-interval"` - FileSize *int `form:"file-size"` + WorkerCount *int `form:"worker-count"` + FlushInterval *string `form:"flush-interval"` + FileSize *int `form:"file-size"` + SpoolDiskQuota *int64 `form:"spool-disk-quota"` } // Config is the configuration for cloud storage sink. @@ -82,6 +85,7 @@ type Config struct { OutputColumnID bool FlushConcurrency int EnableTableAcrossNodes bool + SpoolDiskQuota int64 } // NewConfig returns the default cloud storage sink config. @@ -93,6 +97,7 @@ func NewConfig() *Config { FileExpirationDays: defaultFileExpirationDays, FileCleanupCronSpec: defaultFileCleanupCronSpec, EnableTableAcrossNodes: defaultEnableTableAcrossNodes, + SpoolDiskQuota: defaultSpoolDiskQuota, } } @@ -132,6 +137,10 @@ func (c *Config) Apply( if err != nil { return err } + err = getSpoolDiskQuota(urlParameter, &c.SpoolDiskQuota) + if err != nil { + return err + } c.DateSeparator = util.GetOrZero(sinkConfig.DateSeparator) c.EnablePartitionSeparator = util.GetOrZero(sinkConfig.EnablePartitionSeparator) @@ -164,9 +173,22 @@ func mergeConfig( ) (*urlConfig, error) { dest := &urlConfig{} if sinkConfig != nil && sinkConfig.CloudStorageConfig != nil { - dest.WorkerCount = sinkConfig.CloudStorageConfig.WorkerCount - dest.FlushInterval = sinkConfig.CloudStorageConfig.FlushInterval - dest.FileSize = sinkConfig.CloudStorageConfig.FileSize + if sinkConfig.CloudStorageConfig.WorkerCount != nil { + workerCount := *sinkConfig.CloudStorageConfig.WorkerCount + dest.WorkerCount = &workerCount + } + if sinkConfig.CloudStorageConfig.FlushInterval != nil { + flushInterval := *sinkConfig.CloudStorageConfig.FlushInterval + dest.FlushInterval = &flushInterval + } + if sinkConfig.CloudStorageConfig.FileSize != nil { + fileSize := *sinkConfig.CloudStorageConfig.FileSize + dest.FileSize = &fileSize + } + if sinkConfig.CloudStorageConfig.SpoolDiskQuota != nil { + spoolDiskQuota := *sinkConfig.CloudStorageConfig.SpoolDiskQuota + dest.SpoolDiskQuota = &spoolDiskQuota + } } if err := mergo.Merge(dest, urlParameters, mergo.WithOverride); err != nil { return nil, cerror.WrapError(cerror.ErrStorageSinkInvalidConfig, err) @@ -238,3 +260,18 @@ func getFileSize(values *urlConfig, fileSize *int) error { *fileSize = sz return nil } + +func getSpoolDiskQuota(values *urlConfig, spoolDiskQuota *int64) error { + if values.SpoolDiskQuota == nil { + return nil + } + + quota := *values.SpoolDiskQuota + if quota <= 0 { + return cerror.WrapError(cerror.ErrStorageSinkInvalidConfig, + fmt.Errorf("invalid spool-disk-quota %d, it must be greater than 0", quota)) + } + + *spoolDiskQuota = quota + return nil +} diff --git a/pkg/sink/cloudstorage/config_test.go b/pkg/sink/cloudstorage/config_test.go index 442d95a2eb..85510372a5 100644 --- a/pkg/sink/cloudstorage/config_test.go +++ b/pkg/sink/cloudstorage/config_test.go @@ -33,6 +33,8 @@ func TestConfigApply(t *testing.T) { expected.DateSeparator = config.DateSeparatorDay.String() expected.EnablePartitionSeparator = true expected.FlushConcurrency = 1 + expected.SpoolDiskQuota = 10 * 1024 * 1024 * 1024 + expected.EnableTableAcrossNodes = true uri := "s3://bucket/prefix?worker-count=32&flush-interval=10s&file-size=16777216&protocol=csv" sinkURI, err := url.Parse(uri) require.Nil(t, err) @@ -165,3 +167,27 @@ func TestMergeConfig(t *testing.T) { require.Equal(t, 33554432, c.FileSize) require.Equal(t, "2m2s", c.FlushInterval.String()) } + +func TestSpoolDiskQuotaConfig(t *testing.T) { + uri := "s3://bucket/prefix?spool-disk-quota=2147483648" + sinkURI, err := url.Parse(uri) + require.NoError(t, err) + + replicaConfig := config.GetDefaultReplicaConfig() + replicaConfig.Sink.CloudStorageConfig = &config.CloudStorageConfig{ + SpoolDiskQuota: aws.Int64(3221225472), + } + + cfg := NewConfig() + err = cfg.Apply(context.Background(), sinkURI, replicaConfig.Sink, true) + require.NoError(t, err) + require.Equal(t, int64(2147483648), cfg.SpoolDiskQuota) + + uri = "s3://bucket/prefix" + sinkURI, err = url.Parse(uri) + require.NoError(t, err) + cfg = NewConfig() + err = cfg.Apply(context.Background(), sinkURI, replicaConfig.Sink, true) + require.NoError(t, err) + require.Equal(t, int64(3221225472), cfg.SpoolDiskQuota) +} diff --git a/pkg/sink/cloudstorage/path.go b/pkg/sink/cloudstorage/path.go index 9faf405eb9..7f0fce0cd6 100644 --- a/pkg/sink/cloudstorage/path.go +++ b/pkg/sink/cloudstorage/path.go @@ -30,7 +30,6 @@ import ( appcontext "github.com/pingcap/ticdc/pkg/common/context" "github.com/pingcap/ticdc/pkg/config" "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/hash" "github.com/pingcap/ticdc/pkg/pdutil" "github.com/pingcap/ticdc/pkg/util" "github.com/pingcap/tidb/br/pkg/storage" @@ -53,6 +52,9 @@ const ( // The table schema is stored in the following path: // //meta/schema_{tableVersion}_{checksum}.json tableSchemaPrefix = "%s/%s/meta/" + + defaultPathStateTTL = 24 * time.Hour + defaultPathStateCleanupInterval = 10 * time.Minute ) var schemaRE = regexp.MustCompile(`meta/schema_\d+_\d{10}\.json$`) @@ -123,6 +125,18 @@ type indexWithDate struct { currDate, prevDate string } +type tablePathStateKey struct { + table commonType.TableName + dispatcherID commonType.DispatcherID +} + +type tablePathState struct { + schemaVersion uint64 + fileIndex indexWithDate + indexReady bool + lastAccess time.Time +} + // VersionedTableName is used to wrap TableNameWithPhysicTableID with a version. type VersionedTableName struct { // Because we need to generate different file paths for different @@ -143,12 +157,18 @@ type FilePathGenerator struct { config *Config pdClock pdutil.Clock storage storage.ExternalStorage - fileIndex map[VersionedTableName]*indexWithDate + pathState map[tablePathStateKey]*tablePathState - hasher *hash.PositionInertia - versionMap map[VersionedTableName]uint64 + stateTTL time.Duration + stateCleanupInterval time.Duration + lastStateCleanupTime time.Time } +// Path state principles: +// 1. State is keyed by (table, dispatcher), so different dispatchers do not share file index. +// 2. schemaVersion change resets file index state to avoid mixing different schema versions. +// 3. state cleanup is lazy and interval-based to keep hot path overhead low. + // NewFilePathGenerator creates a FilePathGenerator. func NewFilePathGenerator( changefeedID commonType.ChangeFeedID, @@ -163,9 +183,10 @@ func NewFilePathGenerator( extension: extension, storage: storage, pdClock: pdClock, - fileIndex: make(map[VersionedTableName]*indexWithDate), - hasher: hash.NewPositionInertia(), - versionMap: make(map[VersionedTableName]uint64), + pathState: make(map[tablePathStateKey]*tablePathState), + + stateTTL: defaultPathStateTTL, + stateCleanupInterval: defaultPathStateCleanupInterval, } } @@ -177,7 +198,10 @@ func (f *FilePathGenerator) CheckOrWriteSchema( table VersionedTableName, tableInfo *commonType.TableInfo, ) (bool, error) { - if _, ok := f.versionMap[table]; ok { + now := f.currentTime() + f.cleanupExpiredPathState(now) + state := f.ensurePathState(table, now) + if state.schemaVersion == table.TableInfoVersion { return false, nil } @@ -196,14 +220,14 @@ func (f *FilePathGenerator) CheckOrWriteSchema( // Case 1: point check if the schema file exists. tblSchemaFile, err := def.GenerateSchemaFilePath() if err != nil { - return false, err + return false, errors.Trace(err) } exist, err := f.storage.FileExists(ctx, tblSchemaFile) if err != nil { - return false, err + return false, errors.Trace(err) } if exist { - f.versionMap[table] = table.TableInfoVersion + f.updateSchemaVersion(state, table.TableInfoVersion, now) return false, nil } @@ -241,7 +265,7 @@ func (f *FilePathGenerator) CheckOrWriteSchema( return nil }) if err != nil { - return false, err + return false, errors.Trace(err) } if hasNewerSchemaVersion { return true, nil @@ -257,7 +281,7 @@ func (f *FilePathGenerator) CheckOrWriteSchema( zap.Uint32("checksum", checksum)) // record the last version of the table schema file. // we don't need to write schema file to external storage again. - f.versionMap[table] = lastVersion + f.updateSchemaVersion(state, lastVersion, now) return false, nil } @@ -273,10 +297,13 @@ func (f *FilePathGenerator) CheckOrWriteSchema( } encodedDetail, err := def.MarshalWithQuery() if err != nil { - return false, err + return false, errors.Trace(err) } - f.versionMap[table] = table.TableInfoVersion - return false, f.storage.WriteFile(ctx, tblSchemaFile, encodedDetail) + if err := f.storage.WriteFile(ctx, tblSchemaFile, encodedDetail); err != nil { + return false, errors.Trace(err) + } + f.updateSchemaVersion(state, table.TableInfoVersion, now) + return false, nil } // SetClock is used for unit test @@ -328,10 +355,17 @@ func (f *FilePathGenerator) GenerateDataFilePath( func (f *FilePathGenerator) generateDataDirPath(tbl VersionedTableName, date string) string { var elems []string + now := f.currentTime() + state := f.ensurePathState(tbl, now) + version := state.schemaVersion + if version == 0 { + version = tbl.TableInfoVersion + f.updateSchemaVersion(state, version, now) + } elems = append(elems, tbl.TableNameWithPhysicTableID.Schema) elems = append(elems, tbl.TableNameWithPhysicTableID.Table) - elems = append(elems, fmt.Sprintf("%d", f.versionMap[tbl])) + elems = append(elems, fmt.Sprintf("%d", version)) if f.config.EnablePartitionSeparator && tbl.TableNameWithPhysicTableID.IsPartition { elems = append(elems, fmt.Sprintf("%d", tbl.TableNameWithPhysicTableID.TableID)) @@ -347,27 +381,33 @@ func (f *FilePathGenerator) generateDataDirPath(tbl VersionedTableName, date str func (f *FilePathGenerator) generateDataFileName( ctx context.Context, tbl VersionedTableName, date string, ) (string, error) { - if idx, ok := f.fileIndex[tbl]; !ok { + now := f.currentTime() + f.cleanupExpiredPathState(now) + state := f.ensurePathState(tbl, now) + if !state.indexReady { fileIdx, err := f.getFileIdxFromIndexFile(ctx, tbl, date) if err != nil { return "", err } - f.fileIndex[tbl] = &indexWithDate{ + state.fileIndex = indexWithDate{ prevDate: date, currDate: date, index: fileIdx, } + state.indexReady = true } else { - idx.currDate = date + state.fileIndex.currDate = date } // if date changed, reset the counter - if f.fileIndex[tbl].prevDate != f.fileIndex[tbl].currDate { - f.fileIndex[tbl].prevDate = f.fileIndex[tbl].currDate - f.fileIndex[tbl].index = 0 - } - f.fileIndex[tbl].index++ - return generateDataFileName(f.config.EnableTableAcrossNodes, tbl.DispatcherID.String(), f.fileIndex[tbl].index, f.extension, f.config.FileIndexWidth), nil + if state.fileIndex.prevDate != state.fileIndex.currDate { + state.fileIndex.prevDate = state.fileIndex.currDate + state.fileIndex.index = 0 + } + // Invariant: file index increases monotonically within the same (table, dispatcher, date) state. + state.fileIndex.index++ + state.lastAccess = now + return generateDataFileName(f.config.EnableTableAcrossNodes, tbl.DispatcherID.String(), state.fileIndex.index, f.extension, f.config.FileIndexWidth), nil } func (f *FilePathGenerator) getFileIdxFromIndexFile( @@ -376,7 +416,7 @@ func (f *FilePathGenerator) getFileIdxFromIndexFile( indexFile := f.GenerateIndexFilePath(tbl, date) exist, err := f.storage.FileExists(ctx, indexFile) if err != nil { - return 0, err + return 0, errors.Trace(err) } if !exist { return 0, nil @@ -384,12 +424,92 @@ func (f *FilePathGenerator) getFileIdxFromIndexFile( data, err := f.storage.ReadFile(ctx, indexFile) if err != nil { - return 0, err + return 0, errors.Trace(err) } fileName := strings.TrimSuffix(string(data), "\n") return FetchIndexFromFileName(fileName, f.extension) } +func (f *FilePathGenerator) setPathStateCleanupConfig(ttl, interval time.Duration) { + if ttl > 0 { + f.stateTTL = ttl + } + if interval > 0 { + f.stateCleanupInterval = interval + } +} + +func (f *FilePathGenerator) pathStateCount() int { + return len(f.pathState) +} + +func (f *FilePathGenerator) currentSchemaVersion(tbl VersionedTableName) uint64 { + state := f.pathState[f.pathStateKey(tbl)] + if state == nil { + return 0 + } + return state.schemaVersion +} + +func (f *FilePathGenerator) setCurrentSchemaVersion(tbl VersionedTableName, version uint64) { + now := f.currentTime() + state := f.ensurePathState(tbl, now) + f.updateSchemaVersion(state, version, now) +} + +func (f *FilePathGenerator) currentTime() time.Time { + if f.pdClock == nil { + return time.Now() + } + return f.pdClock.CurrentTime() +} + +func (f *FilePathGenerator) pathStateKey(tbl VersionedTableName) tablePathStateKey { + return tablePathStateKey{ + table: tbl.TableNameWithPhysicTableID, + dispatcherID: tbl.DispatcherID, + } +} + +func (f *FilePathGenerator) ensurePathState(tbl VersionedTableName, now time.Time) *tablePathState { + key := f.pathStateKey(tbl) + state := f.pathState[key] + if state == nil { + state = &tablePathState{} + f.pathState[key] = state + } + state.lastAccess = now + return state +} + +func (f *FilePathGenerator) updateSchemaVersion(state *tablePathState, version uint64, now time.Time) { + if state.schemaVersion != version { + state.schemaVersion = version + state.fileIndex = indexWithDate{} + state.indexReady = false + } + state.lastAccess = now +} + +func (f *FilePathGenerator) cleanupExpiredPathState(now time.Time) { + if f.stateTTL <= 0 || f.stateCleanupInterval <= 0 { + return + } + if !f.lastStateCleanupTime.IsZero() && now.Sub(f.lastStateCleanupTime) < f.stateCleanupInterval { + return + } + f.lastStateCleanupTime = now + for key, state := range f.pathState { + if state == nil { + delete(f.pathState, key) + continue + } + if now.Sub(state.lastAccess) >= f.stateTTL { + delete(f.pathState, key) + } + } +} + func FetchIndexFromFileName(fileName string, extension string) (uint64, error) { if len(fileName) < minFileNamePrefixLen+len(extension) || !strings.HasPrefix(fileName, "CDC") || @@ -398,17 +518,37 @@ func FetchIndexFromFileName(fileName string, extension string) (uint64, error) { fmt.Errorf("'%s' is a invalid file name", fileName)) } - // CDC[_{dispatcherID}_]{num}.fileExtension - pathRE, err := regexp.Compile(`CDC(?:_(\w+)_)?(\d+).\w+`) - if err != nil { - return 0, err + fileName = strings.TrimSuffix(fileName, extension) + indexPart := strings.TrimPrefix(fileName, "CDC") + + if strings.HasPrefix(indexPart, "_") { + trimmed := strings.TrimPrefix(indexPart, "_") + separatorIdx := strings.LastIndex(trimmed, "_") + if separatorIdx <= 0 || separatorIdx >= len(trimmed)-1 { + return 0, errors.WrapError(errors.ErrStorageSinkInvalidFileName, + fmt.Errorf("'%s' is a invalid file name", fileName)) + } + indexPart = trimmed[separatorIdx+1:] } - matches := pathRE.FindStringSubmatch(fileName) - if len(matches) != 3 { - return 0, fmt.Errorf("cannot match dml path pattern for %s", fileName) + if len(indexPart) < config.MinFileIndexWidth || !isNumberString(indexPart) { + return 0, errors.WrapError(errors.ErrStorageSinkInvalidFileName, + fmt.Errorf("'%s' is a invalid file name", fileName)) + } + + return strconv.ParseUint(indexPart, 10, 64) +} + +func isNumberString(value string) bool { + if value == "" { + return false + } + for _, ch := range value { + if ch < '0' || ch > '9' { + return false + } } - return strconv.ParseUint(matches[2], 10, 64) + return true } var dateSeparatorDayRegexp *regexp.Regexp diff --git a/pkg/sink/cloudstorage/path_key.go b/pkg/sink/cloudstorage/path_key.go deleted file mode 100644 index d948af9873..0000000000 --- a/pkg/sink/cloudstorage/path_key.go +++ /dev/null @@ -1,164 +0,0 @@ -// Copyright 2023 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package cloudstorage - -import ( - "fmt" - "regexp" - "strconv" - "strings" - - "github.com/pingcap/ticdc/pkg/common" - "github.com/pingcap/ticdc/pkg/config" - "github.com/pingcap/ticdc/pkg/errors" -) - -// SchemaPathKey is the key of schema path. -type SchemaPathKey struct { - Schema string - Table string - TableVersion uint64 -} - -// GetKey returns the key of schema path. -func (s *SchemaPathKey) GetKey() string { - return common.QuoteSchema(s.Schema, s.Table) -} - -// ParseSchemaFilePath parses the schema file path and returns the table version and checksum. -func (s *SchemaPathKey) ParseSchemaFilePath(path string) (uint32, error) { - // For /
/meta/schema_{tableVersion}_{checksum}.json, the parts - // should be ["", "
", "meta", "schema_{tableVersion}_{checksum}.json"]. - matches := strings.Split(path, "/") - - var schema, table string - schema = matches[0] - switch len(matches) { - case 3: - table = "" - case 4: - table = matches[1] - default: - return 0, errors.Trace(fmt.Errorf("cannot match schema path pattern for %s", path)) - } - - if matches[len(matches)-2] != "meta" { - return 0, errors.Trace(fmt.Errorf("cannot match schema path pattern for %s", path)) - } - - schemaFileName := matches[len(matches)-1] - version, checksum := mustParseSchemaName(schemaFileName) - - *s = SchemaPathKey{ - Schema: schema, - Table: table, - TableVersion: version, - } - return checksum, nil -} - -type FileIndexKey struct { - DispatcherID string - EnableTableAcrossNodes bool -} - -type FileIndex struct { - FileIndexKey - Idx uint64 -} - -// DmlPathKey is the key of dml path. -type DmlPathKey struct { - SchemaPathKey - PartitionNum int64 - Date string -} - -// GenerateDMLFilePath generates the dml file path. -func (d *DmlPathKey) GenerateDMLFilePath( - fileIndex *FileIndex, extension string, fileIndexWidth int, -) string { - var elems []string - - elems = append(elems, d.Schema) - elems = append(elems, d.Table) - elems = append(elems, fmt.Sprintf("%d", d.TableVersion)) - - if d.PartitionNum != 0 { - elems = append(elems, fmt.Sprintf("%d", d.PartitionNum)) - } - if len(d.Date) != 0 { - elems = append(elems, d.Date) - } - elems = append(elems, generateDataFileName(fileIndex.EnableTableAcrossNodes, fileIndex.DispatcherID, fileIndex.Idx, extension, fileIndexWidth)) - - return strings.Join(elems, "/") -} - -// ParseIndexFilePath parses the index file path and returns the max file index. -// index file path pattern is as follows: -// {schema}/{table}/{table-version-separator}/{partition-separator}/{date-separator}/meta/, where -// partition-separator and date-separator could be empty. -// DML file name pattern is as follows: CDC_{dispatcherID}.index or CDC.index -func (d *DmlPathKey) ParseIndexFilePath(dateSeparator, path string) (string, error) { - var partitionNum int64 - - str := `(\w+)\/(\w+)\/(\d+)\/(\d+)?\/*` - switch dateSeparator { - case config.DateSeparatorNone.String(): - str += `(\d{4})*` - case config.DateSeparatorYear.String(): - str += `(\d{4})\/` - case config.DateSeparatorMonth.String(): - str += `(\d{4}-\d{2})\/` - case config.DateSeparatorDay.String(): - str += `(\d{4}-\d{2}-\d{2})\/` - } - str += `meta\/` - // CDC[_{dispatcherID}].index - str += `CDC(?:_(\w+))?.index` - pathRE, err := regexp.Compile(str) - if err != nil { - return "", err - } - - matches := pathRE.FindStringSubmatch(path) - if len(matches) != 7 { - return "", fmt.Errorf("cannot match dml path pattern for %s", path) - } - - version, err := strconv.ParseUint(matches[3], 10, 64) - if err != nil { - return "", err - } - - if len(matches[4]) > 0 { - partitionNum, err = strconv.ParseInt(matches[4], 10, 64) - if err != nil { - return "", err - } - } - - *d = DmlPathKey{ - SchemaPathKey: SchemaPathKey{ - Schema: matches[1], - Table: matches[2], - TableVersion: version, - }, - PartitionNum: partitionNum, - Date: matches[5], - } - - return matches[6], nil -} diff --git a/pkg/sink/cloudstorage/path_key_test.go b/pkg/sink/cloudstorage/path_key_test.go deleted file mode 100644 index f768f7cf74..0000000000 --- a/pkg/sink/cloudstorage/path_key_test.go +++ /dev/null @@ -1,109 +0,0 @@ -// Copyright 2023 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package cloudstorage - -import ( - "fmt" - "testing" - - "github.com/pingcap/ticdc/pkg/common" - "github.com/stretchr/testify/require" -) - -func TestSchemaPathKey(t *testing.T) { - t.Parallel() - - testCases := []struct { - path string - schemakey SchemaPathKey - checksum uint32 - }{ - // Test for database schema path: /meta/schema_{tableVersion}_{checksum}.json - { - path: "test_schema/meta/schema_1_2.json", - schemakey: SchemaPathKey{ - Schema: "test_schema", - Table: "", - TableVersion: 1, - }, - checksum: 2, - }, - // Test for table schema path: /
/meta/schema_{tableVersion}_{checksum}.json - { - path: "test_schema/test_table/meta/schema_11_22.json", - schemakey: SchemaPathKey{ - Schema: "test_schema", - Table: "test_table", - TableVersion: 11, - }, - checksum: 22, - }, - } - for _, tc := range testCases { - var schemaKey SchemaPathKey - checksum, err := schemaKey.ParseSchemaFilePath(tc.path) - require.NoError(t, err) - require.Equal(t, tc.schemakey, schemaKey) - require.Equal(t, tc.checksum, checksum) - } -} - -func TestDmlPathKey(t *testing.T) { - t.Parallel() - - dispatcherID := common.NewDispatcherID() - testCases := []struct { - index uint64 - fileIndexWidth int - extension string - path string - indexPath string - dmlkey DmlPathKey - }{ - { - index: 10, - fileIndexWidth: 20, - extension: ".csv", - path: fmt.Sprintf("schema1/table1/123456/2023-05-09/CDC_%s_00000000000000000010.csv", dispatcherID.String()), - indexPath: fmt.Sprintf("schema1/table1/123456/2023-05-09/meta/CDC_%s.index", dispatcherID.String()), - dmlkey: DmlPathKey{ - SchemaPathKey: SchemaPathKey{ - Schema: "schema1", - Table: "table1", - TableVersion: 123456, - }, - PartitionNum: 0, - Date: "2023-05-09", - }, - }, - } - - for _, tc := range testCases { - var dmlkey DmlPathKey - id, err := dmlkey.ParseIndexFilePath("day", tc.indexPath) - require.NoError(t, err) - require.Equal(t, tc.dmlkey, dmlkey) - require.Equal(t, id, dispatcherID.String()) - - fileIndex := &FileIndex{ - FileIndexKey: FileIndexKey{ - DispatcherID: id, - EnableTableAcrossNodes: id != "", - }, - Idx: tc.index, - } - fileName := dmlkey.GenerateDMLFilePath(fileIndex, tc.extension, tc.fileIndexWidth) - require.Equal(t, tc.path, fileName) - } -} diff --git a/pkg/sink/cloudstorage/path_test.go b/pkg/sink/cloudstorage/path_test.go index 95a9989cd1..b1fc564b91 100644 --- a/pkg/sink/cloudstorage/path_test.go +++ b/pkg/sink/cloudstorage/path_test.go @@ -74,7 +74,7 @@ func TestGenerateDataFilePath(t *testing.T) { dir := t.TempDir() f := testFilePathGenerator(ctx, t, dir) - f.versionMap[table] = table.TableInfoVersion + f.setCurrentSchemaVersion(table, table.TableInfoVersion) date := f.GenerateDateStr() // date-separator: none path, err := f.GenerateDataFilePath(ctx, table, date) @@ -87,7 +87,7 @@ func TestGenerateDataFilePath(t *testing.T) { // date-separator: year mockClock := clock.NewMock() f = testFilePathGenerator(ctx, t, dir) - f.versionMap[table] = table.TableInfoVersion + f.setCurrentSchemaVersion(table, table.TableInfoVersion) f.config.DateSeparator = config.DateSeparatorYear.String() f.SetClock(pdutil.NewMonotonicClock(mockClock)) mockClock.Set(time.Date(2022, 12, 31, 23, 59, 59, 0, time.UTC)) @@ -111,7 +111,7 @@ func TestGenerateDataFilePath(t *testing.T) { // date-separator: month mockClock = clock.NewMock() f = testFilePathGenerator(ctx, t, dir) - f.versionMap[table] = table.TableInfoVersion + f.setCurrentSchemaVersion(table, table.TableInfoVersion) f.config.DateSeparator = config.DateSeparatorMonth.String() f.SetClock(pdutil.NewMonotonicClock(mockClock)) @@ -136,7 +136,7 @@ func TestGenerateDataFilePath(t *testing.T) { // date-separator: day mockClock = clock.NewMock() f = testFilePathGenerator(ctx, t, dir) - f.versionMap[table] = table.TableInfoVersion + f.setCurrentSchemaVersion(table, table.TableInfoVersion) f.config.DateSeparator = config.DateSeparatorDay.String() f.SetClock(pdutil.NewMonotonicClock(mockClock)) @@ -170,14 +170,22 @@ func TestFetchIndexFromFileName(t *testing.T) { testCases := []struct { fileName string wantErr string + index uint64 }{ { fileName: "CDC000011.json", wantErr: "", + index: 11, }, { fileName: "CDC1000000.json", wantErr: "", + index: 1000000, + }, + { + fileName: "CDC_dispatcher-1_000007.json", + wantErr: "", + index: 7, }, { fileName: "CDC1.json", @@ -195,14 +203,19 @@ func TestFetchIndexFromFileName(t *testing.T) { fileName: "CDChello.json", wantErr: "filename in storage sink is invalid", }, + { + fileName: "CDC_dispatcher_1.json", + wantErr: "filename in storage sink is invalid", + }, } for _, tc := range testCases { - _, err := FetchIndexFromFileName(tc.fileName, f.extension) + index, err := FetchIndexFromFileName(tc.fileName, f.extension) if len(tc.wantErr) != 0 { require.Contains(t, err.Error(), tc.wantErr) } else { require.NoError(t, err) + require.Equal(t, tc.index, index) } } } @@ -229,7 +242,7 @@ func TestGenerateDataFilePathWithIndexFile(t *testing.T) { TableInfoVersion: 5, DispatcherID: dispatcherID, } - f.versionMap[table] = table.TableInfoVersion + f.setCurrentSchemaVersion(table, table.TableInfoVersion) date := f.GenerateDateStr() indexFilePath := f.GenerateIndexFilePath(table, date) err := f.storage.WriteFile(ctx, indexFilePath, []byte(fmt.Sprintf("CDC_%s_000005.json\n", dispatcherID.String()))) @@ -307,21 +320,21 @@ func TestCheckOrWriteSchema(t *testing.T) { hasNewerSchemaVersion, err := f.CheckOrWriteSchema(ctx, table, tableInfo) require.NoError(t, err) require.False(t, hasNewerSchemaVersion) - require.Equal(t, table.TableInfoVersion, f.versionMap[table]) + require.Equal(t, table.TableInfoVersion, f.currentSchemaVersion(table)) // test old dml file can be ignored table.TableInfoVersion = 99 hasNewerSchemaVersion, err = f.CheckOrWriteSchema(ctx, table, tableInfo) require.NoError(t, err) require.True(t, hasNewerSchemaVersion) - require.Equal(t, 1, len(f.versionMap)) + require.Equal(t, 1, f.pathStateCount()) // test only table version changed, schema file should be reused table.TableInfoVersion = 101 hasNewerSchemaVersion, err = f.CheckOrWriteSchema(ctx, table, tableInfo) require.NoError(t, err) require.False(t, hasNewerSchemaVersion) - require.Equal(t, uint64(tidbInfo.Version), f.versionMap[table]) + require.Equal(t, uint64(tidbInfo.Version), f.currentSchemaVersion(table)) dir = filepath.Join(dir, "test/table1/meta") files, err := os.ReadDir(dir) @@ -335,17 +348,64 @@ func TestCheckOrWriteSchema(t *testing.T) { require.NoError(t, err) err = os.Remove(filepath.Join(dir, files[0].Name())) require.NoError(t, err) - delete(f.versionMap, table) + delete(f.pathState, f.pathStateKey(table)) hasNewerSchemaVersion, err = f.CheckOrWriteSchema(ctx, table, tableInfo) require.NoError(t, err) require.False(t, hasNewerSchemaVersion) - require.Equal(t, table.TableInfoVersion, f.versionMap[table]) + require.Equal(t, table.TableInfoVersion, f.currentSchemaVersion(table)) files, err = os.ReadDir(dir) require.NoError(t, err) require.Equal(t, 2, len(files)) } +func TestPathStateCleanup(t *testing.T) { + t.Parallel() + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + dir := t.TempDir() + f := testFilePathGenerator(ctx, t, dir) + + mockClock := clock.NewMock() + mockClock.Set(time.Date(2026, 2, 1, 0, 0, 0, 0, time.UTC)) + f.SetClock(pdutil.NewMonotonicClock(mockClock)) + f.setPathStateCleanupConfig(time.Second, time.Second) + + dispatcherID := commonType.NewDispatcherID() + table1 := VersionedTableName{ + TableNameWithPhysicTableID: commonType.TableName{ + Schema: "test", + Table: "table1", + TableID: 1, + }, + TableInfoVersion: 1, + DispatcherID: dispatcherID, + } + table2 := VersionedTableName{ + TableNameWithPhysicTableID: commonType.TableName{ + Schema: "test", + Table: "table2", + TableID: 2, + }, + TableInfoVersion: 2, + DispatcherID: dispatcherID, + } + + f.setCurrentSchemaVersion(table1, table1.TableInfoVersion) + _, err := f.GenerateDataFilePath(ctx, table1, "") + require.NoError(t, err) + require.Equal(t, 1, f.pathStateCount()) + + mockClock.Add(2 * time.Second) + f.setCurrentSchemaVersion(table2, table2.TableInfoVersion) + _, err = f.GenerateDataFilePath(ctx, table2, "") + require.NoError(t, err) + require.Equal(t, 1, f.pathStateCount()) + require.Equal(t, uint64(0), f.currentSchemaVersion(table1)) + require.Equal(t, table2.TableInfoVersion, f.currentSchemaVersion(table2)) +} + func TestRemoveExpiredFilesWithoutPartition(t *testing.T) { t.Parallel() diff --git a/storage-sink.md b/storage-sink.md new file mode 100644 index 0000000000..eef1fff335 --- /dev/null +++ b/storage-sink.md @@ -0,0 +1,169 @@ +# Storage Sink / Consumer 总览(唯一架构文档) + +> 目标:把 storage sink 与 storage consumer 的关键设计、语义约束、性能与运维信息收敛到一处,减少分散文档阅读成本。 +> +> 文档数量约束(storage sink 相关仅保留 3 份): +> 1) 本文(总体设计与运维) +> 2) `docs/plans/2026-02-04-cloudstorage-sink-ga-task-breakdown.md`(任务清单与完成状态) +> 3) `docs/plans/2026-02-14-cloudstorage-sink-spool-review-recommendations.md`(评审清单) + +## 1. 范围与核心目标 + +### 1.1 范围 +- 生产端:`downstreamadapter/sink/cloudstorage` + `pkg/sink/cloudstorage` +- 消费端:`cmd/storage-consumer` +- 外部系统:`storage.ExternalStorage`(本地文件、S3、GCS、Azure Blob) + +### 1.2 核心目标 +- 解决低流量多 dispatcher 场景下吞吐被 `flush-interval` 锁死的问题。 +- 避免 sink 内无界内存增长(OOM 风险可控)。 +- 保证 DDL / DML 顺序正确性不被 early-wake 破坏。 +- 支持大规模 dispatcher(目标口径:1000)长时间稳定运行。 + +### 1.3 非目标 +- 不改变 checkpoint 推进语义(仍然必须远端写成功后推进)。 +- 不改其他 sink(kafka/mysql/pulsar)的行为。 +- 不引入大量面向用户的新参数;复杂控制尽量内置。 + +## 2. 端到端数据流(sink + consumer) + +### 2.1 DML/DDL 写出链路(sink) +```text +Dispatcher + -> Sink.AddDMLEvent / PassBlockEvent / WriteBlockEvent + -> encoding group (task/future pipeline) + -> spool (memory + disk spill) + -> writer flush (data/index/schema) + -> PostFlush callback +``` + +### 2.2 回放链路(consumer) +```text +WalkDir 扫描 schema/index + -> 解析路径与 file index + -> 读取 data file 解码 + -> 聚合并投递到下游 sink + -> 等待 PostFlush 完成 +``` + +## 3. 必须保持的语义不变量 + +### 3.1 DML two-stage ack +- **enqueue ack(用于 wake)**:数据进入 sink 内可控承载后触发(用于解除 dynstream path 阻塞)。 +- **flush ack(用于 checkpoint)**:只有远端写成功后才触发 `PostFlush`。 +- 结论:wake 可以提前,checkpoint 不能提前。 + +### 3.2 DDL 顺序正确性 +- 对 DDL(commitTs=T),必须保证 ` spool -> writer flush 全链路。 + +### 3.3 路由与顺序 +- 同 dispatcher 路由稳定,避免写出顺序抖动。 +- 同 output shard 单消费者,保证 shard 内顺序可推理。 +- 同 index file 不能被多个 writer 并发写。 + +### 3.4 AddDMLEvent 约束 +- 入口保持 non-blocking,避免反向堵塞 dispatcher hot path。 + +## 4. Spool 机制(为什么要有 memory quota) + +### 4.1 设计动机 +如果仅靠内存积压“待写远端数据”,远端变慢时会出现: +- 队列持续增长 +- GC 放大 +- 最终 OOM + +因此需要 spool 做“可控承载 + 背压信号”,而不是把内存当无底洞。 + +### 4.2 语义边界 +- spool 是内部缓冲层,不是 checkpoint 边界。 +- 数据进入 spool 只能用于 wake 相关逻辑,不代表可推进 checkpoint。 + +### 4.3 memory quota 与水位控制 +- 按 changefeed 独立记账(包括 memory 与 spill 状态)。 +- 通过高/低水位(hysteresis)控制 wake: + - 超过高水位:抑制 wake,把背压留在上游调度层; + - 低于低水位:恢复 wake。 + +### 4.4 disk spill、segment、rotate、delete +- spill 采用 segment 追加写,减少小文件和随机写开销。 +- 一个 segment **可以混合多个 dispatcher(也可能包含多表数据)**,这是有意的文件数控制策略。 +- 快速读取不靠扫描文件,而靠每条记录的 `diskPtr`(segmentID + offset + length)直接定位。 +- rotate:当前 segment 达到阈值后切换到新 segment 写入。 +- delete:当旧 segment 中所有记录都被消费并释放后,删除该 segment 文件。 + +## 5. 文件布局与契约(sink 与 consumer 共识) + +### 5.1 schema/data/index +- schema:`/
/meta/schema_{version}_{checksum}.json` +- data:`.../CDC{index}.ext` 或 `.../CDC_{dispatcher}_{index}.ext` +- index:`.../meta/CDC.index` 或 `.../meta/CDC_{dispatcher}.index` + +### 5.2 关键契约 +- consumer 必须按路径中的 schema version 对应地读取与回放。 +- index 文件与 data 文件命名必须严格匹配,避免“读到错误文件序号”。 +- 路径解析逻辑已经就近放到 consumer 侧(`cmd/storage-consumer/path_key.go`),避免 `pkg` 承担仅 consumer 使用的职责。 + +## 6. DDL 行为矩阵(精简版) + +### 6.1 单表 DDL +- 未拆表:通常可不经 barrier,单 dispatcher 处理。 +- 拆表后:同表多个 dispatcher,需要 barrier 协调 write/pass。 + +### 6.2 多表 / DB / All DDL +- 走 barrier,由 maintainer 协调,通常一个 dispatcher write,其余 pass。 + +### 6.3 早唤醒下的额外要求 +- 不管是否 barrier,只要 DDL 会写 schema 文件,都要先完成 `PassBlockEvent` 对应 drain。 + +## 7. 性能瓶颈与优化优先级 + +### P0(必须优先) +- flush 作为放行边界导致吞吐节拍化。 +- 背压失控导致内存高水位与 OOM 风险。 +- DDL drain 覆盖不完整导致顺序风险。 + +### P1(本轮建议) +- index/schema 路径解析热路径开销(已做无正则化)。 +- consumer flush 错误链路必须完整上抛(已修复)。 +- consumer 状态(`tableDefMap` / `tableDMLIdxMap`)回收策略。 + +### P2(后续) +- 大目录下全量 `WalkDir` 的增量化扫描。 +- consumer 模块进一步拆分(scan / decode / apply / state)。 + +## 8. 监控与排障(最小集) + +### 8.1 先看吞吐是否被 interval 主导 +- flush reason:`size` vs `interval` 比例。 +- flush file size 分布(是否长期偏小)。 + +### 8.2 再看是否内存/背压问题 +- spool bytes / items +- wake suppressed 次数 +- writer 输入积压长度 + +### 8.3 DDL 风险定位 +- `ddl_drain_duration_seconds` +- DDL 前后 `