diff --git a/downstreamadapter/sink/cloudstorage.go b/downstreamadapter/sink/cloudstorage.go new file mode 100644 index 00000000..f4429eec --- /dev/null +++ b/downstreamadapter/sink/cloudstorage.go @@ -0,0 +1,198 @@ +// Copyright 2025 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package sink + +import ( + "context" + "math" + "net/url" + "strings" + "sync/atomic" + + "github.com/pingcap/errors" + "github.com/pingcap/log" + "github.com/pingcap/ticdc/downstreamadapter/sink/helper" + "github.com/pingcap/ticdc/downstreamadapter/worker" + "github.com/pingcap/ticdc/pkg/common" + commonType "github.com/pingcap/ticdc/pkg/common" + commonEvent "github.com/pingcap/ticdc/pkg/common/event" + "github.com/pingcap/ticdc/pkg/config" + "github.com/pingcap/ticdc/pkg/metrics" + "github.com/pingcap/ticdc/pkg/sink/cloudstorage" + "github.com/pingcap/ticdc/pkg/sink/util" + "github.com/pingcap/tidb/br/pkg/storage" + putil "github.com/pingcap/tiflow/pkg/util" + "go.uber.org/zap" + "golang.org/x/sync/errgroup" +) + +// It will send the events to cloud storage systems. +// Messages are encoded in the specific protocol and then sent to the defragmenter. +// The data flow is as follows: **data** -> encodingWorkers -> defragmenter -> dmlWorkers -> external storage +// The defragmenter will defragment the out-of-order encoded messages and sends encoded +// messages to individual dmlWorkers. +// The dmlWorkers will write the encoded messages to external storage in parallel between different tables. +type CloudStorageSink struct { + changefeedID commonType.ChangeFeedID + scheme string + outputRawChangeEvent bool + + // workers defines a group of workers for writing events to external storage. + dmlWorker *worker.CloudStorageDMLWorker + ddlWorker *worker.CloudStorageDDLWorker + + statistics *metrics.Statistics + + isNormal uint32 +} + +func verifyCloudStorageSink(ctx context.Context, changefeedID common.ChangeFeedID, sinkURI *url.URL, sinkConfig *config.SinkConfig) error { + var ( + protocol config.Protocol + storage storage.ExternalStorage + err error + ) + cfg := cloudstorage.NewConfig() + if err = cfg.Apply(ctx, sinkURI, sinkConfig); err != nil { + return err + } + if protocol, err = helper.GetProtocol(putil.GetOrZero(sinkConfig.Protocol)); err != nil { + return err + } + if _, err = util.GetEncoderConfig(changefeedID, sinkURI, protocol, sinkConfig, math.MaxInt); err != nil { + return err + } + if storage, err = helper.GetExternalStorageFromURI(ctx, sinkURI.String()); err != nil { + return err + } + storage.Close() + return nil +} + +func newCloudStorageSink( + ctx context.Context, changefeedID common.ChangeFeedID, sinkURI *url.URL, sinkConfig *config.SinkConfig, + cleanupJobs []func(), /* only for test */ +) (*CloudStorageSink, error) { + // create cloud storage config and then apply the params of sinkURI to it. + cfg := cloudstorage.NewConfig() + err := cfg.Apply(ctx, sinkURI, sinkConfig) + if err != nil { + return nil, err + } + // fetch protocol from replicaConfig defined by changefeed config file. + protocol, err := helper.GetProtocol( + putil.GetOrZero(sinkConfig.Protocol), + ) + if err != nil { + return nil, errors.Trace(err) + } + // get cloud storage file extension according to the specific protocol. + ext := helper.GetFileExtension(protocol) + // the last param maxMsgBytes is mainly to limit the size of a single message for + // batch protocols in mq scenario. In cloud storage sink, we just set it to max int. + encoderConfig, err := util.GetEncoderConfig(changefeedID, sinkURI, protocol, sinkConfig, math.MaxInt) + if err != nil { + return nil, errors.Trace(err) + } + storage, err := helper.GetExternalStorageFromURI(ctx, sinkURI.String()) + if err != nil { + return nil, err + } + s := &CloudStorageSink{ + changefeedID: changefeedID, + scheme: strings.ToLower(sinkURI.Scheme), + outputRawChangeEvent: sinkConfig.CloudStorageConfig.GetOutputRawChangeEvent(), + statistics: metrics.NewStatistics(changefeedID, "CloudStorageSink"), + } + + s.dmlWorker, err = worker.NewCloudStorageDMLWorker(changefeedID, storage, cfg, encoderConfig, ext, s.statistics) + if err != nil { + return nil, err + } + s.ddlWorker = worker.NewCloudStorageDDLWorker(changefeedID, sinkURI, cfg, cleanupJobs, storage, s.statistics) + return s, nil +} + +func (s *CloudStorageSink) SinkType() common.SinkType { + return common.CloudStorageSinkType +} + +func (s *CloudStorageSink) Run(ctx context.Context) error { + eg, ctx := errgroup.WithContext(ctx) + + eg.Go(func() error { + return s.dmlWorker.Run(ctx) + }) + + eg.Go(func() error { + return s.ddlWorker.Run(ctx) + }) + + return eg.Wait() +} + +func (s *CloudStorageSink) IsNormal() bool { + return atomic.LoadUint32(&s.isNormal) == 1 +} + +func (s *CloudStorageSink) AddDMLEvent(event *commonEvent.DMLEvent) { + s.dmlWorker.AddDMLEvent(event) +} + +func (s *CloudStorageSink) PassBlockEvent(event commonEvent.BlockEvent) { + event.PostFlush() +} + +func (s *CloudStorageSink) WriteBlockEvent(event commonEvent.BlockEvent) error { + switch e := event.(type) { + case *commonEvent.DDLEvent: + if e.TiDBOnly { + // run callback directly and return + e.PostFlush() + return nil + } + err := s.ddlWorker.WriteBlockEvent(e) + if err != nil { + atomic.StoreUint32(&s.isNormal, 0) + return errors.Trace(err) + } + case *commonEvent.SyncPointEvent: + log.Error("CloudStorageSink doesn't support Sync Point Event", + zap.String("namespace", s.changefeedID.Namespace()), + zap.String("changefeed", s.changefeedID.Name()), + zap.Any("event", event)) + default: + log.Error("CloudStorageSink doesn't support this type of block event", + zap.String("namespace", s.changefeedID.Namespace()), + zap.String("changefeed", s.changefeedID.Name()), + zap.Any("eventType", event.GetType())) + } + return nil +} + +func (s *CloudStorageSink) AddCheckpointTs(ts uint64) { + s.ddlWorker.AddCheckpointTs(ts) +} + +func (s *CloudStorageSink) SetTableSchemaStore(tableSchemaStore *util.TableSchemaStore) { + s.ddlWorker.SetTableSchemaStore(tableSchemaStore) +} + +func (s *CloudStorageSink) Close(_ bool) { + s.dmlWorker.Close() + s.ddlWorker.Close() + if s.statistics != nil { + s.statistics.Close() + } +} diff --git a/downstreamadapter/sink/cloudstorage_test.go b/downstreamadapter/sink/cloudstorage_test.go new file mode 100644 index 00000000..5c5aadf9 --- /dev/null +++ b/downstreamadapter/sink/cloudstorage_test.go @@ -0,0 +1,33 @@ +// Copyright 2025 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package sink + +import ( + "context" + + "github.com/pingcap/ticdc/pkg/common" +) + +func newCloudStorageSinkForTest() (*CloudStorageSink, error) { + ctx := context.Background() + changefeedID := common.NewChangefeedID4Test("test", "test") + // csvProtocol := "csv-protocol" + // sinkConfig := &config.SinkConfig{Protocol: &csvProtocol} + + sink := &CloudStorageSink{ + changefeedID: changefeedID, + } + go sink.Run(ctx) + return sink, nil +} diff --git a/downstreamadapter/sink/helper/helper.go b/downstreamadapter/sink/helper/helper.go index a0d0e448..8236877d 100644 --- a/downstreamadapter/sink/helper/helper.go +++ b/downstreamadapter/sink/helper/helper.go @@ -14,13 +14,83 @@ package helper import ( + "context" "net/url" "strings" + "time" + "github.com/aws/aws-sdk-go/aws/client" + "github.com/aws/aws-sdk-go/aws/request" "github.com/pingcap/ticdc/pkg/config" cerror "github.com/pingcap/ticdc/pkg/errors" + "github.com/pingcap/tidb/br/pkg/storage" + "github.com/pingcap/tiflow/pkg/errors" ) +// GetExternalStorageFromURI creates a new storage.ExternalStorage from a uri. +func GetExternalStorageFromURI( + ctx context.Context, uri string, +) (storage.ExternalStorage, error) { + return GetExternalStorage(ctx, uri, nil, DefaultS3Retryer()) +} + +// GetExternalStorage creates a new storage.ExternalStorage based on the uri and options. +func GetExternalStorage( + ctx context.Context, uri string, + opts *storage.BackendOptions, + retryer request.Retryer, +) (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: retryer, + }) + if err != nil { + retErr := errors.ErrFailToCreateExternalStorage.Wrap(errors.Trace(err)) + return nil, retErr.GenWithStackByArgs("creating ExternalStorage for s3") + } + + // Check the connection and ignore the returned bool value, since we don't care if the file exists. + _, err = ret.FileExists(ctx, "test") + 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 +} + +// 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, + }, + } +} + +// NewS3Retryer creates a new s3 retryer. +func NewS3Retryer(maxRetries int, minRetryDelay, minThrottleDelay time.Duration) request.Retryer { + return retryerWithLog{ + DefaultRetryer: client.DefaultRetryer{ + NumMaxRetries: maxRetries, + MinRetryDelay: minRetryDelay, + MinThrottleDelay: minThrottleDelay, + }, + } +} + // GetTopic returns the topic name from the sink URI. func GetTopic(sinkURI *url.URL) (string, error) { topic := strings.TrimFunc(sinkURI.Path, func(r rune) bool { diff --git a/downstreamadapter/sink/sink.go b/downstreamadapter/sink/sink.go index 04bf230f..99f5a5e5 100644 --- a/downstreamadapter/sink/sink.go +++ b/downstreamadapter/sink/sink.go @@ -50,6 +50,8 @@ func NewSink(ctx context.Context, config *config.ChangefeedConfig, changefeedID return newMySQLSink(ctx, changefeedID, 16, config, sinkURI) case sink.KafkaScheme, sink.KafkaSSLScheme: return newKafkaSink(ctx, changefeedID, sinkURI, config.SinkConfig) + case sink.S3Scheme, sink.FileScheme, sink.GCSScheme, sink.GSScheme, sink.AzblobScheme, sink.AzureScheme, sink.CloudStorageNoopScheme: + return newCloudStorageSink(ctx, changefeedID, sinkURI, config.SinkConfig, nil) case sink.BlackHoleScheme: return newBlackHoleSink() } @@ -67,6 +69,8 @@ func VerifySink(ctx context.Context, config *config.ChangefeedConfig, changefeed return verifyMySQLSink(ctx, sinkURI, config) case sink.KafkaScheme, sink.KafkaSSLScheme: return verifyKafkaSink(ctx, changefeedID, sinkURI, config.SinkConfig) + case sink.S3Scheme, sink.FileScheme, sink.GCSScheme, sink.GSScheme, sink.AzblobScheme, sink.AzureScheme, sink.CloudStorageNoopScheme: + return verifyCloudStorageSink(ctx, changefeedID, sinkURI, config.SinkConfig) case sink.BlackHoleScheme: return nil } diff --git a/downstreamadapter/worker/cloudstorage_ddl_worker.go b/downstreamadapter/worker/cloudstorage_ddl_worker.go new file mode 100644 index 00000000..20d84eb6 --- /dev/null +++ b/downstreamadapter/worker/cloudstorage_ddl_worker.go @@ -0,0 +1,260 @@ +// Copyright 2025 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. +package worker + +import ( + "context" + "encoding/json" + "net/url" + "sync/atomic" + "time" + + "github.com/pingcap/log" + commonType "github.com/pingcap/ticdc/pkg/common" + commonEvent "github.com/pingcap/ticdc/pkg/common/event" + "github.com/pingcap/ticdc/pkg/config" + "github.com/pingcap/ticdc/pkg/metrics" + "github.com/pingcap/ticdc/pkg/sink/cloudstorage" + "github.com/pingcap/ticdc/pkg/sink/util" + "github.com/pingcap/tidb/br/pkg/storage" + "github.com/pingcap/tiflow/pkg/errors" + "github.com/robfig/cron" + "go.uber.org/zap" +) + +type CloudStorageDDLWorker struct { + changefeedID commonType.ChangeFeedID + sinkURI *url.URL + statistics *metrics.Statistics + storage storage.ExternalStorage + cfg *cloudstorage.Config + cron *cron.Cron + + lastCheckpointTs atomic.Uint64 + lastSendCheckpointTsTime time.Time + tableSchemaStore *util.TableSchemaStore + + cleanupJobs []func() /* only for test */ +} + +// NewCloudStorageDDLWorker return a ddl worker instance. +func NewCloudStorageDDLWorker( + changefeedID commonType.ChangeFeedID, + sinkURI *url.URL, + cfg *cloudstorage.Config, + cleanupJobs []func(), + storage storage.ExternalStorage, + statistics *metrics.Statistics, +) *CloudStorageDDLWorker { + return &CloudStorageDDLWorker{ + changefeedID: changefeedID, + sinkURI: sinkURI, + cfg: cfg, + cleanupJobs: cleanupJobs, + storage: storage, + statistics: statistics, + lastSendCheckpointTsTime: time.Now(), + } +} + +func (w *CloudStorageDDLWorker) Run(ctx context.Context) error { + if err := w.initCron(ctx, w.sinkURI, w.cleanupJobs); err != nil { + return errors.Trace(err) + } + w.bgCleanup(ctx) + return nil +} + +func (w *CloudStorageDDLWorker) WriteBlockEvent(event *commonEvent.DDLEvent) error { + for _, e := range event.GetEvents() { + var def cloudstorage.TableDefinition + def.FromDDLEvent(e, w.cfg.OutputColumnID) + if err := w.writeFile(e, def); err != nil { + return err + } + } + event.PostFlush() + return nil +} + +func (w *CloudStorageDDLWorker) AddCheckpointTs(ts uint64) { + if time.Since(w.lastSendCheckpointTsTime) < 2*time.Second { + log.Debug("skip write checkpoint ts to external storage", + zap.Any("changefeedID", w.changefeedID), + zap.Uint64("ts", ts)) + return + } + + defer func() { + w.lastSendCheckpointTsTime = time.Now() + w.lastCheckpointTs.Store(ts) + }() + ckpt, err := json.Marshal(map[string]uint64{"checkpoint-ts": ts}) + if err != nil { + log.Error("CloudStorageSink marshal checkpoint-ts failed", + zap.String("namespace", w.changefeedID.Namespace()), + zap.String("changefeed", w.changefeedID.Name()), + zap.Error(err)) + return + } + err = w.storage.WriteFile(context.Background(), "metadata", ckpt) + if err != nil { + log.Error("CloudStorageSink storage write file failed", + zap.String("namespace", w.changefeedID.Namespace()), + zap.String("changefeed", w.changefeedID.Name()), + zap.Error(err)) + } +} + +func (w *CloudStorageDDLWorker) SetTableSchemaStore(tableSchemaStore *util.TableSchemaStore) { + w.tableSchemaStore = tableSchemaStore +} + +func (w *CloudStorageDDLWorker) Close() { + if w.cron != nil { + w.cron.Stop() + } +} + +func (w *CloudStorageDDLWorker) writeFile(v *commonEvent.DDLEvent, def cloudstorage.TableDefinition) error { + encodedDef, err := def.MarshalWithQuery() + if err != nil { + return errors.Trace(err) + } + + path, err := def.GenerateSchemaFilePath() + if err != nil { + return errors.Trace(err) + } + log.Debug("write ddl event to external storage", + zap.String("path", path), zap.Any("ddl", v)) + return w.statistics.RecordDDLExecution(func() error { + err1 := w.storage.WriteFile(context.Background(), path, encodedDef) + if err1 != nil { + return err1 + } + return nil + }) +} + +func (w *CloudStorageDDLWorker) initCron( + ctx context.Context, sinkURI *url.URL, cleanupJobs []func(), +) (err error) { + if cleanupJobs == nil { + cleanupJobs = w.genCleanupJob(ctx, sinkURI) + } + + w.cron = cron.New() + for _, job := range cleanupJobs { + err = w.cron.AddFunc(w.cfg.FileCleanupCronSpec, job) + if err != nil { + return err + } + } + return nil +} + +func (w *CloudStorageDDLWorker) bgCleanup(ctx context.Context) { + if w.cfg.DateSeparator != config.DateSeparatorDay.String() || w.cfg.FileExpirationDays <= 0 { + log.Info("skip cleanup expired files for storage sink", + zap.String("namespace", w.changefeedID.Namespace()), + zap.Stringer("changefeedID", w.changefeedID.ID()), + zap.String("dateSeparator", w.cfg.DateSeparator), + zap.Int("expiredFileTTL", w.cfg.FileExpirationDays)) + return + } + + w.cron.Start() + defer w.cron.Stop() + log.Info("start schedule cleanup expired files for storage sink", + zap.String("namespace", w.changefeedID.Namespace()), + zap.Stringer("changefeedID", w.changefeedID.ID()), + zap.String("dateSeparator", w.cfg.DateSeparator), + zap.Int("expiredFileTTL", w.cfg.FileExpirationDays)) + + // wait for the context done + <-ctx.Done() + log.Info("stop schedule cleanup expired files for storage sink", + zap.String("namespace", w.changefeedID.Namespace()), + zap.Stringer("changefeedID", w.changefeedID.ID()), + zap.Error(ctx.Err())) +} + +func (w *CloudStorageDDLWorker) genCleanupJob(ctx context.Context, uri *url.URL) []func() { + ret := []func(){} + + isLocal := uri.Scheme == "file" || uri.Scheme == "local" || uri.Scheme == "" + isRemoveEmptyDirsRuning := atomic.Bool{} + if isLocal { + ret = append(ret, func() { + if !isRemoveEmptyDirsRuning.CompareAndSwap(false, true) { + log.Warn("remove empty dirs is already running, skip this round", + zap.String("namespace", w.changefeedID.Namespace()), + zap.Stringer("changefeedID", w.changefeedID.ID())) + return + } + + checkpointTs := w.lastCheckpointTs.Load() + start := time.Now() + cnt, err := cloudstorage.RemoveEmptyDirs(ctx, w.changefeedID, uri.Path) + if err != nil { + log.Error("failed to remove empty dirs", + zap.String("namespace", w.changefeedID.Namespace()), + zap.Stringer("changefeedID", w.changefeedID.ID()), + zap.Uint64("checkpointTs", checkpointTs), + zap.Duration("cost", time.Since(start)), + zap.Error(err), + ) + return + } + log.Info("remove empty dirs", + zap.String("namespace", w.changefeedID.Namespace()), + zap.Stringer("changefeedID", w.changefeedID.ID()), + zap.Uint64("checkpointTs", checkpointTs), + zap.Uint64("count", cnt), + zap.Duration("cost", time.Since(start))) + }) + } + + isCleanupRunning := atomic.Bool{} + ret = append(ret, func() { + if !isCleanupRunning.CompareAndSwap(false, true) { + log.Warn("cleanup expired files is already running, skip this round", + zap.String("namespace", w.changefeedID.Namespace()), + zap.Stringer("changefeedID", w.changefeedID.ID())) + return + } + + defer isCleanupRunning.Store(false) + start := time.Now() + checkpointTs := w.lastCheckpointTs.Load() + cnt, err := cloudstorage.RemoveExpiredFiles(ctx, w.changefeedID, w.storage, w.cfg, checkpointTs) + if err != nil { + log.Error("failed to remove expired files", + zap.String("namespace", w.changefeedID.Namespace()), + zap.Stringer("changefeedID", w.changefeedID.ID()), + zap.Uint64("checkpointTs", checkpointTs), + zap.Duration("cost", time.Since(start)), + zap.Error(err), + ) + return + } + log.Info("remove expired files", + zap.String("namespace", w.changefeedID.Namespace()), + zap.Stringer("changefeedID", w.changefeedID.ID()), + zap.Uint64("checkpointTs", checkpointTs), + zap.Uint64("count", cnt), + zap.Duration("cost", time.Since(start))) + }) + return ret +} diff --git a/downstreamadapter/worker/cloudstorage_dml_worker.go b/downstreamadapter/worker/cloudstorage_dml_worker.go new file mode 100644 index 00000000..40414c35 --- /dev/null +++ b/downstreamadapter/worker/cloudstorage_dml_worker.go @@ -0,0 +1,178 @@ +// Copyright 2025 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. +package worker + +import ( + "context" + "sync" + "sync/atomic" + + "github.com/pingcap/log" + "github.com/pingcap/ticdc/downstreamadapter/worker/writer" + commonType "github.com/pingcap/ticdc/pkg/common" + commonEvent "github.com/pingcap/ticdc/pkg/common/event" + "github.com/pingcap/ticdc/pkg/errors" + "github.com/pingcap/ticdc/pkg/metrics" + "github.com/pingcap/ticdc/pkg/sink/cloudstorage" + "github.com/pingcap/ticdc/pkg/sink/codec" + "github.com/pingcap/ticdc/pkg/sink/codec/common" + "github.com/pingcap/ticdc/utils/chann" + "github.com/pingcap/tidb/br/pkg/storage" + "go.uber.org/zap" + "golang.org/x/sync/errgroup" +) + +const ( + defaultEncodingConcurrency = 8 + defaultChannelSize = 1024 +) + +// CloudStorageDMLWorker denotes a worker responsible for writing messages to cloud storage. +type CloudStorageDMLWorker struct { + changefeedID commonType.ChangeFeedID + storage storage.ExternalStorage + config *cloudstorage.Config + statistics *metrics.Statistics + + // last sequence number + lastSeqNum uint64 + // workers defines a group of workers for encoding events. + workers []*writer.Worker + writers []*writer.Writer + // defragmenter is used to defragment the out-of-order encoded messages and + // sends encoded messages to individual dmlWorkers. + defragmenter *writer.Defragmenter + alive struct { + sync.RWMutex + // msgCh is a channel to hold eventFragment. + // The caller of WriteEvents will write eventFragment to msgCh and + // the encodingWorkers will read eventFragment from msgCh to encode events. + msgCh *chann.DrainableChann[writer.EventFragment] + isDead bool + } +} + +func NewCloudStorageDMLWorker( + changefeedID commonType.ChangeFeedID, + storage storage.ExternalStorage, + config *cloudstorage.Config, + encoderConfig *common.Config, + extension string, + statistics *metrics.Statistics, +) (*CloudStorageDMLWorker, error) { + w := &CloudStorageDMLWorker{ + changefeedID: changefeedID, + storage: storage, + config: config, + statistics: statistics, + workers: make([]*writer.Worker, defaultEncodingConcurrency), + writers: make([]*writer.Writer, config.WorkerCount), + } + w.alive.msgCh = chann.NewAutoDrainChann[writer.EventFragment]() + encodedOutCh := make(chan writer.EventFragment, defaultChannelSize) + workerChannels := make([]*chann.DrainableChann[writer.EventFragment], config.WorkerCount) + // create a group of encoding workers. + for i := 0; i < defaultEncodingConcurrency; i++ { + encoderBuilder, err := codec.NewTxnEventEncoder(encoderConfig) + if err != nil { + return nil, err + } + w.workers[i] = writer.NewWorker(i, w.changefeedID, encoderBuilder, w.alive.msgCh.Out(), encodedOutCh) + } + // create a group of dml workers. + for i := 0; i < w.config.WorkerCount; i++ { + inputCh := chann.NewAutoDrainChann[writer.EventFragment]() + w.writers[i] = writer.NewWriter(i, w.changefeedID, storage, config, extension, + inputCh, w.statistics) + workerChannels[i] = inputCh + } + // create defragmenter. + // The defragmenter is used to defragment the out-of-order encoded messages from encoding workers and + // sends encoded messages to related dmlWorkers in order. Messages of the same table will be sent to + // the same dml + w.defragmenter = writer.NewDefragmenter(encodedOutCh, workerChannels) + + return w, nil +} + +// run creates a set of background goroutines. +func (w *CloudStorageDMLWorker) Run(ctx context.Context) error { + eg, ctx := errgroup.WithContext(ctx) + + for i := 0; i < len(w.workers); i++ { + encodingWorker := w.workers[i] + eg.Go(func() error { + return encodingWorker.Run(ctx) + }) + } + + eg.Go(func() error { + return w.defragmenter.Run(ctx) + }) + + for i := 0; i < len(w.writers); i++ { + worker := w.writers[i] + eg.Go(func() error { + return worker.Run(ctx) + }) + } + + return eg.Wait() +} + +func (w *CloudStorageDMLWorker) AddDMLEvent(event *commonEvent.DMLEvent) { + w.alive.RLock() + defer w.alive.RUnlock() + if w.alive.isDead { + log.Error("dead dmlSink", zap.Error(errors.Trace(errors.New("dead dmlSink")))) + return + } + + if event.State != commonEvent.EventSenderStateNormal { + // The table where the event comes from is in stopping, so it's safe + // to drop the event directly. + event.PostFlush() + return + } + + tbl := cloudstorage.VersionedTableName{ + TableNameWithPhysicTableID: commonType.TableName{ + Schema: event.TableInfo.GetSchemaName(), + Table: event.TableInfo.GetTableName(), + TableID: event.PhysicalTableID, + IsPartition: event.TableInfo.IsPartitionTable(), + }, + TableInfoVersion: event.TableInfoVersion, + } + seq := atomic.AddUint64(&w.lastSeqNum, 1) + + // TODO: add ObserveRows interface + // w.statistics.ObserveRows(event.Rows...) + // emit a TxnCallbackableEvent encoupled with a sequence number starting from one. + w.alive.msgCh.In() <- writer.NewEventFragment(seq, tbl, event) +} + +func (w *CloudStorageDMLWorker) Close() { + w.alive.Lock() + w.alive.isDead = true + w.alive.msgCh.CloseAndDrain() + w.alive.Unlock() + + for _, worker := range w.workers { + worker.Close() + } + + for _, writer := range w.writers { + writer.Close() + } +} diff --git a/downstreamadapter/worker/writer/defragmenter.go b/downstreamadapter/worker/writer/defragmenter.go new file mode 100644 index 00000000..ede568a8 --- /dev/null +++ b/downstreamadapter/worker/writer/defragmenter.go @@ -0,0 +1,131 @@ +// Copyright 2025 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. +package writer + +import ( + "context" + + commonEvent "github.com/pingcap/ticdc/pkg/common/event" + "github.com/pingcap/ticdc/pkg/sink/cloudstorage" + "github.com/pingcap/ticdc/pkg/sink/codec/common" + "github.com/pingcap/ticdc/utils/chann" + "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/hash" +) + +// EventFragment is used to attach a sequence number to TxnCallbackableEvent. +type EventFragment struct { + event *commonEvent.DMLEvent + versionedTable cloudstorage.VersionedTableName + + // The sequence number is mainly useful for TxnCallbackableEvent defragmentation. + // e.g. TxnCallbackableEvent 1~5 are dispatched to a group of encoding workers, but the + // encoding completion time varies. Let's say the final completion sequence are 1,3,2,5,4, + // we can use the sequence numbers to do defragmentation so that the events can arrive + // at dmlWorker sequentially. + seqNumber uint64 + // encodedMsgs denote the encoded messages after the event is handled in encodingWorker. + encodedMsgs []*common.Message +} + +func NewEventFragment(seq uint64, version cloudstorage.VersionedTableName, event *commonEvent.DMLEvent) EventFragment { + return EventFragment{ + seqNumber: seq, + versionedTable: version, + event: event, + } +} + +// Defragmenter is used to handle event fragments which can be registered +// out of order. +type Defragmenter struct { + lastDispatchedSeq uint64 + future map[uint64]EventFragment + inputCh <-chan EventFragment + outputChs []*chann.DrainableChann[EventFragment] + hasher *hash.PositionInertia +} + +func NewDefragmenter( + inputCh <-chan EventFragment, + outputChs []*chann.DrainableChann[EventFragment], +) *Defragmenter { + return &Defragmenter{ + future: make(map[uint64]EventFragment), + inputCh: inputCh, + outputChs: outputChs, + hasher: hash.NewPositionInertia(), + } +} + +func (d *Defragmenter) Run(ctx context.Context) error { + defer d.close() + for { + select { + case <-ctx.Done(): + d.future = nil + return errors.Trace(ctx.Err()) + case frag, ok := <-d.inputCh: + if !ok { + return nil + } + // check whether to write messages to output channel right now + next := d.lastDispatchedSeq + 1 + if frag.seqNumber == next { + d.writeMsgsConsecutive(ctx, frag) + } else if frag.seqNumber > next { + d.future[frag.seqNumber] = frag + } else { + return nil + } + } + } +} + +func (d *Defragmenter) writeMsgsConsecutive( + ctx context.Context, + start EventFragment, +) { + d.dispatchFragToDMLWorker(start) + + // try to dispatch more fragments to DML workers + for { + select { + case <-ctx.Done(): + return + default: + } + next := d.lastDispatchedSeq + 1 + if frag, ok := d.future[next]; ok { + delete(d.future, next) + d.dispatchFragToDMLWorker(frag) + } else { + return + } + } +} + +func (d *Defragmenter) dispatchFragToDMLWorker(frag EventFragment) { + tableName := frag.versionedTable.TableNameWithPhysicTableID + d.hasher.Reset() + d.hasher.Write([]byte(tableName.Schema), []byte(tableName.Table)) + workerID := d.hasher.Sum32() % uint32(len(d.outputChs)) + d.outputChs[workerID].In() <- frag + d.lastDispatchedSeq = frag.seqNumber +} + +func (d *Defragmenter) close() { + for _, ch := range d.outputChs { + ch.CloseAndDrain() + } +} diff --git a/downstreamadapter/worker/writer/worker.go b/downstreamadapter/worker/writer/worker.go new file mode 100644 index 00000000..ff61baf3 --- /dev/null +++ b/downstreamadapter/worker/writer/worker.go @@ -0,0 +1,80 @@ +// Copyright 2025 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. +package writer + +import ( + "context" + "sync/atomic" + + commonType "github.com/pingcap/ticdc/pkg/common" + "github.com/pingcap/ticdc/pkg/sink/codec/common" + "github.com/pingcap/tiflow/pkg/errors" +) + +// Worker denotes the worker responsible for encoding RowChangedEvents +// to messages formatted in the specific protocol. +type Worker struct { + id int + changeFeedID commonType.ChangeFeedID + encoder common.TxnEventEncoder + isClosed uint64 + inputCh <-chan EventFragment + outputCh chan<- EventFragment +} + +func NewWorker( + workerID int, + changefeedID commonType.ChangeFeedID, + encoder common.TxnEventEncoder, + inputCh <-chan EventFragment, + outputCh chan<- EventFragment, +) *Worker { + return &Worker{ + id: workerID, + changeFeedID: changefeedID, + encoder: encoder, + inputCh: inputCh, + outputCh: outputCh, + } +} + +func (w *Worker) Run(ctx context.Context) error { + for { + select { + case <-ctx.Done(): + return errors.Trace(ctx.Err()) + case frag, ok := <-w.inputCh: + if !ok || atomic.LoadUint64(&w.isClosed) == 1 { + return nil + } + err := w.encodeEvents(frag) + if err != nil { + return errors.Trace(err) + } + } + } +} + +func (w *Worker) encodeEvents(frag EventFragment) error { + w.encoder.AppendTxnEvent(frag.event) + frag.encodedMsgs = w.encoder.Build() + w.outputCh <- frag + + return nil +} + +func (w *Worker) Close() { + if !atomic.CompareAndSwapUint64(&w.isClosed, 0, 1) { + return + } +} diff --git a/downstreamadapter/worker/writer/writer.go b/downstreamadapter/worker/writer/writer.go new file mode 100644 index 00000000..653daa2e --- /dev/null +++ b/downstreamadapter/worker/writer/writer.go @@ -0,0 +1,368 @@ +// Copyright 2025 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. +package writer + +import ( + "bytes" + "context" + "path" + "strconv" + "sync/atomic" + "time" + + "github.com/pingcap/failpoint" + "github.com/pingcap/log" + commonType "github.com/pingcap/ticdc/pkg/common" + "github.com/pingcap/ticdc/pkg/metrics" + "github.com/pingcap/ticdc/pkg/sink/cloudstorage" + "github.com/pingcap/ticdc/pkg/sink/codec/common" + "github.com/pingcap/ticdc/utils/chann" + "github.com/pingcap/tidb/br/pkg/storage" + mcloudstorage "github.com/pingcap/tiflow/cdc/sink/metrics/cloudstorage" + "github.com/pingcap/tiflow/pkg/errors" + "github.com/prometheus/client_golang/prometheus" + "go.uber.org/zap" + "golang.org/x/sync/errgroup" +) + +// Writer denotes a worker responsible for writing messages to cloud storage. +type Writer struct { + // worker id + id int + changeFeedID commonType.ChangeFeedID + storage storage.ExternalStorage + config *cloudstorage.Config + // toBeFlushedCh contains a set of batchedTask waiting to be flushed to cloud storage. + toBeFlushedCh chan batchedTask + inputCh *chann.DrainableChann[EventFragment] + isClosed uint64 + statistics *metrics.Statistics + filePathGenerator *cloudstorage.FilePathGenerator + metricWriteBytes prometheus.Gauge + metricFileCount prometheus.Gauge + metricWriteDuration prometheus.Observer + metricFlushDuration prometheus.Observer + metricsWorkerBusyRatio prometheus.Counter +} + +func NewWriter( + id int, + changefeedID commonType.ChangeFeedID, + storage storage.ExternalStorage, + config *cloudstorage.Config, + extension string, + inputCh *chann.DrainableChann[EventFragment], + statistics *metrics.Statistics, +) *Writer { + d := &Writer{ + id: id, + changeFeedID: changefeedID, + storage: storage, + config: config, + inputCh: inputCh, + toBeFlushedCh: make(chan batchedTask, 64), + statistics: statistics, + filePathGenerator: cloudstorage.NewFilePathGenerator(changefeedID, config, storage, extension), + metricWriteBytes: mcloudstorage.CloudStorageWriteBytesGauge. + WithLabelValues(changefeedID.Namespace(), changefeedID.ID().String()), + metricFileCount: mcloudstorage.CloudStorageFileCountGauge. + WithLabelValues(changefeedID.Namespace(), changefeedID.ID().String()), + metricWriteDuration: mcloudstorage.CloudStorageWriteDurationHistogram. + WithLabelValues(changefeedID.Namespace(), changefeedID.ID().String()), + metricFlushDuration: mcloudstorage.CloudStorageFlushDurationHistogram. + WithLabelValues(changefeedID.Namespace(), changefeedID.ID().String()), + metricsWorkerBusyRatio: mcloudstorage.CloudStorageWorkerBusyRatioCounter. + WithLabelValues(changefeedID.Namespace(), changefeedID.ID().String(), strconv.Itoa(id)), + } + + return d +} + +// Run creates a set of background goroutines. +func (d *Writer) Run(ctx context.Context) error { + eg, ctx := errgroup.WithContext(ctx) + eg.Go(func() error { + return d.flushMessages(ctx) + }) + + eg.Go(func() error { + return d.genAndDispatchTask(ctx, d.inputCh) + }) + + return eg.Wait() +} + +// flushMessages flushed messages of active tables to cloud storage. +// active tables are those tables that have received events after the last flush. +func (d *Writer) flushMessages(ctx context.Context) error { + var flushTimeSlice time.Duration + overseerDuration := d.config.FlushInterval * 2 + overseerTicker := time.NewTicker(overseerDuration) + defer overseerTicker.Stop() + for { + select { + case <-ctx.Done(): + return errors.Trace(ctx.Err()) + case <-overseerTicker.C: + d.metricsWorkerBusyRatio.Add(flushTimeSlice.Seconds()) + flushTimeSlice = 0 + case batchedTask := <-d.toBeFlushedCh: + if atomic.LoadUint64(&d.isClosed) == 1 { + return nil + } + start := time.Now() + for table, task := range batchedTask.batch { + if len(task.msgs) == 0 { + continue + } + + // generate scheme.json file before generating the first data file if necessary + err := d.filePathGenerator.CheckOrWriteSchema(ctx, table, task.tableInfo) + if err != nil { + log.Error("failed to write schema file to external storage", + zap.Int("workerID", d.id), + zap.String("namespace", d.changeFeedID.Namespace()), + zap.Stringer("changefeed", d.changeFeedID.ID()), + zap.Error(err)) + return errors.Trace(err) + } + + // make sure that `generateDateStr()` is invoked ONLY once before + // generating data file path and index file path. Because we don't expect the index + // file is written to a different dir if date change happens between + // generating data and index file. + date := d.filePathGenerator.GenerateDateStr() + dataFilePath, err := d.filePathGenerator.GenerateDataFilePath(ctx, table, date) + if err != nil { + log.Error("failed to generate data file path", + zap.Int("workerID", d.id), + zap.String("namespace", d.changeFeedID.Namespace()), + zap.Stringer("changefeed", d.changeFeedID.ID()), + zap.Error(err)) + return errors.Trace(err) + } + indexFilePath := d.filePathGenerator.GenerateIndexFilePath(table, date) + + // first write the index file to external storage. + // the file content is simply the last element of the data file path + err = d.writeIndexFile(ctx, indexFilePath, path.Base(dataFilePath)+"\n") + if err != nil { + log.Error("failed to write index file to external storage", + zap.Int("workerID", d.id), + zap.String("namespace", d.changeFeedID.Namespace()), + zap.Stringer("changefeed", d.changeFeedID.ID()), + zap.String("path", indexFilePath), + zap.Error(err)) + } + + // then write the data file to external storage. + err = d.writeDataFile(ctx, dataFilePath, task) + if err != nil { + log.Error("failed to write data file to external storage", + zap.Int("workerID", d.id), + zap.String("namespace", d.changeFeedID.Namespace()), + zap.Stringer("changefeed", d.changeFeedID.ID()), + zap.String("path", dataFilePath), + zap.Error(err)) + return errors.Trace(err) + } + + log.Debug("write file to storage success", zap.Int("workerID", d.id), + zap.String("namespace", d.changeFeedID.Namespace()), + zap.Stringer("changefeed", d.changeFeedID.ID()), + zap.String("schema", table.TableNameWithPhysicTableID.Schema), + zap.String("table", table.TableNameWithPhysicTableID.Table), + zap.String("path", dataFilePath), + ) + for _, msg := range task.msgs { + msg.Callback() + } + } + flushTimeSlice += time.Since(start) + } + } +} + +func (d *Writer) writeIndexFile(ctx context.Context, path, content string) error { + start := time.Now() + err := d.storage.WriteFile(ctx, path, []byte(content)) + d.metricFlushDuration.Observe(time.Since(start).Seconds()) + return err +} + +func (d *Writer) writeDataFile(ctx context.Context, path string, task *singleTableTask) error { + var callbacks []func() + buf := bytes.NewBuffer(make([]byte, 0, task.size)) + rowsCnt := 0 + bytesCnt := int64(0) + for _, msg := range task.msgs { + bytesCnt += int64(len(msg.Value)) + rowsCnt += msg.GetRowsCount() + buf.Write(msg.Value) + callbacks = append(callbacks, msg.Callback) + } + + if err := d.statistics.RecordBatchExecution(func() (int, int64, error) { + start := time.Now() + if d.config.FlushConcurrency <= 1 { + return rowsCnt, bytesCnt, d.storage.WriteFile(ctx, path, buf.Bytes()) + } + + writer, inErr := d.storage.Create(ctx, path, &storage.WriterOption{ + Concurrency: d.config.FlushConcurrency, + }) + if inErr != nil { + return 0, 0, inErr + } + + defer func() { + closeErr := writer.Close(ctx) + if inErr != nil { + log.Error("failed to close writer", zap.Error(closeErr), + zap.Int("workerID", d.id), + zap.Any("table", task.tableInfo.TableName), + zap.String("namespace", d.changeFeedID.Namespace()), + zap.Stringer("changefeed", d.changeFeedID.ID())) + if inErr == nil { + inErr = closeErr + } + } + }() + if _, inErr = writer.Write(ctx, buf.Bytes()); inErr != nil { + return 0, 0, inErr + } + + d.metricFlushDuration.Observe(time.Since(start).Seconds()) + return rowsCnt, bytesCnt, nil + }); err != nil { + return err + } + + d.metricWriteBytes.Add(float64(bytesCnt)) + d.metricFileCount.Add(1) + for _, cb := range callbacks { + if cb != nil { + cb() + } + } + + return nil +} + +// genAndDispatchTask dispatches flush tasks in two conditions: +// 1. the flush interval exceeds the upper limit. +// 2. the file size exceeds the upper limit. +func (d *Writer) genAndDispatchTask(ctx context.Context, + ch *chann.DrainableChann[EventFragment], +) error { + batchedTask := newBatchedTask() + ticker := time.NewTicker(d.config.FlushInterval) + + for { + // this failpoint is use to pass this ticker once + // to make writeEvent in the test case can write into the same file + failpoint.Inject("passTickerOnce", func() { + <-ticker.C + }) + + select { + case <-ctx.Done(): + return errors.Trace(ctx.Err()) + case <-ticker.C: + if atomic.LoadUint64(&d.isClosed) == 1 { + return nil + } + select { + case <-ctx.Done(): + return errors.Trace(ctx.Err()) + case d.toBeFlushedCh <- batchedTask: + log.Debug("flush task is emitted successfully when flush interval exceeds", + zap.Int("tablesLength", len(batchedTask.batch))) + batchedTask = newBatchedTask() + default: + } + case frag, ok := <-ch.Out(): + if !ok || atomic.LoadUint64(&d.isClosed) == 1 { + return nil + } + batchedTask.handleSingleTableEvent(frag) + // if the file size exceeds the upper limit, emit the flush task containing the table + // as soon as possible. + table := frag.versionedTable + if batchedTask.batch[table].size >= uint64(d.config.FileSize) { + task := batchedTask.generateTaskByTable(table) + select { + case <-ctx.Done(): + return errors.Trace(ctx.Err()) + case d.toBeFlushedCh <- task: + log.Debug("flush task is emitted successfully when file size exceeds", + zap.Any("table", table), + zap.Int("eventsLenth", len(task.batch[table].msgs))) + } + } + } + } +} + +func (d *Writer) Close() { + if !atomic.CompareAndSwapUint64(&d.isClosed, 0, 1) { + return + } +} + +// batchedTask contains a set of singleTableTask. +// We batch message of different tables together to reduce the overhead of calling external storage API. +type batchedTask struct { + batch map[cloudstorage.VersionedTableName]*singleTableTask +} + +// singleTableTask contains a set of messages belonging to the same table. +type singleTableTask struct { + size uint64 + tableInfo *commonType.TableInfo + msgs []*common.Message +} + +func newBatchedTask() batchedTask { + return batchedTask{ + batch: make(map[cloudstorage.VersionedTableName]*singleTableTask), + } +} + +func (t *batchedTask) handleSingleTableEvent(event EventFragment) { + table := event.versionedTable + if _, ok := t.batch[table]; !ok { + t.batch[table] = &singleTableTask{ + size: 0, + tableInfo: event.event.TableInfo, + } + } + + v := t.batch[table] + for _, msg := range event.encodedMsgs { + v.size += uint64(len(msg.Value)) + } + v.msgs = append(v.msgs, event.encodedMsgs...) +} + +func (t *batchedTask) generateTaskByTable(table cloudstorage.VersionedTableName) batchedTask { + v := t.batch[table] + if v == nil { + log.Panic("table not found in dml task", zap.Any("table", table), zap.Any("task", t)) + } + delete(t.batch, table) + + return batchedTask{ + batch: map[cloudstorage.VersionedTableName]*singleTableTask{table: v}, + } +} diff --git a/go.mod b/go.mod index b4203876..dbd8d2e6 100644 --- a/go.mod +++ b/go.mod @@ -9,6 +9,7 @@ require ( github.com/DATA-DOG/go-sqlmock v1.5.0 github.com/IBM/sarama v1.41.2 github.com/apache/pulsar-client-go v0.11.0 + github.com/aws/aws-sdk-go v1.55.5 github.com/aws/aws-sdk-go-v2 v1.19.1 github.com/aws/aws-sdk-go-v2/config v1.18.30 github.com/aws/aws-sdk-go-v2/credentials v1.13.29 @@ -48,6 +49,7 @@ require ( github.com/prometheus/client_golang v1.20.4 github.com/r3labs/diff v1.1.0 github.com/rcrowley/go-metrics v0.0.0-20201227073835-cf1acfcdf475 + github.com/robfig/cron v1.2.0 github.com/segmentio/kafka-go v0.4.41-0.20230526171612-f057b1d369cd github.com/spf13/cobra v1.8.1 github.com/spf13/pflag v1.0.5 @@ -102,7 +104,6 @@ require ( github.com/apache/thrift v0.16.0 // indirect github.com/ardielle/ardielle-go v1.5.2 // indirect github.com/asaskevich/govalidator v0.0.0-20230301143203-a9d515a09cc2 // indirect - github.com/aws/aws-sdk-go v1.55.5 // indirect github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.13.6 // indirect github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.36 // indirect github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.4.30 // indirect @@ -266,7 +267,6 @@ require ( github.com/qri-io/jsonschema v0.2.1 // indirect github.com/remyoudompheng/bigfft v0.0.0-20230129092748-24d4a6f8daec // indirect github.com/rivo/uniseg v0.4.7 // indirect - github.com/robfig/cron v1.2.0 // indirect github.com/robfig/cron/v3 v3.0.1 // indirect github.com/rogpeppe/go-internal v1.12.0 // indirect github.com/rs/cors v1.7.0 // indirect diff --git a/logservice/schemastore/persist_storage_ddl_handlers.go b/logservice/schemastore/persist_storage_ddl_handlers.go index eee6bdd9..759c66a1 100644 --- a/logservice/schemastore/persist_storage_ddl_handlers.go +++ b/logservice/schemastore/persist_storage_ddl_handlers.go @@ -1839,6 +1839,8 @@ func buildDDLEventForExchangeTablePartition(rawEvent *PersistedDDLEvent, tableFi if !ok { return ddlEvent, false } + ddlEvent.ExtraSchemaName = rawEvent.ExtraSchemaName + ddlEvent.ExtraTableName = rawEvent.ExtraTableName // TODO: rawEvent.TableInfo is not correct for ignoreNormalTable ignoreNormalTable := tableFilter != nil && tableFilter.ShouldIgnoreTable(rawEvent.SchemaName, rawEvent.TableName, rawEvent.TableInfo) ignorePartitionTable := tableFilter != nil && tableFilter.ShouldIgnoreTable(rawEvent.ExtraSchemaName, rawEvent.ExtraTableName, rawEvent.TableInfo) @@ -1901,6 +1903,10 @@ func buildDDLEventForExchangeTablePartition(rawEvent *PersistedDDLEvent, tableFi } else { log.Fatal("should not happen") } + ddlEvent.MultipleTableInfos = []*common.TableInfo{ + common.WrapTableInfo(rawEvent.SchemaID, rawEvent.SchemaName, rawEvent.TableInfo), + rawEvent.ExtraTableInfo, + } return ddlEvent, true } @@ -1932,7 +1938,7 @@ func buildDDLEventForRenameTables(rawEvent *PersistedDDLEvent, tableFilter filte allPhysicalIDs := getAllPartitionIDs(rawEvent.TableInfo) if !ignorePrevTable { resultQuerys = append(resultQuerys, querys[i]) - tableInfos = append(tableInfos, common.WrapTableInfo(rawEvent.SchemaID, rawEvent.SchemaName, tableInfo)) + tableInfos = append(tableInfos, common.WrapTableInfo(rawEvent.SchemaID, rawEvent.SchemaNames[i], tableInfo)) ddlEvent.BlockedTables.TableIDs = append(ddlEvent.BlockedTables.TableIDs, allPhysicalIDs...) if !ignoreCurrentTable { // check whether schema change @@ -1975,7 +1981,7 @@ func buildDDLEventForRenameTables(rawEvent *PersistedDDLEvent, tableFilter filte } else { if !ignorePrevTable { resultQuerys = append(resultQuerys, querys[i]) - tableInfos = append(tableInfos, common.WrapTableInfo(rawEvent.SchemaID, rawEvent.SchemaName, tableInfo)) + tableInfos = append(tableInfos, common.WrapTableInfo(rawEvent.SchemaID, rawEvent.SchemaNames[i], tableInfo)) ddlEvent.BlockedTables.TableIDs = append(ddlEvent.BlockedTables.TableIDs, tableInfo.ID) if !ignoreCurrentTable { if rawEvent.ExtraSchemaIDs[i] != rawEvent.SchemaIDs[i] { diff --git a/pkg/common/event/ddl_event.go b/pkg/common/event/ddl_event.go index bd0ad450..1196af4b 100644 --- a/pkg/common/event/ddl_event.go +++ b/pkg/common/event/ddl_event.go @@ -130,6 +130,9 @@ func (d *DDLEvent) GetEvents() []*DDLEvent { // Such as rename table test.table1 to test.table10, test.table2 to test.table20 switch model.ActionType(d.Type) { case model.ActionExchangeTablePartition: + if len(d.MultipleTableInfos) != 2 { + log.Panic("multipleTableInfos length should be equal to 2", zap.Any("multipleTableInfos", d.MultipleTableInfos)) + } return []*DDLEvent{ // partition table before exchange { @@ -139,6 +142,7 @@ func (d *DDLEvent) GetEvents() []*DDLEvent { // TableID: d.TableID, SchemaName: d.SchemaName, TableName: d.TableName, + TableInfo: d.MultipleTableInfos[0], Query: d.Query, FinishedTs: d.FinishedTs, }, @@ -148,6 +152,7 @@ func (d *DDLEvent) GetEvents() []*DDLEvent { Type: d.Type, // SchemaID: d.TableInfo.SchemaID, // TableID: d.TableInfo.TableName.TableID, + TableInfo: d.MultipleTableInfos[1], SchemaName: d.ExtraSchemaName, TableName: d.ExtraTableName, Query: d.Query, diff --git a/pkg/common/types.go b/pkg/common/types.go index c1a1596f..f6f96097 100644 --- a/pkg/common/types.go +++ b/pkg/common/types.go @@ -260,5 +260,6 @@ type SinkType int const ( MysqlSinkType SinkType = iota KafkaSinkType + CloudStorageSinkType BlackHoleSinkType ) diff --git a/pkg/sink/cloudstorage/config.go b/pkg/sink/cloudstorage/config.go new file mode 100644 index 00000000..f0c1ecd0 --- /dev/null +++ b/pkg/sink/cloudstorage/config.go @@ -0,0 +1,235 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package cloudstorage + +import ( + "context" + "fmt" + "net/http" + "net/url" + "strings" + "time" + + "github.com/gin-gonic/gin/binding" + "github.com/imdario/mergo" + "github.com/pingcap/log" + "github.com/pingcap/ticdc/pkg/config" + cerror "github.com/pingcap/tiflow/pkg/errors" + psink "github.com/pingcap/tiflow/pkg/sink" + "github.com/pingcap/tiflow/pkg/util" + "go.uber.org/zap" +) + +const ( + // defaultWorkerCount is the default value of worker-count. + defaultWorkerCount = 16 + // the upper limit of worker-count. + maxWorkerCount = 512 + // defaultFlushInterval is the default value of flush-interval. + defaultFlushInterval = 5 * time.Second + // the lower limit of flush-interval. + minFlushInterval = 2 * time.Second + // the upper limit of flush-interval. + maxFlushInterval = 10 * time.Minute + // defaultFlushConcurrency is the default value of flush-concurrency. + defaultFlushConcurrency = 1 + // the lower limit of flush-concurrency. + minFlushConcurrency = 1 + // the upper limit of flush-concurrency. + maxFlushConcurrency = 512 + // defaultFileSize is the default value of file-size. + defaultFileSize = 64 * 1024 * 1024 + // the lower limit of file size + minFileSize = 1024 * 1024 + // the upper limit of file size + maxFileSize = 512 * 1024 * 1024 + + // disable file cleanup by default + defaultFileExpirationDays = 0 + // Second | Minute | Hour | Dom | Month | DowOptional + // `0 0 2 * * ?` means 2:00:00 AM every day + defaultFileCleanupCronSpec = "0 0 2 * * *" +) + +type urlConfig struct { + WorkerCount *int `form:"worker-count"` + FlushInterval *string `form:"flush-interval"` + FileSize *int `form:"file-size"` +} + +// Config is the configuration for cloud storage sink. +type Config struct { + WorkerCount int + FlushInterval time.Duration + FileSize int + FileIndexWidth int + DateSeparator string + FileExpirationDays int + FileCleanupCronSpec string + EnablePartitionSeparator bool + OutputColumnID bool + FlushConcurrency int +} + +// NewConfig returns the default cloud storage sink config. +func NewConfig() *Config { + return &Config{ + WorkerCount: defaultWorkerCount, + FlushInterval: defaultFlushInterval, + FileSize: defaultFileSize, + FileExpirationDays: defaultFileExpirationDays, + FileCleanupCronSpec: defaultFileCleanupCronSpec, + } +} + +// Apply applies the sink URI parameters to the config. +func (c *Config) Apply( + ctx context.Context, + sinkURI *url.URL, + sinkConfig *config.SinkConfig, +) (err error) { + if sinkURI == nil { + return cerror.ErrStorageSinkInvalidConfig.GenWithStack( + "failed to open cloud storage sink, empty SinkURI") + } + + scheme := strings.ToLower(sinkURI.Scheme) + if !psink.IsStorageScheme(scheme) { + return cerror.ErrStorageSinkInvalidConfig.GenWithStack( + "can't create cloud storage sink with unsupported scheme: %s", scheme) + } + req := &http.Request{URL: sinkURI} + urlParameter := &urlConfig{} + if err := binding.Query.Bind(req, urlParameter); err != nil { + return cerror.WrapError(cerror.ErrStorageSinkInvalidConfig, err) + } + if urlParameter, err = mergeConfig(sinkConfig, urlParameter); err != nil { + return err + } + if err = getWorkerCount(urlParameter, &c.WorkerCount); err != nil { + return err + } + err = getFlushInterval(urlParameter, &c.FlushInterval) + if err != nil { + return err + } + err = getFileSize(urlParameter, &c.FileSize) + if err != nil { + return err + } + + c.DateSeparator = util.GetOrZero(sinkConfig.DateSeparator) + c.EnablePartitionSeparator = util.GetOrZero(sinkConfig.EnablePartitionSeparator) + c.FileIndexWidth = util.GetOrZero(sinkConfig.FileIndexWidth) + if sinkConfig.CloudStorageConfig != nil { + c.OutputColumnID = util.GetOrZero(sinkConfig.CloudStorageConfig.OutputColumnID) + if sinkConfig.CloudStorageConfig.FileExpirationDays != nil { + c.FileExpirationDays = *sinkConfig.CloudStorageConfig.FileExpirationDays + } + if sinkConfig.CloudStorageConfig.FileCleanupCronSpec != nil { + c.FileCleanupCronSpec = *sinkConfig.CloudStorageConfig.FileCleanupCronSpec + } + c.FlushConcurrency = util.GetOrZero(sinkConfig.CloudStorageConfig.FlushConcurrency) + } + + if c.FileIndexWidth < config.MinFileIndexWidth || c.FileIndexWidth > config.MaxFileIndexWidth { + c.FileIndexWidth = config.DefaultFileIndexWidth + } + if c.FlushConcurrency < minFlushConcurrency || c.FlushConcurrency > maxFlushConcurrency { + c.FlushConcurrency = defaultFlushConcurrency + } + + return nil +} + +func mergeConfig( + sinkConfig *config.SinkConfig, + urlParameters *urlConfig, +) (*urlConfig, error) { + dest := &urlConfig{} + if sinkConfig != nil && sinkConfig.CloudStorageConfig != nil { + dest.WorkerCount = sinkConfig.CloudStorageConfig.WorkerCount + dest.FlushInterval = sinkConfig.CloudStorageConfig.FlushInterval + dest.FileSize = sinkConfig.CloudStorageConfig.FileSize + } + if err := mergo.Merge(dest, urlParameters, mergo.WithOverride); err != nil { + return nil, cerror.WrapError(cerror.ErrStorageSinkInvalidConfig, err) + } + return dest, nil +} + +func getWorkerCount(values *urlConfig, workerCount *int) error { + if values.WorkerCount == nil { + return nil + } + + c := *values.WorkerCount + if c <= 0 { + return cerror.WrapError(cerror.ErrStorageSinkInvalidConfig, + fmt.Errorf("invalid worker-count %d, it must be greater than 0", c)) + } + if c > maxWorkerCount { + log.Warn("worker-count is too large", + zap.Int("original", c), zap.Int("override", maxWorkerCount)) + c = maxWorkerCount + } + + *workerCount = c + return nil +} + +func getFlushInterval(values *urlConfig, flushInterval *time.Duration) error { + if values.FlushInterval == nil || len(*values.FlushInterval) == 0 { + return nil + } + + d, err := time.ParseDuration(*values.FlushInterval) + if err != nil { + return cerror.WrapError(cerror.ErrStorageSinkInvalidConfig, err) + } + + if d > maxFlushInterval { + log.Warn("flush-interval is too large", zap.Duration("original", d), + zap.Duration("override", maxFlushInterval)) + d = maxFlushInterval + } + if d < minFlushInterval { + log.Warn("flush-interval is too small", zap.Duration("original", d), + zap.Duration("override", minFlushInterval)) + d = minFlushInterval + } + + *flushInterval = d + return nil +} + +func getFileSize(values *urlConfig, fileSize *int) error { + if values.FileSize == nil { + return nil + } + + sz := *values.FileSize + if sz > maxFileSize { + log.Warn("file-size is too large", + zap.Int("original", sz), zap.Int("override", maxFileSize)) + sz = maxFileSize + } + if sz < minFileSize { + log.Warn("file-size is too small", + zap.Int("original", sz), zap.Int("override", minFileSize)) + sz = minFileSize + } + *fileSize = sz + return nil +} diff --git a/pkg/sink/cloudstorage/config_test.go b/pkg/sink/cloudstorage/config_test.go new file mode 100644 index 00000000..af6515d5 --- /dev/null +++ b/pkg/sink/cloudstorage/config_test.go @@ -0,0 +1,167 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package cloudstorage + +import ( + "context" + "net/url" + "testing" + "time" + + "github.com/aws/aws-sdk-go/aws" + "github.com/pingcap/ticdc/pkg/config" + "github.com/stretchr/testify/require" +) + +func TestConfigApply(t *testing.T) { + expected := NewConfig() + expected.WorkerCount = 32 + expected.FlushInterval = 10 * time.Second + expected.FileSize = 16 * 1024 * 1024 + expected.FileIndexWidth = config.DefaultFileIndexWidth + expected.DateSeparator = config.DateSeparatorDay.String() + expected.EnablePartitionSeparator = true + expected.FlushConcurrency = 1 + uri := "s3://bucket/prefix?worker-count=32&flush-interval=10s&file-size=16777216&protocol=csv" + sinkURI, err := url.Parse(uri) + require.Nil(t, err) + + replicaConfig := config.GetDefaultReplicaConfig() + err = replicaConfig.ValidateAndAdjust(sinkURI) + require.NoError(t, err) + cfg := NewConfig() + err = cfg.Apply(context.TODO(), sinkURI, replicaConfig.Sink) + require.Nil(t, err) + require.Equal(t, expected, cfg) +} + +func TestVerifySinkURIParams(t *testing.T) { + testCases := []struct { + name string + uri string + expectedErr string + }{ + { + name: "valid sink uri with local/nfs scheme", + uri: "file://tmp/test", + expectedErr: "", + }, + { + name: "valid sink uri with s3 scheme", + uri: "s3://bucket/prefix", + expectedErr: "", + }, + { + name: "valid sink uri with gcs scheme", + uri: "gcs://bucket/prefix", + expectedErr: "", + }, + { + name: "valid sink uri with azblob scheme", + uri: "azblob://bucket/prefix", + expectedErr: "", + }, + { + name: "sink uri with valid scheme, worker-count, flush-interval and file-size", + uri: "s3://bucket/prefix?worker-count=64&flush-interval=1m30s&file-size=33554432", + expectedErr: "", + }, + { + name: "invalid sink uri with unknown storage scheme", + uri: "xxx://tmp/test", + expectedErr: "can't create cloud storage sink with unsupported scheme", + }, + { + name: "invalid sink uri with worker-count number less than lower limit", + uri: "file://tmp/test?worker-count=-1", + expectedErr: "invalid worker-count -1, it must be greater than 0", + }, + { + name: "invalid sink uri with worker-count number greater than upper limit", + uri: "s3://bucket/prefix?worker-count=10000", + expectedErr: "", + }, + { + name: "invalid sink uri with flush-interval less than lower limit", + uri: "s3://bucket/prefix?flush-interval=-10s", + expectedErr: "", + }, + { + name: "invalid sink uri with flush-interval greater than upper limit", + uri: "s3://bucket/prefix?flush=interval=1h", + expectedErr: "", + }, + { + name: "invalid sink uri with file-size less than lower limit", + uri: "s3://bucket/prefix?file-size=1024", + expectedErr: "", + }, + { + name: "invalid sink uri with file-size greater than upper limit", + uri: "s3://bucket/prefix?file-size=1073741824", + expectedErr: "", + }, + } + + for _, tc := range testCases { + sinkURI, err := url.Parse(tc.uri) + require.Nil(t, err) + cfg := NewConfig() + err = cfg.Apply(context.TODO(), sinkURI, config.GetDefaultReplicaConfig().Sink) + if tc.expectedErr == "" { + require.Nil(t, err) + require.LessOrEqual(t, cfg.WorkerCount, maxWorkerCount) + require.LessOrEqual(t, cfg.FlushInterval, maxFlushInterval) + require.LessOrEqual(t, cfg.FileSize, maxFileSize) + } else { + require.Regexp(t, tc.expectedErr, err) + } + } +} + +func TestMergeConfig(t *testing.T) { + uri := "s3://bucket/prefix" + sinkURI, err := url.Parse(uri) + require.NoError(t, err) + replicaConfig := config.GetDefaultReplicaConfig() + replicaConfig.Sink.CloudStorageConfig = &config.CloudStorageConfig{ + WorkerCount: aws.Int(12), + FileSize: aws.Int(1485760), + FlushInterval: aws.String("1m2s"), + OutputColumnID: aws.Bool(false), + } + c := NewConfig() + err = c.Apply(context.TODO(), sinkURI, replicaConfig.Sink) + require.NoError(t, err) + require.Equal(t, 12, c.WorkerCount) + require.Equal(t, 1485760, c.FileSize) + require.Equal(t, "1m2s", c.FlushInterval.String()) + + // test override + uri = "s3://bucket/prefix?worker-count=64&flush-interval=2m2s&file-size=33554432" + sinkURI, err = url.Parse(uri) + require.NoError(t, err) + replicaConfig.Sink.CloudStorageConfig = &config.CloudStorageConfig{ + WorkerCount: aws.Int(12), + FileSize: aws.Int(10485760), + FlushInterval: aws.String("1m2s"), + OutputColumnID: aws.Bool(false), + } + c = NewConfig() + err = c.Apply(context.TODO(), sinkURI, replicaConfig.Sink) + require.NoError(t, err) + require.Equal(t, 64, c.WorkerCount) + require.Equal(t, 33554432, c.FileSize) + require.Equal(t, "2m2s", c.FlushInterval.String()) +} diff --git a/pkg/sink/cloudstorage/path.go b/pkg/sink/cloudstorage/path.go new file mode 100644 index 00000000..1e6c306e --- /dev/null +++ b/pkg/sink/cloudstorage/path.go @@ -0,0 +1,501 @@ +// Copyright 2023 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package cloudstorage + +import ( + "context" + "fmt" + "io" + "io/fs" + "os" + "path" + "path/filepath" + "regexp" + "strconv" + "strings" + "time" + + "github.com/pingcap/log" + commonType "github.com/pingcap/ticdc/pkg/common" + appcontext "github.com/pingcap/ticdc/pkg/common/context" + "github.com/pingcap/ticdc/pkg/pdutil" + "github.com/pingcap/tidb/br/pkg/storage" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/engine/pkg/clock" + "github.com/pingcap/tiflow/pkg/config" + "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/hash" + "github.com/pingcap/tiflow/pkg/util" + "github.com/tikv/client-go/v2/oracle" + "go.uber.org/zap" +) + +const ( + // 3 is the length of "CDC", and the file number contains + // at least 6 digits (e.g. CDC000001.csv). + minFileNamePrefixLen = 3 + config.MinFileIndexWidth + defaultIndexFileName = "meta/CDC.index" + + // The following constants are used to generate file paths. + schemaFileNameFormat = "schema_%d_%010d.json" + // The database schema is stored in the following path: + // /meta/schema_{tableVersion}_{checksum}.json + dbSchemaPrefix = "%s/meta/" + // The table schema is stored in the following path: + // //meta/schema_{tableVersion}_{checksum}.json + tableSchemaPrefix = "%s/%s/meta/" +) + +var schemaRE = regexp.MustCompile(`meta/schema_\d+_\d{10}\.json$`) + +// IsSchemaFile checks whether the file is a schema file. +func IsSchemaFile(path string) bool { + return schemaRE.MatchString(path) +} + +// mustParseSchemaName parses the version from the schema file name. +func mustParseSchemaName(path string) (uint64, uint32) { + reportErr := func(err error) { + log.Panic("failed to parse schema file name", + zap.String("schemaPath", path), + zap.Any("error", err)) + } + + // For /
/meta/schema_{tableVersion}_{checksum}.json, the parts + // should be ["/
/meta/schema", "{tableVersion}", "{checksum}.json"]. + parts := strings.Split(path, "_") + if len(parts) < 3 { + reportErr(errors.New("invalid path format")) + } + + checksum := strings.TrimSuffix(parts[len(parts)-1], ".json") + tableChecksum, err := strconv.ParseUint(checksum, 10, 64) + if err != nil { + reportErr(err) + } + version := parts[len(parts)-2] + tableVersion, err := strconv.ParseUint(version, 10, 64) + if err != nil { + reportErr(err) + } + return tableVersion, uint32(tableChecksum) +} + +func generateSchemaFilePath( + schema, table string, tableVersion uint64, checksum uint32, +) string { + if schema == "" || tableVersion == 0 { + log.Panic("invalid schema or tableVersion", + zap.String("schema", schema), zap.String("table", table), zap.Uint64("tableVersion", tableVersion)) + } + + var dir string + if table == "" { + // Generate db schema file path. + dir = fmt.Sprintf(dbSchemaPrefix, schema) + } else { + // Generate table schema file path. + dir = fmt.Sprintf(tableSchemaPrefix, schema, table) + } + name := fmt.Sprintf(schemaFileNameFormat, tableVersion, checksum) + return path.Join(dir, name) +} + +func generateDataFileName(index uint64, extension string, fileIndexWidth int) string { + indexFmt := "%0" + strconv.Itoa(fileIndexWidth) + "d" + return fmt.Sprintf("CDC"+indexFmt+"%s", index, extension) +} + +type indexWithDate struct { + index uint64 + currDate, prevDate string +} + +// VersionedTableName is used to wrap TableNameWithPhysicTableID with a version. +type VersionedTableName struct { + // Because we need to generate different file paths for different + // tables, we need to use the physical table ID instead of the + // logical table ID.(Especially when the table is a partitioned table). + TableNameWithPhysicTableID commonType.TableName + // TableInfoVersion is consistent with the version of TableInfo recorded in + // schema storage. It can either be finished ts of a DDL event, + // or be the checkpoint ts when processor is restarted. + TableInfoVersion uint64 +} + +// FilePathGenerator is used to generate data file path and index file path. +type FilePathGenerator struct { + changefeedID commonType.ChangeFeedID + extension string + config *Config + pdClock pdutil.Clock + storage storage.ExternalStorage + fileIndex map[VersionedTableName]*indexWithDate + + hasher *hash.PositionInertia + versionMap map[VersionedTableName]uint64 +} + +// NewFilePathGenerator creates a FilePathGenerator. +func NewFilePathGenerator( + changefeedID commonType.ChangeFeedID, + config *Config, + storage storage.ExternalStorage, + extension string, +) *FilePathGenerator { + pdClock := appcontext.GetService[pdutil.Clock](appcontext.DefaultPDClock) + if pdClock == nil { + pdClock = pdutil.NewMonotonicClock(clock.New()) + log.Warn("pd clock is not set in storage sink, use local clock instead", + zap.String("namespace", changefeedID.Namespace()), + zap.Stringer("changefeedID", changefeedID.ID())) + } + return &FilePathGenerator{ + changefeedID: changefeedID, + config: config, + extension: extension, + storage: storage, + pdClock: pdClock, + fileIndex: make(map[VersionedTableName]*indexWithDate), + hasher: hash.NewPositionInertia(), + versionMap: make(map[VersionedTableName]uint64), + } +} + +// CheckOrWriteSchema checks whether the schema file exists in the storage and +// write scheme.json if necessary. +func (f *FilePathGenerator) CheckOrWriteSchema( + ctx context.Context, + table VersionedTableName, + tableInfo *commonType.TableInfo, +) error { + if _, ok := f.versionMap[table]; ok { + return nil + } + + var def TableDefinition + def.FromTableInfo(tableInfo.GetSchemaName(), tableInfo.GetTableName(), tableInfo, table.TableInfoVersion, f.config.OutputColumnID) + if !def.IsTableSchema() { + // only check schema for table + log.Error("invalid table schema", + zap.String("namespace", f.changefeedID.Namespace()), + zap.Stringer("changefeedID", f.changefeedID.ID()), + zap.Any("versionedTableName", table), + zap.Any("tableInfo", tableInfo)) + return errors.ErrInternalCheckFailed.GenWithStackByArgs("invalid table schema in FilePathGenerator") + } + + // Case 1: point check if the schema file exists. + tblSchemaFile, err := def.GenerateSchemaFilePath() + if err != nil { + return err + } + exist, err := f.storage.FileExists(ctx, tblSchemaFile) + if err != nil { + return err + } + if exist { + f.versionMap[table] = table.TableInfoVersion + return nil + } + + // walk the table meta path to find the last schema file + _, checksum := mustParseSchemaName(tblSchemaFile) + schemaFileCnt := 0 + lastVersion := uint64(0) + subDir := fmt.Sprintf(tableSchemaPrefix, def.Schema, def.Table) + checksumSuffix := fmt.Sprintf("%010d.json", checksum) + err = f.storage.WalkDir(ctx, &storage.WalkOption{ + SubDir: subDir, /* use subDir to prevent walk the whole storage */ + ObjPrefix: subDir + "schema_", + }, func(path string, _ int64) error { + schemaFileCnt++ + if !strings.HasSuffix(path, checksumSuffix) { + return nil + } + version, parsedChecksum := mustParseSchemaName(path) + if parsedChecksum != checksum { + log.Error("invalid schema file name", + zap.String("namespace", f.changefeedID.Namespace()), + zap.Stringer("changefeedID", f.changefeedID.ID()), + zap.String("path", path), zap.Any("checksum", checksum)) + errMsg := fmt.Sprintf("invalid schema filename in storage sink, "+ + "expected checksum: %d, actual checksum: %d", checksum, parsedChecksum) + return errors.ErrInternalCheckFailed.GenWithStackByArgs(errMsg) + } + if version > lastVersion { + lastVersion = version + } + return nil + }) + if err != nil { + return err + } + + // Case 2: the table meta path is not empty. + if schemaFileCnt != 0 && lastVersion != 0 { + f.versionMap[table] = lastVersion + return nil + } + + // Case 3: the table meta path is empty, which happens when: + // a. the table is existed before changefeed started. We need to write schema file to external storage. + // b. the schema file is deleted by the consumer. We write schema file to external storage too. + if schemaFileCnt != 0 && lastVersion == 0 { + log.Warn("no table schema file found in an non-empty meta path", + zap.String("namespace", f.changefeedID.Namespace()), + zap.Stringer("changefeedID", f.changefeedID.ID()), + zap.Any("versionedTableName", table), + zap.Uint32("checksum", checksum)) + } + encodedDetail, err := def.MarshalWithQuery() + if err != nil { + return err + } + f.versionMap[table] = table.TableInfoVersion + return f.storage.WriteFile(ctx, tblSchemaFile, encodedDetail) +} + +// SetClock is used for unit test +func (f *FilePathGenerator) SetClock(pdClock pdutil.Clock) { + f.pdClock = pdClock +} + +// GenerateDateStr generates a date string base on current time +// and the date-separator configuration item. +func (f *FilePathGenerator) GenerateDateStr() string { + var dateStr string + + currTime := f.pdClock.CurrentTime() + // Note: `dateStr` is formatted using local TZ. + switch f.config.DateSeparator { + case config.DateSeparatorYear.String(): + dateStr = currTime.Format("2006") + case config.DateSeparatorMonth.String(): + dateStr = currTime.Format("2006-01") + case config.DateSeparatorDay.String(): + dateStr = currTime.Format("2006-01-02") + default: + } + + return dateStr +} + +// GenerateIndexFilePath generates a canonical path for index file. +func (f *FilePathGenerator) GenerateIndexFilePath(tbl VersionedTableName, date string) string { + dir := f.generateDataDirPath(tbl, date) + name := defaultIndexFileName + return path.Join(dir, name) +} + +// GenerateDataFilePath generates a canonical path for data file. +func (f *FilePathGenerator) GenerateDataFilePath( + ctx context.Context, tbl VersionedTableName, date string, +) (string, error) { + dir := f.generateDataDirPath(tbl, date) + name, err := f.generateDataFileName(ctx, tbl, date) + if err != nil { + return "", err + } + return path.Join(dir, name), nil +} + +func (f *FilePathGenerator) generateDataDirPath(tbl VersionedTableName, date string) string { + var elems []string + + elems = append(elems, tbl.TableNameWithPhysicTableID.Schema) + elems = append(elems, tbl.TableNameWithPhysicTableID.Table) + elems = append(elems, fmt.Sprintf("%d", f.versionMap[tbl])) + + if f.config.EnablePartitionSeparator && tbl.TableNameWithPhysicTableID.IsPartition { + elems = append(elems, fmt.Sprintf("%d", tbl.TableNameWithPhysicTableID.TableID)) + } + + if len(date) != 0 { + elems = append(elems, date) + } + + return path.Join(elems...) +} + +func (f *FilePathGenerator) generateDataFileName( + ctx context.Context, tbl VersionedTableName, date string, +) (string, error) { + if idx, ok := f.fileIndex[tbl]; !ok { + fileIdx, err := f.getNextFileIdxFromIndexFile(ctx, tbl, date) + if err != nil { + return "", err + } + f.fileIndex[tbl] = &indexWithDate{ + prevDate: date, + currDate: date, + index: fileIdx, + } + } else { + idx.currDate = date + } + + // if date changed, reset the counter + if f.fileIndex[tbl].prevDate != f.fileIndex[tbl].currDate { + f.fileIndex[tbl].prevDate = f.fileIndex[tbl].currDate + f.fileIndex[tbl].index = 0 + } + f.fileIndex[tbl].index++ + return generateDataFileName(f.fileIndex[tbl].index, f.extension, f.config.FileIndexWidth), nil +} + +func (f *FilePathGenerator) getNextFileIdxFromIndexFile( + ctx context.Context, tbl VersionedTableName, date string, +) (uint64, error) { + indexFile := f.GenerateIndexFilePath(tbl, date) + exist, err := f.storage.FileExists(ctx, indexFile) + if err != nil { + return 0, err + } + if !exist { + return 0, nil + } + + data, err := f.storage.ReadFile(ctx, indexFile) + if err != nil { + return 0, err + } + fileName := strings.TrimSuffix(string(data), "\n") + maxFileIdx, err := f.fetchIndexFromFileName(fileName) + if err != nil { + return 0, err + } + + lastFilePath := path.Join( + f.generateDataDirPath(tbl, date), // file dir + generateDataFileName(maxFileIdx, f.extension, f.config.FileIndexWidth), // file name + ) + var lastFileExists, lastFileIsEmpty bool + lastFileExists, err = f.storage.FileExists(ctx, lastFilePath) + if err != nil { + return 0, err + } + + if lastFileExists { + fileReader, err := f.storage.Open(ctx, lastFilePath, nil) + if err != nil { + return 0, err + } + readBytes, err := fileReader.Read(make([]byte, 1)) + if err != nil && err != io.EOF { + return 0, err + } + lastFileIsEmpty = readBytes == 0 + if err := fileReader.Close(); err != nil { + return 0, err + } + } + + var fileIdx uint64 + if lastFileExists && !lastFileIsEmpty { + fileIdx = maxFileIdx + } else { + // Reuse the old index number if the last file does not exist. + fileIdx = maxFileIdx - 1 + } + return fileIdx, nil +} + +func (f *FilePathGenerator) fetchIndexFromFileName(fileName string) (uint64, error) { + var fileIdx uint64 + var err error + + if len(fileName) < minFileNamePrefixLen+len(f.extension) || + !strings.HasPrefix(fileName, "CDC") || + !strings.HasSuffix(fileName, f.extension) { + return 0, errors.WrapError(errors.ErrStorageSinkInvalidFileName, + fmt.Errorf("'%s' is a invalid file name", fileName)) + } + + extIdx := strings.Index(fileName, f.extension) + fileIdxStr := fileName[3:extIdx] + if fileIdx, err = strconv.ParseUint(fileIdxStr, 10, 64); err != nil { + return 0, errors.WrapError(errors.ErrStorageSinkInvalidFileName, err) + } + + return fileIdx, nil +} + +var dateSeparatorDayRegexp *regexp.Regexp + +// RemoveExpiredFiles removes expired files from external storage. +func RemoveExpiredFiles( + ctx context.Context, + _ commonType.ChangeFeedID, + storage storage.ExternalStorage, + cfg *Config, + checkpointTs model.Ts, +) (uint64, error) { + if cfg.DateSeparator != config.DateSeparatorDay.String() { + return 0, nil + } + if dateSeparatorDayRegexp == nil { + dateSeparatorDayRegexp = regexp.MustCompile(config.DateSeparatorDay.GetPattern()) + } + + ttl := time.Duration(cfg.FileExpirationDays) * time.Hour * 24 + currTime := oracle.GetTimeFromTS(checkpointTs).Add(-ttl) + // Note: `expiredDate` is formatted using local TZ. + expiredDate := currTime.Format("2006-01-02") + + cnt := uint64(0) + err := util.RemoveFilesIf(ctx, storage, func(path string) bool { + // the path is like: /
////CDC{num}.extension + match := dateSeparatorDayRegexp.FindString(path) + if match != "" && match < expiredDate { + cnt++ + return true + } + return false + }, nil) + return cnt, err +} + +// RemoveEmptyDirs removes empty directories from external storage. +func RemoveEmptyDirs( + ctx context.Context, + id commonType.ChangeFeedID, + target string, +) (uint64, error) { + cnt := uint64(0) + err := filepath.Walk(target, func(path string, info fs.FileInfo, err error) error { + if os.IsNotExist(err) || path == target || info == nil { + // if path not exists, we should return nil to continue. + return nil + } + if err != nil { + return err + } + if info.IsDir() { + files, err := os.ReadDir(path) + if err == nil && len(files) == 0 { + log.Debug("Deleting empty directory", + zap.String("namespace", id.Namespace()), + zap.Stringer("changeFeedID", id.ID()), + zap.String("path", path)) + os.Remove(path) + cnt++ + return filepath.SkipDir + } + } + return nil + }) + + return cnt, err +} diff --git a/pkg/sink/cloudstorage/path_key.go b/pkg/sink/cloudstorage/path_key.go new file mode 100644 index 00000000..e18636f1 --- /dev/null +++ b/pkg/sink/cloudstorage/path_key.go @@ -0,0 +1,156 @@ +// Copyright 2023 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package cloudstorage + +import ( + "fmt" + "regexp" + "strconv" + "strings" + + "github.com/pingcap/tiflow/pkg/config" + "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/quotes" +) + +// SchemaPathKey is the key of schema path. +type SchemaPathKey struct { + Schema string + Table string + TableVersion uint64 +} + +// GetKey returns the key of schema path. +func (s *SchemaPathKey) GetKey() string { + return quotes.QuoteSchema(s.Schema, s.Table) +} + +// ParseSchemaFilePath parses the schema file path and returns the table version and checksum. +func (s *SchemaPathKey) ParseSchemaFilePath(path string) (uint32, error) { + // For /
/meta/schema_{tableVersion}_{checksum}.json, the parts + // should be ["", "
", "meta", "schema_{tableVersion}_{checksum}.json"]. + matches := strings.Split(path, "/") + + var schema, table string + schema = matches[0] + switch len(matches) { + case 3: + table = "" + case 4: + table = matches[1] + default: + return 0, errors.Trace(fmt.Errorf("cannot match schema path pattern for %s", path)) + } + + if matches[len(matches)-2] != "meta" { + return 0, errors.Trace(fmt.Errorf("cannot match schema path pattern for %s", path)) + } + + schemaFileName := matches[len(matches)-1] + version, checksum := mustParseSchemaName(schemaFileName) + + *s = SchemaPathKey{ + Schema: schema, + Table: table, + TableVersion: version, + } + return checksum, nil +} + +// DmlPathKey is the key of dml path. +type DmlPathKey struct { + SchemaPathKey + PartitionNum int64 + Date string +} + +// GenerateDMLFilePath generates the dml file path. +func (d *DmlPathKey) GenerateDMLFilePath( + idx uint64, extension string, fileIndexWidth int, +) string { + var elems []string + + elems = append(elems, d.Schema) + elems = append(elems, d.Table) + elems = append(elems, fmt.Sprintf("%d", d.TableVersion)) + + if d.PartitionNum != 0 { + elems = append(elems, fmt.Sprintf("%d", d.PartitionNum)) + } + if len(d.Date) != 0 { + elems = append(elems, d.Date) + } + elems = append(elems, generateDataFileName(idx, extension, fileIndexWidth)) + + return strings.Join(elems, "/") +} + +// ParseDMLFilePath parses the dml file path and returns the max file index. +// DML file path pattern is as follows: +// {schema}/{table}/{table-version-separator}/{partition-separator}/{date-separator}/, where +// partition-separator and date-separator could be empty. +// DML file name pattern is as follows: CDC{num}.extension. +func (d *DmlPathKey) ParseDMLFilePath(dateSeparator, path string) (uint64, error) { + var partitionNum int64 + + str := `(\w+)\/(\w+)\/(\d+)\/(\d+)?\/*` + switch dateSeparator { + case config.DateSeparatorNone.String(): + str += `(\d{4})*` + case config.DateSeparatorYear.String(): + str += `(\d{4})\/` + case config.DateSeparatorMonth.String(): + str += `(\d{4}-\d{2})\/` + case config.DateSeparatorDay.String(): + str += `(\d{4}-\d{2}-\d{2})\/` + } + str += `CDC(\d+).\w+` + pathRE, err := regexp.Compile(str) + if err != nil { + return 0, err + } + + matches := pathRE.FindStringSubmatch(path) + if len(matches) != 7 { + return 0, fmt.Errorf("cannot match dml path pattern for %s", path) + } + + version, err := strconv.ParseUint(matches[3], 10, 64) + if err != nil { + return 0, err + } + + if len(matches[4]) > 0 { + partitionNum, err = strconv.ParseInt(matches[4], 10, 64) + if err != nil { + return 0, err + } + } + fileIdx, err := strconv.ParseUint(strings.TrimLeft(matches[6], "0"), 10, 64) + if err != nil { + return 0, err + } + + *d = DmlPathKey{ + SchemaPathKey: SchemaPathKey{ + Schema: matches[1], + Table: matches[2], + TableVersion: version, + }, + PartitionNum: partitionNum, + Date: matches[5], + } + + return fileIdx, nil +} diff --git a/pkg/sink/cloudstorage/path_key_test.go b/pkg/sink/cloudstorage/path_key_test.go new file mode 100644 index 00000000..a78e91d1 --- /dev/null +++ b/pkg/sink/cloudstorage/path_key_test.go @@ -0,0 +1,96 @@ +// Copyright 2023 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package cloudstorage + +import ( + "testing" + + "github.com/stretchr/testify/require" +) + +func TestSchemaPathKey(t *testing.T) { + t.Parallel() + + testCases := []struct { + path string + schemakey SchemaPathKey + checksum uint32 + }{ + // Test for database schema path: /meta/schema_{tableVersion}_{checksum}.json + { + path: "test_schema/meta/schema_1_2.json", + schemakey: SchemaPathKey{ + Schema: "test_schema", + Table: "", + TableVersion: 1, + }, + checksum: 2, + }, + // Test for table schema path: /
/meta/schema_{tableVersion}_{checksum}.json + { + path: "test_schema/test_table/meta/schema_11_22.json", + schemakey: SchemaPathKey{ + Schema: "test_schema", + Table: "test_table", + TableVersion: 11, + }, + checksum: 22, + }, + } + for _, tc := range testCases { + var schemaKey SchemaPathKey + checksum, err := schemaKey.ParseSchemaFilePath(tc.path) + require.NoError(t, err) + require.Equal(t, tc.schemakey, schemaKey) + require.Equal(t, tc.checksum, checksum) + } +} + +func TestDmlPathKey(t *testing.T) { + t.Parallel() + + testCases := []struct { + index int + fileIndexWidth int + extension string + path string + dmlkey DmlPathKey + }{ + { + index: 10, + fileIndexWidth: 20, + extension: ".csv", + path: "schema1/table1/123456/2023-05-09/CDC00000000000000000010.csv", + dmlkey: DmlPathKey{ + SchemaPathKey: SchemaPathKey{ + Schema: "schema1", + Table: "table1", + TableVersion: 123456, + }, + PartitionNum: 0, + Date: "2023-05-09", + }, + }, + } + + for _, tc := range testCases { + var dmlkey DmlPathKey + idx, err := dmlkey.ParseDMLFilePath("day", tc.path) + require.NoError(t, err) + require.Equal(t, tc.dmlkey, dmlkey) + + fileName := dmlkey.GenerateDMLFilePath(idx, tc.extension, tc.fileIndexWidth) + require.Equal(t, tc.path, fileName) + } +} diff --git a/pkg/sink/cloudstorage/path_test.go b/pkg/sink/cloudstorage/path_test.go new file mode 100644 index 00000000..bb0609e2 --- /dev/null +++ b/pkg/sink/cloudstorage/path_test.go @@ -0,0 +1,433 @@ +// Copyright 2023 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package cloudstorage + +import ( + "context" + "fmt" + "net/url" + "os" + "path/filepath" + "testing" + "time" + + "github.com/google/uuid" + commonType "github.com/pingcap/ticdc/pkg/common" + appcontext "github.com/pingcap/ticdc/pkg/common/context" + "github.com/pingcap/ticdc/pkg/config" + "github.com/pingcap/ticdc/pkg/pdutil" + timodel "github.com/pingcap/tidb/pkg/meta/model" + pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/mysql" + "github.com/pingcap/tidb/pkg/parser/types" + "github.com/pingcap/tiflow/engine/pkg/clock" + "github.com/pingcap/tiflow/pkg/util" + "github.com/stretchr/testify/require" + "github.com/tikv/client-go/v2/oracle" +) + +func testFilePathGenerator(ctx context.Context, t *testing.T, dir string) *FilePathGenerator { + uri := fmt.Sprintf("file:///%s?flush-interval=2s", dir) + storage, err := util.GetExternalStorageFromURI(ctx, uri) + require.NoError(t, err) + + sinkURI, err := url.Parse(uri) + require.NoError(t, err) + replicaConfig := config.GetDefaultReplicaConfig() + replicaConfig.Sink.DateSeparator = util.AddressOf(config.DateSeparatorNone.String()) + replicaConfig.Sink.Protocol = util.AddressOf(config.ProtocolOpen.String()) + replicaConfig.Sink.FileIndexWidth = util.AddressOf(6) + cfg := NewConfig() + err = cfg.Apply(ctx, sinkURI, replicaConfig.Sink) + require.NoError(t, err) + + mockPDClock := pdutil.NewClock4Test() + appcontext.SetService(appcontext.DefaultPDClock, mockPDClock) + f := NewFilePathGenerator(commonType.ChangeFeedID{}, cfg, storage, ".json") + return f +} + +func TestGenerateDataFilePath(t *testing.T) { + t.Parallel() + ctx, cancel := context.WithCancel(context.TODO()) + defer cancel() + + table := VersionedTableName{ + TableNameWithPhysicTableID: commonType.TableName{ + Schema: "test", + Table: "table1", + }, + TableInfoVersion: 5, + } + + dir := t.TempDir() + f := testFilePathGenerator(ctx, t, dir) + f.versionMap[table] = table.TableInfoVersion + date := f.GenerateDateStr() + // date-separator: none + path, err := f.GenerateDataFilePath(ctx, table, date) + require.NoError(t, err) + require.Equal(t, "test/table1/5/CDC000001.json", path) + path, err = f.GenerateDataFilePath(ctx, table, date) + require.NoError(t, err) + require.Equal(t, "test/table1/5/CDC000002.json", path) + + // date-separator: year + mockClock := clock.NewMock() + f = testFilePathGenerator(ctx, t, dir) + f.versionMap[table] = table.TableInfoVersion + f.config.DateSeparator = config.DateSeparatorYear.String() + f.SetClock(pdutil.NewMonotonicClock(mockClock)) + mockClock.Set(time.Date(2022, 12, 31, 23, 59, 59, 0, time.UTC)) + date = f.GenerateDateStr() + path, err = f.GenerateDataFilePath(ctx, table, date) + require.NoError(t, err) + require.Equal(t, "test/table1/5/2022/CDC000001.json", path) + path, err = f.GenerateDataFilePath(ctx, table, date) + require.NoError(t, err) + require.Equal(t, "test/table1/5/2022/CDC000002.json", path) + // year changed + mockClock.Set(time.Date(2023, 1, 1, 0, 0, 20, 0, time.UTC)) + date = f.GenerateDateStr() + path, err = f.GenerateDataFilePath(ctx, table, date) + require.NoError(t, err) + require.Equal(t, "test/table1/5/2023/CDC000001.json", path) + path, err = f.GenerateDataFilePath(ctx, table, date) + require.NoError(t, err) + require.Equal(t, "test/table1/5/2023/CDC000002.json", path) + + // date-separator: month + mockClock = clock.NewMock() + f = testFilePathGenerator(ctx, t, dir) + f.versionMap[table] = table.TableInfoVersion + f.config.DateSeparator = config.DateSeparatorMonth.String() + f.SetClock(pdutil.NewMonotonicClock(mockClock)) + + mockClock.Set(time.Date(2022, 12, 31, 23, 59, 59, 0, time.UTC)) + date = f.GenerateDateStr() + path, err = f.GenerateDataFilePath(ctx, table, date) + require.NoError(t, err) + require.Equal(t, "test/table1/5/2022-12/CDC000001.json", path) + path, err = f.GenerateDataFilePath(ctx, table, date) + require.NoError(t, err) + require.Equal(t, "test/table1/5/2022-12/CDC000002.json", path) + // month changed + mockClock.Set(time.Date(2023, 1, 1, 0, 0, 20, 0, time.UTC)) + date = f.GenerateDateStr() + path, err = f.GenerateDataFilePath(ctx, table, date) + require.NoError(t, err) + require.Equal(t, "test/table1/5/2023-01/CDC000001.json", path) + path, err = f.GenerateDataFilePath(ctx, table, date) + require.NoError(t, err) + require.Equal(t, "test/table1/5/2023-01/CDC000002.json", path) + + // date-separator: day + mockClock = clock.NewMock() + f = testFilePathGenerator(ctx, t, dir) + f.versionMap[table] = table.TableInfoVersion + f.config.DateSeparator = config.DateSeparatorDay.String() + f.SetClock(pdutil.NewMonotonicClock(mockClock)) + + mockClock.Set(time.Date(2022, 12, 31, 23, 59, 59, 0, time.UTC)) + date = f.GenerateDateStr() + path, err = f.GenerateDataFilePath(ctx, table, date) + require.NoError(t, err) + require.Equal(t, "test/table1/5/2022-12-31/CDC000001.json", path) + path, err = f.GenerateDataFilePath(ctx, table, date) + require.NoError(t, err) + require.Equal(t, "test/table1/5/2022-12-31/CDC000002.json", path) + // day changed + mockClock.Set(time.Date(2023, 1, 1, 0, 0, 20, 0, time.UTC)) + date = f.GenerateDateStr() + path, err = f.GenerateDataFilePath(ctx, table, date) + require.NoError(t, err) + require.Equal(t, "test/table1/5/2023-01-01/CDC000001.json", path) + path, err = f.GenerateDataFilePath(ctx, table, date) + require.NoError(t, err) + require.Equal(t, "test/table1/5/2023-01-01/CDC000002.json", path) +} + +func TestFetchIndexFromFileName(t *testing.T) { + t.Parallel() + + ctx, cancel := context.WithCancel(context.TODO()) + defer cancel() + + dir := t.TempDir() + f := testFilePathGenerator(ctx, t, dir) + testCases := []struct { + fileName string + wantErr string + }{ + { + fileName: "CDC000011.json", + wantErr: "", + }, + { + fileName: "CDC1000000.json", + wantErr: "", + }, + { + fileName: "CDC1.json", + wantErr: "filename in storage sink is invalid", + }, + { + fileName: "cdc000001.json", + wantErr: "filename in storage sink is invalid", + }, + { + fileName: "CDC000005.xxx", + wantErr: "filename in storage sink is invalid", + }, + { + fileName: "CDChello.json", + wantErr: "filename in storage sink is invalid", + }, + } + + for _, tc := range testCases { + _, err := f.fetchIndexFromFileName(tc.fileName) + if len(tc.wantErr) != 0 { + require.Contains(t, err.Error(), tc.wantErr) + } else { + require.NoError(t, err) + } + } +} + +func TestGenerateDataFilePathWithIndexFile(t *testing.T) { + t.Parallel() + + ctx, cancel := context.WithCancel(context.TODO()) + defer cancel() + + dir := t.TempDir() + f := testFilePathGenerator(ctx, t, dir) + mockClock := clock.NewMock() + f.config.DateSeparator = config.DateSeparatorDay.String() + f.SetClock(pdutil.NewMonotonicClock(mockClock)) + + mockClock.Set(time.Date(2023, 3, 9, 23, 59, 59, 0, time.UTC)) + table := VersionedTableName{ + TableNameWithPhysicTableID: commonType.TableName{ + Schema: "test", + Table: "table1", + }, + TableInfoVersion: 5, + } + f.versionMap[table] = table.TableInfoVersion + date := f.GenerateDateStr() + indexFilePath := f.GenerateIndexFilePath(table, date) + err := f.storage.WriteFile(ctx, indexFilePath, []byte("CDC000005.json\n")) + require.NoError(t, err) + + // index file exists, but the file is not exist + dataFilePath, err := f.GenerateDataFilePath(ctx, table, date) + require.NoError(t, err) + require.Equal(t, "test/table1/5/2023-03-09/CDC000005.json", dataFilePath) + + // cleanup cached file index + delete(f.fileIndex, table) + // index file exists, and the file is empty + err = f.storage.WriteFile(ctx, dataFilePath, []byte("")) + require.NoError(t, err) + dataFilePath, err = f.GenerateDataFilePath(ctx, table, date) + require.NoError(t, err) + require.Equal(t, "test/table1/5/2023-03-09/CDC000005.json", dataFilePath) + + // cleanup cached file index + delete(f.fileIndex, table) + // index file exists, and the file is not empty + err = f.storage.WriteFile(ctx, dataFilePath, []byte("test")) + require.NoError(t, err) + dataFilePath, err = f.GenerateDataFilePath(ctx, table, date) + require.NoError(t, err) + require.Equal(t, "test/table1/5/2023-03-09/CDC000006.json", dataFilePath) +} + +func TestIsSchemaFile(t *testing.T) { + t.Parallel() + + tests := []struct { + name string + path string + expect bool + }{ + { + "valid database schema /meta/", + "schema2/meta/schema_123_0123456789.json", true, + }, + { + "valid table schema /
/meta/", + "schema1/table1/meta/schema_123_0123456789.json", true, + }, + {"valid special prefix", "meta/meta/schema_123_0123456789.json", true}, + {"valid schema1", "meta/schema_123_0123456789.json", true}, + {"missing field1", "meta/schema_012345678_.json", false}, + {"missing field2", "meta/schema_012345678.json", false}, + {"invalid checksum1", "meta/schema_123_012345678.json", false}, + {"invalid checksum2", "meta/schema_123_012a4567c9.json", false}, + {"invalid table version", "meta/schema_abc_0123456789.json", false}, + {"invalid extension1", "meta/schema_123_0123456789.txt", false}, + {"invalid extension2", "meta/schema_123_0123456789.json ", false}, + {"invalid path", "meta/schema1/schema_123_0123456789.json", false}, + } + + for _, tt := range tests { + require.Equal(t, tt.expect, IsSchemaFile(tt.path), + "testCase: %s, path: %v", tt.name, tt.path) + } +} + +func TestCheckOrWriteSchema(t *testing.T) { + t.Parallel() + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + dir := t.TempDir() + f := testFilePathGenerator(ctx, t, dir) + + var columns []*timodel.ColumnInfo + ft := types.NewFieldType(mysql.TypeLong) + ft.SetFlag(mysql.PriKeyFlag | mysql.NotNullFlag) + col := &timodel.ColumnInfo{ + Name: pmodel.NewCIStr("Id"), + FieldType: *ft, + DefaultValue: 10, + } + columns = append(columns, col) + tableInfo := commonType.WrapTableInfo(101, "test", &timodel.TableInfo{Columns: columns}) + + table := VersionedTableName{ + TableNameWithPhysicTableID: tableInfo.TableName, + TableInfoVersion: 100, + } + + err := f.CheckOrWriteSchema(ctx, table, tableInfo) + require.NoError(t, err) + require.Equal(t, table.TableInfoVersion, f.versionMap[table]) + + // test only table version changed, schema file should be reused + table.TableInfoVersion = 101 + err = f.CheckOrWriteSchema(ctx, table, tableInfo) + require.NoError(t, err) + require.Equal(t, table.TableInfoVersion, f.versionMap[table]) + + dir = filepath.Join(dir, "test/table1/meta") + files, err := os.ReadDir(dir) + require.NoError(t, err) + require.Equal(t, 1, len(files)) + + // test schema file is invalid + err = os.WriteFile(filepath.Join(dir, + fmt.Sprintf("%s.tmp.%s", files[0].Name(), uuid.NewString())), + []byte("invalid"), 0o644) + require.NoError(t, err) + err = os.Remove(filepath.Join(dir, files[0].Name())) + require.NoError(t, err) + delete(f.versionMap, table) + err = f.CheckOrWriteSchema(ctx, table, tableInfo) + require.NoError(t, err) + require.Equal(t, table.TableInfoVersion, f.versionMap[table]) + + files, err = os.ReadDir(dir) + require.NoError(t, err) + require.Equal(t, 2, len(files)) +} + +func TestRemoveExpiredFilesWithoutPartition(t *testing.T) { + t.Parallel() + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + dir := t.TempDir() + uri := fmt.Sprintf("file:///%s?flush-interval=2s", dir) + storage, err := util.GetExternalStorageFromURI(ctx, uri) + require.NoError(t, err) + sinkURI, err := url.Parse(uri) + require.NoError(t, err) + replicaConfig := config.GetDefaultReplicaConfig() + replicaConfig.Sink.DateSeparator = util.AddressOf(config.DateSeparatorDay.String()) + replicaConfig.Sink.Protocol = util.AddressOf(config.ProtocolCsv.String()) + replicaConfig.Sink.FileIndexWidth = util.AddressOf(6) + replicaConfig.Sink.CloudStorageConfig = &config.CloudStorageConfig{ + FileExpirationDays: util.AddressOf(1), + FileCleanupCronSpec: util.AddressOf("* * * * * *"), + } + cfg := NewConfig() + err = cfg.Apply(ctx, sinkURI, replicaConfig.Sink) + require.NoError(t, err) + + // generate some expired files + filesWithoutPartition := []string{ + // schma1-table1 + "schema1/table1/5/2021-01-01/CDC000001.csv", + "schema1/table1/5/2021-01-01/CDC000002.csv", + "schema1/table1/5/2021-01-01/CDC000003.csv", + "schema1/table1/5/2021-01-01/" + defaultIndexFileName, // index + "schema1/table1/meta/schema_5_20210101.json", // schema should never be cleaned + // schma1-table2 + "schema1/table2/5/2021-01-01/CDC000001.csv", + "schema1/table2/5/2021-01-01/CDC000002.csv", + "schema1/table2/5/2021-01-01/CDC000003.csv", + "schema1/table2/5/2021-01-01/" + defaultIndexFileName, // index + "schema1/table2/meta/schema_5_20210101.json", // schema should never be cleaned + } + for _, file := range filesWithoutPartition { + err := storage.WriteFile(ctx, file, []byte("test")) + require.NoError(t, err) + } + + filesWithPartition := []string{ + // schma1-table1 + "schema1/table1/400200133/12/2021-01-01/20210101/CDC000001.csv", + "schema1/table1/400200133/12/2021-01-01/20210101/CDC000002.csv", + "schema1/table1/400200133/12/2021-01-01/20210101/CDC000003.csv", + "schema1/table1/400200133/12/2021-01-01/20210101/" + defaultIndexFileName, // index + "schema1/table1/meta/schema_5_20210101.json", // schema should never be cleaned + // schma2-table1 + "schema2/table1/400200150/12/2021-01-01/20210101/CDC000001.csv", + "schema2/table1/400200150/12/2021-01-01/20210101/CDC000002.csv", + "schema2/table1/400200150/12/2021-01-01/20210101/CDC000003.csv", + "schema2/table1/400200150/12/2021-01-01/20210101/" + defaultIndexFileName, // index + "schema2/table1/meta/schema_5_20210101.json", // schema should never be cleaned + } + for _, file := range filesWithPartition { + err := storage.WriteFile(ctx, file, []byte("test")) + require.NoError(t, err) + } + + filesNotExpired := []string{ + // schma1-table1 + "schema1/table1/5/2021-01-02/CDC000001.csv", + "schema1/table1/5/2021-01-02/CDC000002.csv", + "schema1/table1/5/2021-01-02/CDC000003.csv", + "schema1/table1/5/2021-01-02/" + defaultIndexFileName, // index + // schma1-table2 + "schema1/table2/5/2021-01-02/CDC000001.csv", + "schema1/table2/5/2021-01-02/CDC000002.csv", + "schema1/table2/5/2021-01-02/CDC000003.csv", + "schema1/table2/5/2021-01-02/" + defaultIndexFileName, // index + } + for _, file := range filesNotExpired { + err := storage.WriteFile(ctx, file, []byte("test")) + require.NoError(t, err) + } + + currTime := time.Date(2021, 1, 3, 0, 0, 0, 0, time.Local) + checkpointTs := oracle.GoTimeToTS(currTime) + cnt, err := RemoveExpiredFiles(ctx, commonType.ChangeFeedID{}, storage, cfg, checkpointTs) + require.NoError(t, err) + require.Equal(t, uint64(16), cnt) +} diff --git a/pkg/sink/cloudstorage/table_definition.go b/pkg/sink/cloudstorage/table_definition.go new file mode 100644 index 00000000..9ed1f6e9 --- /dev/null +++ b/pkg/sink/cloudstorage/table_definition.go @@ -0,0 +1,343 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. +package cloudstorage + +import ( + "encoding/json" + "sort" + "strconv" + "strings" + + "github.com/pingcap/log" + "github.com/pingcap/ticdc/pkg/common" + commonEvent "github.com/pingcap/ticdc/pkg/common/event" + timodel "github.com/pingcap/tidb/pkg/meta/model" + "github.com/pingcap/tidb/pkg/parser/charset" + pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/mysql" + "github.com/pingcap/tidb/pkg/parser/types" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/hash" + "go.uber.org/zap" +) + +const ( + defaultTableDefinitionVersion = 1 + marshalPrefix = "" + marshalIndent = " " +) + +// TableCol denotes the column info for a table definition. +type TableCol struct { + ID string `json:"ColumnId,omitempty"` + Name string `json:"ColumnName" ` + Tp string `json:"ColumnType"` + Default interface{} `json:"ColumnDefault,omitempty"` + Precision string `json:"ColumnPrecision,omitempty"` + Scale string `json:"ColumnScale,omitempty"` + Nullable string `json:"ColumnNullable,omitempty"` + IsPK string `json:"ColumnIsPk,omitempty"` +} + +// FromTiColumnInfo converts from TiDB ColumnInfo to TableCol. +func (t *TableCol) FromTiColumnInfo(col *timodel.ColumnInfo, outputColumnID bool) { + defaultFlen, defaultDecimal := mysql.GetDefaultFieldLengthAndDecimal(col.GetType()) + isDecimalNotDefault := col.GetDecimal() != defaultDecimal && + col.GetDecimal() != 0 && + col.GetDecimal() != types.UnspecifiedLength + + displayFlen, displayDecimal := col.GetFlen(), col.GetDecimal() + if displayFlen == types.UnspecifiedLength { + displayFlen = defaultFlen + } + if displayDecimal == types.UnspecifiedLength { + displayDecimal = defaultDecimal + } + + if outputColumnID { + t.ID = strconv.FormatInt(col.ID, 10) + } + t.Name = col.Name.O + t.Tp = strings.ToUpper(types.TypeToStr(col.GetType(), col.GetCharset())) + if mysql.HasUnsignedFlag(col.GetFlag()) { + t.Tp += " UNSIGNED" + } + if mysql.HasPriKeyFlag(col.GetFlag()) { + t.IsPK = "true" + } + if mysql.HasNotNullFlag(col.GetFlag()) { + t.Nullable = "false" + } + t.Default = model.GetColumnDefaultValue(col) + + switch col.GetType() { + case mysql.TypeTimestamp, mysql.TypeDatetime, mysql.TypeDuration: + if isDecimalNotDefault { + t.Scale = strconv.Itoa(displayDecimal) + } + case mysql.TypeDouble, mysql.TypeFloat: + t.Precision = strconv.Itoa(displayFlen) + if isDecimalNotDefault { + t.Scale = strconv.Itoa(displayDecimal) + } + case mysql.TypeNewDecimal: + t.Precision = strconv.Itoa(displayFlen) + t.Scale = strconv.Itoa(displayDecimal) + case mysql.TypeTiny, mysql.TypeShort, mysql.TypeInt24, mysql.TypeLong, mysql.TypeLonglong, + mysql.TypeBit, mysql.TypeVarchar, mysql.TypeString, mysql.TypeVarString, mysql.TypeBlob, + mysql.TypeTinyBlob, mysql.TypeMediumBlob, mysql.TypeLongBlob: + t.Precision = strconv.Itoa(displayFlen) + case mysql.TypeYear: + t.Precision = strconv.Itoa(displayFlen) + } +} + +// ToTiColumnInfo converts from TableCol to TiDB ColumnInfo. +func (t *TableCol) ToTiColumnInfo(colID int64) (*timodel.ColumnInfo, error) { + col := new(timodel.ColumnInfo) + + if t.ID != "" { + var err error + col.ID, err = strconv.ParseInt(t.ID, 10, 64) + if err != nil { + return nil, errors.Trace(err) + } + } + + col.ID = colID + col.Name = pmodel.NewCIStr(t.Name) + tp := types.StrToType(strings.ToLower(strings.TrimSuffix(t.Tp, " UNSIGNED"))) + col.FieldType = *types.NewFieldType(tp) + if strings.Contains(t.Tp, "UNSIGNED") { + col.AddFlag(mysql.UnsignedFlag) + } + if t.IsPK == "true" { + col.AddFlag(mysql.PriKeyFlag) + } + if t.Nullable == "false" { + col.AddFlag(mysql.NotNullFlag) + } + col.DefaultValue = t.Default + if strings.Contains(t.Tp, "BLOB") || strings.Contains(t.Tp, "BINARY") { + col.SetCharset(charset.CharsetBin) + } else { + col.SetCharset(charset.CharsetUTF8MB4) + } + setFlen := func(precision string) error { + if len(precision) > 0 { + flen, err := strconv.Atoi(precision) + if err != nil { + return errors.Trace(err) + } + col.SetFlen(flen) + } + return nil + } + setDecimal := func(scale string) error { + if len(scale) > 0 { + decimal, err := strconv.Atoi(scale) + if err != nil { + return errors.Trace(err) + } + col.SetDecimal(decimal) + } + return nil + } + switch col.GetType() { + case mysql.TypeTimestamp, mysql.TypeDatetime, mysql.TypeDuration: + err := setDecimal(t.Scale) + if err != nil { + return nil, errors.Trace(err) + } + case mysql.TypeDouble, mysql.TypeFloat, mysql.TypeNewDecimal: + err := setFlen(t.Precision) + if err != nil { + return nil, errors.Trace(err) + } + err = setDecimal(t.Scale) + if err != nil { + return nil, errors.Trace(err) + } + case mysql.TypeTiny, mysql.TypeShort, mysql.TypeInt24, mysql.TypeLong, mysql.TypeLonglong, + mysql.TypeBit, mysql.TypeVarchar, mysql.TypeString, mysql.TypeVarString, mysql.TypeBlob, + mysql.TypeTinyBlob, mysql.TypeMediumBlob, mysql.TypeLongBlob, mysql.TypeYear: + err := setFlen(t.Precision) + if err != nil { + return nil, errors.Trace(err) + } + } + + return col, nil +} + +// TableDefinition is the detailed table definition used for cloud storage sink. +// TODO: find a better name for this struct. +type TableDefinition struct { + Table string `json:"Table"` + Schema string `json:"Schema"` + Version uint64 `json:"Version"` + TableVersion uint64 `json:"TableVersion"` + Query string `json:"Query"` + Type byte `json:"Type"` + Columns []TableCol `json:"TableColumns"` + TotalColumns int `json:"TableColumnsTotal"` +} + +// tableDefWithoutQuery is the table definition without query, which ignores the +// Query, Type and TableVersion field. +type tableDefWithoutQuery struct { + Table string `json:"Table"` + Schema string `json:"Schema"` + Version uint64 `json:"Version"` + Columns []TableCol `json:"TableColumns"` + TotalColumns int `json:"TableColumnsTotal"` +} + +// FromDDLEvent converts from DDLEvent to TableDefinition. +func (t *TableDefinition) FromDDLEvent(event *commonEvent.DDLEvent, outputColumnID bool) { + // if event.GetCommitTs() != event.TableInfo.UpdateTS() { + // log.Warn("commit ts and table info version should be equal", + // zap.Uint64("tableInfoVersion", event.TableInfo.UpdateTS()), + // zap.Any("commitTs", event.GetCommitTs()), + // zap.Any("tableInfo", event.TableInfo), + // ) + // } + t.FromTableInfo(event.SchemaName, event.TableName, event.TableInfo, event.GetCommitTs(), outputColumnID) + t.Query = event.Query + t.Type = event.Type +} + +// ToDDLEvent converts from TableDefinition to DDLEvent. +func (t *TableDefinition) ToDDLEvent() (*commonEvent.DDLEvent, error) { + tableInfo, err := t.ToTableInfo() + if err != nil { + return nil, err + } + return &commonEvent.DDLEvent{ + TableInfo: tableInfo, + FinishedTs: t.TableVersion, + Type: t.Type, + Query: t.Query, + }, nil +} + +// FromTableInfo converts from TableInfo to TableDefinition. +func (t *TableDefinition) FromTableInfo( + schemaName string, tableName string, info *common.TableInfo, tableInfoVersion uint64, outputColumnID bool, +) { + t.Version = defaultTableDefinitionVersion + t.TableVersion = tableInfoVersion + + t.Schema = schemaName + t.Table = tableName + if info == nil { + return + } + t.TotalColumns = len(info.GetColumns()) + for _, col := range info.GetColumns() { + var tableCol TableCol + tableCol.FromTiColumnInfo(col, outputColumnID) + t.Columns = append(t.Columns, tableCol) + } +} + +// ToTableInfo converts from TableDefinition to DDLEvent. +func (t *TableDefinition) ToTableInfo() (*common.TableInfo, error) { + tidbTableInfo := &timodel.TableInfo{ + Name: pmodel.NewCIStr(t.Table), + } + nextMockID := int64(100) // 100 is an arbitrary number + for _, col := range t.Columns { + tiCol, err := col.ToTiColumnInfo(nextMockID) + if err != nil { + return nil, err + } + if mysql.HasPriKeyFlag(tiCol.GetFlag()) { + // use PKIsHandle to make sure that the primary keys can be detected by `WrapTableInfo` + tidbTableInfo.PKIsHandle = true + } + tidbTableInfo.Columns = append(tidbTableInfo.Columns, tiCol) + nextMockID += 1 + } + info := common.WrapTableInfo(100, t.Schema, tidbTableInfo) + + return info, nil +} + +// IsTableSchema returns whether the TableDefinition is a table schema. +func (t *TableDefinition) IsTableSchema() bool { + if len(t.Columns) != t.TotalColumns { + log.Panic("invalid table definition", zap.Any("tableDef", t)) + } + return t.TotalColumns != 0 +} + +// MarshalWithQuery marshals TableDefinition with Query field. +func (t *TableDefinition) MarshalWithQuery() ([]byte, error) { + data, err := json.MarshalIndent(t, marshalPrefix, marshalIndent) + if err != nil { + return nil, errors.WrapError(errors.ErrMarshalFailed, err) + } + return data, nil +} + +// marshalWithoutQuery marshals TableDefinition without Query field. +func (t *TableDefinition) marshalWithoutQuery() ([]byte, error) { + // sort columns by name + sortedColumns := make([]TableCol, len(t.Columns)) + copy(sortedColumns, t.Columns) + sort.Slice(sortedColumns, func(i, j int) bool { + return sortedColumns[i].Name < sortedColumns[j].Name + }) + + defWithoutQuery := tableDefWithoutQuery{ + Table: t.Table, + Schema: t.Schema, + Columns: sortedColumns, + TotalColumns: t.TotalColumns, + } + + data, err := json.MarshalIndent(defWithoutQuery, marshalPrefix, marshalIndent) + if err != nil { + return nil, errors.WrapError(errors.ErrMarshalFailed, err) + } + return data, nil +} + +// Sum32 returns the 32-bits hash value of TableDefinition. +func (t *TableDefinition) Sum32(hasher *hash.PositionInertia) (uint32, error) { + if hasher == nil { + hasher = hash.NewPositionInertia() + } + hasher.Reset() + data, err := t.marshalWithoutQuery() + if err != nil { + return 0, err + } + + hasher.Write(data) + return hasher.Sum32(), nil +} + +// GenerateSchemaFilePath generates the schema file path for TableDefinition. +func (t *TableDefinition) GenerateSchemaFilePath() (string, error) { + checksum, err := t.Sum32(nil) + if err != nil { + return "", err + } + if !t.IsTableSchema() && t.Table != "" { + log.Panic("invalid table definition", zap.Any("tableDef", t)) + } + return generateSchemaFilePath(t.Schema, t.Table, t.TableVersion, checksum), nil +} diff --git a/pkg/sink/cloudstorage/table_definition_test.go b/pkg/sink/cloudstorage/table_definition_test.go new file mode 100644 index 00000000..fc9401f0 --- /dev/null +++ b/pkg/sink/cloudstorage/table_definition_test.go @@ -0,0 +1,524 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. +package cloudstorage + +import ( + "encoding/json" + "math" + "math/rand" + "testing" + + "github.com/pingcap/ticdc/pkg/common" + commonEvent "github.com/pingcap/ticdc/pkg/common/event" + timodel "github.com/pingcap/tidb/pkg/meta/model" + "github.com/pingcap/tidb/pkg/parser/charset" + pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/mysql" + "github.com/pingcap/tidb/pkg/types" + "github.com/stretchr/testify/require" +) + +func generateTableDef() (TableDefinition, *common.TableInfo) { + var columns []*timodel.ColumnInfo + ft := types.NewFieldType(mysql.TypeLong) + ft.SetFlag(mysql.PriKeyFlag | mysql.NotNullFlag) + col := &timodel.ColumnInfo{ + Name: pmodel.NewCIStr("Id"), + FieldType: *ft, + DefaultValue: 10, + } + columns = append(columns, col) + + ft = types.NewFieldType(mysql.TypeVarchar) + ft.SetFlag(mysql.NotNullFlag) + ft.SetFlen(128) + col = &timodel.ColumnInfo{ + Name: pmodel.NewCIStr("LastName"), + FieldType: *ft, + DefaultValue: "Default LastName", + } + columns = append(columns, col) + + ft = types.NewFieldType(mysql.TypeVarchar) + ft.SetFlen(64) + col = &timodel.ColumnInfo{ + Name: pmodel.NewCIStr("FirstName"), + FieldType: *ft, + DefaultValue: "Default FirstName", + } + columns = append(columns, col) + + ft = types.NewFieldType(mysql.TypeDatetime) + col = &timodel.ColumnInfo{ + Name: pmodel.NewCIStr("Birthday"), + FieldType: *ft, + DefaultValue: 12345678, + } + columns = append(columns, col) + + tableInfo := &common.TableInfo{ + TableName: common.TableName{ + Schema: "schema1", + Table: "table1", + TableID: 20, + }, + } + + var def TableDefinition + def.FromTableInfo(tableInfo.GetSchemaName(), tableInfo.GetTableName(), tableInfo, tableInfo.UpdateTS(), false) + return def, tableInfo +} + +func TestTableCol(t *testing.T) { + t.Parallel() + + testCases := []struct { + name string + filedType byte + flen int + decimal int + flag uint + charset string + expected string + }{ + { + name: "time", + filedType: mysql.TypeDuration, + flen: math.MinInt, + decimal: 5, + flag: 0, + expected: `{"ColumnName":"","ColumnType":"TIME","ColumnScale":"5"}`, + }, + { + name: "int(5) UNSIGNED", + filedType: mysql.TypeLong, + flen: 5, + decimal: math.MinInt, + flag: mysql.UnsignedFlag, + expected: `{"ColumnName":"","ColumnType":"INT UNSIGNED","ColumnPrecision":"5"}`, + }, + { + name: "float(12,3)", + filedType: mysql.TypeFloat, + flen: 12, + decimal: 3, + flag: 0, + expected: `{"ColumnName":"","ColumnType":"FLOAT","ColumnPrecision":"12","ColumnScale":"3"}`, + }, + { + name: "float", + filedType: mysql.TypeFloat, + flen: 12, + decimal: -1, + flag: 0, + expected: `{"ColumnName":"","ColumnType":"FLOAT","ColumnPrecision":"12"}`, + }, + { + name: "float", + filedType: mysql.TypeFloat, + flen: 5, + decimal: -1, + flag: 0, + expected: `{"ColumnName":"","ColumnType":"FLOAT","ColumnPrecision":"5"}`, + }, + { + name: "float(7,3)", + filedType: mysql.TypeFloat, + flen: 7, + decimal: 3, + flag: 0, + expected: `{"ColumnName":"","ColumnType":"FLOAT","ColumnPrecision":"7","ColumnScale":"3"}`, + }, + { + name: "double(12,3)", + filedType: mysql.TypeDouble, + flen: 12, + decimal: 3, + flag: 0, + expected: `{"ColumnName":"","ColumnType":"DOUBLE","ColumnPrecision":"12","ColumnScale":"3"}`, + }, + { + name: "double", + filedType: mysql.TypeDouble, + flen: 12, + decimal: -1, + flag: 0, + expected: `{"ColumnName":"","ColumnType":"DOUBLE","ColumnPrecision":"12"}`, + }, + { + name: "double", + filedType: mysql.TypeDouble, + flen: 5, + decimal: -1, + flag: 0, + expected: `{"ColumnName":"","ColumnType":"DOUBLE","ColumnPrecision":"5"}`, + }, + { + name: "double(7,3)", + filedType: mysql.TypeDouble, + flen: 7, + decimal: 3, + flag: 0, + expected: `{"ColumnName":"","ColumnType":"DOUBLE","ColumnPrecision":"7","ColumnScale":"3"}`, + }, + { + name: "tinyint(5)", + filedType: mysql.TypeTiny, + flen: 5, + decimal: math.MinInt, + flag: 0, + expected: `{"ColumnName":"","ColumnType":"TINYINT","ColumnPrecision":"5"}`, + }, + { + name: "smallint(5)", + filedType: mysql.TypeShort, + flen: 5, + decimal: math.MinInt, + flag: 0, + expected: `{"ColumnName":"","ColumnType":"SMALLINT","ColumnPrecision":"5"}`, + }, + { + name: "mediumint(10)", + filedType: mysql.TypeInt24, + flen: 10, + decimal: math.MinInt, + flag: 0, + expected: `{"ColumnName":"","ColumnType":"MEDIUMINT","ColumnPrecision":"10"}`, + }, + { + name: "int(11)", + filedType: mysql.TypeLong, + flen: math.MinInt, + decimal: math.MinInt, + flag: mysql.PriKeyFlag, + expected: `{"ColumnIsPk":"true", "ColumnName":"", "ColumnPrecision":"11", "ColumnType":"INT"}`, + }, + { + name: "bigint(20)", + filedType: mysql.TypeLonglong, + flen: 20, + decimal: math.MinInt, + flag: 0, + expected: `{"ColumnName":"","ColumnType":"BIGINT","ColumnPrecision":"20"}`, + }, + { + name: "bit(5)", + filedType: mysql.TypeBit, + flen: 5, + decimal: math.MinInt, + flag: 0, + expected: `{"ColumnName":"","ColumnType":"BIT","ColumnPrecision":"5"}`, + }, + { + name: "varchar(128) CHARACTER SET utf8mb4 COLLATE utf8mb4_bin", + filedType: mysql.TypeVarchar, + flen: 128, + decimal: math.MinInt, + flag: 0, + expected: `{"ColumnName":"","ColumnType":"VARCHAR","ColumnPrecision":"128"}`, + }, + { + name: "char(32) CHARACTER SET utf8mb4 COLLATE utf8mb4_bin", + filedType: mysql.TypeString, + flen: 32, + decimal: math.MinInt, + flag: 0, + expected: `{"ColumnName":"","ColumnType":"CHAR","ColumnPrecision":"32"}`, + }, + { + name: "var_string(64)", + filedType: mysql.TypeVarString, + flen: 64, + decimal: math.MinInt, + flag: 0, + expected: `{"ColumnName":"","ColumnType":"VAR_STRING","ColumnPrecision":"64"}`, + }, + { + name: "blob", + filedType: mysql.TypeBlob, + flen: 100, + decimal: math.MinInt, + flag: 0, + expected: `{"ColumnName":"","ColumnType":"BLOB","ColumnPrecision":"100"}`, + }, + { + name: "text", + filedType: mysql.TypeBlob, + flen: 100, + decimal: math.MinInt, + flag: 0, + charset: charset.CharsetUTF8MB4, + expected: `{"ColumnName":"","ColumnType":"TEXT","ColumnPrecision":"100"}`, + }, + { + name: "tinyblob", + filedType: mysql.TypeTinyBlob, + flen: 120, + decimal: math.MinInt, + flag: 0, + expected: `{"ColumnName":"","ColumnType":"TINYBLOB","ColumnPrecision":"120"}`, + }, + { + name: "mediumblob", + filedType: mysql.TypeMediumBlob, + flen: 100, + decimal: math.MinInt, + flag: 0, + expected: `{"ColumnName":"","ColumnType":"MEDIUMBLOB","ColumnPrecision":"100"}`, + }, + { + name: "longblob", + filedType: mysql.TypeLongBlob, + flen: 5, + decimal: math.MinInt, + flag: 0, + expected: `{"ColumnName":"","ColumnType":"LONGBLOB","ColumnPrecision":"5"}`, + }, + { + name: "enum", + filedType: mysql.TypeEnum, + expected: `{"ColumnName":"","ColumnType":"ENUM"}`, + }, + { + name: "set", + filedType: mysql.TypeSet, + expected: `{"ColumnName":"","ColumnType":"SET"}`, + }, + { + name: "timestamp(2)", + filedType: mysql.TypeTimestamp, + flen: 8, + decimal: 2, + expected: `{"ColumnName":"","ColumnType":"TIMESTAMP","ColumnScale":"2"}`, + }, + { + name: "timestamp", + filedType: mysql.TypeTimestamp, + flen: 8, + decimal: 0, + expected: `{"ColumnName":"","ColumnType":"TIMESTAMP"}`, + }, + { + name: "datetime(2)", + filedType: mysql.TypeDatetime, + flen: 8, + decimal: 2, + expected: `{"ColumnName":"","ColumnType":"DATETIME","ColumnScale":"2"}`, + }, + { + name: "datetime", + filedType: mysql.TypeDatetime, + flen: 8, + decimal: 0, + expected: `{"ColumnName":"","ColumnType":"DATETIME"}`, + }, + { + name: "date", + filedType: mysql.TypeDate, + flen: 8, + decimal: 2, + expected: `{"ColumnName":"","ColumnType":"DATE"}`, + }, + { + name: "date", + filedType: mysql.TypeDate, + flen: 8, + decimal: 0, + expected: `{"ColumnName":"","ColumnType":"DATE"}`, + }, + { + name: "year(4)", + filedType: mysql.TypeYear, + flen: 4, + decimal: 0, + expected: `{"ColumnName":"","ColumnType":"YEAR","ColumnPrecision":"4"}`, + }, + { + name: "year(2)", + filedType: mysql.TypeYear, + flen: 2, + decimal: 2, + expected: `{"ColumnName":"","ColumnType":"YEAR","ColumnPrecision":"2"}`, + }, + } + + for _, tc := range testCases { + ft := types.NewFieldType(tc.filedType) + if tc.flen != math.MinInt { + ft.SetFlen(tc.flen) + } + if tc.decimal != math.MinInt { + ft.SetDecimal(tc.decimal) + } + if tc.flag != 0 { + ft.SetFlag(tc.flag) + } + if len(tc.charset) != 0 { + ft.SetCharset(tc.charset) + } + col := &timodel.ColumnInfo{FieldType: *ft} + var tableCol TableCol + tableCol.FromTiColumnInfo(col, false) + encodedCol, err := json.Marshal(tableCol) + require.Nil(t, err, tc.name) + require.JSONEq(t, tc.expected, string(encodedCol), tc.name) + + _, err = tableCol.ToTiColumnInfo(100) + require.NoError(t, err) + } +} + +func TestTableDefinition(t *testing.T) { + t.Parallel() + + def, tableInfo := generateTableDef() + encodedDef, err := json.MarshalIndent(def, "", " ") + require.NoError(t, err) + require.JSONEq(t, `{ + "Table": "table1", + "Schema": "schema1", + "Version": 1, + "TableVersion": 100, + "Query": "", + "Type": 0, + "TableColumns": [ + { + "ColumnName": "Id", + "ColumnType": "INT", + "ColumnPrecision": "11", + "ColumnDefault":10, + "ColumnNullable": "false", + "ColumnIsPk": "true" + }, + { + "ColumnName": "LastName", + "ColumnType": "VARCHAR", + "ColumnDefault":"Default LastName", + "ColumnPrecision": "128", + "ColumnNullable": "false" + }, + { + "ColumnName": "FirstName", + "ColumnDefault":"Default FirstName", + "ColumnType": "VARCHAR", + "ColumnPrecision": "64" + }, + { + "ColumnName": "Birthday", + "ColumnDefault":1.2345678e+07, + "ColumnType": "DATETIME" + } + ], + "TableColumnsTotal": 4 + }`, string(encodedDef)) + + def = TableDefinition{} + event := &commonEvent.DDLEvent{ + FinishedTs: tableInfo.UpdateTS(), + Type: byte(timodel.ActionAddColumn), + Query: "alter table schema1.table1 add Birthday date", + TableInfo: tableInfo, + } + def.FromDDLEvent(event, false) + encodedDef, err = json.MarshalIndent(def, "", " ") + require.NoError(t, err) + require.JSONEq(t, `{ + "Table": "table1", + "Schema": "schema1", + "Version": 1, + "TableVersion": 100, + "Query": "alter table schema1.table1 add Birthday date", + "Type": 5, + "TableColumns": [ + { + "ColumnName": "Id", + "ColumnType": "INT", + "ColumnPrecision": "11", + "ColumnDefault":10, + "ColumnNullable": "false", + "ColumnIsPk": "true" + }, + { + "ColumnName": "LastName", + "ColumnType": "VARCHAR", + "ColumnDefault":"Default LastName", + "ColumnPrecision": "128", + "ColumnNullable": "false" + }, + { + "ColumnName": "FirstName", + "ColumnDefault":"Default FirstName", + "ColumnType": "VARCHAR", + "ColumnPrecision": "64" + }, + { + "ColumnName": "Birthday", + "ColumnDefault":1.2345678e+07, + "ColumnType": "DATETIME" + } + ], + "TableColumnsTotal": 4 + }`, string(encodedDef)) + + tableInfo, err = def.ToTableInfo() + require.NoError(t, err) + require.Len(t, tableInfo.GetColumns(), 4) + + event, err = def.ToDDLEvent() + require.NoError(t, err) + require.Equal(t, timodel.ActionAddColumn, event.Type) + require.Equal(t, uint64(100), event.FinishedTs) +} + +func TestTableDefinitionGenFilePath(t *testing.T) { + t.Parallel() + + schemaDef := &TableDefinition{ + Schema: "schema1", + Version: defaultTableDefinitionVersion, + TableVersion: 100, + } + schemaPath, err := schemaDef.GenerateSchemaFilePath() + require.NoError(t, err) + require.Equal(t, "schema1/meta/schema_100_3233644819.json", schemaPath) + + def, _ := generateTableDef() + tablePath, err := def.GenerateSchemaFilePath() + require.NoError(t, err) + require.Equal(t, "schema1/table1/meta/schema_100_3752767265.json", tablePath) +} + +func TestTableDefinitionSum32(t *testing.T) { + t.Parallel() + + def, _ := generateTableDef() + checksum1, err := def.Sum32(nil) + require.NoError(t, err) + checksum2, err := def.Sum32(nil) + require.NoError(t, err) + require.Equal(t, checksum1, checksum2) + + n := len(def.Columns) + newCol := make([]TableCol, n) + copy(newCol, def.Columns) + newDef := def + newDef.Columns = newCol + + for i := 0; i < n; i++ { + target := rand.Intn(n) + newDef.Columns[i], newDef.Columns[target] = newDef.Columns[target], newDef.Columns[i] + newChecksum, err := newDef.Sum32(nil) + require.NoError(t, err) + require.Equal(t, checksum1, newChecksum) + } +} diff --git a/pkg/sink/codec/canal/encoder.go b/pkg/sink/codec/canal/encoder.go index 52b2eaab..99df824a 100644 --- a/pkg/sink/codec/canal/encoder.go +++ b/pkg/sink/codec/canal/encoder.go @@ -14,6 +14,7 @@ package canal import ( + "bytes" "context" "time" @@ -27,6 +28,7 @@ import ( "github.com/pingcap/ticdc/pkg/sink/codec/common" "github.com/pingcap/ticdc/pkg/sink/codec/internal" "github.com/pingcap/ticdc/pkg/sink/kafka/claimcheck" + "github.com/pingcap/tiflow/pkg/sink/codec" "github.com/pingcap/tiflow/pkg/sink/codec/utils" "go.uber.org/zap" "golang.org/x/text/encoding" @@ -584,3 +586,86 @@ func (b *JSONRowEventEncoder) Clean() { b.claimCheck.CleanMetrics() } } + +// JSONTxnEventEncoder encodes txn event in JSON format +type JSONTxnEventEncoder struct { + config *common.Config + + // the symbol separating two lines + terminator []byte + valueBuf *bytes.Buffer + batchSize int + callback func() + + // Store some fields of the txn event. + txnCommitTs uint64 + txnSchema *string + txnTable *string +} + +// NewJSONTxnEventEncoderBuilder creates a new JSONTxnEventEncoder +func NewJSONTxnEventEncoderBuilder(config *common.Config) common.TxnEventEncoder { + return &JSONTxnEventEncoder{ + valueBuf: &bytes.Buffer{}, + terminator: []byte(config.Terminator), + config: config, + } +} + +// AppendTxnEvent appends a txn event to the encoder. +func (j *JSONTxnEventEncoder) AppendTxnEvent(event *commonEvent.DMLEvent) error { + for { + row, ok := event.GetNextRow() + if !ok { + break + } + value, err := newJSONMessageForDML(&commonEvent.RowEvent{ + TableInfo: event.TableInfo, + CommitTs: event.CommitTs, + Event: row, + }, j.config, false, "") + if err != nil { + return err + } + length := len(value) + common.MaxRecordOverhead + // For single message that is longer than max-message-bytes, do not send it. + if length > j.config.MaxMessageBytes { + log.Warn("Single message is too large for canal-json", + zap.Int("maxMessageBytes", j.config.MaxMessageBytes), + zap.Int("length", length), + zap.Any("table", event.TableInfo.TableName)) + return errors.ErrMessageTooLarge.GenWithStackByArgs() + } + j.valueBuf.Write(value) + j.valueBuf.Write(j.terminator) + j.batchSize++ + } + j.callback = event.PostFlush + j.txnCommitTs = event.CommitTs + j.txnSchema = event.TableInfo.GetSchemaNamePtr() + j.txnTable = event.TableInfo.GetTableNamePtr() + return nil +} + +// Build builds a message from the encoder and resets the encoder. +func (j *JSONTxnEventEncoder) Build() []*common.Message { + if j.batchSize == 0 { + return nil + } + + ret := common.NewMsg(nil, j.valueBuf.Bytes()) + ret.SetRowsCount(j.batchSize) + ret.Callback = j.callback + if j.valueBuf.Cap() > codec.MemBufShrinkThreshold { + j.valueBuf = &bytes.Buffer{} + } else { + j.valueBuf.Reset() + } + j.callback = nil + j.batchSize = 0 + j.txnCommitTs = 0 + j.txnSchema = nil + j.txnTable = nil + + return []*common.Message{ret} +} diff --git a/pkg/sink/codec/common/encoder.go b/pkg/sink/codec/common/encoder.go index 672a68f8..bcb9afd3 100644 --- a/pkg/sink/codec/common/encoder.go +++ b/pkg/sink/codec/common/encoder.go @@ -35,6 +35,15 @@ type EventEncoder interface { Clean() } +// TxnEventEncoder is an abstraction for events encoder +type TxnEventEncoder interface { + // AppendTxnEvent append a txn event into the buffer. + AppendTxnEvent(*commonEvent.DMLEvent) error + // Build builds the batch and returns the bytes of key and value. + // Should be called after `AppendTxnEvent` + Build() []*Message +} + // IsColumnValueEqual checks whether the preValue and updatedValue are equal. func IsColumnValueEqual(preValue, updatedValue interface{}) bool { if preValue == nil || updatedValue == nil { diff --git a/pkg/sink/codec/csv/csv_encoder.go b/pkg/sink/codec/csv/csv_encoder.go new file mode 100644 index 00000000..bca4eb3e --- /dev/null +++ b/pkg/sink/codec/csv/csv_encoder.go @@ -0,0 +1,79 @@ +// Copyright 2025 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package csv + +import ( + "bytes" + + commonEvent "github.com/pingcap/ticdc/pkg/common/event" + "github.com/pingcap/ticdc/pkg/sink/codec/common" + "github.com/pingcap/tiflow/pkg/sink/codec" +) + +// batchEncoder encodes the events into the byte of a batch into. +type batchEncoder struct { + valueBuf *bytes.Buffer + callback func() + batchSize int + config *common.Config +} + +// NewTxnEventEncoder creates a new csv BatchEncoder. +func NewTxnEventEncoder(config *common.Config) common.TxnEventEncoder { + return &batchEncoder{ + config: config, + valueBuf: &bytes.Buffer{}, + } +} + +// AppendTxnEvent implements the TxnEventEncoder interface +func (b *batchEncoder) AppendTxnEvent(event *commonEvent.DMLEvent) error { + for { + row, ok := event.GetNextRow() + if !ok { + break + } + msg, err := rowChangedEvent2CSVMsg(b.config, &commonEvent.RowEvent{ + TableInfo: event.TableInfo, + CommitTs: event.CommitTs, + Event: row, + }) + if err != nil { + return err + } + b.valueBuf.Write(msg.encode()) + b.batchSize++ + } + b.callback = event.PostFlush + return nil +} + +// Build implements the RowEventEncoder interface +func (b *batchEncoder) Build() (messages []*common.Message) { + if b.batchSize == 0 { + return nil + } + ret := common.NewMsg(nil, b.valueBuf.Bytes()) + ret.SetRowsCount(b.batchSize) + ret.Callback = b.callback + if b.valueBuf.Cap() > codec.MemBufShrinkThreshold { + b.valueBuf = &bytes.Buffer{} + } else { + b.valueBuf.Reset() + } + b.callback = nil + b.batchSize = 0 + + return []*common.Message{ret} +} diff --git a/pkg/sink/codec/csv/csv_encoder_test.go b/pkg/sink/codec/csv/csv_encoder_test.go new file mode 100644 index 00000000..588c91b7 --- /dev/null +++ b/pkg/sink/codec/csv/csv_encoder_test.go @@ -0,0 +1,104 @@ +// 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 csv + +import ( + "testing" + + "github.com/pingcap/tiflow/cdc/entry" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/sink/codec/common" + "github.com/stretchr/testify/require" +) + +func TestCSVBatchCodec(t *testing.T) { + helper := entry.NewSchemaTestHelper(t) + defer helper.Close() + + ddl := helper.DDL2Event("create table test.table1(col1 int primary key)") + event1 := helper.DML2Event("insert into test.table1 values (1)", "test", "table1") + event2 := helper.DML2Event("insert into test.table1 values (2)", "test", "table1") + + testCases := []*model.SingleTableTxn{ + { + Rows: []*model.RowChangedEvent{ + event1, + event2, + }, + }, + { + TableInfo: ddl.TableInfo, + Rows: nil, + }, + } + + for _, cs := range testCases { + encoder := newBatchEncoder(&common.Config{ + Delimiter: ",", + Quote: "\"", + Terminator: "\n", + NullString: "\\N", + IncludeCommitTs: true, + }) + err := encoder.AppendTxnEvent(cs, nil) + require.Nil(t, err) + messages := encoder.Build() + if len(cs.Rows) == 0 { + require.Nil(t, messages) + continue + } + require.Len(t, messages, 1) + require.Equal(t, len(cs.Rows), messages[0].GetRowsCount()) + } +} + +func TestCSVAppendRowChangedEventWithCallback(t *testing.T) { + encoder := newBatchEncoder(&common.Config{ + Delimiter: ",", + Quote: "\"", + Terminator: "\n", + NullString: "\\N", + IncludeCommitTs: true, + }) + require.NotNil(t, encoder) + + count := 0 + + helper := entry.NewSchemaTestHelper(t) + defer helper.Close() + + _ = helper.DDL2Event("create table test.table1(col1 int primary key)") + row := helper.DML2Event("insert into test.table1 values (1)", "test", "table1") + txn := &model.SingleTableTxn{ + TableInfo: row.TableInfo, + Rows: []*model.RowChangedEvent{row}, + } + callback := func() { + count += 1 + } + + // Empty build makes sure that the callback build logic not broken. + msgs := encoder.Build() + require.Len(t, msgs, 0, "no message should be built and no panic") + + // Append the event. + err := encoder.AppendTxnEvent(txn, callback) + require.Nil(t, err) + require.Equal(t, 0, count, "nothing should be called") + + msgs = encoder.Build() + require.Len(t, msgs, 1, "expected one message") + msgs[0].Callback() + require.Equal(t, 1, count, "expected all callbacks to be called") +} diff --git a/pkg/sink/codec/csv/csv_message.go b/pkg/sink/codec/csv/csv_message.go new file mode 100644 index 00000000..bc24b4f4 --- /dev/null +++ b/pkg/sink/codec/csv/csv_message.go @@ -0,0 +1,408 @@ +// Copyright 2025 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package csv + +import ( + "encoding/base64" + "encoding/hex" + "fmt" + "strconv" + "strings" + + "github.com/pingcap/errors" + "github.com/pingcap/log" + commonType "github.com/pingcap/ticdc/pkg/common" + "github.com/pingcap/ticdc/pkg/common/event" + "github.com/pingcap/ticdc/pkg/sink/codec/common" + "github.com/pingcap/tidb/pkg/kv" + timodel "github.com/pingcap/tidb/pkg/meta/model" + "github.com/pingcap/tidb/pkg/parser/mysql" + "github.com/pingcap/tidb/pkg/types" + "github.com/pingcap/tidb/pkg/util/chunk" + "github.com/pingcap/tiflow/pkg/config" + cerror "github.com/pingcap/tiflow/pkg/errors" +) + +// a csv row should at least contain operation-type, table-name, schema-name and one table column +const minimumColsCnt = 4 + +// operation specifies the operation type +type operation int + +// enum types of operation +const ( + operationInsert operation = iota + operationDelete + operationUpdate +) + +func (o operation) String() string { + switch o { + case operationInsert: + return "I" + case operationDelete: + return "D" + case operationUpdate: + return "U" + default: + return "unknown" + } +} + +func (o *operation) FromString(op string) error { + switch op { + case "I": + *o = operationInsert + case "D": + *o = operationDelete + case "U": + *o = operationUpdate + default: + return fmt.Errorf("invalid operation type %s", op) + } + + return nil +} + +type csvMessage struct { + // config hold the codec configuration items. + config *common.Config + // opType denotes the specific operation type. + opType operation + tableName string + schemaName string + commitTs uint64 + columns []any + preColumns []any + // newRecord indicates whether we encounter a new record. + newRecord bool + HandleKey kv.Handle +} + +func newCSVMessage(config *common.Config) *csvMessage { + return &csvMessage{ + config: config, + newRecord: true, + } +} + +// encode returns a byte slice composed of the columns as follows: +// Col1: The operation-type indicator: I, D, U. +// Col2: Table name, the name of the source table. +// Col3: Schema name, the name of the source schema. +// Col4: Commit TS, the commit-ts of the source txn (optional). +// Col5-n: one or more columns that represent the data to be changed. +func (c *csvMessage) encode() []byte { + strBuilder := new(strings.Builder) + if c.opType == operationUpdate && c.config.OutputOldValue && len(c.preColumns) != 0 { + // Encode the old value first as a dedicated row. + c.encodeMeta("D", strBuilder) + c.encodeColumns(c.preColumns, strBuilder) + + // Encode the after value as a dedicated row. + c.newRecord = true // reset newRecord to true, so that the first column will not start with delimiter. + c.encodeMeta("I", strBuilder) + c.encodeColumns(c.columns, strBuilder) + } else { + c.encodeMeta(c.opType.String(), strBuilder) + c.encodeColumns(c.columns, strBuilder) + } + return []byte(strBuilder.String()) +} + +func (c *csvMessage) encodeMeta(opType string, b *strings.Builder) { + c.formatValue(opType, b) + c.formatValue(c.tableName, b) + c.formatValue(c.schemaName, b) + if c.config.IncludeCommitTs { + c.formatValue(c.commitTs, b) + } + if c.config.OutputOldValue { + // When c.config.OutputOldValue, we need an extra column "is-updated" + // to indicate whether the row is updated or just original insert/delete + if c.opType == operationUpdate { + c.formatValue(true, b) + } else { + c.formatValue(false, b) + } + } + if c.config.OutputHandleKey { + c.formatValue(c.HandleKey.String(), b) + } +} + +func (c *csvMessage) encodeColumns(columns []any, b *strings.Builder) { + for _, col := range columns { + c.formatValue(col, b) + } + b.WriteString(c.config.Terminator) +} + +func (c *csvMessage) decode(datums []types.Datum) error { + var dataColIdx int + if len(datums) < minimumColsCnt { + return cerror.WrapError(cerror.ErrCSVDecodeFailed, + errors.New("the csv row should have at least four columns"+ + "(operation-type, table-name, schema-name, commit-ts)")) + } + + if err := c.opType.FromString(datums[0].GetString()); err != nil { + return cerror.WrapError(cerror.ErrCSVDecodeFailed, err) + } + dataColIdx++ + c.tableName = datums[1].GetString() + dataColIdx++ + c.schemaName = datums[2].GetString() + dataColIdx++ + if c.config.IncludeCommitTs { + commitTs, err := strconv.ParseUint(datums[3].GetString(), 10, 64) + if err != nil { + return cerror.WrapError(cerror.ErrCSVDecodeFailed, + fmt.Errorf("the 4th column(%s) of csv row should be a valid commit-ts", datums[3].GetString())) + } + c.commitTs = commitTs + dataColIdx++ + } else { + c.commitTs = 0 + } + if c.config.OutputOldValue { + // When c.config.OutputOldValue, we need an extra column "is-updated". + // TODO: use this flag to guarantee data consistency in update uk/pk scenario. + dataColIdx++ + } + c.columns = c.columns[:0] + + for i := dataColIdx; i < len(datums); i++ { + if datums[i].IsNull() { + c.columns = append(c.columns, nil) + } else { + c.columns = append(c.columns, datums[i].GetString()) + } + } + + return nil +} + +// as stated in https://datatracker.ietf.org/doc/html/rfc4180, +// if double-quotes are used to enclose fields, then a double-quote +// appearing inside a field must be escaped by preceding it with +// another double quote. +func (c *csvMessage) formatWithQuotes(value string, strBuilder *strings.Builder) { + quote := c.config.Quote + + strBuilder.WriteString(quote) + // replace any quote in csv column with two quotes. + strBuilder.WriteString(strings.ReplaceAll(value, quote, quote+quote)) + strBuilder.WriteString(quote) +} + +// formatWithEscapes escapes the csv column if necessary. +func (c *csvMessage) formatWithEscapes(value string, strBuilder *strings.Builder) { + lastPos := 0 + delimiter := c.config.Delimiter + + for i := 0; i < len(value); i++ { + ch := value[i] + isDelimiterStart := strings.HasPrefix(value[i:], delimiter) + // if '\r', '\n', '\' or the delimiter (may have multiple characters) are contained in + // csv column, we should escape these characters. + if ch == config.CR || ch == config.LF || ch == config.Backslash || isDelimiterStart { + // write out characters up until this position. + strBuilder.WriteString(value[lastPos:i]) + switch ch { + case config.LF: + ch = 'n' + case config.CR: + ch = 'r' + } + strBuilder.WriteRune(config.Backslash) + strBuilder.WriteRune(rune(ch)) + + // escape each characters in delimiter. + if isDelimiterStart { + for k := 1; k < len(c.config.Delimiter); k++ { + strBuilder.WriteRune(config.Backslash) + strBuilder.WriteRune(rune(delimiter[k])) + } + lastPos = i + len(delimiter) + } else { + lastPos = i + 1 + } + } + } + strBuilder.WriteString(value[lastPos:]) +} + +// formatValue formats the csv column and appends it to a string builder. +func (c *csvMessage) formatValue(value any, strBuilder *strings.Builder) { + defer func() { + // reset newRecord to false after handing the first csv column + c.newRecord = false + }() + + if !c.newRecord { + strBuilder.WriteString(c.config.Delimiter) + } + + if value == nil { + strBuilder.WriteString(c.config.NullString) + return + } + + switch v := value.(type) { + case string: + // if quote is configured, format the csv column with quotes, + // otherwise escape this csv column. + if len(c.config.Quote) != 0 { + c.formatWithQuotes(v, strBuilder) + } else { + c.formatWithEscapes(v, strBuilder) + } + default: + strBuilder.WriteString(fmt.Sprintf("%v", v)) + } +} + +// fromColValToCsvVal converts column from TiDB type to csv type. +func fromColValToCsvVal(csvConfig *common.Config, row *chunk.Row, idx int, colInfo *timodel.ColumnInfo, flag *commonType.ColumnFlagType) (any, error) { + if row.IsNull(idx) { + return nil, nil + } + + switch colInfo.GetType() { + case mysql.TypeVarchar, mysql.TypeString, mysql.TypeVarString, mysql.TypeTinyBlob, + mysql.TypeMediumBlob, mysql.TypeLongBlob, mysql.TypeBlob: + if flag.IsBinary() { + v := row.GetBytes(idx) + switch csvConfig.BinaryEncodingMethod { + case config.BinaryEncodingBase64: + return base64.StdEncoding.EncodeToString(v), nil + case config.BinaryEncodingHex: + return hex.EncodeToString(v), nil + default: + return nil, cerror.WrapError(cerror.ErrCSVEncodeFailed, + errors.Errorf("unsupported binary encoding method %s", + csvConfig.BinaryEncodingMethod)) + } + } + return row.GetString(idx), nil + case mysql.TypeDate, mysql.TypeDatetime, mysql.TypeNewDate, mysql.TypeTimestamp: + return row.GetTime(idx).String(), nil + case mysql.TypeDuration: + return row.GetDuration(idx, colInfo.GetDecimal()).String(), nil + case mysql.TypeEnum: + enumValue := row.GetEnum(idx).Value + enumVar, err := types.ParseEnumValue(colInfo.GetElems(), enumValue) + if err != nil { + return nil, cerror.WrapError(cerror.ErrCSVEncodeFailed, err) + } + return enumVar.Name, nil + case mysql.TypeSet: + bitValue := row.GetEnum(idx).Value + setVar, err := types.ParseSetValue(colInfo.GetElems(), bitValue) + if err != nil { + return nil, cerror.WrapError(cerror.ErrCSVEncodeFailed, err) + } + return setVar.Name, nil + case mysql.TypeBit: + d := row.GetDatum(idx, &colInfo.FieldType) + // Encode bits as integers to avoid pingcap/tidb#10988 (which also affects MySQL itself) + return d.GetBinaryLiteral().ToInt(types.DefaultStmtNoWarningContext) + case mysql.TypeNewDecimal: + return row.GetMyDecimal(idx).String(), nil + case mysql.TypeJSON: + return row.GetJSON(idx).String(), nil + case mysql.TypeTiDBVectorFloat32: + vec := row.GetVectorFloat32(idx) + return vec.String(), nil + default: + d := row.GetDatum(idx, &colInfo.FieldType) + return d.GetValue(), nil + } +} + +// rowChangedEvent2CSVMsg converts a RowChangedEvent to a csv record. +func rowChangedEvent2CSVMsg(csvConfig *common.Config, e *event.RowEvent) (*csvMessage, error) { + var err error + + csvMsg := &csvMessage{ + config: csvConfig, + tableName: e.TableInfo.GetTableName(), + schemaName: e.TableInfo.GetSchemaName(), + commitTs: e.CommitTs, + newRecord: true, + } + + if csvConfig.OutputHandleKey { + log.Warn("not support output handle key") + // csvMsg.HandleKey = e.HandleKey + } + + if e.IsDelete() { + csvMsg.opType = operationDelete + csvMsg.columns, err = rowChangeColumns2CSVColumns(csvConfig, e.GetPreRows(), e.TableInfo) + if err != nil { + return nil, err + } + } else { + if e.GetPreRows() == nil { + // This is a insert operation. + csvMsg.opType = operationInsert + csvMsg.columns, err = rowChangeColumns2CSVColumns(csvConfig, e.GetRows(), e.TableInfo) + if err != nil { + return nil, err + } + } else { + // This is a update operation. + csvMsg.opType = operationUpdate + if csvConfig.OutputOldValue { + if e.GetPreRows().Len() != e.GetRows().Len() { + return nil, cerror.WrapError(cerror.ErrCSVDecodeFailed, + fmt.Errorf("the column length of preColumns %d doesn't equal to that of columns %d", + e.GetPreRows().Len(), e.GetRows().Len())) + } + csvMsg.preColumns, err = rowChangeColumns2CSVColumns(csvConfig, e.GetPreRows(), e.TableInfo) + if err != nil { + return nil, err + } + } + csvMsg.columns, err = rowChangeColumns2CSVColumns(csvConfig, e.GetRows(), e.TableInfo) + if err != nil { + return nil, err + } + } + } + return csvMsg, nil +} + +func rowChangeColumns2CSVColumns(csvConfig *common.Config, row *chunk.Row, tableInfo *commonType.TableInfo) ([]any, error) { + var csvColumns []any + + for i, col := range tableInfo.GetColumns() { + // column could be nil in a condition described in + // https://github.com/pingcap/tiflow/issues/6198#issuecomment-1191132951 + if col == nil { + continue + } + + flag := tableInfo.GetColumnFlags()[col.ID] + converted, err := fromColValToCsvVal(csvConfig, row, i, col, flag) + if err != nil { + return nil, errors.Trace(err) + } + csvColumns = append(csvColumns, converted) + } + + return csvColumns, nil +} diff --git a/pkg/sink/codec/csv/csv_message_test.go b/pkg/sink/codec/csv/csv_message_test.go new file mode 100644 index 00000000..27b62ce1 --- /dev/null +++ b/pkg/sink/codec/csv/csv_message_test.go @@ -0,0 +1,1103 @@ +// 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 csv + +import ( + "fmt" + "strings" + "testing" + + "github.com/pingcap/tidb/pkg/kv" + "github.com/pingcap/tidb/pkg/parser/mysql" + "github.com/pingcap/tidb/pkg/types" + "github.com/pingcap/tidb/pkg/util/rowcodec" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/config" + "github.com/pingcap/tiflow/pkg/sink/codec/common" + "github.com/pingcap/tiflow/pkg/util" + "github.com/stretchr/testify/require" +) + +type csvTestColumnTuple struct { + col model.Column + colInfo rowcodec.ColInfo + want interface{} + BinaryEncodingMethod string +} + +var csvTestColumnsGroup = [][]*csvTestColumnTuple{ + { + { + model.Column{Name: "tiny", Value: int64(1), Type: mysql.TypeTiny}, + rowcodec.ColInfo{ + ID: 1, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeTiny), + }, + int64(1), + config.BinaryEncodingBase64, + }, + { + model.Column{Name: "short", Value: int64(1), Type: mysql.TypeShort}, + rowcodec.ColInfo{ + ID: 2, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeShort), + }, + int64(1), + config.BinaryEncodingBase64, + }, + { + model.Column{Name: "int24", Value: int64(1), Type: mysql.TypeInt24}, + rowcodec.ColInfo{ + ID: 3, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeInt24), + }, + int64(1), + config.BinaryEncodingBase64, + }, + { + model.Column{Name: "long", Value: int64(1), Type: mysql.TypeLong}, + rowcodec.ColInfo{ + ID: 4, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeLong), + }, + int64(1), + config.BinaryEncodingBase64, + }, + { + model.Column{Name: "longlong", Value: int64(1), Type: mysql.TypeLonglong}, + rowcodec.ColInfo{ + ID: 5, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeLonglong), + }, + int64(1), + config.BinaryEncodingBase64, + }, + { + model.Column{ + Name: "tinyunsigned", + Value: uint64(1), + Type: mysql.TypeTiny, + Flag: model.UnsignedFlag, + }, + rowcodec.ColInfo{ + ID: 6, + IsPKHandle: false, + VirtualGenCol: false, + Ft: setFlag(types.NewFieldType(mysql.TypeTiny), uint(model.UnsignedFlag)), + }, + uint64(1), + config.BinaryEncodingBase64, + }, + { + model.Column{ + Name: "shortunsigned", + Value: uint64(1), + Type: mysql.TypeShort, + Flag: model.UnsignedFlag, + }, + rowcodec.ColInfo{ + ID: 7, + IsPKHandle: false, + VirtualGenCol: false, + Ft: setFlag(types.NewFieldType(mysql.TypeShort), uint(model.UnsignedFlag)), + }, + uint64(1), + config.BinaryEncodingBase64, + }, + { + model.Column{ + Name: "int24unsigned", + Value: uint64(1), + Type: mysql.TypeInt24, + Flag: model.UnsignedFlag, + }, + rowcodec.ColInfo{ + ID: 8, + IsPKHandle: false, + VirtualGenCol: false, + Ft: setFlag(types.NewFieldType(mysql.TypeInt24), uint(model.UnsignedFlag)), + }, + uint64(1), + config.BinaryEncodingBase64, + }, + { + model.Column{ + Name: "longunsigned", + Value: uint64(1), + Type: mysql.TypeLong, + Flag: model.UnsignedFlag, + }, + rowcodec.ColInfo{ + ID: 9, + IsPKHandle: false, + VirtualGenCol: false, + Ft: setFlag(types.NewFieldType(mysql.TypeLong), uint(model.UnsignedFlag)), + }, + uint64(1), + config.BinaryEncodingBase64, + }, + { + model.Column{ + Name: "longlongunsigned", + Value: uint64(1), + Type: mysql.TypeLonglong, + Flag: model.UnsignedFlag, + }, + rowcodec.ColInfo{ + ID: 10, + IsPKHandle: false, + VirtualGenCol: false, + Ft: setFlag( + types.NewFieldType(mysql.TypeLonglong), + uint(model.UnsignedFlag), + ), + }, + uint64(1), + config.BinaryEncodingBase64, + }, + }, + { + { + model.Column{Name: "float", Value: float64(3.14), Type: mysql.TypeFloat}, + rowcodec.ColInfo{ + ID: 11, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeFloat), + }, + float64(3.14), + config.BinaryEncodingBase64, + }, + { + model.Column{Name: "double", Value: float64(3.14), Type: mysql.TypeDouble}, + rowcodec.ColInfo{ + ID: 12, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeDouble), + }, + float64(3.14), + config.BinaryEncodingBase64, + }, + }, + { + { + model.Column{Name: "bit", Value: uint64(683), Type: mysql.TypeBit}, + rowcodec.ColInfo{ + ID: 13, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeBit), + }, + uint64(683), + config.BinaryEncodingBase64, + }, + }, + { + { + model.Column{Name: "decimal", Value: "129012.1230000", Type: mysql.TypeNewDecimal}, + rowcodec.ColInfo{ + ID: 14, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeNewDecimal), + }, + "129012.1230000", + config.BinaryEncodingBase64, + }, + }, + { + { + model.Column{Name: "tinytext", Value: []byte("hello world"), Type: mysql.TypeTinyBlob}, + rowcodec.ColInfo{ + ID: 15, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeBlob), + }, + "hello world", + config.BinaryEncodingBase64, + }, + { + model.Column{Name: "mediumtext", Value: []byte("hello world"), Type: mysql.TypeMediumBlob}, + rowcodec.ColInfo{ + ID: 16, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeMediumBlob), + }, + "hello world", + config.BinaryEncodingBase64, + }, + { + model.Column{Name: "text", Value: []byte("hello world"), Type: mysql.TypeBlob}, + rowcodec.ColInfo{ + ID: 17, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeBlob), + }, + "hello world", + config.BinaryEncodingBase64, + }, + { + model.Column{Name: "longtext", Value: []byte("hello world"), Type: mysql.TypeLongBlob}, + rowcodec.ColInfo{ + ID: 18, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeLongBlob), + }, + "hello world", + config.BinaryEncodingBase64, + }, + { + model.Column{Name: "varchar", Value: []byte("hello world"), Type: mysql.TypeVarchar}, + rowcodec.ColInfo{ + ID: 19, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeVarchar), + }, + "hello world", + config.BinaryEncodingBase64, + }, + { + model.Column{Name: "varstring", Value: []byte("hello world"), Type: mysql.TypeVarString}, + rowcodec.ColInfo{ + ID: 20, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeVarString), + }, + "hello world", + config.BinaryEncodingBase64, + }, + { + model.Column{Name: "string", Value: []byte("hello world"), Type: mysql.TypeString}, + rowcodec.ColInfo{ + ID: 21, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeString), + }, + "hello world", + config.BinaryEncodingBase64, + }, + { + model.Column{Name: "json", Value: `{"key": "value"}`, Type: mysql.TypeJSON}, + rowcodec.ColInfo{ + ID: 31, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeJSON), + }, + `{"key": "value"}`, + config.BinaryEncodingBase64, + }, + }, + { + { + model.Column{ + Name: "tinyblob", + Value: []byte("hello world"), + Type: mysql.TypeTinyBlob, + Flag: model.BinaryFlag, + }, + rowcodec.ColInfo{ + ID: 22, + IsPKHandle: false, + VirtualGenCol: false, + Ft: setBinChsClnFlag(types.NewFieldType(mysql.TypeTinyBlob)), + }, + "aGVsbG8gd29ybGQ=", + config.BinaryEncodingBase64, + }, + { + model.Column{ + Name: "mediumblob", + Value: []byte("hello world"), + Type: mysql.TypeMediumBlob, + Flag: model.BinaryFlag, + }, + rowcodec.ColInfo{ + ID: 23, + IsPKHandle: false, + VirtualGenCol: false, + Ft: setBinChsClnFlag(types.NewFieldType(mysql.TypeMediumBlob)), + }, + "aGVsbG8gd29ybGQ=", + config.BinaryEncodingBase64, + }, + { + model.Column{ + Name: "blob", + Value: []byte("hello world"), + Type: mysql.TypeBlob, + Flag: model.BinaryFlag, + }, + rowcodec.ColInfo{ + ID: 24, + IsPKHandle: false, + VirtualGenCol: false, + Ft: setBinChsClnFlag(types.NewFieldType(mysql.TypeBlob)), + }, + "aGVsbG8gd29ybGQ=", + config.BinaryEncodingBase64, + }, + { + model.Column{ + Name: "longblob", + Value: []byte("hello world"), + Type: mysql.TypeLongBlob, + Flag: model.BinaryFlag, + }, + rowcodec.ColInfo{ + ID: 25, + IsPKHandle: false, + VirtualGenCol: false, + Ft: setBinChsClnFlag(types.NewFieldType(mysql.TypeLongBlob)), + }, + "aGVsbG8gd29ybGQ=", + config.BinaryEncodingBase64, + }, + { + model.Column{ + Name: "varbinary", + Value: []byte("hello world"), + Type: mysql.TypeVarchar, + Flag: model.BinaryFlag, + }, + rowcodec.ColInfo{ + ID: 26, + IsPKHandle: false, + VirtualGenCol: false, + Ft: setBinChsClnFlag(types.NewFieldType(mysql.TypeVarchar)), + }, + "aGVsbG8gd29ybGQ=", + config.BinaryEncodingBase64, + }, + { + model.Column{ + Name: "varbinary1", + Value: []byte("hello world"), + Type: mysql.TypeVarString, + Flag: model.BinaryFlag, + }, + rowcodec.ColInfo{ + ID: 27, + IsPKHandle: false, + VirtualGenCol: false, + Ft: setBinChsClnFlag(types.NewFieldType(mysql.TypeVarString)), + }, + "aGVsbG8gd29ybGQ=", + config.BinaryEncodingBase64, + }, + { + model.Column{ + Name: "binary", + Value: []byte("hello world"), + Type: mysql.TypeString, + Flag: model.BinaryFlag, + }, + rowcodec.ColInfo{ + ID: 28, + IsPKHandle: false, + VirtualGenCol: false, + Ft: setBinChsClnFlag(types.NewFieldType(mysql.TypeString)), + }, + "aGVsbG8gd29ybGQ=", + config.BinaryEncodingBase64, + }, + }, + { + { + model.Column{ + Name: "tinyblob", + Value: []byte("hello world"), + Type: mysql.TypeTinyBlob, + Flag: model.BinaryFlag, + }, + rowcodec.ColInfo{ + ID: 22, + IsPKHandle: false, + VirtualGenCol: false, + Ft: setBinChsClnFlag(types.NewFieldType(mysql.TypeTinyBlob)), + }, + "68656c6c6f20776f726c64", + config.BinaryEncodingHex, + }, + { + model.Column{ + Name: "mediumblob", + Value: []byte("hello world"), + Type: mysql.TypeMediumBlob, + Flag: model.BinaryFlag, + }, + rowcodec.ColInfo{ + ID: 23, + IsPKHandle: false, + VirtualGenCol: false, + Ft: setBinChsClnFlag(types.NewFieldType(mysql.TypeMediumBlob)), + }, + "68656c6c6f20776f726c64", + config.BinaryEncodingHex, + }, + { + model.Column{ + Name: "blob", + Value: []byte("hello world"), + Type: mysql.TypeBlob, + Flag: model.BinaryFlag, + }, + rowcodec.ColInfo{ + ID: 24, + IsPKHandle: false, + VirtualGenCol: false, + Ft: setBinChsClnFlag(types.NewFieldType(mysql.TypeBlob)), + }, + "68656c6c6f20776f726c64", + config.BinaryEncodingHex, + }, + { + model.Column{ + Name: "longblob", + Value: []byte("hello world"), + Type: mysql.TypeLongBlob, + Flag: model.BinaryFlag, + }, + rowcodec.ColInfo{ + ID: 25, + IsPKHandle: false, + VirtualGenCol: false, + Ft: setBinChsClnFlag(types.NewFieldType(mysql.TypeLongBlob)), + }, + "68656c6c6f20776f726c64", + config.BinaryEncodingHex, + }, + { + model.Column{ + Name: "varbinary", + Value: []byte("hello world"), + Type: mysql.TypeVarchar, + Flag: model.BinaryFlag, + }, + rowcodec.ColInfo{ + ID: 26, + IsPKHandle: false, + VirtualGenCol: false, + Ft: setBinChsClnFlag(types.NewFieldType(mysql.TypeVarchar)), + }, + "68656c6c6f20776f726c64", + config.BinaryEncodingHex, + }, + { + model.Column{ + Name: "varbinary1", + Value: []byte("hello world"), + Type: mysql.TypeVarString, + Flag: model.BinaryFlag, + }, + rowcodec.ColInfo{ + ID: 27, + IsPKHandle: false, + VirtualGenCol: false, + Ft: setBinChsClnFlag(types.NewFieldType(mysql.TypeVarString)), + }, + "68656c6c6f20776f726c64", + config.BinaryEncodingHex, + }, + { + model.Column{ + Name: "binary", + Value: []byte("hello world"), + Type: mysql.TypeString, + Flag: model.BinaryFlag, + }, + rowcodec.ColInfo{ + ID: 28, + IsPKHandle: false, + VirtualGenCol: false, + Ft: setBinChsClnFlag(types.NewFieldType(mysql.TypeString)), + }, + "68656c6c6f20776f726c64", + config.BinaryEncodingHex, + }, + }, + { + { + model.Column{Name: "enum", Value: uint64(1), Type: mysql.TypeEnum}, + rowcodec.ColInfo{ + ID: 29, + IsPKHandle: false, + VirtualGenCol: false, + Ft: setElems(types.NewFieldType(mysql.TypeEnum), []string{"a,", "b"}), + }, + "a,", + config.BinaryEncodingBase64, + }, + }, + { + { + model.Column{Name: "set", Value: uint64(9), Type: mysql.TypeSet}, + rowcodec.ColInfo{ + ID: 30, + IsPKHandle: false, + VirtualGenCol: false, + Ft: setElems(types.NewFieldType(mysql.TypeSet), []string{"a", "b", "c", "d"}), + }, + "a,d", + config.BinaryEncodingBase64, + }, + }, + { + { + model.Column{Name: "date", Value: "2000-01-01", Type: mysql.TypeDate}, + rowcodec.ColInfo{ + ID: 32, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeDate), + }, + "2000-01-01", + config.BinaryEncodingBase64, + }, + { + model.Column{Name: "datetime", Value: "2015-12-20 23:58:58", Type: mysql.TypeDatetime}, + rowcodec.ColInfo{ + ID: 33, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeDatetime), + }, + "2015-12-20 23:58:58", + config.BinaryEncodingBase64, + }, + { + model.Column{Name: "timestamp", Value: "1973-12-30 15:30:00", Type: mysql.TypeTimestamp}, + rowcodec.ColInfo{ + ID: 34, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeTimestamp), + }, + "1973-12-30 15:30:00", + config.BinaryEncodingBase64, + }, + { + model.Column{Name: "time", Value: "23:59:59", Type: mysql.TypeDuration}, + rowcodec.ColInfo{ + ID: 35, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeDuration), + }, + "23:59:59", + config.BinaryEncodingBase64, + }, + }, + { + { + model.Column{Name: "year", Value: int64(1970), Type: mysql.TypeYear}, + rowcodec.ColInfo{ + ID: 36, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeYear), + }, + int64(1970), + config.BinaryEncodingBase64, + }, + }, + { + { + model.Column{Name: "vectorfloat32", Value: util.Must(types.ParseVectorFloat32("[1,2,3,4,5]")), Type: mysql.TypeTiDBVectorFloat32}, + rowcodec.ColInfo{ + ID: 37, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeTiDBVectorFloat32), + }, + "[1,2,3,4,5]", + config.BinaryEncodingBase64, + }, + }, +} + +func setBinChsClnFlag(ft *types.FieldType) *types.FieldType { + types.SetBinChsClnFlag(ft) + return ft +} + +//nolint:unparam +func setFlag(ft *types.FieldType, flag uint) *types.FieldType { + ft.SetFlag(flag) + return ft +} + +func setElems(ft *types.FieldType, elems []string) *types.FieldType { + ft.SetElems(elems) + return ft +} + +func TestFormatWithQuotes(t *testing.T) { + config := &common.Config{ + Quote: "\"", + } + + testCases := []struct { + name string + input string + expected string + }{ + { + name: "string does not contain quote mark", + input: "a,b,c", + expected: `"a,b,c"`, + }, + { + name: "string contains quote mark", + input: `"a,b,c`, + expected: `"""a,b,c"`, + }, + { + name: "empty string", + input: "", + expected: `""`, + }, + } + for _, tc := range testCases { + csvMessage := newCSVMessage(config) + strBuilder := new(strings.Builder) + csvMessage.formatWithQuotes(tc.input, strBuilder) + require.Equal(t, tc.expected, strBuilder.String(), tc.name) + } +} + +func TestFormatWithEscape(t *testing.T) { + testCases := []struct { + name string + config *common.Config + input string + expected string + }{ + { + name: "string does not contain CR/LF/backslash/delimiter", + config: &common.Config{Delimiter: ","}, + input: "abcdef", + expected: "abcdef", + }, + { + name: "string contains CRLF", + config: &common.Config{Delimiter: ","}, + input: "abc\r\ndef", + expected: "abc\\r\\ndef", + }, + { + name: "string contains backslash", + config: &common.Config{Delimiter: ","}, + input: `abc\def`, + expected: `abc\\def`, + }, + { + name: "string contains a single character delimiter", + config: &common.Config{Delimiter: ","}, + input: "abc,def", + expected: `abc\,def`, + }, + { + name: "string contains multi-character delimiter", + config: &common.Config{Delimiter: "***"}, + input: "abc***def", + expected: `abc\*\*\*def`, + }, + { + name: "string contains CR, LF, backslash and delimiter", + config: &common.Config{Delimiter: "?"}, + input: `abc\def?ghi\r\n`, + expected: `abc\\def\?ghi\\r\\n`, + }, + } + + for _, tc := range testCases { + csvMessage := newCSVMessage(tc.config) + strBuilder := new(strings.Builder) + csvMessage.formatWithEscapes(tc.input, strBuilder) + require.Equal(t, tc.expected, strBuilder.String()) + } +} + +func TestCSVMessageEncode(t *testing.T) { + type fields struct { + config *common.Config + opType operation + tableName string + schemaName string + commitTs uint64 + preColumns []any + columns []any + HandleKey kv.Handle + } + testCases := []struct { + name string + fields fields + want []byte + }{ + { + name: "csv encode with typical configurations", + fields: fields{ + config: &common.Config{ + Delimiter: ",", + Quote: "\"", + Terminator: "\n", + NullString: "\\N", + IncludeCommitTs: true, + }, + opType: operationInsert, + tableName: "table1", + schemaName: "test", + commitTs: 435661838416609281, + columns: []any{123, "hello,world"}, + }, + want: []byte("\"I\",\"table1\",\"test\",435661838416609281,123,\"hello,world\"\n"), + }, + { + name: "csv encode values containing single-character delimter string, without quote mark", + fields: fields{ + config: &common.Config{ + Delimiter: "!", + Quote: "", + Terminator: "\n", + NullString: "\\N", + IncludeCommitTs: true, + }, + opType: operationUpdate, + tableName: "table2", + schemaName: "test", + commitTs: 435661838416609281, + columns: []any{"a!b!c", "def"}, + }, + want: []byte(`U!table2!test!435661838416609281!a\!b\!c!def` + "\n"), + }, + { + name: "csv encode values containing single-character delimter string, without quote mark, update with old value", + fields: fields{ + config: &common.Config{ + Delimiter: "!", + Quote: "", + Terminator: "\n", + NullString: "\\N", + IncludeCommitTs: true, + OutputOldValue: true, + OutputHandleKey: true, + }, + opType: operationUpdate, + tableName: "table2", + schemaName: "test", + commitTs: 435661838416609281, + preColumns: []any{"a!b!c", "abc"}, + columns: []any{"a!b!c", "def"}, + HandleKey: kv.IntHandle(1), + }, + want: []byte(`D!table2!test!435661838416609281!true!1!a\!b\!c!abc` + "\n" + + `I!table2!test!435661838416609281!true!1!a\!b\!c!def` + "\n"), + }, + { + name: "csv encode values containing single-character delimter string, without quote mark, update with old value", + fields: fields{ + config: &common.Config{ + Delimiter: "!", + Quote: "", + Terminator: "\n", + NullString: "\\N", + IncludeCommitTs: true, + OutputOldValue: true, + }, + opType: operationInsert, + tableName: "table2", + schemaName: "test", + commitTs: 435661838416609281, + columns: []any{"a!b!c", "def"}, + }, + want: []byte(`I!table2!test!435661838416609281!false!a\!b\!c!def` + "\n"), + }, + { + name: "csv encode values containing single-character delimter string, with quote mark", + fields: fields{ + config: &common.Config{ + Delimiter: ",", + Quote: "\"", + Terminator: "\n", + NullString: "\\N", + IncludeCommitTs: true, + }, + opType: operationUpdate, + tableName: "table3", + schemaName: "test", + commitTs: 435661838416609281, + columns: []any{"a,b,c", "def", "2022-08-31 17:07:00"}, + }, + want: []byte(`"U","table3","test",435661838416609281,"a,b,c","def","2022-08-31 17:07:00"` + "\n"), + }, + { + name: "csv encode values containing multi-character delimiter string, without quote mark", + fields: fields{ + config: &common.Config{ + Delimiter: "[*]", + Quote: "", + Terminator: "\r\n", + NullString: "\\N", + IncludeCommitTs: false, + }, + opType: operationDelete, + tableName: "table4", + schemaName: "test", + commitTs: 435661838416609281, + columns: []any{"a[*]b[*]c", "def"}, + }, + want: []byte(`D[*]table4[*]test[*]a\[\*\]b\[\*\]c[*]def` + "\r\n"), + }, + { + name: "csv encode with values containing multi-character delimiter string, with quote mark", + fields: fields{ + config: &common.Config{ + Delimiter: "[*]", + Quote: "'", + Terminator: "\n", + NullString: "\\N", + IncludeCommitTs: false, + }, + opType: operationInsert, + tableName: "table5", + schemaName: "test", + commitTs: 435661838416609281, + columns: []any{"a[*]b[*]c", "def", nil, 12345.678}, + }, + want: []byte(`'I'[*]'table5'[*]'test'[*]'a[*]b[*]c'[*]'def'[*]\N[*]12345.678` + "\n"), + }, + { + name: "csv encode with values containing backslash and LF, without quote mark", + fields: fields{ + config: &common.Config{ + Delimiter: ",", + Quote: "", + Terminator: "\n", + NullString: "\\N", + IncludeCommitTs: true, + }, + opType: operationUpdate, + tableName: "table6", + schemaName: "test", + commitTs: 435661838416609281, + columns: []any{"a\\b\\c", "def\n"}, + }, + want: []byte(`U,table6,test,435661838416609281,a\\b\\c,def\n` + "\n"), + }, + { + name: "csv encode with values containing backslash and CR, with quote mark", + fields: fields{ + config: &common.Config{ + Delimiter: ",", + Quote: "'", + Terminator: "\n", + NullString: "\\N", + IncludeCommitTs: false, + }, + opType: operationInsert, + tableName: "table7", + schemaName: "test", + commitTs: 435661838416609281, + columns: []any{"\\", "\\\r", "\\\\"}, + }, + want: []byte("'I','table7','test','\\','\\\r','\\\\'" + "\n"), + }, + { + name: "csv encode with values containing unicode characters", + fields: fields{ + config: &common.Config{ + Delimiter: "\t", + Quote: "\"", + Terminator: "\n", + NullString: "\\N", + IncludeCommitTs: true, + }, + opType: operationDelete, + tableName: "table8", + schemaName: "test", + commitTs: 435661838416609281, + columns: []any{"a\tb", 123.456, "你好,世界"}, + }, + want: []byte("\"D\"\t\"table8\"\t\"test\"\t435661838416609281\t\"a\tb\"\t123.456\t\"你好,世界\"\n"), + }, + } + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { + c := &csvMessage{ + config: tc.fields.config, + opType: tc.fields.opType, + tableName: tc.fields.tableName, + schemaName: tc.fields.schemaName, + commitTs: tc.fields.commitTs, + columns: tc.fields.columns, + preColumns: tc.fields.preColumns, + newRecord: true, + HandleKey: tc.fields.HandleKey, + } + + require.Equal(t, tc.want, c.encode()) + }) + } +} + +func TestConvertToCSVType(t *testing.T) { + for _, group := range csvTestColumnsGroup { + for _, c := range group { + cfg := &common.Config{BinaryEncodingMethod: c.BinaryEncodingMethod} + col := model.Column2ColumnDataXForTest(&c.col) + val, _ := fromColValToCsvVal(cfg, col, c.colInfo.Ft) + require.Equal(t, c.want, val, c.col.Name) + } + } +} + +func TestRowChangeEventConversion(t *testing.T) { + for idx, group := range csvTestColumnsGroup { + row := &model.RowChangedEvent{} + cols := make([]*model.Column, 0) + colInfos := make([]rowcodec.ColInfo, 0) + for _, c := range group { + cols = append(cols, &c.col) + colInfos = append(colInfos, c.colInfo) + } + tidbTableInfo := model.BuildTiDBTableInfo(fmt.Sprintf("table%d", idx), cols, nil) + model.AddExtraColumnInfo(tidbTableInfo, colInfos) + row.TableInfo = model.WrapTableInfo(100, "test", 100, tidbTableInfo) + + if idx%3 == 0 { // delete operation + row.PreColumns = model.Columns2ColumnDatas(cols, row.TableInfo) + } else if idx%3 == 1 { // insert operation + row.Columns = model.Columns2ColumnDatas(cols, row.TableInfo) + } else { // update operation + row.PreColumns = model.Columns2ColumnDatas(cols, row.TableInfo) + row.Columns = model.Columns2ColumnDatas(cols, row.TableInfo) + } + csvMsg, err := rowChangedEvent2CSVMsg(&common.Config{ + Delimiter: "\t", + Quote: "\"", + Terminator: "\n", + NullString: "\\N", + IncludeCommitTs: true, + BinaryEncodingMethod: group[0].BinaryEncodingMethod, + }, row) + require.NotNil(t, csvMsg) + require.Nil(t, err) + + row2, err := csvMsg2RowChangedEvent(&common.Config{ + BinaryEncodingMethod: group[0].BinaryEncodingMethod, + }, csvMsg, row.TableInfo) + require.Nil(t, err) + require.NotNil(t, row2) + } +} + +func TestCSVMessageDecode(t *testing.T) { + // datums := make([][]types.Datum, 0, 4) + testCases := []struct { + row []types.Datum + expectedCommitTs uint64 + expectedColsCnt int + expectedErr string + }{ + { + row: []types.Datum{ + types.NewStringDatum("I"), + types.NewStringDatum("employee"), + types.NewStringDatum("hr"), + types.NewStringDatum("433305438660591626"), + types.NewStringDatum("101"), + types.NewStringDatum("Smith"), + types.NewStringDatum("Bob"), + types.NewStringDatum("2014-06-04"), + types.NewDatum(nil), + }, + expectedCommitTs: 433305438660591626, + expectedColsCnt: 5, + expectedErr: "", + }, + { + row: []types.Datum{ + types.NewStringDatum("U"), + types.NewStringDatum("employee"), + types.NewStringDatum("hr"), + types.NewStringDatum("433305438660591627"), + types.NewStringDatum("101"), + types.NewStringDatum("Smith"), + types.NewStringDatum("Bob"), + types.NewStringDatum("2015-10-08"), + types.NewStringDatum("Los Angeles"), + }, + expectedCommitTs: 433305438660591627, + expectedColsCnt: 5, + expectedErr: "", + }, + { + row: []types.Datum{ + types.NewStringDatum("D"), + types.NewStringDatum("employee"), + types.NewStringDatum("hr"), + }, + expectedCommitTs: 0, + expectedColsCnt: 0, + expectedErr: "the csv row should have at least four columns", + }, + { + row: []types.Datum{ + types.NewStringDatum("D"), + types.NewStringDatum("employee"), + types.NewStringDatum("hr"), + types.NewStringDatum("hello world"), + }, + expectedCommitTs: 0, + expectedColsCnt: 0, + expectedErr: "the 4th column(hello world) of csv row should be a valid commit-ts", + }, + } + for _, tc := range testCases { + csvMsg := newCSVMessage(&common.Config{ + Delimiter: ",", + Quote: "\"", + Terminator: "\n", + NullString: "\\N", + IncludeCommitTs: true, + }) + err := csvMsg.decode(tc.row) + if tc.expectedErr != "" { + require.Contains(t, err.Error(), tc.expectedErr) + } else { + require.Nil(t, err) + require.Equal(t, tc.expectedCommitTs, csvMsg.commitTs) + require.Equal(t, tc.expectedColsCnt, len(csvMsg.columns)) + } + } +} diff --git a/pkg/sink/codec/encoder_builder.go b/pkg/sink/codec/encoder_builder.go index 64500999..bfcb55f7 100644 --- a/pkg/sink/codec/encoder_builder.go +++ b/pkg/sink/codec/encoder_builder.go @@ -20,6 +20,7 @@ import ( "github.com/pingcap/ticdc/pkg/errors" "github.com/pingcap/ticdc/pkg/sink/codec/canal" "github.com/pingcap/ticdc/pkg/sink/codec/common" + "github.com/pingcap/ticdc/pkg/sink/codec/csv" "github.com/pingcap/ticdc/pkg/sink/codec/open" ) @@ -39,3 +40,17 @@ func NewEventEncoder(ctx context.Context, cfg *common.Config) (common.EventEncod return nil, errors.ErrSinkUnknownProtocol.GenWithStackByArgs(cfg.Protocol) } } + +// NewTxnEventEncoder returns an TxnEventEncoderBuilder. +func NewTxnEventEncoder( + c *common.Config, +) (common.TxnEventEncoder, error) { + switch c.Protocol { + case config.ProtocolCsv: + return csv.NewTxnEventEncoder(c), nil + case config.ProtocolCanalJSON: + return canal.NewJSONTxnEventEncoderBuilder(c), nil + default: + return nil, errors.ErrSinkUnknownProtocol.GenWithStackByArgs(c.Protocol) + } +}