diff --git a/cdc/http_handler.go b/cdc/http_handler.go index a7bc397367c..291cb97bf29 100644 --- a/cdc/http_handler.go +++ b/cdc/http_handler.go @@ -26,7 +26,6 @@ import ( "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/cdc/owner" - "github.com/pingcap/tiflow/pkg/config" cerror "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/logutil" "go.etcd.io/etcd/clientv3/concurrency" @@ -76,42 +75,17 @@ func (s *Server) handleResignOwner(w http.ResponseWriter, req *http.Request) { writeError(w, http.StatusBadRequest, cerror.ErrSupportPostOnly.GenWithStackByArgs()) return } - if config.NewReplicaImpl { - if s.captureV2 == nil { - // for test only - handleOwnerResp(w, concurrency.ErrElectionNotLeader) - return - } - err := s.captureV2.OperateOwnerUnderLock(func(owner *owner.Owner) error { - owner.AsyncStop() - return nil - }) - handleOwnerResp(w, err) - return - } - s.ownerLock.RLock() - if s.owner == nil { + if s.capture == nil { + // for test only handleOwnerResp(w, concurrency.ErrElectionNotLeader) - s.ownerLock.RUnlock() return } - // Resign is a complex process that needs to be synchronized because - // it happens in two separate goroutines - // - // Imagine that we have goroutines A and B - // A1. Notify the owner to exit - // B1. The owner exits gracefully - // A2. Delete the leader key until the owner has exited - // B2. Restart to campaign - // - // A2 must occur between B1 and B2, so we register the Resign process - // as the stepDown function which is called when the owner exited. - s.owner.Close(req.Context(), func(ctx context.Context) error { - return s.capture.Resign(ctx) + err := s.capture.OperateOwnerUnderLock(func(owner *owner.Owner) error { + owner.AsyncStop() + return nil }) - s.ownerLock.RUnlock() - s.setOwner(nil) - handleOwnerResp(w, nil) + handleOwnerResp(w, err) + return } func (s *Server) handleChangefeedAdmin(w http.ResponseWriter, req *http.Request) { @@ -119,19 +93,11 @@ func (s *Server) handleChangefeedAdmin(w http.ResponseWriter, req *http.Request) writeError(w, http.StatusBadRequest, cerror.ErrSupportPostOnly.GenWithStackByArgs()) return } - if !config.NewReplicaImpl { - s.ownerLock.RLock() - defer s.ownerLock.RUnlock() - if s.owner == nil { - handleOwnerResp(w, concurrency.ErrElectionNotLeader) - return - } - } else { - if s.captureV2 == nil { - // for test only - handleOwnerResp(w, concurrency.ErrElectionNotLeader) - return - } + + if s.capture == nil { + // for test only + handleOwnerResp(w, concurrency.ErrElectionNotLeader) + return } err := req.ParseForm() @@ -160,14 +126,12 @@ func (s *Server) handleChangefeedAdmin(w http.ResponseWriter, req *http.Request) Type: model.AdminJobType(typ), Opts: opts, } - if config.NewReplicaImpl { - err = s.captureV2.OperateOwnerUnderLock(func(owner *owner.Owner) error { - owner.EnqueueJob(job) - return nil - }) - } else { - err = s.owner.EnqueueJob(job) - } + + err = s.capture.OperateOwnerUnderLock(func(owner *owner.Owner) error { + owner.EnqueueJob(job) + return nil + }) + handleOwnerResp(w, err) } @@ -176,19 +140,10 @@ func (s *Server) handleRebalanceTrigger(w http.ResponseWriter, req *http.Request writeError(w, http.StatusBadRequest, cerror.ErrSupportPostOnly.GenWithStackByArgs()) return } - if !config.NewReplicaImpl { - s.ownerLock.RLock() - defer s.ownerLock.RUnlock() - if s.owner == nil { - handleOwnerResp(w, concurrency.ErrElectionNotLeader) - return - } - } else { - if s.captureV2 == nil { - // for test only - handleOwnerResp(w, concurrency.ErrElectionNotLeader) - return - } + if s.capture == nil { + // for test only + handleOwnerResp(w, concurrency.ErrElectionNotLeader) + return } err := req.ParseForm() @@ -202,14 +157,11 @@ func (s *Server) handleRebalanceTrigger(w http.ResponseWriter, req *http.Request cerror.ErrAPIInvalidParam.GenWithStack("invalid changefeed id: %s", changefeedID)) return } - if config.NewReplicaImpl { - err = s.captureV2.OperateOwnerUnderLock(func(owner *owner.Owner) error { - owner.TriggerRebalance(changefeedID) - return nil - }) - } else { - s.owner.TriggerRebalance(changefeedID) - } + err = s.capture.OperateOwnerUnderLock(func(owner *owner.Owner) error { + owner.TriggerRebalance(changefeedID) + return nil + }) + handleOwnerResp(w, err) } @@ -218,19 +170,11 @@ func (s *Server) handleMoveTable(w http.ResponseWriter, req *http.Request) { writeError(w, http.StatusBadRequest, cerror.ErrSupportPostOnly.GenWithStackByArgs()) return } - if !config.NewReplicaImpl { - s.ownerLock.RLock() - defer s.ownerLock.RUnlock() - if s.owner == nil { - handleOwnerResp(w, concurrency.ErrElectionNotLeader) - return - } - } else { - if s.captureV2 == nil { - // for test only - handleOwnerResp(w, concurrency.ErrElectionNotLeader) - return - } + + if s.capture == nil { + // for test only + handleOwnerResp(w, concurrency.ErrElectionNotLeader) + return } err := req.ParseForm() @@ -257,14 +201,12 @@ func (s *Server) handleMoveTable(w http.ResponseWriter, req *http.Request) { cerror.ErrAPIInvalidParam.GenWithStack("invalid tableID: %s", tableIDStr)) return } - if config.NewReplicaImpl { - err = s.captureV2.OperateOwnerUnderLock(func(owner *owner.Owner) error { - owner.ManualSchedule(changefeedID, to, tableID) - return nil - }) - } else { - s.owner.ManualSchedule(changefeedID, to, tableID) - } + + err = s.capture.OperateOwnerUnderLock(func(owner *owner.Owner) error { + owner.ManualSchedule(changefeedID, to, tableID) + return nil + }) + handleOwnerResp(w, err) } @@ -273,19 +215,10 @@ func (s *Server) handleChangefeedQuery(w http.ResponseWriter, req *http.Request) writeError(w, http.StatusBadRequest, cerror.ErrSupportPostOnly.GenWithStackByArgs()) return } - if !config.NewReplicaImpl { - s.ownerLock.RLock() - defer s.ownerLock.RUnlock() - if s.owner == nil { - handleOwnerResp(w, concurrency.ErrElectionNotLeader) - return - } - } else { - if s.captureV2 == nil { - // for test only - handleOwnerResp(w, concurrency.ErrElectionNotLeader) - return - } + if s.capture == nil { + // for test only + handleOwnerResp(w, concurrency.ErrElectionNotLeader) + return } err := req.ParseForm() diff --git a/cdc/http_status.go b/cdc/http_status.go index 14265b14212..efddc52e453 100644 --- a/cdc/http_status.go +++ b/cdc/http_status.go @@ -113,31 +113,10 @@ func (s *Server) writeEtcdInfo(ctx context.Context, cli *kv.CDCEtcdClient, w io. } func (s *Server) handleDebugInfo(w http.ResponseWriter, req *http.Request) { - if config.NewReplicaImpl { - s.captureV2.WriteDebugInfo(w) - fmt.Fprintf(w, "\n\n*** etcd info ***:\n\n") - s.writeEtcdInfo(req.Context(), s.etcdClient, w) - return - } - s.ownerLock.RLock() - defer s.ownerLock.RUnlock() - if s.owner != nil { - fmt.Fprintf(w, "\n\n*** owner info ***:\n\n") - s.owner.writeDebugInfo(w) - } - - fmt.Fprintf(w, "\n\n*** processors info ***:\n\n") - if config.NewReplicaImpl { - s.capture.processorManager.WriteDebugInfo(w) - } else { - for _, p := range s.capture.processors { - p.writeDebugInfo(w) - fmt.Fprintf(w, "\n") - } - } - + s.capture.WriteDebugInfo(w) fmt.Fprintf(w, "\n\n*** etcd info ***:\n\n") - s.writeEtcdInfo(req.Context(), &s.capture.etcdClient, w) + s.writeEtcdInfo(req.Context(), s.etcdClient, w) + return } func (s *Server) handleStatus(w http.ResponseWriter, req *http.Request) { @@ -146,21 +125,13 @@ func (s *Server) handleStatus(w http.ResponseWriter, req *http.Request) { GitHash: version.GitHash, Pid: os.Getpid(), } - if config.NewReplicaImpl { - if s.captureV2 != nil { - st.ID = s.captureV2.Info().ID - st.IsOwner = s.captureV2.IsOwner() - } - writeData(w, st) - return - } - s.ownerLock.RLock() - defer s.ownerLock.RUnlock() + if s.capture != nil { - st.ID = s.capture.info.ID + st.ID = s.capture.Info().ID + st.IsOwner = s.capture.IsOwner() } - st.IsOwner = s.owner != nil writeData(w, st) + return } func writeInternalServerError(w http.ResponseWriter, err error) { diff --git a/cdc/metrics.go b/cdc/metrics.go index c05241fec33..adeee63d056 100644 --- a/cdc/metrics.go +++ b/cdc/metrics.go @@ -20,7 +20,6 @@ import ( "github.com/pingcap/tiflow/cdc/processor" tablepipeline "github.com/pingcap/tiflow/cdc/processor/pipeline" "github.com/pingcap/tiflow/cdc/puller" - "github.com/pingcap/tiflow/cdc/puller/sorter" "github.com/pingcap/tiflow/cdc/sink" "github.com/pingcap/tiflow/pkg/orchestrator" "github.com/prometheus/client_golang/prometheus" @@ -37,20 +36,8 @@ func init() { sink.InitMetrics(registry) entry.InitMetrics(registry) orchestrator.InitMetrics(registry) -<<<<<<< HEAD - sorter.InitMetrics(registry) - if config.NewReplicaImpl { - processor.InitMetrics(registry) - tablepipeline.InitMetrics(registry) - owner.InitMetrics(registry) - } else { - initProcessorMetrics(registry) - initOwnerMetrics(registry) - } -======= processor.InitMetrics(registry) tablepipeline.InitMetrics(registry) owner.InitMetrics(registry) ->>>>>>> 6591f62df (Clean old owner and old processor in release 5.2 branch (#4019)) initServerMetrics(registry) } diff --git a/cdc/owner.go b/cdc/owner.go deleted file mode 100644 index ef1537d2a53..00000000000 --- a/cdc/owner.go +++ /dev/null @@ -1,1734 +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 cdc - -import ( - "context" - "fmt" - "io" - "math" - "sync" - "sync/atomic" - "time" - - "github.com/pingcap/errors" - "github.com/pingcap/failpoint" - "github.com/pingcap/log" - "github.com/pingcap/tidb/store/tikv/oracle" - "github.com/pingcap/tiflow/cdc/entry" - "github.com/pingcap/tiflow/cdc/kv" - "github.com/pingcap/tiflow/cdc/model" - "github.com/pingcap/tiflow/cdc/sink" - "github.com/pingcap/tiflow/pkg/config" - "github.com/pingcap/tiflow/pkg/cyclic/mark" - cerror "github.com/pingcap/tiflow/pkg/errors" - "github.com/pingcap/tiflow/pkg/filter" - "github.com/pingcap/tiflow/pkg/notify" - "github.com/pingcap/tiflow/pkg/scheduler" - "github.com/pingcap/tiflow/pkg/txnutil/gc" - "github.com/pingcap/tiflow/pkg/util" - pd "github.com/tikv/pd/client" - "go.etcd.io/etcd/clientv3" - "go.etcd.io/etcd/clientv3/concurrency" - "go.etcd.io/etcd/mvcc" - "go.uber.org/zap" - "golang.org/x/time/rate" -) - -type ownership struct { - lastTickTime time.Time - tickTime time.Duration -} - -func newOwnership(tickTime time.Duration) ownership { - minTickTime := 5 * time.Second - if tickTime > minTickTime { - log.Panic("ownership counter must be incearsed every 5 seconds") - } - return ownership{ - tickTime: minTickTime, - } -} - -func (o *ownership) inc() { - now := time.Now() - if now.Sub(o.lastTickTime) > o.tickTime { - // Keep the value of promtheus expression `rate(counter)` = 1 - // Please also change alert rule in ticdc.rules.yml when change the expression value. - ownershipCounter.Add(float64(o.tickTime / time.Second)) - o.lastTickTime = now - } -} - -type minGCSafePointCacheEntry struct { - ts model.Ts - lastUpdated time.Time -} - -func (o *Owner) getMinGCSafePointCache(ctx context.Context) model.Ts { - if time.Now().After(o.minGCSafePointCache.lastUpdated.Add(MinGCSafePointCacheUpdateInterval)) { - physicalTs, logicalTs, err := o.pdClient.GetTS(ctx) - if err != nil { - log.Warn("Fail to update minGCSafePointCache.", zap.Error(err)) - return o.minGCSafePointCache.ts - } - o.minGCSafePointCache.ts = oracle.ComposeTS(physicalTs-(o.gcTTL*1000), logicalTs) - - // o.pdGCSafePoint pd is the smallest gcSafePoint across all services. - // If tikv_gc_life_time > gcTTL, means that tikv_gc_safe_point < o.minGCSafePointCache.ts here. - // It also means that pd.pdGCSafePoint < o.minGCSafePointCache.ts here, we should use its value as the min value. - // This ensures that when tikv_gc_life_time > gcTTL , cdc will not advance the gcSafePoint. - if o.pdGCSafePoint < o.minGCSafePointCache.ts { - o.minGCSafePointCache.ts = o.pdGCSafePoint - } - - o.minGCSafePointCache.lastUpdated = time.Now() - } - return o.minGCSafePointCache.ts -} - -// Owner manages the cdc cluster -type Owner struct { - done chan struct{} - session *concurrency.Session - changeFeeds map[model.ChangeFeedID]*changeFeed - // failInitFeeds record changefeeds that meet error during initialization - failInitFeeds map[model.ChangeFeedID]struct{} - // stoppedFeeds record changefeeds that meet running error - stoppedFeeds map[model.ChangeFeedID]*model.ChangeFeedStatus - rebalanceTigger map[model.ChangeFeedID]bool - rebalanceForAllChangefeed bool - manualScheduleCommand map[model.ChangeFeedID][]*model.MoveTableJob - rebalanceMu sync.Mutex - - cfRWriter ChangeFeedRWriter - - l sync.RWMutex - - pdEndpoints []string - grpcPool kv.GrpcPool - pdClient pd.Client - etcdClient kv.CDCEtcdClient - - captureLoaded int32 - captures map[model.CaptureID]*model.CaptureInfo - - adminJobs []model.AdminJob - adminJobsLock sync.Mutex - - stepDown func(ctx context.Context) error - - // gcTTL is the ttl of cdc gc safepoint ttl. - gcTTL int64 - // last update gc safepoint time. zero time means has not updated or cleared - gcSafepointLastUpdate time.Time - // stores the ts obtained from PD and is updated every MinGCSafePointCacheUpdateInterval. - minGCSafePointCache minGCSafePointCacheEntry - // stores the actual gcSafePoint stored in pd - pdGCSafePoint model.Ts - // record last time that flushes all changefeeds' replication status - lastFlushChangefeeds time.Time - flushChangefeedInterval time.Duration - feedChangeNotifier *notify.Notifier -} - -const ( - // CDCServiceSafePointID is the ID of CDC service in pd.UpdateServiceGCSafePoint. - CDCServiceSafePointID = "ticdc" - // GCSafepointUpdateInterval is the minimual interval that CDC can update gc safepoint - GCSafepointUpdateInterval = 2 * time.Second - // MinGCSafePointCacheUpdateInterval is the interval that update minGCSafePointCache - MinGCSafePointCacheUpdateInterval = time.Second * 2 -) - -// NewOwner creates a new Owner instance -func NewOwner( - ctx context.Context, - pdClient pd.Client, - grpcPool kv.GrpcPool, - sess *concurrency.Session, - gcTTL int64, - flushChangefeedInterval time.Duration, -) (*Owner, error) { - cli := kv.NewCDCEtcdClient(ctx, sess.Client()) - endpoints := sess.Client().Endpoints() - - failpoint.Inject("ownerFlushIntervalInject", func(val failpoint.Value) { - flushChangefeedInterval = time.Millisecond * time.Duration(val.(int)) - }) - - owner := &Owner{ - done: make(chan struct{}), - session: sess, - pdClient: pdClient, - grpcPool: grpcPool, - changeFeeds: make(map[model.ChangeFeedID]*changeFeed), - failInitFeeds: make(map[model.ChangeFeedID]struct{}), - stoppedFeeds: make(map[model.ChangeFeedID]*model.ChangeFeedStatus), - captures: make(map[model.CaptureID]*model.CaptureInfo), - rebalanceTigger: make(map[model.ChangeFeedID]bool), - manualScheduleCommand: make(map[model.ChangeFeedID][]*model.MoveTableJob), - pdEndpoints: endpoints, - cfRWriter: cli, - etcdClient: cli, - gcTTL: gcTTL, - flushChangefeedInterval: flushChangefeedInterval, - feedChangeNotifier: new(notify.Notifier), - } - - return owner, nil -} - -func (o *Owner) addCapture(_ context.Context, info *model.CaptureInfo) { - o.l.Lock() - o.captures[info.ID] = info - o.l.Unlock() - o.rebalanceMu.Lock() - o.rebalanceForAllChangefeed = true - o.rebalanceMu.Unlock() -} - -// When a table is moved from one capture to another, the workflow is as follows -// 1. Owner deletes the table from the original capture (we call it capture-1), -// and adds an table operation record in the task status -// 2. The processor in capture-1 reads the operation record, and waits the table -// checkpoint ts reaches the boundary ts in operation, which often equals to -// the global resovled ts, larger the current checkpoint ts of this table. -// 3. After table checkpoint ts reaches boundary ts, capture-1 marks the table -// operation as finished. -// 4. Owner reads the finished mark and re-dispatches this table to another capture. -// -// When capture-1 crashes between step-2 and step-3, this function should be -// called to let owner re dispatch the table. Besides owner could also crash at -// the same time, in that case this function should also be called. In addtition, -// this function only handles move table job: 1) the add table job persists both -// table replicaInfo and operation, we can recover enough information from table -// replicaInfo; 2) if a table is deleted from a capture and that capture crashes, -// we just ignore this table. -func (o *Owner) rebuildTableFromOperations(cf *changeFeed, taskStatus *model.TaskStatus, startTs uint64) { - for tableID, op := range taskStatus.Operation { - if op.Delete && op.Flag&model.OperFlagMoveTable > 0 { - cf.orphanTables[tableID] = startTs - if job, ok := cf.moveTableJobs[tableID]; ok { - log.Info("remove outdated move table job", zap.Reflect("job", job), zap.Uint64("start-ts", startTs)) - delete(cf.moveTableJobs, tableID) - } - } - } -} - -func (o *Owner) removeCapture(ctx context.Context, info *model.CaptureInfo) { - o.l.Lock() - defer o.l.Unlock() - - delete(o.captures, info.ID) - - for _, feed := range o.changeFeeds { - task, ok := feed.taskStatus[info.ID] - if !ok { - log.Warn("task status not found", zap.String("capture-id", info.ID), zap.String("changefeed", feed.id)) - continue - } - var startTs uint64 - pos, ok := feed.taskPositions[info.ID] - if ok { - startTs = pos.CheckPointTs - } else { - log.Warn("task position not found, fallback to use changefeed checkpointts", - zap.String("capture-id", info.ID), zap.String("changefeed", feed.id)) - // maybe the processor hasn't added table yet, fallback to use the - // global checkpoint ts as the start ts of the table. - startTs = feed.status.CheckpointTs - } - - for tableID, replicaInfo := range task.Tables { - feed.orphanTables[tableID] = startTs - if startTs < replicaInfo.StartTs { - log.Warn("table startTs not consistent", - zap.Uint64("table-start-ts", replicaInfo.StartTs), - zap.Uint64("checkpoint-ts", startTs), - zap.Reflect("status", feed.status)) - feed.orphanTables[tableID] = replicaInfo.StartTs - } - } - - o.rebuildTableFromOperations(feed, task, startTs) - - if err := o.etcdClient.LeaseGuardDeleteTaskStatus(ctx, feed.id, info.ID, o.session.Lease()); err != nil { - log.Warn("failed to delete task status", - zap.String("capture-id", info.ID), zap.String("changefeed", feed.id), zap.Error(err)) - } - if err := o.etcdClient.LeaseGuardDeleteTaskPosition(ctx, feed.id, info.ID, o.session.Lease()); err != nil { - log.Warn("failed to delete task position", - zap.String("capture-id", info.ID), zap.String("changefeed", feed.id), zap.Error(err)) - } - if err := o.etcdClient.LeaseGuardDeleteTaskWorkload(ctx, feed.id, info.ID, o.session.Lease()); err != nil { - log.Warn("failed to delete task workload", - zap.String("capture-id", info.ID), zap.String("changefeed", feed.id), zap.Error(err)) - } - ownerMaintainTableNumGauge.DeleteLabelValues(feed.id, info.AdvertiseAddr, maintainTableTypeTotal) - ownerMaintainTableNumGauge.DeleteLabelValues(feed.id, info.AdvertiseAddr, maintainTableTypeWip) - } -} - -func (o *Owner) addOrphanTable(cid model.CaptureID, tableID model.TableID, startTs model.Ts) { - if cf, ok := o.changeFeeds[cid]; ok { - cf.orphanTables[tableID] = startTs - } else { - log.Warn("changefeed not found", zap.String("changefeed", cid)) - } -} - -func (o *Owner) newChangeFeed( - ctx context.Context, - id model.ChangeFeedID, - processorsInfos model.ProcessorsInfos, - taskPositions map[string]*model.TaskPosition, - info *model.ChangeFeedInfo, - checkpointTs uint64) (cf *changeFeed, resultErr error) { - log.Info("Find new changefeed", zap.Stringer("info", info), - zap.String("changefeed", id), zap.Uint64("checkpoint ts", checkpointTs)) - if info.Config.CheckGCSafePoint { - ensureTTL := int64(10 * 60) - err := gc.EnsureChangefeedStartTsSafety( - ctx, o.pdClient, id, ensureTTL, checkpointTs) - if err != nil { - return nil, errors.Trace(err) - } - } - failpoint.Inject("NewChangefeedNoRetryError", func() { - failpoint.Return(nil, cerror.ErrStartTsBeforeGC.GenWithStackByArgs(checkpointTs-300, checkpointTs)) - }) - - failpoint.Inject("NewChangefeedRetryError", func() { - failpoint.Return(nil, errors.New("failpoint injected retriable error")) - }) - - kvStore, err := util.KVStorageFromCtx(ctx) - if err != nil { - return nil, errors.Trace(err) - } - meta, err := kv.GetSnapshotMeta(kvStore, checkpointTs) - if err != nil { - return nil, errors.Trace(err) - } - schemaSnap, err := entry.NewSingleSchemaSnapshotFromMeta(meta, checkpointTs, info.Config.ForceReplicate) - if err != nil { - return nil, errors.Trace(err) - } - - filter, err := filter.NewFilter(info.Config) - if err != nil { - return nil, errors.Trace(err) - } - - ddlHandler := newDDLHandler(o.pdClient, o.grpcPool, kvStore, checkpointTs) - defer func() { - if resultErr != nil { - ddlHandler.Close() - } - }() - - existingTables := make(map[model.TableID]model.Ts) - for captureID, taskStatus := range processorsInfos { - var checkpointTs uint64 - if pos, exist := taskPositions[captureID]; exist { - checkpointTs = pos.CheckPointTs - } - for tableID, replicaInfo := range taskStatus.Tables { - if replicaInfo.StartTs > checkpointTs { - checkpointTs = replicaInfo.StartTs - } - existingTables[tableID] = checkpointTs - } - } - - ctx, cancel := context.WithCancel(ctx) - defer func() { - if resultErr != nil { - cancel() - } - }() - schemas := make(map[model.SchemaID]tableIDMap) - tables := make(map[model.TableID]model.TableName) - partitions := make(map[model.TableID][]int64) - orphanTables := make(map[model.TableID]model.Ts) - sinkTableInfo := make([]*model.SimpleTableInfo, len(schemaSnap.CloneTables())) - j := 0 - for tid, table := range schemaSnap.CloneTables() { - j++ - if filter.ShouldIgnoreTable(table.Schema, table.Table) { - continue - } - if info.Config.Cyclic.IsEnabled() && mark.IsMarkTable(table.Schema, table.Table) { - // skip the mark table if cyclic is enabled - continue - } - - tables[tid] = table - schema, ok := schemaSnap.SchemaByTableID(tid) - if !ok { - log.Warn("schema not found for table", zap.Int64("tid", tid)) - } else { - sid := schema.ID - if _, ok := schemas[sid]; !ok { - schemas[sid] = make(tableIDMap) - } - schemas[sid][tid] = struct{}{} - } - tblInfo, ok := schemaSnap.TableByID(tid) - if !ok { - log.Warn("table not found for table ID", zap.Int64("tid", tid)) - continue - } - if !tblInfo.IsEligible(info.Config.ForceReplicate) { - log.Warn("skip ineligible table", zap.Int64("tid", tid), zap.Stringer("table", table)) - continue - } - // `existingTables` are tables dispatched to a processor, however the - // capture that this processor belongs to could have crashed or exited. - // So we check this before task dispatching, but after the update of - // changefeed schema information. - if ts, ok := existingTables[tid]; ok { - log.Info("ignore known table", zap.Int64("tid", tid), zap.Stringer("table", table), zap.Uint64("ts", ts)) - continue - } - if pi := tblInfo.GetPartitionInfo(); pi != nil { - delete(partitions, tid) - for _, partition := range pi.Definitions { - id := partition.ID - partitions[tid] = append(partitions[tid], id) - if ts, ok := existingTables[id]; ok { - log.Info("ignore known table partition", zap.Int64("tid", tid), zap.Int64("partitionID", id), zap.Stringer("table", table), zap.Uint64("ts", ts)) - continue - } - orphanTables[id] = checkpointTs - } - } else { - orphanTables[tid] = checkpointTs - } - - sinkTableInfo[j-1] = new(model.SimpleTableInfo) - sinkTableInfo[j-1].TableID = tid - sinkTableInfo[j-1].ColumnInfo = make([]*model.ColumnInfo, len(tblInfo.Cols())) - sinkTableInfo[j-1].Schema = table.Schema - sinkTableInfo[j-1].Table = table.Table - - for i, colInfo := range tblInfo.Cols() { - sinkTableInfo[j-1].ColumnInfo[i] = new(model.ColumnInfo) - sinkTableInfo[j-1].ColumnInfo[i].FromTiColumnInfo(colInfo) - } - - } - errCh := make(chan error, 1) - - primarySink, err := sink.NewSink(ctx, id, info.SinkURI, filter, info.Config, info.Opts, errCh) - if err != nil { - return nil, errors.Trace(err) - } - defer func() { - if resultErr != nil && primarySink != nil { - // The Close of backend sink here doesn't use context, it is ok to pass - // a canceled context here. - primarySink.Close(ctx) - } - }() - go func() { - var err error - select { - case <-ctx.Done(): - case err = <-errCh: - cancel() - } - if err != nil && errors.Cause(err) != context.Canceled { - log.Error("error on running changefeed", zap.Error(err), zap.String("changefeed", id)) - } else { - log.Info("changefeed exited", zap.String("changfeed", id)) - } - }() - - err = primarySink.Initialize(ctx, sinkTableInfo) - if err != nil { - log.Error("error on running owner", zap.Error(err)) - } - - var syncpointStore sink.SyncpointStore - if info.SyncPointEnabled { - syncpointStore, err = sink.NewSyncpointStore(ctx, id, info.SinkURI) - if err != nil { - return nil, errors.Trace(err) - } - } - - cf = &changeFeed{ - info: info, - id: id, - ddlHandler: ddlHandler, - schema: schemaSnap, - schemas: schemas, - tables: tables, - partitions: partitions, - orphanTables: orphanTables, - toCleanTables: make(map[model.TableID]model.Ts), - status: &model.ChangeFeedStatus{ - ResolvedTs: 0, - CheckpointTs: checkpointTs, - }, - appliedCheckpointTs: checkpointTs, - scheduler: scheduler.NewScheduler(info.Config.Scheduler.Tp), - ddlState: model.ChangeFeedSyncDML, - ddlExecutedTs: checkpointTs, - targetTs: info.GetTargetTs(), - ddlTs: 0, - updateResolvedTs: true, - startTimer: make(chan bool), - syncpointStore: syncpointStore, - syncCancel: nil, - taskStatus: processorsInfos, - taskPositions: taskPositions, - etcdCli: o.etcdClient, - leaseID: o.session.Lease(), - filter: filter, - sink: primarySink, - cyclicEnabled: info.Config.Cyclic.IsEnabled(), - lastRebalanceTime: time.Now(), - cancel: cancel, - } - return cf, nil -} - -// This is a compatibility hack between v4.0.0 and v4.0.1 -// This function will try to decode the task status, if that throw a unmarshal error, -// it will remove the invalid task status -func (o *Owner) checkAndCleanTasksInfo(ctx context.Context) error { - _, details, err := o.cfRWriter.GetChangeFeeds(ctx) - if err != nil { - return err - } - cleaned := false - for changefeedID := range details { - _, err := o.cfRWriter.GetAllTaskStatus(ctx, changefeedID) - if err != nil { - if cerror.ErrDecodeFailed.NotEqual(err) { - return errors.Trace(err) - } - err := o.cfRWriter.LeaseGuardRemoveAllTaskStatus(ctx, changefeedID, o.session.Lease()) - if err != nil { - return errors.Trace(err) - } - cleaned = true - } - } - if cleaned { - log.Warn("the task status is outdated, clean them") - } - return nil -} - -func (o *Owner) loadChangeFeeds(ctx context.Context) error { - _, details, err := o.cfRWriter.GetChangeFeeds(ctx) - if err != nil { - return err - } - errorFeeds := make(map[model.ChangeFeedID]*model.RunningError) - for changeFeedID, cfInfoRawValue := range details { - taskStatus, err := o.cfRWriter.GetAllTaskStatus(ctx, changeFeedID) - if err != nil { - return err - } - taskPositions, err := o.cfRWriter.GetAllTaskPositions(ctx, changeFeedID) - if err != nil { - return err - } - if cf, exist := o.changeFeeds[changeFeedID]; exist { - cf.updateProcessorInfos(taskStatus, taskPositions) - for _, pos := range taskPositions { - // TODO: only record error of one capture, - // is it necessary to record all captures' error - if pos.Error != nil { - errorFeeds[changeFeedID] = pos.Error - break - } - } - continue - } - - // we find a new changefeed, init changefeed here. - cfInfo := &model.ChangeFeedInfo{} - err = cfInfo.Unmarshal(cfInfoRawValue.Value) - if err != nil { - return err - } - if cfInfo.State == model.StateFailed { - if _, ok := o.failInitFeeds[changeFeedID]; ok { - continue - } - log.Warn("changefeed is not in normal state", zap.String("changefeed", changeFeedID)) - o.failInitFeeds[changeFeedID] = struct{}{} - continue - } - if _, ok := o.failInitFeeds[changeFeedID]; ok { - log.Info("changefeed recovered from failure", zap.String("changefeed", changeFeedID)) - delete(o.failInitFeeds, changeFeedID) - } - needSave, canInit := cfInfo.CheckErrorHistory() - if needSave { - err := o.etcdClient.LeaseGuardSaveChangeFeedInfo(ctx, cfInfo, changeFeedID, o.session.Lease()) - if err != nil { - return err - } - } - if !canInit { - // avoid too many logs here - if time.Now().Unix()%60 == 0 { - log.Warn("changefeed fails reach rate limit, try to initialize it later", zap.Int64s("history", cfInfo.ErrorHis)) - } - continue - } - err = cfInfo.VerifyAndFix() - if err != nil { - return err - } - - status, _, err := o.cfRWriter.GetChangeFeedStatus(ctx, changeFeedID) - if err != nil && cerror.ErrChangeFeedNotExists.NotEqual(err) { - return err - } - if status != nil && status.AdminJobType.IsStopState() { - if status.AdminJobType == model.AdminStop { - if _, ok := o.stoppedFeeds[changeFeedID]; !ok { - o.stoppedFeeds[changeFeedID] = status - } - } - continue - } - - // remaining task status means some processors are not exited, wait until - // all these statuses cleaned. If the capture of pending processor loses - // etcd session, the cleanUpStaleTasks will clean these statuses later. - allMetadataCleaned := true - allTaskStatus, err := o.etcdClient.GetAllTaskStatus(ctx, changeFeedID) - if err != nil { - return err - } - for _, taskStatus := range allTaskStatus { - if taskStatus.AdminJobType == model.AdminStop || taskStatus.AdminJobType == model.AdminRemove { - log.Info("stale task status is not deleted, wait metadata cleaned to create new changefeed", - zap.Reflect("task status", taskStatus), zap.String("changefeed", changeFeedID)) - allMetadataCleaned = false - break - } - } - if !allMetadataCleaned { - continue - } - - checkpointTs := cfInfo.GetCheckpointTs(status) - - newCf, err := o.newChangeFeed(ctx, changeFeedID, taskStatus, taskPositions, cfInfo, checkpointTs) - if err != nil { - cfInfo.Error = &model.RunningError{ - Addr: util.CaptureAddrFromCtx(ctx), - Code: "CDC-owner-1001", - Message: err.Error(), - } - cfInfo.ErrorHis = append(cfInfo.ErrorHis, time.Now().UnixNano()/1e6) - - if cerror.ChangefeedFastFailError(err) { - log.Error("create changefeed with fast fail error, mark changefeed as failed", - zap.Error(err), zap.String("changefeed", changeFeedID)) - cfInfo.State = model.StateFailed - err := o.etcdClient.LeaseGuardSaveChangeFeedInfo(ctx, cfInfo, changeFeedID, o.session.Lease()) - if err != nil { - return err - } - continue - } - - err2 := o.etcdClient.LeaseGuardSaveChangeFeedInfo(ctx, cfInfo, changeFeedID, o.session.Lease()) - if err2 != nil { - return err2 - } - // changefeed error has been recorded in etcd, log error here and - // don't need to return an error. - log.Warn("create changefeed failed, retry later", - zap.String("changefeed", changeFeedID), zap.Error(err)) - continue - } - - if newCf.info.SyncPointEnabled { - log.Info("syncpoint is on, creating the sync table") - // create the sync table - err := newCf.syncpointStore.CreateSynctable(ctx) - if err != nil { - return err - } - newCf.startSyncPointTicker(ctx, newCf.info.SyncPointInterval) - } else { - log.Info("syncpoint is off") - } - - o.changeFeeds[changeFeedID] = newCf - delete(o.stoppedFeeds, changeFeedID) - } - o.adminJobsLock.Lock() - for cfID, err := range errorFeeds { - job := model.AdminJob{ - CfID: cfID, - Type: model.AdminStop, - Error: err, - } - o.adminJobs = append(o.adminJobs, job) - } - o.adminJobsLock.Unlock() - return nil -} - -func (o *Owner) balanceTables(ctx context.Context) error { - rebalanceForAllChangefeed := false - o.rebalanceMu.Lock() - if o.rebalanceForAllChangefeed { - rebalanceForAllChangefeed = true - o.rebalanceForAllChangefeed = false - } - o.rebalanceMu.Unlock() - for id, changefeed := range o.changeFeeds { - rebalanceNow := false - var scheduleCommands []*model.MoveTableJob - o.rebalanceMu.Lock() - if r, exist := o.rebalanceTigger[id]; exist { - rebalanceNow = r - delete(o.rebalanceTigger, id) - } - if rebalanceForAllChangefeed { - rebalanceNow = true - } - if c, exist := o.manualScheduleCommand[id]; exist { - scheduleCommands = c - delete(o.manualScheduleCommand, id) - } - o.rebalanceMu.Unlock() - err := changefeed.tryBalance(ctx, o.captures, rebalanceNow, scheduleCommands) - if err != nil { - return errors.Trace(err) - } - } - return nil -} - -func (o *Owner) flushChangeFeedInfos(ctx context.Context) error { - // no running or stopped changefeed, clear gc safepoint. - if len(o.changeFeeds) == 0 && len(o.stoppedFeeds) == 0 { - if !o.gcSafepointLastUpdate.IsZero() { - log.Info("clean service safe point", zap.String("service-id", CDCServiceSafePointID)) - _, err := o.pdClient.UpdateServiceGCSafePoint(ctx, CDCServiceSafePointID, 0, 0) - if err != nil { - log.Warn("failed to update service safe point", zap.Error(err)) - } else { - o.gcSafepointLastUpdate = time.Time{} - } - } - return nil - } - - staleChangeFeeds := make(map[model.ChangeFeedID]*model.ChangeFeedStatus, len(o.changeFeeds)) - gcSafePoint := uint64(math.MaxUint64) - - // get the lower bound of gcSafePoint - minGCSafePoint := o.getMinGCSafePointCache(ctx) - - if len(o.changeFeeds) > 0 { - snapshot := make(map[model.ChangeFeedID]*model.ChangeFeedStatus, len(o.changeFeeds)) - for id, changefeed := range o.changeFeeds { - snapshot[id] = changefeed.status - if changefeed.status.CheckpointTs < gcSafePoint { - gcSafePoint = changefeed.status.CheckpointTs - } - // 1. If changefeed's appliedCheckpoinTs <= minGCSafePoint, it means this changefeed is stagnant. - // They are collected into this map, and then handleStaleChangeFeed() is called to deal with these stagnant changefeed. - // A changefeed will not enter the map twice, because in run(), - // handleAdminJob() will always be executed before flushChangeFeedInfos(), - // ensuring that the previous changefeed in staleChangeFeeds has been stopped and removed from o.changeFeeds. - // 2. We need the `<=` check here is because when a changefeed is stagnant, its checkpointTs will be updated to pd, - // and it would be the minimum gcSafePoint across all services. - // So as described above(line 92) minGCSafePoint = gcSafePoint = CheckpointTs would happens. - // In this case, if we check `<` here , this changefeed will not be put into staleChangeFeeds, and its checkpoints will be updated to pd again and again. - // This will cause the cdc's gcSafePoint never advance. - // If we check `<=` here, when we encounter the changefeed again, we will put it into staleChangeFeeds. - if changefeed.status.CheckpointTs <= minGCSafePoint { - staleChangeFeeds[id] = changefeed.status - } - - phyTs := oracle.ExtractPhysical(changefeed.status.CheckpointTs) - changefeedCheckpointTsGauge.WithLabelValues(id).Set(float64(phyTs)) - // It is more accurate to get tso from PD, but in most cases we have - // deployed NTP service, a little bias is acceptable here. - changefeedCheckpointTsLagGauge.WithLabelValues(id).Set(float64(oracle.GetPhysical(time.Now())-phyTs) / 1e3) - } - if time.Since(o.lastFlushChangefeeds) > o.flushChangefeedInterval { - err := o.cfRWriter.LeaseGuardPutAllChangeFeedStatus(ctx, snapshot, o.session.Lease()) - if err != nil { - return errors.Trace(err) - } - for id, changefeedStatus := range snapshot { - o.changeFeeds[id].appliedCheckpointTs = changefeedStatus.CheckpointTs - } - o.lastFlushChangefeeds = time.Now() - } - } - - for _, status := range o.stoppedFeeds { - // If a stopped changefeed's CheckpoinTs <= minGCSafePoint, means this changefeed is stagnant. - // It should never be resumed. This part of the logic is in newChangeFeed() - // So here we can skip it. - if status.CheckpointTs <= minGCSafePoint { - continue - } - - if status.CheckpointTs < gcSafePoint { - gcSafePoint = status.CheckpointTs - } - } - - // handle stagnant changefeed collected above - err := o.handleStaleChangeFeed(ctx, staleChangeFeeds, minGCSafePoint) - if err != nil { - log.Warn("failed to handleStaleChangeFeed ", zap.Error(err)) - } - - if time.Since(o.gcSafepointLastUpdate) > GCSafepointUpdateInterval { - actual, err := o.pdClient.UpdateServiceGCSafePoint(ctx, CDCServiceSafePointID, o.gcTTL, gcSafePoint) - if err != nil { - sinceLastUpdate := time.Since(o.gcSafepointLastUpdate) - log.Warn("failed to update service safe point", zap.Error(err), - zap.Duration("since-last-update", sinceLastUpdate)) - // We do not throw an error unless updating GC safepoint has been failing for more than gcTTL. - if sinceLastUpdate >= time.Second*time.Duration(o.gcTTL) { - return cerror.ErrUpdateServiceSafepointFailed.Wrap(err) - } - } else { - o.pdGCSafePoint = actual - o.gcSafepointLastUpdate = time.Now() - } - - failpoint.Inject("InjectActualGCSafePoint", func(val failpoint.Value) { - actual = uint64(val.(int)) - }) - - if actual > gcSafePoint { - // UpdateServiceGCSafePoint has failed. - log.Warn("updating an outdated service safe point", zap.Uint64("checkpoint-ts", gcSafePoint), zap.Uint64("actual-safepoint", actual)) - - for cfID, cf := range o.changeFeeds { - if cf.status.CheckpointTs < actual { - runningError := &model.RunningError{ - Addr: util.CaptureAddrFromCtx(ctx), - Code: "CDC-owner-1001", - Message: cerror.ErrServiceSafepointLost.GenWithStackByArgs(actual).Error(), - } - - err := o.EnqueueJob(model.AdminJob{ - CfID: cfID, - Type: model.AdminStop, - Error: runningError, - }) - if err != nil { - return errors.Trace(err) - } - } - } - } - } - return nil -} - -// calcResolvedTs call calcResolvedTs of every changefeeds -func (o *Owner) calcResolvedTs(ctx context.Context) error { - for id, cf := range o.changeFeeds { - if err := cf.calcResolvedTs(ctx); err != nil { - log.Error("fail to calculate checkpoint ts, so it will be stopped", zap.String("changefeed", cf.id), zap.Error(err)) - // error may cause by sink.EmitCheckpointTs`, just stop the changefeed at the moment - // todo: make the method mentioned above more robust. - var code string - if rfcCode, ok := cerror.RFCCode(err); ok { - code = string(rfcCode) - } else { - code = string(cerror.ErrOwnerUnknown.RFCCode()) - } - - job := model.AdminJob{ - CfID: id, - Type: model.AdminStop, - Error: &model.RunningError{ - Addr: util.CaptureAddrFromCtx(ctx), - Code: code, - Message: err.Error(), - }, - } - - if err := o.EnqueueJob(job); err != nil { - return errors.Trace(err) - } - } - } - return nil -} - -// handleDDL call handleDDL of every changefeeds -func (o *Owner) handleDDL(ctx context.Context) error { - for _, cf := range o.changeFeeds { - err := cf.handleDDL(ctx) - if err != nil { - var code string - if terror, ok := err.(*errors.Error); ok { - code = string(terror.RFCCode()) - } else { - code = string(cerror.ErrExecDDLFailed.RFCCode()) - } - err = o.EnqueueJob(model.AdminJob{ - CfID: cf.id, - Type: model.AdminStop, - Error: &model.RunningError{ - Addr: util.CaptureAddrFromCtx(ctx), - Code: code, - Message: err.Error(), - }, - }) - if err != nil { - return errors.Trace(err) - } - } - } - return nil -} - -// handleSyncPoint call handleSyncPoint of every changefeeds -func (o *Owner) handleSyncPoint(ctx context.Context) error { - for _, cf := range o.changeFeeds { - if err := cf.handleSyncPoint(ctx); err != nil { - return errors.Trace(err) - } - } - return nil -} - -// dispatchJob dispatches job to processors -// Note job type in this function contains pause, remove and finish -func (o *Owner) dispatchJob(ctx context.Context, job model.AdminJob) error { - cf, ok := o.changeFeeds[job.CfID] - if !ok { - return cerror.ErrOwnerChangefeedNotFound.GenWithStackByArgs(job.CfID) - } - for captureID := range cf.taskStatus { - newStatus, _, err := cf.etcdCli.LeaseGuardAtomicPutTaskStatus( - ctx, cf.id, captureID, o.session.Lease(), - func(modRevision int64, taskStatus *model.TaskStatus) (bool, error) { - taskStatus.AdminJobType = job.Type - return true, nil - }, - ) - if err != nil { - return errors.Trace(err) - } - cf.taskStatus[captureID] = newStatus.Clone() - } - // record admin job in changefeed status - cf.status.AdminJobType = job.Type - infos := map[model.ChangeFeedID]*model.ChangeFeedStatus{job.CfID: cf.status} - err := o.cfRWriter.LeaseGuardPutAllChangeFeedStatus(ctx, infos, o.session.Lease()) - if err != nil { - return errors.Trace(err) - } - cf.Close() - // Only need to process stoppedFeeds with `AdminStop` command here. - // For `AdminResume`, we remove stopped feed in changefeed initialization phase. - // For `AdminRemove`, we need to update stoppedFeeds when removing a stopped changefeed. - if job.Type == model.AdminStop { - log.Debug("put changefeed into stoppedFeeds queue", zap.String("changefeed", job.CfID)) - o.stoppedFeeds[job.CfID] = cf.status - } - for captureID := range cf.taskStatus { - capture, ok := o.captures[captureID] - if !ok { - log.Warn("capture not found", zap.String("capture-id", captureID)) - continue - } - ownerMaintainTableNumGauge.DeleteLabelValues(cf.id, capture.AdvertiseAddr, maintainTableTypeTotal) - ownerMaintainTableNumGauge.DeleteLabelValues(cf.id, capture.AdvertiseAddr, maintainTableTypeWip) - } - delete(o.changeFeeds, job.CfID) - return nil -} - -func (o *Owner) collectChangefeedInfo(ctx context.Context, cid model.ChangeFeedID) ( - cf *changeFeed, - status *model.ChangeFeedStatus, - feedState model.FeedState, - err error, -) { - var ok bool - cf, ok = o.changeFeeds[cid] - if ok { - return cf, cf.status, cf.info.State, nil - } - feedState = model.StateNormal - - var cfInfo *model.ChangeFeedInfo - cfInfo, err = o.etcdClient.GetChangeFeedInfo(ctx, cid) - if err != nil && cerror.ErrChangeFeedNotExists.NotEqual(err) { - return - } - - status, _, err = o.etcdClient.GetChangeFeedStatus(ctx, cid) - if err != nil { - if cerror.ErrChangeFeedNotExists.Equal(err) { - // Only changefeed info exists and error field is not nil means - // the changefeed has met error, mark it as failed. - if cfInfo != nil && cfInfo.Error != nil { - feedState = model.StateFailed - } - } - return - } - switch status.AdminJobType { - case model.AdminNone, model.AdminResume: - if cfInfo != nil && cfInfo.Error != nil { - feedState = model.StateFailed - } - case model.AdminStop: - feedState = model.StateStopped - case model.AdminRemove: - feedState = model.StateRemoved - case model.AdminFinish: - feedState = model.StateFinished - } - return -} - -func (o *Owner) checkClusterHealth(_ context.Context) error { - // check whether a changefeed has finished by comparing checkpoint-ts and target-ts - for _, cf := range o.changeFeeds { - if cf.status.CheckpointTs == cf.info.GetTargetTs() { - log.Info("changefeed replication finished", zap.String("changefeed", cf.id), zap.Uint64("checkpointTs", cf.status.CheckpointTs)) - err := o.EnqueueJob(model.AdminJob{ - CfID: cf.id, - Type: model.AdminFinish, - }) - if err != nil { - return err - } - } - } - for _, cf := range o.changeFeeds { - for captureID, pinfo := range cf.taskStatus { - capture, ok := o.captures[captureID] - if !ok { - log.Warn("capture not found", zap.String("capture-id", captureID)) - continue - } - ownerMaintainTableNumGauge.WithLabelValues(cf.id, capture.AdvertiseAddr, maintainTableTypeTotal).Set(float64(len(pinfo.Tables))) - ownerMaintainTableNumGauge.WithLabelValues(cf.id, capture.AdvertiseAddr, maintainTableTypeWip).Set(float64(len(pinfo.Operation))) - } - } - // TODO: check processor normal exited - return nil -} - -func (o *Owner) handleAdminJob(ctx context.Context) error { - removeIdx := 0 - o.adminJobsLock.Lock() - defer func() { - o.adminJobs = o.adminJobs[removeIdx:] - o.adminJobsLock.Unlock() - }() - for i, job := range o.adminJobs { - log.Info("handle admin job", zap.String("changefeed", job.CfID), zap.Stringer("type", job.Type)) - removeIdx = i + 1 - - cf, status, feedState, err := o.collectChangefeedInfo(ctx, job.CfID) - if err != nil { - if cerror.ErrChangeFeedNotExists.NotEqual(err) { - return err - } - if feedState == model.StateFailed && job.Type == model.AdminRemove { - // changefeed in failed state, but changefeed status has not - // been created yet. Try to remove changefeed info only. - err := o.etcdClient.LeaseGuardDeleteChangeFeedInfo(ctx, job.CfID, o.session.Lease()) - if err != nil { - return errors.Trace(err) - } - } else { - log.Warn("invalid admin job, changefeed status not found", zap.String("changefeed", job.CfID)) - } - continue - } - switch job.Type { - case model.AdminStop: - switch feedState { - case model.StateStopped: - log.Info("changefeed has been stopped, pause command will do nothing") - continue - case model.StateRemoved: - log.Info("changefeed has been removed, pause command will do nothing") - continue - case model.StateFinished: - log.Info("changefeed has finished, pause command will do nothing") - continue - } - if cf == nil { - log.Warn("invalid admin job, changefeed not found", zap.String("changefeed", job.CfID)) - continue - } - - cf.info.AdminJobType = model.AdminStop - cf.info.Error = job.Error - if job.Error != nil { - cf.info.ErrorHis = append(cf.info.ErrorHis, time.Now().UnixNano()/1e6) - } - - err := o.etcdClient.LeaseGuardSaveChangeFeedInfo(ctx, cf.info, job.CfID, o.session.Lease()) - if err != nil { - return errors.Trace(err) - } - err = o.dispatchJob(ctx, job) - if err != nil { - return errors.Trace(err) - } - cf.stopSyncPointTicker() - case model.AdminRemove, model.AdminFinish: - if cf != nil { - cf.stopSyncPointTicker() - err := o.dispatchJob(ctx, job) - if err != nil { - return errors.Trace(err) - } - } else { - switch feedState { - case model.StateRemoved, model.StateFinished: - // remove a removed or finished changefeed - if job.Opts != nil && job.Opts.ForceRemove { - err := o.etcdClient.LeaseGuardRemoveChangeFeedStatus(ctx, job.CfID, o.session.Lease()) - if err != nil { - return errors.Trace(err) - } - } else { - log.Info("changefeed has been removed or finished, remove command will do nothing") - } - continue - case model.StateStopped, model.StateFailed: - // remove a paused or failed changefeed - status.AdminJobType = model.AdminRemove - err = o.etcdClient.LeaseGuardPutChangeFeedStatus(ctx, job.CfID, status, o.session.Lease()) - if err != nil { - return errors.Trace(err) - } - delete(o.stoppedFeeds, job.CfID) - default: - return cerror.ErrChangefeedAbnormalState.GenWithStackByArgs(feedState, status) - } - } - // remove changefeed info - err := o.etcdClient.DeleteChangeFeedInfo(ctx, job.CfID) - if err != nil { - return errors.Trace(err) - } - if job.Opts != nil && job.Opts.ForceRemove { - // if `ForceRemove` is enabled, remove all information related to this changefeed - err := o.etcdClient.LeaseGuardRemoveChangeFeedStatus(ctx, job.CfID, o.session.Lease()) - if err != nil { - return errors.Trace(err) - } - } else { - // set ttl to changefeed status - err = o.etcdClient.SetChangeFeedStatusTTL(ctx, job.CfID, 24*3600 /*24 hours*/) - if err != nil { - return errors.Trace(err) - } - } - case model.AdminResume: - // resume changefeed must read checkpoint from ChangeFeedStatus - if cerror.ErrChangeFeedNotExists.Equal(err) { - log.Warn("invalid admin job, changefeed not found", zap.String("changefeed", job.CfID)) - continue - } - if feedState == model.StateRemoved || feedState == model.StateFinished { - log.Info("changefeed has been removed or finished, cannot be resumed anymore") - continue - } - cfInfo, err := o.etcdClient.GetChangeFeedInfo(ctx, job.CfID) - if err != nil { - return errors.Trace(err) - } - - // set admin job in changefeed status to tell owner resume changefeed - status.AdminJobType = model.AdminResume - err = o.etcdClient.LeaseGuardPutChangeFeedStatus(ctx, job.CfID, status, o.session.Lease()) - if err != nil { - return errors.Trace(err) - } - - // set admin job in changefeed cfInfo to trigger each capture's changefeed list watch event - cfInfo.AdminJobType = model.AdminResume - // clear last running error - cfInfo.State = model.StateNormal - cfInfo.Error = nil - err = o.etcdClient.LeaseGuardSaveChangeFeedInfo(ctx, cfInfo, job.CfID, o.session.Lease()) - if err != nil { - return errors.Trace(err) - } - if config.NewReplicaImpl { - // remove all positions because the old positions may be include an error - err = o.etcdClient.RemoveAllTaskPositions(ctx, job.CfID) - if err != nil { - return errors.Trace(err) - } - } - } - // TODO: we need a better admin job workflow. Supposing uses create - // multiple admin jobs to a specific changefeed at the same time, such - // as pause -> resume -> pause, should the one job handler waits for - // the previous job finished? However it is difficult to distinguish - // whether a job is totally finished in some cases, for example when - // resuming a changefeed, seems we should mark the job finished if all - // processors have started. Currently the owner only processes one - // admin job in each tick loop as a workaround. - break - } - return nil -} - -func (o *Owner) throne(ctx context.Context) error { - // Start a routine to keep watching on the liveness of - // captures. - o.startCaptureWatcher(ctx) - return nil -} - -// Close stops a running owner -func (o *Owner) Close(ctx context.Context, stepDown func(ctx context.Context) error) { - // stepDown is called after exiting the main loop by the owner, it is useful - // to clean up some resource, like dropping the leader key. - o.stepDown = stepDown - - // Close and Run should be in separated goroutines - // A channel is used here to synchronize the steps. - - // Single the Run function to exit - select { - case o.done <- struct{}{}: - case <-ctx.Done(): - } - - // Wait until it exited - select { - case <-o.done: - case <-ctx.Done(): - } -} - -// Run the owner -// TODO avoid this tick style, this means we get `tickTime` latency here. -func (o *Owner) Run(ctx context.Context, tickTime time.Duration) error { - failpoint.Inject("owner-run-with-error", func() { - failpoint.Return(errors.New("owner run with injected error")) - }) - - ctx, cancel := context.WithCancel(ctx) - defer cancel() - - go func() { - if err := o.watchCampaignKey(ctx); err != nil { - cancel() - } - }() - - if err := o.throne(ctx); err != nil { - return err - } - - ctx1, cancel1 := context.WithCancel(ctx) - defer cancel1() - feedChangeReceiver, err := o.feedChangeNotifier.NewReceiver(tickTime) - if err != nil { - return err - } - defer feedChangeReceiver.Stop() - o.watchFeedChange(ctx1) - - ownership := newOwnership(tickTime) -loop: - for { - select { - case <-o.done: - close(o.done) - break loop - case <-ctx.Done(): - // FIXME: cancel the context doesn't ensure all resources are destructed, is it reasonable? - // Anyway we just break loop here to ensure the following destruction. - err = ctx.Err() - break loop - case <-feedChangeReceiver.C: - ownership.inc() - } - - err = o.run(ctx) - if err != nil { - switch errors.Cause(err) { - case context.DeadlineExceeded: - // context timeout means the o.run doesn't finish in a safe owner - // lease cycle, it is safe to retry. If the lease is revoked, - // another run loop will detect it. - continue loop - case context.Canceled: - default: - log.Error("owner exited with error", zap.Error(err)) - } - break loop - } - } - for _, cf := range o.changeFeeds { - cf.Close() - changefeedCheckpointTsGauge.DeleteLabelValues(cf.id) - changefeedCheckpointTsLagGauge.DeleteLabelValues(cf.id) - } - if o.stepDown != nil { - if err := o.stepDown(ctx); err != nil { - return err - } - } - - return err -} - -// watchCampaignKey watches the aliveness of campaign owner key in etcd -func (o *Owner) watchCampaignKey(ctx context.Context) error { - key := fmt.Sprintf("%s/%x", kv.CaptureOwnerKey, o.session.Lease()) -restart: - resp, err := o.etcdClient.Client.Get(ctx, key) - if err != nil { - return cerror.WrapError(cerror.ErrPDEtcdAPIError, err) - } - if resp.Count == 0 { - return cerror.ErrOwnerCampaignKeyDeleted.GenWithStackByArgs() - } - // watch the key change from the next revision relatived to the current - wch := o.etcdClient.Client.Watch(ctx, key, clientv3.WithRev(resp.Header.Revision+1)) - for resp := range wch { - err := resp.Err() - if err != nil { - if err != mvcc.ErrCompacted { - log.Error("watch owner campaign key failed, restart the watcher", zap.Error(err)) - } - goto restart - } - for _, ev := range resp.Events { - if ev.Type == clientv3.EventTypeDelete { - log.Warn("owner campaign key deleted", zap.String("key", key)) - return cerror.ErrOwnerCampaignKeyDeleted.GenWithStackByArgs() - } - } - } - return nil -} - -func (o *Owner) watchFeedChange(ctx context.Context) { - go func() { - for { - select { - case <-ctx.Done(): - return - default: - } - cctx, cancel := context.WithCancel(ctx) - wch := o.etcdClient.Client.Watch(cctx, kv.TaskPositionKeyPrefix, clientv3.WithFilterDelete(), clientv3.WithPrefix()) - - for resp := range wch { - if resp.Err() != nil { - log.Error("position watcher restarted with error", zap.Error(resp.Err())) - break - } - - // TODO: because the main loop has many serial steps, it is hard to do a partial update without change - // majority logical. For now just to wakeup the main loop ASAP to reduce latency, the efficiency of etcd - // operations should be resolved in future release. - - o.feedChangeNotifier.Notify() - } - cancel() - } - }() -} - -func (o *Owner) run(ctx context.Context) error { - // captureLoaded == 0 means capture information is not built, owner can't - // run normal jobs now. - if atomic.LoadInt32(&o.captureLoaded) == int32(0) { - return nil - } - - o.l.Lock() - defer o.l.Unlock() - - var err error - - err = o.cleanUpStaleTasks(ctx) - if err != nil { - return errors.Trace(err) - } - - err = o.loadChangeFeeds(ctx) - if err != nil { - return errors.Trace(err) - } - - err = o.balanceTables(ctx) - if err != nil { - return errors.Trace(err) - } - - err = o.handleDDL(ctx) - if err != nil { - return errors.Trace(err) - } - - err = o.handleSyncPoint(ctx) - if err != nil { - return errors.Trace(err) - } - - err = o.handleAdminJob(ctx) - if err != nil { - return errors.Trace(err) - } - - err = o.calcResolvedTs(ctx) - if err != nil { - return errors.Trace(err) - } - - // It is better for flushChangeFeedInfos to follow calcResolvedTs immediately, - // because operations such as handleDDL and rebalancing rely on proper progress of the checkpoint in Etcd. - err = o.flushChangeFeedInfos(ctx) - if err != nil { - return errors.Trace(err) - } - - err = o.checkClusterHealth(ctx) - if err != nil { - return errors.Trace(err) - } - - return nil -} - -// EnqueueJob adds an admin job -func (o *Owner) EnqueueJob(job model.AdminJob) error { - switch job.Type { - case model.AdminResume, model.AdminRemove, model.AdminStop, model.AdminFinish: - default: - return cerror.ErrInvalidAdminJobType.GenWithStackByArgs(job.Type) - } - o.adminJobsLock.Lock() - o.adminJobs = append(o.adminJobs, job) - o.adminJobsLock.Unlock() - return nil -} - -// TriggerRebalance triggers the rebalance in the specified changefeed -func (o *Owner) TriggerRebalance(changefeedID model.ChangeFeedID) { - o.rebalanceMu.Lock() - defer o.rebalanceMu.Unlock() - o.rebalanceTigger[changefeedID] = true - // TODO(leoppro) throw an error if the changefeed is not exist -} - -// ManualSchedule moves the table from a capture to another capture -func (o *Owner) ManualSchedule(changefeedID model.ChangeFeedID, to model.CaptureID, tableID model.TableID) { - o.rebalanceMu.Lock() - defer o.rebalanceMu.Unlock() - o.manualScheduleCommand[changefeedID] = append(o.manualScheduleCommand[changefeedID], &model.MoveTableJob{ - To: to, - TableID: tableID, - }) -} - -func (o *Owner) writeDebugInfo(w io.Writer) { - fmt.Fprintf(w, "** active changefeeds **:\n") - for _, info := range o.changeFeeds { - fmt.Fprintf(w, "%s\n", info) - } - fmt.Fprintf(w, "** stopped changefeeds **:\n") - for _, feedStatus := range o.stoppedFeeds { - fmt.Fprintf(w, "%+v\n", *feedStatus) - } - fmt.Fprintf(w, "\n** captures **:\n") - for _, capture := range o.captures { - fmt.Fprintf(w, "%+v\n", *capture) - } -} - -// cleanUpStaleTasks cleans up the task status which does not associated -// with an active processor. This function is not thread safe. -// -// When a new owner is elected, it does not know the events occurs before, like -// processor deletion. In this case, the new owner should check if the task -// status is stale because of the processor deletion. -func (o *Owner) cleanUpStaleTasks(ctx context.Context) error { - _, changefeeds, err := o.etcdClient.GetChangeFeeds(ctx) - if err != nil { - return errors.Trace(err) - } - for changeFeedID := range changefeeds { - statuses, err := o.etcdClient.GetAllTaskStatus(ctx, changeFeedID) - if err != nil { - return errors.Trace(err) - } - positions, err := o.etcdClient.GetAllTaskPositions(ctx, changeFeedID) - if err != nil { - return errors.Trace(err) - } - workloads, err := o.etcdClient.GetAllTaskWorkloads(ctx, changeFeedID) - if err != nil { - return errors.Trace(err) - } - // in most cases statuses and positions have the same keys, or positions - // are more than statuses, as we always delete task status first. - captureIDs := make(map[string]struct{}, len(statuses)) - for captureID := range statuses { - captureIDs[captureID] = struct{}{} - } - for captureID := range positions { - captureIDs[captureID] = struct{}{} - } - for captureID := range workloads { - captureIDs[captureID] = struct{}{} - } - - log.Debug("cleanUpStaleTasks", - zap.Reflect("statuses", statuses), - zap.Reflect("positions", positions), - zap.Reflect("workloads", workloads)) - - for captureID := range captureIDs { - if _, ok := o.captures[captureID]; !ok { - status, ok1 := statuses[captureID] - if ok1 { - pos, taskPosFound := positions[captureID] - if !taskPosFound { - log.Warn("task position not found, fallback to use original start ts", - zap.String("capture", captureID), - zap.String("changefeed", changeFeedID), - zap.Reflect("task status", status), - ) - } - for tableID, replicaInfo := range status.Tables { - startTs := replicaInfo.StartTs - if taskPosFound { - if startTs < pos.CheckPointTs { - startTs = pos.CheckPointTs - } - } - o.addOrphanTable(changeFeedID, tableID, startTs) - } - if cf, ok := o.changeFeeds[changeFeedID]; ok { - o.rebuildTableFromOperations(cf, status, cf.status.CheckpointTs) - } - } - - if err := o.etcdClient.LeaseGuardDeleteTaskStatus(ctx, changeFeedID, captureID, o.session.Lease()); err != nil { - return errors.Trace(err) - } - if err := o.etcdClient.LeaseGuardDeleteTaskPosition(ctx, changeFeedID, captureID, o.session.Lease()); err != nil { - return errors.Trace(err) - } - if err := o.etcdClient.LeaseGuardDeleteTaskWorkload(ctx, changeFeedID, captureID, o.session.Lease()); err != nil { - return errors.Trace(err) - } - log.Info("cleanup stale task", zap.String("capture-id", captureID), zap.String("changefeed", changeFeedID)) - } - } - } - return nil -} - -func (o *Owner) watchCapture(ctx context.Context) error { - ctx = clientv3.WithRequireLeader(ctx) - - failpoint.Inject("sleep-before-watch-capture", nil) - - // When an owner just starts, changefeed information is not updated at once. - // Supposing a crashed capture should be removed now, the owner will miss deleting - // task status and task position if changefeed information is not loaded. - // If the task positions and status decode failed, remove them. - if err := o.checkAndCleanTasksInfo(ctx); err != nil { - return errors.Trace(err) - } - o.l.Lock() - if err := o.loadChangeFeeds(ctx); err != nil { - o.l.Unlock() - return errors.Trace(err) - } - o.l.Unlock() - - rev, captureList, err := o.etcdClient.GetCaptures(ctx) - if err != nil { - return errors.Trace(err) - } - captures := make(map[model.CaptureID]*model.CaptureInfo) - for _, c := range captureList { - captures[c.ID] = c - } - // before watching, rebuild events according to - // the existed captures. This is necessary because - // the etcd events may be compacted. - if err := o.rebuildCaptureEvents(ctx, captures); err != nil { - return errors.Trace(err) - } - - log.Info("monitoring captures", - zap.String("key", kv.CaptureInfoKeyPrefix), - zap.Int64("rev", rev)) - ch := o.etcdClient.Client.Watch(ctx, kv.CaptureInfoKeyPrefix, - clientv3.WithPrefix(), - clientv3.WithRev(rev+1), - clientv3.WithPrevKV()) - - for resp := range ch { - err := resp.Err() - failpoint.Inject("restart-capture-watch", func() { - err = mvcc.ErrCompacted - }) - if err != nil { - return cerror.WrapError(cerror.ErrOwnerEtcdWatch, resp.Err()) - } - for _, ev := range resp.Events { - c := &model.CaptureInfo{} - switch ev.Type { - case clientv3.EventTypeDelete: - if err := c.Unmarshal(ev.PrevKv.Value); err != nil { - return errors.Trace(err) - } - log.Info("delete capture", - zap.String("capture-id", c.ID), - zap.String("capture", c.AdvertiseAddr)) - o.removeCapture(ctx, c) - case clientv3.EventTypePut: - if !ev.IsCreate() { - continue - } - if err := c.Unmarshal(ev.Kv.Value); err != nil { - return errors.Trace(err) - } - log.Info("add capture", - zap.String("capture-id", c.ID), - zap.String("capture", c.AdvertiseAddr)) - o.addCapture(ctx, c) - } - } - } - return nil -} - -func (o *Owner) rebuildCaptureEvents(ctx context.Context, captures map[model.CaptureID]*model.CaptureInfo) error { - for _, c := range captures { - o.addCapture(ctx, c) - } - for _, c := range o.captures { - if _, ok := captures[c.ID]; !ok { - o.removeCapture(ctx, c) - } - } - // captureLoaded is used to check whether the owner can execute cleanup stale tasks job. - // Because at the very beginning of a new owner, it doesn't have capture information in - // memory, cleanup stale tasks could have a false positive (where positive means owner - // should cleanup the stale task of a specific capture). After the first time of capture - // rebuild, even the etcd compaction and watch capture is rerun, we don't need to check - // captureLoaded anymore because existing tasks must belong to a capture which is still - // maintained in owner's memory. - atomic.StoreInt32(&o.captureLoaded, 1) - - // clean up stale tasks each time before watch capture event starts, - // for two reasons: - // 1. when a new owner is elected, it must clean up stale task status and positions. - // 2. when error happens in owner's capture event watch, the owner just resets - // the watch loop, with the following two steps: - // 1) load all captures from PD, having a revision for data - // 2) start a new watch from revision in step1 - // the step-2 may meet an error such as ErrCompacted, and we will continue - // from step-1, however other capture may crash just after step-2 returns - // and before step-1 starts, the longer time gap between step-2 to step-1, - // missing a crashed capture is more likely to happen. - o.l.Lock() - defer o.l.Unlock() - return errors.Trace(o.cleanUpStaleTasks(ctx)) -} - -func (o *Owner) startCaptureWatcher(ctx context.Context) { - log.Info("start to watch captures") - go func() { - rl := rate.NewLimiter(0.05, 2) - for { - err := rl.Wait(ctx) - if err != nil { - if errors.Cause(err) == context.Canceled { - return - } - log.Error("capture watcher wait limit token error", zap.Error(err)) - return - } - if err := o.watchCapture(ctx); err != nil { - // When the watching routine returns, the error must not - // be nil, it may be caused by a temporary error or a context - // error(ctx.Err()) - if ctx.Err() != nil { - if errors.Cause(ctx.Err()) != context.Canceled { - // The context error indicates the termination of the owner - log.Error("watch capture failed", zap.Error(ctx.Err())) - } else { - log.Info("watch capture exited") - } - return - } - log.Warn("watch capture returned", zap.Error(err)) - // Otherwise, a temporary error occurred(ErrCompact), - // restart the watching routine. - } - } - }() -} - -// handle the StaleChangeFeed -// By setting the AdminJob type to AdminStop and the Error code to indicate that the changefeed is stagnant. -func (o *Owner) handleStaleChangeFeed(ctx context.Context, staleChangeFeeds map[model.ChangeFeedID]*model.ChangeFeedStatus, minGCSafePoint uint64) error { - for id, status := range staleChangeFeeds { - err := cerror.ErrSnapshotLostByGC.GenWithStackByArgs(status.CheckpointTs, minGCSafePoint) - log.Warn("changefeed checkpoint is lagging too much, so it will be stopped.", zap.String("changefeed", id), zap.Error(err)) - runningError := &model.RunningError{ - Addr: util.CaptureAddrFromCtx(ctx), - Code: string(cerror.ErrSnapshotLostByGC.RFCCode()), // changefeed is stagnant - Message: err.Error(), - } - - err = o.EnqueueJob(model.AdminJob{ - CfID: id, - Type: model.AdminStop, - Error: runningError, - }) - if err != nil { - return errors.Trace(err) - } - delete(staleChangeFeeds, id) - } - return nil -} diff --git a/cdc/owner_test.go b/cdc/owner_test.go deleted file mode 100644 index 1af45032a5b..00000000000 --- a/cdc/owner_test.go +++ /dev/null @@ -1,1478 +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 cdc - -import ( - "bytes" - "context" - "fmt" - "net/url" - "sync" - "sync/atomic" - "time" - - "github.com/google/uuid" - "github.com/pingcap/check" - "github.com/pingcap/errors" - timodel "github.com/pingcap/parser/model" - "github.com/pingcap/parser/mysql" - "github.com/pingcap/parser/types" - "github.com/pingcap/tidb/meta" - "github.com/pingcap/tidb/store/mockstore" - "github.com/pingcap/tidb/store/tikv/oracle" - "github.com/pingcap/tiflow/cdc/entry" - "github.com/pingcap/tiflow/cdc/kv" - "github.com/pingcap/tiflow/cdc/model" - "github.com/pingcap/tiflow/cdc/sink" - "github.com/pingcap/tiflow/pkg/config" - cerror "github.com/pingcap/tiflow/pkg/errors" - "github.com/pingcap/tiflow/pkg/etcd" - "github.com/pingcap/tiflow/pkg/filter" - "github.com/pingcap/tiflow/pkg/security" - "github.com/pingcap/tiflow/pkg/util" - "github.com/pingcap/tiflow/pkg/util/testleak" - pd "github.com/tikv/pd/client" - "go.etcd.io/etcd/clientv3" - "go.etcd.io/etcd/clientv3/concurrency" - "go.etcd.io/etcd/embed" - "golang.org/x/sync/errgroup" -) - -const TiKVGCLifeTime = 10 * 60 * time.Second // 10 min - -type ownerSuite struct { - e *embed.Etcd - clientURL *url.URL - client kv.CDCEtcdClient - ctx context.Context - cancel context.CancelFunc - errg *errgroup.Group -} - -var _ = check.Suite(&ownerSuite{}) - -func (s *ownerSuite) SetUpTest(c *check.C) { - dir := c.MkDir() - var err error - s.clientURL, s.e, err = etcd.SetupEmbedEtcd(dir) - c.Assert(err, check.IsNil) - client, err := clientv3.New(clientv3.Config{ - Endpoints: []string{s.clientURL.String()}, - DialTimeout: 3 * time.Second, - }) - c.Assert(err, check.IsNil) - s.client = kv.NewCDCEtcdClient(context.TODO(), client) - s.ctx, s.cancel = context.WithCancel(context.Background()) - s.errg = util.HandleErrWithErrGroup(s.ctx, s.e.Err(), func(e error) { c.Log(e) }) -} - -func (s *ownerSuite) TearDownTest(c *check.C) { - s.e.Close() - s.cancel() - err := s.errg.Wait() - if err != nil { - c.Errorf("Error group error: %s", err) - } - s.client.Close() //nolint:errcheck -} - -type mockPDClient struct { - pd.Client - invokeCounter int - mockSafePointLost bool - mockPDFailure bool - mockTiKVGCLifeTime bool -} - -func (m *mockPDClient) GetTS(ctx context.Context) (int64, int64, error) { - if m.mockPDFailure { - return 0, 0, errors.New("injected PD failure") - } - if m.mockSafePointLost { - return 0, 0, nil - } - return oracle.GetPhysical(time.Now()), 0, nil -} - -func (m *mockPDClient) UpdateServiceGCSafePoint(ctx context.Context, serviceID string, ttl int64, safePoint uint64) (uint64, error) { - m.invokeCounter++ - - if m.mockSafePointLost { - return 1000, nil - } - if m.mockPDFailure { - return 0, errors.New("injected PD failure") - } - if m.mockTiKVGCLifeTime { - Ts := oracle.GoTimeToTS(time.Now().Add(-TiKVGCLifeTime)) - return Ts, nil - } - return safePoint, nil -} - -type mockSink struct { - sink.Sink - checkpointTs model.Ts - - checkpointMu sync.Mutex - checkpointError error -} - -func (m *mockSink) EmitCheckpointTs(ctx context.Context, ts uint64) error { - m.checkpointMu.Lock() - defer m.checkpointMu.Unlock() - atomic.StoreUint64(&m.checkpointTs, ts) - return m.checkpointError -} - -func (m *mockSink) Close(ctx context.Context) error { - return nil -} - -func (m *mockSink) Barrier(ctx context.Context) error { - return nil -} - -// Test whether the owner can tolerate sink caused error, it won't be killed. -// also set the specific changefeed to stop -func (s *ownerSuite) TestOwnerCalcResolvedTs(c *check.C) { - defer testleak.AfterTest(c)() - mockPDCli := &mockPDClient{} - - sink := &mockSink{checkpointError: cerror.ErrKafkaSendMessage} - changeFeeds := map[model.ChangeFeedID]*changeFeed{ - "test_change_feed_1": { - info: &model.ChangeFeedInfo{State: model.StateNormal}, - etcdCli: s.client, - status: &model.ChangeFeedStatus{ - CheckpointTs: 0, - }, - targetTs: 2000, - ddlResolvedTs: 2000, - ddlState: model.ChangeFeedSyncDML, - taskStatus: model.ProcessorsInfos{ - "capture_1": {}, - "capture_2": {}, - }, - taskPositions: map[string]*model.TaskPosition{ - "capture_1": { - CheckPointTs: 2333, - ResolvedTs: 2333, - }, - "capture_2": { - CheckPointTs: 2333, - ResolvedTs: 2333, - }, - }, - sink: sink, - }, - } - - session, err := concurrency.NewSession(s.client.Client.Unwrap(), - concurrency.WithTTL(config.GetDefaultServerConfig().CaptureSessionTTL)) - c.Assert(err, check.IsNil) - mockOwner := Owner{ - session: session, - pdClient: mockPDCli, - etcdClient: s.client, - lastFlushChangefeeds: time.Now(), - flushChangefeedInterval: 1 * time.Hour, - changeFeeds: changeFeeds, - cfRWriter: s.client, - stoppedFeeds: make(map[model.ChangeFeedID]*model.ChangeFeedStatus), - minGCSafePointCache: minGCSafePointCacheEntry{}, - } - - err = mockOwner.calcResolvedTs(s.ctx) - c.Assert(err, check.IsNil) - - err = mockOwner.handleAdminJob(s.ctx) - c.Assert(err, check.IsNil) - c.Assert(mockOwner.stoppedFeeds["test_change_feed_1"], check.NotNil) - - err = mockOwner.flushChangeFeedInfos(s.ctx) - c.Assert(err, check.IsNil) - c.Assert(mockPDCli.invokeCounter, check.Equals, 1) - - s.TearDownTest(c) -} - -func (s *ownerSuite) TestOwnerFlushChangeFeedInfos(c *check.C) { - defer testleak.AfterTest(c)() - session, err := concurrency.NewSession(s.client.Client.Unwrap(), - concurrency.WithTTL(config.GetDefaultServerConfig().CaptureSessionTTL)) - c.Assert(err, check.IsNil) - mockPDCli := &mockPDClient{} - mockOwner := Owner{ - session: session, - etcdClient: s.client, - pdClient: mockPDCli, - gcSafepointLastUpdate: time.Now(), - } - - err = mockOwner.flushChangeFeedInfos(s.ctx) - c.Assert(err, check.IsNil) - c.Assert(mockPDCli.invokeCounter, check.Equals, 1) - s.TearDownTest(c) -} - -func (s *ownerSuite) TestOwnerFlushChangeFeedInfosFailed(c *check.C) { - defer testleak.AfterTest(c)() - mockPDCli := &mockPDClient{ - mockPDFailure: true, - } - - changeFeeds := map[model.ChangeFeedID]*changeFeed{ - "test_change_feed_1": { - info: &model.ChangeFeedInfo{State: model.StateNormal}, - status: &model.ChangeFeedStatus{ - CheckpointTs: 100, - }, - targetTs: 2000, - ddlState: model.ChangeFeedSyncDML, - taskStatus: model.ProcessorsInfos{ - "capture_1": {}, - "capture_2": {}, - }, - taskPositions: map[string]*model.TaskPosition{ - "capture_1": {}, - "capture_2": {}, - }, - }, - } - - session, err := concurrency.NewSession(s.client.Client.Unwrap(), - concurrency.WithTTL(config.GetDefaultServerConfig().CaptureSessionTTL)) - c.Assert(err, check.IsNil) - mockOwner := Owner{ - session: session, - pdClient: mockPDCli, - etcdClient: s.client, - lastFlushChangefeeds: time.Now(), - flushChangefeedInterval: 1 * time.Hour, - gcSafepointLastUpdate: time.Now(), - gcTTL: 6, // 6 seconds - changeFeeds: changeFeeds, - } - - time.Sleep(3 * time.Second) - err = mockOwner.flushChangeFeedInfos(s.ctx) - c.Assert(err, check.IsNil) - c.Assert(mockPDCli.invokeCounter, check.Equals, 1) - - time.Sleep(6 * time.Second) - err = mockOwner.flushChangeFeedInfos(s.ctx) - c.Assert(err, check.ErrorMatches, ".*CDC:ErrUpdateServiceSafepointFailed.*") - c.Assert(mockPDCli.invokeCounter, check.Equals, 2) - - s.TearDownTest(c) -} - -// Test whether it is possible to successfully create a changefeed -// with startTs less than currentTs - gcTTL when tikv_gc_life_time is greater than gc-ttl -func (s *ownerSuite) TestTiKVGCLifeTimeLargeThanGCTTL(c *check.C) { - defer testleak.AfterTest(c) - mockPDCli := &mockPDClient{} - mockPDCli.mockTiKVGCLifeTime = true - - changeFeeds := map[model.ChangeFeedID]*changeFeed{ - "test_change_feed_1": { - info: &model.ChangeFeedInfo{State: model.StateNormal}, - etcdCli: s.client, - status: &model.ChangeFeedStatus{ - CheckpointTs: oracle.GoTimeToTS(time.Now().Add(-6 * time.Second)), - }, - targetTs: 2000, - ddlState: model.ChangeFeedSyncDML, - taskStatus: model.ProcessorsInfos{ - "capture_1": {}, - "capture_2": {}, - }, - taskPositions: map[string]*model.TaskPosition{ - "capture_1": {}, - "capture_2": {}, - }, - }, - } - - session, err := concurrency.NewSession(s.client.Client.Unwrap(), - concurrency.WithTTL(config.GetDefaultServerConfig().CaptureSessionTTL)) - c.Assert(err, check.IsNil) - - mockOwner := Owner{ - session: session, - pdClient: mockPDCli, - etcdClient: s.client, - lastFlushChangefeeds: time.Now(), - flushChangefeedInterval: 1 * time.Hour, - // gcSafepointLastUpdate: time.Now(), - gcTTL: 6, // 6 seconds - changeFeeds: changeFeeds, - cfRWriter: s.client, - stoppedFeeds: make(map[model.ChangeFeedID]*model.ChangeFeedStatus), - minGCSafePointCache: minGCSafePointCacheEntry{}, - } - - err = mockOwner.flushChangeFeedInfos(s.ctx) - c.Assert(err, check.IsNil) - c.Assert(mockPDCli.invokeCounter, check.Equals, 1) - - err = mockOwner.handleAdminJob(s.ctx) - c.Assert(err, check.IsNil) - c.Assert(mockOwner.stoppedFeeds["test_change_feed_1"], check.IsNil) - c.Assert(mockOwner.changeFeeds["test_change_feed_1"].info.State, check.Equals, model.StateNormal) - - time.Sleep(7 * time.Second) // wait for gcTTL time pass - err = mockOwner.flushChangeFeedInfos(s.ctx) - c.Assert(err, check.IsNil) - c.Assert(mockPDCli.invokeCounter, check.Equals, 2) - - err = mockOwner.handleAdminJob(s.ctx) - c.Assert(err, check.IsNil) - c.Assert(mockOwner.stoppedFeeds["test_change_feed_1"], check.IsNil) - - s.TearDownTest(c) -} - -// Test whether the owner handles the stagnant task correctly, so that it can't block the update of gcSafePoint. -// If a changefeed is put into the stop queue due to stagnation, it can no longer affect the update of gcSafePoint. -// So we just need to test whether the stagnant changefeed is put into the stop queue. -func (s *ownerSuite) TestOwnerHandleStaleChangeFeed(c *check.C) { - defer testleak.AfterTest(c)() - mockPDCli := &mockPDClient{} - changeFeeds := map[model.ChangeFeedID]*changeFeed{ - "test_change_feed_1": { - info: &model.ChangeFeedInfo{State: model.StateNormal}, - etcdCli: s.client, - status: &model.ChangeFeedStatus{ - CheckpointTs: 1000, - }, - targetTs: 2000, - ddlState: model.ChangeFeedSyncDML, - taskStatus: model.ProcessorsInfos{ - "capture_1": {}, - "capture_2": {}, - }, - taskPositions: map[string]*model.TaskPosition{ - "capture_1": {}, - "capture_2": {}, - }, - }, - "test_change_feed_2": { - info: &model.ChangeFeedInfo{State: model.StateNormal}, - etcdCli: s.client, - status: &model.ChangeFeedStatus{ - CheckpointTs: oracle.EncodeTSO(oracle.GetPhysical(time.Now())), - }, - targetTs: 2000, - ddlState: model.ChangeFeedSyncDML, - taskStatus: model.ProcessorsInfos{ - "capture_1": {}, - "capture_2": {}, - }, - taskPositions: map[string]*model.TaskPosition{ - "capture_1": {}, - "capture_2": {}, - }, - }, - } - - session, err := concurrency.NewSession(s.client.Client.Unwrap(), - concurrency.WithTTL(config.GetDefaultServerConfig().CaptureSessionTTL)) - c.Assert(err, check.IsNil) - - mockOwner := Owner{ - session: session, - pdClient: mockPDCli, - etcdClient: s.client, - lastFlushChangefeeds: time.Now(), - flushChangefeedInterval: 1 * time.Hour, - gcSafepointLastUpdate: time.Now().Add(-4 * time.Second), - gcTTL: 6, // 6 seconds - changeFeeds: changeFeeds, - cfRWriter: s.client, - stoppedFeeds: make(map[model.ChangeFeedID]*model.ChangeFeedStatus), - minGCSafePointCache: minGCSafePointCacheEntry{}, - } - - err = mockOwner.flushChangeFeedInfos(s.ctx) - c.Assert(err, check.IsNil) - c.Assert(mockPDCli.invokeCounter, check.Equals, 1) - err = mockOwner.handleAdminJob(s.ctx) - c.Assert(err, check.IsNil) - - time.Sleep(2 * time.Second) - err = mockOwner.flushChangeFeedInfos(s.ctx) - c.Assert(err, check.IsNil) - c.Assert(mockPDCli.invokeCounter, check.Equals, 2) - err = mockOwner.handleAdminJob(s.ctx) - c.Assert(err, check.IsNil) - - c.Assert(mockOwner.stoppedFeeds["test_change_feed_1"], check.NotNil) - c.Assert(mockOwner.changeFeeds["test_change_feed_2"].info.State, check.Equals, model.StateNormal) - - time.Sleep(6 * time.Second) - err = mockOwner.flushChangeFeedInfos(s.ctx) - c.Assert(err, check.IsNil) - c.Assert(mockPDCli.invokeCounter, check.Equals, 3) - err = mockOwner.handleAdminJob(s.ctx) - c.Assert(err, check.IsNil) - - time.Sleep(2 * time.Second) - err = mockOwner.flushChangeFeedInfos(s.ctx) - c.Assert(err, check.IsNil) - c.Assert(mockPDCli.invokeCounter, check.Equals, 4) - err = mockOwner.handleAdminJob(s.ctx) - c.Assert(err, check.IsNil) - c.Assert(mockOwner.stoppedFeeds["test_change_feed_2"], check.NotNil) - - s.TearDownTest(c) -} - -func (s *ownerSuite) TestOwnerUploadGCSafePointOutdated(c *check.C) { - defer testleak.AfterTest(c)() - mockPDCli := &mockPDClient{ - mockSafePointLost: true, - } - changeFeeds := map[model.ChangeFeedID]*changeFeed{ - "test_change_feed_1": { - info: &model.ChangeFeedInfo{State: model.StateNormal}, - etcdCli: s.client, - status: &model.ChangeFeedStatus{ - CheckpointTs: 100, - }, - targetTs: 2000, - ddlState: model.ChangeFeedSyncDML, - taskStatus: model.ProcessorsInfos{ - "capture_1": {}, - "capture_2": {}, - }, - taskPositions: map[string]*model.TaskPosition{ - "capture_1": {}, - "capture_2": {}, - }, - }, - "test_change_feed_2": { - info: &model.ChangeFeedInfo{State: model.StateNormal}, - etcdCli: s.client, - status: &model.ChangeFeedStatus{ - CheckpointTs: 1100, - }, - targetTs: 2000, - ddlState: model.ChangeFeedSyncDML, - taskStatus: model.ProcessorsInfos{ - "capture_1": {}, - "capture_2": {}, - }, - taskPositions: map[string]*model.TaskPosition{ - "capture_1": {}, - "capture_2": {}, - }, - }, - } - - session, err := concurrency.NewSession(s.client.Client.Unwrap(), - concurrency.WithTTL(config.GetDefaultServerConfig().CaptureSessionTTL)) - c.Assert(err, check.IsNil) - - mockOwner := Owner{ - pdClient: mockPDCli, - session: session, - etcdClient: s.client, - lastFlushChangefeeds: time.Now(), - flushChangefeedInterval: 1 * time.Hour, - changeFeeds: changeFeeds, - cfRWriter: s.client, - stoppedFeeds: make(map[model.ChangeFeedID]*model.ChangeFeedStatus), - minGCSafePointCache: minGCSafePointCacheEntry{}, - } - - err = mockOwner.flushChangeFeedInfos(s.ctx) - c.Assert(err, check.IsNil) - c.Assert(mockPDCli.invokeCounter, check.Equals, 1) - - err = mockOwner.handleAdminJob(s.ctx) - c.Assert(err, check.IsNil) - - c.Assert(mockOwner.stoppedFeeds["test_change_feed_1"], check.NotNil) - c.Assert(changeFeeds["test_change_feed_2"].info.State, check.Equals, model.StateNormal) - s.TearDownTest(c) -} - -/* -type handlerForPrueDMLTest struct { - mu sync.RWMutex - index int - resolvedTs1 []uint64 - resolvedTs2 []uint64 - expectResolvedTs []uint64 - c *check.C - cancel func() -} - -func (h *handlerForPrueDMLTest) PullDDL() (resolvedTs uint64, ddl []*model.DDL, err error) { - return uint64(math.MaxUint64), nil, nil -} - -func (h *handlerForPrueDMLTest) ExecDDL(context.Context, string, map[string]string, model.SingleTableTxn) error { - panic("unreachable") -} - -func (h *handlerForPrueDMLTest) Close() error { - return nil -} - -var _ ChangeFeedRWriter = &handlerForPrueDMLTest{} - -func (h *handlerForPrueDMLTest) GetChangeFeeds(ctx context.Context) (int64, map[string]*mvccpb.KeyValue, error) { - h.mu.RLock() - defer h.mu.RUnlock() - cfInfo := &model.ChangeFeedInfo{ - TargetTs: 100, - } - cfInfoJSON, err := cfInfo.Marshal() - h.c.Assert(err, check.IsNil) - rawKV := &mvccpb.KeyValue{ - Value: []byte(cfInfoJSON), - } - return 0, map[model.ChangeFeedID]*mvccpb.KeyValue{ - "test_change_feed": rawKV, - }, nil -} - -func (h *handlerForPrueDMLTest) GetAllTaskStatus(ctx context.Context, changefeedID string) (model.ProcessorsInfos, error) { - if changefeedID != "test_change_feed" { - return nil, cerror.ErrTaskStatusNotExists.GenWithStackByArgs("test_change_feed) - } - h.mu.RLock() - defer h.mu.RUnlock() - h.index++ - return model.ProcessorsInfos{ - "capture_1": {}, - "capture_2": {}, - }, nil -} - -func (h *handlerForPrueDMLTest) GetAllTaskPositions(ctx context.Context, changefeedID string) (map[string]*model.TaskPosition, error) { - if changefeedID != "test_change_feed" { - return nil, cerror.ErrTaskStatusNotExists.GenWithStackByArgs("test_change_feed) - } - h.mu.RLock() - defer h.mu.RUnlock() - h.index++ - return map[string]*model.TaskPosition{ - "capture_1": { - ResolvedTs: h.resolvedTs1[h.index], - }, - "capture_2": { - ResolvedTs: h.resolvedTs2[h.index], - }, - }, nil -} - -func (h *handlerForPrueDMLTest) GetChangeFeedStatus(ctx context.Context, id string) (*model.ChangeFeedStatus, error) { - return nil, cerror.ErrChangeFeedNotExists.GenWithStackByArgs(id) -} - -func (h *handlerForPrueDMLTest) PutAllChangeFeedStatus(ctx context.Context, infos map[model.ChangeFeedID]*model.ChangeFeedStatus) error { - h.mu.Lock() - defer h.mu.Unlock() - info, exist := infos["test_change_feed"] - h.c.Assert(exist, check.IsTrue) - h.c.Assert(info.ResolvedTs, check.Equals, h.expectResolvedTs[h.index]) - // h.c.Assert(info.State, check.Equals, model.ChangeFeedSyncDML) - if h.index >= len(h.expectResolvedTs)-1 { - log.Info("cancel") - h.cancel() - } - return nil -} - -func (s *ownerSuite) TestPureDML(c *check.C) { - defer testleak.AfterTest(c)() - ctx, cancel := context.WithCancel(context.Background()) - handler := &handlerForPrueDMLTest{ - index: -1, - resolvedTs1: []uint64{10, 22, 64, 92, 99, 120}, - resolvedTs2: []uint64{8, 36, 53, 88, 103, 108}, - expectResolvedTs: []uint64{8, 22, 53, 88, 99, 100}, - cancel: cancel, - c: c, - } - - tables := map[uint64]model.TableName{1: {Schema: "any"}} - - changeFeeds := map[model.ChangeFeedID]*changeFeed{ - "test_change_feed": { - tables: tables, - status: &model.ChangeFeedStatus{}, - targetTs: 100, - ddlState: model.ChangeFeedSyncDML, - taskStatus: model.ProcessorsInfos{ - "capture_1": {}, - "capture_2": {}, - }, - taskPositions: map[string]*model.TaskPosition{ - "capture_1": {}, - "capture_2": {}, - }, - ddlHandler: handler, - }, - } - - manager := roles.NewMockManager(uuid.New().String(), cancel) - err := manager.CampaignOwner(ctx) - c.Assert(err, check.IsNil) - owner := &ownerImpl{ - cancelWatchCapture: cancel, - changeFeeds: changeFeeds, - cfRWriter: handler, - etcdClient: s.client, - manager: manager, - } - s.owner = owner - err = owner.Run(ctx, 50*time.Millisecond) - c.Assert(err.Error(), check.Equals, "context canceled") -} - -type handlerForDDLTest struct { - mu sync.RWMutex - - ddlIndex int - ddls []*model.DDL - ddlResolvedTs []uint64 - - ddlExpectIndex int - - dmlIndex int - resolvedTs1 []uint64 - resolvedTs2 []uint64 - currentGlobalResolvedTs uint64 - - dmlExpectIndex int - expectResolvedTs []uint64 - expectStatus []model.ChangeFeedDDLState - - c *check.C - cancel func() -} - -func (h *handlerForDDLTest) PullDDL() (resolvedTs uint64, jobs []*model.DDL, err error) { - h.mu.RLock() - defer h.mu.RUnlock() - if h.ddlIndex < len(h.ddls)-1 { - h.ddlIndex++ - } - return h.ddlResolvedTs[h.ddlIndex], []*model.DDL{h.ddls[h.ddlIndex]}, nil -} - -func (h *handlerForDDLTest) ExecDDL(ctx context.Context, sinkURI string, _ map[string]string, txn model.SingleTableTxn) error { - h.mu.Lock() - defer h.mu.Unlock() - h.ddlExpectIndex++ - h.c.Assert(txn.DDL, check.DeepEquals, h.ddls[h.ddlExpectIndex]) - h.c.Assert(txn.DDL.Job.BinlogInfo.FinishedTS, check.Equals, h.currentGlobalResolvedTs) - return nil -} - -func (h *handlerForDDLTest) Close() error { - return nil -} - -func (h *handlerForDDLTest) GetChangeFeeds(ctx context.Context) (int64, map[string]*mvccpb.KeyValue, error) { - h.mu.RLock() - defer h.mu.RUnlock() - cfInfo := &model.ChangeFeedInfo{ - TargetTs: 100, - } - cfInfoJSON, err := cfInfo.Marshal() - h.c.Assert(err, check.IsNil) - rawKV := &mvccpb.KeyValue{ - Value: []byte(cfInfoJSON), - } - return 0, map[model.ChangeFeedID]*mvccpb.KeyValue{ - "test_change_feed": rawKV, - }, nil -} - -func (h *handlerForDDLTest) GetAllTaskStatus(ctx context.Context, changefeedID string) (model.ProcessorsInfos, error) { - if changefeedID != "test_change_feed" { - return nil, cerror.ErrTaskStatusNotExists.GenWithStackByArgs("test_change_feed") - } - h.mu.RLock() - defer h.mu.RUnlock() - if h.dmlIndex < len(h.resolvedTs1)-1 { - h.dmlIndex++ - } - return model.ProcessorsInfos{ - "capture_1": {}, - "capture_2": {}, - }, nil -} - -func (h *handlerForDDLTest) GetAllTaskPositions(ctx context.Context, changefeedID string) (map[string]*model.TaskPosition, error) { - if changefeedID != "test_change_feed" { - return nil, cerror.ErrTaskStatusNotExists.GenWithStackByArgs("test_change_feed") - } - h.mu.RLock() - defer h.mu.RUnlock() - if h.dmlIndex < len(h.resolvedTs1)-1 { - h.dmlIndex++ - } - return map[string]*model.TaskPosition{ - "capture_1": { - ResolvedTs: h.resolvedTs1[h.dmlIndex], - CheckPointTs: h.currentGlobalResolvedTs, - }, - "capture_2": { - ResolvedTs: h.resolvedTs2[h.dmlIndex], - CheckPointTs: h.currentGlobalResolvedTs, - }, - }, nil -} - -func (h *handlerForDDLTest) GetChangeFeedStatus(ctx context.Context, id string) (*model.ChangeFeedStatus, error) { - return nil, cerror.ErrChangeFeedNotExists.GenWithStackByArgs(id) -} - -func (h *handlerForDDLTest) PutAllChangeFeedStatus(ctx context.Context, infos map[model.ChangeFeedID]*model.ChangeFeedStatus) error { - h.mu.Lock() - defer h.mu.Unlock() - h.dmlExpectIndex++ - info, exist := infos["test_change_feed"] - h.c.Assert(exist, check.IsTrue) - h.currentGlobalResolvedTs = info.ResolvedTs - h.c.Assert(info.ResolvedTs, check.Equals, h.expectResolvedTs[h.dmlExpectIndex]) - // h.c.Assert(info.State, check.Equals, h.expectStatus[h.dmlExpectIndex]) - if h.dmlExpectIndex >= len(h.expectResolvedTs)-1 { - log.Info("cancel") - h.cancel() - } - return nil -} - -func (s *ownerSuite) TestDDL(c *check.C) { - defer testleak.AfterTest(c)() - ctx, cancel := context.WithCancel(context.Background()) - - handler := &handlerForDDLTest{ - ddlIndex: -1, - ddlResolvedTs: []uint64{5, 8, 49, 91, 113}, - ddls: []*model.DDL{ - {Job: &timodel.Job{ - ID: 1, - BinlogInfo: &timodel.HistoryInfo{ - FinishedTS: 3, - }, - }}, - {Job: &timodel.Job{ - ID: 2, - BinlogInfo: &timodel.HistoryInfo{ - FinishedTS: 7, - }, - }}, - {Job: &timodel.Job{ - ID: 3, - BinlogInfo: &timodel.HistoryInfo{ - FinishedTS: 11, - }, - }}, - {Job: &timodel.Job{ - ID: 4, - BinlogInfo: &timodel.HistoryInfo{ - FinishedTS: 89, - }, - }}, - {Job: &timodel.Job{ - ID: 5, - BinlogInfo: &timodel.HistoryInfo{ - FinishedTS: 111, - }, - }}, - }, - - ddlExpectIndex: -1, - - dmlIndex: -1, - resolvedTs1: []uint64{10, 22, 64, 92, 99, 120}, - resolvedTs2: []uint64{8, 36, 53, 88, 103, 108}, - currentGlobalResolvedTs: 0, - - dmlExpectIndex: -1, - expectResolvedTs: []uint64{ - 3, 3, - 7, 7, - 11, 11, - 89, 89, - 100}, - expectStatus: []model.ChangeFeedDDLState{ - model.ChangeFeedWaitToExecDDL, model.ChangeFeedExecDDL, - model.ChangeFeedWaitToExecDDL, model.ChangeFeedExecDDL, - model.ChangeFeedWaitToExecDDL, model.ChangeFeedExecDDL, - model.ChangeFeedWaitToExecDDL, model.ChangeFeedExecDDL, - model.ChangeFeedSyncDML}, - - cancel: cancel, - c: c, - } - - tables := map[uint64]model.TableName{1: {Schema: "any"}} - - filter, err := newTxnFilter(&model.ReplicaConfig{}) - c.Assert(err, check.IsNil) - changeFeeds := map[model.ChangeFeedID]*changeFeed{ - "test_change_feed": { - tables: tables, - info: &model.ChangeFeedInfo{}, - status: &model.ChangeFeedStatus{}, - targetTs: 100, - ddlState: model.ChangeFeedSyncDML, - taskStatus: model.ProcessorsInfos{ - "capture_1": {}, - "capture_2": {}, - }, - taskPositions: map[string]*model.TaskPosition{ - "capture_1": {}, - "capture_2": {}, - }, - ddlHandler: handler, - filter: filter, - }, - } - - manager := roles.NewMockManager(uuid.New().String(), cancel) - err = manager.CampaignOwner(ctx) - c.Assert(err, check.IsNil) - owner := &ownerImpl{ - cancelWatchCapture: cancel, - changeFeeds: changeFeeds, - - // ddlHandler: handler, - etcdClient: s.client, - cfRWriter: handler, - manager: manager, - } - s.owner = owner - err = owner.Run(ctx, 50*time.Millisecond) - c.Assert(errors.Cause(err), check.DeepEquals, context.Canceled) -} -*/ -var cdcGCSafePointTTL4Test = int64(24 * 60 * 60) - -func (s *ownerSuite) TestHandleAdmin(c *check.C) { - defer testleak.AfterTest(c)() - defer s.TearDownTest(c) - cfID := "test_handle_admin" - - ctx, cancel0 := context.WithCancel(context.Background()) - defer cancel0() - cctx, cancel := context.WithCancel(ctx) - errg, _ := errgroup.WithContext(cctx) - - replicaConf := config.GetDefaultReplicaConfig() - f, err := filter.NewFilter(replicaConf) - c.Assert(err, check.IsNil) - - sampleCF := &changeFeed{ - id: cfID, - info: &model.ChangeFeedInfo{}, - status: &model.ChangeFeedStatus{}, - ddlState: model.ChangeFeedSyncDML, - taskStatus: model.ProcessorsInfos{ - "capture_1": {}, - "capture_2": {}, - }, - taskPositions: map[string]*model.TaskPosition{ - "capture_1": {ResolvedTs: 10001}, - "capture_2": {}, - }, - ddlHandler: &ddlHandler{ - cancel: cancel, - wg: errg, - }, - cancel: cancel, - } - errCh := make(chan error, 1) - sink, err := sink.NewSink(ctx, cfID, "blackhole://", f, replicaConf, map[string]string{}, errCh) - c.Assert(err, check.IsNil) - defer sink.Close(cctx) //nolint:errcheck - sampleCF.sink = sink - - capture, err := NewCapture(ctx, []string{s.clientURL.String()}, nil, nil) - c.Assert(err, check.IsNil) - err = capture.Campaign(ctx) - c.Assert(err, check.IsNil) - - grpcPool := kv.NewGrpcPoolImpl(ctx, &security.Credential{}) - defer grpcPool.Close() - owner, err := NewOwner(ctx, nil, grpcPool, capture.session, cdcGCSafePointTTL4Test, time.Millisecond*200) - c.Assert(err, check.IsNil) - - sampleCF.etcdCli = owner.etcdClient - owner.changeFeeds = map[model.ChangeFeedID]*changeFeed{cfID: sampleCF} - for cid, pinfo := range sampleCF.taskPositions { - key := kv.GetEtcdKeyTaskStatus(cfID, cid) - pinfoStr, err := pinfo.Marshal() - c.Assert(err, check.IsNil) - _, err = s.client.Client.Put(ctx, key, pinfoStr) - c.Assert(err, check.IsNil) - } - err = owner.etcdClient.PutChangeFeedStatus(ctx, cfID, &model.ChangeFeedStatus{}) - c.Assert(err, check.IsNil) - err = owner.etcdClient.SaveChangeFeedInfo(ctx, sampleCF.info, cfID) - c.Assert(err, check.IsNil) - checkAdminJobLen := func(length int) { - owner.adminJobsLock.Lock() - c.Assert(owner.adminJobs, check.HasLen, length) - owner.adminJobsLock.Unlock() - } - - c.Assert(owner.EnqueueJob(model.AdminJob{CfID: cfID, Type: model.AdminStop}), check.IsNil) - checkAdminJobLen(1) - c.Assert(owner.handleAdminJob(ctx), check.IsNil) - checkAdminJobLen(0) - c.Assert(len(owner.changeFeeds), check.Equals, 0) - // check changefeed info is set admin job - info, err := owner.etcdClient.GetChangeFeedInfo(ctx, cfID) - c.Assert(err, check.IsNil) - c.Assert(info.AdminJobType, check.Equals, model.AdminStop) - // check processor is set admin job - for cid := range sampleCF.taskPositions { - _, subInfo, err := owner.etcdClient.GetTaskStatus(ctx, cfID, cid) - c.Assert(err, check.IsNil) - c.Assert(subInfo.AdminJobType, check.Equals, model.AdminStop) - } - // check changefeed status is set admin job - st, _, err := owner.etcdClient.GetChangeFeedStatus(ctx, cfID) - c.Assert(err, check.IsNil) - c.Assert(st.AdminJobType, check.Equals, model.AdminStop) - // check changefeed context is canceled - select { - case <-cctx.Done(): - default: - c.Fatal("changefeed context is expected canceled") - } - - cctx, cancel = context.WithCancel(ctx) - sampleCF.cancel = cancel - - c.Assert(owner.EnqueueJob(model.AdminJob{CfID: cfID, Type: model.AdminResume}), check.IsNil) - c.Assert(owner.handleAdminJob(ctx), check.IsNil) - checkAdminJobLen(0) - // check changefeed info is set admin job - info, err = owner.etcdClient.GetChangeFeedInfo(ctx, cfID) - c.Assert(err, check.IsNil) - c.Assert(info.AdminJobType, check.Equals, model.AdminResume) - // check changefeed status is set admin job - st, _, err = owner.etcdClient.GetChangeFeedStatus(ctx, cfID) - c.Assert(err, check.IsNil) - c.Assert(st.AdminJobType, check.Equals, model.AdminResume) - - owner.changeFeeds[cfID] = sampleCF - c.Assert(owner.EnqueueJob(model.AdminJob{CfID: cfID, Type: model.AdminRemove}), check.IsNil) - c.Assert(owner.handleAdminJob(ctx), check.IsNil) - checkAdminJobLen(0) - c.Assert(len(owner.changeFeeds), check.Equals, 0) - // check changefeed info is deleted - _, err = owner.etcdClient.GetChangeFeedInfo(ctx, cfID) - c.Assert(cerror.ErrChangeFeedNotExists.Equal(err), check.IsTrue) - // check processor is set admin job - for cid := range sampleCF.taskPositions { - _, subInfo, err := owner.etcdClient.GetTaskStatus(ctx, cfID, cid) - c.Assert(err, check.IsNil) - c.Assert(subInfo.AdminJobType, check.Equals, model.AdminRemove) - } - // check changefeed status is set admin job - st, _, err = owner.etcdClient.GetChangeFeedStatus(ctx, cfID) - c.Assert(err, check.IsNil) - c.Assert(st.AdminJobType, check.Equals, model.AdminRemove) - // check changefeed context is canceled - select { - case <-cctx.Done(): - default: - c.Fatal("changefeed context is expected canceled") - } - owner.etcdClient.Close() //nolint:errcheck -} - -func (s *ownerSuite) TestChangefeedApplyDDLJob(c *check.C) { - defer testleak.AfterTest(c)() - var ( - jobs = []*timodel.Job{ - { - ID: 1, - SchemaID: 1, - Type: timodel.ActionCreateSchema, - State: timodel.JobStateSynced, - Query: "create database test", - BinlogInfo: &timodel.HistoryInfo{ - SchemaVersion: 1, - DBInfo: &timodel.DBInfo{ - ID: 1, - Name: timodel.NewCIStr("test"), - }, - }, - }, - { - ID: 2, - SchemaID: 1, - Type: timodel.ActionCreateTable, - State: timodel.JobStateSynced, - Query: "create table t1 (id int primary key)", - BinlogInfo: &timodel.HistoryInfo{ - SchemaVersion: 2, - DBInfo: &timodel.DBInfo{ - ID: 1, - Name: timodel.NewCIStr("test"), - }, - TableInfo: &timodel.TableInfo{ - ID: 47, - Name: timodel.NewCIStr("t1"), - PKIsHandle: true, - Columns: []*timodel.ColumnInfo{ - {ID: 1, FieldType: types.FieldType{Flag: mysql.PriKeyFlag}, State: timodel.StatePublic}, - }, - }, - }, - }, - { - ID: 2, - SchemaID: 1, - Type: timodel.ActionCreateTable, - State: timodel.JobStateSynced, - Query: "create table t2 (id int primary key)", - BinlogInfo: &timodel.HistoryInfo{ - SchemaVersion: 2, - DBInfo: &timodel.DBInfo{ - ID: 1, - Name: timodel.NewCIStr("test"), - }, - TableInfo: &timodel.TableInfo{ - ID: 49, - Name: timodel.NewCIStr("t2"), - PKIsHandle: true, - Columns: []*timodel.ColumnInfo{ - {ID: 1, FieldType: types.FieldType{Flag: mysql.PriKeyFlag}, State: timodel.StatePublic}, - }, - }, - }, - }, - { - ID: 2, - SchemaID: 1, - TableID: 49, - Type: timodel.ActionDropTable, - State: timodel.JobStateSynced, - Query: "drop table t2", - BinlogInfo: &timodel.HistoryInfo{ - SchemaVersion: 3, - DBInfo: &timodel.DBInfo{ - ID: 1, - Name: timodel.NewCIStr("test"), - }, - TableInfo: &timodel.TableInfo{ - ID: 49, - Name: timodel.NewCIStr("t2"), - }, - }, - }, - { - ID: 2, - SchemaID: 1, - TableID: 47, - Type: timodel.ActionTruncateTable, - State: timodel.JobStateSynced, - Query: "truncate table t1", - BinlogInfo: &timodel.HistoryInfo{ - SchemaVersion: 4, - DBInfo: &timodel.DBInfo{ - ID: 1, - Name: timodel.NewCIStr("test"), - }, - TableInfo: &timodel.TableInfo{ - ID: 51, - Name: timodel.NewCIStr("t1"), - PKIsHandle: true, - Columns: []*timodel.ColumnInfo{ - {ID: 1, FieldType: types.FieldType{Flag: mysql.PriKeyFlag}, State: timodel.StatePublic}, - }, - }, - }, - }, - { - ID: 2, - SchemaID: 1, - TableID: 51, - Type: timodel.ActionDropTable, - State: timodel.JobStateSynced, - Query: "drop table t1", - BinlogInfo: &timodel.HistoryInfo{ - SchemaVersion: 5, - DBInfo: &timodel.DBInfo{ - ID: 1, - Name: timodel.NewCIStr("test"), - }, - TableInfo: &timodel.TableInfo{ - ID: 51, - Name: timodel.NewCIStr("t1"), - }, - }, - }, - { - ID: 2, - SchemaID: 1, - Type: timodel.ActionDropSchema, - State: timodel.JobStateSynced, - Query: "drop database test", - BinlogInfo: &timodel.HistoryInfo{ - SchemaVersion: 6, - DBInfo: &timodel.DBInfo{ - ID: 1, - Name: timodel.NewCIStr("test"), - }, - }, - }, - } - - expectSchemas = []map[int64]tableIDMap{ - {1: make(tableIDMap)}, - {1: {47: struct{}{}}}, - {1: {47: struct{}{}, 49: struct{}{}}}, - {1: {47: struct{}{}}}, - {1: {51: struct{}{}}}, - {1: make(tableIDMap)}, - {}, - } - - expectTables = []map[int64]model.TableName{ - {}, - {47: {Schema: "test", Table: "t1"}}, - {47: {Schema: "test", Table: "t1"}, 49: {Schema: "test", Table: "t2"}}, - {47: {Schema: "test", Table: "t1"}}, - {51: {Schema: "test", Table: "t1"}}, - {}, - {}, - } - ) - f, err := filter.NewFilter(config.GetDefaultReplicaConfig()) - c.Assert(err, check.IsNil) - - store, err := mockstore.NewMockStore() - c.Assert(err, check.IsNil) - defer func() { - _ = store.Close() - }() - - txn, err := store.Begin() - c.Assert(err, check.IsNil) - defer func() { - _ = txn.Rollback() - }() - t := meta.NewMeta(txn) - - schemaSnap, err := entry.NewSingleSchemaSnapshotFromMeta(t, 0, false) - c.Assert(err, check.IsNil) - - cf := &changeFeed{ - schema: schemaSnap, - schemas: make(map[model.SchemaID]tableIDMap), - tables: make(map[model.TableID]model.TableName), - partitions: make(map[model.TableID][]int64), - orphanTables: make(map[model.TableID]model.Ts), - toCleanTables: make(map[model.TableID]model.Ts), - filter: f, - info: &model.ChangeFeedInfo{Config: config.GetDefaultReplicaConfig()}, - } - for i, job := range jobs { - err = cf.schema.HandleDDL(job) - c.Assert(err, check.IsNil) - err = cf.schema.FillSchemaName(job) - c.Assert(err, check.IsNil) - _, err = cf.applyJob(job) - c.Assert(err, check.IsNil) - c.Assert(cf.schemas, check.DeepEquals, expectSchemas[i]) - c.Assert(cf.tables, check.DeepEquals, expectTables[i]) - } - s.TearDownTest(c) -} - -func (s *ownerSuite) TestWatchCampaignKey(c *check.C) { - defer testleak.AfterTest(c)() - defer s.TearDownTest(c) - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - capture, err := NewCapture(ctx, []string{s.clientURL.String()}, nil, nil) - c.Assert(err, check.IsNil) - err = capture.Campaign(ctx) - c.Assert(err, check.IsNil) - - grpcPool := kv.NewGrpcPoolImpl(ctx, &security.Credential{}) - defer grpcPool.Close() - ctx1, cancel1 := context.WithCancel(ctx) - owner, err := NewOwner(ctx1, nil, grpcPool, capture.session, - cdcGCSafePointTTL4Test, time.Millisecond*200) - c.Assert(err, check.IsNil) - - // check campaign key deleted can be detected - var wg sync.WaitGroup - wg.Add(1) - go func() { - defer wg.Done() - err := owner.watchCampaignKey(ctx1) - c.Assert(cerror.ErrOwnerCampaignKeyDeleted.Equal(err), check.IsTrue) - cancel1() - }() - // ensure the watch loop has started - time.Sleep(time.Millisecond * 100) - etcdCli := owner.etcdClient.Client.Unwrap() - key := fmt.Sprintf("%s/%x", kv.CaptureOwnerKey, owner.session.Lease()) - _, err = etcdCli.Delete(ctx, key) - c.Assert(err, check.IsNil) - wg.Wait() - - // check key is deleted before watch loop starts - ctx1, cancel1 = context.WithCancel(ctx) - err = owner.watchCampaignKey(ctx1) - c.Assert(cerror.ErrOwnerCampaignKeyDeleted.Equal(err), check.IsTrue) - - // check the watch routine can be canceled - err = capture.Campaign(ctx) - c.Assert(err, check.IsNil) - wg.Add(1) - go func() { - defer wg.Done() - err := owner.watchCampaignKey(ctx1) - c.Assert(err, check.IsNil) - }() - // ensure the watch loop has started - time.Sleep(time.Millisecond * 100) - cancel1() - wg.Wait() - - err = capture.etcdClient.Close() - c.Assert(err, check.IsNil) -} - -func (s *ownerSuite) TestCleanUpStaleTasks(c *check.C) { - defer testleak.AfterTest(c)() - defer s.TearDownTest(c) - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - addr := "127.0.0.1:12034" - ctx = util.PutCaptureAddrInCtx(ctx, addr) - capture, err := NewCapture(ctx, []string{s.clientURL.String()}, nil, nil) - c.Assert(err, check.IsNil) - err = s.client.PutCaptureInfo(ctx, capture.info, capture.session.Lease()) - c.Assert(err, check.IsNil) - - changefeed := "changefeed-name" - invalidCapture := uuid.New().String() - for _, captureID := range []string{capture.info.ID, invalidCapture} { - taskStatus := &model.TaskStatus{} - if captureID == invalidCapture { - taskStatus.Tables = map[model.TableID]*model.TableReplicaInfo{ - 51: {StartTs: 110}, - } - } - err = s.client.PutTaskStatus(ctx, changefeed, captureID, taskStatus) - c.Assert(err, check.IsNil) - _, err = s.client.PutTaskPositionOnChange(ctx, changefeed, captureID, &model.TaskPosition{CheckPointTs: 100, ResolvedTs: 120}) - c.Assert(err, check.IsNil) - err = s.client.PutTaskWorkload(ctx, changefeed, captureID, &model.TaskWorkload{}) - c.Assert(err, check.IsNil) - } - err = s.client.SaveChangeFeedInfo(ctx, &model.ChangeFeedInfo{}, changefeed) - c.Assert(err, check.IsNil) - - _, captureList, err := s.client.GetCaptures(ctx) - c.Assert(err, check.IsNil) - captures := make(map[model.CaptureID]*model.CaptureInfo) - for _, c := range captureList { - captures[c.ID] = c - } - grpcPool := kv.NewGrpcPoolImpl(ctx, &security.Credential{}) - defer grpcPool.Close() - owner, err := NewOwner(ctx, nil, grpcPool, capture.session, - cdcGCSafePointTTL4Test, time.Millisecond*200) - c.Assert(err, check.IsNil) - // It is better to update changefeed information by `loadChangeFeeds`, however - // `loadChangeFeeds` is too overweight, just mock enough information here. - owner.changeFeeds = map[model.ChangeFeedID]*changeFeed{ - changefeed: { - id: changefeed, - orphanTables: make(map[model.TableID]model.Ts), - status: &model.ChangeFeedStatus{ - CheckpointTs: 100, - }, - }, - } - - // capture information is not built, owner.run does nothing - err = owner.run(ctx) - c.Assert(err, check.IsNil) - statuses, err := s.client.GetAllTaskStatus(ctx, changefeed) - c.Assert(err, check.IsNil) - // stale tasks are not cleaned up, since `cleanUpStaleTasks` does not run - c.Assert(len(statuses), check.Equals, 2) - c.Assert(len(owner.captures), check.Equals, 0) - - err = owner.rebuildCaptureEvents(ctx, captures) - c.Assert(err, check.IsNil) - c.Assert(len(owner.captures), check.Equals, 1) - c.Assert(owner.captures, check.HasKey, capture.info.ID) - c.Assert(owner.changeFeeds[changefeed].orphanTables, check.DeepEquals, map[model.TableID]model.Ts{51: 110}) - c.Assert(atomic.LoadInt32(&owner.captureLoaded), check.Equals, int32(1)) - // check stale tasks are cleaned up - statuses, err = s.client.GetAllTaskStatus(ctx, changefeed) - c.Assert(err, check.IsNil) - c.Assert(len(statuses), check.Equals, 1) - c.Assert(statuses, check.HasKey, capture.info.ID) - positions, err := s.client.GetAllTaskPositions(ctx, changefeed) - c.Assert(err, check.IsNil) - c.Assert(len(positions), check.Equals, 1) - c.Assert(positions, check.HasKey, capture.info.ID) - workloads, err := s.client.GetAllTaskWorkloads(ctx, changefeed) - c.Assert(err, check.IsNil) - c.Assert(len(workloads), check.Equals, 1) - c.Assert(workloads, check.HasKey, capture.info.ID) - - err = capture.etcdClient.Close() - c.Assert(err, check.IsNil) -} - -func (s *ownerSuite) TestWatchFeedChange(c *check.C) { - defer testleak.AfterTest(c)() - defer s.TearDownTest(c) - - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - addr := "127.0.0.1:12034" - ctx = util.PutCaptureAddrInCtx(ctx, addr) - capture, err := NewCapture(ctx, []string{s.clientURL.String()}, nil, nil) - c.Assert(err, check.IsNil) - grpcPool := kv.NewGrpcPoolImpl(ctx, &security.Credential{}) - defer grpcPool.Close() - owner, err := NewOwner(ctx, nil, grpcPool, capture.session, - cdcGCSafePointTTL4Test, time.Millisecond*200) - c.Assert(err, check.IsNil) - - var ( - wg sync.WaitGroup - updateCount = 0 - recvChangeCount = 0 - ) - ctx1, cancel1 := context.WithCancel(ctx) - wg.Add(1) - go func() { - defer wg.Done() - changefeedID := "test-changefeed" - pos := &model.TaskPosition{CheckPointTs: 100, ResolvedTs: 102} - for { - select { - case <-ctx1.Done(): - return - default: - } - pos.ResolvedTs++ - pos.CheckPointTs++ - updated, err := capture.etcdClient.PutTaskPositionOnChange(ctx1, changefeedID, capture.info.ID, pos) - if errors.Cause(err) == context.Canceled { - return - } - c.Assert(err, check.IsNil) - c.Assert(updated, check.IsTrue) - updateCount++ - // sleep to avoid other goroutine starvation - time.Sleep(time.Millisecond) - } - }() - - feedChangeReceiver, err := owner.feedChangeNotifier.NewReceiver(ownerRunInterval) - c.Assert(err, check.IsNil) - defer feedChangeReceiver.Stop() - owner.watchFeedChange(ctx) - wg.Add(1) - go func() { - defer func() { - // there could be one message remaining in notification receiver, try to consume it - select { - case <-feedChangeReceiver.C: - default: - } - wg.Done() - }() - for { - select { - case <-ctx1.Done(): - return - case <-feedChangeReceiver.C: - recvChangeCount++ - // sleep to simulate some owner work - time.Sleep(time.Millisecond * 50) - } - } - }() - - time.Sleep(time.Second * 2) - // use cancel1 to avoid cancel the watchFeedChange - cancel1() - wg.Wait() - c.Assert(recvChangeCount, check.Greater, 0) - c.Assert(recvChangeCount, check.Less, updateCount) - select { - case <-feedChangeReceiver.C: - c.Error("should not receive message from feed change chan any more") - default: - } - - err = capture.etcdClient.Close() - if err != nil { - c.Assert(errors.Cause(err), check.Equals, context.Canceled) - } -} - -func (s *ownerSuite) TestWriteDebugInfo(c *check.C) { - defer testleak.AfterTest(c)() - defer s.TearDownTest(c) - owner := &Owner{ - changeFeeds: map[model.ChangeFeedID]*changeFeed{ - "test": { - id: "test", - info: &model.ChangeFeedInfo{ - SinkURI: "blackhole://", - Config: config.GetDefaultReplicaConfig(), - }, - status: &model.ChangeFeedStatus{ - ResolvedTs: 120, - CheckpointTs: 100, - }, - }, - }, - stoppedFeeds: map[model.ChangeFeedID]*model.ChangeFeedStatus{ - "test-2": { - ResolvedTs: 120, - CheckpointTs: 100, - }, - }, - captures: map[model.CaptureID]*model.CaptureInfo{ - "capture-1": { - ID: "capture-1", - AdvertiseAddr: "127.0.0.1:8301", - }, - }, - } - var buf bytes.Buffer - owner.writeDebugInfo(&buf) - c.Assert(buf.String(), check.Matches, `[\s\S]*active changefeeds[\s\S]*stopped changefeeds[\s\S]*captures[\s\S]*`) -} diff --git a/cdc/processor.go b/cdc/processor.go deleted file mode 100644 index 14543047eb2..00000000000 --- a/cdc/processor.go +++ /dev/null @@ -1,1291 +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 cdc - -import ( - "context" - "fmt" - "io" - "strconv" - "sync" - "sync/atomic" - "time" - - "github.com/cenkalti/backoff" - "github.com/google/uuid" - "github.com/pingcap/errors" - "github.com/pingcap/failpoint" - "github.com/pingcap/log" - tidbkv "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/store/tikv/oracle" - "github.com/pingcap/tiflow/cdc/entry" - "github.com/pingcap/tiflow/cdc/kv" - "github.com/pingcap/tiflow/cdc/model" - "github.com/pingcap/tiflow/cdc/puller" - psorter "github.com/pingcap/tiflow/cdc/puller/sorter" - "github.com/pingcap/tiflow/cdc/sink" - cerror "github.com/pingcap/tiflow/pkg/errors" - "github.com/pingcap/tiflow/pkg/filter" - "github.com/pingcap/tiflow/pkg/notify" - "github.com/pingcap/tiflow/pkg/regionspan" - "github.com/pingcap/tiflow/pkg/retry" - "github.com/pingcap/tiflow/pkg/util" - pd "github.com/tikv/pd/client" - "go.etcd.io/etcd/clientv3" - "go.etcd.io/etcd/clientv3/concurrency" - "go.etcd.io/etcd/mvcc" - "go.uber.org/zap" - "golang.org/x/sync/errgroup" -) - -const ( - defaultSyncResolvedBatch = 1024 - - schemaStorageGCLag = time.Minute * 20 - - maxTries = 3 -) - -type oldProcessor struct { - id string - captureInfo model.CaptureInfo - changefeedID string - changefeed model.ChangeFeedInfo - stopped int32 - - pdCli pd.Client - etcdCli kv.CDCEtcdClient - grpcPool kv.GrpcPool - session *concurrency.Session - - sinkManager *sink.Manager - - globalResolvedTs uint64 - localResolvedTs uint64 - checkpointTs uint64 - globalCheckpointTs uint64 - appliedLocalCheckpointTs uint64 - flushCheckpointInterval time.Duration - - ddlPuller puller.Puller - ddlPullerCancel context.CancelFunc - schemaStorage entry.SchemaStorage - - mounter entry.Mounter - - stateMu sync.Mutex - status *model.TaskStatus - position *model.TaskPosition - tables map[int64]*tableInfo - markTableIDs map[int64]struct{} - statusModRevision int64 - - globalResolvedTsNotifier *notify.Notifier - localResolvedNotifier *notify.Notifier - localResolvedReceiver *notify.Receiver - localCheckpointTsNotifier *notify.Notifier - localCheckpointTsReceiver *notify.Receiver - - wg *errgroup.Group - errCh chan<- error - opDoneCh chan int64 -} - -type tableInfo struct { - id int64 - name string // quoted schema and table, used in metircs only - resolvedTs uint64 - checkpointTs uint64 - - markTableID int64 - mResolvedTs uint64 - mCheckpointTs uint64 - workload model.WorkloadInfo - cancel context.CancelFunc -} - -func (t *tableInfo) loadResolvedTs() uint64 { - tableRts := atomic.LoadUint64(&t.resolvedTs) - if t.markTableID != 0 { - mTableRts := atomic.LoadUint64(&t.mResolvedTs) - if mTableRts < tableRts { - return mTableRts - } - } - return tableRts -} - -func (t *tableInfo) loadCheckpointTs() uint64 { - tableCkpt := atomic.LoadUint64(&t.checkpointTs) - if t.markTableID != 0 { - mTableCkpt := atomic.LoadUint64(&t.mCheckpointTs) - if mTableCkpt < tableCkpt { - return mTableCkpt - } - } - return tableCkpt -} - -// newProcessor creates and returns a processor for the specified change feed -func newProcessor( - ctx context.Context, - pdCli pd.Client, - grpcPool kv.GrpcPool, - session *concurrency.Session, - changefeed model.ChangeFeedInfo, - sinkManager *sink.Manager, - changefeedID string, - captureInfo model.CaptureInfo, - checkpointTs uint64, - errCh chan error, - flushCheckpointInterval time.Duration, -) (*oldProcessor, error) { - etcdCli := session.Client() - cdcEtcdCli := kv.NewCDCEtcdClient(ctx, etcdCli) - - log.Info("start processor with startts", - zap.Uint64("startts", checkpointTs), util.ZapFieldChangefeed(ctx)) - kvStorage, err := util.KVStorageFromCtx(ctx) - if err != nil { - return nil, errors.Trace(err) - } - ddlspans := []regionspan.Span{regionspan.GetDDLSpan(), regionspan.GetAddIndexDDLSpan()} - ddlPuller := puller.NewPuller(ctx, pdCli, grpcPool, kvStorage, checkpointTs, ddlspans, false) - filter, err := filter.NewFilter(changefeed.Config) - if err != nil { - return nil, errors.Trace(err) - } - schemaStorage, err := createSchemaStorage(kvStorage, checkpointTs, filter, changefeed.Config.ForceReplicate) - if err != nil { - return nil, errors.Trace(err) - } - - localResolvedNotifier := new(notify.Notifier) - localCheckpointTsNotifier := new(notify.Notifier) - globalResolvedTsNotifier := new(notify.Notifier) - localResolvedReceiver, err := localResolvedNotifier.NewReceiver(50 * time.Millisecond) - if err != nil { - return nil, err - } - localCheckpointTsReceiver, err := localCheckpointTsNotifier.NewReceiver(50 * time.Millisecond) - if err != nil { - return nil, err - } - - p := &oldProcessor{ - id: uuid.New().String(), - captureInfo: captureInfo, - changefeedID: changefeedID, - changefeed: changefeed, - pdCli: pdCli, - grpcPool: grpcPool, - etcdCli: cdcEtcdCli, - session: session, - sinkManager: sinkManager, - ddlPuller: ddlPuller, - mounter: entry.NewMounter(schemaStorage, changefeed.Config.Mounter.WorkerNum, changefeed.Config.EnableOldValue), - schemaStorage: schemaStorage, - errCh: errCh, - - flushCheckpointInterval: flushCheckpointInterval, - - position: &model.TaskPosition{CheckPointTs: checkpointTs}, - - globalResolvedTsNotifier: globalResolvedTsNotifier, - localResolvedNotifier: localResolvedNotifier, - localResolvedReceiver: localResolvedReceiver, - - checkpointTs: checkpointTs, - localCheckpointTsNotifier: localCheckpointTsNotifier, - localCheckpointTsReceiver: localCheckpointTsReceiver, - - tables: make(map[int64]*tableInfo), - markTableIDs: make(map[int64]struct{}), - - opDoneCh: make(chan int64, 256), - } - modRevision, status, err := p.etcdCli.GetTaskStatus(ctx, p.changefeedID, p.captureInfo.ID) - if err != nil { - return nil, errors.Trace(err) - } - p.status = status - p.statusModRevision = modRevision - - info, _, err := p.etcdCli.GetChangeFeedStatus(ctx, p.changefeedID) - if err != nil && cerror.ErrChangeFeedNotExists.NotEqual(err) { - return nil, errors.Trace(err) - } - - if err == nil { - p.globalCheckpointTs = info.CheckpointTs - } - - for tableID, replicaInfo := range p.status.Tables { - p.addTable(ctx, tableID, replicaInfo) - } - return p, nil -} - -func (p *oldProcessor) Run(ctx context.Context) { - wg, cctx := errgroup.WithContext(ctx) - p.wg = wg - ddlPullerCtx, ddlPullerCancel := - context.WithCancel(util.PutTableInfoInCtx(cctx, 0, "ticdc-processor-ddl")) - p.ddlPullerCancel = ddlPullerCancel - - wg.Go(func() error { - return p.positionWorker(cctx) - }) - - wg.Go(func() error { - return p.globalStatusWorker(cctx) - }) - - wg.Go(func() error { - return p.ddlPuller.Run(ddlPullerCtx) - }) - - wg.Go(func() error { - return p.ddlPullWorker(cctx) - }) - - wg.Go(func() error { - return p.mounter.Run(cctx) - }) - - wg.Go(func() error { - return p.workloadWorker(cctx) - }) - - go func() { - if err := wg.Wait(); err != nil { - p.sendError(err) - } - }() -} - -// wait blocks until all routines in processor are returned -func (p *oldProcessor) wait() { - err := p.wg.Wait() - if err != nil && errors.Cause(err) != context.Canceled { - log.Error("processor wait error", - zap.String("capture-id", p.captureInfo.ID), - zap.String("capture", p.captureInfo.AdvertiseAddr), - zap.String("changefeed", p.changefeedID), - zap.Error(err), - ) - } -} - -func (p *oldProcessor) writeDebugInfo(w io.Writer) { - fmt.Fprintf(w, "changefeedID:\n\t%s\ninfo:\n\t%s\nstatus:\n\t%+v\nposition:\n\t%s\n", - p.changefeedID, p.changefeed.String(), p.status, p.position.String()) - - fmt.Fprintf(w, "tables:\n") - p.stateMu.Lock() - for _, table := range p.tables { - fmt.Fprintf(w, "\ttable id: %d, resolveTS: %d\n", table.id, table.loadResolvedTs()) - } - p.stateMu.Unlock() -} - -// localResolvedWorker do the flowing works. -// 1, update resolve ts by scanning all table's resolve ts. -// 2, update checkpoint ts by consuming entry from p.executedTxns. -// 3, sync TaskStatus between in memory and storage. -// 4, check admin command in TaskStatus and apply corresponding command -func (p *oldProcessor) positionWorker(ctx context.Context) error { - lastFlushTime := time.Now() - retryFlushTaskStatusAndPosition := func() error { - t0Update := time.Now() - err := retry.Do(ctx, func() error { - inErr := p.flushTaskStatusAndPosition(ctx) - if inErr != nil { - if errors.Cause(inErr) != context.Canceled { - logError := log.Error - errField := zap.Error(inErr) - if cerror.ErrAdminStopProcessor.Equal(inErr) { - logError = log.Warn - errField = zap.String("error", inErr.Error()) - } - logError("update info failed", util.ZapFieldChangefeed(ctx), errField) - } - if p.isStopped() || cerror.ErrAdminStopProcessor.Equal(inErr) { - return cerror.ErrAdminStopProcessor.FastGenByArgs() - } - } - return inErr - }, retry.WithBackoffBaseDelay(500), retry.WithMaxTries(maxTries), retry.WithIsRetryableErr(isRetryable)) - updateInfoDuration. - WithLabelValues(p.captureInfo.AdvertiseAddr). - Observe(time.Since(t0Update).Seconds()) - if err != nil { - return errors.Annotate(err, "failed to update info") - } - return nil - } - - defer func() { - p.localResolvedReceiver.Stop() - p.localCheckpointTsReceiver.Stop() - - if !p.isStopped() { - err := retryFlushTaskStatusAndPosition() - if err != nil && errors.Cause(err) != context.Canceled { - log.Warn("failed to update info before exit", util.ZapFieldChangefeed(ctx), zap.Error(err)) - } - } - - log.Info("Local resolved worker exited", util.ZapFieldChangefeed(ctx)) - }() - - resolvedTsGauge := resolvedTsGauge.WithLabelValues(p.changefeedID, p.captureInfo.AdvertiseAddr) - metricResolvedTsLagGauge := resolvedTsLagGauge.WithLabelValues(p.changefeedID, p.captureInfo.AdvertiseAddr) - checkpointTsGauge := checkpointTsGauge.WithLabelValues(p.changefeedID, p.captureInfo.AdvertiseAddr) - metricCheckpointTsLagGauge := checkpointTsLagGauge.WithLabelValues(p.changefeedID, p.captureInfo.AdvertiseAddr) - for { - select { - case <-ctx.Done(): - return ctx.Err() - case <-p.localResolvedReceiver.C: - minResolvedTs := p.ddlPuller.GetResolvedTs() - p.stateMu.Lock() - for _, table := range p.tables { - ts := table.loadResolvedTs() - - if ts < minResolvedTs { - minResolvedTs = ts - } - } - p.stateMu.Unlock() - atomic.StoreUint64(&p.localResolvedTs, minResolvedTs) - - phyTs := oracle.ExtractPhysical(minResolvedTs) - // It is more accurate to get tso from PD, but in most cases we have - // deployed NTP service, a little bias is acceptable here. - metricResolvedTsLagGauge.Set(float64(oracle.GetPhysical(time.Now())-phyTs) / 1e3) - resolvedTsGauge.Set(float64(phyTs)) - - if p.position.ResolvedTs < minResolvedTs { - p.position.ResolvedTs = minResolvedTs - if err := retryFlushTaskStatusAndPosition(); err != nil { - return errors.Trace(err) - } - } - case <-p.localCheckpointTsReceiver.C: - checkpointTs := atomic.LoadUint64(&p.globalResolvedTs) - p.stateMu.Lock() - for _, table := range p.tables { - ts := table.loadCheckpointTs() - if ts < checkpointTs { - checkpointTs = ts - } - } - p.stateMu.Unlock() - if checkpointTs == 0 { - log.Debug("0 is not a valid checkpointTs", util.ZapFieldChangefeed(ctx)) - continue - } - atomic.StoreUint64(&p.checkpointTs, checkpointTs) - phyTs := oracle.ExtractPhysical(checkpointTs) - // It is more accurate to get tso from PD, but in most cases we have - // deployed NTP service, a little bias is acceptable here. - metricCheckpointTsLagGauge.Set(float64(oracle.GetPhysical(time.Now())-phyTs) / 1e3) - - if time.Since(lastFlushTime) < p.flushCheckpointInterval { - continue - } - - p.position.CheckPointTs = checkpointTs - checkpointTsGauge.Set(float64(phyTs)) - if err := retryFlushTaskStatusAndPosition(); err != nil { - return errors.Trace(err) - } - atomic.StoreUint64(&p.appliedLocalCheckpointTs, checkpointTs) - lastFlushTime = time.Now() - } - } -} - -func isRetryable(err error) bool { - return cerror.IsRetryableError(err) && cerror.ErrAdminStopProcessor.NotEqual(err) -} - -func (p *oldProcessor) ddlPullWorker(ctx context.Context) error { - ddlRawKVCh := puller.SortOutput(ctx, p.ddlPuller.Output()) - var ddlRawKV *model.RawKVEntry - for { - select { - case <-ctx.Done(): - return errors.Trace(ctx.Err()) - case ddlRawKV = <-ddlRawKVCh: - } - if ddlRawKV == nil { - continue - } - failpoint.Inject("processorDDLResolved", func() {}) - if ddlRawKV.OpType == model.OpTypeResolved { - p.schemaStorage.AdvanceResolvedTs(ddlRawKV.CRTs) - p.localResolvedNotifier.Notify() - } - job, err := entry.UnmarshalDDL(ddlRawKV) - if err != nil { - return errors.Trace(err) - } - if job == nil { - continue - } - if err := p.schemaStorage.HandleDDLJob(job); err != nil { - return errors.Trace(err) - } - } -} - -func (p *oldProcessor) workloadWorker(ctx context.Context) error { - t := time.NewTicker(10 * time.Second) - err := p.etcdCli.PutTaskWorkload(ctx, p.changefeedID, p.captureInfo.ID, nil) - if err != nil { - return errors.Trace(err) - } - for { - select { - case <-ctx.Done(): - return errors.Trace(ctx.Err()) - case <-t.C: - } - if p.isStopped() { - continue - } - p.stateMu.Lock() - workload := make(model.TaskWorkload, len(p.tables)) - for _, table := range p.tables { - workload[table.id] = table.workload - } - p.stateMu.Unlock() - err := p.etcdCli.PutTaskWorkload(ctx, p.changefeedID, p.captureInfo.ID, &workload) - if err != nil { - return errors.Trace(err) - } - } -} - -func (p *oldProcessor) flushTaskPosition(ctx context.Context) error { - failpoint.Inject("ProcessorUpdatePositionDelaying", func() { - time.Sleep(1 * time.Second) - }) - if p.isStopped() { - return cerror.ErrAdminStopProcessor.GenWithStackByArgs() - } - // p.position.Count = p.sink.Count() - updated, err := p.etcdCli.PutTaskPositionOnChange(ctx, p.changefeedID, p.captureInfo.ID, p.position) - if err != nil { - if errors.Cause(err) != context.Canceled { - log.Error("failed to flush task position", util.ZapFieldChangefeed(ctx), zap.Error(err)) - return errors.Trace(err) - } - } - if updated { - log.Debug("flushed task position", util.ZapFieldChangefeed(ctx), zap.Stringer("position", p.position)) - } - return nil -} - -// First try to synchronize task status from etcd. -// If local cached task status is outdated (caused by new table scheduling), -// update it to latest value, and force update task position, since add new -// tables may cause checkpoint ts fallback in processor. -func (p *oldProcessor) flushTaskStatusAndPosition(ctx context.Context) error { - if p.isStopped() { - return cerror.ErrAdminStopProcessor.GenWithStackByArgs() - } - var tablesToRemove []model.TableID - newTaskStatus, newModRevision, err := p.etcdCli.AtomicPutTaskStatus(ctx, p.changefeedID, p.captureInfo.ID, - func(modRevision int64, taskStatus *model.TaskStatus) (bool, error) { - // if the task status is not changed and not operation to handle - // we need not to change the task status - if p.statusModRevision == modRevision && !taskStatus.SomeOperationsUnapplied() { - return false, nil - } - // task will be stopped in capture task handler, do nothing - if taskStatus.AdminJobType.IsStopState() { - return false, backoff.Permanent(cerror.ErrAdminStopProcessor.GenWithStackByArgs()) - } - toRemove, err := p.handleTables(ctx, taskStatus) - tablesToRemove = append(tablesToRemove, toRemove...) - if err != nil { - return false, backoff.Permanent(errors.Trace(err)) - } - // processor reads latest task status from etcd, analyzes operation - // field and processes table add or delete. If operation is unapplied - // but stays unchanged after processor handling tables, it means no - // status is changed and we don't need to flush task status neigher. - if !taskStatus.Dirty { - return false, nil - } - err = p.flushTaskPosition(ctx) - return true, err - }) - if err != nil { - // not need to check error - //nolint:errcheck - p.flushTaskPosition(ctx) - return errors.Trace(err) - } - for _, tableID := range tablesToRemove { - p.removeTable(tableID) - } - // newModRevision == 0 means status is not updated - if newModRevision > 0 { - p.statusModRevision = newModRevision - p.status = newTaskStatus - } - syncTableNumGauge. - WithLabelValues(p.changefeedID, p.captureInfo.AdvertiseAddr). - Set(float64(len(p.status.Tables))) - - return p.flushTaskPosition(ctx) -} - -func (p *oldProcessor) removeTable(tableID int64) { - p.stateMu.Lock() - defer p.stateMu.Unlock() - - log.Debug("remove table", zap.String("changefeed", p.changefeedID), zap.Int64("id", tableID)) - - table, ok := p.tables[tableID] - if !ok { - log.Warn("table not found", zap.String("changefeed", p.changefeedID), zap.Int64("tableID", tableID)) - return - } - - table.cancel() - delete(p.tables, tableID) - if table.markTableID != 0 { - delete(p.markTableIDs, table.markTableID) - } - tableResolvedTsGauge.DeleteLabelValues(p.changefeedID, p.captureInfo.AdvertiseAddr) - syncTableNumGauge.WithLabelValues(p.changefeedID, p.captureInfo.AdvertiseAddr).Dec() -} - -// handleTables handles table scheduler on this processor, add or remove table puller -func (p *oldProcessor) handleTables(ctx context.Context, status *model.TaskStatus) (tablesToRemove []model.TableID, err error) { - for tableID, opt := range status.Operation { - if opt.TableProcessed() { - continue - } - if opt.Delete { - if opt.BoundaryTs <= p.position.CheckPointTs { - if opt.BoundaryTs != p.position.CheckPointTs { - log.Warn("the replication progresses beyond the BoundaryTs and duplicate data may be received by downstream", - zap.Uint64("local resolved TS", p.position.ResolvedTs), zap.Any("opt", opt)) - } - table, exist := p.tables[tableID] - if !exist { - log.Warn("table which will be deleted is not found", - util.ZapFieldChangefeed(ctx), zap.Int64("tableID", tableID)) - opt.Done = true - opt.Status = model.OperFinished - status.Dirty = true - continue - } - table.cancel() - checkpointTs := table.loadCheckpointTs() - log.Debug("stop table", zap.Int64("tableID", tableID), - util.ZapFieldChangefeed(ctx), - zap.Any("opt", opt), - zap.Uint64("checkpointTs", checkpointTs)) - opt.BoundaryTs = checkpointTs - tablesToRemove = append(tablesToRemove, tableID) - opt.Done = true - opt.Status = model.OperFinished - status.Dirty = true - tableResolvedTsGauge.DeleteLabelValues(p.changefeedID, p.captureInfo.AdvertiseAddr) - } - } else { - replicaInfo, exist := status.Tables[tableID] - if !exist { - return tablesToRemove, cerror.ErrProcessorTableNotFound.GenWithStack("replicaInfo of table(%d)", tableID) - } - if p.changefeed.Config.Cyclic.IsEnabled() && replicaInfo.MarkTableID == 0 { - return tablesToRemove, cerror.ErrProcessorTableNotFound.GenWithStack("normal table(%d) and mark table not match ", tableID) - } - p.addTable(ctx, tableID, replicaInfo) - opt.Status = model.OperProcessed - status.Dirty = true - } - } - - for { - select { - case <-ctx.Done(): - return nil, ctx.Err() - case tableID := <-p.opDoneCh: - log.Debug("Operation done signal received", - util.ZapFieldChangefeed(ctx), - zap.Int64("tableID", tableID), - zap.Reflect("operation", status.Operation[tableID])) - if status.Operation[tableID] == nil { - log.Debug("TableID does not exist, probably a mark table, ignore", - util.ZapFieldChangefeed(ctx), zap.Int64("tableID", tableID)) - continue - } - status.Operation[tableID].Done = true - status.Operation[tableID].Status = model.OperFinished - status.Dirty = true - default: - goto done - } - } -done: - if !status.SomeOperationsUnapplied() { - status.Operation = nil - // status.Dirty must be true when status changes from `unapplied` to `applied`, - // setting status.Dirty = true is not **must** here. - status.Dirty = true - } - return tablesToRemove, nil -} - -// globalStatusWorker read global resolve ts from changefeed level info and forward `tableInputChans` regularly. -func (p *oldProcessor) globalStatusWorker(ctx context.Context) error { - log.Info("Global status worker started", util.ZapFieldChangefeed(ctx)) - - var ( - changefeedStatus *model.ChangeFeedStatus - statusRev int64 - lastCheckPointTs uint64 - lastResolvedTs uint64 - watchKey = kv.GetEtcdKeyJob(p.changefeedID) - ) - - updateStatus := func(changefeedStatus *model.ChangeFeedStatus) { - atomic.StoreUint64(&p.globalCheckpointTs, changefeedStatus.CheckpointTs) - if lastResolvedTs == changefeedStatus.ResolvedTs && - lastCheckPointTs == changefeedStatus.CheckpointTs { - return - } - if lastCheckPointTs < changefeedStatus.CheckpointTs { - // Delay GC to accommodate pullers starting from a startTs that's too small - // TODO fix startTs problem and remove GC delay, or use other mechanism that prevents the problem deterministically - gcTime := oracle.GetTimeFromTS(changefeedStatus.CheckpointTs).Add(-schemaStorageGCLag) - gcTs := oracle.ComposeTS(gcTime.Unix(), 0) - p.schemaStorage.DoGC(gcTs) - lastCheckPointTs = changefeedStatus.CheckpointTs - } - if lastResolvedTs < changefeedStatus.ResolvedTs { - lastResolvedTs = changefeedStatus.ResolvedTs - atomic.StoreUint64(&p.globalResolvedTs, lastResolvedTs) - log.Debug("Update globalResolvedTs", - zap.Uint64("globalResolvedTs", lastResolvedTs), util.ZapFieldChangefeed(ctx)) - p.globalResolvedTsNotifier.Notify() - } - } - - retryCfg := backoff.WithMaxRetries( - backoff.WithContext( - backoff.NewExponentialBackOff(), ctx), - 5, - ) - for { - select { - case <-ctx.Done(): - log.Info("Global resolved worker exited", util.ZapFieldChangefeed(ctx)) - return ctx.Err() - default: - } - - err := backoff.Retry(func() error { - var err error - changefeedStatus, statusRev, err = p.etcdCli.GetChangeFeedStatus(ctx, p.changefeedID) - if err != nil { - if errors.Cause(err) == context.Canceled { - return backoff.Permanent(err) - } - log.Error("Global resolved worker: read global resolved ts failed", - util.ZapFieldChangefeed(ctx), zap.Error(err)) - } - return err - }, retryCfg) - if err != nil { - return errors.Trace(err) - } - - updateStatus(changefeedStatus) - - ch := p.etcdCli.Client.Watch(ctx, watchKey, clientv3.WithRev(statusRev+1), clientv3.WithFilterDelete()) - for resp := range ch { - if resp.Err() == mvcc.ErrCompacted { - break - } - if resp.Err() != nil { - return cerror.WrapError(cerror.ErrProcessorEtcdWatch, err) - } - for _, ev := range resp.Events { - var status model.ChangeFeedStatus - if err := status.Unmarshal(ev.Kv.Value); err != nil { - return err - } - updateStatus(&status) - } - } - } -} - -func createSchemaStorage( - kvStorage tidbkv.Storage, - checkpointTs uint64, - filter *filter.Filter, - forceReplicate bool, -) (entry.SchemaStorage, error) { - meta, err := kv.GetSnapshotMeta(kvStorage, checkpointTs) - if err != nil { - return nil, errors.Trace(err) - } - return entry.NewSchemaStorage(meta, checkpointTs, filter, forceReplicate) -} - -func (p *oldProcessor) addTable(ctx context.Context, tableID int64, replicaInfo *model.TableReplicaInfo) { - p.stateMu.Lock() - defer p.stateMu.Unlock() - - var tableName string - - err := retry.Do(ctx, func() error { - if name, ok := p.schemaStorage.GetLastSnapshot().GetTableNameByID(tableID); ok { - tableName = name.QuoteString() - return nil - } - return errors.Errorf("failed to get table name, fallback to use table id: %d", tableID) - }, retry.WithBackoffBaseDelay(5), retry.WithMaxTries(maxTries), retry.WithIsRetryableErr(cerror.IsRetryableError)) - if err != nil { - log.Warn("get table name for metric", util.ZapFieldChangefeed(ctx), zap.String("error", err.Error())) - tableName = strconv.Itoa(int(tableID)) - } - - if _, ok := p.tables[tableID]; ok { - log.Warn("Ignore existing table", util.ZapFieldChangefeed(ctx), zap.Int64("ID", tableID)) - return - } - - globalCheckpointTs := atomic.LoadUint64(&p.globalCheckpointTs) - - if replicaInfo.StartTs < globalCheckpointTs { - // use Warn instead of Panic in case that p.globalCheckpointTs has not been initialized. - // The cdc_state_checker will catch a real inconsistency in integration tests. - log.Warn("addTable: startTs < checkpoint", - util.ZapFieldChangefeed(ctx), - zap.Int64("tableID", tableID), - zap.Uint64("checkpoint", globalCheckpointTs), - zap.Uint64("startTs", replicaInfo.StartTs)) - } - - globalResolvedTs := atomic.LoadUint64(&p.globalResolvedTs) - log.Debug("Add table", zap.Int64("tableID", tableID), - util.ZapFieldChangefeed(ctx), - zap.String("name", tableName), - zap.Any("replicaInfo", replicaInfo), - zap.Uint64("globalResolvedTs", globalResolvedTs)) - - ctx = util.PutTableInfoInCtx(ctx, tableID, tableName) - ctx, cancel := context.WithCancel(ctx) - table := &tableInfo{ - id: tableID, - name: tableName, - resolvedTs: replicaInfo.StartTs, - } - // TODO(leoppro) calculate the workload of this table - // We temporarily set the value to constant 1 - table.workload = model.WorkloadInfo{Workload: 1} - - startPuller := func(tableID model.TableID, pResolvedTs *uint64, pCheckpointTs *uint64) sink.Sink { - // start table puller - span := regionspan.GetTableSpan(tableID) - kvStorage, err := util.KVStorageFromCtx(ctx) - if err != nil { - p.sendError(err) - return nil - } - // NOTICE: always pull the old value internally - // See also: TODO(hi-rustin): add issue link here. - plr := puller.NewPuller(ctx, p.pdCli, p.grpcPool, kvStorage, - replicaInfo.StartTs, []regionspan.Span{span}, true) - go func() { - err := plr.Run(ctx) - if errors.Cause(err) != context.Canceled { - p.sendError(err) - } - }() - - var sorter puller.EventSorter - switch p.changefeed.Engine { - case model.SortInMemory: - sorter = puller.NewEntrySorter() - case model.SortUnified, model.SortInFile /* `file` becomes an alias of `unified` for backward compatibility */ : - if p.changefeed.Engine == model.SortInFile { - log.Warn("File sorter is obsolete. Please revise your changefeed settings and use unified sorter", - util.ZapFieldChangefeed(ctx)) - } - err := psorter.UnifiedSorterCheckDir(p.changefeed.SortDir) - if err != nil { - p.sendError(errors.Trace(err)) - return nil - } - sorter, err = psorter.NewUnifiedSorter(p.changefeed.SortDir, p.changefeedID, tableName, tableID, util.CaptureAddrFromCtx(ctx)) - if err != nil { - p.sendError(errors.Trace(err)) - return nil - } - default: - p.sendError(cerror.ErrUnknownSortEngine.GenWithStackByArgs(p.changefeed.Engine)) - return nil - } - failpoint.Inject("ProcessorAddTableError", func() { - p.sendError(errors.New("processor add table injected error")) - failpoint.Return(nil) - }) - go func() { - err := sorter.Run(ctx) - if errors.Cause(err) != context.Canceled { - p.sendError(err) - } - }() - - go func() { - p.pullerConsume(ctx, plr, sorter) - }() - - tableSink := p.sinkManager.CreateTableSink(tableID, replicaInfo.StartTs) - go func() { - p.sorterConsume(ctx, tableID, sorter, pResolvedTs, pCheckpointTs, replicaInfo, tableSink) - }() - return tableSink - } - var tableSink, mTableSink sink.Sink - if p.changefeed.Config.Cyclic.IsEnabled() && replicaInfo.MarkTableID != 0 { - mTableID := replicaInfo.MarkTableID - // we should to make sure a mark table is only listened once. - if _, exist := p.markTableIDs[mTableID]; !exist { - p.markTableIDs[mTableID] = struct{}{} - table.markTableID = mTableID - table.mResolvedTs = replicaInfo.StartTs - - mTableSink = startPuller(mTableID, &table.mResolvedTs, &table.mCheckpointTs) - } - } - - p.tables[tableID] = table - if p.position.CheckPointTs > replicaInfo.StartTs { - p.position.CheckPointTs = replicaInfo.StartTs - } - if p.position.ResolvedTs > replicaInfo.StartTs { - p.position.ResolvedTs = replicaInfo.StartTs - } - - atomic.StoreUint64(&p.localResolvedTs, p.position.ResolvedTs) - tableSink = startPuller(tableID, &table.resolvedTs, &table.checkpointTs) - table.cancel = func() { - cancel() - if tableSink != nil { - tableSink.Close(ctx) - } - if mTableSink != nil { - mTableSink.Close(ctx) - } - } - syncTableNumGauge.WithLabelValues(p.changefeedID, p.captureInfo.AdvertiseAddr).Inc() -} - -const maxLagWithCheckpointTs = (30 * 1000) << 18 // 30s - -// sorterConsume receives sorted PolymorphicEvent from sorter of each table and -// sends to processor's output chan -func (p *oldProcessor) sorterConsume( - ctx context.Context, - tableID int64, - sorter puller.EventSorter, - pResolvedTs *uint64, - pCheckpointTs *uint64, - replicaInfo *model.TableReplicaInfo, - sink sink.Sink, -) { - var lastResolvedTs, lastCheckPointTs uint64 - opDone := false - resolvedGauge := tableResolvedTsGauge.WithLabelValues(p.changefeedID, p.captureInfo.AdvertiseAddr) - checkDoneTicker := time.NewTicker(1 * time.Second) - checkDone := func() { - localResolvedTs := atomic.LoadUint64(&p.localResolvedTs) - globalResolvedTs := atomic.LoadUint64(&p.globalResolvedTs) - tableCheckPointTs := atomic.LoadUint64(pCheckpointTs) - localCheckpoint := atomic.LoadUint64(&p.appliedLocalCheckpointTs) - - if !opDone && lastResolvedTs >= localResolvedTs && localResolvedTs >= globalResolvedTs && - tableCheckPointTs >= localCheckpoint { - - log.Debug("localResolvedTs >= globalResolvedTs, sending operation done signal", - zap.Uint64("localResolvedTs", localResolvedTs), zap.Uint64("globalResolvedTs", globalResolvedTs), - zap.Int64("tableID", tableID), util.ZapFieldChangefeed(ctx)) - - opDone = true - checkDoneTicker.Stop() - select { - case <-ctx.Done(): - if errors.Cause(ctx.Err()) != context.Canceled { - p.sendError(ctx.Err()) - } - return - case p.opDoneCh <- tableID: - } - } - if !opDone { - log.Debug("addTable not done", - util.ZapFieldChangefeed(ctx), - zap.Uint64("tableResolvedTs", lastResolvedTs), - zap.Uint64("localResolvedTs", localResolvedTs), - zap.Uint64("globalResolvedTs", globalResolvedTs), - zap.Uint64("tableCheckpointTs", tableCheckPointTs), - zap.Uint64("localCheckpointTs", localCheckpoint), - zap.Int64("tableID", tableID)) - } - } - - events := make([]*model.PolymorphicEvent, 0, defaultSyncResolvedBatch) - rows := make([]*model.RowChangedEvent, 0, defaultSyncResolvedBatch) - - flushRowChangedEvents := func() error { - for _, ev := range events { - err := ev.WaitPrepare(ctx) - if err != nil { - return errors.Trace(err) - } - if ev.Row == nil { - continue - } - rows = append(rows, ev.Row) - } - failpoint.Inject("ProcessorSyncResolvedPreEmit", func() { - log.Info("Prepare to panic for ProcessorSyncResolvedPreEmit") - time.Sleep(10 * time.Second) - panic("ProcessorSyncResolvedPreEmit") - }) - err := sink.EmitRowChangedEvents(ctx, rows...) - if err != nil { - return errors.Trace(err) - } - events = events[:0] - rows = rows[:0] - return nil - } - - processRowChangedEvent := func(row *model.PolymorphicEvent) error { - events = append(events, row) - - if len(events) >= defaultSyncResolvedBatch { - err := flushRowChangedEvents() - if err != nil { - return errors.Trace(err) - } - } - return nil - } - - globalResolvedTsReceiver, err := p.globalResolvedTsNotifier.NewReceiver(500 * time.Millisecond) - if err != nil { - if errors.Cause(err) != context.Canceled { - p.errCh <- errors.Trace(err) - } - return - } - defer globalResolvedTsReceiver.Stop() - - sendResolvedTs2Sink := func() error { - localResolvedTs := atomic.LoadUint64(&p.localResolvedTs) - globalResolvedTs := atomic.LoadUint64(&p.globalResolvedTs) - var minTs uint64 - if localResolvedTs < globalResolvedTs { - minTs = localResolvedTs - log.Warn("the local resolved ts is less than the global resolved ts", - zap.Uint64("localResolvedTs", localResolvedTs), zap.Uint64("globalResolvedTs", globalResolvedTs)) - } else { - minTs = globalResolvedTs - } - if minTs == 0 { - return nil - } - - checkpointTs, err := sink.FlushRowChangedEvents(ctx, minTs) - if err != nil { - if errors.Cause(err) != context.Canceled { - p.sendError(errors.Trace(err)) - } - return err - } - lastCheckPointTs = checkpointTs - - if checkpointTs < replicaInfo.StartTs { - checkpointTs = replicaInfo.StartTs - } - - if checkpointTs != 0 { - atomic.StoreUint64(pCheckpointTs, checkpointTs) - p.localCheckpointTsNotifier.Notify() - } - return nil - } - for { - select { - case <-ctx.Done(): - if errors.Cause(ctx.Err()) != context.Canceled { - p.sendError(ctx.Err()) - } - return - case pEvent := <-sorter.Output(): - if pEvent == nil { - continue - } - - for lastResolvedTs > maxLagWithCheckpointTs+lastCheckPointTs { - log.Debug("the lag between local checkpoint Ts and local resolved Ts is too lang", - zap.Uint64("resolvedTs", lastResolvedTs), zap.Uint64("lastCheckPointTs", lastCheckPointTs), - zap.Int64("tableID", tableID), util.ZapFieldChangefeed(ctx)) - select { - case <-ctx.Done(): - if ctx.Err() != context.Canceled { - p.sendError(errors.Trace(ctx.Err())) - } - return - case <-globalResolvedTsReceiver.C: - if err := sendResolvedTs2Sink(); err != nil { - // error is already sent to processor, so we can just ignore it - return - } - case <-checkDoneTicker.C: - if !opDone { - checkDone() - } - } - } - - pEvent.SetUpFinishedChan() - select { - case <-ctx.Done(): - if errors.Cause(ctx.Err()) != context.Canceled { - p.sendError(ctx.Err()) - } - return - case p.mounter.Input() <- pEvent: - } - - if pEvent.RawKV != nil && pEvent.RawKV.OpType == model.OpTypeResolved { - if pEvent.CRTs == 0 { - continue - } - err := flushRowChangedEvents() - if err != nil { - if errors.Cause(err) != context.Canceled { - p.errCh <- errors.Trace(err) - } - return - } - atomic.StoreUint64(pResolvedTs, pEvent.CRTs) - lastResolvedTs = pEvent.CRTs - p.localResolvedNotifier.Notify() - resolvedGauge.Set(float64(oracle.ExtractPhysical(pEvent.CRTs))) - if !opDone { - checkDone() - } - continue - } - if pEvent.CRTs <= lastResolvedTs || pEvent.CRTs < replicaInfo.StartTs { - log.Panic("The CRTs of event is not expected, please report a bug", - util.ZapFieldChangefeed(ctx), - zap.String("model", "sorter"), - zap.Uint64("resolvedTs", lastResolvedTs), - zap.Int64("tableID", tableID), - zap.Any("replicaInfo", replicaInfo), - zap.Any("row", pEvent)) - } - failpoint.Inject("ProcessorSyncResolvedError", func() { - p.errCh <- errors.New("processor sync resolved injected error") - failpoint.Return() - }) - err := processRowChangedEvent(pEvent) - if err != nil { - if errors.Cause(err) != context.Canceled { - p.sendError(ctx.Err()) - } - return - } - case <-globalResolvedTsReceiver.C: - if err := sendResolvedTs2Sink(); err != nil { - // error is already sent to processor, so we can just ignore it - return - } - case <-checkDoneTicker.C: - if !opDone { - checkDone() - } - } - } -} - -// pullerConsume receives RawKVEntry from a given puller and sends to sorter -// for data sorting and mounter for data encode -func (p *oldProcessor) pullerConsume( - ctx context.Context, - plr puller.Puller, - sorter puller.EventSorter, -) { - for { - select { - case <-ctx.Done(): - if errors.Cause(ctx.Err()) != context.Canceled { - p.sendError(ctx.Err()) - } - return - case rawKV := <-plr.Output(): - if rawKV == nil { - continue - } - pEvent := model.NewPolymorphicEvent(rawKV) - sorter.AddEntry(ctx, pEvent) - } - } -} - -func (p *oldProcessor) stop(ctx context.Context) error { - log.Info("stop processor", zap.String("id", p.id), zap.String("capture", p.captureInfo.AdvertiseAddr), zap.String("changefeed", p.changefeedID)) - p.stateMu.Lock() - for _, tbl := range p.tables { - tbl.cancel() - tableResolvedTsGauge.DeleteLabelValues(p.changefeedID, p.captureInfo.AdvertiseAddr) - } - p.ddlPullerCancel() - // mark tables share the same context with its original table, don't need to cancel - p.stateMu.Unlock() - p.globalResolvedTsNotifier.Close() - p.localCheckpointTsNotifier.Close() - p.localResolvedNotifier.Close() - failpoint.Inject("processorStopDelay", nil) - atomic.StoreInt32(&p.stopped, 1) - syncTableNumGauge.WithLabelValues(p.changefeedID, p.captureInfo.AdvertiseAddr).Set(0) - if err := p.etcdCli.DeleteTaskPosition(ctx, p.changefeedID, p.captureInfo.ID); err != nil { - return err - } - if err := p.etcdCli.DeleteTaskStatus(ctx, p.changefeedID, p.captureInfo.ID); err != nil { - return err - } - if err := p.etcdCli.DeleteTaskWorkload(ctx, p.changefeedID, p.captureInfo.ID); err != nil { - return err - } - return p.sinkManager.Close(ctx) -} - -func (p *oldProcessor) isStopped() bool { - return atomic.LoadInt32(&p.stopped) == 1 -} - -var runProcessorImpl = runProcessor - -// runProcessor creates a new processor then starts it. -func runProcessor( - ctx context.Context, - pdCli pd.Client, - grpcPool kv.GrpcPool, - session *concurrency.Session, - info model.ChangeFeedInfo, - changefeedID string, - captureInfo model.CaptureInfo, - checkpointTs uint64, - flushCheckpointInterval time.Duration, -) (*oldProcessor, error) { - opts := make(map[string]string, len(info.Opts)+2) - for k, v := range info.Opts { - opts[k] = v - } - opts[sink.OptChangefeedID] = changefeedID - opts[sink.OptCaptureAddr] = captureInfo.AdvertiseAddr - ctx = util.PutChangefeedIDInCtx(ctx, changefeedID) - filter, err := filter.NewFilter(info.Config) - if err != nil { - return nil, errors.Trace(err) - } - ctx, cancel := context.WithCancel(ctx) - // processor only receives one error from the channel, all producers to this - // channel must use the non-blocking way to send error. - errCh := make(chan error, 1) - s, err := sink.NewSink(ctx, changefeedID, info.SinkURI, filter, info.Config, opts, errCh) - if err != nil { - cancel() - return nil, errors.Trace(err) - } - sinkManager := sink.NewManager(ctx, s, errCh, checkpointTs, captureInfo.AdvertiseAddr, changefeedID) - processor, err := newProcessor(ctx, pdCli, grpcPool, session, info, sinkManager, - changefeedID, captureInfo, checkpointTs, errCh, flushCheckpointInterval) - if err != nil { - cancel() - return nil, err - } - log.Info("start to run processor", zap.String("changefeed", changefeedID), zap.String("processor", processor.id)) - - processorErrorCounter.WithLabelValues(changefeedID, captureInfo.AdvertiseAddr).Add(0) - processor.Run(ctx) - - go func() { - err := <-errCh - cancel() - processor.wait() - cause := errors.Cause(err) - if cause != nil && cause != context.Canceled && cerror.ErrAdminStopProcessor.NotEqual(cause) { - processorErrorCounter.WithLabelValues(changefeedID, captureInfo.AdvertiseAddr).Inc() - log.Error("error on running processor", - util.ZapFieldCapture(ctx), - zap.String("changefeed", changefeedID), - zap.String("processor", processor.id), - zap.Error(err)) - // record error information in etcd - var code string - if terror, ok := err.(*errors.Error); ok { - code = string(terror.RFCCode()) - } else { - code = string(cerror.ErrProcessorUnknown.RFCCode()) - } - processor.position.Error = &model.RunningError{ - Addr: captureInfo.AdvertiseAddr, - Code: code, - Message: err.Error(), - } - timeoutCtx, timeoutCancel := context.WithTimeout(context.Background(), 5*time.Second) - _, err = processor.etcdCli.PutTaskPositionOnChange(timeoutCtx, processor.changefeedID, processor.captureInfo.ID, processor.position) - if err != nil { - log.Warn("upload processor error failed", util.ZapFieldChangefeed(ctx), zap.Error(err)) - } - timeoutCancel() - } else { - log.Info("processor exited", - util.ZapFieldCapture(ctx), - zap.String("changefeed", changefeedID), - zap.String("processor", processor.id)) - } - }() - - return processor, nil -} - -func (p *oldProcessor) sendError(err error) { - select { - case p.errCh <- err: - default: - log.Error("processor receives redundant error", zap.Error(err)) - } -} diff --git a/cdc/server.go b/cdc/server.go index 91260a52e05..3f00925b23a 100644 --- a/cdc/server.go +++ b/cdc/server.go @@ -37,12 +37,10 @@ import ( "github.com/prometheus/client_golang/prometheus" pd "github.com/tikv/pd/client" "go.etcd.io/etcd/clientv3" - "go.etcd.io/etcd/mvcc" "go.etcd.io/etcd/pkg/logutil" "go.uber.org/zap" "go.uber.org/zap/zapcore" "golang.org/x/sync/errgroup" - "golang.org/x/time/rate" "google.golang.org/grpc" "google.golang.org/grpc/backoff" ) @@ -55,10 +53,8 @@ const ( // Server is the capture server type Server struct { - captureV2 *capture.Capture + capture *capture.Capture - capture *Capture - owner *Owner ownerLock sync.RWMutex statusServer *http.Server pdClient pd.Client @@ -108,39 +104,37 @@ func (s *Server) Run(ctx context.Context) error { return cerror.WrapError(cerror.ErrServerNewPDClient, err) } s.pdClient = pdClient - if config.NewReplicaImpl { - tlsConfig, err := conf.Security.ToTLSConfig() - if err != nil { - return errors.Trace(err) - } - logConfig := logutil.DefaultZapLoggerConfig - logConfig.Level = zap.NewAtomicLevelAt(zapcore.ErrorLevel) - etcdCli, err := clientv3.New(clientv3.Config{ - Endpoints: s.pdEndpoints, - TLS: tlsConfig, - Context: ctx, - LogConfig: &logConfig, - DialTimeout: 5 * time.Second, - DialOptions: []grpc.DialOption{ - grpcTLSOption, - grpc.WithBlock(), - grpc.WithConnectParams(grpc.ConnectParams{ - Backoff: backoff.Config{ - BaseDelay: time.Second, - Multiplier: 1.1, - Jitter: 0.1, - MaxDelay: 3 * time.Second, - }, - MinConnectTimeout: 3 * time.Second, - }), - }, - }) - if err != nil { - return errors.Annotate(cerror.WrapError(cerror.ErrNewCaptureFailed, err), "new etcd client") - } - etcdClient := kv.NewCDCEtcdClient(ctx, etcdCli) - s.etcdClient = &etcdClient + tlsConfig, err := conf.Security.ToTLSConfig() + if err != nil { + return errors.Trace(err) + } + logConfig := logutil.DefaultZapLoggerConfig + logConfig.Level = zap.NewAtomicLevelAt(zapcore.ErrorLevel) + etcdCli, err := clientv3.New(clientv3.Config{ + Endpoints: s.pdEndpoints, + TLS: tlsConfig, + Context: ctx, + LogConfig: &logConfig, + DialTimeout: 5 * time.Second, + DialOptions: []grpc.DialOption{ + grpcTLSOption, + grpc.WithBlock(), + grpc.WithConnectParams(grpc.ConnectParams{ + Backoff: backoff.Config{ + BaseDelay: time.Second, + Multiplier: 1.1, + Jitter: 0.1, + MaxDelay: 3 * time.Second, + }, + MinConnectTimeout: 3 * time.Second, + }), + }, + }) + if err != nil { + return errors.Annotate(cerror.WrapError(cerror.ErrNewCaptureFailed, err), "new etcd client") } + etcdClient := kv.NewCDCEtcdClient(ctx, etcdCli) + s.etcdClient = &etcdClient if err := s.initDataDir(ctx); err != nil { return errors.Trace(err) @@ -176,81 +170,9 @@ func (s *Server) Run(ctx context.Context) error { }() s.kvStorage = kvStore ctx = util.PutKVStorageInCtx(ctx, kvStore) - if config.NewReplicaImpl { - s.captureV2 = capture.NewCapture(s.pdClient, s.kvStorage, s.etcdClient) - return s.run(ctx) - } - // When a capture suicided, restart it - for { - if err := s.run(ctx); cerror.ErrCaptureSuicide.NotEqual(err) { - return err - } - log.Info("server recovered", zap.String("capture-id", s.capture.info.ID)) - } -} - -func (s *Server) setOwner(owner *Owner) { - s.ownerLock.Lock() - defer s.ownerLock.Unlock() - s.owner = owner -} - -func (s *Server) campaignOwnerLoop(ctx context.Context) error { - // In most failure cases, we don't return error directly, just run another - // campaign loop. We treat campaign loop as a special background routine. - - conf := config.GetGlobalServerConfig() - rl := rate.NewLimiter(0.05, 2) - for { - err := rl.Wait(ctx) - if err != nil { - if errors.Cause(err) == context.Canceled { - return nil - } - return errors.Trace(err) - } - // Campaign to be an owner, it blocks until it becomes the owner - if err := s.capture.Campaign(ctx); err != nil { - switch errors.Cause(err) { - case context.Canceled: - return nil - case mvcc.ErrCompacted: - continue - } - log.Warn("campaign owner failed", zap.Error(err)) - continue - } - captureID := s.capture.info.ID - log.Info("campaign owner successfully", zap.String("capture-id", captureID)) - owner, err := NewOwner(ctx, s.pdClient, s.capture.grpcPool, s.capture.session, conf.GcTTL, time.Duration(conf.OwnerFlushInterval)) - if err != nil { - log.Warn("create new owner failed", zap.Error(err)) - continue - } - - s.setOwner(owner) - if err := owner.Run(ctx, ownerRunInterval); err != nil { - if errors.Cause(err) == context.Canceled { - log.Info("owner exited", zap.String("capture-id", captureID)) - select { - case <-ctx.Done(): - // only exits the campaignOwnerLoop if parent context is done - return ctx.Err() - default: - } - log.Info("owner exited", zap.String("capture-id", captureID)) - } - err2 := s.capture.Resign(ctx) - if err2 != nil { - // if resign owner failed, return error to let capture exits - return errors.Annotatef(err2, "resign owner failed, capture: %s", captureID) - } - log.Warn("run owner failed", zap.Error(err)) - } - // owner is resigned by API, reset owner and continue the campaign loop - s.setOwner(nil) - } + s.capture = capture.NewCapture(s.pdClient, s.kvStorage, s.etcdClient) + return s.run(ctx) } func (s *Server) etcdHealthChecker(ctx context.Context) error { @@ -296,35 +218,15 @@ func (s *Server) etcdHealthChecker(ctx context.Context) error { } func (s *Server) run(ctx context.Context) (err error) { - if !config.NewReplicaImpl { - kvStorage, err := util.KVStorageFromCtx(ctx) - if err != nil { - return errors.Trace(err) - } - capture, err := NewCapture(ctx, s.pdEndpoints, s.pdClient, kvStorage) - if err != nil { - return err - } - s.capture = capture - s.etcdClient = &capture.etcdClient - } ctx, cancel := context.WithCancel(ctx) defer cancel() wg, cctx := errgroup.WithContext(ctx) - if config.NewReplicaImpl { - wg.Go(func() error { - return s.captureV2.Run(cctx) - }) - } else { - wg.Go(func() error { - return s.campaignOwnerLoop(cctx) - }) - - wg.Go(func() error { - return s.capture.Run(cctx) - }) - } + + wg.Go(func() error { + return s.capture.Run(cctx) + }) + wg.Go(func() error { return s.etcdHealthChecker(cctx) }) @@ -343,18 +245,7 @@ func (s *Server) run(ctx context.Context) (err error) { // Close closes the server. func (s *Server) Close() { if s.capture != nil { - if !config.NewReplicaImpl { - s.capture.Cleanup() - } - closeCtx, closeCancel := context.WithTimeout(context.Background(), time.Second*2) - err := s.capture.Close(closeCtx) - if err != nil { - log.Error("close capture", zap.Error(err)) - } - closeCancel() - } - if s.captureV2 != nil { - s.captureV2.AsyncClose() + s.capture.AsyncClose() } if s.statusServer != nil { err := s.statusServer.Close() diff --git a/cdc/task_test.go b/cdc/task_test.go deleted file mode 100644 index d801d50bc58..00000000000 --- a/cdc/task_test.go +++ /dev/null @@ -1,310 +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 cdc - -import ( - "context" - "math" - "time" - - "github.com/pingcap/check" - "github.com/pingcap/failpoint" - "github.com/pingcap/tiflow/cdc/kv" - "github.com/pingcap/tiflow/cdc/model" - "github.com/pingcap/tiflow/pkg/config" - "github.com/pingcap/tiflow/pkg/etcd" - "github.com/pingcap/tiflow/pkg/util/testleak" - "go.etcd.io/etcd/clientv3" - "go.etcd.io/etcd/embed" -) - -type taskSuite struct { - s *embed.Etcd - c *clientv3.Client - w *TaskWatcher - endpoints []string -} - -var _ = check.Suite(&taskSuite{}) - -func (s *taskSuite) SetUpTest(c *check.C) { - dir := c.MkDir() - url, etcd, err := etcd.SetupEmbedEtcd(dir) - c.Assert(err, check.IsNil) - - endpoints := []string{url.String()} - client, err := clientv3.New(clientv3.Config{ - Endpoints: endpoints, - }) - c.Assert(err, check.IsNil) - - // Create a task watcher - capture := &Capture{ - etcdClient: kv.NewCDCEtcdClient(context.TODO(), client), - processors: make(map[string]*oldProcessor), - info: &model.CaptureInfo{ID: "task-suite-capture", AdvertiseAddr: "task-suite-addr"}, - } - c.Assert(capture, check.NotNil) - watcher := NewTaskWatcher(capture, &TaskWatcherConfig{ - Prefix: kv.TaskStatusKeyPrefix + "/" + capture.info.ID, - }) - c.Assert(watcher, check.NotNil) - - s.s = etcd - s.c = client - s.w = watcher - s.endpoints = endpoints -} - -func (s *taskSuite) TearDownTest(c *check.C) { - s.s.Close() - s.c.Close() -} - -func (s *taskSuite) TestNewTaskWatcher(c *check.C) { - defer testleak.AfterTest(c)() - defer s.TearDownTest(c) - // Create a capture instance by initialize the struct, - // NewCapture can not be used because it requires to - // initialize the PD service witch does not support to - // be embeded. - if config.NewReplicaImpl { - c.Skip("this case is designed for old processor") - } - capture := &Capture{ - etcdClient: kv.NewCDCEtcdClient(context.TODO(), s.c), - processors: make(map[string]*oldProcessor), - info: &model.CaptureInfo{ID: "task-suite-capture", AdvertiseAddr: "task-suite-addr"}, - } - c.Assert(capture, check.NotNil) - c.Assert(NewTaskWatcher(capture, &TaskWatcherConfig{ - Prefix: kv.TaskStatusKeyPrefix + "/" + capture.info.ID, - }), check.NotNil) - capture.Close(context.Background()) -} - -func (s *taskSuite) setupFeedInfo(c *check.C, changeFeedID string) { - client := kv.NewCDCEtcdClient(context.TODO(), s.c) - // Create the change feed - c.Assert(client.SaveChangeFeedInfo(s.c.Ctx(), &model.ChangeFeedInfo{ - SinkURI: "mysql://fake", - StartTs: 0, - TargetTs: math.MaxUint64, - CreateTime: time.Now(), - }, changeFeedID), check.IsNil) - - // Fake the change feed status - c.Assert(client.PutChangeFeedStatus(s.c.Ctx(), changeFeedID, - &model.ChangeFeedStatus{ - ResolvedTs: 1, - CheckpointTs: 1, - }), check.IsNil) -} - -func (s *taskSuite) teardownFeedInfo(c *check.C, changeFeedID string) { - etcd := s.c - // Delete change feed info - resp, err := etcd.Delete(s.c.Ctx(), kv.GetEtcdKeyChangeFeedInfo(changeFeedID), clientv3.WithPrefix()) - c.Assert(err, check.IsNil) - c.Assert(resp, check.NotNil) - - // Delete change feed status(job status) - resp, err = etcd.Delete(s.c.Ctx(), kv.GetEtcdKeyJob(changeFeedID), clientv3.WithPrefix()) - c.Assert(err, check.IsNil) - c.Assert(resp, check.NotNil) -} - -func (s *taskSuite) TestParseTask(c *check.C) { - defer testleak.AfterTest(c)() - defer s.TearDownTest(c) - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - changeFeedID := "task-suite-changefeed" - s.setupFeedInfo(c, changeFeedID) - defer s.teardownFeedInfo(c, changeFeedID) - - tests := []struct { - Desc string - Key []byte - Expected *Task - }{ - {"nil task key", nil, nil}, - {"short task key", []byte("test"), nil}, - { - "normal task key", - []byte(kv.GetEtcdKeyTaskStatus(changeFeedID, s.w.capture.info.ID)), - &Task{changeFeedID, 1}, - }, - } - for _, t := range tests { - c.Log("testing ", t.Desc) - task, err := s.w.parseTask(ctx, t.Key) - if t.Expected == nil { - c.Assert(err, check.NotNil) - c.Assert(task, check.IsNil) - } else { - c.Assert(task, check.DeepEquals, t.Expected) - } - } -} - -func (s *taskSuite) TestWatch(c *check.C) { - defer testleak.AfterTest(c)() - defer s.TearDownTest(c) - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - client := kv.NewCDCEtcdClient(ctx, s.c) - defer client.Close() //nolint:errcheck - - s.setupFeedInfo(c, "changefeed-1") - defer s.teardownFeedInfo(c, "changefeed-1") - - // Watch with a canceled context - failedCtx, cancel := context.WithCancel(context.Background()) - cancel() - ev := <-s.w.Watch(failedCtx) - if ev != nil { - c.Assert(ev.Err, check.NotNil) - } - - // Watch with a normal context - ctx = context.Background() - ctx, cancel = context.WithCancel(ctx) - defer cancel() - ch := s.w.Watch(ctx) - - // Trigger the ErrCompacted error - c.Assert(failpoint.Enable("github.com/pingcap/tiflow/cdc.restart_task_watch", "50%off"), check.IsNil) - - // Put task changefeed-1 - c.Assert(client.PutTaskStatus(s.c.Ctx(), "changefeed-1", - s.w.capture.info.ID, - &model.TaskStatus{}), check.IsNil) - ev = <-ch - c.Assert(len(ch), check.Equals, 0) - c.Assert(ev, check.NotNil) - c.Assert(ev.Err, check.IsNil) - c.Assert(ev.Op, check.Equals, TaskOpCreate) - c.Assert(ev.Task.ChangeFeedID, check.Equals, "changefeed-1") - c.Assert(ev.Task.CheckpointTS, check.Equals, uint64(1)) - - // Stop the task changefeed-1 - c.Assert(client.PutTaskStatus(s.c.Ctx(), "changefeed-1", - s.w.capture.info.ID, - &model.TaskStatus{AdminJobType: model.AdminStop}), check.IsNil) - ev = <-ch - c.Assert(len(ch), check.Equals, 0) - c.Assert(ev, check.NotNil) - c.Assert(ev.Err, check.IsNil) - c.Assert(ev.Op, check.Equals, TaskOpDelete) - c.Assert(ev.Task.ChangeFeedID, check.Equals, "changefeed-1") - c.Assert(ev.Task.CheckpointTS, check.Equals, uint64(1)) - - // Resume the task changefeed-1 - c.Assert(client.PutTaskStatus(s.c.Ctx(), "changefeed-1", - s.w.capture.info.ID, - &model.TaskStatus{AdminJobType: model.AdminResume}), check.IsNil) - ev = <-ch - c.Assert(len(ch), check.Equals, 0) - c.Assert(ev, check.NotNil) - c.Assert(ev.Err, check.IsNil) - c.Assert(ev.Op, check.Equals, TaskOpCreate) - c.Assert(ev.Task.ChangeFeedID, check.Equals, "changefeed-1") - c.Assert(ev.Task.CheckpointTS, check.Equals, uint64(1)) - - // Delete the task changefeed-1 - c.Assert(client.DeleteTaskStatus(ctx, "changefeed-1", - s.w.capture.info.ID), check.IsNil) - ev = <-ch - c.Assert(len(ch), check.Equals, 0) - c.Assert(ev, check.NotNil) - c.Assert(ev.Err, check.IsNil) - c.Assert(ev.Op, check.Equals, TaskOpDelete) - c.Assert(ev.Task.ChangeFeedID, check.Equals, "changefeed-1") - c.Assert(ev.Task.CheckpointTS, check.Equals, uint64(1)) - - // Put task changefeed-2 which does not exist - c.Assert(client.PutTaskStatus(s.c.Ctx(), "changefeed-2", - s.w.capture.info.ID, - &model.TaskStatus{}), check.IsNil) - c.Assert(len(ch), check.Equals, 0) -} - -func (s *taskSuite) TestRebuildTaskEvents(c *check.C) { - defer testleak.AfterTest(c)() - defer s.TearDownTest(c) - type T map[string]*TaskEvent - tests := []struct { - desc string - outdated T - latest T - expected T - }{ - { - desc: "nil outdated", - outdated: nil, - latest: T{"changefeed-1": &TaskEvent{TaskOpCreate, &Task{"changeed-1", 0}, nil}}, - expected: T{"changefeed-1": &TaskEvent{TaskOpCreate, &Task{"changeed-1", 0}, nil}}, - }, - { - desc: "empty outdated", - outdated: nil, - latest: T{"changefeed-1": &TaskEvent{TaskOpCreate, &Task{"changeed-1", 0}, nil}}, - expected: T{"changefeed-1": &TaskEvent{TaskOpCreate, &Task{"changeed-1", 0}, nil}}, - }, - { - desc: "need to be updated", - outdated: T{"changefeed-1": &TaskEvent{TaskOpCreate, &Task{"changeed-1", 0}, nil}}, - latest: T{"changefeed-1": &TaskEvent{TaskOpDelete, &Task{"changeed-1", 0}, nil}}, - expected: T{"changefeed-1": &TaskEvent{TaskOpDelete, &Task{"changeed-1", 0}, nil}}, - }, - { - desc: "miss some events", - outdated: T{"changefeed-1": &TaskEvent{TaskOpCreate, &Task{"changeed-1", 0}, nil}}, - latest: T{ - "changefeed-1": &TaskEvent{TaskOpDelete, &Task{"changeed-1", 0}, nil}, - "changefeed-2": &TaskEvent{TaskOpCreate, &Task{"changefeed-2", 0}, nil}, - }, - expected: T{ - "changefeed-1": &TaskEvent{TaskOpDelete, &Task{"changeed-1", 0}, nil}, - "changefeed-2": &TaskEvent{TaskOpCreate, &Task{"changefeed-2", 0}, nil}, - }, - }, - { - desc: "left some events", - outdated: T{ - "changefeed-1": &TaskEvent{TaskOpDelete, &Task{"changeed-1", 0}, nil}, - "changefeed-2": &TaskEvent{TaskOpCreate, &Task{"changefeed-2", 0}, nil}, - }, - latest: T{"changefeed-1": &TaskEvent{TaskOpCreate, &Task{"changeed-1", 0}, nil}}, - expected: T{ - "changefeed-1": &TaskEvent{TaskOpCreate, &Task{"changeed-1", 0}, nil}, - "changefeed-2": &TaskEvent{TaskOpDelete, &Task{"changefeed-2", 0}, nil}, - }, - }, - } - - for _, t := range tests { - c.Log("RUN CASE: ", t.desc) - s.w.events = t.outdated - got := s.w.rebuildTaskEvents(t.latest) - c.Assert(len(got), check.Equals, len(t.expected)) - for k, v := range got { - e := t.expected[k] - c.Assert(v.Err, check.IsNil) - c.Assert(v.Op, check.Equals, e.Op) - c.Assert(v.Task, check.DeepEquals, e.Task) - } - } -} diff --git a/cmd/client_unsafe.go b/cmd/client_unsafe.go index 34dd2c6c640..7cef8e27435 100644 --- a/cmd/client_unsafe.go +++ b/cmd/client_unsafe.go @@ -18,7 +18,7 @@ import ( "strings" "github.com/pingcap/errors" - "github.com/pingcap/tiflow/cdc" + "github.com/pingcap/tiflow/pkg/txnutil/gc" "github.com/spf13/cobra" ) @@ -63,7 +63,7 @@ func newResetCommand() *cobra.Command { return errors.Trace(err) } - _, err = pdCli.UpdateServiceGCSafePoint(ctx, cdc.CDCServiceSafePointID, 0, 0) + _, err = pdCli.UpdateServiceGCSafePoint(ctx, gc.CDCServiceSafePointID, 0, 0) if err != nil { return errors.Trace(err) } @@ -107,7 +107,7 @@ func newDeleteServiceGcSafepointCommand() *cobra.Command { return err } ctx := defaultContext - _, err := pdCli.UpdateServiceGCSafePoint(ctx, cdc.CDCServiceSafePointID, 0, 0) + _, err := pdCli.UpdateServiceGCSafePoint(ctx, gc.CDCServiceSafePointID, 0, 0) if err == nil { cmd.Println("CDC service GC safepoint truncated in PD!") } diff --git a/tests/integration_tests/changefeed_error/run.sh b/tests/integration_tests/changefeed_error/run.sh index 9bb9f168022..6c5b878c187 100755 --- a/tests/integration_tests/changefeed_error/run.sh +++ b/tests/integration_tests/changefeed_error/run.sh @@ -163,12 +163,7 @@ function run() { cleanup_process $CDC_BINARY # updating GC safepoint failure case -<<<<<<< HEAD - export GO_FAILPOINTS='github.com/pingcap/tiflow/pkg/txnutil/gc/InjectActualGCSafePoint=return(9223372036854775807)' # new owner - # export GO_FAILPOINTS='github.com/pingcap/tiflow/cdc/InjectActualGCSafePoint=return(9223372036854775807)' # old owner -======= export GO_FAILPOINTS='github.com/pingcap/tiflow/pkg/txnutil/gc/InjectActualGCSafePoint=return(9223372036854775807)' ->>>>>>> 6591f62df (Clean old owner and old processor in release 5.2 branch (#4019)) run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY changefeedid_2="changefeed-error-2"