diff --git a/cdc/api/v2/api.go b/cdc/api/v2/api.go index 5bd47aad907..b5ff726d012 100644 --- a/cdc/api/v2/api.go +++ b/cdc/api/v2/api.go @@ -57,17 +57,18 @@ func RegisterOpenAPIV2Routes(router *gin.Engine, api OpenAPIV2) { changefeedGroup.POST("/:changefeed_id/resume", api.resumeChangefeed) changefeedGroup.POST("/:changefeed_id/pause", api.pauseChangefeed) - // processor apis - processorGroup := v2.Group("/processors") - processorGroup.Use(middleware.ForwardToOwnerMiddleware(api.capture)) - processorGroup.GET("/:changefeed_id/:capture_id", api.getProcessor) - // capture apis captureGroup := v2.Group("/captures") captureGroup.Use(middleware.ForwardToOwnerMiddleware(api.capture)) captureGroup.POST("/:capture_id/drain", api.drainCapture) captureGroup.GET("", api.listCaptures) + // processor apis + processorGroup := v2.Group("/processors") + processorGroup.Use(middleware.ForwardToOwnerMiddleware(api.capture)) + processorGroup.GET("/:changefeed_id/:capture_id", api.getProcessor) + processorGroup.GET("", api.listProcessors) + verifyTableGroup := v2.Group("/verify_table") verifyTableGroup.Use(middleware.ForwardToOwnerMiddleware(api.capture)) verifyTableGroup.POST("", api.verifyTable) diff --git a/cdc/api/v2/model.go b/cdc/api/v2/model.go index 3408daeea03..7a77197ae3a 100644 --- a/cdc/api/v2/model.go +++ b/cdc/api/v2/model.go @@ -101,6 +101,13 @@ type ChangefeedConfig struct { PDConfig } +// ProcessorCommonInfo holds the common info of a processor +type ProcessorCommonInfo struct { + Namespace string `json:"namespace"` + ChangeFeedID string `json:"changefeed_id"` + CaptureID string `json:"capture_id"` +} + // ReplicaConfig is a duplicate of config.ReplicaConfig type ReplicaConfig struct { MemoryQuota uint64 `json:"memory_quota"` @@ -553,7 +560,8 @@ type ChangeFeedInfo struct { CreatorVersion string `json:"creator_version,omitempty"` } -// RunningError represents some running error from cdc components, such as processor. +// RunningError represents some running error from cdc components, +// such as processor. type RunningError struct { Addr string `json:"addr"` Code string `json:"code"` diff --git a/cdc/api/v2/processor.go b/cdc/api/v2/processor.go index 4f899459d86..6289483122f 100644 --- a/cdc/api/v2/processor.go +++ b/cdc/api/v2/processor.go @@ -53,7 +53,11 @@ func (h *OpenAPIV2) getProcessor(c *gin.Context) { ) return } - info, err := h.capture.StatusProvider().GetChangeFeedInfo(ctx, changefeedID) + + info, err := h.capture.StatusProvider().GetChangeFeedInfo( + ctx, + changefeedID, + ) if err != nil { _ = c.Error(err) return @@ -63,7 +67,7 @@ func (h *OpenAPIV2) getProcessor(c *gin.Context) { cerror.WrapError( cerror.ErrAPIInvalidParam, fmt.Errorf("changefeed in abnormal state: %s, "+ - "can't get processors of an abnormal changefeed", + "can't get processor of an abnormal changefeed", string(info.State), ), ), @@ -111,3 +115,35 @@ func (h *OpenAPIV2) getProcessor(c *gin.Context) { } c.JSON(http.StatusOK, &processorDetail) } + +// listProcessors lists all processors in the TiCDC cluster +// @Summary List processors +// @Description list all processors in the TiCDC cluster +// @Tags processor +// @Produce json +// @Success 200 {array} model.ProcessorCommonInfo +// @Failure 500,400 {object} model.HTTPError +// @Router /api/v2/processors [get] +func (h *OpenAPIV2) listProcessors(c *gin.Context) { + ctx := c.Request.Context() + infos, err := h.capture.StatusProvider().GetProcessors(ctx) + if err != nil { + _ = c.Error(err) + return + } + prcInfos := make([]ProcessorCommonInfo, 0, len(infos)) + for i, info := range infos { + resp := ProcessorCommonInfo{ + Namespace: info.CfID.Namespace, + ChangeFeedID: info.CfID.ID, + CaptureID: info.CaptureID, + } + prcInfos[i] = resp + } + resp := &ListResponse[ProcessorCommonInfo]{ + Total: len(prcInfos), + Items: prcInfos, + } + + c.JSON(http.StatusOK, resp) +} diff --git a/cdc/capture/capture.go b/cdc/capture/capture.go index e1f587cca34..f7444755132 100644 --- a/cdc/capture/capture.go +++ b/cdc/capture/capture.go @@ -28,7 +28,6 @@ import ( "github.com/pingcap/tiflow/cdc/owner" "github.com/pingcap/tiflow/cdc/processor" "github.com/pingcap/tiflow/cdc/processor/sourcemanager/engine/factory" - ssystem "github.com/pingcap/tiflow/cdc/sorter/db/system" "github.com/pingcap/tiflow/pkg/config" cdcContext "github.com/pingcap/tiflow/pkg/context" cerror "github.com/pingcap/tiflow/pkg/errors" @@ -95,11 +94,6 @@ type captureImpl struct { createEtcdClient createEtcdClientFunc EtcdClient etcd.CDCEtcdClient - // useSortEngine indicates whether to use the new pull based sort engine or - // the old push based sorter system. the latter will be removed after all sorter - // have been transformed into pull based sort engine. - useSortEngine bool - sorterSystem *ssystem.System sortEngineFactory *factory.SortEngineFactory // MessageServer is the receiver of the messages from the other nodes. @@ -134,7 +128,6 @@ func NewCapture(pdEndpoints []string, createEtcdClient createEtcdClientFunc, grpcService *p2p.ServerWrapper, sortEngineMangerFactory *factory.SortEngineFactory, - sorterSystem *ssystem.System, ) Capture { return &captureImpl{ config: config.GetGlobalServerConfig(), @@ -147,10 +140,7 @@ func NewCapture(pdEndpoints []string, newOwner: owner.NewOwner, info: &model.CaptureInfo{}, createEtcdClient: createEtcdClient, - - useSortEngine: sortEngineMangerFactory != nil, - sortEngineFactory: sortEngineMangerFactory, - sorterSystem: sorterSystem, + sortEngineFactory: sortEngineMangerFactory, } } @@ -328,7 +318,6 @@ func (c *captureImpl) run(stdCtx context.Context) error { EtcdClient: c.EtcdClient, MessageServer: c.MessageServer, MessageRouter: c.MessageRouter, - SorterSystem: c.sorterSystem, SortEngineFactory: c.sortEngineFactory, }) diff --git a/cdc/processor/metrics.go b/cdc/processor/metrics.go index e85bc102e9f..1cfe4055384 100644 --- a/cdc/processor/metrics.go +++ b/cdc/processor/metrics.go @@ -56,16 +56,6 @@ var ( Help: "Bucketed histogram of processorManager close processor time (s).", Buckets: prometheus.ExponentialBuckets(0.01 /* 10 ms */, 2, 18), }) - - tableMemoryHistogram = prometheus.NewHistogramVec( - prometheus.HistogramOpts{ - Namespace: "ticdc", - Subsystem: "processor", - Name: "table_memory_consumption", - Help: "each table's memory consumption after sorter, in bytes", - Buckets: prometheus.ExponentialBuckets(256, 2.0, 20), - }, []string{"namespace", "changefeed"}) - processorMemoryGauge = prometheus.NewGaugeVec( prometheus.GaugeOpts{ Namespace: "ticdc", @@ -73,7 +63,6 @@ var ( Name: "memory_consumption", Help: "processor's memory consumption estimated in bytes", }, []string{"namespace", "changefeed"}) - remainKVEventsGauge = prometheus.NewGaugeVec( prometheus.GaugeOpts{ Namespace: "ticdc", @@ -90,7 +79,6 @@ func InitMetrics(registry *prometheus.Registry) { registry.MustRegister(processorSchemaStorageGcTsGauge) registry.MustRegister(processorTickDuration) registry.MustRegister(processorCloseDuration) - registry.MustRegister(tableMemoryHistogram) registry.MustRegister(processorMemoryGauge) registry.MustRegister(remainKVEventsGauge) sinkmanager.InitMetrics(registry) diff --git a/cdc/processor/processor.go b/cdc/processor/processor.go index d914bbc2d5c..c1c6e330b24 100644 --- a/cdc/processor/processor.go +++ b/cdc/processor/processor.go @@ -91,7 +91,6 @@ type processor struct { metricProcessorErrorCounter prometheus.Counter metricProcessorTickDuration prometheus.Observer metricsTableSinkTotalRows prometheus.Counter - metricsTableMemoryHistogram prometheus.Observer metricsProcessorMemoryGauge prometheus.Gauge metricRemainKVEventGauge prometheus.Gauge } @@ -424,8 +423,6 @@ func newProcessor( WithLabelValues(changefeedID.Namespace, changefeedID.ID), metricsTableSinkTotalRows: sinkmetric.TableSinkTotalRowsCountCounter. WithLabelValues(changefeedID.Namespace, changefeedID.ID), - metricsTableMemoryHistogram: tableMemoryHistogram. - WithLabelValues(changefeedID.Namespace, changefeedID.ID), metricsProcessorMemoryGauge: processorMemoryGauge. WithLabelValues(changefeedID.Namespace, changefeedID.ID), metricRemainKVEventGauge: remainKVEventsGauge. @@ -1016,12 +1013,8 @@ func (p *processor) cleanupMetrics() { processorErrorCounter.DeleteLabelValues(p.changefeedID.Namespace, p.changefeedID.ID) processorSchemaStorageGcTsGauge.DeleteLabelValues(p.changefeedID.Namespace, p.changefeedID.ID) processorTickDuration.DeleteLabelValues(p.changefeedID.Namespace, p.changefeedID.ID) - - tableMemoryHistogram.DeleteLabelValues(p.changefeedID.Namespace, p.changefeedID.ID) processorMemoryGauge.DeleteLabelValues(p.changefeedID.Namespace, p.changefeedID.ID) - remainKVEventsGauge.DeleteLabelValues(p.changefeedID.Namespace, p.changefeedID.ID) - sinkmetric.TableSinkTotalRowsCountCounter. DeleteLabelValues(p.changefeedID.Namespace, p.changefeedID.ID) } diff --git a/cdc/processor/sinkmanager/metrics.go b/cdc/processor/sinkmanager/metrics.go index bfc9d7f36ee..d8f10e80117 100644 --- a/cdc/processor/sinkmanager/metrics.go +++ b/cdc/processor/sinkmanager/metrics.go @@ -49,6 +49,14 @@ var ( }, // type includes hit and miss. []string{"namespace", "changefeed", "type"}) + + // outputEventCount is the metric that counts events output by the sorter. + outputEventCount = prometheus.NewCounterVec(prometheus.CounterOpts{ + Namespace: "ticdc", + Subsystem: "sorter", + Name: "output_event_count", + Help: "The number of events output by the sorter", + }, []string{"namespace", "changefeed", "type"}) ) // InitMetrics registers all metrics in this file. @@ -56,4 +64,5 @@ func InitMetrics(registry *prometheus.Registry) { registry.MustRegister(MemoryQuota) registry.MustRegister(RedoEventCache) registry.MustRegister(RedoEventCacheAccess) + registry.MustRegister(outputEventCount) } diff --git a/cdc/processor/sinkmanager/table_sink_worker.go b/cdc/processor/sinkmanager/table_sink_worker.go index 459e0cfd80b..4a974f8a89d 100644 --- a/cdc/processor/sinkmanager/table_sink_worker.go +++ b/cdc/processor/sinkmanager/table_sink_worker.go @@ -21,7 +21,6 @@ import ( "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/cdc/processor/sourcemanager" "github.com/pingcap/tiflow/cdc/processor/sourcemanager/engine" - metrics "github.com/pingcap/tiflow/cdc/sorter" "github.com/prometheus/client_golang/prometheus" "go.uber.org/zap" ) @@ -155,7 +154,7 @@ func (w *sinkWorker) handleTask(ctx context.Context, task *sinkTask) (finalErr e // All transactions before currTxnCommitTs are resolved. err = w.advanceTableSink(task, currTxnCommitTs, committedTxnSize+pendingTxnSize) } else { - // This means all events of the currenet transaction have been fetched, but we can't + // This means all events of the current transaction have been fetched, but we can't // ensure whether there are more transaction with the same CommitTs or not. err = w.advanceTableSinkWithBatchID(task, currTxnCommitTs, committedTxnSize+pendingTxnSize, batchID) batchID += 1 @@ -164,7 +163,7 @@ func (w *sinkWorker) handleTask(ctx context.Context, task *sinkTask) (finalErr e pendingTxnSize = 0 } else if w.splitTxn && currTxnCommitTs > 0 { // This branch will advance some complete transactions before currTxnCommitTs, - // and one partail transaction with `batchID`. + // and one partial transaction with `batchID`. err = w.advanceTableSinkWithBatchID(task, currTxnCommitTs, committedTxnSize+pendingTxnSize, batchID) batchID += 1 committedTxnSize = 0 @@ -251,7 +250,7 @@ func (w *sinkWorker) handleTask(ctx context.Context, task *sinkTask) (finalErr e defer func() { w.metricRedoEventCacheMiss.Add(float64(allEventSize)) task.tableSink.receivedEventCount.Add(int64(allEventCount)) - metrics.OutputEventCount.WithLabelValues( + outputEventCount.WithLabelValues( task.tableSink.changefeed.Namespace, task.tableSink.changefeed.ID, "kv", @@ -360,7 +359,7 @@ func (w *sinkWorker) fetchFromCache( newLowerBound = popRes.boundary.Next() if len(popRes.events) > 0 { task.tableSink.receivedEventCount.Add(int64(popRes.pushCount)) - metrics.OutputEventCount.WithLabelValues( + outputEventCount.WithLabelValues( task.tableSink.changefeed.Namespace, task.tableSink.changefeed.ID, "kv", diff --git a/cdc/processor/sourcemanager/engine/factory/factory.go b/cdc/processor/sourcemanager/engine/factory/factory.go index e1c373dd518..8ba10344b5e 100644 --- a/cdc/processor/sourcemanager/engine/factory/factory.go +++ b/cdc/processor/sourcemanager/engine/factory/factory.go @@ -25,9 +25,7 @@ import ( "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/cdc/processor/sourcemanager/engine" epebble "github.com/pingcap/tiflow/cdc/processor/sourcemanager/engine/pebble" - metrics "github.com/pingcap/tiflow/cdc/sorter" "github.com/pingcap/tiflow/pkg/config" - dbMetrics "github.com/pingcap/tiflow/pkg/db" "go.uber.org/atomic" "go.uber.org/multierr" ) @@ -171,17 +169,20 @@ func (f *SortEngineFactory) collectMetrics() { for i, db := range f.dbs { stats := db.Metrics() id := strconv.Itoa(i + 1) - metrics.OnDiskDataSizeGauge.WithLabelValues(id).Set(float64(stats.DiskSpaceUsage())) - metrics.InMemoryDataSizeGauge.WithLabelValues(id).Set(float64(stats.BlockCache.Size)) - dbMetrics.IteratorGauge().WithLabelValues(id).Set(float64(stats.TableIters)) - dbMetrics.WriteDelayCount().WithLabelValues(id).Set(float64(stdatomic.LoadUint64(&f.writeStalls[i].counter))) + engine.OnDiskDataSize().WithLabelValues(id).Set(float64(stats.DiskSpaceUsage())) + engine.InMemoryDataSize().WithLabelValues(id).Set(float64(stats.BlockCache.Size)) + engine.IteratorGauge().WithLabelValues(id).Set(float64(stats.TableIters)) + engine.WriteDelayCount().WithLabelValues(id). + Set(float64(stdatomic.LoadUint64(&f.writeStalls[i].counter))) - metricLevelCount := dbMetrics.LevelCount().MustCurryWith(map[string]string{"id": id}) + metricLevelCount := engine.LevelCount().MustCurryWith(map[string]string{"id": id}) for level, metric := range stats.Levels { metricLevelCount.WithLabelValues(fmt.Sprint(level)).Set(float64(metric.NumFiles)) } - dbMetrics.BlockCacheAccess().WithLabelValues(id, "hit").Set(float64(stats.BlockCache.Hits)) - dbMetrics.BlockCacheAccess().WithLabelValues(id, "miss").Set(float64(stats.BlockCache.Misses)) + engine.BlockCacheAccess().WithLabelValues(id, "hit"). + Set(float64(stats.BlockCache.Hits)) + engine.BlockCacheAccess().WithLabelValues(id, "miss"). + Set(float64(stats.BlockCache.Misses)) } } } diff --git a/cdc/processor/sourcemanager/engine/factory/pebble.go b/cdc/processor/sourcemanager/engine/factory/pebble.go index 6b8bdf39b9f..d3a71a35305 100644 --- a/cdc/processor/sourcemanager/engine/factory/pebble.go +++ b/cdc/processor/sourcemanager/engine/factory/pebble.go @@ -21,8 +21,8 @@ import ( "github.com/cockroachdb/pebble" "github.com/pingcap/log" + "github.com/pingcap/tiflow/cdc/processor/sourcemanager/engine" epebble "github.com/pingcap/tiflow/cdc/processor/sourcemanager/engine/pebble" - metrics "github.com/pingcap/tiflow/cdc/sorter/db" "github.com/pingcap/tiflow/pkg/config" "go.uber.org/zap" ) @@ -52,7 +52,7 @@ func createPebbleDBs( } opts.EventListener.CompactionEnd = func(job pebble.CompactionInfo) { idstr := strconv.Itoa(id + 1) - x := metrics.SorterCompactionDuration().WithLabelValues(idstr) + x := engine.SorterCompactionDuration().WithLabelValues(idstr) x.Observe(job.TotalDuration.Seconds()) } } diff --git a/cdc/sorter/db/metrics.go b/cdc/processor/sourcemanager/engine/metrics.go similarity index 51% rename from cdc/sorter/db/metrics.go rename to cdc/processor/sourcemanager/engine/metrics.go index ec08d54a5ea..28aa13e74a6 100644 --- a/cdc/sorter/db/metrics.go +++ b/cdc/processor/sourcemanager/engine/metrics.go @@ -1,4 +1,4 @@ -// Copyright 2021 PingCAP, Inc. +// 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. @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package db +package engine import ( "github.com/prometheus/client_golang/prometheus" @@ -49,6 +49,50 @@ var ( Help: "Bucketed histogram of db sorter iterator read duration", Buckets: prometheus.ExponentialBuckets(0.004, 2.0, 20), }, []string{"namespace", "id", "call"}) + + // inMemoryDataSizeGauge is the metric that records sorter memory usage. + inMemoryDataSizeGauge = prometheus.NewGaugeVec(prometheus.GaugeOpts{ + Namespace: "ticdc", + Subsystem: "sorter", + Name: "in_memory_data_size_gauge", + Help: "The amount of pending data stored in-memory by the sorter", + }, []string{"id"}) + + // onDiskDataSizeGauge is the metric that records sorter disk usage. + onDiskDataSizeGauge = prometheus.NewGaugeVec(prometheus.GaugeOpts{ + Namespace: "ticdc", + Subsystem: "sorter", + Name: "on_disk_data_size_gauge", + Help: "The amount of pending data stored on-disk by the sorter", + }, []string{"id"}) + + dbIteratorGauge = prometheus.NewGaugeVec(prometheus.GaugeOpts{ + Namespace: "ticdc", + Subsystem: "db", + Name: "iterator_count_gauge", + Help: "The number of iterator by the db", + }, []string{"id"}) + + dbLevelCount = prometheus.NewGaugeVec(prometheus.GaugeOpts{ + Namespace: "ticdc", + Subsystem: "db", + Name: "level_count", + Help: "The number of files in each level by the db", + }, []string{"level", "id"}) + + dbWriteDelayCount = prometheus.NewGaugeVec(prometheus.GaugeOpts{ + Namespace: "ticdc", + Subsystem: "db", + Name: "write_delay_total", + Help: "The total number of db delay", + }, []string{"id"}) + + dbBlockCacheAccess = prometheus.NewGaugeVec(prometheus.GaugeOpts{ + Namespace: "ticdc", + Subsystem: "db", + Name: "block_cache_access_total", + Help: "The total number of db block cache access", + }, []string{"id", "type"}) ) /* Some metrics are shared in pipeline sorter and pull-based-sink sort engine */ @@ -68,10 +112,46 @@ func SorterWriteBytes() *prometheus.HistogramVec { return sorterWriteBytesHistogram } +// InMemoryDataSize returns inMemoryDataSizeGauge. +func InMemoryDataSize() *prometheus.GaugeVec { + return inMemoryDataSizeGauge +} + +// OnDiskDataSize returns onDiskDataSizeGauge. +func OnDiskDataSize() *prometheus.GaugeVec { + return onDiskDataSizeGauge +} + +// IteratorGauge returns dbIteratorGauge. +func IteratorGauge() *prometheus.GaugeVec { + return dbIteratorGauge +} + +// WriteDelayCount returns dbWriteDelayCount. +func WriteDelayCount() *prometheus.GaugeVec { + return dbWriteDelayCount +} + +// LevelCount returns dbLevelCount. +func LevelCount() *prometheus.GaugeVec { + return dbLevelCount +} + +// BlockCacheAccess returns dbBlockCacheAccess. +func BlockCacheAccess() *prometheus.GaugeVec { + return dbBlockCacheAccess +} + // InitMetrics registers all metrics in this file func InitMetrics(registry *prometheus.Registry) { registry.MustRegister(sorterWriteDurationHistogram) registry.MustRegister(sorterCompactDurationHistogram) registry.MustRegister(sorterWriteBytesHistogram) registry.MustRegister(sorterIterReadDurationHistogram) + registry.MustRegister(inMemoryDataSizeGauge) + registry.MustRegister(onDiskDataSizeGauge) + registry.MustRegister(dbIteratorGauge) + registry.MustRegister(dbLevelCount) + registry.MustRegister(dbWriteDelayCount) + registry.MustRegister(dbBlockCacheAccess) } diff --git a/cdc/processor/sourcemanager/engine/pebble/event_sorter.go b/cdc/processor/sourcemanager/engine/pebble/event_sorter.go index f467a11885e..004e576cfcf 100644 --- a/cdc/processor/sourcemanager/engine/pebble/event_sorter.go +++ b/cdc/processor/sourcemanager/engine/pebble/event_sorter.go @@ -28,7 +28,6 @@ import ( "github.com/pingcap/tiflow/cdc/processor/sourcemanager/engine" "github.com/pingcap/tiflow/cdc/processor/sourcemanager/engine/pebble/encoding" "github.com/pingcap/tiflow/cdc/processor/tablepb" - metrics "github.com/pingcap/tiflow/cdc/sorter/db" "github.com/pingcap/tiflow/pkg/chann" "github.com/pingcap/tiflow/pkg/spanz" "go.uber.org/zap" @@ -398,8 +397,8 @@ func (s *EventSorter) handleEvents( fetchTokens, ioTokens chan struct{}, ) { idstr := strconv.Itoa(id + 1) - writeDuration := metrics.SorterWriteDuration().WithLabelValues(idstr) - writeBytes := metrics.SorterWriteBytes().WithLabelValues(idstr) + writeDuration := engine.SorterWriteDuration().WithLabelValues(idstr) + writeBytes := engine.SorterWriteBytes().WithLabelValues(idstr) batch := db.NewBatch() writeOpts := &pebble.WriteOptions{Sync: false} @@ -552,8 +551,8 @@ func (s *EventSorter) cleanTable( // ----- Some internal variable and functions ----- const ( - batchCommitSize int = 16 * 1024 * 1024 - batchCommitInterval time.Duration = 20 * time.Millisecond + batchCommitSize int = 16 * 1024 * 1024 + batchCommitInterval = 20 * time.Millisecond ) var uniqueIDGen uint32 = 0 diff --git a/cdc/puller/ddl_puller.go b/cdc/puller/ddl_puller.go index e05ab6cac49..68be10612bb 100644 --- a/cdc/puller/ddl_puller.go +++ b/cdc/puller/ddl_puller.go @@ -31,7 +31,7 @@ import ( "github.com/pingcap/tiflow/cdc/entry/schema" "github.com/pingcap/tiflow/cdc/kv" "github.com/pingcap/tiflow/cdc/model" - "github.com/pingcap/tiflow/cdc/sorter/memory" + "github.com/pingcap/tiflow/cdc/puller/memorysorter" "github.com/pingcap/tiflow/pkg/config" cerror "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/filter" @@ -92,7 +92,7 @@ func (p *ddlJobPullerImpl) Run(ctx context.Context) error { return errors.Trace(p.puller.Run(ctx)) }) - rawDDLCh := memory.SortOutput(ctx, p.puller.Output()) + rawDDLCh := memorysorter.SortOutput(ctx, p.puller.Output()) eg.Go( func() error { for { diff --git a/cdc/sorter/memory/doc.go b/cdc/puller/memorysorter/doc.go similarity index 85% rename from cdc/sorter/memory/doc.go rename to cdc/puller/memorysorter/doc.go index 13cbaf3feab..a09df5ee113 100644 --- a/cdc/sorter/memory/doc.go +++ b/cdc/puller/memorysorter/doc.go @@ -11,5 +11,5 @@ // See the License for the specific language governing permissions and // limitations under the License. -// Package memory is an in-memory EventSorter implementation. -package memory +// Package memorysorter is an in-memory event sorter implementation. +package memorysorter diff --git a/cdc/sorter/memory/entry_sorter.go b/cdc/puller/memorysorter/entry_sorter.go similarity index 96% rename from cdc/sorter/memory/entry_sorter.go rename to cdc/puller/memorysorter/entry_sorter.go index f49f4d93c68..2559f92856a 100644 --- a/cdc/sorter/memory/entry_sorter.go +++ b/cdc/puller/memorysorter/entry_sorter.go @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package memory +package memorysorter import ( "context" @@ -24,7 +24,6 @@ import ( "github.com/pingcap/log" "github.com/pingcap/tiflow/cdc/contextutil" "github.com/pingcap/tiflow/cdc/model" - "github.com/pingcap/tiflow/cdc/sorter" "github.com/pingcap/tiflow/pkg/notify" "go.uber.org/zap" "golang.org/x/sync/errgroup" @@ -155,12 +154,6 @@ func (es *EntrySorter) Output() <-chan *model.PolymorphicEvent { return es.outputCh } -// EmitStartTs implement sorter interface -func (es *EntrySorter) EmitStartTs(_ context.Context, _ uint64) {} - -// Stats implement sorter interface -func (es *EntrySorter) Stats() sorter.Stats { return sorter.Stats{} } - func eventLess(i *model.PolymorphicEvent, j *model.PolymorphicEvent) bool { return model.ComparePolymorphicEvents(i, j) } diff --git a/cdc/sorter/memory/entry_sorter_test.go b/cdc/puller/memorysorter/entry_sorter_test.go similarity index 99% rename from cdc/sorter/memory/entry_sorter_test.go rename to cdc/puller/memorysorter/entry_sorter_test.go index 29d83c27a72..178cc5da936 100644 --- a/cdc/sorter/memory/entry_sorter_test.go +++ b/cdc/puller/memorysorter/entry_sorter_test.go @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package memory +package memorysorter import ( "context" diff --git a/cdc/sorter/memory/main_test.go b/cdc/puller/memorysorter/main_test.go similarity index 96% rename from cdc/sorter/memory/main_test.go rename to cdc/puller/memorysorter/main_test.go index 37efe307b32..bd423096999 100644 --- a/cdc/sorter/memory/main_test.go +++ b/cdc/puller/memorysorter/main_test.go @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package memory +package memorysorter import ( "testing" diff --git a/cdc/sorter/memory/metrics.go b/cdc/puller/memorysorter/metrics.go similarity index 99% rename from cdc/sorter/memory/metrics.go rename to cdc/puller/memorysorter/metrics.go index 72874bdc428..4aa4a55b531 100644 --- a/cdc/sorter/memory/metrics.go +++ b/cdc/puller/memorysorter/metrics.go @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package memory +package memorysorter import ( "time" diff --git a/cdc/redo/reader/file_test.go b/cdc/redo/reader/file_test.go index 2be067047b6..a4d6eac8024 100644 --- a/cdc/redo/reader/file_test.go +++ b/cdc/redo/reader/file_test.go @@ -15,6 +15,7 @@ package reader import ( "bufio" + "context" "fmt" "io" "os" @@ -27,7 +28,6 @@ import ( "github.com/pingcap/tiflow/pkg/redo" "github.com/pingcap/tiflow/pkg/uuid" "github.com/stretchr/testify/require" - "golang.org/x/net/context" ) func TestReaderNewReader(t *testing.T) { diff --git a/cdc/server/metrics.go b/cdc/server/metrics.go index 98df0552d61..a9e455f56b2 100644 --- a/cdc/server/metrics.go +++ b/cdc/server/metrics.go @@ -18,18 +18,15 @@ import ( "github.com/pingcap/tiflow/cdc/kv" "github.com/pingcap/tiflow/cdc/owner" "github.com/pingcap/tiflow/cdc/processor" + "github.com/pingcap/tiflow/cdc/processor/sourcemanager/engine" "github.com/pingcap/tiflow/cdc/puller" + "github.com/pingcap/tiflow/cdc/puller/memorysorter" redo "github.com/pingcap/tiflow/cdc/redo/common" "github.com/pingcap/tiflow/cdc/scheduler" sink "github.com/pingcap/tiflow/cdc/sink/metrics" "github.com/pingcap/tiflow/cdc/sink/mq/producer/kafka" sinkv2 "github.com/pingcap/tiflow/cdc/sinkv2/metrics" - "github.com/pingcap/tiflow/cdc/sorter" - dbsroter "github.com/pingcap/tiflow/cdc/sorter/db" - "github.com/pingcap/tiflow/cdc/sorter/memory" - "github.com/pingcap/tiflow/cdc/sorter/unified" "github.com/pingcap/tiflow/pkg/actor" - "github.com/pingcap/tiflow/pkg/db" "github.com/pingcap/tiflow/pkg/etcd" "github.com/pingcap/tiflow/pkg/orchestrator" "github.com/pingcap/tiflow/pkg/p2p" @@ -58,12 +55,9 @@ func init() { actor.InitMetrics(registry) orchestrator.InitMetrics(registry) p2p.InitMetrics(registry) - sorter.InitMetrics(registry) - memory.InitMetrics(registry) - unified.InitMetrics(registry) - dbsroter.InitMetrics(registry) + engine.InitMetrics(registry) + memorysorter.InitMetrics(registry) redo.InitMetrics(registry) - db.InitMetrics(registry) kafka.InitMetrics(registry) scheduler.InitMetrics(registry) observer.InitMetrics(registry) diff --git a/cdc/server/server.go b/cdc/server/server.go index 88058f6a10d..c1dfa1ab5f4 100644 --- a/cdc/server/server.go +++ b/cdc/server/server.go @@ -31,8 +31,6 @@ import ( "github.com/pingcap/tiflow/cdc/capture" "github.com/pingcap/tiflow/cdc/kv" "github.com/pingcap/tiflow/cdc/processor/sourcemanager/engine/factory" - ssystem "github.com/pingcap/tiflow/cdc/sorter/db/system" - "github.com/pingcap/tiflow/cdc/sorter/unified" "github.com/pingcap/tiflow/pkg/config" cerror "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/etcd" @@ -77,17 +75,13 @@ type Server interface { // TODO: we need to make server more unit testable and add more test cases. // Especially we need to decouple the HTTPServer out of server. type server struct { - capture capture.Capture - tcpServer tcpserver.TCPServer - grpcService *p2p.ServerWrapper - statusServer *http.Server - etcdClient etcd.CDCEtcdClient - pdEndpoints []string - - // If it's true sortEngineManager will be used, otherwise sorterSystem will be used. - useEventSortEngine bool - sortEngineFactory *factory.SortEngineFactory - sorterSystem *ssystem.System + capture capture.Capture + tcpServer tcpserver.TCPServer + grpcService *p2p.ServerWrapper + statusServer *http.Server + etcdClient etcd.CDCEtcdClient + pdEndpoints []string + sortEngineFactory *factory.SortEngineFactory } // New creates a server instance. @@ -112,16 +106,11 @@ func New(pdEndpoints []string) (*server, error) { return nil, errors.Trace(err) } - // TODO(qupeng): adjust it after all sorters are transformed into EventSortEngine. debugConfig := config.GetGlobalServerConfig().Debug - useEventSortEngine := debugConfig.EnablePullBasedSink && debugConfig.EnableDBSorter - s := &server{ pdEndpoints: pdEndpoints, grpcService: p2p.NewServerWrapper(debugConfig.Messages.ToMessageServerConfig()), tcpServer: tcpServer, - - useEventSortEngine: useEventSortEngine, } log.Info("CDC server created", @@ -189,57 +178,36 @@ func (s *server) prepare(ctx context.Context) error { return errors.Trace(err) } - if err := s.startActorSystems(ctx); err != nil { + if err := s.createSortEngineFactory(); err != nil { return errors.Trace(err) } s.capture = capture.NewCapture( s.pdEndpoints, createEtcdClient, s.grpcService, - s.sortEngineFactory, s.sorterSystem) + s.sortEngineFactory) return nil } -func (s *server) startActorSystems(ctx context.Context) error { +func (s *server) createSortEngineFactory() error { conf := config.GetGlobalServerConfig() - if !conf.Debug.EnableDBSorter { - return nil - } - - if s.useEventSortEngine && s.sortEngineFactory != nil { + if s.sortEngineFactory != nil { if err := s.sortEngineFactory.Close(); err != nil { log.Error("fails to close sort engine manager", zap.Error(err)) } s.sortEngineFactory = nil } - if !s.useEventSortEngine && s.sorterSystem != nil { - s.sorterSystem.Stop() - } // Sorter dir has been set and checked when server starts. // See https://github.com/pingcap/tiflow/blob/9dad09/cdc/server.go#L275 sortDir := config.GetGlobalServerConfig().Sorter.SortDir - - if s.useEventSortEngine { - totalMemory, err := memory.MemTotal() - if err != nil { - return errors.Trace(err) - } - memPercentage := float64(conf.Sorter.MaxMemoryPercentage) / 100 - memInBytes := uint64(float64(totalMemory) * memPercentage) - if config.GetGlobalServerConfig().Debug.EnableDBSorter { - s.sortEngineFactory = factory.NewForPebble(sortDir, memInBytes, conf.Debug.DB) - } else { - panic("only pebble is transformed to EventSortEngine") - } - } else { - memPercentage := float64(conf.Sorter.MaxMemoryPercentage) / 100 - s.sorterSystem = ssystem.NewSystem(sortDir, memPercentage, conf.Debug.DB) - err := s.sorterSystem.Start(ctx) - if err != nil { - return errors.Trace(err) - } + totalMemory, err := memory.MemTotal() + if err != nil { + return errors.Trace(err) } + memPercentage := float64(conf.Sorter.MaxMemoryPercentage) / 100 + memInBytes := uint64(float64(totalMemory) * memPercentage) + s.sortEngineFactory = factory.NewForPebble(sortDir, memInBytes, conf.Debug.DB) return nil } @@ -359,14 +327,6 @@ func (s *server) run(ctx context.Context) (err error) { return s.tcpServer.Run(cctx) }) - conf := config.GetGlobalServerConfig() - - if !conf.Debug.EnableDBSorter { - wg.Go(func() error { - return unified.RunWorkerPool(cctx) - }) - } - grpcServer := grpc.NewServer(s.grpcService.ServerOptions()...) p2pProto.RegisterCDCPeerToPeerServer(grpcServer, s.grpcService) @@ -390,7 +350,7 @@ func (s *server) Drain() <-chan struct{} { // Close closes the server. func (s *server) Close() { - s.stopActorSystems() + s.closeSortEngineFactory() if s.capture != nil { s.capture.AsyncClose() @@ -411,19 +371,14 @@ func (s *server) Close() { } } -func (s *server) stopActorSystems() { +func (s *server) closeSortEngineFactory() { start := time.Now() - if s.useEventSortEngine && s.sortEngineFactory != nil { + if s.sortEngineFactory != nil { if err := s.sortEngineFactory.Close(); err != nil { log.Error("fails to close sort engine manager", zap.Error(err)) } log.Info("sort engine manager closed", zap.Duration("duration", time.Since(start))) } - if !s.useEventSortEngine && s.sorterSystem != nil { - s.sorterSystem.Stop() - s.sorterSystem = nil - log.Info("sorter actor system closed", zap.Duration("duration", time.Since(start))) - } } func (s *server) initDir(ctx context.Context) error { diff --git a/cdc/sink/mq/manager/kafka_manager.go b/cdc/sink/mq/manager/kafka_manager.go index 0e7a1741582..7ecd0a18a47 100644 --- a/cdc/sink/mq/manager/kafka_manager.go +++ b/cdc/sink/mq/manager/kafka_manager.go @@ -42,6 +42,7 @@ type kafkaTopicManager struct { // NewKafkaTopicManager creates a new topic manager. func NewKafkaTopicManager( + ctx context.Context, client kafka.Client, admin kafka.ClusterAdminClient, cfg *kafka.AutoCreateTopicConfig, @@ -53,7 +54,7 @@ func NewKafkaTopicManager( } // do an initial metadata fetching using ListTopics - err := mgr.listTopics() + err := mgr.listTopics(ctx) if err != nil { return nil, err } @@ -63,7 +64,10 @@ func NewKafkaTopicManager( // GetPartitionNum returns the number of partitions of the topic. // It may also try to update the topics' information maintained by manager. -func (m *kafkaTopicManager) GetPartitionNum(topic string) (int32, error) { +func (m *kafkaTopicManager) GetPartitionNum( + ctx context.Context, + topic string, +) (int32, error) { err := m.tryRefreshMeta() if err != nil { return 0, errors.Trace(err) @@ -73,7 +77,7 @@ func (m *kafkaTopicManager) GetPartitionNum(topic string) (int32, error) { return partitions.(int32), nil } - partitionNum, err := m.CreateTopicAndWaitUntilVisible(topic) + partitionNum, err := m.CreateTopicAndWaitUntilVisible(ctx, topic) if err != nil { return 0, errors.Trace(err) } @@ -125,7 +129,9 @@ func (m *kafkaTopicManager) tryUpdatePartitionsAndLogging(topic string, partitio } } -func (m *kafkaTopicManager) getMetadataOfTopics() (map[string]kafka.TopicDetail, error) { +func (m *kafkaTopicManager) getMetadataOfTopics( + ctx context.Context, +) (map[string]kafka.TopicDetail, error) { var topicList []string m.topics.Range(func(key, value any) bool { @@ -137,7 +143,7 @@ func (m *kafkaTopicManager) getMetadataOfTopics() (map[string]kafka.TopicDetail, start := time.Now() // ignore the topic with error, return a subset of all topics. - topicMetaList, err := m.admin.GetTopicsMeta(context.Background(), topicList, true) + topicMetaList, err := m.admin.GetTopicsMeta(ctx, topicList, true) if err != nil { log.Warn( "Kafka admin client describe topics failed", @@ -159,11 +165,14 @@ func (m *kafkaTopicManager) getMetadataOfTopics() (map[string]kafka.TopicDetail, // CreateTopic returns success for all the brokers to become aware that the // topics have been created. // See https://kafka.apache.org/23/javadoc/org/apache/kafka/clients/admin/AdminClient.html -func (m *kafkaTopicManager) waitUntilTopicVisible(topicName string) error { +func (m *kafkaTopicManager) waitUntilTopicVisible( + ctx context.Context, + topicName string, +) error { topics := []string{topicName} - err := retry.Do(context.Background(), func() error { + err := retry.Do(ctx, func() error { start := time.Now() - meta, err := m.admin.GetTopicsMeta(context.Background(), topics, false) + meta, err := m.admin.GetTopicsMeta(ctx, topics, false) if err != nil { log.Warn(" topic not found, retry it", zap.Error(err), @@ -185,9 +194,9 @@ func (m *kafkaTopicManager) waitUntilTopicVisible(topicName string) error { } // listTopics is used to do an initial metadata fetching. -func (m *kafkaTopicManager) listTopics() error { +func (m *kafkaTopicManager) listTopics(ctx context.Context) error { start := time.Now() - topics, err := m.admin.GetAllTopicsMeta(context.Background()) + topics, err := m.admin.GetAllTopicsMeta(ctx) if err != nil { log.Error( "Kafka admin client list topics failed", @@ -213,8 +222,11 @@ func (m *kafkaTopicManager) listTopics() error { // createTopic creates a topic with the given name // and returns the number of partitions. -func (m *kafkaTopicManager) createTopic(topicName string) (int32, error) { - topicMetaList, err := m.getMetadataOfTopics() +func (m *kafkaTopicManager) createTopic( + ctx context.Context, + topicName string, +) (int32, error) { + topicMetaList, err := m.getMetadataOfTopics(ctx) if err != nil { return 0, errors.Trace(err) } @@ -241,7 +253,7 @@ func (m *kafkaTopicManager) createTopic(topicName string) (int32, error) { } start := time.Now() - err = m.admin.CreateTopic(context.Background(), &kafka.TopicDetail{ + err = m.admin.CreateTopic(ctx, &kafka.TopicDetail{ Name: topicName, NumPartitions: m.cfg.PartitionNum, ReplicationFactor: m.cfg.ReplicationFactor, @@ -271,13 +283,16 @@ func (m *kafkaTopicManager) createTopic(topicName string) (int32, error) { } // CreateTopicAndWaitUntilVisible wraps createTopic and waitUntilTopicVisible together. -func (m *kafkaTopicManager) CreateTopicAndWaitUntilVisible(topicName string) (int32, error) { - partitionNum, err := m.createTopic(topicName) +func (m *kafkaTopicManager) CreateTopicAndWaitUntilVisible( + ctx context.Context, + topicName string, +) (int32, error) { + partitionNum, err := m.createTopic(ctx, topicName) if err != nil { return 0, errors.Trace(err) } - err = m.waitUntilTopicVisible(topicName) + err = m.waitUntilTopicVisible(ctx, topicName) if err != nil { return 0, errors.Trace(err) } diff --git a/cdc/sink/mq/manager/kafka_manager_test.go b/cdc/sink/mq/manager/kafka_manager_test.go index e1ab5191bb3..cdfedf0514b 100644 --- a/cdc/sink/mq/manager/kafka_manager_test.go +++ b/cdc/sink/mq/manager/kafka_manager_test.go @@ -14,6 +14,7 @@ package manager import ( + "context" "testing" "time" @@ -35,9 +36,11 @@ func TestPartitions(t *testing.T) { ReplicationFactor: 1, } - manager, err := NewKafkaTopicManager(client, adminClient, cfg) + ctx := context.Background() + manager, err := NewKafkaTopicManager(ctx, client, adminClient, cfg) require.Nil(t, err) partitionsNum, err := manager.GetPartitionNum( + ctx, kafka.DefaultMockTopicName) require.Nil(t, err) require.Equal(t, int32(3), partitionsNum) @@ -57,9 +60,11 @@ func TestTryRefreshMeta(t *testing.T) { ReplicationFactor: 1, } - manager, err := NewKafkaTopicManager(client, adminClient, cfg) + ctx := context.Background() + manager, err := NewKafkaTopicManager(ctx, client, adminClient, cfg) require.Nil(t, err) partitionsNum, err := manager.GetPartitionNum( + ctx, kafka.DefaultMockTopicName) require.Nil(t, err) require.Equal(t, int32(3), partitionsNum) @@ -67,14 +72,14 @@ func TestTryRefreshMeta(t *testing.T) { // Mock create a topic. client.AddTopic("test", 4) manager.lastMetadataRefresh.Store(time.Now().Add(-2 * time.Minute).Unix()) - partitionsNum, err = manager.GetPartitionNum("test") + partitionsNum, err = manager.GetPartitionNum(ctx, "test") require.Nil(t, err) require.Equal(t, int32(4), partitionsNum) // Mock delete a topic. // NOTICE: we do not refresh metadata for the deleted topic. client.DeleteTopic("test") - partitionsNum, err = manager.GetPartitionNum("test") + partitionsNum, err = manager.GetPartitionNum(ctx, "test") require.Nil(t, err) require.Equal(t, int32(4), partitionsNum) } @@ -94,24 +99,25 @@ func TestCreateTopic(t *testing.T) { ReplicationFactor: 1, } - manager, err := NewKafkaTopicManager(client, adminClient, cfg) + ctx := context.Background() + manager, err := NewKafkaTopicManager(ctx, client, adminClient, cfg) require.Nil(t, err) - partitionNum, err := manager.createTopic(kafka.DefaultMockTopicName) + partitionNum, err := manager.createTopic(ctx, kafka.DefaultMockTopicName) require.Nil(t, err) require.Equal(t, int32(3), partitionNum) - partitionNum, err = manager.createTopic("new-topic") + partitionNum, err = manager.createTopic(ctx, "new-topic") require.Nil(t, err) require.Equal(t, int32(2), partitionNum) - partitionsNum, err := manager.GetPartitionNum("new-topic") + partitionsNum, err := manager.GetPartitionNum(ctx, "new-topic") require.Nil(t, err) require.Equal(t, int32(2), partitionsNum) // Try to create a topic without auto create. cfg.AutoCreate = false - manager, err = NewKafkaTopicManager(client, adminClient, cfg) + manager, err = NewKafkaTopicManager(ctx, client, adminClient, cfg) require.Nil(t, err) - _, err = manager.createTopic("new-topic2") + _, err = manager.createTopic(ctx, "new-topic2") require.Regexp( t, "`auto-create-topic` is false, and new-topic2 not found", @@ -125,9 +131,9 @@ func TestCreateTopic(t *testing.T) { PartitionNum: 2, ReplicationFactor: 4, } - manager, err = NewKafkaTopicManager(client, adminClient, cfg) + manager, err = NewKafkaTopicManager(ctx, client, adminClient, cfg) require.Nil(t, err) - _, err = manager.createTopic("new-topic-failed") + _, err = manager.createTopic(ctx, "new-topic-failed") require.Regexp( t, "kafka create topic failed: kafka server: Replication-factor is invalid", @@ -149,13 +155,14 @@ func TestCreateTopicWithDelay(t *testing.T) { ReplicationFactor: 1, } - manager, err := NewKafkaTopicManager(client, adminClient, cfg) + ctx := context.Background() + manager, err := NewKafkaTopicManager(ctx, client, adminClient, cfg) require.Nil(t, err) - partitionNum, err := manager.createTopic("new_topic") + partitionNum, err := manager.createTopic(ctx, "new_topic") require.Nil(t, err) err = adminClient.SetRemainingFetchesUntilTopicVisible("new_topic", 3) require.Nil(t, err) - err = manager.waitUntilTopicVisible("new_topic") + err = manager.waitUntilTopicVisible(ctx, "new_topic") require.Nil(t, err) require.Equal(t, int32(2), partitionNum) } diff --git a/cdc/sink/mq/manager/manager.go b/cdc/sink/mq/manager/manager.go index 5c20a672465..90a4afd31c0 100644 --- a/cdc/sink/mq/manager/manager.go +++ b/cdc/sink/mq/manager/manager.go @@ -13,12 +13,14 @@ package manager +import "context" + // TopicManager is the interface of topic manager. // It will be responsible for creating and // updating the information of the topic. type TopicManager interface { // GetPartitionNum returns the number of partitions of the topic. - GetPartitionNum(topic string) (int32, error) + GetPartitionNum(ctx context.Context, topic string) (int32, error) // CreateTopicAndWaitUntilVisible creates the topic and wait for the topic completion. - CreateTopicAndWaitUntilVisible(topicName string) (int32, error) + CreateTopicAndWaitUntilVisible(ctx context.Context, topicName string) (int32, error) } diff --git a/cdc/sink/mq/mq.go b/cdc/sink/mq/mq.go index fe0531e84d6..80727782b9a 100644 --- a/cdc/sink/mq/mq.go +++ b/cdc/sink/mq/mq.go @@ -138,7 +138,7 @@ func (k *mqSink) EmitRowChangedEvents(ctx context.Context, rows ...*model.RowCha rowsCount := 0 for _, row := range rows { topic := k.eventRouter.GetTopicForRowChange(row) - partitionNum, err := k.topicManager.GetPartitionNum(topic) + partitionNum, err := k.topicManager.GetPartitionNum(ctx, topic) if err != nil { return errors.Trace(err) } @@ -248,7 +248,7 @@ func (k *mqSink) EmitCheckpointTs(ctx context.Context, ts uint64, tables []*mode // This will be compatible with the old behavior. if len(tables) == 0 { topic := k.eventRouter.GetDefaultTopic() - partitionNum, err := k.topicManager.GetPartitionNum(topic) + partitionNum, err := k.topicManager.GetPartitionNum(ctx, topic) if err != nil { return errors.Trace(err) } @@ -264,7 +264,7 @@ func (k *mqSink) EmitCheckpointTs(ctx context.Context, ts uint64, tables []*mode topics := k.eventRouter.GetActiveTopics(tableNames) log.Debug("MQ sink current active topics", zap.Any("topics", topics)) for _, topic := range topics { - partitionNum, err := k.topicManager.GetPartitionNum(topic) + partitionNum, err := k.topicManager.GetPartitionNum(ctx, topic) if err != nil { return errors.Trace(err) } @@ -300,7 +300,7 @@ func (k *mqSink) EmitDDLEvent(ctx context.Context, ddl *model.DDLEvent) error { zap.String("changefeed", k.id.ID), zap.Any("role", k.role)) if partitionRule == dispatcher.PartitionAll { - partitionNum, err := k.topicManager.GetPartitionNum(topic) + partitionNum, err := k.topicManager.GetPartitionNum(ctx, topic) if err != nil { return errors.Trace(err) } @@ -311,7 +311,7 @@ func (k *mqSink) EmitDDLEvent(ctx context.Context, ddl *model.DDLEvent) error { // which will be responsible for automatically creating topics when they don't exist. // If it is not called here and kafka has `auto.create.topics.enable` turned on, // then the auto-created topic will not be created as configured by ticdc. - _, err = k.topicManager.GetPartitionNum(topic) + _, err = k.topicManager.GetPartitionNum(ctx, topic) if err != nil { return errors.Trace(err) } @@ -410,7 +410,7 @@ func NewKafkaSaramaSink(ctx context.Context, sinkURI *url.URL, } }() - if err := kafka.AdjustOptions(adminClient, options, topic); err != nil { + if err := kafka.AdjustOptions(ctx, adminClient, options, topic); err != nil { return nil, cerror.WrapError(cerror.ErrKafkaNewSaramaProducer, err) } @@ -437,6 +437,7 @@ func NewKafkaSaramaSink(ctx context.Context, sinkURI *url.URL, } topicManager, err := manager.NewKafkaTopicManager( + ctx, client, adminClient, options.DeriveTopicConfig(), @@ -445,7 +446,7 @@ func NewKafkaSaramaSink(ctx context.Context, sinkURI *url.URL, return nil, cerror.WrapError(cerror.ErrKafkaNewSaramaProducer, err) } - if _, err := topicManager.CreateTopicAndWaitUntilVisible(topic); err != nil { + if _, err := topicManager.CreateTopicAndWaitUntilVisible(ctx, topic); err != nil { return nil, cerror.WrapError(cerror.ErrKafkaCreateTopic, err) } diff --git a/cdc/sink/mq/producer/kafka/kafka.go b/cdc/sink/mq/producer/kafka/kafka.go index 6b493dffdf2..0eb57ebc013 100644 --- a/cdc/sink/mq/producer/kafka/kafka.go +++ b/cdc/sink/mq/producer/kafka/kafka.go @@ -332,16 +332,17 @@ func NewKafkaSaramaProducer( // AdjustOptions adjust the `Options` and `sarama.Config` by condition. func AdjustOptions( + ctx context.Context, admin kafka.ClusterAdminClient, options *kafka.Options, topic string, ) error { - topics, err := admin.GetAllTopicsMeta(context.Background()) + topics, err := admin.GetAllTopicsMeta(ctx) if err != nil { return errors.Trace(err) } - err = validateMinInsyncReplicas(admin, topics, topic, int(options.ReplicationFactor)) + err = validateMinInsyncReplicas(ctx, admin, topics, topic, int(options.ReplicationFactor)) if err != nil { return errors.Trace(err) } @@ -350,8 +351,11 @@ func AdjustOptions( // once we have found the topic, no matter `auto-create-topic`, make sure user input parameters are valid. if exists { // make sure that producer's `MaxMessageBytes` smaller than topic's `max.message.bytes` - topicMaxMessageBytesStr, err := getTopicConfig(admin, info, kafka.TopicMaxMessageBytesConfigName, - kafka.BrokerMessageMaxBytesConfigName) + topicMaxMessageBytesStr, err := getTopicConfig( + ctx, admin, info, + kafka.TopicMaxMessageBytesConfigName, + kafka.BrokerMessageMaxBytesConfigName, + ) if err != nil { return errors.Trace(err) } @@ -381,8 +385,10 @@ func AdjustOptions( return nil } - brokerMessageMaxBytesStr, err := admin.GetBrokerConfig(context.Background(), - kafka.BrokerMessageMaxBytesConfigName) + brokerMessageMaxBytesStr, err := admin.GetBrokerConfig( + ctx, + kafka.BrokerMessageMaxBytesConfigName, + ) if err != nil { log.Warn("TiCDC cannot find `message.max.bytes` from broker's configuration") return errors.Trace(err) @@ -414,13 +420,15 @@ func AdjustOptions( } func validateMinInsyncReplicas( + ctx context.Context, admin kafka.ClusterAdminClient, topics map[string]kafka.TopicDetail, topic string, replicationFactor int, ) error { minInsyncReplicasConfigGetter := func() (string, bool, error) { info, exists := topics[topic] if exists { - minInsyncReplicasStr, err := getTopicConfig(admin, info, + minInsyncReplicasStr, err := getTopicConfig( + ctx, admin, info, kafka.MinInsyncReplicasConfigName, kafka.MinInsyncReplicasConfigName) if err != nil { @@ -429,7 +437,7 @@ func validateMinInsyncReplicas( return minInsyncReplicasStr, true, nil } - minInsyncReplicasStr, err := admin.GetBrokerConfig(context.Background(), + minInsyncReplicasStr, err := admin.GetBrokerConfig(ctx, kafka.MinInsyncReplicasConfigName) if err != nil { return "", false, err @@ -475,6 +483,7 @@ func validateMinInsyncReplicas( // If the topic does not have this configuration, we will try to get it from the broker's configuration. // NOTICE: The configuration names of topic and broker may be different for the same configuration. func getTopicConfig( + ctx context.Context, admin kafka.ClusterAdminClient, detail kafka.TopicDetail, topicConfigName string, @@ -484,5 +493,5 @@ func getTopicConfig( return a, nil } - return admin.GetBrokerConfig(context.Background(), brokerConfigName) + return admin.GetBrokerConfig(ctx, brokerConfigName) } diff --git a/cdc/sink/mq/producer/kafka/kafka_test.go b/cdc/sink/mq/producer/kafka/kafka_test.go index ab5b6e0013a..d024e71cdf8 100644 --- a/cdc/sink/mq/producer/kafka/kafka_test.go +++ b/cdc/sink/mq/producer/kafka/kafka_test.go @@ -163,28 +163,29 @@ func TestAdjustConfigTopicNotExist(t *testing.T) { // When the topic does not exist, use the broker's configuration to create the topic. // topic not exist, `max-message-bytes` = `message.max.bytes` options.MaxMessageBytes = adminClient.GetBrokerMessageMaxBytes() - saramaConfig, err := kafka.NewSaramaConfig(context.Background(), options) + ctx := context.Background() + saramaConfig, err := kafka.NewSaramaConfig(ctx, options) require.Nil(t, err) - err = AdjustOptions(adminClient, options, "create-random1") + err = AdjustOptions(ctx, adminClient, options, "create-random1") require.Nil(t, err) expectedSaramaMaxMessageBytes := options.MaxMessageBytes require.Equal(t, expectedSaramaMaxMessageBytes, saramaConfig.Producer.MaxMessageBytes) // topic not exist, `max-message-bytes` > `message.max.bytes` options.MaxMessageBytes = adminClient.GetBrokerMessageMaxBytes() + 1 - saramaConfig, err = kafka.NewSaramaConfig(context.Background(), options) + saramaConfig, err = kafka.NewSaramaConfig(ctx, options) require.Nil(t, err) - err = AdjustOptions(adminClient, options, "create-random2") + err = AdjustOptions(ctx, adminClient, options, "create-random2") require.Nil(t, err) expectedSaramaMaxMessageBytes = adminClient.GetBrokerMessageMaxBytes() require.Equal(t, expectedSaramaMaxMessageBytes, options.MaxMessageBytes) // topic not exist, `max-message-bytes` < `message.max.bytes` options.MaxMessageBytes = adminClient.GetBrokerMessageMaxBytes() - 1 - saramaConfig, err = kafka.NewSaramaConfig(context.Background(), options) + saramaConfig, err = kafka.NewSaramaConfig(ctx, options) require.Nil(t, err) - err = AdjustOptions(adminClient, options, "create-random3") + err = AdjustOptions(ctx, adminClient, options, "create-random3") require.Nil(t, err) expectedSaramaMaxMessageBytes = options.MaxMessageBytes require.Equal(t, expectedSaramaMaxMessageBytes, options.MaxMessageBytes) @@ -201,10 +202,12 @@ func TestAdjustConfigTopicExist(t *testing.T) { // topic exists, `max-message-bytes` = `max.message.bytes`. options.MaxMessageBytes = adminClient.GetTopicMaxMessageBytes() - saramaConfig, err := kafka.NewSaramaConfig(context.Background(), options) + + ctx := context.Background() + saramaConfig, err := kafka.NewSaramaConfig(ctx, options) require.Nil(t, err) - err = AdjustOptions(adminClient, options, adminClient.GetDefaultMockTopicName()) + err = AdjustOptions(ctx, adminClient, options, adminClient.GetDefaultMockTopicName()) require.Nil(t, err) expectedSaramaMaxMessageBytes := options.MaxMessageBytes @@ -215,7 +218,7 @@ func TestAdjustConfigTopicExist(t *testing.T) { saramaConfig, err = kafka.NewSaramaConfig(context.Background(), options) require.Nil(t, err) - err = AdjustOptions(adminClient, options, adminClient.GetDefaultMockTopicName()) + err = AdjustOptions(ctx, adminClient, options, adminClient.GetDefaultMockTopicName()) require.Nil(t, err) expectedSaramaMaxMessageBytes = adminClient.GetTopicMaxMessageBytes() @@ -223,10 +226,10 @@ func TestAdjustConfigTopicExist(t *testing.T) { // topic exists, `max-message-bytes` < `max.message.bytes` options.MaxMessageBytes = adminClient.GetTopicMaxMessageBytes() - 1 - saramaConfig, err = kafka.NewSaramaConfig(context.Background(), options) + saramaConfig, err = kafka.NewSaramaConfig(ctx, options) require.Nil(t, err) - err = AdjustOptions(adminClient, options, adminClient.GetDefaultMockTopicName()) + err = AdjustOptions(ctx, adminClient, options, adminClient.GetDefaultMockTopicName()) require.Nil(t, err) expectedSaramaMaxMessageBytes = options.MaxMessageBytes @@ -245,23 +248,23 @@ func TestAdjustConfigTopicExist(t *testing.T) { require.Nil(t, err) options.MaxMessageBytes = adminClient.GetBrokerMessageMaxBytes() - 1 - saramaConfig, err = kafka.NewSaramaConfig(context.Background(), options) + saramaConfig, err = kafka.NewSaramaConfig(ctx, options) require.Nil(t, err) - err = AdjustOptions(adminClient, options, topicName) + err = AdjustOptions(ctx, adminClient, options, topicName) require.Nil(t, err) - // since `max.message.bytes` cannot found, use broker's `message.max.bytes` instead. + // since `max.message.bytes` cannot be found, use broker's `message.max.bytes` instead. expectedSaramaMaxMessageBytes = options.MaxMessageBytes require.Equal(t, expectedSaramaMaxMessageBytes, saramaConfig.Producer.MaxMessageBytes) // When the topic exists, but the topic doesn't have `max.message.bytes` // `max-message-bytes` > `message.max.bytes` options.MaxMessageBytes = adminClient.GetBrokerMessageMaxBytes() + 1 - saramaConfig, err = kafka.NewSaramaConfig(context.Background(), options) + saramaConfig, err = kafka.NewSaramaConfig(ctx, options) require.Nil(t, err) - err = AdjustOptions(adminClient, options, topicName) + err = AdjustOptions(ctx, adminClient, options, topicName) require.Nil(t, err) expectedSaramaMaxMessageBytes = adminClient.GetBrokerMessageMaxBytes() require.Equal(t, expectedSaramaMaxMessageBytes, options.MaxMessageBytes) @@ -279,7 +282,10 @@ func TestAdjustConfigMinInsyncReplicas(t *testing.T) { // Report an error if the replication-factor is less than min.insync.replicas // when the topic does not exist. adminClient.SetMinInsyncReplicas("2") + + ctx := context.Background() err := AdjustOptions( + ctx, adminClient, options, "create-new-fail-invalid-min-insync-replicas", @@ -293,7 +299,7 @@ func TestAdjustConfigMinInsyncReplicas(t *testing.T) { // topic not exist, and `min.insync.replicas` not found in broker's configuration adminClient.DropBrokerConfig(kafka.MinInsyncReplicasConfigName) topicName := "no-topic-no-min-insync-replicas" - err = AdjustOptions(adminClient, options, "no-topic-no-min-insync-replicas") + err = AdjustOptions(ctx, adminClient, options, "no-topic-no-min-insync-replicas") require.Nil(t, err) err = adminClient.CreateTopic(context.Background(), &kafka.TopicDetail{ Name: topicName, @@ -312,12 +318,12 @@ func TestAdjustConfigMinInsyncReplicas(t *testing.T) { NumPartitions: 3, }, false) require.Nil(t, err) - err = AdjustOptions(adminClient, options, topicName) + err = AdjustOptions(ctx, adminClient, options, topicName) require.Nil(t, err) // topic found, and have `min.insync.replicas`, but set to 2, larger than `replication-factor`. adminClient.SetMinInsyncReplicas("2") - err = AdjustOptions(adminClient, options, adminClient.GetDefaultMockTopicName()) + err = AdjustOptions(ctx, adminClient, options, adminClient.GetDefaultMockTopicName()) require.Regexp(t, ".*`replication-factor` is smaller than the `min.insync.replicas` of topic.*", errors.Cause(err), @@ -699,7 +705,7 @@ func TestConfigurationCombinations(t *testing.T) { topic, ok := a.uriParams[0].(string) require.True(t, ok) require.NotEqual(t, "", topic) - err = AdjustOptions(adminClient, options, topic) + err = AdjustOptions(ctx, adminClient, options, topic) require.Nil(t, err) encoderConfig := common.NewConfig(config.ProtocolOpen) diff --git a/cdc/sinkv2/ddlsink/mq/kafka_ddl_sink.go b/cdc/sinkv2/ddlsink/mq/kafka_ddl_sink.go index 5c80ed7a65a..2f57ba476ce 100644 --- a/cdc/sinkv2/ddlsink/mq/kafka_ddl_sink.go +++ b/cdc/sinkv2/ddlsink/mq/kafka_ddl_sink.go @@ -64,7 +64,7 @@ func NewKafkaDDLSink( } }() - if err := kafka.AdjustOptions(adminClient, options, topic); err != nil { + if err := kafka.AdjustOptions(ctx, adminClient, options, topic); err != nil { return nil, cerror.WrapError(cerror.ErrKafkaNewSaramaProducer, err) } @@ -95,6 +95,7 @@ func NewKafkaDDLSink( }() topicManager, err := util.GetTopicManagerAndTryCreateTopic( + ctx, topic, options.DeriveTopicConfig(), client, diff --git a/cdc/sinkv2/ddlsink/mq/mq_ddl_sink.go b/cdc/sinkv2/ddlsink/mq/mq_ddl_sink.go index 6abdeadda8c..b110c5b4884 100644 --- a/cdc/sinkv2/ddlsink/mq/mq_ddl_sink.go +++ b/cdc/sinkv2/ddlsink/mq/mq_ddl_sink.go @@ -105,7 +105,7 @@ func (k *ddlSink) WriteDDLEvent(ctx context.Context, ddl *model.DDLEvent) error zap.String("namespace", k.id.Namespace), zap.String("changefeed", k.id.ID)) if partitionRule == dispatcher.PartitionAll { - partitionNum, err := k.topicManager.GetPartitionNum(topic) + partitionNum, err := k.topicManager.GetPartitionNum(ctx, topic) if err != nil { return errors.Trace(err) } @@ -118,7 +118,7 @@ func (k *ddlSink) WriteDDLEvent(ctx context.Context, ddl *model.DDLEvent) error // which will be responsible for automatically creating topics when they don't exist. // If it is not called here and kafka has `auto.create.topics.enable` turned on, // then the auto-created topic will not be created as configured by ticdc. - _, err = k.topicManager.GetPartitionNum(topic) + _, err = k.topicManager.GetPartitionNum(ctx, topic) if err != nil { return errors.Trace(err) } @@ -144,7 +144,7 @@ func (k *ddlSink) WriteCheckpointTs(ctx context.Context, // This will be compatible with the old behavior. if len(tables) == 0 { topic := k.eventRouter.GetDefaultTopic() - partitionNum, err := k.topicManager.GetPartitionNum(topic) + partitionNum, err := k.topicManager.GetPartitionNum(ctx, topic) if err != nil { return errors.Trace(err) } @@ -159,7 +159,7 @@ func (k *ddlSink) WriteCheckpointTs(ctx context.Context, } topics := k.eventRouter.GetActiveTopics(tableNames) for _, topic := range topics { - partitionNum, err := k.topicManager.GetPartitionNum(topic) + partitionNum, err := k.topicManager.GetPartitionNum(ctx, topic) if err != nil { return errors.Trace(err) } diff --git a/cdc/sinkv2/eventsink/mq/kafka_dml_sink.go b/cdc/sinkv2/eventsink/mq/kafka_dml_sink.go index 8f055aa8c97..cfec911a387 100644 --- a/cdc/sinkv2/eventsink/mq/kafka_dml_sink.go +++ b/cdc/sinkv2/eventsink/mq/kafka_dml_sink.go @@ -65,7 +65,7 @@ func NewKafkaDMLSink( }() // adjust the option configuration before creating the kafka client - if err = kafka.AdjustOptions(adminClient, options, topic); err != nil { + if err = kafka.AdjustOptions(ctx, adminClient, options, topic); err != nil { return nil, cerror.WrapError(cerror.ErrKafkaNewSaramaProducer, err) } @@ -94,6 +94,7 @@ func NewKafkaDMLSink( }() topicManager, err := util.GetTopicManagerAndTryCreateTopic( + ctx, topic, options.DeriveTopicConfig(), client, diff --git a/cdc/sinkv2/eventsink/mq/mq_dml_sink.go b/cdc/sinkv2/eventsink/mq/mq_dml_sink.go index 96afe02597d..66aa53e8907 100644 --- a/cdc/sinkv2/eventsink/mq/mq_dml_sink.go +++ b/cdc/sinkv2/eventsink/mq/mq_dml_sink.go @@ -52,9 +52,13 @@ type dmlSink struct { // topicManager used to manage topics. // It is also responsible for creating topics. topicManager manager.TopicManager + + ctx context.Context + cancel context.CancelFunc } -func newSink(ctx context.Context, +func newSink( + ctx context.Context, producer dmlproducer.DMLProducer, topicManager manager.TopicManager, eventRouter *dispatcher.EventRouter, @@ -69,6 +73,7 @@ func newSink(ctx context.Context, return nil, cerror.WrapError(cerror.ErrKafkaInvalidConfig, err) } + ctx, cancel := context.WithCancel(ctx) statistics := metrics.NewStatistics(ctx, sink.RowSink) worker := newWorker(changefeedID, encoderConfig.Protocol, encoderBuilder, encoderConcurrency, producer, statistics) @@ -78,6 +83,8 @@ func newSink(ctx context.Context, worker: worker, eventRouter: eventRouter, topicManager: topicManager, + ctx: ctx, + cancel: cancel, } // Spawn a goroutine to send messages by the worker. @@ -110,7 +117,7 @@ func (s *dmlSink) WriteEvents(rows ...*eventsink.RowChangeCallbackableEvent) err continue } topic := s.eventRouter.GetTopicForRowChange(row.Event) - partitionNum, err := s.topicManager.GetPartitionNum(topic) + partitionNum, err := s.topicManager.GetPartitionNum(s.ctx, topic) if err != nil { return errors.Trace(err) } @@ -129,6 +136,9 @@ func (s *dmlSink) WriteEvents(rows ...*eventsink.RowChangeCallbackableEvent) err // Close closes the sink. func (s *dmlSink) Close() error { + if s.cancel != nil { + s.cancel() + } s.worker.close() return nil } diff --git a/cdc/sinkv2/util/helper.go b/cdc/sinkv2/util/helper.go index e8420042569..4a76cccc0c9 100644 --- a/cdc/sinkv2/util/helper.go +++ b/cdc/sinkv2/util/helper.go @@ -14,6 +14,7 @@ package util import ( + "context" "net/url" "strings" @@ -87,12 +88,14 @@ func GetEncoderConfig( // GetTopicManagerAndTryCreateTopic returns the topic manager and try to create the topic. func GetTopicManagerAndTryCreateTopic( + ctx context.Context, topic string, topicCfg *kafka.AutoCreateTopicConfig, client kafka.Client, adminClient kafka.ClusterAdminClient, ) (manager.TopicManager, error) { topicManager, err := manager.NewKafkaTopicManager( + ctx, client, adminClient, topicCfg, @@ -101,7 +104,7 @@ func GetTopicManagerAndTryCreateTopic( return nil, cerror.WrapError(cerror.ErrKafkaNewSaramaProducer, err) } - if _, err := topicManager.CreateTopicAndWaitUntilVisible(topic); err != nil { + if _, err := topicManager.CreateTopicAndWaitUntilVisible(ctx, topic); err != nil { return nil, cerror.WrapError(cerror.ErrKafkaCreateTopic, err) } diff --git a/cdc/sorter/db/buffer.go b/cdc/sorter/db/buffer.go deleted file mode 100644 index 3b6cdade46f..00000000000 --- a/cdc/sorter/db/buffer.go +++ /dev/null @@ -1,115 +0,0 @@ -// Copyright 2021 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package db - -import ( - "github.com/pingcap/log" - "github.com/pingcap/tiflow/cdc/model" - "github.com/pingcap/tiflow/cdc/sorter/db/message" - "go.uber.org/zap" -) - -// outputBuffer a struct that facilitate db sorter. -type outputBuffer struct { - // A slice of keys need to be deleted. - deleteKeys []message.Key - // A slice of resolved events that have the same commit ts. - resolvedEvents []*model.PolymorphicEvent - // partialReadTxn is set to true when a transaction is partially read. - partialReadTxn bool - - advisedCapacity int -} - -func newOutputBuffer(advisedCapacity int) *outputBuffer { - return &outputBuffer{ - deleteKeys: make([]message.Key, 0, advisedCapacity), - resolvedEvents: make([]*model.PolymorphicEvent, 0, advisedCapacity), - advisedCapacity: advisedCapacity, - } -} - -// maybeShrink try to shrink slices to the advised capacity. -func (b *outputBuffer) maybeShrink() { - if len(b.deleteKeys) < b.advisedCapacity { - if cap(b.deleteKeys) > b.advisedCapacity { - buf := make([]message.Key, 0, b.advisedCapacity) - buf = append(buf, b.deleteKeys...) - b.deleteKeys = buf - } - } - if len(b.resolvedEvents) < b.advisedCapacity { - if cap(b.resolvedEvents) > b.advisedCapacity { - buf := make([]*model.PolymorphicEvent, 0, b.advisedCapacity) - buf = append(buf, b.resolvedEvents...) - b.resolvedEvents = buf - } - } -} - -// In place left shift resolved events slice. After the call, -// `index` will become the first element in the slice -func (b *outputBuffer) shiftResolvedEvents(index int) { - if index > len(b.resolvedEvents) { - log.Panic("index out of range", zap.Int("len", len(b.resolvedEvents))) - } - if index != 0 { - length := len(b.resolvedEvents) - for left, right := 0, index; right < length; right++ { - b.resolvedEvents[left] = b.resolvedEvents[right] - // Set original element to nil to help GC. - b.resolvedEvents[right] = nil - left++ - } - b.resolvedEvents = b.resolvedEvents[:length-index] - } -} - -// tryAppendResolvedEvent try to append resolved events to the buffer. -// Return false if the buffer is full and append fails. -func (b *outputBuffer) tryAppendResolvedEvent(event *model.PolymorphicEvent) bool { - if len(b.resolvedEvents) > 0 { - if b.resolvedEvents[0].CRTs != event.CRTs { - log.Panic("commit ts must be equal", - zap.Uint64("newCommitTs", event.CRTs), - zap.Uint64("commitTs", b.resolvedEvents[0].CRTs)) - } - } else if len(b.resolvedEvents) == 0 { - // Reset if it appends a new transaction. - b.partialReadTxn = false - } - if len(b.resolvedEvents) >= b.advisedCapacity { - // buffer is full, and the commit ts of event is the same, - // we must be in the middle of a transaction. - b.partialReadTxn = true - return false - } - b.resolvedEvents = append(b.resolvedEvents, event) - return true -} - -// appendDeleteKey appends to-be-deleted keys to the buffer. -func (b *outputBuffer) appendDeleteKey(key message.Key) { - b.deleteKeys = append(b.deleteKeys, key) -} - -// resetDeleteKey reset deleteKeys to a zero len slice. -func (b *outputBuffer) resetDeleteKey() { - b.deleteKeys = b.deleteKeys[:0] -} - -// len returns the length of resolvedEvents and delete keys. -func (b *outputBuffer) len() (int, int) { - return len(b.resolvedEvents), len(b.deleteKeys) -} diff --git a/cdc/sorter/db/buffer_test.go b/cdc/sorter/db/buffer_test.go deleted file mode 100644 index 7c6c11f38a6..00000000000 --- a/cdc/sorter/db/buffer_test.go +++ /dev/null @@ -1,116 +0,0 @@ -// Copyright 2021 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package db - -import ( - "testing" - - "github.com/pingcap/tiflow/cdc/model" - "github.com/pingcap/tiflow/cdc/sorter/db/message" - "github.com/stretchr/testify/require" -) - -func TestOutputBufferMaybeShrink(t *testing.T) { - t.Parallel() - advisedCapacity := 4 - buf := newOutputBuffer(advisedCapacity) - require.Equal(t, 0, len(buf.resolvedEvents)) - require.Equal(t, 0, len(buf.deleteKeys)) - require.Equal(t, advisedCapacity, cap(buf.resolvedEvents)) - require.Equal(t, advisedCapacity, cap(buf.deleteKeys)) - - // len == cap == advisedCapacity. - buf.resolvedEvents = make([]*model.PolymorphicEvent, advisedCapacity) - buf.resolvedEvents[0] = model.NewResolvedPolymorphicEvent(0, 1) - buf.deleteKeys = make([]message.Key, advisedCapacity) - buf.deleteKeys[0] = message.Key([]byte{1}) - resolvedEvents := append([]*model.PolymorphicEvent{}, buf.resolvedEvents...) - deleteKeys := append([]message.Key{}, buf.deleteKeys...) - - buf.maybeShrink() - require.Equal(t, advisedCapacity, len(buf.resolvedEvents)) - require.Equal(t, advisedCapacity, cap(buf.resolvedEvents)) - require.EqualValues(t, resolvedEvents, buf.resolvedEvents) - require.EqualValues(t, deleteKeys, buf.deleteKeys) - - // len < cap == 2*advisedCapacity. - buf.resolvedEvents = make([]*model.PolymorphicEvent, 2*advisedCapacity-1, 2*advisedCapacity) - buf.resolvedEvents[0] = model.NewResolvedPolymorphicEvent(0, 1) - buf.deleteKeys = make([]message.Key, 2*advisedCapacity-1, 2*advisedCapacity) - buf.deleteKeys[0] = message.Key([]byte{1}) - resolvedEvents = append([]*model.PolymorphicEvent{}, buf.resolvedEvents...) - deleteKeys = append([]message.Key{}, buf.deleteKeys...) - - buf.maybeShrink() - require.Equal(t, 2*advisedCapacity-1, len(buf.resolvedEvents)) - require.Equal(t, 2*advisedCapacity-1, len(buf.deleteKeys)) - require.EqualValues(t, resolvedEvents, buf.resolvedEvents) - require.EqualValues(t, deleteKeys, buf.deleteKeys) - - // len < cap/2 == advisedCapacity. - buf.resolvedEvents = make([]*model.PolymorphicEvent, advisedCapacity-1, 2*advisedCapacity) - buf.resolvedEvents[0] = model.NewResolvedPolymorphicEvent(0, 1) - buf.deleteKeys = make([]message.Key, advisedCapacity-1, 2*advisedCapacity) - buf.deleteKeys[0] = message.Key([]byte{1}) - resolvedEvents = append([]*model.PolymorphicEvent{}, buf.resolvedEvents...) - deleteKeys = append([]message.Key{}, buf.deleteKeys...) - - buf.maybeShrink() - require.Equal(t, advisedCapacity-1, len(buf.resolvedEvents)) - require.Equal(t, advisedCapacity-1, len(buf.deleteKeys)) - require.EqualValues(t, resolvedEvents, buf.resolvedEvents) - require.EqualValues(t, deleteKeys, buf.deleteKeys) -} - -func TestOutputBufferShiftResolvedEvents(t *testing.T) { - t.Parallel() - advisedCapacity := 64 - buf := newOutputBuffer(advisedCapacity) - - events := make([]*model.PolymorphicEvent, advisedCapacity) - for i := range events { - events[i] = &model.PolymorphicEvent{CRTs: uint64(1)} - } - - for i := 0; i < advisedCapacity; i++ { - buf.resolvedEvents = append([]*model.PolymorphicEvent{}, events...) - buf.shiftResolvedEvents(i) - require.EqualValues(t, buf.resolvedEvents, events[i:]) - } -} - -func TestOutputBufferTryAppendResolvedEvent(t *testing.T) { - t.Parallel() - - advisedCapacity := 2 - buf := newOutputBuffer(advisedCapacity) - require.False(t, buf.partialReadTxn) - - require.True(t, buf.tryAppendResolvedEvent(&model.PolymorphicEvent{})) - require.False(t, buf.partialReadTxn) - require.True(t, buf.tryAppendResolvedEvent(&model.PolymorphicEvent{})) - require.False(t, buf.partialReadTxn) - - // Failed append sets partialReadTxn - require.False(t, buf.tryAppendResolvedEvent(&model.PolymorphicEvent{})) - require.True(t, buf.partialReadTxn) - - buf.shiftResolvedEvents(2) - l, _ := buf.len() - require.Equal(t, 0, l) - - // A successful append resets partialReadTxn - require.True(t, buf.tryAppendResolvedEvent(&model.PolymorphicEvent{})) - require.False(t, buf.partialReadTxn) -} diff --git a/cdc/sorter/db/compactor.go b/cdc/sorter/db/compactor.go deleted file mode 100644 index 2c6769c87a3..00000000000 --- a/cdc/sorter/db/compactor.go +++ /dev/null @@ -1,174 +0,0 @@ -// Copyright 2021 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package db - -import ( - "bytes" - "context" - "math/rand" - "strconv" - "sync" - "time" - - "github.com/pingcap/log" - "github.com/pingcap/tiflow/cdc/sorter/db/message" - "github.com/pingcap/tiflow/pkg/actor" - actormsg "github.com/pingcap/tiflow/pkg/actor/message" - "github.com/pingcap/tiflow/pkg/config" - "github.com/pingcap/tiflow/pkg/db" - cerrors "github.com/pingcap/tiflow/pkg/errors" - "github.com/prometheus/client_golang/prometheus" - "go.uber.org/zap" -) - -type deleteThrottle struct { - count int - nextTime time.Time - rnd *rand.Rand - - // The number of delete keys that triggers delete. - countThreshold int - period time.Duration -} - -func (d *deleteThrottle) reset(now time.Time) { - // Randomize next time to avoid thundering herd problem. - randFactor := d.rnd.Int63n(int64(d.period)) - period := d.period + time.Duration(randFactor) - d.nextTime = now.Add(period) - d.count = 0 -} - -func (d *deleteThrottle) trigger(count int, now time.Time) bool { - if d.rnd == nil { - // Init rnd. - d.rnd = rand.New(rand.NewSource(rand.Int63())) - d.reset(now) - } - d.count += count - if d.count >= d.countThreshold || now.After(d.nextTime) { - // Throttle is triggered, reset before returning true. - d.reset(now) - return true - } - return false -} - -// CompactActor is an actor that compacts db. -// It GCs delete kv entries and reclaim disk space. -type CompactActor struct { - id actor.ID - db db.DB - delete deleteThrottle - closedWg *sync.WaitGroup - - metricCompactDuration prometheus.Observer -} - -var _ actor.Actor[message.Task] = (*CompactActor)(nil) - -// NewCompactActor returns a compactor actor. -func NewCompactActor( - id int, db db.DB, wg *sync.WaitGroup, cfg *config.DBConfig, -) (*CompactActor, actor.Mailbox[message.Task], error) { - wg.Add(1) - idTag := strconv.Itoa(id) - // Compact is CPU intensive, set capacity to 1 to reduce unnecessary tasks. - mb := actor.NewMailbox[message.Task](actor.ID(id), 1) - return &CompactActor{ - id: actor.ID(id), - db: db, - closedWg: wg, - delete: deleteThrottle{ - countThreshold: cfg.CompactionDeletionThreshold, - period: time.Duration(cfg.CompactionPeriod * int(time.Second)), - }, - - metricCompactDuration: sorterCompactDurationHistogram.WithLabelValues(idTag), - }, mb, nil -} - -// Poll implements actor.Actor. -func (c *CompactActor) Poll(ctx context.Context, tasks []actormsg.Message[message.Task]) bool { - select { - case <-ctx.Done(): - return false - default: - } - - // Only compact once for every batch. - count := 0 - for pos := range tasks { - msg := tasks[pos] - switch msg.Tp { - case actormsg.TypeValue: - count += msg.Value.DeleteReq.Count - case actormsg.TypeStop: - return false - default: - log.Panic("unexpected message", zap.Any("message", msg)) - } - } - - now := time.Now() - if !c.delete.trigger(count, now) { - return true - } - - // A range that is large enough to cover entire db effectively. - // See sorter/encoding/key.go. - start, end := []byte{0x0}, bytes.Repeat([]byte{0xff}, 128) - if err := c.db.Compact(start, end); err != nil { - log.Error("db compact error", zap.Error(err)) - } - c.metricCompactDuration.Observe(time.Since(now).Seconds()) - - return true -} - -// OnClose releases CompactActor resource. -func (c *CompactActor) OnClose() { - log.Info("compactor actor quit", zap.Uint64("ID", uint64(c.id))) - c.closedWg.Done() -} - -// NewCompactScheduler returns a new compact scheduler. -func NewCompactScheduler(router *actor.Router[message.Task]) *CompactScheduler { - return &CompactScheduler{router: router} -} - -// CompactScheduler schedules compact tasks to compactors. -type CompactScheduler struct { - // A router to compactors. - router *actor.Router[message.Task] -} - -// tryScheduleCompact try to schedule a compact task. -// Returns true if it schedules compact task successfully. -func (s *CompactScheduler) tryScheduleCompact(id actor.ID, deleteCount int) bool { - task := message.Task{ - DeleteReq: &message.DeleteRequest{ - // Compactor only needs count. DeleteRange is wrote by db actor. - Count: deleteCount, - }, - } - err := s.router.Send(id, actormsg.ValueMessage(task)) - // An ongoing compaction may block compactor and cause channel full, - // skip send the task as there is a pending task. - if err != nil && cerrors.ErrMailboxFull.NotEqual(err) { - log.Warn("schedule compact failed", zap.Error(err)) - return false - } - return true -} diff --git a/cdc/sorter/db/compactor_test.go b/cdc/sorter/db/compactor_test.go deleted file mode 100644 index 5d479fbd2a3..00000000000 --- a/cdc/sorter/db/compactor_test.go +++ /dev/null @@ -1,153 +0,0 @@ -// Copyright 2021 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package db - -import ( - "context" - "sync" - "testing" - "time" - - "github.com/pingcap/tiflow/cdc/sorter/db/message" - "github.com/pingcap/tiflow/pkg/actor" - actormsg "github.com/pingcap/tiflow/pkg/actor/message" - "github.com/pingcap/tiflow/pkg/config" - "github.com/pingcap/tiflow/pkg/db" - "github.com/stretchr/testify/require" -) - -type mockCompactDB struct { - db.DB - compact chan struct{} -} - -func (m *mockCompactDB) Compact(_, _ []byte) error { - m.compact <- struct{}{} - return nil -} - -func TestCompactorPoll(t *testing.T) { - t.Parallel() - ctx := context.Background() - cfg := config.GetDefaultServerConfig().Clone().Debug.DB - cfg.Count = 1 - - db, err := db.OpenPebble(ctx, 1, t.TempDir(), cfg, db.WithTableCRTsCollectors()) - require.Nil(t, err) - mockDB := mockCompactDB{DB: db, compact: make(chan struct{}, 1)} - closedWg := new(sync.WaitGroup) - cfg.CompactionDeletionThreshold = 2 - cfg.CompactionPeriod = 1 - compactor, _, err := NewCompactActor(1, &mockDB, closedWg, cfg) - require.Nil(t, err) - - // Must not trigger compact. - task := message.Task{DeleteReq: &message.DeleteRequest{}} - task.DeleteReq.Count = 0 - closed := !compactor.Poll(ctx, []actormsg.Message[message.Task]{actormsg.ValueMessage(task)}) - require.False(t, closed) - select { - case <-mockDB.compact: - t.Fatal("Must trigger compact") - case <-time.After(500 * time.Millisecond): - } - - // Must trigger compact. - task.DeleteReq.Count = 2 * cfg.CompactionDeletionThreshold - closed = !compactor.Poll(ctx, []actormsg.Message[message.Task]{actormsg.ValueMessage(task)}) - require.False(t, closed) - select { - case <-time.After(5 * time.Second): - t.Fatal("Must trigger compact") - case <-mockDB.compact: - } - - // Must trigger compact. - time.Sleep(time.Duration(cfg.CompactionPeriod) * time.Second * 2) - task.DeleteReq.Count = cfg.CompactionDeletionThreshold / 2 - closed = !compactor.Poll(ctx, []actormsg.Message[message.Task]{actormsg.ValueMessage(task)}) - require.False(t, closed) - select { - case <-time.After(5 * time.Second): - t.Fatal("Must trigger compact") - case <-mockDB.compact: - } - - // Close db. - stopMsg := actormsg.StopMessage[message.Task]() - closed = !compactor.Poll(ctx, []actormsg.Message[message.Task]{stopMsg}) - require.True(t, closed) - compactor.OnClose() - closedWg.Wait() - require.Nil(t, db.Close()) -} - -func TestComactorContextCancel(t *testing.T) { - t.Parallel() - ctx, cancel := context.WithCancel(context.Background()) - cfg := config.GetDefaultServerConfig().Clone().Debug.DB - cfg.Count = 1 - - db, err := db.OpenPebble(ctx, 1, t.TempDir(), cfg, db.WithTableCRTsCollectors()) - require.Nil(t, err) - closedWg := new(sync.WaitGroup) - ldb, _, err := NewCompactActor(0, db, closedWg, cfg) - require.Nil(t, err) - - cancel() - closed := !ldb.Poll( - ctx, []actormsg.Message[message.Task]{actormsg.ValueMessage(message.Task{})}) - require.True(t, closed) - ldb.OnClose() - closedWg.Wait() - require.Nil(t, db.Close()) -} - -func TestScheduleCompact(t *testing.T) { - t.Parallel() - router := actor.NewRouter[message.Task](t.Name()) - mb := actor.NewMailbox[message.Task](actor.ID(1), 1) - router.InsertMailbox4Test(mb.ID(), mb) - compact := NewCompactScheduler(router) - - // Must schedule successfully. - require.True(t, compact.tryScheduleCompact(mb.ID(), 3)) - msg, ok := mb.Receive() - require.True(t, ok) - task := message.Task{DeleteReq: &message.DeleteRequest{}} - task.DeleteReq.Count = 3 - require.EqualValues(t, actormsg.ValueMessage(task), msg) - - // Skip sending unnecessary tasks. - require.True(t, compact.tryScheduleCompact(mb.ID(), 3)) - require.True(t, compact.tryScheduleCompact(mb.ID(), 3)) - msg, ok = mb.Receive() - require.True(t, ok) - require.EqualValues(t, actormsg.ValueMessage(task), msg) - _, ok = mb.Receive() - require.False(t, ok) -} - -func TestDeleteThrottle(t *testing.T) { - t.Parallel() - dt := deleteThrottle{ - countThreshold: 2, - period: 1 * time.Second, - } - - require.False(t, dt.trigger(1, time.Now())) - require.True(t, dt.trigger(3, time.Now())) - time.Sleep(2 * dt.period) - require.True(t, dt.trigger(0, time.Now())) -} diff --git a/cdc/sorter/db/db.go b/cdc/sorter/db/db.go deleted file mode 100644 index 7c7c6bf0f86..00000000000 --- a/cdc/sorter/db/db.go +++ /dev/null @@ -1,267 +0,0 @@ -// Copyright 2021 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package db - -import ( - "container/list" - "context" - "strconv" - "sync" - "time" - - "github.com/pingcap/log" - "github.com/pingcap/tiflow/cdc/sorter/db/message" - "github.com/pingcap/tiflow/pkg/actor" - actormsg "github.com/pingcap/tiflow/pkg/actor/message" - "github.com/pingcap/tiflow/pkg/config" - "github.com/pingcap/tiflow/pkg/db" - cerrors "github.com/pingcap/tiflow/pkg/errors" - "github.com/prometheus/client_golang/prometheus" - "go.uber.org/zap" - "golang.org/x/sync/semaphore" -) - -// Queue of IterRequest -type iterQueue struct { - *list.List - // TableID set. - tables map[tableKey]struct{} -} - -type iterItem struct { - key tableKey - req *message.IterRequest -} - -type tableKey struct { - UID uint32 - TableID uint64 -} - -func (q *iterQueue) push(uid uint32, tableID uint64, req *message.IterRequest) { - key := tableKey{UID: uid, TableID: tableID} - _, ok := q.tables[key] - if ok { - log.Panic("A table should not issue two concurrent iterator requests", - zap.Uint64("tableID", tableID), - zap.Uint32("UID", uid), - zap.Uint64("resolvedTs", req.ResolvedTs)) - } - q.tables[key] = struct{}{} - q.List.PushBack(iterItem{req: req, key: key}) -} - -func (q *iterQueue) pop() (*message.IterRequest, bool) { - item := q.List.Front() - if item == nil { - return nil, false - } - q.List.Remove(item) - req := item.Value.(iterItem) - delete(q.tables, req.key) - return req.req, true -} - -// Actor is a db actor, it reads, writes and deletes key value pair in its db. -type Actor struct { - id actor.ID - db db.DB - wb db.Batch - wbSize int - wbCap int - iterSem *semaphore.Weighted - iterQ iterQueue - - deleteCount int - compact *CompactScheduler - - stopped bool - closedWg *sync.WaitGroup - - metricWriteDuration prometheus.Observer - metricWriteBytes prometheus.Observer -} - -var _ actor.Actor[message.Task] = (*Actor)(nil) - -// NewDBActor returns a db actor. -func NewDBActor( - id int, db db.DB, cfg *config.DBConfig, compact *CompactScheduler, - wg *sync.WaitGroup, -) (*Actor, actor.Mailbox[message.Task], error) { - idTag := strconv.Itoa(id) - // Write batch size should be larger than block size to save CPU. - const writeBatchSizeFactor = 16 - wbSize := cfg.BlockSize * writeBatchSizeFactor - // Double batch capacity to avoid memory reallocation. - const writeBatchCapFactor = 2 - wbCap := wbSize * writeBatchCapFactor - wb := db.Batch(wbCap) - // IterCount limits the total number of opened iterators to release db - // resources in time. - iterSema := semaphore.NewWeighted(int64(cfg.Concurrency)) - mb := actor.NewMailbox[message.Task](actor.ID(id), cfg.Concurrency) - wg.Add(1) - - return &Actor{ - id: actor.ID(id), - db: db, - wb: wb, - iterSem: iterSema, - iterQ: iterQueue{ - List: list.New(), - tables: make(map[tableKey]struct{}), - }, - wbSize: wbSize, - wbCap: wbCap, - compact: compact, - - closedWg: wg, - - metricWriteDuration: sorterWriteDurationHistogram.WithLabelValues(idTag), - metricWriteBytes: sorterWriteBytesHistogram.WithLabelValues(idTag), - }, mb, nil -} - -func (ldb *Actor) tryScheduleCompact() { - // Schedule a compact task when there are too many deletion. - if ldb.compact.tryScheduleCompact(ldb.id, ldb.deleteCount) { - // Reset delete key count if schedule compaction successfully. - ldb.deleteCount = 0 - } -} - -func (ldb *Actor) maybeWrite(force bool) error { - bytes := len(ldb.wb.Repr()) - if bytes >= ldb.wbSize || (force && bytes != 0) { - startTime := time.Now() - err := ldb.wb.Commit() - if err != nil { - return cerrors.ErrDBSorterError.GenWithStackByArgs(err) - } - ldb.metricWriteDuration.Observe(time.Since(startTime).Seconds()) - ldb.metricWriteBytes.Observe(float64(bytes)) - - // Reset write batch or reclaim memory if it grows too large. - if cap(ldb.wb.Repr()) <= ldb.wbCap { - ldb.wb.Reset() - } else { - ldb.wb = ldb.db.Batch(ldb.wbCap) - } - } - return nil -} - -// Batch acquire iterators for requests in the queue. -func (ldb *Actor) acquireIterators() { - for { - succeed := ldb.iterSem.TryAcquire(1) - if !succeed { - break - } - req, ok := ldb.iterQ.pop() - if !ok { - ldb.iterSem.Release(1) - break - } - - iterRange := req.Range - iter := ldb.db.Iterator(iterRange[0], iterRange[1], req.CRTsFilter[0], req.CRTsFilter[1]) - req.IterCallback(&message.LimitedIterator{ - Iterator: iter, - Sema: ldb.iterSem, - ResolvedTs: req.ResolvedTs, - }) - } -} - -// Poll implements actor.Actor. -// It handles tasks by writing kv, deleting kv and taking iterators. -func (ldb *Actor) Poll(ctx context.Context, tasks []actormsg.Message[message.Task]) bool { - select { - case <-ctx.Done(): - return false - default: - } - requireIter := false - for i := range tasks { - var task message.Task - msg := tasks[i] - switch msg.Tp { - case actormsg.TypeValue: - task = msg.Value - case actormsg.TypeStop: - return false - default: - log.Panic("unexpected message", zap.Any("message", msg)) - } - - for k, v := range task.WriteReq { - if len(v) != 0 { - ldb.wb.Put([]byte(k), v) - } else { - // Delete the key if value is empty - ldb.wb.Delete([]byte(k)) - ldb.deleteCount++ - } - - // Do not force write, batching for efficiency. - if err := ldb.maybeWrite(false); err != nil { - log.Panic("db error", zap.Error(err)) - } - } - if task.DeleteReq != nil { - ldb.deleteCount += task.DeleteReq.Count - if len(task.DeleteReq.Range[0]) != 0 && len(task.DeleteReq.Range[1]) != 0 { - // Force write pending write batch before delete range. - if err := ldb.maybeWrite(true); err != nil { - log.Panic("db error", - zap.Error(err), zap.Stringer("span", task.Span)) - } - start, end := task.DeleteReq.Range[0], task.DeleteReq.Range[1] - if err := ldb.db.DeleteRange(start, end); err != nil { - log.Panic("db error", - zap.Error(err), zap.Stringer("span", task.Span)) - } - ldb.tryScheduleCompact() - } - } - if task.IterReq != nil { - // Append to slice for later batch acquiring iterators. - ldb.iterQ.push(task.UID, uint64(task.Span.TableID), task.IterReq) - requireIter = true - } - if task.Test != nil { - time.Sleep(task.Test.Sleep) - } - } - - // Force write only if there is a task requires an iterator. - if err := ldb.maybeWrite(requireIter); err != nil { - log.Panic("db error", zap.Error(err)) - } - ldb.acquireIterators() - - return true -} - -// OnClose releases Actor resource. -func (ldb *Actor) OnClose() { - if ldb.stopped { - return - } - ldb.stopped = true - log.Info("db actor quit", zap.Uint64("ID", uint64(ldb.id))) - ldb.closedWg.Done() -} diff --git a/cdc/sorter/db/db_test.go b/cdc/sorter/db/db_test.go deleted file mode 100644 index dc276274be0..00000000000 --- a/cdc/sorter/db/db_test.go +++ /dev/null @@ -1,435 +0,0 @@ -// Copyright 2021 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package db - -import ( - "bytes" - "context" - "math/rand" - "sort" - "sync" - "testing" - "time" - - "github.com/pingcap/tiflow/cdc/processor/tablepb" - "github.com/pingcap/tiflow/cdc/sorter/db/message" - "github.com/pingcap/tiflow/pkg/actor" - actormsg "github.com/pingcap/tiflow/pkg/actor/message" - "github.com/pingcap/tiflow/pkg/config" - "github.com/pingcap/tiflow/pkg/db" - "github.com/pingcap/tiflow/pkg/leakutil" - "github.com/pingcap/tiflow/pkg/spanz" - "github.com/stretchr/testify/require" -) - -func TestMain(m *testing.M) { - leakutil.SetUpLeakTest(m) -} - -func TestMaybeWrite(t *testing.T) { - t.Parallel() - - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - - cfg := config.GetDefaultServerConfig().Clone().Debug.DB - cfg.Count = 1 - - db, err := db.OpenPebble(ctx, 1, t.TempDir(), cfg, db.WithTableCRTsCollectors()) - require.Nil(t, err) - closedWg := new(sync.WaitGroup) - compact := NewCompactScheduler(actor.NewRouter[message.Task](t.Name())) - ldb, _, err := NewDBActor(0, db, cfg, compact, closedWg) - require.Nil(t, err) - - // Empty batch - err = ldb.maybeWrite(false) - require.Nil(t, err) - - // None empty batch - ldb.wb.Put([]byte("abc"), []byte("abc")) - err = ldb.maybeWrite(false) - require.Nil(t, err) - require.EqualValues(t, ldb.wb.Count(), 1) - - // None empty batch - err = ldb.maybeWrite(true) - require.Nil(t, err) - require.EqualValues(t, ldb.wb.Count(), 0) - - ldb.wb.Put([]byte("abc"), []byte("abc")) - ldb.wbSize = 1 - require.Greater(t, len(ldb.wb.Repr()), ldb.wbSize) - err = ldb.maybeWrite(false) - require.Nil(t, err) - require.EqualValues(t, ldb.wb.Count(), 0) - - // Close db. - closed := !ldb.Poll(ctx, []actormsg.Message[message.Task]{actormsg.StopMessage[message.Task]()}) - require.True(t, closed) - ldb.OnClose() - closedWg.Wait() - require.Nil(t, db.Close()) -} - -func TestCompact(t *testing.T) { - t.Parallel() - - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - - cfg := config.GetDefaultServerConfig().Clone().Debug.DB - cfg.Count = 1 - - id := 1 - db, err := db.OpenPebble(ctx, id, t.TempDir(), cfg, db.WithTableCRTsCollectors()) - require.Nil(t, err) - closedWg := new(sync.WaitGroup) - compactRouter := actor.NewRouter[message.Task](t.Name()) - compactMB := actor.NewMailbox[message.Task](actor.ID(id), 1) - compactRouter.InsertMailbox4Test(compactMB.ID(), compactMB) - compact := NewCompactScheduler(compactRouter) - ldb, _, err := NewDBActor(id, db, cfg, compact, closedWg) - require.Nil(t, err) - - // Empty task must not trigger compact. - task, iterCh := makeTask(make(map[message.Key][]byte), [][]byte{{0x00}, {0xff}}) - require.True(t, ldb.Poll(ctx, task)) - iter := <-iterCh - iter.Release() - _, ok := compactMB.Receive() - require.False(t, ok) - - // Empty delete range task must not trigger compact. - task = makeDelTask([2][]byte{}, 0) - require.True(t, ldb.Poll(ctx, task)) - _, ok = compactMB.Receive() - require.False(t, ok) - - // A valid delete range task must trigger compact. - task = makeDelTask([2][]byte{{0x00}, {0xff}}, 3) - require.True(t, ldb.Poll(ctx, task)) - _, ok = compactMB.Receive() - require.True(t, ok) - - // Close db. - closed := !ldb.Poll(ctx, []actormsg.Message[message.Task]{actormsg.StopMessage[message.Task]()}) - require.True(t, closed) - ldb.OnClose() - closedWg.Wait() - require.Nil(t, db.Close()) -} - -func makeDelTask(delRange [2][]byte, count int) []actormsg.Message[message.Task] { - return []actormsg.Message[message.Task]{actormsg.ValueMessage(message.Task{ - DeleteReq: &message.DeleteRequest{ - Range: delRange, - Count: count, - }, - })} -} - -func makeTask( - writes map[message.Key][]byte, rg [][]byte, -) ( - []actormsg.Message[message.Task], chan *message.LimitedIterator, -) { - var iterReq *message.IterRequest - var iterCh chan *message.LimitedIterator - if len(rg) != 0 { - iterCh = make(chan *message.LimitedIterator, 1) - iterReq = &message.IterRequest{ - Range: [2][]byte{rg[0], rg[1]}, - IterCallback: func(iter *message.LimitedIterator) { - iterCh <- iter - close(iterCh) - }, - } - } - return []actormsg.Message[message.Task]{actormsg.ValueMessage(message.Task{ - Span: &tablepb.Span{}, - WriteReq: writes, - IterReq: iterReq, - })}, iterCh -} - -func TestPutReadDelete(t *testing.T) { - t.Parallel() - - ctx := context.Background() - cfg := config.GetDefaultServerConfig().Clone().Debug.DB - cfg.Count = 1 - - db, err := db.OpenPebble(ctx, 1, t.TempDir(), cfg, db.WithTableCRTsCollectors()) - require.Nil(t, err) - closedWg := new(sync.WaitGroup) - compact := NewCompactScheduler(actor.NewRouter[message.Task](t.Name())) - ldb, _, err := NewDBActor(0, db, cfg, compact, closedWg) - require.Nil(t, err) - - // Put only. - tasks, iterCh := makeTask(map[message.Key][]byte{"key": []byte("value")}, nil) - require.Nil(t, iterCh) - closed := !ldb.Poll(ctx, tasks) - require.False(t, closed) - - // Put and read. - tasks, iterCh = makeTask(map[message.Key][]byte{"key": []byte("value")}, - [][]byte{{0x00}, {0xff}}) - closed = !ldb.Poll(ctx, tasks) - require.False(t, closed) - iter, ok := <-iterCh - require.True(t, ok) - require.NotNil(t, iter) - ok = iter.Seek([]byte("")) - require.True(t, ok) - require.EqualValues(t, iter.Key(), "key") - ok = iter.Next() - require.False(t, ok) - require.Nil(t, iter.Release()) - - // Read only. - tasks, iterCh = makeTask(make(map[message.Key][]byte), [][]byte{{0x00}, {0xff}}) - closed = !ldb.Poll(ctx, tasks) - require.False(t, closed) - iter, ok = <-iterCh - require.True(t, ok) - require.NotNil(t, iter) - ok = iter.Seek([]byte("")) - require.True(t, ok) - require.EqualValues(t, iter.Key(), "key") - ok = iter.Next() - require.False(t, ok) - require.Nil(t, iter.Release()) - - // Delete and read. - tasks = makeDelTask([2][]byte{{0x00}, {0xff}}, 0) - iterTasks, iterCh := makeTask(make(map[message.Key][]byte), [][]byte{{0x00}, {0xff}}) - tasks = append(tasks, iterTasks...) - closed = !ldb.Poll(ctx, tasks) - require.False(t, closed) - iter, ok = <-iterCh - require.True(t, ok) - require.NotNil(t, iter) - ok = iter.Seek([]byte("")) - require.False(t, ok, string(iter.Key())) - require.Nil(t, iter.Release()) - - // Close db. - closed = !ldb.Poll(ctx, []actormsg.Message[message.Task]{actormsg.StopMessage[message.Task]()}) - require.True(t, closed) - ldb.OnClose() - closedWg.Wait() - require.Nil(t, db.Close()) -} - -func TestAcquireIterators(t *testing.T) { - ctx := context.Background() - cfg := config.GetDefaultServerConfig().Clone().Debug.DB - cfg.Count = 1 - - db, err := db.OpenPebble(ctx, 1, t.TempDir(), cfg, db.WithTableCRTsCollectors()) - require.Nil(t, err) - closedWg := new(sync.WaitGroup) - - // Set max iterator count to 1. - cfg.Concurrency = 1 - compact := NewCompactScheduler(actor.NewRouter[message.Task](t.Name())) - ldb, _, err := NewDBActor(0, db, cfg, compact, closedWg) - require.Nil(t, err) - - // Poll two tasks. - tasks, iterCh1 := makeTask(make(map[message.Key][]byte), [][]byte{{0x00}, {0xff}}) - span1 := spanz.TableIDToComparableSpan(1) - tasks[0].Value.Span = &span1 - tasks2, iterCh2 := makeTask(make(map[message.Key][]byte), [][]byte{{0x00}, {0xff}}) - span2 := spanz.TableIDToComparableSpan(2) - tasks2[0].Value.Span = &span2 - tasks = append(tasks, tasks2...) - closed := !ldb.Poll(ctx, tasks) - require.False(t, closed) - iter, ok := <-iterCh1 - require.True(t, ok) - require.NotNil(t, iter) - - // Require iterator is not allow for now. - closed = !ldb.Poll(ctx, []actormsg.Message[message.Task]{actormsg.ValueMessage(message.Task{})}) - require.False(t, closed) - select { - case <-iterCh2: - require.FailNow(t, "should not acquire an iterator") - default: - } - - // Release iter and iterCh2 should be able to receive an iterator. - require.Nil(t, iter.Release()) - closed = !ldb.Poll(ctx, []actormsg.Message[message.Task]{actormsg.ValueMessage(message.Task{})}) - require.False(t, closed) - iter, ok = <-iterCh2 - require.True(t, ok) - require.Nil(t, iter.Release()) - - // Close db. - closed = !ldb.Poll(ctx, []actormsg.Message[message.Task]{actormsg.StopMessage[message.Task]()}) - require.True(t, closed) - ldb.OnClose() - closedWg.Wait() - require.Nil(t, db.Close()) -} - -type sortedMap struct { - // sorted keys - kvs map[message.Key][]byte -} - -func (s *sortedMap) put(k message.Key, v []byte) { - s.kvs[k] = v -} - -func (s *sortedMap) delete(k message.Key) { - delete(s.kvs, k) -} - -func (s *sortedMap) iter(start, end message.Key) []message.Key { - keys := make([]message.Key, 0) - for k := range s.kvs { - key := k - // [start, end) - if bytes.Compare([]byte(key), []byte(start)) >= 0 && - bytes.Compare([]byte(key), []byte(end)) < 0 { - keys = append(keys, key) - } - } - sort.Sort(sortableKeys(keys)) - return keys -} - -type sortableKeys []message.Key - -func (x sortableKeys) Len() int { return len(x) } -func (x sortableKeys) Less(i, j int) bool { return bytes.Compare([]byte(x[i]), []byte(x[j])) < 0 } -func (x sortableKeys) Swap(i, j int) { x[i], x[j] = x[j], x[i] } - -func TestModelChecking(t *testing.T) { - t.Parallel() - - seed := time.Now().Unix() - rd := rand.New(rand.NewSource(seed)) - ctx := context.Background() - cfg := config.GetDefaultServerConfig().Clone().Debug.DB - cfg.Count = 1 - - db, err := db.OpenPebble(ctx, 1, t.TempDir(), cfg, db.WithTableCRTsCollectors()) - require.Nil(t, err) - closedWg := new(sync.WaitGroup) - compact := NewCompactScheduler(actor.NewRouter[message.Task](t.Name())) - ldb, _, err := NewDBActor(0, db, cfg, compact, closedWg) - require.Nil(t, err) - - minKey := message.Key("") - maxKey := message.Key(bytes.Repeat([]byte{0xff}, 100)) - randKey := func() []byte { - // At least 10 bytes. - key := make([]byte, rd.Intn(90)+10) - n, err := rd.Read(key) - require.Greater(t, n, 0) - require.Nil(t, err) - return key - } - model := sortedMap{kvs: make(map[message.Key][]byte)} - // Prepare 100 key value pairs. - for i := 0; i < 100; i++ { - key := randKey() - value := key - - // Put to model. - model.put(message.Key(key), value) - // Put to db. - tasks, _ := makeTask(map[message.Key][]byte{message.Key(key): value}, nil) - closed := !ldb.Poll(ctx, tasks) - require.False(t, closed) - } - - // 100 random tests. - for i := 0; i < 100; i++ { - // [1, 4] ops - ops := rd.Intn(4) + 1 - for j := 0; j < ops; j++ { - switch rd.Intn(2) { - // 0 for put. - case 0: - key := randKey() - value := key - - model.put(message.Key(key), value) - tasks, _ := makeTask(map[message.Key][]byte{message.Key(key): value}, nil) - closed := !ldb.Poll(ctx, tasks) - require.False(t, closed) - - // 1 for delete. - case 1: - keys := model.iter(minKey, maxKey) - delKey := keys[rd.Intn(len(keys))] - model.delete(delKey) - tasks, _ := makeTask(map[message.Key][]byte{delKey: {}}, nil) - closed := !ldb.Poll(ctx, tasks) - require.False(t, closed) - } - } - - tasks, iterCh := makeTask( - map[message.Key][]byte{}, [][]byte{[]byte(minKey), []byte(maxKey)}) - closed := !ldb.Poll(ctx, tasks) - require.False(t, closed) - iter := <-iterCh - iter.Seek([]byte(minKey)) - keys := model.iter(minKey, maxKey) - for idx, key := range keys { - require.EqualValues(t, key, iter.Key()) - require.EqualValues(t, model.kvs[key], iter.Value()) - ok := iter.Next() - require.Equal(t, ok, idx != len(keys)-1, - "index %d, len(model): %d, seed: %d", idx, len(model.kvs), seed) - } - require.Nil(t, iter.Release()) - } - - // Close db. - closed := !ldb.Poll(ctx, []actormsg.Message[message.Task]{actormsg.StopMessage[message.Task]()}) - require.True(t, closed) - require.Nil(t, db.Close()) -} - -func TestContextCancel(t *testing.T) { - t.Parallel() - - ctx, cancel := context.WithCancel(context.Background()) - cfg := config.GetDefaultServerConfig().Clone().Debug.DB - cfg.Count = 1 - - db, err := db.OpenPebble(ctx, 1, t.TempDir(), cfg, db.WithTableCRTsCollectors()) - require.Nil(t, err) - closedWg := new(sync.WaitGroup) - compact := NewCompactScheduler(actor.NewRouter[message.Task](t.Name())) - ldb, _, err := NewDBActor(0, db, cfg, compact, closedWg) - require.Nil(t, err) - - cancel() - tasks, _ := makeTask(map[message.Key][]byte{"key": {}}, [][]byte{{0x00}, {0xff}}) - closed := !ldb.Poll(ctx, tasks) - require.True(t, closed) - ldb.OnClose() - closedWg.Wait() - require.Nil(t, db.Close()) -} diff --git a/cdc/sorter/db/message/task.go b/cdc/sorter/db/message/task.go deleted file mode 100644 index 8b21fe2cf2b..00000000000 --- a/cdc/sorter/db/message/task.go +++ /dev/null @@ -1,116 +0,0 @@ -// Copyright 2021 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package message - -import ( - "fmt" - "time" - - "github.com/pingcap/errors" - "github.com/pingcap/tiflow/cdc/model" - "github.com/pingcap/tiflow/cdc/processor/tablepb" - "github.com/pingcap/tiflow/cdc/sorter/encoding" - "github.com/pingcap/tiflow/pkg/db" - "golang.org/x/sync/semaphore" -) - -// Task is a db actor task. It carries write and read request. -type Task struct { - UID uint32 - Span *tablepb.Span - - // Input unsorted event for writers. - // Sorter.AddEntry -> writer. - InputEvent *model.PolymorphicEvent - // Latest resolved ts / commit ts for readers. - // An empty ReadTs works like a tick. - // writer -> reader - ReadTs ReadTs - // A batch of events (bytes encoded) need to be wrote. - // writer -> db - WriteReq map[Key][]byte - // Requests an iterator when it is not nil. - // reader -> db - IterReq *IterRequest - // Deletes all of the key-values in the range. - // reader -> db and db -> compactor - DeleteReq *DeleteRequest - - // StartTs let reader now the lower bound timestamp for reading data from db - StartTs uint64 - - // A test message. - Test *Test -} - -// DeleteRequest a request to delete range. -type DeleteRequest struct { - Range [2][]byte - // Approximately key value pairs in the range. - Count int -} - -// ReadTs wraps the latest resolved ts and commit ts. -type ReadTs struct { - MaxCommitTs uint64 - MaxResolvedTs uint64 -} - -// IterRequest contains parameters that necessary to build an iterator. -type IterRequest struct { - UID uint32 - - // The resolved ts at the time of issuing the request. - ResolvedTs uint64 - // Range of a requested iterator. - Range [2][]byte - - // Lower bound and upper bound for CRTs, both are included. - // They can be used to filter out some table files when fetching entries from DB. - CRTsFilter [2]uint64 - - // IterCallback is callback to send iterator back. - // It must be buffered channel to avoid blocking. - IterCallback func(*LimitedIterator) `json:"-"` // Make Task JSON printable. -} - -// Test is a message for testing actors. -type Test struct { - Sleep time.Duration -} - -// Key is the key that is written to db. -type Key string - -// String returns a pretty printed string. -func (k Key) String() string { - uid, tableID, startTs, CRTs := encoding.DecodeKey([]byte(k)) - return fmt.Sprintf( - "uid: %d, tableID: %d, startTs: %d, CRTs: %d", - uid, tableID, startTs, CRTs) -} - -// LimitedIterator is a wrapper of db.Iterator that has a sema to limit -// the total number of alive iterator. -type LimitedIterator struct { - db.Iterator - Sema *semaphore.Weighted - ResolvedTs uint64 -} - -// Release resources of the snapshot. -func (s *LimitedIterator) Release() error { - s.Sema.Release(1) - return errors.Trace(s.Iterator.Release()) -} diff --git a/cdc/sorter/db/message/task_test.go b/cdc/sorter/db/message/task_test.go deleted file mode 100644 index 3ae0722a676..00000000000 --- a/cdc/sorter/db/message/task_test.go +++ /dev/null @@ -1,37 +0,0 @@ -// Copyright 2021 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package message - -import ( - "testing" - - "github.com/pingcap/tiflow/cdc/model" - "github.com/pingcap/tiflow/cdc/sorter/encoding" - "github.com/stretchr/testify/require" -) - -func TestPrint(t *testing.T) { - t.Parallel() - event := model.NewPolymorphicEvent(&model.RawKVEntry{ - OpType: model.OpTypeDelete, - Key: []byte{1}, - StartTs: 3, - CRTs: 4, - }) - - require.Equal(t, "uid: 1, tableID: 2, startTs: 3, CRTs: 4", - Key(encoding.EncodeKey(1, 2, event)).String()) - require.Equal(t, "uid: 1, tableID: 2, startTs: 0, CRTs: 3", - Key(encoding.EncodeTsKey(1, 2, 3)).String()) -} diff --git a/cdc/sorter/db/reader.go b/cdc/sorter/db/reader.go deleted file mode 100644 index 8c8801514a4..00000000000 --- a/cdc/sorter/db/reader.go +++ /dev/null @@ -1,559 +0,0 @@ -// Copyright 2022 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package db - -import ( - "context" - "sync/atomic" - "time" - - "github.com/pingcap/errors" - "github.com/pingcap/log" - "github.com/pingcap/tiflow/cdc/model" - "github.com/pingcap/tiflow/cdc/processor/tablepb" - "github.com/pingcap/tiflow/cdc/sorter" - "github.com/pingcap/tiflow/cdc/sorter/db/message" - "github.com/pingcap/tiflow/cdc/sorter/encoding" - "github.com/pingcap/tiflow/pkg/actor" - actormsg "github.com/pingcap/tiflow/pkg/actor/message" - "github.com/pingcap/tiflow/pkg/db" - "github.com/prometheus/client_golang/prometheus" - "go.uber.org/zap" -) - -// reader accepts out-of-order raw kv entries and output sorted entries -type reader struct { - common - stopped bool - - state pollState - - lastSentCommitTs uint64 - lastSentResolvedTs uint64 - lastEvent *model.PolymorphicEvent - - outputCh chan *model.PolymorphicEvent - delete deleteThrottle - - metricIterReadDuration prometheus.Observer - metricIterNextDuration prometheus.Observer - metricTotalEventsKV prometheus.Counter - metricTotalEventsResolved prometheus.Counter -} - -var _ actor.Actor[message.Task] = (*reader)(nil) - -func (r *reader) stats() sorter.Stats { - return sorter.Stats{ - CheckpointTsEgress: atomic.LoadUint64(&r.lastSentCommitTs), - ResolvedTsEgress: atomic.LoadUint64(&r.lastSentResolvedTs), - } -} - -// setTaskDelete set delete range if there are too many events can be deleted or -// it has been a long time since last delete. -func (r *reader) setTaskDelete(task *message.Task, deleteKeys []message.Key) { - if len(deleteKeys) <= 0 { - return - } - - totalDelete := r.delete.count - if !r.delete.trigger(len(deleteKeys), time.Now()) { - return - } - - // Delete events that less than the last delete keys' commit ts. - task.DeleteReq = &message.DeleteRequest{} - task.DeleteReq.Range[0] = encoding.EncodeTsKey(r.uid, uint64(r.span.TableID), 0) - task.DeleteReq.Range[1] = []byte(deleteKeys[len(deleteKeys)-1]) - task.DeleteReq.Count = totalDelete + len(deleteKeys) -} - -// output nonblocking outputs an event. Caller should retry when it returns false. -func (r *reader) output(event *model.PolymorphicEvent) bool { - if r.lastEvent == nil { - r.lastEvent = event - } - if r.lastEvent.CRTs > event.CRTs { - log.Panic("regression", - zap.Any("lastEntry", r.lastEvent), zap.Any("event", event), - zap.Uint64("regionID", event.RegionID())) - } - select { - case r.outputCh <- event: - r.lastEvent = event - atomic.StoreUint64(&r.lastSentCommitTs, event.CRTs) - return true - default: - return false - } -} - -// outputResolvedTs nonblocking outputs a resolved ts event. -func (r *reader) outputResolvedTs(rts model.Ts) { - ok := r.output(model.NewResolvedPolymorphicEvent(0, rts)) - if ok { - r.metricTotalEventsResolved.Inc() - atomic.StoreUint64(&r.lastSentResolvedTs, rts) - } -} - -// outputBufferedResolvedEvents nonblocking output resolved events and -// resolved ts that are buffered in outputBuffer. -// It pops outputted events in the buffer and append their key to deleteKeys. -func (r *reader) outputBufferedResolvedEvents(buffer *outputBuffer) { - hasRemainEvents := false - // Index of remaining output events - remainIdx := 0 - // Commit ts of the last outputted events. - lastCommitTs := uint64(0) - for idx := range buffer.resolvedEvents { - event := buffer.resolvedEvents[idx] - ok := r.output(event) - if !ok { - hasRemainEvents = true - break - } - lastCommitTs = event.CRTs - - // Delete sent events. - key := encoding.EncodeKey(r.uid, uint64(r.span.TableID), event) - buffer.appendDeleteKey(message.Key(key)) - remainIdx = idx + 1 - } - r.metricTotalEventsKV.Add(float64(remainIdx)) - // Remove outputted events. - buffer.shiftResolvedEvents(remainIdx) - - // If all buffered resolved events are sent, send its resolved ts too. - if lastCommitTs != 0 && !hasRemainEvents && !buffer.partialReadTxn { - r.outputResolvedTs(lastCommitTs) - } -} - -// outputIterEvents nonblocking output resolved events that are buffered -// in db. -// It appends outputted events's key to outputBuffer deleteKeys to delete them -// later, and appends resolved events to outputBuffer resolvedEvents to send -// them later. -// -// It returns a new read position. -// -// Note: outputBuffer must be empty. -func (r *reader) outputIterEvents( - iter db.Iterator, position readPosition, buffer *outputBuffer, resolvedTs uint64, -) (readPosition, error) { - lenResolvedEvents, lenDeleteKeys := buffer.len() - if lenDeleteKeys > 0 || lenResolvedEvents > 0 { - log.Panic("buffer is not empty", - zap.Int("deleteKeys", lenDeleteKeys), - zap.Int("resolvedEvents", lenResolvedEvents)) - } - - // Commit ts of buffered resolved events. - commitTs := uint64(0) - start := time.Now() - lastNext := start - if position.iterHasRead { - // We have read the last key/value, move the Next. - iter.Next() - r.metricIterNextDuration.Observe(time.Since(start).Seconds()) - } // else the last is not read, we need to skip calling Next and read again. - hasReadNext := true - hasNext := iter.Valid() - for ; hasNext; hasNext = iter.Next() { - now := time.Now() - r.metricIterNextDuration.Observe(now.Sub(lastNext).Seconds()) - lastNext = now - - if iter.Error() != nil { - return readPosition{}, errors.Trace(iter.Error()) - } - event := new(model.PolymorphicEvent) - _, err := r.serde.Unmarshal(event, iter.Value()) - if err != nil { - return readPosition{}, errors.Trace(err) - } - if commitTs > event.CRTs || commitTs > resolvedTs { - log.Panic("event commit ts regression", - zap.Any("event", event), zap.Stringer("key", message.Key(iter.Key())), - zap.Uint64("ts", commitTs), zap.Uint64("resolvedTs", resolvedTs)) - } - - if commitTs == 0 { - commitTs = event.CRTs - } - // Read all resolved events that have the same commit ts. - if commitTs == event.CRTs { - ok := buffer.tryAppendResolvedEvent(event) - if !ok { - // append fails and buffer is full, we need to flush buffer to - // prevent OOM. - // It means we have not read value in to buffer after calling Next. - hasReadNext = false - break - } - continue - } - - // Commit ts has changed, the new event belongs to a new txn group, - // we need to output all buffered events before append the event. - r.outputBufferedResolvedEvents(buffer) - lenResolvedEvents, _ = buffer.len() - if lenResolvedEvents > 0 { - // Output blocked, skip append new event. - // It means we have not read value in to buffer after calling Next. - hasReadNext = false - break - } - - // Append new event to the buffer. - commitTs = event.CRTs - buffer.tryAppendResolvedEvent(event) - } - elapsed := time.Since(start) - r.metricIterReadDuration.Observe(elapsed.Seconds()) - - // When iter exhausts, buffer may never get a chance to output in the above - // for loop. We retry output buffer again. - r.outputBufferedResolvedEvents(buffer) - - // Try shrink buffer to release memory. - buffer.maybeShrink() - - newPos := readPosition{ - iterHasRead: hasReadNext, - } - if !buffer.partialReadTxn { - // All resolved events whose commit ts are less or equal to the commitTs - // have read into buffer. - newPos.exhaustedResolvedTs = commitTs - if !hasNext { - // Iter is exhausted, it means resolved events whose commit ts are - // less or equal to the commitTs have read into buffer. - if resolvedTs != 0 { - newPos.exhaustedResolvedTs = resolvedTs - } - } - } else { - // Copy current iter key to position. - newPos.partialTxnKey = append([]byte{}, iter.Key()...) - } - - return newPos, nil -} - -type readPosition struct { - // A flag to mark whether the current position has been read. - iterHasRead bool - // partialTxnKey is set when a transaction is partially read. - partialTxnKey []byte - // All resolved events before the resolved ts are read into buffer. - exhaustedResolvedTs uint64 -} - -func (r *readPosition) update(position readPosition) { - if position.exhaustedResolvedTs > r.exhaustedResolvedTs { - r.exhaustedResolvedTs = position.exhaustedResolvedTs - } - r.iterHasRead = position.iterHasRead - r.partialTxnKey = position.partialTxnKey -} - -type pollState struct { - span tablepb.Span - // Buffer for resolved events and to-be-deleted events. - outputBuf *outputBuffer - // The position of a reader. - position readPosition - - // The maximum commit ts for all events. - maxCommitTs uint64 - // The maximum commit ts for all resolved ts events. - maxResolvedTs uint64 - - // read data after `startTs` - startTs uint64 - - // ID and router of the reader itself. - readerID actor.ID - readerRouter *actor.Router[message.Task] - - // Compactor actor ID. - compactorID actor.ID - // A scheduler that triggers db compaction to speed up Iterator.Seek(). - compact *CompactScheduler - // A threshold of triggering db compaction. - iterFirstSlowDuration time.Duration - // A timestamp when iterator was created. - // Iterator is released once it exceeds `iterMaxAliveDuration`. - iterAliveTime time.Time - iterMaxAliveDuration time.Duration - // A timestamp when we request an iterator. - iterRequestTime time.Time - // A channel for receiving iterator asynchronously. - iterCh chan *message.LimitedIterator - // An iterator for reading resolved events, up to the `iterResolvedTs`. - iter *message.LimitedIterator - iterResolvedTs uint64 - - metricIterRequest prometheus.Observer - metricIterFirst prometheus.Observer - metricIterRelease prometheus.Observer -} - -func (state *pollState) hasResolvedEvents() bool { - // It has resolved events, if 1) it has buffer resolved events, - lenResolvedEvents, _ := state.outputBuf.len() - if lenResolvedEvents > 0 { - return true - } - // or 2) there are some events that can be resolved. - // -------|-----------------|-------------|-------> time - // exhaustedResolvedTs - // maxCommitTs - // maxResolvedTs - // -------|-----------------|-------------|-------> time - // exhaustedResolvedTs - // maxResolvedTs - // maxCommitTs - if state.position.exhaustedResolvedTs < state.maxCommitTs && - state.position.exhaustedResolvedTs < state.maxResolvedTs { - return true - } - - // Otherwise, there is no event can be resolved. - // -------|-----------------|-------------|-------> time - // maxCommitTs - // exhaustedResolvedTs - // maxResolvedTs - return false -} - -func (state *pollState) advanceMaxTs(maxCommitTs, maxResolvedTs uint64) { - // The max commit ts of all received events. - if maxCommitTs > state.maxCommitTs { - state.maxCommitTs = maxCommitTs - } - // The max resolved ts of all received resolvedTs events. - if maxResolvedTs > state.maxResolvedTs { - state.maxResolvedTs = maxResolvedTs - } -} - -// tryGetIterator tries to get an iterator. -// When it returns a request, caller must send it. -// When it returns true, it means there is an iterator that can be used. -func (state *pollState) tryGetIterator(uid uint32, tableID uint64) (*message.IterRequest, bool) { - if state.iter != nil && state.iterCh != nil { - log.Panic("assert failed, there can only be one of iter or iterCh", - zap.Any("iter", state.iter), zap.Uint64("tableID", tableID), - zap.Uint32("uid", uid)) - } - - if state.iter != nil { - return nil, true - } - - if state.iterCh == nil { - // We haven't sent request. - iterCh := make(chan *message.LimitedIterator, 1) - state.iterCh = iterCh - state.iterRequestTime = time.Now() - readerRouter := state.readerRouter - readerID := state.readerID - lowerBoundTs := atomic.LoadUint64(&state.startTs) - if lowerBoundTs < state.position.exhaustedResolvedTs { - lowerBoundTs = state.position.exhaustedResolvedTs - } - return &message.IterRequest{ - Range: [2][]byte{ - encoding.EncodeTsKey(uid, tableID, lowerBoundTs+1), - encoding.EncodeTsKey(uid, tableID, state.maxResolvedTs+1), - }, - ResolvedTs: state.maxResolvedTs, - CRTsFilter: [2]uint64{state.position.exhaustedResolvedTs + 1, state.maxResolvedTs}, - IterCallback: func(iter *message.LimitedIterator) { - iterCh <- iter - close(iterCh) - // Notify itself that iterator has acquired. - _ = readerRouter.Send(readerID, actormsg.ValueMessage( - message.Task{ - UID: uid, - Span: &state.span, - ReadTs: message.ReadTs{}, - })) - }, - }, false - } - - // Try to receive iterator. - select { - case iter := <-state.iterCh: - // Iterator received, reset state.iterCh - state.iterCh = nil - state.iter = iter - start := time.Now() - requestDuration := start.Sub(state.iterRequestTime) - state.metricIterRequest.Observe(requestDuration.Seconds()) - state.iterAliveTime = start - state.iterResolvedTs = iter.ResolvedTs - state.position.iterHasRead = false - state.iter.Seek(state.position.partialTxnKey) - duration := time.Since(start) - state.metricIterFirst.Observe(duration.Seconds()) - if duration >= state.iterFirstSlowDuration { - // Try trigger a compaction if Iterator.Seek is too slow. - state.compact.tryScheduleCompact(state.compactorID, 0) - } - return nil, true - default: - // Iterator is not ready yet. - return nil, false - } -} - -func (state *pollState) tryReleaseIterator(force bool) error { - if state.iter == nil { - return nil - } - now := time.Now() - if !state.iter.Valid() || now.Sub(state.iterAliveTime) > state.iterMaxAliveDuration || force { - err := state.iter.Release() - if err != nil { - return errors.Trace(err) - } - state.metricIterRelease.Observe(time.Since(now).Seconds()) - state.iter = nil - state.position.iterHasRead = true - - if state.iterCh != nil { - log.Panic("there must not be iterCh", zap.Any("iter", state.iter)) - } - } - - return nil -} - -// Poll receives ReadTs and send resolved events. -func (r *reader) Poll(ctx context.Context, msgs []actormsg.Message[message.Task]) (running bool) { - for i := range msgs { - switch msgs[i].Tp { - case actormsg.TypeValue: - case actormsg.TypeStop: - r.reportError("receive stop message", nil) - return false - default: - log.Panic("unexpected message", zap.Any("message", msgs[i])) - } - if msgs[i].Value.StartTs != 0 { - atomic.StoreUint64(&r.state.startTs, msgs[i].Value.StartTs) - continue - } - // Update the max commit ts and resolved ts of all received events. - ts := msgs[i].Value.ReadTs - r.state.advanceMaxTs(ts.MaxCommitTs, ts.MaxResolvedTs) - - // Test only message. - if msgs[i].Value.Test != nil { - time.Sleep(msgs[i].Value.Test.Sleep) - } - } - - // Length of buffered resolved events. - lenResolvedEvents, _ := r.state.outputBuf.len() - if lenResolvedEvents != 0 { - // Try output buffered resolved events. - r.outputBufferedResolvedEvents(r.state.outputBuf) - lenResolvedEvents, _ = r.state.outputBuf.len() - } - // Build task for new events and delete sent keys. - task := message.Task{UID: r.uid, Span: &r.span} - r.setTaskDelete(&task, r.state.outputBuf.deleteKeys) - // Reset buffer as delete keys are scheduled. - r.state.outputBuf.resetDeleteKey() - // Try shrink buffer to release memory. - r.state.outputBuf.maybeShrink() - - // It can only read an iterator when - // 1. No buffered resolved events, they must be sent before - // sending further resolved events from iterator. - readIter := lenResolvedEvents == 0 - // 2. There are some events that can be resolved. - readIter = readIter && r.state.hasResolvedEvents() - if !readIter { - // No buffered resolved events, try to send resolved ts. - if !r.state.hasResolvedEvents() && r.state.maxResolvedTs != 0 { - // To avoid ping-pong busy loop, we only send resolved ts - // when it advances. - if r.state.maxResolvedTs > r.lastSentResolvedTs { - r.outputResolvedTs(r.state.maxResolvedTs) - } - } - // Release iterator as we do not need to read. - err := r.state.tryReleaseIterator(false) - if err != nil { - r.reportError("failed to release iterator", err) - return false - } - // Send delete task to db. - if task.DeleteReq != nil { - err = r.dbRouter.SendB(ctx, r.dbActorID, actormsg.ValueMessage(task)) - if err != nil { - r.reportError("failed to send delete request", err) - return false - } - } - return true - } - - var hasIter bool - task.IterReq, hasIter = r.state.tryGetIterator(r.uid, uint64(r.span.TableID)) - // Send delete/read task to db. - err := r.dbRouter.SendB(ctx, r.dbActorID, actormsg.ValueMessage(task)) - if err != nil { - r.reportError("failed to send delete request", err) - return false - } - if !hasIter { - // Skip read iterator if there is no iterator - return true - } - - // Read and send resolved events from iterator. - position, err := r.outputIterEvents( - r.state.iter, r.state.position, r.state.outputBuf, r.state.iterResolvedTs) - if err != nil { - r.reportError("failed to read iterator", err) - return false - } - r.state.position.update(position) - err = r.state.tryReleaseIterator(false) - if err != nil { - r.reportError("failed to release iterator", err) - return false - } - return true -} - -// OnClose releases reader resource. -func (r *reader) OnClose() { - if r.stopped { - return - } - r.stopped = true - // Must release iterator before stopping, otherwise it leaks iterator. - _ = r.state.tryReleaseIterator(true) - r.common.closedWg.Done() -} diff --git a/cdc/sorter/db/reader_test.go b/cdc/sorter/db/reader_test.go deleted file mode 100644 index 1b7dbc6e776..00000000000 --- a/cdc/sorter/db/reader_test.go +++ /dev/null @@ -1,994 +0,0 @@ -// Copyright 2021 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package db - -import ( - "context" - "encoding/hex" - "fmt" - "math" - "testing" - "time" - - "github.com/pingcap/tiflow/cdc/model" - "github.com/pingcap/tiflow/cdc/sorter" - "github.com/pingcap/tiflow/cdc/sorter/db/message" - "github.com/pingcap/tiflow/cdc/sorter/encoding" - "github.com/pingcap/tiflow/pkg/actor" - actormsg "github.com/pingcap/tiflow/pkg/actor/message" - "github.com/pingcap/tiflow/pkg/config" - "github.com/pingcap/tiflow/pkg/db" - "github.com/pingcap/tiflow/pkg/spanz" - "github.com/prometheus/client_golang/prometheus" - "github.com/stretchr/testify/require" - "golang.org/x/sync/semaphore" -) - -func newTestReader() *reader { - metricIterDuration := sorterIterReadDurationHistogram.MustCurryWith( - prometheus.Labels{ - "namespace": "default", - "id": "test", - }) - - metricOutputKV := sorter.OutputEventCount. - WithLabelValues("default", "test", "kv") - metricOutputResolved := sorter.InputEventCount. - WithLabelValues("default", "test", "resolved") - return &reader{ - common: common{ - dbActorID: 1, - // dbRouter: dbRouter, - uid: 2, - span: spanz.TableIDToComparableSpan(3), - serde: &encoding.MsgPackGenSerde{}, - }, - state: pollState{ - metricIterRequest: metricIterDuration.WithLabelValues("request"), - metricIterFirst: metricIterDuration.WithLabelValues("first"), - metricIterRelease: metricIterDuration.WithLabelValues("release"), - }, - metricIterReadDuration: metricIterDuration.WithLabelValues("read"), - metricIterNextDuration: metricIterDuration.WithLabelValues("next"), - metricTotalEventsKV: metricOutputKV, - metricTotalEventsResolved: metricOutputResolved, - } -} - -func TestReaderSetTaskDelete(t *testing.T) { - t.Parallel() - - r := newTestReader() - r.delete = deleteThrottle{ - countThreshold: 2, - period: 2 * time.Second, - } - - cases := []struct { - deleteKeys []message.Key - sleep time.Duration - expectDelete *message.DeleteRequest - }{ - // Empty delete does not set delete. - { - deleteKeys: []message.Key{}, - }, - // 1 delete key does not set delete. - { - deleteKeys: []message.Key{ - message.Key(encoding.EncodeKey(r.uid, uint64(r.span.TableID), - model.NewPolymorphicEvent(&model.RawKVEntry{CRTs: 2}))), - }, - }, - // One more delete key sets delete. - { - deleteKeys: []message.Key{ - message.Key(encoding.EncodeKey(r.uid, uint64(r.span.TableID), - model.NewPolymorphicEvent(&model.RawKVEntry{CRTs: 1}))), - }, - expectDelete: &message.DeleteRequest{ - Count: 2, - Range: [2][]byte{ - encoding.EncodeTsKey(r.uid, uint64(r.span.TableID), 0), - encoding.EncodeKey(r.uid, uint64(r.span.TableID), - model.NewPolymorphicEvent(&model.RawKVEntry{CRTs: 1})), - }, - }, - }, - // Waiting long period sets delete. - { - deleteKeys: []message.Key{ - message.Key(encoding.EncodeKey(r.uid, uint64(r.span.TableID), - model.NewPolymorphicEvent(&model.RawKVEntry{CRTs: 3}))), - }, - sleep: 4 * time.Second, - expectDelete: &message.DeleteRequest{ - Count: 1, - Range: [2][]byte{ - encoding.EncodeTsKey(r.uid, uint64(r.span.TableID), 0), - encoding.EncodeKey(r.uid, uint64(r.span.TableID), - model.NewPolymorphicEvent(&model.RawKVEntry{CRTs: 3})), - }, - }, - }, - } - for i, cs := range cases { - if cs.sleep != 0 { - time.Sleep(cs.sleep) - } - deleteKeys := cs.deleteKeys - task := &message.Task{} - r.setTaskDelete(task, deleteKeys) - - require.EqualValues(t, &message.Task{ - DeleteReq: cs.expectDelete, - }, task, "case #%d, %v", i, cs) - } -} - -func TestReaderOutput(t *testing.T) { - t.Parallel() - - r := newTestReader() - - r.outputCh = make(chan *model.PolymorphicEvent, 1) - ok := r.output(&model.PolymorphicEvent{CRTs: 1}) - require.True(t, ok) - require.EqualValues(t, &model.PolymorphicEvent{CRTs: 1}, r.lastEvent) - ok = r.output(&model.PolymorphicEvent{CRTs: 1}) - require.False(t, ok) - r.outputResolvedTs(2) - require.EqualValues(t, 1, r.lastSentCommitTs) - - <-r.outputCh - r.outputResolvedTs(2) - require.EqualValues(t, 2, r.lastSentResolvedTs) - - <-r.outputCh - ok = r.output(&model.PolymorphicEvent{CRTs: 3}) - require.True(t, ok) -} - -func TestReaderOutputBufferedResolvedEvents(t *testing.T) { - t.Parallel() - - capacity := 4 - r := newTestReader() - - buf := newOutputBuffer(capacity) - - cases := []struct { - outputChCap int - inputEvents []*model.PolymorphicEvent - inputDeleteKeys []message.Key - - expectEvents []*model.PolymorphicEvent - expectDeleteKeys []message.Key - expectOutputs []*model.PolymorphicEvent - }{ - // Empty buffer. - { - outputChCap: 1, - inputEvents: []*model.PolymorphicEvent{}, - inputDeleteKeys: []message.Key{}, - - expectEvents: []*model.PolymorphicEvent{}, - expectDeleteKeys: []message.Key{}, - expectOutputs: []*model.PolymorphicEvent{}, - }, - // Output one event, delete one event. - { - outputChCap: 2, - inputEvents: []*model.PolymorphicEvent{ - model.NewPolymorphicEvent(&model.RawKVEntry{CRTs: 1}), - }, - inputDeleteKeys: []message.Key{}, - - expectEvents: []*model.PolymorphicEvent{}, - expectDeleteKeys: []message.Key{ - message.Key(encoding.EncodeKey(r.uid, uint64(r.span.TableID), - model.NewPolymorphicEvent(&model.RawKVEntry{CRTs: 1}))), - }, - expectOutputs: []*model.PolymorphicEvent{ - model.NewPolymorphicEvent(&model.RawKVEntry{CRTs: 1}), - // All inputEvent are sent, it also outputs a resolved ts event. - model.NewResolvedPolymorphicEvent(0, 1), - }, - }, - // Delete one event. - { - outputChCap: 2, - inputEvents: []*model.PolymorphicEvent{}, - inputDeleteKeys: []message.Key{ - message.Key(encoding.EncodeKey(r.uid, uint64(r.span.TableID), - model.NewPolymorphicEvent(&model.RawKVEntry{CRTs: 1}))), - }, - - expectEvents: []*model.PolymorphicEvent{}, - expectDeleteKeys: []message.Key{ - message.Key(encoding.EncodeKey(r.uid, uint64(r.span.TableID), - model.NewPolymorphicEvent(&model.RawKVEntry{CRTs: 1}))), - }, - expectOutputs: []*model.PolymorphicEvent{}, - }, - // Output one event, delete two event. - { - outputChCap: 2, - inputEvents: []*model.PolymorphicEvent{ - model.NewPolymorphicEvent(&model.RawKVEntry{CRTs: 2}), - }, - inputDeleteKeys: []message.Key{ - message.Key(encoding.EncodeKey(r.uid, uint64(r.span.TableID), - model.NewPolymorphicEvent(&model.RawKVEntry{CRTs: 1}))), - }, - - expectEvents: []*model.PolymorphicEvent{}, - expectDeleteKeys: []message.Key{ - message.Key(encoding.EncodeKey(r.uid, uint64(r.span.TableID), - model.NewPolymorphicEvent(&model.RawKVEntry{CRTs: 1}))), - message.Key(encoding.EncodeKey(r.uid, uint64(r.span.TableID), - model.NewPolymorphicEvent(&model.RawKVEntry{CRTs: 2}))), - }, - expectOutputs: []*model.PolymorphicEvent{ - model.NewPolymorphicEvent(&model.RawKVEntry{CRTs: 2}), - // All inputEvent are sent, it also outputs a resolved ts event. - model.NewResolvedPolymorphicEvent(0, 2), - }, - }, - // Output two events, left one event. - { - outputChCap: 2, - inputEvents: []*model.PolymorphicEvent{ - model.NewPolymorphicEvent(&model.RawKVEntry{CRTs: 3, RegionID: 1}), - model.NewPolymorphicEvent(&model.RawKVEntry{CRTs: 3, RegionID: 2}), - model.NewPolymorphicEvent(&model.RawKVEntry{CRTs: 3, RegionID: 3}), - }, - inputDeleteKeys: []message.Key{}, - - expectEvents: []*model.PolymorphicEvent{ - model.NewPolymorphicEvent(&model.RawKVEntry{CRTs: 3, RegionID: 3}), - }, - expectDeleteKeys: []message.Key{ - message.Key(encoding.EncodeKey(r.uid, uint64(r.span.TableID), - model.NewPolymorphicEvent(&model.RawKVEntry{CRTs: 3, RegionID: 1}))), - message.Key(encoding.EncodeKey(r.uid, uint64(r.span.TableID), - model.NewPolymorphicEvent(&model.RawKVEntry{CRTs: 3, RegionID: 2}))), - }, - expectOutputs: []*model.PolymorphicEvent{ - model.NewPolymorphicEvent(&model.RawKVEntry{CRTs: 3, RegionID: 1}), - model.NewPolymorphicEvent(&model.RawKVEntry{CRTs: 3, RegionID: 2}), - // No resolved ts event because not all events are sent. - }, - }, - // Output zero event, left two events. - { - outputChCap: 0, - inputEvents: []*model.PolymorphicEvent{ - model.NewPolymorphicEvent(&model.RawKVEntry{CRTs: 4, RegionID: 1}), - model.NewPolymorphicEvent(&model.RawKVEntry{CRTs: 4, RegionID: 2}), - }, - inputDeleteKeys: []message.Key{}, - - expectEvents: []*model.PolymorphicEvent{ - model.NewPolymorphicEvent(&model.RawKVEntry{CRTs: 4, RegionID: 1}), - model.NewPolymorphicEvent(&model.RawKVEntry{CRTs: 4, RegionID: 2}), - }, - expectDeleteKeys: []message.Key{}, - expectOutputs: []*model.PolymorphicEvent{}, - }, - } - - for i, cs := range cases { - r.outputCh = make(chan *model.PolymorphicEvent, cs.outputChCap) - buf.resolvedEvents = append([]*model.PolymorphicEvent{}, cs.inputEvents...) - buf.deleteKeys = append([]message.Key{}, cs.inputDeleteKeys...) - - r.outputBufferedResolvedEvents(buf) - require.EqualValues(t, cs.expectDeleteKeys, buf.deleteKeys, "case #%d, %v", i, cs) - require.EqualValues(t, cs.expectEvents, buf.resolvedEvents, "case #%d, %v", i, cs) - - outputEvents := []*model.PolymorphicEvent{} - RECV: - for { - select { - case ev := <-r.outputCh: - outputEvents = append(outputEvents, ev) - default: - break RECV - } - } - require.EqualValues(t, cs.expectOutputs, outputEvents, "case #%d, %v", i, cs) - } -} - -func prepareTxnData( - t *testing.T, r *reader, txnCount, txnSize int, -) db.DB { - cfg := config.GetDefaultServerConfig().Clone().Debug.DB - db, err := db.OpenPebble( - context.Background(), 1, t.TempDir(), cfg, db.WithTableCRTsCollectors()) - require.Nil(t, err) - wb := db.Batch(0) - for i := 1; i < txnCount+1; i++ { // txns. - for j := 0; j < txnSize; j++ { // events. - event := newTestEvent(uint64(i)+2, uint64(i), j) - key := encoding.EncodeKey(r.uid, uint64(r.span.TableID), event) - value, err := r.serde.Marshal(event, []byte{}) - require.Nil(t, err) - t.Logf("key: %s, value: %s\n", message.Key(key), hex.EncodeToString(value)) - wb.Put(key, value) - } - } - require.Nil(t, wb.Commit()) - return db -} - -func TestReaderOutputIterEvents(t *testing.T) { - t.Parallel() - - capacity := 4 - r := newTestReader() - tableID := r.span.TableID - - // Prepare data, 3 txns, 3 events for each. - // CRTs 3, StartTs 1, keys (0|1|2) - // CRTs 4, StartTs 2, keys (0|1|2) - // CRTs 5, StartTs 3, keys (0|1|2) - // CRTs 6, StartTs 4, keys (0|1|2) - // CRTs 7, StartTs 4, keys (0|1|2) - db := prepareTxnData(t, r, 5, 3) - - cases := []struct { - outputChCap int - maxResolvedTs uint64 - hasReadNext bool - - expectEvents []*model.PolymorphicEvent - expectDeleteKeys []message.Key - expectOutputs []*model.PolymorphicEvent - expectExhaustedRTs uint64 - expectHasReadNext bool - }{ - // Empty resolved event. - { - outputChCap: 1, - maxResolvedTs: 0, - - expectEvents: []*model.PolymorphicEvent{}, - expectDeleteKeys: []message.Key{}, - expectOutputs: []*model.PolymorphicEvent{}, - expectExhaustedRTs: 0, - expectHasReadNext: true, - }, - // Nonblocking output three events. - { - outputChCap: 3, - maxResolvedTs: 3, // CRTs 3 has 3 events. - - expectEvents: []*model.PolymorphicEvent{}, - expectDeleteKeys: []message.Key{ - message.Key(encoding.EncodeKey(r.uid, uint64(tableID), newTestEvent(3, 1, 0))), - message.Key(encoding.EncodeKey(r.uid, uint64(tableID), newTestEvent(3, 1, 1))), - message.Key(encoding.EncodeKey(r.uid, uint64(tableID), newTestEvent(3, 1, 2))), - }, - expectOutputs: []*model.PolymorphicEvent{ - newTestEvent(3, 1, 0), - newTestEvent(3, 1, 1), - newTestEvent(3, 1, 2), - }, - expectExhaustedRTs: 3, // Iter is exhausted and no buffered resolved events. - expectHasReadNext: true, - }, - // Blocking output two events of CRTs 4. - { - outputChCap: 2, - maxResolvedTs: 4, // CRTs 4 has 3 events. - - expectEvents: []*model.PolymorphicEvent{newTestEvent(4, 2, 2)}, - expectDeleteKeys: []message.Key{ - message.Key(encoding.EncodeKey(r.uid, uint64(tableID), newTestEvent(4, 2, 0))), - message.Key(encoding.EncodeKey(r.uid, uint64(tableID), newTestEvent(4, 2, 1))), - }, - expectOutputs: []*model.PolymorphicEvent{ - newTestEvent(4, 2, 0), - newTestEvent(4, 2, 1), - }, - // Events of CRTs 4 have been read and buffered. - expectExhaustedRTs: 4, - expectHasReadNext: true, - }, - // Output remaining event of CRTs 4. - { - outputChCap: 3, - maxResolvedTs: 4, // CRTs 4 has 1 events. - - expectEvents: []*model.PolymorphicEvent{}, - expectDeleteKeys: []message.Key{ - message.Key(encoding.EncodeKey(r.uid, uint64(tableID), newTestEvent(4, 2, 2))), - }, - expectOutputs: []*model.PolymorphicEvent{ - newTestEvent(4, 2, 2), - model.NewResolvedPolymorphicEvent(0, 4), - }, - expectExhaustedRTs: 4, // Iter is exhausted and no buffered resolved events. - expectHasReadNext: true, - }, - // Resolved ts covers all resolved events, - // blocking output events of CRTs 5 (3 events) and 6 (1 event). - { - outputChCap: 5, - maxResolvedTs: 7, - - expectEvents: []*model.PolymorphicEvent{ - newTestEvent(6, 4, 1), - newTestEvent(6, 4, 2), - }, - expectDeleteKeys: []message.Key{ - message.Key(encoding.EncodeKey(r.uid, uint64(tableID), newTestEvent(5, 3, 0))), - message.Key(encoding.EncodeKey(r.uid, uint64(tableID), newTestEvent(5, 3, 1))), - message.Key(encoding.EncodeKey(r.uid, uint64(tableID), newTestEvent(5, 3, 2))), - message.Key(encoding.EncodeKey(r.uid, uint64(tableID), newTestEvent(6, 4, 0))), - }, - expectOutputs: []*model.PolymorphicEvent{ - newTestEvent(5, 3, 0), - newTestEvent(5, 3, 1), - newTestEvent(5, 3, 2), - model.NewResolvedPolymorphicEvent(0, 5), - newTestEvent(6, 4, 0), - }, - // Iter is not exhausted, but all events with commit ts 6 have been - // read into buffer. - expectExhaustedRTs: 6, - expectHasReadNext: false, // (6, 4, 1) is neither output nor buffered. - }, - // Resolved ts covers all resolved events, nonblocking output all events. - { - outputChCap: 7, - maxResolvedTs: 7, - - expectEvents: []*model.PolymorphicEvent{}, - expectDeleteKeys: []message.Key{ - message.Key(encoding.EncodeKey(r.uid, uint64(tableID), newTestEvent(6, 4, 1))), - message.Key(encoding.EncodeKey(r.uid, uint64(tableID), newTestEvent(6, 4, 2))), - message.Key(encoding.EncodeKey(r.uid, uint64(tableID), newTestEvent(7, 5, 0))), - message.Key(encoding.EncodeKey(r.uid, uint64(tableID), newTestEvent(7, 5, 1))), - message.Key(encoding.EncodeKey(r.uid, uint64(tableID), newTestEvent(7, 5, 2))), - }, - expectOutputs: []*model.PolymorphicEvent{ - newTestEvent(6, 4, 1), - newTestEvent(6, 4, 2), - model.NewResolvedPolymorphicEvent(0, 6), - newTestEvent(7, 5, 0), - newTestEvent(7, 5, 1), - newTestEvent(7, 5, 2), - model.NewResolvedPolymorphicEvent(0, 7), - }, - expectExhaustedRTs: 7, // Iter is exhausted and no buffered resolved events. - expectHasReadNext: true, - }, - // All resolved events outputted, as resolved ts continues advance, - // exhausted resolved ts advances too. - { - outputChCap: 1, - maxResolvedTs: 8, - - expectEvents: []*model.PolymorphicEvent{}, - expectDeleteKeys: []message.Key{}, - expectOutputs: []*model.PolymorphicEvent{}, - expectExhaustedRTs: 8, - expectHasReadNext: true, - }, - } - - for i, cs := range cases { - r.outputCh = make(chan *model.PolymorphicEvent, cs.outputChCap) - buf := newOutputBuffer(capacity) - - iter := db.Iterator( - encoding.EncodeTsKey(r.uid, uint64(tableID), 0), - encoding.EncodeTsKey(r.uid, uint64(tableID), cs.maxResolvedTs+1), - 0, math.MaxUint64) - iter.Seek([]byte{}) - require.Nil(t, iter.Error(), "case #%d, %v", i, cs) - pos, err := r.outputIterEvents( - iter, readPosition{iterHasRead: cs.hasReadNext}, buf, cs.maxResolvedTs) - require.Nil(t, err, "case #%d, %v", i, cs) - require.EqualValues( - t, cs.expectExhaustedRTs, pos.exhaustedResolvedTs, "case #%d, %v", i, cs) - for _, k := range buf.deleteKeys { - fmt.Printf("%s\n", k) - } - require.EqualValues(t, cs.expectDeleteKeys, buf.deleteKeys, "case #%d, %v", i, cs) - require.EqualValues(t, cs.expectEvents, buf.resolvedEvents, "case #%d, %v", i, cs) - require.EqualValues(t, cs.expectHasReadNext, pos.iterHasRead, "case #%d, %v", i, cs) - outputEvents := receiveOutputEvents(r.outputCh) - require.EqualValues(t, cs.expectOutputs, outputEvents, "case #%d, %v", i, cs) - - wb := db.Batch(0) - for _, key := range cs.expectDeleteKeys { - wb.Delete([]byte(key)) - } - require.Nil(t, wb.Commit()) - require.Nil(t, iter.Release()) - } - - require.Nil(t, db.Close()) -} - -func TestReaderStateIterator(t *testing.T) { - t.Parallel() - - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - r := newTestReader() - // Prepare data, 1 txn. - db := prepareTxnData(t, r, 1, 1) - sema := semaphore.NewWeighted(1) - router := actor.NewRouter[message.Task](t.Name()) - compactMb := actor.NewMailbox[message.Task](1, 1) - router.InsertMailbox4Test(compactMb.ID(), compactMb) - readerMb := actor.NewMailbox[message.Task](2, 1) - router.InsertMailbox4Test(readerMb.ID(), readerMb) - state := r.state - state.readerID = readerMb.ID() - state.readerRouter = router - state.compactorID = compactMb.ID() - state.iterFirstSlowDuration = 100 * time.Second - state.compact = NewCompactScheduler(router) - state.iterMaxAliveDuration = 100 * time.Millisecond - - // First get returns a request. - req, ok := state.tryGetIterator(1, 1) - require.False(t, ok) - require.NotNil(t, req) - - // Still wait for iterator response. - req1, ok := state.tryGetIterator(1, 1) - require.False(t, ok) - require.Nil(t, req1) - - // Send iterator. - require.Nil(t, sema.Acquire(ctx, 1)) - req.IterCallback(&message.LimitedIterator{ - Iterator: db.Iterator([]byte{}, []byte{}, 0, math.MaxUint64), - Sema: sema, - }) - // Must notify reader - _, ok = readerMb.Receive() - require.True(t, ok) - // Get iterator successfully. - req2, ok := state.tryGetIterator(1, 1) - require.True(t, ok) - require.Nil(t, req2) - // Get iterator successfully again. - req2, ok = state.tryGetIterator(1, 1) - require.True(t, ok) - require.Nil(t, req2) - - // Release an invalid iterator. - require.False(t, state.iter.Valid()) - require.Nil(t, state.tryReleaseIterator(false)) - require.Nil(t, state.iter) - - // Release an outdated iterator. - require.Nil(t, sema.Acquire(ctx, 1)) - state.iter = &message.LimitedIterator{ - Iterator: db.Iterator([]byte{}, []byte{0xff}, 0, math.MaxUint64), - Sema: sema, - } - require.True(t, state.iter.Seek([]byte{})) - state.iterAliveTime = time.Now() - time.Sleep(2 * state.iterMaxAliveDuration) - require.Nil(t, state.tryReleaseIterator(false)) - require.Nil(t, state.iter) - - // Release empty iterator. - require.Nil(t, state.tryReleaseIterator(false)) - - // Slow first must send a compaction task. - req3, ok := state.tryGetIterator(1, 1) - require.False(t, ok) - require.NotNil(t, req3) - require.Nil(t, sema.Acquire(ctx, 1)) - req3.IterCallback(&message.LimitedIterator{ - Iterator: db.Iterator([]byte{}, []byte{}, 0, math.MaxUint64), - Sema: sema, - }) - // Must notify reader - _, ok = readerMb.Receive() - require.True(t, ok) - // No compaction task yet. - _, ok = compactMb.Receive() - require.False(t, ok) - // Always slow. - state.iterFirstSlowDuration = time.Duration(0) - _, ok = state.tryGetIterator(1, 1) - require.True(t, ok) - require.NotNil(t, state.iter) - // Must recv a compaction task. - _, ok = compactMb.Receive() - require.True(t, ok) - // Release iterator. - time.Sleep(2 * state.iterMaxAliveDuration) - require.Nil(t, state.tryReleaseIterator(false)) - require.Nil(t, state.iter) - - require.Nil(t, db.Close()) -} - -func TestReaderPoll(t *testing.T) { - t.Parallel() - - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - - capacity := 4 - router := actor.NewRouter[message.Task](t.Name()) - sema := semaphore.NewWeighted(1) - dbMb := actor.NewMailbox[message.Task](1, capacity) - router.InsertMailbox4Test(dbMb.ID(), dbMb) - readerMb := actor.NewMailbox[message.Task](2, capacity) - router.InsertMailbox4Test(readerMb.ID(), readerMb) - r := newTestReader() - r.common.dbRouter = router - r.common.dbActorID = dbMb.ID() - r.outputCh = make(chan *model.PolymorphicEvent, sorterOutputCap) - tableID := r.span.TableID - - // Prepare data, 3 txns, 3 events for each. - // CRTs 3, StartTs 1, keys (0|1|2) - // CRTs 4, StartTs 2, keys (0|1|2) - db := prepareTxnData(t, r, 2, 3) - - // We need to poll twice to read resolved events, so we need a slice of - // two cases. - cases := [][]struct { - inputReadTs message.ReadTs - inputIter func([2][]byte) *message.LimitedIterator - state pollState - releaseIter bool - - expectEvents []*model.PolymorphicEvent - expectDeleteKeys []message.Key - expectOutputs []*model.PolymorphicEvent - expectPartialTxnKey []byte - expectMaxCommitTs uint64 - expectMaxResolvedTs uint64 - expectExhaustedRTs uint64 - }{ - {{ // The first poll - inputReadTs: message.ReadTs{MaxResolvedTs: 1}, - state: pollState{ - outputBuf: newOutputBuffer(1), - }, - inputIter: nil, - - expectEvents: []*model.PolymorphicEvent{}, - expectDeleteKeys: []message.Key{}, - expectOutputs: []*model.PolymorphicEvent{model.NewResolvedPolymorphicEvent(0, 1)}, - expectMaxCommitTs: 0, - expectMaxResolvedTs: 1, - expectExhaustedRTs: 0, - }, { // The second poll - inputReadTs: message.ReadTs{MaxResolvedTs: 1}, - // state is inherited from the first poll. - inputIter: nil, // no need to make an iterator. - - expectEvents: []*model.PolymorphicEvent{}, - expectDeleteKeys: []message.Key{}, - // Does not output the same resolved ts twice. - expectOutputs: []*model.PolymorphicEvent{}, - expectMaxCommitTs: 0, - expectMaxResolvedTs: 1, - expectExhaustedRTs: 0, - }}, - // maxCommitTs and maxResolvedTs must advance according to inputs. - // And exhaustedResolvedTs must advance if there is no resolved event. - {{ // The first poll - inputReadTs: message.ReadTs{MaxResolvedTs: 2, MaxCommitTs: 3}, - state: pollState{ - outputBuf: newOutputBuffer(1), - }, - // An empty iterator. - inputIter: newEmptyIterator(ctx, t, db, sema), - - expectEvents: []*model.PolymorphicEvent{}, - expectDeleteKeys: []message.Key{}, - expectOutputs: []*model.PolymorphicEvent{}, - expectMaxCommitTs: 3, - expectMaxResolvedTs: 2, - expectExhaustedRTs: 0, - }, { // The second poll - inputReadTs: message.ReadTs{MaxResolvedTs: 2}, - // state is inherited from the first poll. - inputIter: nil, // no need to make an iterator. - - expectEvents: []*model.PolymorphicEvent{}, - expectDeleteKeys: []message.Key{}, - expectOutputs: []*model.PolymorphicEvent{}, - expectMaxCommitTs: 3, - expectMaxResolvedTs: 2, - // exhaustedResolvedTs must advance if there is no resolved event. - expectExhaustedRTs: 2, - }}, - // exhaustedResolvedTs must not advance if a txn is partially read. - // Output: CRTs 3, StartTs 1, keys (0|1|2) - {{ // The first poll - inputReadTs: message.ReadTs{MaxResolvedTs: 3, MaxCommitTs: 3}, - state: pollState{ - // A smaller buffer so that it can not hold all txn events. - outputBuf: newOutputBuffer(1), - }, - inputIter: newIterator(ctx, t, db, sema), - - expectEvents: []*model.PolymorphicEvent{}, - expectDeleteKeys: []message.Key{}, - expectOutputs: []*model.PolymorphicEvent{}, - expectMaxCommitTs: 3, - expectMaxResolvedTs: 3, - expectExhaustedRTs: 0, - }, { // The second poll - inputReadTs: message.ReadTs{MaxResolvedTs: 3}, - // state is inherited from the first poll. - inputIter: nil, // no need to make an iterator. - - expectEvents: []*model.PolymorphicEvent{}, - expectDeleteKeys: []message.Key{ - message.Key(encoding.EncodeKey(r.uid, uint64(tableID), newTestEvent(3, 1, 0))), - }, - expectOutputs: []*model.PolymorphicEvent{ - newTestEvent(3, 1, 0), - }, - expectPartialTxnKey: encoding.EncodeKey(r.uid, uint64(tableID), newTestEvent(3, 1, 1)), - expectMaxCommitTs: 3, - expectMaxResolvedTs: 3, - // exhaustedResolvedTs must not advance if a txn is partially read. - expectExhaustedRTs: 0, - }, { // The third poll - inputReadTs: message.ReadTs{MaxResolvedTs: 3}, - // state is inherited from the first poll. - inputIter: nil, // no need to make an iterator. - - expectEvents: []*model.PolymorphicEvent{}, - expectDeleteKeys: []message.Key{ - message.Key(encoding.EncodeKey(r.uid, uint64(tableID), newTestEvent(3, 1, 1))), - }, - expectOutputs: []*model.PolymorphicEvent{ - newTestEvent(3, 1, 1), - }, - expectPartialTxnKey: encoding.EncodeKey(r.uid, uint64(tableID), newTestEvent(3, 1, 2)), - expectMaxCommitTs: 3, - expectMaxResolvedTs: 3, - // exhaustedResolvedTs must not advance if a txn is partially read. - expectExhaustedRTs: 0, - }, { // The fourth poll, mock releasing iterator during read. - inputReadTs: message.ReadTs{MaxResolvedTs: 3}, - // Release iterator to make reader request iter again. - releaseIter: true, - inputIter: newIterator(ctx, t, db, sema), - - expectEvents: []*model.PolymorphicEvent{}, - expectDeleteKeys: []message.Key{}, - expectOutputs: []*model.PolymorphicEvent{}, - expectPartialTxnKey: encoding.EncodeKey(r.uid, uint64(tableID), newTestEvent(3, 1, 2)), - expectMaxCommitTs: 3, - expectMaxResolvedTs: 3, - // exhaustedResolvedTs must advance if a txn is completely read. - expectExhaustedRTs: 0, - }, { // The fifth poll, all events read. - inputReadTs: message.ReadTs{MaxResolvedTs: 3}, - // state is inherited from the fourth poll. - inputIter: nil, - - expectEvents: []*model.PolymorphicEvent{}, - expectDeleteKeys: []message.Key{ - message.Key(encoding.EncodeKey(r.uid, uint64(tableID), newTestEvent(3, 1, 2))), - }, - expectOutputs: []*model.PolymorphicEvent{ - newTestEvent(3, 1, 2), - model.NewResolvedPolymorphicEvent(0, 3), - }, - expectMaxCommitTs: 3, - expectMaxResolvedTs: 3, - // exhaustedResolvedTs must advance if a txn is completely read. - expectExhaustedRTs: 3, - }}, - // exhaustedResolvedTs must advance if all resolved events are outputted. - // Output: CRTs 3, StartTs 1, keys (0|1|2) - {{ // The first poll - inputReadTs: message.ReadTs{MaxResolvedTs: 3, MaxCommitTs: 3}, - state: pollState{ - outputBuf: newOutputBuffer(3), - }, - inputIter: newIterator(ctx, t, db, sema), - - expectEvents: []*model.PolymorphicEvent{}, - expectDeleteKeys: []message.Key{}, - expectOutputs: []*model.PolymorphicEvent{}, - expectMaxCommitTs: 3, - expectMaxResolvedTs: 3, - expectExhaustedRTs: 0, - }, { // The second poll - inputReadTs: message.ReadTs{MaxResolvedTs: 3}, - // state is inherited from the first poll. - inputIter: nil, // no need to make an iterator. - - expectEvents: []*model.PolymorphicEvent{}, - expectDeleteKeys: []message.Key{ - message.Key(encoding.EncodeKey(r.uid, uint64(tableID), newTestEvent(3, 1, 0))), - message.Key(encoding.EncodeKey(r.uid, uint64(tableID), newTestEvent(3, 1, 1))), - message.Key(encoding.EncodeKey(r.uid, uint64(tableID), newTestEvent(3, 1, 2))), - }, - expectOutputs: []*model.PolymorphicEvent{ - newTestEvent(3, 1, 0), - newTestEvent(3, 1, 1), - newTestEvent(3, 1, 2), - model.NewResolvedPolymorphicEvent(0, 3), - }, - expectMaxCommitTs: 3, - expectMaxResolvedTs: 3, - // exhaustedResolvedTs must advance if there is no resolved event. - expectExhaustedRTs: 3, - }}, - // Batch output buffered resolved events - {{ // The first poll - inputReadTs: message.ReadTs{}, - state: pollState{ - outputBuf: &outputBuffer{ - deleteKeys: make([]message.Key, 0, 2), - resolvedEvents: []*model.PolymorphicEvent{ - model.NewPolymorphicEvent(&model.RawKVEntry{CRTs: 4}), - model.NewPolymorphicEvent(&model.RawKVEntry{CRTs: 4}), - model.NewPolymorphicEvent(&model.RawKVEntry{CRTs: 4}), - }, - advisedCapacity: 2, - }, - }, - inputIter: nil, - - expectEvents: []*model.PolymorphicEvent{}, - expectDeleteKeys: []message.Key{}, - expectOutputs: []*model.PolymorphicEvent{ - model.NewPolymorphicEvent(&model.RawKVEntry{CRTs: 4}), - model.NewPolymorphicEvent(&model.RawKVEntry{CRTs: 4}), - model.NewPolymorphicEvent(&model.RawKVEntry{CRTs: 4}), - model.NewResolvedPolymorphicEvent(0, 4), - }, - expectMaxCommitTs: 0, - expectMaxResolvedTs: 0, - expectExhaustedRTs: 0, - }, { // The second poll - inputReadTs: message.ReadTs{MaxResolvedTs: 4}, - // state is inherited from the first poll. - inputIter: nil, // no need to make an iterator. - - expectEvents: []*model.PolymorphicEvent{}, - expectDeleteKeys: []message.Key{}, - // Does not output the same resolved ts twice. - expectOutputs: []*model.PolymorphicEvent{}, - expectMaxCommitTs: 0, - expectMaxResolvedTs: 4, - expectExhaustedRTs: 0, - }}, - } - - metricIterDuration := sorterIterReadDurationHistogram.MustCurryWith( - prometheus.Labels{ - "namespace": "default", - "id": t.Name(), - }) - for i, css := range cases { - r.state = css[0].state - r.state.readerRouter = router - r.state.readerID = readerMb.ID() - r.state.iterFirstSlowDuration = 100 * time.Second - r.state.iterMaxAliveDuration = 100 * time.Second - // Do not send delete range. - r.delete.countThreshold = 1000 - r.delete.period = 100 * time.Second - r.state.metricIterRequest = metricIterDuration.WithLabelValues("request") - r.state.metricIterFirst = metricIterDuration.WithLabelValues("first") - r.state.metricIterRelease = metricIterDuration.WithLabelValues("release") - for j, cs := range css { - t.Logf("test case #%d[%d], %v", i, j, cs) - if cs.releaseIter { - require.Nil(t, r.state.tryReleaseIterator(true)) - } - msg := actormsg.ValueMessage(message.Task{ReadTs: cs.inputReadTs}) - require.True(t, r.Poll(ctx, []actormsg.Message[message.Task]{msg})) - require.EqualValues( - t, cs.expectEvents, r.state.outputBuf.resolvedEvents, - "case #%d[%d], %v", i, j, cs) - require.EqualValues( - t, cs.expectDeleteKeys, r.state.outputBuf.deleteKeys, - "case #%d[%d], %v", i, j, cs) - require.EqualValues( - t, cs.expectMaxCommitTs, r.state.maxCommitTs, - "case #%d[%d], %v", i, j, cs) - require.EqualValues( - t, cs.expectMaxResolvedTs, r.state.maxResolvedTs, - "case #%d[%d], %v", i, j, cs) - require.EqualValues( - t, cs.expectExhaustedRTs, r.state.position.exhaustedResolvedTs, - "case #%d[%d], %v", i, j, cs) - require.EqualValues( - t, cs.expectPartialTxnKey, r.state.position.partialTxnKey, - "case #%d[%d], %v", i, j, cs) - outputEvents := receiveOutputEvents(r.outputCh) - require.EqualValues( - t, cs.expectOutputs, outputEvents, - "case #%d[%d], %v", i, j, cs) - - select { - case err := <-r.errCh: - require.Fail(t, "must not receive error", err) - default: - } - task, ok := dbMb.Receive() - if !ok { - // No task, so there must be nil inputIter. - require.Nil(t, cs.inputIter, "case #%d[%d], %v", i, j, cs) - continue - } - handleTask(t, task, cs.inputIter, readerMb) - } - if r.state.iter != nil { - require.Nil(t, r.state.iter.Release()) - } - } - - require.Nil(t, db.Close()) -} - -func handleTask( - t *testing.T, task actormsg.Message[message.Task], - iterFn func(rg [2][]byte) *message.LimitedIterator, readerMb actor.Mailbox[message.Task], -) { - if task.Value.IterReq == nil || iterFn == nil { - return - } - iter := iterFn(task.Value.IterReq.Range) - if iter != nil { - iter.ResolvedTs = task.Value.IterReq.ResolvedTs - task.Value.IterReq.IterCallback(iter) - // Must notify reader - _, ok := readerMb.Receive() - require.True(t, ok) - } -} - -func newIterator( - ctx context.Context, t *testing.T, db db.DB, sema *semaphore.Weighted, -) func(rg [2][]byte) *message.LimitedIterator { - return func(rg [2][]byte) *message.LimitedIterator { - require.Nil(t, sema.Acquire(ctx, 1)) - t.Logf("newIterator %s %s\n", message.Key(rg[0]), message.Key(rg[1])) - return &message.LimitedIterator{ - Iterator: db.Iterator(rg[0], rg[1], 0, math.MaxUint64), - Sema: sema, - } - } -} - -func newEmptyIterator( - ctx context.Context, t *testing.T, db db.DB, sema *semaphore.Weighted, -) func(rg [2][]byte) *message.LimitedIterator { - return func(rg [2][]byte) *message.LimitedIterator { - require.Nil(t, sema.Acquire(ctx, 1)) - return &message.LimitedIterator{ - Iterator: db.Iterator([]byte{}, []byte{}, 0, math.MaxUint64), - Sema: sema, - } - } -} diff --git a/cdc/sorter/db/sorter.go b/cdc/sorter/db/sorter.go deleted file mode 100644 index 848dc64db56..00000000000 --- a/cdc/sorter/db/sorter.go +++ /dev/null @@ -1,281 +0,0 @@ -// Copyright 2022 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package db - -import ( - "context" - "sync" - "sync/atomic" - "time" - - "github.com/pingcap/errors" - "github.com/pingcap/log" - "github.com/pingcap/tiflow/cdc/model" - "github.com/pingcap/tiflow/cdc/processor/tablepb" - "github.com/pingcap/tiflow/cdc/sorter" - "github.com/pingcap/tiflow/cdc/sorter/db/message" - "github.com/pingcap/tiflow/cdc/sorter/encoding" - "github.com/pingcap/tiflow/pkg/actor" - actormsg "github.com/pingcap/tiflow/pkg/actor/message" - "github.com/pingcap/tiflow/pkg/config" - "github.com/prometheus/client_golang/prometheus" - "go.uber.org/zap" -) - -const ( - // Capacity of db sorter input and output channels. - sorterInputCap, sorterOutputCap = 1, 64 - // Max size of received event batch. - batchReceiveEventSize = 32 -) - -var dbSorterIDAlloc uint32 = 0 - -func allocID() uint32 { - return atomic.AddUint32(&dbSorterIDAlloc, 1) -} - -type common struct { - dbActorID actor.ID - dbRouter *actor.Router[message.Task] - - uid uint32 - span tablepb.Span - serde *encoding.MsgPackGenSerde - errCh chan error - closedWg *sync.WaitGroup -} - -// reportError notifies Sorter to return an error and close. -func (c *common) reportError(msg string, err error) { - if errors.Cause(err) != context.Canceled { - log.L().WithOptions(zap.AddCallerSkip(1)). - Warn(msg, zap.Stringer("span", &c.span), zap.Error(err)) - } - select { - case c.errCh <- err: - default: - // It means there is an error already. - } -} - -// Sorter accepts out-of-order raw kv entries and output sorted entries -type Sorter struct { - common - - writerRouter *actor.Router[message.Task] - writerActorID actor.ID - writer *writer - - readerRouter *actor.Router[message.Task] - ReaderActorID actor.ID - reader *reader - - outputCh chan *model.PolymorphicEvent - closed int32 -} - -// NewSorter creates a new Sorter -func NewSorter( - ctx context.Context, changefeedID model.ChangeFeedID, span tablepb.Span, startTs uint64, - dbRouter *actor.Router[message.Task], dbActorID actor.ID, - writerSystem *actor.System[message.Task], writerRouter *actor.Router[message.Task], - readerSystem *actor.System[message.Task], readerRouter *actor.Router[message.Task], - compact *CompactScheduler, cfg *config.DBConfig, -) (*Sorter, error) { - metricIterDuration := sorterIterReadDurationHistogram.MustCurryWith( - prometheus.Labels{ - "namespace": changefeedID.Namespace, - "id": changefeedID.ID, - }) - metricInputKV := sorter.InputEventCount. - WithLabelValues(changefeedID.Namespace, changefeedID.ID, "kv") - metricInputResolved := sorter.InputEventCount. - WithLabelValues(changefeedID.Namespace, changefeedID.ID, "resolved") - metricOutputKV := sorter.OutputEventCount. - WithLabelValues(changefeedID.Namespace, changefeedID.ID, "kv") - metricOutputResolved := sorter.OutputEventCount. - WithLabelValues(changefeedID.Namespace, changefeedID.ID, "resolved") - - // TODO: test capture the same table multiple times. - uid := allocID() - actorID := actor.ID(uid) - c := common{ - dbActorID: dbActorID, - dbRouter: dbRouter, - uid: uid, - span: span, - serde: &encoding.MsgPackGenSerde{}, - errCh: make(chan error, 1), - closedWg: &sync.WaitGroup{}, - } - - w := &writer{ - common: c, - readerRouter: readerRouter, - readerActorID: actorID, - - metricTotalEventsKV: metricInputKV, - metricTotalEventsResolved: metricInputResolved, - } - wmb := actor.NewMailbox[message.Task](actorID, sorterInputCap) - err := writerSystem.Spawn(wmb, w) - if err != nil { - return nil, errors.Trace(err) - } - c.closedWg.Add(1) - - r := &reader{ - common: c, - - state: pollState{ - outputBuf: newOutputBuffer(batchReceiveEventSize), - - maxCommitTs: uint64(0), - maxResolvedTs: uint64(0), - startTs: uint64(0), - - readerID: actorID, - readerRouter: readerRouter, - - compactorID: dbActorID, - compact: compact, - iterMaxAliveDuration: time.Duration(cfg.IteratorMaxAliveDuration) * time.Millisecond, - iterFirstSlowDuration: time.Duration(cfg.IteratorSlowReadDuration) * time.Millisecond, - - metricIterRequest: metricIterDuration.WithLabelValues("request"), - metricIterFirst: metricIterDuration.WithLabelValues("first"), - metricIterRelease: metricIterDuration.WithLabelValues("release"), - }, - delete: deleteThrottle{ - countThreshold: cfg.CompactionDeletionThreshold, - period: time.Duration(cfg.CompactionPeriod * int(time.Second)), - }, - - lastSentResolvedTs: startTs, - outputCh: make(chan *model.PolymorphicEvent, sorterOutputCap), - - metricIterReadDuration: metricIterDuration.WithLabelValues("read"), - metricIterNextDuration: metricIterDuration.WithLabelValues("next"), - metricTotalEventsKV: metricOutputKV, - metricTotalEventsResolved: metricOutputResolved, - } - rmb := actor.NewMailbox[message.Task](actorID, sorterInputCap) - err = readerSystem.Spawn(rmb, r) - if err != nil { - return nil, errors.Trace(err) - } - c.closedWg.Add(1) - - return &Sorter{ - common: c, - writerRouter: writerRouter, - writerActorID: actorID, - writer: w, - readerRouter: readerRouter, - ReaderActorID: actorID, - reader: r, - outputCh: r.outputCh, - }, nil -} - -// Run runs Sorter -func (ls *Sorter) Run(ctx context.Context) error { - var err error - select { - case <-ctx.Done(): - err = ctx.Err() - case err = <-ls.errCh: - } - atomic.StoreInt32(&ls.closed, 1) - // We should never have lost message, make sure StopMessage is sent. - ctx1 := context.TODO() - // As the context can't be cancelled. SendB can only return an error - // ActorStopped or ActorNotFound, and they mean actors have closed. - _ = ls.writerRouter.SendB( - ctx1, ls.writerActorID, actormsg.StopMessage[message.Task]()) - _ = ls.readerRouter.SendB( - ctx1, ls.ReaderActorID, actormsg.StopMessage[message.Task]()) - ls.closedWg.Wait() - - _ = ls.cleanup(ctx1) - return errors.Trace(err) -} - -// AddEntry adds an RawKVEntry to the EntryGroup -func (ls *Sorter) AddEntry(ctx context.Context, event *model.PolymorphicEvent) { - if atomic.LoadInt32(&ls.closed) != 0 { - return - } - msg := actormsg.ValueMessage(message.Task{ - UID: ls.uid, - Span: &ls.span, - InputEvent: event, - }) - _ = ls.writerRouter.SendB(ctx, ls.writerActorID, msg) -} - -// Output returns the sorted raw kv output channel -func (ls *Sorter) Output() <-chan *model.PolymorphicEvent { - // Notify reader to read sorted events - msg := actormsg.ValueMessage(message.Task{ - UID: ls.uid, - Span: &ls.span, - ReadTs: message.ReadTs{}, - }) - // It's ok to ignore error, as reader is either channel full or stopped. - // If it's channel full, it has been notified by others, and caller will - // receive new resolved events eventually. - // - // TODO: Consider if we are sending too many msgs here. - // It may waste CPU and be a bottleneck. - _ = ls.readerRouter.Send(ls.ReaderActorID, msg) - return ls.outputCh -} - -// cleanup cleans up sorter's data. -func (ls *Sorter) cleanup(ctx context.Context) error { - task := message.Task{UID: ls.uid, Span: &ls.span} - task.DeleteReq = &message.DeleteRequest{ - // We do not set task.Delete.Count, because we don't know - // how many key-value pairs in the range. - Range: [2][]byte{ - encoding.EncodeTsKey(ls.uid, uint64(ls.span.TableID), 0), - encoding.EncodeTsKey(ls.uid, uint64(ls.span.TableID)+1, 0), - }, - } - return ls.dbRouter.SendB(ctx, ls.dbActorID, actormsg.ValueMessage(task)) -} - -// EmitStartTs implement sorter interface -func (ls *Sorter) EmitStartTs(ctx context.Context, ts uint64) { - msg := actormsg.ValueMessage(message.Task{ - UID: ls.uid, - Span: &ls.span, - StartTs: ts, - }) - _ = ls.readerRouter.SendB(ctx, ls.ReaderActorID, msg) -} - -// Stats implement sorter interface -func (ls *Sorter) Stats() sorter.Stats { - ingress := ls.writer.stats() - egress := ls.reader.stats() - return sorter.Stats{ - CheckpointTsEgress: egress.CheckpointTsEgress, - ResolvedTsEgress: egress.ResolvedTsEgress, - CheckpointTsIngress: ingress.CheckpointTsIngress, - ResolvedTsIngress: ingress.ResolvedTsIngress, - } -} diff --git a/cdc/sorter/db/sorter_test.go b/cdc/sorter/db/sorter_test.go deleted file mode 100644 index 2b26ec9d24e..00000000000 --- a/cdc/sorter/db/sorter_test.go +++ /dev/null @@ -1,128 +0,0 @@ -// Copyright 2022 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package db - -import ( - "context" - "sync" - "testing" - "time" - - "github.com/pingcap/tiflow/cdc/model" - "github.com/pingcap/tiflow/cdc/sorter/db/message" - "github.com/pingcap/tiflow/cdc/sorter/encoding" - "github.com/pingcap/tiflow/pkg/actor" - actormsg "github.com/pingcap/tiflow/pkg/actor/message" - "github.com/pingcap/tiflow/pkg/errors" - "github.com/stretchr/testify/require" -) - -func newTestSorter(name string, capacity int) (Sorter, actor.Mailbox[message.Task]) { - mb := actor.NewMailbox[message.Task](1, capacity) - router := actor.NewRouter[message.Task](name) - router.InsertMailbox4Test(mb.ID(), mb) - - s := Sorter{ - common: common{ - dbRouter: router, - dbActorID: mb.ID(), - errCh: make(chan error, 1), - closedWg: &sync.WaitGroup{}, - }, - writerRouter: router, - writerActorID: mb.ID(), - readerRouter: router, - ReaderActorID: mb.ID(), - } - return s, mb -} - -func TestAddEntry(t *testing.T) { - t.Parallel() - - s, mb := newTestSorter(t.Name(), 1) - - event := model.NewResolvedPolymorphicEvent(0, 1) - s.AddEntry(context.Background(), event) - task, ok := mb.Receive() - require.True(t, ok) - require.EqualValues(t, - message.Task{ - UID: s.uid, - Span: &s.span, - InputEvent: event, - }, task.Value) -} - -func TestOutput(t *testing.T) { - t.Parallel() - - s, mb := newTestSorter(t.Name(), 1) - - s.Output() - task, ok := mb.Receive() - require.True(t, ok) - require.EqualValues(t, - message.Task{ - UID: s.uid, - Span: &s.span, - ReadTs: message.ReadTs{}, - }, task.Value) -} - -func TestRunAndReportError(t *testing.T) { - t.Parallel() - - // Run exits with three messages - cap := 3 - s, mb := newTestSorter(t.Name(), cap) - go func() { - time.Sleep(100 * time.Millisecond) - s.common.reportError( - "test", errors.ErrDBSorterError.GenWithStackByArgs()) - }() - require.Error(t, s.Run(context.Background())) - - // Stop writer and reader. - msg, ok := mb.Receive() - require.True(t, ok) - require.EqualValues(t, actormsg.StopMessage[message.Task](), msg) - msg, ok = mb.Receive() - require.True(t, ok) - require.EqualValues(t, actormsg.StopMessage[message.Task](), msg) - // Cleanup - msg, ok = mb.Receive() - require.True(t, ok) - require.EqualValues(t, - message.Task{ - UID: s.uid, - Span: &s.span, - DeleteReq: &message.DeleteRequest{ - Range: [2][]byte{ - encoding.EncodeTsKey(s.uid, uint64(s.span.TableID), 0), - encoding.EncodeTsKey(s.uid, uint64(s.span.TableID)+1, 0), - }, - }, - }, msg.Value) - - // No more message. - msg, ok = mb.Receive() - require.False(t, ok) - - // Must be nonblock. - s.common.reportError( - "test", errors.ErrDBSorterError.GenWithStackByArgs()) - s.common.reportError( - "test", errors.ErrDBSorterError.GenWithStackByArgs()) -} diff --git a/cdc/sorter/db/system/system.go b/cdc/sorter/db/system/system.go deleted file mode 100644 index 8c2d0c7afa7..00000000000 --- a/cdc/sorter/db/system/system.go +++ /dev/null @@ -1,228 +0,0 @@ -// Copyright 2021 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package system - -import ( - "context" - "encoding/binary" - "hash/fnv" - "sync" - "time" - - "github.com/pingcap/errors" - "github.com/pingcap/log" - "github.com/pingcap/tidb/util/memory" - "github.com/pingcap/tiflow/cdc/processor/tablepb" - dbsorter "github.com/pingcap/tiflow/cdc/sorter/db" - "github.com/pingcap/tiflow/cdc/sorter/db/message" - "github.com/pingcap/tiflow/pkg/actor" - "github.com/pingcap/tiflow/pkg/config" - "github.com/pingcap/tiflow/pkg/db" - cerrors "github.com/pingcap/tiflow/pkg/errors" - "go.uber.org/zap" -) - -// The interval of collecting db metrics. -const defaultMetricInterval = 15 * time.Second - -// State of a system. -type sysState int - -const ( - sysStateInit sysState = iota - sysStateStarted - sysStateStopped -) - -// System manages db sorter resource. -type System struct { - dbs []db.DB - dbSystem *actor.System[message.Task] - DBRouter *actor.Router[message.Task] - WriterSystem *actor.System[message.Task] - WriterRouter *actor.Router[message.Task] - ReaderSystem *actor.System[message.Task] - ReaderRouter *actor.Router[message.Task] - compactSystem *actor.System[message.Task] - compactRouter *actor.Router[message.Task] - compactSched *dbsorter.CompactScheduler - dir string - memPercentage float64 - cfg *config.DBConfig - closedCh chan struct{} - closedWg *sync.WaitGroup - - state sysState - stateMu *sync.Mutex -} - -// NewSystem returns a system. -func NewSystem(dir string, memPercentage float64, cfg *config.DBConfig) *System { - // A system polles actors that read and write db. - dbSystem, dbRouter := actor.NewSystemBuilder[message.Task]("sorter-db"). - WorkerNumber(cfg.Count).Build() - // A system polles actors that compact db, garbage collection. - compactSystem, compactRouter := actor.NewSystemBuilder[message.Task]("sorter-compactor"). - WorkerNumber(cfg.Count).Build() - // A system polles actors that receive events from Puller and batch send - // writes to db. - writerSystem, writerRouter := actor.NewSystemBuilder[message.Task]("sorter-writer"). - WorkerNumber(cfg.Count).Throughput(4, 64).Build() - readerSystem, readerRouter := actor.NewSystemBuilder[message.Task]("sorter-reader"). - WorkerNumber(cfg.Count).Throughput(4, 64).Build() - compactSched := dbsorter.NewCompactScheduler(compactRouter) - return &System{ - dbSystem: dbSystem, - DBRouter: dbRouter, - WriterSystem: writerSystem, - WriterRouter: writerRouter, - ReaderSystem: readerSystem, - ReaderRouter: readerRouter, - compactSystem: compactSystem, - compactRouter: compactRouter, - compactSched: compactSched, - dir: dir, - memPercentage: memPercentage, - cfg: cfg, - closedCh: make(chan struct{}), - closedWg: new(sync.WaitGroup), - state: sysStateInit, - stateMu: new(sync.Mutex), - } -} - -// DBActorID returns an DBActorID correspond with tableID. -func (s *System) DBActorID(span tablepb.Span) actor.ID { - h := fnv.New64() - b := [8]byte{} - binary.LittleEndian.PutUint64(b[:], uint64(span.TableID)) - h.Write(b[:]) - h.Write(span.StartKey) - h.Write(span.EndKey) - return actor.ID(h.Sum64() % uint64(s.cfg.Count)) -} - -// CompactScheduler returns compaction scheduler. -func (s *System) CompactScheduler() *dbsorter.CompactScheduler { - return s.compactSched -} - -// Start starts a system. -func (s *System) Start(ctx context.Context) error { - s.stateMu.Lock() - defer s.stateMu.Unlock() - if s.state == sysStateStarted { - // Already started. - return nil - } else if s.state == sysStateStopped { - return cerrors.ErrStartAStoppedDBSystem.GenWithStackByArgs() - } - s.state = sysStateStarted - - s.compactSystem.Start(ctx) - s.dbSystem.Start(ctx) - s.WriterSystem.Start(ctx) - s.ReaderSystem.Start(ctx) - totalMemory, err := memory.MemTotal() - if err != nil { - return errors.Trace(err) - } - memInBytePerDB := float64(totalMemory) * s.memPercentage / float64(s.cfg.Count) - for id := 0; id < s.cfg.Count; id++ { - // Open db. - db, err := db.OpenPebble( - ctx, id, s.dir, s.cfg, db.WithCache(int(memInBytePerDB)), db.WithTableCRTsCollectors()) - if err != nil { - return errors.Trace(err) - } - s.dbs = append(s.dbs, db) - // Create and spawn compactor actor. - compactor, cmb, err := dbsorter.NewCompactActor(id, db, s.closedWg, s.cfg) - if err != nil { - return errors.Trace(err) - } - err = s.compactSystem.Spawn(cmb, compactor) - if err != nil { - return errors.Trace(err) - } - // Create and spawn db actor. - dbac, dbmb, err := dbsorter.NewDBActor(id, db, s.cfg, s.compactSched, s.closedWg) - if err != nil { - return errors.Trace(err) - } - err = s.dbSystem.Spawn(dbmb, dbac) - if err != nil { - return errors.Trace(err) - } - } - s.closedWg.Add(1) - go func() { - defer s.closedWg.Done() - metricsTimer := time.NewTimer(defaultMetricInterval) - defer metricsTimer.Stop() - for { - select { - case <-ctx.Done(): - return - case <-s.closedCh: - return - case <-metricsTimer.C: - collectMetrics(s.dbs) - metricsTimer.Reset(defaultMetricInterval) - } - } - }() - return nil -} - -// Stop stops a system. -func (s *System) Stop() { - s.stateMu.Lock() - defer s.stateMu.Unlock() - switch s.state { - case sysStateStopped: - // Already stopped. - return - case sysStateInit: - // Not started. - return - } - s.state = sysStateStopped - - // Stop all actors and system to release resource. - s.WriterSystem.Stop() - s.ReaderSystem.Stop() - // TODO: compact is not context-aware, it may block. - s.compactSystem.Stop() - s.dbSystem.Stop() - // Close metrics goroutine. - close(s.closedCh) - - // Close dbs. - for i, db := range s.dbs { - err := db.Close() - if err != nil { - log.Warn("db close error", zap.Int("ID", i), zap.Error(err)) - } - } - // Wait actors and metrics goroutine. - s.closedWg.Wait() -} - -func collectMetrics(dbs []db.DB) { - for i := range dbs { - db := dbs[i] - db.CollectMetrics(i) - } -} diff --git a/cdc/sorter/db/system/system_test.go b/cdc/sorter/db/system/system_test.go deleted file mode 100644 index cf3d926618e..00000000000 --- a/cdc/sorter/db/system/system_test.go +++ /dev/null @@ -1,168 +0,0 @@ -// Copyright 2021 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package system - -import ( - "context" - "testing" - "time" - - "github.com/pingcap/tiflow/cdc/model" - "github.com/pingcap/tiflow/cdc/sorter/db" - "github.com/pingcap/tiflow/cdc/sorter/db/message" - "github.com/pingcap/tiflow/pkg/actor" - actormsg "github.com/pingcap/tiflow/pkg/actor/message" - "github.com/pingcap/tiflow/pkg/config" - "github.com/pingcap/tiflow/pkg/spanz" - "github.com/stretchr/testify/require" -) - -func TestSystemStartStop(t *testing.T) { - t.Parallel() - ctx := context.Background() - cfg := config.GetDefaultServerConfig().Clone().Debug.DB - cfg.Count = 1 - - sys := NewSystem(t.TempDir(), 1, cfg) - require.Nil(t, sys.Start(ctx)) - sys.Stop() - - // Start a closed system. - require.Error(t, sys.Start(ctx)) -} - -func TestSystemStopUnstarted(t *testing.T) { - t.Parallel() - cfg := config.GetDefaultServerConfig().Clone().Debug.DB - cfg.Count = 1 - - sys := NewSystem(t.TempDir(), 1, cfg) - sys.Stop() -} - -func TestCollectMetrics(t *testing.T) { - t.Parallel() - ctx := context.Background() - cfg := config.GetDefaultServerConfig().Clone().Debug.DB - cfg.Count = 2 - - sys := NewSystem(t.TempDir(), 1, cfg) - require.Nil(t, sys.Start(ctx)) - collectMetrics(sys.dbs) - sys.Stop() -} - -func TestDBActorID(t *testing.T) { - t.Parallel() - ctx := context.Background() - cfg := config.GetDefaultServerConfig().Clone().Debug.DB - cfg.Count = 2 - - sys := NewSystem(t.TempDir(), 1, cfg) - require.Nil(t, sys.Start(ctx)) - id1 := sys.DBActorID(spanz.TableIDToComparableSpan(1)) - id2 := sys.DBActorID(spanz.TableIDToComparableSpan(1)) - // tableID to actor ID must be deterministic. - require.Equal(t, id1, id2) - sys.Stop() -} - -// Slow actor should not block system.Stop() forever. -func TestSystemStopSlowly(t *testing.T) { - t.Parallel() - ctx := context.Background() - cfg := config.GetDefaultServerConfig().Clone().Debug.DB - cfg.Count = 2 - - sys := NewSystem(t.TempDir(), 1, cfg) - require.Nil(t, sys.Start(ctx)) - msg := message.Task{Test: &message.Test{Sleep: 2 * time.Second}} - sys.DBRouter.Broadcast(ctx, actormsg.ValueMessage(msg)) - sys.Stop() -} - -// Mailbox full should not cause system.Stop() being blocked forever. -func TestSystemStopMailboxFull(t *testing.T) { - t.Parallel() - ctx := context.Background() - cfg := config.GetDefaultServerConfig().Clone().Debug.DB - cfg.Count = 2 - - sys := NewSystem(t.TempDir(), 1, cfg) - require.Nil(t, sys.Start(ctx)) - msg := message.Task{Test: &message.Test{Sleep: 2 * time.Second}} - sys.DBRouter.Broadcast(ctx, actormsg.ValueMessage(msg)) - for { - err := sys.DBRouter.Send(actor.ID(1), actormsg.ValueMessage(message.Task{})) - if err != nil { - break - } - } - sys.Stop() -} - -func TestSystemStopWithManyTablesAndFewStragglers(t *testing.T) { - t.Parallel() - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - cfg := config.GetDefaultServerConfig().Clone().Debug.DB - cfg.Count = 8 - - sys := NewSystem(t.TempDir(), 1, cfg) - require.Nil(t, sys.Start(ctx)) - - ss := make([]*db.Sorter, 0, 1000) - scancels := make([]context.CancelFunc, 0, 1000) - for i := uint64(0); i < 1000; i++ { - dbActorID := sys.DBActorID(spanz.TableIDToComparableSpan(int64(i))) - s, err := db.NewSorter( - ctx, model.ChangeFeedID4Test("test", "test"), - spanz.TableIDToComparableSpan(int64(i)), - i, sys.DBRouter, dbActorID, - sys.WriterSystem, sys.WriterRouter, - sys.ReaderSystem, sys.ReaderRouter, - sys.CompactScheduler(), cfg) - require.Nil(t, err) - ss = append(ss, s) - sctx, scancel := context.WithCancel(ctx) - scancels = append(scancels, scancel) - go func() { - _ = s.Run(sctx) - }() - } - time.Sleep(500 * time.Millisecond) - - // Close 100 tables ahead. - for i := 0; i < 100; i++ { - scancels[i]() - } - // 10 stragglers - for i := 100; i < 110; i++ { - id := ss[i].ReaderActorID - sleep := message.Task{Test: &message.Test{Sleep: 2 * time.Second}} - require.Nil(t, sys.ReaderRouter.SendB(ctx, id, actormsg.ValueMessage(sleep))) - if i%2 == 0 { - continue - } - // Make it channel full. - for { - err := sys.ReaderRouter.Send(id, actormsg.ValueMessage(message.Task{})) - if err != nil { - break - } - } - } - // Close system. - sys.Stop() -} diff --git a/cdc/sorter/db/writer.go b/cdc/sorter/db/writer.go deleted file mode 100644 index 1a31c572ec8..00000000000 --- a/cdc/sorter/db/writer.go +++ /dev/null @@ -1,159 +0,0 @@ -// Copyright 2022 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package db - -import ( - "context" - "sync/atomic" - - "github.com/pingcap/log" - "github.com/pingcap/tiflow/cdc/sorter" - "github.com/pingcap/tiflow/cdc/sorter/db/message" - "github.com/pingcap/tiflow/cdc/sorter/encoding" - "github.com/pingcap/tiflow/pkg/actor" - actormsg "github.com/pingcap/tiflow/pkg/actor/message" - "github.com/prometheus/client_golang/prometheus" - "go.uber.org/zap" -) - -// writer is a thin shim that batches, translates events into key-value pairs -// and writes to db. -type writer struct { - common - stopped bool - - readerRouter *actor.Router[message.Task] - readerActorID actor.ID - - maxResolvedTs uint64 - maxCommitTs uint64 - - metricTotalEventsKV prometheus.Counter - metricTotalEventsResolved prometheus.Counter -} - -var _ actor.Actor[message.Task] = (*writer)(nil) - -func (w *writer) Poll(ctx context.Context, msgs []actormsg.Message[message.Task]) (running bool) { - kvEventCount, resolvedEventCount := 0, 0 - maxCommitTs, maxResolvedTs := uint64(0), uint64(0) - writes := make(map[message.Key][]byte) - for i := range msgs { - switch msgs[i].Tp { - case actormsg.TypeValue: - case actormsg.TypeStop: - return false - default: - log.Panic("unexpected message", zap.Any("message", msgs[i])) - } - - ev := msgs[i].Value.InputEvent - if ev.IsResolved() { - if maxResolvedTs < ev.CRTs { - maxResolvedTs = ev.CRTs - } - resolvedEventCount++ - continue - } - if maxCommitTs < ev.CRTs { - maxCommitTs = ev.CRTs - } - kvEventCount++ - - key := encoding.EncodeKey(w.uid, uint64(w.span.TableID), ev) - value := []byte{} - var err error - value, err = w.serde.Marshal(ev, value) - if err != nil { - log.Panic("failed to marshal events", zap.Error(err)) - } - writes[message.Key(key)] = value - } - w.metricTotalEventsKV.Add(float64(kvEventCount)) - w.metricTotalEventsResolved.Add(float64(resolvedEventCount)) - if atomic.LoadUint64(&w.maxCommitTs) < maxCommitTs { - atomic.StoreUint64(&w.maxCommitTs, maxCommitTs) - } - if atomic.LoadUint64(&w.maxResolvedTs) < maxResolvedTs { - atomic.StoreUint64(&w.maxResolvedTs, maxResolvedTs) - } - - if len(writes) != 0 { - // Send write task to db. - task := message.Task{UID: w.uid, Span: &w.span, WriteReq: writes} - err := w.dbRouter.SendB(ctx, w.dbActorID, actormsg.ValueMessage(task)) - if err != nil { - w.reportError("failed to send write request", err) - return false - } - } - - if w.maxResolvedTs == 0 { - // Resolved ts has not advanced yet, skip notify reader. - return true - } - // Notify reader that there is something to read. - // - // It's ok to notify reader immediately without waiting writes done, - // because reader will see these writes: - // 1. reader/writer send tasks to the same db, so tasks are ordered. - // 2. ReadTs will trigger reader to take iterator from db, - // it happens after writer send writes to db. - // 3. Before db takes iterator, it flushes all buffered writes. - msg := actormsg.ValueMessage(message.Task{ - UID: w.uid, - Span: &w.span, - ReadTs: message.ReadTs{ - // The maxCommitTs and maxResolvedTs must be sent together, - // otherwise reader may output resolved ts wrongly. - // As reader employs maxCommitTs and maxResolvedTs to skip taking - // iterators when maxResolvedTs > maxCommitTs and - // exhaustedResolvedTs >= maxCommitTs. - // - // If maxCommitTs and maxResolvedTs are sent separately, - // data in (exhaustedResolvedTs, actual maxCommitTs] is lost: - // ---------------------------------------------> - // writer: ^ actual maxCommitTs - // reader: ^ maxCommitTs ^ exhaustedResolvedTs ^ maxResolvedTs - MaxCommitTs: atomic.LoadUint64(&w.maxCommitTs), - MaxResolvedTs: atomic.LoadUint64(&w.maxResolvedTs), - }, - }) - // It's ok if send fails, as resolved ts events are received periodically. - _ = w.readerRouter.Send(w.readerActorID, msg) - return true -} - -// OnClose releases writer resource. -func (w *writer) OnClose() { - if w.stopped { - return - } - w.stopped = true - w.common.closedWg.Done() -} - -func (w *writer) stats() sorter.Stats { - maxCommitTs := atomic.LoadUint64(&w.maxCommitTs) - maxResolvedTs := atomic.LoadUint64(&w.maxResolvedTs) - if maxCommitTs < maxResolvedTs { - // In case, there is no write for the table, - // we use maxResolvedTs as maxCommitTs to make the stats meaningful. - maxCommitTs = maxResolvedTs - } - return sorter.Stats{ - CheckpointTsIngress: maxCommitTs, - ResolvedTsIngress: maxResolvedTs, - } -} diff --git a/cdc/sorter/db/writer_test.go b/cdc/sorter/db/writer_test.go deleted file mode 100644 index fbe3b36c1f2..00000000000 --- a/cdc/sorter/db/writer_test.go +++ /dev/null @@ -1,205 +0,0 @@ -// Copyright 2021 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package db - -import ( - "context" - "testing" - - "github.com/pingcap/tiflow/cdc/model" - "github.com/pingcap/tiflow/cdc/sorter" - "github.com/pingcap/tiflow/cdc/sorter/db/message" - "github.com/pingcap/tiflow/cdc/sorter/encoding" - "github.com/pingcap/tiflow/pkg/actor" - actormsg "github.com/pingcap/tiflow/pkg/actor/message" - "github.com/stretchr/testify/require" -) - -func newTestWriter( - c common, readerRouter *actor.Router[message.Task], readerActorID actor.ID, -) *writer { - return &writer{ - common: c, - readerRouter: readerRouter, - readerActorID: readerActorID, - - metricTotalEventsKV: sorter.OutputEventCount. - WithLabelValues("default", "test", "kv"), - metricTotalEventsResolved: sorter.OutputEventCount. - WithLabelValues("default", "test", "resolved"), - } -} - -func newTestEvent(crts, startTs uint64, key int) *model.PolymorphicEvent { - return model.NewPolymorphicEvent(&model.RawKVEntry{ - OpType: model.OpTypePut, - Key: []byte{byte(key)}, - StartTs: startTs, - CRTs: crts, - }) -} - -func receiveOutputEvents( - outputCh chan *model.PolymorphicEvent, -) []*model.PolymorphicEvent { - outputEvents := []*model.PolymorphicEvent{} -RECV: - for { - select { - case ev := <-outputCh: - outputEvents = append(outputEvents, ev) - default: - break RECV - } - } - return outputEvents -} - -func TestWriterPoll(t *testing.T) { - t.Parallel() - - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - - capacity := 4 - router := actor.NewRouter[message.Task](t.Name()) - readerID := actor.ID(1) - readerMB := actor.NewMailbox[message.Task](readerID, capacity) - router.InsertMailbox4Test(readerID, readerMB) - dbID := actor.ID(2) - dbMB := actor.NewMailbox[message.Task](dbID, capacity) - router.InsertMailbox4Test(dbID, dbMB) - c := common{dbActorID: dbID, dbRouter: router} - writer := newTestWriter(c, router, readerID) - - // We need to poll twice to read resolved events, so we need a slice of - // two cases. - cases := []struct { - inputEvents []*model.PolymorphicEvent - - expectWrites [][]byte - expectMaxCommitTs uint64 - expectMaxResolvedTs uint64 - }{{ - // Only resoved ts events. - inputEvents: []*model.PolymorphicEvent{ - model.NewResolvedPolymorphicEvent(0, 1), - model.NewResolvedPolymorphicEvent(0, 2), - }, - - expectWrites: [][]byte{}, - expectMaxCommitTs: 0, - expectMaxResolvedTs: 2, - }, { - // Only rawkv events. - inputEvents: []*model.PolymorphicEvent{ - newTestEvent(3, 1, 0), // crts 3, startts 2, key 0 - newTestEvent(3, 1, 1), // crts 3, startts 2, key 1 - }, - - expectWrites: [][]byte{ - encoding.EncodeKey(c.uid, uint64(c.span.TableID), newTestEvent(3, 1, 0)), - encoding.EncodeKey(c.uid, uint64(c.span.TableID), newTestEvent(3, 1, 1)), - }, - expectMaxCommitTs: 3, - expectMaxResolvedTs: 2, - }, { - // Mix rawkv events and resolved ts events. - inputEvents: []*model.PolymorphicEvent{ - newTestEvent(4, 2, 0), // crts 4, startts 2 - model.NewResolvedPolymorphicEvent(0, 3), - newTestEvent(6, 3, 0), // crts 6, startts 3 - model.NewResolvedPolymorphicEvent(0, 3), - }, - - expectWrites: [][]byte{ - encoding.EncodeKey(c.uid, uint64(c.span.TableID), newTestEvent(4, 2, 0)), - encoding.EncodeKey(c.uid, uint64(c.span.TableID), newTestEvent(6, 3, 0)), - }, - expectMaxCommitTs: 6, - expectMaxResolvedTs: 3, - }, { - // Duplicate commit events. - inputEvents: []*model.PolymorphicEvent{ - newTestEvent(6, 3, 0), // crts 6, startts 3 - newTestEvent(6, 3, 0), // crts 6, startts 3 - }, - - expectWrites: [][]byte{ - encoding.EncodeKey(c.uid, uint64(c.span.TableID), newTestEvent(6, 3, 0)), - }, - expectMaxCommitTs: 6, - expectMaxResolvedTs: 3, - }, { - // Commit ts regress and bounce. - inputEvents: []*model.PolymorphicEvent{ - newTestEvent(4, 3, 0), // crts 4, startts 3 - newTestEvent(5, 3, 0), // crts 5, startts 3 - newTestEvent(4, 2, 0), // crts 4, startts 3 - }, - - expectWrites: [][]byte{ - encoding.EncodeKey(c.uid, uint64(c.span.TableID), newTestEvent(4, 3, 0)), - encoding.EncodeKey(c.uid, uint64(c.span.TableID), newTestEvent(5, 3, 0)), - encoding.EncodeKey(c.uid, uint64(c.span.TableID), newTestEvent(4, 2, 0)), - }, - expectMaxCommitTs: 6, - expectMaxResolvedTs: 3, - }, { - // Resolved ts regress. It should not happen, but we test it anyway. - inputEvents: []*model.PolymorphicEvent{ - model.NewResolvedPolymorphicEvent(0, 2), - }, - - expectWrites: [][]byte{}, - expectMaxCommitTs: 6, - expectMaxResolvedTs: 3, - }} - - for i, cs := range cases { - msgs := make([]actormsg.Message[message.Task], 0, len(cs.inputEvents)) - for i := range cs.inputEvents { - msgs = append(msgs, actormsg.ValueMessage(message.Task{ - InputEvent: cs.inputEvents[i], - })) - } - t.Logf("test case #%d, %v", i, cs) - require.True(t, writer.Poll(ctx, msgs), "case #%d, %v", i, cs) - if len(cs.expectWrites) != 0 { - msg, ok := dbMB.Receive() - require.True(t, ok, "case #%d, %v", i, cs) - writeReq := msg.Value.WriteReq - require.EqualValues(t, len(cs.expectWrites), len(writeReq)) - for _, k := range cs.expectWrites { - _, ok := writeReq[message.Key(k)] - require.True(t, ok, "case #%d, %v, %v, %v", i, cs, writeReq) - } - } else { - _, ok := dbMB.Receive() - require.False(t, ok, "case #%d, %v", i, cs) - } - msg, ok := readerMB.Receive() - require.True(t, ok, "case #%d, %v", i, cs) - require.EqualValues(t, - cs.expectMaxCommitTs, msg.Value.ReadTs.MaxCommitTs, - "case #%d, %v", i, cs) - require.EqualValues(t, - cs.expectMaxResolvedTs, msg.Value.ReadTs.MaxResolvedTs, - "case #%d, %v", i, cs) - } - - // writer should stop once it receives Stop message. - msg := actormsg.StopMessage[message.Task]() - require.False(t, writer.Poll(ctx, []actormsg.Message[message.Task]{msg})) -} diff --git a/cdc/sorter/metrics.go b/cdc/sorter/metrics.go deleted file mode 100644 index dd200d15865..00000000000 --- a/cdc/sorter/metrics.go +++ /dev/null @@ -1,78 +0,0 @@ -// Copyright 2021 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package sorter - -import ( - "github.com/prometheus/client_golang/prometheus" -) - -var ( - // InputEventCount is the metric that counts events input to sorter. - InputEventCount = prometheus.NewCounterVec(prometheus.CounterOpts{ - Namespace: "ticdc", - Subsystem: "sorter", - Name: "input_event_count", - Help: "The number of events input to sorter", - }, []string{"namespace", "changefeed", "type"}) - - // OutputEventCount is the metric that counts events output by the sorter. - OutputEventCount = prometheus.NewCounterVec(prometheus.CounterOpts{ - Namespace: "ticdc", - Subsystem: "sorter", - Name: "output_event_count", - Help: "The number of events output by the sorter", - }, []string{"namespace", "changefeed", "type"}) - - // ResolvedTsGauge is the metric that records sorter resolved ts. - ResolvedTsGauge = prometheus.NewGaugeVec(prometheus.GaugeOpts{ - Namespace: "ticdc", - Subsystem: "sorter", - Name: "resolved_ts_gauge", - Help: "the resolved ts of the sorter", - }, []string{"namespace", "changefeed"}) - - // InMemoryDataSizeGauge is the metric that records sorter memory usage. - InMemoryDataSizeGauge = prometheus.NewGaugeVec(prometheus.GaugeOpts{ - Namespace: "ticdc", - Subsystem: "sorter", - Name: "in_memory_data_size_gauge", - Help: "The amount of pending data stored in-memory by the sorter", - }, []string{"id"}) - - // OnDiskDataSizeGauge is the metric that records sorter disk usage. - OnDiskDataSizeGauge = prometheus.NewGaugeVec(prometheus.GaugeOpts{ - Namespace: "ticdc", - Subsystem: "sorter", - Name: "on_disk_data_size_gauge", - Help: "The amount of pending data stored on-disk by the sorter", - }, []string{"id"}) - - // OpenFileCountGauge is the metric that records sorter open files. - OpenFileCountGauge = prometheus.NewGaugeVec(prometheus.GaugeOpts{ - Namespace: "ticdc", - Subsystem: "sorter", - Name: "open_file_count_gauge", - Help: "The number of open file descriptors held by the sorter", - }, []string{"id"}) -) - -// InitMetrics registers all metrics in this file -func InitMetrics(registry *prometheus.Registry) { - registry.MustRegister(InputEventCount) - registry.MustRegister(OutputEventCount) - registry.MustRegister(ResolvedTsGauge) - registry.MustRegister(InMemoryDataSizeGauge) - registry.MustRegister(OnDiskDataSizeGauge) - registry.MustRegister(OpenFileCountGauge) -} diff --git a/cdc/sorter/sorter.go b/cdc/sorter/sorter.go deleted file mode 100644 index 3fb88656759..00000000000 --- a/cdc/sorter/sorter.go +++ /dev/null @@ -1,58 +0,0 @@ -// Copyright 2020 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 sorter - -import ( - "context" - - "github.com/pingcap/tiflow/cdc/model" -) - -// Stats of a sorter. -type Stats struct { - CheckpointTsIngress model.Ts - ResolvedTsIngress model.Ts - CheckpointTsEgress model.Ts - ResolvedTsEgress model.Ts -} - -// EventSorter accepts unsorted PolymorphicEvents, sort them in background and returns -// sorted PolymorphicEvents in Output channel -type EventSorter interface { - Run(ctx context.Context) error - // TODO add constraints to entries, e.g., order and duplication guarantees. - AddEntry(ctx context.Context, entry *model.PolymorphicEvent) - // Output sorted events, ordered by commit ts. - // - // Callers must not cache the returned channel, as sorter may not output - // any resolved events if callers skip calling `Output`. - // - // func caller(ctx context.Context, sorter EventSorter) { - // for { - // output := sorter.Output() - // select { - // case <-ctx.Done(): - // return - // case ev := <-output: - // // Do something with ev. - // } - // } - // } - Output() <-chan *model.PolymorphicEvent - - // EmitStartTs let sorter know the start timestamp for consuming data - EmitStartTs(ctx context.Context, ts uint64) - - Stats() Stats -} diff --git a/cdc/sorter/unified/backend.go b/cdc/sorter/unified/backend.go deleted file mode 100644 index 6f546bfcfab..00000000000 --- a/cdc/sorter/unified/backend.go +++ /dev/null @@ -1,34 +0,0 @@ -// Copyright 2020 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 unified - -import "github.com/pingcap/tiflow/cdc/model" - -type backEnd interface { - reader() (backEndReader, error) - writer() (backEndWriter, error) - free() error -} - -type backEndReader interface { - readNext() (*model.PolymorphicEvent, error) - resetAndClose() error -} - -type backEndWriter interface { - writeNext(event *model.PolymorphicEvent) error - writtenCount() int - dataSize() uint64 - flushAndClose() error -} diff --git a/cdc/sorter/unified/backend_pool.go b/cdc/sorter/unified/backend_pool.go deleted file mode 100644 index 66b177a9e9f..00000000000 --- a/cdc/sorter/unified/backend_pool.go +++ /dev/null @@ -1,405 +0,0 @@ -// Copyright 2020 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 unified - -import ( - "context" - "fmt" - "os" - "path/filepath" - "reflect" - "sync" - "sync/atomic" - "time" - "unsafe" - - "github.com/pingcap/errors" - "github.com/pingcap/failpoint" - "github.com/pingcap/log" - "github.com/pingcap/tidb/util/memory" - "github.com/pingcap/tiflow/cdc/contextutil" - "github.com/pingcap/tiflow/cdc/sorter" - sorterencoding "github.com/pingcap/tiflow/cdc/sorter/encoding" - "github.com/pingcap/tiflow/pkg/config" - cerrors "github.com/pingcap/tiflow/pkg/errors" - "github.com/pingcap/tiflow/pkg/fsutil" - "go.uber.org/zap" -) - -const ( - backgroundJobInterval = time.Second * 15 - sortDirLockFileName = "ticdc_lock" - sortDirDataFileMagicPrefix = "sort" -) - -var ( - pool *backEndPool // this is the singleton instance of backEndPool - poolMu sync.Mutex // this mutex is for delayed initialization of `pool` only -) - -type backEndPool struct { - memoryUseEstimate int64 - onDiskDataSize int64 - fileNameCounter uint64 - memPressure int32 - cache [256]unsafe.Pointer - dir string - filePrefix string - - // to prevent `dir` from being accidentally used by another TiCDC server process. - fileLock *fsutil.FileLock - - // cancelCh needs to be unbuffered to prevent races - cancelCh chan struct{} - // cancelRWLock protects cache against races when the backEnd is exiting - cancelRWLock sync.RWMutex - isTerminating bool -} - -func newBackEndPool(dir string) (*backEndPool, error) { - ret := &backEndPool{ - memoryUseEstimate: 0, - fileNameCounter: 0, - dir: dir, - cancelCh: make(chan struct{}), - filePrefix: fmt.Sprintf("%s/%s-%d-", dir, sortDirDataFileMagicPrefix, os.Getpid()), - } - - err := ret.lockSortDir() - if err != nil { - log.Warn("failed to lock file prefix", - zap.String("prefix", ret.filePrefix), - zap.Error(err)) - return nil, errors.Trace(err) - } - - err = ret.cleanUpStaleFiles() - if err != nil { - log.Warn("Unified Sorter: failed to clean up stale temporary files. Report a bug if you believe this is unexpected", zap.Error(err)) - return nil, errors.Trace(err) - } - - go func() { - ticker := time.NewTicker(backgroundJobInterval) - defer ticker.Stop() - - id := "0" // A placeholder for ID label in metrics. - metricSorterInMemoryDataSizeGauge := sorter.InMemoryDataSizeGauge.WithLabelValues(id) - metricSorterOnDiskDataSizeGauge := sorter.OnDiskDataSizeGauge.WithLabelValues(id) - metricSorterOpenFileCountGauge := sorter.OpenFileCountGauge.WithLabelValues(id) - - // TODO: The underlaying implementation only recognizes cgroups set by - // containers, we need to support cgroups set by systemd or manually. - // See https://github.com/pingcap/tidb/issues/22132 - totalMemory, err := memory.MemTotal() - if err != nil { - log.Panic("read memory stat failed", zap.Error(err)) - } - for { - select { - case <-ret.cancelCh: - log.Info("Unified Sorter backEnd is being cancelled") - return - case <-ticker.C: - } - - metricSorterInMemoryDataSizeGauge.Set(float64(atomic.LoadInt64(&ret.memoryUseEstimate))) - metricSorterOnDiskDataSizeGauge.Set(float64(atomic.LoadInt64(&ret.onDiskDataSize))) - metricSorterOpenFileCountGauge.Set(float64(atomic.LoadInt64(&openFDCount))) - - // update memPressure - usedMemory, err := memory.MemUsed() - if err != nil || totalMemory == 0 { - failpoint.Inject("sorterDebug", func() { - log.Panic("unified sorter: getting system memory usage failed", zap.Error(err)) - }) - - log.Warn("unified sorter: getting system memory usage failed", zap.Error(err)) - // Reports a 100% memory pressure, so that the backEndPool will allocate fileBackEnds. - // We default to fileBackEnds because they are unlikely to cause OOMs. If IO errors are - // encountered, we can fail gracefully. - atomic.StoreInt32(&ret.memPressure, 100) - } else { - memPressure := usedMemory * 100 / totalMemory - atomic.StoreInt32(&ret.memPressure, int32(memPressure)) - } - - // garbage collect temporary files in batches - freedCount := 0 - for i := range ret.cache { - ptr := &ret.cache[i] - innerPtr := atomic.SwapPointer(ptr, nil) - if innerPtr == nil { - continue - } - backEnd := (*fileBackEnd)(innerPtr) - err := backEnd.free() - if err != nil { - log.Warn("Cannot remove temporary file for sorting", zap.String("file", backEnd.fileName), zap.Error(err)) - } else { - log.Debug("Temporary file removed", zap.String("file", backEnd.fileName)) - freedCount += 1 - } - if freedCount >= 16 { - freedCount = 0 - break - } - } - } - }() - - return ret, nil -} - -func (p *backEndPool) alloc(ctx context.Context) (backEnd, error) { - sorterConfig := config.GetGlobalServerConfig().Sorter - if p.sorterMemoryUsage() < int64(sorterConfig.MaxMemoryConsumption) && - p.memoryPressure() < int32(sorterConfig.MaxMemoryPercentage) { - - ret := newMemoryBackEnd() - return ret, nil - } - - p.cancelRWLock.RLock() - defer p.cancelRWLock.RUnlock() - - if p.isTerminating { - return nil, cerrors.ErrUnifiedSorterBackendTerminating.GenWithStackByArgs() - } - - for i := range p.cache { - ptr := &p.cache[i] - ret := atomic.SwapPointer(ptr, nil) - if ret != nil { - return (*fileBackEnd)(ret), nil - } - } - - fname := fmt.Sprintf("%s%d.tmp", p.filePrefix, atomic.AddUint64(&p.fileNameCounter, 1)) - tableID, tableName := contextutil.TableIDFromCtx(ctx) - log.Debug("Unified Sorter: trying to create file backEnd", - zap.String("filename", fname), - zap.Int64("tableID", tableID), - zap.String("tableName", tableName)) - - if err := checkDataDirSatisfied(); err != nil { - return nil, errors.Trace(err) - } - - ret, err := newFileBackEnd(fname, &sorterencoding.MsgPackGenSerde{}) - if err != nil { - return nil, errors.Trace(err) - } - - return ret, nil -} - -func (p *backEndPool) dealloc(backEnd backEnd) error { - switch b := backEnd.(type) { - case *memoryBackEnd: - err := b.free() - if err != nil { - log.Warn("error freeing memory backend", zap.Error(err)) - } - // Let GC do its job - return nil - case *fileBackEnd: - failpoint.Inject("sorterDebug", func() { - if atomic.LoadInt32(&b.borrowed) != 0 { - log.Warn("Deallocating a fileBackEnd in use", zap.String("filename", b.fileName)) - failpoint.Return(nil) - } - }) - - b.cleanStats() - - p.cancelRWLock.RLock() - defer p.cancelRWLock.RUnlock() - - if p.isTerminating { - return cerrors.ErrUnifiedSorterBackendTerminating.GenWithStackByArgs() - } - - for i := range p.cache { - ptr := &p.cache[i] - if atomic.CompareAndSwapPointer(ptr, nil, unsafe.Pointer(b)) { - return nil - } - } - // Cache is full. - err := b.free() - if err != nil { - return errors.Trace(err) - } - - return nil - default: - log.Panic("backEndPool: unexpected backEnd type to be deallocated", zap.Reflect("type", reflect.TypeOf(backEnd))) - } - return nil -} - -func (p *backEndPool) terminate() { - defer func() { - if p.fileLock == nil { - return - } - err := p.unlockSortDir() - if err != nil { - log.Warn("failed to unlock file prefix", zap.String("prefix", p.filePrefix)) - } - }() - - p.cancelCh <- struct{}{} - defer close(p.cancelCh) - // the background goroutine can be considered terminated here - - log.Debug("Unified Sorter terminating...") - p.cancelRWLock.Lock() - defer p.cancelRWLock.Unlock() - p.isTerminating = true - - log.Debug("Unified Sorter cleaning up before exiting") - // any new allocs and deallocs will not succeed from this point - // accessing p.cache without atomics is safe from now - - for i := range p.cache { - ptr := &p.cache[i] - backend := (*fileBackEnd)(*ptr) - if backend == nil { - continue - } - _ = backend.free() - } - - if p.filePrefix == "" { - // This should not happen. But to prevent accidents in production, we add this anyway. - log.Panic("Empty filePrefix, please report a bug") - } - - files, err := filepath.Glob(p.filePrefix + "*") - if err != nil { - log.Warn("Unified Sorter clean-up failed", zap.Error(err)) - } - for _, file := range files { - log.Debug("Unified Sorter backEnd removing file", zap.String("file", file)) - err = os.RemoveAll(file) - if err != nil { - log.Warn("Unified Sorter clean-up failed: failed to remove", - zap.String("fileName", file), zap.Error(err)) - } - } - - log.Debug("Unified Sorter backEnd terminated") -} - -func (p *backEndPool) sorterMemoryUsage() int64 { - failpoint.Inject("memoryUsageInjectPoint", func(val failpoint.Value) { - failpoint.Return(int64(val.(int))) - }) - return atomic.LoadInt64(&p.memoryUseEstimate) -} - -func (p *backEndPool) memoryPressure() int32 { - failpoint.Inject("memoryPressureInjectPoint", func(val failpoint.Value) { - failpoint.Return(int32(val.(int))) - }) - return atomic.LoadInt32(&p.memPressure) -} - -func (p *backEndPool) lockSortDir() error { - lockFileName := fmt.Sprintf("%s/%s", p.dir, sortDirLockFileName) - fileLock, err := fsutil.NewFileLock(lockFileName) - if err != nil { - return cerrors.ErrSortDirLockError.Wrap(err).GenWithStackByCause() - } - - err = fileLock.Lock() - if err != nil { - if cerrors.ErrConflictingFileLocks.Equal(err) { - log.Warn("TiCDC failed to lock sorter temporary file directory. "+ - "Make sure that another instance of TiCDC, or any other program, is not using the directory. "+ - "If you believe you should not see this error, try deleting the lock file and resume the changefeed. "+ - "Report a bug or contact support if the problem persists.", - zap.String("lockFile", lockFileName)) - return errors.Trace(err) - } - return cerrors.ErrSortDirLockError.Wrap(err).GenWithStackByCause() - } - - p.fileLock = fileLock - return nil -} - -func (p *backEndPool) unlockSortDir() error { - err := p.fileLock.Unlock() - if err != nil { - return cerrors.ErrSortDirLockError.Wrap(err).FastGenWithCause() - } - return nil -} - -func (p *backEndPool) cleanUpStaleFiles() error { - if p.dir == "" { - // guard against programmer error. Must be careful when we are deleting user files. - log.Panic("unexpected sort-dir", zap.String("sortDir", p.dir)) - } - - files, err := filepath.Glob(filepath.Join(p.dir, fmt.Sprintf("%s-*", sortDirDataFileMagicPrefix))) - if err != nil { - return errors.Trace(err) - } - - for _, toRemoveFilePath := range files { - log.Debug("Removing stale sorter temporary file", zap.String("file", toRemoveFilePath)) - err := os.Remove(toRemoveFilePath) - if err != nil { - // In production, we do not want an error here to interfere with normal operation, - // because in most situations, failure to remove files only indicates non-fatal misconfigurations - // such as permission problems, rather than fatal errors. - // If the directory is truly unusable, other errors would be raised when we try to write to it. - log.Warn("failed to remove file", - zap.String("file", toRemoveFilePath), - zap.Error(err)) - // For fail-fast in integration tests - failpoint.Inject("sorterDebug", func() { - log.Panic("panicking", zap.Error(err)) - }) - } - } - - return nil -} - -// checkDataDirSatisfied check if the data-dir meet the requirement during server running -// the caller should guarantee that dir exist -func checkDataDirSatisfied() error { - const dataDirAvailLowThreshold = 10 // percentage - - conf := config.GetGlobalServerConfig() - diskInfo, err := fsutil.GetDiskInfo(conf.DataDir) - if err != nil { - return cerrors.WrapError(cerrors.ErrCheckDataDirSatisfied, err) - } - if diskInfo.AvailPercentage < dataDirAvailLowThreshold { - failpoint.Inject("InjectCheckDataDirSatisfied", func() { - log.Info("inject check data dir satisfied error") - failpoint.Return(nil) - }) - return cerrors.WrapError(cerrors.ErrCheckDataDirSatisfied, errors.Errorf("disk is almost full, TiCDC require that the disk mount data-dir "+ - "have 10%% available space, and the total amount has at least 500GB is preferred. disk info: %+v", diskInfo)) - } - - return nil -} diff --git a/cdc/sorter/unified/backend_pool_test.go b/cdc/sorter/unified/backend_pool_test.go deleted file mode 100644 index 356cd7525af..00000000000 --- a/cdc/sorter/unified/backend_pool_test.go +++ /dev/null @@ -1,354 +0,0 @@ -// Copyright 2020 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 unified - -import ( - "context" - "fmt" - "os" - "path/filepath" - "strconv" - "testing" - "time" - - "github.com/pingcap/failpoint" - "github.com/pingcap/tidb/util/memory" - "github.com/pingcap/tiflow/pkg/config" - "github.com/pingcap/tiflow/pkg/fsutil" - "github.com/stretchr/testify/require" -) - -func TestBasicFunction(t *testing.T) { - dataDir := t.TempDir() - err := os.MkdirAll(dataDir, 0o755) - require.Nil(t, err) - - sortDir := filepath.Join(dataDir, config.DefaultSortDir) - err = os.MkdirAll(sortDir, 0o755) - require.Nil(t, err) - - conf := config.GetDefaultServerConfig() - conf.DataDir = dataDir - conf.Sorter.SortDir = sortDir - conf.Sorter.MaxMemoryPercentage = 90 // 90% - conf.Sorter.MaxMemoryConsumption = 16 * 1024 * 1024 * 1024 // 16G - config.StoreGlobalServerConfig(conf) - - err = failpoint.Enable("github.com/pingcap/tiflow/cdc/sorter/unified/memoryPressureInjectPoint", "return(100)") - require.Nil(t, err) - - ctx, cancel := context.WithTimeout(context.Background(), time.Second*20) - defer cancel() - - backEndPool, err := newBackEndPool(sortDir) - require.Nil(t, err) - require.NotNil(t, backEndPool) - defer backEndPool.terminate() - - backEnd, err := backEndPool.alloc(ctx) - require.Nil(t, err) - require.IsType(t, &fileBackEnd{}, backEnd) - fileName := backEnd.(*fileBackEnd).fileName - require.NotEqual(t, "", fileName) - - err = failpoint.Enable("github.com/pingcap/tiflow/cdc/sorter/unified/memoryPressureInjectPoint", "return(0)") - require.Nil(t, err) - err = failpoint.Enable("github.com/pingcap/tiflow/cdc/sorter/unified/memoryUsageInjectPoint", "return(34359738368)") - require.Nil(t, err) - - backEnd1, err := backEndPool.alloc(ctx) - require.Nil(t, err) - require.IsType(t, &fileBackEnd{}, backEnd1) - fileName1 := backEnd1.(*fileBackEnd).fileName - require.NotEqual(t, "", fileName1) - require.NotEqual(t, fileName, fileName1) - - err = failpoint.Enable("github.com/pingcap/tiflow/cdc/sorter/unified/memoryPressureInjectPoint", "return(0)") - require.Nil(t, err) - err = failpoint.Enable("github.com/pingcap/tiflow/cdc/sorter/unified/memoryUsageInjectPoint", "return(0)") - require.Nil(t, err) - - backEnd2, err := backEndPool.alloc(ctx) - require.Nil(t, err) - require.IsType(t, &memoryBackEnd{}, backEnd2) - - err = backEndPool.dealloc(backEnd) - require.Nil(t, err) - - err = backEndPool.dealloc(backEnd1) - require.Nil(t, err) - - err = backEndPool.dealloc(backEnd2) - require.Nil(t, err) - - time.Sleep(backgroundJobInterval * 3 / 2) - - _, err = os.Stat(fileName) - require.True(t, os.IsNotExist(err)) - - _, err = os.Stat(fileName1) - require.True(t, os.IsNotExist(err)) -} - -// TestDirectoryBadPermission verifies that no permission to ls the directory does not prevent using it -// as a temporary file directory. -func TestDirectoryBadPermission(t *testing.T) { - dataDir := t.TempDir() - sortDir := filepath.Join(dataDir, config.DefaultSortDir) - err := os.MkdirAll(sortDir, 0o755) - require.Nil(t, err) - - err = os.Chmod(sortDir, 0o311) // no permission to `ls` - defer func() { - err := os.Chmod(sortDir, 0o755) - require.Nil(t, err) - }() - require.Nil(t, err) - - conf := config.GetGlobalServerConfig() - conf.DataDir = dataDir - conf.Sorter.SortDir = sortDir - conf.Sorter.MaxMemoryPercentage = 0 // force using files - - backEndPool, err := newBackEndPool(sortDir) - require.Nil(t, err) - require.NotNil(t, backEndPool) - defer backEndPool.terminate() - - backEnd, err := backEndPool.alloc(context.Background()) - require.Nil(t, err) - defer backEnd.free() //nolint:errcheck - - fileName := backEnd.(*fileBackEnd).fileName - _, err = os.Stat(fileName) - require.Nil(t, err) - - err = backEndPool.dealloc(backEnd) - require.Nil(t, err) -} - -// TestCleanUpSelf verifies that the backendPool correctly cleans up files used by itself on exit. -func TestCleanUpSelf(t *testing.T) { - dataDir := t.TempDir() - err := os.Chmod(dataDir, 0o755) - require.Nil(t, err) - - sorterDir := filepath.Join(dataDir, config.DefaultSortDir) - err = os.MkdirAll(sorterDir, 0o755) - require.Nil(t, err) - - conf := config.GetDefaultServerConfig() - conf.DataDir = dataDir - conf.Sorter.SortDir = sorterDir - conf.Sorter.MaxMemoryPercentage = 90 // 90% - conf.Sorter.MaxMemoryConsumption = 16 * 1024 * 1024 * 1024 // 16G - config.StoreGlobalServerConfig(conf) - - err = failpoint.Enable("github.com/pingcap/tiflow/cdc/sorter/unified/memoryPressureInjectPoint", "return(100)") - require.Nil(t, err) - defer failpoint.Disable("github.com/pingcap/tiflow/cdc/sorter/unified/memoryPressureInjectPoint") //nolint:errcheck - - backEndPool, err := newBackEndPool(sorterDir) - require.Nil(t, err) - require.NotNil(t, backEndPool) - - ctx, cancel := context.WithTimeout(context.Background(), time.Second*20) - defer cancel() - - var fileNames []string - for i := 0; i < 20; i++ { - backEnd, err := backEndPool.alloc(ctx) - require.Nil(t, err) - require.IsType(t, &fileBackEnd{}, backEnd) - - fileName := backEnd.(*fileBackEnd).fileName - _, err = os.Stat(fileName) - require.Nil(t, err) - - fileNames = append(fileNames, fileName) - } - - prefix := backEndPool.filePrefix - require.NotEqual(t, "", prefix) - - for j := 100; j < 120; j++ { - fileName := prefix + strconv.Itoa(j) + ".tmp" - f, err := os.Create(fileName) - require.Nil(t, err) - err = f.Close() - require.Nil(t, err) - - fileNames = append(fileNames, fileName) - } - - backEndPool.terminate() - - for _, fileName := range fileNames { - _, err = os.Stat(fileName) - require.True(t, os.IsNotExist(err)) - } -} - -type mockOtherProcess struct { - dir string - prefix string - flock *fsutil.FileLock - files []string -} - -func newMockOtherProcess(t *testing.T, dir string, prefix string) *mockOtherProcess { - prefixLockPath := fmt.Sprintf("%s/%s", dir, sortDirLockFileName) - flock, err := fsutil.NewFileLock(prefixLockPath) - require.Nil(t, err) - - err = flock.Lock() - require.Nil(t, err) - - return &mockOtherProcess{ - dir: dir, - prefix: prefix, - flock: flock, - } -} - -func (p *mockOtherProcess) writeMockFiles(t *testing.T, num int) { - for i := 0; i < num; i++ { - fileName := fmt.Sprintf("%s%d", p.prefix, i) - f, err := os.Create(fileName) - require.Nil(t, err) - _ = f.Close() - p.files = append(p.files, fileName) - } -} - -func (p *mockOtherProcess) changeLockPermission(t *testing.T, mode os.FileMode) { - prefixLockPath := fmt.Sprintf("%s/%s", p.dir, sortDirLockFileName) - err := os.Chmod(prefixLockPath, mode) - require.Nil(t, err) -} - -func (p *mockOtherProcess) unlock(t *testing.T) { - err := p.flock.Unlock() - require.Nil(t, err) -} - -func (p *mockOtherProcess) assertFilesExist(t *testing.T) { - for _, file := range p.files { - _, err := os.Stat(file) - require.Nil(t, err) - } -} - -func (p *mockOtherProcess) assertFilesNotExist(t *testing.T) { - for _, file := range p.files { - _, err := os.Stat(file) - require.True(t, os.IsNotExist(err)) - } -} - -// TestCleanUpStaleBasic verifies that the backendPool correctly cleans up stale temporary files -// left by other CDC processes that have exited abnormally. -func TestCleanUpStaleBasic(t *testing.T) { - dir := t.TempDir() - prefix := dir + "/sort-1-" - - mockP := newMockOtherProcess(t, dir, prefix) - mockP.writeMockFiles(t, 100) - mockP.unlock(t) - mockP.assertFilesExist(t) - - backEndPool, err := newBackEndPool(dir) - require.Nil(t, err) - require.NotNil(t, backEndPool) - defer backEndPool.terminate() - - mockP.assertFilesNotExist(t) -} - -// TestFileLockConflict tests that if two backEndPools were to use the same sort-dir, -// and error would be returned by one of them. -func TestFileLockConflict(t *testing.T) { - dir := t.TempDir() - - backEndPool1, err := newBackEndPool(dir) - require.Nil(t, err) - require.NotNil(t, backEndPool1) - defer backEndPool1.terminate() - - backEndPool2, err := newBackEndPool(dir) - require.Regexp(t, ".*file lock conflict.*", err) - require.Nil(t, backEndPool2) -} - -// TestCleanUpStaleBasic verifies that the backendPool correctly cleans up stale temporary files -// left by other CDC processes that have exited abnormally. -func TestCleanUpStaleLockNoPermission(t *testing.T) { - dir := t.TempDir() - prefix := dir + "/sort-1-" - - mockP := newMockOtherProcess(t, dir, prefix) - mockP.writeMockFiles(t, 100) - // set a bad permission - mockP.changeLockPermission(t, 0o000) - - backEndPool, err := newBackEndPool(dir) - require.Regexp(t, ".*permission denied.*", err) - require.Nil(t, backEndPool) - - mockP.assertFilesExist(t) -} - -// TestGetMemoryPressureFailure verifies that the backendPool can handle gracefully failures that happen when -// getting the current system memory pressure. Such a failure is usually caused by a lack of file descriptor quota -// set by the operating system. -func TestGetMemoryPressureFailure(t *testing.T) { - origin := memory.MemTotal - defer func() { - memory.MemTotal = origin - }() - memory.MemTotal = func() (uint64, error) { return 0, nil } - - dir := t.TempDir() - backEndPool, err := newBackEndPool(dir) - require.Nil(t, err) - require.NotNil(t, backEndPool) - defer backEndPool.terminate() - - after := time.After(time.Second * 20) - tick := time.Tick(time.Millisecond * 100) - for { - select { - case <-after: - t.Fatal("TestGetMemoryPressureFailure timed out") - case <-tick: - if backEndPool.memoryPressure() == 100 { - return - } - } - } -} - -func TestCheckDataDirSatisfied(t *testing.T) { - dir := t.TempDir() - conf := config.GetGlobalServerConfig() - conf.DataDir = dir - config.StoreGlobalServerConfig(conf) - - p := "github.com/pingcap/tiflow/cdc/sorter/unified/" + - "InjectCheckDataDirSatisfied" - require.Nil(t, failpoint.Enable(p, "")) - err := checkDataDirSatisfied() - require.Nil(t, err) - require.Nil(t, failpoint.Disable(p)) -} diff --git a/cdc/sorter/unified/file_backend.go b/cdc/sorter/unified/file_backend.go deleted file mode 100644 index 77304d73d2f..00000000000 --- a/cdc/sorter/unified/file_backend.go +++ /dev/null @@ -1,449 +0,0 @@ -// Copyright 2021 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package unified - -import ( - "bufio" - "encoding/binary" - "io" - "os" - "sync/atomic" - - "github.com/pingcap/errors" - "github.com/pingcap/failpoint" - "github.com/pingcap/log" - "github.com/pingcap/tiflow/cdc/model" - "github.com/pingcap/tiflow/cdc/sorter/encoding" - cerrors "github.com/pingcap/tiflow/pkg/errors" - "go.uber.org/zap" -) - -const ( - fileBufferSize = 4 * 1024 // 4KB - fileMagic = 0x12345678 - numFileEntriesOffset = 4 - blockMagic = 0xbeefbeef -) - -var openFDCount int64 - -type fileBackEnd struct { - fileName string - serde encoding.SerializerDeserializer - borrowed int32 - size int64 -} - -func newFileBackEnd(fileName string, serde encoding.SerializerDeserializer) (*fileBackEnd, error) { - f, err := os.Create(fileName) - if err != nil { - return nil, errors.Trace(wrapIOError(err)) - } - - err = f.Close() - if err != nil { - return nil, errors.Trace(wrapIOError(err)) - } - - log.Debug("new FileSorterBackEnd created", zap.String("filename", fileName)) - return &fileBackEnd{ - fileName: fileName, - serde: serde, - borrowed: 0, - }, nil -} - -func (f *fileBackEnd) reader() (backEndReader, error) { - fd, err := os.OpenFile(f.fileName, os.O_RDWR, 0o600) - if err != nil { - return nil, errors.Trace(wrapIOError(err)) - } - - atomic.AddInt64(&openFDCount, 1) - - var totalSize int64 - failpoint.Inject("sorterDebug", func() { - info, err := fd.Stat() - if err != nil { - failpoint.Return(nil, errors.Trace(wrapIOError(err))) - } - totalSize = info.Size() - }) - - failpoint.Inject("sorterDebug", func() { - if atomic.SwapInt32(&f.borrowed, 1) != 0 { - log.Panic("fileBackEnd: already borrowed", zap.String("fileName", f.fileName)) - } - }) - - ret := &fileBackEndReader{ - backEnd: f, - f: fd, - reader: bufio.NewReaderSize(fd, fileBufferSize), - totalSize: totalSize, - } - - err = ret.readHeader() - if err != nil { - return nil, errors.Trace(wrapIOError(err)) - } - - return ret, nil -} - -func (f *fileBackEnd) writer() (backEndWriter, error) { - fd, err := os.OpenFile(f.fileName, os.O_TRUNC|os.O_RDWR, 0o600) - if err != nil { - return nil, errors.Trace(wrapIOError(err)) - } - - atomic.AddInt64(&openFDCount, 1) - - failpoint.Inject("sorterDebug", func() { - if atomic.SwapInt32(&f.borrowed, 1) != 0 { - log.Panic("fileBackEnd: already borrowed", zap.String("fileName", f.fileName)) - } - }) - - ret := &fileBackEndWriter{ - backEnd: f, - f: fd, - writer: bufio.NewWriterSize(fd, fileBufferSize), - } - - err = ret.writeFileHeader() - if err != nil { - return nil, errors.Trace(wrapIOError(err)) - } - - return ret, nil -} - -func (f *fileBackEnd) free() error { - failpoint.Inject("sorterDebug", func() { - if atomic.LoadInt32(&f.borrowed) != 0 { - log.Panic("fileBackEnd: trying to free borrowed file", zap.String("fileName", f.fileName)) - } - }) - - log.Debug("Removing file", zap.String("file", f.fileName)) - - f.cleanStats() - - err := os.Remove(f.fileName) - if err != nil { - failpoint.Inject("sorterDebug", func() { - failpoint.Return(errors.Trace(wrapIOError(err))) - }) - // ignore this error in production to provide some resilience - log.Warn("fileBackEnd: failed to remove file", zap.Error(wrapIOError(err))) - } - - return nil -} - -func (f *fileBackEnd) cleanStats() { - if pool != nil { - atomic.AddInt64(&pool.onDiskDataSize, -f.size) - } - f.size = 0 -} - -type fileBackEndReader struct { - backEnd *fileBackEnd - f *os.File - reader *bufio.Reader - isEOF bool - - // to prevent truncation-like corruption - totalEvents uint64 - readEvents uint64 - - // debug only fields - readBytes int64 - totalSize int64 -} - -func (r *fileBackEndReader) readHeader() error { - failpoint.Inject("sorterDebug", func() { - pos, err := r.f.Seek(0, 1 /* relative to the current position */) - if err != nil { - failpoint.Return(errors.Trace(err)) - } - // verify that we are reading from the beginning of the file - if pos != 0 { - log.Panic("unexpected file descriptor cursor position", zap.Int64("pos", pos)) - } - }) - - var m uint32 - err := binary.Read(r.reader, binary.LittleEndian, &m) - if err != nil { - return errors.Trace(err) - } - if m != fileMagic { - log.Panic("fileSorterBackEnd: wrong fileMagic. Damaged file or bug?", zap.Uint32("actual", m)) - } - - err = binary.Read(r.reader, binary.LittleEndian, &r.totalEvents) - if err != nil { - return errors.Trace(err) - } - - return nil -} - -func (r *fileBackEndReader) readNext() (*model.PolymorphicEvent, error) { - if r.isEOF { - // guaranteed EOF idempotency - return nil, nil - } - - var m uint32 - err := binary.Read(r.reader, binary.LittleEndian, &m) - if err != nil { - if err == io.EOF { - r.isEOF = true - // verifies that the file has not been truncated unexpectedly. - if r.totalEvents != r.readEvents { - log.Panic("unexpected EOF", - zap.String("file", r.backEnd.fileName), - zap.Uint64("expectedNumEvents", r.totalEvents), - zap.Uint64("actualNumEvents", r.readEvents)) - } - return nil, nil - } - return nil, errors.Trace(wrapIOError(err)) - } - - if m != blockMagic { - log.Panic("fileSorterBackEnd: wrong blockMagic. Damaged file or bug?", zap.Uint32("actual", m)) - } - - var size uint32 - err = binary.Read(r.reader, binary.LittleEndian, &size) - if err != nil { - return nil, errors.Trace(wrapIOError(err)) - } - - // Note, do not hold the buffer in reader to avoid hogging memory. - rawBytesBuf := make([]byte, size) - - // short reads are possible with bufio, hence the need for io.ReadFull - n, err := io.ReadFull(r.reader, rawBytesBuf) - if err != nil { - return nil, errors.Trace(wrapIOError(err)) - } - - if n != int(size) { - return nil, errors.Errorf("fileSorterBackEnd: expected %d bytes, actually read %d bytes", size, n) - } - - event := new(model.PolymorphicEvent) - _, err = r.backEnd.serde.Unmarshal(event, rawBytesBuf) - if err != nil { - return nil, errors.Trace(err) - } - - r.readEvents++ - - failpoint.Inject("sorterDebug", func() { - r.readBytes += int64(4 + 4 + int(size)) - if r.readBytes > r.totalSize { - log.Panic("fileSorterBackEnd: read more bytes than expected, check concurrent use of file", - zap.String("fileName", r.backEnd.fileName)) - } - }) - - return event, nil -} - -func (r *fileBackEndReader) resetAndClose() error { - defer func() { - // fail-fast for double-close - r.f = nil - - r.backEnd.cleanStats() - - failpoint.Inject("sorterDebug", func() { - atomic.StoreInt32(&r.backEnd.borrowed, 0) - }) - }() - - if r.f == nil { - failpoint.Inject("sorterDebug", func() { - log.Panic("Double closing of file", zap.String("filename", r.backEnd.fileName)) - }) - log.Warn("Double closing of file", zap.String("filename", r.backEnd.fileName)) - return nil - } - - err := r.f.Truncate(0) - if err != nil { - failpoint.Inject("sorterDebug", func() { - info, err1 := r.f.Stat() - if err1 != nil { - failpoint.Return(errors.Trace(wrapIOError(err))) - } - - log.Info("file debug info", zap.String("filename", info.Name()), - zap.Int64("size", info.Size())) - - failpoint.Return(nil) - }) - log.Warn("fileBackEndReader: could not truncate file", zap.Error(err)) - } - - err = r.f.Close() - if err != nil { - failpoint.Inject("sorterDebug", func() { - failpoint.Return(errors.Trace(err)) - }) - log.Warn("fileBackEndReader: could not close file", zap.Error(err)) - return nil - } - - atomic.AddInt64(&openFDCount, -1) - - return nil -} - -type fileBackEndWriter struct { - backEnd *fileBackEnd - f *os.File - writer *bufio.Writer - - bytesWritten int64 - eventsWritten int64 -} - -func (w *fileBackEndWriter) writeFileHeader() error { - err := binary.Write(w.writer, binary.LittleEndian, uint32(fileMagic)) - if err != nil { - return errors.Trace(err) - } - - // reserves the space for writing the total number of entries in this file - err = binary.Write(w.writer, binary.LittleEndian, uint64(0)) - if err != nil { - return errors.Trace(err) - } - - return nil -} - -func (w *fileBackEndWriter) writeNext(event *model.PolymorphicEvent) error { - var err error - // Note, do not hold the buffer in writer to avoid hogging memory. - var rawBytesBuf []byte - rawBytesBuf, err = w.backEnd.serde.Marshal(event, rawBytesBuf) - if err != nil { - return errors.Trace(wrapIOError(err)) - } - - size := len(rawBytesBuf) - if size == 0 { - log.Panic("fileSorterBackEnd: serialized to empty byte array. Bug?") - } - - err = binary.Write(w.writer, binary.LittleEndian, uint32(blockMagic)) - if err != nil { - return errors.Trace(wrapIOError(err)) - } - - err = binary.Write(w.writer, binary.LittleEndian, uint32(size)) - if err != nil { - return errors.Trace(wrapIOError(err)) - } - - // short writes are possible with bufio - offset := 0 - for offset < size { - n, err := w.writer.Write(rawBytesBuf[offset:]) - if err != nil { - return errors.Trace(wrapIOError(err)) - } - offset += n - } - if offset != size { - return errors.Errorf("fileSorterBackEnd: expected to write %d bytes, actually wrote %d bytes", size, offset) - } - - w.eventsWritten++ - w.bytesWritten += int64(size) - return nil -} - -func (w *fileBackEndWriter) writtenCount() int { - return int(w.eventsWritten) -} - -func (w *fileBackEndWriter) dataSize() uint64 { - return uint64(w.bytesWritten) -} - -func (w *fileBackEndWriter) flushAndClose() error { - defer func() { - // fail-fast for double-close - w.f = nil - }() - - err := w.writer.Flush() - if err != nil { - return errors.Trace(wrapIOError(err)) - } - - _, err = w.f.Seek(numFileEntriesOffset, 0 /* relative to the beginning of the file */) - if err != nil { - return errors.Trace(wrapIOError(err)) - } - - // write the total number of entries in the file to the header - err = binary.Write(w.f, binary.LittleEndian, uint64(w.eventsWritten)) - if err != nil { - return errors.Trace(wrapIOError(err)) - } - - err = w.f.Close() - if err != nil { - failpoint.Inject("sorterDebug", func() { - failpoint.Return(errors.Trace(wrapIOError(err))) - }) - log.Warn("fileBackEndReader: could not close file", zap.Error(err)) - return nil - } - - atomic.AddInt64(&openFDCount, -1) - w.backEnd.size = w.bytesWritten - atomic.AddInt64(&pool.onDiskDataSize, w.bytesWritten) - - failpoint.Inject("sorterDebug", func() { - atomic.StoreInt32(&w.backEnd.borrowed, 0) - }) - - return nil -} - -// wrapIOError should be called when the error is to be returned to an caller outside this file and -// if the error could be caused by a filesystem-related error. -func wrapIOError(err error) error { - cause := errors.Cause(err) - switch cause.(type) { - case *os.PathError: - // We don't generate stack in this helper function to avoid confusion. - return cerrors.ErrUnifiedSorterIOError.FastGenByArgs(err.Error()) - default: - return err - } -} diff --git a/cdc/sorter/unified/file_backend_test.go b/cdc/sorter/unified/file_backend_test.go deleted file mode 100644 index ddada6d8411..00000000000 --- a/cdc/sorter/unified/file_backend_test.go +++ /dev/null @@ -1,90 +0,0 @@ -// Copyright 2021 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package unified - -import ( - "io" - "os" - "testing" - - "github.com/pingcap/tiflow/cdc/model" - "github.com/pingcap/tiflow/cdc/sorter/encoding" - "github.com/pingcap/tiflow/pkg/errors" - "github.com/stretchr/testify/require" -) - -func TestWrapIOError(t *testing.T) { - fullFile, err := os.OpenFile("/dev/full", os.O_RDWR, 0) - require.Nil(t, err) - defer fullFile.Close() //nolint:errcheck - - _, err = fullFile.WriteString("test") - wrapped := wrapIOError(err) - // tests that the error message gives the user some informative description - require.Regexp(t, ".*review the settings.*no space.*", wrapped.Error()) - - eof := wrapIOError(io.EOF) - // tests that the function does not change io.EOF - require.Equal(t, io.EOF, eof) -} - -func TestNoSpace(t *testing.T) { - fb := &fileBackEnd{ - fileName: "/dev/full", - serde: &encoding.MsgPackGenSerde{}, - } - w, err := fb.writer() - require.Nil(t, err) - - err = w.writeNext(model.NewPolymorphicEvent(generateMockRawKV(0))) - if err == nil { - // Due to write buffering, `writeNext` might not return an error when the filesystem is full. - err = w.flushAndClose() - } - - require.Regexp(t, ".*review the settings.*no space.*", err.Error()) - require.True(t, errors.ErrUnifiedSorterIOError.Equal(err)) -} - -func TestWrittenCount(t *testing.T) { - f, err := os.CreateTemp("", "writer-test") - require.Nil(t, err) - defer os.Remove(f.Name()) - - fb := &fileBackEnd{ - fileName: f.Name(), - serde: &encoding.MsgPackGenSerde{}, - } - w, err := fb.writer() - require.Nil(t, err) - err = w.writeNext(model.NewPolymorphicEvent(generateMockRawKV(0))) - require.Nil(t, err) - require.Equal(t, 1, w.writtenCount()) -} - -func TestDataSize(t *testing.T) { - f, err := os.CreateTemp("", "writer-test") - require.Nil(t, err) - defer os.Remove(f.Name()) - - fb := &fileBackEnd{ - fileName: f.Name(), - serde: &encoding.MsgPackGenSerde{}, - } - w, err := fb.writer() - require.Nil(t, err) - err = w.writeNext(model.NewPolymorphicEvent(generateMockRawKV(0))) - require.Nil(t, err) - require.Equal(t, uint64(71), w.dataSize()) -} diff --git a/cdc/sorter/unified/heap.go b/cdc/sorter/unified/heap.go deleted file mode 100644 index 9145556c740..00000000000 --- a/cdc/sorter/unified/heap.go +++ /dev/null @@ -1,41 +0,0 @@ -// Copyright 2020 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 unified - -import "github.com/pingcap/tiflow/cdc/model" - -type sortItem struct { - entry *model.PolymorphicEvent - data interface{} -} - -type sortHeap []*sortItem - -func (h sortHeap) Len() int { return len(h) } -func (h sortHeap) Less(i, j int) bool { - return model.ComparePolymorphicEvents(h[i].entry, h[j].entry) -} -func (h sortHeap) Swap(i, j int) { h[i], h[j] = h[j], h[i] } -func (h *sortHeap) Push(x interface{}) { - *h = append(*h, x.(*sortItem)) -} - -func (h *sortHeap) Pop() interface{} { - old := *h - n := len(old) - x := old[n-1] - old[n-1] = nil - *h = old[0 : n-1] - return x -} diff --git a/cdc/sorter/unified/heap_sorter.go b/cdc/sorter/unified/heap_sorter.go deleted file mode 100644 index 30c4930472c..00000000000 --- a/cdc/sorter/unified/heap_sorter.go +++ /dev/null @@ -1,394 +0,0 @@ -// Copyright 2021 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package unified - -import ( - "container/heap" - "context" - "sync" - "sync/atomic" - "time" - - "github.com/pingcap/errors" - "github.com/pingcap/failpoint" - "github.com/pingcap/log" - "github.com/pingcap/tiflow/cdc/contextutil" - "github.com/pingcap/tiflow/cdc/model" - "github.com/pingcap/tiflow/pkg/config" - cerrors "github.com/pingcap/tiflow/pkg/errors" - "github.com/pingcap/tiflow/pkg/workerpool" - "go.uber.org/zap" -) - -const ( - flushRateLimitPerSecond = 10 - sortHeapCapacity = 32 - sortHeapInputChSize = 1024 -) - -type flushTask struct { - taskID int - heapSorterID int - reader backEndReader - tsLowerBound uint64 - maxResolvedTs uint64 - finished chan error - dealloc func() error - dataSize int64 - lastTs uint64 // for debugging TODO remove - canceller *asyncCanceller - - isEmpty bool // read only field - - deallocLock sync.RWMutex - isDeallocated bool // do not access directly - backend backEnd // do not access directly -} - -func (t *flushTask) markDeallocated() { - t.deallocLock.Lock() - defer t.deallocLock.Unlock() - - t.backend = nil - t.isDeallocated = true -} - -func (t *flushTask) GetBackEnd() backEnd { - t.deallocLock.RLock() - defer t.deallocLock.RUnlock() - - return t.backend -} - -type heapSorter struct { - id int - taskCounter int - inputCh chan *model.PolymorphicEvent - outputCh chan *flushTask - heap sortHeap - canceller *asyncCanceller - - poolHandle workerpool.EventHandle - internalState *heapSorterInternalState -} - -func newHeapSorter(id int, out chan *flushTask) *heapSorter { - return &heapSorter{ - id: id, - inputCh: make(chan *model.PolymorphicEvent, sortHeapInputChSize), - outputCh: out, - heap: make(sortHeap, 0, sortHeapCapacity), - canceller: new(asyncCanceller), - } -} - -// flush should only be called in the same goroutine where the heap is being written to. -func (h *heapSorter) flush(ctx context.Context, maxResolvedTs uint64) error { - changefeedID := contextutil.ChangefeedIDFromCtx(ctx) - - var ( - backEnd backEnd - lowerBound uint64 - ) - - if h.heap.Len() > 0 { - lowerBound = h.heap[0].entry.CRTs - } else { - return nil - } - - sorterFlushCountHistogram. - WithLabelValues(changefeedID.Namespace, changefeedID.ID). - Observe(float64(h.heap.Len())) - - // We check if the heap contains only one entry and that entry is a ResolvedEvent. - // As an optimization, when the condition is true, we clear the heap and send an empty flush. - // Sending an empty flush saves CPU and potentially IO. - // Since when a table is mostly idle or near-idle, most flushes would contain one ResolvedEvent alone, - // this optimization will greatly improve performance when (1) total number of table is large, - // and (2) most tables do not have many events. - if h.heap.Len() == 1 && h.heap[0].entry.IsResolved() { - h.heap.Pop() - } - - isEmptyFlush := h.heap.Len() == 0 - var finishCh chan error - if !isEmptyFlush { - failpoint.Inject("InjectErrorBackEndAlloc", func() { - failpoint.Return(cerrors.ErrUnifiedSorterIOError.Wrap(errors.New("injected alloc error")).FastGenWithCause()) - }) - - var err error - backEnd, err = pool.alloc(ctx) - if err != nil { - return errors.Trace(err) - } - - finishCh = make(chan error, 1) - } - - task := &flushTask{ - taskID: h.taskCounter, - heapSorterID: h.id, - backend: backEnd, - tsLowerBound: lowerBound, - maxResolvedTs: maxResolvedTs, - finished: finishCh, - canceller: h.canceller, - isEmpty: isEmptyFlush, - } - h.taskCounter++ - - var oldHeap sortHeap - if !isEmptyFlush { - task.dealloc = func() error { - backEnd := task.GetBackEnd() - if backEnd != nil { - defer task.markDeallocated() - return pool.dealloc(backEnd) - } - return nil - } - oldHeap = h.heap - h.heap = make(sortHeap, 0, sortHeapCapacity) - } else { - task.dealloc = func() error { - task.markDeallocated() - return nil - } - } - failpoint.Inject("sorterDebug", func() { - tableID, tableName := contextutil.TableIDFromCtx(ctx) - log.Debug("Unified Sorter new flushTask", - zap.Int64("tableID", tableID), - zap.String("tableName", tableName), - zap.Int("heapID", task.heapSorterID), - zap.Uint64("resolvedTs", task.maxResolvedTs)) - }) - - if !isEmptyFlush { - backEndFinal := backEnd - err := heapSorterIOPool.Go(ctx, func() { - failpoint.Inject("asyncFlushStartDelay", func() { - log.Debug("asyncFlushStartDelay") - }) - - h.canceller.EnterAsyncOp() - defer h.canceller.FinishAsyncOp() - - if h.canceller.IsCanceled() { - if backEndFinal != nil { - _ = task.dealloc() - } - task.finished <- cerrors.ErrAsyncIOCancelled.GenWithStackByArgs() - return - } - - writer, err := backEnd.writer() - if err != nil { - if backEndFinal != nil { - _ = task.dealloc() - } - task.finished <- errors.Trace(err) - return - } - - defer func() { - // handle errors (or aborts) gracefully to prevent resource leaking (especially FD's) - if writer != nil { - _ = writer.flushAndClose() - } - if backEndFinal != nil { - _ = task.dealloc() - } - close(task.finished) - }() - - failpoint.Inject("InjectErrorBackEndWrite", func() { - task.finished <- cerrors.ErrUnifiedSorterIOError.Wrap(errors.New("injected write error")).FastGenWithCause() - failpoint.Return() - }) - - counter := 0 - for oldHeap.Len() > 0 { - failpoint.Inject("asyncFlushInProcessDelay", func() { - log.Debug("asyncFlushInProcessDelay") - }) - // no need to check for cancellation so frequently. - if counter%10000 == 0 && h.canceller.IsCanceled() { - task.finished <- cerrors.ErrAsyncIOCancelled.GenWithStackByArgs() - return - } - counter++ - - event := heap.Pop(&oldHeap).(*sortItem).entry - err := writer.writeNext(event) - if err != nil { - task.finished <- errors.Trace(err) - return - } - } - - dataSize := writer.dataSize() - atomic.StoreInt64(&task.dataSize, int64(dataSize)) - eventCount := writer.writtenCount() - - writer1 := writer - writer = nil - err = writer1.flushAndClose() - if err != nil { - task.finished <- errors.Trace(err) - return - } - - backEndFinal = nil - - failpoint.Inject("sorterDebug", func() { - tableID, tableName := contextutil.TableIDFromCtx(ctx) - log.Debug("Unified Sorter flushTask finished", - zap.Int("heapID", task.heapSorterID), - zap.Int64("tableID", tableID), - zap.String("tableName", tableName), - zap.Uint64("resolvedTs", task.maxResolvedTs), - zap.Uint64("dataSize", dataSize), - zap.Int("size", eventCount)) - }) - - task.finished <- nil // DO NOT access `task` beyond this point in this function - }) - if err != nil { - close(task.finished) - return errors.Trace(err) - } - } - - select { - case <-ctx.Done(): - return ctx.Err() - case h.outputCh <- task: - } - return nil -} - -var ( - heapSorterPool workerpool.WorkerPool - heapSorterIOPool workerpool.AsyncPool - poolOnce sync.Once -) - -type heapSorterInternalState struct { - maxResolved uint64 - heapSizeBytesEstimate int64 - rateCounter int - sorterConfig *config.SorterConfig - timerMultiplier int -} - -func (h *heapSorter) init(ctx context.Context, onError func(err error)) { - state := &heapSorterInternalState{ - sorterConfig: config.GetGlobalServerConfig().Sorter, - } - - poolHandle := heapSorterPool.RegisterEvent(func(ctx context.Context, eventI interface{}) error { - event := eventI.(*model.PolymorphicEvent) - heap.Push(&h.heap, &sortItem{entry: event}) - isResolvedEvent := event.RawKV != nil && event.IsResolved() - - if isResolvedEvent { - if event.RawKV.CRTs < state.maxResolved { - log.Panic("ResolvedTs regression, bug?", zap.Uint64("resolvedTs", event.RawKV.CRTs), - zap.Uint64("maxResolvedTs", state.maxResolved)) - } - state.maxResolved = event.RawKV.CRTs - } - - if event.RawKV.CRTs < state.maxResolved { - log.Panic("Bad input to sorter", zap.Uint64("curTs", event.RawKV.CRTs), zap.Uint64("maxResolved", state.maxResolved)) - } - - // 5 * 8 is for the 5 fields in PolymorphicEvent - state.heapSizeBytesEstimate += event.RawKV.ApproximateDataSize() + 40 - needFlush := state.heapSizeBytesEstimate >= int64(state.sorterConfig.ChunkSizeLimit) || - (isResolvedEvent && state.rateCounter < flushRateLimitPerSecond) - - if needFlush { - state.rateCounter++ - err := h.flush(ctx, state.maxResolved) - if err != nil { - return errors.Trace(err) - } - state.heapSizeBytesEstimate = 0 - } - - return nil - }).SetTimer(ctx, 1*time.Second, func(ctx context.Context) error { - state.rateCounter = 0 - state.timerMultiplier = (state.timerMultiplier + 1) % 5 - if state.timerMultiplier == 0 && state.rateCounter < flushRateLimitPerSecond { - err := h.flush(ctx, state.maxResolved) - if err != nil { - return errors.Trace(err) - } - state.heapSizeBytesEstimate = 0 - } - return nil - }).OnExit(onError) - - h.poolHandle = poolHandle - h.internalState = state -} - -// asyncCanceller is a shared object used to cancel async IO operations. -// We do not use `context.Context` because (1) selecting on `ctx.Done()` is expensive -// especially if the context is shared by many goroutines, and (2) due to the complexity -// of managing contexts through the workerpools, using a special shared object seems more reasonable -// and readable. -type asyncCanceller struct { - exitRWLock sync.RWMutex // held when an asynchronous flush is taking place - hasExited int32 // this flag should be accessed atomically -} - -func (c *asyncCanceller) EnterAsyncOp() { - c.exitRWLock.RLock() -} - -func (c *asyncCanceller) FinishAsyncOp() { - c.exitRWLock.RUnlock() -} - -func (c *asyncCanceller) IsCanceled() bool { - return atomic.LoadInt32(&c.hasExited) == 1 -} - -func (c *asyncCanceller) Cancel() { - // Sets the flag - atomic.StoreInt32(&c.hasExited, 1) - - // By taking the lock, we are making sure that all IO operations that started before setting the flag have finished, - // so that by the returning of this function, no more IO operations will finish successfully. - // Since IO operations that are NOT successful will clean up themselves, the goroutine in which this - // function was called is responsible for releasing files written by only those IO operations that complete BEFORE - // this function returns. - // In short, we are creating a linearization point here. - c.exitRWLock.Lock() - defer c.exitRWLock.Unlock() -} - -func lazyInitWorkerPool() { - poolOnce.Do(func() { - sorterConfig := config.GetGlobalServerConfig().Sorter - heapSorterPool = workerpool.NewDefaultWorkerPool(sorterConfig.NumWorkerPoolGoroutine) - heapSorterIOPool = workerpool.NewDefaultAsyncPool(sorterConfig.NumWorkerPoolGoroutine * 2) - }) -} diff --git a/cdc/sorter/unified/main_test.go b/cdc/sorter/unified/main_test.go deleted file mode 100644 index f9c3cd4b1f6..00000000000 --- a/cdc/sorter/unified/main_test.go +++ /dev/null @@ -1,24 +0,0 @@ -// Copyright 2022 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package unified - -import ( - "testing" - - "github.com/pingcap/tiflow/pkg/leakutil" -) - -func TestMain(m *testing.M) { - leakutil.SetUpLeakTest(m) -} diff --git a/cdc/sorter/unified/memory_backend.go b/cdc/sorter/unified/memory_backend.go deleted file mode 100644 index 66db83c30b8..00000000000 --- a/cdc/sorter/unified/memory_backend.go +++ /dev/null @@ -1,146 +0,0 @@ -// Copyright 2021 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package unified - -import ( - "sync/atomic" - - "github.com/pingcap/failpoint" - "github.com/pingcap/log" - "github.com/pingcap/tiflow/cdc/model" - "go.uber.org/zap" -) - -type memoryBackEnd struct { - events []*model.PolymorphicEvent - estimatedSize int64 - borrowed int32 -} - -func newMemoryBackEnd() *memoryBackEnd { - return &memoryBackEnd{} -} - -func (m *memoryBackEnd) reader() (backEndReader, error) { - failpoint.Inject("sorterDebug", func() { - if atomic.SwapInt32(&m.borrowed, 1) != 0 { - log.Panic("memoryBackEnd: already borrowed") - } - }) - - return &memoryBackEndReader{ - backEnd: m, - readIndex: 0, - }, nil -} - -func (m *memoryBackEnd) writer() (backEndWriter, error) { - failpoint.Inject("sorterDebug", func() { - if atomic.SwapInt32(&m.borrowed, 1) != 0 { - log.Panic("memoryBackEnd: already borrowed") - } - }) - - return &memoryBackEndWriter{backEnd: m}, nil -} - -func (m *memoryBackEnd) free() error { - failpoint.Inject("sorterDebug", func() { - if atomic.LoadInt32(&m.borrowed) != 0 { - log.Panic("fileBackEnd: trying to free borrowed file") - } - }) - - if pool != nil { - atomic.AddInt64(&pool.memoryUseEstimate, -m.estimatedSize) - } - - return nil -} - -type memoryBackEndReader struct { - backEnd *memoryBackEnd - readIndex int -} - -func (r *memoryBackEndReader) readNext() (*model.PolymorphicEvent, error) { - // Check for "EOF" - if r.readIndex >= len(r.backEnd.events) { - return nil, nil - } - - ret := r.backEnd.events[r.readIndex] - // Sets the slot to nil to prevent delaying GC. - r.backEnd.events[r.readIndex] = nil - r.readIndex++ - return ret, nil -} - -func (r *memoryBackEndReader) resetAndClose() error { - failpoint.Inject("sorterDebug", func() { - atomic.StoreInt32(&r.backEnd.borrowed, 0) - }) - - if pool != nil { - atomic.AddInt64(&pool.memoryUseEstimate, -r.backEnd.estimatedSize) - } - r.backEnd.estimatedSize = 0 - - return nil -} - -type memoryBackEndWriter struct { - backEnd *memoryBackEnd - bytesWritten int64 - // for debugging only - maxTs uint64 -} - -func (w *memoryBackEndWriter) writeNext(event *model.PolymorphicEvent) error { - w.backEnd.events = append(w.backEnd.events, event) - // 8 * 5 is for the 5 fields in PolymorphicEvent, each of which is thought of as a 64-bit pointer - w.bytesWritten += 8*5 + event.RawKV.ApproximateDataSize() - - failpoint.Inject("sorterDebug", func() { - if event.CRTs < w.maxTs { - log.Panic("memoryBackEnd: ts regressed, bug?", - zap.Uint64("prevTs", w.maxTs), - zap.Uint64("curTs", event.CRTs)) - } - w.maxTs = event.CRTs - }) - return nil -} - -func (w *memoryBackEndWriter) writtenCount() int { - return len(w.backEnd.events) -} - -// dataSize for the memoryBackEnd returns only an estimation, as there is no serialization taking place. -func (w *memoryBackEndWriter) dataSize() uint64 { - return uint64(w.bytesWritten) -} - -func (w *memoryBackEndWriter) flushAndClose() error { - failpoint.Inject("sorterDebug", func() { - atomic.StoreInt32(&w.backEnd.borrowed, 0) - }) - - w.backEnd.estimatedSize = w.bytesWritten - if pool != nil { - atomic.AddInt64(&pool.memoryUseEstimate, w.bytesWritten) - } - - return nil -} diff --git a/cdc/sorter/unified/memory_backend_test.go b/cdc/sorter/unified/memory_backend_test.go deleted file mode 100644 index 090c4faf4cc..00000000000 --- a/cdc/sorter/unified/memory_backend_test.go +++ /dev/null @@ -1,72 +0,0 @@ -// Copyright 2021 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package unified - -import ( - "runtime" - "sync/atomic" - "testing" - "time" - - "github.com/pingcap/tiflow/cdc/model" - "github.com/stretchr/testify/require" -) - -func TestNoLeaking(t *testing.T) { - bknd := newMemoryBackEnd() - wrtr, err := bknd.writer() - require.Nil(t, err) - - var objCount int64 - for i := 0; i < 10000; i++ { - atomic.AddInt64(&objCount, 1) - event := model.NewResolvedPolymorphicEvent(0, 1) - runtime.SetFinalizer(event, func(*model.PolymorphicEvent) { - atomic.AddInt64(&objCount, -1) - }) - err := wrtr.writeNext(event) - require.Nil(t, err) - } - err = wrtr.flushAndClose() - require.Nil(t, err) - - rdr, err := bknd.reader() - require.Nil(t, err) - - for i := 0; i < 5000; i++ { - _, err := rdr.readNext() - require.Nil(t, err) - } - - for i := 0; i < 10; i++ { - runtime.GC() - if atomic.LoadInt64(&objCount) <= 5000 { - break - } - time.Sleep(100 * time.Millisecond) - } - require.LessOrEqual(t, atomic.LoadInt64(&objCount), int64(5000)) - - err = rdr.resetAndClose() - require.Nil(t, err) - - for i := 0; i < 10; i++ { - runtime.GC() - if atomic.LoadInt64(&objCount) == 0 { - break - } - time.Sleep(100 * time.Millisecond) - } - require.Equal(t, int64(0), atomic.LoadInt64(&objCount)) -} diff --git a/cdc/sorter/unified/merger.go b/cdc/sorter/unified/merger.go deleted file mode 100644 index efd0a08d43a..00000000000 --- a/cdc/sorter/unified/merger.go +++ /dev/null @@ -1,521 +0,0 @@ -// Copyright 2021 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package unified - -import ( - "container/heap" - "context" - "math" - "strings" - "sync" - "sync/atomic" - "time" - - "github.com/pingcap/errors" - "github.com/pingcap/failpoint" - "github.com/pingcap/log" - "github.com/pingcap/tiflow/cdc/contextutil" - "github.com/pingcap/tiflow/cdc/model" - "github.com/pingcap/tiflow/cdc/sorter" - cerrors "github.com/pingcap/tiflow/pkg/errors" - "github.com/tikv/client-go/v2/oracle" - "go.uber.org/zap" - "golang.org/x/sync/errgroup" -) - -// TODO refactor this into a struct Merger. -func runMerger(ctx context.Context, numSorters int, in <-chan *flushTask, out chan *model.PolymorphicEvent, onExit func()) error { - changefeedID := contextutil.ChangefeedIDFromCtx(ctx) - - metricSorterEventCount := sorter.OutputEventCount.MustCurryWith(map[string]string{ - "namespace": changefeedID.Namespace, - "changefeed": changefeedID.ID, - }) - metricSorterResolvedTsGauge := sorter.ResolvedTsGauge. - WithLabelValues(changefeedID.Namespace, changefeedID.ID) - metricSorterMergerStartTsGauge := sorterMergerStartTsGauge. - WithLabelValues(changefeedID.Namespace, changefeedID.ID) - metricSorterMergeCountHistogram := sorterMergeCountHistogram. - WithLabelValues(changefeedID.Namespace, changefeedID.ID) - - lastResolvedTs := make([]uint64, numSorters) - minResolvedTs := uint64(0) - var workingSet map[*flushTask]struct{} - pendingSet := &sync.Map{} - - defer func() { - log.Debug("Unified Sorter: merger exiting, cleaning up resources") - // cancel pending async IO operations. - onExit() - cleanUpTask := func(task *flushTask) { - select { - case err := <-task.finished: - _ = printError(err) - default: - // The task has not finished, so we give up. - // It does not matter because: - // 1) if the async workerpool has exited, it means the CDC process is exiting, CleanUp will - // take care of the temp files, - // 2) if the async workerpool is not exiting, the unfinished tasks will eventually be executed, - // and by that time, since the `onExit` have canceled them, they will not do any IO and clean up themselves. - return - } - - if task.reader != nil { - _ = printError(task.reader.resetAndClose()) - task.reader = nil - } - _ = printError(task.dealloc()) - } - - LOOP: - for { - var task *flushTask - select { - case task = <-in: - default: - break LOOP - } - - if task == nil { - log.Debug("Merger exiting, in-channel is exhausted") - break - } - - cleanUpTask(task) - } - - pendingSet.Range(func(task, _ interface{}) bool { - cleanUpTask(task.(*flushTask)) - return true - }) - for task := range workingSet { - cleanUpTask(task) - } - }() - - lastOutputTs := uint64(0) - lastOutputResolvedTs := uint64(0) - var lastEvent *model.PolymorphicEvent - var lastTask *flushTask - - sendResolvedEvent := func(ts uint64) error { - lastOutputResolvedTs = ts - if ts == 0 { - return nil - } - select { - case <-ctx.Done(): - return ctx.Err() - case out <- model.NewResolvedPolymorphicEvent(0, ts): - metricSorterEventCount.WithLabelValues("resolved").Inc() - metricSorterResolvedTsGauge.Set(float64(oracle.ExtractPhysical(ts))) - return nil - } - } - - onMinResolvedTsUpdate := func(minResolvedTs /* note the shadowing */ uint64) error { - metricSorterMergerStartTsGauge.Set(float64(oracle.ExtractPhysical(minResolvedTs))) - workingSet = make(map[*flushTask]struct{}) - sortHeap := new(sortHeap) - - // loopErr is used to return an error out of the closure taken by `pendingSet.Range`. - var loopErr error - // NOTE 1: We can block the closure passed to `pendingSet.Range` WITHOUT worrying about - // deadlocks because the closure is NOT called with any lock acquired in the implementation - // of Sync.Map. - // NOTE 2: It is safe to used `Range` to iterate through the pendingSet, in spite of NOT having - // a snapshot consistency because (1) pendingSet is updated first before minResolvedTs is updated, - // which guarantees that useful new flushTasks are not missed, and (2) by design, once minResolvedTs is updated, - // new flushTasks will satisfy `task.tsLowerBound > minResolvedTs`, and such flushTasks are ignored in - // the closure. - pendingSet.Range(func(iTask, iCache interface{}) bool { - task := iTask.(*flushTask) - var cache *model.PolymorphicEvent - if iCache != nil { - cache = iCache.(*model.PolymorphicEvent) - } - - if task.tsLowerBound > minResolvedTs { - // the condition above implies that for any event in task.backend, CRTs > minResolvedTs. - return true - } - var event *model.PolymorphicEvent - if cache != nil { - event = cache - } else { - select { - case <-ctx.Done(): - loopErr = ctx.Err() - // terminates the loop - return false - case err := <-task.finished: - if err != nil { - loopErr = errors.Trace(err) - // terminates the loop - return false - } - } - - if task.reader == nil { - var err error - task.reader, err = task.GetBackEnd().reader() - if err != nil { - loopErr = errors.Trace(err) - // terminates the loop - return false - } - } - - var err error - event, err = task.reader.readNext() - if err != nil { - loopErr = errors.Trace(err) - // terminates the loop - return false - } - - if event == nil { - log.Panic("Unexpected end of backEnd data, bug?", - zap.Uint64("minResolvedTs", task.maxResolvedTs)) - } - } - - if event.CRTs > minResolvedTs { - pendingSet.Store(task, event) - // continues the loop - return true - } - - pendingSet.Store(task, nil) - workingSet[task] = struct{}{} - - heap.Push(sortHeap, &sortItem{ - entry: event, - data: task, - }) - return true - }) - if loopErr != nil { - return errors.Trace(loopErr) - } - - resolvedTicker := time.NewTicker(1 * time.Second) - defer resolvedTicker.Stop() - - retire := func(task *flushTask) error { - delete(workingSet, task) - cached, ok := pendingSet.Load(task) - if !ok { - log.Panic("task not found in pendingSet") - } - - if cached != nil { - return nil - } - - nextEvent, err := task.reader.readNext() - if err != nil { - _ = task.reader.resetAndClose() // prevents fd leak - task.reader = nil - return errors.Trace(err) - } - - if nextEvent == nil { - pendingSet.Delete(task) - - err := task.reader.resetAndClose() - if err != nil { - return errors.Trace(err) - } - task.reader = nil - - err = task.dealloc() - if err != nil { - return errors.Trace(err) - } - } else { - pendingSet.Store(task, nextEvent) - if nextEvent.CRTs < minResolvedTs { - log.Panic("remaining event CRTs too small", - zap.Uint64("nextTs", nextEvent.CRTs), - zap.Uint64("minResolvedTs", minResolvedTs)) - } - } - return nil - } - - failpoint.Inject("sorterDebug", func() { - if sortHeap.Len() > 0 { - tableID, tableName := contextutil.TableIDFromCtx(ctx) - log.Debug("Unified Sorter: start merging", - zap.Int64("tableID", tableID), - zap.String("tableName", tableName), - zap.Uint64("minResolvedTs", minResolvedTs)) - } - }) - - counter := 0 - for sortHeap.Len() > 0 { - failpoint.Inject("sorterMergeDelay", func() {}) - - item := heap.Pop(sortHeap).(*sortItem) - task := item.data.(*flushTask) - event := item.entry - - if event.CRTs < task.lastTs { - log.Panic("unified sorter: ts regressed in one backEnd, bug?", zap.Uint64("curTs", event.CRTs), zap.Uint64("lastTs", task.lastTs)) - } - task.lastTs = event.CRTs - - if event.RawKV != nil && event.RawKV.OpType != model.OpTypeResolved { - if event.CRTs < lastOutputTs { - for sortHeap.Len() > 0 { - item := heap.Pop(sortHeap).(*sortItem) - task := item.data.(*flushTask) - event := item.entry - log.Debug("dump", zap.Reflect("event", event), zap.Int("heapID", task.heapSorterID)) - } - log.Panic("unified sorter: output ts regressed, bug?", - zap.Int("counter", counter), - zap.Uint64("minResolvedTs", minResolvedTs), - zap.Int("curHeapID", task.heapSorterID), - zap.Int("curTaskID", task.taskID), - zap.Uint64("curTaskResolved", task.maxResolvedTs), - zap.Reflect("curEvent", event), - zap.Uint64("curTs", event.CRTs), - zap.Int("lastHeapID", lastTask.heapSorterID), - zap.Int("lastTaskID", lastTask.taskID), - zap.Uint64("lastTaskResolved", task.maxResolvedTs), - zap.Reflect("lastEvent", lastEvent), - zap.Uint64("lastTs", lastOutputTs), - zap.Int("sortHeapLen", sortHeap.Len())) - } - - if event.CRTs <= lastOutputResolvedTs { - log.Panic("unified sorter: output ts smaller than resolved ts, bug?", zap.Uint64("minResolvedTs", minResolvedTs), - zap.Uint64("lastOutputResolvedTs", lastOutputResolvedTs), zap.Uint64("eventCrts", event.CRTs)) - } - lastOutputTs = event.CRTs - lastEvent = event - lastTask = task - select { - case <-ctx.Done(): - return ctx.Err() - case out <- event: - metricSorterEventCount.WithLabelValues("kv").Inc() - } - } - counter += 1 - - select { - case <-resolvedTicker.C: - err := sendResolvedEvent(event.CRTs - 1) - if err != nil { - return errors.Trace(err) - } - default: - } - - event, err := task.reader.readNext() - if err != nil { - return errors.Trace(err) - } - - if event == nil { - // EOF - delete(workingSet, task) - pendingSet.Delete(task) - - err := task.reader.resetAndClose() - if err != nil { - return errors.Trace(err) - } - task.reader = nil - - err = task.dealloc() - if err != nil { - return errors.Trace(err) - } - - continue - } - - if event.CRTs > minResolvedTs || (event.CRTs == minResolvedTs && event.IsResolved()) { - // we have processed all events from this task that need to be processed in this merge - if event.CRTs > minResolvedTs || event.RawKV.OpType != model.OpTypeResolved { - pendingSet.Store(task, event) - } - err := retire(task) - if err != nil { - return errors.Trace(err) - } - continue - } - - failpoint.Inject("sorterDebug", func() { - if counter%10 == 0 { - tableID, tableName := contextutil.TableIDFromCtx(ctx) - log.Debug("Merging progress", - zap.Int64("tableID", tableID), - zap.String("tableName", tableName), - zap.Int("counter", counter)) - } - }) - - heap.Push(sortHeap, &sortItem{ - entry: event, - data: task, - }) - } - - if len(workingSet) != 0 { - log.Panic("unified sorter: merging ended prematurely, bug?", zap.Uint64("resolvedTs", minResolvedTs)) - } - - failpoint.Inject("sorterDebug", func() { - if counter > 0 { - tableID, tableName := contextutil.TableIDFromCtx(ctx) - log.Debug("Unified Sorter: merging ended", - zap.Int64("tableID", tableID), - zap.String("tableName", tableName), - zap.Uint64("resolvedTs", minResolvedTs), zap.Int("count", counter)) - } - }) - err := sendResolvedEvent(minResolvedTs) - if err != nil { - return errors.Trace(err) - } - - if counter > 0 { - // ignore empty merges for better visualization of metrics - metricSorterMergeCountHistogram.Observe(float64(counter)) - } - - return nil - } - - resolvedTsNotifierChan := make(chan struct{}, 1) - errg, ctx := errgroup.WithContext(ctx) - - errg.Go(func() error { - for { - var task *flushTask - select { - case <-ctx.Done(): - return ctx.Err() - case task = <-in: - } - - if task == nil { - tableID, tableName := contextutil.TableIDFromCtx(ctx) - log.Debug("Merger input channel closed, exiting", - zap.Int64("tableID", tableID), - zap.String("tableName", tableName)) - return nil - } - - if !task.isEmpty { - pendingSet.Store(task, nil) - } // otherwise it is an empty flush - - if lastResolvedTs[task.heapSorterID] < task.maxResolvedTs { - lastResolvedTs[task.heapSorterID] = task.maxResolvedTs - } - - minTemp := uint64(math.MaxUint64) - for _, ts := range lastResolvedTs { - if minTemp > ts { - minTemp = ts - } - } - - if minTemp > minResolvedTs { - atomic.StoreUint64(&minResolvedTs, minTemp) - select { - case resolvedTsNotifierChan <- struct{}{}: - default: - } - } - } - }) - - errg.Go(func() error { - resolvedTsTicker := time.NewTicker(time.Second * 1) - - defer resolvedTsTicker.Stop() - - var lastResolvedTs uint64 - resolvedTsTickFunc := func() error { - curResolvedTs := atomic.LoadUint64(&minResolvedTs) - if curResolvedTs > lastResolvedTs { - err := onMinResolvedTsUpdate(curResolvedTs) - if err != nil { - return errors.Trace(err) - } - } else if curResolvedTs < lastResolvedTs { - log.Panic("resolved-ts regressed in sorter", - zap.Uint64("curResolvedTs", curResolvedTs), - zap.Uint64("lastResolvedTs", lastResolvedTs)) - } - return nil - } - - for { - select { - case <-ctx.Done(): - return ctx.Err() - case <-resolvedTsTicker.C: - if err := resolvedTsTickFunc(); err != nil { - return err - } - case <-resolvedTsNotifierChan: - if err := resolvedTsTickFunc(); err != nil { - return err - } - } - } - }) - - return errg.Wait() -} - -func mergerCleanUp(in <-chan *flushTask) { - for task := range in { - select { - case err := <-task.finished: - _ = printError(err) - default: - break - } - - if task.reader != nil { - _ = printError(task.reader.resetAndClose()) - } - _ = printError(task.dealloc()) - } -} - -// printError is a helper for tracing errors on function returns -func printError(err error) error { - if err != nil && errors.Cause(err) != context.Canceled && - errors.Cause(err) != context.DeadlineExceeded && - !strings.Contains(err.Error(), "context canceled") && - !strings.Contains(err.Error(), "context deadline exceeded") && - cerrors.ErrAsyncIOCancelled.NotEqual(errors.Cause(err)) { - - log.Warn("Unified Sorter: Error detected", zap.Error(err), zap.Stack("stack")) - } - return err -} diff --git a/cdc/sorter/unified/merger_test.go b/cdc/sorter/unified/merger_test.go deleted file mode 100644 index a3634b2bda9..00000000000 --- a/cdc/sorter/unified/merger_test.go +++ /dev/null @@ -1,546 +0,0 @@ -// Copyright 2021 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package unified - -import ( - "context" - "sync/atomic" - "testing" - "time" - - "github.com/pingcap/failpoint" - "github.com/pingcap/log" - "github.com/pingcap/tiflow/cdc/model" - "github.com/stretchr/testify/require" - "go.uber.org/zap" - "go.uber.org/zap/zapcore" - "golang.org/x/sync/errgroup" -) - -type mockFlushTaskBuilder struct { - task *flushTask - writer backEndWriter - totalCount int -} - -var backEndCounterForTest int64 - -func newMockFlushTaskBuilder() *mockFlushTaskBuilder { - backEnd := newMemoryBackEnd() - atomic.AddInt64(&backEndCounterForTest, 1) - - task := &flushTask{ - backend: backEnd, - tsLowerBound: 0, - maxResolvedTs: 0, - finished: make(chan error, 2), - } - - task.dealloc = func() error { - if task.backend != nil { - atomic.AddInt64(&backEndCounterForTest, -1) - task.backend = nil - return backEnd.free() - } - return nil - } - - writer, _ := backEnd.writer() - - return &mockFlushTaskBuilder{ - task: task, - writer: writer, - } -} - -func (b *mockFlushTaskBuilder) generateRowChanges(tsRangeBegin, tsRangeEnd uint64, count int) *mockFlushTaskBuilder { - if b.task.tsLowerBound == 0 { - b.task.tsLowerBound = tsRangeBegin - } - density := float64(tsRangeEnd-tsRangeBegin) / float64(count) - for fTs := float64(tsRangeBegin); fTs < float64(tsRangeEnd); fTs += density { - ts := uint64(fTs) - kvEntry := generateMockRawKV(ts) - _ = b.writer.writeNext(model.NewPolymorphicEvent(kvEntry)) - b.totalCount++ - } - return b -} - -func (b *mockFlushTaskBuilder) addResolved(ts uint64) *mockFlushTaskBuilder { - _ = b.writer.writeNext(model.NewResolvedPolymorphicEvent(0, ts)) - b.task.maxResolvedTs = ts - return b -} - -func (b *mockFlushTaskBuilder) build() *flushTask { - _ = b.writer.flushAndClose() - return b.task -} - -// TestMergerSingleHeap simulates a situation where there is only one data stream -// It tests the most basic scenario. -func TestMergerSingleHeap(t *testing.T) { - err := failpoint.Enable("github.com/pingcap/tiflow/cdc/sorter/unified/sorterDebug", "return(true)") - if err != nil { - log.Panic("Could not enable failpoint", zap.Error(err)) - } - - ctx, cancel := context.WithTimeout(context.TODO(), time.Second*10) - defer cancel() - wg, ctx := errgroup.WithContext(ctx) - inChan := make(chan *flushTask, 1024) - outChan := make(chan *model.PolymorphicEvent, 1024) - - wg.Go(func() error { - return runMerger(ctx, 1, inChan, outChan, func() {}) - }) - - totalCount := 0 - builder := newMockFlushTaskBuilder() - task1 := builder.generateRowChanges(1000, 100000, 2048).addResolved(100001).build() - totalCount += builder.totalCount - builder = newMockFlushTaskBuilder() - task2 := builder.generateRowChanges(100002, 200000, 2048).addResolved(200001).build() - totalCount += builder.totalCount - builder = newMockFlushTaskBuilder() - task3 := builder.generateRowChanges(200002, 300000, 2048).addResolved(300001).build() - totalCount += builder.totalCount - - wg.Go(func() error { - inChan <- task1 - close(task1.finished) - inChan <- task2 - close(task2.finished) - inChan <- task3 - close(task3.finished) - - return nil - }) - - wg.Go(func() error { - count := 0 - lastTs := uint64(0) - lastResolved := uint64(0) - for { - select { - case <-ctx.Done(): - return ctx.Err() - case event := <-outChan: - switch event.RawKV.OpType { - case model.OpTypePut: - count++ - require.GreaterOrEqual(t, event.CRTs, lastTs) - require.GreaterOrEqual(t, event.CRTs, lastResolved) - lastTs = event.CRTs - case model.OpTypeResolved: - require.GreaterOrEqual(t, event.CRTs, lastResolved) - lastResolved = event.CRTs - } - if lastResolved >= 300001 { - require.Equal(t, totalCount, count) - cancel() - return nil - } - } - } - }) - require.Regexp(t, ".*context canceled.*", wg.Wait()) - require.Equal(t, int64(0), atomic.LoadInt64(&backEndCounterForTest)) -} - -// TestMergerSingleHeapRetire simulates a situation where the resolved event is not the last event in a flushTask -func TestMergerSingleHeapRetire(t *testing.T) { - err := failpoint.Enable("github.com/pingcap/tiflow/cdc/sorter/unified/sorterDebug", "return(true)") - if err != nil { - log.Panic("Could not enable failpoint", zap.Error(err)) - } - - ctx, cancel := context.WithTimeout(context.TODO(), time.Second*10) - defer cancel() - wg, ctx := errgroup.WithContext(ctx) - inChan := make(chan *flushTask, 1024) - outChan := make(chan *model.PolymorphicEvent, 1024) - - wg.Go(func() error { - return runMerger(ctx, 1, inChan, outChan, func() {}) - }) - - totalCount := 0 - builder := newMockFlushTaskBuilder() - task1 := builder.generateRowChanges(1000, 100000, 2048).addResolved(100001).build() - totalCount += builder.totalCount - builder = newMockFlushTaskBuilder() - task2 := builder.generateRowChanges(100002, 200000, 2048).build() - totalCount += builder.totalCount - builder = newMockFlushTaskBuilder() - task3 := builder.generateRowChanges(200002, 300000, 2048).addResolved(300001).build() - totalCount += builder.totalCount - - wg.Go(func() error { - inChan <- task1 - close(task1.finished) - inChan <- task2 - close(task2.finished) - inChan <- task3 - close(task3.finished) - - return nil - }) - - wg.Go(func() error { - count := 0 - lastTs := uint64(0) - lastResolved := uint64(0) - for { - select { - case <-ctx.Done(): - return ctx.Err() - case event := <-outChan: - switch event.RawKV.OpType { - case model.OpTypePut: - count++ - require.GreaterOrEqual(t, event.CRTs, lastResolved) - require.GreaterOrEqual(t, event.CRTs, lastTs) - lastTs = event.CRTs - case model.OpTypeResolved: - require.GreaterOrEqual(t, event.CRTs, lastResolved) - lastResolved = event.CRTs - } - if lastResolved >= 300001 { - require.Equal(t, totalCount, count) - cancel() - return nil - } - } - } - }) - - require.Regexp(t, ".*context canceled.*", wg.Wait()) - require.Equal(t, int64(0), atomic.LoadInt64(&backEndCounterForTest)) -} - -// TestMergerSortDelay simulates a situation where merging takes a long time. -// Expects intermediate resolved events to be generated, so that the sink would not get stuck in a real life situation. -func TestMergerSortDelay(t *testing.T) { - err := failpoint.Enable("github.com/pingcap/tiflow/cdc/sorter/unified/sorterDebug", "return(true)") - require.Nil(t, err) - - // enable the failpoint to simulate delays - err = failpoint.Enable("github.com/pingcap/tiflow/cdc/sorter/unified/sorterMergeDelay", "sleep(5)") - require.Nil(t, err) - defer func() { - _ = failpoint.Disable("github.com/pingcap/tiflow/cdc/sorter/unified/sorterMergeDelay") - }() - - log.SetLevel(zapcore.DebugLevel) - defer log.SetLevel(zapcore.InfoLevel) - - ctx, cancel := context.WithTimeout(context.TODO(), time.Second*10) - defer cancel() - wg, ctx := errgroup.WithContext(ctx) - inChan := make(chan *flushTask, 1024) - outChan := make(chan *model.PolymorphicEvent, 1024) - - wg.Go(func() error { - return runMerger(ctx, 1, inChan, outChan, func() {}) - }) - - totalCount := 0 - builder := newMockFlushTaskBuilder() - task1 := builder.generateRowChanges(1000, 1000000, 1024).addResolved(1000001).build() - totalCount += builder.totalCount - - wg.Go(func() error { - inChan <- task1 - close(task1.finished) - return nil - }) - - wg.Go(func() error { - var ( - count int - lastTs uint64 - lastResolved uint64 - lastResolvedTime time.Time - ) - for { - select { - case <-ctx.Done(): - return ctx.Err() - case event := <-outChan: - switch event.RawKV.OpType { - case model.OpTypePut: - count++ - require.GreaterOrEqual(t, event.CRTs, lastResolved) - require.GreaterOrEqual(t, event.CRTs, lastTs) - lastTs = event.CRTs - case model.OpTypeResolved: - require.GreaterOrEqual(t, event.CRTs, lastResolved) - if !lastResolvedTime.IsZero() { - require.LessOrEqual(t, time.Since(lastResolvedTime), 2*time.Second) - } - log.Debug("resolved event received", zap.Uint64("ts", event.CRTs)) - lastResolvedTime = time.Now() - lastResolved = event.CRTs - } - if lastResolved >= 1000001 { - require.Equal(t, totalCount, count) - cancel() - return nil - } - } - } - }) - - require.Regexp(t, ".*context canceled.*", wg.Wait()) - close(inChan) - mergerCleanUp(inChan) - require.Equal(t, int64(0), atomic.LoadInt64(&backEndCounterForTest)) -} - -// TestMergerCancel simulates a situation where the merger is cancelled with pending data. -// Expects proper clean-up of the data. -func TestMergerCancel(t *testing.T) { - err := failpoint.Enable("github.com/pingcap/tiflow/cdc/sorter/unified/sorterDebug", "return(true)") - require.Nil(t, err) - - // enable the failpoint to simulate delays - err = failpoint.Enable("github.com/pingcap/tiflow/cdc/sorter/unified/sorterMergeDelay", "sleep(10)") - require.Nil(t, err) - defer func() { - _ = failpoint.Disable("github.com/pingcap/tiflow/cdc/sorter/unified/sorterMergeDelay") - }() - - log.SetLevel(zapcore.DebugLevel) - defer log.SetLevel(zapcore.InfoLevel) - - ctx, cancel := context.WithTimeout(context.TODO(), time.Second*10) - defer cancel() - wg, ctx := errgroup.WithContext(ctx) - inChan := make(chan *flushTask, 1024) - outChan := make(chan *model.PolymorphicEvent, 1024) - - wg.Go(func() error { - return runMerger(ctx, 1, inChan, outChan, func() {}) - }) - - builder := newMockFlushTaskBuilder() - task1 := builder.generateRowChanges(1000, 100000, 2048).addResolved(100001).build() - builder = newMockFlushTaskBuilder() - task2 := builder.generateRowChanges(100002, 200000, 2048).addResolved(200001).build() - builder = newMockFlushTaskBuilder() - task3 := builder.generateRowChanges(200002, 300000, 2048).addResolved(300001).build() - - wg.Go(func() error { - inChan <- task1 - close(task1.finished) - inChan <- task2 - close(task2.finished) - inChan <- task3 - close(task3.finished) - return nil - }) - - wg.Go(func() error { - for { - select { - case <-ctx.Done(): - return ctx.Err() - case <-outChan: - // We just drain the data here. We don't care about it. - } - } - }) - - time.Sleep(5 * time.Second) - cancel() - require.Regexp(t, ".*context canceled.*", wg.Wait()) - close(inChan) - mergerCleanUp(inChan) - require.Equal(t, int64(0), atomic.LoadInt64(&backEndCounterForTest)) -} - -// TestMergerCancel simulates a situation where the merger is cancelled with pending data. -// Expects proper clean-up of the data. -func TestMergerCancelWithUnfinishedFlushTasks(t *testing.T) { - err := failpoint.Enable("github.com/pingcap/tiflow/cdc/sorter/unified/sorterDebug", "return(true)") - require.Nil(t, err) - - log.SetLevel(zapcore.DebugLevel) - defer log.SetLevel(zapcore.InfoLevel) - - ctx, cancel := context.WithTimeout(context.TODO(), time.Second*10) - wg, ctx := errgroup.WithContext(ctx) - inChan := make(chan *flushTask, 1024) - outChan := make(chan *model.PolymorphicEvent, 1024) - - wg.Go(func() error { - return runMerger(ctx, 1, inChan, outChan, func() {}) - }) - - builder := newMockFlushTaskBuilder() - task1 := builder.generateRowChanges(1000, 100000, 2048).addResolved(100001).build() - builder = newMockFlushTaskBuilder() - task2 := builder.generateRowChanges(100002, 200000, 2048).addResolved(200001).build() - builder = newMockFlushTaskBuilder() - task3 := builder.generateRowChanges(200002, 300000, 2048).addResolved(300001).build() - - wg.Go(func() error { - inChan <- task1 - inChan <- task2 - inChan <- task3 - close(task2.finished) - close(task1.finished) - time.Sleep(1 * time.Second) - cancel() - return nil - }) - - wg.Go(func() error { - for { - select { - case <-ctx.Done(): - return ctx.Err() - case <-outChan: - // We just drain the data here. We don't care about it. - } - } - }) - - require.Regexp(t, ".*context canceled.*", wg.Wait()) - close(inChan) - mergerCleanUp(inChan) - // Leaking one task is expected - require.Equal(t, int64(1), atomic.LoadInt64(&backEndCounterForTest)) - atomic.StoreInt64(&backEndCounterForTest, 0) -} - -// TestMergerCancel simulates a situation where the input channel is abruptly closed. -// There is expected to be NO fatal error. -func TestMergerCloseChannel(t *testing.T) { - err := failpoint.Enable("github.com/pingcap/tiflow/cdc/sorter/unified/sorterDebug", "return(true)") - require.Nil(t, err) - - log.SetLevel(zapcore.DebugLevel) - defer log.SetLevel(zapcore.InfoLevel) - - ctx, cancel := context.WithTimeout(context.TODO(), time.Second*15) - defer cancel() - wg, ctx := errgroup.WithContext(ctx) - inChan := make(chan *flushTask, 1024) - outChan := make(chan *model.PolymorphicEvent, 1024) - - builder := newMockFlushTaskBuilder() - task1 := builder.generateRowChanges(1000, 100000, 2048).addResolved(100001).build() - - inChan <- task1 - close(task1.finished) - - wg.Go(func() error { - return runMerger(ctx, 1, inChan, outChan, func() {}) - }) - - wg.Go(func() error { - for { - select { - case <-ctx.Done(): - return ctx.Err() - case <-outChan: - // We just drain the data here. We don't care about it. - } - } - }) - - time.Sleep(5 * time.Second) - close(inChan) - time.Sleep(5 * time.Second) - cancel() - require.Regexp(t, ".*context canceled.*", wg.Wait()) - mergerCleanUp(inChan) - require.Equal(t, int64(0), atomic.LoadInt64(&backEndCounterForTest)) -} - -// TestMergerOutputBlocked simulates a situation where the output channel is blocked for -// a significant period of time. -func TestMergerOutputBlocked(t *testing.T) { - err := failpoint.Enable("github.com/pingcap/tiflow/cdc/sorter/unified/sorterDebug", "return(true)") - require.Nil(t, err) - defer failpoint.Disable("github.com/pingcap/tiflow/cdc/sorter/unified/sorterDebug") //nolint:errcheck - - ctx, cancel := context.WithTimeout(context.TODO(), time.Second*25) - defer cancel() - wg, ctx := errgroup.WithContext(ctx) - // use unbuffered channel to make sure that the input has been processed - inChan := make(chan *flushTask) - // make a small channel to test blocking - outChan := make(chan *model.PolymorphicEvent, 1) - - wg.Go(func() error { - return runMerger(ctx, 1, inChan, outChan, func() {}) - }) - - totalCount := 0 - builder := newMockFlushTaskBuilder() - task1 := builder.generateRowChanges(1000, 100000, 2048).addResolved(100001).build() - totalCount += builder.totalCount - builder = newMockFlushTaskBuilder() - task2 := builder.generateRowChanges(100002, 200000, 2048).addResolved(200001).build() - totalCount += builder.totalCount - builder = newMockFlushTaskBuilder() - task3 := builder.generateRowChanges(200002, 300000, 2048).addResolved(300001).build() - totalCount += builder.totalCount - - wg.Go(func() error { - inChan <- task1 - close(task1.finished) - inChan <- task2 - close(task2.finished) - inChan <- task3 - close(task3.finished) - - return nil - }) - - wg.Go(func() error { - time.Sleep(10 * time.Second) - count := 0 - lastTs := uint64(0) - lastResolved := uint64(0) - for { - select { - case <-ctx.Done(): - return ctx.Err() - case event := <-outChan: - switch event.RawKV.OpType { - case model.OpTypePut: - count++ - require.GreaterOrEqual(t, event.CRTs, lastTs) - require.GreaterOrEqual(t, event.CRTs, lastResolved) - lastTs = event.CRTs - case model.OpTypeResolved: - require.GreaterOrEqual(t, event.CRTs, lastResolved) - lastResolved = event.CRTs - } - if lastResolved >= 300001 { - require.Equal(t, totalCount, count) - cancel() - return nil - } - } - } - }) - require.Regexp(t, ".*context canceled.*", wg.Wait()) - require.Equal(t, int64(0), atomic.LoadInt64(&backEndCounterForTest)) -} diff --git a/cdc/sorter/unified/metrics.go b/cdc/sorter/unified/metrics.go deleted file mode 100644 index f1c552ff128..00000000000 --- a/cdc/sorter/unified/metrics.go +++ /dev/null @@ -1,58 +0,0 @@ -// Copyright 2020 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 unified - -import ( - "github.com/prometheus/client_golang/prometheus" -) - -var ( - sorterConsumeCount = prometheus.NewCounterVec(prometheus.CounterOpts{ - Namespace: "ticdc", - Subsystem: "sorter", - Name: "consume_count", - Help: "the number of events consumed by the sorter", - }, []string{"namespace", "changefeed", "type"}) - - sorterMergerStartTsGauge = prometheus.NewGaugeVec(prometheus.GaugeOpts{ - Namespace: "ticdc", - Subsystem: "sorter", - Name: "merger_start_ts_gauge", - Help: "the start TS of each merge in the sorter", - }, []string{"namespace", "changefeed"}) - - sorterFlushCountHistogram = prometheus.NewHistogramVec(prometheus.HistogramOpts{ - Namespace: "ticdc", - Subsystem: "sorter", - Name: "flush_count_histogram", - Help: "Bucketed histogram of the number of events in individual flushes performed by the sorter", - Buckets: prometheus.ExponentialBuckets(4, 4, 10), - }, []string{"namespace", "changefeed"}) - - sorterMergeCountHistogram = prometheus.NewHistogramVec(prometheus.HistogramOpts{ - Namespace: "ticdc", - Subsystem: "sorter", - Name: "merge_count_histogram", - Help: "Bucketed histogram of the number of events in individual merges performed by the sorter", - Buckets: prometheus.ExponentialBuckets(16, 4, 10), - }, []string{"namespace", "changefeed"}) -) - -// InitMetrics registers all metrics in this file -func InitMetrics(registry *prometheus.Registry) { - registry.MustRegister(sorterConsumeCount) - registry.MustRegister(sorterMergerStartTsGauge) - registry.MustRegister(sorterFlushCountHistogram) - registry.MustRegister(sorterMergeCountHistogram) -} diff --git a/cdc/sorter/unified/sorter_test.go b/cdc/sorter/unified/sorter_test.go deleted file mode 100644 index d8533b41b90..00000000000 --- a/cdc/sorter/unified/sorter_test.go +++ /dev/null @@ -1,469 +0,0 @@ -// Copyright 2020 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 unified - -import ( - "context" - "math" - _ "net/http/pprof" - "os" - "path/filepath" - "sync/atomic" - "testing" - "time" - - "github.com/pingcap/failpoint" - "github.com/pingcap/log" - "github.com/pingcap/tiflow/cdc/model" - "github.com/pingcap/tiflow/cdc/sorter" - "github.com/pingcap/tiflow/pkg/config" - "github.com/stretchr/testify/require" - "go.uber.org/zap" - "go.uber.org/zap/zapcore" - "golang.org/x/sync/errgroup" -) - -const ( - numProducers = 16 -) - -func generateMockRawKV(ts uint64) *model.RawKVEntry { - return &model.RawKVEntry{ - OpType: model.OpTypePut, - Key: []byte{}, - Value: []byte{}, - OldValue: nil, - StartTs: ts - 5, - CRTs: ts, - RegionID: 0, - } -} - -func TestSorterBasic(t *testing.T) { - defer CleanUp() - - conf := config.GetDefaultServerConfig() - conf.DataDir = t.TempDir() - sortDir := filepath.Join(conf.DataDir, config.DefaultSortDir) - conf.Sorter = &config.SorterConfig{ - NumConcurrentWorker: 8, - ChunkSizeLimit: 1 * 1024 * 1024 * 1024, - MaxMemoryPercentage: 60, - MaxMemoryConsumption: 16 * 1024 * 1024 * 1024, - NumWorkerPoolGoroutine: 4, - SortDir: sortDir, - } - config.StoreGlobalServerConfig(conf) - - err := os.MkdirAll(conf.Sorter.SortDir, 0o755) - require.Nil(t, err) - sorter, err := NewUnifiedSorter(conf.Sorter.SortDir, - model.DefaultChangeFeedID("test-cf"), "test", 0) - require.Nil(t, err) - - ctx, cancel := context.WithTimeout(context.Background(), 5*time.Minute) - defer cancel() - err = testSorter(ctx, t, sorter, 10000) - require.Regexp(t, ".*context cancel.*", err) -} - -func TestSorterCancel(t *testing.T) { - defer CleanUp() - - conf := config.GetDefaultServerConfig() - conf.DataDir = t.TempDir() - sortDir := filepath.Join(conf.DataDir, config.DefaultSortDir) - conf.Sorter = &config.SorterConfig{ - NumConcurrentWorker: 8, - ChunkSizeLimit: 1 * 1024 * 1024 * 1024, - MaxMemoryPercentage: 60, - MaxMemoryConsumption: 0, - NumWorkerPoolGoroutine: 4, - SortDir: sortDir, - } - config.StoreGlobalServerConfig(conf) - - err := os.MkdirAll(conf.Sorter.SortDir, 0o755) - require.Nil(t, err) - sorter, err := NewUnifiedSorter(conf.Sorter.SortDir, - model.DefaultChangeFeedID("cf-1"), - "test", 0) - require.Nil(t, err) - - ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) - defer cancel() - - finishedCh := make(chan struct{}) - go func() { - err := testSorter(ctx, t, sorter, 10000000) - require.Regexp(t, ".*context deadline exceeded.*", err) - close(finishedCh) - }() - - after := time.After(30 * time.Second) - select { - case <-after: - t.Fatal("TestSorterCancel timed out") - case <-finishedCh: - } - - log.Info("Sorter successfully cancelled") -} - -func testSorter(ctx context.Context, t *testing.T, sorter sorter.EventSorter, count int) error { - err := failpoint.Enable("github.com/pingcap/tiflow/cdc/sorter/unified/sorterDebug", "return(true)") - if err != nil { - log.Panic("Could not enable failpoint", zap.Error(err)) - } - - p := "github.com/pingcap/tiflow/pkg/util/InjectCheckDataDirSatisfied" - require.Nil(t, failpoint.Enable(p, "")) - defer func() { - require.Nil(t, failpoint.Disable(p)) - }() - - ctx, cancel := context.WithCancel(ctx) - errg, ctx := errgroup.WithContext(ctx) - errg.Go(func() error { - return sorter.Run(ctx) - }) - errg.Go(func() error { - return RunWorkerPool(ctx) - }) - - producerProgress := make([]uint64, numProducers) - - // launch the producers - for i := 0; i < numProducers; i++ { - finalI := i - errg.Go(func() error { - for j := 1; j <= count; j++ { - select { - case <-ctx.Done(): - return ctx.Err() - default: - } - - sorter.AddEntry(ctx, model.NewPolymorphicEvent(generateMockRawKV(uint64(j)<<5))) - if j%10000 == 0 { - atomic.StoreUint64(&producerProgress[finalI], uint64(j)<<5) - } - } - sorter.AddEntry(ctx, model.NewPolymorphicEvent(generateMockRawKV(uint64(count+1)<<5))) - atomic.StoreUint64(&producerProgress[finalI], uint64(count+1)<<5) - return nil - }) - } - - // launch the resolver - errg.Go(func() error { - ticker := time.NewTicker(1 * time.Second) - defer ticker.Stop() - for { - select { - case <-ctx.Done(): - return ctx.Err() - case <-ticker.C: - resolvedTs := uint64(math.MaxUint64) - for i := range producerProgress { - ts := atomic.LoadUint64(&producerProgress[i]) - if resolvedTs > ts { - resolvedTs = ts - } - } - sorter.AddEntry(ctx, model.NewResolvedPolymorphicEvent(0, resolvedTs)) - if resolvedTs == uint64(count)<<5 { - return nil - } - } - } - }) - - // launch the consumer - errg.Go(func() error { - counter := 0 - lastTs := uint64(0) - ticker := time.NewTicker(1 * time.Second) - defer ticker.Stop() - for { - select { - case <-ctx.Done(): - return ctx.Err() - case event := <-sorter.Output(): - if event.RawKV.OpType != model.OpTypeResolved { - if event.CRTs < lastTs { - panic("regressed") - } - lastTs = event.CRTs - counter += 1 - if counter%10000 == 0 { - log.Debug("Messages received", zap.Int("counter", counter)) - } - if counter >= numProducers*count { - log.Debug("Unified Sorter test successful") - cancel() - } - } - case <-ticker.C: - log.Debug("Consumer is alive") - } - } - }) - - return errg.Wait() -} - -func TestSortDirConfigChangeFeed(t *testing.T) { - defer CleanUp() - - poolMu.Lock() - // Clean up the back-end pool if one has been created - pool = nil - poolMu.Unlock() - - dir := t.TempDir() - // We expect the changefeed setting to take effect - config.GetGlobalServerConfig().Sorter.SortDir = "" - - _, err := NewUnifiedSorter(dir, /* the changefeed setting */ - model.DefaultChangeFeedID("cf-1"), "test", 0) - require.Nil(t, err) - - poolMu.Lock() - defer poolMu.Unlock() - - require.NotNil(t, pool) - require.Equal(t, dir, pool.dir) -} - -// TestSorterCancelRestart tests the situation where the Unified Sorter is repeatedly canceled and -// restarted. There should not be any problem, especially file corruptions. -func TestSorterCancelRestart(t *testing.T) { - defer CleanUp() - - conf := config.GetDefaultServerConfig() - conf.DataDir = t.TempDir() - sortDir := filepath.Join(conf.DataDir, config.DefaultSortDir) - conf.Sorter = &config.SorterConfig{ - NumConcurrentWorker: 8, - ChunkSizeLimit: 1 * 1024 * 1024 * 1024, - MaxMemoryPercentage: 0, // disable memory sort - MaxMemoryConsumption: 0, - NumWorkerPoolGoroutine: 4, - SortDir: sortDir, - } - config.StoreGlobalServerConfig(conf) - - err := os.MkdirAll(conf.Sorter.SortDir, 0o755) - require.Nil(t, err) - - // enable the failpoint to simulate delays - err = failpoint.Enable("github.com/pingcap/tiflow/cdc/sorter/unified/asyncFlushStartDelay", "sleep(100)") - require.Nil(t, err) - defer func() { - _ = failpoint.Disable("github.com/pingcap/tiflow/cdc/sorter/unified/asyncFlushStartDelay") - }() - - // enable the failpoint to simulate delays - err = failpoint.Enable("github.com/pingcap/tiflow/cdc/sorter/unified/asyncFlushInProcessDelay", "1%sleep(1)") - require.Nil(t, err) - defer func() { - _ = failpoint.Disable("github.com/pingcap/tiflow/cdc/sorter/unified/asyncFlushInProcessDelay") - }() - - for i := 0; i < 5; i++ { - sorter, err := NewUnifiedSorter(conf.Sorter.SortDir, - model.DefaultChangeFeedID("cf-1"), - "test", 0) - require.Nil(t, err) - ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) - err = testSorter(ctx, t, sorter, 100000000) - require.Regexp(t, ".*context deadline exceeded.*", err) - cancel() - } -} - -func TestSorterIOError(t *testing.T) { - defer CleanUp() - - conf := config.GetDefaultServerConfig() - conf.DataDir = t.TempDir() - sortDir := filepath.Join(conf.DataDir, config.DefaultSortDir) - conf.Sorter = &config.SorterConfig{ - NumConcurrentWorker: 8, - ChunkSizeLimit: 1 * 1024 * 1024 * 1024, - MaxMemoryPercentage: 60, - MaxMemoryConsumption: 0, - NumWorkerPoolGoroutine: 4, - SortDir: sortDir, - } - config.StoreGlobalServerConfig(conf) - - err := os.MkdirAll(conf.Sorter.SortDir, 0o755) - require.Nil(t, err) - sorter, err := NewUnifiedSorter(conf.Sorter.SortDir, - model.DefaultChangeFeedID("cf-1"), - "test", 0) - require.Nil(t, err) - - ctx, cancel := context.WithTimeout(context.Background(), 60*time.Second) - defer cancel() - - // enable the failpoint to simulate backEnd allocation error (usually would happen when creating a file) - err = failpoint.Enable("github.com/pingcap/tiflow/cdc/sorter/unified/InjectErrorBackEndAlloc", "return(true)") - require.Nil(t, err) - defer func() { - _ = failpoint.Disable("github.com/pingcap/tiflow/cdc/sorter/unified/InjectErrorBackEndAlloc") - }() - - finishedCh := make(chan struct{}) - go func() { - err := testSorter(ctx, t, sorter, 10000) - require.Regexp(t, ".*injected alloc error.*", err) - close(finishedCh) - }() - - after := time.After(60 * time.Second) - select { - case <-after: - t.Fatal("TestSorterIOError timed out") - case <-finishedCh: - } - - CleanUp() - _ = failpoint.Disable("github.com/pingcap/tiflow/cdc/sorter/unified/InjectErrorBackEndAlloc") - // enable the failpoint to simulate backEnd write error (usually would happen when writing to a file) - err = failpoint.Enable("github.com/pingcap/tiflow/cdc/sorter/unified/InjectErrorBackEndWrite", "return(true)") - require.Nil(t, err) - defer func() { - _ = failpoint.Disable("github.com/pingcap/tiflow/cdc/sorter/unified/InjectErrorBackEndWrite") - }() - - // recreate the sorter - sorter, err = NewUnifiedSorter(conf.Sorter.SortDir, - model.DefaultChangeFeedID("cf-1"), "test", 0) - require.Nil(t, err) - - finishedCh = make(chan struct{}) - go func() { - err := testSorter(ctx, t, sorter, 10000) - require.Regexp(t, ".*injected write error.*", err) - close(finishedCh) - }() - - after = time.After(60 * time.Second) - select { - case <-after: - t.Fatal("TestSorterIOError timed out") - case <-finishedCh: - } -} - -func TestSorterErrorReportCorrect(t *testing.T) { - defer CleanUp() - - log.SetLevel(zapcore.DebugLevel) - defer log.SetLevel(zapcore.InfoLevel) - - conf := config.GetDefaultServerConfig() - conf.DataDir = t.TempDir() - sortDir := filepath.Join(conf.DataDir, config.DefaultSortDir) - conf.Sorter = &config.SorterConfig{ - NumConcurrentWorker: 8, - ChunkSizeLimit: 1 * 1024 * 1024 * 1024, - MaxMemoryPercentage: 60, - MaxMemoryConsumption: 0, - NumWorkerPoolGoroutine: 4, - SortDir: sortDir, - } - config.StoreGlobalServerConfig(conf) - - err := os.MkdirAll(conf.Sorter.SortDir, 0o755) - require.Nil(t, err) - sorter, err := NewUnifiedSorter(conf.Sorter.SortDir, - model.DefaultChangeFeedID("cf-1"), - "test", 0) - require.Nil(t, err) - - ctx, cancel := context.WithTimeout(context.Background(), 60*time.Second) - defer cancel() - - // enable the failpoint to simulate backEnd allocation error (usually would happen when creating a file) - err = failpoint.Enable("github.com/pingcap/tiflow/cdc/sorter/unified/InjectHeapSorterExitDelay", "sleep(2000)") - require.Nil(t, err) - defer func() { - _ = failpoint.Disable("github.com/pingcap/tiflow/cdc/sorter/unified/InjectHeapSorterExitDelay") - }() - - err = failpoint.Enable("github.com/pingcap/tiflow/cdc/sorter/unified/InjectErrorBackEndAlloc", "return(true)") - require.Nil(t, err) - defer func() { - _ = failpoint.Disable("github.com/pingcap/tiflow/cdc/sorter/unified/InjectErrorBackEndAlloc") - }() - - finishedCh := make(chan struct{}) - go func() { - err := testSorter(ctx, t, sorter, 10000) - require.Regexp(t, ".*injected alloc error.*", err) - close(finishedCh) - }() - - after := time.After(60 * time.Second) - select { - case <-after: - t.Fatal("TestSorterIOError timed out") - case <-finishedCh: - } -} - -func TestSortClosedAddEntry(t *testing.T) { - defer CleanUp() - - sorter, err := NewUnifiedSorter(t.TempDir(), - model.DefaultChangeFeedID("cf-1"), "test", 0) - require.Nil(t, err) - - ctx, cancel := context.WithTimeout(context.Background(), time.Millisecond*100) - defer cancel() - err = sorter.Run(ctx) - require.Regexp(t, ".*deadline.*", err) - - ctx1, cancel1 := context.WithTimeout(context.Background(), time.Second*10) - defer cancel1() - for i := 0; i < 10000; i++ { - sorter.AddEntry(ctx1, model.NewPolymorphicEvent(generateMockRawKV(uint64(i)))) - } - - select { - case <-ctx1.Done(): - t.Fatal("TestSortClosedAddEntry timed out") - default: - } - cancel1() -} - -func TestUnifiedSorterFileLockConflict(t *testing.T) { - defer CleanUp() - - dir := t.TempDir() - backEndPool, err := newBackEndPool(dir) - defer backEndPool.terminate() - require.Nil(t, err) - - // GlobalServerConfig overrides dir parameter in NewUnifiedSorter. - config.GetGlobalServerConfig().Sorter.SortDir = dir - _, err = NewUnifiedSorter(dir, - model.DefaultChangeFeedID("cf-1"), "test", 0) - require.Regexp(t, ".*file lock conflict.*", err) -} diff --git a/cdc/sorter/unified/unified_sorter.go b/cdc/sorter/unified/unified_sorter.go deleted file mode 100644 index a7773925c31..00000000000 --- a/cdc/sorter/unified/unified_sorter.go +++ /dev/null @@ -1,269 +0,0 @@ -// Copyright 2021 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package unified - -import ( - "context" - "sync" - - "github.com/pingcap/errors" - "github.com/pingcap/failpoint" - "github.com/pingcap/log" - "github.com/pingcap/tiflow/cdc/contextutil" - "github.com/pingcap/tiflow/cdc/model" - "github.com/pingcap/tiflow/cdc/sorter" - "github.com/pingcap/tiflow/pkg/config" - cerror "github.com/pingcap/tiflow/pkg/errors" - "github.com/pingcap/tiflow/pkg/util" - "golang.org/x/sync/errgroup" -) - -const ( - inputChSize = 128 - outputChSize = 128 - heapCollectChSize = 128 // this should be not be too small, to guarantee IO concurrency -) - -// Sorter provides both sorting in memory and in file. Memory pressure is used to determine which one to use. -type Sorter struct { - inputCh chan *model.PolymorphicEvent - outputCh chan *model.PolymorphicEvent - dir string - metricsInfo *metricsInfo - - closeCh chan struct{} -} - -type metricsInfo struct { - changeFeedID model.ChangeFeedID - tableName string - tableID model.TableID -} - -type ctxKey struct{} - -// NewUnifiedSorter creates a new Sorter -func NewUnifiedSorter( - dir string, changeFeedID model.ChangeFeedID, tableName string, tableID model.TableID, -) (*Sorter, error) { - poolMu.Lock() - defer poolMu.Unlock() - - if pool == nil { - var err error - pool, err = newBackEndPool(dir) - if err != nil { - return nil, errors.Trace(err) - } - } - - lazyInitWorkerPool() - return &Sorter{ - inputCh: make(chan *model.PolymorphicEvent, inputChSize), - outputCh: make(chan *model.PolymorphicEvent, outputChSize), - dir: dir, - metricsInfo: &metricsInfo{ - changeFeedID: changeFeedID, - tableName: tableName, - tableID: tableID, - }, - closeCh: make(chan struct{}, 1), - }, nil -} - -// CleanUp cleans up the files that might have been used. -func CleanUp() { - poolMu.Lock() - defer poolMu.Unlock() - - if pool != nil { - log.Info("Unified Sorter: starting cleaning up files") - pool.terminate() - pool = nil - } -} - -// ResetGlobalPoolWithoutCleanup reset the pool without cleaning up files. -// Note that it is used in tests only. -func ResetGlobalPoolWithoutCleanup() { - poolMu.Lock() - defer poolMu.Unlock() - - pool = nil -} - -// Run implements the EventSorter interface -func (s *Sorter) Run(ctx context.Context) error { - failpoint.Inject("sorterDebug", func() { - log.Info("sorterDebug: Running Unified Sorter in debug mode") - }) - - defer close(s.closeCh) - - finish, startCancel := util.MonitorCancelLatency(ctx, "Unified Sorter") - defer finish() - - ctx = context.WithValue(ctx, ctxKey{}, s) - ctx = contextutil.PutChangefeedIDInCtx(ctx, s.metricsInfo.changeFeedID) - ctx = contextutil.PutTableInfoInCtx(ctx, s.metricsInfo.tableID, s.metricsInfo.tableName) - - sorterConfig := config.GetGlobalServerConfig().Sorter - numConcurrentHeaps := sorterConfig.NumConcurrentWorker - - errg, subctx := errgroup.WithContext(ctx) - heapSorterCollectCh := make(chan *flushTask, heapCollectChSize) - // mergerCleanUp will consumer the remaining elements in heapSorterCollectCh to prevent any FD leak. - defer mergerCleanUp(heapSorterCollectCh) - - heapSorterErrCh := make(chan error, 1) - defer close(heapSorterErrCh) - heapSorterErrOnce := &sync.Once{} - heapSorters := make([]*heapSorter, sorterConfig.NumConcurrentWorker) - for i := range heapSorters { - heapSorters[i] = newHeapSorter(i, heapSorterCollectCh) - heapSorters[i].init(subctx, func(err error) { - heapSorterErrOnce.Do(func() { - heapSorterErrCh <- err - }) - }) - } - - ioCancelFunc := func() { - for _, heapSorter := range heapSorters { - // cancels async IO operations - heapSorter.canceller.Cancel() - } - } - - errg.Go(func() error { - defer func() { - // cancelling the heapSorters from the outside - for _, hs := range heapSorters { - hs.poolHandle.Unregister() - } - // must wait for all writers to exit to close the channel. - close(heapSorterCollectCh) - failpoint.Inject("InjectHeapSorterExitDelay", func() {}) - }() - - select { - case <-subctx.Done(): - startCancel() - return errors.Trace(subctx.Err()) - case err := <-heapSorterErrCh: - return errors.Trace(err) - } - }) - - errg.Go(func() error { - return printError(runMerger(subctx, numConcurrentHeaps, heapSorterCollectCh, s.outputCh, ioCancelFunc)) - }) - - errg.Go(func() error { - changefeedID := contextutil.ChangefeedIDFromCtx(ctx) - - metricSorterConsumeCount := sorterConsumeCount.MustCurryWith(map[string]string{ - "namespace": changefeedID.Namespace, - "changefeed": changefeedID.ID, - }) - - nextSorterID := 0 - for { - select { - case <-subctx.Done(): - return subctx.Err() - case event := <-s.inputCh: - if event.RawKV != nil && event.IsResolved() { - // broadcast resolved events - for _, sorter := range heapSorters { - select { - case <-subctx.Done(): - return subctx.Err() - default: - } - err := sorter.poolHandle.AddEvent(subctx, event) - if cerror.ErrWorkerPoolHandleCancelled.Equal(err) { - // no need to report ErrWorkerPoolHandleCancelled, - // as it may confuse the user - return nil - } - if err != nil { - return errors.Trace(err) - } - metricSorterConsumeCount.WithLabelValues("resolved").Inc() - } - continue - } - - // dispatch a row changed event - targetID := nextSorterID % numConcurrentHeaps - nextSorterID++ - select { - case <-subctx.Done(): - return subctx.Err() - default: - err := heapSorters[targetID].poolHandle.AddEvent(subctx, event) - if err != nil { - if cerror.ErrWorkerPoolHandleCancelled.Equal(err) { - // no need to report ErrWorkerPoolHandleCancelled, - // as it may confuse the user - return nil - } - return errors.Trace(err) - } - metricSorterConsumeCount.WithLabelValues("kv").Inc() - } - } - } - }) - - return printError(errg.Wait()) -} - -// AddEntry implements the EventSorter interface -func (s *Sorter) AddEntry(ctx context.Context, entry *model.PolymorphicEvent) { - select { - case <-ctx.Done(): - return - case <-s.closeCh: - case s.inputCh <- entry: - } -} - -// Output implements the EventSorter interface -func (s *Sorter) Output() <-chan *model.PolymorphicEvent { - return s.outputCh -} - -// RunWorkerPool runs the worker pool used by the heapSorters -// It **must** be running for Unified Sorter to work. -func RunWorkerPool(ctx context.Context) error { - lazyInitWorkerPool() - errg, ctx := errgroup.WithContext(ctx) - errg.Go(func() error { - return errors.Trace(heapSorterPool.Run(ctx)) - }) - - errg.Go(func() error { - return errors.Trace(heapSorterIOPool.Run(ctx)) - }) - - return errors.Trace(errg.Wait()) -} - -// EmitStartTs implement sorter interface -func (s *Sorter) EmitStartTs(ctx context.Context, ts uint64) {} - -// Stats implement sorter interface -func (s *Sorter) Stats() sorter.Stats { return sorter.Stats{} } diff --git a/dm/config/source_config.go b/dm/config/source_config.go index df8381cae41..584eb1b5db4 100644 --- a/dm/config/source_config.go +++ b/dm/config/source_config.go @@ -82,7 +82,7 @@ type SourceConfig struct { // relay synchronous starting point (if specified) RelayBinLogName string `yaml:"relay-binlog-name" toml:"relay-binlog-name" json:"relay-binlog-name"` RelayBinlogGTID string `yaml:"relay-binlog-gtid" toml:"relay-binlog-gtid" json:"relay-binlog-gtid"` - // only use when worker bound source, do not marsh it + // only use when the source is bound to a worker, do not marsh it UUIDSuffix int `yaml:"-" toml:"-" json:"-"` SourceID string `yaml:"source-id" toml:"source-id" json:"source-id"` diff --git a/dm/config/task.go b/dm/config/task.go index f99a0823b45..fd1e2e19239 100644 --- a/dm/config/task.go +++ b/dm/config/task.go @@ -125,7 +125,7 @@ type Meta struct { } // Verify does verification on configs -// NOTE: we can't decide to verify `binlog-name` or `binlog-gtid` until bound to a source (with `enable-gtid` set). +// NOTE: we can't decide to verify `binlog-name` or `binlog-gtid` until being bound to a source (with `enable-gtid` set). func (m *Meta) Verify() error { if m != nil && len(m.BinLogName) == 0 && len(m.BinLogGTID) == 0 { return terror.ErrConfigMetaInvalid.Generate() diff --git a/dm/master/openapi_view_test.go b/dm/master/openapi_view_test.go index 11ba5c1258b..b0fa7e802f0 100644 --- a/dm/master/openapi_view_test.go +++ b/dm/master/openapi_view_test.go @@ -718,7 +718,7 @@ func (s *OpenAPIViewSuite) TestSourceAPI() { ctrl := gomock.NewController(s.T()) defer ctrl.Finish() - // add mock worker the unbounded sources should be bounded + // add mock worker to which the unbound sources should be bound ctx1, cancel1 := context.WithCancel(ctx) defer cancel1() workerName1 := "worker1" @@ -737,7 +737,7 @@ func (s *OpenAPIViewSuite) TestSourceAPI() { mockRelayQueryStatus(mockWorkerClient, source1.SourceName, workerName1, pb.Stage_InvalidStage) s1.scheduler.SetWorkerClientForTest(workerName1, newMockRPCClient(mockWorkerClient)) - // get source status again,source should be bounded by worker1,but relay not started + // get source status again,source should be bound by worker1,but relay not started result = testutil.NewRequest().Get(source1StatusURL).GoWithHTTPHandler(s.T(), s1.openapiHandles) s.Equal(http.StatusOK, result.Code()) s.NoError(result.UnmarshalBodyToObject(&source1Status)) @@ -967,7 +967,7 @@ func (s *OpenAPIViewSuite) TestTaskAPI() { // check http status code s.Equal(http.StatusCreated, result.Code()) - // add mock worker start workers, the unbounded sources should be bounded + // add mock worker, the unbound sources should be bound before starting workers ctx1, cancel1 := context.WithCancel(ctx) defer cancel1() workerName1 := "worker-1" diff --git a/dm/master/scheduler/scheduler.go b/dm/master/scheduler/scheduler.go index 68a83de101a..1bd841e4357 100644 --- a/dm/master/scheduler/scheduler.go +++ b/dm/master/scheduler/scheduler.go @@ -149,7 +149,7 @@ type Scheduler struct { // - when the bounding worker become offline, in updateStatusToUnbound. // delete: // - remove source by user request (calling `RemoveSourceCfg`). - // - when bounded the source to a worker, in updateStatusToBound. + // - when bound the source to a worker, in updateStatusToBound. unbounds map[string]struct{} // a mirror of bounds whose element is not deleted when worker unbound. worker -> SourceBound @@ -548,10 +548,10 @@ func (s *Scheduler) GetSourceCfgByID(source string) *config.SourceConfig { // transferWorkerAndSource swaps two sources between two workers (maybe empty). The input means before invocation of // this function, left worker and left source are bound, right worker and right source are bound. After this function, // left worker should be bound to right source and vice versa. -// lworker, "", "", rsource This means an unbounded source bounded to a free worker +// lworker, "", "", rsource This means an unbound source bound to a free worker // lworker, lsource, rworker, "" This means transfer a source from a worker to another free worker -// lworker, lsource, "", rsource This means transfer a worker from a bounded source to another unbounded source -// lworker, lsource, rworker, rsource This means transfer two bounded relations. +// lworker, lsource, "", rsource This means transfer a worker from a bound source to another unbound source +// lworker, lsource, rworker, rsource This means transfer two bound relations. func (s *Scheduler) transferWorkerAndSource(lworker, lsource, rworker, rsource string) error { // in first four arrays, index 0 is for left worker, index 1 is for right worker var ( @@ -645,7 +645,7 @@ func (s *Scheduler) transferWorkerAndSource(lworker, lsource, rworker, rsource s } } - // if one of the workers/sources become free/unbounded + // if one of the workers/sources become free/unbound // try bound it. for i := range inputWorkers { another := i ^ 1 // make use of XOR to flip 0 and 1 @@ -2125,7 +2125,7 @@ func (s *Scheduler) observeWorkerEvent(ctx context.Context, rev int64) error { } // handleWorkerOnline handles the scheduler when a DM-worker become online. -// This should try to bound an unbounded source to it. +// This should try to bound an unbound source to it. // NOTE: this func need to hold the mutex. func (s *Scheduler) handleWorkerOnline(ev ha.WorkerEvent, toLock bool) error { if toLock { @@ -2178,13 +2178,13 @@ func (s *Scheduler) handleWorkerOnline(ev ha.WorkerEvent, toLock bool) error { } } - // 4. try to bound an unbounded source. + // 4. try to bind an unbound source. _, err := s.tryBoundForWorker(w) return err } // handleWorkerOffline handles the scheduler when a DM-worker become offline. -// This should unbound any previous bounded source. +// This should unbind any previous bound source. // NOTE: this func need to hold the mutex. func (s *Scheduler) handleWorkerOffline(ev ha.WorkerEvent, toLock bool) error { if toLock { @@ -2236,7 +2236,7 @@ func (s *Scheduler) handleWorkerOffline(ev ha.WorkerEvent, toLock bool) error { // - try to bind any unbound sources // if the source is bound to a relay enabled worker, we must check that the source is also the relay source of worker. // pulling binlog using relay or not is determined by whether the worker has enabled relay. -func (s *Scheduler) tryBoundForWorker(w *Worker) (bounded bool, err error) { +func (s *Scheduler) tryBoundForWorker(w *Worker) (bound bool, err error) { // 1. handle this worker has unfinished load task. worker, sourceID := s.getNextLoadTaskTransfer(w.BaseInfo().Name, "") if sourceID != "" { @@ -2600,17 +2600,17 @@ func (s *Scheduler) RemoveLoadTaskAndLightningStatus(task string) error { // return (worker, source) that is used by transferWorkerAndSource, to try to resolve a paused load task that the source can't be bound to the worker which has its dump files. // worker, source This means a subtask finish load stage, often called by handleLoadTaskDel. // worker, "" This means a free worker online, often called by tryBoundForWorker. -// "", source This means a unbounded source online, often called by tryBoundForSource. +// "", source This means a unbound source online, often called by tryBoundForSource. func (s *Scheduler) getNextLoadTaskTransfer(worker, source string) (string, string) { // origin worker not free, try to get a source. if worker != "" { - // try to get a unbounded source + // try to get a unbound source for sourceID := range s.unbounds { if sourceID != source && s.hasLoadTaskByWorkerAndSource(worker, sourceID) { return "", sourceID } } - // try to get a bounded source + // try to get a bound source for sourceID, w := range s.bounds { if sourceID != source && s.hasLoadTaskByWorkerAndSource(worker, sourceID) && !s.hasLoadTaskByWorkerAndSource(w.baseInfo.Name, sourceID) { return w.baseInfo.Name, sourceID @@ -2618,7 +2618,7 @@ func (s *Scheduler) getNextLoadTaskTransfer(worker, source string) (string, stri } } - // origin source bounded, try to get a worker + // origin source is bound, try to get a worker if source != "" { // try to get a free worker for _, w := range s.workers { @@ -2628,7 +2628,7 @@ func (s *Scheduler) getNextLoadTaskTransfer(worker, source string) (string, stri } } - // try to get a bounded worker + // try to get a bound worker for _, w := range s.workers { workerName := w.baseInfo.Name if workerName != worker && w.Stage() == WorkerBound { diff --git a/dm/master/scheduler/scheduler_test.go b/dm/master/scheduler/scheduler_test.go index 937e6b0bec1..e8178807015 100644 --- a/dm/master/scheduler/scheduler_test.go +++ b/dm/master/scheduler/scheduler_test.go @@ -199,7 +199,7 @@ func (t *testSchedulerSuite) testSchedulerProgress(restart int) { t.workerOffline(s, workerName1) // still no bounds (because the worker is offline). t.sourceBounds(s, []string{}, []string{sourceID1}) - // no expect relay stage exist (because the source has never been bounded). + // no expect relay stage exist (because the source has never been bound). t.relayStageMatch(s, sourceID1, pb.Stage_InvalidStage) rebuildScheduler(ctx) @@ -212,7 +212,7 @@ func (t *testSchedulerSuite) testSchedulerProgress(restart int) { defer wg.Done() require.NoError(t.T(), ha.KeepAlive(ctx1, t.etcdTestCli, workerName1, keepAliveTTL)) }() - // wait for source1 bound to worker1. + // wait for source1 being bound to worker1. require.Eventually(t.T(), func() bool { bounds := s.BoundSources() return len(bounds) == 1 && bounds[0] == sourceID1 @@ -1156,40 +1156,40 @@ func (t *testSchedulerSuite) TestLastBound() { s.unbounds[sourceID1] = struct{}{} s.unbounds[sourceID2] = struct{}{} - // worker1 goes to last bounded source + // worker1 goes to last bound source worker1.ToFree() - bounded, err := s.tryBoundForWorker(worker1) + bound, err := s.tryBoundForWorker(worker1) require.NoError(t.T(), err) - require.True(t.T(), bounded) + require.True(t.T(), bound) require.Equal(t.T(), worker1, s.bounds[sourceID1]) - // worker3 has to bounded to source2 + // worker3 has to bind source2 worker3.ToFree() - bounded, err = s.tryBoundForWorker(worker3) + bound, err = s.tryBoundForWorker(worker3) require.NoError(t.T(), err) - require.True(t.T(), bounded) + require.True(t.T(), bound) require.Equal(t.T(), worker3, s.bounds[sourceID2]) // though worker2 has a previous source, that source is not available, so not bound worker2.ToFree() - bounded, err = s.tryBoundForWorker(worker2) + bound, err = s.tryBoundForWorker(worker2) require.NoError(t.T(), err) - require.False(t.T(), bounded) + require.False(t.T(), bound) - // worker4 is used to test whether source2 should be bounded to worker2 rather than a new worker + // worker4 is used to test whether source2 should be bound to worker2 rather than a new worker worker4.ToFree() - bounded, err = s.tryBoundForWorker(worker4) + bound, err = s.tryBoundForWorker(worker4) require.NoError(t.T(), err) - require.False(t.T(), bounded) + require.False(t.T(), bound) - // after worker3 become offline, source2 should be bounded to worker2 + // after worker3 become offline, source2 should be bound to worker2 s.updateStatusToUnbound(sourceID2) _, ok := s.bounds[sourceID2] require.False(t.T(), ok) worker3.ToOffline() - bounded, err = s.tryBoundForSource(sourceID2) + bound, err = s.tryBoundForSource(sourceID2) require.NoError(t.T(), err) - require.True(t.T(), bounded) + require.True(t.T(), bound) require.Equal(t.T(), worker2, s.bounds[sourceID2]) } @@ -1217,11 +1217,11 @@ func (t *testSchedulerSuite) TestInvalidLastBound() { s.sourceCfgs[sourceID1] = sourceCfg1 s.lastBound[workerName1] = ha.SourceBound{Source: sourceID2} s.unbounds[sourceID1] = struct{}{} - // step2: worker1 doesn't go to last bounded source, because last source doesn't have a source config (might be removed) + // step2: worker1 doesn't go to last bound source, because last source doesn't have a source config (might be removed) worker1.ToFree() - bounded, err := s.tryBoundForWorker(worker1) + bound, err := s.tryBoundForWorker(worker1) require.NoError(t.T(), err) - require.True(t.T(), bounded) + require.True(t.T(), bound) require.Equal(t.T(), worker1, s.bounds[sourceID1]) } @@ -1629,12 +1629,12 @@ func (t *testSchedulerSuite) TestStartSourcesWithoutSourceConfigsInEtcd() { s.workers[workerName1].stage = WorkerFree s.workers[workerName2].stage = WorkerFree - bounded, err := s.tryBoundForSource(sourceID1) + bound, err := s.tryBoundForSource(sourceID1) require.NoError(t.T(), err) - require.True(t.T(), bounded) - bounded, err = s.tryBoundForSource(sourceID2) + require.True(t.T(), bound) + bound, err = s.tryBoundForSource(sourceID2) require.NoError(t.T(), err) - require.True(t.T(), bounded) + require.True(t.T(), bound) s.started.Store(false) sbm, _, err := ha.GetSourceBound(t.etcdTestCli, "") @@ -1693,14 +1693,14 @@ func (t *testSchedulerSuite) TestTransferWorkerAndSource() { s.unbounds[sourceID1] = struct{}{} s.unbounds[sourceID2] = struct{}{} - // test free worker and unbounded source + // test free worker and unbound source require.NoError(t.T(), s.transferWorkerAndSource(workerName1, "", "", sourceID1)) require.NoError(t.T(), s.transferWorkerAndSource("", sourceID2, workerName2, "")) require.Equal(t.T(), worker1, s.bounds[sourceID1]) require.Equal(t.T(), worker2, s.bounds[sourceID2]) require.Equal(t.T(), 0, len(s.unbounds)) - // test transfer bounded source to free worker + // test transfer bound source to free worker require.NoError(t.T(), s.transferWorkerAndSource(workerName1, sourceID1, workerName4, "")) require.Equal(t.T(), worker4, s.bounds[sourceID1]) require.Equal(t.T(), WorkerFree, worker1.Stage()) @@ -1711,7 +1711,7 @@ func (t *testSchedulerSuite) TestTransferWorkerAndSource() { require.Equal(t.T(), WorkerFree, worker2.Stage()) require.Equal(t.T(), WorkerBound, worker3.Stage()) - // test transfer bounded worker to unbounded source + // test transfer bound worker to unbound source s.unbounds[sourceID3] = struct{}{} s.unbounds[sourceID4] = struct{}{} require.NoError(t.T(), s.transferWorkerAndSource("", sourceID3, workerName3, sourceID2)) @@ -1726,7 +1726,7 @@ func (t *testSchedulerSuite) TestTransferWorkerAndSource() { require.Equal(t.T(), 0, len(s.unbounds)) - // test transfer two bounded sources + // test transfer two bound sources require.NoError(t.T(), s.transferWorkerAndSource(workerName1, sourceID1, workerName2, sourceID2)) require.NoError(t.T(), s.transferWorkerAndSource(workerName4, sourceID4, workerName3, sourceID3)) require.Equal(t.T(), worker2, s.bounds[sourceID1]) diff --git a/dm/master/scheduler/worker.go b/dm/master/scheduler/worker.go index a7a96dd0a1d..8ac1246159d 100644 --- a/dm/master/scheduler/worker.go +++ b/dm/master/scheduler/worker.go @@ -85,7 +85,7 @@ type Worker struct { cli workerrpc.Client // the gRPC client proxy. baseInfo ha.WorkerInfo // the base information of the DM-worker instance. - bound ha.SourceBound // the source bound relationship, null value if not bounded. + bound ha.SourceBound // the source bound relationship, null value if not bound stage WorkerStage // the current stage. // the source ID from which the worker is pulling relay log. should keep consistent with Scheduler.relayWorkers @@ -162,7 +162,7 @@ func (w *Worker) Unbound() error { defer w.mu.Unlock() if w.stage != WorkerBound { // caller should not do this. - return terror.ErrSchedulerWorkerInvalidTrans.Generatef("can't unbound a worker that is not in bound stage.") + return terror.ErrSchedulerWorkerInvalidTrans.Generatef("can't unbind a worker that is not in bound stage.") } w.bound = nullBound @@ -227,8 +227,8 @@ func (w *Worker) Stage() WorkerStage { return w.stage } -// Bound returns the current source ID bounded to, -// returns null value if not bounded. +// Bound returns the current source ID bound to, +// returns null value if not be bound. func (w *Worker) Bound() ha.SourceBound { w.mu.RLock() defer w.mu.RUnlock() diff --git a/dm/master/server.go b/dm/master/server.go index e5e63a6868f..8792573978d 100644 --- a/dm/master/server.go +++ b/dm/master/server.go @@ -2653,7 +2653,7 @@ func (s *Server) OperateRelay(ctx context.Context, req *pb.OperateRelayRequest) } else { resp2.Sources = append(resp2.Sources, &pb.CommonWorkerResponse{ Result: true, - Msg: "source relay is operated but the bounded worker is offline", + Msg: "source relay is operated but the bound worker is offline", Source: req.Source, Worker: worker, }) diff --git a/dm/master/server_test.go b/dm/master/server_test.go index 3ecbffefb67..bf33341b039 100644 --- a/dm/master/server_test.go +++ b/dm/master/server_test.go @@ -1841,7 +1841,7 @@ func (t *testMasterSuite) TestOperateSource() { require.False(t.T(), resp.Result) require.Contains(t.T(), resp.Msg, "source config with ID "+mysqlCfg.SourceID+" not exists") - // 5. start workers, the unbounded sources should be bounded + // 5. start workers, the unbound sources should be bound var wg sync.WaitGroup workerName1 := "worker1" workerName2 := "worker2" diff --git a/dm/openapi/spec/dm.yaml b/dm/openapi/spec/dm.yaml index a8a88239cf9..07e6772acba 100644 --- a/dm/openapi/spec/dm.yaml +++ b/dm/openapi/spec/dm.yaml @@ -1864,7 +1864,7 @@ components: bound_source_name: type: string example: "mysql-01" - description: "bound source name of this worker node" + description: "source name bound to this worker node" required: - "name" - "addr" diff --git a/dm/pkg/ha/bound.go b/dm/pkg/ha/bound.go index 6155e577d12..5699bea0b95 100644 --- a/dm/pkg/ha/bound.go +++ b/dm/pkg/ha/bound.go @@ -42,7 +42,7 @@ const ( // SourceBound represents the bound relationship between the DM-worker instance and the upstream MySQL source. type SourceBound struct { Source string `json:"source"` // the source ID of the upstream. - Worker string `json:"worker"` // the name of the bounded DM-worker for the source. + Worker string `json:"worker"` // the name of the bound DM-worker for the source. // only used to report to the caller of the watcher, do not marsh it. // if it's true, it means the bound has been deleted in etcd. diff --git a/dm/pkg/ha/doc.go b/dm/pkg/ha/doc.go index a69e4b6ff7f..71fb16f5fe9 100644 --- a/dm/pkg/ha/doc.go +++ b/dm/pkg/ha/doc.go @@ -52,7 +52,7 @@ package ha // - PUT when scheduling the source to a DM-worker instance by DM-master. // - PUT with `the expectant stage of the relay` in one txn for the first time. // - GET (through GET/WATCH) by DM-worker to know relevant relay/subtasks have to do. -// - DELETE when the bounded DM-worker become offline. +// - DELETE when the bound DM-worker become offline. // - DELETE when removing an upstream by DM-master. // - DELETE with `the upstream MySQL config` in one txn. // - DELETE with `the expectant stage of the relay` in one txn. diff --git a/dm/pkg/shardddl/optimism/lock.go b/dm/pkg/shardddl/optimism/lock.go index 5b215dbc895..0e7776d2cab 100644 --- a/dm/pkg/shardddl/optimism/lock.go +++ b/dm/pkg/shardddl/optimism/lock.go @@ -15,6 +15,7 @@ package optimism import ( "bytes" + "context" "encoding/json" "fmt" "sync" @@ -32,7 +33,6 @@ import ( "github.com/pingcap/tiflow/dm/pkg/terror" clientv3 "go.etcd.io/etcd/client/v3" "go.uber.org/zap" - "golang.org/x/net/context" ) // DropColumnStage represents whether drop column done for a sharding table. diff --git a/dm/pkg/utils/storage_freebsd.go b/dm/pkg/utils/storage_freebsd.go index c3f6f86d6f8..38eddf99080 100644 --- a/dm/pkg/utils/storage_freebsd.go +++ b/dm/pkg/utils/storage_freebsd.go @@ -12,7 +12,6 @@ // limitations under the License. //go:build freebsd -// +build freebsd package utils diff --git a/dm/pkg/utils/storage_unix.go b/dm/pkg/utils/storage_unix.go index 89bfd4d11db..4ac72a7e8d3 100644 --- a/dm/pkg/utils/storage_unix.go +++ b/dm/pkg/utils/storage_unix.go @@ -12,7 +12,6 @@ // limitations under the License. //go:build !windows && !freebsd -// +build !windows,!freebsd package utils diff --git a/dm/pkg/utils/storage_windows.go b/dm/pkg/utils/storage_windows.go index 7ea6b4185c2..613e155669d 100644 --- a/dm/pkg/utils/storage_windows.go +++ b/dm/pkg/utils/storage_windows.go @@ -12,7 +12,6 @@ // limitations under the License. //go:build windows -// +build windows package utils diff --git a/dm/proto/dmworker.proto b/dm/proto/dmworker.proto index 1dba4c26044..792b7822368 100644 --- a/dm/proto/dmworker.proto +++ b/dm/proto/dmworker.proto @@ -169,7 +169,7 @@ message SyncStatus { // SourceStatus represents status for source runing on dm-worker message SourceStatus { string source = 1; // sourceID, set by dm-master - string worker = 2; // bounded worker name for this source + string worker = 2; // worker name bound to this source ProcessResult result = 3; RelayStatus relayStatus = 4; } diff --git a/dm/tests/dmctl_basic/check_list/start_relay.sh b/dm/tests/dmctl_basic/check_list/start_relay.sh index f48fd00fbe7..b059135f9e7 100644 --- a/dm/tests/dmctl_basic/check_list/start_relay.sh +++ b/dm/tests/dmctl_basic/check_list/start_relay.sh @@ -54,7 +54,7 @@ function start_relay_on_offline_worker() { run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "start-relay -s $SOURCE_ID2 worker2" \ "\"result\": true" 2 \ - "\"msg\": \"source relay is operated but the bounded worker is offline\"" 1 \ + "\"msg\": \"source relay is operated but the bound worker is offline\"" 1 \ "\"source\": \"$SOURCE_ID2\"" 1 \ "\"worker\": \"worker2\"" 1 } diff --git a/dm/tests/dmctl_basic/check_list/stop_relay.sh b/dm/tests/dmctl_basic/check_list/stop_relay.sh index 86045a4c64e..f64ffa496b0 100644 --- a/dm/tests/dmctl_basic/check_list/stop_relay.sh +++ b/dm/tests/dmctl_basic/check_list/stop_relay.sh @@ -49,7 +49,7 @@ function stop_relay_on_offline_worker() { run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "stop-relay -s $SOURCE_ID2 worker2" \ "\"result\": true" 2 \ - "\"msg\": \"source relay is operated but the bounded worker is offline\"" 1 \ + "\"msg\": \"source relay is operated but the bound worker is offline\"" 1 \ "\"source\": \"$SOURCE_ID2\"" 1 \ "\"worker\": \"worker2\"" 1 } diff --git a/dm/tests/lightning_load_task/run.sh b/dm/tests/lightning_load_task/run.sh index a40eb188fe0..471d7f947ef 100755 --- a/dm/tests/lightning_load_task/run.sh +++ b/dm/tests/lightning_load_task/run.sh @@ -18,7 +18,7 @@ function test_worker_restart() { kill_process dm-worker1 check_port_offline $WORKER1_PORT 20 - # source1 bound to worker3 + # source1 is bound to worker3 run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "list-member -w -n worker3" \ "\"stage\": \"bound\"" 1 \ @@ -95,7 +95,7 @@ function test_transfer_two_sources() { kill_process dm-worker1 check_port_offline $WORKER1_PORT 20 - # source1 bound to worker2 + # source1 is bound to worker2 run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "list-member -w -n worker2" \ "\"stage\": \"bound\"" 1 \ @@ -124,7 +124,7 @@ function test_transfer_two_sources() { kill_process dm-worker3 check_port_offline $WORKER3_PORT 20 - # source2 bound to worker1 + # source2 is bound to worker1 run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "list-member -w -n worker1" \ "\"stage\": \"bound\"" 1 \ @@ -160,7 +160,7 @@ function test_transfer_two_sources() { run_dm_worker $WORK_DIR/worker3 $WORKER3_PORT $CONF_DIR/dm-worker3.toml check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER3_PORT - # source2 bound to worker3 since load_task3 + # source2 is bound to worker3 since load_task3 run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "list-member -w -n worker2" \ "\"stage\": \"bound\"" 1 \ diff --git a/dm/tests/openapi/run.sh b/dm/tests/openapi/run.sh index 38106cdb777..5dcdfa3c124 100644 --- a/dm/tests/openapi/run.sh +++ b/dm/tests/openapi/run.sh @@ -113,7 +113,7 @@ function test_relay() { # create source successfully openapi_source_check "create_source1_success" - # we need make sure that source is bounded by worker1 because we will start relay on worker1 + # we need make sure that source is bound by worker1 because we will start relay on worker1 openapi_source_check "transfer_source_success" "mysql-01" "worker1" run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ diff --git a/dm/tests/validator_basic/run.sh b/dm/tests/validator_basic/run.sh index 5f8f5fb1033..be3f71ad6a1 100644 --- a/dm/tests/validator_basic/run.sh +++ b/dm/tests/validator_basic/run.sh @@ -20,7 +20,7 @@ function prepare_dm_and_source() { run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $cur/conf/dm-worker1.toml check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER1_PORT - # source 1 should bound to worker 1 + # source 1 should be bound to worker 1 dmctl_operate_source create $cur/conf/source1.yaml $SOURCE_ID1 run_dm_worker $WORK_DIR/worker2 $WORKER2_PORT $cur/conf/dm-worker2.toml @@ -564,7 +564,7 @@ function stopped_validator_fail_over() { "new\/ignored\/resolved: 1\/0\/0" 1 \ "\"stage\": \"Running\"" 1 \ "\"stage\": \"Stopped\"" 1 - # source1 bound to worker1, so source1 will bound to worker2. see prepare_dm_and_source + # source1 is bound to worker1, so source1 will be bound to worker2. see prepare_dm_and_source kill_process worker1 # stopped task fail over, processed row status and table status is not loaded into memory, so they're zero # but we can see errors, since it's loaded from db all the time diff --git a/dm/ui/embedded_asserts.go b/dm/ui/embedded_asserts.go index ae09f02ad54..36c704039c6 100644 --- a/dm/ui/embedded_asserts.go +++ b/dm/ui/embedded_asserts.go @@ -1,5 +1,4 @@ //go:build dm_webui -// +build dm_webui // Copyright 2021 PingCAP, Inc. // diff --git a/dm/ui/empty_asserts.go b/dm/ui/empty_asserts.go index f22ca32d0c2..745457f3d8c 100644 --- a/dm/ui/empty_asserts.go +++ b/dm/ui/empty_asserts.go @@ -1,5 +1,4 @@ //go:build !dm_webui -// +build !dm_webui // Copyright 2021 PingCAP, Inc. // diff --git a/docs/swagger/docs.go b/docs/swagger/docs.go index e94bc13c8cf..83e901f4d66 100644 --- a/docs/swagger/docs.go +++ b/docs/swagger/docs.go @@ -741,6 +741,41 @@ var doc = `{ } } }, + "/api/v2/processors": { + "get": { + "description": "list all processors in the TiCDC cluster", + "produces": [ + "application/json" + ], + "tags": [ + "processor" + ], + "summary": "List processors", + "responses": { + "200": { + "description": "OK", + "schema": { + "type": "array", + "items": { + "$ref": "#/definitions/model.ProcessorCommonInfo" + } + } + }, + "400": { + "description": "Bad Request", + "schema": { + "$ref": "#/definitions/model.HTTPError" + } + }, + "500": { + "description": "Internal Server Error", + "schema": { + "$ref": "#/definitions/model.HTTPError" + } + } + } + } + }, "/api/v2/processors/{changefeed_id}/{capture_id}": { "get": { "description": "get the detail information of a processor", diff --git a/docs/swagger/swagger.json b/docs/swagger/swagger.json index 8a2d5ba02bb..30f02296eb5 100644 --- a/docs/swagger/swagger.json +++ b/docs/swagger/swagger.json @@ -722,6 +722,41 @@ } } }, + "/api/v2/processors": { + "get": { + "description": "list all processors in the TiCDC cluster", + "produces": [ + "application/json" + ], + "tags": [ + "processor" + ], + "summary": "List processors", + "responses": { + "200": { + "description": "OK", + "schema": { + "type": "array", + "items": { + "$ref": "#/definitions/model.ProcessorCommonInfo" + } + } + }, + "400": { + "description": "Bad Request", + "schema": { + "$ref": "#/definitions/model.HTTPError" + } + }, + "500": { + "description": "Internal Server Error", + "schema": { + "$ref": "#/definitions/model.HTTPError" + } + } + } + } + }, "/api/v2/processors/{changefeed_id}/{capture_id}": { "get": { "description": "get the detail information of a processor", diff --git a/docs/swagger/swagger.yaml b/docs/swagger/swagger.yaml index 0129e2f5df5..e688444e9ed 100644 --- a/docs/swagger/swagger.yaml +++ b/docs/swagger/swagger.yaml @@ -725,6 +725,29 @@ paths: summary: Get server status tags: - common + /api/v2/processors: + get: + description: list all processors in the TiCDC cluster + produces: + - application/json + responses: + "200": + description: OK + schema: + items: + $ref: '#/definitions/model.ProcessorCommonInfo' + type: array + "400": + description: Bad Request + schema: + $ref: '#/definitions/model.HTTPError' + "500": + description: Internal Server Error + schema: + $ref: '#/definitions/model.HTTPError' + summary: List processors + tags: + - processor /api/v2/processors/{changefeed_id}/{capture_id}: get: description: get the detail information of a processor diff --git a/engine/servermaster/jobop/operator.go b/engine/servermaster/jobop/operator.go index ec731ace54e..bc2d46de030 100644 --- a/engine/servermaster/jobop/operator.go +++ b/engine/servermaster/jobop/operator.go @@ -14,13 +14,14 @@ package jobop import ( + "context" + "github.com/pingcap/log" frameworkModel "github.com/pingcap/tiflow/engine/framework/model" pkgOrm "github.com/pingcap/tiflow/engine/pkg/orm" ormModel "github.com/pingcap/tiflow/engine/pkg/orm/model" "go.uber.org/multierr" "go.uber.org/zap" - "golang.org/x/net/context" ) type operateRouter interface { diff --git a/engine/servermaster/jobop/operator_test.go b/engine/servermaster/jobop/operator_test.go index 1d8cede654c..e1660c6b24c 100644 --- a/engine/servermaster/jobop/operator_test.go +++ b/engine/servermaster/jobop/operator_test.go @@ -14,6 +14,7 @@ package jobop import ( + "context" "testing" frameModel "github.com/pingcap/tiflow/engine/framework/model" @@ -21,7 +22,6 @@ import ( ormModel "github.com/pingcap/tiflow/engine/pkg/orm/model" "github.com/pingcap/tiflow/pkg/errors" "github.com/stretchr/testify/require" - "golang.org/x/net/context" ) type mockOperatorRouter struct { diff --git a/pkg/cmd/server/server.go b/pkg/cmd/server/server.go index ca020feb946..64d03691ffe 100644 --- a/pkg/cmd/server/server.go +++ b/pkg/cmd/server/server.go @@ -25,7 +25,6 @@ import ( ticonfig "github.com/pingcap/tidb/config" "github.com/pingcap/tiflow/cdc/contextutil" "github.com/pingcap/tiflow/cdc/server" - "github.com/pingcap/tiflow/cdc/sorter/unified" cmdcontext "github.com/pingcap/tiflow/pkg/cmd/context" "github.com/pingcap/tiflow/pkg/cmd/util" "github.com/pingcap/tiflow/pkg/config" @@ -161,7 +160,6 @@ func (o *options) run(cmd *cobra.Command) error { log.Info("cdc server exits normally") } server.Close() - unified.CleanUp() return nil } diff --git a/pkg/config/debug.go b/pkg/config/debug.go index 1f4de695a5e..768250c972a 100644 --- a/pkg/config/debug.go +++ b/pkg/config/debug.go @@ -27,9 +27,9 @@ type DebugConfig struct { // We will always use pull-based sink. EnablePullBasedSink bool `toml:"enable-pull-based-sink" json:"enable-pull-based-sink"` - // EnableDBSorter enables db sorter. - // - // The default value is true. + // EnableDBSorter enables db sorter. The default value is true. + // DEPRECATED: this option will be removed in the future. + // We will always use db sorter. EnableDBSorter bool `toml:"enable-db-sorter" json:"enable-db-sorter"` DB *DBConfig `toml:"db" json:"db"` diff --git a/pkg/context/context.go b/pkg/context/context.go index 739fa5f88d0..ce2acdac8c1 100644 --- a/pkg/context/context.go +++ b/pkg/context/context.go @@ -20,7 +20,6 @@ import ( "github.com/pingcap/log" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/cdc/processor/sourcemanager/engine/factory" - ssystem "github.com/pingcap/tiflow/cdc/sorter/db/system" "github.com/pingcap/tiflow/pkg/config" "github.com/pingcap/tiflow/pkg/etcd" "github.com/pingcap/tiflow/pkg/p2p" @@ -36,10 +35,6 @@ type GlobalVars struct { EtcdClient etcd.CDCEtcdClient // SortEngineManager is introduced for pull-based sinks. - // - // TODO(qupeng): remove SorterSystem after all sorters are transformed - // to adapt pull-based sinks. - SorterSystem *ssystem.System SortEngineFactory *factory.SortEngineFactory // OwnerRevision is the Etcd revision when the owner got elected. diff --git a/pkg/db/db.go b/pkg/db/db.go index f62b681d883..22ddef7fe46 100644 --- a/pkg/db/db.go +++ b/pkg/db/db.go @@ -27,7 +27,6 @@ type DB interface { DeleteRange(start, end []byte) error Compact(start, end []byte) error Close() error - CollectMetrics(id int) } // A Batch is a sequence of Puts and Deletes that Commit to DB. diff --git a/pkg/db/db_test.go b/pkg/db/db_test.go index 2db62042d03..d093bd4a67b 100644 --- a/pkg/db/db_test.go +++ b/pkg/db/db_test.go @@ -21,7 +21,6 @@ import ( "math/rand" "path/filepath" "testing" - "time" "github.com/cockroachdb/pebble" "github.com/pingcap/tiflow/pkg/config" @@ -51,9 +50,6 @@ func testDB(t *testing.T, db DB) { pdb, err := pebble.Open(t.TempDir(), &option) require.Nil(t, err) - // Collect metrics - db.CollectMetrics(0) - // Batch pbatch := pdb.NewBatch() err = pbatch.Set([]byte("k1"), []byte("v1"), nil) @@ -123,50 +119,6 @@ func testDB(t *testing.T, db DB) { require.Nil(t, pdb.Close()) } -func TestPebbleMetrics(t *testing.T) { - t.Parallel() - - cfg := config.GetDefaultServerConfig().Clone().Debug.DB - cfg.Count = 1 - - id := 1 - option, ws := buildPebbleOption(id, cfg) - db, err := pebble.Open(t.TempDir(), &option) - require.Nil(t, err) - pdb := &pebbleDB{ - db: db, - metricWriteStall: ws, - } - - // Collect empty metrics. - pdb.CollectMetrics(id) - - // Write stall. - option.EventListener.WriteStallBegin(pebble.WriteStallBeginInfo{}) - time.Sleep(100 * time.Millisecond) - option.EventListener.WriteStallEnd() - require.EqualValues(t, 1, ws.counter) - require.Less(t, time.Duration(0), ws.duration.Load().(time.Duration)) - - // Collect write stall metrics. - pdb.CollectMetrics(id) - require.EqualValues(t, 1, ws.counter) - require.Equal(t, time.Duration(0), ws.duration.Load().(time.Duration)) - - // Filter out of order write stall end. - option.EventListener.WriteStallEnd() - require.Equal(t, time.Duration(0), ws.duration.Load().(time.Duration)) - - // Write stall again. - option.EventListener.WriteStallBegin(pebble.WriteStallBeginInfo{}) - time.Sleep(10 * time.Millisecond) - option.EventListener.WriteStallEnd() - require.EqualValues(t, 2, ws.counter) - require.Less(t, time.Duration(0), ws.duration.Load().(time.Duration)) - - require.Nil(t, pdb.Close()) -} - // goos: linux // goarch: amd64 // pkg: github.com/pingcap/tiflow/pkg/db diff --git a/cdc/sorter/encoding/key.go b/pkg/db/encoding/key.go similarity index 100% rename from cdc/sorter/encoding/key.go rename to pkg/db/encoding/key.go diff --git a/cdc/sorter/encoding/key_test.go b/pkg/db/encoding/key_test.go similarity index 100% rename from cdc/sorter/encoding/key_test.go rename to pkg/db/encoding/key_test.go diff --git a/cdc/sorter/encoding/value.go b/pkg/db/encoding/value.go similarity index 100% rename from cdc/sorter/encoding/value.go rename to pkg/db/encoding/value.go diff --git a/pkg/db/metrics.go b/pkg/db/metrics.go deleted file mode 100644 index 8df1b361428..00000000000 --- a/pkg/db/metrics.go +++ /dev/null @@ -1,110 +0,0 @@ -// Copyright 2021 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package db - -import ( - "github.com/prometheus/client_golang/prometheus" -) - -var ( - dbWriteBytes = prometheus.NewGaugeVec(prometheus.GaugeOpts{ - Namespace: "ticdc", - Subsystem: "db", - Name: "write_bytes_total", - Help: "The total number of write bytes by the db", - }, []string{"id"}) - - dbReadBytes = prometheus.NewGaugeVec(prometheus.GaugeOpts{ - Namespace: "ticdc", - Subsystem: "db", - Name: "read_bytes_total", - Help: "The total number of read bytes by the db", - }, []string{"id"}) - - dbSnapshotGauge = prometheus.NewGaugeVec(prometheus.GaugeOpts{ - Namespace: "ticdc", - Subsystem: "db", - Name: "snapshot_count_gauge", - Help: "The number of snapshot by the db", - }, []string{"id"}) - - dbIteratorGauge = prometheus.NewGaugeVec(prometheus.GaugeOpts{ - Namespace: "ticdc", - Subsystem: "db", - Name: "iterator_count_gauge", - Help: "The number of iterator by the db", - }, []string{"id"}) - - dbLevelCount = prometheus.NewGaugeVec(prometheus.GaugeOpts{ - Namespace: "ticdc", - Subsystem: "db", - Name: "level_count", - Help: "The number of files in each level by the db", - }, []string{"level", "id"}) - - dbWriteDelayDuration = prometheus.NewGaugeVec(prometheus.GaugeOpts{ - Namespace: "ticdc", - Subsystem: "db", - Name: "write_delay_seconds", - Help: "The duration of db write delay seconds", - }, []string{"id"}) - - dbWriteDelayCount = prometheus.NewGaugeVec(prometheus.GaugeOpts{ - Namespace: "ticdc", - Subsystem: "db", - Name: "write_delay_total", - Help: "The total number of db delay", - }, []string{"id"}) - - dbBlockCacheAccess = prometheus.NewGaugeVec(prometheus.GaugeOpts{ - Namespace: "ticdc", - Subsystem: "db", - Name: "block_cache_access_total", - Help: "The total number of db block cache access", - }, []string{"id", "type"}) -) - -// InitMetrics registers all metrics in this file -func InitMetrics(registry *prometheus.Registry) { - registry.MustRegister(dbSnapshotGauge) - registry.MustRegister(dbIteratorGauge) - registry.MustRegister(dbLevelCount) - registry.MustRegister(dbWriteBytes) - registry.MustRegister(dbReadBytes) - registry.MustRegister(dbWriteDelayDuration) - registry.MustRegister(dbWriteDelayCount) - registry.MustRegister(dbBlockCacheAccess) -} - -/* There are some metrics shared with pipeline sorter and pull-based-sink sort engine. */ - -// IteratorGauge returns dbIteratorGauge. -func IteratorGauge() *prometheus.GaugeVec { - return dbIteratorGauge -} - -// WriteDelayCount returns dbWriteDelayCount. -func WriteDelayCount() *prometheus.GaugeVec { - return dbWriteDelayCount -} - -// LevelCount returns dbLevelCount. -func LevelCount() *prometheus.GaugeVec { - return dbLevelCount -} - -// BlockCacheAccess returns dbBlockCacheAccess. -func BlockCacheAccess() *prometheus.GaugeVec { - return dbBlockCacheAccess -} diff --git a/pkg/db/pebble.go b/pkg/db/pebble.go index 77790d1bc3d..7de7c663ba8 100644 --- a/pkg/db/pebble.go +++ b/pkg/db/pebble.go @@ -27,9 +27,8 @@ import ( "github.com/cockroachdb/pebble/bloom" "github.com/pingcap/errors" "github.com/pingcap/log" - "github.com/pingcap/tiflow/cdc/sorter" - "github.com/pingcap/tiflow/cdc/sorter/encoding" "github.com/pingcap/tiflow/pkg/config" + "github.com/pingcap/tiflow/pkg/db/encoding" "github.com/pingcap/tiflow/pkg/retry" "go.uber.org/zap" ) @@ -197,43 +196,6 @@ func (p *pebbleDB) Close() error { return p.db.Close() } -// TODO: Update metrics once we switch to pebble, -// as some metrics are not applicable to pebble. -func (p *pebbleDB) CollectMetrics(i int) { - db := p.db - stats := db.Metrics() - id := strconv.Itoa(i) - sum := 0 - for i := range stats.Levels { - sum += int(stats.Levels[i].Size) - } - sorter.OnDiskDataSizeGauge. - WithLabelValues(id).Set(float64(stats.DiskSpaceUsage())) - sorter.InMemoryDataSizeGauge. - WithLabelValues(id).Set(float64(stats.BlockCache.Size)) - dbIteratorGauge. - WithLabelValues(id).Set(float64(stats.TableIters)) - dbWriteDelayCount. - WithLabelValues(id). - Set(float64(atomic.LoadInt64(&p.metricWriteStall.counter))) - stallDuration := p.metricWriteStall.duration.Load() - if stallDuration != nil && stallDuration.(time.Duration) != time.Duration(0) { - p.metricWriteStall.duration.Store(time.Duration(0)) - dbWriteDelayDuration. - WithLabelValues(id). - Set(stallDuration.(time.Duration).Seconds()) - } - metricLevelCount := dbLevelCount. - MustCurryWith(map[string]string{"id": id}) - for level, metric := range stats.Levels { - metricLevelCount.WithLabelValues(fmt.Sprint(level)).Set(float64(metric.NumFiles)) - } - dbBlockCacheAccess. - WithLabelValues(id, "hit").Set(float64(stats.BlockCache.Hits)) - dbBlockCacheAccess. - WithLabelValues(id, "miss").Set(float64(stats.BlockCache.Misses)) -} - type pebbleBatch struct { *pebble.Batch } diff --git a/pkg/db/pebble_test.go b/pkg/db/pebble_test.go index 50d40afb7c9..ff1135da18e 100644 --- a/pkg/db/pebble_test.go +++ b/pkg/db/pebble_test.go @@ -20,8 +20,8 @@ import ( "time" "github.com/cockroachdb/pebble" - "github.com/pingcap/tiflow/cdc/sorter/encoding" "github.com/pingcap/tiflow/pkg/config" + "github.com/pingcap/tiflow/pkg/db/encoding" "github.com/stretchr/testify/require" ) diff --git a/pkg/fsutil/disk_info_freebsd.go b/pkg/fsutil/disk_info_freebsd.go index 6711d23b3d9..b474a6aa1ff 100644 --- a/pkg/fsutil/disk_info_freebsd.go +++ b/pkg/fsutil/disk_info_freebsd.go @@ -12,7 +12,6 @@ // limitations under the License. // //go:build freebsd -// +build freebsd package fsutil diff --git a/pkg/fsutil/disk_info_generic.go b/pkg/fsutil/disk_info_generic.go index 2a5ded8b577..27aa1cf2052 100644 --- a/pkg/fsutil/disk_info_generic.go +++ b/pkg/fsutil/disk_info_generic.go @@ -12,7 +12,6 @@ // limitations under the License. // //go:build !freebsd -// +build !freebsd package fsutil diff --git a/pkg/fsutil/fadvise_linux.go b/pkg/fsutil/fadvise_linux.go index ab91b06f99f..39eaca817b9 100644 --- a/pkg/fsutil/fadvise_linux.go +++ b/pkg/fsutil/fadvise_linux.go @@ -12,7 +12,6 @@ // limitations under the License. // //go:build linux -// +build linux package fsutil diff --git a/pkg/fsutil/fadvise_other.go b/pkg/fsutil/fadvise_other.go index 26a3a8b10f5..2eb6fba322c 100644 --- a/pkg/fsutil/fadvise_other.go +++ b/pkg/fsutil/fadvise_other.go @@ -12,7 +12,6 @@ // limitations under the License. // //go:build !linux -// +build !linux package fsutil diff --git a/pkg/fsutil/preallocate_linux.go b/pkg/fsutil/preallocate_linux.go index fe15690da77..a395ea112e5 100644 --- a/pkg/fsutil/preallocate_linux.go +++ b/pkg/fsutil/preallocate_linux.go @@ -12,7 +12,6 @@ // limitations under the License. // //go:build linux -// +build linux package fsutil diff --git a/pkg/fsutil/preallocate_other.go b/pkg/fsutil/preallocate_other.go index 64fe99f5b9f..699007a9725 100644 --- a/pkg/fsutil/preallocate_other.go +++ b/pkg/fsutil/preallocate_other.go @@ -12,7 +12,6 @@ // limitations under the License. // //go:build !linux -// +build !linux package fsutil diff --git a/tests/integration_tests/sorter/run.sh b/tests/integration_tests/sorter/run.sh index b0997b742a9..9fdb6b6e7aa 100755 --- a/tests/integration_tests/sorter/run.sh +++ b/tests/integration_tests/sorter/run.sh @@ -54,39 +54,15 @@ function run() { cd $WORK_DIR - # FIXME: pull-based sink does not support unified sorter. - # echo "test unified sorter" - # - # start_ts=$(run_cdc_cli_tso_query ${UP_PD_HOST_1} ${UP_PD_PORT_1}) - # reset_and_prepare_data - # - # # Run cdc server with unified sorter. - # run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY --config $CUR/conf/ticdc_unified_sorter.toml - # - # TOPIC_NAME="ticdc-unified-sorter-test-$RANDOM" - # CF_NAME=$TOPIC_NAME - # case $SINK_TYPE in - # kafka) SINK_URI="kafka://127.0.0.1:9092/$TOPIC_NAME?protocol=open-protocol&partition-num=4&kafka-version=${KAFKA_VERSION}&max-message-bytes=10485760" ;; - # *) SINK_URI="mysql://normal:123456@127.0.0.1:3306/?max-txn-row=1" ;; - # esac - # run_cdc_cli changefeed create -c $CF_NAME --start-ts=$start_ts --sink-uri="$SINK_URI" --sort-engine="unified" - # if [ "$SINK_TYPE" == "kafka" ]; then - # run_kafka_consumer $WORK_DIR "kafka://127.0.0.1:9092/$TOPIC_NAME?protocol=open-protocol&partition-num=4&version=${KAFKA_VERSION}&max-message-bytes=10485760" - # fi - # - # run_test - # cleanup_process $CDC_BINARY - # run_cdc_cli unsafe reset --no-confirm --pd=http://${UP_PD_HOST_1}:${UP_PD_PORT_1} - - echo "test leveldb sorter" + echo "test db sorter" start_ts=$(run_cdc_cli_tso_query ${UP_PD_HOST_1} ${UP_PD_PORT_1}) reset_and_prepare_data - # Run cdc server with leveldb sorter. + # Run cdc server with db sorter. run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY - TOPIC_NAME="ticdc-leveldb-sorter-test-$RANDOM" + TOPIC_NAME="ticdc-db-sorter-test-$RANDOM" CF_NAME=$TOPIC_NAME case $SINK_TYPE in kafka) SINK_URI="kafka://127.0.0.1:9092/$TOPIC_NAME?protocol=open-protocol&partition-num=4&kafka-version=${KAFKA_VERSION}&max-message-bytes=10485760" ;; diff --git a/tests/mq_protocol_tests/framework/docker_compose_op.go b/tests/mq_protocol_tests/framework/docker_compose_op.go index 110dad3d19f..b1bf9ad162e 100644 --- a/tests/mq_protocol_tests/framework/docker_compose_op.go +++ b/tests/mq_protocol_tests/framework/docker_compose_op.go @@ -14,6 +14,7 @@ package framework import ( + "context" "database/sql" "fmt" "os" @@ -25,7 +26,6 @@ import ( cerrors "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/retry" "go.uber.org/zap" - "golang.org/x/net/context" ) // DockerComposeOperator represent a docker compose diff --git a/tests/utils/many_sorters_test/many_sorters.go b/tests/utils/many_sorters_test/many_sorters.go deleted file mode 100644 index 80f8d756b12..00000000000 --- a/tests/utils/many_sorters_test/many_sorters.go +++ /dev/null @@ -1,166 +0,0 @@ -// Copyright 2020 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package main - -import ( - "context" - "flag" - "fmt" - "math/rand" - "net/http" - "os" - "strings" - "sync/atomic" - - "github.com/pingcap/errors" - "github.com/pingcap/failpoint" - "github.com/pingcap/log" - "github.com/pingcap/tiflow/cdc/model" - "github.com/pingcap/tiflow/cdc/sorter" - "github.com/pingcap/tiflow/cdc/sorter/unified" - "github.com/pingcap/tiflow/pkg/config" - cerrors "github.com/pingcap/tiflow/pkg/errors" - "go.uber.org/zap" - "go.uber.org/zap/zapcore" - "golang.org/x/sync/errgroup" -) - -var ( - sorterDir = flag.String("dir", "./sorter", "temporary directory used for sorting") - numSorters = flag.Int("num-sorters", 256, "number of instances of sorters") - numEvents = flag.Int("num-events-per-sorter", 10000, "number of events sent to a sorter") - percentageResolves = flag.Int("percentage-resolve-events", 70, "percentage of resolved events") -) - -func main() { - flag.Parse() - err := failpoint.Enable("github.com/pingcap/tiflow/cdc/sorter/unified/sorterDebug", "return(true)") - if err != nil { - log.Fatal("Could not enable failpoint", zap.Error(err)) - } - log.SetLevel(zapcore.DebugLevel) - - conf := config.GetDefaultServerConfig() - conf.Sorter = &config.SorterConfig{ - NumConcurrentWorker: 8, - ChunkSizeLimit: 1 * 1024 * 1024 * 1024, - MaxMemoryPercentage: 60, - MaxMemoryConsumption: 16 * 1024 * 1024 * 1024, - NumWorkerPoolGoroutine: 16, - } - config.StoreGlobalServerConfig(conf) - - go func() { - _ = http.ListenAndServe("localhost:6060", nil) - }() - - err = os.MkdirAll(*sorterDir, 0o700) - if err != nil { - log.Error("sorter_stress_test:", zap.Error(err)) - } - - sorters := make([]sorter.EventSorter, *numSorters) - ctx0, cancel := context.WithCancel(context.Background()) - errg, ctx := errgroup.WithContext(ctx0) - - errg.Go(func() error { - return unified.RunWorkerPool(ctx) - }) - - var finishCount int32 - for i := 0; i < *numSorters; i++ { - sorters[i], err = unified.NewUnifiedSorter(*sorterDir, - model.DefaultChangeFeedID("test-cf"), fmt.Sprintf("test-%d", i), model.TableID(i)) - if err != nil { - log.Panic("many_sorters", zap.Error(err)) - } - finalI := i - - // run sorter - errg.Go(func() error { - return printError(sorters[finalI].Run(ctx)) - }) - - // run producer - errg.Go(func() error { - for j := 0; j < *numEvents; j++ { - select { - case <-ctx.Done(): - return ctx.Err() - default: - } - - ev := generateEvent(uint64(finalI), uint64(j<<5)) - sorters[finalI].AddEntry(ctx, ev) - } - sorters[finalI].AddEntry(ctx, model.NewResolvedPolymorphicEvent(uint64(finalI), uint64(((*numEvents)<<5)+1))) - return nil - }) - - // run consumer - errg.Go(func() error { - for { - var ev *model.PolymorphicEvent - select { - case <-ctx.Done(): - return ctx.Err() - case ev = <-sorters[finalI].Output(): - } - - if ev.CRTs == uint64(((*numEvents)<<5)+1) { - log.Info("Sorter finished", zap.Int("sorterID", finalI)) - if atomic.AddInt32(&finishCount, 1) == int32(*numSorters) { - log.Info("Many Sorters test finished, cancelling all goroutines") - cancel() - } - return nil - } - } - }) - } - - _ = printError(errg.Wait()) - if atomic.LoadInt32(&finishCount) == int32(*numSorters) { - log.Info("Test was successful!") - } -} - -func generateEvent(region uint64, ts uint64) *model.PolymorphicEvent { - r := rand.Int() % 100 - if r < *percentageResolves { - return model.NewResolvedPolymorphicEvent(region, ts) - } - return model.NewPolymorphicEvent(&model.RawKVEntry{ - OpType: model.OpTypePut, - Key: []byte("keykeykey"), - Value: []byte("valuevaluevalue"), - OldValue: nil, - StartTs: ts - 5, - CRTs: ts, - RegionID: region, - }) -} - -// printError is a helper for tracing errors on function returns -func printError(err error) error { - if err != nil && errors.Cause(err) != context.Canceled && - errors.Cause(err) != context.DeadlineExceeded && - !strings.Contains(err.Error(), "context canceled") && - !strings.Contains(err.Error(), "context deadline exceeded") && - cerrors.ErrWorkerPoolHandleCancelled.NotEqual(errors.Cause(err)) { - - log.Warn("Unified Sorter: Error detected", zap.Error(err)) - } - return err -} diff --git a/tests/utils/many_sorters_test/many_sorters_test.go b/tests/utils/many_sorters_test/many_sorters_test.go deleted file mode 100644 index 7ffeb2f99c5..00000000000 --- a/tests/utils/many_sorters_test/many_sorters_test.go +++ /dev/null @@ -1,20 +0,0 @@ -// Copyright 2020 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package main - -import "testing" - -func TestFoo(_ *testing.T) { - main() -} diff --git a/tests/utils/sorter_stress_test/sorter_stress.go b/tests/utils/sorter_stress_test/sorter_stress.go deleted file mode 100644 index beda91ceb11..00000000000 --- a/tests/utils/sorter_stress_test/sorter_stress.go +++ /dev/null @@ -1,170 +0,0 @@ -// Copyright 2020 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package main - -import ( - "context" - "flag" - "math/rand" - "net/http" - _ "net/http/pprof" // #nosec G108 - "os" - "strings" - - "github.com/pingcap/failpoint" - "github.com/pingcap/log" - "github.com/pingcap/tiflow/cdc/model" - "github.com/pingcap/tiflow/cdc/sorter" - "github.com/pingcap/tiflow/cdc/sorter/unified" - "github.com/pingcap/tiflow/pkg/config" - "go.uber.org/zap" - "golang.org/x/sync/errgroup" -) - -var ( - sorterDir = flag.String("dir", "./sorter", "temporary directory used for sorting") - numBatches = flag.Int("num-batches", 256, "number of batches of ordered events") - msgsPerBatch = flag.Int("num-messages-per-batch", 1024, "number of events in a batch") - bytesPerMsg = flag.Int("bytes-per-message", 1024, "number of bytes in an event") -) - -func main() { - flag.Parse() - log.SetLevel(zap.DebugLevel) - err := failpoint.Enable("github.com/pingcap/tiflow/cdc/sorter/unified/sorterDebug", "return(true)") - if err != nil { - log.Fatal("Could not enable failpoint", zap.Error(err)) - } - - conf := config.GetDefaultServerConfig() - conf.Sorter = &config.SorterConfig{ - NumConcurrentWorker: 8, - ChunkSizeLimit: 1 * 1024 * 1024 * 1024, - MaxMemoryPercentage: 60, - MaxMemoryConsumption: 16 * 1024 * 1024 * 1024, - } - config.StoreGlobalServerConfig(conf) - - go func() { - _ = http.ListenAndServe("localhost:6060", nil) - }() - - err = os.MkdirAll(*sorterDir, 0o700) - if err != nil { - log.Error("sorter_stress_test:", zap.Error(err)) - } - - sorter, err := unified.NewUnifiedSorter(*sorterDir, model.DefaultChangeFeedID("test-cf"), "test", 0) - if err != nil { - log.Panic("sorter_stress_test:", zap.Error(err)) - } - - ctx1, cancel := context.WithCancel(context.Background()) - - eg, ctx := errgroup.WithContext(ctx1) - - eg.Go(func() error { - return unified.RunWorkerPool(ctx) - }) - - eg.Go(func() error { - return sorter.Run(ctx) - }) - - // launch the consumer - eg.Go(func() error { - counter := 0 - lastTs := uint64(0) - for { - select { - case <-ctx.Done(): - return ctx.Err() - case event := <-sorter.Output(): - if event.RawKV.OpType != model.OpTypeResolved { - if event.CRTs < lastTs { - panic("regressed") - } - lastTs = event.CRTs - counter += 1 - if counter%10000 == 0 { - log.Debug("Messages received", zap.Int("counter", counter)) - } - if counter >= *numBatches**msgsPerBatch { - log.Debug("Unified Sorter test successful") - cancel() - return nil - } - } - } - } - }) - - eg1 := errgroup.Group{} - for i := 0; i < *numBatches; i++ { - eg1.Go(func() error { - generateGroup(ctx, sorter) - return nil - }) - } - - err = eg1.Wait() - if err != nil { - log.Error("sorter_stress_test:", zap.Error(err)) - } - - sorter.AddEntry(ctx, model.NewResolvedPolymorphicEvent(0, uint64((*msgsPerBatch<<5)+256))) - - err = eg.Wait() - if err != nil { - if strings.Contains(err.Error(), "context canceled") { - return - } - log.Error("sorter_stress_test:", zap.Error(err)) - } -} - -func generateGroup(ctx context.Context, sorter sorter.EventSorter) { - for i := 0; i < *msgsPerBatch; i++ { - ts := (i << 5) + rand.Intn(256) - event := model.NewPolymorphicEvent(newMockRawKV(uint64(ts))) - sorter.AddEntry(ctx, event) - } -} - -var ( - key = []byte(randSeq(10)) - value = []byte(randSeq(*bytesPerMsg)) -) - -func newMockRawKV(ts uint64) *model.RawKVEntry { - return &model.RawKVEntry{ - OpType: model.OpTypePut, - Key: key, - Value: value, - OldValue: nil, - StartTs: ts - 5, - CRTs: ts, - RegionID: 0, - } -} - -var letters = []rune("abcdefghijklmnopqrstuvwxyzABCDEFGHIJKLMNOPQRSTUVWXYZ") - -func randSeq(n int) string { - b := make([]rune, n) - for i := range b { - b[i] = letters[rand.Intn(len(letters))] - } - return string(b) -}