From 2898d74576f936b2c46734a0796df42644cc9aa2 Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Wed, 22 Nov 2023 15:36:16 +0800 Subject: [PATCH 01/35] first commit --- cdc/api/v2/api.go | 1 + cdc/api/v2/changefeed.go | 78 +++++++++++ cdc/model/changefeed.go | 6 + cdc/owner/changefeed.go | 32 ++--- cdc/owner/owner.go | 13 +- cdc/owner/status_provider.go | 20 +++ cdc/processor/processor.go | 1 + cdc/processor/sinkmanager/manager.go | 4 + .../sinkmanager/table_sink_wrapper.go | 6 + cdc/processor/tablepb/table.pb.go | 125 ++++++++++++------ cdc/processor/tablepb/table.proto | 1 + cdc/scheduler/internal/scheduler.go | 2 +- cdc/scheduler/internal/v3/coordinator.go | 22 +-- .../v3/replication/replication_manager.go | 19 +-- .../v3/replication/replication_set.go | 4 + .../cloudstorage/cloud_storage_dml_sink.go | 1 + cdc/sink/tablesink/table_sink.go | 3 + cdc/sink/tablesink/table_sink_impl.go | 33 ++++- 18 files changed, 289 insertions(+), 82 deletions(-) diff --git a/cdc/api/v2/api.go b/cdc/api/v2/api.go index 09a2849cdb2..a6b8b48c4ba 100644 --- a/cdc/api/v2/api.go +++ b/cdc/api/v2/api.go @@ -63,6 +63,7 @@ func RegisterOpenAPIV2Routes(router *gin.Engine, api OpenAPIV2) { changefeedGroup.POST("/:changefeed_id/resume", changefeedOwnerMiddleware, api.resumeChangefeed) changefeedGroup.POST("/:changefeed_id/pause", changefeedOwnerMiddleware, api.pauseChangefeed) changefeedGroup.GET("/:changefeed_id/status", changefeedOwnerMiddleware, api.status) + changefeedGroup.GET("/:changefeed_id/synced", changefeedOwnerMiddleware, api.synced) // capture apis captureGroup := v2.Group("/captures") diff --git a/cdc/api/v2/changefeed.go b/cdc/api/v2/changefeed.go index 7d5373c526d..cbbbc8550ff 100644 --- a/cdc/api/v2/changefeed.go +++ b/cdc/api/v2/changefeed.go @@ -895,6 +895,84 @@ func (h *OpenAPIV2) status(c *gin.Context) { }) } +func (h *OpenAPIV2) synced(c *gin.Context) { + ctx := c.Request.Context() + + namespace := getNamespaceValueWithDefault(c) + changefeedID := model.ChangeFeedID{Namespace: namespace, ID: c.Param(apiOpVarChangefeedID)} + if err := model.ValidateChangefeedID(changefeedID.ID); err != nil { + _ = c.Error(cerror.ErrAPIInvalidParam.GenWithStack("invalid changefeed_id: %s", + changefeedID.ID)) + return + } + status, err := h.capture.StatusProvider().GetChangeFeedSyncedStatus( + ctx, + changefeedID, + ) + if err != nil { + _ = c.Error(err) + return + } + + // get pd_now + cfg := &ChangefeedConfig{ReplicaConfig: GetDefaultReplicaConfig()} + + if err := c.BindJSON(&cfg); err != nil { + _ = c.Error(cerror.WrapError(cerror.ErrAPIInvalidParam, err)) + return + } + if len(cfg.PDAddrs) == 0 { + up, err := getCaptureDefaultUpstream(h.capture) + if err != nil { + _ = c.Error(err) + return + } + cfg.PDConfig = getUpstreamPDConfig(up) + } + credential := cfg.PDConfig.toCredential() + + timeoutCtx, cancel := context.WithTimeout(ctx, 30*time.Second) + defer cancel() + pdClient, err := h.helpers.getPDClient(timeoutCtx, cfg.PDAddrs, credential) + defer pdClient.Close() + if err != nil { + // _ = c.Error(cerror.WrapError(cerror.ErrAPIGetPDClientFailed, err)) + // return + // means pd is offline + c.JSON(http.StatusOK, map[string]any{ + "Synced": false, + "CheckpointTs": status.CheckpointTs, + "ResolvedTs": status.ResolvedTs, + "LastSyncTime": status.LastSyncTime, + "info": "xxxxx", + }) + return + } + + //TSO 是啥? + now, _, _ := pdClient.GetTS(ctx) + // get pd_now + // 随便写一个先 + if (now-int64(status.LastSyncTime) > 60*5) && (now-int64(status.CheckpointTs) < 5) { + c.JSON(http.StatusOK, map[string]any{ + "Synced": true, + "CheckpointTs": status.CheckpointTs, + "ResolvedTs": status.ResolvedTs, + "LastSyncTime": status.LastSyncTime, + "info": "", + }) + } else { + c.JSON(http.StatusOK, map[string]any{ + "Synced": false, + "CheckpointTs": status.CheckpointTs, + "ResolvedTs": status.ResolvedTs, + "LastSyncTime": status.LastSyncTime, + "info": "xxxxx", + }) + } + +} + func toAPIModel( info *model.ChangeFeedInfo, resolvedTs uint64, diff --git a/cdc/model/changefeed.go b/cdc/model/changefeed.go index 805d1a56729..34a5aff5c3e 100644 --- a/cdc/model/changefeed.go +++ b/cdc/model/changefeed.go @@ -629,3 +629,9 @@ type ChangeFeedStatusForAPI struct { ResolvedTs uint64 `json:"resolved-ts"` CheckpointTs uint64 `json:"checkpoint-ts"` } + +type ChangeFeedSyncedStatusForAPI struct { + ResolvedTs uint64 `json:"resolved-ts"` + CheckpointTs uint64 `json:"checkpoint-ts"` + LastSyncTime uint64 `json:"last-sync-time"` +} diff --git a/cdc/owner/changefeed.go b/cdc/owner/changefeed.go index 436485e46f4..47e80d954e4 100644 --- a/cdc/owner/changefeed.go +++ b/cdc/owner/changefeed.go @@ -93,6 +93,7 @@ type changefeed struct { barriers *barriers feedStateManager FeedStateManager resolvedTs model.Ts + lastSyncTime model.Ts // ddl related fields ddlManager *ddlManager @@ -272,7 +273,8 @@ func (c *changefeed) Tick(ctx cdcContext.Context, } return nil }) - checkpointTs, minTableBarrierTs, err := c.tick(ctx, captures) + checkpointTs, minTableBarrierTs, maxLastSyncTime, err := c.tick(ctx, captures) + c.lastSyncTime = maxLastSyncTime // The tick duration is recorded only if changefeed has completed initialization if c.initialized { @@ -350,50 +352,50 @@ func (c *changefeed) checkStaleCheckpointTs(ctx cdcContext.Context, // tick returns the checkpointTs and minTableBarrierTs. func (c *changefeed) tick(ctx cdcContext.Context, captures map[model.CaptureID]*model.CaptureInfo, -) (model.Ts, model.Ts, error) { +) (model.Ts, model.Ts, model.Ts, error) { adminJobPending := c.feedStateManager.Tick(c.resolvedTs, c.latestStatus, c.latestInfo) preCheckpointTs := c.latestInfo.GetCheckpointTs(c.latestStatus) // checkStaleCheckpointTs must be called before `feedStateManager.ShouldRunning()` // to ensure all changefeeds, no matter whether they are running or not, will be checked. if err := c.checkStaleCheckpointTs(ctx, c.latestInfo, preCheckpointTs); err != nil { - return 0, 0, errors.Trace(err) + return 0, 0, 0, errors.Trace(err) } if !c.feedStateManager.ShouldRunning() { c.isRemoved = c.feedStateManager.ShouldRemoved() c.releaseResources(ctx) - return 0, 0, nil + return 0, 0, 0, nil } if adminJobPending { - return 0, 0, nil + return 0, 0, 0, nil } if err := c.initialize(ctx); err != nil { - return 0, 0, errors.Trace(err) + return 0, 0, 0, errors.Trace(err) } select { case err := <-c.errCh: - return 0, 0, errors.Trace(err) + return 0, 0, 0, errors.Trace(err) default: } if c.redoMetaMgr.Enabled() { if !c.redoMetaMgr.Running() { - return 0, 0, nil + return 0, 0, 0, nil } } // TODO: pass table checkpointTs when we support concurrent process ddl allPhysicalTables, barrier, err := c.ddlManager.tick(ctx, preCheckpointTs, nil) if err != nil { - return 0, 0, errors.Trace(err) + return 0, 0, 0, errors.Trace(err) } err = c.handleBarrier(ctx, c.latestInfo, c.latestStatus, barrier) if err != nil { - return 0, 0, errors.Trace(err) + return 0, 0, 0, errors.Trace(err) } log.Debug("owner handles barrier", @@ -409,14 +411,14 @@ func (c *changefeed) tick(ctx cdcContext.Context, // This condition implies that the DDL resolved-ts has not yet reached checkpointTs, // which implies that it would be premature to schedule tables or to update status. // So we return here. - return 0, 0, nil + return 0, 0, 0, nil } - newCheckpointTs, newResolvedTs, err := c.scheduler.Tick( + newCheckpointTs, newResolvedTs, newLastSyncTime, err := c.scheduler.Tick( ctx, preCheckpointTs, allPhysicalTables, captures, barrier) if err != nil { - return 0, 0, errors.Trace(err) + return 0, 0, 0, errors.Trace(err) } pdTime := c.upstream.PDClock.CurrentTime() @@ -430,7 +432,7 @@ func (c *changefeed) tick(ctx cdcContext.Context, // advance the watermarks for now. c.updateMetrics(currentTs, c.latestStatus.CheckpointTs, c.resolvedTs) } - return 0, 0, nil + return 0, 0, 0, nil } log.Debug("owner prepares to update status", @@ -463,7 +465,7 @@ func (c *changefeed) tick(ctx cdcContext.Context, c.updateMetrics(currentTs, newCheckpointTs, c.resolvedTs) c.tickDownstreamObserver(ctx) - return newCheckpointTs, barrier.MinTableBarrierTs, nil + return newCheckpointTs, barrier.MinTableBarrierTs, newLastSyncTime, nil } func (c *changefeed) initialize(ctx cdcContext.Context) (err error) { diff --git a/cdc/owner/owner.go b/cdc/owner/owner.go index 966e26da00f..166cfc78df1 100644 --- a/cdc/owner/owner.go +++ b/cdc/owner/owner.go @@ -587,7 +587,7 @@ func (o *ownerImpl) handleJobs(ctx context.Context) { func (o *ownerImpl) handleQueries(query *Query) error { switch query.Tp { - case QueryChangeFeedStatuses: + case QueryChangeFeedSyncedStatus: cfReactor, ok := o.changefeeds[query.ChangeFeedID] if !ok { query.Data = nil @@ -597,6 +597,17 @@ func (o *ownerImpl) handleQueries(query *Query) error { ret.ResolvedTs = cfReactor.resolvedTs ret.CheckpointTs = cfReactor.latestStatus.CheckpointTs query.Data = ret + case QueryChangeFeedStatuses: + cfReactor, ok := o.changefeeds[query.ChangeFeedID] + if !ok { + query.Data = nil + return nil + } + ret := &model.ChangeFeedSyncedStatusForAPI{} + ret.ResolvedTs = cfReactor.resolvedTs + ret.LastSyncTime = cfReactor.lastSyncTime + ret.CheckpointTs = cfReactor.latestStatus.CheckpointTs + query.Data = ret case QueryChangefeedInfo: cfReactor, ok := o.changefeeds[query.ChangeFeedID] if !ok { diff --git a/cdc/owner/status_provider.go b/cdc/owner/status_provider.go index f1b6a057ddd..2cea1e2cd81 100644 --- a/cdc/owner/status_provider.go +++ b/cdc/owner/status_provider.go @@ -27,6 +27,8 @@ type StatusProvider interface { // GetChangeFeedStatus returns a changefeeds' runtime status. GetChangeFeedStatus(ctx context.Context, changefeedID model.ChangeFeedID) (*model.ChangeFeedStatusForAPI, error) + GetChangeFeedSyncedStatus(ctx context.Context, changefeedID model.ChangeFeedID) (*model.ChangeFeedSyncedStatusForAPI, error) + // GetChangeFeedInfo returns a changefeeds' info. GetChangeFeedInfo(ctx context.Context, changefeedID model.ChangeFeedID) (*model.ChangeFeedInfo, error) @@ -63,6 +65,8 @@ const ( QueryChangefeedInfo // QueryChangeFeedStatuses is the type of query changefeed status QueryChangeFeedStatuses + // QueryChangeFeedSyncedStatus is the type of query changefeed synced status + QueryChangeFeedSyncedStatus ) // Query wraps query command and return results. @@ -98,6 +102,22 @@ func (p *ownerStatusProvider) GetChangeFeedStatus(ctx context.Context, return query.Data.(*model.ChangeFeedStatusForAPI), nil } +func (p *ownerStatusProvider) GetChangeFeedSyncedStatus(ctx context.Context, + changefeedID model.ChangeFeedID, +) (*model.ChangeFeedSyncedStatusForAPI, error) { + query := &Query{ + Tp: QueryChangeFeedSyncedStatus, + ChangeFeedID: changefeedID, + } + if err := p.sendQueryToOwner(ctx, query); err != nil { + return nil, errors.Trace(err) + } + if query.Data == nil { + return nil, cerror.ErrChangeFeedNotExists.GenWithStackByArgs(changefeedID) + } + return query.Data.(*model.ChangeFeedSyncedStatusForAPI), nil +} + func (p *ownerStatusProvider) GetChangeFeedInfo(ctx context.Context, changefeedID model.ChangeFeedID, ) (*model.ChangeFeedInfo, error) { diff --git a/cdc/processor/processor.go b/cdc/processor/processor.go index bff2b877445..269eadfcf1d 100644 --- a/cdc/processor/processor.go +++ b/cdc/processor/processor.go @@ -366,6 +366,7 @@ func (p *processor) GetTableSpanStatus(span tablepb.Span, collectStat bool) tabl Checkpoint: tablepb.Checkpoint{ CheckpointTs: sinkStats.CheckpointTs, ResolvedTs: sinkStats.ResolvedTs, + LastSyncTime: sinkStats.LastSyncTime, }, State: state, Stats: stats, diff --git a/cdc/processor/sinkmanager/manager.go b/cdc/processor/sinkmanager/manager.go index 74c5513888c..c34ec1bc3ed 100644 --- a/cdc/processor/sinkmanager/manager.go +++ b/cdc/processor/sinkmanager/manager.go @@ -58,6 +58,7 @@ const ( type TableStats struct { CheckpointTs model.Ts ResolvedTs model.Ts + LastSyncTime model.Ts BarrierTs model.Ts } @@ -140,6 +141,7 @@ func New( redoDMLMgr redo.DMLManager, sourceManager *sourcemanager.SourceManager, ) *SinkManager { + m := &SinkManager{ changefeedID: changefeedID, changefeedInfo: changefeedInfo, @@ -961,6 +963,7 @@ func (m *SinkManager) GetTableStats(span tablepb.Span) TableStats { tableSink := value.(*tableSinkWrapper) checkpointTs := tableSink.getCheckpointTs() + lastSyncTime := tableSink.getLastSyncTime() m.sinkMemQuota.Release(span, checkpointTs) m.redoMemQuota.Release(span, checkpointTs) @@ -1003,6 +1006,7 @@ func (m *SinkManager) GetTableStats(span tablepb.Span) TableStats { return TableStats{ CheckpointTs: checkpointTs.ResolvedMark(), ResolvedTs: resolvedTs, + LastSyncTime: lastSyncTime, BarrierTs: tableSink.barrierTs.Load(), } } diff --git a/cdc/processor/sinkmanager/table_sink_wrapper.go b/cdc/processor/sinkmanager/table_sink_wrapper.go index 8d21d69709b..ac946ea1446 100644 --- a/cdc/processor/sinkmanager/table_sink_wrapper.go +++ b/cdc/processor/sinkmanager/table_sink_wrapper.go @@ -218,6 +218,12 @@ func (t *tableSinkWrapper) updateResolvedTs(ts model.ResolvedTs) error { return t.tableSink.s.UpdateResolvedTs(ts) } +func (t *tableSinkWrapper) getLastSyncTime() uint64 { + t.tableSink.RLock() + defer t.tableSink.RUnlock() + return t.tableSink.s.GetLastSyncTime() +} + func (t *tableSinkWrapper) getCheckpointTs() model.ResolvedTs { t.tableSink.RLock() defer t.tableSink.RUnlock() diff --git a/cdc/processor/tablepb/table.pb.go b/cdc/processor/tablepb/table.pb.go index da01e6d59ef..001685818e8 100644 --- a/cdc/processor/tablepb/table.pb.go +++ b/cdc/processor/tablepb/table.pb.go @@ -117,6 +117,7 @@ var xxx_messageInfo_Span proto.InternalMessageInfo type Checkpoint struct { CheckpointTs Ts `protobuf:"varint,1,opt,name=checkpoint_ts,json=checkpointTs,proto3,casttype=Ts" json:"checkpoint_ts,omitempty"` ResolvedTs Ts `protobuf:"varint,2,opt,name=resolved_ts,json=resolvedTs,proto3,casttype=Ts" json:"resolved_ts,omitempty"` + LastSyncTime Ts `protobuf:"varint,3,opt,name=last_sync_time,json=lastSyncTime,proto3,casttype=Ts" json:"last_sync_time,omitempty"` } func (m *Checkpoint) Reset() { *m = Checkpoint{} } @@ -166,6 +167,13 @@ func (m *Checkpoint) GetResolvedTs() Ts { return 0 } +func (m *Checkpoint) GetLastSyncTime() Ts { + if m != nil { + return m.LastSyncTime + } + return 0 +} + // Stats holds a statistic for a table. type Stats struct { // Number of captured regions. @@ -329,50 +337,52 @@ func init() { func init() { proto.RegisterFile("processor/tablepb/table.proto", fileDescriptor_ae83c9c6cf5ef75c) } var fileDescriptor_ae83c9c6cf5ef75c = []byte{ - // 688 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x54, 0xbf, 0x6f, 0xd3, 0x4e, - 0x1c, 0xb5, 0x9d, 0x5f, 0xcd, 0x27, 0xf9, 0x7e, 0xe5, 0xde, 0xb7, 0xed, 0x37, 0x44, 0x22, 0x31, - 0x51, 0x81, 0xaa, 0x95, 0x1c, 0x28, 0x0b, 0xea, 0xd6, 0xb4, 0x80, 0xaa, 0x0a, 0x09, 0xb9, 0x81, - 0x81, 0x25, 0xf2, 0x8f, 0xc3, 0xb5, 0x1a, 0xce, 0x96, 0xef, 0xd2, 0x2a, 0x1b, 0x23, 0xca, 0x02, - 0x13, 0x62, 0x89, 0xd4, 0x3f, 0xa7, 0x63, 0x47, 0x06, 0x14, 0x41, 0x2a, 0x66, 0xf6, 0x4e, 0xe8, - 0xee, 0xdc, 0xb8, 0x09, 0x0c, 0xa1, 0x4b, 0x72, 0xbe, 0xf7, 0x3e, 0xcf, 0xef, 0xbd, 0x3b, 0x19, - 0x6e, 0x47, 0x71, 0xe8, 0x62, 0x4a, 0xc3, 0xb8, 0xc9, 0x6c, 0xa7, 0x8b, 0x23, 0x47, 0xfe, 0x9b, - 0x51, 0x1c, 0xb2, 0x10, 0xad, 0x46, 0x01, 0xf1, 0x5d, 0x3b, 0x32, 0x59, 0xf0, 0xa6, 0x1b, 0x9e, - 0x98, 0xae, 0xe7, 0x9a, 0x93, 0x09, 0x33, 0x99, 0xa8, 0x2e, 0xf9, 0xa1, 0x1f, 0x8a, 0x81, 0x26, - 0x5f, 0xc9, 0xd9, 0xc6, 0x07, 0x15, 0xb2, 0x07, 0x91, 0x4d, 0xd0, 0x43, 0x58, 0x10, 0xcc, 0x4e, - 0xe0, 0x55, 0x54, 0x43, 0x5d, 0xcb, 0xb4, 0x56, 0xc6, 0xa3, 0x7a, 0xa1, 0xcd, 0xf7, 0xf6, 0x76, - 0x2f, 0xd3, 0xa5, 0x55, 0x10, 0xbc, 0x3d, 0x0f, 0xad, 0x42, 0x91, 0x32, 0x3b, 0x66, 0x9d, 0x23, - 0xdc, 0xaf, 0x68, 0x86, 0xba, 0x56, 0x6e, 0x15, 0x2e, 0x47, 0xf5, 0xcc, 0x3e, 0xee, 0x5b, 0x0b, - 0x02, 0xd9, 0xc7, 0x7d, 0x64, 0x40, 0x01, 0x13, 0x4f, 0x70, 0x32, 0xd3, 0x9c, 0x3c, 0x26, 0xde, - 0x3e, 0xee, 0x6f, 0x95, 0xdf, 0x9f, 0xd6, 0x95, 0xcf, 0xa7, 0x75, 0xe5, 0xdd, 0x57, 0x43, 0x69, - 0x38, 0x00, 0x3b, 0x87, 0xd8, 0x3d, 0x8a, 0xc2, 0x80, 0x30, 0xb4, 0x01, 0xff, 0xb8, 0x93, 0xa7, - 0x0e, 0xa3, 0xc2, 0x5b, 0xb6, 0x95, 0xbf, 0x1c, 0xd5, 0xb5, 0x36, 0xb5, 0xca, 0x29, 0xd8, 0xa6, - 0xe8, 0x3e, 0x94, 0x62, 0x4c, 0xc3, 0xee, 0x31, 0xf6, 0x38, 0x55, 0x9b, 0xa2, 0xc2, 0x15, 0xd4, - 0xa6, 0x8d, 0x1f, 0x1a, 0xe4, 0x0e, 0x98, 0xcd, 0x28, 0xba, 0x03, 0xe5, 0x18, 0xfb, 0x41, 0x48, - 0x3a, 0x6e, 0xd8, 0x23, 0x4c, 0xca, 0x5b, 0x25, 0xb9, 0xb7, 0xc3, 0xb7, 0xd0, 0x5d, 0x00, 0xb7, - 0x17, 0xc7, 0x58, 0xbe, 0x7f, 0x5a, 0xb4, 0x98, 0x20, 0x6d, 0x8a, 0x18, 0x2c, 0x52, 0x66, 0xfb, - 0xb8, 0x93, 0x5a, 0xa2, 0x95, 0x8c, 0x91, 0x59, 0x2b, 0x6d, 0x6e, 0x9b, 0xf3, 0x9c, 0x90, 0x29, - 0x1c, 0xf1, 0x5f, 0x1f, 0xa7, 0x0d, 0xd0, 0x27, 0x84, 0xc5, 0xfd, 0x56, 0xf6, 0x6c, 0x54, 0x57, - 0x2c, 0x9d, 0xce, 0x80, 0xdc, 0x9c, 0x63, 0xc7, 0x71, 0x80, 0x63, 0x6e, 0x2e, 0x3b, 0x6d, 0x2e, - 0x41, 0xda, 0xb4, 0xda, 0x83, 0xe5, 0x3f, 0xea, 0x22, 0x1d, 0x32, 0xfc, 0x64, 0x78, 0xec, 0xa2, - 0xc5, 0x97, 0xe8, 0x29, 0xe4, 0x8e, 0xed, 0x6e, 0x0f, 0x8b, 0xa4, 0xa5, 0xcd, 0x07, 0xf3, 0x79, - 0x4f, 0x85, 0x2d, 0x39, 0xbe, 0xa5, 0x3d, 0x56, 0x1b, 0x3f, 0x35, 0x28, 0x89, 0x6b, 0xc3, 0xa3, - 0xf5, 0xe8, 0x4d, 0x2e, 0xd9, 0x2e, 0x64, 0x69, 0x64, 0x93, 0x4a, 0x4e, 0xb8, 0x59, 0x9f, 0xb3, - 0xc9, 0xc8, 0x26, 0x49, 0x65, 0x62, 0x9a, 0x87, 0xa2, 0xcc, 0x66, 0x32, 0xd4, 0xbf, 0xf3, 0x86, - 0x9a, 0x58, 0xc7, 0x96, 0x1c, 0x47, 0xaf, 0x00, 0xd2, 0xe3, 0x15, 0xf7, 0xf9, 0x06, 0x0d, 0x25, - 0xce, 0xae, 0x29, 0xa1, 0x67, 0xd2, 0x9f, 0x3c, 0xc1, 0xd2, 0xe6, 0xc6, 0x5f, 0x5c, 0x98, 0x44, - 0x4d, 0xce, 0xaf, 0x7f, 0xd2, 0x00, 0x52, 0xdb, 0xa8, 0x01, 0x85, 0x97, 0xe4, 0x88, 0x84, 0x27, - 0x44, 0x57, 0xaa, 0xcb, 0x83, 0xa1, 0xb1, 0x98, 0x82, 0x09, 0x80, 0x0c, 0xc8, 0x6f, 0x3b, 0x14, - 0x13, 0xa6, 0xab, 0xd5, 0xa5, 0xc1, 0xd0, 0xd0, 0x53, 0x8a, 0xdc, 0x47, 0xf7, 0xa0, 0xf8, 0x22, - 0xc6, 0x91, 0x1d, 0x07, 0xc4, 0xd7, 0xb5, 0xea, 0xff, 0x83, 0xa1, 0xf1, 0x5f, 0x4a, 0x9a, 0x40, - 0x68, 0x15, 0x16, 0xe4, 0x03, 0xf6, 0xf4, 0x4c, 0x75, 0x65, 0x30, 0x34, 0xd0, 0x2c, 0x0d, 0x7b, - 0x68, 0x1d, 0x4a, 0x16, 0x8e, 0xba, 0x81, 0x6b, 0x33, 0xae, 0x97, 0xad, 0xde, 0x1a, 0x0c, 0x8d, - 0xe5, 0x6b, 0x5d, 0xa7, 0x20, 0x57, 0x3c, 0x60, 0x61, 0xc4, 0xdb, 0xd0, 0x73, 0xb3, 0x8a, 0x57, - 0x08, 0x4f, 0x29, 0xd6, 0xd8, 0xd3, 0xf3, 0xb3, 0x29, 0x13, 0xa0, 0xf5, 0xfc, 0xfc, 0x7b, 0x4d, - 0x39, 0x1b, 0xd7, 0xd4, 0xf3, 0x71, 0x4d, 0xfd, 0x36, 0xae, 0xa9, 0x1f, 0x2f, 0x6a, 0xca, 0xf9, - 0x45, 0x4d, 0xf9, 0x72, 0x51, 0x53, 0x5e, 0x37, 0xfd, 0x80, 0x1d, 0xf6, 0x1c, 0xd3, 0x0d, 0xdf, - 0x36, 0x93, 0xea, 0x9b, 0xb2, 0xfa, 0xa6, 0xeb, 0xb9, 0xcd, 0xdf, 0xbe, 0xbf, 0x4e, 0x5e, 0x7c, - 0x3e, 0x1f, 0xfd, 0x0a, 0x00, 0x00, 0xff, 0xff, 0x80, 0x0e, 0x45, 0x99, 0x9b, 0x05, 0x00, 0x00, + // 712 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x54, 0xbf, 0x6f, 0xd3, 0x40, + 0x18, 0xb5, 0x9d, 0x5f, 0xcd, 0x97, 0x50, 0xb9, 0x47, 0x5b, 0x42, 0x24, 0x12, 0x13, 0x15, 0xa8, + 0x5a, 0xe4, 0x40, 0x59, 0x50, 0xb7, 0xa6, 0x05, 0x54, 0x55, 0x48, 0xc8, 0x09, 0x0c, 0x2c, 0x91, + 0x63, 0x1f, 0xae, 0xd5, 0xf4, 0x6c, 0xf9, 0x2e, 0xad, 0xbc, 0x31, 0xa2, 0x2c, 0x74, 0x42, 0x2c, + 0x91, 0xfa, 0xe7, 0x74, 0xec, 0xc8, 0x80, 0x22, 0x48, 0xc5, 0xcc, 0xde, 0x09, 0xdd, 0xd9, 0x8d, + 0x9b, 0xc0, 0x10, 0xba, 0x24, 0xe7, 0x7b, 0xef, 0x7b, 0x7a, 0xef, 0xdd, 0xe9, 0xe0, 0x9e, 0x1f, + 0x78, 0x16, 0xa6, 0xd4, 0x0b, 0xea, 0xcc, 0xec, 0x74, 0xb1, 0xdf, 0x89, 0xfe, 0x75, 0x3f, 0xf0, + 0x98, 0x87, 0x56, 0x7c, 0x97, 0x38, 0x96, 0xe9, 0xeb, 0xcc, 0xfd, 0xd0, 0xf5, 0x8e, 0x75, 0xcb, + 0xb6, 0xf4, 0xf1, 0x84, 0x1e, 0x4f, 0x94, 0x17, 0x1d, 0xcf, 0xf1, 0xc4, 0x40, 0x9d, 0xaf, 0xa2, + 0xd9, 0xda, 0x67, 0x19, 0xd2, 0x4d, 0xdf, 0x24, 0xe8, 0x29, 0xcc, 0x09, 0x66, 0xdb, 0xb5, 0x4b, + 0xb2, 0x26, 0xaf, 0xa6, 0x1a, 0xcb, 0xa3, 0x61, 0x35, 0xd7, 0xe2, 0x7b, 0xbb, 0x3b, 0x97, 0xc9, + 0xd2, 0xc8, 0x09, 0xde, 0xae, 0x8d, 0x56, 0x20, 0x4f, 0x99, 0x19, 0xb0, 0xf6, 0x01, 0x0e, 0x4b, + 0x8a, 0x26, 0xaf, 0x16, 0x1b, 0xb9, 0xcb, 0x61, 0x35, 0xb5, 0x87, 0x43, 0x63, 0x4e, 0x20, 0x7b, + 0x38, 0x44, 0x1a, 0xe4, 0x30, 0xb1, 0x05, 0x27, 0x35, 0xc9, 0xc9, 0x62, 0x62, 0xef, 0xe1, 0x70, + 0xb3, 0xf8, 0xe9, 0xb4, 0x2a, 0x7d, 0x3d, 0xad, 0x4a, 0x1f, 0xbf, 0x6b, 0x52, 0xed, 0x44, 0x06, + 0xd8, 0xde, 0xc7, 0xd6, 0x81, 0xef, 0xb9, 0x84, 0xa1, 0x75, 0xb8, 0x65, 0x8d, 0xbf, 0xda, 0x8c, + 0x0a, 0x73, 0xe9, 0x46, 0xf6, 0x72, 0x58, 0x55, 0x5a, 0xd4, 0x28, 0x26, 0x60, 0x8b, 0xa2, 0x47, + 0x50, 0x08, 0x30, 0xf5, 0xba, 0x47, 0xd8, 0xe6, 0x54, 0x65, 0x82, 0x0a, 0x57, 0x50, 0x8b, 0xa2, + 0xc7, 0x30, 0xdf, 0x35, 0x29, 0x6b, 0xd3, 0x90, 0x58, 0x6d, 0xe6, 0x1e, 0x62, 0xe1, 0xed, 0x9a, + 0x2c, 0x47, 0x9b, 0x21, 0xb1, 0x5a, 0xee, 0x21, 0xae, 0xfd, 0x52, 0x20, 0xd3, 0x64, 0x26, 0xa3, + 0xe8, 0x3e, 0x14, 0x03, 0xec, 0xb8, 0x1e, 0x69, 0x5b, 0x5e, 0x8f, 0xb0, 0xc8, 0x8c, 0x51, 0x88, + 0xf6, 0xb6, 0xf9, 0x16, 0x7a, 0x00, 0x60, 0xf5, 0x82, 0x00, 0x47, 0x6e, 0x27, 0x2d, 0xe4, 0x63, + 0xa4, 0x45, 0x11, 0x83, 0x05, 0xca, 0x4c, 0x07, 0xb7, 0x93, 0x00, 0xb4, 0x94, 0xd2, 0x52, 0xab, + 0x85, 0x8d, 0x2d, 0x7d, 0x96, 0x03, 0xd5, 0x85, 0x23, 0xfe, 0xeb, 0xe0, 0xa4, 0x2f, 0xfa, 0x82, + 0xb0, 0x20, 0x6c, 0xa4, 0xcf, 0x86, 0x55, 0xc9, 0x50, 0xe9, 0x14, 0xc8, 0xcd, 0x75, 0xcc, 0x20, + 0x70, 0x71, 0xc0, 0xcd, 0xa5, 0x27, 0xcd, 0xc5, 0x48, 0x8b, 0x96, 0x7b, 0xb0, 0xf4, 0x4f, 0x5d, + 0xa4, 0x42, 0x8a, 0x1f, 0x24, 0x8f, 0x9d, 0x37, 0xf8, 0x12, 0xbd, 0x84, 0xcc, 0x91, 0xd9, 0xed, + 0x61, 0x91, 0xb4, 0xb0, 0xf1, 0x64, 0x36, 0xef, 0x89, 0xb0, 0x11, 0x8d, 0x6f, 0x2a, 0xcf, 0xe5, + 0xda, 0x6f, 0x05, 0x0a, 0xe2, 0x96, 0xf1, 0x68, 0x3d, 0x7a, 0x93, 0x3b, 0xb9, 0x03, 0x69, 0xea, + 0x9b, 0xa4, 0x94, 0x11, 0x6e, 0xd6, 0x66, 0x6c, 0xd2, 0x37, 0x49, 0x5c, 0x99, 0x98, 0xe6, 0xa1, + 0x28, 0x33, 0x59, 0x14, 0x6a, 0x7e, 0xd6, 0x50, 0x63, 0xeb, 0xd8, 0x88, 0xc6, 0xd1, 0x3b, 0x80, + 0xe4, 0x78, 0xc5, 0x15, 0xbb, 0x41, 0x43, 0xb1, 0xb3, 0x6b, 0x4a, 0xe8, 0x55, 0xe4, 0x2f, 0x3a, + 0xc1, 0xc2, 0xc6, 0xfa, 0x7f, 0x5c, 0x98, 0x58, 0x2d, 0x9a, 0x5f, 0xfb, 0xa2, 0x00, 0x24, 0xb6, + 0x51, 0x0d, 0x72, 0x6f, 0xc9, 0x01, 0xf1, 0x8e, 0x89, 0x2a, 0x95, 0x97, 0xfa, 0x03, 0x6d, 0x21, + 0x01, 0x63, 0x00, 0x69, 0x90, 0xdd, 0xea, 0x50, 0x4c, 0x98, 0x2a, 0x97, 0x17, 0xfb, 0x03, 0x4d, + 0x4d, 0x28, 0xd1, 0x3e, 0x7a, 0x08, 0xf9, 0x37, 0x01, 0xf6, 0xcd, 0xc0, 0x25, 0x8e, 0xaa, 0x94, + 0xef, 0xf4, 0x07, 0xda, 0xed, 0x84, 0x34, 0x86, 0xd0, 0x0a, 0xcc, 0x45, 0x1f, 0xd8, 0x56, 0x53, + 0xe5, 0xe5, 0xfe, 0x40, 0x43, 0xd3, 0x34, 0x6c, 0xa3, 0x35, 0x28, 0x18, 0xd8, 0xef, 0xba, 0x96, + 0xc9, 0xb8, 0x5e, 0xba, 0x7c, 0xb7, 0x3f, 0xd0, 0x96, 0xae, 0x75, 0x9d, 0x80, 0x5c, 0xb1, 0xc9, + 0x3c, 0x9f, 0xb7, 0xa1, 0x66, 0xa6, 0x15, 0xaf, 0x10, 0x9e, 0x52, 0xac, 0xb1, 0xad, 0x66, 0xa7, + 0x53, 0xc6, 0x40, 0xe3, 0xf5, 0xf9, 0xcf, 0x8a, 0x74, 0x36, 0xaa, 0xc8, 0xe7, 0xa3, 0x8a, 0xfc, + 0x63, 0x54, 0x91, 0x4f, 0x2e, 0x2a, 0xd2, 0xf9, 0x45, 0x45, 0xfa, 0x76, 0x51, 0x91, 0xde, 0xd7, + 0x1d, 0x97, 0xed, 0xf7, 0x3a, 0xba, 0xe5, 0x1d, 0xd6, 0xe3, 0xea, 0xeb, 0x51, 0xf5, 0x75, 0xcb, + 0xb6, 0xea, 0x7f, 0x3d, 0xd7, 0x9d, 0xac, 0x78, 0x6d, 0x9f, 0xfd, 0x09, 0x00, 0x00, 0xff, 0xff, + 0x8b, 0x24, 0x01, 0x79, 0xca, 0x05, 0x00, 0x00, } func (m *Span) Marshal() (dAtA []byte, err error) { @@ -437,6 +447,11 @@ func (m *Checkpoint) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + if m.LastSyncTime != 0 { + i = encodeVarintTable(dAtA, i, uint64(m.LastSyncTime)) + i-- + dAtA[i] = 0x18 + } if m.ResolvedTs != 0 { i = encodeVarintTable(dAtA, i, uint64(m.ResolvedTs)) i-- @@ -618,6 +633,9 @@ func (m *Checkpoint) Size() (n int) { if m.ResolvedTs != 0 { n += 1 + sovTable(uint64(m.ResolvedTs)) } + if m.LastSyncTime != 0 { + n += 1 + sovTable(uint64(m.LastSyncTime)) + } return n } @@ -879,6 +897,25 @@ func (m *Checkpoint) Unmarshal(dAtA []byte) error { break } } + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field LastSyncTime", wireType) + } + m.LastSyncTime = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTable + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.LastSyncTime |= Ts(b&0x7F) << shift + if b < 0x80 { + break + } + } default: iNdEx = preIndex skippy, err := skipTable(dAtA[iNdEx:]) diff --git a/cdc/processor/tablepb/table.proto b/cdc/processor/tablepb/table.proto index 97de3a23e57..d0143cc219c 100644 --- a/cdc/processor/tablepb/table.proto +++ b/cdc/processor/tablepb/table.proto @@ -57,6 +57,7 @@ enum TableState { message Checkpoint { uint64 checkpoint_ts = 1 [(gogoproto.casttype) = "Ts"]; uint64 resolved_ts = 2 [(gogoproto.casttype) = "Ts"]; + uint64 last_sync_time = 3 [(gogoproto.casttype) = "Ts"]; } // Stats holds a statistic for a table. diff --git a/cdc/scheduler/internal/scheduler.go b/cdc/scheduler/internal/scheduler.go index 111911b5632..8e47a779d2a 100644 --- a/cdc/scheduler/internal/scheduler.go +++ b/cdc/scheduler/internal/scheduler.go @@ -47,7 +47,7 @@ type Scheduler interface { // ddl jobs that need to be replicated. The Scheduler will // broadcast the barrierTs to all captures through the Heartbeat. barrier *schedulepb.BarrierWithMinTs, - ) (newCheckpointTs, newResolvedTs model.Ts, err error) + ) (newCheckpointTs, newResolvedTs, newLastSyncTime model.Ts, err error) // MoveTable requests that a table be moved to target. // It is thread-safe. diff --git a/cdc/scheduler/internal/v3/coordinator.go b/cdc/scheduler/internal/v3/coordinator.go index e27bdf49c97..76291c818ab 100644 --- a/cdc/scheduler/internal/v3/coordinator.go +++ b/cdc/scheduler/internal/v3/coordinator.go @@ -126,7 +126,7 @@ func (c *coordinator) Tick( // All captures that are alive according to the latest Etcd states. aliveCaptures map[model.CaptureID]*model.CaptureInfo, barrier *schedulepb.BarrierWithMinTs, -) (newCheckpointTs, newResolvedTs model.Ts, err error) { +) (newCheckpointTs, newResolvedTs, newLastSyncTime model.Ts, err error) { startTime := time.Now() defer func() { costTime := time.Since(startTime) @@ -272,7 +272,7 @@ func (c *coordinator) poll( currentTables []model.TableID, aliveCaptures map[model.CaptureID]*model.CaptureInfo, barrier *schedulepb.BarrierWithMinTs, -) (newCheckpointTs, newResolvedTs model.Ts, err error) { +) (newCheckpointTs, newResolvedTs, newLastSyncTime model.Ts, err error) { c.maybeCollectMetrics() if c.compat.UpdateCaptureInfo(aliveCaptures) { spanReplicationEnabled := c.compat.CheckSpanReplicationEnabled() @@ -283,7 +283,7 @@ func (c *coordinator) poll( recvMsgs, err := c.recvMsgs(ctx) if err != nil { - return checkpointCannotProceed, checkpointCannotProceed, errors.Trace(err) + return checkpointCannotProceed, checkpointCannotProceed, checkpointCannotProceed, errors.Trace(err) } var msgBuf []*schedulepb.Message @@ -295,7 +295,7 @@ func (c *coordinator) poll( // Handle received messages to advance replication set. msgs, err = c.replicationM.HandleMessage(recvMsgs) if err != nil { - return checkpointCannotProceed, checkpointCannotProceed, errors.Trace(err) + return checkpointCannotProceed, checkpointCannotProceed, checkpointCannotProceed, errors.Trace(err) } msgBuf = append(msgBuf, msgs...) @@ -309,13 +309,13 @@ func (c *coordinator) poll( if !c.captureM.CheckAllCaptureInitialized() { // Skip generating schedule tasks for replication manager, // as not all capture are initialized. - newCheckpointTs, newResolvedTs = c.replicationM.AdvanceCheckpoint(&c.tableRanges, pdTime, barrier, c.redoMetaManager) + newCheckpointTs, newResolvedTs, newLastSyncTime = c.replicationM.AdvanceCheckpoint(&c.tableRanges, pdTime, barrier, c.redoMetaManager) // tick capture manager after checkpoint calculation to take account resolvedTs in barrier // when redo is enabled msgs = c.captureM.Tick(c.replicationM.ReplicationSets(), c.schedulerM.DrainingTarget(), barrier.Barrier) msgBuf = append(msgBuf, msgs...) - return newCheckpointTs, newResolvedTs, c.sendMsgs(ctx, msgBuf) + return newCheckpointTs, newResolvedTs, newLastSyncTime, c.sendMsgs(ctx, msgBuf) } // Handle capture membership changes. @@ -323,7 +323,7 @@ func (c *coordinator) poll( msgs, err = c.replicationM.HandleCaptureChanges( changes.Init, changes.Removed, checkpointTs) if err != nil { - return checkpointCannotProceed, checkpointCannotProceed, errors.Trace(err) + return checkpointCannotProceed, checkpointCannotProceed, checkpointCannotProceed, errors.Trace(err) } msgBuf = append(msgBuf, msgs...) } @@ -339,12 +339,12 @@ func (c *coordinator) poll( // Handle generated schedule tasks. msgs, err = c.replicationM.HandleTasks(allTasks) if err != nil { - return checkpointCannotProceed, checkpointCannotProceed, errors.Trace(err) + return checkpointCannotProceed, checkpointCannotProceed, checkpointCannotProceed, errors.Trace(err) } msgBuf = append(msgBuf, msgs...) // Checkpoint calculation - newCheckpointTs, newResolvedTs = c.replicationM.AdvanceCheckpoint(&c.tableRanges, pdTime, barrier, c.redoMetaManager) + newCheckpointTs, newResolvedTs, newLastSyncTime = c.replicationM.AdvanceCheckpoint(&c.tableRanges, pdTime, barrier, c.redoMetaManager) // tick capture manager after checkpoint calculation to take account resolvedTs in barrier // when redo is enabled @@ -355,10 +355,10 @@ func (c *coordinator) poll( // Send new messages. err = c.sendMsgs(ctx, msgBuf) if err != nil { - return checkpointCannotProceed, checkpointCannotProceed, errors.Trace(err) + return checkpointCannotProceed, checkpointCannotProceed, checkpointCannotProceed, errors.Trace(err) } - return newCheckpointTs, newResolvedTs, nil + return newCheckpointTs, newResolvedTs, newLastSyncTime, nil } func (c *coordinator) recvMsgs(ctx context.Context) ([]*schedulepb.Message, error) { diff --git a/cdc/scheduler/internal/v3/replication/replication_manager.go b/cdc/scheduler/internal/v3/replication/replication_manager.go index 4228cc0ec75..6e2fdea5e19 100644 --- a/cdc/scheduler/internal/v3/replication/replication_manager.go +++ b/cdc/scheduler/internal/v3/replication/replication_manager.go @@ -557,9 +557,9 @@ func (r *Manager) AdvanceCheckpoint( currentPDTime time.Time, barrier *schedulepb.BarrierWithMinTs, redoMetaManager redo.MetaManager, -) (newCheckpointTs, newResolvedTs model.Ts) { +) (newCheckpointTs, newResolvedTs model.Ts, newLastSyncTime model.Ts) { var redoFlushedResolvedTs model.Ts - limitBarrierWithRedo := func(newCheckpointTs, newResolvedTs uint64) (uint64, uint64) { + limitBarrierWithRedo := func(newCheckpointTs, newResolvedTs, newLastSyncTime uint64) (uint64, uint64, uint64) { flushedMeta := redoMetaManager.GetFlushedMeta() redoFlushedResolvedTs = flushedMeta.ResolvedTs log.Debug("owner gets flushed redo meta", @@ -578,7 +578,7 @@ func (r *Manager) AdvanceCheckpoint( if barrier.GlobalBarrierTs > newResolvedTs { barrier.GlobalBarrierTs = newResolvedTs } - return newCheckpointTs, newResolvedTs + return newCheckpointTs, newResolvedTs, newLastSyncTime } defer func() { if redoFlushedResolvedTs != 0 && barrier.GlobalBarrierTs > redoFlushedResolvedTs { @@ -594,7 +594,7 @@ func (r *Manager) AdvanceCheckpoint( r.slowestSink = tablepb.Span{} var slowestPullerResolvedTs uint64 = math.MaxUint64 - newCheckpointTs, newResolvedTs = math.MaxUint64, math.MaxUint64 + newCheckpointTs, newResolvedTs, newLastSyncTime = math.MaxUint64, math.MaxUint64, 0 cannotProceed := false currentTables.Iter(func(tableID model.TableID, tableStart, tableEnd tablepb.Span) bool { tableSpanFound, tableHasHole := false, false @@ -629,6 +629,9 @@ func (r *Manager) AdvanceCheckpoint( newResolvedTs = table.Checkpoint.ResolvedTs } + if newLastSyncTime < table.Checkpoint.LastSyncTime { + newLastSyncTime = table.Checkpoint.LastSyncTime + } // Find the minimum puller resolved ts. if pullerCkpt, ok := table.Stats.StageCheckpoints["puller-egress"]; ok { if slowestPullerResolvedTs > pullerCkpt.ResolvedTs { @@ -664,9 +667,9 @@ func (r *Manager) AdvanceCheckpoint( if redoMetaManager.Enabled() { // If redo is enabled, GlobalBarrierTs should be limited by redo flushed meta. newResolvedTs = barrier.RedoBarrierTs - limitBarrierWithRedo(newCheckpointTs, newResolvedTs) + limitBarrierWithRedo(newCheckpointTs, newResolvedTs, newLastSyncTime) } - return checkpointCannotProceed, checkpointCannotProceed + return checkpointCannotProceed, checkpointCannotProceed, checkpointCannotProceed } // If currentTables is empty, we should advance newResolvedTs to global barrier ts and @@ -714,10 +717,10 @@ func (r *Manager) AdvanceCheckpoint( zap.String("changefeed", r.changefeedID.ID), zap.Uint64("newCheckpointTs", newCheckpointTs), zap.Uint64("newResolvedTs", newResolvedTs)) - return limitBarrierWithRedo(newCheckpointTs, newResolvedTs) + return limitBarrierWithRedo(newCheckpointTs, newResolvedTs, newLastSyncTime) } - return newCheckpointTs, newResolvedTs + return newCheckpointTs, newResolvedTs, newLastSyncTime } func (r *Manager) logSlowTableInfo(currentPDTime time.Time) { diff --git a/cdc/scheduler/internal/v3/replication/replication_set.go b/cdc/scheduler/internal/v3/replication/replication_set.go index fec432499a8..d99a297e65c 100644 --- a/cdc/scheduler/internal/v3/replication/replication_set.go +++ b/cdc/scheduler/internal/v3/replication/replication_set.go @@ -1020,6 +1020,10 @@ func (r *ReplicationSet) updateCheckpointAndStats( zap.Any("checkpointTs", r.Checkpoint.CheckpointTs), zap.Any("resolvedTs", r.Checkpoint.ResolvedTs)) } + + if r.Checkpoint.LastSyncTime < checkpoint.LastSyncTime { + r.Checkpoint.LastSyncTime = checkpoint.LastSyncTime + } r.Stats = stats } diff --git a/cdc/sink/dmlsink/cloudstorage/cloud_storage_dml_sink.go b/cdc/sink/dmlsink/cloudstorage/cloud_storage_dml_sink.go index 14043124524..b3e235578b0 100644 --- a/cdc/sink/dmlsink/cloudstorage/cloud_storage_dml_sink.go +++ b/cdc/sink/dmlsink/cloudstorage/cloud_storage_dml_sink.go @@ -224,6 +224,7 @@ func (s *DMLSink) WriteEvents(txns ...*dmlsink.CallbackableEvent[*model.SingleTa if txn.GetTableSinkState() != state.TableSinkSinking { // The table where the event comes from is in stopping, so it's safe // to drop the event directly. + // 这个状态是在干嘛的? txn.Callback() continue } diff --git a/cdc/sink/tablesink/table_sink.go b/cdc/sink/tablesink/table_sink.go index 588b30aabf9..6dac7c87417 100644 --- a/cdc/sink/tablesink/table_sink.go +++ b/cdc/sink/tablesink/table_sink.go @@ -32,6 +32,9 @@ type TableSink interface { // For example, calculating the current progress from the statistics of the table sink. // This is a thread-safe method. GetCheckpointTs() model.ResolvedTs + // GetLastSyncTime returns the lastSyncTime of table sink. + // This is a thread-safe method. + GetLastSyncTime() uint64 // Close closes the table sink. // After it returns, no more events will be sent out from this capture. Close() diff --git a/cdc/sink/tablesink/table_sink_impl.go b/cdc/sink/tablesink/table_sink_impl.go index da067501727..76908c9fac0 100644 --- a/cdc/sink/tablesink/table_sink_impl.go +++ b/cdc/sink/tablesink/table_sink_impl.go @@ -15,6 +15,7 @@ package tablesink import ( "sort" + "sync" "github.com/pingcap/log" "github.com/pingcap/tiflow/cdc/model" @@ -31,7 +32,13 @@ var ( _ TableSink = (*EventTableSink[*model.SingleTableTxn, *dmlsink.TxnEventAppender])(nil) ) +type LastSyncTimeRecord struct { + sync.Mutex + lastSyncTime uint64 +} + // EventTableSink is a table sink that can write events. + type EventTableSink[E dmlsink.TableEvent, P dmlsink.Appender[E]] struct { changefeedID model.ChangeFeedID span tablepb.Span @@ -46,6 +53,8 @@ type EventTableSink[E dmlsink.TableEvent, P dmlsink.Appender[E]] struct { eventBuffer []E state state.TableSinkState + lastSyncTime LastSyncTimeRecord + // For dataflow metrics. metricsTableSinkTotalRows prometheus.Counter } @@ -69,6 +78,7 @@ func New[E dmlsink.TableEvent, P dmlsink.Appender[E]]( eventAppender: appender, eventBuffer: make([]E, 0, 1024), state: state.TableSinkSinking, + lastSyncTime: LastSyncTimeRecord{lastSyncTime: 0}, // use 0 to initialize lastSyncTime metricsTableSinkTotalRows: totalRowsCounter, } } @@ -115,8 +125,20 @@ func (e *EventTableSink[E, P]) UpdateResolvedTs(resolvedTs model.ResolvedTs) err } // We have to record the event ID for the callback. ce := &dmlsink.CallbackableEvent[E]{ - Event: ev, - Callback: e.progressTracker.addEvent(), + Event: ev, + Callback: func() { + // Due to multi workers will call this callback concurrently, + // we need to add lock to protect lastSyncTime + // we need make a performance test for it + { + e.lastSyncTime.Lock() + defer e.lastSyncTime.Unlock() + if e.lastSyncTime.lastSyncTime < ev.GetCommitTs() { + e.lastSyncTime.lastSyncTime = ev.GetCommitTs() + } + } + e.progressTracker.addEvent() + }, SinkState: &e.state, } resolvedCallbackableEvents = append(resolvedCallbackableEvents, ce) @@ -140,6 +162,13 @@ func (e *EventTableSink[E, P]) GetCheckpointTs() model.ResolvedTs { return e.progressTracker.advance() } +// GetLastSyncTime returns the lastSyncTime ts of the table sink. +func (e *EventTableSink[E, P]) GetLastSyncTime() uint64 { + e.lastSyncTime.Lock() + defer e.lastSyncTime.Unlock() + return e.lastSyncTime.lastSyncTime +} + // Close closes the table sink. // After it returns, no more events will be sent out from this capture. func (e *EventTableSink[E, P]) Close() { From 73bc607bf4e0fa092f7686b35e82126f9bdb3ad4 Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Wed, 22 Nov 2023 16:19:44 +0800 Subject: [PATCH 02/35] update code --- cdc/owner/changefeed.go | 36 +++++++++++++++++++++--------------- cdc/owner/owner.go | 4 ++-- 2 files changed, 23 insertions(+), 17 deletions(-) diff --git a/cdc/owner/changefeed.go b/cdc/owner/changefeed.go index 47e80d954e4..6d4e68f55ef 100644 --- a/cdc/owner/changefeed.go +++ b/cdc/owner/changefeed.go @@ -273,8 +273,7 @@ func (c *changefeed) Tick(ctx cdcContext.Context, } return nil }) - checkpointTs, minTableBarrierTs, maxLastSyncTime, err := c.tick(ctx, captures) - c.lastSyncTime = maxLastSyncTime + checkpointTs, minTableBarrierTs, err := c.tick(ctx, captures) // The tick duration is recorded only if changefeed has completed initialization if c.initialized { @@ -352,50 +351,50 @@ func (c *changefeed) checkStaleCheckpointTs(ctx cdcContext.Context, // tick returns the checkpointTs and minTableBarrierTs. func (c *changefeed) tick(ctx cdcContext.Context, captures map[model.CaptureID]*model.CaptureInfo, -) (model.Ts, model.Ts, model.Ts, error) { +) (model.Ts, model.Ts, error) { adminJobPending := c.feedStateManager.Tick(c.resolvedTs, c.latestStatus, c.latestInfo) preCheckpointTs := c.latestInfo.GetCheckpointTs(c.latestStatus) // checkStaleCheckpointTs must be called before `feedStateManager.ShouldRunning()` // to ensure all changefeeds, no matter whether they are running or not, will be checked. if err := c.checkStaleCheckpointTs(ctx, c.latestInfo, preCheckpointTs); err != nil { - return 0, 0, 0, errors.Trace(err) + return 0, 0, errors.Trace(err) } if !c.feedStateManager.ShouldRunning() { c.isRemoved = c.feedStateManager.ShouldRemoved() c.releaseResources(ctx) - return 0, 0, 0, nil + return 0, 0, nil } if adminJobPending { - return 0, 0, 0, nil + return 0, 0, nil } if err := c.initialize(ctx); err != nil { - return 0, 0, 0, errors.Trace(err) + return 0, 0, errors.Trace(err) } select { case err := <-c.errCh: - return 0, 0, 0, errors.Trace(err) + return 0, 0, errors.Trace(err) default: } if c.redoMetaMgr.Enabled() { if !c.redoMetaMgr.Running() { - return 0, 0, 0, nil + return 0, 0, nil } } // TODO: pass table checkpointTs when we support concurrent process ddl allPhysicalTables, barrier, err := c.ddlManager.tick(ctx, preCheckpointTs, nil) if err != nil { - return 0, 0, 0, errors.Trace(err) + return 0, 0, errors.Trace(err) } err = c.handleBarrier(ctx, c.latestInfo, c.latestStatus, barrier) if err != nil { - return 0, 0, 0, errors.Trace(err) + return 0, 0, errors.Trace(err) } log.Debug("owner handles barrier", @@ -411,14 +410,21 @@ func (c *changefeed) tick(ctx cdcContext.Context, // This condition implies that the DDL resolved-ts has not yet reached checkpointTs, // which implies that it would be premature to schedule tables or to update status. // So we return here. - return 0, 0, 0, nil + return 0, 0, nil } newCheckpointTs, newResolvedTs, newLastSyncTime, err := c.scheduler.Tick( ctx, preCheckpointTs, allPhysicalTables, captures, barrier) if err != nil { - return 0, 0, 0, errors.Trace(err) + return 0, 0, errors.Trace(err) + } + if c.lastSyncTime > newLastSyncTime { + c.lastSyncTime = newLastSyncTime + } else { + log.Warn("lastSyncTime should not be greater than newLastSyncTime", + zap.Uint64("c.lastSyncTime", c.lastSyncTime), + zap.Uint64("newLastSyncTime", newLastSyncTime)) } pdTime := c.upstream.PDClock.CurrentTime() @@ -432,7 +438,7 @@ func (c *changefeed) tick(ctx cdcContext.Context, // advance the watermarks for now. c.updateMetrics(currentTs, c.latestStatus.CheckpointTs, c.resolvedTs) } - return 0, 0, 0, nil + return 0, 0, nil } log.Debug("owner prepares to update status", @@ -465,7 +471,7 @@ func (c *changefeed) tick(ctx cdcContext.Context, c.updateMetrics(currentTs, newCheckpointTs, c.resolvedTs) c.tickDownstreamObserver(ctx) - return newCheckpointTs, barrier.MinTableBarrierTs, newLastSyncTime, nil + return newCheckpointTs, barrier.MinTableBarrierTs, nil } func (c *changefeed) initialize(ctx cdcContext.Context) (err error) { diff --git a/cdc/owner/owner.go b/cdc/owner/owner.go index 166cfc78df1..c2972ec8a05 100644 --- a/cdc/owner/owner.go +++ b/cdc/owner/owner.go @@ -587,7 +587,7 @@ func (o *ownerImpl) handleJobs(ctx context.Context) { func (o *ownerImpl) handleQueries(query *Query) error { switch query.Tp { - case QueryChangeFeedSyncedStatus: + case QueryChangeFeedStatuses: cfReactor, ok := o.changefeeds[query.ChangeFeedID] if !ok { query.Data = nil @@ -597,7 +597,7 @@ func (o *ownerImpl) handleQueries(query *Query) error { ret.ResolvedTs = cfReactor.resolvedTs ret.CheckpointTs = cfReactor.latestStatus.CheckpointTs query.Data = ret - case QueryChangeFeedStatuses: + case QueryChangeFeedSyncedStatus: cfReactor, ok := o.changefeeds[query.ChangeFeedID] if !ok { query.Data = nil From 6b92aa4a845e95dd87ecca488ddb84f24059bb1b Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Thu, 23 Nov 2023 16:03:36 +0800 Subject: [PATCH 03/35] fix code --- cdc/api/v2/changefeed.go | 76 +++++++----- cdc/model/changefeed.go | 7 +- cdc/owner/changefeed.go | 47 ++++++-- cdc/owner/owner.go | 3 +- cdc/processor/processor.go | 2 +- cdc/processor/sinkmanager/manager.go | 7 +- .../sinkmanager/table_sink_wrapper.go | 4 +- cdc/processor/tablepb/table.pb.go | 112 +++++++++--------- cdc/processor/tablepb/table.proto | 2 +- cdc/scheduler/internal/scheduler.go | 2 +- cdc/scheduler/internal/v3/coordinator.go | 24 ++-- .../v3/replication/replication_manager.go | 33 ++++-- .../v3/replication/replication_set.go | 4 +- .../cloudstorage/cloud_storage_dml_sink.go | 1 - cdc/sink/tablesink/table_sink.go | 6 +- cdc/sink/tablesink/table_sink_impl.go | 36 +++--- 16 files changed, 218 insertions(+), 148 deletions(-) diff --git a/cdc/api/v2/changefeed.go b/cdc/api/v2/changefeed.go index cbbbc8550ff..3d873d0b151 100644 --- a/cdc/api/v2/changefeed.go +++ b/cdc/api/v2/changefeed.go @@ -29,6 +29,7 @@ import ( "github.com/pingcap/tiflow/cdc/api" "github.com/pingcap/tiflow/cdc/capture" "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/dm/pkg/terror" "github.com/pingcap/tiflow/pkg/config" cerror "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/retry" @@ -935,42 +936,63 @@ func (h *OpenAPIV2) synced(c *gin.Context) { defer cancel() pdClient, err := h.helpers.getPDClient(timeoutCtx, cfg.PDAddrs, credential) defer pdClient.Close() - if err != nil { - // _ = c.Error(cerror.WrapError(cerror.ErrAPIGetPDClientFailed, err)) - // return - // means pd is offline + + if err != nil { // pd 不可用 + var message string + if (status.PullerIngressResolvedTs - status.CheckpointTs) > (5*1000)<<18 { // 5s + message = fmt.Sprintf("we get pd client failed with err is %s. Besides the data is not finish syncing", terror.Message(err)) + } else { + message = fmt.Sprintf("we get pd client failed with err is %s. "+ + "You can check the pd first, and if pd is available, means we don't finish sync data. "+ + "If pd is not available, please check the whether we satisfy the condition that"+ + "The time difference from lastSyncedTs to the current time from the time zone of pd is greater than 5 min"+ + "If it's satisfied, means the data syncing is totally finished", err) + } c.JSON(http.StatusOK, map[string]any{ - "Synced": false, - "CheckpointTs": status.CheckpointTs, - "ResolvedTs": status.ResolvedTs, - "LastSyncTime": status.LastSyncTime, - "info": "xxxxx", + "Synced": false, + "Sink-CheckpointTs": status.CheckpointTs, + "Puller-ResolvedTs": status.PullerIngressResolvedTs, + "LastSyncedTs": status.LastSyncedTs, + "info": message, }) - return } - //TSO 是啥? - now, _, _ := pdClient.GetTS(ctx) - // get pd_now - // 随便写一个先 - if (now-int64(status.LastSyncTime) > 60*5) && (now-int64(status.CheckpointTs) < 5) { + physical, logical, _ := pdClient.GetTS(ctx) + now := oracle.ComposeTS(physical, logical) + + if (now-status.LastSyncedTs > (5*60*1000)<<18) && (now-status.CheckpointTs < (5*1000)<<18) { // 达到 synced 严格条件 c.JSON(http.StatusOK, map[string]any{ - "Synced": true, - "CheckpointTs": status.CheckpointTs, - "ResolvedTs": status.ResolvedTs, - "LastSyncTime": status.LastSyncTime, - "info": "", + "Synced": true, + "Sink-CheckpointTs": status.CheckpointTs, + "Puller-ResolvedTs": status.PullerIngressResolvedTs, + "LastSyncedTs": status.LastSyncedTs, + "info": "Data syncing is finished", }) - } else { + } else if now-status.LastSyncedTs > (5*60*1000)<<18 { // lastSyncedTs 条件达到,checkpoint-ts 未达到 + var message string + if (status.PullerIngressResolvedTs - status.CheckpointTs) > (5*1000)<<18 { // 5s + message = fmt.Sprintf("Please check whether pd is health and tikv region is all available. " + + "If pd is not health or tikv region is not available, the data syncing is finished. " + + " Otherwise the data syncing is not finished, please wait") + } else { + message = fmt.Sprintf("The data syncing is not finished, please wait") + } c.JSON(http.StatusOK, map[string]any{ - "Synced": false, - "CheckpointTs": status.CheckpointTs, - "ResolvedTs": status.ResolvedTs, - "LastSyncTime": status.LastSyncTime, - "info": "xxxxx", + "Synced": false, + "Sink-CheckpointTs": status.CheckpointTs, + "Puller-ResolvedTs": status.PullerIngressResolvedTs, + "LastSyncedTs": status.LastSyncedTs, + "info": message, + }) + } else { // lastSyncedTs 条件达到 + c.JSON(http.StatusOK, map[string]any{ + "Synced": false, + "Sink-CheckpointTs": status.CheckpointTs, + "Puller-ResolvedTs": status.PullerIngressResolvedTs, + "LastSyncedTs": status.LastSyncedTs, + "info": "The data syncing is not finished, please wait", }) } - } func toAPIModel( diff --git a/cdc/model/changefeed.go b/cdc/model/changefeed.go index 34a5aff5c3e..91d3a2111ce 100644 --- a/cdc/model/changefeed.go +++ b/cdc/model/changefeed.go @@ -631,7 +631,8 @@ type ChangeFeedStatusForAPI struct { } type ChangeFeedSyncedStatusForAPI struct { - ResolvedTs uint64 `json:"resolved-ts"` - CheckpointTs uint64 `json:"checkpoint-ts"` - LastSyncTime uint64 `json:"last-sync-time"` + ResolvedTs uint64 `json:"resolved-ts"` + CheckpointTs uint64 `json:"checkpoint-ts"` + LastSyncedTs uint64 `json:"last-sync-time"` + PullerIngressResolvedTs uint64 `json:"puller-ingress-resolved-ts"` } diff --git a/cdc/owner/changefeed.go b/cdc/owner/changefeed.go index 6d4e68f55ef..a78b9ce29eb 100644 --- a/cdc/owner/changefeed.go +++ b/cdc/owner/changefeed.go @@ -90,10 +90,11 @@ type changefeed struct { scheduler scheduler.Scheduler // barriers will be created when a changefeed is initialized // and will be destroyed when a changefeed is closed. - barriers *barriers - feedStateManager FeedStateManager - resolvedTs model.Ts - lastSyncTime model.Ts + barriers *barriers + feedStateManager FeedStateManager + resolvedTs model.Ts + lastSyncedTs model.Ts + pullerIngressResolvedTs model.Ts // record the latest min puller ingress resolved ts of all pullers // ddl related fields ddlManager *ddlManager @@ -413,18 +414,28 @@ func (c *changefeed) tick(ctx cdcContext.Context, return 0, 0, nil } - newCheckpointTs, newResolvedTs, newLastSyncTime, err := c.scheduler.Tick( + newCheckpointTs, newResolvedTs, newLastSyncedTs, newPullerIngressResolvedTs, err := c.scheduler.Tick( ctx, preCheckpointTs, allPhysicalTables, captures, barrier) if err != nil { return 0, 0, errors.Trace(err) } - if c.lastSyncTime > newLastSyncTime { - c.lastSyncTime = newLastSyncTime - } else { - log.Warn("lastSyncTime should not be greater than newLastSyncTime", - zap.Uint64("c.lastSyncTime", c.lastSyncTime), - zap.Uint64("newLastSyncTime", newLastSyncTime)) + if c.lastSyncedTs < newLastSyncedTs { + c.lastSyncedTs = newLastSyncedTs + } else if c.lastSyncedTs > newLastSyncedTs { + log.Warn("LastSyncedTs should not be greater than newLastSyncedTs", + zap.Uint64("c.LastSyncedTs", c.lastSyncedTs), + zap.Uint64("newLastSyncedTs", newLastSyncedTs)) + } + + if newPullerIngressResolvedTs != scheduler.CheckpointCannotProceed { + if newPullerIngressResolvedTs > c.pullerIngressResolvedTs { + c.pullerIngressResolvedTs = newPullerIngressResolvedTs + } else if newPullerIngressResolvedTs < c.pullerIngressResolvedTs { + log.Warn("the newPullerIngressResolvedTs should not be smaller than c.PullerIngressResolvedTs", + zap.Uint64("c.pullerIngressResolvedTs", c.pullerIngressResolvedTs), + zap.Uint64("newPullerIngressResolvedTs", newPullerIngressResolvedTs)) + } } pdTime := c.upstream.PDClock.CurrentTime() @@ -508,6 +519,20 @@ LOOP2: if c.resolvedTs == 0 { c.resolvedTs = checkpointTs } + + if c.lastSyncedTs == 0 { + // Set LastSyncedTs with current pd time when do initialize. + + // we don't save lastSyncedTs in etcd because we want to reduce the number of etcd write. + // Based on the assumption that owner will not be replaced frequently, + // and we only change owners when oom or some panic happens, + // use pd time to initialize lastSyncedTs can work well enough. + // Even if there are no more data send into ticdc after changing owner, + // we just need to wait synced-check-time to reach synced = true. + // We regard the situation never happens that owner is replaced frequently and then leading to + // lastSyncedTs always increase even if there are no more data send into ticdc. + c.lastSyncedTs = uint64(oracle.GetPhysical(c.upstream.PDClock.CurrentTime())) + } minTableBarrierTs := c.latestStatus.MinTableBarrierTs failpoint.Inject("NewChangefeedNoRetryError", func() { diff --git a/cdc/owner/owner.go b/cdc/owner/owner.go index c2972ec8a05..747b40bb0a7 100644 --- a/cdc/owner/owner.go +++ b/cdc/owner/owner.go @@ -605,8 +605,9 @@ func (o *ownerImpl) handleQueries(query *Query) error { } ret := &model.ChangeFeedSyncedStatusForAPI{} ret.ResolvedTs = cfReactor.resolvedTs - ret.LastSyncTime = cfReactor.lastSyncTime + ret.LastSyncedTs = cfReactor.lastSyncedTs ret.CheckpointTs = cfReactor.latestStatus.CheckpointTs + ret.PullerIngressResolvedTs = cfReactor.pullerIngressResolvedTs query.Data = ret case QueryChangefeedInfo: cfReactor, ok := o.changefeeds[query.ChangeFeedID] diff --git a/cdc/processor/processor.go b/cdc/processor/processor.go index 269eadfcf1d..c1440bd6e15 100644 --- a/cdc/processor/processor.go +++ b/cdc/processor/processor.go @@ -366,7 +366,7 @@ func (p *processor) GetTableSpanStatus(span tablepb.Span, collectStat bool) tabl Checkpoint: tablepb.Checkpoint{ CheckpointTs: sinkStats.CheckpointTs, ResolvedTs: sinkStats.ResolvedTs, - LastSyncTime: sinkStats.LastSyncTime, + LastSyncedTs: sinkStats.LastSyncedTs, }, State: state, Stats: stats, diff --git a/cdc/processor/sinkmanager/manager.go b/cdc/processor/sinkmanager/manager.go index c34ec1bc3ed..ee96d3caddd 100644 --- a/cdc/processor/sinkmanager/manager.go +++ b/cdc/processor/sinkmanager/manager.go @@ -58,7 +58,7 @@ const ( type TableStats struct { CheckpointTs model.Ts ResolvedTs model.Ts - LastSyncTime model.Ts + LastSyncedTs model.Ts BarrierTs model.Ts } @@ -141,7 +141,6 @@ func New( redoDMLMgr redo.DMLManager, sourceManager *sourcemanager.SourceManager, ) *SinkManager { - m := &SinkManager{ changefeedID: changefeedID, changefeedInfo: changefeedInfo, @@ -963,7 +962,7 @@ func (m *SinkManager) GetTableStats(span tablepb.Span) TableStats { tableSink := value.(*tableSinkWrapper) checkpointTs := tableSink.getCheckpointTs() - lastSyncTime := tableSink.getLastSyncTime() + lastSyncedTs := tableSink.getLastSyncedTs() m.sinkMemQuota.Release(span, checkpointTs) m.redoMemQuota.Release(span, checkpointTs) @@ -1006,7 +1005,7 @@ func (m *SinkManager) GetTableStats(span tablepb.Span) TableStats { return TableStats{ CheckpointTs: checkpointTs.ResolvedMark(), ResolvedTs: resolvedTs, - LastSyncTime: lastSyncTime, + LastSyncedTs: lastSyncedTs, BarrierTs: tableSink.barrierTs.Load(), } } diff --git a/cdc/processor/sinkmanager/table_sink_wrapper.go b/cdc/processor/sinkmanager/table_sink_wrapper.go index ac946ea1446..98129e7c032 100644 --- a/cdc/processor/sinkmanager/table_sink_wrapper.go +++ b/cdc/processor/sinkmanager/table_sink_wrapper.go @@ -218,10 +218,10 @@ func (t *tableSinkWrapper) updateResolvedTs(ts model.ResolvedTs) error { return t.tableSink.s.UpdateResolvedTs(ts) } -func (t *tableSinkWrapper) getLastSyncTime() uint64 { +func (t *tableSinkWrapper) getLastSyncedTs() uint64 { t.tableSink.RLock() defer t.tableSink.RUnlock() - return t.tableSink.s.GetLastSyncTime() + return t.tableSink.s.GetLastSyncedTs() } func (t *tableSinkWrapper) getCheckpointTs() model.ResolvedTs { diff --git a/cdc/processor/tablepb/table.pb.go b/cdc/processor/tablepb/table.pb.go index 001685818e8..c4b86e83406 100644 --- a/cdc/processor/tablepb/table.pb.go +++ b/cdc/processor/tablepb/table.pb.go @@ -117,7 +117,7 @@ var xxx_messageInfo_Span proto.InternalMessageInfo type Checkpoint struct { CheckpointTs Ts `protobuf:"varint,1,opt,name=checkpoint_ts,json=checkpointTs,proto3,casttype=Ts" json:"checkpoint_ts,omitempty"` ResolvedTs Ts `protobuf:"varint,2,opt,name=resolved_ts,json=resolvedTs,proto3,casttype=Ts" json:"resolved_ts,omitempty"` - LastSyncTime Ts `protobuf:"varint,3,opt,name=last_sync_time,json=lastSyncTime,proto3,casttype=Ts" json:"last_sync_time,omitempty"` + LastSyncedTs Ts `protobuf:"varint,3,opt,name=last_synced_ts,json=lastSyncedTs,proto3,casttype=Ts" json:"last_synced_ts,omitempty"` } func (m *Checkpoint) Reset() { *m = Checkpoint{} } @@ -167,9 +167,9 @@ func (m *Checkpoint) GetResolvedTs() Ts { return 0 } -func (m *Checkpoint) GetLastSyncTime() Ts { +func (m *Checkpoint) GetLastSyncedTs() Ts { if m != nil { - return m.LastSyncTime + return m.LastSyncedTs } return 0 } @@ -337,52 +337,52 @@ func init() { func init() { proto.RegisterFile("processor/tablepb/table.proto", fileDescriptor_ae83c9c6cf5ef75c) } var fileDescriptor_ae83c9c6cf5ef75c = []byte{ - // 712 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x54, 0xbf, 0x6f, 0xd3, 0x40, - 0x18, 0xb5, 0x9d, 0x5f, 0xcd, 0x97, 0x50, 0xb9, 0x47, 0x5b, 0x42, 0x24, 0x12, 0x13, 0x15, 0xa8, - 0x5a, 0xe4, 0x40, 0x59, 0x50, 0xb7, 0xa6, 0x05, 0x54, 0x55, 0x48, 0xc8, 0x09, 0x0c, 0x2c, 0x91, - 0x63, 0x1f, 0xae, 0xd5, 0xf4, 0x6c, 0xf9, 0x2e, 0xad, 0xbc, 0x31, 0xa2, 0x2c, 0x74, 0x42, 0x2c, - 0x91, 0xfa, 0xe7, 0x74, 0xec, 0xc8, 0x80, 0x22, 0x48, 0xc5, 0xcc, 0xde, 0x09, 0xdd, 0xd9, 0x8d, - 0x9b, 0xc0, 0x10, 0xba, 0x24, 0xe7, 0x7b, 0xef, 0x7b, 0x7a, 0xef, 0xdd, 0xe9, 0xe0, 0x9e, 0x1f, - 0x78, 0x16, 0xa6, 0xd4, 0x0b, 0xea, 0xcc, 0xec, 0x74, 0xb1, 0xdf, 0x89, 0xfe, 0x75, 0x3f, 0xf0, - 0x98, 0x87, 0x56, 0x7c, 0x97, 0x38, 0x96, 0xe9, 0xeb, 0xcc, 0xfd, 0xd0, 0xf5, 0x8e, 0x75, 0xcb, - 0xb6, 0xf4, 0xf1, 0x84, 0x1e, 0x4f, 0x94, 0x17, 0x1d, 0xcf, 0xf1, 0xc4, 0x40, 0x9d, 0xaf, 0xa2, - 0xd9, 0xda, 0x67, 0x19, 0xd2, 0x4d, 0xdf, 0x24, 0xe8, 0x29, 0xcc, 0x09, 0x66, 0xdb, 0xb5, 0x4b, - 0xb2, 0x26, 0xaf, 0xa6, 0x1a, 0xcb, 0xa3, 0x61, 0x35, 0xd7, 0xe2, 0x7b, 0xbb, 0x3b, 0x97, 0xc9, - 0xd2, 0xc8, 0x09, 0xde, 0xae, 0x8d, 0x56, 0x20, 0x4f, 0x99, 0x19, 0xb0, 0xf6, 0x01, 0x0e, 0x4b, - 0x8a, 0x26, 0xaf, 0x16, 0x1b, 0xb9, 0xcb, 0x61, 0x35, 0xb5, 0x87, 0x43, 0x63, 0x4e, 0x20, 0x7b, - 0x38, 0x44, 0x1a, 0xe4, 0x30, 0xb1, 0x05, 0x27, 0x35, 0xc9, 0xc9, 0x62, 0x62, 0xef, 0xe1, 0x70, - 0xb3, 0xf8, 0xe9, 0xb4, 0x2a, 0x7d, 0x3d, 0xad, 0x4a, 0x1f, 0xbf, 0x6b, 0x52, 0xed, 0x44, 0x06, - 0xd8, 0xde, 0xc7, 0xd6, 0x81, 0xef, 0xb9, 0x84, 0xa1, 0x75, 0xb8, 0x65, 0x8d, 0xbf, 0xda, 0x8c, - 0x0a, 0x73, 0xe9, 0x46, 0xf6, 0x72, 0x58, 0x55, 0x5a, 0xd4, 0x28, 0x26, 0x60, 0x8b, 0xa2, 0x47, - 0x50, 0x08, 0x30, 0xf5, 0xba, 0x47, 0xd8, 0xe6, 0x54, 0x65, 0x82, 0x0a, 0x57, 0x50, 0x8b, 0xa2, - 0xc7, 0x30, 0xdf, 0x35, 0x29, 0x6b, 0xd3, 0x90, 0x58, 0x6d, 0xe6, 0x1e, 0x62, 0xe1, 0xed, 0x9a, - 0x2c, 0x47, 0x9b, 0x21, 0xb1, 0x5a, 0xee, 0x21, 0xae, 0xfd, 0x52, 0x20, 0xd3, 0x64, 0x26, 0xa3, - 0xe8, 0x3e, 0x14, 0x03, 0xec, 0xb8, 0x1e, 0x69, 0x5b, 0x5e, 0x8f, 0xb0, 0xc8, 0x8c, 0x51, 0x88, - 0xf6, 0xb6, 0xf9, 0x16, 0x7a, 0x00, 0x60, 0xf5, 0x82, 0x00, 0x47, 0x6e, 0x27, 0x2d, 0xe4, 0x63, - 0xa4, 0x45, 0x11, 0x83, 0x05, 0xca, 0x4c, 0x07, 0xb7, 0x93, 0x00, 0xb4, 0x94, 0xd2, 0x52, 0xab, - 0x85, 0x8d, 0x2d, 0x7d, 0x96, 0x03, 0xd5, 0x85, 0x23, 0xfe, 0xeb, 0xe0, 0xa4, 0x2f, 0xfa, 0x82, - 0xb0, 0x20, 0x6c, 0xa4, 0xcf, 0x86, 0x55, 0xc9, 0x50, 0xe9, 0x14, 0xc8, 0xcd, 0x75, 0xcc, 0x20, - 0x70, 0x71, 0xc0, 0xcd, 0xa5, 0x27, 0xcd, 0xc5, 0x48, 0x8b, 0x96, 0x7b, 0xb0, 0xf4, 0x4f, 0x5d, - 0xa4, 0x42, 0x8a, 0x1f, 0x24, 0x8f, 0x9d, 0x37, 0xf8, 0x12, 0xbd, 0x84, 0xcc, 0x91, 0xd9, 0xed, - 0x61, 0x91, 0xb4, 0xb0, 0xf1, 0x64, 0x36, 0xef, 0x89, 0xb0, 0x11, 0x8d, 0x6f, 0x2a, 0xcf, 0xe5, - 0xda, 0x6f, 0x05, 0x0a, 0xe2, 0x96, 0xf1, 0x68, 0x3d, 0x7a, 0x93, 0x3b, 0xb9, 0x03, 0x69, 0xea, - 0x9b, 0xa4, 0x94, 0x11, 0x6e, 0xd6, 0x66, 0x6c, 0xd2, 0x37, 0x49, 0x5c, 0x99, 0x98, 0xe6, 0xa1, - 0x28, 0x33, 0x59, 0x14, 0x6a, 0x7e, 0xd6, 0x50, 0x63, 0xeb, 0xd8, 0x88, 0xc6, 0xd1, 0x3b, 0x80, - 0xe4, 0x78, 0xc5, 0x15, 0xbb, 0x41, 0x43, 0xb1, 0xb3, 0x6b, 0x4a, 0xe8, 0x55, 0xe4, 0x2f, 0x3a, - 0xc1, 0xc2, 0xc6, 0xfa, 0x7f, 0x5c, 0x98, 0x58, 0x2d, 0x9a, 0x5f, 0xfb, 0xa2, 0x00, 0x24, 0xb6, - 0x51, 0x0d, 0x72, 0x6f, 0xc9, 0x01, 0xf1, 0x8e, 0x89, 0x2a, 0x95, 0x97, 0xfa, 0x03, 0x6d, 0x21, - 0x01, 0x63, 0x00, 0x69, 0x90, 0xdd, 0xea, 0x50, 0x4c, 0x98, 0x2a, 0x97, 0x17, 0xfb, 0x03, 0x4d, - 0x4d, 0x28, 0xd1, 0x3e, 0x7a, 0x08, 0xf9, 0x37, 0x01, 0xf6, 0xcd, 0xc0, 0x25, 0x8e, 0xaa, 0x94, - 0xef, 0xf4, 0x07, 0xda, 0xed, 0x84, 0x34, 0x86, 0xd0, 0x0a, 0xcc, 0x45, 0x1f, 0xd8, 0x56, 0x53, - 0xe5, 0xe5, 0xfe, 0x40, 0x43, 0xd3, 0x34, 0x6c, 0xa3, 0x35, 0x28, 0x18, 0xd8, 0xef, 0xba, 0x96, - 0xc9, 0xb8, 0x5e, 0xba, 0x7c, 0xb7, 0x3f, 0xd0, 0x96, 0xae, 0x75, 0x9d, 0x80, 0x5c, 0xb1, 0xc9, - 0x3c, 0x9f, 0xb7, 0xa1, 0x66, 0xa6, 0x15, 0xaf, 0x10, 0x9e, 0x52, 0xac, 0xb1, 0xad, 0x66, 0xa7, - 0x53, 0xc6, 0x40, 0xe3, 0xf5, 0xf9, 0xcf, 0x8a, 0x74, 0x36, 0xaa, 0xc8, 0xe7, 0xa3, 0x8a, 0xfc, - 0x63, 0x54, 0x91, 0x4f, 0x2e, 0x2a, 0xd2, 0xf9, 0x45, 0x45, 0xfa, 0x76, 0x51, 0x91, 0xde, 0xd7, - 0x1d, 0x97, 0xed, 0xf7, 0x3a, 0xba, 0xe5, 0x1d, 0xd6, 0xe3, 0xea, 0xeb, 0x51, 0xf5, 0x75, 0xcb, - 0xb6, 0xea, 0x7f, 0x3d, 0xd7, 0x9d, 0xac, 0x78, 0x6d, 0x9f, 0xfd, 0x09, 0x00, 0x00, 0xff, 0xff, - 0x8b, 0x24, 0x01, 0x79, 0xca, 0x05, 0x00, 0x00, + // 706 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x54, 0xbf, 0x6f, 0xd3, 0x4e, + 0x1c, 0xb5, 0x9d, 0x5f, 0xcd, 0x27, 0xf9, 0x56, 0xee, 0x7d, 0xdb, 0x52, 0x22, 0x91, 0x98, 0xa8, + 0x40, 0xd5, 0x22, 0x07, 0xca, 0x82, 0xba, 0x35, 0x2d, 0xa0, 0xaa, 0x42, 0x42, 0x4e, 0x60, 0x60, + 0x89, 0x1c, 0xfb, 0x70, 0xad, 0x86, 0xb3, 0xe5, 0xbb, 0xb4, 0xca, 0xc6, 0x88, 0xb2, 0xd0, 0x09, + 0xb1, 0x44, 0xea, 0x9f, 0xd3, 0xb1, 0x23, 0x03, 0x8a, 0x20, 0x15, 0x33, 0x7b, 0x27, 0x74, 0x77, + 0x6e, 0xdc, 0x04, 0x86, 0xd0, 0x25, 0x39, 0xdf, 0x7b, 0x9f, 0xa7, 0xf7, 0xde, 0x9d, 0x0e, 0xee, + 0x84, 0x51, 0xe0, 0x60, 0x4a, 0x83, 0xa8, 0xc6, 0xec, 0x76, 0x07, 0x87, 0x6d, 0xf9, 0x6f, 0x86, + 0x51, 0xc0, 0x02, 0xb4, 0x1a, 0xfa, 0xc4, 0x73, 0xec, 0xd0, 0x64, 0xfe, 0xbb, 0x4e, 0x70, 0x6c, + 0x3a, 0xae, 0x63, 0x8e, 0x27, 0xcc, 0x78, 0xa2, 0xb4, 0xe8, 0x05, 0x5e, 0x20, 0x06, 0x6a, 0x7c, + 0x25, 0x67, 0xab, 0x9f, 0x54, 0x48, 0x37, 0x42, 0x9b, 0xa0, 0xc7, 0x30, 0x27, 0x98, 0x2d, 0xdf, + 0x5d, 0x51, 0x0d, 0x75, 0x2d, 0x55, 0x5f, 0x1e, 0x0d, 0x2b, 0xb9, 0x26, 0xdf, 0xdb, 0xdb, 0xbd, + 0x4c, 0x96, 0x56, 0x4e, 0xf0, 0xf6, 0x5c, 0xb4, 0x0a, 0x79, 0xca, 0xec, 0x88, 0xb5, 0x0e, 0x71, + 0x6f, 0x45, 0x33, 0xd4, 0xb5, 0x62, 0x3d, 0x77, 0x39, 0xac, 0xa4, 0xf6, 0x71, 0xcf, 0x9a, 0x13, + 0xc8, 0x3e, 0xee, 0x21, 0x03, 0x72, 0x98, 0xb8, 0x82, 0x93, 0x9a, 0xe4, 0x64, 0x31, 0x71, 0xf7, + 0x71, 0x6f, 0xab, 0xf8, 0xf1, 0xb4, 0xa2, 0x7c, 0x39, 0xad, 0x28, 0x1f, 0xbe, 0x19, 0x4a, 0xf5, + 0x44, 0x05, 0xd8, 0x39, 0xc0, 0xce, 0x61, 0x18, 0xf8, 0x84, 0xa1, 0x0d, 0xf8, 0xcf, 0x19, 0x7f, + 0xb5, 0x18, 0x15, 0xe6, 0xd2, 0xf5, 0xec, 0xe5, 0xb0, 0xa2, 0x35, 0xa9, 0x55, 0x4c, 0xc0, 0x26, + 0x45, 0x0f, 0xa0, 0x10, 0x61, 0x1a, 0x74, 0x8e, 0xb0, 0xcb, 0xa9, 0xda, 0x04, 0x15, 0xae, 0xa0, + 0x26, 0x45, 0x0f, 0x61, 0xbe, 0x63, 0x53, 0xd6, 0xa2, 0x3d, 0xe2, 0x48, 0x6e, 0x6a, 0x52, 0x96, + 0xa3, 0x0d, 0x01, 0x36, 0x69, 0xf5, 0xa7, 0x06, 0x99, 0x06, 0xb3, 0x19, 0x45, 0x77, 0xa1, 0x18, + 0x61, 0xcf, 0x0f, 0x48, 0xcb, 0x09, 0xba, 0x84, 0x49, 0x33, 0x56, 0x41, 0xee, 0xed, 0xf0, 0x2d, + 0x74, 0x0f, 0xc0, 0xe9, 0x46, 0x11, 0x96, 0x6e, 0x27, 0x2d, 0xe4, 0x63, 0xa4, 0x49, 0x11, 0x83, + 0x05, 0xca, 0x6c, 0x0f, 0xb7, 0x92, 0x00, 0xdc, 0x44, 0x6a, 0xad, 0xb0, 0xb9, 0x6d, 0xce, 0x72, + 0xa0, 0xa6, 0x70, 0xc4, 0x7f, 0x3d, 0x9c, 0xf4, 0x45, 0x9f, 0x11, 0x16, 0xf5, 0xea, 0xe9, 0xb3, + 0x61, 0x45, 0xb1, 0x74, 0x3a, 0x05, 0x72, 0x73, 0x6d, 0x3b, 0x8a, 0x7c, 0x1c, 0x71, 0x73, 0xe9, + 0x49, 0x73, 0x31, 0xd2, 0xa4, 0xa5, 0x2e, 0x2c, 0xfd, 0x55, 0x17, 0xe9, 0x90, 0xe2, 0x07, 0xc9, + 0x63, 0xe7, 0x2d, 0xbe, 0x44, 0xcf, 0x21, 0x73, 0x64, 0x77, 0xba, 0x58, 0x24, 0x2d, 0x6c, 0x3e, + 0x9a, 0xcd, 0x7b, 0x22, 0x6c, 0xc9, 0xf1, 0x2d, 0xed, 0xa9, 0x5a, 0xfd, 0xa5, 0x41, 0x41, 0xdc, + 0x32, 0x1e, 0xad, 0x4b, 0x6f, 0x72, 0x27, 0x77, 0x21, 0x4d, 0x43, 0x9b, 0xac, 0x64, 0x84, 0x9b, + 0xf5, 0x19, 0x9b, 0x0c, 0x6d, 0x12, 0x57, 0x26, 0xa6, 0x79, 0x28, 0xca, 0x6c, 0x26, 0x43, 0xcd, + 0xcf, 0x1a, 0x6a, 0x6c, 0x1d, 0x5b, 0x72, 0x1c, 0xbd, 0x01, 0x48, 0x8e, 0x57, 0x5c, 0xb1, 0x1b, + 0x34, 0x14, 0x3b, 0xbb, 0xa6, 0x84, 0x5e, 0x48, 0x7f, 0xf2, 0x04, 0x0b, 0x9b, 0x1b, 0xff, 0x70, + 0x61, 0x62, 0x35, 0x39, 0xbf, 0xfe, 0x59, 0x03, 0x48, 0x6c, 0xa3, 0x2a, 0xe4, 0x5e, 0x93, 0x43, + 0x12, 0x1c, 0x13, 0x5d, 0x29, 0x2d, 0xf5, 0x07, 0xc6, 0x42, 0x02, 0xc6, 0x00, 0x32, 0x20, 0xbb, + 0xdd, 0xa6, 0x98, 0x30, 0x5d, 0x2d, 0x2d, 0xf6, 0x07, 0x86, 0x9e, 0x50, 0xe4, 0x3e, 0xba, 0x0f, + 0xf9, 0x57, 0x11, 0x0e, 0xed, 0xc8, 0x27, 0x9e, 0xae, 0x95, 0x6e, 0xf5, 0x07, 0xc6, 0xff, 0x09, + 0x69, 0x0c, 0xa1, 0x55, 0x98, 0x93, 0x1f, 0xd8, 0xd5, 0x53, 0xa5, 0xe5, 0xfe, 0xc0, 0x40, 0xd3, + 0x34, 0xec, 0xa2, 0x75, 0x28, 0x58, 0x38, 0xec, 0xf8, 0x8e, 0xcd, 0xb8, 0x5e, 0xba, 0x74, 0xbb, + 0x3f, 0x30, 0x96, 0xae, 0x75, 0x9d, 0x80, 0x5c, 0xb1, 0xc1, 0x82, 0x90, 0xb7, 0xa1, 0x67, 0xa6, + 0x15, 0xaf, 0x10, 0x9e, 0x52, 0xac, 0xb1, 0xab, 0x67, 0xa7, 0x53, 0xc6, 0x40, 0xfd, 0xe5, 0xf9, + 0x8f, 0xb2, 0x72, 0x36, 0x2a, 0xab, 0xe7, 0xa3, 0xb2, 0xfa, 0x7d, 0x54, 0x56, 0x4f, 0x2e, 0xca, + 0xca, 0xf9, 0x45, 0x59, 0xf9, 0x7a, 0x51, 0x56, 0xde, 0xd6, 0x3c, 0x9f, 0x1d, 0x74, 0xdb, 0xa6, + 0x13, 0xbc, 0xaf, 0xc5, 0xd5, 0xd7, 0x64, 0xf5, 0x35, 0xc7, 0x75, 0x6a, 0x7f, 0x3c, 0xd7, 0xed, + 0xac, 0x78, 0x6d, 0x9f, 0xfc, 0x0e, 0x00, 0x00, 0xff, 0xff, 0xa3, 0xeb, 0x06, 0x9b, 0xca, 0x05, + 0x00, 0x00, } func (m *Span) Marshal() (dAtA []byte, err error) { @@ -447,8 +447,8 @@ func (m *Checkpoint) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l - if m.LastSyncTime != 0 { - i = encodeVarintTable(dAtA, i, uint64(m.LastSyncTime)) + if m.LastSyncedTs != 0 { + i = encodeVarintTable(dAtA, i, uint64(m.LastSyncedTs)) i-- dAtA[i] = 0x18 } @@ -633,8 +633,8 @@ func (m *Checkpoint) Size() (n int) { if m.ResolvedTs != 0 { n += 1 + sovTable(uint64(m.ResolvedTs)) } - if m.LastSyncTime != 0 { - n += 1 + sovTable(uint64(m.LastSyncTime)) + if m.LastSyncedTs != 0 { + n += 1 + sovTable(uint64(m.LastSyncedTs)) } return n } @@ -899,9 +899,9 @@ func (m *Checkpoint) Unmarshal(dAtA []byte) error { } case 3: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field LastSyncTime", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field LastSyncedTs", wireType) } - m.LastSyncTime = 0 + m.LastSyncedTs = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowTable @@ -911,7 +911,7 @@ func (m *Checkpoint) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.LastSyncTime |= Ts(b&0x7F) << shift + m.LastSyncedTs |= Ts(b&0x7F) << shift if b < 0x80 { break } diff --git a/cdc/processor/tablepb/table.proto b/cdc/processor/tablepb/table.proto index d0143cc219c..3727edc0c13 100644 --- a/cdc/processor/tablepb/table.proto +++ b/cdc/processor/tablepb/table.proto @@ -57,7 +57,7 @@ enum TableState { message Checkpoint { uint64 checkpoint_ts = 1 [(gogoproto.casttype) = "Ts"]; uint64 resolved_ts = 2 [(gogoproto.casttype) = "Ts"]; - uint64 last_sync_time = 3 [(gogoproto.casttype) = "Ts"]; + uint64 last_synced_ts = 3 [(gogoproto.casttype) = "Ts"]; } // Stats holds a statistic for a table. diff --git a/cdc/scheduler/internal/scheduler.go b/cdc/scheduler/internal/scheduler.go index 8e47a779d2a..70bff9bfeff 100644 --- a/cdc/scheduler/internal/scheduler.go +++ b/cdc/scheduler/internal/scheduler.go @@ -47,7 +47,7 @@ type Scheduler interface { // ddl jobs that need to be replicated. The Scheduler will // broadcast the barrierTs to all captures through the Heartbeat. barrier *schedulepb.BarrierWithMinTs, - ) (newCheckpointTs, newResolvedTs, newLastSyncTime model.Ts, err error) + ) (newCheckpointTs, newResolvedTs, newLastSyncedTs, newPullerIngressResolvedTs model.Ts, err error) // MoveTable requests that a table be moved to target. // It is thread-safe. diff --git a/cdc/scheduler/internal/v3/coordinator.go b/cdc/scheduler/internal/v3/coordinator.go index 76291c818ab..a2312401e2d 100644 --- a/cdc/scheduler/internal/v3/coordinator.go +++ b/cdc/scheduler/internal/v3/coordinator.go @@ -126,7 +126,7 @@ func (c *coordinator) Tick( // All captures that are alive according to the latest Etcd states. aliveCaptures map[model.CaptureID]*model.CaptureInfo, barrier *schedulepb.BarrierWithMinTs, -) (newCheckpointTs, newResolvedTs, newLastSyncTime model.Ts, err error) { +) (newCheckpointTs, newResolvedTs, newLastSyncedTs, newPullerIngressResolvedTs model.Ts, err error) { startTime := time.Now() defer func() { costTime := time.Since(startTime) @@ -272,7 +272,7 @@ func (c *coordinator) poll( currentTables []model.TableID, aliveCaptures map[model.CaptureID]*model.CaptureInfo, barrier *schedulepb.BarrierWithMinTs, -) (newCheckpointTs, newResolvedTs, newLastSyncTime model.Ts, err error) { +) (newCheckpointTs, newResolvedTs, newLastSyncedTs, newPullerIngressResolvedTs model.Ts, err error) { c.maybeCollectMetrics() if c.compat.UpdateCaptureInfo(aliveCaptures) { spanReplicationEnabled := c.compat.CheckSpanReplicationEnabled() @@ -283,7 +283,7 @@ func (c *coordinator) poll( recvMsgs, err := c.recvMsgs(ctx) if err != nil { - return checkpointCannotProceed, checkpointCannotProceed, checkpointCannotProceed, errors.Trace(err) + return checkpointCannotProceed, checkpointCannotProceed, checkpointCannotProceed, checkpointCannotProceed, errors.Trace(err) } var msgBuf []*schedulepb.Message @@ -295,7 +295,7 @@ func (c *coordinator) poll( // Handle received messages to advance replication set. msgs, err = c.replicationM.HandleMessage(recvMsgs) if err != nil { - return checkpointCannotProceed, checkpointCannotProceed, checkpointCannotProceed, errors.Trace(err) + return checkpointCannotProceed, checkpointCannotProceed, checkpointCannotProceed, checkpointCannotProceed, errors.Trace(err) } msgBuf = append(msgBuf, msgs...) @@ -309,13 +309,14 @@ func (c *coordinator) poll( if !c.captureM.CheckAllCaptureInitialized() { // Skip generating schedule tasks for replication manager, // as not all capture are initialized. - newCheckpointTs, newResolvedTs, newLastSyncTime = c.replicationM.AdvanceCheckpoint(&c.tableRanges, pdTime, barrier, c.redoMetaManager) + newCheckpointTs, newResolvedTs, newLastSyncedTs, + newPullerIngressResolvedTs = c.replicationM.AdvanceCheckpoint(&c.tableRanges, pdTime, barrier, c.redoMetaManager) // tick capture manager after checkpoint calculation to take account resolvedTs in barrier // when redo is enabled msgs = c.captureM.Tick(c.replicationM.ReplicationSets(), c.schedulerM.DrainingTarget(), barrier.Barrier) msgBuf = append(msgBuf, msgs...) - return newCheckpointTs, newResolvedTs, newLastSyncTime, c.sendMsgs(ctx, msgBuf) + return newCheckpointTs, newResolvedTs, newLastSyncedTs, newPullerIngressResolvedTs, c.sendMsgs(ctx, msgBuf) } // Handle capture membership changes. @@ -323,7 +324,7 @@ func (c *coordinator) poll( msgs, err = c.replicationM.HandleCaptureChanges( changes.Init, changes.Removed, checkpointTs) if err != nil { - return checkpointCannotProceed, checkpointCannotProceed, checkpointCannotProceed, errors.Trace(err) + return checkpointCannotProceed, checkpointCannotProceed, checkpointCannotProceed, checkpointCannotProceed, errors.Trace(err) } msgBuf = append(msgBuf, msgs...) } @@ -339,12 +340,13 @@ func (c *coordinator) poll( // Handle generated schedule tasks. msgs, err = c.replicationM.HandleTasks(allTasks) if err != nil { - return checkpointCannotProceed, checkpointCannotProceed, checkpointCannotProceed, errors.Trace(err) + return checkpointCannotProceed, checkpointCannotProceed, checkpointCannotProceed, checkpointCannotProceed, errors.Trace(err) } msgBuf = append(msgBuf, msgs...) // Checkpoint calculation - newCheckpointTs, newResolvedTs, newLastSyncTime = c.replicationM.AdvanceCheckpoint(&c.tableRanges, pdTime, barrier, c.redoMetaManager) + newCheckpointTs, newResolvedTs, newLastSyncedTs, + newPullerIngressResolvedTs = c.replicationM.AdvanceCheckpoint(&c.tableRanges, pdTime, barrier, c.redoMetaManager) // tick capture manager after checkpoint calculation to take account resolvedTs in barrier // when redo is enabled @@ -355,10 +357,10 @@ func (c *coordinator) poll( // Send new messages. err = c.sendMsgs(ctx, msgBuf) if err != nil { - return checkpointCannotProceed, checkpointCannotProceed, checkpointCannotProceed, errors.Trace(err) + return checkpointCannotProceed, checkpointCannotProceed, checkpointCannotProceed, checkpointCannotProceed, errors.Trace(err) } - return newCheckpointTs, newResolvedTs, newLastSyncTime, nil + return newCheckpointTs, newResolvedTs, newLastSyncedTs, newPullerIngressResolvedTs, nil } func (c *coordinator) recvMsgs(ctx context.Context) ([]*schedulepb.Message, error) { diff --git a/cdc/scheduler/internal/v3/replication/replication_manager.go b/cdc/scheduler/internal/v3/replication/replication_manager.go index 6e2fdea5e19..4b96567d665 100644 --- a/cdc/scheduler/internal/v3/replication/replication_manager.go +++ b/cdc/scheduler/internal/v3/replication/replication_manager.go @@ -557,9 +557,11 @@ func (r *Manager) AdvanceCheckpoint( currentPDTime time.Time, barrier *schedulepb.BarrierWithMinTs, redoMetaManager redo.MetaManager, -) (newCheckpointTs, newResolvedTs model.Ts, newLastSyncTime model.Ts) { +) (newCheckpointTs, newResolvedTs, newLastSyncedTs, newPullerIngressResolvedTs model.Ts) { var redoFlushedResolvedTs model.Ts - limitBarrierWithRedo := func(newCheckpointTs, newResolvedTs, newLastSyncTime uint64) (uint64, uint64, uint64) { + limitBarrierWithRedo := func(newCheckpointTs, newResolvedTs, newLastSyncedTs, + newPullerIngressResolvedTs uint64, + ) (uint64, uint64, uint64, uint64) { flushedMeta := redoMetaManager.GetFlushedMeta() redoFlushedResolvedTs = flushedMeta.ResolvedTs log.Debug("owner gets flushed redo meta", @@ -578,7 +580,7 @@ func (r *Manager) AdvanceCheckpoint( if barrier.GlobalBarrierTs > newResolvedTs { barrier.GlobalBarrierTs = newResolvedTs } - return newCheckpointTs, newResolvedTs, newLastSyncTime + return newCheckpointTs, newResolvedTs, newLastSyncedTs, newPullerIngressResolvedTs } defer func() { if redoFlushedResolvedTs != 0 && barrier.GlobalBarrierTs > redoFlushedResolvedTs { @@ -594,7 +596,9 @@ func (r *Manager) AdvanceCheckpoint( r.slowestSink = tablepb.Span{} var slowestPullerResolvedTs uint64 = math.MaxUint64 - newCheckpointTs, newResolvedTs, newLastSyncTime = math.MaxUint64, math.MaxUint64, 0 + // newPullerIngressResolvedTs to record the min ingress resolved ts of all pullers + newCheckpointTs, newResolvedTs, newLastSyncedTs, newPullerIngressResolvedTs = math.MaxUint64, math.MaxUint64, 0, math.MaxUint64 + cannotProceed := false currentTables.Iter(func(tableID model.TableID, tableStart, tableEnd tablepb.Span) bool { tableSpanFound, tableHasHole := false, false @@ -629,8 +633,9 @@ func (r *Manager) AdvanceCheckpoint( newResolvedTs = table.Checkpoint.ResolvedTs } - if newLastSyncTime < table.Checkpoint.LastSyncTime { - newLastSyncTime = table.Checkpoint.LastSyncTime + // Find the max lastSyncedTs of all tables. + if newLastSyncedTs < table.Checkpoint.LastSyncedTs { + newLastSyncedTs = table.Checkpoint.LastSyncedTs } // Find the minimum puller resolved ts. if pullerCkpt, ok := table.Stats.StageCheckpoints["puller-egress"]; ok { @@ -639,6 +644,14 @@ func (r *Manager) AdvanceCheckpoint( r.slowestPuller = span } } + + // Find the minimum puller ingress resolved ts. + if pullerIngressCkpt, ok := table.Stats.StageCheckpoints["puller-ingress"]; ok { + if newPullerIngressResolvedTs > pullerIngressCkpt.ResolvedTs { + newPullerIngressResolvedTs = pullerIngressCkpt.ResolvedTs + } + } + return true }) if !tableSpanFound || !tableSpanStartFound || !tableSpanEndFound || tableHasHole { @@ -667,9 +680,9 @@ func (r *Manager) AdvanceCheckpoint( if redoMetaManager.Enabled() { // If redo is enabled, GlobalBarrierTs should be limited by redo flushed meta. newResolvedTs = barrier.RedoBarrierTs - limitBarrierWithRedo(newCheckpointTs, newResolvedTs, newLastSyncTime) + limitBarrierWithRedo(newCheckpointTs, newResolvedTs, newLastSyncedTs, newPullerIngressResolvedTs) } - return checkpointCannotProceed, checkpointCannotProceed, checkpointCannotProceed + return checkpointCannotProceed, checkpointCannotProceed, checkpointCannotProceed, checkpointCannotProceed } // If currentTables is empty, we should advance newResolvedTs to global barrier ts and @@ -717,10 +730,10 @@ func (r *Manager) AdvanceCheckpoint( zap.String("changefeed", r.changefeedID.ID), zap.Uint64("newCheckpointTs", newCheckpointTs), zap.Uint64("newResolvedTs", newResolvedTs)) - return limitBarrierWithRedo(newCheckpointTs, newResolvedTs, newLastSyncTime) + return limitBarrierWithRedo(newCheckpointTs, newResolvedTs, newLastSyncedTs, newPullerIngressResolvedTs) } - return newCheckpointTs, newResolvedTs, newLastSyncTime + return newCheckpointTs, newResolvedTs, newLastSyncedTs, newPullerIngressResolvedTs } func (r *Manager) logSlowTableInfo(currentPDTime time.Time) { diff --git a/cdc/scheduler/internal/v3/replication/replication_set.go b/cdc/scheduler/internal/v3/replication/replication_set.go index d99a297e65c..8e6f8ec6122 100644 --- a/cdc/scheduler/internal/v3/replication/replication_set.go +++ b/cdc/scheduler/internal/v3/replication/replication_set.go @@ -1021,8 +1021,8 @@ func (r *ReplicationSet) updateCheckpointAndStats( zap.Any("resolvedTs", r.Checkpoint.ResolvedTs)) } - if r.Checkpoint.LastSyncTime < checkpoint.LastSyncTime { - r.Checkpoint.LastSyncTime = checkpoint.LastSyncTime + if r.Checkpoint.LastSyncedTs < checkpoint.LastSyncedTs { + r.Checkpoint.LastSyncedTs = checkpoint.LastSyncedTs } r.Stats = stats } diff --git a/cdc/sink/dmlsink/cloudstorage/cloud_storage_dml_sink.go b/cdc/sink/dmlsink/cloudstorage/cloud_storage_dml_sink.go index b3e235578b0..14043124524 100644 --- a/cdc/sink/dmlsink/cloudstorage/cloud_storage_dml_sink.go +++ b/cdc/sink/dmlsink/cloudstorage/cloud_storage_dml_sink.go @@ -224,7 +224,6 @@ func (s *DMLSink) WriteEvents(txns ...*dmlsink.CallbackableEvent[*model.SingleTa if txn.GetTableSinkState() != state.TableSinkSinking { // The table where the event comes from is in stopping, so it's safe // to drop the event directly. - // 这个状态是在干嘛的? txn.Callback() continue } diff --git a/cdc/sink/tablesink/table_sink.go b/cdc/sink/tablesink/table_sink.go index 6dac7c87417..83cf988ad63 100644 --- a/cdc/sink/tablesink/table_sink.go +++ b/cdc/sink/tablesink/table_sink.go @@ -32,9 +32,11 @@ type TableSink interface { // For example, calculating the current progress from the statistics of the table sink. // This is a thread-safe method. GetCheckpointTs() model.ResolvedTs - // GetLastSyncTime returns the lastSyncTime of table sink. + // GetLastSyncedTs returns the last synced ts of table sink. + // the last synced ts means the biggest committs of the events + // that have been flushed to the downstream. // This is a thread-safe method. - GetLastSyncTime() uint64 + GetLastSyncedTs() model.Ts // 先全部用这个吧,最后再看是否合适 // Close closes the table sink. // After it returns, no more events will be sent out from this capture. Close() diff --git a/cdc/sink/tablesink/table_sink_impl.go b/cdc/sink/tablesink/table_sink_impl.go index 76908c9fac0..1a33a21be19 100644 --- a/cdc/sink/tablesink/table_sink_impl.go +++ b/cdc/sink/tablesink/table_sink_impl.go @@ -32,9 +32,12 @@ var ( _ TableSink = (*EventTableSink[*model.SingleTableTxn, *dmlsink.TxnEventAppender])(nil) ) -type LastSyncTimeRecord struct { +// LastSyncedTsRecord is used to record the last synced ts of table sink with lock +// lastSyncedTs means the biggest committs of the events +// that have been flushed to the downstream. +type LastSyncedTsRecord struct { sync.Mutex - lastSyncTime uint64 + lastSyncedTs model.Ts } // EventTableSink is a table sink that can write events. @@ -53,7 +56,7 @@ type EventTableSink[E dmlsink.TableEvent, P dmlsink.Appender[E]] struct { eventBuffer []E state state.TableSinkState - lastSyncTime LastSyncTimeRecord + lastSyncedTs LastSyncedTsRecord // For dataflow metrics. metricsTableSinkTotalRows prometheus.Counter @@ -78,7 +81,7 @@ func New[E dmlsink.TableEvent, P dmlsink.Appender[E]]( eventAppender: appender, eventBuffer: make([]E, 0, 1024), state: state.TableSinkSinking, - lastSyncTime: LastSyncTimeRecord{lastSyncTime: 0}, // use 0 to initialize lastSyncTime + lastSyncedTs: LastSyncedTsRecord{lastSyncedTs: 0}, metricsTableSinkTotalRows: totalRowsCounter, } } @@ -124,20 +127,21 @@ func (e *EventTableSink[E, P]) UpdateResolvedTs(resolvedTs model.ResolvedTs) err return SinkInternalError{err} } // We have to record the event ID for the callback. + postEventFlushFunc := e.progressTracker.addEvent() ce := &dmlsink.CallbackableEvent[E]{ Event: ev, Callback: func() { // Due to multi workers will call this callback concurrently, - // we need to add lock to protect lastSyncTime + // we need to add lock to protect lastSyncedTs // we need make a performance test for it { - e.lastSyncTime.Lock() - defer e.lastSyncTime.Unlock() - if e.lastSyncTime.lastSyncTime < ev.GetCommitTs() { - e.lastSyncTime.lastSyncTime = ev.GetCommitTs() + e.lastSyncedTs.Lock() + defer e.lastSyncedTs.Unlock() + if e.lastSyncedTs.lastSyncedTs < ev.GetCommitTs() { + e.lastSyncedTs.lastSyncedTs = ev.GetCommitTs() } } - e.progressTracker.addEvent() + postEventFlushFunc() }, SinkState: &e.state, } @@ -162,11 +166,13 @@ func (e *EventTableSink[E, P]) GetCheckpointTs() model.ResolvedTs { return e.progressTracker.advance() } -// GetLastSyncTime returns the lastSyncTime ts of the table sink. -func (e *EventTableSink[E, P]) GetLastSyncTime() uint64 { - e.lastSyncTime.Lock() - defer e.lastSyncTime.Unlock() - return e.lastSyncTime.lastSyncTime +// GetLastSyncedTs returns the last synced ts of table sink. +// lastSyncedTs means the biggest committs of all the events +// that have been flushed to the downstream. +func (e *EventTableSink[E, P]) GetLastSyncedTs() model.Ts { + e.lastSyncedTs.Lock() + defer e.lastSyncedTs.Unlock() + return e.lastSyncedTs.lastSyncedTs } // Close closes the table sink. From f45c7aefc89a9b827f57f296403afbb51331cad0 Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Thu, 23 Nov 2023 21:23:02 +0800 Subject: [PATCH 04/35] add test --- cdc/sink/tablesink/table_sink.go | 2 +- cdc/sink/tablesink/table_sink_impl.go | 16 +++++++++++----- cdc/sink/tablesink/table_sink_impl_test.go | 9 +++++++++ 3 files changed, 21 insertions(+), 6 deletions(-) diff --git a/cdc/sink/tablesink/table_sink.go b/cdc/sink/tablesink/table_sink.go index 83cf988ad63..de8d197a208 100644 --- a/cdc/sink/tablesink/table_sink.go +++ b/cdc/sink/tablesink/table_sink.go @@ -36,7 +36,7 @@ type TableSink interface { // the last synced ts means the biggest committs of the events // that have been flushed to the downstream. // This is a thread-safe method. - GetLastSyncedTs() model.Ts // 先全部用这个吧,最后再看是否合适 + GetLastSyncedTs() model.Ts // Close closes the table sink. // After it returns, no more events will be sent out from this capture. Close() diff --git a/cdc/sink/tablesink/table_sink_impl.go b/cdc/sink/tablesink/table_sink_impl.go index 1a33a21be19..3c4a4407045 100644 --- a/cdc/sink/tablesink/table_sink_impl.go +++ b/cdc/sink/tablesink/table_sink_impl.go @@ -40,6 +40,12 @@ type LastSyncedTsRecord struct { lastSyncedTs model.Ts } +func (r *LastSyncedTsRecord) getLastSyncedTs() model.Ts { + r.Lock() + defer r.Unlock() + return r.lastSyncedTs +} + // EventTableSink is a table sink that can write events. type EventTableSink[E dmlsink.TableEvent, P dmlsink.Appender[E]] struct { @@ -128,6 +134,7 @@ func (e *EventTableSink[E, P]) UpdateResolvedTs(resolvedTs model.ResolvedTs) err } // We have to record the event ID for the callback. postEventFlushFunc := e.progressTracker.addEvent() + evCommitTs := ev.GetCommitTs() ce := &dmlsink.CallbackableEvent[E]{ Event: ev, Callback: func() { @@ -137,8 +144,9 @@ func (e *EventTableSink[E, P]) UpdateResolvedTs(resolvedTs model.ResolvedTs) err { e.lastSyncedTs.Lock() defer e.lastSyncedTs.Unlock() - if e.lastSyncedTs.lastSyncedTs < ev.GetCommitTs() { - e.lastSyncedTs.lastSyncedTs = ev.GetCommitTs() + + if e.lastSyncedTs.lastSyncedTs < evCommitTs { + e.lastSyncedTs.lastSyncedTs = evCommitTs } } postEventFlushFunc() @@ -170,9 +178,7 @@ func (e *EventTableSink[E, P]) GetCheckpointTs() model.ResolvedTs { // lastSyncedTs means the biggest committs of all the events // that have been flushed to the downstream. func (e *EventTableSink[E, P]) GetLastSyncedTs() model.Ts { - e.lastSyncedTs.Lock() - defer e.lastSyncedTs.Unlock() - return e.lastSyncedTs.lastSyncedTs + return e.lastSyncedTs.getLastSyncedTs() } // Close closes the table sink. diff --git a/cdc/sink/tablesink/table_sink_impl_test.go b/cdc/sink/tablesink/table_sink_impl_test.go index 95a26711a25..647328d7747 100644 --- a/cdc/sink/tablesink/table_sink_impl_test.go +++ b/cdc/sink/tablesink/table_sink_impl_test.go @@ -38,6 +38,9 @@ type mockEventSink struct { func (m *mockEventSink) WriteEvents(rows ...*dmlsink.TxnCallbackableEvent) error { m.events = append(m.events, rows...) + // for _, event := range rows { + // event.Callback() + // } return nil } @@ -247,6 +250,7 @@ func TestGetCheckpointTs(t *testing.T) { tb.AppendRowChangedEvents(getTestRows()...) require.Equal(t, model.NewResolvedTs(0), tb.GetCheckpointTs(), "checkpointTs should be 0") + require.Equal(t, tb.lastSyncedTs.getLastSyncedTs(), uint64(0), "lastSyncedTs should be not updated") // One event will be flushed. err := tb.UpdateResolvedTs(model.NewResolvedTs(101)) @@ -254,11 +258,13 @@ func TestGetCheckpointTs(t *testing.T) { require.Equal(t, model.NewResolvedTs(0), tb.GetCheckpointTs(), "checkpointTs should be 0") sink.acknowledge(101) require.Equal(t, model.NewResolvedTs(101), tb.GetCheckpointTs(), "checkpointTs should be 101") + require.Equal(t, tb.lastSyncedTs.getLastSyncedTs(), uint64(101), "lastSyncedTs should be the same as the flushed event") // Flush all events. err = tb.UpdateResolvedTs(model.NewResolvedTs(105)) require.Nil(t, err) require.Equal(t, model.NewResolvedTs(101), tb.GetCheckpointTs(), "checkpointTs should be 101") + require.Equal(t, tb.lastSyncedTs.getLastSyncedTs(), uint64(101), "lastSyncedTs should be not updated") // Only acknowledge some events. sink.acknowledge(102) @@ -268,10 +274,12 @@ func TestGetCheckpointTs(t *testing.T) { tb.GetCheckpointTs(), "checkpointTs should still be 101", ) + require.Equal(t, tb.lastSyncedTs.getLastSyncedTs(), uint64(102), "lastSyncedTs should be updated") // Ack all events. sink.acknowledge(105) require.Equal(t, model.NewResolvedTs(105), tb.GetCheckpointTs(), "checkpointTs should be 105") + require.Equal(t, tb.lastSyncedTs.getLastSyncedTs(), uint64(105), "lastSyncedTs should be updated") } func TestClose(t *testing.T) { @@ -404,4 +412,5 @@ func TestCheckpointTsFrozenWhenStopping(t *testing.T) { currentTs := tb.GetCheckpointTs() sink.acknowledge(105) require.Equal(t, currentTs, tb.GetCheckpointTs(), "checkpointTs should not be updated") + require.Equal(t, tb.lastSyncedTs.getLastSyncedTs(), uint64(105), "lastSyncedTs should not change") } From cbd4b7de730880d757e81b03dcf72deb06555a14 Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Mon, 27 Nov 2023 09:34:27 +0800 Subject: [PATCH 05/35] add test --- cdc/api/v2/changefeed.go | 12 +- cdc/model/changefeed.go | 8 +- cdc/owner/changefeed.go | 32 +++-- cdc/owner/changefeed_test.go | 5 +- cdc/owner/owner.go | 2 +- cdc/processor/sinkmanager/manager_test.go | 29 +++-- .../sinkmanager/table_sink_wrapper.go | 7 +- cdc/scheduler/internal/scheduler.go | 2 +- cdc/scheduler/internal/v3/coordinator.go | 12 +- cdc/scheduler/internal/v3/coordinator_test.go | 56 +++++++-- .../v3/replication/replication_manager.go | 22 ++-- .../replication/replication_manager_test.go | 113 ++++++++++++++++-- 12 files changed, 220 insertions(+), 80 deletions(-) diff --git a/cdc/api/v2/changefeed.go b/cdc/api/v2/changefeed.go index 3d873d0b151..af27dcb1941 100644 --- a/cdc/api/v2/changefeed.go +++ b/cdc/api/v2/changefeed.go @@ -939,7 +939,7 @@ func (h *OpenAPIV2) synced(c *gin.Context) { if err != nil { // pd 不可用 var message string - if (status.PullerIngressResolvedTs - status.CheckpointTs) > (5*1000)<<18 { // 5s + if (status.PullerResolvedTs - status.CheckpointTs) > (5*1000)<<18 { // 5s message = fmt.Sprintf("we get pd client failed with err is %s. Besides the data is not finish syncing", terror.Message(err)) } else { message = fmt.Sprintf("we get pd client failed with err is %s. "+ @@ -951,7 +951,7 @@ func (h *OpenAPIV2) synced(c *gin.Context) { c.JSON(http.StatusOK, map[string]any{ "Synced": false, "Sink-CheckpointTs": status.CheckpointTs, - "Puller-ResolvedTs": status.PullerIngressResolvedTs, + "Puller-ResolvedTs": status.PullerResolvedTs, "LastSyncedTs": status.LastSyncedTs, "info": message, }) @@ -964,13 +964,13 @@ func (h *OpenAPIV2) synced(c *gin.Context) { c.JSON(http.StatusOK, map[string]any{ "Synced": true, "Sink-CheckpointTs": status.CheckpointTs, - "Puller-ResolvedTs": status.PullerIngressResolvedTs, + "Puller-ResolvedTs": status.PullerResolvedTs, "LastSyncedTs": status.LastSyncedTs, "info": "Data syncing is finished", }) } else if now-status.LastSyncedTs > (5*60*1000)<<18 { // lastSyncedTs 条件达到,checkpoint-ts 未达到 var message string - if (status.PullerIngressResolvedTs - status.CheckpointTs) > (5*1000)<<18 { // 5s + if (status.PullerResolvedTs - status.CheckpointTs) > (5*1000)<<18 { // 5s message = fmt.Sprintf("Please check whether pd is health and tikv region is all available. " + "If pd is not health or tikv region is not available, the data syncing is finished. " + " Otherwise the data syncing is not finished, please wait") @@ -980,7 +980,7 @@ func (h *OpenAPIV2) synced(c *gin.Context) { c.JSON(http.StatusOK, map[string]any{ "Synced": false, "Sink-CheckpointTs": status.CheckpointTs, - "Puller-ResolvedTs": status.PullerIngressResolvedTs, + "Puller-ResolvedTs": status.PullerResolvedTs, "LastSyncedTs": status.LastSyncedTs, "info": message, }) @@ -988,7 +988,7 @@ func (h *OpenAPIV2) synced(c *gin.Context) { c.JSON(http.StatusOK, map[string]any{ "Synced": false, "Sink-CheckpointTs": status.CheckpointTs, - "Puller-ResolvedTs": status.PullerIngressResolvedTs, + "Puller-ResolvedTs": status.PullerResolvedTs, "LastSyncedTs": status.LastSyncedTs, "info": "The data syncing is not finished, please wait", }) diff --git a/cdc/model/changefeed.go b/cdc/model/changefeed.go index 91d3a2111ce..586fa0020e1 100644 --- a/cdc/model/changefeed.go +++ b/cdc/model/changefeed.go @@ -631,8 +631,8 @@ type ChangeFeedStatusForAPI struct { } type ChangeFeedSyncedStatusForAPI struct { - ResolvedTs uint64 `json:"resolved-ts"` - CheckpointTs uint64 `json:"checkpoint-ts"` - LastSyncedTs uint64 `json:"last-sync-time"` - PullerIngressResolvedTs uint64 `json:"puller-ingress-resolved-ts"` + ResolvedTs uint64 `json:"resolved-ts"` + CheckpointTs uint64 `json:"checkpoint-ts"` + LastSyncedTs uint64 `json:"last-sync-time"` + PullerResolvedTs uint64 `json:"puller-resolved-ts"` } diff --git a/cdc/owner/changefeed.go b/cdc/owner/changefeed.go index a78b9ce29eb..93ac2fba83a 100644 --- a/cdc/owner/changefeed.go +++ b/cdc/owner/changefeed.go @@ -90,11 +90,16 @@ type changefeed struct { scheduler scheduler.Scheduler // barriers will be created when a changefeed is initialized // and will be destroyed when a changefeed is closed. - barriers *barriers - feedStateManager FeedStateManager - resolvedTs model.Ts - lastSyncedTs model.Ts - pullerIngressResolvedTs model.Ts // record the latest min puller ingress resolved ts of all pullers + barriers *barriers + feedStateManager FeedStateManager + resolvedTs model.Ts + + // lastSyncedTs is the lastest resolvedTs that has been synced to downstream. + // pullerResolvedTs is the minimum resolvedTs of all pullers. + // we don't need to initialize lastSyncedTs and pullerResolvedTs specially + // because it will be updated in tick. + lastSyncedTs model.Ts + pullerResolvedTs model.Ts // ddl related fields ddlManager *ddlManager @@ -414,7 +419,7 @@ func (c *changefeed) tick(ctx cdcContext.Context, return 0, 0, nil } - newCheckpointTs, newResolvedTs, newLastSyncedTs, newPullerIngressResolvedTs, err := c.scheduler.Tick( + newCheckpointTs, newResolvedTs, newLastSyncedTs, newPullerResolvedTs, err := c.scheduler.Tick( ctx, preCheckpointTs, allPhysicalTables, captures, barrier) if err != nil { @@ -428,13 +433,13 @@ func (c *changefeed) tick(ctx cdcContext.Context, zap.Uint64("newLastSyncedTs", newLastSyncedTs)) } - if newPullerIngressResolvedTs != scheduler.CheckpointCannotProceed { - if newPullerIngressResolvedTs > c.pullerIngressResolvedTs { - c.pullerIngressResolvedTs = newPullerIngressResolvedTs - } else if newPullerIngressResolvedTs < c.pullerIngressResolvedTs { - log.Warn("the newPullerIngressResolvedTs should not be smaller than c.PullerIngressResolvedTs", - zap.Uint64("c.pullerIngressResolvedTs", c.pullerIngressResolvedTs), - zap.Uint64("newPullerIngressResolvedTs", newPullerIngressResolvedTs)) + if newPullerResolvedTs != scheduler.CheckpointCannotProceed { + if newPullerResolvedTs > c.pullerResolvedTs { + c.pullerResolvedTs = newPullerResolvedTs + } else if newPullerResolvedTs < c.pullerResolvedTs { + log.Warn("the newPullerResolvedTs should not be smaller than c.pullerResolvedTs", + zap.Uint64("c.pullerResolvedTs", c.pullerResolvedTs), + zap.Uint64("newPullerResolvedTs", newPullerResolvedTs)) } } @@ -533,6 +538,7 @@ LOOP2: // lastSyncedTs always increase even if there are no more data send into ticdc. c.lastSyncedTs = uint64(oracle.GetPhysical(c.upstream.PDClock.CurrentTime())) } + minTableBarrierTs := c.latestStatus.MinTableBarrierTs failpoint.Inject("NewChangefeedNoRetryError", func() { diff --git a/cdc/owner/changefeed_test.go b/cdc/owner/changefeed_test.go index 28eaec00a03..bd3f5714219 100644 --- a/cdc/owner/changefeed_test.go +++ b/cdc/owner/changefeed_test.go @@ -165,9 +165,10 @@ func (m *mockScheduler) Tick( currentTables []model.TableID, captures map[model.CaptureID]*model.CaptureInfo, barrier *schedulepb.BarrierWithMinTs, -) (newCheckpointTs, newResolvedTs model.Ts, err error) { +) (newCheckpointTs, newResolvedTs, newLastSyncedTs, newPullerResolvedTs model.Ts, err error) { m.currentTables = currentTables - return barrier.MinTableBarrierTs, barrier.GlobalBarrierTs, nil + // todo:这边到底应该 mock 一个什么呢 + return barrier.MinTableBarrierTs, barrier.GlobalBarrierTs, scheduler.CheckpointCannotProceed, scheduler.CheckpointCannotProceed, nil } // MoveTable is used to trigger manual table moves. diff --git a/cdc/owner/owner.go b/cdc/owner/owner.go index 747b40bb0a7..99571338523 100644 --- a/cdc/owner/owner.go +++ b/cdc/owner/owner.go @@ -607,7 +607,7 @@ func (o *ownerImpl) handleQueries(query *Query) error { ret.ResolvedTs = cfReactor.resolvedTs ret.LastSyncedTs = cfReactor.lastSyncedTs ret.CheckpointTs = cfReactor.latestStatus.CheckpointTs - ret.PullerIngressResolvedTs = cfReactor.pullerIngressResolvedTs + ret.PullerResolvedTs = cfReactor.pullerResolvedTs query.Data = ret case QueryChangefeedInfo: cfReactor, ok := o.changefeeds[query.ChangeFeedID] diff --git a/cdc/processor/sinkmanager/manager_test.go b/cdc/processor/sinkmanager/manager_test.go index b70e17fc3ab..ca3eef085d2 100644 --- a/cdc/processor/sinkmanager/manager_test.go +++ b/cdc/processor/sinkmanager/manager_test.go @@ -93,6 +93,13 @@ func addTableAndAddEventsToSortEngine( CRTs: 4, }, }, + { + CRTs: 6, + RawKV: &model.RawKVEntry{ + OpType: model.OpTypeResolved, + CRTs: 6, + }, + }, } for _, event := range events { engine.Add(span, event) @@ -195,10 +202,16 @@ func TestGenerateTableSinkTaskWithBarrierTs(t *testing.T) { require.NoError(t, err) require.Eventually(t, func() bool { - tableSink, ok := manager.tableSinks.Load(span) - require.True(t, ok) - checkpointTS := tableSink.(*tableSinkWrapper).getCheckpointTs() - return checkpointTS.ResolvedMark() == 4 + s := manager.GetTableStats(span) + return s.CheckpointTs == 4 && s.LastSyncedTs == 4 + }, 5*time.Second, 10*time.Millisecond) + + manager.UpdateReceivedSorterResolvedTs(span, 6) + manager.schemaStorage.AdvanceResolvedTs(6) + manager.UpdateBarrierTs(6, nil) + require.Eventually(t, func() bool { + s := manager.GetTableStats(span) + return s.CheckpointTs == 6 && s.LastSyncedTs == 4 }, 5*time.Second, 10*time.Millisecond) } @@ -226,10 +239,8 @@ func TestGenerateTableSinkTaskWithResolvedTs(t *testing.T) { require.NoError(t, err) require.Eventually(t, func() bool { - tableSink, ok := manager.tableSinks.Load(span) - require.True(t, ok) - checkpointTS := tableSink.(*tableSinkWrapper).getCheckpointTs() - return checkpointTS.ResolvedMark() == 3 + s := manager.GetTableStats(span) + return s.CheckpointTs == 3 && s.LastSyncedTs == 3 }, 5*time.Second, 10*time.Millisecond) } @@ -257,7 +268,7 @@ func TestGetTableStatsToReleaseMemQuota(t *testing.T) { require.Eventually(t, func() bool { s := manager.GetTableStats(span) - return manager.sinkMemQuota.GetUsedBytes() == 0 && s.CheckpointTs == 4 + return manager.sinkMemQuota.GetUsedBytes() == 0 && s.CheckpointTs == 4 && s.LastSyncedTs == 4 }, 5*time.Second, 10*time.Millisecond) } diff --git a/cdc/processor/sinkmanager/table_sink_wrapper.go b/cdc/processor/sinkmanager/table_sink_wrapper.go index 98129e7c032..931748e4599 100644 --- a/cdc/processor/sinkmanager/table_sink_wrapper.go +++ b/cdc/processor/sinkmanager/table_sink_wrapper.go @@ -58,6 +58,7 @@ type tableSinkWrapper struct { advanced time.Time resolvedTs model.ResolvedTs checkpointTs model.ResolvedTs + lastSyncedTs model.Ts } // state used to control the lifecycle of the table. @@ -221,7 +222,10 @@ func (t *tableSinkWrapper) updateResolvedTs(ts model.ResolvedTs) error { func (t *tableSinkWrapper) getLastSyncedTs() uint64 { t.tableSink.RLock() defer t.tableSink.RUnlock() - return t.tableSink.s.GetLastSyncedTs() + if t.tableSink.s != nil { + return t.tableSink.s.GetLastSyncedTs() + } + return t.tableSink.lastSyncedTs } func (t *tableSinkWrapper) getCheckpointTs() model.ResolvedTs { @@ -363,6 +367,7 @@ func (t *tableSinkWrapper) doTableSinkClear() { t.tableSink.checkpointTs = checkpointTs } t.tableSink.resolvedTs = checkpointTs + t.tableSink.lastSyncedTs = t.tableSink.s.GetLastSyncedTs() t.tableSink.advanced = time.Now() t.tableSink.innerMu.Unlock() t.tableSink.s = nil diff --git a/cdc/scheduler/internal/scheduler.go b/cdc/scheduler/internal/scheduler.go index 70bff9bfeff..86e72570d5d 100644 --- a/cdc/scheduler/internal/scheduler.go +++ b/cdc/scheduler/internal/scheduler.go @@ -47,7 +47,7 @@ type Scheduler interface { // ddl jobs that need to be replicated. The Scheduler will // broadcast the barrierTs to all captures through the Heartbeat. barrier *schedulepb.BarrierWithMinTs, - ) (newCheckpointTs, newResolvedTs, newLastSyncedTs, newPullerIngressResolvedTs model.Ts, err error) + ) (newCheckpointTs, newResolvedTs, newLastSyncedTs, newPullerResolvedTs model.Ts, err error) // MoveTable requests that a table be moved to target. // It is thread-safe. diff --git a/cdc/scheduler/internal/v3/coordinator.go b/cdc/scheduler/internal/v3/coordinator.go index a2312401e2d..211f293c831 100644 --- a/cdc/scheduler/internal/v3/coordinator.go +++ b/cdc/scheduler/internal/v3/coordinator.go @@ -126,7 +126,7 @@ func (c *coordinator) Tick( // All captures that are alive according to the latest Etcd states. aliveCaptures map[model.CaptureID]*model.CaptureInfo, barrier *schedulepb.BarrierWithMinTs, -) (newCheckpointTs, newResolvedTs, newLastSyncedTs, newPullerIngressResolvedTs model.Ts, err error) { +) (newCheckpointTs, newResolvedTs, newLastSyncedTs, newPullerResolvedTs model.Ts, err error) { startTime := time.Now() defer func() { costTime := time.Since(startTime) @@ -272,7 +272,7 @@ func (c *coordinator) poll( currentTables []model.TableID, aliveCaptures map[model.CaptureID]*model.CaptureInfo, barrier *schedulepb.BarrierWithMinTs, -) (newCheckpointTs, newResolvedTs, newLastSyncedTs, newPullerIngressResolvedTs model.Ts, err error) { +) (newCheckpointTs, newResolvedTs, newLastSyncedTs, newPullerResolvedTs model.Ts, err error) { c.maybeCollectMetrics() if c.compat.UpdateCaptureInfo(aliveCaptures) { spanReplicationEnabled := c.compat.CheckSpanReplicationEnabled() @@ -310,13 +310,13 @@ func (c *coordinator) poll( // Skip generating schedule tasks for replication manager, // as not all capture are initialized. newCheckpointTs, newResolvedTs, newLastSyncedTs, - newPullerIngressResolvedTs = c.replicationM.AdvanceCheckpoint(&c.tableRanges, pdTime, barrier, c.redoMetaManager) + newPullerResolvedTs = c.replicationM.AdvanceCheckpoint(&c.tableRanges, pdTime, barrier, c.redoMetaManager) // tick capture manager after checkpoint calculation to take account resolvedTs in barrier // when redo is enabled msgs = c.captureM.Tick(c.replicationM.ReplicationSets(), c.schedulerM.DrainingTarget(), barrier.Barrier) msgBuf = append(msgBuf, msgs...) - return newCheckpointTs, newResolvedTs, newLastSyncedTs, newPullerIngressResolvedTs, c.sendMsgs(ctx, msgBuf) + return newCheckpointTs, newResolvedTs, newLastSyncedTs, newPullerResolvedTs, c.sendMsgs(ctx, msgBuf) } // Handle capture membership changes. @@ -346,7 +346,7 @@ func (c *coordinator) poll( // Checkpoint calculation newCheckpointTs, newResolvedTs, newLastSyncedTs, - newPullerIngressResolvedTs = c.replicationM.AdvanceCheckpoint(&c.tableRanges, pdTime, barrier, c.redoMetaManager) + newPullerResolvedTs = c.replicationM.AdvanceCheckpoint(&c.tableRanges, pdTime, barrier, c.redoMetaManager) // tick capture manager after checkpoint calculation to take account resolvedTs in barrier // when redo is enabled @@ -360,7 +360,7 @@ func (c *coordinator) poll( return checkpointCannotProceed, checkpointCannotProceed, checkpointCannotProceed, checkpointCannotProceed, errors.Trace(err) } - return newCheckpointTs, newResolvedTs, newLastSyncedTs, newPullerIngressResolvedTs, nil + return newCheckpointTs, newResolvedTs, newLastSyncedTs, newPullerResolvedTs, nil } func (c *coordinator) recvMsgs(ctx context.Context) ([]*schedulepb.Message, error) { diff --git a/cdc/scheduler/internal/v3/coordinator_test.go b/cdc/scheduler/internal/v3/coordinator_test.go index 8d691b4e01f..3a14bfa83ed 100644 --- a/cdc/scheduler/internal/v3/coordinator_test.go +++ b/cdc/scheduler/internal/v3/coordinator_test.go @@ -251,7 +251,7 @@ func TestCoordinatorHeartbeat(t *testing.T) { ctx := context.Background() currentTables := []model.TableID{1, 2, 3} aliveCaptures := map[model.CaptureID]*model.CaptureInfo{"a": {}, "b": {}} - _, _, err := coord.poll(ctx, 0, currentTables, aliveCaptures, schedulepb.NewBarrierWithMinTs(0)) + _, _, _, _, err := coord.poll(ctx, 0, currentTables, aliveCaptures, schedulepb.NewBarrierWithMinTs(0)) require.Nil(t, err) msgs := trans.SendBuffer require.Len(t, msgs, 2) @@ -283,7 +283,7 @@ func TestCoordinatorHeartbeat(t *testing.T) { }, }) trans.SendBuffer = []*schedulepb.Message{} - _, _, err = coord.poll(ctx, 0, currentTables, aliveCaptures, schedulepb.NewBarrierWithMinTs(0)) + _, _, _, _, err = coord.poll(ctx, 0, currentTables, aliveCaptures, schedulepb.NewBarrierWithMinTs(0)) require.Nil(t, err) require.True(t, coord.captureM.CheckAllCaptureInitialized()) msgs = trans.SendBuffer @@ -324,7 +324,7 @@ func TestCoordinatorAddCapture(t *testing.T) { ctx := context.Background() currentTables := []model.TableID{1, 2, 3} aliveCaptures := map[model.CaptureID]*model.CaptureInfo{"a": {}, "b": {}} - _, _, err = coord.poll(ctx, 0, currentTables, aliveCaptures, schedulepb.NewBarrierWithMinTs(0)) + _, _, _, _, err = coord.poll(ctx, 0, currentTables, aliveCaptures, schedulepb.NewBarrierWithMinTs(0)) require.Nil(t, err) msgs = trans.SendBuffer require.Len(t, msgs, 1) @@ -340,7 +340,7 @@ func TestCoordinatorAddCapture(t *testing.T) { HeartbeatResponse: &schedulepb.HeartbeatResponse{}, }) trans.SendBuffer = []*schedulepb.Message{} - _, _, err = coord.poll(ctx, 0, currentTables, aliveCaptures, schedulepb.NewBarrierWithMinTs(0)) + _, _, _, _, err = coord.poll(ctx, 0, currentTables, aliveCaptures, schedulepb.NewBarrierWithMinTs(0)) require.Nil(t, err) msgs = trans.SendBuffer require.Len(t, msgs, 1) @@ -381,7 +381,7 @@ func TestCoordinatorRemoveCapture(t *testing.T) { ctx := context.Background() currentTables := []model.TableID{1, 2, 3} aliveCaptures := map[model.CaptureID]*model.CaptureInfo{"a": {}, "b": {}} - _, _, err = coord.poll(ctx, 0, currentTables, aliveCaptures, schedulepb.NewBarrierWithMinTs(0)) + _, _, _, _, err = coord.poll(ctx, 0, currentTables, aliveCaptures, schedulepb.NewBarrierWithMinTs(0)) require.Nil(t, err) msgs = trans.SendBuffer require.Len(t, msgs, 1) @@ -456,7 +456,7 @@ func TestCoordinatorAdvanceCheckpoint(t *testing.T) { ctx := context.Background() currentTables := []model.TableID{1, 2} aliveCaptures := map[model.CaptureID]*model.CaptureInfo{"a": {}, "b": {}} - _, _, err := coord.poll(ctx, 0, currentTables, aliveCaptures, schedulepb.NewBarrierWithMinTs(0)) + _, _, _, _, err := coord.poll(ctx, 0, currentTables, aliveCaptures, schedulepb.NewBarrierWithMinTs(0)) require.Nil(t, err) // Initialize captures. @@ -482,24 +482,40 @@ func TestCoordinatorAdvanceCheckpoint(t *testing.T) { Span: spanz.TableIDToComparableSpan(1), State: tablepb.TableStateReplicating, Checkpoint: tablepb.Checkpoint{ - CheckpointTs: 2, ResolvedTs: 4, + CheckpointTs: 2, ResolvedTs: 4, LastSyncedTs: 3, + }, + Stats: tablepb.Stats{ + StageCheckpoints: map[string]tablepb.Checkpoint{ + "puller-egress": { + ResolvedTs: model.Ts(5), + }, + }, }, }, { Span: spanz.TableIDToComparableSpan(2), State: tablepb.TableStateReplicating, Checkpoint: tablepb.Checkpoint{ - CheckpointTs: 2, ResolvedTs: 4, + CheckpointTs: 2, ResolvedTs: 4, LastSyncedTs: 4, + }, + Stats: tablepb.Stats{ + StageCheckpoints: map[string]tablepb.Checkpoint{ + "puller-egress": { + ResolvedTs: model.Ts(6), + }, + }, }, }, }, }, }) - cts, rts, err := coord.poll(ctx, 0, currentTables, aliveCaptures, schedulepb.NewBarrierWithMinTs(5)) + cts, rts, lastSyncedTs, pullerResolvedTs, err := coord.poll(ctx, 0, currentTables, aliveCaptures, schedulepb.NewBarrierWithMinTs(5)) require.Nil(t, err) require.True(t, coord.captureM.CheckAllCaptureInitialized()) require.EqualValues(t, 2, cts) require.EqualValues(t, 4, rts) + require.EqualValues(t, 4, lastSyncedTs) + require.EqualValues(t, 5, pullerResolvedTs) // Checkpoint should be advanced even if there is an uninitialized capture. aliveCaptures["c"] = &model.CaptureInfo{} @@ -517,24 +533,40 @@ func TestCoordinatorAdvanceCheckpoint(t *testing.T) { Span: spanz.TableIDToComparableSpan(1), State: tablepb.TableStateReplicating, Checkpoint: tablepb.Checkpoint{ - CheckpointTs: 3, ResolvedTs: 5, + CheckpointTs: 3, ResolvedTs: 5, LastSyncedTs: 4, + }, + Stats: tablepb.Stats{ + StageCheckpoints: map[string]tablepb.Checkpoint{ + "puller-egress": { + ResolvedTs: model.Ts(7), + }, + }, }, }, { Span: spanz.TableIDToComparableSpan(2), State: tablepb.TableStateReplicating, Checkpoint: tablepb.Checkpoint{ - CheckpointTs: 4, ResolvedTs: 5, + CheckpointTs: 4, ResolvedTs: 5, LastSyncedTs: 6, + }, + Stats: tablepb.Stats{ + StageCheckpoints: map[string]tablepb.Checkpoint{ + "puller-egress": { + ResolvedTs: model.Ts(7), + }, + }, }, }, }, }, }) - cts, rts, err = coord.poll(ctx, 0, currentTables, aliveCaptures, schedulepb.NewBarrierWithMinTs(5)) + cts, rts, lastSyncedTs, pullerResolvedTs, err = coord.poll(ctx, 0, currentTables, aliveCaptures, schedulepb.NewBarrierWithMinTs(5)) require.Nil(t, err) require.False(t, coord.captureM.CheckAllCaptureInitialized()) require.EqualValues(t, 3, cts) require.EqualValues(t, 5, rts) + require.EqualValues(t, 6, lastSyncedTs) + require.EqualValues(t, 7, pullerResolvedTs) } func TestCoordinatorDropMsgIfChangefeedEpochMismatch(t *testing.T) { diff --git a/cdc/scheduler/internal/v3/replication/replication_manager.go b/cdc/scheduler/internal/v3/replication/replication_manager.go index 4b96567d665..84101e07b3e 100644 --- a/cdc/scheduler/internal/v3/replication/replication_manager.go +++ b/cdc/scheduler/internal/v3/replication/replication_manager.go @@ -557,10 +557,10 @@ func (r *Manager) AdvanceCheckpoint( currentPDTime time.Time, barrier *schedulepb.BarrierWithMinTs, redoMetaManager redo.MetaManager, -) (newCheckpointTs, newResolvedTs, newLastSyncedTs, newPullerIngressResolvedTs model.Ts) { +) (newCheckpointTs, newResolvedTs, newLastSyncedTs, newPullerResolvedTs model.Ts) { var redoFlushedResolvedTs model.Ts limitBarrierWithRedo := func(newCheckpointTs, newResolvedTs, newLastSyncedTs, - newPullerIngressResolvedTs uint64, + newPullerResolvedTs uint64, ) (uint64, uint64, uint64, uint64) { flushedMeta := redoMetaManager.GetFlushedMeta() redoFlushedResolvedTs = flushedMeta.ResolvedTs @@ -580,7 +580,7 @@ func (r *Manager) AdvanceCheckpoint( if barrier.GlobalBarrierTs > newResolvedTs { barrier.GlobalBarrierTs = newResolvedTs } - return newCheckpointTs, newResolvedTs, newLastSyncedTs, newPullerIngressResolvedTs + return newCheckpointTs, newResolvedTs, newLastSyncedTs, newPullerResolvedTs } defer func() { if redoFlushedResolvedTs != 0 && barrier.GlobalBarrierTs > redoFlushedResolvedTs { @@ -596,8 +596,7 @@ func (r *Manager) AdvanceCheckpoint( r.slowestSink = tablepb.Span{} var slowestPullerResolvedTs uint64 = math.MaxUint64 - // newPullerIngressResolvedTs to record the min ingress resolved ts of all pullers - newCheckpointTs, newResolvedTs, newLastSyncedTs, newPullerIngressResolvedTs = math.MaxUint64, math.MaxUint64, 0, math.MaxUint64 + newCheckpointTs, newResolvedTs, newLastSyncedTs = math.MaxUint64, math.MaxUint64, 0 cannotProceed := false currentTables.Iter(func(tableID model.TableID, tableStart, tableEnd tablepb.Span) bool { @@ -645,13 +644,6 @@ func (r *Manager) AdvanceCheckpoint( } } - // Find the minimum puller ingress resolved ts. - if pullerIngressCkpt, ok := table.Stats.StageCheckpoints["puller-ingress"]; ok { - if newPullerIngressResolvedTs > pullerIngressCkpt.ResolvedTs { - newPullerIngressResolvedTs = pullerIngressCkpt.ResolvedTs - } - } - return true }) if !tableSpanFound || !tableSpanStartFound || !tableSpanEndFound || tableHasHole { @@ -680,7 +672,7 @@ func (r *Manager) AdvanceCheckpoint( if redoMetaManager.Enabled() { // If redo is enabled, GlobalBarrierTs should be limited by redo flushed meta. newResolvedTs = barrier.RedoBarrierTs - limitBarrierWithRedo(newCheckpointTs, newResolvedTs, newLastSyncedTs, newPullerIngressResolvedTs) + limitBarrierWithRedo(newCheckpointTs, newResolvedTs, checkpointCannotProceed, checkpointCannotProceed) } return checkpointCannotProceed, checkpointCannotProceed, checkpointCannotProceed, checkpointCannotProceed } @@ -730,10 +722,10 @@ func (r *Manager) AdvanceCheckpoint( zap.String("changefeed", r.changefeedID.ID), zap.Uint64("newCheckpointTs", newCheckpointTs), zap.Uint64("newResolvedTs", newResolvedTs)) - return limitBarrierWithRedo(newCheckpointTs, newResolvedTs, newLastSyncedTs, newPullerIngressResolvedTs) + return limitBarrierWithRedo(newCheckpointTs, newResolvedTs, newLastSyncedTs, slowestPullerResolvedTs) } - return newCheckpointTs, newResolvedTs, newLastSyncedTs, newPullerIngressResolvedTs + return newCheckpointTs, newResolvedTs, newLastSyncedTs, slowestPullerResolvedTs } func (r *Manager) logSlowTableInfo(currentPDTime time.Time) { diff --git a/cdc/scheduler/internal/v3/replication/replication_manager_test.go b/cdc/scheduler/internal/v3/replication/replication_manager_test.go index 632a0835aa3..9e0791d99ef 100644 --- a/cdc/scheduler/internal/v3/replication/replication_manager_test.go +++ b/cdc/scheduler/internal/v3/replication/replication_manager_test.go @@ -15,6 +15,7 @@ package replication import ( "context" + "math" "testing" "time" @@ -620,6 +621,14 @@ func TestReplicationManagerAdvanceCheckpoint(t *testing.T) { Checkpoint: tablepb.Checkpoint{ CheckpointTs: model.Ts(10), ResolvedTs: model.Ts(20), + LastSyncedTs: model.Ts(15), + }, + Stats: tablepb.Stats{ + StageCheckpoints: map[string]tablepb.Checkpoint{ + "puller-egress": { + ResolvedTs: model.Ts(30), + }, + }, }, }, }, model.ChangeFeedID{}) @@ -635,6 +644,14 @@ func TestReplicationManagerAdvanceCheckpoint(t *testing.T) { Checkpoint: tablepb.Checkpoint{ CheckpointTs: model.Ts(15), ResolvedTs: model.Ts(30), + LastSyncedTs: model.Ts(20), + }, + Stats: tablepb.Stats{ + StageCheckpoints: map[string]tablepb.Checkpoint{ + "puller-egress": { + ResolvedTs: model.Ts(40), + }, + }, }, }, }, model.ChangeFeedID{}) @@ -646,21 +663,27 @@ func TestReplicationManagerAdvanceCheckpoint(t *testing.T) { // no tables are replicating, resolvedTs should be advanced to globalBarrierTs and checkpoint // should be advanced to minTableBarrierTs. currentTables := &TableRanges{} - checkpoint, resolved := r.AdvanceCheckpoint(currentTables, time.Now(), schedulepb.NewBarrierWithMinTs(5), redoMetaManager) + checkpoint, resolved, lastSyncedTs, pullerResolvedTs := r.AdvanceCheckpoint(currentTables, time.Now(), schedulepb.NewBarrierWithMinTs(5), redoMetaManager) require.Equal(t, model.Ts(5), checkpoint) require.Equal(t, model.Ts(5), resolved) + require.Equal(t, model.Ts(0), lastSyncedTs) + require.Equal(t, model.Ts(math.MaxUint64), pullerResolvedTs) // all tables are replicating currentTables.UpdateTables([]model.TableID{1, 2}) - checkpoint, resolved = r.AdvanceCheckpoint(currentTables, time.Now(), schedulepb.NewBarrierWithMinTs(30), redoMetaManager) + checkpoint, resolved, lastSyncedTs, pullerResolvedTs = r.AdvanceCheckpoint(currentTables, time.Now(), schedulepb.NewBarrierWithMinTs(30), redoMetaManager) require.Equal(t, model.Ts(10), checkpoint) require.Equal(t, model.Ts(20), resolved) + require.Equal(t, model.Ts(20), lastSyncedTs) + require.Equal(t, model.Ts(30), pullerResolvedTs) // some table not exist yet. currentTables.UpdateTables([]model.TableID{1, 2, 3}) - checkpoint, resolved = r.AdvanceCheckpoint(currentTables, time.Now(), schedulepb.NewBarrierWithMinTs(30), redoMetaManager) + checkpoint, resolved, lastSyncedTs, pullerResolvedTs = r.AdvanceCheckpoint(currentTables, time.Now(), schedulepb.NewBarrierWithMinTs(30), redoMetaManager) require.Equal(t, checkpointCannotProceed, checkpoint) require.Equal(t, checkpointCannotProceed, resolved) + require.Equal(t, checkpointCannotProceed, lastSyncedTs) + require.Equal(t, checkpointCannotProceed, pullerResolvedTs) span3 := spanz.TableIDToComparableSpan(3) rs, err = NewReplicationSet(span3, model.Ts(5), @@ -671,6 +694,14 @@ func TestReplicationManagerAdvanceCheckpoint(t *testing.T) { Checkpoint: tablepb.Checkpoint{ CheckpointTs: model.Ts(5), ResolvedTs: model.Ts(40), + LastSyncedTs: model.Ts(30), + }, + Stats: tablepb.Stats{ + StageCheckpoints: map[string]tablepb.Checkpoint{ + "puller-egress": { + ResolvedTs: model.Ts(50), + }, + }, }, }, "2": { @@ -679,14 +710,24 @@ func TestReplicationManagerAdvanceCheckpoint(t *testing.T) { Checkpoint: tablepb.Checkpoint{ CheckpointTs: model.Ts(5), ResolvedTs: model.Ts(40), + LastSyncedTs: model.Ts(32), + }, + Stats: tablepb.Stats{ + StageCheckpoints: map[string]tablepb.Checkpoint{ + "puller-egress": { + ResolvedTs: model.Ts(50), + }, + }, }, }, }, model.ChangeFeedID{}) require.NoError(t, err) r.spans.ReplaceOrInsert(span3, rs) - checkpoint, resolved = r.AdvanceCheckpoint(currentTables, time.Now(), schedulepb.NewBarrierWithMinTs(30), redoMetaManager) + checkpoint, resolved, lastSyncedTs, pullerResolvedTs = r.AdvanceCheckpoint(currentTables, time.Now(), schedulepb.NewBarrierWithMinTs(30), redoMetaManager) require.Equal(t, model.Ts(5), checkpoint) require.Equal(t, model.Ts(20), resolved) + require.Equal(t, model.Ts(32), lastSyncedTs) + require.Equal(t, model.Ts(30), pullerResolvedTs) currentTables.UpdateTables([]model.TableID{1, 2, 3, 4}) span4 := spanz.TableIDToComparableSpan(4) @@ -698,14 +739,24 @@ func TestReplicationManagerAdvanceCheckpoint(t *testing.T) { Checkpoint: tablepb.Checkpoint{ CheckpointTs: model.Ts(3), ResolvedTs: model.Ts(10), + LastSyncedTs: model.Ts(5), + }, + Stats: tablepb.Stats{ + StageCheckpoints: map[string]tablepb.Checkpoint{ + "puller-egress": { + ResolvedTs: model.Ts(12), + }, + }, }, }, }, model.ChangeFeedID{}) require.NoError(t, err) r.spans.ReplaceOrInsert(span4, rs) - checkpoint, resolved = r.AdvanceCheckpoint(currentTables, time.Now(), schedulepb.NewBarrierWithMinTs(30), redoMetaManager) + checkpoint, resolved, lastSyncedTs, pullerResolvedTs = r.AdvanceCheckpoint(currentTables, time.Now(), schedulepb.NewBarrierWithMinTs(30), redoMetaManager) require.Equal(t, model.Ts(3), checkpoint) require.Equal(t, model.Ts(10), resolved) + require.Equal(t, model.Ts(32), lastSyncedTs) + require.Equal(t, model.Ts(12), pullerResolvedTs) // Split table 5 into 2 spans. currentTables.UpdateTables([]model.TableID{1, 2, 3, 4, 5}) @@ -722,28 +773,42 @@ func TestReplicationManagerAdvanceCheckpoint(t *testing.T) { Checkpoint: tablepb.Checkpoint{ CheckpointTs: model.Ts(3), ResolvedTs: model.Ts(10), + LastSyncedTs: model.Ts(8), + }, + Stats: tablepb.Stats{ + StageCheckpoints: map[string]tablepb.Checkpoint{ + "puller-egress": { + ResolvedTs: model.Ts(11), + }, + }, }, }, }, model.ChangeFeedID{}) require.NoError(t, err) r.spans.ReplaceOrInsert(span, rs) } - checkpoint, resolved = r.AdvanceCheckpoint(currentTables, time.Now(), schedulepb.NewBarrierWithMinTs(30), redoMetaManager) + checkpoint, resolved, lastSyncedTs, pullerResolvedTs = r.AdvanceCheckpoint(currentTables, time.Now(), schedulepb.NewBarrierWithMinTs(30), redoMetaManager) require.Equal(t, model.Ts(3), checkpoint) require.Equal(t, model.Ts(10), resolved) + require.Equal(t, model.Ts(32), lastSyncedTs) + require.Equal(t, model.Ts(11), pullerResolvedTs) // The start span is missing rs5_1, _ := r.spans.Delete(span5_1) - checkpoint, resolved = r.AdvanceCheckpoint(currentTables, time.Now(), schedulepb.NewBarrierWithMinTs(30), redoMetaManager) + checkpoint, resolved, lastSyncedTs, pullerResolvedTs = r.AdvanceCheckpoint(currentTables, time.Now(), schedulepb.NewBarrierWithMinTs(30), redoMetaManager) require.Equal(t, checkpointCannotProceed, checkpoint) require.Equal(t, checkpointCannotProceed, resolved) + require.Equal(t, checkpointCannotProceed, lastSyncedTs) + require.Equal(t, checkpointCannotProceed, pullerResolvedTs) // The end span is missing r.spans.ReplaceOrInsert(span5_1, rs5_1) r.spans.Delete(span5_2) - checkpoint, resolved = r.AdvanceCheckpoint(currentTables, time.Now(), schedulepb.NewBarrierWithMinTs(30), redoMetaManager) + checkpoint, resolved, lastSyncedTs, pullerResolvedTs = r.AdvanceCheckpoint(currentTables, time.Now(), schedulepb.NewBarrierWithMinTs(30), redoMetaManager) require.Equal(t, checkpointCannotProceed, checkpoint) require.Equal(t, checkpointCannotProceed, resolved) + require.Equal(t, checkpointCannotProceed, lastSyncedTs) + require.Equal(t, checkpointCannotProceed, pullerResolvedTs) // redo is enabled currentTables.UpdateTables([]model.TableID{4}) @@ -756,6 +821,14 @@ func TestReplicationManagerAdvanceCheckpoint(t *testing.T) { Checkpoint: tablepb.Checkpoint{ CheckpointTs: model.Ts(10), ResolvedTs: model.Ts(15), + LastSyncedTs: model.Ts(12), + }, + Stats: tablepb.Stats{ + StageCheckpoints: map[string]tablepb.Checkpoint{ + "puller-egress": { + ResolvedTs: model.Ts(16), + }, + }, }, }, }, model.ChangeFeedID{}) @@ -764,9 +837,11 @@ func TestReplicationManagerAdvanceCheckpoint(t *testing.T) { barrier := schedulepb.NewBarrierWithMinTs(30) redoMetaManager.enable = true redoMetaManager.resolvedTs = 9 - checkpoint, resolved = r.AdvanceCheckpoint(currentTables, time.Now(), barrier, redoMetaManager) + checkpoint, resolved, lastSyncedTs, pullerResolvedTs = r.AdvanceCheckpoint(currentTables, time.Now(), barrier, redoMetaManager) require.Equal(t, model.Ts(9), resolved) require.Equal(t, model.Ts(9), checkpoint) + require.Equal(t, model.Ts(12), lastSyncedTs) + require.Equal(t, model.Ts(16), pullerResolvedTs) require.Equal(t, model.Ts(9), barrier.GetGlobalBarrierTs()) } @@ -782,6 +857,14 @@ func TestReplicationManagerAdvanceCheckpointWithRedoEnabled(t *testing.T) { Checkpoint: tablepb.Checkpoint{ CheckpointTs: model.Ts(10), ResolvedTs: model.Ts(20), + LastSyncedTs: model.Ts(12), + }, + Stats: tablepb.Stats{ + StageCheckpoints: map[string]tablepb.Checkpoint{ + "puller-egress": { + ResolvedTs: model.Ts(26), + }, + }, }, }, }, model.ChangeFeedID{}) @@ -797,6 +880,14 @@ func TestReplicationManagerAdvanceCheckpointWithRedoEnabled(t *testing.T) { Checkpoint: tablepb.Checkpoint{ CheckpointTs: model.Ts(15), ResolvedTs: model.Ts(30), + LastSyncedTs: model.Ts(18), + }, + Stats: tablepb.Stats{ + StageCheckpoints: map[string]tablepb.Checkpoint{ + "puller-egress": { + ResolvedTs: model.Ts(39), + }, + }, }, }, }, model.ChangeFeedID{}) @@ -809,9 +900,11 @@ func TestReplicationManagerAdvanceCheckpointWithRedoEnabled(t *testing.T) { currentTables := &TableRanges{} currentTables.UpdateTables([]model.TableID{1, 2, 3}) barrier := schedulepb.NewBarrierWithMinTs(30) - checkpoint, resolved := r.AdvanceCheckpoint(currentTables, time.Now(), barrier, redoMetaManager) + checkpoint, resolved, lastSyncedTs, pullerResolvedTs := r.AdvanceCheckpoint(currentTables, time.Now(), barrier, redoMetaManager) require.Equal(t, checkpointCannotProceed, checkpoint) require.Equal(t, checkpointCannotProceed, resolved) + require.Equal(t, checkpointCannotProceed, lastSyncedTs) + require.Equal(t, checkpointCannotProceed, pullerResolvedTs) require.Equal(t, uint64(25), barrier.Barrier.GetGlobalBarrierTs()) } From 37add5b3746cfecb341e33d9190401872292c95e Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Tue, 28 Nov 2023 17:00:39 +0800 Subject: [PATCH 06/35] update test --- cdc/api/v2/api_test.go | 25 ++- cdc/api/v2/changefeed.go | 94 ++++---- cdc/api/v2/changefeed_test.go | 212 ++++++++++++++++++ cdc/api/v2/model.go | 9 + cdc/model/changefeed.go | 7 +- cdc/owner/changefeed.go | 27 +-- cdc/owner/owner.go | 8 +- .../replication/replication_manager_test.go | 31 ++- 8 files changed, 334 insertions(+), 79 deletions(-) diff --git a/cdc/api/v2/api_test.go b/cdc/api/v2/api_test.go index 96929887ae6..3ab0c9d0291 100644 --- a/cdc/api/v2/api_test.go +++ b/cdc/api/v2/api_test.go @@ -49,7 +49,7 @@ func (c *mockPDClient) UpdateServiceGCSafePoint(ctx context.Context, // GetTS of mockPDClient returns a mock tso func (c *mockPDClient) GetTS(ctx context.Context) (int64, int64, error) { - return c.logicTime, c.timestamp, nil + return c.timestamp, c.logicTime, nil } // GetClusterID of mockPDClient returns a mock ClusterID @@ -62,13 +62,14 @@ func (c *mockPDClient) Close() {} type mockStatusProvider struct { owner.StatusProvider - changefeedStatus *model.ChangeFeedStatusForAPI - changefeedInfo *model.ChangeFeedInfo - processors []*model.ProcInfoSnap - taskStatus map[model.CaptureID]*model.TaskStatus - changefeedInfos map[model.ChangeFeedID]*model.ChangeFeedInfo - changefeedStatuses map[model.ChangeFeedID]*model.ChangeFeedStatusForAPI - err error + changefeedStatus *model.ChangeFeedStatusForAPI + changefeedInfo *model.ChangeFeedInfo + processors []*model.ProcInfoSnap + taskStatus map[model.CaptureID]*model.TaskStatus + changefeedInfos map[model.ChangeFeedID]*model.ChangeFeedInfo + changefeedStatuses map[model.ChangeFeedID]*model.ChangeFeedStatusForAPI + changeFeedSyncedStatusForAPI *model.ChangeFeedSyncedStatusForAPI + err error } // GetChangeFeedStatus returns a changefeeds' runtime status. @@ -120,6 +121,14 @@ func (m *mockStatusProvider) GetAllChangeFeedStatuses(_ context.Context) ( return m.changefeedStatuses, m.err } +// GetChangeFeedSyncedStatusForAPI returns a mock changefeed status. +func (m *mockStatusProvider) GetChangeFeedSyncedStatus(_ context.Context, changefeedID model.ChangeFeedID) ( + *model.ChangeFeedSyncedStatusForAPI, + error, +) { + return m.changeFeedSyncedStatusForAPI, m.err +} + func (m *mockStatusProvider) IsChangefeedOwner(_ context.Context, id model.ChangeFeedID) (bool, error) { return true, nil } diff --git a/cdc/api/v2/changefeed.go b/cdc/api/v2/changefeed.go index af27dcb1941..5472667fe0b 100644 --- a/cdc/api/v2/changefeed.go +++ b/cdc/api/v2/changefeed.go @@ -896,6 +896,12 @@ func (h *OpenAPIV2) status(c *gin.Context) { }) } +func transformerTime(timestamp int64) string { + location := time.Local + tm := time.Unix((timestamp / 1000), 0).In(location) + return tm.Format("2006-01-02 15:04:05") +} + func (h *OpenAPIV2) synced(c *gin.Context) { ctx := c.Request.Context() @@ -906,6 +912,7 @@ func (h *OpenAPIV2) synced(c *gin.Context) { changefeedID.ID)) return } + status, err := h.capture.StatusProvider().GetChangeFeedSyncedStatus( ctx, changefeedID, @@ -915,13 +922,14 @@ func (h *OpenAPIV2) synced(c *gin.Context) { return } + log.Info("synced status", zap.Any("status", status)) // get pd_now cfg := &ChangefeedConfig{ReplicaConfig: GetDefaultReplicaConfig()} - if err := c.BindJSON(&cfg); err != nil { - _ = c.Error(cerror.WrapError(cerror.ErrAPIInvalidParam, err)) - return - } + // if err := c.BindJSON(&cfg); err != nil { + // _ = c.Error(cerror.WrapError(cerror.ErrAPIInvalidParam, err)) + // return + // } if len(cfg.PDAddrs) == 0 { up, err := getCaptureDefaultUpstream(h.capture) if err != nil { @@ -934,63 +942,69 @@ func (h *OpenAPIV2) synced(c *gin.Context) { timeoutCtx, cancel := context.WithTimeout(ctx, 30*time.Second) defer cancel() - pdClient, err := h.helpers.getPDClient(timeoutCtx, cfg.PDAddrs, credential) - defer pdClient.Close() + pdClient, err := h.helpers.getPDClient(timeoutCtx, cfg.PDAddrs, credential) if err != nil { // pd 不可用 var message string - if (status.PullerResolvedTs - status.CheckpointTs) > (5*1000)<<18 { // 5s - message = fmt.Sprintf("we get pd client failed with err is %s. Besides the data is not finish syncing", terror.Message(err)) + if (status.PullerResolvedTs - status.CheckpointTs) > 5*1000 { // 5s + message = fmt.Sprintf("%s. Besides the data is not finish syncing", terror.Message(err)) } else { - message = fmt.Sprintf("we get pd client failed with err is %s. "+ - "You can check the pd first, and if pd is available, means we don't finish sync data. "+ + message = fmt.Sprintf("%s. You can check the pd first, and if pd is available, means we don't finish sync data. "+ "If pd is not available, please check the whether we satisfy the condition that"+ "The time difference from lastSyncedTs to the current time from the time zone of pd is greater than 5 min"+ "If it's satisfied, means the data syncing is totally finished", err) } - c.JSON(http.StatusOK, map[string]any{ - "Synced": false, - "Sink-CheckpointTs": status.CheckpointTs, - "Puller-ResolvedTs": status.PullerResolvedTs, - "LastSyncedTs": status.LastSyncedTs, - "info": message, + c.JSON(http.StatusOK, SyncedStatus{ + Synced: false, + SinkCheckpointTs: transformerTime(status.CheckpointTs), + PullerResolvedTs: transformerTime(status.PullerResolvedTs), + LastSyncedTs: transformerTime(status.LastSyncedTs), + NowTs: transformerTime(0), + Info: message, }) + return } + defer pdClient.Close() + + physical_now, _, _ := pdClient.GetTS(ctx) - physical, logical, _ := pdClient.GetTS(ctx) - now := oracle.ComposeTS(physical, logical) + log.Info("time info", zap.Int64("physical", physical_now), zap.Int64("checkpointTs", status.CheckpointTs), + zap.Int64("pullerResolvedTs", status.PullerResolvedTs), zap.Int64("LastSyncedTs", status.LastSyncedTs)) - if (now-status.LastSyncedTs > (5*60*1000)<<18) && (now-status.CheckpointTs < (5*1000)<<18) { // 达到 synced 严格条件 - c.JSON(http.StatusOK, map[string]any{ - "Synced": true, - "Sink-CheckpointTs": status.CheckpointTs, - "Puller-ResolvedTs": status.PullerResolvedTs, - "LastSyncedTs": status.LastSyncedTs, - "info": "Data syncing is finished", + if (physical_now-status.LastSyncedTs > 5*60*1000) && (physical_now-status.CheckpointTs < 5*1000) { // 达到 synced 严格条件 + c.JSON(http.StatusOK, SyncedStatus{ + Synced: true, + SinkCheckpointTs: transformerTime(status.CheckpointTs), + PullerResolvedTs: transformerTime(status.PullerResolvedTs), + LastSyncedTs: transformerTime(status.LastSyncedTs), + NowTs: transformerTime(physical_now), + Info: "Data syncing is finished", }) - } else if now-status.LastSyncedTs > (5*60*1000)<<18 { // lastSyncedTs 条件达到,checkpoint-ts 未达到 + } else if physical_now-status.LastSyncedTs > 5*60*1000 { // lastSyncedTs 条件达到,checkpoint-ts 未达到 var message string - if (status.PullerResolvedTs - status.CheckpointTs) > (5*1000)<<18 { // 5s + if (status.PullerResolvedTs - status.CheckpointTs) < 5*1000 { // 5s message = fmt.Sprintf("Please check whether pd is health and tikv region is all available. " + "If pd is not health or tikv region is not available, the data syncing is finished. " + " Otherwise the data syncing is not finished, please wait") } else { message = fmt.Sprintf("The data syncing is not finished, please wait") } - c.JSON(http.StatusOK, map[string]any{ - "Synced": false, - "Sink-CheckpointTs": status.CheckpointTs, - "Puller-ResolvedTs": status.PullerResolvedTs, - "LastSyncedTs": status.LastSyncedTs, - "info": message, + c.JSON(http.StatusOK, SyncedStatus{ + Synced: false, + SinkCheckpointTs: transformerTime(status.CheckpointTs), + PullerResolvedTs: transformerTime(status.PullerResolvedTs), + LastSyncedTs: transformerTime(status.LastSyncedTs), + NowTs: transformerTime(physical_now), + Info: message, }) - } else { // lastSyncedTs 条件达到 - c.JSON(http.StatusOK, map[string]any{ - "Synced": false, - "Sink-CheckpointTs": status.CheckpointTs, - "Puller-ResolvedTs": status.PullerResolvedTs, - "LastSyncedTs": status.LastSyncedTs, - "info": "The data syncing is not finished, please wait", + } else { // lastSyncedTs 条件未达到 + c.JSON(http.StatusOK, SyncedStatus{ + Synced: false, + SinkCheckpointTs: transformerTime(status.CheckpointTs), + PullerResolvedTs: transformerTime(status.PullerResolvedTs), + LastSyncedTs: transformerTime(status.LastSyncedTs), + NowTs: transformerTime(physical_now), + Info: "The data syncing is not finished, please wait", }) } } diff --git a/cdc/api/v2/changefeed_test.go b/cdc/api/v2/changefeed_test.go index 8ce4b13d4fa..0ba80d08e5b 100644 --- a/cdc/api/v2/changefeed_test.go +++ b/cdc/api/v2/changefeed_test.go @@ -32,6 +32,7 @@ import ( "github.com/pingcap/tiflow/cdc/model" mock_owner "github.com/pingcap/tiflow/cdc/owner/mock" "github.com/pingcap/tiflow/pkg/config" + cerror "github.com/pingcap/tiflow/pkg/errors" cerrors "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/etcd" mock_etcd "github.com/pingcap/tiflow/pkg/etcd/mock" @@ -963,6 +964,217 @@ func TestPauseChangefeed(t *testing.T) { require.Equal(t, "{}", w.Body.String()) } +func TestChangefeedSynced(t *testing.T) { + syncedInfo := testCase{url: "/api/v2/changefeeds/%s/synced?namespace=abc", method: "GET"} + helpers := NewMockAPIV2Helpers(gomock.NewController(t)) + cp := mock_capture.NewMockCapture(gomock.NewController(t)) + owner := mock_owner.NewMockOwner(gomock.NewController(t)) + apiV2 := NewOpenAPIV2ForTest(cp, helpers) + router := newRouter(apiV2) + + statusProvider := &mockStatusProvider{} + + cp.EXPECT().StatusProvider().Return(statusProvider).AnyTimes() + cp.EXPECT().IsReady().Return(true).AnyTimes() + cp.EXPECT().IsController().Return(true).AnyTimes() + cp.EXPECT().GetOwner().Return(owner, nil).AnyTimes() + + pdClient := &mockPDClient{} + mockUpManager := upstream.NewManager4Test(pdClient) + cp.EXPECT().GetUpstreamManager().Return(mockUpManager, nil).AnyTimes() + + { + // case 1: invalid changefeed id + w := httptest.NewRecorder() + invalidID := "@^Invalid" + req, _ := http.NewRequestWithContext(context.Background(), + syncedInfo.method, fmt.Sprintf(syncedInfo.url, invalidID), nil) + router.ServeHTTP(w, req) + respErr := model.HTTPError{} + err := json.NewDecoder(w.Body).Decode(&respErr) + require.Nil(t, err) + require.Contains(t, respErr.Code, "ErrAPIInvalidParam") + } + + { + // case 2: not existed changefeed id + validID := changeFeedID.ID + statusProvider.err = cerrors.ErrChangeFeedNotExists.GenWithStackByArgs(validID) + w := httptest.NewRecorder() + req, _ := http.NewRequestWithContext(context.Background(), syncedInfo.method, + fmt.Sprintf(syncedInfo.url, validID), nil) + router.ServeHTTP(w, req) + respErr := model.HTTPError{} + err := json.NewDecoder(w.Body).Decode(&respErr) + require.Nil(t, err) + require.Contains(t, respErr.Code, "ErrChangeFeedNotExists") + require.Equal(t, http.StatusBadRequest, w.Code) + } + + { + validID := changeFeedID.ID + helpers.EXPECT().getPDClient(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil, cerror.ErrAPIGetPDClientFailed).Times(1) + statusProvider.err = nil + statusProvider.changefeedInfo = &model.ChangeFeedInfo{ID: validID} + // case3: pd is offline,resolvedTs - checkpointTs > 5s + statusProvider.changeFeedSyncedStatusForAPI = &model.ChangeFeedSyncedStatusForAPI{ + CheckpointTs: 1701153217279, + LastSyncedTs: 1701153217279, + PullerResolvedTs: 1701153227279, + } + w := httptest.NewRecorder() + req, _ := http.NewRequestWithContext( + context.Background(), + syncedInfo.method, + fmt.Sprintf(syncedInfo.url, validID), + nil, + ) + router.ServeHTTP(w, req) + require.Equal(t, http.StatusOK, w.Code) + resp := SyncedStatus{} + err := json.NewDecoder(w.Body).Decode(&resp) + require.Nil(t, err) + require.Equal(t, false, resp.Synced) + require.Equal(t, "1970-01-01 08:00:00", resp.NowTs) + require.Equal(t, "[CDC:ErrAPIGetPDClientFailed]failed to get PDClient to connect PD, "+ + "please recheck. Besides the data is not finish syncing", resp.Info) + } + + { + validID := changeFeedID.ID + helpers.EXPECT().getPDClient(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil, cerror.ErrAPIGetPDClientFailed).Times(1) + statusProvider.err = nil + statusProvider.changefeedInfo = &model.ChangeFeedInfo{ID: validID} + // case4: pd is offline,resolvedTs - checkpointTs < 5s + statusProvider.changeFeedSyncedStatusForAPI = &model.ChangeFeedSyncedStatusForAPI{ + CheckpointTs: 1701153217279, + LastSyncedTs: 1701153217279, + PullerResolvedTs: 1701153217479, + } + w := httptest.NewRecorder() + req, _ := http.NewRequestWithContext( + context.Background(), + syncedInfo.method, + fmt.Sprintf(syncedInfo.url, validID), + nil, + ) + router.ServeHTTP(w, req) + require.Equal(t, http.StatusOK, w.Code) + resp := SyncedStatus{} + err := json.NewDecoder(w.Body).Decode(&resp) + require.Nil(t, err) + require.Equal(t, false, resp.Synced) + require.Equal(t, "1970-01-01 08:00:00", resp.NowTs) + require.Equal(t, "[CDC:ErrAPIGetPDClientFailed]failed to get PDClient to connect PD, please recheck. "+ + "You can check the pd first, and if pd is available, means we don't finish sync data. "+ + "If pd is not available, please check the whether we satisfy the condition that"+ + "The time difference from lastSyncedTs to the current time from the time zone of pd is greater than 5 min"+ + "If it's satisfied, means the data syncing is totally finished", resp.Info) + } + + helpers.EXPECT().getPDClient(gomock.Any(), gomock.Any(), gomock.Any()).Return(pdClient, nil).AnyTimes() + pdClient.logicTime = 1000 + pdClient.timestamp = 1701153217279 + statusProvider.err = nil + validID := changeFeedID.ID + statusProvider.changefeedInfo = &model.ChangeFeedInfo{ID: validID} + + { + // case5: pdTs - lastSyncedTs > 5min, pdTs - checkpointTs < 5s + statusProvider.changeFeedSyncedStatusForAPI = &model.ChangeFeedSyncedStatusForAPI{ + CheckpointTs: 1701153217209, + LastSyncedTs: 1701152217279, + PullerResolvedTs: 1701153217229, + } + w := httptest.NewRecorder() + req, _ := http.NewRequestWithContext( + context.Background(), + syncedInfo.method, + fmt.Sprintf(syncedInfo.url, validID), + nil, + ) + router.ServeHTTP(w, req) + require.Equal(t, http.StatusOK, w.Code) + resp := SyncedStatus{} + err := json.NewDecoder(w.Body).Decode(&resp) + require.Nil(t, err) + require.Equal(t, true, resp.Synced) + require.Equal(t, "2023-11-28 14:33:37", resp.NowTs) + require.Equal(t, "Data syncing is finished", resp.Info) + } + + { + // case6: pdTs - lastSyncedTs > 5min, pdTs - checkpointTs > 5s, resolvedTs - checkpointTs < 5s + statusProvider.changeFeedSyncedStatusForAPI = &model.ChangeFeedSyncedStatusForAPI{ + CheckpointTs: 1701153211279, + LastSyncedTs: 1701152217279, + PullerResolvedTs: 1701153211379, + } + w := httptest.NewRecorder() + req, _ := http.NewRequestWithContext( + context.Background(), + syncedInfo.method, + fmt.Sprintf(syncedInfo.url, validID), + nil, + ) + router.ServeHTTP(w, req) + require.Equal(t, http.StatusOK, w.Code) + resp := SyncedStatus{} + err := json.NewDecoder(w.Body).Decode(&resp) + require.Nil(t, err) + require.Equal(t, false, resp.Synced) + require.Equal(t, "Please check whether pd is health and tikv region is all available. "+ + "If pd is not health or tikv region is not available, the data syncing is finished. "+ + " Otherwise the data syncing is not finished, please wait", resp.Info) + } + + { + // case7: pdTs - lastSyncedTs > 5min, pdTs - checkpointTs > 5s, resolvedTs - checkpointTs > 5s + statusProvider.changeFeedSyncedStatusForAPI = &model.ChangeFeedSyncedStatusForAPI{ + CheckpointTs: 1701153211279, + LastSyncedTs: 1701152217279, + PullerResolvedTs: 1701153218279, + } + w := httptest.NewRecorder() + req, _ := http.NewRequestWithContext( + context.Background(), + syncedInfo.method, + fmt.Sprintf(syncedInfo.url, validID), + nil, + ) + router.ServeHTTP(w, req) + require.Equal(t, http.StatusOK, w.Code) + resp := SyncedStatus{} + err := json.NewDecoder(w.Body).Decode(&resp) + require.Nil(t, err) + require.Equal(t, false, resp.Synced) + require.Equal(t, "The data syncing is not finished, please wait", resp.Info) + } + + { + // case8: pdTs - lastSyncedTs < 5min + statusProvider.changeFeedSyncedStatusForAPI = &model.ChangeFeedSyncedStatusForAPI{ + CheckpointTs: 1701153217279, + LastSyncedTs: 1701153213279, + PullerResolvedTs: 1701153217279, + } + w := httptest.NewRecorder() + req, _ := http.NewRequestWithContext( + context.Background(), + syncedInfo.method, + fmt.Sprintf(syncedInfo.url, validID), + nil, + ) + router.ServeHTTP(w, req) + require.Equal(t, http.StatusOK, w.Code) + resp := SyncedStatus{} + err := json.NewDecoder(w.Body).Decode(&resp) + require.Nil(t, err) + require.Equal(t, false, resp.Synced) + require.Equal(t, "The data syncing is not finished, please wait", resp.Info) + } +} + func TestHasRunningImport(t *testing.T) { integration.BeforeTestExternal(t) testEtcdCluster := integration.NewClusterV3( diff --git a/cdc/api/v2/model.go b/cdc/api/v2/model.go index 9ff979894e6..5c08fbd9057 100644 --- a/cdc/api/v2/model.go +++ b/cdc/api/v2/model.go @@ -1011,6 +1011,15 @@ type ChangeFeedInfo struct { TaskStatus []model.CaptureTaskStatus `json:"task_status,omitempty"` } +type SyncedStatus struct { + Synced bool `json:"synced"` + SinkCheckpointTs string `json:"sink_checkpoint_ts"` + PullerResolvedTs string `json:"puller_resolved_ts"` + LastSyncedTs string `json:"last_synced_ts"` + NowTs string `json:"now_ts"` + Info string `json:"info"` +} + // RunningError represents some running error from cdc components, // such as processor. type RunningError struct { diff --git a/cdc/model/changefeed.go b/cdc/model/changefeed.go index 586fa0020e1..4ee230dfd37 100644 --- a/cdc/model/changefeed.go +++ b/cdc/model/changefeed.go @@ -631,8 +631,7 @@ type ChangeFeedStatusForAPI struct { } type ChangeFeedSyncedStatusForAPI struct { - ResolvedTs uint64 `json:"resolved-ts"` - CheckpointTs uint64 `json:"checkpoint-ts"` - LastSyncedTs uint64 `json:"last-sync-time"` - PullerResolvedTs uint64 `json:"puller-resolved-ts"` + CheckpointTs int64 `json:"checkpoint-ts"` + LastSyncedTs int64 `json:"last-sync-time"` + PullerResolvedTs int64 `json:"puller-resolved-ts"` } diff --git a/cdc/owner/changefeed.go b/cdc/owner/changefeed.go index 93ac2fba83a..0cf902ac425 100644 --- a/cdc/owner/changefeed.go +++ b/cdc/owner/changefeed.go @@ -16,6 +16,7 @@ package owner import ( "context" "fmt" + "math" "strings" "sync" "time" @@ -433,7 +434,7 @@ func (c *changefeed) tick(ctx cdcContext.Context, zap.Uint64("newLastSyncedTs", newLastSyncedTs)) } - if newPullerResolvedTs != scheduler.CheckpointCannotProceed { + if newPullerResolvedTs != scheduler.CheckpointCannotProceed && newPullerResolvedTs != math.MaxUint64 { if newPullerResolvedTs > c.pullerResolvedTs { c.pullerResolvedTs = newPullerResolvedTs } else if newPullerResolvedTs < c.pullerResolvedTs { @@ -525,19 +526,19 @@ LOOP2: c.resolvedTs = checkpointTs } - if c.lastSyncedTs == 0 { - // Set LastSyncedTs with current pd time when do initialize. + // if c.lastSyncedTs == 0 { + // // Set LastSyncedTs with current pd time when do initialize. - // we don't save lastSyncedTs in etcd because we want to reduce the number of etcd write. - // Based on the assumption that owner will not be replaced frequently, - // and we only change owners when oom or some panic happens, - // use pd time to initialize lastSyncedTs can work well enough. - // Even if there are no more data send into ticdc after changing owner, - // we just need to wait synced-check-time to reach synced = true. - // We regard the situation never happens that owner is replaced frequently and then leading to - // lastSyncedTs always increase even if there are no more data send into ticdc. - c.lastSyncedTs = uint64(oracle.GetPhysical(c.upstream.PDClock.CurrentTime())) - } + // // we don't save lastSyncedTs in etcd because we want to reduce the number of etcd write. + // // Based on the assumption that owner will not be replaced frequently, + // // and we only change owners when oom or some panic happens, + // // use pd time to initialize lastSyncedTs can work well enough. + // // Even if there are no more data send into ticdc after changing owner, + // // we just need to wait synced-check-time to reach synced = true. + // // We regard the situation never happens that owner is replaced frequently and then leading to + // // lastSyncedTs always increase even if there are no more data send into ticdc. + // c.lastSyncedTs = uint64(oracle.GetPhysical(c.upstream.PDClock.CurrentTime())) + // } minTableBarrierTs := c.latestStatus.MinTableBarrierTs diff --git a/cdc/owner/owner.go b/cdc/owner/owner.go index 99571338523..1db45f66306 100644 --- a/cdc/owner/owner.go +++ b/cdc/owner/owner.go @@ -32,6 +32,7 @@ import ( "github.com/pingcap/tiflow/pkg/orchestrator" "github.com/pingcap/tiflow/pkg/upstream" "github.com/pingcap/tiflow/pkg/version" + "github.com/tikv/client-go/v2/oracle" "go.uber.org/zap" "golang.org/x/time/rate" ) @@ -604,10 +605,9 @@ func (o *ownerImpl) handleQueries(query *Query) error { return nil } ret := &model.ChangeFeedSyncedStatusForAPI{} - ret.ResolvedTs = cfReactor.resolvedTs - ret.LastSyncedTs = cfReactor.lastSyncedTs - ret.CheckpointTs = cfReactor.latestStatus.CheckpointTs - ret.PullerResolvedTs = cfReactor.pullerResolvedTs + ret.LastSyncedTs = oracle.ExtractPhysical(cfReactor.lastSyncedTs) + ret.CheckpointTs = oracle.ExtractPhysical(cfReactor.latestStatus.CheckpointTs) + ret.PullerResolvedTs = oracle.ExtractPhysical(cfReactor.pullerResolvedTs) query.Data = ret case QueryChangefeedInfo: cfReactor, ok := o.changefeeds[query.ChangeFeedID] diff --git a/cdc/scheduler/internal/v3/replication/replication_manager_test.go b/cdc/scheduler/internal/v3/replication/replication_manager_test.go index 9e0791d99ef..050b07f9707 100644 --- a/cdc/scheduler/internal/v3/replication/replication_manager_test.go +++ b/cdc/scheduler/internal/v3/replication/replication_manager_test.go @@ -663,7 +663,8 @@ func TestReplicationManagerAdvanceCheckpoint(t *testing.T) { // no tables are replicating, resolvedTs should be advanced to globalBarrierTs and checkpoint // should be advanced to minTableBarrierTs. currentTables := &TableRanges{} - checkpoint, resolved, lastSyncedTs, pullerResolvedTs := r.AdvanceCheckpoint(currentTables, time.Now(), schedulepb.NewBarrierWithMinTs(5), redoMetaManager) + checkpoint, resolved, lastSyncedTs, pullerResolvedTs := r.AdvanceCheckpoint( + currentTables, time.Now(), schedulepb.NewBarrierWithMinTs(5), redoMetaManager) require.Equal(t, model.Ts(5), checkpoint) require.Equal(t, model.Ts(5), resolved) require.Equal(t, model.Ts(0), lastSyncedTs) @@ -671,7 +672,8 @@ func TestReplicationManagerAdvanceCheckpoint(t *testing.T) { // all tables are replicating currentTables.UpdateTables([]model.TableID{1, 2}) - checkpoint, resolved, lastSyncedTs, pullerResolvedTs = r.AdvanceCheckpoint(currentTables, time.Now(), schedulepb.NewBarrierWithMinTs(30), redoMetaManager) + checkpoint, resolved, lastSyncedTs, pullerResolvedTs = r.AdvanceCheckpoint( + currentTables, time.Now(), schedulepb.NewBarrierWithMinTs(30), redoMetaManager) require.Equal(t, model.Ts(10), checkpoint) require.Equal(t, model.Ts(20), resolved) require.Equal(t, model.Ts(20), lastSyncedTs) @@ -679,7 +681,8 @@ func TestReplicationManagerAdvanceCheckpoint(t *testing.T) { // some table not exist yet. currentTables.UpdateTables([]model.TableID{1, 2, 3}) - checkpoint, resolved, lastSyncedTs, pullerResolvedTs = r.AdvanceCheckpoint(currentTables, time.Now(), schedulepb.NewBarrierWithMinTs(30), redoMetaManager) + checkpoint, resolved, lastSyncedTs, pullerResolvedTs = r.AdvanceCheckpoint( + currentTables, time.Now(), schedulepb.NewBarrierWithMinTs(30), redoMetaManager) require.Equal(t, checkpointCannotProceed, checkpoint) require.Equal(t, checkpointCannotProceed, resolved) require.Equal(t, checkpointCannotProceed, lastSyncedTs) @@ -723,7 +726,8 @@ func TestReplicationManagerAdvanceCheckpoint(t *testing.T) { }, model.ChangeFeedID{}) require.NoError(t, err) r.spans.ReplaceOrInsert(span3, rs) - checkpoint, resolved, lastSyncedTs, pullerResolvedTs = r.AdvanceCheckpoint(currentTables, time.Now(), schedulepb.NewBarrierWithMinTs(30), redoMetaManager) + checkpoint, resolved, lastSyncedTs, pullerResolvedTs = r.AdvanceCheckpoint( + currentTables, time.Now(), schedulepb.NewBarrierWithMinTs(30), redoMetaManager) require.Equal(t, model.Ts(5), checkpoint) require.Equal(t, model.Ts(20), resolved) require.Equal(t, model.Ts(32), lastSyncedTs) @@ -752,7 +756,8 @@ func TestReplicationManagerAdvanceCheckpoint(t *testing.T) { }, model.ChangeFeedID{}) require.NoError(t, err) r.spans.ReplaceOrInsert(span4, rs) - checkpoint, resolved, lastSyncedTs, pullerResolvedTs = r.AdvanceCheckpoint(currentTables, time.Now(), schedulepb.NewBarrierWithMinTs(30), redoMetaManager) + checkpoint, resolved, lastSyncedTs, pullerResolvedTs = r.AdvanceCheckpoint( + currentTables, time.Now(), schedulepb.NewBarrierWithMinTs(30), redoMetaManager) require.Equal(t, model.Ts(3), checkpoint) require.Equal(t, model.Ts(10), resolved) require.Equal(t, model.Ts(32), lastSyncedTs) @@ -787,7 +792,8 @@ func TestReplicationManagerAdvanceCheckpoint(t *testing.T) { require.NoError(t, err) r.spans.ReplaceOrInsert(span, rs) } - checkpoint, resolved, lastSyncedTs, pullerResolvedTs = r.AdvanceCheckpoint(currentTables, time.Now(), schedulepb.NewBarrierWithMinTs(30), redoMetaManager) + checkpoint, resolved, lastSyncedTs, pullerResolvedTs = r.AdvanceCheckpoint( + currentTables, time.Now(), schedulepb.NewBarrierWithMinTs(30), redoMetaManager) require.Equal(t, model.Ts(3), checkpoint) require.Equal(t, model.Ts(10), resolved) require.Equal(t, model.Ts(32), lastSyncedTs) @@ -795,7 +801,8 @@ func TestReplicationManagerAdvanceCheckpoint(t *testing.T) { // The start span is missing rs5_1, _ := r.spans.Delete(span5_1) - checkpoint, resolved, lastSyncedTs, pullerResolvedTs = r.AdvanceCheckpoint(currentTables, time.Now(), schedulepb.NewBarrierWithMinTs(30), redoMetaManager) + checkpoint, resolved, lastSyncedTs, pullerResolvedTs = r.AdvanceCheckpoint( + currentTables, time.Now(), schedulepb.NewBarrierWithMinTs(30), redoMetaManager) require.Equal(t, checkpointCannotProceed, checkpoint) require.Equal(t, checkpointCannotProceed, resolved) require.Equal(t, checkpointCannotProceed, lastSyncedTs) @@ -804,7 +811,8 @@ func TestReplicationManagerAdvanceCheckpoint(t *testing.T) { // The end span is missing r.spans.ReplaceOrInsert(span5_1, rs5_1) r.spans.Delete(span5_2) - checkpoint, resolved, lastSyncedTs, pullerResolvedTs = r.AdvanceCheckpoint(currentTables, time.Now(), schedulepb.NewBarrierWithMinTs(30), redoMetaManager) + checkpoint, resolved, lastSyncedTs, pullerResolvedTs = r.AdvanceCheckpoint( + currentTables, time.Now(), schedulepb.NewBarrierWithMinTs(30), redoMetaManager) require.Equal(t, checkpointCannotProceed, checkpoint) require.Equal(t, checkpointCannotProceed, resolved) require.Equal(t, checkpointCannotProceed, lastSyncedTs) @@ -837,7 +845,8 @@ func TestReplicationManagerAdvanceCheckpoint(t *testing.T) { barrier := schedulepb.NewBarrierWithMinTs(30) redoMetaManager.enable = true redoMetaManager.resolvedTs = 9 - checkpoint, resolved, lastSyncedTs, pullerResolvedTs = r.AdvanceCheckpoint(currentTables, time.Now(), barrier, redoMetaManager) + checkpoint, resolved, lastSyncedTs, pullerResolvedTs = r.AdvanceCheckpoint( + currentTables, time.Now(), barrier, redoMetaManager) require.Equal(t, model.Ts(9), resolved) require.Equal(t, model.Ts(9), checkpoint) require.Equal(t, model.Ts(12), lastSyncedTs) @@ -900,7 +909,9 @@ func TestReplicationManagerAdvanceCheckpointWithRedoEnabled(t *testing.T) { currentTables := &TableRanges{} currentTables.UpdateTables([]model.TableID{1, 2, 3}) barrier := schedulepb.NewBarrierWithMinTs(30) - checkpoint, resolved, lastSyncedTs, pullerResolvedTs := r.AdvanceCheckpoint(currentTables, time.Now(), barrier, redoMetaManager) + checkpoint, resolved, lastSyncedTs, pullerResolvedTs := r.AdvanceCheckpoint( + currentTables, + time.Now(), barrier, redoMetaManager) require.Equal(t, checkpointCannotProceed, checkpoint) require.Equal(t, checkpointCannotProceed, resolved) require.Equal(t, checkpointCannotProceed, lastSyncedTs) From 45e02738440ef5a4caa9d3c3c9d47bd3ba8fee7a Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Tue, 28 Nov 2023 21:49:04 +0800 Subject: [PATCH 07/35] update code --- cdc/api/v2/changefeed.go | 16 +++++++------- cdc/api/v2/model.go | 1 + cdc/model/changefeed.go | 1 + cdc/owner/mock/status_provider_mock.go | 15 +++++++++++++ cdc/sink/tablesink/table_sink.go | 2 +- cdc/sink/tablesink/table_sink_impl.go | 4 ++-- tests/integration_tests/synced_status/run.sh | 22 ++++++++++++++++++++ 7 files changed, 50 insertions(+), 11 deletions(-) create mode 100644 tests/integration_tests/synced_status/run.sh diff --git a/cdc/api/v2/changefeed.go b/cdc/api/v2/changefeed.go index 5472667fe0b..c257131b696 100644 --- a/cdc/api/v2/changefeed.go +++ b/cdc/api/v2/changefeed.go @@ -966,35 +966,35 @@ func (h *OpenAPIV2) synced(c *gin.Context) { } defer pdClient.Close() - physical_now, _, _ := pdClient.GetTS(ctx) + physicalNow, _, _ := pdClient.GetTS(ctx) - log.Info("time info", zap.Int64("physical", physical_now), zap.Int64("checkpointTs", status.CheckpointTs), + log.Info("time info", zap.Int64("physical", physicalNow), zap.Int64("checkpointTs", status.CheckpointTs), zap.Int64("pullerResolvedTs", status.PullerResolvedTs), zap.Int64("LastSyncedTs", status.LastSyncedTs)) - if (physical_now-status.LastSyncedTs > 5*60*1000) && (physical_now-status.CheckpointTs < 5*1000) { // 达到 synced 严格条件 + if (physicalNow-status.LastSyncedTs > 5*60*1000) && (physicalNow-status.CheckpointTs < 5*1000) { // 达到 synced 严格条件 c.JSON(http.StatusOK, SyncedStatus{ Synced: true, SinkCheckpointTs: transformerTime(status.CheckpointTs), PullerResolvedTs: transformerTime(status.PullerResolvedTs), LastSyncedTs: transformerTime(status.LastSyncedTs), - NowTs: transformerTime(physical_now), + NowTs: transformerTime(physicalNow), Info: "Data syncing is finished", }) - } else if physical_now-status.LastSyncedTs > 5*60*1000 { // lastSyncedTs 条件达到,checkpoint-ts 未达到 + } else if physicalNow-status.LastSyncedTs > 5*60*1000 { // lastSyncedTs 条件达到,checkpoint-ts 未达到 var message string if (status.PullerResolvedTs - status.CheckpointTs) < 5*1000 { // 5s message = fmt.Sprintf("Please check whether pd is health and tikv region is all available. " + "If pd is not health or tikv region is not available, the data syncing is finished. " + " Otherwise the data syncing is not finished, please wait") } else { - message = fmt.Sprintf("The data syncing is not finished, please wait") + message = "The data syncing is not finished, please wait" } c.JSON(http.StatusOK, SyncedStatus{ Synced: false, SinkCheckpointTs: transformerTime(status.CheckpointTs), PullerResolvedTs: transformerTime(status.PullerResolvedTs), LastSyncedTs: transformerTime(status.LastSyncedTs), - NowTs: transformerTime(physical_now), + NowTs: transformerTime(physicalNow), Info: message, }) } else { // lastSyncedTs 条件未达到 @@ -1003,7 +1003,7 @@ func (h *OpenAPIV2) synced(c *gin.Context) { SinkCheckpointTs: transformerTime(status.CheckpointTs), PullerResolvedTs: transformerTime(status.PullerResolvedTs), LastSyncedTs: transformerTime(status.LastSyncedTs), - NowTs: transformerTime(physical_now), + NowTs: transformerTime(physicalNow), Info: "The data syncing is not finished, please wait", }) } diff --git a/cdc/api/v2/model.go b/cdc/api/v2/model.go index 5c08fbd9057..f06a6f6644a 100644 --- a/cdc/api/v2/model.go +++ b/cdc/api/v2/model.go @@ -1011,6 +1011,7 @@ type ChangeFeedInfo struct { TaskStatus []model.CaptureTaskStatus `json:"task_status,omitempty"` } +// SyncedStatus describes the detail of a changefeed's synced status type SyncedStatus struct { Synced bool `json:"synced"` SinkCheckpointTs string `json:"sink_checkpoint_ts"` diff --git a/cdc/model/changefeed.go b/cdc/model/changefeed.go index 4ee230dfd37..700b3663f5a 100644 --- a/cdc/model/changefeed.go +++ b/cdc/model/changefeed.go @@ -630,6 +630,7 @@ type ChangeFeedStatusForAPI struct { CheckpointTs uint64 `json:"checkpoint-ts"` } +// ChangeFeedSyncedStatusForAPI uses to transfer the synced status of changefeed for API. type ChangeFeedSyncedStatusForAPI struct { CheckpointTs int64 `json:"checkpoint-ts"` LastSyncedTs int64 `json:"last-sync-time"` diff --git a/cdc/owner/mock/status_provider_mock.go b/cdc/owner/mock/status_provider_mock.go index 2cc69039b23..8751cc4f0bd 100644 --- a/cdc/owner/mock/status_provider_mock.go +++ b/cdc/owner/mock/status_provider_mock.go @@ -95,6 +95,21 @@ func (mr *MockStatusProviderMockRecorder) GetChangeFeedStatus(ctx, changefeedID return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetChangeFeedStatus", reflect.TypeOf((*MockStatusProvider)(nil).GetChangeFeedStatus), ctx, changefeedID) } +// GetChangeFeedSyncedStatus mocks base method. +func (m *MockStatusProvider) GetChangeFeedSyncedStatus(ctx context.Context, changefeedID model.ChangeFeedID) (*model.ChangeFeedSyncedStatusForAPI, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "GetChangeFeedSyncedStatus", ctx, changefeedID) + ret0, _ := ret[0].(*model.ChangeFeedSyncedStatusForAPI) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// GetChangeFeedSyncedStatus indicates an expected call of GetChangeFeedSyncedStatus. +func (mr *MockStatusProviderMockRecorder) GetChangeFeedSyncedStatus(ctx, changefeedID interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetChangeFeedSyncedStatus", reflect.TypeOf((*MockStatusProvider)(nil).GetChangeFeedSyncedStatus), ctx, changefeedID) +} + // GetProcessors mocks base method. func (m *MockStatusProvider) GetProcessors(ctx context.Context) ([]*model.ProcInfoSnap, error) { m.ctrl.T.Helper() diff --git a/cdc/sink/tablesink/table_sink.go b/cdc/sink/tablesink/table_sink.go index de8d197a208..f9f0d9703c5 100644 --- a/cdc/sink/tablesink/table_sink.go +++ b/cdc/sink/tablesink/table_sink.go @@ -33,7 +33,7 @@ type TableSink interface { // This is a thread-safe method. GetCheckpointTs() model.ResolvedTs // GetLastSyncedTs returns the last synced ts of table sink. - // the last synced ts means the biggest committs of the events + // the last synced ts means the biggest commits of the events // that have been flushed to the downstream. // This is a thread-safe method. GetLastSyncedTs() model.Ts diff --git a/cdc/sink/tablesink/table_sink_impl.go b/cdc/sink/tablesink/table_sink_impl.go index 3c4a4407045..2f1c2f6faf5 100644 --- a/cdc/sink/tablesink/table_sink_impl.go +++ b/cdc/sink/tablesink/table_sink_impl.go @@ -33,7 +33,7 @@ var ( ) // LastSyncedTsRecord is used to record the last synced ts of table sink with lock -// lastSyncedTs means the biggest committs of the events +// lastSyncedTs means the biggest commits of the events // that have been flushed to the downstream. type LastSyncedTsRecord struct { sync.Mutex @@ -175,7 +175,7 @@ func (e *EventTableSink[E, P]) GetCheckpointTs() model.ResolvedTs { } // GetLastSyncedTs returns the last synced ts of table sink. -// lastSyncedTs means the biggest committs of all the events +// lastSyncedTs means the biggest commits of all the events // that have been flushed to the downstream. func (e *EventTableSink[E, P]) GetLastSyncedTs() model.Ts { return e.lastSyncedTs.getLastSyncedTs() diff --git a/tests/integration_tests/synced_status/run.sh b/tests/integration_tests/synced_status/run.sh new file mode 100644 index 00000000000..013fc933f91 --- /dev/null +++ b/tests/integration_tests/synced_status/run.sh @@ -0,0 +1,22 @@ +#!/bin/bash + +# [DISCRIPTION]: +# This test is related to +# It will test the sync status request of cdc server in the following scenarios:(including both enable redo and disable redo) +# 1. The sync status request of cdc server when the upstream cluster is available +# 1.1 pdNow - lastSyncedTs > 5min, pdNow - checkpointTs < 5s +# 1.2 pdNow - lastSyncedTs < 5min +# 2. The sync status request of cdc server when the upstream pd is unavailable +# 2.1 resolvedTs - checkpointTs < 5s +# 3. The sync status request of cdc server when the upstream tikv is unavailable +# 3.1 pdNow - lastSyncedTs > 5min, pdNow - checkpointTs > 5s, resolvedTs - checkpointTs < 5s +# 3.2 pdNow - lastSyncedTs < 5min +# 4. The sync status request of cdc server when the downstream tidb is available +# 4.1 pdNow - lastSyncedTs > 5min, pdNow - checkpointTs < 5s +# 4.2 pdNow - lastSyncedTs < 5min +# [STEP]: +# 1. Create changefeed with synced-time-config = xx +# 2. insert data to upstream cluster, and do the related actions for each scenarios +# 3. do the query of synced status of cdc server +# 4. check the info and status of query + From 8dedd2a63c213e9b1be717d84403958fceeaab42 Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Wed, 29 Nov 2023 22:05:53 +0800 Subject: [PATCH 08/35] add ft --- cdc/api/v2/changefeed.go | 31 ++- cdc/api/v2/changefeed_test.go | 25 +- cdc/api/v2/model.go | 22 ++ cdc/model/changefeed.go | 8 +- cdc/owner/owner.go | 8 + pkg/config/replica_config.go | 8 +- pkg/config/synced_status_config.go | 23 ++ .../synced_status/conf/changefeed.toml | 3 + tests/integration_tests/synced_status/run.sh | 256 +++++++++++++++++- 9 files changed, 343 insertions(+), 41 deletions(-) create mode 100644 pkg/config/synced_status_config.go create mode 100644 tests/integration_tests/synced_status/conf/changefeed.toml diff --git a/cdc/api/v2/changefeed.go b/cdc/api/v2/changefeed.go index c257131b696..6a3a93cc356 100644 --- a/cdc/api/v2/changefeed.go +++ b/cdc/api/v2/changefeed.go @@ -922,14 +922,14 @@ func (h *OpenAPIV2) synced(c *gin.Context) { return } - log.Info("synced status", zap.Any("status", status)) - // get pd_now + log.Info("Get changefeed synced status:", zap.Any("status", status)) + cfg := &ChangefeedConfig{ReplicaConfig: GetDefaultReplicaConfig()} + if (status.SyncedCheckInterval != 0) && (status.CheckpointInterval != 0) { + cfg.ReplicaConfig.SyncedStatus.CheckpointInterval = status.CheckpointInterval + cfg.ReplicaConfig.SyncedStatus.SyncedCheckInterval = status.SyncedCheckInterval + } - // if err := c.BindJSON(&cfg); err != nil { - // _ = c.Error(cerror.WrapError(cerror.ErrAPIInvalidParam, err)) - // return - // } if len(cfg.PDAddrs) == 0 { up, err := getCaptureDefaultUpstream(h.capture) if err != nil { @@ -946,13 +946,13 @@ func (h *OpenAPIV2) synced(c *gin.Context) { pdClient, err := h.helpers.getPDClient(timeoutCtx, cfg.PDAddrs, credential) if err != nil { // pd 不可用 var message string - if (status.PullerResolvedTs - status.CheckpointTs) > 5*1000 { // 5s + if (status.PullerResolvedTs - status.CheckpointTs) > cfg.ReplicaConfig.SyncedStatus.CheckpointInterval*1000 { // 5s message = fmt.Sprintf("%s. Besides the data is not finish syncing", terror.Message(err)) } else { message = fmt.Sprintf("%s. You can check the pd first, and if pd is available, means we don't finish sync data. "+ - "If pd is not available, please check the whether we satisfy the condition that"+ - "The time difference from lastSyncedTs to the current time from the time zone of pd is greater than 5 min"+ - "If it's satisfied, means the data syncing is totally finished", err) + "If pd is not available, please check the whether we satisfy the condition that "+ + "the time difference from lastSyncedTs to the current time from the time zone of pd is greater than %v secs. "+ + "If it's satisfied, means the data syncing is totally finished", err, cfg.ReplicaConfig.SyncedStatus.SyncedCheckInterval) } c.JSON(http.StatusOK, SyncedStatus{ Synced: false, @@ -969,9 +969,12 @@ func (h *OpenAPIV2) synced(c *gin.Context) { physicalNow, _, _ := pdClient.GetTS(ctx) log.Info("time info", zap.Int64("physical", physicalNow), zap.Int64("checkpointTs", status.CheckpointTs), - zap.Int64("pullerResolvedTs", status.PullerResolvedTs), zap.Int64("LastSyncedTs", status.LastSyncedTs)) + zap.Int64("pullerResolvedTs", status.PullerResolvedTs), zap.Int64("LastSyncedTs", status.LastSyncedTs), + zap.Int64("SyncedCheckInterval", cfg.ReplicaConfig.SyncedStatus.SyncedCheckInterval), + zap.Int64("CheckpointInterval", cfg.ReplicaConfig.SyncedStatus.CheckpointInterval)) - if (physicalNow-status.LastSyncedTs > 5*60*1000) && (physicalNow-status.CheckpointTs < 5*1000) { // 达到 synced 严格条件 + if (physicalNow-status.LastSyncedTs > cfg.ReplicaConfig.SyncedStatus.SyncedCheckInterval*1000) && + (physicalNow-status.CheckpointTs < cfg.ReplicaConfig.SyncedStatus.CheckpointInterval*1000) { // 达到 synced 严格条件 c.JSON(http.StatusOK, SyncedStatus{ Synced: true, SinkCheckpointTs: transformerTime(status.CheckpointTs), @@ -980,9 +983,9 @@ func (h *OpenAPIV2) synced(c *gin.Context) { NowTs: transformerTime(physicalNow), Info: "Data syncing is finished", }) - } else if physicalNow-status.LastSyncedTs > 5*60*1000 { // lastSyncedTs 条件达到,checkpoint-ts 未达到 + } else if physicalNow-status.LastSyncedTs > cfg.ReplicaConfig.SyncedStatus.SyncedCheckInterval*1000 { // lastSyncedTs 条件达到,checkpoint-ts 未达到 var message string - if (status.PullerResolvedTs - status.CheckpointTs) < 5*1000 { // 5s + if (status.PullerResolvedTs - status.CheckpointTs) < cfg.ReplicaConfig.SyncedStatus.CheckpointInterval*1000 { // 5s message = fmt.Sprintf("Please check whether pd is health and tikv region is all available. " + "If pd is not health or tikv region is not available, the data syncing is finished. " + " Otherwise the data syncing is not finished, please wait") diff --git a/cdc/api/v2/changefeed_test.go b/cdc/api/v2/changefeed_test.go index 0ba80d08e5b..3ff0a81ab24 100644 --- a/cdc/api/v2/changefeed_test.go +++ b/cdc/api/v2/changefeed_test.go @@ -1011,11 +1011,15 @@ func TestChangefeedSynced(t *testing.T) { require.Equal(t, http.StatusBadRequest, w.Code) } + validID := changeFeedID.ID + cfInfo := &model.ChangeFeedInfo{ + ID: validID, + } + statusProvider.err = nil + statusProvider.changefeedInfo = cfInfo { - validID := changeFeedID.ID + helpers.EXPECT().getPDClient(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil, cerror.ErrAPIGetPDClientFailed).Times(1) - statusProvider.err = nil - statusProvider.changefeedInfo = &model.ChangeFeedInfo{ID: validID} // case3: pd is offline,resolvedTs - checkpointTs > 5s statusProvider.changeFeedSyncedStatusForAPI = &model.ChangeFeedSyncedStatusForAPI{ CheckpointTs: 1701153217279, @@ -1037,14 +1041,11 @@ func TestChangefeedSynced(t *testing.T) { require.Equal(t, false, resp.Synced) require.Equal(t, "1970-01-01 08:00:00", resp.NowTs) require.Equal(t, "[CDC:ErrAPIGetPDClientFailed]failed to get PDClient to connect PD, "+ - "please recheck. Besides the data is not finish syncing", resp.Info) + "please recheck: [pd] failed to get cluster id. Besides the data is not finish syncing", resp.Info) } { - validID := changeFeedID.ID helpers.EXPECT().getPDClient(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil, cerror.ErrAPIGetPDClientFailed).Times(1) - statusProvider.err = nil - statusProvider.changefeedInfo = &model.ChangeFeedInfo{ID: validID} // case4: pd is offline,resolvedTs - checkpointTs < 5s statusProvider.changeFeedSyncedStatusForAPI = &model.ChangeFeedSyncedStatusForAPI{ CheckpointTs: 1701153217279, @@ -1065,20 +1066,16 @@ func TestChangefeedSynced(t *testing.T) { require.Nil(t, err) require.Equal(t, false, resp.Synced) require.Equal(t, "1970-01-01 08:00:00", resp.NowTs) - require.Equal(t, "[CDC:ErrAPIGetPDClientFailed]failed to get PDClient to connect PD, please recheck. "+ + require.Equal(t, "[CDC:ErrAPIGetPDClientFailed]failed to get PDClient to connect PD, please recheck: [pd] failed to get cluster id. "+ "You can check the pd first, and if pd is available, means we don't finish sync data. "+ - "If pd is not available, please check the whether we satisfy the condition that"+ - "The time difference from lastSyncedTs to the current time from the time zone of pd is greater than 5 min"+ + "If pd is not available, please check the whether we satisfy the condition that "+ + "the time difference from lastSyncedTs to the current time from the time zone of pd is greater than 300 secs. "+ "If it's satisfied, means the data syncing is totally finished", resp.Info) } helpers.EXPECT().getPDClient(gomock.Any(), gomock.Any(), gomock.Any()).Return(pdClient, nil).AnyTimes() pdClient.logicTime = 1000 pdClient.timestamp = 1701153217279 - statusProvider.err = nil - validID := changeFeedID.ID - statusProvider.changefeedInfo = &model.ChangeFeedInfo{ID: validID} - { // case5: pdTs - lastSyncedTs > 5min, pdTs - checkpointTs < 5s statusProvider.changeFeedSyncedStatusForAPI = &model.ChangeFeedSyncedStatusForAPI{ diff --git a/cdc/api/v2/model.go b/cdc/api/v2/model.go index f06a6f6644a..bedd6944bdd 100644 --- a/cdc/api/v2/model.go +++ b/cdc/api/v2/model.go @@ -103,6 +103,15 @@ type ChangefeedCommonInfo struct { RunningError *model.RunningError `json:"error"` } +// SyncedStatusConfig represents synced check interval config for a changefeed +type SyncedStatusConfig struct { + // The minimum interval between the latest synced ts and now required to reach synced state + SyncedCheckInterval int64 `json:"synced-check-interval"` + // The maximum interval between latest checkpoint ts and now or + // between latest sink's checkpoint ts and puller's checkpoint ts required to reach synced state + CheckpointInterval int64 `json:"checkpoint-interval"` +} + // MarshalJSON marshal changefeed common info to json // we need to set feed state to normal if it is uninitialized and pending to warning // to hide the detail of uninitialized and pending state from user @@ -195,6 +204,7 @@ type ReplicaConfig struct { Integrity *IntegrityConfig `json:"integrity"` ChangefeedErrorStuckDuration *JSONDuration `json:"changefeed_error_stuck_duration,omitempty"` SQLMode string `json:"sql_mode,omitempty"` + SyncedStatus *SyncedStatusConfig `json:"synced_status,omitempty"` } // ToInternalReplicaConfig coverts *v2.ReplicaConfig into *config.ReplicaConfig @@ -488,6 +498,12 @@ func (c *ReplicaConfig) toInternalReplicaConfigWithOriginConfig( if c.ChangefeedErrorStuckDuration != nil { res.ChangefeedErrorStuckDuration = &c.ChangefeedErrorStuckDuration.duration } + if c.SyncedStatus != nil { + res.SyncedStatus = &config.SyncedStatusConfig{ + SyncedCheckInterval: c.SyncedStatus.SyncedCheckInterval, + CheckpointInterval: c.SyncedStatus.CheckpointInterval, + } + } return res } @@ -781,6 +797,12 @@ func ToAPIReplicaConfig(c *config.ReplicaConfig) *ReplicaConfig { if cloned.ChangefeedErrorStuckDuration != nil { res.ChangefeedErrorStuckDuration = &JSONDuration{*cloned.ChangefeedErrorStuckDuration} } + if cloned.SyncedStatus != nil { + res.SyncedStatus = &SyncedStatusConfig{ + SyncedCheckInterval: cloned.SyncedStatus.SyncedCheckInterval, + CheckpointInterval: cloned.SyncedStatus.CheckpointInterval, + } + } return res } diff --git a/cdc/model/changefeed.go b/cdc/model/changefeed.go index 700b3663f5a..3d61a19a339 100644 --- a/cdc/model/changefeed.go +++ b/cdc/model/changefeed.go @@ -632,7 +632,9 @@ type ChangeFeedStatusForAPI struct { // ChangeFeedSyncedStatusForAPI uses to transfer the synced status of changefeed for API. type ChangeFeedSyncedStatusForAPI struct { - CheckpointTs int64 `json:"checkpoint-ts"` - LastSyncedTs int64 `json:"last-sync-time"` - PullerResolvedTs int64 `json:"puller-resolved-ts"` + CheckpointTs int64 `json:"checkpoint-ts"` + LastSyncedTs int64 `json:"last-sync-time"` + PullerResolvedTs int64 `json:"puller-resolved-ts"` + SyncedCheckInterval int64 `json:"synced-check-interval"` + CheckpointInterval int64 `json:"checkpoint-interval"` } diff --git a/cdc/owner/owner.go b/cdc/owner/owner.go index 1db45f66306..8c52bb9b1f3 100644 --- a/cdc/owner/owner.go +++ b/cdc/owner/owner.go @@ -608,6 +608,14 @@ func (o *ownerImpl) handleQueries(query *Query) error { ret.LastSyncedTs = oracle.ExtractPhysical(cfReactor.lastSyncedTs) ret.CheckpointTs = oracle.ExtractPhysical(cfReactor.latestStatus.CheckpointTs) ret.PullerResolvedTs = oracle.ExtractPhysical(cfReactor.pullerResolvedTs) + + if cfReactor.latestInfo == nil { + ret.CheckpointInterval = 0 + ret.SyncedCheckInterval = 0 + } else { + ret.CheckpointInterval = cfReactor.latestInfo.Config.SyncedStatus.CheckpointInterval + ret.SyncedCheckInterval = cfReactor.latestInfo.Config.SyncedStatus.SyncedCheckInterval + } query.Data = ret case QueryChangefeedInfo: cfReactor, ok := o.changefeeds[query.ChangeFeedID] diff --git a/pkg/config/replica_config.go b/pkg/config/replica_config.go index 869a4458e92..be52ddb66c8 100644 --- a/pkg/config/replica_config.go +++ b/pkg/config/replica_config.go @@ -100,6 +100,7 @@ var defaultReplicaConfig = &ReplicaConfig{ }, ChangefeedErrorStuckDuration: util.AddressOf(time.Minute * 30), SQLMode: defaultSQLMode, + SyncedStatus: &SyncedStatusConfig{SyncedCheckInterval: 5 * 60, CheckpointInterval: 5}, } // GetDefaultReplicaConfig returns the default replica config. @@ -149,9 +150,10 @@ type replicaConfig struct { // Scheduler is the configuration for scheduler. Scheduler *ChangefeedSchedulerConfig `toml:"scheduler" json:"scheduler"` // Integrity is only available when the downstream is MQ. - Integrity *integrity.Config `toml:"integrity" json:"integrity"` - ChangefeedErrorStuckDuration *time.Duration `toml:"changefeed-error-stuck-duration" json:"changefeed-error-stuck-duration,omitempty"` - SQLMode string `toml:"sql-mode" json:"sql-mode"` + Integrity *integrity.Config `toml:"integrity" json:"integrity"` + ChangefeedErrorStuckDuration *time.Duration `toml:"changefeed-error-stuck-duration" json:"changefeed-error-stuck-duration,omitempty"` + SQLMode string `toml:"sql-mode" json:"sql-mode"` + SyncedStatus *SyncedStatusConfig `toml:"synced-status" json:"synced-status,omitempty"` } // Value implements the driver.Valuer interface diff --git a/pkg/config/synced_status_config.go b/pkg/config/synced_status_config.go new file mode 100644 index 00000000000..144851f8354 --- /dev/null +++ b/pkg/config/synced_status_config.go @@ -0,0 +1,23 @@ +// 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 config + +// SyncedStatusConfig represents synced check interval config for a changefeed +type SyncedStatusConfig struct { + // The minimum interval between the latest synced ts and now required to reach synced state + SyncedCheckInterval int64 `toml:"synced-check-interval" json:"synced-check-interval"` + // The maximum interval between latest checkpoint ts and now or + // between latest sink's checkpoint ts and puller's checkpoint ts required to reach synced state + CheckpointInterval int64 `toml:"checkpoint-interval" json:"checkpoint-interval"` +} diff --git a/tests/integration_tests/synced_status/conf/changefeed.toml b/tests/integration_tests/synced_status/conf/changefeed.toml new file mode 100644 index 00000000000..0d8aa19fbf5 --- /dev/null +++ b/tests/integration_tests/synced_status/conf/changefeed.toml @@ -0,0 +1,3 @@ +[synced-status] +synced-check-interval = 120 +checkpoint-interval = 20 \ No newline at end of file diff --git a/tests/integration_tests/synced_status/run.sh b/tests/integration_tests/synced_status/run.sh index 013fc933f91..f418f09811f 100644 --- a/tests/integration_tests/synced_status/run.sh +++ b/tests/integration_tests/synced_status/run.sh @@ -4,19 +4,261 @@ # This test is related to # It will test the sync status request of cdc server in the following scenarios:(including both enable redo and disable redo) # 1. The sync status request of cdc server when the upstream cluster is available -# 1.1 pdNow - lastSyncedTs > 5min, pdNow - checkpointTs < 5s -# 1.2 pdNow - lastSyncedTs < 5min +# 1.1 pdNow - lastSyncedTs > threshold, pdNow - checkpointTs < threshold +# 1.2 pdNow - lastSyncedTs < threshold # 2. The sync status request of cdc server when the upstream pd is unavailable -# 2.1 resolvedTs - checkpointTs < 5s +# 2.1 resolvedTs - checkpointTs < threshold # 3. The sync status request of cdc server when the upstream tikv is unavailable -# 3.1 pdNow - lastSyncedTs > 5min, pdNow - checkpointTs > 5s, resolvedTs - checkpointTs < 5s -# 3.2 pdNow - lastSyncedTs < 5min +# 3.1 pdNow - lastSyncedTs > threshold, pdNow - checkpointTs > threshold, resolvedTs - checkpointTs < threshold +# 3.2 pdNow - lastSyncedTs < threshold # 4. The sync status request of cdc server when the downstream tidb is available -# 4.1 pdNow - lastSyncedTs > 5min, pdNow - checkpointTs < 5s -# 4.2 pdNow - lastSyncedTs < 5min +# 4.1 pdNow - lastSyncedTs > threshold, pdNow - checkpointTs < threshold +# 4.2 pdNow - lastSyncedTs < threshold # [STEP]: # 1. Create changefeed with synced-time-config = xx # 2. insert data to upstream cluster, and do the related actions for each scenarios # 3. do the query of synced status of cdc server # 4. check the info and status of query +set -xeu + +CUR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +source $CUR/../_utils/test_prepare +WORK_DIR=$OUT_DIR/$TEST_NAME +CDC_BINARY=cdc.test +SINK_TYPE=$1 + +CDC_COUNT=3 +DB_COUNT=4 + +function kill_pd(){ + info=`ps aux | grep pd-server | grep $WORK_DIR` || true + `ps aux | grep pd-server | grep $WORK_DIR | awk '{print $2}' | xargs kill -9 &>/dev/null ` || true +} + +function kill_tikv(){ + info=`ps aux | grep tikv-server | grep $WORK_DIR` || true + `ps aux | grep tikv-server | grep $WORK_DIR | awk '{print $2}' | xargs kill -9 &>/dev/null ` || true +} + +function kill_tidb(){ + info=`ps aux | grep tidb-server | grep $WORK_DIR` || true + `ps aux | grep tidb-server | grep $WORK_DIR | awk '{print $2}' | xargs kill -9 &>/dev/null ` || true +} + +function run_normal_case_and_unavailable_pd() { + rm -rf $WORK_DIR && mkdir -p $WORK_DIR + + start_tidb_cluster --workdir $WORK_DIR + + cd $WORK_DIR + + start_ts=$(run_cdc_cli_tso_query ${UP_PD_HOST_1} ${UP_PD_PORT_1}) + run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY + + SINK_URI="mysql://root@127.0.0.1:3306/?max-txn-row=1" + run_cdc_cli changefeed create --start-ts=$start_ts --sink-uri="$SINK_URI" --changefeed-id="test-1" --config="$CUR/conf/changefeed.toml" + + # case 1: test in available cluster + synced_status=`curl -X GET http://127.0.0.1:8300/api/v2/changefeeds/test-1/synced` + + status=$(echo $synced_status | jq '.synced') + sink_checkpoint_ts=$(echo $synced_status | jq -r '.sink_checkpoint_ts') + puller_resolved_ts=$(echo $synced_status | jq -r '.puller_resolved_ts') + last_synced_ts=$(echo $synced_status | jq -r '.last_synced_ts') + if [ $status != true ]; then + echo "synced status isn't correct" + exit 1 + fi + # the timestamp for puller_resolved_ts is 0 when do data insert + if [ "$puller_resolved_ts" != "1970-01-01 08:00:00" ]; then + echo "puller_resolved_ts is not 1970-01-01 08:00:00" + exit 1 + fi + # the timestamp for last_synced_ts is 0 when do data insert + if [ "$last_synced_ts" != "1970-01-01 08:00:00" ]; then + echo "last_synced_ts is not 1970-01-01 08:00:00" + exit 1 + fi + + # compare sink_checkpoint_ts with current time + current=$(date +"%Y-%m-%d %H:%M:%S") + echo "sink_checkpoint_ts is "$sink_checkpoint_ts + checkpoint_timestamp=$(date -d "$sink_checkpoint_ts" +%s) + current_timestamp=$(date -d "$current" +%s) + if [ $(($current_timestamp-$checkpoint_timestamp)) -gt 300 ]; then # give a soft check + echo "sink_checkpoint_ts is not correct" + exit 1 + fi + + run_sql "USE TEST;Create table t1(a int primary key, b int);insert into t1 values(1,2);insert into t1 values(2,3);" + check_table_exists "test.t1" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + + sleep 5 # wait data insert + synced_status=`curl -X GET http://127.0.0.1:8300/api/v2/changefeeds/test-1/synced` + status=$(echo $synced_status | jq '.synced') + if [ $status != false ]; then + echo "synced status isn't correct" + exit 1 + fi + info=$(echo $synced_status | jq -r '.info') + if [ "$info" != "The data syncing is not finished, please wait" ]; then + echo "synced status info is not correct" + exit 1 + fi + + sleep 130 # wait enough time for pass synced-check-interval + synced_status=`curl -X GET http://127.0.0.1:8300/api/v2/changefeeds/test-1/synced` + status=$(echo $synced_status | jq '.synced') + if [ $status != true ]; then + echo "synced status isn't correct" + exit 1 + fi + + + #========== + # case 2: test with unavailable pd + kill_pd + + synced_status=`curl -X GET http://127.0.0.1:8300/api/v2/changefeeds/test-1/synced` + status=$(echo $synced_status | jq '.synced') + if [ $status != false ]; then + echo "synced status isn't correct" + exit 1 + fi + info=$(echo $synced_status | jq -r '.info') + target_message="[CDC:ErrAPIGetPDClientFailed]failed to get PDClient to connect PD, please recheck: [pd] failed to get cluster id. \ +You can check the pd first, and if pd is available, means we don't finish sync data. \ +If pd is not available, please check the whether we satisfy the condition that \ +the time difference from lastSyncedTs to the current time from the time zone of pd is greater than 120 secs. \ +If it's satisfied, means the data syncing is totally finished" + + if [ "$info" != "$target_message" ]; then + echo "synced status info is not correct" + exit 1 + fi + + cleanup_process $CDC_BINARY + stop_tidb_cluster +} + +function run_case_with_unavailable_tikv() { + rm -rf $WORK_DIR && mkdir -p $WORK_DIR + + start_tidb_cluster --workdir $WORK_DIR + + cd $WORK_DIR + + start_ts=$(run_cdc_cli_tso_query ${UP_PD_HOST_1} ${UP_PD_PORT_1}) + run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY + + SINK_URI="mysql://root@127.0.0.1:3306/?max-txn-row=1" + run_cdc_cli changefeed create --start-ts=$start_ts --sink-uri="$SINK_URI" --changefeed-id="test-1" --config="$CUR/conf/changefeed.toml" + + # case 3: test in unavailable tikv cluster + run_sql "USE TEST;Create table t1(a int primary key, b int);insert into t1 values(1,2);insert into t1 values(2,3);" + check_table_exists "test.t1" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + + sleep 5 # make data inserted into downstream + kill_tikv + + # test the case when pdNow - lastSyncedTs < threshold + synced_status=`curl -X GET http://127.0.0.1:8300/api/v2/changefeeds/test-1/synced` + status=$(echo $synced_status | jq '.synced') + if [ $status != false ]; then + echo "synced status isn't correct" + exit 1 + fi + info=$(echo $synced_status | jq -r '.info') + target_message="The data syncing is not finished, please wait" + + if [ "$info" != "$target_message" ]; then + echo "synced status info is not correct" + exit 1 + fi + + sleep 130 # wait enough time for pass synced-check-interval + # test the case when pdNow - lastSyncedTs > threshold + synced_status=`curl -X GET http://127.0.0.1:8300/api/v2/changefeeds/test-1/synced` + status=$(echo $synced_status | jq '.synced') + if [ $status != false ]; then + echo "synced status isn't correct" + exit 1 + fi + info=$(echo $synced_status | jq -r '.info') + target_message="Please check whether pd is health and tikv region is all available. \ +If pd is not health or tikv region is not available, the data syncing is finished. \ +Otherwise the data syncing is not finished, please wait" + + if [ "$info" != "$target_message" ]; then + echo "synced status info is not correct" + exit 1 + fi + + cleanup_process $CDC_BINARY + stop_tidb_cluster +} + +function run_case_with_unavailable_tidb() { + rm -rf $WORK_DIR && mkdir -p $WORK_DIR + + start_tidb_cluster --workdir $WORK_DIR + + cd $WORK_DIR + + start_ts=$(run_cdc_cli_tso_query ${UP_PD_HOST_1} ${UP_PD_PORT_1}) + run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY + + SINK_URI="mysql://root@127.0.0.1:3306/?max-txn-row=1" + run_cdc_cli changefeed create --start-ts=$start_ts --sink-uri="$SINK_URI" --changefeed-id="test-1" --config="$CUR/conf/changefeed.toml" + + # case 3: test in unavailable tikv cluster + run_sql "USE TEST;Create table t1(a int primary key, b int);insert into t1 values(1,2);insert into t1 values(2,3);" + check_table_exists "test.t1" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + + sleep 5 # make data inserted into downstream + kill_tidb + + # test the case when pdNow - lastSyncedTs < threshold + synced_status=`curl -X GET http://127.0.0.1:8300/api/v2/changefeeds/test-1/synced` + status=$(echo $synced_status | jq '.synced') + if [ $status != false ]; then + echo "synced status isn't correct" + exit 1 + fi + info=$(echo $synced_status | jq -r '.info') + target_message="The data syncing is not finished, please wait" + + if [ "$info" != "$target_message" ]; then + echo "synced status info is not correct" + exit 1 + fi + + sleep 130 # wait enough time for pass synced-check-interval + # test the case when pdNow - lastSyncedTs > threshold + synced_status=`curl -X GET http://127.0.0.1:8300/api/v2/changefeeds/test-1/synced` + status=$(echo $synced_status | jq '.synced') + if [ $status != true ]; then + echo "synced status isn't correct" + exit 1 + fi + info=$(echo $synced_status | jq -r '.info') + target_message="Data syncing is finished" + + if [ "$info" != "$target_message" ]; then + echo "synced status info is not correct" + exit 1 + fi + + cleanup_process $CDC_BINARY + stop_tidb_cluster +} + + + +trap stop_tidb_cluster EXIT +run_normal_case_and_unavailable_pd $* +run_case_with_unavailable_tikv $* +run_case_with_unavailable_tidb $* +check_logs $WORK_DIR +echo "[$(date)] <<<<<< run test case $TEST_NAME success! >>>>>>" \ No newline at end of file From 95f9204c0a29369b6be7d6e02e6213ad922a47fd Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Thu, 30 Nov 2023 11:00:13 +0800 Subject: [PATCH 09/35] update ft --- cdc/owner/changefeed.go | 16 ++--- tests/integration_tests/synced_status/run.sh | 61 +++++++++++++++++++- 2 files changed, 66 insertions(+), 11 deletions(-) mode change 100644 => 100755 cdc/owner/changefeed.go diff --git a/cdc/owner/changefeed.go b/cdc/owner/changefeed.go old mode 100644 new mode 100755 index 0cf902ac425..9fe259d1328 --- a/cdc/owner/changefeed.go +++ b/cdc/owner/changefeed.go @@ -168,8 +168,6 @@ type changefeed struct { opts ...observer.NewObserverOption, ) (observer.Observer, error) - lastDDLTs uint64 // Timestamp of the last executed DDL. Only used for tests. - // The latest changefeed info and status from meta storage. they are updated in every Tick. latestInfo *model.ChangeFeedInfo latestStatus *model.ChangeFeedStatus @@ -475,14 +473,12 @@ func (c *changefeed) tick(ctx cdcContext.Context, } failpoint.Inject("ChangefeedOwnerDontUpdateCheckpoint", func() { - if c.lastDDLTs != 0 && c.latestStatus.CheckpointTs >= c.lastDDLTs { - log.Info("owner won't update checkpoint because of failpoint", - zap.String("namespace", c.id.Namespace), - zap.String("changefeed", c.id.ID), - zap.Uint64("keepCheckpoint", c.latestStatus.CheckpointTs), - zap.Uint64("skipCheckpoint", newCheckpointTs)) - newCheckpointTs = c.latestStatus.CheckpointTs - } + log.Info("owner won't update checkpoint because of failpoint", + zap.String("namespace", c.id.Namespace), + zap.String("changefeed", c.id.ID), + zap.Uint64("keepCheckpoint", c.latestStatus.CheckpointTs), + zap.Uint64("skipCheckpoint", newCheckpointTs)) + newCheckpointTs = c.latestStatus.CheckpointTs }) c.updateMetrics(currentTs, newCheckpointTs, c.resolvedTs) diff --git a/tests/integration_tests/synced_status/run.sh b/tests/integration_tests/synced_status/run.sh index f418f09811f..ddf6302b276 100644 --- a/tests/integration_tests/synced_status/run.sh +++ b/tests/integration_tests/synced_status/run.sh @@ -2,10 +2,11 @@ # [DISCRIPTION]: # This test is related to -# It will test the sync status request of cdc server in the following scenarios:(including both enable redo and disable redo) +# It will test the sync status request of cdc server in the following scenarios: # 1. The sync status request of cdc server when the upstream cluster is available # 1.1 pdNow - lastSyncedTs > threshold, pdNow - checkpointTs < threshold # 1.2 pdNow - lastSyncedTs < threshold +# 1.3 pdNow - lastSyncedTs > threshold, pdNow - checkpointTs < threshold, resolvedTs - checkpointTs > threshold # 2. The sync status request of cdc server when the upstream pd is unavailable # 2.1 resolvedTs - checkpointTs < threshold # 3. The sync status request of cdc server when the upstream tikv is unavailable @@ -115,6 +116,25 @@ function run_normal_case_and_unavailable_pd() { exit 1 fi + # make failpoint to block checkpoint-ts + export GO_FAILPOINTS='github.com/pingcap/ticdc/cdc/owner/ChangefeedOwnerDontUpdateCheckpoint=return("")' + sleep 20 # wait enough time for pass checkpoint-check-interval + synced_status=`curl -X GET http://127.0.0.1:8300/api/v2/changefeeds/test-1/synced` + status=$(echo $synced_status | jq '.synced') + if [ $status != false ]; then + echo "synced status isn't correct" + exit 1 + fi + info=$(echo $synced_status | jq -r '.info') + target_message="Please check whether pd is health and tikv region is all available. \ +If pd is not health or tikv region is not available, the data syncing is finished. \ +Otherwise the data syncing is not finished, please wait" + if [ "$info" != "$target_message" ]; then + echo "synced status info is not correct" + exit 1 + fi + + export GO_FAILPOINTS='' #========== # case 2: test with unavailable pd @@ -254,11 +274,50 @@ function run_case_with_unavailable_tidb() { stop_tidb_cluster } +function run_case_with_failpoint() { + rm -rf $WORK_DIR && mkdir -p $WORK_DIR + + start_tidb_cluster --workdir $WORK_DIR + + cd $WORK_DIR + + # make failpoint to block checkpoint-ts + export GO_FAILPOINTS='github.com/pingcap/tiflow/cdc/owner/ChangefeedOwnerDontUpdateCheckpoint=return(true)' + + start_ts=$(run_cdc_cli_tso_query ${UP_PD_HOST_1} ${UP_PD_PORT_1}) + run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY + + SINK_URI="mysql://root@127.0.0.1:3306/?max-txn-row=1" + run_cdc_cli changefeed create --start-ts=$start_ts --sink-uri="$SINK_URI" --changefeed-id="test-1" --config="$CUR/conf/changefeed.toml" + + sleep 20 # wait enough time for pass checkpoint-check-interval + synced_status=`curl -X GET http://127.0.0.1:8300/api/v2/changefeeds/test-1/synced` + status=$(echo $synced_status | jq '.synced') + if [ $status != false ]; then + echo "synced status isn't correct" + exit 1 + fi + info=$(echo $synced_status | jq -r '.info') + target_message="Please check whether pd is health and tikv region is all available. \ +If pd is not health or tikv region is not available, the data syncing is finished. \ +Otherwise the data syncing is not finished, please wait" + if [ "$info" != "$target_message" ]; then + echo "synced status info is not correct" + exit 1 + fi + + export GO_FAILPOINTS='' + + cleanup_process $CDC_BINARY + stop_tidb_cluster +} + trap stop_tidb_cluster EXIT run_normal_case_and_unavailable_pd $* run_case_with_unavailable_tikv $* run_case_with_unavailable_tidb $* +run_case_with_failpoint $* check_logs $WORK_DIR echo "[$(date)] <<<<<< run test case $TEST_NAME success! >>>>>>" \ No newline at end of file From 8431839702e0caa17593cdab93ce5e5e2d79a38e Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Thu, 30 Nov 2023 14:05:35 +0800 Subject: [PATCH 10/35] update test --- .../synced_status/conf/changefeed-redo.toml | 7 +++ tests/integration_tests/synced_status/run.sh | 52 ++++++++----------- 2 files changed, 30 insertions(+), 29 deletions(-) create mode 100644 tests/integration_tests/synced_status/conf/changefeed-redo.toml diff --git a/tests/integration_tests/synced_status/conf/changefeed-redo.toml b/tests/integration_tests/synced_status/conf/changefeed-redo.toml new file mode 100644 index 00000000000..8f67ecb9d8f --- /dev/null +++ b/tests/integration_tests/synced_status/conf/changefeed-redo.toml @@ -0,0 +1,7 @@ +[synced-status] +synced-check-interval = 120 +checkpoint-interval = 20 + +[consistent] +level="eventual" +storage = "file:///tmp/tidb_cdc_test/synced_status/redo" \ No newline at end of file diff --git a/tests/integration_tests/synced_status/run.sh b/tests/integration_tests/synced_status/run.sh index ddf6302b276..5d72eb87143 100644 --- a/tests/integration_tests/synced_status/run.sh +++ b/tests/integration_tests/synced_status/run.sh @@ -57,8 +57,10 @@ function run_normal_case_and_unavailable_pd() { start_ts=$(run_cdc_cli_tso_query ${UP_PD_HOST_1} ${UP_PD_PORT_1}) run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY + config_path=$1 + SINK_URI="mysql://root@127.0.0.1:3306/?max-txn-row=1" - run_cdc_cli changefeed create --start-ts=$start_ts --sink-uri="$SINK_URI" --changefeed-id="test-1" --config="$CUR/conf/changefeed.toml" + run_cdc_cli changefeed create --start-ts=$start_ts --sink-uri="$SINK_URI" --changefeed-id="test-1" --config="$CUR/$config_path" # case 1: test in available cluster synced_status=`curl -X GET http://127.0.0.1:8300/api/v2/changefeeds/test-1/synced` @@ -116,26 +118,6 @@ function run_normal_case_and_unavailable_pd() { exit 1 fi - # make failpoint to block checkpoint-ts - export GO_FAILPOINTS='github.com/pingcap/ticdc/cdc/owner/ChangefeedOwnerDontUpdateCheckpoint=return("")' - sleep 20 # wait enough time for pass checkpoint-check-interval - synced_status=`curl -X GET http://127.0.0.1:8300/api/v2/changefeeds/test-1/synced` - status=$(echo $synced_status | jq '.synced') - if [ $status != false ]; then - echo "synced status isn't correct" - exit 1 - fi - info=$(echo $synced_status | jq -r '.info') - target_message="Please check whether pd is health and tikv region is all available. \ -If pd is not health or tikv region is not available, the data syncing is finished. \ -Otherwise the data syncing is not finished, please wait" - if [ "$info" != "$target_message" ]; then - echo "synced status info is not correct" - exit 1 - fi - - export GO_FAILPOINTS='' - #========== # case 2: test with unavailable pd kill_pd @@ -172,8 +154,10 @@ function run_case_with_unavailable_tikv() { start_ts=$(run_cdc_cli_tso_query ${UP_PD_HOST_1} ${UP_PD_PORT_1}) run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY + config_path=$1 + SINK_URI="mysql://root@127.0.0.1:3306/?max-txn-row=1" - run_cdc_cli changefeed create --start-ts=$start_ts --sink-uri="$SINK_URI" --changefeed-id="test-1" --config="$CUR/conf/changefeed.toml" + run_cdc_cli changefeed create --start-ts=$start_ts --sink-uri="$SINK_URI" --changefeed-id="test-1" --config="$CUR/$config_path" # case 3: test in unavailable tikv cluster run_sql "USE TEST;Create table t1(a int primary key, b int);insert into t1 values(1,2);insert into t1 values(2,3);" @@ -228,9 +212,11 @@ function run_case_with_unavailable_tidb() { start_ts=$(run_cdc_cli_tso_query ${UP_PD_HOST_1} ${UP_PD_PORT_1}) run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY + + config_path=$1 SINK_URI="mysql://root@127.0.0.1:3306/?max-txn-row=1" - run_cdc_cli changefeed create --start-ts=$start_ts --sink-uri="$SINK_URI" --changefeed-id="test-1" --config="$CUR/conf/changefeed.toml" + run_cdc_cli changefeed create --start-ts=$start_ts --sink-uri="$SINK_URI" --changefeed-id="test-1" --config="$CUR/$config_path" # case 3: test in unavailable tikv cluster run_sql "USE TEST;Create table t1(a int primary key, b int);insert into t1 values(1,2);insert into t1 values(2,3);" @@ -287,8 +273,10 @@ function run_case_with_failpoint() { start_ts=$(run_cdc_cli_tso_query ${UP_PD_HOST_1} ${UP_PD_PORT_1}) run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY + config_path=$1 + SINK_URI="mysql://root@127.0.0.1:3306/?max-txn-row=1" - run_cdc_cli changefeed create --start-ts=$start_ts --sink-uri="$SINK_URI" --changefeed-id="test-1" --config="$CUR/conf/changefeed.toml" + run_cdc_cli changefeed create --start-ts=$start_ts --sink-uri="$SINK_URI" --changefeed-id="test-1" --config="$CUR/$config_path" sleep 20 # wait enough time for pass checkpoint-check-interval synced_status=`curl -X GET http://127.0.0.1:8300/api/v2/changefeeds/test-1/synced` @@ -313,11 +301,17 @@ Otherwise the data syncing is not finished, please wait" } - trap stop_tidb_cluster EXIT -run_normal_case_and_unavailable_pd $* -run_case_with_unavailable_tikv $* -run_case_with_unavailable_tidb $* -run_case_with_failpoint $* +run_normal_case_and_unavailable_pd "conf/changefeed.toml" +run_case_with_unavailable_tikv "conf/changefeed.toml" +run_case_with_unavailable_tidb "conf/changefeed.toml" +run_case_with_failpoint "conf/changefeed.toml" + +# enable redo +run_normal_case_and_unavailable_pd "conf/changefeed-redo.toml" +run_case_with_unavailable_tikv "conf/changefeed-redo.toml" +run_case_with_unavailable_tidb "conf/changefeed-redo.toml" +run_case_with_failpoint "conf/changefeed-redo.toml" + check_logs $WORK_DIR echo "[$(date)] <<<<<< run test case $TEST_NAME success! >>>>>>" \ No newline at end of file From 9b37e32a913f29252835c7a43b9b00dfd9bfbdba Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Mon, 11 Dec 2023 17:18:17 +0800 Subject: [PATCH 11/35] update run_group --- tests/integration_tests/run_group.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration_tests/run_group.sh b/tests/integration_tests/run_group.sh index 0a394227ebb..0bd52764d6d 100755 --- a/tests/integration_tests/run_group.sh +++ b/tests/integration_tests/run_group.sh @@ -54,7 +54,7 @@ groups=( ["G18"]='availability http_proxies sequence' ["G19"]='changefeed_fast_fail batch_update_to_no_batch changefeed_resume_with_checkpoint_ts' ["G20"]='tidb_mysql_test ddl_reentrant multi_cdc_cluster' - ["G21"]='bank kill_owner_with_ddl owner_remove_table_error' + ["G21"]='bank kill_owner_with_ddl owner_remove_table_error synced_status' ) # Get other cases not in groups, to avoid missing any case From a3f8736979024fd2b3308adcf61093c369b814fe Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Mon, 11 Dec 2023 17:54:10 +0800 Subject: [PATCH 12/35] fix typo --- cdc/api/v2/changefeed.go | 3 +- tests/integration_tests/synced_status/run.sh | 365 +++++++++---------- 2 files changed, 184 insertions(+), 184 deletions(-) diff --git a/cdc/api/v2/changefeed.go b/cdc/api/v2/changefeed.go index 6a3a93cc356..0d97db7ecac 100644 --- a/cdc/api/v2/changefeed.go +++ b/cdc/api/v2/changefeed.go @@ -983,7 +983,8 @@ func (h *OpenAPIV2) synced(c *gin.Context) { NowTs: transformerTime(physicalNow), Info: "Data syncing is finished", }) - } else if physicalNow-status.LastSyncedTs > cfg.ReplicaConfig.SyncedStatus.SyncedCheckInterval*1000 { // lastSyncedTs 条件达到,checkpoint-ts 未达到 + } else if physicalNow-status.LastSyncedTs > cfg.ReplicaConfig.SyncedStatus.SyncedCheckInterval*1000 { + // lastSyncedTs 条件达到,checkpoint-ts 未达到 var message string if (status.PullerResolvedTs - status.CheckpointTs) < cfg.ReplicaConfig.SyncedStatus.CheckpointInterval*1000 { // 5s message = fmt.Sprintf("Please check whether pd is health and tikv region is all available. " + diff --git a/tests/integration_tests/synced_status/run.sh b/tests/integration_tests/synced_status/run.sh index 5d72eb87143..735ecdc3d89 100644 --- a/tests/integration_tests/synced_status/run.sh +++ b/tests/integration_tests/synced_status/run.sh @@ -1,7 +1,7 @@ #!/bin/bash # [DISCRIPTION]: -# This test is related to +# This test is related to # It will test the sync status request of cdc server in the following scenarios: # 1. The sync status request of cdc server when the upstream cluster is available # 1.1 pdNow - lastSyncedTs > threshold, pdNow - checkpointTs < threshold @@ -32,19 +32,19 @@ SINK_TYPE=$1 CDC_COUNT=3 DB_COUNT=4 -function kill_pd(){ - info=`ps aux | grep pd-server | grep $WORK_DIR` || true - `ps aux | grep pd-server | grep $WORK_DIR | awk '{print $2}' | xargs kill -9 &>/dev/null ` || true +function kill_pd() { + info=$(ps aux | grep pd-server | grep $WORK_DIR) || true + $(ps aux | grep pd-server | grep $WORK_DIR | awk '{print $2}' | xargs kill -9 &>/dev/null) || true } -function kill_tikv(){ - info=`ps aux | grep tikv-server | grep $WORK_DIR` || true - `ps aux | grep tikv-server | grep $WORK_DIR | awk '{print $2}' | xargs kill -9 &>/dev/null ` || true +function kill_tikv() { + info=$(ps aux | grep tikv-server | grep $WORK_DIR) || true + $(ps aux | grep tikv-server | grep $WORK_DIR | awk '{print $2}' | xargs kill -9 &>/dev/null) || true } -function kill_tidb(){ - info=`ps aux | grep tidb-server | grep $WORK_DIR` || true - `ps aux | grep tidb-server | grep $WORK_DIR | awk '{print $2}' | xargs kill -9 &>/dev/null ` || true +function kill_tidb() { + info=$(ps aux | grep tidb-server | grep $WORK_DIR) || true + $(ps aux | grep tidb-server | grep $WORK_DIR | awk '{print $2}' | xargs kill -9 &>/dev/null) || true } function run_normal_case_and_unavailable_pd() { @@ -57,91 +57,91 @@ function run_normal_case_and_unavailable_pd() { start_ts=$(run_cdc_cli_tso_query ${UP_PD_HOST_1} ${UP_PD_PORT_1}) run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY - config_path=$1 + config_path=$1 SINK_URI="mysql://root@127.0.0.1:3306/?max-txn-row=1" run_cdc_cli changefeed create --start-ts=$start_ts --sink-uri="$SINK_URI" --changefeed-id="test-1" --config="$CUR/$config_path" - # case 1: test in available cluster - synced_status=`curl -X GET http://127.0.0.1:8300/api/v2/changefeeds/test-1/synced` - - status=$(echo $synced_status | jq '.synced') - sink_checkpoint_ts=$(echo $synced_status | jq -r '.sink_checkpoint_ts') - puller_resolved_ts=$(echo $synced_status | jq -r '.puller_resolved_ts') - last_synced_ts=$(echo $synced_status | jq -r '.last_synced_ts') - if [ $status != true ]; then - echo "synced status isn't correct" - exit 1 - fi - # the timestamp for puller_resolved_ts is 0 when do data insert - if [ "$puller_resolved_ts" != "1970-01-01 08:00:00" ]; then - echo "puller_resolved_ts is not 1970-01-01 08:00:00" - exit 1 - fi - # the timestamp for last_synced_ts is 0 when do data insert - if [ "$last_synced_ts" != "1970-01-01 08:00:00" ]; then - echo "last_synced_ts is not 1970-01-01 08:00:00" - exit 1 - fi - - # compare sink_checkpoint_ts with current time - current=$(date +"%Y-%m-%d %H:%M:%S") - echo "sink_checkpoint_ts is "$sink_checkpoint_ts - checkpoint_timestamp=$(date -d "$sink_checkpoint_ts" +%s) - current_timestamp=$(date -d "$current" +%s) - if [ $(($current_timestamp-$checkpoint_timestamp)) -gt 300 ]; then # give a soft check - echo "sink_checkpoint_ts is not correct" - exit 1 - fi - - run_sql "USE TEST;Create table t1(a int primary key, b int);insert into t1 values(1,2);insert into t1 values(2,3);" - check_table_exists "test.t1" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} - - sleep 5 # wait data insert - synced_status=`curl -X GET http://127.0.0.1:8300/api/v2/changefeeds/test-1/synced` - status=$(echo $synced_status | jq '.synced') - if [ $status != false ]; then - echo "synced status isn't correct" - exit 1 - fi - info=$(echo $synced_status | jq -r '.info') - if [ "$info" != "The data syncing is not finished, please wait" ]; then - echo "synced status info is not correct" - exit 1 - fi - - sleep 130 # wait enough time for pass synced-check-interval - synced_status=`curl -X GET http://127.0.0.1:8300/api/v2/changefeeds/test-1/synced` - status=$(echo $synced_status | jq '.synced') - if [ $status != true ]; then - echo "synced status isn't correct" - exit 1 - fi - - #========== - # case 2: test with unavailable pd - kill_pd - - synced_status=`curl -X GET http://127.0.0.1:8300/api/v2/changefeeds/test-1/synced` - status=$(echo $synced_status | jq '.synced') - if [ $status != false ]; then - echo "synced status isn't correct" - exit 1 - fi - info=$(echo $synced_status | jq -r '.info') - target_message="[CDC:ErrAPIGetPDClientFailed]failed to get PDClient to connect PD, please recheck: [pd] failed to get cluster id. \ + # case 1: test in available cluster + synced_status=$(curl -X GET http://127.0.0.1:8300/api/v2/changefeeds/test-1/synced) + + status=$(echo $synced_status | jq '.synced') + sink_checkpoint_ts=$(echo $synced_status | jq -r '.sink_checkpoint_ts') + puller_resolved_ts=$(echo $synced_status | jq -r '.puller_resolved_ts') + last_synced_ts=$(echo $synced_status | jq -r '.last_synced_ts') + if [ $status != true ]; then + echo "synced status isn't correct" + exit 1 + fi + # the timestamp for puller_resolved_ts is 0 when do data insert + if [ "$puller_resolved_ts" != "1970-01-01 08:00:00" ]; then + echo "puller_resolved_ts is not 1970-01-01 08:00:00" + exit 1 + fi + # the timestamp for last_synced_ts is 0 when do data insert + if [ "$last_synced_ts" != "1970-01-01 08:00:00" ]; then + echo "last_synced_ts is not 1970-01-01 08:00:00" + exit 1 + fi + + # compare sink_checkpoint_ts with current time + current=$(date +"%Y-%m-%d %H:%M:%S") + echo "sink_checkpoint_ts is "$sink_checkpoint_ts + checkpoint_timestamp=$(date -d "$sink_checkpoint_ts" +%s) + current_timestamp=$(date -d "$current" +%s) + if [ $(($current_timestamp - $checkpoint_timestamp)) -gt 300 ]; then # give a soft check + echo "sink_checkpoint_ts is not correct" + exit 1 + fi + + run_sql "USE TEST;Create table t1(a int primary key, b int);insert into t1 values(1,2);insert into t1 values(2,3);" + check_table_exists "test.t1" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + + sleep 5 # wait data insert + synced_status=$(curl -X GET http://127.0.0.1:8300/api/v2/changefeeds/test-1/synced) + status=$(echo $synced_status | jq '.synced') + if [ $status != false ]; then + echo "synced status isn't correct" + exit 1 + fi + info=$(echo $synced_status | jq -r '.info') + if [ "$info" != "The data syncing is not finished, please wait" ]; then + echo "synced status info is not correct" + exit 1 + fi + + sleep 130 # wait enough time for pass synced-check-interval + synced_status=$(curl -X GET http://127.0.0.1:8300/api/v2/changefeeds/test-1/synced) + status=$(echo $synced_status | jq '.synced') + if [ $status != true ]; then + echo "synced status isn't correct" + exit 1 + fi + + #========== + # case 2: test with unavailable pd + kill_pd + + synced_status=$(curl -X GET http://127.0.0.1:8300/api/v2/changefeeds/test-1/synced) + status=$(echo $synced_status | jq '.synced') + if [ $status != false ]; then + echo "synced status isn't correct" + exit 1 + fi + info=$(echo $synced_status | jq -r '.info') + target_message="[CDC:ErrAPIGetPDClientFailed]failed to get PDClient to connect PD, please recheck: [pd] failed to get cluster id. \ You can check the pd first, and if pd is available, means we don't finish sync data. \ If pd is not available, please check the whether we satisfy the condition that \ the time difference from lastSyncedTs to the current time from the time zone of pd is greater than 120 secs. \ If it's satisfied, means the data syncing is totally finished" - if [ "$info" != "$target_message" ]; then - echo "synced status info is not correct" - exit 1 - fi + if [ "$info" != "$target_message" ]; then + echo "synced status info is not correct" + exit 1 + fi cleanup_process $CDC_BINARY - stop_tidb_cluster + stop_tidb_cluster } function run_case_with_unavailable_tikv() { @@ -154,53 +154,53 @@ function run_case_with_unavailable_tikv() { start_ts=$(run_cdc_cli_tso_query ${UP_PD_HOST_1} ${UP_PD_PORT_1}) run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY - config_path=$1 + config_path=$1 SINK_URI="mysql://root@127.0.0.1:3306/?max-txn-row=1" run_cdc_cli changefeed create --start-ts=$start_ts --sink-uri="$SINK_URI" --changefeed-id="test-1" --config="$CUR/$config_path" - # case 3: test in unavailable tikv cluster - run_sql "USE TEST;Create table t1(a int primary key, b int);insert into t1 values(1,2);insert into t1 values(2,3);" - check_table_exists "test.t1" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} - - sleep 5 # make data inserted into downstream - kill_tikv - - # test the case when pdNow - lastSyncedTs < threshold - synced_status=`curl -X GET http://127.0.0.1:8300/api/v2/changefeeds/test-1/synced` - status=$(echo $synced_status | jq '.synced') - if [ $status != false ]; then - echo "synced status isn't correct" - exit 1 - fi - info=$(echo $synced_status | jq -r '.info') - target_message="The data syncing is not finished, please wait" - - if [ "$info" != "$target_message" ]; then - echo "synced status info is not correct" - exit 1 - fi - - sleep 130 # wait enough time for pass synced-check-interval - # test the case when pdNow - lastSyncedTs > threshold - synced_status=`curl -X GET http://127.0.0.1:8300/api/v2/changefeeds/test-1/synced` - status=$(echo $synced_status | jq '.synced') - if [ $status != false ]; then - echo "synced status isn't correct" - exit 1 - fi - info=$(echo $synced_status | jq -r '.info') - target_message="Please check whether pd is health and tikv region is all available. \ + # case 3: test in unavailable tikv cluster + run_sql "USE TEST;Create table t1(a int primary key, b int);insert into t1 values(1,2);insert into t1 values(2,3);" + check_table_exists "test.t1" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + + sleep 5 # make data inserted into downstream + kill_tikv + + # test the case when pdNow - lastSyncedTs < threshold + synced_status=$(curl -X GET http://127.0.0.1:8300/api/v2/changefeeds/test-1/synced) + status=$(echo $synced_status | jq '.synced') + if [ $status != false ]; then + echo "synced status isn't correct" + exit 1 + fi + info=$(echo $synced_status | jq -r '.info') + target_message="The data syncing is not finished, please wait" + + if [ "$info" != "$target_message" ]; then + echo "synced status info is not correct" + exit 1 + fi + + sleep 130 # wait enough time for pass synced-check-interval + # test the case when pdNow - lastSyncedTs > threshold + synced_status=$(curl -X GET http://127.0.0.1:8300/api/v2/changefeeds/test-1/synced) + status=$(echo $synced_status | jq '.synced') + if [ $status != false ]; then + echo "synced status isn't correct" + exit 1 + fi + info=$(echo $synced_status | jq -r '.info') + target_message="Please check whether pd is health and tikv region is all available. \ If pd is not health or tikv region is not available, the data syncing is finished. \ Otherwise the data syncing is not finished, please wait" - if [ "$info" != "$target_message" ]; then - echo "synced status info is not correct" - exit 1 - fi + if [ "$info" != "$target_message" ]; then + echo "synced status info is not correct" + exit 1 + fi cleanup_process $CDC_BINARY - stop_tidb_cluster + stop_tidb_cluster } function run_case_with_unavailable_tidb() { @@ -212,52 +212,52 @@ function run_case_with_unavailable_tidb() { start_ts=$(run_cdc_cli_tso_query ${UP_PD_HOST_1} ${UP_PD_PORT_1}) run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY - - config_path=$1 + + config_path=$1 SINK_URI="mysql://root@127.0.0.1:3306/?max-txn-row=1" run_cdc_cli changefeed create --start-ts=$start_ts --sink-uri="$SINK_URI" --changefeed-id="test-1" --config="$CUR/$config_path" - # case 3: test in unavailable tikv cluster - run_sql "USE TEST;Create table t1(a int primary key, b int);insert into t1 values(1,2);insert into t1 values(2,3);" - check_table_exists "test.t1" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} - - sleep 5 # make data inserted into downstream - kill_tidb - - # test the case when pdNow - lastSyncedTs < threshold - synced_status=`curl -X GET http://127.0.0.1:8300/api/v2/changefeeds/test-1/synced` - status=$(echo $synced_status | jq '.synced') - if [ $status != false ]; then - echo "synced status isn't correct" - exit 1 - fi - info=$(echo $synced_status | jq -r '.info') - target_message="The data syncing is not finished, please wait" - - if [ "$info" != "$target_message" ]; then - echo "synced status info is not correct" - exit 1 - fi - - sleep 130 # wait enough time for pass synced-check-interval - # test the case when pdNow - lastSyncedTs > threshold - synced_status=`curl -X GET http://127.0.0.1:8300/api/v2/changefeeds/test-1/synced` - status=$(echo $synced_status | jq '.synced') - if [ $status != true ]; then - echo "synced status isn't correct" - exit 1 - fi - info=$(echo $synced_status | jq -r '.info') - target_message="Data syncing is finished" - - if [ "$info" != "$target_message" ]; then - echo "synced status info is not correct" - exit 1 - fi + # case 3: test in unavailable tikv cluster + run_sql "USE TEST;Create table t1(a int primary key, b int);insert into t1 values(1,2);insert into t1 values(2,3);" + check_table_exists "test.t1" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + + sleep 5 # make data inserted into downstream + kill_tidb + + # test the case when pdNow - lastSyncedTs < threshold + synced_status=$(curl -X GET http://127.0.0.1:8300/api/v2/changefeeds/test-1/synced) + status=$(echo $synced_status | jq '.synced') + if [ $status != false ]; then + echo "synced status isn't correct" + exit 1 + fi + info=$(echo $synced_status | jq -r '.info') + target_message="The data syncing is not finished, please wait" + + if [ "$info" != "$target_message" ]; then + echo "synced status info is not correct" + exit 1 + fi + + sleep 130 # wait enough time for pass synced-check-interval + # test the case when pdNow - lastSyncedTs > threshold + synced_status=$(curl -X GET http://127.0.0.1:8300/api/v2/changefeeds/test-1/synced) + status=$(echo $synced_status | jq '.synced') + if [ $status != true ]; then + echo "synced status isn't correct" + exit 1 + fi + info=$(echo $synced_status | jq -r '.info') + target_message="Data syncing is finished" + + if [ "$info" != "$target_message" ]; then + echo "synced status info is not correct" + exit 1 + fi cleanup_process $CDC_BINARY - stop_tidb_cluster + stop_tidb_cluster } function run_case_with_failpoint() { @@ -267,40 +267,39 @@ function run_case_with_failpoint() { cd $WORK_DIR - # make failpoint to block checkpoint-ts - export GO_FAILPOINTS='github.com/pingcap/tiflow/cdc/owner/ChangefeedOwnerDontUpdateCheckpoint=return(true)' + # make failpoint to block checkpoint-ts + export GO_FAILPOINTS='github.com/pingcap/tiflow/cdc/owner/ChangefeedOwnerDontUpdateCheckpoint=return(true)' - start_ts=$(run_cdc_cli_tso_query ${UP_PD_HOST_1} ${UP_PD_PORT_1}) + start_ts=$(run_cdc_cli_tso_query ${UP_PD_HOST_1} ${UP_PD_PORT_1}) run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY - config_path=$1 + config_path=$1 SINK_URI="mysql://root@127.0.0.1:3306/?max-txn-row=1" run_cdc_cli changefeed create --start-ts=$start_ts --sink-uri="$SINK_URI" --changefeed-id="test-1" --config="$CUR/$config_path" - sleep 20 # wait enough time for pass checkpoint-check-interval - synced_status=`curl -X GET http://127.0.0.1:8300/api/v2/changefeeds/test-1/synced` - status=$(echo $synced_status | jq '.synced') - if [ $status != false ]; then - echo "synced status isn't correct" - exit 1 - fi - info=$(echo $synced_status | jq -r '.info') - target_message="Please check whether pd is health and tikv region is all available. \ + sleep 20 # wait enough time for pass checkpoint-check-interval + synced_status=$(curl -X GET http://127.0.0.1:8300/api/v2/changefeeds/test-1/synced) + status=$(echo $synced_status | jq '.synced') + if [ $status != false ]; then + echo "synced status isn't correct" + exit 1 + fi + info=$(echo $synced_status | jq -r '.info') + target_message="Please check whether pd is health and tikv region is all available. \ If pd is not health or tikv region is not available, the data syncing is finished. \ Otherwise the data syncing is not finished, please wait" - if [ "$info" != "$target_message" ]; then - echo "synced status info is not correct" - exit 1 - fi + if [ "$info" != "$target_message" ]; then + echo "synced status info is not correct" + exit 1 + fi - export GO_FAILPOINTS='' + export GO_FAILPOINTS='' cleanup_process $CDC_BINARY - stop_tidb_cluster + stop_tidb_cluster } - trap stop_tidb_cluster EXIT run_normal_case_and_unavailable_pd "conf/changefeed.toml" run_case_with_unavailable_tikv "conf/changefeed.toml" @@ -314,4 +313,4 @@ run_case_with_unavailable_tidb "conf/changefeed-redo.toml" run_case_with_failpoint "conf/changefeed-redo.toml" check_logs $WORK_DIR -echo "[$(date)] <<<<<< run test case $TEST_NAME success! >>>>>>" \ No newline at end of file +echo "[$(date)] <<<<<< run test case $TEST_NAME success! >>>>>>" From 275ac708f0edd10a2a49c035ddde7d5caaac2590 Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Mon, 11 Dec 2023 21:32:18 +0800 Subject: [PATCH 13/35] update --- cdc/api/v1/api_test.go | 7 +++++++ cdc/sink/tablesink/table_sink_impl.go | 2 +- 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/cdc/api/v1/api_test.go b/cdc/api/v1/api_test.go index 24b08e2d491..eb3f702bfd4 100644 --- a/cdc/api/v1/api_test.go +++ b/cdc/api/v1/api_test.go @@ -100,6 +100,13 @@ func (p *mockStatusProvider) GetCaptures(ctx context.Context) ([]*model.CaptureI return args.Get(0).([]*model.CaptureInfo), args.Error(1) } +func (p *mockStatusProvider) GetChangeFeedSyncedStatus(ctx context.Context, + changefeedID model.ChangeFeedID, +) (*model.ChangeFeedSyncedStatusForAPI, error) { + args := p.Called(ctx) + return args.Get(0).(*model.ChangeFeedSyncedStatusForAPI), args.Error(1) +} + func (p *mockStatusProvider) IsHealthy(ctx context.Context) (bool, error) { args := p.Called(ctx) return args.Get(0).(bool), args.Error(1) diff --git a/cdc/sink/tablesink/table_sink_impl.go b/cdc/sink/tablesink/table_sink_impl.go index c86639350ff..24a2984704d 100644 --- a/cdc/sink/tablesink/table_sink_impl.go +++ b/cdc/sink/tablesink/table_sink_impl.go @@ -40,6 +40,7 @@ type LastSyncedTsRecord struct { lastSyncedTs model.Ts } +// getLastSyncedTs get value from LastSyncedTsRecord func (r *LastSyncedTsRecord) getLastSyncedTs() model.Ts { r.Lock() defer r.Unlock() @@ -47,7 +48,6 @@ func (r *LastSyncedTsRecord) getLastSyncedTs() model.Ts { } // EventTableSink is a table sink that can write events. - type EventTableSink[E dmlsink.TableEvent, P dmlsink.Appender[E]] struct { changefeedID model.ChangeFeedID span tablepb.Span From e32aef51ec5852faf77a798a40be84fd7527d335 Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Mon, 11 Dec 2023 21:38:37 +0800 Subject: [PATCH 14/35] update --- tests/integration_tests/run_group.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration_tests/run_group.sh b/tests/integration_tests/run_group.sh index 3043feedb63..57400100d8d 100755 --- a/tests/integration_tests/run_group.sh +++ b/tests/integration_tests/run_group.sh @@ -40,7 +40,7 @@ groups=( ["G06"]='sink_retry changefeed_error ddl_sequence resourcecontrol' ["G07"]='kv_client_stream_reconnect cdc split_region' ["G08"]='processor_err_chan changefeed_reconstruct multi_capture' - ["G09"]='gc_safepoint changefeed_pause_resume cli savepoint' + ["G09"]='gc_safepoint changefeed_pause_resume cli savepoint synced_status' ["G10"]='default_value simple cdc_server_tips event_filter' ["G11"]='resolve_lock move_table autorandom generate_column' ["G12"]='many_pk_or_uk capture_session_done_during_task ddl_attributes' @@ -54,7 +54,7 @@ groups=( ["G18"]='availability http_proxies sequence' ["G19"]='changefeed_fast_fail batch_update_to_no_batch changefeed_resume_with_checkpoint_ts' ["G20"]='tidb_mysql_test ddl_reentrant multi_cdc_cluster' - ["G21"]='bank kill_owner_with_ddl owner_remove_table_error synced_status' + ["G21"]='bank kill_owner_with_ddl owner_remove_table_error' ) # Get other cases not in groups, to avoid missing any case From c6bf6159a1e98056041fcbbd1fd5d314a22cd9ae Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Mon, 11 Dec 2023 22:18:31 +0800 Subject: [PATCH 15/35] fix ut --- cdc/api/v2/changefeed_test.go | 4 ++-- cdc/api/v2/model_test.go | 3 ++- cdc/model/changefeed.go | 3 +++ pkg/config/config_test_data.go | 18 +++++++++++++++--- pkg/config/replica_config.go | 2 +- 5 files changed, 23 insertions(+), 7 deletions(-) diff --git a/cdc/api/v2/changefeed_test.go b/cdc/api/v2/changefeed_test.go index 3ff0a81ab24..73ef0134c06 100644 --- a/cdc/api/v2/changefeed_test.go +++ b/cdc/api/v2/changefeed_test.go @@ -1041,7 +1041,7 @@ func TestChangefeedSynced(t *testing.T) { require.Equal(t, false, resp.Synced) require.Equal(t, "1970-01-01 08:00:00", resp.NowTs) require.Equal(t, "[CDC:ErrAPIGetPDClientFailed]failed to get PDClient to connect PD, "+ - "please recheck: [pd] failed to get cluster id. Besides the data is not finish syncing", resp.Info) + "please recheck. Besides the data is not finish syncing", resp.Info) } { @@ -1066,7 +1066,7 @@ func TestChangefeedSynced(t *testing.T) { require.Nil(t, err) require.Equal(t, false, resp.Synced) require.Equal(t, "1970-01-01 08:00:00", resp.NowTs) - require.Equal(t, "[CDC:ErrAPIGetPDClientFailed]failed to get PDClient to connect PD, please recheck: [pd] failed to get cluster id. "+ + require.Equal(t, "[CDC:ErrAPIGetPDClientFailed]failed to get PDClient to connect PD, please recheck. "+ "You can check the pd first, and if pd is available, means we don't finish sync data. "+ "If pd is not available, please check the whether we satisfy the condition that "+ "the time difference from lastSyncedTs to the current time from the time zone of pd is greater than 300 secs. "+ diff --git a/cdc/api/v2/model_test.go b/cdc/api/v2/model_test.go index 8214068ab07..2ad93496329 100644 --- a/cdc/api/v2/model_test.go +++ b/cdc/api/v2/model_test.go @@ -84,7 +84,8 @@ var defaultAPIConfig = &ReplicaConfig{ }, ChangefeedErrorStuckDuration: &JSONDuration{*config. GetDefaultReplicaConfig().ChangefeedErrorStuckDuration}, - SQLMode: config.GetDefaultReplicaConfig().SQLMode, + SQLMode: config.GetDefaultReplicaConfig().SQLMode, + SyncedStatus: (*SyncedStatusConfig)(config.GetDefaultReplicaConfig().SyncedStatus), } func TestDefaultReplicaConfig(t *testing.T) { diff --git a/cdc/model/changefeed.go b/cdc/model/changefeed.go index a0cab212ee2..6abaf00e249 100644 --- a/cdc/model/changefeed.go +++ b/cdc/model/changefeed.go @@ -337,6 +337,9 @@ func (info *ChangeFeedInfo) VerifyAndComplete() { if info.Config.SQLMode == "" { info.Config.SQLMode = defaultConfig.SQLMode } + if info.Config.SyncedStatus == nil { + info.Config.SyncedStatus = defaultConfig.SyncedStatus + } info.RmUnusedFields() } diff --git a/pkg/config/config_test_data.go b/pkg/config/config_test_data.go index 13703e25eb6..14b896ab3ef 100644 --- a/pkg/config/config_test_data.go +++ b/pkg/config/config_test_data.go @@ -87,7 +87,11 @@ const ( "corruption-handle-level": "warn" }, "changefeed-error-stuck-duration": 1800000000000, - "sql-mode":"ONLY_FULL_GROUP_BY,STRICT_TRANS_TABLES,NO_ZERO_IN_DATE,NO_ZERO_DATE,ERROR_FOR_DIVISION_BY_ZERO,NO_AUTO_CREATE_USER,NO_ENGINE_SUBSTITUTION" + "sql-mode":"ONLY_FULL_GROUP_BY,STRICT_TRANS_TABLES,NO_ZERO_IN_DATE,NO_ZERO_DATE,ERROR_FOR_DIVISION_BY_ZERO,NO_AUTO_CREATE_USER,NO_ENGINE_SUBSTITUTION", + "synced-status": { + "synced-check-interval": 300, + "checkpoint-interval": 15 + } }` testCfgTestServerConfigMarshal = `{ @@ -333,7 +337,11 @@ const ( "corruption-handle-level": "warn" }, "changefeed-error-stuck-duration": 1800000000000, - "sql-mode":"ONLY_FULL_GROUP_BY,STRICT_TRANS_TABLES,NO_ZERO_IN_DATE,NO_ZERO_DATE,ERROR_FOR_DIVISION_BY_ZERO,NO_AUTO_CREATE_USER,NO_ENGINE_SUBSTITUTION" + "sql-mode":"ONLY_FULL_GROUP_BY,STRICT_TRANS_TABLES,NO_ZERO_IN_DATE,NO_ZERO_DATE,ERROR_FOR_DIVISION_BY_ZERO,NO_AUTO_CREATE_USER,NO_ENGINE_SUBSTITUTION", + "synced-status": { + "synced-check-interval": 300, + "checkpoint-interval": 15 + } }` testCfgTestReplicaConfigMarshal2 = `{ @@ -484,6 +492,10 @@ const ( "corruption-handle-level": "warn" }, "changefeed-error-stuck-duration": 1800000000000, - "sql-mode":"ONLY_FULL_GROUP_BY,STRICT_TRANS_TABLES,NO_ZERO_IN_DATE,NO_ZERO_DATE,ERROR_FOR_DIVISION_BY_ZERO,NO_AUTO_CREATE_USER,NO_ENGINE_SUBSTITUTION" + "sql-mode":"ONLY_FULL_GROUP_BY,STRICT_TRANS_TABLES,NO_ZERO_IN_DATE,NO_ZERO_DATE,ERROR_FOR_DIVISION_BY_ZERO,NO_AUTO_CREATE_USER,NO_ENGINE_SUBSTITUTION", + "synced-status": { + "synced-check-interval": 300, + "checkpoint-interval": 15 + } }` ) diff --git a/pkg/config/replica_config.go b/pkg/config/replica_config.go index 2b67b5accd7..a6101d3ddc6 100644 --- a/pkg/config/replica_config.go +++ b/pkg/config/replica_config.go @@ -101,7 +101,7 @@ var defaultReplicaConfig = &ReplicaConfig{ }, ChangefeedErrorStuckDuration: util.AddressOf(time.Minute * 30), SQLMode: defaultSQLMode, - SyncedStatus: &SyncedStatusConfig{SyncedCheckInterval: 5 * 60, CheckpointInterval: 5}, + SyncedStatus: &SyncedStatusConfig{SyncedCheckInterval: 5 * 60, CheckpointInterval: 15}, } // GetDefaultReplicaConfig returns the default replica config. From aee4463c9f0bfd982d55a3bff084214c1542b156 Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Tue, 12 Dec 2023 10:19:19 +0800 Subject: [PATCH 16/35] fix test --- cdc/api/v2/changefeed_test.go | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/cdc/api/v2/changefeed_test.go b/cdc/api/v2/changefeed_test.go index 73ef0134c06..cedcc61fc44 100644 --- a/cdc/api/v2/changefeed_test.go +++ b/cdc/api/v2/changefeed_test.go @@ -1020,11 +1020,11 @@ func TestChangefeedSynced(t *testing.T) { { helpers.EXPECT().getPDClient(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil, cerror.ErrAPIGetPDClientFailed).Times(1) - // case3: pd is offline,resolvedTs - checkpointTs > 5s + // case3: pd is offline,resolvedTs - checkpointTs > 15s statusProvider.changeFeedSyncedStatusForAPI = &model.ChangeFeedSyncedStatusForAPI{ CheckpointTs: 1701153217279, LastSyncedTs: 1701153217279, - PullerResolvedTs: 1701153227279, + PullerResolvedTs: 1701153247279, } w := httptest.NewRecorder() req, _ := http.NewRequestWithContext( @@ -1101,11 +1101,11 @@ func TestChangefeedSynced(t *testing.T) { } { - // case6: pdTs - lastSyncedTs > 5min, pdTs - checkpointTs > 5s, resolvedTs - checkpointTs < 5s + // case6: pdTs - lastSyncedTs > 5min, pdTs - checkpointTs > 15s, resolvedTs - checkpointTs < 15s statusProvider.changeFeedSyncedStatusForAPI = &model.ChangeFeedSyncedStatusForAPI{ - CheckpointTs: 1701153211279, + CheckpointTs: 1701153201279, LastSyncedTs: 1701152217279, - PullerResolvedTs: 1701153211379, + PullerResolvedTs: 1701153201379, } w := httptest.NewRecorder() req, _ := http.NewRequestWithContext( @@ -1126,10 +1126,10 @@ func TestChangefeedSynced(t *testing.T) { } { - // case7: pdTs - lastSyncedTs > 5min, pdTs - checkpointTs > 5s, resolvedTs - checkpointTs > 5s + // case7: pdTs - lastSyncedTs > 5min, pdTs - checkpointTs > 15s, resolvedTs - checkpointTs > 15s statusProvider.changeFeedSyncedStatusForAPI = &model.ChangeFeedSyncedStatusForAPI{ - CheckpointTs: 1701153211279, - LastSyncedTs: 1701152217279, + CheckpointTs: 1701153201279, + LastSyncedTs: 1701152207279, PullerResolvedTs: 1701153218279, } w := httptest.NewRecorder() From c62086d77ef5e97213d65c674fc4d1bfdb3d1d9e Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Tue, 12 Dec 2023 14:35:50 +0800 Subject: [PATCH 17/35] update --- cdc/api/v2/model.go | 4 ++-- pkg/orchestrator/reactor_state_test.go | 11 ++++++++--- 2 files changed, 10 insertions(+), 5 deletions(-) diff --git a/cdc/api/v2/model.go b/cdc/api/v2/model.go index c54e97e0ff9..084537eff3e 100644 --- a/cdc/api/v2/model.go +++ b/cdc/api/v2/model.go @@ -106,10 +106,10 @@ type ChangefeedCommonInfo struct { // SyncedStatusConfig represents synced check interval config for a changefeed type SyncedStatusConfig struct { // The minimum interval between the latest synced ts and now required to reach synced state - SyncedCheckInterval int64 `json:"synced-check-interval"` + SyncedCheckInterval int64 `json:"synced_check_interval"` // The maximum interval between latest checkpoint ts and now or // between latest sink's checkpoint ts and puller's checkpoint ts required to reach synced state - CheckpointInterval int64 `json:"checkpoint-interval"` + CheckpointInterval int64 `json:"checkpoint_interval"` } // MarshalJSON marshal changefeed common info to json diff --git a/pkg/orchestrator/reactor_state_test.go b/pkg/orchestrator/reactor_state_test.go index 1468264d5ba..97eb8e8aaaa 100644 --- a/pkg/orchestrator/reactor_state_test.go +++ b/pkg/orchestrator/reactor_state_test.go @@ -132,7 +132,8 @@ func TestChangefeedStateUpdate(t *testing.T) { Integrity: config.GetDefaultReplicaConfig().Integrity, ChangefeedErrorStuckDuration: config. GetDefaultReplicaConfig().ChangefeedErrorStuckDuration, - SQLMode: config.GetDefaultReplicaConfig().SQLMode, + SQLMode: config.GetDefaultReplicaConfig().SQLMode, + SyncedStatus: config.GetDefaultReplicaConfig().SyncedStatus, }, }, Status: &model.ChangeFeedStatus{CheckpointTs: 421980719742451713}, @@ -196,7 +197,8 @@ func TestChangefeedStateUpdate(t *testing.T) { Consistent: config.GetDefaultReplicaConfig().Consistent, ChangefeedErrorStuckDuration: config. GetDefaultReplicaConfig().ChangefeedErrorStuckDuration, - SQLMode: config.GetDefaultReplicaConfig().SQLMode, + SQLMode: config.GetDefaultReplicaConfig().SQLMode, + SyncedStatus: config.GetDefaultReplicaConfig().SyncedStatus, }, }, Status: &model.ChangeFeedStatus{CheckpointTs: 421980719742451713}, @@ -265,7 +267,8 @@ func TestChangefeedStateUpdate(t *testing.T) { Integrity: config.GetDefaultReplicaConfig().Integrity, ChangefeedErrorStuckDuration: config. GetDefaultReplicaConfig().ChangefeedErrorStuckDuration, - SQLMode: config.GetDefaultReplicaConfig().SQLMode, + SQLMode: config.GetDefaultReplicaConfig().SQLMode, + SyncedStatus: config.GetDefaultReplicaConfig().SyncedStatus, }, }, Status: &model.ChangeFeedStatus{CheckpointTs: 421980719742451713}, @@ -361,6 +364,7 @@ func TestPatchInfo(t *testing.T) { Integrity: defaultConfig.Integrity, ChangefeedErrorStuckDuration: defaultConfig.ChangefeedErrorStuckDuration, SQLMode: defaultConfig.SQLMode, + SyncedStatus: defaultConfig.SyncedStatus, }, } cfInfo.RmUnusedFields() @@ -384,6 +388,7 @@ func TestPatchInfo(t *testing.T) { Integrity: defaultConfig.Integrity, ChangefeedErrorStuckDuration: defaultConfig.ChangefeedErrorStuckDuration, SQLMode: defaultConfig.SQLMode, + SyncedStatus: defaultConfig.SyncedStatus, }, } cfInfo.RmUnusedFields() From d5ead0a5f7554bcce7c138f2d6ec2a4d01fe6509 Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Tue, 12 Dec 2023 14:36:43 +0800 Subject: [PATCH 18/35] revert change --- cdc/owner/changefeed.go | 18 +++++++++++++----- tests/integration_tests/synced_status/run.sh | 2 +- 2 files changed, 14 insertions(+), 6 deletions(-) diff --git a/cdc/owner/changefeed.go b/cdc/owner/changefeed.go index 212cac7c649..1fecd62c56a 100755 --- a/cdc/owner/changefeed.go +++ b/cdc/owner/changefeed.go @@ -167,6 +167,8 @@ type changefeed struct { opts ...observer.NewObserverOption, ) (observer.Observer, error) + lastDDLTs uint64 // Timestamp of the last executed DDL. Only used for tests. + // The latest changefeed info and status from meta storage. they are updated in every Tick. latestInfo *model.ChangeFeedInfo latestStatus *model.ChangeFeedStatus @@ -469,11 +471,17 @@ func (c *changefeed) tick(ctx cdcContext.Context, } failpoint.Inject("ChangefeedOwnerDontUpdateCheckpoint", func() { - log.Info("owner won't update checkpoint because of failpoint", - zap.String("namespace", c.id.Namespace), - zap.String("changefeed", c.id.ID), - zap.Uint64("keepCheckpoint", c.latestStatus.CheckpointTs), - zap.Uint64("skipCheckpoint", newCheckpointTs)) + if c.lastDDLTs != 0 && c.latestStatus.CheckpointTs >= c.lastDDLTs { + log.Info("owner won't update checkpoint because of failpoint", + zap.String("namespace", c.id.Namespace), + zap.String("changefeed", c.id.ID), + zap.Uint64("keepCheckpoint", c.latestStatus.CheckpointTs), + zap.Uint64("skipCheckpoint", newCheckpointTs)) + newCheckpointTs = c.latestStatus.CheckpointTs + } + }) + + failpoint.Inject("ChangefeedOwnerNotUpdateCheckpoint", func() { newCheckpointTs = c.latestStatus.CheckpointTs }) diff --git a/tests/integration_tests/synced_status/run.sh b/tests/integration_tests/synced_status/run.sh index 735ecdc3d89..0f3f92f7867 100644 --- a/tests/integration_tests/synced_status/run.sh +++ b/tests/integration_tests/synced_status/run.sh @@ -268,7 +268,7 @@ function run_case_with_failpoint() { cd $WORK_DIR # make failpoint to block checkpoint-ts - export GO_FAILPOINTS='github.com/pingcap/tiflow/cdc/owner/ChangefeedOwnerDontUpdateCheckpoint=return(true)' + export GO_FAILPOINTS='github.com/pingcap/tiflow/cdc/owner/ChangefeedOwnerNotUpdateCheckpoint=return(true)' start_ts=$(run_cdc_cli_tso_query ${UP_PD_HOST_1} ${UP_PD_PORT_1}) run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY From 6edef57f6b59a410dc19afa013d7b7aedfb9f5ff Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Tue, 12 Dec 2023 17:47:18 +0800 Subject: [PATCH 19/35] update it --- tests/integration_tests/synced_status/run.sh | 21 ++++++-------------- 1 file changed, 6 insertions(+), 15 deletions(-) diff --git a/tests/integration_tests/synced_status/run.sh b/tests/integration_tests/synced_status/run.sh index 0f3f92f7867..cc8c8df1b52 100644 --- a/tests/integration_tests/synced_status/run.sh +++ b/tests/integration_tests/synced_status/run.sh @@ -122,24 +122,15 @@ function run_normal_case_and_unavailable_pd() { # case 2: test with unavailable pd kill_pd - synced_status=$(curl -X GET http://127.0.0.1:8300/api/v2/changefeeds/test-1/synced) - status=$(echo $synced_status | jq '.synced') - if [ $status != false ]; then - echo "synced status isn't correct" - exit 1 - fi - info=$(echo $synced_status | jq -r '.info') - target_message="[CDC:ErrAPIGetPDClientFailed]failed to get PDClient to connect PD, please recheck: [pd] failed to get cluster id. \ -You can check the pd first, and if pd is available, means we don't finish sync data. \ -If pd is not available, please check the whether we satisfy the condition that \ -the time difference from lastSyncedTs to the current time from the time zone of pd is greater than 120 secs. \ -If it's satisfied, means the data syncing is totally finished" + sleep 20 - if [ "$info" != "$target_message" ]; then - echo "synced status info is not correct" + synced_status=$(curl -X GET http://127.0.0.1:8300/api/v2/changefeeds/test-1/synced) + error_code=$(echo $synced_status | jq -r '.error_code') + if [ $error_code != "CDC:ErrPDEtcdAPIError" ]; then + echo "error_code isn't correct" exit 1 fi - + cleanup_process $CDC_BINARY stop_tidb_cluster } From 440b4e8dfb806349ce27039267fb936fcbc5d0d6 Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Tue, 12 Dec 2023 19:02:55 +0800 Subject: [PATCH 20/35] update --- tests/integration_tests/synced_status/run.sh | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/integration_tests/synced_status/run.sh b/tests/integration_tests/synced_status/run.sh index cc8c8df1b52..a1bb3d823f4 100644 --- a/tests/integration_tests/synced_status/run.sh +++ b/tests/integration_tests/synced_status/run.sh @@ -130,7 +130,6 @@ function run_normal_case_and_unavailable_pd() { echo "error_code isn't correct" exit 1 fi - cleanup_process $CDC_BINARY stop_tidb_cluster } From 1cc43cfa742525a09bb765e98d6efb99b5b52a7e Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Tue, 12 Dec 2023 20:08:22 +0800 Subject: [PATCH 21/35] update --- tests/integration_tests/synced_status/run.sh | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/tests/integration_tests/synced_status/run.sh b/tests/integration_tests/synced_status/run.sh index a1bb3d823f4..4713d02a26e 100644 --- a/tests/integration_tests/synced_status/run.sh +++ b/tests/integration_tests/synced_status/run.sh @@ -119,17 +119,14 @@ function run_normal_case_and_unavailable_pd() { fi #========== - # case 2: test with unavailable pd + # case 2: test with unavailable pd, query will not get the available response kill_pd sleep 20 synced_status=$(curl -X GET http://127.0.0.1:8300/api/v2/changefeeds/test-1/synced) error_code=$(echo $synced_status | jq -r '.error_code') - if [ $error_code != "CDC:ErrPDEtcdAPIError" ]; then - echo "error_code isn't correct" - exit 1 - fi + cleanup_process $CDC_BINARY stop_tidb_cluster } From a92161331efebafb5e8bfea8b470d512f23fd5b9 Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Tue, 12 Dec 2023 20:38:30 +0800 Subject: [PATCH 22/35] update --- tests/integration_tests/synced_status/run.sh | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/integration_tests/synced_status/run.sh b/tests/integration_tests/synced_status/run.sh index 4713d02a26e..387e7d03f0a 100644 --- a/tests/integration_tests/synced_status/run.sh +++ b/tests/integration_tests/synced_status/run.sh @@ -126,7 +126,6 @@ function run_normal_case_and_unavailable_pd() { synced_status=$(curl -X GET http://127.0.0.1:8300/api/v2/changefeeds/test-1/synced) error_code=$(echo $synced_status | jq -r '.error_code') - cleanup_process $CDC_BINARY stop_tidb_cluster } From b9f7b1bd66a86db4925b4aa1d6206c0f03548ae4 Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Tue, 12 Dec 2023 22:02:20 +0800 Subject: [PATCH 23/35] update --- cdc/processor/sinkmanager/manager_test.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/cdc/processor/sinkmanager/manager_test.go b/cdc/processor/sinkmanager/manager_test.go index d8f63109c3a..189a2e9adbc 100644 --- a/cdc/processor/sinkmanager/manager_test.go +++ b/cdc/processor/sinkmanager/manager_test.go @@ -27,6 +27,7 @@ import ( "github.com/pingcap/tiflow/pkg/config" "github.com/pingcap/tiflow/pkg/spanz" "github.com/stretchr/testify/require" + "go.uber.org/zap" ) func getChangefeedInfo() *model.ChangeFeedInfo { @@ -211,6 +212,7 @@ func TestGenerateTableSinkTaskWithBarrierTs(t *testing.T) { manager.UpdateBarrierTs(6, nil) require.Eventually(t, func() bool { s := manager.GetTableStats(span) + log.Info("current value", zap.Uint64("checkpoint ts", s.CheckpointTs), zap.Uint64("last synced ts", s.LastSyncedTs)) return s.CheckpointTs == 6 && s.LastSyncedTs == 4 }, 5*time.Second, 10*time.Millisecond) } From 573ef5556c0ac64b930533831959981e8aa46916 Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Tue, 12 Dec 2023 22:20:29 +0800 Subject: [PATCH 24/35] update --- cdc/processor/sinkmanager/manager_test.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/cdc/processor/sinkmanager/manager_test.go b/cdc/processor/sinkmanager/manager_test.go index 189a2e9adbc..6818554cb3f 100644 --- a/cdc/processor/sinkmanager/manager_test.go +++ b/cdc/processor/sinkmanager/manager_test.go @@ -212,7 +212,8 @@ func TestGenerateTableSinkTaskWithBarrierTs(t *testing.T) { manager.UpdateBarrierTs(6, nil) require.Eventually(t, func() bool { s := manager.GetTableStats(span) - log.Info("current value", zap.Uint64("checkpoint ts", s.CheckpointTs), zap.Uint64("last synced ts", s.LastSyncedTs)) + log.Info("current value", zap.Uint64("checkpointTs", s.CheckpointTs), + zap.Uint64("lastSyncedTs", s.LastSyncedTs)) return s.CheckpointTs == 6 && s.LastSyncedTs == 4 }, 5*time.Second, 10*time.Millisecond) } From e276bde32bbdddadbfe9e3d3e47e94551d614447 Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Wed, 13 Dec 2023 09:15:18 +0800 Subject: [PATCH 25/35] for test --- cdc/processor/sinkmanager/manager_test.go | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/cdc/processor/sinkmanager/manager_test.go b/cdc/processor/sinkmanager/manager_test.go index 6818554cb3f..fdb851c2552 100644 --- a/cdc/processor/sinkmanager/manager_test.go +++ b/cdc/processor/sinkmanager/manager_test.go @@ -207,14 +207,15 @@ func TestGenerateTableSinkTaskWithBarrierTs(t *testing.T) { return s.CheckpointTs == 4 && s.LastSyncedTs == 4 }, 5*time.Second, 10*time.Millisecond) + manager.UpdateBarrierTs(6, nil) manager.UpdateReceivedSorterResolvedTs(span, 6) manager.schemaStorage.AdvanceResolvedTs(6) - manager.UpdateBarrierTs(6, nil) require.Eventually(t, func() bool { s := manager.GetTableStats(span) log.Info("current value", zap.Uint64("checkpointTs", s.CheckpointTs), zap.Uint64("lastSyncedTs", s.LastSyncedTs)) - return s.CheckpointTs == 6 && s.LastSyncedTs == 4 + // return s.CheckpointTs == 6 && s.LastSyncedTs == 4 + return s.CheckpointTs == 6 }, 5*time.Second, 10*time.Millisecond) } From f6f8d0a38b7352d3287b5c2a1e1e976b411e500d Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Wed, 13 Dec 2023 09:35:10 +0800 Subject: [PATCH 26/35] update --- cdc/processor/sinkmanager/manager_test.go | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/cdc/processor/sinkmanager/manager_test.go b/cdc/processor/sinkmanager/manager_test.go index fdb851c2552..749fb68c248 100644 --- a/cdc/processor/sinkmanager/manager_test.go +++ b/cdc/processor/sinkmanager/manager_test.go @@ -214,8 +214,7 @@ func TestGenerateTableSinkTaskWithBarrierTs(t *testing.T) { s := manager.GetTableStats(span) log.Info("current value", zap.Uint64("checkpointTs", s.CheckpointTs), zap.Uint64("lastSyncedTs", s.LastSyncedTs)) - // return s.CheckpointTs == 6 && s.LastSyncedTs == 4 - return s.CheckpointTs == 6 + return s.CheckpointTs == 6 && s.LastSyncedTs == 4 }, 5*time.Second, 10*time.Millisecond) } From 9cfd6f20d4cddf43bdb4c8eb7a6cec74af085953 Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Wed, 13 Dec 2023 10:19:43 +0800 Subject: [PATCH 27/35] update --- cdc/api/v2/api_test.go | 20 ++++++------- cdc/api/v2/changefeed.go | 34 +++++++++++++++------- cdc/api/v2/changefeed_test.go | 16 +++++----- cdc/owner/changefeed.go | 14 --------- cdc/owner/changefeed_test.go | 1 - cdc/processor/sinkmanager/manager_test.go | 3 -- cdc/sink/tablesink/table_sink_impl_test.go | 3 -- 7 files changed, 41 insertions(+), 50 deletions(-) diff --git a/cdc/api/v2/api_test.go b/cdc/api/v2/api_test.go index 3ab0c9d0291..98072ecc68d 100644 --- a/cdc/api/v2/api_test.go +++ b/cdc/api/v2/api_test.go @@ -62,14 +62,14 @@ func (c *mockPDClient) Close() {} type mockStatusProvider struct { owner.StatusProvider - changefeedStatus *model.ChangeFeedStatusForAPI - changefeedInfo *model.ChangeFeedInfo - processors []*model.ProcInfoSnap - taskStatus map[model.CaptureID]*model.TaskStatus - changefeedInfos map[model.ChangeFeedID]*model.ChangeFeedInfo - changefeedStatuses map[model.ChangeFeedID]*model.ChangeFeedStatusForAPI - changeFeedSyncedStatusForAPI *model.ChangeFeedSyncedStatusForAPI - err error + changefeedStatus *model.ChangeFeedStatusForAPI + changefeedInfo *model.ChangeFeedInfo + processors []*model.ProcInfoSnap + taskStatus map[model.CaptureID]*model.TaskStatus + changefeedInfos map[model.ChangeFeedID]*model.ChangeFeedInfo + changefeedStatuses map[model.ChangeFeedID]*model.ChangeFeedStatusForAPI + changeFeedSyncedStatus *model.ChangeFeedSyncedStatusForAPI + err error } // GetChangeFeedStatus returns a changefeeds' runtime status. @@ -121,12 +121,12 @@ func (m *mockStatusProvider) GetAllChangeFeedStatuses(_ context.Context) ( return m.changefeedStatuses, m.err } -// GetChangeFeedSyncedStatusForAPI returns a mock changefeed status. +// GetChangeFeedSyncedStatus returns a mock changefeed status. func (m *mockStatusProvider) GetChangeFeedSyncedStatus(_ context.Context, changefeedID model.ChangeFeedID) ( *model.ChangeFeedSyncedStatusForAPI, error, ) { - return m.changeFeedSyncedStatusForAPI, m.err + return m.changeFeedSyncedStatus, m.err } func (m *mockStatusProvider) IsChangefeedOwner(_ context.Context, id model.ChangeFeedID) (bool, error) { diff --git a/cdc/api/v2/changefeed.go b/cdc/api/v2/changefeed.go index 0d97db7ecac..9606090f40f 100644 --- a/cdc/api/v2/changefeed.go +++ b/cdc/api/v2/changefeed.go @@ -896,12 +896,24 @@ func (h *OpenAPIV2) status(c *gin.Context) { }) } +// transformer timestamp to readable format func transformerTime(timestamp int64) string { location := time.Local tm := time.Unix((timestamp / 1000), 0).In(location) return tm.Format("2006-01-02 15:04:05") } +// synced get the synced status of a changefeed +// @Summary Get synced status +// @Description get the synced status of a changefeed +// @Tags changefeed,v2 +// @Accept json +// @Produce json +// @Param changefeed_id path string true "changefeed_id" +// @Param namespace query string false "default" +// @Success 200 {object} SyncedStatus +// @Failure 500,400 {object} model.HTTPError +// @Router /api/v2/changefeeds/{changefeed_id}/synced [get] func (h *OpenAPIV2) synced(c *gin.Context) { ctx := c.Request.Context() @@ -922,7 +934,7 @@ func (h *OpenAPIV2) synced(c *gin.Context) { return } - log.Info("Get changefeed synced status:", zap.Any("status", status)) + log.Info("Get changefeed synced status:", zap.Any("status", status), zap.Any("changefeedID", changefeedID)) cfg := &ChangefeedConfig{ReplicaConfig: GetDefaultReplicaConfig()} if (status.SyncedCheckInterval != 0) && (status.CheckpointInterval != 0) { @@ -930,6 +942,7 @@ func (h *OpenAPIV2) synced(c *gin.Context) { cfg.ReplicaConfig.SyncedStatus.SyncedCheckInterval = status.SyncedCheckInterval } + // get pd client if len(cfg.PDAddrs) == 0 { up, err := getCaptureDefaultUpstream(h.capture) if err != nil { @@ -944,7 +957,8 @@ func (h *OpenAPIV2) synced(c *gin.Context) { defer cancel() pdClient, err := h.helpers.getPDClient(timeoutCtx, cfg.PDAddrs, credential) - if err != nil { // pd 不可用 + if err != nil { + // pd is unavailable var message string if (status.PullerResolvedTs - status.CheckpointTs) > cfg.ReplicaConfig.SyncedStatus.CheckpointInterval*1000 { // 5s message = fmt.Sprintf("%s. Besides the data is not finish syncing", terror.Message(err)) @@ -966,15 +980,12 @@ func (h *OpenAPIV2) synced(c *gin.Context) { } defer pdClient.Close() + // get time from pd physicalNow, _, _ := pdClient.GetTS(ctx) - log.Info("time info", zap.Int64("physical", physicalNow), zap.Int64("checkpointTs", status.CheckpointTs), - zap.Int64("pullerResolvedTs", status.PullerResolvedTs), zap.Int64("LastSyncedTs", status.LastSyncedTs), - zap.Int64("SyncedCheckInterval", cfg.ReplicaConfig.SyncedStatus.SyncedCheckInterval), - zap.Int64("CheckpointInterval", cfg.ReplicaConfig.SyncedStatus.CheckpointInterval)) - if (physicalNow-status.LastSyncedTs > cfg.ReplicaConfig.SyncedStatus.SyncedCheckInterval*1000) && - (physicalNow-status.CheckpointTs < cfg.ReplicaConfig.SyncedStatus.CheckpointInterval*1000) { // 达到 synced 严格条件 + (physicalNow-status.CheckpointTs < cfg.ReplicaConfig.SyncedStatus.CheckpointInterval*1000) { + // reach strict synced condition c.JSON(http.StatusOK, SyncedStatus{ Synced: true, SinkCheckpointTs: transformerTime(status.CheckpointTs), @@ -984,9 +995,9 @@ func (h *OpenAPIV2) synced(c *gin.Context) { Info: "Data syncing is finished", }) } else if physicalNow-status.LastSyncedTs > cfg.ReplicaConfig.SyncedStatus.SyncedCheckInterval*1000 { - // lastSyncedTs 条件达到,checkpoint-ts 未达到 + // lastSyncedTs reach the synced condition, while checkpoint-ts doesn't var message string - if (status.PullerResolvedTs - status.CheckpointTs) < cfg.ReplicaConfig.SyncedStatus.CheckpointInterval*1000 { // 5s + if (status.PullerResolvedTs - status.CheckpointTs) < cfg.ReplicaConfig.SyncedStatus.CheckpointInterval*1000 { message = fmt.Sprintf("Please check whether pd is health and tikv region is all available. " + "If pd is not health or tikv region is not available, the data syncing is finished. " + " Otherwise the data syncing is not finished, please wait") @@ -1001,7 +1012,8 @@ func (h *OpenAPIV2) synced(c *gin.Context) { NowTs: transformerTime(physicalNow), Info: message, }) - } else { // lastSyncedTs 条件未达到 + } else { + // lastSyncedTs doesn't reach the synced condition c.JSON(http.StatusOK, SyncedStatus{ Synced: false, SinkCheckpointTs: transformerTime(status.CheckpointTs), diff --git a/cdc/api/v2/changefeed_test.go b/cdc/api/v2/changefeed_test.go index cedcc61fc44..54cd7a90d96 100644 --- a/cdc/api/v2/changefeed_test.go +++ b/cdc/api/v2/changefeed_test.go @@ -1021,7 +1021,7 @@ func TestChangefeedSynced(t *testing.T) { helpers.EXPECT().getPDClient(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil, cerror.ErrAPIGetPDClientFailed).Times(1) // case3: pd is offline,resolvedTs - checkpointTs > 15s - statusProvider.changeFeedSyncedStatusForAPI = &model.ChangeFeedSyncedStatusForAPI{ + statusProvider.changeFeedSyncedStatus = &model.ChangeFeedSyncedStatusForAPI{ CheckpointTs: 1701153217279, LastSyncedTs: 1701153217279, PullerResolvedTs: 1701153247279, @@ -1046,8 +1046,8 @@ func TestChangefeedSynced(t *testing.T) { { helpers.EXPECT().getPDClient(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil, cerror.ErrAPIGetPDClientFailed).Times(1) - // case4: pd is offline,resolvedTs - checkpointTs < 5s - statusProvider.changeFeedSyncedStatusForAPI = &model.ChangeFeedSyncedStatusForAPI{ + // case4: pd is offline,resolvedTs - checkpointTs < 15s + statusProvider.changeFeedSyncedStatus = &model.ChangeFeedSyncedStatusForAPI{ CheckpointTs: 1701153217279, LastSyncedTs: 1701153217279, PullerResolvedTs: 1701153217479, @@ -1077,8 +1077,8 @@ func TestChangefeedSynced(t *testing.T) { pdClient.logicTime = 1000 pdClient.timestamp = 1701153217279 { - // case5: pdTs - lastSyncedTs > 5min, pdTs - checkpointTs < 5s - statusProvider.changeFeedSyncedStatusForAPI = &model.ChangeFeedSyncedStatusForAPI{ + // case5: pdTs - lastSyncedTs > 5min, pdTs - checkpointTs < 15s + statusProvider.changeFeedSyncedStatus = &model.ChangeFeedSyncedStatusForAPI{ CheckpointTs: 1701153217209, LastSyncedTs: 1701152217279, PullerResolvedTs: 1701153217229, @@ -1102,7 +1102,7 @@ func TestChangefeedSynced(t *testing.T) { { // case6: pdTs - lastSyncedTs > 5min, pdTs - checkpointTs > 15s, resolvedTs - checkpointTs < 15s - statusProvider.changeFeedSyncedStatusForAPI = &model.ChangeFeedSyncedStatusForAPI{ + statusProvider.changeFeedSyncedStatus = &model.ChangeFeedSyncedStatusForAPI{ CheckpointTs: 1701153201279, LastSyncedTs: 1701152217279, PullerResolvedTs: 1701153201379, @@ -1127,7 +1127,7 @@ func TestChangefeedSynced(t *testing.T) { { // case7: pdTs - lastSyncedTs > 5min, pdTs - checkpointTs > 15s, resolvedTs - checkpointTs > 15s - statusProvider.changeFeedSyncedStatusForAPI = &model.ChangeFeedSyncedStatusForAPI{ + statusProvider.changeFeedSyncedStatus = &model.ChangeFeedSyncedStatusForAPI{ CheckpointTs: 1701153201279, LastSyncedTs: 1701152207279, PullerResolvedTs: 1701153218279, @@ -1150,7 +1150,7 @@ func TestChangefeedSynced(t *testing.T) { { // case8: pdTs - lastSyncedTs < 5min - statusProvider.changeFeedSyncedStatusForAPI = &model.ChangeFeedSyncedStatusForAPI{ + statusProvider.changeFeedSyncedStatus = &model.ChangeFeedSyncedStatusForAPI{ CheckpointTs: 1701153217279, LastSyncedTs: 1701153213279, PullerResolvedTs: 1701153217279, diff --git a/cdc/owner/changefeed.go b/cdc/owner/changefeed.go index 1fecd62c56a..983947b0b01 100755 --- a/cdc/owner/changefeed.go +++ b/cdc/owner/changefeed.go @@ -526,20 +526,6 @@ LOOP2: c.resolvedTs = checkpointTs } - // if c.lastSyncedTs == 0 { - // // Set LastSyncedTs with current pd time when do initialize. - - // // we don't save lastSyncedTs in etcd because we want to reduce the number of etcd write. - // // Based on the assumption that owner will not be replaced frequently, - // // and we only change owners when oom or some panic happens, - // // use pd time to initialize lastSyncedTs can work well enough. - // // Even if there are no more data send into ticdc after changing owner, - // // we just need to wait synced-check-time to reach synced = true. - // // We regard the situation never happens that owner is replaced frequently and then leading to - // // lastSyncedTs always increase even if there are no more data send into ticdc. - // c.lastSyncedTs = uint64(oracle.GetPhysical(c.upstream.PDClock.CurrentTime())) - // } - minTableBarrierTs := c.latestStatus.MinTableBarrierTs failpoint.Inject("NewChangefeedNoRetryError", func() { diff --git a/cdc/owner/changefeed_test.go b/cdc/owner/changefeed_test.go index 9ff5befa6a4..c3238560f42 100644 --- a/cdc/owner/changefeed_test.go +++ b/cdc/owner/changefeed_test.go @@ -171,7 +171,6 @@ func (m *mockScheduler) Tick( barrier *schedulepb.BarrierWithMinTs, ) (newCheckpointTs, newResolvedTs, newLastSyncedTs, newPullerResolvedTs model.Ts, err error) { m.currentTables = currentTables - // todo:这边到底应该 mock 一个什么呢 return barrier.MinTableBarrierTs, barrier.GlobalBarrierTs, scheduler.CheckpointCannotProceed, scheduler.CheckpointCannotProceed, nil } diff --git a/cdc/processor/sinkmanager/manager_test.go b/cdc/processor/sinkmanager/manager_test.go index 749fb68c248..249e0832299 100644 --- a/cdc/processor/sinkmanager/manager_test.go +++ b/cdc/processor/sinkmanager/manager_test.go @@ -27,7 +27,6 @@ import ( "github.com/pingcap/tiflow/pkg/config" "github.com/pingcap/tiflow/pkg/spanz" "github.com/stretchr/testify/require" - "go.uber.org/zap" ) func getChangefeedInfo() *model.ChangeFeedInfo { @@ -212,8 +211,6 @@ func TestGenerateTableSinkTaskWithBarrierTs(t *testing.T) { manager.schemaStorage.AdvanceResolvedTs(6) require.Eventually(t, func() bool { s := manager.GetTableStats(span) - log.Info("current value", zap.Uint64("checkpointTs", s.CheckpointTs), - zap.Uint64("lastSyncedTs", s.LastSyncedTs)) return s.CheckpointTs == 6 && s.LastSyncedTs == 4 }, 5*time.Second, 10*time.Millisecond) } diff --git a/cdc/sink/tablesink/table_sink_impl_test.go b/cdc/sink/tablesink/table_sink_impl_test.go index 647328d7747..34e334f334b 100644 --- a/cdc/sink/tablesink/table_sink_impl_test.go +++ b/cdc/sink/tablesink/table_sink_impl_test.go @@ -38,9 +38,6 @@ type mockEventSink struct { func (m *mockEventSink) WriteEvents(rows ...*dmlsink.TxnCallbackableEvent) error { m.events = append(m.events, rows...) - // for _, event := range rows { - // event.Callback() - // } return nil } From 317d58bf8489ced5caf011e39d23274f3b184beb Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Mon, 18 Dec 2023 12:12:02 +0800 Subject: [PATCH 28/35] update --- cdc/api/v2/changefeed.go | 52 +++++++++++--------- cdc/api/v2/changefeed_test.go | 46 +++++++++-------- cdc/api/v2/model.go | 12 ++--- cdc/model/changefeed.go | 10 ++-- cdc/owner/owner.go | 7 ++- tests/integration_tests/synced_status/run.sh | 10 ++-- 6 files changed, 70 insertions(+), 67 deletions(-) diff --git a/cdc/api/v2/changefeed.go b/cdc/api/v2/changefeed.go index 9606090f40f..ee17803b524 100644 --- a/cdc/api/v2/changefeed.go +++ b/cdc/api/v2/changefeed.go @@ -960,7 +960,8 @@ func (h *OpenAPIV2) synced(c *gin.Context) { if err != nil { // pd is unavailable var message string - if (status.PullerResolvedTs - status.CheckpointTs) > cfg.ReplicaConfig.SyncedStatus.CheckpointInterval*1000 { // 5s + if (oracle.ExtractPhysical(status.PullerResolvedTs) - oracle.ExtractPhysical(status.CheckpointTs)) > + cfg.ReplicaConfig.SyncedStatus.CheckpointInterval*1000 { message = fmt.Sprintf("%s. Besides the data is not finish syncing", terror.Message(err)) } else { message = fmt.Sprintf("%s. You can check the pd first, and if pd is available, means we don't finish sync data. "+ @@ -970,10 +971,10 @@ func (h *OpenAPIV2) synced(c *gin.Context) { } c.JSON(http.StatusOK, SyncedStatus{ Synced: false, - SinkCheckpointTs: transformerTime(status.CheckpointTs), - PullerResolvedTs: transformerTime(status.PullerResolvedTs), - LastSyncedTs: transformerTime(status.LastSyncedTs), - NowTs: transformerTime(0), + SinkCheckpointTs: model.JSONTime(oracle.GetTimeFromTS(status.CheckpointTs)), + PullerResolvedTs: model.JSONTime(oracle.GetTimeFromTS(status.PullerResolvedTs)), + LastSyncedTs: model.JSONTime(oracle.GetTimeFromTS(status.LastSyncedTs)), + NowTs: model.JSONTime(time.Unix(0, 0)), Info: message, }) return @@ -983,43 +984,46 @@ func (h *OpenAPIV2) synced(c *gin.Context) { // get time from pd physicalNow, _, _ := pdClient.GetTS(ctx) - if (physicalNow-status.LastSyncedTs > cfg.ReplicaConfig.SyncedStatus.SyncedCheckInterval*1000) && - (physicalNow-status.CheckpointTs < cfg.ReplicaConfig.SyncedStatus.CheckpointInterval*1000) { + if (physicalNow-oracle.ExtractPhysical(status.LastSyncedTs) > cfg.ReplicaConfig.SyncedStatus.SyncedCheckInterval*1000) && + (physicalNow-oracle.ExtractPhysical(status.CheckpointTs) < cfg.ReplicaConfig.SyncedStatus.CheckpointInterval*1000) { // reach strict synced condition c.JSON(http.StatusOK, SyncedStatus{ Synced: true, - SinkCheckpointTs: transformerTime(status.CheckpointTs), - PullerResolvedTs: transformerTime(status.PullerResolvedTs), - LastSyncedTs: transformerTime(status.LastSyncedTs), - NowTs: transformerTime(physicalNow), + SinkCheckpointTs: model.JSONTime(oracle.GetTimeFromTS(status.CheckpointTs)), + PullerResolvedTs: model.JSONTime(oracle.GetTimeFromTS(status.PullerResolvedTs)), + LastSyncedTs: model.JSONTime(oracle.GetTimeFromTS(status.LastSyncedTs)), + NowTs: model.JSONTime(time.Unix(physicalNow/1e3, 0)), Info: "Data syncing is finished", }) - } else if physicalNow-status.LastSyncedTs > cfg.ReplicaConfig.SyncedStatus.SyncedCheckInterval*1000 { + } else if physicalNow-oracle.ExtractPhysical(status.LastSyncedTs) > cfg.ReplicaConfig.SyncedStatus.SyncedCheckInterval*1000 { // lastSyncedTs reach the synced condition, while checkpoint-ts doesn't var message string - if (status.PullerResolvedTs - status.CheckpointTs) < cfg.ReplicaConfig.SyncedStatus.CheckpointInterval*1000 { - message = fmt.Sprintf("Please check whether pd is health and tikv region is all available. " + - "If pd is not health or tikv region is not available, the data syncing is finished. " + - " Otherwise the data syncing is not finished, please wait") + if (oracle.ExtractPhysical(status.PullerResolvedTs) - oracle.ExtractPhysical(status.CheckpointTs)) < + cfg.ReplicaConfig.SyncedStatus.CheckpointInterval*1000 { + message = fmt.Sprintf("Please check whether pd is healthy and tikv region is all available. "+ + "If pd is not healthy or tikv region is not available, the data syncing is finished. "+ + "Because in this case, the resolvedTs will not advance anymore, "+ + "thus we only need to care whether last_synced_ts is more than %v secs from the current time."+ + " Otherwise the data syncing is not finished, please wait", cfg.ReplicaConfig.SyncedStatus.SyncedCheckInterval) } else { message = "The data syncing is not finished, please wait" } c.JSON(http.StatusOK, SyncedStatus{ Synced: false, - SinkCheckpointTs: transformerTime(status.CheckpointTs), - PullerResolvedTs: transformerTime(status.PullerResolvedTs), - LastSyncedTs: transformerTime(status.LastSyncedTs), - NowTs: transformerTime(physicalNow), + SinkCheckpointTs: model.JSONTime(oracle.GetTimeFromTS(status.CheckpointTs)), + PullerResolvedTs: model.JSONTime(oracle.GetTimeFromTS(status.PullerResolvedTs)), + LastSyncedTs: model.JSONTime(oracle.GetTimeFromTS(status.LastSyncedTs)), + NowTs: model.JSONTime(time.Unix(physicalNow/1e3, 0)), Info: message, }) } else { // lastSyncedTs doesn't reach the synced condition c.JSON(http.StatusOK, SyncedStatus{ Synced: false, - SinkCheckpointTs: transformerTime(status.CheckpointTs), - PullerResolvedTs: transformerTime(status.PullerResolvedTs), - LastSyncedTs: transformerTime(status.LastSyncedTs), - NowTs: transformerTime(physicalNow), + SinkCheckpointTs: model.JSONTime(oracle.GetTimeFromTS(status.CheckpointTs)), + PullerResolvedTs: model.JSONTime(oracle.GetTimeFromTS(status.PullerResolvedTs)), + LastSyncedTs: model.JSONTime(oracle.GetTimeFromTS(status.LastSyncedTs)), + NowTs: model.JSONTime(time.Unix(physicalNow/1e3, 0)), Info: "The data syncing is not finished, please wait", }) } diff --git a/cdc/api/v2/changefeed_test.go b/cdc/api/v2/changefeed_test.go index 54cd7a90d96..6200cf8857a 100644 --- a/cdc/api/v2/changefeed_test.go +++ b/cdc/api/v2/changefeed_test.go @@ -1018,13 +1018,12 @@ func TestChangefeedSynced(t *testing.T) { statusProvider.err = nil statusProvider.changefeedInfo = cfInfo { - helpers.EXPECT().getPDClient(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil, cerror.ErrAPIGetPDClientFailed).Times(1) // case3: pd is offline,resolvedTs - checkpointTs > 15s statusProvider.changeFeedSyncedStatus = &model.ChangeFeedSyncedStatusForAPI{ - CheckpointTs: 1701153217279, - LastSyncedTs: 1701153217279, - PullerResolvedTs: 1701153247279, + CheckpointTs: 1701153217279 << 18, + LastSyncedTs: 1701153217279 << 18, + PullerResolvedTs: 1701153247279 << 18, } w := httptest.NewRecorder() req, _ := http.NewRequestWithContext( @@ -1039,7 +1038,6 @@ func TestChangefeedSynced(t *testing.T) { err := json.NewDecoder(w.Body).Decode(&resp) require.Nil(t, err) require.Equal(t, false, resp.Synced) - require.Equal(t, "1970-01-01 08:00:00", resp.NowTs) require.Equal(t, "[CDC:ErrAPIGetPDClientFailed]failed to get PDClient to connect PD, "+ "please recheck. Besides the data is not finish syncing", resp.Info) } @@ -1048,9 +1046,9 @@ func TestChangefeedSynced(t *testing.T) { helpers.EXPECT().getPDClient(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil, cerror.ErrAPIGetPDClientFailed).Times(1) // case4: pd is offline,resolvedTs - checkpointTs < 15s statusProvider.changeFeedSyncedStatus = &model.ChangeFeedSyncedStatusForAPI{ - CheckpointTs: 1701153217279, - LastSyncedTs: 1701153217279, - PullerResolvedTs: 1701153217479, + CheckpointTs: 1701153217279 << 18, + LastSyncedTs: 1701153217279 << 18, + PullerResolvedTs: 1701153217479 << 18, } w := httptest.NewRecorder() req, _ := http.NewRequestWithContext( @@ -1065,7 +1063,6 @@ func TestChangefeedSynced(t *testing.T) { err := json.NewDecoder(w.Body).Decode(&resp) require.Nil(t, err) require.Equal(t, false, resp.Synced) - require.Equal(t, "1970-01-01 08:00:00", resp.NowTs) require.Equal(t, "[CDC:ErrAPIGetPDClientFailed]failed to get PDClient to connect PD, please recheck. "+ "You can check the pd first, and if pd is available, means we don't finish sync data. "+ "If pd is not available, please check the whether we satisfy the condition that "+ @@ -1079,9 +1076,9 @@ func TestChangefeedSynced(t *testing.T) { { // case5: pdTs - lastSyncedTs > 5min, pdTs - checkpointTs < 15s statusProvider.changeFeedSyncedStatus = &model.ChangeFeedSyncedStatusForAPI{ - CheckpointTs: 1701153217209, - LastSyncedTs: 1701152217279, - PullerResolvedTs: 1701153217229, + CheckpointTs: 1701153217209 << 18, + LastSyncedTs: 1701152217279 << 18, + PullerResolvedTs: 1701153217229 << 18, } w := httptest.NewRecorder() req, _ := http.NewRequestWithContext( @@ -1096,16 +1093,15 @@ func TestChangefeedSynced(t *testing.T) { err := json.NewDecoder(w.Body).Decode(&resp) require.Nil(t, err) require.Equal(t, true, resp.Synced) - require.Equal(t, "2023-11-28 14:33:37", resp.NowTs) require.Equal(t, "Data syncing is finished", resp.Info) } { // case6: pdTs - lastSyncedTs > 5min, pdTs - checkpointTs > 15s, resolvedTs - checkpointTs < 15s statusProvider.changeFeedSyncedStatus = &model.ChangeFeedSyncedStatusForAPI{ - CheckpointTs: 1701153201279, - LastSyncedTs: 1701152217279, - PullerResolvedTs: 1701153201379, + CheckpointTs: 1701153201279 << 18, + LastSyncedTs: 1701152217279 << 18, + PullerResolvedTs: 1701153201379 << 18, } w := httptest.NewRecorder() req, _ := http.NewRequestWithContext( @@ -1120,17 +1116,19 @@ func TestChangefeedSynced(t *testing.T) { err := json.NewDecoder(w.Body).Decode(&resp) require.Nil(t, err) require.Equal(t, false, resp.Synced) - require.Equal(t, "Please check whether pd is health and tikv region is all available. "+ - "If pd is not health or tikv region is not available, the data syncing is finished. "+ + require.Equal(t, "Please check whether pd is healthy and tikv region is all available. "+ + "If pd is not healthy or tikv region is not available, the data syncing is finished. "+ + "Because in this case, the resolvedTs will not advance anymore, "+ + "thus we only need to care whether last_synced_ts is more than 300 secs from the current time."+ " Otherwise the data syncing is not finished, please wait", resp.Info) } { // case7: pdTs - lastSyncedTs > 5min, pdTs - checkpointTs > 15s, resolvedTs - checkpointTs > 15s statusProvider.changeFeedSyncedStatus = &model.ChangeFeedSyncedStatusForAPI{ - CheckpointTs: 1701153201279, - LastSyncedTs: 1701152207279, - PullerResolvedTs: 1701153218279, + CheckpointTs: 1701153201279 << 18, + LastSyncedTs: 1701152207279 << 18, + PullerResolvedTs: 1701153218279 << 18, } w := httptest.NewRecorder() req, _ := http.NewRequestWithContext( @@ -1151,9 +1149,9 @@ func TestChangefeedSynced(t *testing.T) { { // case8: pdTs - lastSyncedTs < 5min statusProvider.changeFeedSyncedStatus = &model.ChangeFeedSyncedStatusForAPI{ - CheckpointTs: 1701153217279, - LastSyncedTs: 1701153213279, - PullerResolvedTs: 1701153217279, + CheckpointTs: 1701153217279 << 18, + LastSyncedTs: 1701153213279 << 18, + PullerResolvedTs: 1701153217279 << 18, } w := httptest.NewRecorder() req, _ := http.NewRequestWithContext( diff --git a/cdc/api/v2/model.go b/cdc/api/v2/model.go index 1ea95057138..be9f47648a6 100644 --- a/cdc/api/v2/model.go +++ b/cdc/api/v2/model.go @@ -1048,12 +1048,12 @@ type ChangeFeedInfo struct { // SyncedStatus describes the detail of a changefeed's synced status type SyncedStatus struct { - Synced bool `json:"synced"` - SinkCheckpointTs string `json:"sink_checkpoint_ts"` - PullerResolvedTs string `json:"puller_resolved_ts"` - LastSyncedTs string `json:"last_synced_ts"` - NowTs string `json:"now_ts"` - Info string `json:"info"` + Synced bool `json:"synced"` + SinkCheckpointTs model.JSONTime `json:"sink_checkpoint_ts"` + PullerResolvedTs model.JSONTime `json:"puller_resolved_ts"` + LastSyncedTs model.JSONTime `json:"last_synced_ts"` + NowTs model.JSONTime `json:"now_ts"` + Info string `json:"info"` } // RunningError represents some running error from cdc components, diff --git a/cdc/model/changefeed.go b/cdc/model/changefeed.go index 6abaf00e249..9b8aa7d4744 100644 --- a/cdc/model/changefeed.go +++ b/cdc/model/changefeed.go @@ -653,9 +653,9 @@ type ChangeFeedStatusForAPI struct { // ChangeFeedSyncedStatusForAPI uses to transfer the synced status of changefeed for API. type ChangeFeedSyncedStatusForAPI struct { - CheckpointTs int64 `json:"checkpoint-ts"` - LastSyncedTs int64 `json:"last-sync-time"` - PullerResolvedTs int64 `json:"puller-resolved-ts"` - SyncedCheckInterval int64 `json:"synced-check-interval"` - CheckpointInterval int64 `json:"checkpoint-interval"` + CheckpointTs uint64 `json:"checkpoint-ts"` + LastSyncedTs uint64 `json:"last-sync-time"` + PullerResolvedTs uint64 `json:"puller-resolved-ts"` + SyncedCheckInterval int64 `json:"synced-check-interval"` + CheckpointInterval int64 `json:"checkpoint-interval"` } diff --git a/cdc/owner/owner.go b/cdc/owner/owner.go index 9fb81b91dae..0950eb62d2f 100644 --- a/cdc/owner/owner.go +++ b/cdc/owner/owner.go @@ -32,7 +32,6 @@ import ( "github.com/pingcap/tiflow/pkg/orchestrator" "github.com/pingcap/tiflow/pkg/upstream" "github.com/pingcap/tiflow/pkg/version" - "github.com/tikv/client-go/v2/oracle" "go.uber.org/zap" "golang.org/x/time/rate" ) @@ -605,9 +604,9 @@ func (o *ownerImpl) handleQueries(query *Query) error { return nil } ret := &model.ChangeFeedSyncedStatusForAPI{} - ret.LastSyncedTs = oracle.ExtractPhysical(cfReactor.lastSyncedTs) - ret.CheckpointTs = oracle.ExtractPhysical(cfReactor.latestStatus.CheckpointTs) - ret.PullerResolvedTs = oracle.ExtractPhysical(cfReactor.pullerResolvedTs) + ret.LastSyncedTs = cfReactor.lastSyncedTs + ret.CheckpointTs = cfReactor.latestStatus.CheckpointTs + ret.PullerResolvedTs = cfReactor.pullerResolvedTs if cfReactor.latestInfo == nil { ret.CheckpointInterval = 0 diff --git a/tests/integration_tests/synced_status/run.sh b/tests/integration_tests/synced_status/run.sh index 387e7d03f0a..b436667d85e 100644 --- a/tests/integration_tests/synced_status/run.sh +++ b/tests/integration_tests/synced_status/run.sh @@ -176,8 +176,8 @@ function run_case_with_unavailable_tikv() { exit 1 fi info=$(echo $synced_status | jq -r '.info') - target_message="Please check whether pd is health and tikv region is all available. \ -If pd is not health or tikv region is not available, the data syncing is finished. \ + target_message="Please check whether pd is healthy and tikv region is all available. \ +If pd is not healthy or tikv region is not available, the data syncing is finished. \ Otherwise the data syncing is not finished, please wait" if [ "$info" != "$target_message" ]; then @@ -272,8 +272,10 @@ function run_case_with_failpoint() { exit 1 fi info=$(echo $synced_status | jq -r '.info') - target_message="Please check whether pd is health and tikv region is all available. \ -If pd is not health or tikv region is not available, the data syncing is finished. \ + target_message="Please check whether pd is healthy and tikv region is all available. \ +If pd is not healthy or tikv region is not available, the data syncing is finished. \ +Because in this case, the resolvedTs will not advance anymore, \ +thus we only need to care whether last_synced_ts is more than 120 secs from the current time. \ Otherwise the data syncing is not finished, please wait" if [ "$info" != "$target_message" ]; then echo "synced status info is not correct" From 50b0a15fc053017e2f7614f6f68747a501b84ba2 Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Tue, 19 Dec 2023 18:32:34 +0800 Subject: [PATCH 29/35] fix comments --- cdc/api/v2/changefeed.go | 46 ++++---- cdc/api/v2/changefeed_test.go | 5 +- cdc/owner/changefeed.go | 40 +++---- cdc/owner/changefeed_test.go | 9 +- cdc/owner/status_provider.go | 1 + cdc/scheduler/internal/scheduler.go | 2 +- cdc/scheduler/internal/v3/coordinator.go | 49 ++++++--- cdc/scheduler/internal/v3/coordinator_test.go | 32 +++--- .../v3/replication/replication_manager.go | 93 ++++++++-------- .../replication/replication_manager_test.go | 100 +++++++++--------- cdc/scheduler/schedulepb/watermark.go | 24 +++++ tests/integration_tests/synced_status/run.sh | 8 +- 12 files changed, 242 insertions(+), 167 deletions(-) create mode 100644 cdc/scheduler/schedulepb/watermark.go diff --git a/cdc/api/v2/changefeed.go b/cdc/api/v2/changefeed.go index ee17803b524..9a53227d21b 100644 --- a/cdc/api/v2/changefeed.go +++ b/cdc/api/v2/changefeed.go @@ -49,6 +49,8 @@ const ( apiOpVarChangefeedID = "changefeed_id" // apiOpVarNamespace is the key of changefeed namespace in HTTP API apiOpVarNamespace = "namespace" + // timeout for pd client + timeout = 30 * time.Second ) // createChangefeed handles create changefeed request, @@ -925,10 +927,7 @@ func (h *OpenAPIV2) synced(c *gin.Context) { return } - status, err := h.capture.StatusProvider().GetChangeFeedSyncedStatus( - ctx, - changefeedID, - ) + status, err := h.capture.StatusProvider().GetChangeFeedSyncedStatus(ctx, changefeedID) if err != nil { _ = c.Error(err) return @@ -942,7 +941,7 @@ func (h *OpenAPIV2) synced(c *gin.Context) { cfg.ReplicaConfig.SyncedStatus.SyncedCheckInterval = status.SyncedCheckInterval } - // get pd client + // try to get pd client to get pd time, and determine synced status based on the pd time if len(cfg.PDAddrs) == 0 { up, err := getCaptureDefaultUpstream(h.capture) if err != nil { @@ -953,20 +952,23 @@ func (h *OpenAPIV2) synced(c *gin.Context) { } credential := cfg.PDConfig.toCredential() - timeoutCtx, cancel := context.WithTimeout(ctx, 30*time.Second) + timeoutCtx, cancel := context.WithTimeout(ctx, timeout) defer cancel() pdClient, err := h.helpers.getPDClient(timeoutCtx, cfg.PDAddrs, credential) if err != nil { - // pd is unavailable + // case 1. we can't get pd client, pd may be unavailable. + // if pullerResolvedTs - checkpointTs > checkpointInterval, data is not synced + // otherwise, if pd is unavailable, we decide data whether is synced based on + // the time difference between current time and lastSyncedTs. var message string if (oracle.ExtractPhysical(status.PullerResolvedTs) - oracle.ExtractPhysical(status.CheckpointTs)) > cfg.ReplicaConfig.SyncedStatus.CheckpointInterval*1000 { message = fmt.Sprintf("%s. Besides the data is not finish syncing", terror.Message(err)) } else { - message = fmt.Sprintf("%s. You can check the pd first, and if pd is available, means we don't finish sync data. "+ - "If pd is not available, please check the whether we satisfy the condition that "+ - "the time difference from lastSyncedTs to the current time from the time zone of pd is greater than %v secs. "+ + message = fmt.Sprintf("%s. You should check the pd status first. If pd status is normal, means we don't finish sync data. "+ + "If pd is offline, please check the whether we satisfy the condition that "+ + "the time difference from lastSyncedTs to the current time in the time zone of pd is greater than %v secs. "+ "If it's satisfied, means the data syncing is totally finished", err, cfg.ReplicaConfig.SyncedStatus.SyncedCheckInterval) } c.JSON(http.StatusOK, SyncedStatus{ @@ -980,13 +982,14 @@ func (h *OpenAPIV2) synced(c *gin.Context) { return } defer pdClient.Close() - // get time from pd physicalNow, _, _ := pdClient.GetTS(ctx) + // We can normally get pd time. Thus we determine synced status based on physicalNow, lastSyncedTs, checkpointTs and pullerResolvedTs if (physicalNow-oracle.ExtractPhysical(status.LastSyncedTs) > cfg.ReplicaConfig.SyncedStatus.SyncedCheckInterval*1000) && (physicalNow-oracle.ExtractPhysical(status.CheckpointTs) < cfg.ReplicaConfig.SyncedStatus.CheckpointInterval*1000) { - // reach strict synced condition + // case 2: If physcialNow - lastSyncedTs > SyncedCheckInterval && physcialNow - CheckpointTs < CheckpointInterval + // --> reach strict synced status c.JSON(http.StatusOK, SyncedStatus{ Synced: true, SinkCheckpointTs: model.JSONTime(oracle.GetTimeFromTS(status.CheckpointTs)), @@ -996,15 +999,20 @@ func (h *OpenAPIV2) synced(c *gin.Context) { Info: "Data syncing is finished", }) } else if physicalNow-oracle.ExtractPhysical(status.LastSyncedTs) > cfg.ReplicaConfig.SyncedStatus.SyncedCheckInterval*1000 { - // lastSyncedTs reach the synced condition, while checkpoint-ts doesn't + // case 3: If physcialNow - lastSyncedTs > SyncedCheckInterval && physcialNow - CheckpointTs > CheckpointInterval + // we should consider the situation that pd or tikv region is not healthy to block the advancing resolveTs. + // if pullerResolvedTs - checkpointTs > CheckpointInterval--> data is not synced + // otherwise, if pd & tikv is healthy --> data is not synced + // if not healthy --> data is synced var message string if (oracle.ExtractPhysical(status.PullerResolvedTs) - oracle.ExtractPhysical(status.CheckpointTs)) < cfg.ReplicaConfig.SyncedStatus.CheckpointInterval*1000 { - message = fmt.Sprintf("Please check whether pd is healthy and tikv region is all available. "+ - "If pd is not healthy or tikv region is not available, the data syncing is finished. "+ - "Because in this case, the resolvedTs will not advance anymore, "+ - "thus we only need to care whether last_synced_ts is more than %v secs from the current time."+ - " Otherwise the data syncing is not finished, please wait", cfg.ReplicaConfig.SyncedStatus.SyncedCheckInterval) + message = fmt.Sprintf("Please check whether pd is healthy and tikv region is all available. " + + "If pd is not healthy or tikv region is not available, the data syncing is finished. " + + "When pd is offline means that pd is not healthy. For tikv region, you can check the grafana info " + + "in 'TiKV-Details-Resolved-Ts-Max Leader Resolved TS gap'. If the gap is a large value, such as a few minutes, " + + "it means some regions in tikv are unavailable. " + + " Otherwise the data syncing is not finished, please wait") } else { message = "The data syncing is not finished, please wait" } @@ -1017,7 +1025,7 @@ func (h *OpenAPIV2) synced(c *gin.Context) { Info: message, }) } else { - // lastSyncedTs doesn't reach the synced condition + // case 4: If physcialNow - lastSyncedTs < SyncedCheckInterval --> data is not synced c.JSON(http.StatusOK, SyncedStatus{ Synced: false, SinkCheckpointTs: model.JSONTime(oracle.GetTimeFromTS(status.CheckpointTs)), diff --git a/cdc/api/v2/changefeed_test.go b/cdc/api/v2/changefeed_test.go index 6200cf8857a..9dec0790046 100644 --- a/cdc/api/v2/changefeed_test.go +++ b/cdc/api/v2/changefeed_test.go @@ -1118,8 +1118,9 @@ func TestChangefeedSynced(t *testing.T) { require.Equal(t, false, resp.Synced) require.Equal(t, "Please check whether pd is healthy and tikv region is all available. "+ "If pd is not healthy or tikv region is not available, the data syncing is finished. "+ - "Because in this case, the resolvedTs will not advance anymore, "+ - "thus we only need to care whether last_synced_ts is more than 300 secs from the current time."+ + "When pd is offline means that pd is not healthy. For tikv region, you can check the grafana info "+ + "in 'TiKV-Details-Resolved-Ts-Max Leader Resolved TS gap'. If the gap is a large value, such as a few minutes, "+ + "it means some regions in tikv are unavailable. "+ " Otherwise the data syncing is not finished, please wait", resp.Info) } diff --git a/cdc/owner/changefeed.go b/cdc/owner/changefeed.go index 983947b0b01..016994041d5 100755 --- a/cdc/owner/changefeed.go +++ b/cdc/owner/changefeed.go @@ -416,27 +416,27 @@ func (c *changefeed) tick(ctx cdcContext.Context, return 0, 0, nil } - newCheckpointTs, newResolvedTs, newLastSyncedTs, newPullerResolvedTs, err := c.scheduler.Tick( + watermark, err := c.scheduler.Tick( ctx, preCheckpointTs, allPhysicalTables, captures, barrier) if err != nil { return 0, 0, errors.Trace(err) } - if c.lastSyncedTs < newLastSyncedTs { - c.lastSyncedTs = newLastSyncedTs - } else if c.lastSyncedTs > newLastSyncedTs { + if c.lastSyncedTs < watermark.LastSyncedTs { + c.lastSyncedTs = watermark.LastSyncedTs + } else if c.lastSyncedTs > watermark.LastSyncedTs { log.Warn("LastSyncedTs should not be greater than newLastSyncedTs", zap.Uint64("c.LastSyncedTs", c.lastSyncedTs), - zap.Uint64("newLastSyncedTs", newLastSyncedTs)) + zap.Uint64("newLastSyncedTs", watermark.LastSyncedTs)) } - if newPullerResolvedTs != scheduler.CheckpointCannotProceed && newPullerResolvedTs != math.MaxUint64 { - if newPullerResolvedTs > c.pullerResolvedTs { - c.pullerResolvedTs = newPullerResolvedTs - } else if newPullerResolvedTs < c.pullerResolvedTs { + if watermark.PullerResolvedTs != scheduler.CheckpointCannotProceed && watermark.PullerResolvedTs != math.MaxUint64 { + if watermark.PullerResolvedTs > c.pullerResolvedTs { + c.pullerResolvedTs = watermark.PullerResolvedTs + } else if watermark.PullerResolvedTs < c.pullerResolvedTs { log.Warn("the newPullerResolvedTs should not be smaller than c.pullerResolvedTs", zap.Uint64("c.pullerResolvedTs", c.pullerResolvedTs), - zap.Uint64("newPullerResolvedTs", newPullerResolvedTs)) + zap.Uint64("newPullerResolvedTs", watermark.PullerResolvedTs)) } } @@ -445,7 +445,7 @@ func (c *changefeed) tick(ctx cdcContext.Context, // CheckpointCannotProceed implies that not all tables are being replicated normally, // so in that case there is no need to advance the global watermarks. - if newCheckpointTs == scheduler.CheckpointCannotProceed { + if watermark.CheckpointTs == scheduler.CheckpointCannotProceed { if c.latestStatus != nil { // We should keep the metrics updated even if the scheduler cannot // advance the watermarks for now. @@ -456,13 +456,13 @@ func (c *changefeed) tick(ctx cdcContext.Context, log.Debug("owner prepares to update status", zap.Uint64("prevResolvedTs", c.resolvedTs), - zap.Uint64("newResolvedTs", newResolvedTs), - zap.Uint64("newCheckpointTs", newCheckpointTs), + zap.Uint64("newResolvedTs", watermark.ResolvedTs), + zap.Uint64("newCheckpointTs", watermark.CheckpointTs), zap.String("namespace", c.id.Namespace), zap.String("changefeed", c.id.ID)) // resolvedTs should never regress. - if newResolvedTs > c.resolvedTs { - c.resolvedTs = newResolvedTs + if watermark.ResolvedTs > c.resolvedTs { + c.resolvedTs = watermark.ResolvedTs } // MinTableBarrierTs should never regress @@ -476,19 +476,19 @@ func (c *changefeed) tick(ctx cdcContext.Context, zap.String("namespace", c.id.Namespace), zap.String("changefeed", c.id.ID), zap.Uint64("keepCheckpoint", c.latestStatus.CheckpointTs), - zap.Uint64("skipCheckpoint", newCheckpointTs)) - newCheckpointTs = c.latestStatus.CheckpointTs + zap.Uint64("skipCheckpoint", watermark.CheckpointTs)) + watermark.CheckpointTs = c.latestStatus.CheckpointTs } }) failpoint.Inject("ChangefeedOwnerNotUpdateCheckpoint", func() { - newCheckpointTs = c.latestStatus.CheckpointTs + watermark.CheckpointTs = c.latestStatus.CheckpointTs }) - c.updateMetrics(currentTs, newCheckpointTs, c.resolvedTs) + c.updateMetrics(currentTs, watermark.CheckpointTs, c.resolvedTs) c.tickDownstreamObserver(ctx) - return newCheckpointTs, barrier.MinTableBarrierTs, nil + return watermark.CheckpointTs, barrier.MinTableBarrierTs, nil } func (c *changefeed) initialize(ctx cdcContext.Context) (err error) { diff --git a/cdc/owner/changefeed_test.go b/cdc/owner/changefeed_test.go index 755687acffc..492732711c9 100644 --- a/cdc/owner/changefeed_test.go +++ b/cdc/owner/changefeed_test.go @@ -169,9 +169,14 @@ func (m *mockScheduler) Tick( currentTables []model.TableID, captures map[model.CaptureID]*model.CaptureInfo, barrier *schedulepb.BarrierWithMinTs, -) (newCheckpointTs, newResolvedTs, newLastSyncedTs, newPullerResolvedTs model.Ts, err error) { +) (watermark schedulepb.Watermark, err error) { m.currentTables = currentTables - return barrier.MinTableBarrierTs, barrier.GlobalBarrierTs, scheduler.CheckpointCannotProceed, scheduler.CheckpointCannotProceed, nil + return schedulepb.Watermark{ + CheckpointTs: barrier.MinTableBarrierTs, + ResolvedTs: barrier.GlobalBarrierTs, + LastSyncedTs: scheduler.CheckpointCannotProceed, + PullerResolvedTs: scheduler.CheckpointCannotProceed, + }, nil } // MoveTable is used to trigger manual table moves. diff --git a/cdc/owner/status_provider.go b/cdc/owner/status_provider.go index 2cea1e2cd81..373654552be 100644 --- a/cdc/owner/status_provider.go +++ b/cdc/owner/status_provider.go @@ -27,6 +27,7 @@ type StatusProvider interface { // GetChangeFeedStatus returns a changefeeds' runtime status. GetChangeFeedStatus(ctx context.Context, changefeedID model.ChangeFeedID) (*model.ChangeFeedStatusForAPI, error) + // GetChangeFeedSyncedStatus returns a changefeeds' synced status. GetChangeFeedSyncedStatus(ctx context.Context, changefeedID model.ChangeFeedID) (*model.ChangeFeedSyncedStatusForAPI, error) // GetChangeFeedInfo returns a changefeeds' info. diff --git a/cdc/scheduler/internal/scheduler.go b/cdc/scheduler/internal/scheduler.go index 86e72570d5d..28e241b9dd8 100644 --- a/cdc/scheduler/internal/scheduler.go +++ b/cdc/scheduler/internal/scheduler.go @@ -47,7 +47,7 @@ type Scheduler interface { // ddl jobs that need to be replicated. The Scheduler will // broadcast the barrierTs to all captures through the Heartbeat. barrier *schedulepb.BarrierWithMinTs, - ) (newCheckpointTs, newResolvedTs, newLastSyncedTs, newPullerResolvedTs model.Ts, err error) + ) (watermark schedulepb.Watermark, err error) // MoveTable requests that a table be moved to target. // It is thread-safe. diff --git a/cdc/scheduler/internal/v3/coordinator.go b/cdc/scheduler/internal/v3/coordinator.go index 211f293c831..aaaa77300a6 100644 --- a/cdc/scheduler/internal/v3/coordinator.go +++ b/cdc/scheduler/internal/v3/coordinator.go @@ -126,7 +126,7 @@ func (c *coordinator) Tick( // All captures that are alive according to the latest Etcd states. aliveCaptures map[model.CaptureID]*model.CaptureInfo, barrier *schedulepb.BarrierWithMinTs, -) (newCheckpointTs, newResolvedTs, newLastSyncedTs, newPullerResolvedTs model.Ts, err error) { +) (watermark schedulepb.Watermark, err error) { startTime := time.Now() defer func() { costTime := time.Since(startTime) @@ -272,7 +272,7 @@ func (c *coordinator) poll( currentTables []model.TableID, aliveCaptures map[model.CaptureID]*model.CaptureInfo, barrier *schedulepb.BarrierWithMinTs, -) (newCheckpointTs, newResolvedTs, newLastSyncedTs, newPullerResolvedTs model.Ts, err error) { +) (watermark schedulepb.Watermark, err error) { c.maybeCollectMetrics() if c.compat.UpdateCaptureInfo(aliveCaptures) { spanReplicationEnabled := c.compat.CheckSpanReplicationEnabled() @@ -283,7 +283,12 @@ func (c *coordinator) poll( recvMsgs, err := c.recvMsgs(ctx) if err != nil { - return checkpointCannotProceed, checkpointCannotProceed, checkpointCannotProceed, checkpointCannotProceed, errors.Trace(err) + return schedulepb.Watermark{ + CheckpointTs: checkpointCannotProceed, + ResolvedTs: checkpointCannotProceed, + LastSyncedTs: checkpointCannotProceed, + PullerResolvedTs: checkpointCannotProceed, + }, errors.Trace(err) } var msgBuf []*schedulepb.Message @@ -295,7 +300,12 @@ func (c *coordinator) poll( // Handle received messages to advance replication set. msgs, err = c.replicationM.HandleMessage(recvMsgs) if err != nil { - return checkpointCannotProceed, checkpointCannotProceed, checkpointCannotProceed, checkpointCannotProceed, errors.Trace(err) + return schedulepb.Watermark{ + CheckpointTs: checkpointCannotProceed, + ResolvedTs: checkpointCannotProceed, + LastSyncedTs: checkpointCannotProceed, + PullerResolvedTs: checkpointCannotProceed, + }, errors.Trace(err) } msgBuf = append(msgBuf, msgs...) @@ -309,14 +319,13 @@ func (c *coordinator) poll( if !c.captureM.CheckAllCaptureInitialized() { // Skip generating schedule tasks for replication manager, // as not all capture are initialized. - newCheckpointTs, newResolvedTs, newLastSyncedTs, - newPullerResolvedTs = c.replicationM.AdvanceCheckpoint(&c.tableRanges, pdTime, barrier, c.redoMetaManager) + watermark = c.replicationM.AdvanceCheckpoint(&c.tableRanges, pdTime, barrier, c.redoMetaManager) // tick capture manager after checkpoint calculation to take account resolvedTs in barrier // when redo is enabled msgs = c.captureM.Tick(c.replicationM.ReplicationSets(), c.schedulerM.DrainingTarget(), barrier.Barrier) msgBuf = append(msgBuf, msgs...) - return newCheckpointTs, newResolvedTs, newLastSyncedTs, newPullerResolvedTs, c.sendMsgs(ctx, msgBuf) + return watermark, c.sendMsgs(ctx, msgBuf) } // Handle capture membership changes. @@ -324,7 +333,12 @@ func (c *coordinator) poll( msgs, err = c.replicationM.HandleCaptureChanges( changes.Init, changes.Removed, checkpointTs) if err != nil { - return checkpointCannotProceed, checkpointCannotProceed, checkpointCannotProceed, checkpointCannotProceed, errors.Trace(err) + return schedulepb.Watermark{ + CheckpointTs: checkpointCannotProceed, + ResolvedTs: checkpointCannotProceed, + LastSyncedTs: checkpointCannotProceed, + PullerResolvedTs: checkpointCannotProceed, + }, errors.Trace(err) } msgBuf = append(msgBuf, msgs...) } @@ -340,13 +354,17 @@ func (c *coordinator) poll( // Handle generated schedule tasks. msgs, err = c.replicationM.HandleTasks(allTasks) if err != nil { - return checkpointCannotProceed, checkpointCannotProceed, checkpointCannotProceed, checkpointCannotProceed, errors.Trace(err) + return schedulepb.Watermark{ + CheckpointTs: checkpointCannotProceed, + ResolvedTs: checkpointCannotProceed, + LastSyncedTs: checkpointCannotProceed, + PullerResolvedTs: checkpointCannotProceed, + }, errors.Trace(err) } msgBuf = append(msgBuf, msgs...) // Checkpoint calculation - newCheckpointTs, newResolvedTs, newLastSyncedTs, - newPullerResolvedTs = c.replicationM.AdvanceCheckpoint(&c.tableRanges, pdTime, barrier, c.redoMetaManager) + watermark = c.replicationM.AdvanceCheckpoint(&c.tableRanges, pdTime, barrier, c.redoMetaManager) // tick capture manager after checkpoint calculation to take account resolvedTs in barrier // when redo is enabled @@ -357,10 +375,15 @@ func (c *coordinator) poll( // Send new messages. err = c.sendMsgs(ctx, msgBuf) if err != nil { - return checkpointCannotProceed, checkpointCannotProceed, checkpointCannotProceed, checkpointCannotProceed, errors.Trace(err) + return schedulepb.Watermark{ + CheckpointTs: checkpointCannotProceed, + ResolvedTs: checkpointCannotProceed, + LastSyncedTs: checkpointCannotProceed, + PullerResolvedTs: checkpointCannotProceed, + }, errors.Trace(err) } - return newCheckpointTs, newResolvedTs, newLastSyncedTs, newPullerResolvedTs, nil + return watermark, nil } func (c *coordinator) recvMsgs(ctx context.Context) ([]*schedulepb.Message, error) { diff --git a/cdc/scheduler/internal/v3/coordinator_test.go b/cdc/scheduler/internal/v3/coordinator_test.go index 3a14bfa83ed..8caacc30ae3 100644 --- a/cdc/scheduler/internal/v3/coordinator_test.go +++ b/cdc/scheduler/internal/v3/coordinator_test.go @@ -251,7 +251,7 @@ func TestCoordinatorHeartbeat(t *testing.T) { ctx := context.Background() currentTables := []model.TableID{1, 2, 3} aliveCaptures := map[model.CaptureID]*model.CaptureInfo{"a": {}, "b": {}} - _, _, _, _, err := coord.poll(ctx, 0, currentTables, aliveCaptures, schedulepb.NewBarrierWithMinTs(0)) + _, err := coord.poll(ctx, 0, currentTables, aliveCaptures, schedulepb.NewBarrierWithMinTs(0)) require.Nil(t, err) msgs := trans.SendBuffer require.Len(t, msgs, 2) @@ -283,7 +283,7 @@ func TestCoordinatorHeartbeat(t *testing.T) { }, }) trans.SendBuffer = []*schedulepb.Message{} - _, _, _, _, err = coord.poll(ctx, 0, currentTables, aliveCaptures, schedulepb.NewBarrierWithMinTs(0)) + _, err = coord.poll(ctx, 0, currentTables, aliveCaptures, schedulepb.NewBarrierWithMinTs(0)) require.Nil(t, err) require.True(t, coord.captureM.CheckAllCaptureInitialized()) msgs = trans.SendBuffer @@ -324,7 +324,7 @@ func TestCoordinatorAddCapture(t *testing.T) { ctx := context.Background() currentTables := []model.TableID{1, 2, 3} aliveCaptures := map[model.CaptureID]*model.CaptureInfo{"a": {}, "b": {}} - _, _, _, _, err = coord.poll(ctx, 0, currentTables, aliveCaptures, schedulepb.NewBarrierWithMinTs(0)) + _, err = coord.poll(ctx, 0, currentTables, aliveCaptures, schedulepb.NewBarrierWithMinTs(0)) require.Nil(t, err) msgs = trans.SendBuffer require.Len(t, msgs, 1) @@ -340,7 +340,7 @@ func TestCoordinatorAddCapture(t *testing.T) { HeartbeatResponse: &schedulepb.HeartbeatResponse{}, }) trans.SendBuffer = []*schedulepb.Message{} - _, _, _, _, err = coord.poll(ctx, 0, currentTables, aliveCaptures, schedulepb.NewBarrierWithMinTs(0)) + _, err = coord.poll(ctx, 0, currentTables, aliveCaptures, schedulepb.NewBarrierWithMinTs(0)) require.Nil(t, err) msgs = trans.SendBuffer require.Len(t, msgs, 1) @@ -381,7 +381,7 @@ func TestCoordinatorRemoveCapture(t *testing.T) { ctx := context.Background() currentTables := []model.TableID{1, 2, 3} aliveCaptures := map[model.CaptureID]*model.CaptureInfo{"a": {}, "b": {}} - _, _, _, _, err = coord.poll(ctx, 0, currentTables, aliveCaptures, schedulepb.NewBarrierWithMinTs(0)) + _, err = coord.poll(ctx, 0, currentTables, aliveCaptures, schedulepb.NewBarrierWithMinTs(0)) require.Nil(t, err) msgs = trans.SendBuffer require.Len(t, msgs, 1) @@ -456,7 +456,7 @@ func TestCoordinatorAdvanceCheckpoint(t *testing.T) { ctx := context.Background() currentTables := []model.TableID{1, 2} aliveCaptures := map[model.CaptureID]*model.CaptureInfo{"a": {}, "b": {}} - _, _, _, _, err := coord.poll(ctx, 0, currentTables, aliveCaptures, schedulepb.NewBarrierWithMinTs(0)) + _, err := coord.poll(ctx, 0, currentTables, aliveCaptures, schedulepb.NewBarrierWithMinTs(0)) require.Nil(t, err) // Initialize captures. @@ -509,13 +509,13 @@ func TestCoordinatorAdvanceCheckpoint(t *testing.T) { }, }, }) - cts, rts, lastSyncedTs, pullerResolvedTs, err := coord.poll(ctx, 0, currentTables, aliveCaptures, schedulepb.NewBarrierWithMinTs(5)) + watermark, err := coord.poll(ctx, 0, currentTables, aliveCaptures, schedulepb.NewBarrierWithMinTs(5)) require.Nil(t, err) require.True(t, coord.captureM.CheckAllCaptureInitialized()) - require.EqualValues(t, 2, cts) - require.EqualValues(t, 4, rts) - require.EqualValues(t, 4, lastSyncedTs) - require.EqualValues(t, 5, pullerResolvedTs) + require.EqualValues(t, 2, watermark.CheckpointTs) + require.EqualValues(t, 4, watermark.ResolvedTs) + require.EqualValues(t, 4, watermark.LastSyncedTs) + require.EqualValues(t, 5, watermark.PullerResolvedTs) // Checkpoint should be advanced even if there is an uninitialized capture. aliveCaptures["c"] = &model.CaptureInfo{} @@ -560,13 +560,13 @@ func TestCoordinatorAdvanceCheckpoint(t *testing.T) { }, }, }) - cts, rts, lastSyncedTs, pullerResolvedTs, err = coord.poll(ctx, 0, currentTables, aliveCaptures, schedulepb.NewBarrierWithMinTs(5)) + watermark, err = coord.poll(ctx, 0, currentTables, aliveCaptures, schedulepb.NewBarrierWithMinTs(5)) require.Nil(t, err) require.False(t, coord.captureM.CheckAllCaptureInitialized()) - require.EqualValues(t, 3, cts) - require.EqualValues(t, 5, rts) - require.EqualValues(t, 6, lastSyncedTs) - require.EqualValues(t, 7, pullerResolvedTs) + require.EqualValues(t, 3, watermark.CheckpointTs) + require.EqualValues(t, 5, watermark.ResolvedTs) + require.EqualValues(t, 6, watermark.LastSyncedTs) + require.EqualValues(t, 7, watermark.PullerResolvedTs) } func TestCoordinatorDropMsgIfChangefeedEpochMismatch(t *testing.T) { diff --git a/cdc/scheduler/internal/v3/replication/replication_manager.go b/cdc/scheduler/internal/v3/replication/replication_manager.go index 84101e07b3e..a58de4caa81 100644 --- a/cdc/scheduler/internal/v3/replication/replication_manager.go +++ b/cdc/scheduler/internal/v3/replication/replication_manager.go @@ -557,11 +557,9 @@ func (r *Manager) AdvanceCheckpoint( currentPDTime time.Time, barrier *schedulepb.BarrierWithMinTs, redoMetaManager redo.MetaManager, -) (newCheckpointTs, newResolvedTs, newLastSyncedTs, newPullerResolvedTs model.Ts) { +) (watermark schedulepb.Watermark) { var redoFlushedResolvedTs model.Ts - limitBarrierWithRedo := func(newCheckpointTs, newResolvedTs, newLastSyncedTs, - newPullerResolvedTs uint64, - ) (uint64, uint64, uint64, uint64) { + limitBarrierWithRedo := func(watermark *schedulepb.Watermark) { flushedMeta := redoMetaManager.GetFlushedMeta() redoFlushedResolvedTs = flushedMeta.ResolvedTs log.Debug("owner gets flushed redo meta", @@ -569,18 +567,17 @@ func (r *Manager) AdvanceCheckpoint( zap.String("changefeed", r.changefeedID.ID), zap.Uint64("flushedCheckpointTs", flushedMeta.CheckpointTs), zap.Uint64("flushedResolvedTs", flushedMeta.ResolvedTs)) - if flushedMeta.ResolvedTs < newResolvedTs { - newResolvedTs = flushedMeta.ResolvedTs + if flushedMeta.ResolvedTs < watermark.ResolvedTs { + watermark.ResolvedTs = flushedMeta.ResolvedTs } - if newCheckpointTs > newResolvedTs { - newCheckpointTs = newResolvedTs + if watermark.CheckpointTs > watermark.ResolvedTs { + watermark.CheckpointTs = watermark.ResolvedTs } - if barrier.GlobalBarrierTs > newResolvedTs { - barrier.GlobalBarrierTs = newResolvedTs + if barrier.GlobalBarrierTs > watermark.ResolvedTs { + barrier.GlobalBarrierTs = watermark.ResolvedTs } - return newCheckpointTs, newResolvedTs, newLastSyncedTs, newPullerResolvedTs } defer func() { if redoFlushedResolvedTs != 0 && barrier.GlobalBarrierTs > redoFlushedResolvedTs { @@ -594,9 +591,13 @@ func (r *Manager) AdvanceCheckpoint( r.slowestPuller = tablepb.Span{} r.slowestSink = tablepb.Span{} - var slowestPullerResolvedTs uint64 = math.MaxUint64 - newCheckpointTs, newResolvedTs, newLastSyncedTs = math.MaxUint64, math.MaxUint64, 0 + watermark = schedulepb.Watermark{ + CheckpointTs: math.MaxUint64, + ResolvedTs: math.MaxUint64, + LastSyncedTs: 0, + PullerResolvedTs: math.MaxUint64, + } cannotProceed := false currentTables.Iter(func(tableID model.TableID, tableStart, tableEnd tablepb.Span) bool { @@ -624,22 +625,22 @@ func (r *Manager) AdvanceCheckpoint( } // Find the minimum checkpoint ts and resolved ts. - if newCheckpointTs > table.Checkpoint.CheckpointTs { - newCheckpointTs = table.Checkpoint.CheckpointTs + if watermark.CheckpointTs > table.Checkpoint.CheckpointTs { + watermark.CheckpointTs = table.Checkpoint.CheckpointTs r.slowestSink = span } - if newResolvedTs > table.Checkpoint.ResolvedTs { - newResolvedTs = table.Checkpoint.ResolvedTs + if watermark.ResolvedTs > table.Checkpoint.ResolvedTs { + watermark.ResolvedTs = table.Checkpoint.ResolvedTs } // Find the max lastSyncedTs of all tables. - if newLastSyncedTs < table.Checkpoint.LastSyncedTs { - newLastSyncedTs = table.Checkpoint.LastSyncedTs + if watermark.LastSyncedTs < table.Checkpoint.LastSyncedTs { + watermark.LastSyncedTs = table.Checkpoint.LastSyncedTs } // Find the minimum puller resolved ts. if pullerCkpt, ok := table.Stats.StageCheckpoints["puller-egress"]; ok { - if slowestPullerResolvedTs > pullerCkpt.ResolvedTs { - slowestPullerResolvedTs = pullerCkpt.ResolvedTs + if watermark.PullerResolvedTs > pullerCkpt.ResolvedTs { + watermark.PullerResolvedTs = pullerCkpt.ResolvedTs r.slowestPuller = span } } @@ -671,32 +672,40 @@ func (r *Manager) AdvanceCheckpoint( if cannotProceed { if redoMetaManager.Enabled() { // If redo is enabled, GlobalBarrierTs should be limited by redo flushed meta. - newResolvedTs = barrier.RedoBarrierTs - limitBarrierWithRedo(newCheckpointTs, newResolvedTs, checkpointCannotProceed, checkpointCannotProceed) + watermark.ResolvedTs = barrier.RedoBarrierTs + watermark.LastSyncedTs = checkpointCannotProceed + watermark.PullerResolvedTs = checkpointCannotProceed + limitBarrierWithRedo(&watermark) + return watermark + } + return schedulepb.Watermark{ + CheckpointTs: checkpointCannotProceed, + ResolvedTs: checkpointCannotProceed, + LastSyncedTs: checkpointCannotProceed, + PullerResolvedTs: checkpointCannotProceed, } - return checkpointCannotProceed, checkpointCannotProceed, checkpointCannotProceed, checkpointCannotProceed } // If currentTables is empty, we should advance newResolvedTs to global barrier ts and // advance newCheckpointTs to min table barrier ts. - if newResolvedTs == math.MaxUint64 || newCheckpointTs == math.MaxUint64 { - if newCheckpointTs != newResolvedTs || currentTables.Len() != 0 { + if watermark.ResolvedTs == math.MaxUint64 || watermark.CheckpointTs == math.MaxUint64 { + if watermark.CheckpointTs != watermark.ResolvedTs || currentTables.Len() != 0 { log.Panic("schedulerv3: newCheckpointTs and newResolvedTs should be both maxUint64 "+ "if currentTables is empty", - zap.Uint64("newCheckpointTs", newCheckpointTs), - zap.Uint64("newResolvedTs", newResolvedTs), + zap.Uint64("newCheckpointTs", watermark.CheckpointTs), + zap.Uint64("newResolvedTs", watermark.ResolvedTs), zap.Any("currentTables", currentTables)) } - newResolvedTs = barrier.GlobalBarrierTs - newCheckpointTs = barrier.MinTableBarrierTs + watermark.ResolvedTs = barrier.GlobalBarrierTs + watermark.CheckpointTs = barrier.MinTableBarrierTs } - if newCheckpointTs > barrier.MinTableBarrierTs { - newCheckpointTs = barrier.MinTableBarrierTs + if watermark.CheckpointTs > barrier.MinTableBarrierTs { + watermark.CheckpointTs = barrier.MinTableBarrierTs // TODO: add panic after we fix the bug that newCheckpointTs > minTableBarrierTs. // log.Panic("schedulerv3: newCheckpointTs should not be larger than minTableBarrierTs", - // zap.Uint64("newCheckpointTs", newCheckpointTs), - // zap.Uint64("newResolvedTs", newResolvedTs), + // zap.Uint64("newCheckpointTs", watermark.CheckpointTs), + // zap.Uint64("newResolvedTs", watermark.ResolvedTs), // zap.Any("currentTables", currentTables.currentTables), // zap.Any("barrier", barrier.Barrier), // zap.Any("minTableBarrierTs", barrier.MinTableBarrierTs)) @@ -705,7 +714,7 @@ func (r *Manager) AdvanceCheckpoint( // If changefeed's checkpoint lag is larger than 30s, // log the 4 slowlest table infos every minute, which can // help us find the problematic tables. - checkpointLag := currentPDTime.Sub(oracle.GetTimeFromTS(newCheckpointTs)) + checkpointLag := currentPDTime.Sub(oracle.GetTimeFromTS(watermark.CheckpointTs)) if checkpointLag > logSlowTablesLagThreshold && time.Since(r.lastLogSlowTablesTime) > logSlowTablesInterval { r.logSlowTableInfo(currentPDTime) @@ -713,19 +722,19 @@ func (r *Manager) AdvanceCheckpoint( } if redoMetaManager.Enabled() { - if newResolvedTs > barrier.RedoBarrierTs { - newResolvedTs = barrier.RedoBarrierTs + if watermark.ResolvedTs > barrier.RedoBarrierTs { + watermark.ResolvedTs = barrier.RedoBarrierTs } - redoMetaManager.UpdateMeta(newCheckpointTs, newResolvedTs) + redoMetaManager.UpdateMeta(watermark.CheckpointTs, watermark.ResolvedTs) log.Debug("owner updates redo meta", zap.String("namespace", r.changefeedID.Namespace), zap.String("changefeed", r.changefeedID.ID), - zap.Uint64("newCheckpointTs", newCheckpointTs), - zap.Uint64("newResolvedTs", newResolvedTs)) - return limitBarrierWithRedo(newCheckpointTs, newResolvedTs, newLastSyncedTs, slowestPullerResolvedTs) + zap.Uint64("newCheckpointTs", watermark.CheckpointTs), + zap.Uint64("newResolvedTs", watermark.ResolvedTs)) + limitBarrierWithRedo(&watermark) } - return newCheckpointTs, newResolvedTs, newLastSyncedTs, slowestPullerResolvedTs + return watermark } func (r *Manager) logSlowTableInfo(currentPDTime time.Time) { diff --git a/cdc/scheduler/internal/v3/replication/replication_manager_test.go b/cdc/scheduler/internal/v3/replication/replication_manager_test.go index 050b07f9707..a899bc4a985 100644 --- a/cdc/scheduler/internal/v3/replication/replication_manager_test.go +++ b/cdc/scheduler/internal/v3/replication/replication_manager_test.go @@ -663,30 +663,30 @@ func TestReplicationManagerAdvanceCheckpoint(t *testing.T) { // no tables are replicating, resolvedTs should be advanced to globalBarrierTs and checkpoint // should be advanced to minTableBarrierTs. currentTables := &TableRanges{} - checkpoint, resolved, lastSyncedTs, pullerResolvedTs := r.AdvanceCheckpoint( + watermark := r.AdvanceCheckpoint( currentTables, time.Now(), schedulepb.NewBarrierWithMinTs(5), redoMetaManager) - require.Equal(t, model.Ts(5), checkpoint) - require.Equal(t, model.Ts(5), resolved) - require.Equal(t, model.Ts(0), lastSyncedTs) - require.Equal(t, model.Ts(math.MaxUint64), pullerResolvedTs) + require.Equal(t, model.Ts(5), watermark.CheckpointTs) + require.Equal(t, model.Ts(5), watermark.ResolvedTs) + require.Equal(t, model.Ts(0), watermark.LastSyncedTs) + require.Equal(t, model.Ts(math.MaxUint64), watermark.PullerResolvedTs) // all tables are replicating currentTables.UpdateTables([]model.TableID{1, 2}) - checkpoint, resolved, lastSyncedTs, pullerResolvedTs = r.AdvanceCheckpoint( + watermark = r.AdvanceCheckpoint( currentTables, time.Now(), schedulepb.NewBarrierWithMinTs(30), redoMetaManager) - require.Equal(t, model.Ts(10), checkpoint) - require.Equal(t, model.Ts(20), resolved) - require.Equal(t, model.Ts(20), lastSyncedTs) - require.Equal(t, model.Ts(30), pullerResolvedTs) + require.Equal(t, model.Ts(10), watermark.CheckpointTs) + require.Equal(t, model.Ts(20), watermark.ResolvedTs) + require.Equal(t, model.Ts(20), watermark.LastSyncedTs) + require.Equal(t, model.Ts(30), watermark.PullerResolvedTs) // some table not exist yet. currentTables.UpdateTables([]model.TableID{1, 2, 3}) - checkpoint, resolved, lastSyncedTs, pullerResolvedTs = r.AdvanceCheckpoint( + watermark = r.AdvanceCheckpoint( currentTables, time.Now(), schedulepb.NewBarrierWithMinTs(30), redoMetaManager) - require.Equal(t, checkpointCannotProceed, checkpoint) - require.Equal(t, checkpointCannotProceed, resolved) - require.Equal(t, checkpointCannotProceed, lastSyncedTs) - require.Equal(t, checkpointCannotProceed, pullerResolvedTs) + require.Equal(t, checkpointCannotProceed, watermark.CheckpointTs) + require.Equal(t, checkpointCannotProceed, watermark.ResolvedTs) + require.Equal(t, checkpointCannotProceed, watermark.LastSyncedTs) + require.Equal(t, checkpointCannotProceed, watermark.PullerResolvedTs) span3 := spanz.TableIDToComparableSpan(3) rs, err = NewReplicationSet(span3, model.Ts(5), @@ -726,12 +726,12 @@ func TestReplicationManagerAdvanceCheckpoint(t *testing.T) { }, model.ChangeFeedID{}) require.NoError(t, err) r.spans.ReplaceOrInsert(span3, rs) - checkpoint, resolved, lastSyncedTs, pullerResolvedTs = r.AdvanceCheckpoint( + watermark = r.AdvanceCheckpoint( currentTables, time.Now(), schedulepb.NewBarrierWithMinTs(30), redoMetaManager) - require.Equal(t, model.Ts(5), checkpoint) - require.Equal(t, model.Ts(20), resolved) - require.Equal(t, model.Ts(32), lastSyncedTs) - require.Equal(t, model.Ts(30), pullerResolvedTs) + require.Equal(t, model.Ts(5), watermark.CheckpointTs) + require.Equal(t, model.Ts(20), watermark.ResolvedTs) + require.Equal(t, model.Ts(32), watermark.LastSyncedTs) + require.Equal(t, model.Ts(30), watermark.PullerResolvedTs) currentTables.UpdateTables([]model.TableID{1, 2, 3, 4}) span4 := spanz.TableIDToComparableSpan(4) @@ -756,12 +756,12 @@ func TestReplicationManagerAdvanceCheckpoint(t *testing.T) { }, model.ChangeFeedID{}) require.NoError(t, err) r.spans.ReplaceOrInsert(span4, rs) - checkpoint, resolved, lastSyncedTs, pullerResolvedTs = r.AdvanceCheckpoint( + watermark = r.AdvanceCheckpoint( currentTables, time.Now(), schedulepb.NewBarrierWithMinTs(30), redoMetaManager) - require.Equal(t, model.Ts(3), checkpoint) - require.Equal(t, model.Ts(10), resolved) - require.Equal(t, model.Ts(32), lastSyncedTs) - require.Equal(t, model.Ts(12), pullerResolvedTs) + require.Equal(t, model.Ts(3), watermark.CheckpointTs) + require.Equal(t, model.Ts(10), watermark.ResolvedTs) + require.Equal(t, model.Ts(32), watermark.LastSyncedTs) + require.Equal(t, model.Ts(12), watermark.PullerResolvedTs) // Split table 5 into 2 spans. currentTables.UpdateTables([]model.TableID{1, 2, 3, 4, 5}) @@ -792,31 +792,31 @@ func TestReplicationManagerAdvanceCheckpoint(t *testing.T) { require.NoError(t, err) r.spans.ReplaceOrInsert(span, rs) } - checkpoint, resolved, lastSyncedTs, pullerResolvedTs = r.AdvanceCheckpoint( + watermark = r.AdvanceCheckpoint( currentTables, time.Now(), schedulepb.NewBarrierWithMinTs(30), redoMetaManager) - require.Equal(t, model.Ts(3), checkpoint) - require.Equal(t, model.Ts(10), resolved) - require.Equal(t, model.Ts(32), lastSyncedTs) - require.Equal(t, model.Ts(11), pullerResolvedTs) + require.Equal(t, model.Ts(3), watermark.CheckpointTs) + require.Equal(t, model.Ts(10), watermark.ResolvedTs) + require.Equal(t, model.Ts(32), watermark.LastSyncedTs) + require.Equal(t, model.Ts(11), watermark.PullerResolvedTs) // The start span is missing rs5_1, _ := r.spans.Delete(span5_1) - checkpoint, resolved, lastSyncedTs, pullerResolvedTs = r.AdvanceCheckpoint( + watermark = r.AdvanceCheckpoint( currentTables, time.Now(), schedulepb.NewBarrierWithMinTs(30), redoMetaManager) - require.Equal(t, checkpointCannotProceed, checkpoint) - require.Equal(t, checkpointCannotProceed, resolved) - require.Equal(t, checkpointCannotProceed, lastSyncedTs) - require.Equal(t, checkpointCannotProceed, pullerResolvedTs) + require.Equal(t, checkpointCannotProceed, watermark.CheckpointTs) + require.Equal(t, checkpointCannotProceed, watermark.ResolvedTs) + require.Equal(t, checkpointCannotProceed, watermark.LastSyncedTs) + require.Equal(t, checkpointCannotProceed, watermark.PullerResolvedTs) // The end span is missing r.spans.ReplaceOrInsert(span5_1, rs5_1) r.spans.Delete(span5_2) - checkpoint, resolved, lastSyncedTs, pullerResolvedTs = r.AdvanceCheckpoint( + watermark = r.AdvanceCheckpoint( currentTables, time.Now(), schedulepb.NewBarrierWithMinTs(30), redoMetaManager) - require.Equal(t, checkpointCannotProceed, checkpoint) - require.Equal(t, checkpointCannotProceed, resolved) - require.Equal(t, checkpointCannotProceed, lastSyncedTs) - require.Equal(t, checkpointCannotProceed, pullerResolvedTs) + require.Equal(t, checkpointCannotProceed, watermark.CheckpointTs) + require.Equal(t, checkpointCannotProceed, watermark.ResolvedTs) + require.Equal(t, checkpointCannotProceed, watermark.LastSyncedTs) + require.Equal(t, checkpointCannotProceed, watermark.PullerResolvedTs) // redo is enabled currentTables.UpdateTables([]model.TableID{4}) @@ -845,12 +845,12 @@ func TestReplicationManagerAdvanceCheckpoint(t *testing.T) { barrier := schedulepb.NewBarrierWithMinTs(30) redoMetaManager.enable = true redoMetaManager.resolvedTs = 9 - checkpoint, resolved, lastSyncedTs, pullerResolvedTs = r.AdvanceCheckpoint( + watermark = r.AdvanceCheckpoint( currentTables, time.Now(), barrier, redoMetaManager) - require.Equal(t, model.Ts(9), resolved) - require.Equal(t, model.Ts(9), checkpoint) - require.Equal(t, model.Ts(12), lastSyncedTs) - require.Equal(t, model.Ts(16), pullerResolvedTs) + require.Equal(t, model.Ts(9), watermark.ResolvedTs) + require.Equal(t, model.Ts(9), watermark.CheckpointTs) + require.Equal(t, model.Ts(12), watermark.LastSyncedTs) + require.Equal(t, model.Ts(16), watermark.PullerResolvedTs) require.Equal(t, model.Ts(9), barrier.GetGlobalBarrierTs()) } @@ -909,13 +909,13 @@ func TestReplicationManagerAdvanceCheckpointWithRedoEnabled(t *testing.T) { currentTables := &TableRanges{} currentTables.UpdateTables([]model.TableID{1, 2, 3}) barrier := schedulepb.NewBarrierWithMinTs(30) - checkpoint, resolved, lastSyncedTs, pullerResolvedTs := r.AdvanceCheckpoint( + watermark := r.AdvanceCheckpoint( currentTables, time.Now(), barrier, redoMetaManager) - require.Equal(t, checkpointCannotProceed, checkpoint) - require.Equal(t, checkpointCannotProceed, resolved) - require.Equal(t, checkpointCannotProceed, lastSyncedTs) - require.Equal(t, checkpointCannotProceed, pullerResolvedTs) + require.Equal(t, checkpointCannotProceed, watermark.CheckpointTs) + require.Equal(t, checkpointCannotProceed, watermark.ResolvedTs) + require.Equal(t, checkpointCannotProceed, watermark.LastSyncedTs) + require.Equal(t, checkpointCannotProceed, watermark.PullerResolvedTs) require.Equal(t, uint64(25), barrier.Barrier.GetGlobalBarrierTs()) } diff --git a/cdc/scheduler/schedulepb/watermark.go b/cdc/scheduler/schedulepb/watermark.go new file mode 100644 index 00000000000..81123734e74 --- /dev/null +++ b/cdc/scheduler/schedulepb/watermark.go @@ -0,0 +1,24 @@ +// Copyright 2023 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package schedulepb + +import "github.com/pingcap/tiflow/cdc/model" + +// Watermark contains various ts variables to make code easier +type Watermark struct { + CheckpointTs model.Ts + ResolvedTs model.Ts + LastSyncedTs model.Ts + PullerResolvedTs model.Ts +} diff --git a/tests/integration_tests/synced_status/run.sh b/tests/integration_tests/synced_status/run.sh index b436667d85e..efe02884dc1 100644 --- a/tests/integration_tests/synced_status/run.sh +++ b/tests/integration_tests/synced_status/run.sh @@ -178,6 +178,9 @@ function run_case_with_unavailable_tikv() { info=$(echo $synced_status | jq -r '.info') target_message="Please check whether pd is healthy and tikv region is all available. \ If pd is not healthy or tikv region is not available, the data syncing is finished. \ +When pd is offline means that pd is not healthy. For tikv region, you can check the grafana info \ +in 'TiKV-Details-Resolved-Ts-Max Leader Resolved TS gap'. If the gap is a large value, such as a few minutes, \ +it means some regions in tikv are unavailable. \ Otherwise the data syncing is not finished, please wait" if [ "$info" != "$target_message" ]; then @@ -274,8 +277,9 @@ function run_case_with_failpoint() { info=$(echo $synced_status | jq -r '.info') target_message="Please check whether pd is healthy and tikv region is all available. \ If pd is not healthy or tikv region is not available, the data syncing is finished. \ -Because in this case, the resolvedTs will not advance anymore, \ -thus we only need to care whether last_synced_ts is more than 120 secs from the current time. \ +When pd is offline means that pd is not healthy. For tikv region, you can check the grafana info \ +in 'TiKV-Details-Resolved-Ts-Max Leader Resolved TS gap'. If the gap is a large value, such as a few minutes, \ +it means some regions in tikv are unavailable. \ Otherwise the data syncing is not finished, please wait" if [ "$info" != "$target_message" ]; then echo "synced status info is not correct" From 74e9b82b8eee77074c71d94bd83206f09b26167c Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Tue, 19 Dec 2023 19:03:56 +0800 Subject: [PATCH 30/35] fix comments --- cdc/api/v2/changefeed.go | 36 ++++++++++++++++------------------- cdc/api/v2/changefeed_test.go | 1 - 2 files changed, 16 insertions(+), 21 deletions(-) diff --git a/cdc/api/v2/changefeed.go b/cdc/api/v2/changefeed.go index 9a53227d21b..5b5bb1889f9 100644 --- a/cdc/api/v2/changefeed.go +++ b/cdc/api/v2/changefeed.go @@ -29,7 +29,6 @@ import ( "github.com/pingcap/tiflow/cdc/api" "github.com/pingcap/tiflow/cdc/capture" "github.com/pingcap/tiflow/cdc/model" - "github.com/pingcap/tiflow/dm/pkg/terror" "github.com/pingcap/tiflow/pkg/config" cerror "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/retry" @@ -898,13 +897,6 @@ func (h *OpenAPIV2) status(c *gin.Context) { }) } -// transformer timestamp to readable format -func transformerTime(timestamp int64) string { - location := time.Local - tm := time.Unix((timestamp / 1000), 0).In(location) - return tm.Format("2006-01-02 15:04:05") -} - // synced get the synced status of a changefeed // @Summary Get synced status // @Description get the synced status of a changefeed @@ -964,7 +956,7 @@ func (h *OpenAPIV2) synced(c *gin.Context) { var message string if (oracle.ExtractPhysical(status.PullerResolvedTs) - oracle.ExtractPhysical(status.CheckpointTs)) > cfg.ReplicaConfig.SyncedStatus.CheckpointInterval*1000 { - message = fmt.Sprintf("%s. Besides the data is not finish syncing", terror.Message(err)) + message = fmt.Sprintf("%s. Besides the data is not finish syncing", err.Error()) } else { message = fmt.Sprintf("%s. You should check the pd status first. If pd status is normal, means we don't finish sync data. "+ "If pd is offline, please check the whether we satisfy the condition that "+ @@ -998,7 +990,10 @@ func (h *OpenAPIV2) synced(c *gin.Context) { NowTs: model.JSONTime(time.Unix(physicalNow/1e3, 0)), Info: "Data syncing is finished", }) - } else if physicalNow-oracle.ExtractPhysical(status.LastSyncedTs) > cfg.ReplicaConfig.SyncedStatus.SyncedCheckInterval*1000 { + return + } + + if physicalNow-oracle.ExtractPhysical(status.LastSyncedTs) > cfg.ReplicaConfig.SyncedStatus.SyncedCheckInterval*1000 { // case 3: If physcialNow - lastSyncedTs > SyncedCheckInterval && physcialNow - CheckpointTs > CheckpointInterval // we should consider the situation that pd or tikv region is not healthy to block the advancing resolveTs. // if pullerResolvedTs - checkpointTs > CheckpointInterval--> data is not synced @@ -1024,17 +1019,18 @@ func (h *OpenAPIV2) synced(c *gin.Context) { NowTs: model.JSONTime(time.Unix(physicalNow/1e3, 0)), Info: message, }) - } else { - // case 4: If physcialNow - lastSyncedTs < SyncedCheckInterval --> data is not synced - c.JSON(http.StatusOK, SyncedStatus{ - Synced: false, - SinkCheckpointTs: model.JSONTime(oracle.GetTimeFromTS(status.CheckpointTs)), - PullerResolvedTs: model.JSONTime(oracle.GetTimeFromTS(status.PullerResolvedTs)), - LastSyncedTs: model.JSONTime(oracle.GetTimeFromTS(status.LastSyncedTs)), - NowTs: model.JSONTime(time.Unix(physicalNow/1e3, 0)), - Info: "The data syncing is not finished, please wait", - }) + return } + + // case 4: If physcialNow - lastSyncedTs < SyncedCheckInterval --> data is not synced + c.JSON(http.StatusOK, SyncedStatus{ + Synced: false, + SinkCheckpointTs: model.JSONTime(oracle.GetTimeFromTS(status.CheckpointTs)), + PullerResolvedTs: model.JSONTime(oracle.GetTimeFromTS(status.PullerResolvedTs)), + LastSyncedTs: model.JSONTime(oracle.GetTimeFromTS(status.LastSyncedTs)), + NowTs: model.JSONTime(time.Unix(physicalNow/1e3, 0)), + Info: "The data syncing is not finished, please wait", + }) } func toAPIModel( diff --git a/cdc/api/v2/changefeed_test.go b/cdc/api/v2/changefeed_test.go index 9dec0790046..07b8c6e9013 100644 --- a/cdc/api/v2/changefeed_test.go +++ b/cdc/api/v2/changefeed_test.go @@ -32,7 +32,6 @@ import ( "github.com/pingcap/tiflow/cdc/model" mock_owner "github.com/pingcap/tiflow/cdc/owner/mock" "github.com/pingcap/tiflow/pkg/config" - cerror "github.com/pingcap/tiflow/pkg/errors" cerrors "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/etcd" mock_etcd "github.com/pingcap/tiflow/pkg/etcd/mock" From 5f519038ce1fe7ba8fb7fcf893cd3b6a237a5433 Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Tue, 19 Dec 2023 19:17:25 +0800 Subject: [PATCH 31/35] update code --- cdc/sink/tablesink/table_sink_impl.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cdc/sink/tablesink/table_sink_impl.go b/cdc/sink/tablesink/table_sink_impl.go index 24a2984704d..76b65cc7781 100644 --- a/cdc/sink/tablesink/table_sink_impl.go +++ b/cdc/sink/tablesink/table_sink_impl.go @@ -87,7 +87,7 @@ func New[E dmlsink.TableEvent, P dmlsink.Appender[E]]( eventAppender: appender, eventBuffer: make([]E, 0, 1024), state: state.TableSinkSinking, - lastSyncedTs: LastSyncedTsRecord{lastSyncedTs: 0}, + lastSyncedTs: LastSyncedTsRecord{lastSyncedTs: startTs}, metricsTableSinkTotalRows: totalRowsCounter, } } From 9680570f1fa63eff39df45b6e1895828c1312efb Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Wed, 20 Dec 2023 10:05:59 +0800 Subject: [PATCH 32/35] update --- cdc/api/v2/changefeed_test.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/cdc/api/v2/changefeed_test.go b/cdc/api/v2/changefeed_test.go index 69775b7704e..ba7250e8c00 100644 --- a/cdc/api/v2/changefeed_test.go +++ b/cdc/api/v2/changefeed_test.go @@ -1017,7 +1017,7 @@ func TestChangefeedSynced(t *testing.T) { statusProvider.err = nil statusProvider.changefeedInfo = cfInfo { - helpers.EXPECT().getPDClient(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil, cerror.ErrAPIGetPDClientFailed).Times(1) + helpers.EXPECT().getPDClient(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil, cerrors.ErrAPIGetPDClientFailed).Times(1) // case3: pd is offline,resolvedTs - checkpointTs > 15s statusProvider.changeFeedSyncedStatus = &model.ChangeFeedSyncedStatusForAPI{ CheckpointTs: 1701153217279 << 18, @@ -1042,7 +1042,7 @@ func TestChangefeedSynced(t *testing.T) { } { - helpers.EXPECT().getPDClient(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil, cerror.ErrAPIGetPDClientFailed).Times(1) + helpers.EXPECT().getPDClient(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil, cerrors.ErrAPIGetPDClientFailed).Times(1) // case4: pd is offline,resolvedTs - checkpointTs < 15s statusProvider.changeFeedSyncedStatus = &model.ChangeFeedSyncedStatusForAPI{ CheckpointTs: 1701153217279 << 18, From 70866f554f1c45ec5ab5d5a13255144a1eec270f Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Wed, 20 Dec 2023 10:58:16 +0800 Subject: [PATCH 33/35] update --- cdc/scheduler/internal/v3/replication/replication_manager.go | 1 - 1 file changed, 1 deletion(-) diff --git a/cdc/scheduler/internal/v3/replication/replication_manager.go b/cdc/scheduler/internal/v3/replication/replication_manager.go index a58de4caa81..8196a06a0f1 100644 --- a/cdc/scheduler/internal/v3/replication/replication_manager.go +++ b/cdc/scheduler/internal/v3/replication/replication_manager.go @@ -676,7 +676,6 @@ func (r *Manager) AdvanceCheckpoint( watermark.LastSyncedTs = checkpointCannotProceed watermark.PullerResolvedTs = checkpointCannotProceed limitBarrierWithRedo(&watermark) - return watermark } return schedulepb.Watermark{ CheckpointTs: checkpointCannotProceed, From fca13f0fa518f4947552e70e8d63d6080bdf031d Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Wed, 20 Dec 2023 11:22:27 +0800 Subject: [PATCH 34/35] fix --- cdc/api/v2/changefeed_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cdc/api/v2/changefeed_test.go b/cdc/api/v2/changefeed_test.go index ba7250e8c00..3d36f2b9048 100644 --- a/cdc/api/v2/changefeed_test.go +++ b/cdc/api/v2/changefeed_test.go @@ -1064,7 +1064,7 @@ func TestChangefeedSynced(t *testing.T) { require.Equal(t, false, resp.Synced) require.Equal(t, "[CDC:ErrAPIGetPDClientFailed]failed to get PDClient to connect PD, please recheck. "+ "You can check the pd first, and if pd is available, means we don't finish sync data. "+ - "If pd is not available, please check the whether we satisfy the condition that "+ + "If pd is offline, please check the whether we satisfy the condition that "+ "the time difference from lastSyncedTs to the current time from the time zone of pd is greater than 300 secs. "+ "If it's satisfied, means the data syncing is totally finished", resp.Info) } From f9ea16ce25bdf2b109eb502cd02b6abb7caec7e4 Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Thu, 21 Dec 2023 09:58:55 +0800 Subject: [PATCH 35/35] fix typo --- cdc/api/v2/changefeed.go | 4 ++-- cdc/api/v2/changefeed_test.go | 4 ++-- tests/integration_tests/synced_status/run.sh | 8 ++++---- 3 files changed, 8 insertions(+), 8 deletions(-) diff --git a/cdc/api/v2/changefeed.go b/cdc/api/v2/changefeed.go index 57295868d29..cd88fe57643 100644 --- a/cdc/api/v2/changefeed.go +++ b/cdc/api/v2/changefeed.go @@ -959,8 +959,8 @@ func (h *OpenAPIV2) synced(c *gin.Context) { message = fmt.Sprintf("%s. Besides the data is not finish syncing", err.Error()) } else { message = fmt.Sprintf("%s. You should check the pd status first. If pd status is normal, means we don't finish sync data. "+ - "If pd is offline, please check the whether we satisfy the condition that "+ - "the time difference from lastSyncedTs to the current time in the time zone of pd is greater than %v secs. "+ + "If pd is offline, please check whether we satisfy the condition that "+ + "the time difference from lastSyncedTs to the current time from the time zone of pd is greater than %v secs. "+ "If it's satisfied, means the data syncing is totally finished", err, cfg.ReplicaConfig.SyncedStatus.SyncedCheckInterval) } c.JSON(http.StatusOK, SyncedStatus{ diff --git a/cdc/api/v2/changefeed_test.go b/cdc/api/v2/changefeed_test.go index 3d36f2b9048..4348fcef612 100644 --- a/cdc/api/v2/changefeed_test.go +++ b/cdc/api/v2/changefeed_test.go @@ -1063,8 +1063,8 @@ func TestChangefeedSynced(t *testing.T) { require.Nil(t, err) require.Equal(t, false, resp.Synced) require.Equal(t, "[CDC:ErrAPIGetPDClientFailed]failed to get PDClient to connect PD, please recheck. "+ - "You can check the pd first, and if pd is available, means we don't finish sync data. "+ - "If pd is offline, please check the whether we satisfy the condition that "+ + "You should check the pd status first. If pd status is normal, means we don't finish sync data. "+ + "If pd is offline, please check whether we satisfy the condition that "+ "the time difference from lastSyncedTs to the current time from the time zone of pd is greater than 300 secs. "+ "If it's satisfied, means the data syncing is totally finished", resp.Info) } diff --git a/tests/integration_tests/synced_status/run.sh b/tests/integration_tests/synced_status/run.sh index efe02884dc1..04f55009dad 100644 --- a/tests/integration_tests/synced_status/run.sh +++ b/tests/integration_tests/synced_status/run.sh @@ -74,13 +74,13 @@ function run_normal_case_and_unavailable_pd() { exit 1 fi # the timestamp for puller_resolved_ts is 0 when do data insert - if [ "$puller_resolved_ts" != "1970-01-01 08:00:00" ]; then - echo "puller_resolved_ts is not 1970-01-01 08:00:00" + if [ "$puller_resolved_ts" != "1970-01-01 08:00:00.000" ]; then + echo "puller_resolved_ts is not 1970-01-01 08:00:00.000" exit 1 fi # the timestamp for last_synced_ts is 0 when do data insert - if [ "$last_synced_ts" != "1970-01-01 08:00:00" ]; then - echo "last_synced_ts is not 1970-01-01 08:00:00" + if [ "$last_synced_ts" != "1970-01-01 08:00:00.000" ]; then + echo "last_synced_ts is not 1970-01-01 08:00:00.000" exit 1 fi