Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

owner: Fix owner exit due to emit checkpoint failure (#1985) #2031

Merged
Merged
Show file tree
Hide file tree
Changes from 15 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
3 changes: 3 additions & 0 deletions cdc/changefeed.go
Original file line number Diff line number Diff line change
Expand Up @@ -938,6 +938,9 @@ func (c *changeFeed) calcResolvedTs(ctx context.Context) error {
// some DDL is waiting to executed, we can't ensure whether the DDL has been executed.
// so we can't emit checkpoint to sink
if c.ddlState != model.ChangeFeedWaitToExecDDL {
failpoint.Inject("InjectEmitCheckpointTsError", func() {
failpoint.Return(cerror.ErrEmitCheckpointTsFailed.GenWithStackByArgs())
})
err := c.sink.EmitCheckpointTs(ctx, minCheckpointTs)
if err != nil {
return errors.Trace(err)
Expand Down
42 changes: 32 additions & 10 deletions cdc/owner.go
Original file line number Diff line number Diff line change
Expand Up @@ -864,9 +864,31 @@ func (o *Owner) flushChangeFeedInfos(ctx context.Context) error {

// calcResolvedTs call calcResolvedTs of every changefeeds
func (o *Owner) calcResolvedTs(ctx context.Context) error {
for _, cf := range o.changeFeeds {
for id, cf := range o.changeFeeds {
if err := cf.calcResolvedTs(ctx); err != nil {
return errors.Trace(err)
log.Error("fail to calculate checkpoint ts, so it will be stopped", zap.String("changefeed", cf.id), zap.Error(err))
// error may cause by sink.EmitCheckpointTs`, just stop the changefeed at the moment
// todo: make the method mentioned above more robust.
var code string
if rfcCode, ok := cerror.RFCCode(err); ok {
code = string(rfcCode)
} else {
code = string(cerror.ErrOwnerUnknown.RFCCode())
}

job := model.AdminJob{
CfID: id,
Type: model.AdminStop,
Error: &model.RunningError{
Addr: util.CaptureAddrFromCtx(ctx),
Code: code,
Message: err.Error(),
},
}

if err := o.EnqueueJob(job); err != nil {
return errors.Trace(err)
}
}
}
return nil
Expand Down Expand Up @@ -942,7 +964,7 @@ func (o *Owner) dispatchJob(ctx context.Context, job model.AdminJob) error {
// For `AdminResume`, we remove stopped feed in changefeed initialization phase.
// For `AdminRemove`, we need to update stoppedFeeds when removing a stopped changefeed.
if job.Type == model.AdminStop {
log.Debug("put changfeed into stoppedFeeds queue", zap.String("changefeed", job.CfID))
log.Debug("put changefeed into stoppedFeeds queue", zap.String("changefeed", job.CfID))
o.stoppedFeeds[job.CfID] = cf.status
}
for captureID := range cf.taskStatus {
Expand Down Expand Up @@ -1656,7 +1678,7 @@ func (o *Owner) rebuildCaptureEvents(ctx context.Context, captures map[model.Cap
// the step-2 may meet an error such as ErrCompacted, and we will continue
// from step-1, however other capture may crash just after step-2 returns
// and before step-1 starts, the longer time gap between step-2 to step-1,
// missing a crashed capture is more likey to happen.
// missing a crashed capture is more likely to happen.
o.l.Lock()
defer o.l.Unlock()
return errors.Trace(o.cleanUpStaleTasks(ctx))
Expand Down Expand Up @@ -1689,7 +1711,7 @@ func (o *Owner) startCaptureWatcher(ctx context.Context) {
return
}
log.Warn("watch capture returned", zap.Error(err))
// Otherwise, a temporary error occured(ErrCompact),
// Otherwise, a temporary error occurred(ErrCompact),
// restart the watching routine.
}
}
Expand All @@ -1700,15 +1722,15 @@ func (o *Owner) startCaptureWatcher(ctx context.Context) {
// By setting the AdminJob type to AdminStop and the Error code to indicate that the changefeed is stagnant.
func (o *Owner) handleStaleChangeFeed(ctx context.Context, staleChangeFeeds map[model.ChangeFeedID]*model.ChangeFeedStatus, minGCSafePoint uint64) error {
for id, status := range staleChangeFeeds {
message := cerror.ErrSnapshotLostByGC.GenWithStackByArgs(status.CheckpointTs, minGCSafePoint).Error()
log.Warn("changefeed checkpoint is lagging too much, so it will be stopped.", zap.String("changefeed", id), zap.String("Error message", message))
err := cerror.ErrSnapshotLostByGC.GenWithStackByArgs(status.CheckpointTs, minGCSafePoint)
log.Warn("changefeed checkpoint is lagging too much, so it will be stopped.", zap.String("changefeed", id), zap.Error(err))
runningError := &model.RunningError{
Addr: util.CaptureAddrFromCtx(ctx),
Code: string(cerror.ErrSnapshotLostByGC.RFCCode()), // changfeed is stagnant
Message: message,
Code: string(cerror.ErrSnapshotLostByGC.RFCCode()), // changefeed is stagnant
Message: err.Error(),
}

err := o.EnqueueJob(model.AdminJob{
err = o.EnqueueJob(model.AdminJob{
CfID: id,
Type: model.AdminStop,
Error: runningError,
Expand Down
83 changes: 83 additions & 0 deletions cdc/owner_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -121,6 +121,89 @@ func (m *mockPDClient) UpdateServiceGCSafePoint(ctx context.Context, serviceID s
return safePoint, nil
}

type mockSink struct {
sink.Sink
checkpointTs model.Ts

checkpointMu sync.Mutex
checkpointError error
}

func (m *mockSink) EmitCheckpointTs(ctx context.Context, ts uint64) error {
m.checkpointMu.Lock()
defer m.checkpointMu.Unlock()
atomic.StoreUint64(&m.checkpointTs, ts)
return m.checkpointError
}

func (m *mockSink) Close() error {
return nil
}

// Test whether the owner can tolerate sink caused error, it won't be killed.
// also set the specific changefeed to stop
func (s *ownerSuite) TestOwnerCalcResolvedTs(c *check.C) {
defer testleak.AfterTest(c)()
mockPDCli := &mockPDClient{}

sink := &mockSink{checkpointError: cerror.ErrKafkaSendMessage}
changeFeeds := map[model.ChangeFeedID]*changeFeed{
"test_change_feed_1": {
info: &model.ChangeFeedInfo{State: model.StateNormal},
etcdCli: s.client,
status: &model.ChangeFeedStatus{
CheckpointTs: 0,
},
targetTs: 2000,
ddlResolvedTs: 2000,
ddlState: model.ChangeFeedSyncDML,
taskStatus: model.ProcessorsInfos{
"capture_1": {},
"capture_2": {},
},
taskPositions: map[string]*model.TaskPosition{
"capture_1": {
CheckPointTs: 2333,
ResolvedTs: 2333,
},
"capture_2": {
CheckPointTs: 2333,
ResolvedTs: 2333,
},
},
sink: sink,
},
}

session, err := concurrency.NewSession(s.client.Client.Unwrap(),
concurrency.WithTTL(config.GetDefaultServerConfig().CaptureSessionTTL))
c.Assert(err, check.IsNil)
mockOwner := Owner{
session: session,
pdClient: mockPDCli,
etcdClient: s.client,
lastFlushChangefeeds: time.Now(),
flushChangefeedInterval: 1 * time.Hour,
changeFeeds: changeFeeds,
cfRWriter: s.client,
stoppedFeeds: make(map[model.ChangeFeedID]*model.ChangeFeedStatus),
minGCSafePointCache: minGCSafePointCacheEntry{},
}

err = mockOwner.calcResolvedTs(s.ctx)
c.Assert(err, check.IsNil)

err = mockOwner.handleAdminJob(s.ctx)
c.Assert(err, check.IsNil)
c.Assert(mockOwner.stoppedFeeds["test_change_feed_1"], check.NotNil)

err = mockOwner.flushChangeFeedInfos(s.ctx)
c.Assert(err, check.IsNil)
c.Assert(mockPDCli.invokeCounter, check.Equals, 1)

s.TearDownTest(c)
}

func (s *ownerSuite) TestOwnerFlushChangeFeedInfos(c *check.C) {
defer testleak.AfterTest(c)()
session, err := concurrency.NewSession(s.client.Client.Unwrap(),
Expand Down
2 changes: 1 addition & 1 deletion cdc/server.go
Original file line number Diff line number Diff line change
Expand Up @@ -183,7 +183,7 @@ func (s *Server) campaignOwnerLoop(ctx context.Context) error {
}
err2 := s.capture.Resign(ctx)
if err2 != nil {
// if regisn owner failed, return error to let capture exits
// if resign owner failed, return error to let capture exits
return errors.Annotatef(err2, "resign owner failed, capture: %s", captureID)
}
log.Warn("run owner failed", zap.Error(err))
Expand Down
4 changes: 2 additions & 2 deletions cdc/sink/cdclog/s3.go
Original file line number Diff line number Diff line change
Expand Up @@ -356,7 +356,7 @@ func NewS3Sink(ctx context.Context, sinkURI *url.URL, errCh chan error) (*s3Sink
options := &storage.BackendOptions{}
storage.ExtractQueryParameters(sinkURI, &options.S3)
if err := options.S3.Apply(s3); err != nil {
return nil, cerror.WrapError(cerror.ErrS3SinkInitialzie, err)
return nil, cerror.WrapError(cerror.ErrS3SinkInitialize, err)
}
// we should set this to true, since br set it by default in parseBackend
s3.ForcePathStyle = true
Expand All @@ -369,7 +369,7 @@ func NewS3Sink(ctx context.Context, sinkURI *url.URL, errCh chan error) (*s3Sink
HTTPClient: nil,
})
if err != nil {
return nil, cerror.WrapError(cerror.ErrS3SinkInitialzie, err)
return nil, cerror.WrapError(cerror.ErrS3SinkInitialize, err)
}

s := &s3Sink{
Expand Down
2 changes: 1 addition & 1 deletion cdc/sink/mq.go
Original file line number Diff line number Diff line change
Expand Up @@ -371,7 +371,7 @@ func (k *mqSink) writeToProducer(ctx context.Context, message *codec.MQMessage,
if partition >= 0 {
return k.mqProducer.SendMessage(ctx, message, partition)
}
return cerror.ErrAsyncBroadcaseNotSupport.GenWithStackByArgs()
return cerror.ErrAsyncBroadcastNotSupport.GenWithStackByArgs()
case codec.EncoderNeedSyncWrite:
if partition >= 0 {
err := k.mqProducer.SendMessage(ctx, message, partition)
Expand Down
8 changes: 4 additions & 4 deletions cdc/sink/producer/kafka/kafka.go
Original file line number Diff line number Diff line change
Expand Up @@ -97,7 +97,7 @@ func (k *kafkaSaramaProducer) SendMessage(ctx context.Context, message *codec.MQ
msg.Metadata = atomic.AddUint64(&k.partitionOffset[partition].sent, 1)

failpoint.Inject("KafkaSinkAsyncSendError", func() {
// simulate sending message to intput channel successfully but flushing
// simulate sending message to input channel successfully but flushing
// message to Kafka meets error
log.Info("failpoint error injected")
k.failpointCh <- errors.New("kafka sink injected error")
Expand Down Expand Up @@ -180,7 +180,7 @@ flushLoop:
if checkAllPartitionFlushed() {
return nil
}
return cerror.ErrKafkaFlushUnfished.GenWithStackByArgs()
return cerror.ErrKafkaFlushUnfinished.GenWithStackByArgs()
case <-k.flushedReceiver.C:
if !checkAllPartitionFlushed() {
continue flushLoop
Expand Down Expand Up @@ -380,7 +380,7 @@ func init() {
}

var (
validClienID *regexp.Regexp = regexp.MustCompile(`\A[A-Za-z0-9._-]+\z`)
validClientID *regexp.Regexp = regexp.MustCompile(`\A[A-Za-z0-9._-]+\z`)
commonInvalidChar *regexp.Regexp = regexp.MustCompile(`[\?:,"]`)
)

Expand All @@ -391,7 +391,7 @@ func kafkaClientID(role, captureAddr, changefeedID, configuredClientID string) (
clientID = fmt.Sprintf("TiCDC_sarama_producer_%s_%s_%s", role, captureAddr, changefeedID)
clientID = commonInvalidChar.ReplaceAllString(clientID, "_")
}
if !validClienID.MatchString(clientID) {
if !validClientID.MatchString(clientID) {
return "", cerror.ErrKafkaInvalidClientID.GenWithStackByArgs(clientID)
}
return
Expand Down
2 changes: 1 addition & 1 deletion cdc/sink/producer/kafka/kafka_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -121,7 +121,7 @@ func (s *kafkaSuite) TestSaramaProducer(c *check.C) {
c.Assert(err, check.IsNil)
}

// In TiCDC logic, resovled ts event will always notify the flush loop. Here we
// In TiCDC logic, resolved ts event will always notify the flush loop. Here we
// trigger the flushedNotifier periodically to prevent the flush loop block.
var wg sync.WaitGroup
wg.Add(1)
Expand Down
2 changes: 1 addition & 1 deletion cdc/sink/sink.go
Original file line number Diff line number Diff line change
Expand Up @@ -60,7 +60,7 @@ var sinkIniterMap = make(map[string]sinkInitFunc)
type sinkInitFunc func(context.Context, model.ChangeFeedID, *url.URL, *filter.Filter, *config.ReplicaConfig, map[string]string, chan error) (Sink, error)

func init() {
// register blockhole sink
// register blackhole sink
sinkIniterMap["blackhole"] = func(ctx context.Context, changefeedID model.ChangeFeedID, sinkURI *url.URL,
filter *filter.Filter, config *config.ReplicaConfig, opts map[string]string, errCh chan error) (Sink, error) {
return newBlackHoleSink(ctx, opts), nil
Expand Down
13 changes: 9 additions & 4 deletions errors.toml
Original file line number Diff line number Diff line change
Expand Up @@ -11,7 +11,7 @@ error = '''
stop processor by admin command
'''

["CDC:ErrAsyncBroadcaseNotSupport"]
["CDC:ErrAsyncBroadcastNotSupport"]
error = '''
Async broadcasts not supported
'''
Expand Down Expand Up @@ -156,6 +156,11 @@ error = '''
decode row data to datum failed
'''

["CDC:ErrEmitCheckpointTsFailed"]
error = '''
emit checkpoint ts failed
'''

["CDC:ErrEncodeFailed"]
error = '''
encode failed: %s
Expand Down Expand Up @@ -361,7 +366,7 @@ error = '''
kafka async send message failed
'''

["CDC:ErrKafkaFlushUnfished"]
["CDC:ErrKafkaFlushUnfinished"]
error = '''
flush not finished before producer close
'''
Expand Down Expand Up @@ -601,7 +606,7 @@ error = '''
resolve locks failed
'''

["CDC:ErrS3SinkInitialzie"]
["CDC:ErrS3SinkInitialize"]
error = '''
new s3 sink
'''
Expand Down Expand Up @@ -738,7 +743,7 @@ unified sorter backend is terminating

["CDC:ErrUnifiedSorterIOError"]
error = '''
unified sorter IO error
unified sorter IO error. Make sure your sort-dir is configured correctly by passing a valid argument or toml file to `cdc server`, or if you use TiUP, review the settings in `tiup cluster edit-config`. Details: %s
'''

["CDC:ErrUnknownKVEventType"]
Expand Down
7 changes: 4 additions & 3 deletions pkg/errors/errors.go
Original file line number Diff line number Diff line change
Expand Up @@ -68,10 +68,11 @@ var (

// sink related errors
ErrExecDDLFailed = errors.Normalize("exec DDL failed", errors.RFCCodeText("CDC:ErrExecDDLFailed"))
ErrEmitCheckpointTsFailed = errors.Normalize("emit checkpoint ts failed", errors.RFCCodeText("CDC:ErrEmitCheckpointTsFailed"))
ErrDDLEventIgnored = errors.Normalize("ddl event is ignored", errors.RFCCodeText("CDC:ErrDDLEventIgnored"))
ErrKafkaSendMessage = errors.Normalize("kafka send message failed", errors.RFCCodeText("CDC:ErrKafkaSendMessage"))
ErrKafkaAsyncSendMessage = errors.Normalize("kafka async send message failed", errors.RFCCodeText("CDC:ErrKafkaAsyncSendMessage"))
ErrKafkaFlushUnfished = errors.Normalize("flush not finished before producer close", errors.RFCCodeText("CDC:ErrKafkaFlushUnfished"))
ErrKafkaFlushUnfinished = errors.Normalize("flush not finished before producer close", errors.RFCCodeText("CDC:ErrKafkaFlushUnfinished"))
ErrKafkaInvalidPartitionNum = errors.Normalize("invalid partition num %d", errors.RFCCodeText("CDC:ErrKafkaInvalidPartitionNum"))
ErrKafkaNewSaramaProducer = errors.Normalize("new sarama producer", errors.RFCCodeText("CDC:ErrKafkaNewSaramaProducer"))
ErrKafkaInvalidClientID = errors.Normalize("invalid kafka client ID '%s'", errors.RFCCodeText("CDC:ErrKafkaInvalidClientID"))
Expand All @@ -82,10 +83,10 @@ var (
ErrFileSinkFileOp = errors.Normalize("file sink file operation", errors.RFCCodeText("CDC:ErrFileSinkFileOp"))
ErrFileSinkMetaAlreadyExists = errors.Normalize("file sink meta file already exists", errors.RFCCodeText("CDC:ErrFileSinkMetaAlreadyExists"))
ErrS3SinkWriteStorage = errors.Normalize("write to storage", errors.RFCCodeText("CDC:ErrS3SinkWriteStorage"))
ErrS3SinkInitialzie = errors.Normalize("new s3 sink", errors.RFCCodeText("CDC:ErrS3SinkInitialzie"))
ErrS3SinkInitialize = errors.Normalize("new s3 sink", errors.RFCCodeText("CDC:ErrS3SinkInitialize"))
ErrS3SinkStorageAPI = errors.Normalize("s3 sink storage api", errors.RFCCodeText("CDC:ErrS3SinkStorageAPI"))
ErrPrepareAvroFailed = errors.Normalize("prepare avro failed", errors.RFCCodeText("CDC:ErrPrepareAvroFailed"))
ErrAsyncBroadcaseNotSupport = errors.Normalize("Async broadcasts not supported", errors.RFCCodeText("CDC:ErrAsyncBroadcaseNotSupport"))
ErrAsyncBroadcastNotSupport = errors.Normalize("Async broadcasts not supported", errors.RFCCodeText("CDC:ErrAsyncBroadcastNotSupport"))
ErrKafkaInvalidConfig = errors.Normalize("kafka config invalid", errors.RFCCodeText("CDC:ErrKafkaInvalidConfig"))
ErrSinkURIInvalid = errors.Normalize("sink uri invalid", errors.RFCCodeText("CDC:ErrSinkURIInvalid"))
ErrMySQLTxnError = errors.Normalize("MySQL txn error", errors.RFCCodeText("CDC:ErrMySQLTxnError"))
Expand Down