github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/cdc/sink/ddlsink/cloudstorage/cloud_storage_ddl_sink.go (about)

     1  // Copyright 2022 PingCAP, Inc.
     2  //
     3  // Licensed under the Apache License, Version 2.0 (the "License");
     4  // you may not use this file except in compliance with the License.
     5  // You may obtain a copy of the License at
     6  //
     7  //     http://www.apache.org/licenses/LICENSE-2.0
     8  //
     9  // Unless required by applicable law or agreed to in writing, software
    10  // distributed under the License is distributed on an "AS IS" BASIS,
    11  // See the License for the specific language governing permissions and
    12  // limitations under the License.
    13  
    14  package cloudstorage
    15  
    16  import (
    17  	"context"
    18  	"encoding/json"
    19  	"net/url"
    20  	"sync/atomic"
    21  	"time"
    22  
    23  	"github.com/pingcap/log"
    24  	"github.com/pingcap/tidb/br/pkg/storage"
    25  	timodel "github.com/pingcap/tidb/pkg/parser/model"
    26  	"github.com/pingcap/tiflow/cdc/model"
    27  	"github.com/pingcap/tiflow/cdc/sink/ddlsink"
    28  	"github.com/pingcap/tiflow/cdc/sink/metrics"
    29  	"github.com/pingcap/tiflow/pkg/config"
    30  	"github.com/pingcap/tiflow/pkg/errors"
    31  	"github.com/pingcap/tiflow/pkg/sink"
    32  	"github.com/pingcap/tiflow/pkg/sink/cloudstorage"
    33  	"github.com/pingcap/tiflow/pkg/util"
    34  	"github.com/robfig/cron"
    35  	"go.uber.org/zap"
    36  )
    37  
    38  // Assert Sink implementation
    39  var _ ddlsink.Sink = (*DDLSink)(nil)
    40  
    41  // DDLSink is a sink that sends DDL events to the cloud storage system.
    42  type DDLSink struct {
    43  	// id indicates which changefeed this sink belongs to.
    44  	id model.ChangeFeedID
    45  	// statistic is used to record the DDL metrics
    46  	statistics *metrics.Statistics
    47  	storage    storage.ExternalStorage
    48  	cfg        *cloudstorage.Config
    49  	cron       *cron.Cron
    50  
    51  	lastCheckpointTs         atomic.Uint64
    52  	lastSendCheckpointTsTime time.Time
    53  }
    54  
    55  // NewDDLSink creates a ddl sink for cloud storage.
    56  func NewDDLSink(ctx context.Context,
    57  	changefeedID model.ChangeFeedID,
    58  	sinkURI *url.URL,
    59  	replicaConfig *config.ReplicaConfig,
    60  ) (*DDLSink, error) {
    61  	return newDDLSink(ctx, changefeedID, sinkURI, replicaConfig, nil)
    62  }
    63  
    64  func newDDLSink(ctx context.Context,
    65  	changefeedID model.ChangeFeedID,
    66  	sinkURI *url.URL,
    67  	replicaConfig *config.ReplicaConfig,
    68  	cleanupJobs []func(), /* only for test */
    69  ) (*DDLSink, error) {
    70  	// create cloud storage config and then apply the params of sinkURI to it.
    71  	cfg := cloudstorage.NewConfig()
    72  	err := cfg.Apply(ctx, sinkURI, replicaConfig)
    73  	if err != nil {
    74  		return nil, errors.Trace(err)
    75  	}
    76  
    77  	storage, err := util.GetExternalStorageFromURI(ctx, sinkURI.String())
    78  	if err != nil {
    79  		return nil, err
    80  	}
    81  
    82  	d := &DDLSink{
    83  		id:                       changefeedID,
    84  		storage:                  storage,
    85  		statistics:               metrics.NewStatistics(ctx, changefeedID, sink.TxnSink),
    86  		cfg:                      cfg,
    87  		lastSendCheckpointTsTime: time.Now(),
    88  	}
    89  
    90  	if err := d.initCron(ctx, sinkURI, cleanupJobs); err != nil {
    91  		return nil, errors.Trace(err)
    92  	}
    93  	// Note: It is intended to run the cleanup goroutine in the background.
    94  	// we don't wait for it to finish since the gourotine would be stuck if
    95  	// the downstream is abnormal, especially when the downstream is a nfs.
    96  	go d.bgCleanup(ctx)
    97  	return d, nil
    98  }
    99  
   100  // WriteDDLEvent writes the ddl event to the cloud storage.
   101  func (d *DDLSink) WriteDDLEvent(ctx context.Context, ddl *model.DDLEvent) error {
   102  	writeFile := func(def cloudstorage.TableDefinition) error {
   103  		encodedDef, err := def.MarshalWithQuery()
   104  		if err != nil {
   105  			return errors.Trace(err)
   106  		}
   107  
   108  		path, err := def.GenerateSchemaFilePath()
   109  		if err != nil {
   110  			return errors.Trace(err)
   111  		}
   112  		log.Debug("write ddl event to external storage",
   113  			zap.String("path", path), zap.Any("ddl", ddl))
   114  		return d.statistics.RecordDDLExecution(func() error {
   115  			err1 := d.storage.WriteFile(ctx, path, encodedDef)
   116  			if err1 != nil {
   117  				return err1
   118  			}
   119  
   120  			return nil
   121  		})
   122  	}
   123  
   124  	var def cloudstorage.TableDefinition
   125  	def.FromDDLEvent(ddl, d.cfg.OutputColumnID)
   126  	if err := writeFile(def); err != nil {
   127  		return errors.Trace(err)
   128  	}
   129  
   130  	if ddl.Type == timodel.ActionExchangeTablePartition {
   131  		// For exchange partition, we need to write the schema of the source table.
   132  		var sourceTableDef cloudstorage.TableDefinition
   133  		sourceTableDef.FromTableInfo(ddl.PreTableInfo, ddl.TableInfo.Version, d.cfg.OutputColumnID)
   134  		return writeFile(sourceTableDef)
   135  	}
   136  	return nil
   137  }
   138  
   139  // WriteCheckpointTs writes the checkpoint ts to the cloud storage.
   140  func (d *DDLSink) WriteCheckpointTs(ctx context.Context,
   141  	ts uint64, tables []*model.TableInfo,
   142  ) error {
   143  	if time.Since(d.lastSendCheckpointTsTime) < 2*time.Second {
   144  		log.Debug("skip write checkpoint ts to external storage",
   145  			zap.Any("changefeedID", d.id),
   146  			zap.Uint64("ts", ts))
   147  		return nil
   148  	}
   149  
   150  	defer func() {
   151  		d.lastSendCheckpointTsTime = time.Now()
   152  		d.lastCheckpointTs.Store(ts)
   153  	}()
   154  	ckpt, err := json.Marshal(map[string]uint64{"checkpoint-ts": ts})
   155  	if err != nil {
   156  		return errors.Trace(err)
   157  	}
   158  	err = d.storage.WriteFile(ctx, "metadata", ckpt)
   159  	return errors.Trace(err)
   160  }
   161  
   162  func (d *DDLSink) initCron(
   163  	ctx context.Context, sinkURI *url.URL, cleanupJobs []func(),
   164  ) (err error) {
   165  	if cleanupJobs == nil {
   166  		cleanupJobs = d.genCleanupJob(ctx, sinkURI)
   167  	}
   168  
   169  	d.cron = cron.New()
   170  	for _, job := range cleanupJobs {
   171  		err = d.cron.AddFunc(d.cfg.FileCleanupCronSpec, job)
   172  		if err != nil {
   173  			return err
   174  		}
   175  	}
   176  	return nil
   177  }
   178  
   179  func (d *DDLSink) bgCleanup(ctx context.Context) {
   180  	if d.cfg.DateSeparator != config.DateSeparatorDay.String() || d.cfg.FileExpirationDays <= 0 {
   181  		log.Info("skip cleanup expired files for storage sink",
   182  			zap.String("namespace", d.id.Namespace),
   183  			zap.String("changefeedID", d.id.ID),
   184  			zap.String("dateSeparator", d.cfg.DateSeparator),
   185  			zap.Int("expiredFileTTL", d.cfg.FileExpirationDays))
   186  		return
   187  	}
   188  
   189  	d.cron.Start()
   190  	defer d.cron.Stop()
   191  	log.Info("start schedule cleanup expired files for storage sink",
   192  		zap.String("namespace", d.id.Namespace),
   193  		zap.String("changefeedID", d.id.ID),
   194  		zap.String("dateSeparator", d.cfg.DateSeparator),
   195  		zap.Int("expiredFileTTL", d.cfg.FileExpirationDays))
   196  
   197  	// wait for the context done
   198  	<-ctx.Done()
   199  	log.Info("stop schedule cleanup expired files for storage sink",
   200  		zap.String("namespace", d.id.Namespace),
   201  		zap.String("changefeedID", d.id.ID),
   202  		zap.Error(ctx.Err()))
   203  }
   204  
   205  func (d *DDLSink) genCleanupJob(ctx context.Context, uri *url.URL) []func() {
   206  	ret := []func(){}
   207  
   208  	isLocal := uri.Scheme == "file" || uri.Scheme == "local" || uri.Scheme == ""
   209  	isRemoveEmptyDirsRuning := atomic.Bool{}
   210  	if isLocal {
   211  		ret = append(ret, func() {
   212  			if !isRemoveEmptyDirsRuning.CompareAndSwap(false, true) {
   213  				log.Warn("remove empty dirs is already running, skip this round",
   214  					zap.String("namespace", d.id.Namespace),
   215  					zap.String("changefeedID", d.id.ID))
   216  				return
   217  			}
   218  
   219  			checkpointTs := d.lastCheckpointTs.Load()
   220  			start := time.Now()
   221  			cnt, err := cloudstorage.RemoveEmptyDirs(ctx, d.id, uri.Path)
   222  			if err != nil {
   223  				log.Error("failed to remove empty dirs",
   224  					zap.String("namespace", d.id.Namespace),
   225  					zap.String("changefeedID", d.id.ID),
   226  					zap.Uint64("checkpointTs", checkpointTs),
   227  					zap.Duration("cost", time.Since(start)),
   228  					zap.Error(err),
   229  				)
   230  				return
   231  			}
   232  			log.Info("remove empty dirs",
   233  				zap.String("namespace", d.id.Namespace),
   234  				zap.String("changefeedID", d.id.ID),
   235  				zap.Uint64("checkpointTs", checkpointTs),
   236  				zap.Uint64("count", cnt),
   237  				zap.Duration("cost", time.Since(start)))
   238  		})
   239  	}
   240  
   241  	isCleanupRunning := atomic.Bool{}
   242  	ret = append(ret, func() {
   243  		if !isCleanupRunning.CompareAndSwap(false, true) {
   244  			log.Warn("cleanup expired files is already running, skip this round",
   245  				zap.String("namespace", d.id.Namespace),
   246  				zap.String("changefeedID", d.id.ID))
   247  			return
   248  		}
   249  
   250  		defer isCleanupRunning.Store(false)
   251  		start := time.Now()
   252  		checkpointTs := d.lastCheckpointTs.Load()
   253  		cnt, err := cloudstorage.RemoveExpiredFiles(ctx, d.id, d.storage, d.cfg, checkpointTs)
   254  		if err != nil {
   255  			log.Error("failed to remove expired files",
   256  				zap.String("namespace", d.id.Namespace),
   257  				zap.String("changefeedID", d.id.ID),
   258  				zap.Uint64("checkpointTs", checkpointTs),
   259  				zap.Duration("cost", time.Since(start)),
   260  				zap.Error(err),
   261  			)
   262  			return
   263  		}
   264  		log.Info("remove expired files",
   265  			zap.String("namespace", d.id.Namespace),
   266  			zap.String("changefeedID", d.id.ID),
   267  			zap.Uint64("checkpointTs", checkpointTs),
   268  			zap.Uint64("count", cnt),
   269  			zap.Duration("cost", time.Since(start)))
   270  	})
   271  	return ret
   272  }
   273  
   274  // Close closes the sink.
   275  func (d *DDLSink) Close() {
   276  	if d.statistics != nil {
   277  		d.statistics.Close()
   278  	}
   279  }