diff --git a/cdc/api/v2/model.go b/cdc/api/v2/model.go index 9e9218b2ca4..49e910fc526 100644 --- a/cdc/api/v2/model.go +++ b/cdc/api/v2/model.go @@ -341,7 +341,7 @@ func GetDefaultReplicaConfig() *ReplicaConfig { Consistent: &ConsistentConfig{ Level: "none", MaxLogSize: 64, - FlushIntervalInMs: 1000, + FlushIntervalInMs: config.MinFlushIntervalInMs, Storage: "", }, } diff --git a/cdc/owner/changefeed.go b/cdc/owner/changefeed.go index af81600e0b0..5c92040a2df 100644 --- a/cdc/owner/changefeed.go +++ b/cdc/owner/changefeed.go @@ -32,6 +32,7 @@ import ( cerror "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/orchestrator" "github.com/pingcap/tiflow/pkg/pdutil" + redoCfg "github.com/pingcap/tiflow/pkg/redo" "github.com/pingcap/tiflow/pkg/txnutil/gc" "github.com/pingcap/tiflow/pkg/upstream" "github.com/prometheus/client_golang/prometheus" @@ -662,7 +663,7 @@ func (c *changefeed) cleanupRedoManager(ctx context.Context) { log.Warn("changefeed is removed, but state is not complete", zap.Any("state", c.state)) return } - if !redo.IsConsistentEnabled(c.state.Info.Config.Consistent.Level) { + if !redoCfg.IsConsistentEnabled(c.state.Info.Config.Consistent.Level) { return } // when removing a paused changefeed, the redo manager is nil, create a new one diff --git a/cdc/owner/changefeed_test.go b/cdc/owner/changefeed_test.go index 2c96b23f604..aaeec0f79f6 100644 --- a/cdc/owner/changefeed_test.go +++ b/cdc/owner/changefeed_test.go @@ -485,8 +485,9 @@ func TestRemoveChangefeed(t *testing.T) { info := ctx.ChangefeedVars().Info dir := t.TempDir() info.Config.Consistent = &config.ConsistentConfig{ - Level: "eventual", - Storage: filepath.Join("nfs://", dir), + Level: "eventual", + Storage: filepath.Join("nfs://", dir), + FlushIntervalInMs: config.MinFlushIntervalInMs, } ctx = cdcContext.WithChangefeedVars(ctx, &cdcContext.ChangefeedVars{ ID: ctx.ChangefeedVars().ID, diff --git a/cdc/redo/applier.go b/cdc/redo/applier.go deleted file mode 100644 index e4d9ef520a6..00000000000 --- a/cdc/redo/applier.go +++ /dev/null @@ -1,34 +0,0 @@ -// Copyright 2021 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 redo - -import ( - "context" - - "github.com/pingcap/tiflow/cdc/redo/reader" - cerror "github.com/pingcap/tiflow/pkg/errors" -) - -// NewRedoReader creates a new redo log reader -func NewRedoReader(ctx context.Context, storage string, cfg *reader.LogReaderConfig) (rd reader.RedoLogReader, err error) { - switch consistentStorage(storage) { - case consistentStorageBlackhole: - rd = reader.NewBlackHoleReader() - case consistentStorageLocal, consistentStorageNFS, consistentStorageS3: - rd, err = reader.NewLogReader(ctx, cfg) - default: - err = cerror.ErrConsistentStorage.GenWithStackByArgs(storage) - } - return -} diff --git a/cdc/redo/convert.go b/cdc/redo/common/convert.go similarity index 99% rename from cdc/redo/convert.go rename to cdc/redo/common/convert.go index 4d18ab8ca50..534865559e3 100644 --- a/cdc/redo/convert.go +++ b/cdc/redo/common/convert.go @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package redo +package common import ( "bytes" diff --git a/cdc/redo/convert_test.go b/cdc/redo/common/convert_test.go similarity index 99% rename from cdc/redo/convert_test.go rename to cdc/redo/common/convert_test.go index 67d2eae052e..57bb993abff 100644 --- a/cdc/redo/convert_test.go +++ b/cdc/redo/common/convert_test.go @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package redo +package common import ( "testing" diff --git a/cdc/redo/common/redo.go b/cdc/redo/common/redo.go deleted file mode 100644 index 377f8d461e1..00000000000 --- a/cdc/redo/common/redo.go +++ /dev/null @@ -1,75 +0,0 @@ -// Copyright 2021 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. - -//go:generate msgp - -package common - -import ( - "github.com/pingcap/tiflow/cdc/model" -) - -const ( - // MinSectorSize is minimum sector size used when flushing log so that log can safely - // distinguish between torn writes and ordinary data corruption. - MinSectorSize = 512 -) - -const ( - // TmpEXT is the file ext of log file before safely wrote to disk - TmpEXT = ".tmp" - // LogEXT is the file ext of log file after safely wrote to disk - LogEXT = ".log" - // MetaEXT is the meta file ext of meta file after safely wrote to disk - MetaEXT = ".meta" - // MetaTmpEXT is the meta file ext of meta file before safely wrote to disk - MetaTmpEXT = ".mtmp" - // SortLogEXT is the sorted log file ext of log file after safely wrote to disk - SortLogEXT = ".sort" -) - -const ( - // DefaultFileMode is the default mode when operation files - DefaultFileMode = 0o644 - // DefaultDirMode is the default mode when operation dir - DefaultDirMode = 0o755 -) - -const ( - // DefaultMetaFileType is the default file type of meta file - DefaultMetaFileType = "meta" - // DefaultRowLogFileType is the default file type of row log file - DefaultRowLogFileType = "row" - // DefaultDDLLogFileType is the default file type of ddl log file - DefaultDDLLogFileType = "ddl" -) - -// LogMeta is used for store meta info. -type LogMeta struct { - CheckpointTs uint64 `msg:"checkpointTs"` - ResolvedTs uint64 `msg:"resolvedTs"` -} - -// ParseMeta parses meta. -func ParseMeta(metas []*LogMeta, checkpointTs, resolvedTs *model.Ts) { - *checkpointTs = 0 - *resolvedTs = 0 - for _, meta := range metas { - if *checkpointTs < meta.CheckpointTs { - *checkpointTs = meta.CheckpointTs - } - if *resolvedTs < meta.ResolvedTs { - *resolvedTs = meta.ResolvedTs - } - } -} diff --git a/cdc/redo/common/redo_meta.go b/cdc/redo/common/redo_meta.go new file mode 100644 index 00000000000..a62d54e793b --- /dev/null +++ b/cdc/redo/common/redo_meta.go @@ -0,0 +1,40 @@ +// 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. + +//go:generate msgp + +package common + +import ( + "github.com/pingcap/tiflow/cdc/model" +) + +// LogMeta is used for store meta info. +type LogMeta struct { + CheckpointTs uint64 `msg:"checkpointTs"` + ResolvedTs uint64 `msg:"resolvedTs"` +} + +// ParseMeta parses meta. +func ParseMeta(metas []*LogMeta, checkpointTs, resolvedTs *model.Ts) { + *checkpointTs = 0 + *resolvedTs = 0 + for _, meta := range metas { + if *checkpointTs < meta.CheckpointTs { + *checkpointTs = meta.CheckpointTs + } + if *resolvedTs < meta.ResolvedTs { + *resolvedTs = meta.ResolvedTs + } + } +} diff --git a/cdc/redo/common/redo_gen.go b/cdc/redo/common/redo_meta_gen.go similarity index 100% rename from cdc/redo/common/redo_gen.go rename to cdc/redo/common/redo_meta_gen.go diff --git a/cdc/redo/common/redo_gen_test.go b/cdc/redo/common/redo_meta_gen_test.go similarity index 100% rename from cdc/redo/common/redo_gen_test.go rename to cdc/redo/common/redo_meta_gen_test.go diff --git a/cdc/redo/common/util.go b/cdc/redo/common/util.go index 5607bdb99a5..1d3d374d7b5 100644 --- a/cdc/redo/common/util.go +++ b/cdc/redo/common/util.go @@ -14,156 +14,12 @@ package common import ( - "context" "fmt" - "net/url" - "path/filepath" "strings" - "time" - "github.com/aws/aws-sdk-go/aws/client" - "github.com/aws/aws-sdk-go/aws/request" - "github.com/pingcap/errors" - backuppb "github.com/pingcap/kvproto/pkg/brpb" - "github.com/pingcap/log" - "github.com/pingcap/tidb/br/pkg/storage" "github.com/pingcap/tiflow/cdc/model" - cerror "github.com/pingcap/tiflow/pkg/errors" - "go.uber.org/zap" ) -const ( - // RedoLogFileFormatV1 was used before v6.1.0, which doesn't contain namespace information - // layout: captureID_changefeedID_fileType_maxEventCommitTs_uuid.fileExtName - RedoLogFileFormatV1 = "%s_%s_%s_%d_%s%s" - // RedoLogFileFormatV2 is available since v6.1.0, which contains namespace information - // layout: captureID_namespace_changefeedID_fileType_maxEventCommitTs_uuid.fileExtName - RedoLogFileFormatV2 = "%s_%s_%s_%s_%d_%s%s" -) - -// InitS3storage init a storage used for s3, -// s3URI should be like s3URI="s3://logbucket/test-changefeed?endpoint=http://$S3_ENDPOINT/" -var InitS3storage = func(ctx context.Context, uri url.URL) (storage.ExternalStorage, error) { - if len(uri.Host) == 0 { - return nil, cerror.WrapChangefeedUnretryableErr(cerror.ErrS3StorageInitialize, errors.Errorf("please specify the bucket for s3 in %v", uri)) - } - - prefix := strings.Trim(uri.Path, "/") - s3 := &backuppb.S3{Bucket: uri.Host, Prefix: prefix} - options := &storage.BackendOptions{} - storage.ExtractQueryParameters(&uri, &options.S3) - if err := options.S3.Apply(s3); err != nil { - return nil, cerror.WrapChangefeedUnretryableErr(cerror.ErrS3StorageInitialize, err) - } - - // we should set this to true, since br set it by default in parseBackend - s3.ForcePathStyle = true - backend := &backuppb.StorageBackend{ - Backend: &backuppb.StorageBackend_S3{S3: s3}, - } - s3storage, err := storage.New(ctx, backend, &storage.ExternalStorageOptions{ - SendCredentials: false, - HTTPClient: nil, - S3Retryer: DefaultS3Retryer(), - }) - if err != nil { - return nil, cerror.WrapChangefeedUnretryableErr(cerror.ErrS3StorageInitialize, err) - } - - return s3storage, nil -} - -// logFormat2ParseFormat converts redo log file name format to the space separated -// format, which can be read and parsed by sscanf. Besides remove the suffix `%s` -// which is used as file name extension, since we will parse extension first. -func logFormat2ParseFormat(fmtStr string) string { - return strings.TrimSuffix(strings.ReplaceAll(fmtStr, "_", " "), "%s") -} - -// ParseLogFileName extract the commitTs, fileType from log fileName -func ParseLogFileName(name string) (uint64, string, error) { - ext := filepath.Ext(name) - if ext == MetaEXT { - return 0, DefaultMetaFileType, nil - } - - // if .sort, the name should be like - // fmt.Sprintf("%s_%s_%s_%d_%s_%d%s", w.cfg.captureID, - // w.cfg.changeFeedID.Namespace,w.cfg.changeFeedID.ID, - // w.cfg.fileType, w.commitTS.Load(), uuid, LogEXT)+SortLogEXT - if ext == SortLogEXT { - name = strings.TrimSuffix(name, SortLogEXT) - ext = filepath.Ext(name) - } - if ext != LogEXT && ext != TmpEXT { - return 0, "", nil - } - - var commitTs uint64 - var captureID, namespace, changefeedID, fileType, uid string - // if the namespace is not default, the log looks like: - // fmt.Sprintf("%s_%s_%s_%s_%d_%s%s", w.cfg.captureID, - // w.cfg.changeFeedID.Namespace,w.cfg.changeFeedID.ID, - // w.cfg.fileType, w.commitTS.Load(), uuid, redo.LogEXT) - // otherwise it looks like: - // fmt.Sprintf("%s_%s_%s_%d_%s%s", w.cfg.captureID, - // w.cfg.changeFeedID.ID, - // w.cfg.fileType, w.commitTS.Load(), uuid, redo.LogEXT) - var ( - vars []any - formatStr string - ) - if len(strings.Split(name, "_")) == 6 { - formatStr = logFormat2ParseFormat(RedoLogFileFormatV2) - vars = []any{&captureID, &namespace, &changefeedID, &fileType, &commitTs, &uid} - } else { - formatStr = logFormat2ParseFormat(RedoLogFileFormatV1) - vars = []any{&captureID, &changefeedID, &fileType, &commitTs, &uid} - } - name = strings.ReplaceAll(name, "_", " ") - _, err := fmt.Sscanf(name, formatStr, vars...) - if err != nil { - return 0, "", errors.Annotatef(err, "bad log name: %s", name) - } - return commitTs, fileType, nil -} - -// retryerWithLog wraps the client.DefaultRetryer, and logs when retrying. -type retryerWithLog struct { - client.DefaultRetryer -} - -func isDeadlineExceedError(err error) bool { - return strings.Contains(err.Error(), "context deadline exceeded") -} - -func (rl retryerWithLog) ShouldRetry(r *request.Request) bool { - if isDeadlineExceedError(r.Error) { - return false - } - return rl.DefaultRetryer.ShouldRetry(r) -} - -func (rl retryerWithLog) RetryRules(r *request.Request) time.Duration { - backoffTime := rl.DefaultRetryer.RetryRules(r) - if backoffTime > 0 { - log.Warn("failed to request s3, retrying", zap.Error(r.Error), zap.Duration("backoff", backoffTime)) - } - return backoffTime -} - -// DefaultS3Retryer is the default s3 retryer, maybe this function -// should be extracted to another place. -func DefaultS3Retryer() request.Retryer { - return retryerWithLog{ - DefaultRetryer: client.DefaultRetryer{ - NumMaxRetries: 3, - MinRetryDelay: 1 * time.Second, - MinThrottleDelay: 2 * time.Second, - }, - } -} - // FilterChangefeedFiles return the files that match to the changefeed. func FilterChangefeedFiles(files []string, changefeedID model.ChangeFeedID) []string { var ( diff --git a/cdc/redo/common/util_test.go b/cdc/redo/common/util_test.go index e8b08900557..c07085829ef 100644 --- a/cdc/redo/common/util_test.go +++ b/cdc/redo/common/util_test.go @@ -14,162 +14,15 @@ package common import ( - "fmt" "testing" - "github.com/google/uuid" "github.com/pingcap/tiflow/cdc/model" "github.com/stretchr/testify/require" ) -func TestParseLogFileName(t *testing.T) { - type arg struct { - name string - } - tests := []struct { - name string - args arg - wantTs uint64 - wantFileType string - wantErr string - }{ - { - name: "happy row .log", - args: arg{ - name: fmt.Sprintf(RedoLogFileFormatV1, "cp", - "test", - DefaultRowLogFileType, 1, uuid.NewString(), LogEXT), - }, - wantTs: 1, - wantFileType: DefaultRowLogFileType, - }, - { - name: "happy row .log", - args: arg{ - name: fmt.Sprintf(RedoLogFileFormatV2, "cp", - "namespace", "test", - DefaultRowLogFileType, 1, uuid.NewString(), LogEXT), - }, - wantTs: 1, - wantFileType: DefaultRowLogFileType, - }, - { - name: "happy row .tmp", - args: arg{ - name: fmt.Sprintf(RedoLogFileFormatV1, "cp", - "test", - DefaultRowLogFileType, 1, uuid.NewString(), LogEXT) + TmpEXT, - }, - wantTs: 1, - wantFileType: DefaultRowLogFileType, - }, - { - name: "happy row .tmp", - args: arg{ - name: fmt.Sprintf(RedoLogFileFormatV2, "cp", - "namespace", "test", - DefaultRowLogFileType, 1, uuid.NewString(), LogEXT) + TmpEXT, - }, - wantTs: 1, - wantFileType: DefaultRowLogFileType, - }, - { - name: "happy ddl .log", - args: arg{ - name: fmt.Sprintf(RedoLogFileFormatV1, "cp", - "test", - DefaultDDLLogFileType, 1, uuid.NewString(), LogEXT), - }, - wantTs: 1, - wantFileType: DefaultDDLLogFileType, - }, - { - name: "happy ddl .log", - args: arg{ - name: fmt.Sprintf(RedoLogFileFormatV2, "cp", - "namespace", "test", - DefaultDDLLogFileType, 1, uuid.NewString(), LogEXT), - }, - wantTs: 1, - wantFileType: DefaultDDLLogFileType, - }, - { - name: "happy ddl .sort", - args: arg{ - name: fmt.Sprintf(RedoLogFileFormatV2, "cp", - "default", "test", - DefaultDDLLogFileType, 1, uuid.NewString(), LogEXT) + SortLogEXT, - }, - wantTs: 1, - wantFileType: DefaultDDLLogFileType, - }, - { - name: "happy ddl .sort", - args: arg{ - name: fmt.Sprintf(RedoLogFileFormatV2, "cp", - "namespace", "test", - DefaultDDLLogFileType, 1, uuid.NewString(), LogEXT) + SortLogEXT, - }, - wantTs: 1, - wantFileType: DefaultDDLLogFileType, - }, - { - name: "happy ddl .tmp", - args: arg{ - name: fmt.Sprintf(RedoLogFileFormatV1, "cp", - "test", - DefaultDDLLogFileType, 1, uuid.NewString(), LogEXT) + TmpEXT, - }, - wantTs: 1, - wantFileType: DefaultDDLLogFileType, - }, - { - name: "happy ddl .tmp", - args: arg{ - name: fmt.Sprintf(RedoLogFileFormatV2, "cp", - "namespace", "test", - DefaultDDLLogFileType, 1, uuid.NewString(), LogEXT) + TmpEXT, - }, - wantTs: 1, - wantFileType: DefaultDDLLogFileType, - }, - { - name: "happy .meta", - args: arg{ - name: "sdfsdfsf" + MetaEXT, - }, - wantTs: 0, - wantFileType: DefaultMetaFileType, - }, - { - name: "not supported fileType", - args: arg{ - name: "sdfsdfsf.sfsf", - }, - }, - { - name: "err wrong format ddl .tmp", - args: arg{ - name: fmt.Sprintf("%s_%s_%s_%s_%d%s%s", /* a wrong format */ - "cp", "default", "test", - DefaultDDLLogFileType, 1, uuid.NewString(), LogEXT) + TmpEXT, - }, - wantErr: ".*bad log name*.", - }, - } - for _, tt := range tests { - ts, fileType, err := ParseLogFileName(tt.args.name) - if tt.wantErr != "" { - require.Regexp(t, tt.wantErr, err, tt.name) - } else { - require.Nil(t, err, tt.name) - require.EqualValues(t, tt.wantTs, ts, tt.name) - require.Equal(t, tt.wantFileType, fileType, tt.name) - } - } -} - func TestGetChangefeedFiles(t *testing.T) { + t.Parallel() + cases := []struct { fileNames []string changefeed model.ChangeFeedID diff --git a/cdc/redo/manager.go b/cdc/redo/manager.go index 7f4f799f8fc..235f0276dbe 100644 --- a/cdc/redo/manager.go +++ b/cdc/redo/manager.go @@ -16,14 +16,12 @@ package redo import ( "context" "math" - "path/filepath" "sync" "sync/atomic" "time" "github.com/pingcap/errors" "github.com/pingcap/log" - "github.com/pingcap/tidb/br/pkg/storage" "github.com/pingcap/tiflow/cdc/contextutil" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/cdc/redo/common" @@ -31,68 +29,19 @@ import ( "github.com/pingcap/tiflow/pkg/chann" "github.com/pingcap/tiflow/pkg/config" cerror "github.com/pingcap/tiflow/pkg/errors" +<<<<<<< HEAD +======= + "github.com/pingcap/tiflow/pkg/redo" + "github.com/pingcap/tiflow/pkg/spanz" +>>>>>>> dacb992c39 (redo(ticdc): support gcs scheme in redo log (#7993)) "github.com/prometheus/client_golang/prometheus" "go.uber.org/zap" ) -var ( - // flushIntervalInMs is the minimum value of flush interval - flushIntervalInMs int64 = 2000 // 2 seconds - flushTimeout = time.Second * 20 +const redoFlushWarnDuration = time.Second * 20 - // Redo Manager GC interval. It can be changed in tests. - defaultGCIntervalInMs = 5000 // 5 seconds -) - -// ConsistentLevelType is the level of redo log consistent level. -type ConsistentLevelType string - -const ( - // ConsistentLevelNone no consistent guarantee. - ConsistentLevelNone ConsistentLevelType = "none" - // ConsistentLevelEventual eventual consistent. - ConsistentLevelEventual ConsistentLevelType = "eventual" -) - -type consistentStorage string - -const ( - consistentStorageLocal consistentStorage = "local" - consistentStorageNFS consistentStorage = "nfs" - consistentStorageS3 consistentStorage = "s3" - consistentStorageBlackhole consistentStorage = "blackhole" -) - -// IsValidConsistentLevel checks whether a given consistent level is valid -func IsValidConsistentLevel(level string) bool { - switch ConsistentLevelType(level) { - case ConsistentLevelNone, ConsistentLevelEventual: - return true - default: - return false - } -} - -// IsValidConsistentStorage checks whether a give consistent storage is valid -func IsValidConsistentStorage(storage string) bool { - switch consistentStorage(storage) { - case consistentStorageLocal, consistentStorageNFS, - consistentStorageS3, consistentStorageBlackhole: - return true - default: - return false - } -} - -// IsConsistentEnabled returns whether the consistent feature is enabled -func IsConsistentEnabled(level string) bool { - return IsValidConsistentLevel(level) && ConsistentLevelType(level) != ConsistentLevelNone -} - -// IsS3StorageEnabled returns whether s3 storage is enabled -func IsS3StorageEnabled(storage string) bool { - return consistentStorage(storage) == consistentStorageS3 -} +// Redo Manager GC interval. It can be changed in tests. +var defaultGCIntervalInMs = 5000 // 5 seconds // LogManager defines an interface that is used to manage redo log type LogManager interface { @@ -164,8 +113,6 @@ func (s *statefulRts) checkAndSetUnflushed(unflushed model.Ts) { type ManagerImpl struct { changeFeedID model.ChangeFeedID enabled bool - level ConsistentLevelType - storageType consistentStorage opts *ManagerOptions @@ -201,93 +148,41 @@ type ManagerImpl struct { // NewManager creates a new Manager func NewManager(ctx context.Context, cfg *config.ConsistentConfig, opts *ManagerOptions) (*ManagerImpl, error) { // return a disabled Manager if no consistent config or normal consistent level - if cfg == nil || ConsistentLevelType(cfg.Level) == ConsistentLevelNone { + if cfg == nil || !redo.IsConsistentEnabled(cfg.Level) { return &ManagerImpl{enabled: false}, nil } - if cfg.FlushIntervalInMs > flushIntervalInMs { - flushIntervalInMs = cfg.FlushIntervalInMs - } - uri, err := storage.ParseRawURL(cfg.Storage) + writer, err := writer.NewRedoLogWriter(ctx, cfg, opts.FileTypeConfig) if err != nil { return nil, err } - changeFeedID := contextutil.ChangefeedIDFromCtx(ctx) m := &ManagerImpl{ - changeFeedID: changeFeedID, + changeFeedID: contextutil.ChangefeedIDFromCtx(ctx), enabled: true, - level: ConsistentLevelType(cfg.Level), - storageType: consistentStorage(uri.Scheme), opts: opts, rtsMap: sync.Map{}, minResolvedTs: math.MaxInt64, - - metricWriteLogDuration: common.RedoWriteLogDurationHistogram. - WithLabelValues(changeFeedID.Namespace, changeFeedID.ID), - metricFlushLogDuration: common.RedoFlushLogDurationHistogram. - WithLabelValues(changeFeedID.Namespace, changeFeedID.ID), + writer: writer, } - switch m.storageType { - case consistentStorageBlackhole: - m.writer = writer.NewBlackHoleWriter() - case consistentStorageLocal, consistentStorageNFS, consistentStorageS3: - globalConf := config.GetGlobalServerConfig() - // When an external storage such S3 is used, we use redoDir as a temporary dir to store redo logs - // before we flush them to S3. - var redoDir string - if changeFeedID.Namespace == model.DefaultNamespace { - redoDir = filepath.Join(globalConf.DataDir, - config.DefaultRedoDir, changeFeedID.ID) - } else { - redoDir = filepath.Join(globalConf.DataDir, - config.DefaultRedoDir, - changeFeedID.Namespace, changeFeedID.ID) - } - - // When local storage or NFS is used, we use redoDir as the final storage path. - if m.storageType == consistentStorageLocal || m.storageType == consistentStorageNFS { - redoDir = uri.Path - } - - writerCfg := &writer.LogWriterConfig{ - Dir: redoDir, - CaptureID: contextutil.CaptureAddrFromCtx(ctx), - ChangeFeedID: changeFeedID, - CreateTime: time.Now(), - MaxLogSize: cfg.MaxLogSize, - FlushIntervalInMs: cfg.FlushIntervalInMs, - S3Storage: m.storageType == consistentStorageS3, - - EmitMeta: m.opts.EmitMeta, - EmitRowEvents: m.opts.EmitRowEvents, - EmitDDLEvents: m.opts.EmitDDLEvents, - } - if writerCfg.S3Storage { - writerCfg.S3URI = *uri - } - writer, err := writer.NewLogWriter(ctx, writerCfg) - if err != nil { - return nil, err - } - m.writer = writer - - if m.opts.EmitMeta { - checkpointTs, resolvedTs := m.writer.GetMeta() - m.metaCheckpointTs.flushed = checkpointTs - m.metaCheckpointTs.unflushed = checkpointTs - m.metaResolvedTs.flushed = resolvedTs - m.metaResolvedTs.unflushed = resolvedTs - } - default: - return nil, cerror.ErrConsistentStorage.GenWithStackByArgs(m.storageType) + if m.opts.EmitMeta { + checkpointTs, resolvedTs := m.writer.GetMeta() + m.metaCheckpointTs.flushed = checkpointTs + m.metaCheckpointTs.unflushed = checkpointTs + m.metaResolvedTs.flushed = resolvedTs + m.metaResolvedTs.unflushed = resolvedTs } + m.metricWriteLogDuration = common.RedoWriteLogDurationHistogram. + WithLabelValues(m.changeFeedID.Namespace, m.changeFeedID.ID) + m.metricFlushLogDuration = common.RedoFlushLogDurationHistogram. + WithLabelValues(m.changeFeedID.Namespace, m.changeFeedID.ID) + // TODO: better to wait background goroutines after the context is canceled. if m.opts.EnableBgRunner { m.logBuffer = chann.New[cacheEvents]() - go m.bgUpdateLog(ctx, opts.ErrCh) + go m.bgUpdateLog(ctx, cfg.FlushIntervalInMs, opts.ErrCh) } if m.opts.EnableGCRunner { go m.bgGC(ctx) @@ -304,8 +199,9 @@ func NewDisabledManager() *ManagerImpl { // NewMockManager returns a mock redo manager instance, used in test only func NewMockManager(ctx context.Context) (*ManagerImpl, error) { cfg := &config.ConsistentConfig{ - Level: string(ConsistentLevelEventual), - Storage: "blackhole://", + Level: string(redo.ConsistentLevelEventual), + Storage: "blackhole://", + FlushIntervalInMs: config.MinFlushIntervalInMs, } errCh := make(chan error, 1) @@ -390,7 +286,9 @@ func (m *ManagerImpl) UpdateCheckpointTs(ckpt model.Ts) { // EmitDDLEvent sends DDL event to redo log writer func (m *ManagerImpl) EmitDDLEvent(ctx context.Context, ddl *model.DDLEvent) error { - return m.withLock(func(m *ManagerImpl) error { return m.writer.SendDDL(ctx, DDLToRedo(ddl)) }) + return m.withLock(func(m *ManagerImpl) error { + return m.writer.SendDDL(ctx, common.DDLToRedo(ddl)) + }) } // GetResolvedTs returns the resolved ts of a table @@ -538,7 +436,7 @@ func (m *ManagerImpl) flushLog(ctx context.Context, handleErr func(err error)) { if !atomic.CompareAndSwapInt64(&m.flushing, 0, 1) { log.Debug("Fail to update flush flag, " + "the previous flush operation hasn't finished yet") - if time.Since(m.lastFlushTime) > flushTimeout { + if time.Since(m.lastFlushTime) > redoFlushWarnDuration { log.Warn("flushLog blocking too long, the redo manager may be stuck", zap.Duration("duration", time.Since(m.lastFlushTime)), zap.Any("changfeed", m.changeFeedID)) @@ -578,7 +476,9 @@ func (m *ManagerImpl) onResolvedTsMsg(tableID model.TableID, resolvedTs model.Ts value.(*statefulRts).checkAndSetUnflushed(resolvedTs) } -func (m *ManagerImpl) bgUpdateLog(ctx context.Context, errCh chan<- error) { +func (m *ManagerImpl) bgUpdateLog( + ctx context.Context, flushIntervalInMs int64, errCh chan<- error, +) { // logErrCh is used to retrieve errors from log flushing goroutines. // if the channel is full, it's better to block subsequent flushing goroutines. logErrCh := make(chan error, 1) @@ -672,7 +572,7 @@ func (m *ManagerImpl) bgUpdateLog(ctx context.Context, errCh chan<- error) { switch cache.eventType { case model.MessageTypeRow: for _, row := range cache.rows { - logs = append(logs, RowToRedo(row)) + logs = append(logs, common.RowToRedo(row)) } if cache.releaseMemory != nil { releaseMemoryCbs = append(releaseMemoryCbs, cache.releaseMemory) @@ -702,7 +602,7 @@ func (m *ManagerImpl) bgUpdateLog(ctx context.Context, errCh chan<- error) { switch cache.eventType { case model.MessageTypeRow: for _, row := range cache.rows { - logs = append(logs, RowToRedo(row)) + logs = append(logs, common.RowToRedo(row)) } if cache.releaseMemory != nil { releaseMemoryCbs = append(releaseMemoryCbs, cache.releaseMemory) diff --git a/cdc/redo/manager_test.go b/cdc/redo/manager_test.go index eba93903f41..e8b8dd12382 100644 --- a/cdc/redo/manager_test.go +++ b/cdc/redo/manager_test.go @@ -27,6 +27,11 @@ import ( "github.com/pingcap/tiflow/cdc/redo/writer" "github.com/pingcap/tiflow/pkg/chann" "github.com/pingcap/tiflow/pkg/config" +<<<<<<< HEAD +======= + "github.com/pingcap/tiflow/pkg/redo" + "github.com/pingcap/tiflow/pkg/spanz" +>>>>>>> dacb992c39 (redo(ticdc): support gcs scheme in redo log (#7993)) "github.com/stretchr/testify/require" "go.uber.org/zap" ) @@ -43,7 +48,7 @@ func TestConsistentConfig(t *testing.T) { {"", false}, } for _, lc := range levelCases { - require.Equal(t, lc.valid, IsValidConsistentLevel(lc.level)) + require.Equal(t, lc.valid, redo.IsValidConsistentLevel(lc.level)) } levelEnableCases := []struct { @@ -55,7 +60,7 @@ func TestConsistentConfig(t *testing.T) { {"eventual", true}, } for _, lc := range levelEnableCases { - require.Equal(t, lc.consistent, IsConsistentEnabled(lc.level)) + require.Equal(t, lc.consistent, redo.IsConsistentEnabled(lc.level)) } storageCases := []struct { @@ -70,7 +75,7 @@ func TestConsistentConfig(t *testing.T) { {"", false}, } for _, sc := range storageCases { - require.Equal(t, sc.valid, IsValidConsistentStorage(sc.storage)) + require.Equal(t, sc.valid, redo.IsValidConsistentStorage(sc.storage)) } s3StorageCases := []struct { @@ -83,7 +88,7 @@ func TestConsistentConfig(t *testing.T) { {"blackhole", false}, } for _, sc := range s3StorageCases { - require.Equal(t, sc.s3Enabled, IsS3StorageEnabled(sc.storage)) + require.Equal(t, sc.s3Enabled, redo.IsExternalStorage(sc.storage)) } } @@ -99,7 +104,7 @@ func TestLogManagerInProcessor(t *testing.T) { defer logMgr.Cleanup(ctx) checkResolvedTs := func(mgr LogManager, expectedRts uint64) { - time.Sleep(time.Duration(flushIntervalInMs+200) * time.Millisecond) + time.Sleep(time.Duration(config.MinFlushIntervalInMs+200) * time.Millisecond) resolvedTs := mgr.GetMinResolvedTs() require.Equal(t, expectedRts, resolvedTs) } @@ -319,8 +324,9 @@ func TestManagerError(t *testing.T) { defer cancel() cfg := &config.ConsistentConfig{ - Level: string(ConsistentLevelEventual), - Storage: "blackhole://", + Level: string(redo.ConsistentLevelEventual), + Storage: "blackhole://", + FlushIntervalInMs: config.MinFlushIntervalInMs, } errCh := make(chan error, 1) @@ -331,7 +337,7 @@ func TestManagerError(t *testing.T) { require.Nil(t, err) logMgr.writer = writer.NewInvalidBlackHoleWriter(logMgr.writer) logMgr.logBuffer = chann.New[cacheEvents]() - go logMgr.bgUpdateLog(ctx, errCh) + go logMgr.bgUpdateLog(ctx, cfg.FlushIntervalInMs, errCh) testCases := []struct { tableID model.TableID @@ -364,7 +370,7 @@ func TestManagerError(t *testing.T) { require.Nil(t, err) logMgr.writer = writer.NewInvalidBlackHoleWriter(logMgr.writer) logMgr.logBuffer = chann.New[cacheEvents]() - go logMgr.bgUpdateLog(ctx, errCh) + go logMgr.bgUpdateLog(ctx, cfg.FlushIntervalInMs, errCh) // bgUpdateLog exists because of writer.FlushLog failure. select { @@ -383,8 +389,9 @@ func TestReuseWritter(t *testing.T) { dir := t.TempDir() cfg := &config.ConsistentConfig{ - Level: string(ConsistentLevelEventual), - Storage: "local://" + dir, + Level: string(redo.ConsistentLevelEventual), + Storage: "local://" + dir, + FlushIntervalInMs: config.MinFlushIntervalInMs, } errCh := make(chan error, 1) diff --git a/cdc/redo/options.go b/cdc/redo/options.go index 2e48fe4865e..3b97295bd41 100644 --- a/cdc/redo/options.go +++ b/cdc/redo/options.go @@ -13,34 +13,32 @@ package redo +import ( + "github.com/pingcap/tiflow/pkg/redo" +) + // ManagerOptions defines options for redo log manager. type ManagerOptions struct { + redo.FileTypeConfig + // Whether to run background flush goroutine. EnableBgRunner bool - // Whether to start a GC goroutine or not. EnableGCRunner bool - // Whether it's created for emitting redo meta or not. - EmitMeta bool - - // Whether it's created for emitting row events or not. - EmitRowEvents bool - - // Whether it's created for emitting DDL events or not. - EmitDDLEvents bool - ErrCh chan<- error } // NewOwnerManagerOptions creates a manager options for owner. func NewOwnerManagerOptions(errCh chan<- error) *ManagerOptions { return &ManagerOptions{ + FileTypeConfig: redo.FileTypeConfig{ + EmitMeta: true, + EmitRowEvents: false, + EmitDDLEvents: true, + }, EnableBgRunner: true, EnableGCRunner: false, - EmitMeta: true, - EmitRowEvents: false, - EmitDDLEvents: true, ErrCh: errCh, } } @@ -48,11 +46,13 @@ func NewOwnerManagerOptions(errCh chan<- error) *ManagerOptions { // NewProcessorManagerOptions creates a manager options for processor. func NewProcessorManagerOptions(errCh chan<- error) *ManagerOptions { return &ManagerOptions{ + FileTypeConfig: redo.FileTypeConfig{ + EmitMeta: false, + EmitRowEvents: true, + EmitDDLEvents: false, + }, EnableBgRunner: true, EnableGCRunner: true, - EmitMeta: false, - EmitRowEvents: true, - EmitDDLEvents: false, ErrCh: errCh, } } @@ -60,22 +60,26 @@ func NewProcessorManagerOptions(errCh chan<- error) *ManagerOptions { // NewManagerOptionsForClean creates a manager options for cleaning. func NewManagerOptionsForClean() *ManagerOptions { return &ManagerOptions{ + FileTypeConfig: redo.FileTypeConfig{ + EmitMeta: false, + EmitRowEvents: false, + EmitDDLEvents: false, + }, EnableBgRunner: false, EnableGCRunner: false, - EmitMeta: false, - EmitRowEvents: false, - EmitDDLEvents: false, } } // newMockManagerOptions creates a manager options for mock tests. func newMockManagerOptions(errCh chan<- error) *ManagerOptions { return &ManagerOptions{ + FileTypeConfig: redo.FileTypeConfig{ + EmitMeta: true, + EmitRowEvents: true, + EmitDDLEvents: true, + }, EnableBgRunner: true, EnableGCRunner: true, - EmitMeta: true, - EmitRowEvents: true, - EmitDDLEvents: true, ErrCh: errCh, } } diff --git a/cdc/redo/reader/blackhole_reader.go b/cdc/redo/reader/blackhole_reader.go index 724ea6a0a3e..370c137f863 100644 --- a/cdc/redo/reader/blackhole_reader.go +++ b/cdc/redo/reader/blackhole_reader.go @@ -22,8 +22,8 @@ import ( // BlackHoleReader is a blockHole storage which implements LogReader interface type BlackHoleReader struct{} -// NewBlackHoleReader creates a new BlackHoleReader -func NewBlackHoleReader() *BlackHoleReader { +// newBlackHoleReader creates a new BlackHoleReader +func newBlackHoleReader() *BlackHoleReader { return &BlackHoleReader{} } diff --git a/cdc/redo/reader/file.go b/cdc/redo/reader/file.go index 8a79785bb2b..fa184129bd4 100644 --- a/cdc/redo/reader/file.go +++ b/cdc/redo/reader/file.go @@ -30,9 +30,9 @@ import ( "github.com/pingcap/log" "github.com/pingcap/tidb/br/pkg/storage" "github.com/pingcap/tiflow/cdc/model" - "github.com/pingcap/tiflow/cdc/redo/common" "github.com/pingcap/tiflow/cdc/redo/writer" cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/redo" "go.uber.org/multierr" "go.uber.org/zap" "golang.org/x/sync/errgroup" @@ -56,13 +56,14 @@ type fileReader interface { } type readerConfig struct { - dir string - fileType string - startTs uint64 - endTs uint64 - s3Storage bool - s3URI url.URL - workerNums int + startTs uint64 + endTs uint64 + dir string + fileType string + + uri url.URL + useExternalStorage bool + workerNums int } type reader struct { @@ -80,13 +81,13 @@ func newReader(ctx context.Context, cfg *readerConfig) ([]fileReader, error) { return nil, cerror.WrapError(cerror.ErrRedoConfigInvalid, errors.New("readerConfig can not be nil")) } - if cfg.s3Storage { - s3storage, err := common.InitS3storage(ctx, cfg.s3URI) + if cfg.useExternalStorage { + extStorage, err := redo.InitExternalStorage(ctx, cfg.uri) if err != nil { return nil, err } - err = downLoadToLocal(ctx, cfg.dir, s3storage, cfg.fileType) + err = downLoadToLocal(ctx, cfg.dir, extStorage, cfg.fileType) if err != nil { return nil, cerror.WrapError(cerror.ErrRedoDownloadFailed, err) } @@ -114,20 +115,23 @@ func newReader(ctx context.Context, cfg *readerConfig) ([]fileReader, error) { return readers, nil } -func selectDownLoadFile(ctx context.Context, s3storage storage.ExternalStorage, fixedType string) ([]string, error) { +func selectDownLoadFile( + ctx context.Context, extStorage storage.ExternalStorage, fixedType string, +) ([]string, error) { files := []string{} - err := s3storage.WalkDir(ctx, &storage.WalkOption{}, func(path string, size int64) error { - fileName := filepath.Base(path) - _, fileType, err := common.ParseLogFileName(fileName) - if err != nil { - return err - } + err := extStorage.WalkDir(ctx, &storage.WalkOption{}, + func(path string, size int64) error { + fileName := filepath.Base(path) + _, fileType, err := redo.ParseLogFileName(fileName) + if err != nil { + return err + } - if fileType == fixedType { - files = append(files, path) - } - return nil - }) + if fileType == fixedType { + files = append(files, path) + } + return nil + }) if err != nil { return nil, cerror.WrapError(cerror.ErrS3StorageAPI, err) } @@ -135,8 +139,10 @@ func selectDownLoadFile(ctx context.Context, s3storage storage.ExternalStorage, return files, nil } -func downLoadToLocal(ctx context.Context, dir string, s3storage storage.ExternalStorage, fixedType string) error { - files, err := selectDownLoadFile(ctx, s3storage, fixedType) +func downLoadToLocal( + ctx context.Context, dir string, extStorage storage.ExternalStorage, fixedType string, +) error { + files, err := selectDownLoadFile(ctx, extStorage, fixedType) if err != nil { return err } @@ -145,17 +151,17 @@ func downLoadToLocal(ctx context.Context, dir string, s3storage storage.External for _, file := range files { f := file eg.Go(func() error { - data, err := s3storage.ReadFile(eCtx, f) + data, err := extStorage.ReadFile(eCtx, f) if err != nil { return cerror.WrapError(cerror.ErrS3StorageAPI, err) } - err = os.MkdirAll(dir, common.DefaultDirMode) + err = os.MkdirAll(dir, redo.DefaultDirMode) if err != nil { return cerror.WrapError(cerror.ErrRedoFileOp, err) } path := filepath.Join(dir, f) - err = os.WriteFile(path, data, common.DefaultFileMode) + err = os.WriteFile(path, data, redo.DefaultFileMode) return cerror.WrapError(cerror.ErrRedoFileOp, err) }) } @@ -171,7 +177,7 @@ func openSelectedFiles(ctx context.Context, dir, fixedType string, startTs uint6 sortedFileList := map[string]bool{} for _, file := range files { - if filepath.Ext(file.Name()) == common.SortLogEXT { + if filepath.Ext(file.Name()) == redo.SortLogEXT { sortedFileList[file.Name()] = false } } @@ -190,8 +196,8 @@ func openSelectedFiles(ctx context.Context, dir, fixedType string, startTs uint6 if ret { sortedName := name - if filepath.Ext(sortedName) != common.SortLogEXT { - sortedName += common.SortLogEXT + if filepath.Ext(sortedName) != redo.SortLogEXT { + sortedName += redo.SortLogEXT } if opened, ok := sortedFileList[sortedName]; ok { if opened { @@ -220,7 +226,7 @@ func openSelectedFiles(ctx context.Context, dir, fixedType string, startTs uint6 } func openReadFile(name string) (*os.File, error) { - return os.OpenFile(name, os.O_RDONLY, common.DefaultFileMode) + return os.OpenFile(name, os.O_RDONLY, redo.DefaultFileMode) } func readFile(file *os.File) (logHeap, error) { @@ -332,7 +338,7 @@ func createSortedFile(ctx context.Context, dir string, name string, errCh chan e return } - sortFileName := name + common.SortLogEXT + sortFileName := name + redo.SortLogEXT err = writFile(ctx, dir, sortFileName, h) if err != nil { errCh <- err @@ -349,7 +355,7 @@ func createSortedFile(ctx context.Context, dir string, name string, errCh chan e func shouldOpen(startTs uint64, name, fixedType string) (bool, error) { // .sort.tmp will return error - commitTs, fileType, err := common.ParseLogFileName(name) + commitTs, fileType, err := redo.ParseLogFileName(name) if err != nil { return false, err } @@ -357,7 +363,7 @@ func shouldOpen(startTs uint64, name, fixedType string) (bool, error) { return false, nil } // always open .tmp - if filepath.Ext(name) == common.TmpEXT { + if filepath.Ext(name) == redo.TmpEXT { return true, nil } // the commitTs=max(ts of log item in the file), if max > startTs then should open, @@ -435,7 +441,7 @@ func (r *reader) isTornEntry(data []byte) bool { chunks := [][]byte{} // split data on sector boundaries for curOff < len(data) { - chunkLen := int(common.MinSectorSize - (fileOff % common.MinSectorSize)) + chunkLen := int(redo.MinSectorSize - (fileOff % redo.MinSectorSize)) if chunkLen > len(data)-curOff { chunkLen = len(data) - curOff } diff --git a/cdc/redo/reader/file_test.go b/cdc/redo/reader/file_test.go index 61f780d622f..2be067047b6 100644 --- a/cdc/redo/reader/file_test.go +++ b/cdc/redo/reader/file_test.go @@ -23,8 +23,8 @@ import ( "time" "github.com/pingcap/tiflow/cdc/model" - "github.com/pingcap/tiflow/cdc/redo/common" "github.com/pingcap/tiflow/cdc/redo/writer" + "github.com/pingcap/tiflow/pkg/redo" "github.com/pingcap/tiflow/pkg/uuid" "github.com/stretchr/testify/require" "golang.org/x/net/context" @@ -47,8 +47,7 @@ func TestReaderRead(t *testing.T) { Dir: dir, ChangeFeedID: model.DefaultChangeFeedID("test-cf"), CaptureID: "cp", - FileType: common.DefaultRowLogFileType, - CreateTime: time.Date(2000, 1, 1, 1, 1, 1, 1, &time.Location{}), + FileType: redo.RedoRowLogFileType, } ctx, cancel := context.WithCancel(context.Background()) defer cancel() @@ -69,9 +68,9 @@ func TestReaderRead(t *testing.T) { err = w.Close() require.Nil(t, err) require.True(t, !w.IsRunning()) - fileName := fmt.Sprintf(common.RedoLogFileFormatV1, cfg.CaptureID, + fileName := fmt.Sprintf(redo.RedoLogFileFormatV1, cfg.CaptureID, cfg.ChangeFeedID.ID, - cfg.FileType, 11, uuidGen.NewString(), common.LogEXT) + cfg.FileType, 11, uuidGen.NewString(), redo.LogEXT) path := filepath.Join(cfg.Dir, fileName) info, err := os.Stat(path) require.Nil(t, err) @@ -81,7 +80,7 @@ func TestReaderRead(t *testing.T) { dir: dir, startTs: 1, endTs: 12, - fileType: common.DefaultRowLogFileType, + fileType: redo.RedoRowLogFileType, }) require.Nil(t, err) require.Equal(t, 1, len(r)) @@ -103,9 +102,9 @@ func TestReaderOpenSelectedFiles(t *testing.T) { Dir: dir, } uuidGen := uuid.NewGenerator() - fileName := fmt.Sprintf(common.RedoLogFileFormatV2, "cp", - "default", "test-cf", common.DefaultDDLLogFileType, 11, - uuidGen.NewString(), common.LogEXT+common.TmpEXT) + fileName := fmt.Sprintf(redo.RedoLogFileFormatV2, "cp", + "default", "test-cf", redo.RedoDDLLogFileType, 11, + uuidGen.NewString(), redo.LogEXT+redo.TmpEXT) w, err := writer.NewWriter(ctx, cfg, writer.WithLogFileName(func() string { return fileName })) @@ -131,24 +130,24 @@ func TestReaderOpenSelectedFiles(t *testing.T) { require.Nil(t, err) // no data, wil not open - fileName = fmt.Sprintf(common.RedoLogFileFormatV2, "cp", - "default", "test-cf11", common.DefaultDDLLogFileType, 10, - uuidGen.NewString(), common.LogEXT) + fileName = fmt.Sprintf(redo.RedoLogFileFormatV2, "cp", + "default", "test-cf11", redo.RedoDDLLogFileType, 10, + uuidGen.NewString(), redo.LogEXT) path = filepath.Join(dir, fileName) _, err = os.Create(path) require.Nil(t, err) // SortLogEXT, wil open - fileName = fmt.Sprintf(common.RedoLogFileFormatV2, "cp", "default", - "test-cf111", common.DefaultDDLLogFileType, 10, uuidGen.NewString(), - common.LogEXT) + common.SortLogEXT + fileName = fmt.Sprintf(redo.RedoLogFileFormatV2, "cp", "default", + "test-cf111", redo.RedoDDLLogFileType, 10, uuidGen.NewString(), + redo.LogEXT) + redo.SortLogEXT path = filepath.Join(dir, fileName) f1, err := os.Create(path) require.Nil(t, err) dir1 := t.TempDir() - fileName = fmt.Sprintf(common.RedoLogFileFormatV2, "cp", "default", "test-cf", - common.DefaultDDLLogFileType, 11, uuidGen.NewString(), common.LogEXT+"test") + fileName = fmt.Sprintf(redo.RedoLogFileFormatV2, "cp", "default", "test-cf", + redo.RedoDDLLogFileType, 11, uuidGen.NewString(), redo.LogEXT+"test") path = filepath.Join(dir1, fileName) _, err = os.Create(path) require.Nil(t, err) @@ -168,7 +167,7 @@ func TestReaderOpenSelectedFiles(t *testing.T) { name: "dir not exist", args: arg{ dir: dir + "test", - fixedName: common.DefaultDDLLogFileType, + fixedName: redo.RedoDDLLogFileType, startTs: 0, }, wantErr: ".*CDC:ErrRedoFileOp*.", @@ -177,7 +176,7 @@ func TestReaderOpenSelectedFiles(t *testing.T) { name: "happy", args: arg{ dir: dir, - fixedName: common.DefaultDDLLogFileType, + fixedName: redo.RedoDDLLogFileType, startTs: 0, }, wantRet: []io.ReadCloser{f, f1}, @@ -186,7 +185,7 @@ func TestReaderOpenSelectedFiles(t *testing.T) { name: "wrong ts", args: arg{ dir: dir, - fixedName: common.DefaultDDLLogFileType, + fixedName: redo.RedoDDLLogFileType, startTs: 12, }, wantRet: []io.ReadCloser{f}, @@ -195,7 +194,7 @@ func TestReaderOpenSelectedFiles(t *testing.T) { name: "wrong fixedName", args: arg{ dir: dir, - fixedName: common.DefaultDDLLogFileType + "test", + fixedName: redo.RedoDDLLogFileType + "test", startTs: 0, }, }, @@ -203,7 +202,7 @@ func TestReaderOpenSelectedFiles(t *testing.T) { name: "wrong ext", args: arg{ dir: dir1, - fixedName: common.DefaultDDLLogFileType, + fixedName: redo.RedoDDLLogFileType, startTs: 0, }, }, @@ -216,8 +215,8 @@ func TestReaderOpenSelectedFiles(t *testing.T) { require.Equal(t, len(tt.wantRet), len(ret), tt.name) for _, closer := range tt.wantRet { name := closer.(*os.File).Name() - if filepath.Ext(name) != common.SortLogEXT { - name += common.SortLogEXT + if filepath.Ext(name) != redo.SortLogEXT { + name += redo.SortLogEXT } contains := false for _, r := range ret { diff --git a/cdc/redo/reader/mock_RedoLogReader.go b/cdc/redo/reader/mock_RedoLogReader.go deleted file mode 100644 index c4d6f5372ad..00000000000 --- a/cdc/redo/reader/mock_RedoLogReader.go +++ /dev/null @@ -1,132 +0,0 @@ -// Code generated by mockery v2.14.1. DO NOT EDIT. - -package reader - -import ( - context "context" - - model "github.com/pingcap/tiflow/cdc/model" - mock "github.com/stretchr/testify/mock" -) - -// MockRedoLogReader is an autogenerated mock type for the RedoLogReader type -type MockRedoLogReader struct { - mock.Mock -} - -// Close provides a mock function with given fields: -func (_m *MockRedoLogReader) Close() error { - ret := _m.Called() - - var r0 error - if rf, ok := ret.Get(0).(func() error); ok { - r0 = rf() - } else { - r0 = ret.Error(0) - } - - return r0 -} - -// ReadMeta provides a mock function with given fields: ctx -func (_m *MockRedoLogReader) ReadMeta(ctx context.Context) (uint64, uint64, error) { - ret := _m.Called(ctx) - - var r0 uint64 - if rf, ok := ret.Get(0).(func(context.Context) uint64); ok { - r0 = rf(ctx) - } else { - r0 = ret.Get(0).(uint64) - } - - var r1 uint64 - if rf, ok := ret.Get(1).(func(context.Context) uint64); ok { - r1 = rf(ctx) - } else { - r1 = ret.Get(1).(uint64) - } - - var r2 error - if rf, ok := ret.Get(2).(func(context.Context) error); ok { - r2 = rf(ctx) - } else { - r2 = ret.Error(2) - } - - return r0, r1, r2 -} - -// ReadNextDDL provides a mock function with given fields: ctx, maxNumberOfEvents -func (_m *MockRedoLogReader) ReadNextDDL(ctx context.Context, maxNumberOfEvents uint64) ([]*model.RedoDDLEvent, error) { - ret := _m.Called(ctx, maxNumberOfEvents) - - var r0 []*model.RedoDDLEvent - if rf, ok := ret.Get(0).(func(context.Context, uint64) []*model.RedoDDLEvent); ok { - r0 = rf(ctx, maxNumberOfEvents) - } else { - if ret.Get(0) != nil { - r0 = ret.Get(0).([]*model.RedoDDLEvent) - } - } - - var r1 error - if rf, ok := ret.Get(1).(func(context.Context, uint64) error); ok { - r1 = rf(ctx, maxNumberOfEvents) - } else { - r1 = ret.Error(1) - } - - return r0, r1 -} - -// ReadNextLog provides a mock function with given fields: ctx, maxNumberOfEvents -func (_m *MockRedoLogReader) ReadNextLog(ctx context.Context, maxNumberOfEvents uint64) ([]*model.RedoRowChangedEvent, error) { - ret := _m.Called(ctx, maxNumberOfEvents) - - var r0 []*model.RedoRowChangedEvent - if rf, ok := ret.Get(0).(func(context.Context, uint64) []*model.RedoRowChangedEvent); ok { - r0 = rf(ctx, maxNumberOfEvents) - } else { - if ret.Get(0) != nil { - r0 = ret.Get(0).([]*model.RedoRowChangedEvent) - } - } - - var r1 error - if rf, ok := ret.Get(1).(func(context.Context, uint64) error); ok { - r1 = rf(ctx, maxNumberOfEvents) - } else { - r1 = ret.Error(1) - } - - return r0, r1 -} - -// ResetReader provides a mock function with given fields: ctx, startTs, endTs -func (_m *MockRedoLogReader) ResetReader(ctx context.Context, startTs uint64, endTs uint64) error { - ret := _m.Called(ctx, startTs, endTs) - - var r0 error - if rf, ok := ret.Get(0).(func(context.Context, uint64, uint64) error); ok { - r0 = rf(ctx, startTs, endTs) - } else { - r0 = ret.Error(0) - } - - return r0 -} - -type mockConstructorTestingTNewMockRedoLogReader interface { - mock.TestingT - Cleanup(func()) -} - -// NewMockRedoLogReader creates a new instance of MockRedoLogReader. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations. -func NewMockRedoLogReader(t mockConstructorTestingTNewMockRedoLogReader) *MockRedoLogReader { - mock := &MockRedoLogReader{} - mock.Mock.Test(t) - - t.Cleanup(func() { mock.AssertExpectations(t) }) - - return mock -} diff --git a/cdc/redo/reader/reader.go b/cdc/redo/reader/reader.go index e3816b3d8c1..3faf1079912 100644 --- a/cdc/redo/reader/reader.go +++ b/cdc/redo/reader/reader.go @@ -27,13 +27,12 @@ import ( "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/cdc/redo/common" cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/redo" "go.uber.org/multierr" "go.uber.org/zap" ) // RedoLogReader is a reader abstraction for redo log storage layer -// -//go:generate mockery --name=RedoLogReader --inpackage --quiet type RedoLogReader interface { io.Closer @@ -51,20 +50,37 @@ type RedoLogReader interface { ReadMeta(ctx context.Context) (checkpointTs, resolvedTs uint64, err error) } +// NewRedoLogReader creates a new redo log reader +func NewRedoLogReader( + ctx context.Context, storageType string, cfg *LogReaderConfig, +) (rd RedoLogReader, err error) { + if !redo.IsValidConsistentStorage(storageType) { + return nil, cerror.ErrConsistentStorage.GenWithStackByArgs(storageType) + } + if redo.IsBlackholeStorage(storageType) { + return newBlackHoleReader(), nil + } + return newLogReader(ctx, cfg) +} + // LogReaderConfig is the config for LogReader type LogReaderConfig struct { + startTs uint64 + endTs uint64 + // Dir is the folder contains the redo logs need to apply when OP environment or - // the folder used to download redo logs to if s3 enabled - Dir string - S3Storage bool - // S3URI should be like S3URI="s3://logbucket/test-changefeed?endpoint=http://$S3_ENDPOINT/" - S3URI url.URL + // the folder used to download redo logs to if using external storage, such as s3 + // and gcs. + Dir string + + // URI should be like "s3://logbucket/test-changefeed?endpoint=http://$S3_ENDPOINT/" + URI url.URL + UseExternalStorage bool + // WorkerNums is the num of workers used to sort the log file to sorted file, // will load the file to memory first then write the sorted file to disk // the memory used is WorkerNums * defaultMaxLogSize (64 * megabyte) total WorkerNums int - startTs uint64 - endTs uint64 } // LogReader implement RedoLogReader interface @@ -81,10 +97,11 @@ type LogReader struct { sync.Mutex } -// NewLogReader creates a LogReader instance. Need the client to guarantee only one LogReader per changefeed +// newLogReader creates a LogReader instance. +// Need the client to guarantee only one LogReader per changefeed // currently support rewind operation by ResetReader api // if s3 will download logs first, if OP environment need fetch the redo logs to local dir first -func NewLogReader(ctx context.Context, cfg *LogReaderConfig) (*LogReader, error) { +func newLogReader(ctx context.Context, cfg *LogReaderConfig) (*LogReader, error) { if cfg == nil { return nil, cerror.WrapError(cerror.ErrRedoConfigInvalid, errors.New("LogReaderConfig can not be nil")) } @@ -92,8 +109,8 @@ func NewLogReader(ctx context.Context, cfg *LogReaderConfig) (*LogReader, error) logReader := &LogReader{ cfg: cfg, } - if cfg.S3Storage { - s3storage, err := common.InitS3storage(ctx, cfg.S3URI) + if cfg.UseExternalStorage { + extStorage, err := redo.InitExternalStorage(ctx, cfg.URI) if err != nil { return nil, err } @@ -102,7 +119,7 @@ func NewLogReader(ctx context.Context, cfg *LogReaderConfig) (*LogReader, error) if err != nil { return nil, cerror.WrapError(cerror.ErrRedoFileOp, err) } - err = downLoadToLocal(ctx, cfg.Dir, s3storage, common.DefaultMetaFileType) + err = downLoadToLocal(ctx, cfg.Dir, extStorage, redo.RedoMetaFileType) if err != nil { return nil, cerror.WrapError(cerror.ErrRedoDownloadFailed, err) } @@ -154,13 +171,13 @@ func (l *LogReader) setUpRowReader(ctx context.Context, startTs, endTs uint64) e } rowCfg := &readerConfig{ - dir: l.cfg.Dir, - fileType: common.DefaultRowLogFileType, - startTs: startTs, - endTs: endTs, - s3Storage: l.cfg.S3Storage, - s3URI: l.cfg.S3URI, - workerNums: l.cfg.WorkerNums, + startTs: startTs, + endTs: endTs, + dir: l.cfg.Dir, + fileType: redo.RedoRowLogFileType, + uri: l.cfg.URI, + useExternalStorage: l.cfg.UseExternalStorage, + workerNums: l.cfg.WorkerNums, } l.rowReader, err = newReader(ctx, rowCfg) if err != nil { @@ -183,13 +200,13 @@ func (l *LogReader) setUpDDLReader(ctx context.Context, startTs, endTs uint64) e } ddlCfg := &readerConfig{ - dir: l.cfg.Dir, - fileType: common.DefaultDDLLogFileType, - startTs: startTs, - endTs: endTs, - s3Storage: l.cfg.S3Storage, - s3URI: l.cfg.S3URI, - workerNums: l.cfg.WorkerNums, + startTs: startTs, + endTs: endTs, + dir: l.cfg.Dir, + fileType: redo.RedoDDLLogFileType, + uri: l.cfg.URI, + useExternalStorage: l.cfg.UseExternalStorage, + workerNums: l.cfg.WorkerNums, } l.ddlReader, err = newReader(ctx, ddlCfg) if err != nil { @@ -350,7 +367,7 @@ func (l *LogReader) ReadMeta(ctx context.Context) (checkpointTs, resolvedTs uint metas := make([]*common.LogMeta, 0, 64) for _, file := range files { - if filepath.Ext(file.Name()) == common.MetaEXT { + if filepath.Ext(file.Name()) == redo.MetaEXT { path := filepath.Join(l.cfg.Dir, file.Name()) fileData, err := os.ReadFile(path) if err != nil { diff --git a/cdc/redo/reader/reader_test.go b/cdc/redo/reader/reader_test.go index bdac1078962..62472bf6c8d 100644 --- a/cdc/redo/reader/reader_test.go +++ b/cdc/redo/reader/reader_test.go @@ -31,16 +31,17 @@ import ( "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/cdc/redo/common" "github.com/pingcap/tiflow/cdc/redo/writer" + "github.com/pingcap/tiflow/pkg/redo" "github.com/stretchr/testify/mock" "github.com/stretchr/testify/require" "go.uber.org/multierr" ) func TestNewLogReader(t *testing.T) { - _, err := NewLogReader(context.Background(), nil) + _, err := newLogReader(context.Background(), nil) require.NotNil(t, err) - _, err = NewLogReader(context.Background(), &LogReaderConfig{}) + _, err = newLogReader(context.Background(), &LogReaderConfig{}) require.Nil(t, err) dir := t.TempDir() @@ -48,23 +49,25 @@ func TestNewLogReader(t *testing.T) { s3URI, err := url.Parse("s3://logbucket/test-changefeed?endpoint=http://111/") require.Nil(t, err) - origin := common.InitS3storage + origin := redo.InitExternalStorage defer func() { - common.InitS3storage = origin + redo.InitExternalStorage = origin }() controller := gomock.NewController(t) mockStorage := mockstorage.NewMockExternalStorage(controller) // no file to download mockStorage.EXPECT().WalkDir(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil) - common.InitS3storage = func(ctx context.Context, uri url.URL) (storage.ExternalStorage, error) { + redo.InitExternalStorage = func( + ctx context.Context, uri url.URL, + ) (storage.ExternalStorage, error) { return mockStorage, nil } // after init should rm the dir - _, err = NewLogReader(context.Background(), &LogReaderConfig{ - S3Storage: true, - Dir: dir, - S3URI: *s3URI, + _, err = newLogReader(context.Background(), &LogReaderConfig{ + UseExternalStorage: true, + Dir: dir, + URI: *s3URI, }) require.Nil(t, err) _, err = os.Stat(dir) @@ -80,9 +83,9 @@ func TestLogReaderResetReader(t *testing.T) { MaxLogSize: 100000, Dir: dir, } - fileName := fmt.Sprintf(common.RedoLogFileFormatV2, "cp", + fileName := fmt.Sprintf(redo.RedoLogFileFormatV2, "cp", "default", "test-cf100", - common.DefaultDDLLogFileType, 100, uuid.NewString(), common.LogEXT) + redo.RedoDDLLogFileType, 100, uuid.NewString(), redo.LogEXT) w, err := writer.NewWriter(ctx, cfg, writer.WithLogFileName(func() string { return fileName })) @@ -101,9 +104,9 @@ func TestLogReaderResetReader(t *testing.T) { f, err := os.Open(path) require.Nil(t, err) - fileName = fmt.Sprintf(common.RedoLogFileFormatV2, "cp", + fileName = fmt.Sprintf(redo.RedoLogFileFormatV2, "cp", "default", "test-cf10", - common.DefaultRowLogFileType, 10, uuid.NewString(), common.LogEXT) + redo.RedoRowLogFileType, 10, uuid.NewString(), redo.LogEXT) w, err = writer.NewWriter(ctx, cfg, writer.WithLogFileName(func() string { return fileName })) @@ -224,8 +227,10 @@ func TestLogReaderResetReader(t *testing.T) { } else { require.Nil(t, err, tt.name) mockReader.AssertNumberOfCalls(t, "Close", 2) - require.Equal(t, tt.rowFleName+common.SortLogEXT, r.rowReader[0].(*reader).fileName, tt.name) - require.Equal(t, tt.ddlFleName+common.SortLogEXT, r.ddlReader[0].(*reader).fileName, tt.name) + require.Equal(t, tt.rowFleName+redo.SortLogEXT, + r.rowReader[0].(*reader).fileName, tt.name) + require.Equal(t, tt.ddlFleName+redo.SortLogEXT, + r.ddlReader[0].(*reader).fileName, tt.name) require.Equal(t, tt.wantStartTs, r.cfg.startTs, tt.name) require.Equal(t, tt.wantEndTs, r.cfg.endTs, tt.name) @@ -239,7 +244,7 @@ func TestLogReaderReadMeta(t *testing.T) { fileName := fmt.Sprintf("%s_%s_%d_%s%s", "cp", "test-changefeed", - time.Now().Unix(), common.DefaultMetaFileType, common.MetaEXT) + time.Now().Unix(), redo.RedoMetaFileType, redo.MetaEXT) path := filepath.Join(dir, fileName) f, err := os.Create(path) require.Nil(t, err) @@ -254,7 +259,7 @@ func TestLogReaderReadMeta(t *testing.T) { fileName = fmt.Sprintf("%s_%s_%d_%s%s", "cp1", "test-changefeed", - time.Now().Unix(), common.DefaultMetaFileType, common.MetaEXT) + time.Now().Unix(), redo.RedoMetaFileType, redo.MetaEXT) path = filepath.Join(dir, fileName) f, err = os.Create(path) require.Nil(t, err) diff --git a/cdc/redo/writer/file.go b/cdc/redo/writer/file.go index e1679c99e8a..c2b74318d48 100644 --- a/cdc/redo/writer/file.go +++ b/cdc/redo/writer/file.go @@ -32,6 +32,7 @@ import ( "github.com/pingcap/tiflow/cdc/redo/common" cerror "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/fsutil" + "github.com/pingcap/tiflow/pkg/redo" "github.com/pingcap/tiflow/pkg/uuid" "github.com/prometheus/client_golang/prometheus" "github.com/uber-go/atomic" @@ -44,7 +45,7 @@ const ( // pageBytes is the alignment for flushing records to the backing Writer. // It should be a multiple of the minimum sector size so that log can safely // distinguish between torn writes and ordinary data corruption. - pageBytes = 8 * common.MinSectorSize + pageBytes = 8 * redo.MinSectorSize defaultS3Timeout = 15 * time.Second ) @@ -73,15 +74,16 @@ type flusher interface { // FileWriterConfig is the configuration used by a Writer. type FileWriterConfig struct { - Dir string + FileType string ChangeFeedID model.ChangeFeedID CaptureID string - FileType string - CreateTime time.Time + + URI url.URL + UseExternalStorage bool + // MaxLogSize is the maximum size of log in megabyte, defaults to defaultMaxLogSize. MaxLogSize int64 - S3Storage bool - S3URI url.URL + Dir string } // Option define the writerOptions @@ -148,10 +150,10 @@ func NewWriter(ctx context.Context, cfg *FileWriterConfig, opts ...Option) (*Wri if cfg.MaxLogSize == 0 { cfg.MaxLogSize = defaultMaxLogSize } - var s3storage storage.ExternalStorage - if cfg.S3Storage { + var extStorage storage.ExternalStorage + if cfg.UseExternalStorage { var err error - s3storage, err = common.InitS3storage(ctx, cfg.S3URI) + extStorage, err = redo.InitExternalStorage(ctx, cfg.URI) if err != nil { return nil, err } @@ -166,7 +168,7 @@ func NewWriter(ctx context.Context, cfg *FileWriterConfig, opts ...Option) (*Wri cfg: cfg, op: op, uint64buf: make([]byte, 8), - storage: s3storage, + storage: extStorage, metricFsyncDuration: common.RedoFsyncDurationHistogram. WithLabelValues(cfg.ChangeFeedID.Namespace, cfg.ChangeFeedID.ID), @@ -185,7 +187,7 @@ func NewWriter(ctx context.Context, cfg *FileWriterConfig, opts ...Option) (*Wri return nil, cerror.WrapError(cerror.ErrRedoFileOp, errors.New("invalid redo dir path")) } - err := os.MkdirAll(cfg.Dir, common.DefaultDirMode) + err := os.MkdirAll(cfg.Dir, redo.DefaultDirMode) if err != nil { return nil, cerror.WrapError(cerror.ErrRedoFileOp, errors.Annotatef(err, "can't make dir: %s for redo writing", cfg.Dir)) @@ -194,7 +196,7 @@ func NewWriter(ctx context.Context, cfg *FileWriterConfig, opts ...Option) (*Wri // if we use S3 as the remote storage, a file allocator can be leveraged to // pre-allocate files for us. // TODO: test whether this improvement can also be applied to NFS. - if cfg.S3Storage { + if cfg.UseExternalStorage { w.allocator = fsutil.NewFileAllocator(cfg.Dir, cfg.FileType, defaultMaxLogSize) } @@ -317,7 +319,7 @@ func (w *Writer) close() error { return err } - if w.cfg.S3Storage { + if w.cfg.UseExternalStorage { off, err := w.file.Seek(0, io.SeekCurrent) if err != nil { return err @@ -355,7 +357,7 @@ func (w *Writer) close() error { // We only write content to S3 before closing the local file. // By this way, we no longer need renaming object in S3. - if w.cfg.S3Storage { + if w.cfg.UseExternalStorage { ctx, cancel := context.WithTimeout(context.Background(), defaultS3Timeout) defer cancel() @@ -378,13 +380,13 @@ func (w *Writer) getLogFileName() string { } uid := w.uuidGenerator.NewString() if model.DefaultNamespace == w.cfg.ChangeFeedID.Namespace { - return fmt.Sprintf(common.RedoLogFileFormatV1, + return fmt.Sprintf(redo.RedoLogFileFormatV1, w.cfg.CaptureID, w.cfg.ChangeFeedID.ID, w.cfg.FileType, - w.commitTS.Load(), uid, common.LogEXT) + w.commitTS.Load(), uid, redo.LogEXT) } - return fmt.Sprintf(common.RedoLogFileFormatV2, + return fmt.Sprintf(redo.RedoLogFileFormatV2, w.cfg.CaptureID, w.cfg.ChangeFeedID.Namespace, w.cfg.ChangeFeedID.ID, - w.cfg.FileType, w.commitTS.Load(), uid, common.LogEXT) + w.cfg.FileType, w.commitTS.Load(), uid, redo.LogEXT) } // filePath always creates a new, unique file path, note this function is not @@ -396,11 +398,11 @@ func (w *Writer) filePath() string { } func openTruncFile(name string) (*os.File, error) { - return os.OpenFile(name, os.O_CREATE|os.O_WRONLY|os.O_TRUNC, common.DefaultFileMode) + return os.OpenFile(name, os.O_CREATE|os.O_WRONLY|os.O_TRUNC, redo.DefaultFileMode) } func (w *Writer) openNew() error { - err := os.MkdirAll(w.cfg.Dir, common.DefaultDirMode) + err := os.MkdirAll(w.cfg.Dir, redo.DefaultDirMode) if err != nil { return cerror.WrapError(cerror.ErrRedoFileOp, errors.Annotatef(err, "can't make dir: %s for new redo logfile", w.cfg.Dir)) @@ -411,7 +413,7 @@ func (w *Writer) openNew() error { if w.allocator == nil { w.commitTS.Store(w.eventCommitTS.Load()) w.maxCommitTS.Store(w.eventCommitTS.Load()) - path := w.filePath() + common.TmpEXT + path := w.filePath() + redo.TmpEXT f, err = openTruncFile(path) if err != nil { return cerror.WrapError(cerror.ErrRedoFileOp, @@ -478,7 +480,7 @@ func (w *Writer) GC(checkPointTs uint64) error { return cerror.WrapError(cerror.ErrRedoFileOp, errs) } - if w.cfg.S3Storage { + if w.cfg.UseExternalStorage { // since if fail delete in s3, do not block any path, so just log the error if any go func() { var errs error @@ -499,11 +501,11 @@ func (w *Writer) GC(checkPointTs uint64) error { // shouldRemoved remove the file which commitTs in file name (max commitTs of all event ts in the file) < checkPointTs, // since all event ts < checkPointTs already sent to sink, the log is not needed any more for recovery func (w *Writer) shouldRemoved(checkPointTs uint64, f os.FileInfo) (bool, error) { - if filepath.Ext(f.Name()) != common.LogEXT { + if filepath.Ext(f.Name()) != redo.LogEXT { return false, nil } - commitTs, fileType, err := common.ParseLogFileName(f.Name()) + commitTs, fileType, err := redo.ParseLogFileName(f.Name()) if err != nil { return false, err } @@ -558,7 +560,7 @@ func (w *Writer) flushAndRotateFile() error { return err } - if !w.cfg.S3Storage { + if !w.cfg.UseExternalStorage { return nil } diff --git a/cdc/redo/writer/file_test.go b/cdc/redo/writer/file_test.go index 388350171e7..6e5c4b90158 100644 --- a/cdc/redo/writer/file_test.go +++ b/cdc/redo/writer/file_test.go @@ -29,6 +29,7 @@ import ( "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/cdc/redo/common" "github.com/pingcap/tiflow/pkg/fsutil" + "github.com/pingcap/tiflow/pkg/redo" "github.com/pingcap/tiflow/pkg/uuid" "github.com/stretchr/testify/require" "github.com/uber-go/atomic" @@ -61,8 +62,7 @@ func TestWriterWrite(t *testing.T) { Dir: dir, ChangeFeedID: cf, CaptureID: "cp", - FileType: common.DefaultRowLogFileType, - CreateTime: time.Date(2000, 1, 1, 1, 1, 1, 1, &time.Location{}), + FileType: redo.RedoRowLogFileType, }, uint64buf: make([]byte, 8), running: *atomic.NewBool(true), @@ -81,13 +81,13 @@ func TestWriterWrite(t *testing.T) { var fileName string // create a .tmp file if w.cfg.ChangeFeedID.Namespace == model.DefaultNamespace { - fileName = fmt.Sprintf(common.RedoLogFileFormatV1, w.cfg.CaptureID, + fileName = fmt.Sprintf(redo.RedoLogFileFormatV1, w.cfg.CaptureID, w.cfg.ChangeFeedID.ID, - w.cfg.FileType, 1, uuidGen.NewString(), common.LogEXT) + common.TmpEXT + w.cfg.FileType, 1, uuidGen.NewString(), redo.LogEXT) + redo.TmpEXT } else { - fileName = fmt.Sprintf(common.RedoLogFileFormatV2, w.cfg.CaptureID, + fileName = fmt.Sprintf(redo.RedoLogFileFormatV2, w.cfg.CaptureID, w.cfg.ChangeFeedID.Namespace, w.cfg.ChangeFeedID.ID, - w.cfg.FileType, 1, uuidGen.NewString(), common.LogEXT) + common.TmpEXT + w.cfg.FileType, 1, uuidGen.NewString(), redo.LogEXT) + redo.TmpEXT } path := filepath.Join(w.cfg.Dir, fileName) info, err := os.Stat(path) @@ -103,13 +103,13 @@ func TestWriterWrite(t *testing.T) { // after rotate, rename to .log if w.cfg.ChangeFeedID.Namespace == model.DefaultNamespace { - fileName = fmt.Sprintf(common.RedoLogFileFormatV1, w.cfg.CaptureID, + fileName = fmt.Sprintf(redo.RedoLogFileFormatV1, w.cfg.CaptureID, w.cfg.ChangeFeedID.ID, - w.cfg.FileType, 1, uuidGen.NewString(), common.LogEXT) + w.cfg.FileType, 1, uuidGen.NewString(), redo.LogEXT) } else { - fileName = fmt.Sprintf(common.RedoLogFileFormatV2, w.cfg.CaptureID, + fileName = fmt.Sprintf(redo.RedoLogFileFormatV2, w.cfg.CaptureID, w.cfg.ChangeFeedID.Namespace, w.cfg.ChangeFeedID.ID, - w.cfg.FileType, 1, uuidGen.NewString(), common.LogEXT) + w.cfg.FileType, 1, uuidGen.NewString(), redo.LogEXT) } path = filepath.Join(w.cfg.Dir, fileName) info, err = os.Stat(path) @@ -117,13 +117,13 @@ func TestWriterWrite(t *testing.T) { require.Equal(t, fileName, info.Name()) // create a .tmp file with first eventCommitTS as name if w.cfg.ChangeFeedID.Namespace == model.DefaultNamespace { - fileName = fmt.Sprintf(common.RedoLogFileFormatV1, w.cfg.CaptureID, + fileName = fmt.Sprintf(redo.RedoLogFileFormatV1, w.cfg.CaptureID, w.cfg.ChangeFeedID.ID, - w.cfg.FileType, 12, uuidGen.NewString(), common.LogEXT) + common.TmpEXT + w.cfg.FileType, 12, uuidGen.NewString(), redo.LogEXT) + redo.TmpEXT } else { - fileName = fmt.Sprintf(common.RedoLogFileFormatV2, w.cfg.CaptureID, + fileName = fmt.Sprintf(redo.RedoLogFileFormatV2, w.cfg.CaptureID, w.cfg.ChangeFeedID.Namespace, w.cfg.ChangeFeedID.ID, - w.cfg.FileType, 12, uuidGen.NewString(), common.LogEXT) + common.TmpEXT + w.cfg.FileType, 12, uuidGen.NewString(), redo.LogEXT) + redo.TmpEXT } path = filepath.Join(w.cfg.Dir, fileName) info, err = os.Stat(path) @@ -134,13 +134,13 @@ func TestWriterWrite(t *testing.T) { require.False(t, w.IsRunning()) // safe close, rename to .log with max eventCommitTS as name if w.cfg.ChangeFeedID.Namespace == model.DefaultNamespace { - fileName = fmt.Sprintf(common.RedoLogFileFormatV1, w.cfg.CaptureID, + fileName = fmt.Sprintf(redo.RedoLogFileFormatV1, w.cfg.CaptureID, w.cfg.ChangeFeedID.ID, - w.cfg.FileType, 22, uuidGen.NewString(), common.LogEXT) + w.cfg.FileType, 22, uuidGen.NewString(), redo.LogEXT) } else { - fileName = fmt.Sprintf(common.RedoLogFileFormatV2, w.cfg.CaptureID, + fileName = fmt.Sprintf(redo.RedoLogFileFormatV2, w.cfg.CaptureID, w.cfg.ChangeFeedID.Namespace, w.cfg.ChangeFeedID.ID, - w.cfg.FileType, 22, uuidGen.NewString(), common.LogEXT) + w.cfg.FileType, 22, uuidGen.NewString(), redo.LogEXT) } path = filepath.Join(w.cfg.Dir, fileName) info, err = os.Stat(path) @@ -153,8 +153,7 @@ func TestWriterWrite(t *testing.T) { Dir: dir, ChangeFeedID: cf11s[idx], CaptureID: "cp", - FileType: common.DefaultRowLogFileType, - CreateTime: time.Date(2000, 1, 1, 1, 1, 1, 1, &time.Location{}), + FileType: redo.RedoRowLogFileType, }, uint64buf: make([]byte, 8), running: *atomic.NewBool(true), @@ -172,13 +171,13 @@ func TestWriterWrite(t *testing.T) { require.Nil(t, err) // create a .tmp file if w1.cfg.ChangeFeedID.Namespace == model.DefaultNamespace { - fileName = fmt.Sprintf(common.RedoLogFileFormatV1, w1.cfg.CaptureID, + fileName = fmt.Sprintf(redo.RedoLogFileFormatV1, w1.cfg.CaptureID, w1.cfg.ChangeFeedID.ID, - w1.cfg.FileType, 1, uuidGen.NewString(), common.LogEXT) + common.TmpEXT + w1.cfg.FileType, 1, uuidGen.NewString(), redo.LogEXT) + redo.TmpEXT } else { - fileName = fmt.Sprintf(common.RedoLogFileFormatV2, w1.cfg.CaptureID, + fileName = fmt.Sprintf(redo.RedoLogFileFormatV2, w1.cfg.CaptureID, w1.cfg.ChangeFeedID.Namespace, w1.cfg.ChangeFeedID.ID, - w1.cfg.FileType, 1, uuidGen.NewString(), common.LogEXT) + common.TmpEXT + w1.cfg.FileType, 1, uuidGen.NewString(), redo.LogEXT) + redo.TmpEXT } path = filepath.Join(w1.cfg.Dir, fileName) info, err = os.Stat(path) @@ -218,9 +217,9 @@ func TestWriterGC(t *testing.T) { ChangeFeedID: model.DefaultChangeFeedID("test"), CaptureID: "cp", MaxLogSize: 10, - FileType: common.DefaultRowLogFileType, - CreateTime: time.Date(2000, 1, 1, 1, 1, 1, 1, &time.Location{}), - S3Storage: true, + FileType: redo.RedoRowLogFileType, + + UseExternalStorage: true, } w := &Writer{ cfg: cfg, @@ -259,10 +258,10 @@ func TestWriterGC(t *testing.T) { require.Nil(t, err) require.Equal(t, 1, len(files), "should have 1 log left after GC") - ts, fileType, err := common.ParseLogFileName(files[0].Name()) + ts, fileType, err := redo.ParseLogFileName(files[0].Name()) require.Nil(t, err, files[0].Name()) require.EqualValues(t, 3, ts) - require.Equal(t, common.DefaultRowLogFileType, fileType) + require.Equal(t, redo.RedoRowLogFileType, fileType) time.Sleep(time.Duration(100) * time.Millisecond) w1 := &Writer{ @@ -291,8 +290,8 @@ func TestNewWriter(t *testing.T) { uuidGen := uuid.NewConstGenerator("const-uuid") w, err := NewWriter(context.Background(), &FileWriterConfig{ - Dir: "sdfsf", - S3Storage: false, + Dir: "sdfsf", + UseExternalStorage: false, }, WithUUIDGenerator(func() uuid.Generator { return uuidGen }), ) @@ -324,10 +323,10 @@ func TestNewWriter(t *testing.T) { Dir: dir, CaptureID: "cp", ChangeFeedID: changefeed, - FileType: common.DefaultDDLLogFileType, - CreateTime: time.Date(2000, 1, 1, 1, 1, 1, 1, &time.Location{}), - S3Storage: true, - MaxLogSize: defaultMaxLogSize, + FileType: redo.RedoDDLLogFileType, + + UseExternalStorage: true, + MaxLogSize: defaultMaxLogSize, }, uint64buf: make([]byte, 8), storage: mockStorage, @@ -380,10 +379,10 @@ func TestRotateFileWithFileAllocator(t *testing.T) { Dir: dir, CaptureID: "cp", ChangeFeedID: changefeed, - FileType: common.DefaultRowLogFileType, - CreateTime: time.Date(2000, 1, 1, 1, 1, 1, 1, &time.Location{}), - S3Storage: true, - MaxLogSize: defaultMaxLogSize, + FileType: redo.RedoRowLogFileType, + + UseExternalStorage: true, + MaxLogSize: defaultMaxLogSize, }, uint64buf: make([]byte, 8), metricWriteBytes: common.RedoWriteBytesGauge. @@ -396,7 +395,7 @@ func TestRotateFileWithFileAllocator(t *testing.T) { uuidGenerator: uuidGen, } w.allocator = fsutil.NewFileAllocator( - w.cfg.Dir, common.DefaultRowLogFileType, defaultMaxLogSize) + w.cfg.Dir, redo.RedoRowLogFileType, defaultMaxLogSize) w.running.Store(true) _, err = w.Write([]byte("test")) @@ -445,10 +444,10 @@ func TestRotateFileWithoutFileAllocator(t *testing.T) { Dir: dir, CaptureID: "cp", ChangeFeedID: changefeed, - FileType: common.DefaultDDLLogFileType, - CreateTime: time.Date(2000, 1, 1, 1, 1, 1, 1, &time.Location{}), - S3Storage: true, - MaxLogSize: defaultMaxLogSize, + FileType: redo.RedoDDLLogFileType, + + UseExternalStorage: true, + MaxLogSize: defaultMaxLogSize, }, uint64buf: make([]byte, 8), metricWriteBytes: common.RedoWriteBytesGauge. diff --git a/cdc/redo/writer/mock_RedoLogWriter.go b/cdc/redo/writer/mock_RedoLogWriter.go deleted file mode 100644 index ac1a8dcbe8f..00000000000 --- a/cdc/redo/writer/mock_RedoLogWriter.go +++ /dev/null @@ -1,135 +0,0 @@ -// Code generated by mockery v2.14.1. DO NOT EDIT. - -package writer - -import ( - context "context" - - model "github.com/pingcap/tiflow/cdc/model" - mock "github.com/stretchr/testify/mock" -) - -// MockRedoLogWriter is an autogenerated mock type for the RedoLogWriter type -type MockRedoLogWriter struct { - mock.Mock -} - -// Close provides a mock function with given fields: -func (_m *MockRedoLogWriter) Close() error { - ret := _m.Called() - - var r0 error - if rf, ok := ret.Get(0).(func() error); ok { - r0 = rf() - } else { - r0 = ret.Error(0) - } - - return r0 -} - -// DeleteAllLogs provides a mock function with given fields: ctx -func (_m *MockRedoLogWriter) DeleteAllLogs(ctx context.Context) error { - ret := _m.Called(ctx) - - var r0 error - if rf, ok := ret.Get(0).(func(context.Context) error); ok { - r0 = rf(ctx) - } else { - r0 = ret.Error(0) - } - - return r0 -} - -// FlushLog provides a mock function with given fields: ctx, checkpointTs, resolvedTs -func (_m *MockRedoLogWriter) FlushLog(ctx context.Context, checkpointTs uint64, resolvedTs uint64) error { - ret := _m.Called(ctx, checkpointTs, resolvedTs) - - var r0 error - if rf, ok := ret.Get(0).(func(context.Context, uint64, uint64) error); ok { - r0 = rf(ctx, checkpointTs, resolvedTs) - } else { - r0 = ret.Error(0) - } - - return r0 -} - -// GC provides a mock function with given fields: ctx, checkpointTs -func (_m *MockRedoLogWriter) GC(ctx context.Context, checkpointTs uint64) error { - ret := _m.Called(ctx, checkpointTs) - - var r0 error - if rf, ok := ret.Get(0).(func(context.Context, uint64) error); ok { - r0 = rf(ctx, checkpointTs) - } else { - r0 = ret.Error(0) - } - - return r0 -} - -// GetMeta provides a mock function with given fields: -func (_m *MockRedoLogWriter) GetMeta() (uint64, uint64) { - ret := _m.Called() - - var r0 uint64 - if rf, ok := ret.Get(0).(func() uint64); ok { - r0 = rf() - } else { - r0 = ret.Get(0).(uint64) - } - - var r1 uint64 - if rf, ok := ret.Get(1).(func() uint64); ok { - r1 = rf() - } else { - r1 = ret.Get(1).(uint64) - } - - return r0, r1 -} - -// SendDDL provides a mock function with given fields: ctx, ddl -func (_m *MockRedoLogWriter) SendDDL(ctx context.Context, ddl *model.RedoDDLEvent) error { - ret := _m.Called(ctx, ddl) - - var r0 error - if rf, ok := ret.Get(0).(func(context.Context, *model.RedoDDLEvent) error); ok { - r0 = rf(ctx, ddl) - } else { - r0 = ret.Error(0) - } - - return r0 -} - -// WriteLog provides a mock function with given fields: ctx, rows -func (_m *MockRedoLogWriter) WriteLog(ctx context.Context, rows []*model.RedoRowChangedEvent) error { - ret := _m.Called(ctx, rows) - - var r0 error - if rf, ok := ret.Get(0).(func(context.Context, []*model.RedoRowChangedEvent) error); ok { - r0 = rf(ctx, rows) - } else { - r0 = ret.Error(0) - } - - return r0 -} - -type mockConstructorTestingTNewMockRedoLogWriter interface { - mock.TestingT - Cleanup(func()) -} - -// NewMockRedoLogWriter creates a new instance of MockRedoLogWriter. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations. -func NewMockRedoLogWriter(t mockConstructorTestingTNewMockRedoLogWriter) *MockRedoLogWriter { - mock := &MockRedoLogWriter{} - mock.Mock.Test(t) - - t.Cleanup(func() { mock.AssertExpectations(t) }) - - return mock -} diff --git a/cdc/redo/writer/writer.go b/cdc/redo/writer/writer.go index 1d8630376ca..28fa5fc29f1 100644 --- a/cdc/redo/writer/writer.go +++ b/cdc/redo/writer/writer.go @@ -19,16 +19,18 @@ import ( "net/url" "os" "path/filepath" - "time" "github.com/aws/aws-sdk-go/aws/awserr" "github.com/aws/aws-sdk-go/service/s3" "github.com/pingcap/errors" "github.com/pingcap/log" "github.com/pingcap/tidb/br/pkg/storage" + "github.com/pingcap/tiflow/cdc/contextutil" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/cdc/redo/common" + "github.com/pingcap/tiflow/pkg/config" cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/redo" "github.com/prometheus/client_golang/prometheus" "go.uber.org/multierr" "go.uber.org/zap" @@ -36,8 +38,6 @@ import ( ) // RedoLogWriter defines the interfaces used to write redo log, all operations are thread-safe. -// -//go:generate mockery --name=RedoLogWriter --inpackage --quiet type RedoLogWriter interface { // WriteLog writer RedoRowChangedEvent to row log file. WriteLog(ctx context.Context, rows []*model.RedoRowChangedEvent) error @@ -63,82 +63,121 @@ type RedoLogWriter interface { Close() error } -// LogWriterConfig is the configuration used by a Writer. -type LogWriterConfig struct { - Dir string - ChangeFeedID model.ChangeFeedID +// NewRedoLogWriter creates a new RedoLogWriter. +func NewRedoLogWriter( + ctx context.Context, + cfg *config.ConsistentConfig, + fileTypeConfig redo.FileTypeConfig, +) (RedoLogWriter, error) { + uri, err := storage.ParseRawURL(cfg.Storage) + if err != nil { + return nil, err + } + + scheme := uri.Scheme + if !redo.IsValidConsistentStorage(scheme) { + return nil, cerror.ErrConsistentStorage.GenWithStackByArgs(scheme) + } + if redo.IsBlackholeStorage(scheme) { + return NewBlackHoleWriter(), nil + } + + lwCfg := &logWriterConfig{ + FileTypeConfig: fileTypeConfig, + CaptureID: contextutil.CaptureAddrFromCtx(ctx), + ChangeFeedID: contextutil.ChangefeedIDFromCtx(ctx), + URI: *uri, + UseExternalStorage: redo.IsExternalStorage(scheme), + MaxLogSize: cfg.MaxLogSize, + } + + if lwCfg.UseExternalStorage { + // When an external storage is used, we use redoDir as a temporary dir to store redo logs + // before we flush them to S3. + changeFeedID := contextutil.ChangefeedIDFromCtx(ctx) + dataDir := config.GetGlobalServerConfig().DataDir + if changeFeedID.Namespace == model.DefaultNamespace { + lwCfg.Dir = filepath.Join(dataDir, config.DefaultRedoDir, changeFeedID.ID) + } else { + lwCfg.Dir = filepath.Join(dataDir, config.DefaultRedoDir, + changeFeedID.Namespace, changeFeedID.ID) + } + } else { + // When local storage or NFS is used, we use redoDir as the final storage path. + lwCfg.Dir = uri.Path + } + + return newLogWriter(ctx, lwCfg) +} + +type logWriterConfig struct { + redo.FileTypeConfig CaptureID string - CreateTime time.Time + ChangeFeedID model.ChangeFeedID + + URI url.URL + UseExternalStorage bool + // MaxLogSize is the maximum size of log in megabyte, defaults to defaultMaxLogSize. - MaxLogSize int64 - FlushIntervalInMs int64 - S3Storage bool - // S3URI should be like S3URI="s3://logbucket/test-changefeed?endpoint=http://$S3_ENDPOINT/" - S3URI url.URL - - EmitMeta bool - EmitRowEvents bool - EmitDDLEvents bool + MaxLogSize int64 + Dir string } -// LogWriter implement the RedoLogWriter interface -type LogWriter struct { - cfg *LogWriterConfig +// logWriter implement the RedoLogWriter interface +type logWriter struct { + cfg *logWriterConfig rowWriter fileWriter ddlWriter fileWriter - // storage in LogWriter is used to write meta and clean up + // extStorage in LogWriter is used to write meta and clean up // the redo log files when changefeed is created or deleted. - storage storage.ExternalStorage + extStorage storage.ExternalStorage meta *common.LogMeta metricTotalRowsCount prometheus.Gauge } -// NewLogWriter creates a LogWriter instance. It is guaranteed only one LogWriter per changefeed -func NewLogWriter( - ctx context.Context, cfg *LogWriterConfig, opts ...Option, -) (logWriter *LogWriter, err error) { +func newLogWriter( + ctx context.Context, cfg *logWriterConfig, opts ...Option, +) (lw *logWriter, err error) { if cfg == nil { return nil, cerror.WrapError(cerror.ErrRedoConfigInvalid, errors.New("LogWriterConfig can not be nil")) } - logWriter = &LogWriter{cfg: cfg} + lw = &logWriter{cfg: cfg} - if logWriter.cfg.EmitRowEvents { + if lw.cfg.EmitRowEvents { writerCfg := &FileWriterConfig{ - Dir: cfg.Dir, - ChangeFeedID: cfg.ChangeFeedID, - CaptureID: cfg.CaptureID, - FileType: common.DefaultRowLogFileType, - CreateTime: cfg.CreateTime, - MaxLogSize: cfg.MaxLogSize, - S3Storage: cfg.S3Storage, - S3URI: cfg.S3URI, + FileType: redo.RedoRowLogFileType, + ChangeFeedID: cfg.ChangeFeedID, + CaptureID: cfg.CaptureID, + URI: cfg.URI, + UseExternalStorage: cfg.UseExternalStorage, + MaxLogSize: cfg.MaxLogSize, + Dir: cfg.Dir, } - if logWriter.rowWriter, err = NewWriter(ctx, writerCfg, opts...); err != nil { + if lw.rowWriter, err = NewWriter(ctx, writerCfg, opts...); err != nil { return } } - if logWriter.cfg.EmitDDLEvents { + if lw.cfg.EmitDDLEvents { writerCfg := &FileWriterConfig{ - Dir: cfg.Dir, - ChangeFeedID: cfg.ChangeFeedID, - CaptureID: cfg.CaptureID, - FileType: common.DefaultDDLLogFileType, - CreateTime: cfg.CreateTime, - MaxLogSize: cfg.MaxLogSize, - S3Storage: cfg.S3Storage, - S3URI: cfg.S3URI, + FileType: redo.RedoDDLLogFileType, + ChangeFeedID: cfg.ChangeFeedID, + CaptureID: cfg.CaptureID, + URI: cfg.URI, + UseExternalStorage: cfg.UseExternalStorage, + MaxLogSize: cfg.MaxLogSize, + Dir: cfg.Dir, } - if logWriter.ddlWriter, err = NewWriter(ctx, writerCfg, opts...); err != nil { + if lw.ddlWriter, err = NewWriter(ctx, writerCfg, opts...); err != nil { return } } - if logWriter.cfg.EmitMeta { - if err = logWriter.initMeta(ctx); err != nil { + if lw.cfg.EmitMeta { + if err = lw.initMeta(ctx); err != nil { log.Warn("init redo meta fail", zap.String("namespace", cfg.ChangeFeedID.Namespace), zap.String("changefeed", cfg.ChangeFeedID.ID), @@ -147,25 +186,25 @@ func NewLogWriter( } } - if cfg.S3Storage { - logWriter.storage, err = common.InitS3storage(ctx, cfg.S3URI) + if cfg.UseExternalStorage { + lw.extStorage, err = redo.InitExternalStorage(ctx, cfg.URI) if err != nil { return nil, err } // since other process get the remove changefeed job async, may still write some logs after owner delete the log - err = logWriter.preCleanUpS3(ctx) + err = lw.preCleanUpS3(ctx) if err != nil { return nil, err } } - logWriter.metricTotalRowsCount = common.RedoTotalRowsCountGauge. + lw.metricTotalRowsCount = common.RedoTotalRowsCountGauge. WithLabelValues(cfg.ChangeFeedID.Namespace, cfg.ChangeFeedID.ID) return } -func (l *LogWriter) preCleanUpS3(ctx context.Context) error { - ret, err := l.storage.FileExists(ctx, l.getDeletedChangefeedMarker()) +func (l *logWriter) preCleanUpS3(ctx context.Context) error { + ret, err := l.extStorage.FileExists(ctx, l.getDeletedChangefeedMarker()) if err != nil { return cerror.WrapError(cerror.ErrS3StorageAPI, err) } @@ -188,7 +227,7 @@ func (l *LogWriter) preCleanUpS3(ctx context.Context) error { if err != nil { return err } - err = l.storage.DeleteFile(ctx, l.getDeletedChangefeedMarker()) + err = l.extStorage.DeleteFile(ctx, l.getDeletedChangefeedMarker()) if !isNotExistInS3(err) { return cerror.WrapError(cerror.ErrS3StorageAPI, err) } @@ -196,7 +235,7 @@ func (l *LogWriter) preCleanUpS3(ctx context.Context) error { return nil } -func (l *LogWriter) initMeta(ctx context.Context) error { +func (l *logWriter) initMeta(ctx context.Context) error { select { case <-ctx.Done(): return errors.Trace(ctx.Err()) @@ -222,7 +261,7 @@ func (l *LogWriter) initMeta(ctx context.Context) error { } // GC implement GC api -func (l *LogWriter) GC(ctx context.Context, ts model.Ts) error { +func (l *logWriter) GC(ctx context.Context, ts model.Ts) error { var err error if l.cfg.EmitRowEvents { err = multierr.Append(err, l.rowWriter.GC(ts)) @@ -234,7 +273,7 @@ func (l *LogWriter) GC(ctx context.Context, ts model.Ts) error { } // WriteLog implement WriteLog api -func (l *LogWriter) WriteLog(ctx context.Context, rows []*model.RedoRowChangedEvent) error { +func (l *logWriter) WriteLog(ctx context.Context, rows []*model.RedoRowChangedEvent) error { select { case <-ctx.Done(): return errors.Trace(ctx.Err()) @@ -271,7 +310,7 @@ func (l *LogWriter) WriteLog(ctx context.Context, rows []*model.RedoRowChangedEv } // SendDDL implement SendDDL api -func (l *LogWriter) SendDDL(ctx context.Context, ddl *model.RedoDDLEvent) error { +func (l *logWriter) SendDDL(ctx context.Context, ddl *model.RedoDDLEvent) error { select { case <-ctx.Done(): return errors.Trace(ctx.Err()) @@ -297,7 +336,7 @@ func (l *LogWriter) SendDDL(ctx context.Context, ddl *model.RedoDDLEvent) error } // FlushLog implement FlushLog api -func (l *LogWriter) FlushLog(ctx context.Context, checkpointTs, resolvedTs model.Ts) error { +func (l *logWriter) FlushLog(ctx context.Context, checkpointTs, resolvedTs model.Ts) error { select { case <-ctx.Done(): return errors.Trace(ctx.Err()) @@ -312,7 +351,7 @@ func (l *LogWriter) FlushLog(ctx context.Context, checkpointTs, resolvedTs model } // GetMeta implement GetMeta api -func (l *LogWriter) GetMeta() (checkpointTs, resolvedTs model.Ts) { +func (l *logWriter) GetMeta() (checkpointTs, resolvedTs model.Ts) { return l.meta.CheckpointTs, l.meta.ResolvedTs } @@ -320,7 +359,7 @@ func (l *LogWriter) GetMeta() (checkpointTs, resolvedTs model.Ts) { // FIXME: currently only owner will call this. We need to split it into 2 functions: // 1. cleanLocalStorage, which should be called on processors; // 2. cleanRemoteStorage, which should be called on owner. -func (l *LogWriter) DeleteAllLogs(ctx context.Context) (err error) { +func (l *logWriter) DeleteAllLogs(ctx context.Context) (err error) { if err = l.Close(); err != nil { return } @@ -361,7 +400,7 @@ func (l *LogWriter) DeleteAllLogs(ctx context.Context) (err error) { } } - if !l.cfg.S3Storage { + if !l.cfg.UseExternalStorage { return } @@ -385,23 +424,24 @@ func (l *LogWriter) DeleteAllLogs(ctx context.Context) (err error) { return } -func (l *LogWriter) getDeletedChangefeedMarker() string { +func (l *logWriter) getDeletedChangefeedMarker() string { if l.cfg.ChangeFeedID.Namespace == model.DefaultNamespace { return fmt.Sprintf("delete_%s", l.cfg.ChangeFeedID.ID) } return fmt.Sprintf("delete_%s_%s", l.cfg.ChangeFeedID.Namespace, l.cfg.ChangeFeedID.ID) } -func (l *LogWriter) writeDeletedMarkerToS3(ctx context.Context) error { - return cerror.WrapError(cerror.ErrS3StorageAPI, l.storage.WriteFile(ctx, l.getDeletedChangefeedMarker(), []byte("D"))) +func (l *logWriter) writeDeletedMarkerToS3(ctx context.Context) error { + return cerror.WrapError(cerror.ErrS3StorageAPI, + l.extStorage.WriteFile(ctx, l.getDeletedChangefeedMarker(), []byte("D"))) } -func (l *LogWriter) deleteFilesInS3(ctx context.Context, files []string) error { +func (l *logWriter) deleteFilesInS3(ctx context.Context, files []string) error { eg, eCtx := errgroup.WithContext(ctx) for _, f := range files { name := f eg.Go(func() error { - err := l.storage.DeleteFile(eCtx, name) + err := l.extStorage.DeleteFile(eCtx, name) if err != nil { // if fail then retry, may end up with notExit err, ignore the error if !isNotExistInS3(err) { @@ -426,9 +466,9 @@ func isNotExistInS3(err error) bool { return false } -var getAllFilesInS3 = func(ctx context.Context, l *LogWriter) ([]string, error) { +var getAllFilesInS3 = func(ctx context.Context, l *logWriter) ([]string, error) { files := []string{} - err := l.storage.WalkDir(ctx, &storage.WalkOption{}, func(path string, _ int64) error { + err := l.extStorage.WalkDir(ctx, &storage.WalkOption{}, func(path string, _ int64) error { files = append(files, path) return nil }) @@ -440,7 +480,7 @@ var getAllFilesInS3 = func(ctx context.Context, l *LogWriter) ([]string, error) } // Close implements RedoLogWriter.Close. -func (l *LogWriter) Close() (err error) { +func (l *logWriter) Close() (err error) { common.RedoTotalRowsCountGauge. DeleteLabelValues(l.cfg.ChangeFeedID.Namespace, l.cfg.ChangeFeedID.ID) @@ -454,7 +494,7 @@ func (l *LogWriter) Close() (err error) { } // flush flushes all the buffered data to the disk. -func (l *LogWriter) flush(checkpointTs, resolvedTs model.Ts) (err error) { +func (l *logWriter) flush(checkpointTs, resolvedTs model.Ts) (err error) { if l.cfg.EmitDDLEvents { err = multierr.Append(err, l.ddlWriter.Flush()) } @@ -467,7 +507,7 @@ func (l *LogWriter) flush(checkpointTs, resolvedTs model.Ts) (err error) { return } -func (l *LogWriter) isStopped() bool { +func (l *logWriter) isStopped() bool { var rowStopped, ddlStopped bool if l.cfg.EmitRowEvents { rowStopped = !l.rowWriter.IsRunning() @@ -478,17 +518,17 @@ func (l *LogWriter) isStopped() bool { return rowStopped || ddlStopped } -func (l *LogWriter) getMetafileName() string { +func (l *logWriter) getMetafileName() string { if model.DefaultNamespace == l.cfg.ChangeFeedID.Namespace { return fmt.Sprintf("%s_%s_%s%s", l.cfg.CaptureID, l.cfg.ChangeFeedID.ID, - common.DefaultMetaFileType, common.MetaEXT) + redo.RedoMetaFileType, redo.MetaEXT) } return fmt.Sprintf("%s_%s_%s_%s%s", l.cfg.CaptureID, l.cfg.ChangeFeedID.Namespace, l.cfg.ChangeFeedID.ID, - common.DefaultMetaFileType, common.MetaEXT) + redo.RedoMetaFileType, redo.MetaEXT) } -func (l *LogWriter) maybeUpdateMeta(checkpointTs, resolvedTs uint64) ([]byte, error) { +func (l *logWriter) maybeUpdateMeta(checkpointTs, resolvedTs uint64) ([]byte, error) { // NOTE: both checkpoint and resolved can regress if a cdc instance restarts. hasChange := false if checkpointTs > l.meta.CheckpointTs { @@ -523,7 +563,7 @@ func (l *LogWriter) maybeUpdateMeta(checkpointTs, resolvedTs uint64) ([]byte, er return data, err } -func (l *LogWriter) flushLogMeta(checkpointTs, resolvedTs uint64) error { +func (l *logWriter) flushLogMeta(checkpointTs, resolvedTs uint64) error { data, err := l.maybeUpdateMeta(checkpointTs, resolvedTs) if err != nil { return err @@ -532,13 +572,13 @@ func (l *LogWriter) flushLogMeta(checkpointTs, resolvedTs uint64) error { return nil } - err = os.MkdirAll(l.cfg.Dir, common.DefaultDirMode) + err = os.MkdirAll(l.cfg.Dir, redo.DefaultDirMode) if err != nil { return cerror.WrapError(cerror.ErrRedoFileOp, errors.Annotate(err, "can't make dir for new redo logfile")) } // we will create a temp metadata file and then atomically rename it. - tmpFileName := l.filePath() + common.MetaTmpEXT + tmpFileName := l.filePath() + redo.MetaTmpEXT tmpFile, err := openTruncFile(tmpFileName) if err != nil { return cerror.WrapError(cerror.ErrRedoFileOp, err) @@ -570,7 +610,7 @@ func (l *LogWriter) flushLogMeta(checkpointTs, resolvedTs uint64) error { return cerror.WrapError(cerror.ErrRedoFileOp, err) } - if !l.cfg.S3Storage { + if !l.cfg.UseExternalStorage { return nil } @@ -579,23 +619,23 @@ func (l *LogWriter) flushLogMeta(checkpointTs, resolvedTs uint64) error { return l.writeMetaToS3(ctx) } -func (l *LogWriter) writeMetaToS3(ctx context.Context) error { +func (l *logWriter) writeMetaToS3(ctx context.Context) error { name := l.filePath() fileData, err := os.ReadFile(name) if err != nil { return cerror.WrapError(cerror.ErrRedoFileOp, err) } - return cerror.WrapError(cerror.ErrS3StorageAPI, l.storage.WriteFile(ctx, l.getMetafileName(), fileData)) + return cerror.WrapError(cerror.ErrS3StorageAPI, + l.extStorage.WriteFile(ctx, l.getMetafileName(), fileData)) } -func (l *LogWriter) filePath() string { +func (l *logWriter) filePath() string { return filepath.Join(l.cfg.Dir, l.getMetafileName()) } -func (cfg LogWriterConfig) String() string { - return fmt.Sprintf("%s:%s:%s:%s:%d:%d:%s:%t", - cfg.ChangeFeedID.Namespace, cfg.ChangeFeedID.ID, - cfg.CaptureID, cfg.Dir, cfg.MaxLogSize, - cfg.FlushIntervalInMs, cfg.S3URI.String(), cfg.S3Storage) +func (cfg logWriterConfig) String() string { + return fmt.Sprintf("%s:%s:%s:%s:%d:%s:%t", + cfg.ChangeFeedID.Namespace, cfg.ChangeFeedID.ID, cfg.CaptureID, + cfg.Dir, cfg.MaxLogSize, cfg.URI.String(), cfg.UseExternalStorage) } diff --git a/cdc/redo/writer/writer_test.go b/cdc/redo/writer/writer_test.go index 2c5f9b456ea..329a9c39f8d 100644 --- a/cdc/redo/writer/writer_test.go +++ b/cdc/redo/writer/writer_test.go @@ -20,7 +20,6 @@ import ( "os" "path/filepath" "testing" - "time" "github.com/aws/aws-sdk-go/aws/awserr" "github.com/aws/aws-sdk-go/service/s3" @@ -32,6 +31,7 @@ import ( "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/cdc/redo/common" cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/redo" "github.com/pingcap/tiflow/pkg/uuid" "github.com/stretchr/testify/mock" "github.com/stretchr/testify/require" @@ -120,12 +120,12 @@ func TestLogWriterWriteLog(t *testing.T) { mockWriter.On("Write", mock.Anything).Return(1, tt.writerErr) mockWriter.On("IsRunning").Return(tt.isRunning) mockWriter.On("AdvanceTs", mock.Anything) - writer := LogWriter{ - cfg: &LogWriterConfig{ + writer := logWriter{ + cfg: &logWriterConfig{FileTypeConfig: redo.FileTypeConfig{ EmitMeta: true, EmitRowEvents: true, EmitDDLEvents: true, - }, + }}, rowWriter: mockWriter, ddlWriter: mockWriter, meta: &common.LogMeta{}, @@ -223,11 +223,11 @@ func TestLogWriterSendDDL(t *testing.T) { mockWriter.On("Write", mock.Anything).Return(1, tt.writerErr) mockWriter.On("IsRunning").Return(tt.isRunning) mockWriter.On("AdvanceTs", mock.Anything) - writer := LogWriter{ - cfg: &LogWriterConfig{ + writer := logWriter{ + cfg: &logWriterConfig{FileTypeConfig: redo.FileTypeConfig{ EmitRowEvents: true, EmitDDLEvents: true, - }, + }}, rowWriter: mockWriter, ddlWriter: mockWriter, meta: &common.LogMeta{}, @@ -320,25 +320,24 @@ func TestLogWriterFlushLog(t *testing.T) { mockWriter := &mockFileWriter{} mockWriter.On("Flush", mock.Anything).Return(tt.flushErr) mockWriter.On("IsRunning").Return(tt.isRunning) - cfg := &LogWriterConfig{ - Dir: dir, - ChangeFeedID: model.DefaultChangeFeedID("test-cf"), - CaptureID: "cp", - MaxLogSize: 10, - CreateTime: time.Date(2000, 1, 1, 1, 1, 1, 1, &time.Location{}), - FlushIntervalInMs: 5, - S3Storage: true, - - EmitMeta: true, - EmitRowEvents: true, - EmitDDLEvents: true, + cfg := &logWriterConfig{ + FileTypeConfig: redo.FileTypeConfig{ + EmitMeta: true, + EmitRowEvents: true, + EmitDDLEvents: true, + }, + Dir: dir, + ChangeFeedID: model.DefaultChangeFeedID("test-cf"), + CaptureID: "cp", + MaxLogSize: 10, + UseExternalStorage: true, } - writer := LogWriter{ - cfg: cfg, - rowWriter: mockWriter, - ddlWriter: mockWriter, - meta: &common.LogMeta{}, - storage: mockStorage, + writer := logWriter{ + cfg: cfg, + rowWriter: mockWriter, + ddlWriter: mockWriter, + meta: &common.LogMeta{}, + extStorage: mockStorage, } if tt.name == "context cancel" { @@ -358,15 +357,16 @@ func TestLogWriterFlushLog(t *testing.T) { // checkpoint or meta regress should be ignored correctly. func TestLogWriterRegress(t *testing.T) { dir := t.TempDir() - writer, err := NewLogWriter(context.Background(), &LogWriterConfig{ - Dir: dir, - ChangeFeedID: model.DefaultChangeFeedID("test-log-writer-regress"), - CaptureID: "cp", - S3Storage: false, - - EmitMeta: true, - EmitRowEvents: true, - EmitDDLEvents: true, + writer, err := newLogWriter(context.Background(), &logWriterConfig{ + FileTypeConfig: redo.FileTypeConfig{ + EmitMeta: true, + EmitRowEvents: true, + EmitDDLEvents: true, + }, + Dir: dir, + ChangeFeedID: model.DefaultChangeFeedID("test-log-writer-regress"), + CaptureID: "cp", + UseExternalStorage: false, }) require.Nil(t, err) require.Nil(t, writer.FlushLog(context.Background(), 2, 4)) @@ -377,55 +377,51 @@ func TestLogWriterRegress(t *testing.T) { } func TestNewLogWriter(t *testing.T) { - _, err := NewLogWriter(context.Background(), nil) + _, err := newLogWriter(context.Background(), nil) require.NotNil(t, err) ctx, cancel := context.WithCancel(context.Background()) defer cancel() - cfg := &LogWriterConfig{ - Dir: "dirt", - ChangeFeedID: model.DefaultChangeFeedID("test-cf"), - CaptureID: "cp", - MaxLogSize: 10, - CreateTime: time.Date(2000, 1, 1, 1, 1, 1, 1, &time.Location{}), - FlushIntervalInMs: 5, - - EmitMeta: true, - EmitRowEvents: true, - EmitDDLEvents: true, + cfg := &logWriterConfig{ + FileTypeConfig: redo.FileTypeConfig{ + EmitMeta: true, + EmitRowEvents: true, + EmitDDLEvents: true, + }, + Dir: "dirt", + ChangeFeedID: model.DefaultChangeFeedID("test-cf"), + CaptureID: "cp", + MaxLogSize: 10, } uuidGen := uuid.NewConstGenerator("const-uuid") - ll, err := NewLogWriter(ctx, cfg, + ll, err := newLogWriter(ctx, cfg, WithUUIDGenerator(func() uuid.Generator { return uuidGen }), ) require.Nil(t, err) - cfg1 := &LogWriterConfig{ - Dir: "dirt111", - ChangeFeedID: model.DefaultChangeFeedID("test-cf"), - CaptureID: "cp", - MaxLogSize: 10, - CreateTime: time.Date(2000, 1, 1, 1, 1, 1, 1, &time.Location{}), - FlushIntervalInMs: 5, + cfg1 := &logWriterConfig{ + Dir: "dirt111", + ChangeFeedID: model.DefaultChangeFeedID("test-cf"), + CaptureID: "cp", + MaxLogSize: 10, } - ll1, err := NewLogWriter(ctx, cfg1) + ll1, err := newLogWriter(ctx, cfg1) require.Nil(t, err) require.NotSame(t, ll, ll1) dir := t.TempDir() - cfg = &LogWriterConfig{ - Dir: dir, - ChangeFeedID: model.DefaultChangeFeedID("test-cf"), - CaptureID: "cp", - MaxLogSize: 10, - CreateTime: time.Date(2000, 1, 1, 1, 1, 1, 1, &time.Location{}), - FlushIntervalInMs: 5, - - EmitMeta: true, - EmitRowEvents: true, - EmitDDLEvents: true, + cfg = &logWriterConfig{ + FileTypeConfig: redo.FileTypeConfig{ + EmitMeta: true, + EmitRowEvents: true, + EmitDDLEvents: true, + }, + Dir: dir, + ChangeFeedID: model.DefaultChangeFeedID("test-cf"), + CaptureID: "cp", + MaxLogSize: 10, } - l, err := NewLogWriter(ctx, cfg) + l, err := newLogWriter(ctx, cfg) require.Nil(t, err) err = l.Close() require.Nil(t, err) @@ -439,7 +435,7 @@ func TestNewLogWriter(t *testing.T) { _, err = f.Write(data) require.Nil(t, err) - l, err = NewLogWriter(ctx, cfg) + l, err = newLogWriter(ctx, cfg) require.Nil(t, err) err = l.Close() require.Nil(t, err) @@ -448,27 +444,27 @@ func TestNewLogWriter(t *testing.T) { require.Equal(t, meta.CheckpointTs, l.meta.CheckpointTs) require.Equal(t, meta.ResolvedTs, l.meta.ResolvedTs) - origin := common.InitS3storage + origin := redo.InitExternalStorage defer func() { - common.InitS3storage = origin + redo.InitExternalStorage = origin }() controller := gomock.NewController(t) mockStorage := mockstorage.NewMockExternalStorage(controller) // skip pre cleanup mockStorage.EXPECT().FileExists(gomock.Any(), gomock.Any()).Return(false, nil) - common.InitS3storage = func(ctx context.Context, uri url.URL) (storage.ExternalStorage, error) { + redo.InitExternalStorage = func( + ctx context.Context, uri url.URL, + ) (storage.ExternalStorage, error) { return mockStorage, nil } - cfg3 := &LogWriterConfig{ - Dir: dir, - ChangeFeedID: model.DefaultChangeFeedID("test-cf112232"), - CaptureID: "cp", - MaxLogSize: 10, - CreateTime: time.Date(2000, 1, 1, 1, 1, 1, 1, &time.Location{}), - FlushIntervalInMs: 5, - S3Storage: true, + cfg3 := &logWriterConfig{ + Dir: dir, + ChangeFeedID: model.DefaultChangeFeedID("test-cf112232"), + CaptureID: "cp", + MaxLogSize: 10, + UseExternalStorage: true, } - l3, err := NewLogWriter(ctx, cfg3) + l3, err := newLogWriter(ctx, cfg3) require.Nil(t, err) err = l3.Close() require.Nil(t, err) @@ -561,7 +557,7 @@ func TestDeleteAllLogs(t *testing.T) { require.Nil(t, err) origin := getAllFilesInS3 - getAllFilesInS3 = func(ctx context.Context, l *LogWriter) ([]string, error) { + getAllFilesInS3 = func(ctx context.Context, l *logWriter) ([]string, error) { return []string{fileName, fileName1}, tt.getAllFilesInS3Err } controller := gomock.NewController(t) @@ -572,25 +568,24 @@ func TestDeleteAllLogs(t *testing.T) { mockWriter := &mockFileWriter{} mockWriter.On("Close").Return(tt.closeErr) - cfg := &LogWriterConfig{ - Dir: dir, - ChangeFeedID: tt.changefeed, - CaptureID: "cp", - MaxLogSize: 10, - CreateTime: time.Date(2000, 1, 1, 1, 1, 1, 1, &time.Location{}), - FlushIntervalInMs: 5, - S3Storage: tt.args.enableS3, - - EmitMeta: true, - EmitRowEvents: true, - EmitDDLEvents: true, + cfg := &logWriterConfig{ + FileTypeConfig: redo.FileTypeConfig{ + EmitMeta: true, + EmitRowEvents: true, + EmitDDLEvents: true, + }, + Dir: dir, + ChangeFeedID: tt.changefeed, + CaptureID: "cp", + MaxLogSize: 10, + UseExternalStorage: tt.args.enableS3, } - writer := LogWriter{ - rowWriter: mockWriter, - ddlWriter: mockWriter, - meta: &common.LogMeta{}, - cfg: cfg, - storage: mockStorage, + writer := logWriter{ + rowWriter: mockWriter, + ddlWriter: mockWriter, + meta: &common.LogMeta{}, + cfg: cfg, + extStorage: mockStorage, } ret := writer.DeleteAllLogs(context.Background()) if tt.wantErr != "" { @@ -660,7 +655,7 @@ func TestPreCleanUpS3(t *testing.T) { } for _, cf := range cfs { origin := getAllFilesInS3 - getAllFilesInS3 = func(ctx context.Context, l *LogWriter) ([]string, error) { + getAllFilesInS3 = func(ctx context.Context, l *logWriter) ([]string, error) { if cf.Namespace == model.DefaultNamespace { return []string{"1", "11", "delete_test-cf"}, tc.getAllFilesInS3Err } @@ -674,18 +669,15 @@ func TestPreCleanUpS3(t *testing.T) { mockStorage.EXPECT().DeleteFile(gomock.Any(), gomock.Any()). Return(tc.deleteFileErr).MaxTimes(3) - cfg := &LogWriterConfig{ + cfg := &logWriterConfig{ Dir: "dir", ChangeFeedID: cf, CaptureID: "cp", MaxLogSize: 10, - CreateTime: time.Date(2000, 1, 1, 1, 1, 1, - 1, &time.Location{}), - FlushIntervalInMs: 5, } - writer := LogWriter{ - cfg: cfg, - storage: mockStorage, + writer := logWriter{ + cfg: cfg, + extStorage: mockStorage, } ret := writer.preCleanUpS3(context.Background()) if tc.wantErr != "" { diff --git a/cdc/sinkv2/ddlsink/cloudstorage/cloud_storage_ddl_sink.go b/cdc/sinkv2/ddlsink/cloudstorage/cloud_storage_ddl_sink.go index bdb6f61a37d..57a070f82a1 100644 --- a/cdc/sinkv2/ddlsink/cloudstorage/cloud_storage_ddl_sink.go +++ b/cdc/sinkv2/ddlsink/cloudstorage/cloud_storage_ddl_sink.go @@ -23,11 +23,11 @@ import ( "github.com/pingcap/tidb/br/pkg/storage" "github.com/pingcap/tiflow/cdc/contextutil" "github.com/pingcap/tiflow/cdc/model" - "github.com/pingcap/tiflow/cdc/redo/common" "github.com/pingcap/tiflow/cdc/sinkv2/ddlsink" "github.com/pingcap/tiflow/cdc/sinkv2/metrics" "github.com/pingcap/tiflow/pkg/sink" "github.com/pingcap/tiflow/pkg/sink/cloudstorage" + "github.com/pingcap/tiflow/pkg/util" ) // Assert DDLEventSink implementation @@ -43,22 +43,12 @@ type ddlSink struct { // NewCloudStorageDDLSink creates a ddl sink for cloud storage. func NewCloudStorageDDLSink(ctx context.Context, sinkURI *url.URL) (*ddlSink, error) { - // parse backend storage from sinkURI - bs, err := storage.ParseBackend(sinkURI.String(), nil) + storage, err := util.GetExternalStorageFromURI(ctx, sinkURI.String()) if err != nil { return nil, err } - // create an external storage. - storage, err := storage.New(ctx, bs, &storage.ExternalStorageOptions{ - SendCredentials: false, - S3Retryer: common.DefaultS3Retryer(), - }) - if err != nil { - return nil, err - } changefeedID := contextutil.ChangefeedIDFromCtx(ctx) - d := &ddlSink{ id: changefeedID, storage: storage, diff --git a/cdc/sinkv2/eventsink/cloudstorage/cloud_storage_dml_sink.go b/cdc/sinkv2/eventsink/cloudstorage/cloud_storage_dml_sink.go index a3fc42976fc..086b9be9b60 100644 --- a/cdc/sinkv2/eventsink/cloudstorage/cloud_storage_dml_sink.go +++ b/cdc/sinkv2/eventsink/cloudstorage/cloud_storage_dml_sink.go @@ -19,10 +19,8 @@ import ( "sync/atomic" "github.com/pingcap/errors" - "github.com/pingcap/tidb/br/pkg/storage" "github.com/pingcap/tiflow/cdc/contextutil" "github.com/pingcap/tiflow/cdc/model" - rcommon "github.com/pingcap/tiflow/cdc/redo/common" "github.com/pingcap/tiflow/cdc/sink/codec/builder" "github.com/pingcap/tiflow/cdc/sink/codec/common" "github.com/pingcap/tiflow/cdc/sinkv2/eventsink" @@ -33,6 +31,7 @@ import ( cerror "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/sink" "github.com/pingcap/tiflow/pkg/sink/cloudstorage" + putil "github.com/pingcap/tiflow/pkg/util" ) const ( @@ -95,17 +94,8 @@ func NewCloudStorageSink(ctx context.Context, return nil, err } - // parse backend storage from sinkURI. - bs, err := storage.ParseBackend(sinkURI.String(), nil) - if err != nil { - return nil, err - } - // create an external storage. - storage, err := storage.New(ctx, bs, &storage.ExternalStorageOptions{ - SendCredentials: false, - S3Retryer: rcommon.DefaultS3Retryer(), - }) + storage, err := putil.GetExternalStorageFromURI(ctx, sinkURI.String()) if err != nil { return nil, err } diff --git a/cdc/sinkv2/eventsink/cloudstorage/dml_worker_test.go b/cdc/sinkv2/eventsink/cloudstorage/dml_worker_test.go index 25421d9ca21..5f8d375a5fc 100644 --- a/cdc/sinkv2/eventsink/cloudstorage/dml_worker_test.go +++ b/cdc/sinkv2/eventsink/cloudstorage/dml_worker_test.go @@ -21,7 +21,6 @@ import ( "testing" "time" - "github.com/pingcap/tidb/br/pkg/storage" timodel "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/parser/types" @@ -34,14 +33,13 @@ import ( "github.com/pingcap/tiflow/pkg/config" "github.com/pingcap/tiflow/pkg/sink" "github.com/pingcap/tiflow/pkg/sink/cloudstorage" + "github.com/pingcap/tiflow/pkg/util" "github.com/stretchr/testify/require" ) func testDMLWorker(ctx context.Context, t *testing.T, dir string) *dmlWorker { uri := fmt.Sprintf("file:///%s?flush-interval=2s", dir) - bs, err := storage.ParseBackend(uri, &storage.BackendOptions{}) - require.Nil(t, err) - storage, err := storage.New(ctx, bs, nil) + storage, err := util.GetExternalStorageFromURI(ctx, uri) require.Nil(t, err) errCh := make(chan error, 1) sinkURI, err := url.Parse(uri) diff --git a/cmd/storage-consumer/main.go b/cmd/storage-consumer/main.go index d5aa4a665ce..ad0bc72ae74 100644 --- a/cmd/storage-consumer/main.go +++ b/cmd/storage-consumer/main.go @@ -33,7 +33,6 @@ import ( "github.com/pingcap/log" "github.com/pingcap/tidb/br/pkg/storage" "github.com/pingcap/tiflow/cdc/model" - rcommon "github.com/pingcap/tiflow/cdc/redo/common" "github.com/pingcap/tiflow/cdc/sink" "github.com/pingcap/tiflow/cdc/sink/codec" "github.com/pingcap/tiflow/cdc/sink/codec/canal" @@ -46,6 +45,7 @@ import ( "github.com/pingcap/tiflow/pkg/quotes" psink "github.com/pingcap/tiflow/pkg/sink" "github.com/pingcap/tiflow/pkg/sink/cloudstorage" + putil "github.com/pingcap/tiflow/pkg/util" "go.uber.org/zap" ) @@ -264,16 +264,8 @@ func newConsumer(ctx context.Context) (*consumer, error) { } extension := sinkutil.GetFileExtension(protocol) - bs, err := storage.ParseBackend(upstreamURIStr, nil) - if err != nil { - log.Error("failed to parse storage backend", zap.Error(err)) - return nil, err - } - storage, err := storage.New(ctx, bs, &storage.ExternalStorageOptions{ - SendCredentials: false, - S3Retryer: rcommon.DefaultS3Retryer(), - }) + storage, err := putil.GetExternalStorageFromURI(ctx, upstreamURIStr) if err != nil { log.Error("failed to create external storage", zap.Error(err)) return nil, err diff --git a/engine/pkg/externalresource/internal/s3/storage_factory.go b/engine/pkg/externalresource/internal/s3/storage_factory.go index a0af2b0f1bd..996b5b60a40 100644 --- a/engine/pkg/externalresource/internal/s3/storage_factory.go +++ b/engine/pkg/externalresource/internal/s3/storage_factory.go @@ -21,6 +21,7 @@ import ( brStorage "github.com/pingcap/tidb/br/pkg/storage" "github.com/pingcap/tiflow/engine/pkg/externalresource/internal" "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/util" ) // ExternalStorageFactory represents a factory used to create @@ -88,14 +89,9 @@ func GetExternalStorageFromURI( opts := &brStorage.BackendOptions{ S3: s3Opts, } - backEnd, err := brStorage.ParseBackend(uri, opts) - if err != nil { - return nil, errors.Trace(err) - } // Note that we may have network I/O here. - // TODO: use proper retry policy. - ret, err := brStorage.New(ctx, backEnd, &brStorage.ExternalStorageOptions{}) + ret, err := util.GetExternalStorage(ctx, uri, opts) if err != nil { retErr := errors.ErrFailToCreateExternalStorage.Wrap(errors.Trace(err)) return nil, retErr.GenWithStackByArgs("creating ExternalStorage for s3") diff --git a/errors.toml b/errors.toml index 85e3d1bb820..71d1c084886 100755 --- a/errors.toml +++ b/errors.toml @@ -996,11 +996,6 @@ error = ''' s3 storage api ''' -["CDC:ErrS3StorageInitialize"] -error = ''' -new s3 storage for redo log -''' - ["CDC:ErrScanLockFailed"] error = ''' scan lock failed @@ -1111,6 +1106,11 @@ error = ''' fail to create or maintain changefeed because start-ts %d is earlier than or equal to GC safepoint at %d ''' +["CDC:ErrStorageInitialize"] +error = ''' +fail to open storage for redo log +''' + ["CDC:ErrStorageSinkInvalidDateSeparator"] error = ''' date separator in cloud storage sink is invalid diff --git a/pkg/applier/redo.go b/pkg/applier/redo.go index 13c67f74c38..f3227b169d8 100644 --- a/pkg/applier/redo.go +++ b/pkg/applier/redo.go @@ -21,12 +21,13 @@ import ( "github.com/pingcap/log" "github.com/pingcap/tiflow/cdc/contextutil" "github.com/pingcap/tiflow/cdc/model" - "github.com/pingcap/tiflow/cdc/redo" + "github.com/pingcap/tiflow/cdc/redo/common" "github.com/pingcap/tiflow/cdc/redo/reader" "github.com/pingcap/tiflow/cdc/sink" "github.com/pingcap/tiflow/cdc/sink/mysql" "github.com/pingcap/tiflow/pkg/config" cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/redo" "github.com/pingcap/tiflow/pkg/util" "go.uber.org/zap" "golang.org/x/sync/errgroup" @@ -70,12 +71,12 @@ func (rac *RedoApplierConfig) toLogReaderConfig() (string, *reader.LogReaderConf return "", nil, cerror.WrapError(cerror.ErrConsistentStorage, err) } cfg := &reader.LogReaderConfig{ - Dir: uri.Path, - S3Storage: redo.IsS3StorageEnabled(uri.Scheme), + Dir: uri.Path, + UseExternalStorage: redo.IsExternalStorage(uri.Scheme), } - if cfg.S3Storage { - cfg.S3URI = *uri - // If use s3 as backend, applier will download redo logs to local dir. + if cfg.UseExternalStorage { + cfg.URI = *uri + // If use external storage as backend, applier will download redo logs to local dir. cfg.Dir = rac.Dir } return uri.Scheme, cfg, nil @@ -159,7 +160,7 @@ func (ra *RedoApplier) consumeLogs(ctx context.Context) error { } cachedRows = make([]*model.RowChangedEvent, 0, emitBatch) } - cachedRows = append(cachedRows, redo.LogToRow(redoLog)) + cachedRows = append(cachedRows, common.LogToRow(redoLog)) if redoLog.Row.CommitTs > tableResolvedTsMap[tableID] { tableResolvedTsMap[tableID], lastResolvedTs = lastResolvedTs, redoLog.Row.CommitTs @@ -200,7 +201,7 @@ func createRedoReaderImpl(ctx context.Context, cfg *RedoApplierConfig) (reader.R if err != nil { return nil, err } - return redo.NewRedoReader(ctx, storageType, readerCfg) + return reader.NewRedoLogReader(ctx, storageType, readerCfg) } // ReadMeta creates a new redo applier and read meta from reader diff --git a/pkg/applier/redo_test.go b/pkg/applier/redo_test.go index 59a5ae7b769..d8e728aaed1 100644 --- a/pkg/applier/redo_test.go +++ b/pkg/applier/redo_test.go @@ -22,7 +22,7 @@ import ( "github.com/DATA-DOG/go-sqlmock" "github.com/phayes/freeport" "github.com/pingcap/tiflow/cdc/model" - "github.com/pingcap/tiflow/cdc/redo" + "github.com/pingcap/tiflow/cdc/redo/common" "github.com/pingcap/tiflow/cdc/redo/reader" "github.com/pingcap/tiflow/cdc/sink/mysql" "github.com/stretchr/testify/require" @@ -233,7 +233,7 @@ func TestApplyDMLs(t *testing.T) { }, } for _, dml := range dmls { - redoLogCh <- redo.RowToRedo(dml) + redoLogCh <- common.RowToRedo(dml) } close(redoLogCh) close(ddlEventCh) diff --git a/pkg/cmd/util/changefeed.toml b/pkg/cmd/util/changefeed.toml index 3c9528605b7..fd05d27614c 100644 --- a/pkg/cmd/util/changefeed.toml +++ b/pkg/cmd/util/changefeed.toml @@ -52,8 +52,8 @@ level = "none" # file size of single redo log, unit is MB max-log-size = 64 # 刷新或上传 redo log 至 S3 的间隔,单位毫秒 -# interval to flush or upload redo log, default is 1000ms, unit is microseconds -flush-interval = 1000 +# interval to flush or upload redo log, default is 2000ms, unit is microseconds +flush-interval = 2000 # 存储 redo log 的形式,包括 nfs(NFS 目录),S3(上传至S3),blackhole(测试用) # storage type for redo log # nfs: store redo logs in nfs directly diff --git a/pkg/config/consistent.go b/pkg/config/consistent.go index 2b1a42fd16e..4e4c94ca93d 100644 --- a/pkg/config/consistent.go +++ b/pkg/config/consistent.go @@ -13,10 +13,44 @@ package config -// ConsistentConfig represents replication consistency config for a changefeed +import ( + "fmt" + + "github.com/pingcap/tidb/br/pkg/storage" + cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/redo" +) + +const ( + // MinFlushIntervalInMs is the minimum value of flush interval, which is set + // to two seconds to reduce the frequency of accessing external storage. + MinFlushIntervalInMs = 2000 +) + +// ConsistentConfig represents replication consistency config for a changefeed. type ConsistentConfig struct { Level string `toml:"level" json:"level"` MaxLogSize int64 `toml:"max-log-size" json:"max-log-size"` FlushIntervalInMs int64 `toml:"flush-interval" json:"flush-interval"` Storage string `toml:"storage" json:"storage"` } + +// ValidateAndAdjust validates the consistency config and adjusts it if necessary. +func (c *ConsistentConfig) ValidateAndAdjust() error { + if !redo.IsConsistentEnabled(c.Level) { + return nil + } + + if c.FlushIntervalInMs < MinFlushIntervalInMs { + return cerror.ErrInvalidReplicaConfig.FastGenByArgs( + fmt.Sprintf("The consistent.flush-interval:%d must be equal or greater than %d", + c.FlushIntervalInMs, MinFlushIntervalInMs)) + } + + uri, err := storage.ParseRawURL(c.Storage) + if err != nil { + return cerror.ErrInvalidReplicaConfig.GenWithStackByArgs( + fmt.Sprintf("invalid storage uri: %s", c.Storage)) + } + return redo.ValidateStorage(uri) +} diff --git a/pkg/config/replica_config.go b/pkg/config/replica_config.go index 91d042e01a1..9c34562b15b 100644 --- a/pkg/config/replica_config.go +++ b/pkg/config/replica_config.go @@ -62,7 +62,7 @@ var defaultReplicaConfig = &ReplicaConfig{ Consistent: &ConsistentConfig{ Level: "none", MaxLogSize: 64, - FlushIntervalInMs: 2000, + FlushIntervalInMs: MinFlushIntervalInMs, Storage: "", }, } @@ -170,6 +170,13 @@ func (c *ReplicaConfig) ValidateAndAdjust(sinkURI *url.URL) error { return err } } + if c.Consistent != nil { + err := c.Consistent.ValidateAndAdjust() + if err != nil { + return err + } + } + // check sync point config if c.EnableSyncPoint { if c.SyncPointInterval < minSyncPointInterval { diff --git a/pkg/errors/cdc_errors.go b/pkg/errors/cdc_errors.go index 61dcee24df4..2b8aeb6f128 100644 --- a/pkg/errors/cdc_errors.go +++ b/pkg/errors/cdc_errors.go @@ -303,9 +303,9 @@ var ( "s3 storage api", errors.RFCCodeText("CDC:ErrS3StorageAPI"), ) - ErrS3StorageInitialize = errors.Normalize( - "new s3 storage for redo log", - errors.RFCCodeText("CDC:ErrS3StorageInitialize"), + ErrStorageInitialize = errors.Normalize( + "fail to open storage for redo log", + errors.RFCCodeText("CDC:ErrStorageInitialize"), ) ErrCodecInvalidConfig = errors.Normalize( "Codec invalid config", diff --git a/pkg/fsutil/file_allocator.go b/pkg/fsutil/file_allocator.go index afb71041408..acf4f203d95 100644 --- a/pkg/fsutil/file_allocator.go +++ b/pkg/fsutil/file_allocator.go @@ -19,7 +19,7 @@ import ( "path/filepath" "sync" - "github.com/pingcap/tiflow/cdc/redo/common" + "github.com/pingcap/tiflow/pkg/redo" ) // FileAllocator has two functionalities: @@ -84,7 +84,7 @@ func (fl *FileAllocator) alloc() (f *os.File, err error) { } filePath := filepath.Join(fl.dir, fmt.Sprintf("%s_%d.tmp", fl.prefix, fl.count%2)) - f, err = os.OpenFile(filePath, os.O_CREATE|os.O_WRONLY, common.DefaultFileMode) + f, err = os.OpenFile(filePath, os.O_CREATE|os.O_WRONLY, redo.DefaultFileMode) if err != nil { return nil, err } diff --git a/pkg/redo/config.go b/pkg/redo/config.go new file mode 100644 index 00000000000..8016b12f495 --- /dev/null +++ b/pkg/redo/config.go @@ -0,0 +1,260 @@ +// 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 redo + +import ( + "context" + "fmt" + "net/url" + "os" + "path/filepath" + "strings" + "time" + + "github.com/pingcap/tidb/br/pkg/storage" + "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/util" +) + +const ( + // DefaultFileMode is the default mode when operation files + DefaultFileMode = 0o644 + // DefaultDirMode is the default mode when operation dir + DefaultDirMode = 0o755 + + // TmpEXT is the file ext of log file before safely wrote to disk + TmpEXT = ".tmp" + // LogEXT is the file ext of log file after safely wrote to disk + LogEXT = ".log" + // MetaEXT is the meta file ext of meta file after safely wrote to disk + MetaEXT = ".meta" + // MetaTmpEXT is the meta file ext of meta file before safely wrote to disk + MetaTmpEXT = ".mtmp" + // SortLogEXT is the sorted log file ext of log file after safely wrote to disk + SortLogEXT = ".sort" + + // MinSectorSize is minimum sector size used when flushing log so that log can safely + // distinguish between torn writes and ordinary data corruption. + MinSectorSize = 512 +) + +const ( + // RedoMetaFileType is the default file type of meta file + RedoMetaFileType = "meta" + // RedoRowLogFileType is the default file type of row log file + RedoRowLogFileType = "row" + // RedoDDLLogFileType is the default file type of ddl log file + RedoDDLLogFileType = "ddl" +) + +// FileTypeConfig Specifies redo file type config. +type FileTypeConfig struct { + // Whether emitting redo meta or not. + EmitMeta bool + // Whether emitting row events or not. + EmitRowEvents bool + // Whether emitting DDL events or not. + EmitDDLEvents bool +} + +// ConsistentLevelType is the level of redo log consistent level. +type ConsistentLevelType string + +const ( + // ConsistentLevelNone no consistent guarantee. + ConsistentLevelNone ConsistentLevelType = "none" + // ConsistentLevelEventual eventual consistent. + ConsistentLevelEventual ConsistentLevelType = "eventual" +) + +// IsValidConsistentLevel checks whether a given consistent level is valid +func IsValidConsistentLevel(level string) bool { + switch ConsistentLevelType(level) { + case ConsistentLevelNone, ConsistentLevelEventual: + return true + default: + return false + } +} + +// IsConsistentEnabled returns whether the consistent feature is enabled. +func IsConsistentEnabled(level string) bool { + return IsValidConsistentLevel(level) && ConsistentLevelType(level) != ConsistentLevelNone +} + +// ConsistentStorage is the type of consistent storage. +type ConsistentStorage string + +const ( + // consistentStorageBlackhole is a blackhole storage, which will discard all data. + consistentStorageBlackhole ConsistentStorage = "blackhole" + // consistentStorageLocal is a local storage, which will store data in local disk. + consistentStorageLocal ConsistentStorage = "local" + // consistentStorageNFS is a NFS storage, which will store data in NFS. + consistentStorageNFS ConsistentStorage = "nfs" + + // consistentStorageS3 is a S3 storage, which will store data in S3. + consistentStorageS3 ConsistentStorage = "s3" + // consistentStorageGCS is a GCS storage, which will store data in GCS. + consistentStorageGCS ConsistentStorage = "gcs" + // consistentStorageGS is an alias of GCS storage. + consistentStorageGS ConsistentStorage = "gs" + // consistentStorageAzblob is a Azure Blob storage, which will store data in Azure Blob. + consistentStorageAzblob ConsistentStorage = "azblob" + // consistentStorageAzure is an alias of Azure Blob storage. + consistentStorageAzure ConsistentStorage = "azure" + // consistentStorageFile is an external storage based on local files and + // will only be used for testing. + consistentStorageFile ConsistentStorage = "file" + // consistentStorageNoop is a noop storage, which simply discard all data. + consistentStorageNoop ConsistentStorage = "noop" +) + +// IsValidConsistentStorage checks whether a give consistent storage is valid. +func IsValidConsistentStorage(scheme string) bool { + return IsBlackholeStorage(scheme) || + IsLocalStorage(scheme) || + IsExternalStorage(scheme) +} + +// IsExternalStorage returns whether an external storage is used. +func IsExternalStorage(scheme string) bool { + switch ConsistentStorage(scheme) { + case consistentStorageS3, consistentStorageGCS, consistentStorageGS, + consistentStorageAzblob, consistentStorageAzure, consistentStorageFile, + consistentStorageNoop: + return true + default: + return false + } +} + +// IsLocalStorage returns whether a local storage is used. +func IsLocalStorage(scheme string) bool { + switch ConsistentStorage(scheme) { + case consistentStorageLocal, consistentStorageNFS: + return true + default: + return false + } +} + +// IsBlackholeStorage returns whether a blackhole storage is used. +func IsBlackholeStorage(scheme string) bool { + return ConsistentStorage(scheme) == consistentStorageBlackhole +} + +// InitExternalStorage init an external storage. +var InitExternalStorage = func(ctx context.Context, uri url.URL) (storage.ExternalStorage, error) { + if ConsistentStorage(uri.Scheme) == consistentStorageS3 && len(uri.Host) == 0 { + // TODO: this branch is compatible with previous s3 logic and will be removed + // in the future. + return nil, errors.WrapChangefeedUnretryableErr(errors.ErrStorageInitialize, + errors.Errorf("please specify the bucket for %+v", uri)) + } + s, err := util.GetExternalStorage(ctx, uri.String(), nil) + if err != nil { + return nil, errors.WrapChangefeedUnretryableErr(errors.ErrStorageInitialize, err) + } + return s, nil +} + +// ValidateStorage validates the storage used by redo. +func ValidateStorage(uri *url.URL) error { + scheme := uri.Scheme + if !IsValidConsistentStorage(scheme) { + return errors.ErrConsistentStorage.GenWithStackByArgs(scheme) + } + if IsBlackholeStorage(scheme) { + return nil + } + + if IsExternalStorage(scheme) { + ctx, cancel := context.WithTimeout(context.Background(), 3*time.Second) + defer cancel() + _, err := InitExternalStorage(ctx, *uri) + return err + } + + err := os.MkdirAll(uri.Path, DefaultDirMode) + if err != nil { + return errors.WrapError(errors.ErrStorageInitialize, errors.Annotate(err, + fmt.Sprintf("can't make dir for new redo log: %+v", uri))) + } + return nil +} + +const ( + // RedoLogFileFormatV1 was used before v6.1.0, which doesn't contain namespace information + // layout: captureID_changefeedID_fileType_maxEventCommitTs_uuid.fileExtName + RedoLogFileFormatV1 = "%s_%s_%s_%d_%s%s" + // RedoLogFileFormatV2 is available since v6.1.0, which contains namespace information + // layout: captureID_namespace_changefeedID_fileType_maxEventCommitTs_uuid.fileExtName + RedoLogFileFormatV2 = "%s_%s_%s_%s_%d_%s%s" +) + +// logFormat2ParseFormat converts redo log file name format to the space separated +// format, which can be read and parsed by sscanf. Besides remove the suffix `%s` +// which is used as file name extension, since we will parse extension first. +func logFormat2ParseFormat(fmtStr string) string { + return strings.TrimSuffix(strings.ReplaceAll(fmtStr, "_", " "), "%s") +} + +// ParseLogFileName extract the commitTs, fileType from log fileName +func ParseLogFileName(name string) (uint64, string, error) { + ext := filepath.Ext(name) + if ext == MetaEXT { + return 0, RedoMetaFileType, nil + } + + // if .sort, the name should be like + // fmt.Sprintf("%s_%s_%s_%d_%s_%d%s", w.cfg.captureID, + // w.cfg.changeFeedID.Namespace,w.cfg.changeFeedID.ID, + // w.cfg.fileType, w.commitTS.Load(), uuid, LogEXT)+SortLogEXT + if ext == SortLogEXT { + name = strings.TrimSuffix(name, SortLogEXT) + ext = filepath.Ext(name) + } + if ext != LogEXT && ext != TmpEXT { + return 0, "", nil + } + + var commitTs uint64 + var captureID, namespace, changefeedID, fileType, uid string + // if the namespace is not default, the log looks like: + // fmt.Sprintf("%s_%s_%s_%s_%d_%s%s", w.cfg.captureID, + // w.cfg.changeFeedID.Namespace,w.cfg.changeFeedID.ID, + // w.cfg.fileType, w.commitTS.Load(), uuid, redo.LogEXT) + // otherwise it looks like: + // fmt.Sprintf("%s_%s_%s_%d_%s%s", w.cfg.captureID, + // w.cfg.changeFeedID.ID, + // w.cfg.fileType, w.commitTS.Load(), uuid, redo.LogEXT) + var ( + vars []any + formatStr string + ) + if len(strings.Split(name, "_")) == 6 { + formatStr = logFormat2ParseFormat(RedoLogFileFormatV2) + vars = []any{&captureID, &namespace, &changefeedID, &fileType, &commitTs, &uid} + } else { + formatStr = logFormat2ParseFormat(RedoLogFileFormatV1) + vars = []any{&captureID, &changefeedID, &fileType, &commitTs, &uid} + } + name = strings.ReplaceAll(name, "_", " ") + _, err := fmt.Sscanf(name, formatStr, vars...) + if err != nil { + return 0, "", errors.Annotatef(err, "bad log name: %s", name) + } + return commitTs, fileType, nil +} diff --git a/pkg/redo/config_test.go b/pkg/redo/config_test.go new file mode 100644 index 00000000000..c908d7e79fa --- /dev/null +++ b/pkg/redo/config_test.go @@ -0,0 +1,189 @@ +// 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 redo + +import ( + "context" + "fmt" + "testing" + + "github.com/google/uuid" + "github.com/pingcap/tidb/br/pkg/storage" + "github.com/stretchr/testify/require" +) + +func TestParseLogFileName(t *testing.T) { + t.Parallel() + + type arg struct { + name string + } + tests := []struct { + name string + args arg + wantTs uint64 + wantFileType string + wantErr string + }{ + { + name: "happy row .log", + args: arg{ + name: fmt.Sprintf(RedoLogFileFormatV1, "cp", + "test", + RedoRowLogFileType, 1, uuid.NewString(), LogEXT), + }, + wantTs: 1, + wantFileType: RedoRowLogFileType, + }, + { + name: "happy row .log", + args: arg{ + name: fmt.Sprintf(RedoLogFileFormatV2, "cp", + "namespace", "test", + RedoRowLogFileType, 1, uuid.NewString(), LogEXT), + }, + wantTs: 1, + wantFileType: RedoRowLogFileType, + }, + { + name: "happy row .tmp", + args: arg{ + name: fmt.Sprintf(RedoLogFileFormatV1, "cp", + "test", + RedoRowLogFileType, 1, uuid.NewString(), LogEXT) + TmpEXT, + }, + wantTs: 1, + wantFileType: RedoRowLogFileType, + }, + { + name: "happy row .tmp", + args: arg{ + name: fmt.Sprintf(RedoLogFileFormatV2, "cp", + "namespace", "test", + RedoRowLogFileType, 1, uuid.NewString(), LogEXT) + TmpEXT, + }, + wantTs: 1, + wantFileType: RedoRowLogFileType, + }, + { + name: "happy ddl .log", + args: arg{ + name: fmt.Sprintf(RedoLogFileFormatV1, "cp", + "test", + RedoDDLLogFileType, 1, uuid.NewString(), LogEXT), + }, + wantTs: 1, + wantFileType: RedoDDLLogFileType, + }, + { + name: "happy ddl .log", + args: arg{ + name: fmt.Sprintf(RedoLogFileFormatV2, "cp", + "namespace", "test", + RedoDDLLogFileType, 1, uuid.NewString(), LogEXT), + }, + wantTs: 1, + wantFileType: RedoDDLLogFileType, + }, + { + name: "happy ddl .sort", + args: arg{ + name: fmt.Sprintf(RedoLogFileFormatV2, "cp", + "default", "test", + RedoDDLLogFileType, 1, uuid.NewString(), LogEXT) + SortLogEXT, + }, + wantTs: 1, + wantFileType: RedoDDLLogFileType, + }, + { + name: "happy ddl .sort", + args: arg{ + name: fmt.Sprintf(RedoLogFileFormatV2, "cp", + "namespace", "test", + RedoDDLLogFileType, 1, uuid.NewString(), LogEXT) + SortLogEXT, + }, + wantTs: 1, + wantFileType: RedoDDLLogFileType, + }, + { + name: "happy ddl .tmp", + args: arg{ + name: fmt.Sprintf(RedoLogFileFormatV1, "cp", + "test", + RedoDDLLogFileType, 1, uuid.NewString(), LogEXT) + TmpEXT, + }, + wantTs: 1, + wantFileType: RedoDDLLogFileType, + }, + { + name: "happy ddl .tmp", + args: arg{ + name: fmt.Sprintf(RedoLogFileFormatV2, "cp", + "namespace", "test", + RedoDDLLogFileType, 1, uuid.NewString(), LogEXT) + TmpEXT, + }, + wantTs: 1, + wantFileType: RedoDDLLogFileType, + }, + { + name: "happy .meta", + args: arg{ + name: "sdfsdfsf" + MetaEXT, + }, + wantTs: 0, + wantFileType: RedoMetaFileType, + }, + { + name: "not supported fileType", + args: arg{ + name: "sdfsdfsf.sfsf", + }, + }, + { + name: "err wrong format ddl .tmp", + args: arg{ + name: fmt.Sprintf("%s_%s_%s_%s_%d%s%s", /* a wrong format */ + "cp", "default", "test", + RedoDDLLogFileType, 1, uuid.NewString(), LogEXT) + TmpEXT, + }, + wantErr: ".*bad log name*.", + }, + } + for _, tt := range tests { + ts, fileType, err := ParseLogFileName(tt.args.name) + if tt.wantErr != "" { + require.Regexp(t, tt.wantErr, err, tt.name) + } else { + require.Nil(t, err, tt.name) + require.EqualValues(t, tt.wantTs, ts, tt.name) + require.Equal(t, tt.wantFileType, fileType, tt.name) + } + } +} + +func TestInitExternalStorage(t *testing.T) { + t.Parallel() + + dir := t.TempDir() + urls := []string{ + fmt.Sprintf("file://%s/test", dir), + } + + for _, urlStr := range urls { + url, err := storage.ParseRawURL(urlStr) + require.NoError(t, err) + _, err = InitExternalStorage(context.Background(), *url) + require.NoError(t, err) + } +} diff --git a/pkg/util/external_storage.go b/pkg/util/external_storage.go new file mode 100644 index 00000000000..90359c00cce --- /dev/null +++ b/pkg/util/external_storage.go @@ -0,0 +1,92 @@ +// 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 util + +import ( + "context" + "strings" + "time" + + "github.com/aws/aws-sdk-go/aws/client" + "github.com/aws/aws-sdk-go/aws/request" + "github.com/pingcap/log" + "github.com/pingcap/tidb/br/pkg/storage" + "github.com/pingcap/tiflow/pkg/errors" + "go.uber.org/zap" +) + +// GetExternalStorageFromURI creates a new storage.ExternalStorage from a uri. +func GetExternalStorageFromURI( + ctx context.Context, uri string, +) (storage.ExternalStorage, error) { + return GetExternalStorage(ctx, uri, nil) +} + +// GetExternalStorage creates a new storage.ExternalStorage based on the uri and options. +func GetExternalStorage( + ctx context.Context, uri string, opts *storage.BackendOptions, +) (storage.ExternalStorage, error) { + backEnd, err := storage.ParseBackend(uri, opts) + if err != nil { + return nil, errors.Trace(err) + } + + ret, err := storage.New(ctx, backEnd, &storage.ExternalStorageOptions{ + SendCredentials: false, + S3Retryer: DefaultS3Retryer(), + }) + if err != nil { + retErr := errors.ErrFailToCreateExternalStorage.Wrap(errors.Trace(err)) + return nil, retErr.GenWithStackByArgs("creating ExternalStorage for s3") + } + return ret, nil +} + +// retryerWithLog wraps the client.DefaultRetryer, and logs when retrying. +type retryerWithLog struct { + client.DefaultRetryer +} + +func isDeadlineExceedError(err error) bool { + return strings.Contains(err.Error(), "context deadline exceeded") +} + +func (rl retryerWithLog) ShouldRetry(r *request.Request) bool { + if isDeadlineExceedError(r.Error) { + return false + } + return rl.DefaultRetryer.ShouldRetry(r) +} + +func (rl retryerWithLog) RetryRules(r *request.Request) time.Duration { + backoffTime := rl.DefaultRetryer.RetryRules(r) + if backoffTime > 0 { + log.Warn("failed to request s3, retrying", + zap.Error(r.Error), + zap.Duration("backoff", backoffTime)) + } + return backoffTime +} + +// DefaultS3Retryer is the default s3 retryer, maybe this function +// should be extracted to another place. +func DefaultS3Retryer() request.Retryer { + return retryerWithLog{ + DefaultRetryer: client.DefaultRetryer{ + NumMaxRetries: 3, + MinRetryDelay: 1 * time.Second, + MinThrottleDelay: 2 * time.Second, + }, + } +} diff --git a/scripts/check-copyright.sh b/scripts/check-copyright.sh index 468fd02c4b5..fb6d3d111d8 100755 --- a/scripts/check-copyright.sh +++ b/scripts/check-copyright.sh @@ -1,6 +1,6 @@ #!/bin/bash -result=$(find ./ -name "*.go" | grep -vE '\.pb\.go|vendor/|leaktest.go|kv_gen|redo_gen|sink_gen|pbmock|\.pb\.gw\.go|statik.go|openapi/gen\..*\.go|_mock\.go|embedded_asserts.go|empty_asserts.go|docs/swagger|bin|owner/mock' | +result=$(find ./ -name "*.go" | grep -vE '\.pb\.go|vendor/|leaktest.go|kv_gen|redo_meta_gen|sink_gen|pbmock|\.pb\.gw\.go|statik.go|openapi/gen\..*\.go|_mock\.go|embedded_asserts.go|empty_asserts.go|docs/swagger|bin|owner/mock' | while read -r file_path; do head=$(head -n 1 "$file_path") if [[ "$head" == *"Code generated by"* ]]; then diff --git a/tests/integration_tests/_utils/check_redo_resolved_ts b/tests/integration_tests/_utils/check_redo_resolved_ts new file mode 100755 index 00000000000..800644d58b6 --- /dev/null +++ b/tests/integration_tests/_utils/check_redo_resolved_ts @@ -0,0 +1,17 @@ +#!/bin/bash +# parameter 1: changefeed id +# parameter 2: check tso +# parameter 3: external storage path +# parameter 4: temporary download path + +changefeedid=$1 +check_tso=$2 +storage_path=$3 +read_dir=$4 + +# check resolved ts has been persisted in redo log meta +rts=$(cdc redo meta --storage="$storage_path" --tmp-dir="$read_dir" | grep -oE "resolved-ts:[0-9]+" | awk -F: '{print $2}') +if [[ "$rts" -le "$check_tso" ]]; then + echo "global resolved ts $rts not forward to $check_tso" + exit 1 +fi diff --git a/tests/integration_tests/consistent_replicate_gbk/run.sh b/tests/integration_tests/consistent_replicate_gbk/run.sh index d50b0a09dd2..ec2bb63e829 100644 --- a/tests/integration_tests/consistent_replicate_gbk/run.sh +++ b/tests/integration_tests/consistent_replicate_gbk/run.sh @@ -41,23 +41,6 @@ stop() { s3cmd --access_key=$MINIO_ACCESS_KEY --secret_key=$MINIO_SECRET_KEY --host=$S3_ENDPOINT --host-bucket=$S3_ENDPOINT --no-ssl mb s3://logbucket -# check resolved ts has been persisted in redo log meta -function check_resolved_ts() { - export AWS_ACCESS_KEY_ID=$MINIO_ACCESS_KEY - export AWS_SECRET_ACCESS_KEY=$MINIO_SECRET_KEY - changefeedid=$1 - check_tso=$2 - read_dir=$3 - rts=$(cdc redo meta --storage="s3://logbucket/test-changefeed?endpoint=http://127.0.0.1:24927/" --tmp-dir="$read_dir" | grep -oE "resolved-ts:[0-9]+" | awk -F: '{print $2}') - if [[ "$rts" -gt "$check_tso" ]]; then - return - fi - echo "global resolved ts $rts not forward to $check_tso" - exit 1 -} - -export -f check_resolved_ts - function run() { # we only support eventually consistent replication with MySQL sink if [ "$SINK_TYPE" == "kafka" ]; then @@ -92,15 +75,17 @@ function run() { # to ensure row changed events have been replicated to TiCDC sleep 5 + storage_path="s3://logbucket/test-changefeed?endpoint=http://127.0.0.1:24927/" + tmp_download_path=$WORK_DIR/cdc_data/redo/$changefeed_id current_tso=$(cdc cli tso query --pd=http://$UP_PD_HOST_1:$UP_PD_PORT_1) - ensure 20 check_resolved_ts $changefeed_id $current_tso $WORK_DIR/redo/meta + ensure 20 check_redo_resolved_ts $changefeed_id $current_tso $storage_path $tmp_download_path/meta cleanup_process $CDC_BINARY export GO_FAILPOINTS='' export AWS_ACCESS_KEY_ID=$MINIO_ACCESS_KEY export AWS_SECRET_ACCESS_KEY=$MINIO_SECRET_KEY - cdc redo apply --tmp-dir="$WORK_DIR/redo/apply" \ - --storage="s3://logbucket/test-changefeed?endpoint=http://127.0.0.1:24927/" \ + cdc redo apply --tmp-dir="$WORK_DIR/apply" \ + --storage="$storage_path" \ --sink-uri="mysql://normal:123456@127.0.0.1:3306/" check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml } diff --git a/tests/integration_tests/consistent_replicate_nfs/run.sh b/tests/integration_tests/consistent_replicate_nfs/run.sh index 56d398329a2..39284dc32e8 100644 --- a/tests/integration_tests/consistent_replicate_nfs/run.sh +++ b/tests/integration_tests/consistent_replicate_nfs/run.sh @@ -14,21 +14,6 @@ stop() { stop_tidb_cluster } -# check resolved ts has been persisted in redo log meta -function check_resolved_ts() { - changefeedid=$1 - check_tso=$2 - redo_dir=$3 - rts=$(cdc redo meta --storage="nfs:///tmp/tidb_cdc_test/consistent_replicate_nfs/nfs/redo" --tmp-dir="$redo_dir" | grep -oE "resolved-ts:[0-9]+" | awk -F: '{print $2}') - if [[ "$rts" -gt "$check_tso" ]]; then - return - fi - echo "global resolved ts $rts not forward to $check_tso" - exit 1 -} - -export -f check_resolved_ts - function run() { # we only support eventually consistent replication with MySQL sink if [ "$SINK_TYPE" == "kafka" ]; then @@ -64,13 +49,14 @@ function run() { # to ensure row changed events have been replicated to TiCDC sleep 10 - nfs_download_path=$WORK_DIR/cdc_data/redo/$changefeed_id + storage_path="nfs://$WORK_DIR/nfs/redo" + tmp_download_path=$WORK_DIR/cdc_data/redo/$changefeed_id current_tso=$(cdc cli tso query --pd=http://$UP_PD_HOST_1:$UP_PD_PORT_1) - ensure 50 check_resolved_ts $changefeed_id $current_tso $nfs_download_path + ensure 50 check_redo_resolved_ts $changefeed_id $current_tso $storage_path $tmp_download_path/meta cleanup_process $CDC_BINARY export GO_FAILPOINTS='' - cdc redo apply --tmp-dir="$nfs_download_path" --storage="nfs://$WORK_DIR/nfs/redo" --sink-uri="mysql://normal:123456@127.0.0.1:3306/" + cdc redo apply --tmp-dir="$tmp_download_path/apply" --storage="$storage_path" --sink-uri="mysql://normal:123456@127.0.0.1:3306/" check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml } diff --git a/tests/integration_tests/consistent_replicate_storage_file/conf/changefeed.toml b/tests/integration_tests/consistent_replicate_storage_file/conf/changefeed.toml new file mode 100644 index 00000000000..dadf83cf9fd --- /dev/null +++ b/tests/integration_tests/consistent_replicate_storage_file/conf/changefeed.toml @@ -0,0 +1,3 @@ +[consistent] +level = "eventual" +storage = "file:///tmp/tidb_cdc_test/consistent_replicate_storage_file/redo" diff --git a/tests/integration_tests/consistent_replicate_storage_file/conf/diff_config.toml b/tests/integration_tests/consistent_replicate_storage_file/conf/diff_config.toml new file mode 100644 index 00000000000..ab502d7ea53 --- /dev/null +++ b/tests/integration_tests/consistent_replicate_storage_file/conf/diff_config.toml @@ -0,0 +1,29 @@ +# diff Configuration. + +check-thread-count = 4 + +export-fix-sql = true + +check-struct-only = false + +[task] + output-dir = "/tmp/tidb_cdc_test/consistent_replicate_storage_file/sync_diff/output" + + source-instances = ["mysql1"] + + target-instance = "tidb0" + + target-check-tables = ["consistent_replicate_storage_file.usertable*","consistent_replicate_storage_file.t*"] + +[data-sources] +[data-sources.mysql1] + host = "127.0.0.1" + port = 4000 + user = "root" + password = "" + +[data-sources.tidb0] + host = "127.0.0.1" + port = 3306 + user = "root" + password = "" diff --git a/tests/integration_tests/consistent_replicate_s3/conf/workload b/tests/integration_tests/consistent_replicate_storage_file/conf/workload similarity index 100% rename from tests/integration_tests/consistent_replicate_s3/conf/workload rename to tests/integration_tests/consistent_replicate_storage_file/conf/workload diff --git a/tests/integration_tests/consistent_replicate_storage_file/data/prepare.sql b/tests/integration_tests/consistent_replicate_storage_file/data/prepare.sql new file mode 100644 index 00000000000..a1b9a336ebd --- /dev/null +++ b/tests/integration_tests/consistent_replicate_storage_file/data/prepare.sql @@ -0,0 +1,13 @@ +use `consistent_replicate_storage_file`; +set @@global.tidb_enable_exchange_partition=on; + +create table t1 (a int primary key) PARTITION BY RANGE ( a ) ( PARTITION p0 VALUES LESS THAN (6),PARTITION p1 VALUES LESS THAN (11),PARTITION p2 VALUES LESS THAN (21)); +insert into t1 values (1),(2),(3),(4),(5),(6); +insert into t1 values (7),(8),(9); +insert into t1 values (11),(12),(20); +alter table t1 add partition (partition p3 values less than (30), partition p4 values less than (40)); +insert into t1 values (25),(29),(35); /*these values in p3,p4*/ + +create table t2 (a int primary key); + + diff --git a/tests/integration_tests/consistent_replicate_storage_file/run.sh b/tests/integration_tests/consistent_replicate_storage_file/run.sh new file mode 100644 index 00000000000..e8b8ed84f92 --- /dev/null +++ b/tests/integration_tests/consistent_replicate_storage_file/run.sh @@ -0,0 +1,84 @@ +#!/bin/bash + +set -eu + +CUR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +source $CUR/../_utils/test_prepare +WORK_DIR=$OUT_DIR/$TEST_NAME +CDC_BINARY=cdc.test +SINK_TYPE=$1 + +rm -rf "$WORK_DIR" +mkdir -p "$WORK_DIR" + +stop() { + # to distinguish whether the test failed in the DML synchronization phase or the DDL synchronization phase + echo $(mysql -h${DOWN_TIDB_HOST} -P${DOWN_TIDB_PORT} -uroot -e "select count(*) from consistent_replicate_storage_file.USERTABLE;") + stop_tidb_cluster +} + +function run() { + # we only support eventually consistent replication with MySQL sink + if [ "$SINK_TYPE" == "kafka" ]; then + return + fi + + start_tidb_cluster --workdir $WORK_DIR + + cd $WORK_DIR + run_sql "set @@global.tidb_enable_exchange_partition=on" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + + run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY + + SINK_URI="mysql://normal:123456@127.0.0.1:3306/" + changefeed_id=$(cdc cli changefeed create --sink-uri="$SINK_URI" --config="$CUR/conf/changefeed.toml" 2>&1 | tail -n2 | head -n1 | awk '{print $2}') + + run_sql "CREATE DATABASE consistent_replicate_storage_file;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + go-ycsb load mysql -P $CUR/conf/workload -p mysql.host=${UP_TIDB_HOST} -p mysql.port=${UP_TIDB_PORT} -p mysql.user=root -p mysql.db=consistent_replicate_storage_file + run_sql "CREATE table consistent_replicate_storage_file.check1(id int primary key);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql_file $CUR/data/prepare.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} + check_table_exists "consistent_replicate_storage_file.USERTABLE" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists "consistent_replicate_storage_file.check1" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 120 + check_table_exists "consistent_replicate_storage_file.t2" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 120 + + check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml + + # Inject the failpoint to prevent sink execution, but the global resolved can be moved forward. + # Then we can apply redo log to reach an eventual consistent state in downstream. + cleanup_process $CDC_BINARY + export GO_FAILPOINTS='github.com/pingcap/tiflow/cdc/sinkv2/eventsink/txn/mysql/MySQLSinkHangLongTime=return(true)' + run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY + run_sql "create table consistent_replicate_storage_file.USERTABLE2 like consistent_replicate_storage_file.USERTABLE" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "ALTER TABLE consistent_replicate_storage_file.t1 EXCHANGE PARTITION p3 WITH TABLE consistent_replicate_storage_file.t2" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "insert into consistent_replicate_storage_file.t2 values (100),(101),(102),(103),(104),(105);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "insert into consistent_replicate_storage_file.t1 values (25),(29);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "insert into consistent_replicate_storage_file.USERTABLE2 select * from consistent_replicate_storage_file.USERTABLE" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + + # to ensure row changed events have been replicated to TiCDC + sleep 20 + + storage_path="file://$WORK_DIR/redo" + tmp_download_path=$WORK_DIR/cdc_data/redo/$changefeed_id + current_tso=$(cdc cli tso query --pd=http://$UP_PD_HOST_1:$UP_PD_PORT_1) + ensure 50 check_redo_resolved_ts $changefeed_id $current_tso $storage_path $tmp_download_path/meta + cleanup_process $CDC_BINARY + + export GO_FAILPOINTS='' + + # This value is generated by: + # echo -n '123456' | base64 + # MTIzNDU2 + # Use this value here to test redo apply function works well + # when use base64 encoded password + ENPASSWORD="MTIzNDU2" + + cdc redo apply --tmp-dir="$tmp_download_path/apply" \ + --storage="$storage_path" \ + --sink-uri="mysql://normal:${ENPASSWORD}@127.0.0.1:3306/" + check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml +} + +trap stop EXIT +run $* +check_logs $WORK_DIR +echo "[$(date)] <<<<<< run test case $TEST_NAME success! >>>>>>" diff --git a/tests/integration_tests/consistent_replicate_s3/conf/changefeed.toml b/tests/integration_tests/consistent_replicate_storage_s3/conf/changefeed.toml similarity index 100% rename from tests/integration_tests/consistent_replicate_s3/conf/changefeed.toml rename to tests/integration_tests/consistent_replicate_storage_s3/conf/changefeed.toml diff --git a/tests/integration_tests/consistent_replicate_s3/conf/diff_config.toml b/tests/integration_tests/consistent_replicate_storage_s3/conf/diff_config.toml similarity index 65% rename from tests/integration_tests/consistent_replicate_s3/conf/diff_config.toml rename to tests/integration_tests/consistent_replicate_storage_s3/conf/diff_config.toml index e9fc089d25a..c6f6f934957 100644 --- a/tests/integration_tests/consistent_replicate_s3/conf/diff_config.toml +++ b/tests/integration_tests/consistent_replicate_storage_s3/conf/diff_config.toml @@ -7,13 +7,13 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/tidb_cdc_test/consistent_replicate_s3/sync_diff/output" + output-dir = "/tmp/tidb_cdc_test/consistent_replicate_storage_s3/sync_diff/output" source-instances = ["mysql1"] target-instance = "tidb0" - target-check-tables = ["consistent_replicate_s3.usertable*","consistent_replicate_s3.t*"] + target-check-tables = ["consistent_replicate_storage_s3.usertable*","consistent_replicate_storage_s3.t*"] [data-sources] [data-sources.mysql1] diff --git a/tests/integration_tests/consistent_replicate_storage_s3/conf/workload b/tests/integration_tests/consistent_replicate_storage_s3/conf/workload new file mode 100644 index 00000000000..7a3fd48df6f --- /dev/null +++ b/tests/integration_tests/consistent_replicate_storage_s3/conf/workload @@ -0,0 +1,14 @@ +threadcount=10 +recordcount=5000 +operationcount=0 +workload=core +fieldcount=100 + +readallfields=true + +readproportion=0 +updateproportion=0 +scanproportion=0 +insertproportion=0 + +requestdistribution=uniform diff --git a/tests/integration_tests/consistent_replicate_s3/data/prepare.sql b/tests/integration_tests/consistent_replicate_storage_s3/data/prepare.sql similarity index 93% rename from tests/integration_tests/consistent_replicate_s3/data/prepare.sql rename to tests/integration_tests/consistent_replicate_storage_s3/data/prepare.sql index 67ecd05966d..4d0acf98cff 100644 --- a/tests/integration_tests/consistent_replicate_s3/data/prepare.sql +++ b/tests/integration_tests/consistent_replicate_storage_s3/data/prepare.sql @@ -1,4 +1,4 @@ -use `consistent_replicate_s3`; +use `consistent_replicate_storage_s3`; set @@global.tidb_enable_exchange_partition=on; create table t1 (a int primary key) PARTITION BY RANGE ( a ) ( PARTITION p0 VALUES LESS THAN (6),PARTITION p1 VALUES LESS THAN (11),PARTITION p2 VALUES LESS THAN (21)); diff --git a/tests/integration_tests/consistent_replicate_s3/run.sh b/tests/integration_tests/consistent_replicate_storage_s3/run.sh similarity index 60% rename from tests/integration_tests/consistent_replicate_s3/run.sh rename to tests/integration_tests/consistent_replicate_storage_s3/run.sh index f94f07bdb80..dfca5cb5f69 100644 --- a/tests/integration_tests/consistent_replicate_s3/run.sh +++ b/tests/integration_tests/consistent_replicate_storage_s3/run.sh @@ -36,30 +36,13 @@ stop_minio() { stop() { # to distinguish whether the test failed in the DML synchronization phase or the DDL synchronization phase - echo $(mysql -h${DOWN_TIDB_HOST} -P${DOWN_TIDB_PORT} -uroot -e "select count(*) from consistent_replicate_s3.USERTABLE;") + echo $(mysql -h${DOWN_TIDB_HOST} -P${DOWN_TIDB_PORT} -uroot -e "select count(*) from consistent_replicate_storage_s3.USERTABLE;") stop_minio stop_tidb_cluster } s3cmd --access_key=$MINIO_ACCESS_KEY --secret_key=$MINIO_SECRET_KEY --host=$S3_ENDPOINT --host-bucket=$S3_ENDPOINT --no-ssl mb s3://logbucket -# check resolved ts has been persisted in redo log meta -function check_resolved_ts() { - export AWS_ACCESS_KEY_ID=$MINIO_ACCESS_KEY - export AWS_SECRET_ACCESS_KEY=$MINIO_SECRET_KEY - changefeedid=$1 - check_tso=$2 - read_dir=$3 - rts=$(cdc redo meta --storage="s3://logbucket/test-changefeed?endpoint=http://127.0.0.1:24927/" --tmp-dir="$read_dir" | grep -oE "resolved-ts:[0-9]+" | awk -F: '{print $2}') - if [[ "$rts" -gt "$check_tso" ]]; then - return - fi - echo "global resolved ts $rts not forward to $check_tso" - exit 1 -} - -export -f check_resolved_ts - function run() { # we only support eventually consistent replication with MySQL sink if [ "$SINK_TYPE" == "kafka" ]; then @@ -76,13 +59,13 @@ function run() { SINK_URI="mysql://normal:123456@127.0.0.1:3306/" changefeed_id=$(cdc cli changefeed create --sink-uri="$SINK_URI" --config="$CUR/conf/changefeed.toml" 2>&1 | tail -n2 | head -n1 | awk '{print $2}') - run_sql "CREATE DATABASE consistent_replicate_s3;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - go-ycsb load mysql -P $CUR/conf/workload -p mysql.host=${UP_TIDB_HOST} -p mysql.port=${UP_TIDB_PORT} -p mysql.user=root -p mysql.db=consistent_replicate_s3 - run_sql "CREATE table consistent_replicate_s3.check1(id int primary key);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "CREATE DATABASE consistent_replicate_storage_s3;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + go-ycsb load mysql -P $CUR/conf/workload -p mysql.host=${UP_TIDB_HOST} -p mysql.port=${UP_TIDB_PORT} -p mysql.user=root -p mysql.db=consistent_replicate_storage_s3 + run_sql "CREATE table consistent_replicate_storage_s3.check1(id int primary key);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} run_sql_file $CUR/data/prepare.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} - check_table_exists "consistent_replicate_s3.USERTABLE" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} - check_table_exists "consistent_replicate_s3.check1" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 120 - check_table_exists "consistent_replicate_s3.t2" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 120 + check_table_exists "consistent_replicate_storage_s3.USERTABLE" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists "consistent_replicate_storage_s3.check1" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 120 + check_table_exists "consistent_replicate_storage_s3.t2" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 120 check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml @@ -91,17 +74,19 @@ function run() { cleanup_process $CDC_BINARY export GO_FAILPOINTS='github.com/pingcap/tiflow/cdc/sinkv2/eventsink/txn/mysql/MySQLSinkHangLongTime=return(true)' run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY - run_sql "create table consistent_replicate_s3.USERTABLE2 like consistent_replicate_s3.USERTABLE" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - run_sql "ALTER TABLE consistent_replicate_s3.t1 EXCHANGE PARTITION p3 WITH TABLE consistent_replicate_s3.t2" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - run_sql "insert into consistent_replicate_s3.t2 values (100),(101),(102),(103),(104),(105);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - run_sql "insert into consistent_replicate_s3.t1 values (25),(29);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - run_sql "insert into consistent_replicate_s3.USERTABLE2 select * from consistent_replicate_s3.USERTABLE" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "create table consistent_replicate_storage_s3.USERTABLE2 like consistent_replicate_storage_s3.USERTABLE" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "ALTER TABLE consistent_replicate_storage_s3.t1 EXCHANGE PARTITION p3 WITH TABLE consistent_replicate_storage_s3.t2" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "insert into consistent_replicate_storage_s3.t2 values (100),(101),(102),(103),(104),(105);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "insert into consistent_replicate_storage_s3.t1 values (25),(29);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "insert into consistent_replicate_storage_s3.USERTABLE2 select * from consistent_replicate_storage_s3.USERTABLE" ${UP_TIDB_HOST} ${UP_TIDB_PORT} # to ensure row changed events have been replicated to TiCDC sleep 20 + storage_path="s3://logbucket/test-changefeed?endpoint=http://127.0.0.1:24927/" + tmp_download_path=$WORK_DIR/cdc_data/redo/$changefeed_id current_tso=$(cdc cli tso query --pd=http://$UP_PD_HOST_1:$UP_PD_PORT_1) - ensure 50 check_resolved_ts $changefeed_id $current_tso $WORK_DIR/redo/meta + ensure 50 check_redo_resolved_ts $changefeed_id $current_tso $storage_path $tmp_download_path/meta cleanup_process $CDC_BINARY export GO_FAILPOINTS='' @@ -115,8 +100,8 @@ function run() { # when use base64 encoded password ENPASSWORD="MTIzNDU2" - cdc redo apply --tmp-dir="$WORK_DIR/redo/apply" \ - --storage="s3://logbucket/test-changefeed?endpoint=http://127.0.0.1:24927/" \ + cdc redo apply --tmp-dir="$tmp_download_path/apply" \ + --storage="$storage_path" \ --sink-uri="mysql://normal:${ENPASSWORD}@127.0.0.1:3306/" check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml }