diff --git a/Makefile b/Makefile index ae37c21796d..586935fbbf9 100644 --- a/Makefile +++ b/Makefile @@ -172,7 +172,7 @@ check: check-copyright fmt lint check-static tidy errdoc check-leaktest-added coverage: GO111MODULE=off go get github.com/wadey/gocovmerge - gocovmerge "$(TEST_DIR)"/cov.* | grep -vE ".*.pb.go|$(CDC_PKG)/testing_utils/.*|$(CDC_PKG)/cdc/kv/testing.go|$(CDC_PKG)/cdc/sink/simple_mysql_tester.go|.*.__failpoint_binding__.go" > "$(TEST_DIR)/all_cov.out" + gocovmerge "$(TEST_DIR)"/cov.* | grep -vE ".*.pb.go|$(CDC_PKG)/testing_utils/.*|$(CDC_PKG)/cdc/kv/testing.go|$(CDC_PKG)/cdc/entry/schema_test_helper.go|$(CDC_PKG)/cdc/sink/simple_mysql_tester.go|.*.__failpoint_binding__.go" > "$(TEST_DIR)/all_cov.out" grep -vE ".*.pb.go|$(CDC_PKG)/testing_utils/.*|$(CDC_PKG)/cdc/kv/testing.go|$(CDC_PKG)/cdc/sink/simple_mysql_tester.go|.*.__failpoint_binding__.go" "$(TEST_DIR)/cov.unit.out" > "$(TEST_DIR)/unit_cov.out" ifeq ("$(JenkinsCI)", "1") GO111MODULE=off go get github.com/mattn/goveralls diff --git a/README.md b/README.md index 4a0fbc20f5d..11a7423a12b 100644 --- a/README.md +++ b/README.md @@ -24,10 +24,6 @@ See a detailed introduction to [the TiCDC architecture](https://docs.pingcap.com - [English](https://pingcap.com/blog/) - [Chinese](https://pingcap.com/blog-cn/) -## TiDB Monthly - -[TiDB Monthly](https://pingcap.com/weekly/) - ## Building To check the source code, run test cases and build binaries, you can simply run: diff --git a/cdc/capture.go b/cdc/capture.go index 2ffef74e42a..e0be0c7466d 100644 --- a/cdc/capture.go +++ b/cdc/capture.go @@ -18,8 +18,6 @@ import ( "sync" "time" - "github.com/pingcap/ticdc/pkg/version" - "github.com/google/uuid" "github.com/pingcap/errors" "github.com/pingcap/failpoint" @@ -28,10 +26,13 @@ import ( "github.com/pingcap/ticdc/cdc/model" "github.com/pingcap/ticdc/cdc/processor" "github.com/pingcap/ticdc/pkg/config" + cdcContext "github.com/pingcap/ticdc/pkg/context" cerror "github.com/pingcap/ticdc/pkg/errors" "github.com/pingcap/ticdc/pkg/orchestrator" "github.com/pingcap/ticdc/pkg/security" "github.com/pingcap/ticdc/pkg/util" + "github.com/pingcap/ticdc/pkg/version" + tidbkv "github.com/pingcap/tidb/kv" pd "github.com/tikv/pd/client" "go.etcd.io/etcd/clientv3" "go.etcd.io/etcd/clientv3/concurrency" @@ -43,16 +44,11 @@ import ( "google.golang.org/grpc/backoff" ) -// captureOpts records options for capture -type captureOpts struct { - flushCheckpointInterval time.Duration - captureSessionTTL int -} - // Capture represents a Capture server, it monitors the changefeed information in etcd and schedules Task on it. type Capture struct { etcdClient kv.CDCEtcdClient pdCli pd.Client + kvStorage tidbkv.Storage credential *security.Credential processorManager *processor.Manager @@ -66,19 +62,18 @@ type Capture struct { session *concurrency.Session election *concurrency.Election - opts *captureOpts closed chan struct{} } // NewCapture returns a new Capture instance func NewCapture( - ctx context.Context, + stdCtx context.Context, pdEndpoints []string, pdCli pd.Client, - credential *security.Credential, - advertiseAddr string, - opts *captureOpts, + kvStorage tidbkv.Storage, ) (c *Capture, err error) { + conf := config.GetGlobalServerConfig() + credential := conf.Security tlsConfig, err := credential.ToTLSConfig() if err != nil { return nil, errors.Trace(err) @@ -92,7 +87,7 @@ func NewCapture( etcdCli, err := clientv3.New(clientv3.Config{ Endpoints: pdEndpoints, TLS: tlsConfig, - Context: ctx, + Context: stdCtx, LogConfig: &logConfig, DialTimeout: 5 * time.Second, DialOptions: []grpc.DialOption{ @@ -113,20 +108,20 @@ func NewCapture( return nil, errors.Annotate(cerror.WrapError(cerror.ErrNewCaptureFailed, err), "new etcd client") } sess, err := concurrency.NewSession(etcdCli, - concurrency.WithTTL(opts.captureSessionTTL)) + concurrency.WithTTL(conf.CaptureSessionTTL)) if err != nil { return nil, errors.Annotate(cerror.WrapError(cerror.ErrNewCaptureFailed, err), "create capture session") } elec := concurrency.NewElection(sess, kv.CaptureOwnerKey) - cli := kv.NewCDCEtcdClient(ctx, etcdCli) + cli := kv.NewCDCEtcdClient(stdCtx, etcdCli) id := uuid.New().String() info := &model.CaptureInfo{ ID: id, - AdvertiseAddr: advertiseAddr, + AdvertiseAddr: conf.AdvertiseAddr, Version: version.ReleaseVersion, } - processorManager := processor.NewManager(pdCli, credential, info) - log.Info("creating capture", zap.String("capture-id", id), util.ZapFieldCapture(ctx)) + processorManager := processor.NewManager() + log.Info("creating capture", zap.String("capture-id", id), util.ZapFieldCapture(stdCtx)) c = &Capture{ processors: make(map[string]*oldProcessor), @@ -135,8 +130,8 @@ func NewCapture( session: sess, election: elec, info: info, - opts: opts, pdCli: pdCli, + kvStorage: kvStorage, processorManager: processorManager, closed: make(chan struct{}), } @@ -150,13 +145,19 @@ func (c *Capture) Run(ctx context.Context) (err error) { // TODO: we'd better to add some wait mechanism to ensure no routine is blocked defer cancel() defer close(c.closed) + + ctx = cdcContext.NewContext(ctx, &cdcContext.GlobalVars{ + PDClient: c.pdCli, + KVStorage: c.kvStorage, + CaptureInfo: c.info, + }) err = c.register(ctx) if err != nil { return errors.Trace(err) } if config.NewReplicaImpl { sessionCli := c.session.Client() - etcdWorker, err := orchestrator.NewEtcdWorker(kv.NewCDCEtcdClient(ctx, sessionCli).Client, kv.EtcdKeyBase, c.processorManager, processor.NewGlobalState(c.info.ID)) + etcdWorker, err := orchestrator.NewEtcdWorker(kv.NewCDCEtcdClient(ctx, sessionCli).Client, kv.EtcdKeyBase, c.processorManager, model.NewGlobalState()) if err != nil { return errors.Trace(err) } @@ -306,9 +307,9 @@ func (c *Capture) assignTask(ctx context.Context, task *Task) (*oldProcessor, er log.Info("run processor", zap.String("capture-id", c.info.ID), util.ZapFieldCapture(ctx), zap.String("changefeed", task.ChangeFeedID)) - + conf := config.GetGlobalServerConfig() p, err := runProcessorImpl( - ctx, c.pdCli, c.credential, c.session, *cf, task.ChangeFeedID, *c.info, task.CheckpointTS, c.opts.flushCheckpointInterval) + ctx, c.pdCli, c.credential, c.session, *cf, task.ChangeFeedID, *c.info, task.CheckpointTS, time.Duration(conf.ProcessorFlushInterval)) if err != nil { log.Error("run processor failed", zap.String("changefeed", task.ChangeFeedID), diff --git a/cdc/capture/capture.go b/cdc/capture/capture.go new file mode 100644 index 00000000000..41d9e5fec8b --- /dev/null +++ b/cdc/capture/capture.go @@ -0,0 +1,360 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package capture + +import ( + "context" + "fmt" + "io" + "sync" + "time" + + "github.com/google/uuid" + "github.com/pingcap/errors" + "github.com/pingcap/failpoint" + "github.com/pingcap/log" + "github.com/pingcap/ticdc/cdc/kv" + "github.com/pingcap/ticdc/cdc/model" + "github.com/pingcap/ticdc/cdc/owner" + "github.com/pingcap/ticdc/cdc/processor" + "github.com/pingcap/ticdc/pkg/config" + cdcContext "github.com/pingcap/ticdc/pkg/context" + cerror "github.com/pingcap/ticdc/pkg/errors" + "github.com/pingcap/ticdc/pkg/orchestrator" + "github.com/pingcap/ticdc/pkg/version" + tidbkv "github.com/pingcap/tidb/kv" + pd "github.com/tikv/pd/client" + "go.etcd.io/etcd/clientv3/concurrency" + "go.etcd.io/etcd/mvcc" + "go.uber.org/zap" + "golang.org/x/time/rate" +) + +// Capture represents a Capture server, it monitors the changefeed information in etcd and schedules Task on it. +type Capture struct { + captureMu sync.Mutex + info *model.CaptureInfo + + ownerMu sync.Mutex + owner *owner.Owner + processorManager *processor.Manager + + // session keeps alive between the capture and etcd + session *concurrency.Session + election *concurrency.Election + + pdClient pd.Client + kvStorage tidbkv.Storage + etcdClient *kv.CDCEtcdClient + + cancel context.CancelFunc + + newProcessorManager func() *processor.Manager + newOwner func() *owner.Owner +} + +// NewCapture returns a new Capture instance +func NewCapture(pdClient pd.Client, kvStorage tidbkv.Storage, etcdClient *kv.CDCEtcdClient) *Capture { + return &Capture{ + pdClient: pdClient, + kvStorage: kvStorage, + etcdClient: etcdClient, + cancel: func() {}, + + newProcessorManager: processor.NewManager, + newOwner: owner.NewOwner, + } +} + +func (c *Capture) reset() error { + c.captureMu.Lock() + defer c.captureMu.Unlock() + conf := config.GetGlobalServerConfig() + c.info = &model.CaptureInfo{ + ID: uuid.New().String(), + AdvertiseAddr: conf.AdvertiseAddr, + Version: version.ReleaseVersion, + } + c.processorManager = c.newProcessorManager() + if c.session != nil { + c.session.Close() //nolint:errcheck + } + sess, err := concurrency.NewSession(c.etcdClient.Client.Unwrap(), + concurrency.WithTTL(conf.CaptureSessionTTL)) + if err != nil { + return errors.Annotate(cerror.WrapError(cerror.ErrNewCaptureFailed, err), "create capture session") + } + c.session = sess + c.election = concurrency.NewElection(sess, kv.CaptureOwnerKey) + log.Info("init capture", zap.String("capture-id", c.info.ID), zap.String("capture-addr", c.info.AdvertiseAddr)) + return nil +} + +// Run runs the capture +func (c *Capture) Run(ctx context.Context) error { + defer log.Info("the capture routine has exited") + // Limit the frequency of reset capture to avoid frequent recreating of resources + rl := rate.NewLimiter(0.05, 2) + for { + select { + case <-ctx.Done(): + return nil + default: + } + ctx, cancel := context.WithCancel(ctx) + c.cancel = cancel + err := rl.Wait(ctx) + if err != nil { + if errors.Cause(err) == context.Canceled { + return nil + } + return errors.Trace(err) + } + err = c.reset() + if err != nil { + return errors.Trace(err) + } + err = c.run(ctx) + // if capture suicided, reset the capture and run again. + // if the canceled error throw, there are two possible scenarios: + // 1. the internal context canceled, it means some error happened in the internal, and the routine is exited, we should restart the capture + // 2. the parent context canceled, it means that the caller of the capture hope the capture to exit, and this loop will return in the above `select` block + // TODO: make sure the internal cancel should return the real error instead of context.Canceled + if cerror.ErrCaptureSuicide.Equal(err) || context.Canceled == errors.Cause(err) { + log.Info("capture recovered", zap.String("capture-id", c.info.ID)) + continue + } + return errors.Trace(err) + } +} + +func (c *Capture) run(stdCtx context.Context) error { + ctx := cdcContext.NewContext(stdCtx, &cdcContext.GlobalVars{ + PDClient: c.pdClient, + KVStorage: c.kvStorage, + CaptureInfo: c.info, + EtcdClient: c.etcdClient, + }) + err := c.register(ctx) + if err != nil { + return errors.Trace(err) + } + defer func() { + timeoutCtx, cancel := context.WithTimeout(context.Background(), 5*time.Second) + if err := ctx.GlobalVars().EtcdClient.DeleteCaptureInfo(timeoutCtx, c.info.ID); err != nil { + log.Warn("failed to delete capture info when capture exited", zap.Error(err)) + } + cancel() + }() + wg := new(sync.WaitGroup) + wg.Add(2) + var ownerErr, processorErr error + go func() { + defer wg.Done() + defer c.AsyncClose() + // when the campaignOwner returns an error, it means that the the owner throws an unrecoverable serious errors + // (recoverable errors are intercepted in the owner tick) + // so we should also stop the processor and let capture restart or exit + ownerErr = c.campaignOwner(ctx) + log.Info("the owner routine has exited", zap.Error(ownerErr)) + }() + go func() { + defer wg.Done() + defer c.AsyncClose() + conf := config.GetGlobalServerConfig() + processorFlushInterval := time.Duration(conf.ProcessorFlushInterval) + // when the etcd worker of processor returns an error, it means that the the processor throws an unrecoverable serious errors + // (recoverable errors are intercepted in the processor tick) + // so we should also stop the owner and let capture restart or exit + processorErr = c.runEtcdWorker(ctx, c.processorManager, model.NewGlobalState(), processorFlushInterval) + log.Info("the processor routine has exited", zap.Error(processorErr)) + }() + wg.Wait() + if ownerErr != nil { + return errors.Annotate(ownerErr, "owner exited with error") + } + if processorErr != nil { + return errors.Annotate(processorErr, "processor exited with error") + } + return nil +} + +// Info gets the capture info +func (c *Capture) Info() model.CaptureInfo { + c.captureMu.Lock() + defer c.captureMu.Unlock() + return *c.info +} + +func (c *Capture) campaignOwner(ctx cdcContext.Context) error { + // In most failure cases, we don't return error directly, just run another + // campaign loop. We treat campaign loop as a special background routine. + conf := config.GetGlobalServerConfig() + ownerFlushInterval := time.Duration(conf.OwnerFlushInterval) + failpoint.Inject("ownerFlushIntervalInject", func(val failpoint.Value) { + ownerFlushInterval = time.Millisecond * time.Duration(val.(int)) + }) + // Limit the frequency of elections to avoid putting too much pressure on the etcd server + rl := rate.NewLimiter(0.05, 2) + for { + select { + case <-ctx.Done(): + return nil + default: + } + err := rl.Wait(ctx) + if err != nil { + if errors.Cause(err) == context.Canceled { + return nil + } + return errors.Trace(err) + } + // Campaign to be an owner, it blocks until it becomes the owner + if err := c.campaign(ctx); err != nil { + switch errors.Cause(err) { + case context.Canceled: + return nil + case mvcc.ErrCompacted: + // the revision we requested is compacted, just retry + continue + } + log.Warn("campaign owner failed", zap.Error(err)) + // if campaign owner failed, restart capture + return cerror.ErrCaptureSuicide.GenWithStackByArgs() + } + + log.Info("campaign owner successfully", zap.String("capture-id", c.info.ID)) + owner := c.newOwner() + c.setOwner(owner) + err = c.runEtcdWorker(ctx, owner, model.NewGlobalState(), ownerFlushInterval) + c.setOwner(nil) + log.Info("run owner exited", zap.Error(err)) + // if owner exits, resign the owner key + if resignErr := c.resign(ctx); resignErr != nil { + // if resigning owner failed, return error to let capture exits + return errors.Annotatef(resignErr, "resign owner failed, capture: %s", c.info.ID) + } + if err != nil { + // for errors, return error and let capture exits or restart + return errors.Trace(err) + } + // if owner exits normally, continue the campaign loop and try to election owner again + } +} + +func (c *Capture) runEtcdWorker(ctx cdcContext.Context, reactor orchestrator.Reactor, reactorState orchestrator.ReactorState, timerInterval time.Duration) error { + etcdWorker, err := orchestrator.NewEtcdWorker(ctx.GlobalVars().EtcdClient.Client, kv.EtcdKeyBase, reactor, reactorState) + if err != nil { + return errors.Trace(err) + } + if err := etcdWorker.Run(ctx, c.session, timerInterval); err != nil { + // We check ttl of lease instead of check `session.Done`, because + // `session.Done` is only notified when etcd client establish a + // new keepalive request, there could be a time window as long as + // 1/3 of session ttl that `session.Done` can't be triggered even + // the lease is already revoked. + switch { + case cerror.ErrEtcdSessionDone.Equal(err), + cerror.ErrLeaseExpired.Equal(err): + return cerror.ErrCaptureSuicide.GenWithStackByArgs() + } + lease, inErr := ctx.GlobalVars().EtcdClient.Client.TimeToLive(ctx, c.session.Lease()) + if inErr != nil { + return cerror.WrapError(cerror.ErrPDEtcdAPIError, inErr) + } + if lease.TTL == int64(-1) { + log.Warn("session is disconnected", zap.Error(err)) + return cerror.ErrCaptureSuicide.GenWithStackByArgs() + } + return errors.Trace(err) + } + return nil +} + +func (c *Capture) setOwner(owner *owner.Owner) { + c.ownerMu.Lock() + defer c.ownerMu.Unlock() + c.owner = owner +} + +// OperateOwnerUnderLock operates the owner with lock +func (c *Capture) OperateOwnerUnderLock(fn func(*owner.Owner) error) error { + c.ownerMu.Lock() + defer c.ownerMu.Unlock() + if c.owner == nil { + return cerror.ErrNotOwner.GenWithStackByArgs() + } + return fn(c.owner) +} + +// Campaign to be an owner +func (c *Capture) campaign(ctx cdcContext.Context) error { + failpoint.Inject("capture-campaign-compacted-error", func() { + failpoint.Return(errors.Trace(mvcc.ErrCompacted)) + }) + return cerror.WrapError(cerror.ErrCaptureCampaignOwner, c.election.Campaign(ctx, c.info.ID)) +} + +// Resign lets a owner start a new election. +func (c *Capture) resign(ctx cdcContext.Context) error { + failpoint.Inject("capture-resign-failed", func() { + failpoint.Return(errors.New("capture resign failed")) + }) + return cerror.WrapError(cerror.ErrCaptureResignOwner, c.election.Resign(ctx)) +} + +// register registers the capture information in etcd +func (c *Capture) register(ctx cdcContext.Context) error { + err := ctx.GlobalVars().EtcdClient.PutCaptureInfo(ctx, c.info, c.session.Lease()) + if err != nil { + return cerror.WrapError(cerror.ErrCaptureRegister, err) + } + return nil +} + +// AsyncClose closes the capture by unregistering it from etcd +func (c *Capture) AsyncClose() { + defer c.cancel() + c.OperateOwnerUnderLock(func(o *owner.Owner) error { //nolint:errcheck + o.AsyncStop() + return nil + }) + c.captureMu.Lock() + defer c.captureMu.Unlock() + if c.processorManager != nil { + c.processorManager.AsyncClose() + } +} + +// WriteDebugInfo writes the debug info into writer. +func (c *Capture) WriteDebugInfo(w io.Writer) { + c.OperateOwnerUnderLock(func(o *owner.Owner) error { //nolint:errcheck + fmt.Fprintf(w, "\n\n*** owner info ***:\n\n") + o.WriteDebugInfo(w) + return nil + }) + c.captureMu.Lock() + defer c.captureMu.Unlock() + if c.processorManager != nil { + fmt.Fprintf(w, "\n\n*** processors info ***:\n\n") + c.processorManager.WriteDebugInfo(w) + } +} + +// IsOwner returns whether the capture is an owner +func (c *Capture) IsOwner() bool { + return c.OperateOwnerUnderLock(func(o *owner.Owner) error { + return nil + }) == nil +} diff --git a/cdc/capture_test.go b/cdc/capture_test.go index f3f6d53781e..b9c209a62f7 100644 --- a/cdc/capture_test.go +++ b/cdc/capture_test.go @@ -79,9 +79,7 @@ func (s *captureSuite) TestCaptureSuicide(c *check.C) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() - capture, err := NewCapture(ctx, []string{s.clientURL.String()}, nil, - &security.Credential{}, "127.0.0.1:12034", - &captureOpts{flushCheckpointInterval: time.Millisecond * 200}) + capture, err := NewCapture(ctx, []string{s.clientURL.String()}, nil, nil) c.Assert(err, check.IsNil) var wg sync.WaitGroup @@ -112,9 +110,7 @@ func (s *captureSuite) TestCaptureSessionDoneDuringHandleTask(c *check.C) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() - capture, err := NewCapture(ctx, []string{s.clientURL.String()}, nil, - &security.Credential{}, "127.0.0.1:12034", - &captureOpts{flushCheckpointInterval: time.Millisecond * 200}) + capture, err := NewCapture(ctx, []string{s.clientURL.String()}, nil, nil) c.Assert(err, check.IsNil) runProcessorCount := 0 diff --git a/cdc/changefeed.go b/cdc/changefeed.go index 8e7b262b901..ff3229058a4 100644 --- a/cdc/changefeed.go +++ b/cdc/changefeed.go @@ -678,6 +678,7 @@ func (c *changeFeed) handleDDL(ctx context.Context, captures map[string]*model.C log.Info("apply job", zap.Stringer("job", todoDDLJob), zap.String("schema", todoDDLJob.SchemaName), zap.String("query", todoDDLJob.Query), + zap.Uint64("start-ts", todoDDLJob.StartTS), zap.Uint64("ts", todoDDLJob.BinlogInfo.FinishedTS)) ddlEvent := new(model.DDLEvent) @@ -937,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) diff --git a/cdc/entry/schema_storage.go b/cdc/entry/schema_storage.go index b3d7da96831..e8f0505d4e1 100644 --- a/cdc/entry/schema_storage.go +++ b/cdc/entry/schema_storage.go @@ -100,6 +100,12 @@ func (s *SingleSchemaSnapshot) PreTableInfo(job *timodel.Job) (*model.TableInfo, // NewSingleSchemaSnapshotFromMeta creates a new single schema snapshot from a tidb meta func NewSingleSchemaSnapshotFromMeta(meta *timeta.Meta, currentTs uint64, explicitTables bool) (*SingleSchemaSnapshot, error) { + // meta is nil only in unit tests + if meta == nil { + snap := newEmptySchemaSnapshot(explicitTables) + snap.currentTs = currentTs + return snap, nil + } return newSchemaSnapshotFromMeta(meta, currentTs, explicitTables) } @@ -634,6 +640,12 @@ func (s *schemaSnapshot) CloneTables() map[model.TableID]model.TableName { return mp } +// Tables return a map between table id and table info +// the returned map must be READ-ONLY. Any modified of this map will lead to the internal state confusion in schema storage +func (s *schemaSnapshot) Tables() map[model.TableID]*model.TableInfo { + return s.tables +} + // SchemaStorage stores the schema information with multi-version type SchemaStorage interface { // GetSnapshot returns the snapshot which of ts is specified diff --git a/cdc/entry/schema_test_helper.go b/cdc/entry/schema_test_helper.go new file mode 100644 index 00000000000..a6e10532d83 --- /dev/null +++ b/cdc/entry/schema_test_helper.go @@ -0,0 +1,84 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package entry + +import ( + "github.com/pingcap/check" + timodel "github.com/pingcap/parser/model" + ticonfig "github.com/pingcap/tidb/config" + "github.com/pingcap/tidb/domain" + "github.com/pingcap/tidb/kv" + timeta "github.com/pingcap/tidb/meta" + "github.com/pingcap/tidb/session" + "github.com/pingcap/tidb/store/mockstore" + "github.com/pingcap/tidb/util/testkit" +) + +// SchemaTestHelper is a test helper for schema which creates an internal tidb instance to generate DDL jobs with meta information +type SchemaTestHelper struct { + c *check.C + tk *testkit.TestKit + storage kv.Storage + domain *domain.Domain +} + +// NewSchemaTestHelper creates a SchemaTestHelper +func NewSchemaTestHelper(c *check.C) *SchemaTestHelper { + store, err := mockstore.NewMockTikvStore() + c.Assert(err, check.IsNil) + ticonfig.UpdateGlobal(func(conf *ticonfig.Config) { + conf.AlterPrimaryKey = true + }) + session.SetSchemaLease(0) + session.DisableStats4Test() + domain, err := session.BootstrapSession(store) + c.Assert(err, check.IsNil) + domain.SetStatsUpdating(true) + tk := testkit.NewTestKit(c, store) + return &SchemaTestHelper{ + c: c, + tk: tk, + storage: store, + domain: domain, + } +} + +// DDL2Job executes the DDL stmt and returns the DDL job +func (s *SchemaTestHelper) DDL2Job(ddl string) *timodel.Job { + s.tk.MustExec(ddl) + jobs, err := s.GetCurrentMeta().GetLastNHistoryDDLJobs(1) + s.c.Assert(err, check.IsNil) + s.c.Assert(jobs, check.HasLen, 1) + return jobs[0] +} + +// Storage return the tikv storage +func (s *SchemaTestHelper) Storage() kv.Storage { + return s.storage +} + +// GetCurrentMeta return the current meta snapshot +func (s *SchemaTestHelper) GetCurrentMeta() *timeta.Meta { + ver, err := s.storage.CurrentVersion() + s.c.Assert(err, check.IsNil) + snap, err := s.storage.GetSnapshot(ver) + s.c.Assert(err, check.IsNil) + return timeta.NewSnapshotMeta(snap) +} + +// Close closes the helper +func (s *SchemaTestHelper) Close() { + s.domain.Close() + s.storage.Close() //nolint:errcheck +} diff --git a/cdc/http_handler.go b/cdc/http_handler.go index 83a2dd60129..5bba64fb534 100644 --- a/cdc/http_handler.go +++ b/cdc/http_handler.go @@ -19,10 +19,13 @@ import ( "io/ioutil" "net/http" "strconv" + "time" "github.com/pingcap/errors" "github.com/pingcap/log" "github.com/pingcap/ticdc/cdc/model" + "github.com/pingcap/ticdc/cdc/owner" + "github.com/pingcap/ticdc/pkg/config" cerror "github.com/pingcap/ticdc/pkg/errors" "github.com/pingcap/ticdc/pkg/logutil" "github.com/pingcap/tidb/store/tikv/oracle" @@ -73,6 +76,19 @@ func (s *Server) handleResignOwner(w http.ResponseWriter, req *http.Request) { writeError(w, http.StatusBadRequest, cerror.ErrSupportPostOnly.GenWithStackByArgs()) return } + if config.NewReplicaImpl { + if s.captureV2 == nil { + // for test only + handleOwnerResp(w, concurrency.ErrElectionNotLeader) + return + } + err := s.captureV2.OperateOwnerUnderLock(func(owner *owner.Owner) error { + owner.AsyncStop() + return nil + }) + handleOwnerResp(w, err) + return + } s.ownerLock.RLock() if s.owner == nil { handleOwnerResp(w, concurrency.ErrElectionNotLeader) @@ -103,12 +119,19 @@ func (s *Server) handleChangefeedAdmin(w http.ResponseWriter, req *http.Request) writeError(w, http.StatusBadRequest, cerror.ErrSupportPostOnly.GenWithStackByArgs()) return } - - s.ownerLock.RLock() - defer s.ownerLock.RUnlock() - if s.owner == nil { - handleOwnerResp(w, concurrency.ErrElectionNotLeader) - return + if !config.NewReplicaImpl { + s.ownerLock.RLock() + defer s.ownerLock.RUnlock() + if s.owner == nil { + handleOwnerResp(w, concurrency.ErrElectionNotLeader) + return + } + } else { + if s.captureV2 == nil { + // for test only + handleOwnerResp(w, concurrency.ErrElectionNotLeader) + return + } } err := req.ParseForm() @@ -137,7 +160,14 @@ func (s *Server) handleChangefeedAdmin(w http.ResponseWriter, req *http.Request) Type: model.AdminJobType(typ), Opts: opts, } - err = s.owner.EnqueueJob(job) + if config.NewReplicaImpl { + err = s.captureV2.OperateOwnerUnderLock(func(owner *owner.Owner) error { + owner.EnqueueJob(job) + return nil + }) + } else { + err = s.owner.EnqueueJob(job) + } handleOwnerResp(w, err) } @@ -146,12 +176,19 @@ func (s *Server) handleRebalanceTrigger(w http.ResponseWriter, req *http.Request writeError(w, http.StatusBadRequest, cerror.ErrSupportPostOnly.GenWithStackByArgs()) return } - - s.ownerLock.RLock() - defer s.ownerLock.RUnlock() - if s.owner == nil { - handleOwnerResp(w, concurrency.ErrElectionNotLeader) - return + if !config.NewReplicaImpl { + s.ownerLock.RLock() + defer s.ownerLock.RUnlock() + if s.owner == nil { + handleOwnerResp(w, concurrency.ErrElectionNotLeader) + return + } + } else { + if s.captureV2 == nil { + // for test only + handleOwnerResp(w, concurrency.ErrElectionNotLeader) + return + } } err := req.ParseForm() @@ -165,8 +202,15 @@ func (s *Server) handleRebalanceTrigger(w http.ResponseWriter, req *http.Request cerror.ErrAPIInvalidParam.GenWithStack("invalid changefeed id: %s", changefeedID)) return } - s.owner.TriggerRebalance(changefeedID) - handleOwnerResp(w, nil) + if config.NewReplicaImpl { + err = s.captureV2.OperateOwnerUnderLock(func(owner *owner.Owner) error { + owner.TriggerRebalance(changefeedID) + return nil + }) + } else { + s.owner.TriggerRebalance(changefeedID) + } + handleOwnerResp(w, err) } func (s *Server) handleMoveTable(w http.ResponseWriter, req *http.Request) { @@ -174,12 +218,19 @@ func (s *Server) handleMoveTable(w http.ResponseWriter, req *http.Request) { writeError(w, http.StatusBadRequest, cerror.ErrSupportPostOnly.GenWithStackByArgs()) return } - - s.ownerLock.RLock() - defer s.ownerLock.RUnlock() - if s.owner == nil { - handleOwnerResp(w, concurrency.ErrElectionNotLeader) - return + if !config.NewReplicaImpl { + s.ownerLock.RLock() + defer s.ownerLock.RUnlock() + if s.owner == nil { + handleOwnerResp(w, concurrency.ErrElectionNotLeader) + return + } + } else { + if s.captureV2 == nil { + // for test only + handleOwnerResp(w, concurrency.ErrElectionNotLeader) + return + } } err := req.ParseForm() @@ -206,8 +257,15 @@ func (s *Server) handleMoveTable(w http.ResponseWriter, req *http.Request) { cerror.ErrAPIInvalidParam.GenWithStack("invalid tableID: %s", tableIDStr)) return } - s.owner.ManualSchedule(changefeedID, to, tableID) - handleOwnerResp(w, nil) + if config.NewReplicaImpl { + err = s.captureV2.OperateOwnerUnderLock(func(owner *owner.Owner) error { + owner.ManualSchedule(changefeedID, to, tableID) + return nil + }) + } else { + s.owner.ManualSchedule(changefeedID, to, tableID) + } + handleOwnerResp(w, err) } func (s *Server) handleChangefeedQuery(w http.ResponseWriter, req *http.Request) { @@ -215,11 +273,19 @@ func (s *Server) handleChangefeedQuery(w http.ResponseWriter, req *http.Request) writeError(w, http.StatusBadRequest, cerror.ErrSupportPostOnly.GenWithStackByArgs()) return } - s.ownerLock.RLock() - defer s.ownerLock.RUnlock() - if s.owner == nil { - handleOwnerResp(w, concurrency.ErrElectionNotLeader) - return + if !config.NewReplicaImpl { + s.ownerLock.RLock() + defer s.ownerLock.RUnlock() + if s.owner == nil { + handleOwnerResp(w, concurrency.ErrElectionNotLeader) + return + } + } else { + if s.captureV2 == nil { + // for test only + handleOwnerResp(w, concurrency.ErrElectionNotLeader) + return + } } err := req.ParseForm() @@ -233,30 +299,44 @@ func (s *Server) handleChangefeedQuery(w http.ResponseWriter, req *http.Request) cerror.ErrAPIInvalidParam.GenWithStack("invalid changefeed id: %s", changefeedID)) return } - cf, status, feedState, err := s.owner.collectChangefeedInfo(req.Context(), changefeedID) + ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) + defer cancel() + cfInfo, err := s.etcdClient.GetChangeFeedInfo(ctx, changefeedID) if err != nil && cerror.ErrChangeFeedNotExists.NotEqual(err) { - writeInternalServerError(w, err) + writeError(w, http.StatusBadRequest, + cerror.ErrAPIInvalidParam.GenWithStack("invalid changefeed id: %s", changefeedID)) return } - feedInfo, err := s.owner.etcdClient.GetChangeFeedInfo(req.Context(), changefeedID) + cfStatus, _, err := s.etcdClient.GetChangeFeedStatus(ctx, changefeedID) if err != nil && cerror.ErrChangeFeedNotExists.NotEqual(err) { - writeInternalServerError(w, err) + writeError(w, http.StatusBadRequest, err) return } - resp := &ChangefeedResp{ - FeedState: string(feedState), + resp := &ChangefeedResp{} + if cfInfo != nil { + resp.FeedState = string(cfInfo.State) + resp.RunningError = cfInfo.Error } - if cf != nil { - resp.RunningError = cf.info.Error - } else if feedInfo != nil { - resp.RunningError = feedInfo.Error - } - if status != nil { - resp.TSO = status.CheckpointTs - tm := oracle.GetTimeFromTS(status.CheckpointTs) + if cfStatus != nil { + resp.TSO = cfStatus.CheckpointTs + tm := oracle.GetTimeFromTS(cfStatus.CheckpointTs) resp.Checkpoint = tm.Format("2006-01-02 15:04:05.000") } + if !config.NewReplicaImpl && cfStatus != nil { + switch cfStatus.AdminJobType { + case model.AdminNone, model.AdminResume: + if cfInfo != nil && cfInfo.Error != nil { + resp.FeedState = string(model.StateFailed) + } + case model.AdminStop: + resp.FeedState = string(model.StateStopped) + case model.AdminRemove: + resp.FeedState = string(model.StateRemoved) + case model.AdminFinish: + resp.FeedState = string(model.StateFinished) + } + } writeData(w, resp) } diff --git a/cdc/http_status.go b/cdc/http_status.go index c4f25a9bb09..ac58b399a12 100644 --- a/cdc/http_status.go +++ b/cdc/http_status.go @@ -99,7 +99,7 @@ type status struct { IsOwner bool `json:"is_owner"` } -func (s *Server) writeEtcdInfo(ctx context.Context, cli kv.CDCEtcdClient, w io.Writer) { +func (s *Server) writeEtcdInfo(ctx context.Context, cli *kv.CDCEtcdClient, w io.Writer) { resp, err := cli.Client.Get(ctx, kv.EtcdKeyBase, clientv3.WithPrefix()) if err != nil { fmt.Fprintf(w, "failed to get info: %s\n\n", err.Error()) @@ -112,6 +112,12 @@ func (s *Server) writeEtcdInfo(ctx context.Context, cli kv.CDCEtcdClient, w io.W } func (s *Server) handleDebugInfo(w http.ResponseWriter, req *http.Request) { + if config.NewReplicaImpl { + s.captureV2.WriteDebugInfo(w) + fmt.Fprintf(w, "\n\n*** etcd info ***:\n\n") + s.writeEtcdInfo(req.Context(), s.etcdClient, w) + return + } s.ownerLock.RLock() defer s.ownerLock.RUnlock() if s.owner != nil { @@ -130,17 +136,25 @@ func (s *Server) handleDebugInfo(w http.ResponseWriter, req *http.Request) { } fmt.Fprintf(w, "\n\n*** etcd info ***:\n\n") - s.writeEtcdInfo(req.Context(), s.capture.etcdClient, w) + s.writeEtcdInfo(req.Context(), &s.capture.etcdClient, w) } func (s *Server) handleStatus(w http.ResponseWriter, req *http.Request) { - s.ownerLock.RLock() - defer s.ownerLock.RUnlock() st := status{ Version: version.ReleaseVersion, GitHash: version.GitHash, Pid: os.Getpid(), } + if config.NewReplicaImpl { + if s.captureV2 != nil { + st.ID = s.captureV2.Info().ID + st.IsOwner = s.captureV2.IsOwner() + } + writeData(w, st) + return + } + s.ownerLock.RLock() + defer s.ownerLock.RUnlock() if s.capture != nil { st.ID = s.capture.info.ID } diff --git a/cdc/kv/client.go b/cdc/kv/client.go index b93cc8ea8ce..64a39ea76df 100644 --- a/cdc/kv/client.go +++ b/cdc/kv/client.go @@ -68,6 +68,9 @@ const ( // failed region will be reloaded via `BatchLoadRegionsWithKeyRange` API. So we // don't need to force reload region any more. regionScheduleReload = false + + // defines the scan region limit for each table + regionScanLimitPerTable = 6 ) // time interval to force kv client to terminate gRPC stream and reconnect @@ -92,6 +95,8 @@ var ( metricFeedDuplicateRequestCounter = eventFeedErrorCounter.WithLabelValues("DuplicateRequest") metricFeedUnknownErrorCounter = eventFeedErrorCounter.WithLabelValues("Unknown") metricFeedRPCCtxUnavailable = eventFeedErrorCounter.WithLabelValues("RPCCtxUnavailable") + metricStoreSendRequestErr = eventFeedErrorCounter.WithLabelValues("SendRequestToStore") + metricConnectToStoreErr = eventFeedErrorCounter.WithLabelValues("ConnectToStore") ) var ( @@ -115,9 +120,13 @@ func newSingleRegionInfo(verID tikv.RegionVerID, span regionspan.ComparableSpan, // happens, kv client needs to recover region request from singleRegionInfo func (s *singleRegionInfo) partialClone() singleRegionInfo { sri := singleRegionInfo{ - verID: s.verID, - span: s.span.Clone(), - ts: s.ts, + verID: s.verID, + span: s.span.Clone(), + ts: s.ts, + rpcCtx: &tikv.RPCContext{}, + } + if s.rpcCtx != nil { + sri.rpcCtx.Addr = s.rpcCtx.Addr } return sri } @@ -473,7 +482,7 @@ func (c *CDCClient) EventFeed( isPullerInit PullerInitialization, eventCh chan<- *model.RegionFeedEvent, ) error { - s := newEventFeedSession(c, c.regionCache, c.kvStorage, span, + s := newEventFeedSession(ctx, c, c.regionCache, c.kvStorage, span, lockResolver, isPullerInit, enableOldValue, ts, eventCh) return s.eventFeed(ctx, ts) @@ -503,6 +512,9 @@ type eventFeedSession struct { // The channel to send the processed events. eventCh chan<- *model.RegionFeedEvent + // The token based region router, it controls the uninitialzied regions with + // a given size limit. + regionRouter LimitRegionRouter // The channel to put the region that will be sent requests. regionCh chan singleRegionInfo // The channel to notify that an error is happening, so that the error will be handled and the affected region @@ -535,6 +547,7 @@ type rangeRequestTask struct { } func newEventFeedSession( + ctx context.Context, client *CDCClient, regionCache *tikv.RegionCache, kvStorage tikv.Storage, @@ -552,6 +565,7 @@ func newEventFeedSession( kvStorage: kvStorage, totalSpan: totalSpan, eventCh: eventCh, + regionRouter: NewSizedRegionRouter(ctx, regionScanLimitPerTable), regionCh: make(chan singleRegionInfo, 16), errCh: make(chan regionErrorInfo, 16), requestRangeCh: make(chan rangeRequestTask, 16), @@ -582,6 +596,10 @@ func (s *eventFeedSession) eventFeed(ctx context.Context, ts uint64) error { return s.dispatchRequest(ctx, g) }) + g.Go(func() error { + return s.requestRegionToStore(ctx, g) + }) + g.Go(func() error { for { select { @@ -619,6 +637,10 @@ func (s *eventFeedSession) eventFeed(ctx context.Context, ts uint64) error { } }) + g.Go(func() error { + return s.regionRouter.Run(ctx) + }) + s.requestRangeCh <- rangeRequestTask{span: s.totalSpan, ts: ts} s.rangeChSizeGauge.Inc() @@ -648,7 +670,6 @@ func (s *eventFeedSession) scheduleRegionRequest(ctx context.Context, sri single s.regionChSizeGauge.Inc() case <-ctx.Done(): } - case regionspan.LockRangeStatusStale: log.Info("request expired", zap.Uint64("regionID", sri.verID.GetID()), @@ -697,9 +718,12 @@ func (s *eventFeedSession) scheduleRegionRequest(ctx context.Context, sri single // onRegionFail handles a region's failure, which means, unlock the region's range and send the error to the errCh for // error handling. This function is non blocking even if error channel is full. // CAUTION: Note that this should only be called in a context that the region has locked it's range. -func (s *eventFeedSession) onRegionFail(ctx context.Context, errorInfo regionErrorInfo) error { +func (s *eventFeedSession) onRegionFail(ctx context.Context, errorInfo regionErrorInfo, revokeToken bool) error { log.Debug("region failed", zap.Uint64("regionID", errorInfo.verID.GetID()), zap.Error(errorInfo.err)) s.rangeLock.UnlockRange(errorInfo.span.Start, errorInfo.span.End, errorInfo.verID.GetID(), errorInfo.verID.GetVer(), errorInfo.ts) + if revokeToken { + s.regionRouter.Release(errorInfo.rpcCtx.Addr) + } select { case s.errCh <- errorInfo: s.errChSizeGauge.Inc() @@ -715,14 +739,13 @@ func (s *eventFeedSession) onRegionFail(ctx context.Context, errorInfo regionErr return nil } -// dispatchRequest manages a set of streams and dispatch event feed requests -// to these streams. Streams to each store will be created on need. After -// establishing new stream, a goroutine will be spawned to handle events from -// the stream. -// Regions from `regionCh` will be connected. If any error happens to a -// region, the error will be send to `errCh` and the receiver of `errCh` is -// responsible for handling the error. -func (s *eventFeedSession) dispatchRequest( +// requestRegionToStore gets singleRegionInfo from regionRouter, which is a token +// based limitter, sends request to TiKV. +// If the send request to TiKV returns error, fail the region with sendRequestToStoreErr +// and kv client will redispatch the region. +// If initialize gPRC stream with an error, fail the region with connectToStoreErr +// and kv client will also redispatch the region. +func (s *eventFeedSession) requestRegionToStore( ctx context.Context, g *errgroup.Group, ) error { @@ -731,179 +754,202 @@ func (s *eventFeedSession) dispatchRequest( // to pass the region info to the receiver since the region info cannot be inferred from the response from TiKV. storePendingRegions := make(map[string]*syncRegionFeedStateMap) -MainLoop: + var sri singleRegionInfo for { - // Note that when a region is received from the channel, it's range has been already locked. - var sri singleRegionInfo select { case <-ctx.Done(): - return ctx.Err() - case sri = <-s.regionCh: - s.regionChSizeGauge.Dec() + return errors.Trace(ctx.Err()) + case sri = <-s.regionRouter.Chan(): } + requestID := allocID() - log.Debug("dispatching region", zap.Uint64("regionID", sri.verID.GetID())) + extraOp := kvrpcpb.ExtraOp_Noop + if s.enableOldValue { + extraOp = kvrpcpb.ExtraOp_ReadOldValue + } - // Loop for retrying in case the stream has disconnected. - // TODO: Should we break if retries and fails too many times? - for { - rpcCtx, err := s.getRPCContextForRegion(ctx, sri.verID) - if err != nil { - return errors.Trace(err) + rpcCtx := sri.rpcCtx + regionID := rpcCtx.Meta.GetId() + req := &cdcpb.ChangeDataRequest{ + Header: &cdcpb.Header{ + ClusterId: s.client.clusterID, + TicdcVersion: version.ReleaseSemver(), + }, + RegionId: regionID, + RequestId: requestID, + RegionEpoch: rpcCtx.Meta.RegionEpoch, + CheckpointTs: sri.ts, + StartKey: sri.span.Start, + EndKey: sri.span.End, + ExtraOp: extraOp, + } + + failpoint.Inject("kvClientPendingRegionDelay", nil) + + // each TiKV store has an independent pendingRegions. + var pendingRegions *syncRegionFeedStateMap + + var err error + stream, ok := s.getStream(rpcCtx.Addr) + if ok { + var ok bool + pendingRegions, ok = storePendingRegions[rpcCtx.Addr] + if !ok { + // Should never happen + log.Panic("pending regions is not found for store", zap.String("store", rpcCtx.Addr)) } - if rpcCtx == nil { - // The region info is invalid. Retry the span. - log.Info("cannot get rpcCtx, retry span", + } else { + // when a new stream is established, always create a new pending + // regions map, the old map will be used in old `receiveFromStream` + // and won't be deleted until that goroutine exits. + pendingRegions = newSyncRegionFeedStateMap() + storePendingRegions[rpcCtx.Addr] = pendingRegions + storeID := rpcCtx.Peer.GetStoreId() + log.Info("creating new stream to store to send request", + zap.Uint64("regionID", sri.verID.GetID()), + zap.Uint64("requestID", requestID), + zap.Uint64("storeID", storeID), + zap.String("addr", rpcCtx.Addr)) + streamCtx, streamCancel := context.WithCancel(ctx) + _ = streamCancel // to avoid possible context leak warning from govet + stream, err = s.client.newStream(streamCtx, rpcCtx.Addr, storeID) + if err != nil { + // if get stream failed, maybe the store is down permanently, we should try to relocate the active store + log.Warn("get grpc stream client failed", zap.Uint64("regionID", sri.verID.GetID()), - zap.Stringer("span", sri.span)) + zap.Uint64("requestID", requestID), + zap.Uint64("storeID", storeID), + zap.String("error", err.Error())) + if cerror.ErrVersionIncompatible.Equal(err) { + // It often occurs on rolling update. Sleep 20s to reduce logs. + delay := 20 * time.Second + failpoint.Inject("kvClientDelayWhenIncompatible", func() { + delay = 100 * time.Millisecond + }) + time.Sleep(delay) + } + bo := tikv.NewBackoffer(ctx, tikvRequestMaxBackoff) + s.client.regionCache.OnSendFail(bo, rpcCtx, regionScheduleReload, err) err = s.onRegionFail(ctx, regionErrorInfo{ singleRegionInfo: sri, - err: &rpcCtxUnavailableErr{ - verID: sri.verID, - }, - }) + err: &connectToStoreErr{}, + }, false /* revokeToken */) if err != nil { return errors.Trace(err) } - continue MainLoop - } - sri.rpcCtx = rpcCtx - - requestID := allocID() - - extraOp := kvrpcpb.ExtraOp_Noop - if s.enableOldValue { - extraOp = kvrpcpb.ExtraOp_ReadOldValue + continue } + s.addStream(rpcCtx.Addr, stream, streamCancel) - regionID := rpcCtx.Meta.GetId() - req := &cdcpb.ChangeDataRequest{ - Header: &cdcpb.Header{ - ClusterId: s.client.clusterID, - TicdcVersion: version.ReleaseSemver(), - }, - RegionId: regionID, - RequestId: requestID, - RegionEpoch: rpcCtx.Meta.RegionEpoch, - CheckpointTs: sri.ts, - StartKey: sri.span.Start, - EndKey: sri.span.End, - ExtraOp: extraOp, - } + limiter := s.client.getRegionLimiter(regionID) + g.Go(func() error { + if !s.enableKVClientV2 { + return s.receiveFromStream(ctx, g, rpcCtx.Addr, getStoreID(rpcCtx), stream, pendingRegions, limiter) + } + return s.receiveFromStreamV2(ctx, g, rpcCtx.Addr, getStoreID(rpcCtx), stream, pendingRegions, limiter) + }) + } - failpoint.Inject("kvClientPendingRegionDelay", nil) + state := newRegionFeedState(sri, requestID) + pendingRegions.insert(requestID, state) - // each TiKV store has an independent pendingRegions. - var pendingRegions *syncRegionFeedStateMap + logReq := log.Debug + if s.isPullerInit.IsInitialized() { + logReq = log.Info + } + logReq("start new request", zap.Reflect("request", req), zap.String("addr", rpcCtx.Addr)) - stream, ok := s.getStream(rpcCtx.Addr) - if ok { - var ok bool - pendingRegions, ok = storePendingRegions[rpcCtx.Addr] - if !ok { - // Should never happen - log.Panic("pending regions is not found for store", zap.String("store", rpcCtx.Addr)) - } - } else { - // when a new stream is established, always create a new pending - // regions map, the old map will be used in old `receiveFromStream` - // and won't be deleted until that goroutine exits. - pendingRegions = newSyncRegionFeedStateMap() - storePendingRegions[rpcCtx.Addr] = pendingRegions - storeID := rpcCtx.Peer.GetStoreId() - log.Info("creating new stream to store to send request", - zap.Uint64("regionID", sri.verID.GetID()), - zap.Uint64("requestID", requestID), - zap.Uint64("storeID", storeID), - zap.String("addr", rpcCtx.Addr)) - streamCtx, streamCancel := context.WithCancel(ctx) - _ = streamCancel // to avoid possible context leak warning from govet - stream, err = s.client.newStream(streamCtx, rpcCtx.Addr, storeID) - if err != nil { - // if get stream failed, maybe the store is down permanently, we should try to relocate the active store - log.Warn("get grpc stream client failed", - zap.Uint64("regionID", sri.verID.GetID()), - zap.Uint64("requestID", requestID), - zap.Uint64("storeID", storeID), - zap.String("error", err.Error())) - if cerror.ErrVersionIncompatible.Equal(err) { - // It often occurs on rolling update. Sleep 20s to reduce logs. - delay := 20 * time.Second - failpoint.Inject("kvClientDelayWhenIncompatible", func() { - delay = 100 * time.Millisecond - }) - time.Sleep(delay) - } - bo := tikv.NewBackoffer(ctx, tikvRequestMaxBackoff) - s.client.regionCache.OnSendFail(bo, rpcCtx, regionScheduleReload, err) - continue - } - s.addStream(rpcCtx.Addr, stream, streamCancel) + err = stream.Send(req) - limiter := s.client.getRegionLimiter(regionID) - g.Go(func() error { - if !s.enableKVClientV2 { - return s.receiveFromStream(ctx, g, rpcCtx.Addr, getStoreID(rpcCtx), stream, pendingRegions, limiter) - } - return s.receiveFromStreamV2(ctx, g, rpcCtx.Addr, getStoreID(rpcCtx), stream, pendingRegions, limiter) - }) + // If Send error, the receiver should have received error too or will receive error soon. So we doesn't need + // to do extra work here. + if err != nil { + log.Error("send request to stream failed", + zap.String("addr", rpcCtx.Addr), + zap.Uint64("storeID", getStoreID(rpcCtx)), + zap.Uint64("regionID", sri.verID.GetID()), + zap.Uint64("requestID", requestID), + zap.Error(err)) + err1 := stream.CloseSend() + if err1 != nil { + log.Error("failed to close stream", zap.Error(err1)) } - - state := newRegionFeedState(sri, requestID) - pendingRegions.insert(requestID, state) - - logReq := log.Debug - if s.isPullerInit.IsInitialized() { - logReq = log.Info + // Delete the stream from the map so that the next time the store is accessed, the stream will be + // re-established. + s.deleteStream(rpcCtx.Addr) + // Delete `pendingRegions` from `storePendingRegions` so that the next time a region of this store is + // requested, it will create a new one. So if the `receiveFromStream` goroutine tries to stop all + // pending regions, the new pending regions that are requested after reconnecting won't be stopped + // incorrectly. + delete(storePendingRegions, rpcCtx.Addr) + + // Remove the region from pendingRegions. If it's already removed, it should be already retried by + // `receiveFromStream`, so no need to retry here. + _, ok := pendingRegions.take(requestID) + if !ok { + continue } - logReq("start new request", zap.Reflect("request", req), zap.String("addr", rpcCtx.Addr)) - err = stream.Send(req) - - // If Send error, the receiver should have received error too or will receive error soon. So we doesn't need - // to do extra work here. + // Wait for a while and retry sending the request + time.Sleep(time.Millisecond * time.Duration(rand.Intn(100))) + err = s.onRegionFail(ctx, regionErrorInfo{ + singleRegionInfo: sri, + err: &sendRequestToStoreErr{}, + }, false /* revokeToken */) if err != nil { + return errors.Trace(err) + } + } else { + s.regionRouter.Acquire(rpcCtx.Addr) + } + } +} - log.Error("send request to stream failed", - zap.String("addr", rpcCtx.Addr), - zap.Uint64("storeID", getStoreID(rpcCtx)), - zap.Uint64("regionID", sri.verID.GetID()), - zap.Uint64("requestID", requestID), - zap.Error(err)) - err1 := stream.CloseSend() - if err1 != nil { - log.Error("failed to close stream", zap.Error(err1)) - } - // Delete the stream from the map so that the next time the store is accessed, the stream will be - // re-established. - s.deleteStream(rpcCtx.Addr) - // Delete `pendingRegions` from `storePendingRegions` so that the next time a region of this store is - // requested, it will create a new one. So if the `receiveFromStream` goroutine tries to stop all - // pending regions, the new pending regions that are requested after reconnecting won't be stopped - // incorrectly. - delete(storePendingRegions, rpcCtx.Addr) - - // Remove the region from pendingRegions. If it's already removed, it should be already retried by - // `receiveFromStream`, so no need to retry here. - _, ok := pendingRegions.take(requestID) - if !ok { - break - } +// dispatchRequest manages a set of streams and dispatch event feed requests +// to these streams. Streams to each store will be created on need. After +// establishing new stream, a goroutine will be spawned to handle events from +// the stream. +// Regions from `regionCh` will be connected. If any error happens to a +// region, the error will be send to `errCh` and the receiver of `errCh` is +// responsible for handling the error. +func (s *eventFeedSession) dispatchRequest( + ctx context.Context, + g *errgroup.Group, +) error { + for { + // Note that when a region is received from the channel, it's range has been already locked. + var sri singleRegionInfo + select { + case <-ctx.Done(): + return ctx.Err() + case sri = <-s.regionCh: + s.regionChSizeGauge.Dec() + } - // Wait for a while and retry sending the request - time.Sleep(time.Millisecond * time.Duration(rand.Intn(100))) - // Break if ctx has been canceled. - select { - case <-ctx.Done(): - return ctx.Err() - default: - } + log.Debug("dispatching region", zap.Uint64("regionID", sri.verID.GetID())) - continue + rpcCtx, err := s.getRPCContextForRegion(ctx, sri.verID) + if err != nil { + return errors.Trace(err) + } + if rpcCtx == nil { + // The region info is invalid. Retry the span. + log.Info("cannot get rpcCtx, retry span", + zap.Uint64("regionID", sri.verID.GetID()), + zap.Stringer("span", sri.span)) + err = s.onRegionFail(ctx, regionErrorInfo{ + singleRegionInfo: sri, + err: &rpcCtxUnavailableErr{ + verID: sri.verID, + }, + }, false /* revokeToken */) + if err != nil { + return errors.Trace(err) } - - break + continue } + sri.rpcCtx = rpcCtx + s.regionRouter.AddRegion(sri) } } @@ -933,7 +979,8 @@ func (s *eventFeedSession) partialRegionFeed( }() ts := state.sri.ts - maxTs, err := s.singleEventFeed(ctx, state.sri.verID.GetID(), state.sri.span, state.sri.ts, receiver) + maxTs, initialized, err := s.singleEventFeed(ctx, state.sri.verID.GetID(), state.sri.span, + state.sri.ts, state.sri.rpcCtx.Addr, receiver) log.Debug("singleEventFeed quit") if err == nil || errors.Cause(err) == context.Canceled { @@ -994,10 +1041,11 @@ func (s *eventFeedSession) partialRegionFeed( } } + revokeToken := !initialized return s.onRegionFail(ctx, regionErrorInfo{ singleRegionInfo: state.sri, err: err, - }) + }, revokeToken) } // divideAndSendEventFeedToRegions split up the input span into spans aligned @@ -1114,6 +1162,10 @@ func (s *eventFeedSession) handleError(ctx context.Context, errInfo regionErrorI metricFeedRPCCtxUnavailable.Inc() s.scheduleDivideRegionAndRequest(ctx, errInfo.span, errInfo.ts) return nil + case *connectToStoreErr: + metricConnectToStoreErr.Inc() + case *sendRequestToStoreErr: + metricStoreSendRequestErr.Inc() default: bo := tikv.NewBackoffer(ctx, tikvRequestMaxBackoff) if errInfo.rpcCtx.Meta != nil { @@ -1157,7 +1209,7 @@ func (s *eventFeedSession) receiveFromStream( err := s.onRegionFail(ctx, regionErrorInfo{ singleRegionInfo: state.sri, err: cerror.ErrPendingRegionCancel.GenWithStackByArgs(), - }) + }, true /* revokeToken */) if err != nil { // The only possible is that the ctx is cancelled. Simply return. return @@ -1360,8 +1412,9 @@ func (s *eventFeedSession) singleEventFeed( regionID uint64, span regionspan.ComparableSpan, startTs uint64, + storeAddr string, receiverCh <-chan *regionEvent, -) (uint64, error) { +) (lastResolvedTs uint64, initialized bool, err error) { captureAddr := util.CaptureAddrFromCtx(ctx) changefeedID := util.ChangefeedIDFromCtx(ctx) metricEventSize := eventSize.WithLabelValues(captureAddr) @@ -1374,14 +1427,12 @@ func (s *eventFeedSession) singleEventFeed( metricSendEventCommitCounter := sendEventCounter.WithLabelValues("commit", captureAddr, changefeedID) metricSendEventCommittedCounter := sendEventCounter.WithLabelValues("committed", captureAddr, changefeedID) - initialized := false - matcher := newMatcher() advanceCheckTicker := time.NewTicker(time.Second * 5) defer advanceCheckTicker.Stop() lastReceivedEventTime := time.Now() startFeedTime := time.Now() - lastResolvedTs := startTs + lastResolvedTs = startTs handleResolvedTs := func(resolvedTs uint64) error { if !initialized { return nil @@ -1422,7 +1473,8 @@ func (s *eventFeedSession) singleEventFeed( }, }: case <-ctx.Done(): - return lastResolvedTs, errors.Trace(ctx.Err()) + err = errors.Trace(ctx.Err()) + return } resolveLockInterval := 20 * time.Second failpoint.Inject("kvClientResolveLockInterval", func(val failpoint.Value) { @@ -1437,7 +1489,8 @@ func (s *eventFeedSession) singleEventFeed( var ok bool select { case <-ctx.Done(): - return lastResolvedTs, ctx.Err() + err = errors.Trace(err) + return case <-advanceCheckTicker.C: if time.Since(startFeedTime) < resolveLockInterval { continue @@ -1453,7 +1506,8 @@ func (s *eventFeedSession) singleEventFeed( } if sinceLastEvent > reconnectInterval { log.Warn("kv client reconnect triggered", zap.Duration("duration", sinceLastEvent)) - return lastResolvedTs, errReconnect + err = errReconnect + return } version, err := s.kvStorage.(*StorageWithCurVersionCache).GetCachedCurrentVersion() if err != nil { @@ -1480,8 +1534,10 @@ func (s *eventFeedSession) singleEventFeed( if !ok || event == nil { log.Debug("singleEventFeed closed by error") - return lastResolvedTs, cerror.ErrEventFeedAborted.GenWithStackByArgs() + err = cerror.ErrEventFeedAborted.GenWithStackByArgs() + return } + var revent *model.RegionFeedEvent lastReceivedEventTime = time.Now() if event.changeEvent != nil { metricEventSize.Observe(float64(event.changeEvent.Event.Size())) @@ -1500,30 +1556,33 @@ func (s *eventFeedSession) singleEventFeed( switch entry.Type { case cdcpb.Event_INITIALIZED: if time.Since(startFeedTime) > 20*time.Second { - log.Warn("The time cost of initializing is too mush", + log.Warn("The time cost of initializing is too much", zap.Duration("timeCost", time.Since(startFeedTime)), zap.Uint64("regionID", regionID)) } metricPullEventInitializedCounter.Inc() initialized = true + s.regionRouter.Release(storeAddr) cachedEvents := matcher.matchCachedRow() for _, cachedEvent := range cachedEvents { - revent, err := assembleRowEvent(regionID, cachedEvent, s.enableOldValue) + revent, err = assembleRowEvent(regionID, cachedEvent, s.enableOldValue) if err != nil { - return lastResolvedTs, errors.Trace(err) + err = errors.Trace(err) + return } select { case s.eventCh <- revent: metricSendEventCommitCounter.Inc() case <-ctx.Done(): - return lastResolvedTs, errors.Trace(ctx.Err()) + err = errors.Trace(err) + return } } case cdcpb.Event_COMMITTED: metricPullEventCommittedCounter.Inc() - revent, err := assembleRowEvent(regionID, entry, s.enableOldValue) + revent, err = assembleRowEvent(regionID, entry, s.enableOldValue) if err != nil { - return lastResolvedTs, errors.Trace(err) + return } if entry.CommitTs <= lastResolvedTs { @@ -1532,13 +1591,15 @@ func (s *eventFeedSession) singleEventFeed( zap.Uint64("CommitTs", entry.CommitTs), zap.Uint64("resolvedTs", lastResolvedTs), zap.Uint64("regionID", regionID)) - return lastResolvedTs, errUnreachable + err = errUnreachable + return } select { case s.eventCh <- revent: metricSendEventCommittedCounter.Inc() case <-ctx.Done(): - return lastResolvedTs, errors.Trace(ctx.Err()) + err = errors.Trace(ctx.Err()) + return } case cdcpb.Event_PREWRITE: metricPullEventPrewriteCounter.Inc() @@ -1551,7 +1612,8 @@ func (s *eventFeedSession) singleEventFeed( zap.Uint64("CommitTs", entry.CommitTs), zap.Uint64("resolvedTs", lastResolvedTs), zap.Uint64("regionID", regionID)) - return lastResolvedTs, errUnreachable + err = errUnreachable + return } ok := matcher.matchRow(entry) if !ok { @@ -1559,19 +1621,21 @@ func (s *eventFeedSession) singleEventFeed( matcher.cacheCommitRow(entry) continue } - return lastResolvedTs, cerror.ErrPrewriteNotMatch.GenWithStackByArgs(entry.GetKey(), entry.GetStartTs()) + err = cerror.ErrPrewriteNotMatch.GenWithStackByArgs(entry.GetKey(), entry.GetStartTs()) + return } - revent, err := assembleRowEvent(regionID, entry, s.enableOldValue) + revent, err = assembleRowEvent(regionID, entry, s.enableOldValue) if err != nil { - return lastResolvedTs, errors.Trace(err) + return } select { case s.eventCh <- revent: metricSendEventCommitCounter.Inc() case <-ctx.Done(): - return lastResolvedTs, errors.Trace(ctx.Err()) + err = errors.Trace(ctx.Err()) + return } case cdcpb.Event_ROLLBACK: metricPullEventRollbackCounter.Inc() @@ -1581,17 +1645,18 @@ func (s *eventFeedSession) singleEventFeed( case *cdcpb.Event_Admin_: log.Info("receive admin event", zap.Stringer("event", event.changeEvent)) case *cdcpb.Event_Error: - return lastResolvedTs, cerror.WrapError(cerror.ErrEventFeedEventError, &eventError{err: x.Error}) + err = cerror.WrapError(cerror.ErrEventFeedEventError, &eventError{err: x.Error}) + return case *cdcpb.Event_ResolvedTs: - if err := handleResolvedTs(x.ResolvedTs); err != nil { - return lastResolvedTs, errors.Trace(err) + if err = handleResolvedTs(x.ResolvedTs); err != nil { + return } } } if event.resolvedTs != nil { - if err := handleResolvedTs(event.resolvedTs.Ts); err != nil { - return lastResolvedTs, errors.Trace(err) + if err = handleResolvedTs(event.resolvedTs.Ts); err != nil { + return } } } @@ -1675,6 +1740,14 @@ func (e *rpcCtxUnavailableErr) Error() string { e.verID.GetID(), e.verID.GetVer(), e.verID.GetConfVer()) } +type connectToStoreErr struct{} + +func (e *connectToStoreErr) Error() string { return "connect to store error" } + +type sendRequestToStoreErr struct{} + +func (e *sendRequestToStoreErr) Error() string { return "send request to store error" } + func getStoreID(rpcCtx *tikv.RPCContext) uint64 { if rpcCtx != nil && rpcCtx.Peer != nil { return rpcCtx.Peer.GetStoreId() diff --git a/cdc/kv/client_test.go b/cdc/kv/client_test.go index 3ad3a6d18c5..7cda292a153 100644 --- a/cdc/kv/client_test.go +++ b/cdc/kv/client_test.go @@ -1479,7 +1479,7 @@ ReceiveLoop: } } -// TestStreamSendWithErrorNormal mainly tests the scenario that the `Recv` call +// TestStreamRecvWithErrorNormal mainly tests the scenario that the `Recv` call // of a gPRC stream in kv client meets a **logical related** error, and kv client // logs the error and re-establish new request. func (s *etcdSuite) TestStreamRecvWithErrorNormal(c *check.C) { @@ -1497,7 +1497,7 @@ func (s *etcdSuite) TestStreamRecvWithErrorNormal(c *check.C) { s.testStreamRecvWithError(c, "1*return(\"injected stream recv error\")") } -// TestStreamSendWithErrorIOEOF mainly tests the scenario that the `Recv` call +// TestStreamRecvWithErrorIOEOF mainly tests the scenario that the `Recv` call // of a gPRC stream in kv client meets error io.EOF, and kv client logs the error // and re-establish new request func (s *etcdSuite) TestStreamRecvWithErrorIOEOF(c *check.C) { @@ -2422,7 +2422,7 @@ func (s *clientSuite) TestSingleRegionInfoClone(c *check.C) { c.Assert(sri.span.String(), check.Equals, "[61, 63)") c.Assert(sri2.ts, check.Equals, uint64(2000)) c.Assert(sri2.span.String(), check.Equals, "[61, 62)") - c.Assert(sri2.rpcCtx, check.IsNil) + c.Assert(sri2.rpcCtx, check.DeepEquals, &tikv.RPCContext{}) } // TestResolveLockNoCandidate tests the resolved ts manager can work normally @@ -3138,7 +3138,9 @@ func (s *etcdSuite) TestConcurrentProcessRangeRequest(c *check.C) { lockresolver := txnutil.NewLockerResolver(kvStorage.(tikv.Storage)) isPullInit := &mockPullerInit{} cdcClient := NewCDCClient(ctx, pdClient, kvStorage.(tikv.Storage), &security.Credential{}) - eventCh := make(chan *model.RegionFeedEvent, 10) + // The buffer size of event channel must be large enough because in the test + // case we send events first, and then retrive all events from this channel. + eventCh := make(chan *model.RegionFeedEvent, 100) wg.Add(1) go func() { err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("z")}, 100, false, lockresolver, isPullInit, eventCh) @@ -3159,10 +3161,29 @@ func (s *etcdSuite) TestConcurrentProcessRangeRequest(c *check.C) { } // wait for all regions requested from cdc kv client + // Since there exists incremental scan limit in kv client, we must wait for + // the ready region and send initialized event. + sent := make(map[uint64]bool, regionNum) err = retry.Run(time.Millisecond*200, 20, func() error { count := 0 - requestIDs.Range(func(_, _ interface{}) bool { + // send initialized event and a resolved ts event to each region + requestIDs.Range(func(key, value interface{}) bool { count++ + regionID := key.(uint64) + requestID := value.(uint64) + if _, ok := sent[regionID]; !ok { + initialized := mockInitializedEvent(regionID, requestID) + ch1 <- initialized + sent[regionID] = true + resolved := &cdcpb.ChangeDataEvent{Events: []*cdcpb.Event{ + { + RegionId: regionID, + RequestId: requestID, + Event: &cdcpb.Event_ResolvedTs{ResolvedTs: 120}, + }, + }} + ch1 <- resolved + } return true }) if count == regionNum { @@ -3172,23 +3193,6 @@ func (s *etcdSuite) TestConcurrentProcessRangeRequest(c *check.C) { }) c.Assert(err, check.IsNil) - // send initialized event and a resolved ts event to each region - requestIDs.Range(func(key, value interface{}) bool { - regionID := key.(uint64) - requestID := value.(uint64) - initialized := mockInitializedEvent(regionID, requestID) - ch1 <- initialized - resolved := &cdcpb.ChangeDataEvent{Events: []*cdcpb.Event{ - { - RegionId: regionID, - RequestId: requestID, - Event: &cdcpb.Event_ResolvedTs{ResolvedTs: 120}, - }, - }} - ch1 <- resolved - return true - }) - resolvedCount := 0 checkEvent: for { diff --git a/cdc/kv/client_v2.go b/cdc/kv/client_v2.go index 4eb18d121fa..3a792581471 100644 --- a/cdc/kv/client_v2.go +++ b/cdc/kv/client_v2.go @@ -178,7 +178,7 @@ func (s *eventFeedSession) receiveFromStreamV2( err := s.onRegionFail(ctx, regionErrorInfo{ singleRegionInfo: state.sri, err: cerror.ErrPendingRegionCancel.GenWithStackByArgs(), - }) + }, false /* initialized */) if err != nil { // The only possible is that the ctx is cancelled. Simply return. return diff --git a/cdc/kv/etcd.go b/cdc/kv/etcd.go index 19f44b3f2c9..b1a02a18152 100644 --- a/cdc/kv/etcd.go +++ b/cdc/kv/etcd.go @@ -18,6 +18,8 @@ import ( "fmt" "time" + "go.etcd.io/etcd/etcdserver/api/v3rpc/rpctypes" + "github.com/pingcap/errors" "github.com/pingcap/log" "github.com/pingcap/ticdc/cdc/model" @@ -31,6 +33,7 @@ import ( "go.etcd.io/etcd/embed" "go.etcd.io/etcd/mvcc/mvccpb" "go.uber.org/zap" + "google.golang.org/grpc/codes" ) const ( @@ -162,9 +165,13 @@ func (c CDCEtcdClient) GetAllCDCInfo(ctx context.Context) ([]*mvccpb.KeyValue, e func (c CDCEtcdClient) RevokeAllLeases(ctx context.Context, leases map[string]int64) error { for _, lease := range leases { _, err := c.Client.Revoke(ctx, clientv3.LeaseID(lease)) - if err != nil { - return cerror.WrapError(cerror.ErrPDEtcdAPIError, err) + if err == nil { + continue + } else if etcdErr := err.(rpctypes.EtcdError); etcdErr.Code() == codes.NotFound { + // it means the etcd lease is already expired or revoked + continue } + return cerror.WrapError(cerror.ErrPDEtcdAPIError, err) } return nil } diff --git a/cdc/kv/etcd_test.go b/cdc/kv/etcd_test.go index 5fe823ecc9f..f8ad520c1c4 100644 --- a/cdc/kv/etcd_test.go +++ b/cdc/kv/etcd_test.go @@ -527,6 +527,8 @@ func (s *etcdSuite) TestGetAllCaptureLeases(c *check.C) { c.Assert(err, check.IsNil) c.Check(queryLeases, check.DeepEquals, leases) + // make sure the RevokeAllLeases function can ignore the lease not exist + leases["/fake/capture/info"] = 200 err = s.client.RevokeAllLeases(ctx, leases) c.Assert(err, check.IsNil) queryLeases, err = s.client.GetCaptureLeases(ctx) diff --git a/cdc/kv/metrics.go b/cdc/kv/metrics.go index bacfe8d0489..b9fe36c12c3 100644 --- a/cdc/kv/metrics.go +++ b/cdc/kv/metrics.go @@ -67,6 +67,13 @@ var ( Name: "channel_size", Help: "size of each channel in kv client", }, []string{"id", "channel"}) + clientRegionTokenSize = prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Namespace: "ticdc", + Subsystem: "kvclient", + Name: "region_token", + Help: "size of region token in kv client", + }, []string{"store", "table", "changefeed"}) batchResolvedEventSize = prometheus.NewHistogramVec( prometheus.HistogramOpts{ Namespace: "ticdc", @@ -93,6 +100,7 @@ func InitMetrics(registry *prometheus.Registry) { registry.MustRegister(pullEventCounter) registry.MustRegister(sendEventCounter) registry.MustRegister(clientChannelSize) + registry.MustRegister(clientRegionTokenSize) registry.MustRegister(batchResolvedEventSize) registry.MustRegister(etcdRequestCounter) } diff --git a/cdc/kv/region_worker.go b/cdc/kv/region_worker.go index 8e2dd09c121..f71e0fb7bf1 100644 --- a/cdc/kv/region_worker.go +++ b/cdc/kv/region_worker.go @@ -199,10 +199,11 @@ func (w *regionWorker) handleSingleRegionError(ctx context.Context, err error, s } }) + revokeToken := !state.initialized return w.session.onRegionFail(ctx, regionErrorInfo{ singleRegionInfo: state.sri, err: err, - }) + }, revokeToken) } func (w *regionWorker) checkUnInitRegions(ctx context.Context) error { @@ -452,7 +453,7 @@ func (w *regionWorker) handleEventEntry( switch entry.Type { case cdcpb.Event_INITIALIZED: if time.Since(state.startFeedTime) > 20*time.Second { - log.Warn("The time cost of initializing is too mush", + log.Warn("The time cost of initializing is too much", zap.Duration("timeCost", time.Since(state.startFeedTime)), zap.Uint64("regionID", regionID)) } @@ -468,7 +469,7 @@ func (w *regionWorker) handleEventEntry( metricPullEventInitializedCounter.Inc() state.initialized = true - + w.session.regionRouter.Release(state.sri.rpcCtx.Addr) cachedEvents := state.matcher.matchCachedRow() for _, cachedEvent := range cachedEvents { revent, err := assembleRowEvent(regionID, cachedEvent, w.enableOldValue) @@ -605,13 +606,14 @@ func (w *regionWorker) evictAllRegions(ctx context.Context) error { if state.lastResolvedTs > singleRegionInfo.ts { singleRegionInfo.ts = state.lastResolvedTs } + revokeToken := !state.initialized state.lock.Unlock() err = w.session.onRegionFail(ctx, regionErrorInfo{ singleRegionInfo: singleRegionInfo, err: &rpcCtxUnavailableErr{ verID: singleRegionInfo.verID, }, - }) + }, revokeToken) return err == nil }) } diff --git a/cdc/kv/token_region.go b/cdc/kv/token_region.go new file mode 100644 index 00000000000..fee545ce375 --- /dev/null +++ b/cdc/kv/token_region.go @@ -0,0 +1,163 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package kv + +import ( + "context" + "sync" + "time" + + "github.com/pingcap/errors" + "github.com/pingcap/ticdc/pkg/util" + "github.com/prometheus/client_golang/prometheus" +) + +const ( + // buffer size for ranged region consumer + regionRouterChanSize = 16 + // sizedRegionRouter checks region buffer every 100ms + sizedRegionCheckInterval = 100 * time.Millisecond +) + +// LimitRegionRouter defines an interface that can buffer singleRegionInfo +// and provide token based consumption +type LimitRegionRouter interface { + // Chan returns a singleRegionInfo channel that can be consumed from + Chan() <-chan singleRegionInfo + // AddRegion adds an singleRegionInfo to buffer, this function is thread-safe + AddRegion(task singleRegionInfo) + // Acquire acquires one token + Acquire(id string) + // Release gives back one token, this function is thread-safe + Release(id string) + // Run runs in background and does some logic work + Run(ctx context.Context) error +} + +type srrMetrics struct { + changefeed string + table string + tokens map[string]prometheus.Gauge +} + +func newSrrMetrics(ctx context.Context) *srrMetrics { + changefeed := util.ChangefeedIDFromCtx(ctx) + _, table := util.TableIDFromCtx(ctx) + return &srrMetrics{ + changefeed: changefeed, + table: table, + tokens: make(map[string]prometheus.Gauge), + } +} + +type sizedRegionRouter struct { + buffer map[string][]singleRegionInfo + output chan singleRegionInfo + lock sync.Mutex + metrics *srrMetrics + tokens map[string]int + sizeLimit int +} + +// NewSizedRegionRouter creates a new sizedRegionRouter +func NewSizedRegionRouter(ctx context.Context, sizeLimit int) *sizedRegionRouter { + return &sizedRegionRouter{ + buffer: make(map[string][]singleRegionInfo), + output: make(chan singleRegionInfo, regionRouterChanSize), + sizeLimit: sizeLimit, + tokens: make(map[string]int), + metrics: newSrrMetrics(ctx), + } +} + +func (r *sizedRegionRouter) Chan() <-chan singleRegionInfo { + return r.output +} + +func (r *sizedRegionRouter) AddRegion(sri singleRegionInfo) { + r.lock.Lock() + var id string + // if rpcCtx is not provided, use the default "" bucket + if sri.rpcCtx != nil { + id = sri.rpcCtx.Addr + } + if r.sizeLimit > r.tokens[id] && len(r.output) < regionRouterChanSize { + r.output <- sri + } else { + r.buffer[id] = append(r.buffer[id], sri) + } + r.lock.Unlock() +} + +func (r *sizedRegionRouter) Acquire(id string) { + r.lock.Lock() + defer r.lock.Unlock() + r.tokens[id]++ + if _, ok := r.metrics.tokens[id]; !ok { + r.metrics.tokens[id] = clientRegionTokenSize.WithLabelValues(id, r.metrics.table, r.metrics.changefeed) + } + r.metrics.tokens[id].Inc() +} + +func (r *sizedRegionRouter) Release(id string) { + r.lock.Lock() + defer r.lock.Unlock() + r.tokens[id]-- + if _, ok := r.metrics.tokens[id]; !ok { + r.metrics.tokens[id] = clientRegionTokenSize.WithLabelValues(id, r.metrics.table, r.metrics.changefeed) + } + r.metrics.tokens[id].Dec() +} + +func (r *sizedRegionRouter) Run(ctx context.Context) error { + ticker := time.NewTicker(sizedRegionCheckInterval) + defer ticker.Stop() + for { + select { + case <-ctx.Done(): + return errors.Trace(ctx.Err()) + case <-ticker.C: + r.lock.Lock() + for id, buf := range r.buffer { + available := r.sizeLimit - r.tokens[id] + // the tokens used could be more then size limit, since we have + // a sized channel as level1 cache + if available <= 0 { + continue + } + if available > len(buf) { + available = len(buf) + } + // to avoid deadlock because when consuming from the output channel. + // onRegionFail could decrease tokens, which requires lock protection. + if available > regionRouterChanSize-len(r.output) { + available = regionRouterChanSize - len(r.output) + } + if available == 0 { + continue + } + for i := 0; i < available; i++ { + select { + case <-ctx.Done(): + r.lock.Unlock() + return errors.Trace(ctx.Err()) + case r.output <- buf[i]: + } + } + r.buffer[id] = r.buffer[id][available:] + } + r.lock.Unlock() + } + } +} diff --git a/cdc/kv/token_region_test.go b/cdc/kv/token_region_test.go new file mode 100644 index 00000000000..b26d9b02cd5 --- /dev/null +++ b/cdc/kv/token_region_test.go @@ -0,0 +1,181 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package kv + +import ( + "context" + "fmt" + "sync/atomic" + "time" + + "github.com/pingcap/check" + "github.com/pingcap/errors" + "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tidb/store/tikv" + "golang.org/x/sync/errgroup" +) + +type tokenRegionSuite struct { +} + +var _ = check.Suite(&tokenRegionSuite{}) + +func (s *tokenRegionSuite) TestRouter(c *check.C) { + defer testleak.AfterTest(c)() + store := "store-1" + limit := 10 + r := NewSizedRegionRouter(context.Background(), limit) + for i := 0; i < limit; i++ { + r.AddRegion(singleRegionInfo{ts: uint64(i), rpcCtx: &tikv.RPCContext{Addr: store}}) + } + regions := make([]singleRegionInfo, 0, limit) + // limit is less than regionScanLimitPerTable + for i := 0; i < limit; i++ { + select { + case sri := <-r.Chan(): + c.Assert(sri.ts, check.Equals, uint64(i)) + r.Acquire(store) + regions = append(regions, sri) + default: + c.Error("expect region info from router") + } + } + c.Assert(r.tokens[store], check.Equals, limit) + for range regions { + r.Release(store) + } + c.Assert(r.tokens[store], check.Equals, 0) +} + +func (s *tokenRegionSuite) TestRouterWithFastConsumer(c *check.C) { + defer testleak.AfterTest(c)() + s.testRouterWithConsumer(c, func() {}) +} + +func (s *tokenRegionSuite) TestRouterWithSlowConsumer(c *check.C) { + defer testleak.AfterTest(c)() + s.testRouterWithConsumer(c, func() { time.Sleep(time.Millisecond * 15) }) +} + +func (s *tokenRegionSuite) testRouterWithConsumer(c *check.C, funcDoSth func()) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + store := "store-1" + limit := 20 + r := NewSizedRegionRouter(context.Background(), limit) + for i := 0; i < limit*2; i++ { + r.AddRegion(singleRegionInfo{ts: uint64(i), rpcCtx: &tikv.RPCContext{Addr: store}}) + } + received := uint64(0) + for i := 0; i < regionRouterChanSize; i++ { + <-r.Chan() + atomic.AddUint64(&received, 1) + r.Acquire(store) + } + + wg, ctx := errgroup.WithContext(ctx) + wg.Go(func() error { + return r.Run(ctx) + }) + + wg.Go(func() error { + for i := 0; i < regionRouterChanSize; i++ { + r.Release(store) + } + return nil + }) + + wg.Go(func() error { + for { + select { + case <-ctx.Done(): + return ctx.Err() + case <-r.Chan(): + r.Acquire(store) + atomic.AddUint64(&received, 1) + r.Release(store) + funcDoSth() + if atomic.LoadUint64(&received) == uint64(limit*4) { + cancel() + } + } + } + }) + + for i := 0; i < limit*2; i++ { + r.AddRegion(singleRegionInfo{ts: uint64(i), rpcCtx: &tikv.RPCContext{Addr: store}}) + } + + err := wg.Wait() + c.Assert(errors.Cause(err), check.Equals, context.Canceled) + c.Assert(r.tokens[store], check.Equals, 0) +} + +func (s *tokenRegionSuite) TestRouterWithMultiStores(c *check.C) { + defer testleak.AfterTest(c)() + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + storeN := 10 + stores := make([]string, 0, storeN) + for i := 0; i < storeN; i++ { + stores = append(stores, fmt.Sprintf("store-%d", i)) + } + limit := 20 + r := NewSizedRegionRouter(context.Background(), limit) + + for _, store := range stores { + for j := 0; j < limit*2; j++ { + r.AddRegion(singleRegionInfo{ts: uint64(j), rpcCtx: &tikv.RPCContext{Addr: store}}) + } + } + received := uint64(0) + wg, ctx := errgroup.WithContext(ctx) + wg.Go(func() error { + return r.Run(ctx) + }) + + for _, store := range stores { + store := store + wg.Go(func() error { + for { + select { + case <-ctx.Done(): + return ctx.Err() + case <-r.Chan(): + r.Acquire(store) + atomic.AddUint64(&received, 1) + r.Release(store) + if atomic.LoadUint64(&received) == uint64(limit*4*storeN) { + cancel() + } + } + } + }) + } + + for _, store := range stores { + for i := 0; i < limit*2; i++ { + r.AddRegion(singleRegionInfo{ts: uint64(i), rpcCtx: &tikv.RPCContext{Addr: store}}) + } + } + + err := wg.Wait() + c.Assert(errors.Cause(err), check.Equals, context.Canceled) + for _, store := range stores { + c.Assert(r.tokens[store], check.Equals, 0) + } +} diff --git a/cdc/metrics.go b/cdc/metrics.go index cdc9fc8dade..2fde08e12d2 100644 --- a/cdc/metrics.go +++ b/cdc/metrics.go @@ -16,6 +16,7 @@ package cdc import ( "github.com/pingcap/ticdc/cdc/entry" "github.com/pingcap/ticdc/cdc/kv" + "github.com/pingcap/ticdc/cdc/owner" "github.com/pingcap/ticdc/cdc/processor" tablepipeline "github.com/pingcap/ticdc/cdc/processor/pipeline" "github.com/pingcap/ticdc/cdc/puller" @@ -39,9 +40,10 @@ func init() { if config.NewReplicaImpl { processor.InitMetrics(registry) tablepipeline.InitMetrics(registry) + owner.InitMetrics(registry) } else { initProcessorMetrics(registry) + initOwnerMetrics(registry) } - initOwnerMetrics(registry) initServerMetrics(registry) } diff --git a/cdc/model/changefeed.go b/cdc/model/changefeed.go index 2121d84994c..8f513bb9c9b 100644 --- a/cdc/model/changefeed.go +++ b/cdc/model/changefeed.go @@ -45,23 +45,24 @@ type FeedState string // All FeedStates const ( StateNormal FeedState = "normal" + StateError FeedState = "error" StateFailed FeedState = "failed" StateStopped FeedState = "stopped" - StateRemoved FeedState = "removed" + StateRemoved FeedState = "removed" // deprecated, will be removed in the next version StateFinished FeedState = "finished" ) const ( - // ErrorHistoryGCInterval represents how long we keep error record in changefeed info - ErrorHistoryGCInterval = time.Minute * 10 + // errorHistoryGCInterval represents how long we keep error record in changefeed info + errorHistoryGCInterval = time.Minute * 10 // errorHistoryCheckInterval represents time window for failure check errorHistoryCheckInterval = time.Minute * 2 - // errorHistoryThreshold represents failure upper limit in time window. + // ErrorHistoryThreshold represents failure upper limit in time window. // Before a changefeed is initialized, check the the failure count of this - // changefeed, if it is less than errorHistoryThreshold, then initialize it. - errorHistoryThreshold = 5 + // changefeed, if it is less than ErrorHistoryThreshold, then initialize it. + ErrorHistoryThreshold = 5 ) // ChangeFeedInfo describes the detail of a ChangeFeed @@ -215,13 +216,10 @@ func (info *ChangeFeedInfo) VerifyAndFix() error { func (info *ChangeFeedInfo) CheckErrorHistory() (needSave bool, canInit bool) { i := sort.Search(len(info.ErrorHis), func(i int) bool { ts := info.ErrorHis[i] - return time.Since(time.Unix(ts/1e3, (ts%1e3)*1e6)) < ErrorHistoryGCInterval + return time.Since(time.Unix(ts/1e3, (ts%1e3)*1e6)) < errorHistoryGCInterval }) - if i == len(info.ErrorHis) { - info.ErrorHis = info.ErrorHis[:] - } else { - info.ErrorHis = info.ErrorHis[i:] - } + info.ErrorHis = info.ErrorHis[i:] + if i > 0 { needSave = true } @@ -230,6 +228,38 @@ func (info *ChangeFeedInfo) CheckErrorHistory() (needSave bool, canInit bool) { ts := info.ErrorHis[i] return time.Since(time.Unix(ts/1e3, (ts%1e3)*1e6)) < errorHistoryCheckInterval }) - canInit = len(info.ErrorHis)-i < errorHistoryThreshold + canInit = len(info.ErrorHis)-i < ErrorHistoryThreshold return } + +// HasFastFailError returns true if the error in changefeed is fast-fail +func (info *ChangeFeedInfo) HasFastFailError() bool { + if info.Error == nil { + return false + } + return cerror.ChangefeedFastFailErrorCode(errors.RFCErrorCode(info.Error.Code)) +} + +// findActiveErrors finds all errors occurring within errorHistoryCheckInterval +func (info *ChangeFeedInfo) findActiveErrors() []int64 { + i := sort.Search(len(info.ErrorHis), func(i int) bool { + ts := info.ErrorHis[i] + // ts is a errors occurrence time, here to find all errors occurring within errorHistoryCheckInterval + return time.Since(time.Unix(ts/1e3, (ts%1e3)*1e6)) < errorHistoryCheckInterval + }) + return info.ErrorHis[i:] +} + +// ErrorsReachedThreshold checks error history of a changefeed +// returns true if error counts reach threshold +func (info *ChangeFeedInfo) ErrorsReachedThreshold() bool { + return len(info.findActiveErrors()) >= ErrorHistoryThreshold +} + +// CleanUpOutdatedErrorHistory cleans up the outdated error history +// return true if the ErrorHis changed +func (info *ChangeFeedInfo) CleanUpOutdatedErrorHistory() bool { + lastLenOfErrorHis := len(info.ErrorHis) + info.ErrorHis = info.findActiveErrors() + return lastLenOfErrorHis != len(info.ErrorHis) +} diff --git a/cdc/model/changefeed_test.go b/cdc/model/changefeed_test.go index 175517a8ef6..71478c54144 100644 --- a/cdc/model/changefeed_test.go +++ b/cdc/model/changefeed_test.go @@ -223,11 +223,11 @@ func (s *changefeedSuite) TestCheckErrorHistory(c *check.C) { ErrorHis: []int64{}, } for i := 0; i < 5; i++ { - tm := now.Add(-ErrorHistoryGCInterval) + tm := now.Add(-errorHistoryGCInterval) info.ErrorHis = append(info.ErrorHis, tm.UnixNano()/1e6) time.Sleep(time.Millisecond) } - for i := 0; i < errorHistoryThreshold-1; i++ { + for i := 0; i < ErrorHistoryThreshold-1; i++ { info.ErrorHis = append(info.ErrorHis, time.Now().UnixNano()/1e6) time.Sleep(time.Millisecond) } @@ -235,7 +235,7 @@ func (s *changefeedSuite) TestCheckErrorHistory(c *check.C) { needSave, canInit := info.CheckErrorHistory() c.Assert(needSave, check.IsTrue) c.Assert(canInit, check.IsTrue) - c.Assert(info.ErrorHis, check.HasLen, errorHistoryThreshold-1) + c.Assert(info.ErrorHis, check.HasLen, ErrorHistoryThreshold-1) info.ErrorHis = append(info.ErrorHis, time.Now().UnixNano()/1e6) needSave, canInit = info.CheckErrorHistory() diff --git a/cdc/model/reactor_state.go b/cdc/model/reactor_state.go new file mode 100644 index 00000000000..b4e4b8668a3 --- /dev/null +++ b/cdc/model/reactor_state.go @@ -0,0 +1,407 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package model + +import ( + "encoding/json" + "reflect" + + "github.com/pingcap/errors" + "github.com/pingcap/log" + cerrors "github.com/pingcap/ticdc/pkg/errors" + "github.com/pingcap/ticdc/pkg/etcd" + "github.com/pingcap/ticdc/pkg/orchestrator" + "github.com/pingcap/ticdc/pkg/orchestrator/util" + "go.uber.org/zap" +) + +// GlobalReactorState represents a global state which stores all key-value pairs in ETCD +type GlobalReactorState struct { + Owner map[string]struct{} + Captures map[CaptureID]*CaptureInfo + Changefeeds map[ChangeFeedID]*ChangefeedReactorState + pendingPatches [][]orchestrator.DataPatch +} + +// NewGlobalState creates a new global state +func NewGlobalState() orchestrator.ReactorState { + return &GlobalReactorState{ + Owner: map[string]struct{}{}, + Captures: make(map[CaptureID]*CaptureInfo), + Changefeeds: make(map[ChangeFeedID]*ChangefeedReactorState), + } +} + +// Update implements the ReactorState interface +func (s *GlobalReactorState) Update(key util.EtcdKey, value []byte, _ bool) error { + k := new(etcd.CDCKey) + err := k.Parse(key.String()) + if err != nil { + return errors.Trace(err) + } + switch k.Tp { + case etcd.CDCKeyTypeOwner: + if value != nil { + s.Owner[k.OwnerLeaseID] = struct{}{} + } else { + delete(s.Owner, k.OwnerLeaseID) + } + return nil + case etcd.CDCKeyTypeCapture: + if value == nil { + log.Info("remote capture offline", zap.String("capture-id", k.CaptureID)) + delete(s.Captures, k.CaptureID) + return nil + } + + var newCaptureInfo CaptureInfo + err := newCaptureInfo.Unmarshal(value) + if err != nil { + return cerrors.ErrUnmarshalFailed.Wrap(err).GenWithStackByArgs() + } + + log.Info("remote capture online", zap.String("capture-id", k.CaptureID), zap.Any("info", newCaptureInfo)) + s.Captures[k.CaptureID] = &newCaptureInfo + case etcd.CDCKeyTypeChangefeedInfo, + etcd.CDCKeyTypeChangeFeedStatus, + etcd.CDCKeyTypeTaskPosition, + etcd.CDCKeyTypeTaskStatus, + etcd.CDCKeyTypeTaskWorkload: + changefeedState, exist := s.Changefeeds[k.ChangefeedID] + if !exist { + if value == nil { + return nil + } + changefeedState = NewChangefeedReactorState(k.ChangefeedID) + s.Changefeeds[k.ChangefeedID] = changefeedState + } + if err := changefeedState.UpdateCDCKey(k, value); err != nil { + return errors.Trace(err) + } + if value == nil && !changefeedState.Exist() { + s.pendingPatches = append(s.pendingPatches, changefeedState.getPatches()) + delete(s.Changefeeds, k.ChangefeedID) + } + default: + log.Warn("receive an unexpected etcd event", zap.String("key", key.String()), zap.ByteString("value", value)) + } + return nil +} + +// GetPatches implements the ReactorState interface +func (s *GlobalReactorState) GetPatches() [][]orchestrator.DataPatch { + pendingPatches := s.pendingPatches + for _, changefeedState := range s.Changefeeds { + pendingPatches = append(pendingPatches, changefeedState.getPatches()) + } + s.pendingPatches = nil + return pendingPatches +} + +// ChangefeedReactorState represents a changefeed state which stores all key-value pairs of a changefeed in ETCD +type ChangefeedReactorState struct { + ID ChangeFeedID + Info *ChangeFeedInfo + Status *ChangeFeedStatus + TaskPositions map[CaptureID]*TaskPosition + TaskStatuses map[CaptureID]*TaskStatus + Workloads map[CaptureID]TaskWorkload + + pendingPatches []orchestrator.DataPatch + skipPatchesInThisTick bool +} + +// NewChangefeedReactorState creates a new changefeed reactor state +func NewChangefeedReactorState(id ChangeFeedID) *ChangefeedReactorState { + return &ChangefeedReactorState{ + ID: id, + TaskPositions: make(map[CaptureID]*TaskPosition), + TaskStatuses: make(map[CaptureID]*TaskStatus), + Workloads: make(map[CaptureID]TaskWorkload), + } +} + +// Update implements the ReactorState interface +func (s *ChangefeedReactorState) Update(key util.EtcdKey, value []byte, _ bool) error { + k := new(etcd.CDCKey) + if err := k.Parse(key.String()); err != nil { + return errors.Trace(err) + } + if err := s.UpdateCDCKey(k, value); err != nil { + log.Error("failed to update status", zap.String("key", key.String()), zap.ByteString("value", value)) + return errors.Trace(err) + } + return nil +} + +// UpdateCDCKey updates the state by a parsed etcd key +func (s *ChangefeedReactorState) UpdateCDCKey(key *etcd.CDCKey, value []byte) error { + var e interface{} + switch key.Tp { + case etcd.CDCKeyTypeChangefeedInfo: + if key.ChangefeedID != s.ID { + return nil + } + if value == nil { + s.Info = nil + return nil + } + s.Info = new(ChangeFeedInfo) + e = s.Info + case etcd.CDCKeyTypeChangeFeedStatus: + if key.ChangefeedID != s.ID { + return nil + } + if value == nil { + s.Status = nil + return nil + } + s.Status = new(ChangeFeedStatus) + e = s.Status + case etcd.CDCKeyTypeTaskPosition: + if key.ChangefeedID != s.ID { + return nil + } + if value == nil { + delete(s.TaskPositions, key.CaptureID) + return nil + } + position := new(TaskPosition) + s.TaskPositions[key.CaptureID] = position + e = position + case etcd.CDCKeyTypeTaskStatus: + if key.ChangefeedID != s.ID { + return nil + } + if value == nil { + delete(s.TaskStatuses, key.CaptureID) + return nil + } + status := new(TaskStatus) + s.TaskStatuses[key.CaptureID] = status + e = status + case etcd.CDCKeyTypeTaskWorkload: + if key.ChangefeedID != s.ID { + return nil + } + if value == nil { + delete(s.Workloads, key.CaptureID) + return nil + } + workload := make(TaskWorkload) + s.Workloads[key.CaptureID] = workload + e = &workload + default: + return nil + } + if err := json.Unmarshal(value, e); err != nil { + return errors.Trace(err) + } + if key.Tp == etcd.CDCKeyTypeChangefeedInfo { + if err := s.Info.VerifyAndFix(); err != nil { + return errors.Trace(err) + } + } + return nil +} + +// Exist returns false if all keys of this changefeed in ETCD is not exist +func (s *ChangefeedReactorState) Exist() bool { + return s.Info != nil || s.Status != nil || len(s.TaskPositions) != 0 || len(s.TaskStatuses) != 0 || len(s.Workloads) != 0 +} + +// Active return true if the changefeed is ready to be processed +func (s *ChangefeedReactorState) Active(captureID CaptureID) bool { + return s.Info != nil && s.Status != nil && s.TaskStatuses[captureID] != nil +} + +// GetPatches implements the ReactorState interface +func (s *ChangefeedReactorState) GetPatches() [][]orchestrator.DataPatch { + return [][]orchestrator.DataPatch{s.getPatches()} +} + +func (s *ChangefeedReactorState) getPatches() []orchestrator.DataPatch { + pendingPatches := s.pendingPatches + s.pendingPatches = nil + return pendingPatches +} + +// CheckCaptureAlive checks if the capture is alive, if the capture offline, +// the etcd worker will exit and throw the ErrLeaseExpired error. +func (s *ChangefeedReactorState) CheckCaptureAlive(captureID CaptureID) { + k := etcd.CDCKey{ + Tp: etcd.CDCKeyTypeCapture, + CaptureID: captureID, + } + key := k.String() + patch := &orchestrator.SingleDataPatch{ + Key: util.NewEtcdKey(key), + Func: func(v []byte) ([]byte, bool, error) { + // If v is empty, it means that the key-value pair of capture info is not exist. + // The key-value pair of capture info is written with lease, + // so if the capture info is not exist, the lease is expired + if len(v) == 0 { + return v, false, cerrors.ErrLeaseExpired.GenWithStackByArgs() + } + return v, false, nil + }, + } + s.pendingPatches = append(s.pendingPatches, patch) +} + +// CheckChangefeedNormal checks if the changefeed state is runable, +// if the changefeed status is not runable, the etcd worker will skip all patch of this tick +// the processor should call this function every tick to make sure the changefeed is runable +func (s *ChangefeedReactorState) CheckChangefeedNormal() { + s.skipPatchesInThisTick = false + s.PatchInfo(func(info *ChangeFeedInfo) (*ChangeFeedInfo, bool, error) { + if info == nil || info.AdminJobType.IsStopState() { + s.skipPatchesInThisTick = true + return info, false, cerrors.ErrEtcdTryAgain.GenWithStackByArgs() + } + return info, false, nil + }) + s.PatchStatus(func(status *ChangeFeedStatus) (*ChangeFeedStatus, bool, error) { + if status == nil { + return status, false, nil + } + if status.AdminJobType.IsStopState() { + s.skipPatchesInThisTick = true + return status, false, cerrors.ErrEtcdTryAgain.GenWithStackByArgs() + } + return status, false, nil + }) +} + +// PatchInfo appends a DataPatch which can modify the ChangeFeedInfo +func (s *ChangefeedReactorState) PatchInfo(fn func(*ChangeFeedInfo) (*ChangeFeedInfo, bool, error)) { + key := &etcd.CDCKey{ + Tp: etcd.CDCKeyTypeChangefeedInfo, + ChangefeedID: s.ID, + } + s.patchAny(key.String(), changefeedInfoTPI, func(e interface{}) (interface{}, bool, error) { + // e == nil means that the key is not exist before this patch + if e == nil { + return fn(nil) + } + return fn(e.(*ChangeFeedInfo)) + }) +} + +// PatchStatus appends a DataPatch which can modify the ChangeFeedStatus +func (s *ChangefeedReactorState) PatchStatus(fn func(*ChangeFeedStatus) (*ChangeFeedStatus, bool, error)) { + key := &etcd.CDCKey{ + Tp: etcd.CDCKeyTypeChangeFeedStatus, + ChangefeedID: s.ID, + } + s.patchAny(key.String(), changefeedStatusTPI, func(e interface{}) (interface{}, bool, error) { + // e == nil means that the key is not exist before this patch + if e == nil { + return fn(nil) + } + return fn(e.(*ChangeFeedStatus)) + }) +} + +// PatchTaskPosition appends a DataPatch which can modify the TaskPosition of a specified capture +func (s *ChangefeedReactorState) PatchTaskPosition(captureID CaptureID, fn func(*TaskPosition) (*TaskPosition, bool, error)) { + key := &etcd.CDCKey{ + Tp: etcd.CDCKeyTypeTaskPosition, + CaptureID: captureID, + ChangefeedID: s.ID, + } + s.patchAny(key.String(), taskPositionTPI, func(e interface{}) (interface{}, bool, error) { + // e == nil means that the key is not exist before this patch + if e == nil { + return fn(nil) + } + return fn(e.(*TaskPosition)) + }) +} + +// PatchTaskStatus appends a DataPatch which can modify the TaskStatus of a specified capture +func (s *ChangefeedReactorState) PatchTaskStatus(captureID CaptureID, fn func(*TaskStatus) (*TaskStatus, bool, error)) { + key := &etcd.CDCKey{ + Tp: etcd.CDCKeyTypeTaskStatus, + CaptureID: captureID, + ChangefeedID: s.ID, + } + s.patchAny(key.String(), taskStatusTPI, func(e interface{}) (interface{}, bool, error) { + // e == nil means that the key is not exist before this patch + if e == nil { + return fn(nil) + } + return fn(e.(*TaskStatus)) + }) +} + +// PatchTaskWorkload appends a DataPatch which can modify the TaskWorkload of a specified capture +func (s *ChangefeedReactorState) PatchTaskWorkload(captureID CaptureID, fn func(TaskWorkload) (TaskWorkload, bool, error)) { + key := &etcd.CDCKey{ + Tp: etcd.CDCKeyTypeTaskWorkload, + CaptureID: captureID, + ChangefeedID: s.ID, + } + s.patchAny(key.String(), taskWorkloadTPI, func(e interface{}) (interface{}, bool, error) { + // e == nil means that the key is not exist before this patch + if e == nil { + return fn(nil) + } + return fn(*e.(*TaskWorkload)) + }) +} + +var ( + taskPositionTPI *TaskPosition + taskStatusTPI *TaskStatus + taskWorkloadTPI *TaskWorkload + changefeedStatusTPI *ChangeFeedStatus + changefeedInfoTPI *ChangeFeedInfo +) + +func (s *ChangefeedReactorState) patchAny(key string, tpi interface{}, fn func(interface{}) (interface{}, bool, error)) { + patch := &orchestrator.SingleDataPatch{ + Key: util.NewEtcdKey(key), + Func: func(v []byte) ([]byte, bool, error) { + if s.skipPatchesInThisTick { + return v, false, cerrors.ErrEtcdIgnore.GenWithStackByArgs() + } + var e interface{} + if v != nil { + tp := reflect.TypeOf(tpi) + e = reflect.New(tp.Elem()).Interface() + err := json.Unmarshal(v, e) + if err != nil { + return nil, false, errors.Trace(err) + } + } + ne, changed, err := fn(e) + if err != nil { + return nil, false, errors.Trace(err) + } + if !changed { + return v, false, nil + } + if reflect.ValueOf(ne).IsNil() { + return nil, true, nil + } + nv, err := json.Marshal(ne) + if err != nil { + return nil, false, errors.Trace(err) + } + return nv, true, nil + }, + } + s.pendingPatches = append(s.pendingPatches, patch) +} diff --git a/cdc/model/reactor_state_test.go b/cdc/model/reactor_state_test.go new file mode 100644 index 00000000000..df76386c49a --- /dev/null +++ b/cdc/model/reactor_state_test.go @@ -0,0 +1,686 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package model + +import ( + "time" + + "github.com/google/go-cmp/cmp/cmpopts" + + "github.com/google/go-cmp/cmp" + "github.com/pingcap/check" + "github.com/pingcap/ticdc/pkg/config" + "github.com/pingcap/ticdc/pkg/orchestrator" + "github.com/pingcap/ticdc/pkg/orchestrator/util" + "github.com/pingcap/ticdc/pkg/util/testleak" +) + +type stateSuite struct{} + +var _ = check.Suite(&stateSuite{}) + +func (s *stateSuite) TestCheckCaptureAlive(c *check.C) { + defer testleak.AfterTest(c)() + state := NewChangefeedReactorState("test") + stateTester := orchestrator.NewReactorStateTester(c, state, nil) + state.CheckCaptureAlive("6bbc01c8-0605-4f86-a0f9-b3119109b225") + c.Assert(stateTester.ApplyPatches(), check.ErrorMatches, ".*[CDC:ErrLeaseExpired].*") + err := stateTester.Update("/tidb/cdc/capture/6bbc01c8-0605-4f86-a0f9-b3119109b225", []byte(`{"id":"6bbc01c8-0605-4f86-a0f9-b3119109b225","address":"127.0.0.1:8300"}`)) + c.Assert(err, check.IsNil) + state.CheckCaptureAlive("6bbc01c8-0605-4f86-a0f9-b3119109b225") + stateTester.MustApplyPatches() +} + +func (s *stateSuite) TestChangefeedStateUpdate(c *check.C) { + defer testleak.AfterTest(c)() + createTime, err := time.Parse("2006-01-02", "2020-02-02") + c.Assert(err, check.IsNil) + testCases := []struct { + changefeedID string + updateKey []string + updateValue []string + expected ChangefeedReactorState + }{ + { // common case + changefeedID: "test1", + updateKey: []string{ + "/tidb/cdc/changefeed/info/test1", + "/tidb/cdc/job/test1", + "/tidb/cdc/task/position/6bbc01c8-0605-4f86-a0f9-b3119109b225/test1", + "/tidb/cdc/task/status/6bbc01c8-0605-4f86-a0f9-b3119109b225/test1", + "/tidb/cdc/task/workload/6bbc01c8-0605-4f86-a0f9-b3119109b225/test1", + "/tidb/cdc/capture/6bbc01c8-0605-4f86-a0f9-b3119109b225", + }, + updateValue: []string{ + `{"sink-uri":"blackhole://","opts":{},"create-time":"2020-02-02T00:00:00.000000+00:00","start-ts":421980685886554116,"target-ts":0,"admin-job-type":0,"sort-engine":"memory","sort-dir":"","config":{"case-sensitive":true,"enable-old-value":false,"force-replicate":false,"check-gc-safe-point":true,"filter":{"rules":["*.*"],"ignore-txn-start-ts":null,"ddl-allow-list":null},"mounter":{"worker-num":16},"sink":{"dispatchers":null,"protocol":"default"},"cyclic-replication":{"enable":false,"replica-id":0,"filter-replica-ids":null,"id-buckets":0,"sync-ddl":false},"scheduler":{"type":"table-number","polling-time":-1}},"state":"normal","history":null,"error":null,"sync-point-enabled":false,"sync-point-interval":600000000000}`, + `{"resolved-ts":421980720003809281,"checkpoint-ts":421980719742451713,"admin-job-type":0}`, + `{"checkpoint-ts":421980720003809281,"resolved-ts":421980720003809281,"count":0,"error":null}`, + `{"tables":{"45":{"start-ts":421980685886554116,"mark-table-id":0}},"operation":null,"admin-job-type":0}`, + `{"45":{"workload":1}}`, + `{"id":"6bbc01c8-0605-4f86-a0f9-b3119109b225","address":"127.0.0.1:8300"}`, + }, + expected: ChangefeedReactorState{ + ID: "test1", + Info: &ChangeFeedInfo{ + SinkURI: "blackhole://", + Opts: map[string]string{}, + CreateTime: createTime, + StartTs: 421980685886554116, + Engine: SortInMemory, + State: "normal", + SyncPointInterval: time.Minute * 10, + Config: &config.ReplicaConfig{ + CaseSensitive: true, + CheckGCSafePoint: true, + Filter: &config.FilterConfig{Rules: []string{"*.*"}}, + Mounter: &config.MounterConfig{WorkerNum: 16}, + Sink: &config.SinkConfig{Protocol: "default"}, + Cyclic: &config.CyclicConfig{}, + Scheduler: &config.SchedulerConfig{Tp: "table-number", PollingTime: -1}, + }, + }, + Status: &ChangeFeedStatus{CheckpointTs: 421980719742451713, ResolvedTs: 421980720003809281}, + TaskStatuses: map[CaptureID]*TaskStatus{ + "6bbc01c8-0605-4f86-a0f9-b3119109b225": { + Tables: map[int64]*TableReplicaInfo{45: {StartTs: 421980685886554116}}, + }, + }, + TaskPositions: map[CaptureID]*TaskPosition{ + "6bbc01c8-0605-4f86-a0f9-b3119109b225": {CheckPointTs: 421980720003809281, ResolvedTs: 421980720003809281}, + }, + Workloads: map[CaptureID]TaskWorkload{ + "6bbc01c8-0605-4f86-a0f9-b3119109b225": {45: {Workload: 1}}, + }, + }, + }, + { // test multiple capture + changefeedID: "test1", + updateKey: []string{ + "/tidb/cdc/changefeed/info/test1", + "/tidb/cdc/job/test1", + "/tidb/cdc/task/position/6bbc01c8-0605-4f86-a0f9-b3119109b225/test1", + "/tidb/cdc/task/status/6bbc01c8-0605-4f86-a0f9-b3119109b225/test1", + "/tidb/cdc/task/workload/6bbc01c8-0605-4f86-a0f9-b3119109b225/test1", + "/tidb/cdc/capture/6bbc01c8-0605-4f86-a0f9-b3119109b225", + "/tidb/cdc/task/position/666777888/test1", + "/tidb/cdc/task/status/666777888/test1", + "/tidb/cdc/task/workload/666777888/test1", + "/tidb/cdc/capture/666777888", + }, + updateValue: []string{ + `{"sink-uri":"blackhole://","opts":{},"create-time":"2020-02-02T00:00:00.000000+00:00","start-ts":421980685886554116,"target-ts":0,"admin-job-type":0,"sort-engine":"memory","sort-dir":"","config":{"case-sensitive":true,"enable-old-value":false,"force-replicate":false,"check-gc-safe-point":true,"filter":{"rules":["*.*"],"ignore-txn-start-ts":null,"ddl-allow-list":null},"mounter":{"worker-num":16},"sink":{"dispatchers":null,"protocol":"default"},"cyclic-replication":{"enable":false,"replica-id":0,"filter-replica-ids":null,"id-buckets":0,"sync-ddl":false},"scheduler":{"type":"table-number","polling-time":-1}},"state":"normal","history":null,"error":null,"sync-point-enabled":false,"sync-point-interval":600000000000}`, + `{"resolved-ts":421980720003809281,"checkpoint-ts":421980719742451713,"admin-job-type":0}`, + `{"checkpoint-ts":421980720003809281,"resolved-ts":421980720003809281,"count":0,"error":null}`, + `{"tables":{"45":{"start-ts":421980685886554116,"mark-table-id":0}},"operation":null,"admin-job-type":0}`, + `{"45":{"workload":1}}`, + `{"id":"6bbc01c8-0605-4f86-a0f9-b3119109b225","address":"127.0.0.1:8300"}`, + `{"checkpoint-ts":11332244,"resolved-ts":312321,"count":8,"error":null}`, + `{"tables":{"46":{"start-ts":412341234,"mark-table-id":0}},"operation":null,"admin-job-type":0}`, + `{"46":{"workload":3}}`, + `{"id":"666777888","address":"127.0.0.1:8300"}`, + }, + expected: ChangefeedReactorState{ + ID: "test1", + Info: &ChangeFeedInfo{ + SinkURI: "blackhole://", + Opts: map[string]string{}, + CreateTime: createTime, + StartTs: 421980685886554116, + Engine: SortInMemory, + State: "normal", + SyncPointInterval: time.Minute * 10, + Config: &config.ReplicaConfig{ + CaseSensitive: true, + CheckGCSafePoint: true, + Filter: &config.FilterConfig{Rules: []string{"*.*"}}, + Mounter: &config.MounterConfig{WorkerNum: 16}, + Sink: &config.SinkConfig{Protocol: "default"}, + Cyclic: &config.CyclicConfig{}, + Scheduler: &config.SchedulerConfig{Tp: "table-number", PollingTime: -1}, + }, + }, + Status: &ChangeFeedStatus{CheckpointTs: 421980719742451713, ResolvedTs: 421980720003809281}, + TaskStatuses: map[CaptureID]*TaskStatus{ + "6bbc01c8-0605-4f86-a0f9-b3119109b225": { + Tables: map[int64]*TableReplicaInfo{45: {StartTs: 421980685886554116}}, + }, + "666777888": { + Tables: map[int64]*TableReplicaInfo{46: {StartTs: 412341234}}, + }, + }, + TaskPositions: map[CaptureID]*TaskPosition{ + "6bbc01c8-0605-4f86-a0f9-b3119109b225": {CheckPointTs: 421980720003809281, ResolvedTs: 421980720003809281}, + "666777888": {CheckPointTs: 11332244, ResolvedTs: 312321, Count: 8}, + }, + Workloads: map[CaptureID]TaskWorkload{ + "6bbc01c8-0605-4f86-a0f9-b3119109b225": {45: {Workload: 1}}, + "666777888": {46: {Workload: 3}}, + }, + }, + }, + { // testing changefeedID not match + changefeedID: "test1", + updateKey: []string{ + "/tidb/cdc/changefeed/info/test1", + "/tidb/cdc/job/test1", + "/tidb/cdc/task/position/6bbc01c8-0605-4f86-a0f9-b3119109b225/test1", + "/tidb/cdc/task/status/6bbc01c8-0605-4f86-a0f9-b3119109b225/test1", + "/tidb/cdc/task/workload/6bbc01c8-0605-4f86-a0f9-b3119109b225/test1", + "/tidb/cdc/capture/6bbc01c8-0605-4f86-a0f9-b3119109b225", + "/tidb/cdc/changefeed/info/test-fake", + "/tidb/cdc/job/test-fake", + "/tidb/cdc/task/position/6bbc01c8-0605-4f86-a0f9-b3119109b225/test-fake", + "/tidb/cdc/task/status/6bbc01c8-0605-4f86-a0f9-b3119109b225/test-fake", + "/tidb/cdc/task/workload/6bbc01c8-0605-4f86-a0f9-b3119109b225/test-fake", + }, + updateValue: []string{ + `{"sink-uri":"blackhole://","opts":{},"create-time":"2020-02-02T00:00:00.000000+00:00","start-ts":421980685886554116,"target-ts":0,"admin-job-type":0,"sort-engine":"memory","sort-dir":"","config":{"case-sensitive":true,"enable-old-value":false,"force-replicate":false,"check-gc-safe-point":true,"filter":{"rules":["*.*"],"ignore-txn-start-ts":null,"ddl-allow-list":null},"mounter":{"worker-num":16},"sink":{"dispatchers":null,"protocol":"default"},"cyclic-replication":{"enable":false,"replica-id":0,"filter-replica-ids":null,"id-buckets":0,"sync-ddl":false},"scheduler":{"type":"table-number","polling-time":-1}},"state":"normal","history":null,"error":null,"sync-point-enabled":false,"sync-point-interval":600000000000}`, + `{"resolved-ts":421980720003809281,"checkpoint-ts":421980719742451713,"admin-job-type":0}`, + `{"checkpoint-ts":421980720003809281,"resolved-ts":421980720003809281,"count":0,"error":null}`, + `{"tables":{"45":{"start-ts":421980685886554116,"mark-table-id":0}},"operation":null,"admin-job-type":0}`, + `{"45":{"workload":1}}`, + `{"id":"6bbc01c8-0605-4f86-a0f9-b3119109b225","address":"127.0.0.1:8300"}`, + `fake value`, + `fake value`, + `fake value`, + `fake value`, + `fake value`, + }, + expected: ChangefeedReactorState{ + ID: "test1", + Info: &ChangeFeedInfo{ + SinkURI: "blackhole://", + Opts: map[string]string{}, + CreateTime: createTime, + StartTs: 421980685886554116, + Engine: SortInMemory, + State: "normal", + SyncPointInterval: time.Minute * 10, + Config: &config.ReplicaConfig{ + CaseSensitive: true, + CheckGCSafePoint: true, + Filter: &config.FilterConfig{Rules: []string{"*.*"}}, + Mounter: &config.MounterConfig{WorkerNum: 16}, + Sink: &config.SinkConfig{Protocol: "default"}, + Cyclic: &config.CyclicConfig{}, + Scheduler: &config.SchedulerConfig{Tp: "table-number", PollingTime: -1}, + }, + }, + Status: &ChangeFeedStatus{CheckpointTs: 421980719742451713, ResolvedTs: 421980720003809281}, + TaskStatuses: map[CaptureID]*TaskStatus{ + "6bbc01c8-0605-4f86-a0f9-b3119109b225": { + Tables: map[int64]*TableReplicaInfo{45: {StartTs: 421980685886554116}}, + }, + }, + TaskPositions: map[CaptureID]*TaskPosition{ + "6bbc01c8-0605-4f86-a0f9-b3119109b225": {CheckPointTs: 421980720003809281, ResolvedTs: 421980720003809281}, + }, + Workloads: map[CaptureID]TaskWorkload{ + "6bbc01c8-0605-4f86-a0f9-b3119109b225": {45: {Workload: 1}}, + }, + }, + }, + { // testing value is nil + changefeedID: "test1", + updateKey: []string{ + "/tidb/cdc/changefeed/info/test1", + "/tidb/cdc/job/test1", + "/tidb/cdc/task/position/6bbc01c8-0605-4f86-a0f9-b3119109b225/test1", + "/tidb/cdc/task/status/6bbc01c8-0605-4f86-a0f9-b3119109b225/test1", + "/tidb/cdc/task/workload/6bbc01c8-0605-4f86-a0f9-b3119109b225/test1", + "/tidb/cdc/capture/6bbc01c8-0605-4f86-a0f9-b3119109b225", + "/tidb/cdc/task/position/666777888/test1", + "/tidb/cdc/task/status/666777888/test1", + "/tidb/cdc/task/workload/666777888/test1", + "/tidb/cdc/changefeed/info/test1", + "/tidb/cdc/job/test1", + "/tidb/cdc/task/position/6bbc01c8-0605-4f86-a0f9-b3119109b225/test1", + "/tidb/cdc/task/status/6bbc01c8-0605-4f86-a0f9-b3119109b225/test1", + "/tidb/cdc/task/workload/6bbc01c8-0605-4f86-a0f9-b3119109b225/test1", + "/tidb/cdc/capture/6bbc01c8-0605-4f86-a0f9-b3119109b225", + "/tidb/cdc/task/workload/666777888/test1", + "/tidb/cdc/task/status/666777888/test1", + }, + updateValue: []string{ + `{"sink-uri":"blackhole://","opts":{},"create-time":"2020-02-02T00:00:00.000000+00:00","start-ts":421980685886554116,"target-ts":0,"admin-job-type":0,"sort-engine":"memory","sort-dir":"","config":{"case-sensitive":true,"enable-old-value":false,"force-replicate":false,"check-gc-safe-point":true,"filter":{"rules":["*.*"],"ignore-txn-start-ts":null,"ddl-allow-list":null},"mounter":{"worker-num":16},"sink":{"dispatchers":null,"protocol":"default"},"cyclic-replication":{"enable":false,"replica-id":0,"filter-replica-ids":null,"id-buckets":0,"sync-ddl":false},"scheduler":{"type":"table-number","polling-time":-1}},"state":"normal","history":null,"error":null,"sync-point-enabled":false,"sync-point-interval":600000000000}`, + `{"resolved-ts":421980720003809281,"checkpoint-ts":421980719742451713,"admin-job-type":0}`, + `{"checkpoint-ts":421980720003809281,"resolved-ts":421980720003809281,"count":0,"error":null}`, + `{"tables":{"45":{"start-ts":421980685886554116,"mark-table-id":0}},"operation":null,"admin-job-type":0}`, + `{"45":{"workload":1}}`, + `{"id":"6bbc01c8-0605-4f86-a0f9-b3119109b225","address":"127.0.0.1:8300"}`, + `{"checkpoint-ts":11332244,"resolved-ts":312321,"count":8,"error":null}`, + `{"tables":{"46":{"start-ts":412341234,"mark-table-id":0}},"operation":null,"admin-job-type":0}`, + `{"46":{"workload":3}}`, + ``, + ``, + ``, + ``, + ``, + ``, + ``, + ``, + }, + expected: ChangefeedReactorState{ + ID: "test1", + Info: nil, + Status: nil, + TaskStatuses: map[CaptureID]*TaskStatus{}, + TaskPositions: map[CaptureID]*TaskPosition{ + "666777888": {CheckPointTs: 11332244, ResolvedTs: 312321, Count: 8}, + }, + Workloads: map[CaptureID]TaskWorkload{}, + }, + }, + { // testing the same key case + changefeedID: "test1", + updateKey: []string{ + "/tidb/cdc/task/status/6bbc01c8-0605-4f86-a0f9-b3119109b225/test1", + "/tidb/cdc/task/status/6bbc01c8-0605-4f86-a0f9-b3119109b225/test1", + "/tidb/cdc/task/status/6bbc01c8-0605-4f86-a0f9-b3119109b225/test1", + "/tidb/cdc/task/status/6bbc01c8-0605-4f86-a0f9-b3119109b225/test1", + }, + updateValue: []string{ + `{"tables":{"45":{"start-ts":421980685886554116,"mark-table-id":0}},"operation":null,"admin-job-type":0}`, + `{"tables":{"46":{"start-ts":421980685886554116,"mark-table-id":0}},"operation":null,"admin-job-type":0}`, + ``, + `{"tables":{"47":{"start-ts":421980685886554116,"mark-table-id":0}},"operation":null,"admin-job-type":0}`, + }, + expected: ChangefeedReactorState{ + ID: "test1", + TaskStatuses: map[CaptureID]*TaskStatus{ + "6bbc01c8-0605-4f86-a0f9-b3119109b225": { + Tables: map[int64]*TableReplicaInfo{47: {StartTs: 421980685886554116}}, + }, + }, + TaskPositions: map[CaptureID]*TaskPosition{}, + Workloads: map[CaptureID]TaskWorkload{}, + }, + }, + } + for i, tc := range testCases { + state := NewChangefeedReactorState(tc.changefeedID) + for i, k := range tc.updateKey { + value := []byte(tc.updateValue[i]) + if len(value) == 0 { + value = nil + } + err = state.Update(util.NewEtcdKey(k), value, false) + c.Assert(err, check.IsNil) + } + c.Assert(cmp.Equal(state, &tc.expected, cmpopts.IgnoreUnexported(ChangefeedReactorState{})), check.IsTrue, + check.Commentf("%d,%s", i, cmp.Diff(state, &tc.expected, cmpopts.IgnoreUnexported(ChangefeedReactorState{})))) + } +} + +func (s *stateSuite) TestPatchInfo(c *check.C) { + defer testleak.AfterTest(c)() + state := NewChangefeedReactorState("test1") + stateTester := orchestrator.NewReactorStateTester(c, state, nil) + state.PatchInfo(func(info *ChangeFeedInfo) (*ChangeFeedInfo, bool, error) { + c.Assert(info, check.IsNil) + return &ChangeFeedInfo{SinkURI: "123", Config: &config.ReplicaConfig{}}, true, nil + }) + stateTester.MustApplyPatches() + defaultConfig := config.GetDefaultReplicaConfig() + c.Assert(state.Info, check.DeepEquals, &ChangeFeedInfo{ + SinkURI: "123", + Engine: SortUnified, + Config: &config.ReplicaConfig{ + Filter: defaultConfig.Filter, + Mounter: defaultConfig.Mounter, + Sink: defaultConfig.Sink, + Cyclic: defaultConfig.Cyclic, + Scheduler: defaultConfig.Scheduler, + }, + }) + state.PatchInfo(func(info *ChangeFeedInfo) (*ChangeFeedInfo, bool, error) { + info.StartTs = 6 + return info, true, nil + }) + stateTester.MustApplyPatches() + c.Assert(state.Info, check.DeepEquals, &ChangeFeedInfo{ + SinkURI: "123", + StartTs: 6, + Engine: SortUnified, + Config: &config.ReplicaConfig{ + Filter: defaultConfig.Filter, + Mounter: defaultConfig.Mounter, + Sink: defaultConfig.Sink, + Cyclic: defaultConfig.Cyclic, + Scheduler: defaultConfig.Scheduler, + }, + }) + state.PatchInfo(func(info *ChangeFeedInfo) (*ChangeFeedInfo, bool, error) { + return nil, true, nil + }) + stateTester.MustApplyPatches() + c.Assert(state.Info, check.IsNil) +} + +func (s *stateSuite) TestPatchStatus(c *check.C) { + defer testleak.AfterTest(c)() + state := NewChangefeedReactorState("test1") + stateTester := orchestrator.NewReactorStateTester(c, state, nil) + state.PatchStatus(func(status *ChangeFeedStatus) (*ChangeFeedStatus, bool, error) { + c.Assert(status, check.IsNil) + return &ChangeFeedStatus{CheckpointTs: 5}, true, nil + }) + stateTester.MustApplyPatches() + c.Assert(state.Status, check.DeepEquals, &ChangeFeedStatus{CheckpointTs: 5}) + state.PatchStatus(func(status *ChangeFeedStatus) (*ChangeFeedStatus, bool, error) { + status.ResolvedTs = 6 + return status, true, nil + }) + stateTester.MustApplyPatches() + c.Assert(state.Status, check.DeepEquals, &ChangeFeedStatus{CheckpointTs: 5, ResolvedTs: 6}) + state.PatchStatus(func(status *ChangeFeedStatus) (*ChangeFeedStatus, bool, error) { + return nil, true, nil + }) + stateTester.MustApplyPatches() + c.Assert(state.Status, check.IsNil) +} + +func (s *stateSuite) TestPatchTaskPosition(c *check.C) { + defer testleak.AfterTest(c)() + state := NewChangefeedReactorState("test1") + stateTester := orchestrator.NewReactorStateTester(c, state, nil) + captureID1 := "capture1" + captureID2 := "capture2" + state.PatchTaskPosition(captureID1, func(position *TaskPosition) (*TaskPosition, bool, error) { + c.Assert(position, check.IsNil) + return &TaskPosition{ + CheckPointTs: 1, + }, true, nil + }) + state.PatchTaskPosition(captureID2, func(position *TaskPosition) (*TaskPosition, bool, error) { + c.Assert(position, check.IsNil) + return &TaskPosition{ + CheckPointTs: 2, + }, true, nil + }) + stateTester.MustApplyPatches() + c.Assert(state.TaskPositions, check.DeepEquals, map[string]*TaskPosition{ + captureID1: { + CheckPointTs: 1, + }, + captureID2: { + CheckPointTs: 2, + }, + }) + state.PatchTaskPosition(captureID1, func(position *TaskPosition) (*TaskPosition, bool, error) { + position.CheckPointTs = 3 + return position, true, nil + }) + state.PatchTaskPosition(captureID2, func(position *TaskPosition) (*TaskPosition, bool, error) { + position.ResolvedTs = 2 + return position, true, nil + }) + stateTester.MustApplyPatches() + c.Assert(state.TaskPositions, check.DeepEquals, map[string]*TaskPosition{ + captureID1: { + CheckPointTs: 3, + }, + captureID2: { + CheckPointTs: 2, + ResolvedTs: 2, + }, + }) + state.PatchTaskPosition(captureID1, func(position *TaskPosition) (*TaskPosition, bool, error) { + return nil, false, nil + }) + state.PatchTaskPosition(captureID2, func(position *TaskPosition) (*TaskPosition, bool, error) { + return nil, true, nil + }) + state.PatchTaskPosition(captureID1, func(position *TaskPosition) (*TaskPosition, bool, error) { + position.Count = 6 + return position, true, nil + }) + stateTester.MustApplyPatches() + c.Assert(state.TaskPositions, check.DeepEquals, map[string]*TaskPosition{ + captureID1: { + CheckPointTs: 3, + Count: 6, + }, + }) +} + +func (s *stateSuite) TestPatchTaskStatus(c *check.C) { + defer testleak.AfterTest(c)() + state := NewChangefeedReactorState("test1") + stateTester := orchestrator.NewReactorStateTester(c, state, nil) + captureID1 := "capture1" + captureID2 := "capture2" + state.PatchTaskStatus(captureID1, func(status *TaskStatus) (*TaskStatus, bool, error) { + c.Assert(status, check.IsNil) + return &TaskStatus{ + Tables: map[TableID]*TableReplicaInfo{45: {StartTs: 1}}, + }, true, nil + }) + state.PatchTaskStatus(captureID2, func(status *TaskStatus) (*TaskStatus, bool, error) { + c.Assert(status, check.IsNil) + return &TaskStatus{ + Tables: map[TableID]*TableReplicaInfo{46: {StartTs: 1}}, + }, true, nil + }) + stateTester.MustApplyPatches() + c.Assert(state.TaskStatuses, check.DeepEquals, map[CaptureID]*TaskStatus{ + captureID1: {Tables: map[TableID]*TableReplicaInfo{45: {StartTs: 1}}}, + captureID2: {Tables: map[TableID]*TableReplicaInfo{46: {StartTs: 1}}}, + }) + state.PatchTaskStatus(captureID1, func(status *TaskStatus) (*TaskStatus, bool, error) { + status.Tables[46] = &TableReplicaInfo{StartTs: 2} + return status, true, nil + }) + state.PatchTaskStatus(captureID2, func(status *TaskStatus) (*TaskStatus, bool, error) { + status.Tables[46].StartTs++ + return status, true, nil + }) + stateTester.MustApplyPatches() + c.Assert(state.TaskStatuses, check.DeepEquals, map[CaptureID]*TaskStatus{ + captureID1: {Tables: map[TableID]*TableReplicaInfo{45: {StartTs: 1}, 46: {StartTs: 2}}}, + captureID2: {Tables: map[TableID]*TableReplicaInfo{46: {StartTs: 2}}}, + }) + state.PatchTaskStatus(captureID2, func(status *TaskStatus) (*TaskStatus, bool, error) { + return nil, true, nil + }) + stateTester.MustApplyPatches() + c.Assert(state.TaskStatuses, check.DeepEquals, map[CaptureID]*TaskStatus{ + captureID1: {Tables: map[TableID]*TableReplicaInfo{45: {StartTs: 1}, 46: {StartTs: 2}}}, + }) +} + +func (s *stateSuite) TestPatchTaskWorkload(c *check.C) { + defer testleak.AfterTest(c)() + state := NewChangefeedReactorState("test1") + stateTester := orchestrator.NewReactorStateTester(c, state, nil) + captureID1 := "capture1" + captureID2 := "capture2" + state.PatchTaskWorkload(captureID1, func(workload TaskWorkload) (TaskWorkload, bool, error) { + c.Assert(workload, check.IsNil) + return TaskWorkload{45: {Workload: 1}}, true, nil + }) + state.PatchTaskWorkload(captureID2, func(workload TaskWorkload) (TaskWorkload, bool, error) { + c.Assert(workload, check.IsNil) + return TaskWorkload{46: {Workload: 1}}, true, nil + }) + stateTester.MustApplyPatches() + c.Assert(state.Workloads, check.DeepEquals, map[CaptureID]TaskWorkload{ + captureID1: {45: {Workload: 1}}, + captureID2: {46: {Workload: 1}}, + }) + state.PatchTaskWorkload(captureID1, func(workload TaskWorkload) (TaskWorkload, bool, error) { + workload[46] = WorkloadInfo{Workload: 2} + return workload, true, nil + }) + state.PatchTaskWorkload(captureID2, func(workload TaskWorkload) (TaskWorkload, bool, error) { + workload[45] = WorkloadInfo{Workload: 3} + return workload, true, nil + }) + stateTester.MustApplyPatches() + c.Assert(state.Workloads, check.DeepEquals, map[CaptureID]TaskWorkload{ + captureID1: {45: {Workload: 1}, 46: {Workload: 2}}, + captureID2: {45: {Workload: 3}, 46: {Workload: 1}}, + }) + state.PatchTaskWorkload(captureID2, func(workload TaskWorkload) (TaskWorkload, bool, error) { + return nil, true, nil + }) + stateTester.MustApplyPatches() + c.Assert(state.Workloads, check.DeepEquals, map[CaptureID]TaskWorkload{ + captureID1: {45: {Workload: 1}, 46: {Workload: 2}}, + }) +} + +func (s *stateSuite) TestGlobalStateUpdate(c *check.C) { + defer testleak.AfterTest(c)() + testCases := []struct { + updateKey []string + updateValue []string + expected GlobalReactorState + }{ + { // common case + updateKey: []string{ + "/tidb/cdc/owner/22317526c4fc9a37", + "/tidb/cdc/owner/22317526c4fc9a38", + "/tidb/cdc/capture/6bbc01c8-0605-4f86-a0f9-b3119109b225", + "/tidb/cdc/task/position/6bbc01c8-0605-4f86-a0f9-b3119109b225/test1", + "/tidb/cdc/task/workload/6bbc01c8-0605-4f86-a0f9-b3119109b225/test2", + "/tidb/cdc/task/workload/55551111/test2", + }, + updateValue: []string{ + `6bbc01c8-0605-4f86-a0f9-b3119109b225`, + `55551111`, + `{"id":"6bbc01c8-0605-4f86-a0f9-b3119109b225","address":"127.0.0.1:8300"}`, + `{"resolved-ts":421980720003809281,"checkpoint-ts":421980719742451713,"admin-job-type":0}`, + `{"45":{"workload":1}}`, + `{"46":{"workload":1}}`, + }, + expected: GlobalReactorState{ + Owner: map[string]struct{}{"22317526c4fc9a37": {}, "22317526c4fc9a38": {}}, + Captures: map[CaptureID]*CaptureInfo{"6bbc01c8-0605-4f86-a0f9-b3119109b225": { + ID: "6bbc01c8-0605-4f86-a0f9-b3119109b225", + AdvertiseAddr: "127.0.0.1:8300", + }}, + Changefeeds: map[ChangeFeedID]*ChangefeedReactorState{ + "test1": { + ID: "test1", + TaskStatuses: map[string]*TaskStatus{}, + TaskPositions: map[CaptureID]*TaskPosition{ + "6bbc01c8-0605-4f86-a0f9-b3119109b225": {CheckPointTs: 421980719742451713, ResolvedTs: 421980720003809281}, + }, + Workloads: map[string]TaskWorkload{}, + }, + "test2": { + ID: "test2", + TaskStatuses: map[string]*TaskStatus{}, + TaskPositions: map[CaptureID]*TaskPosition{}, + Workloads: map[CaptureID]TaskWorkload{ + "6bbc01c8-0605-4f86-a0f9-b3119109b225": {45: {Workload: 1}}, + "55551111": {46: {Workload: 1}}, + }, + }, + }, + }, + }, + { // testing remove changefeed + updateKey: []string{ + "/tidb/cdc/owner/22317526c4fc9a37", + "/tidb/cdc/owner/22317526c4fc9a38", + "/tidb/cdc/capture/6bbc01c8-0605-4f86-a0f9-b3119109b225", + "/tidb/cdc/task/position/6bbc01c8-0605-4f86-a0f9-b3119109b225/test1", + "/tidb/cdc/task/workload/6bbc01c8-0605-4f86-a0f9-b3119109b225/test2", + "/tidb/cdc/task/workload/55551111/test2", + "/tidb/cdc/owner/22317526c4fc9a37", + "/tidb/cdc/task/position/6bbc01c8-0605-4f86-a0f9-b3119109b225/test1", + "/tidb/cdc/task/workload/6bbc01c8-0605-4f86-a0f9-b3119109b225/test2", + "/tidb/cdc/capture/6bbc01c8-0605-4f86-a0f9-b3119109b225", + }, + updateValue: []string{ + `6bbc01c8-0605-4f86-a0f9-b3119109b225`, + `55551111`, + `{"id":"6bbc01c8-0605-4f86-a0f9-b3119109b225","address":"127.0.0.1:8300"}`, + `{"resolved-ts":421980720003809281,"checkpoint-ts":421980719742451713,"admin-job-type":0}`, + `{"45":{"workload":1}}`, + `{"46":{"workload":1}}`, + ``, + ``, + ``, + ``, + }, + expected: GlobalReactorState{ + Owner: map[string]struct{}{"22317526c4fc9a38": {}}, + Captures: map[CaptureID]*CaptureInfo{}, + Changefeeds: map[ChangeFeedID]*ChangefeedReactorState{ + "test2": { + ID: "test2", + TaskStatuses: map[string]*TaskStatus{}, + TaskPositions: map[CaptureID]*TaskPosition{}, + Workloads: map[CaptureID]TaskWorkload{ + "55551111": {46: {Workload: 1}}, + }, + }, + }, + }, + }, + } + for _, tc := range testCases { + state := NewGlobalState() + for i, k := range tc.updateKey { + value := []byte(tc.updateValue[i]) + if len(value) == 0 { + value = nil + } + err := state.Update(util.NewEtcdKey(k), value, false) + c.Assert(err, check.IsNil) + } + c.Assert(cmp.Equal(state, &tc.expected, cmpopts.IgnoreUnexported(GlobalReactorState{}, ChangefeedReactorState{})), check.IsTrue, + check.Commentf("%s", cmp.Diff(state, &tc.expected, cmpopts.IgnoreUnexported(GlobalReactorState{}, ChangefeedReactorState{})))) + } +} + +func (s *stateSuite) TestCheckChangefeedNormal(c *check.C) { + defer testleak.AfterTest(c)() + state := NewChangefeedReactorState("test1") + stateTester := orchestrator.NewReactorStateTester(c, state, nil) + state.CheckChangefeedNormal() + stateTester.MustApplyPatches() + state.PatchInfo(func(info *ChangeFeedInfo) (*ChangeFeedInfo, bool, error) { + return &ChangeFeedInfo{SinkURI: "123", AdminJobType: AdminNone, Config: &config.ReplicaConfig{}}, true, nil + }) + state.PatchStatus(func(status *ChangeFeedStatus) (*ChangeFeedStatus, bool, error) { + return &ChangeFeedStatus{ResolvedTs: 1, AdminJobType: AdminNone}, true, nil + }) + state.CheckChangefeedNormal() + stateTester.MustApplyPatches() + c.Assert(state.Status.ResolvedTs, check.Equals, uint64(1)) + + state.PatchInfo(func(info *ChangeFeedInfo) (*ChangeFeedInfo, bool, error) { + info.AdminJobType = AdminStop + return info, true, nil + }) + state.PatchStatus(func(status *ChangeFeedStatus) (*ChangeFeedStatus, bool, error) { + status.ResolvedTs = 2 + return status, true, nil + }) + state.CheckChangefeedNormal() + stateTester.MustApplyPatches() + c.Assert(state.Status.ResolvedTs, check.Equals, uint64(1)) + + state.PatchStatus(func(status *ChangeFeedStatus) (*ChangeFeedStatus, bool, error) { + status.ResolvedTs = 2 + return status, true, nil + }) + state.CheckChangefeedNormal() + stateTester.MustApplyPatches() + c.Assert(state.Status.ResolvedTs, check.Equals, uint64(2)) +} diff --git a/cdc/owner.go b/cdc/owner.go index 0e39e209ccc..fb6eb3d9a98 100644 --- a/cdc/owner.go +++ b/cdc/owner.go @@ -85,6 +85,15 @@ func (o *Owner) getMinGCSafePointCache(ctx context.Context) model.Ts { return o.minGCSafePointCache.ts } o.minGCSafePointCache.ts = oracle.ComposeTS(physicalTs-(o.gcTTL*1000), logicalTs) + + // o.pdGCSafePoint pd is the smallest gcSafePoint across all services. + // If tikv_gc_life_time > gcTTL, means that tikv_gc_safe_point < o.minGCSafePointCache.ts here. + // It also means that pd.pdGCSafePoint < o.minGCSafePointCache.ts here, we should use its value as the min value. + // This ensures that when tikv_gc_life_time > gcTTL , cdc will not advance the gcSafePoint. + if o.pdGCSafePoint < o.minGCSafePointCache.ts { + o.minGCSafePointCache.ts = o.pdGCSafePoint + } + o.minGCSafePointCache.lastUpdated = time.Now() } return o.minGCSafePointCache.ts @@ -127,6 +136,8 @@ type Owner struct { gcSafepointLastUpdate time.Time // stores the ts obtained from PD and is updated every MinGCSafePointCacheUpdateInterval. minGCSafePointCache minGCSafePointCacheEntry + // stores the actual gcSafePoint stored in pd + pdGCSafePoint model.Ts // record last time that flushes all changefeeds' replication status lastFlushChangefeeds time.Time flushChangefeedInterval time.Duration @@ -290,7 +301,7 @@ func (o *Owner) newChangeFeed( log.Info("Find new changefeed", zap.Stringer("info", info), zap.String("changefeed", id), zap.Uint64("checkpoint ts", checkpointTs)) if info.Config.CheckGCSafePoint { - err := util.CheckSafetyOfStartTs(ctx, o.pdClient, checkpointTs) + err := util.CheckSafetyOfStartTs(ctx, o.pdClient, id, checkpointTs) if err != nil { return nil, errors.Trace(err) } @@ -640,7 +651,7 @@ func (o *Owner) loadChangeFeeds(ctx context.Context) error { } cfInfo.ErrorHis = append(cfInfo.ErrorHis, time.Now().UnixNano()/1e6) - if filter.ChangefeedFastFailError(err) { + if cerror.ChangefeedFastFailError(err) { log.Error("create changefeed with fast fail error, mark changefeed as failed", zap.Error(err), zap.String("changefeed", changeFeedID)) cfInfo.State = model.StateFailed @@ -750,12 +761,18 @@ func (o *Owner) flushChangeFeedInfos(ctx context.Context) error { if changefeed.status.CheckpointTs < gcSafePoint { gcSafePoint = changefeed.status.CheckpointTs } - // If changefeed's appliedCheckpoinTs < minGCSafePoint, it means this changefeed is stagnant. + // 1. If changefeed's appliedCheckpoinTs <= minGCSafePoint, it means this changefeed is stagnant. // They are collected into this map, and then handleStaleChangeFeed() is called to deal with these stagnant changefeed. // A changefeed will not enter the map twice, because in run(), // handleAdminJob() will always be executed before flushChangeFeedInfos(), // ensuring that the previous changefeed in staleChangeFeeds has been stopped and removed from o.changeFeeds. - if changefeed.status.CheckpointTs < minGCSafePoint { + // 2. We need the `<=` check here is because when a changefeed is stagnant, its checkpointTs will be updated to pd, + // and it would be the minimum gcSafePoint across all services. + // So as described above(line 92) minGCSafePoint = gcSafePoint = CheckpointTs would happens. + // In this case, if we check `<` here , this changefeed will not be put into staleChangeFeeds, and its checkpoints will be updated to pd again and again. + // This will cause the cdc's gcSafePoint never advance. + // If we check `<=` here, when we encounter the changefeed again, we will put it into staleChangeFeeds. + if changefeed.status.CheckpointTs <= minGCSafePoint { staleChangeFeeds[id] = changefeed.status } @@ -778,10 +795,10 @@ func (o *Owner) flushChangeFeedInfos(ctx context.Context) error { } for _, status := range o.stoppedFeeds { - // If a stopped changefeed's CheckpoinTs < minGCSafePoint, means this changefeed is stagnant. + // If a stopped changefeed's CheckpoinTs <= minGCSafePoint, means this changefeed is stagnant. // It should never be resumed. This part of the logic is in newChangeFeed() // So here we can skip it. - if status.CheckpointTs < minGCSafePoint { + if status.CheckpointTs <= minGCSafePoint { continue } @@ -807,6 +824,7 @@ func (o *Owner) flushChangeFeedInfos(ctx context.Context) error { return cerror.ErrUpdateServiceSafepointFailed.Wrap(err) } } else { + o.pdGCSafePoint = actual o.gcSafepointLastUpdate = time.Now() } @@ -843,9 +861,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 @@ -921,7 +961,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 { @@ -1635,7 +1675,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)) @@ -1668,7 +1708,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. } } @@ -1679,15 +1719,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, diff --git a/cdc/owner/async_sink.go b/cdc/owner/async_sink.go new file mode 100644 index 00000000000..b6af446cc94 --- /dev/null +++ b/cdc/owner/async_sink.go @@ -0,0 +1,190 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package owner + +import ( + "context" + "sync" + "sync/atomic" + "time" + + "github.com/pingcap/errors" + "github.com/pingcap/failpoint" + "github.com/pingcap/log" + "github.com/pingcap/ticdc/cdc/model" + "github.com/pingcap/ticdc/cdc/sink" + cdcContext "github.com/pingcap/ticdc/pkg/context" + cerror "github.com/pingcap/ticdc/pkg/errors" + "github.com/pingcap/ticdc/pkg/filter" + "go.uber.org/zap" +) + +const ( + defaultErrChSize = 1024 +) + +// AsyncSink is a async sink design for owner +// The EmitCheckpointTs and EmitDDLEvent is asynchronous function for now +// Other functions are still synchronization +type AsyncSink interface { + Initialize(ctx cdcContext.Context, tableInfo []*model.SimpleTableInfo) error + // EmitCheckpointTs emits the checkpoint Ts to downstream data source + // this function will return after recording the checkpointTs specified in memory immediately + // and the recorded checkpointTs will be sent and updated to downstream data source every second + EmitCheckpointTs(ctx cdcContext.Context, ts uint64) + // EmitDDLEvent emits DDL event asynchronously and return true if the DDL is executed + // the DDL event will be sent to another goroutine and execute to downstream + // the caller of this function can call again and again until a true returned + EmitDDLEvent(ctx cdcContext.Context, ddl *model.DDLEvent) (bool, error) + SinkSyncpoint(ctx cdcContext.Context, checkpointTs uint64) error + Close() error +} + +type asyncSinkImpl struct { + sink sink.Sink + syncpointStore sink.SyncpointStore + + checkpointTs model.Ts + + lastSyncPoint model.Ts + + ddlCh chan *model.DDLEvent + ddlFinishedTs model.Ts + ddlSentTs model.Ts + + cancel context.CancelFunc + wg sync.WaitGroup + errCh chan error +} + +func newAsyncSink(ctx cdcContext.Context) (AsyncSink, error) { + ctx, cancel := cdcContext.WithCancel(ctx) + changefeedID := ctx.ChangefeedVars().ID + changefeedInfo := ctx.ChangefeedVars().Info + filter, err := filter.NewFilter(changefeedInfo.Config) + if err != nil { + return nil, errors.Trace(err) + } + errCh := make(chan error, defaultErrChSize) + s, err := sink.NewSink(ctx, changefeedID, changefeedInfo.SinkURI, filter, changefeedInfo.Config, changefeedInfo.Opts, errCh) + if err != nil { + return nil, errors.Trace(err) + } + asyncSink := &asyncSinkImpl{ + sink: s, + ddlCh: make(chan *model.DDLEvent, 1), + errCh: errCh, + cancel: cancel, + } + if changefeedInfo.SyncPointEnabled { + asyncSink.syncpointStore, err = sink.NewSyncpointStore(ctx, changefeedID, changefeedInfo.SinkURI) + if err != nil { + return nil, errors.Trace(err) + } + if err := asyncSink.syncpointStore.CreateSynctable(ctx); err != nil { + return nil, errors.Trace(err) + } + } + asyncSink.wg.Add(1) + go asyncSink.run(ctx) + return asyncSink, nil +} + +func (s *asyncSinkImpl) Initialize(ctx cdcContext.Context, tableInfo []*model.SimpleTableInfo) error { + return s.sink.Initialize(ctx, tableInfo) +} + +func (s *asyncSinkImpl) run(ctx cdcContext.Context) { + defer s.wg.Done() + // TODO make the tick duration configurable + ticker := time.NewTicker(time.Second) + defer ticker.Stop() + var lastCheckpointTs model.Ts + for { + select { + case <-ctx.Done(): + return + case err := <-s.errCh: + ctx.Throw(err) + return + case <-ticker.C: + checkpointTs := atomic.LoadUint64(&s.checkpointTs) + if checkpointTs == 0 || checkpointTs <= lastCheckpointTs { + continue + } + lastCheckpointTs = checkpointTs + if err := s.sink.EmitCheckpointTs(ctx, checkpointTs); err != nil { + ctx.Throw(errors.Trace(err)) + return + } + case ddl := <-s.ddlCh: + err := s.sink.EmitDDLEvent(ctx, ddl) + failpoint.Inject("InjectChangefeedDDLError", func() { + err = cerror.ErrExecDDLFailed.GenWithStackByArgs() + }) + if err == nil || cerror.ErrDDLEventIgnored.Equal(errors.Cause(err)) { + log.Info("Execute DDL succeeded", zap.String("changefeed", ctx.ChangefeedVars().ID), zap.Bool("ignored", err != nil), zap.Reflect("ddl", ddl)) + atomic.StoreUint64(&s.ddlFinishedTs, ddl.CommitTs) + } else { + // If DDL executing failed, and the error can not be ignored, throw an error and pause the changefeed + log.Error("Execute DDL failed", + zap.String("ChangeFeedID", ctx.ChangefeedVars().ID), + zap.Error(err), + zap.Reflect("ddl", ddl)) + ctx.Throw(errors.Trace(err)) + return + } + } + } +} + +func (s *asyncSinkImpl) EmitCheckpointTs(ctx cdcContext.Context, ts uint64) { + atomic.StoreUint64(&s.checkpointTs, ts) +} + +func (s *asyncSinkImpl) EmitDDLEvent(ctx cdcContext.Context, ddl *model.DDLEvent) (bool, error) { + ddlFinishedTs := atomic.LoadUint64(&s.ddlFinishedTs) + if ddl.CommitTs <= ddlFinishedTs { + return true, nil + } + if ddl.CommitTs <= s.ddlSentTs { + return false, nil + } + select { + case <-ctx.Done(): + return false, errors.Trace(ctx.Err()) + case s.ddlCh <- ddl: + } + s.ddlSentTs = ddl.CommitTs + return false, nil +} + +func (s *asyncSinkImpl) SinkSyncpoint(ctx cdcContext.Context, checkpointTs uint64) error { + if checkpointTs == s.lastSyncPoint { + return nil + } + s.lastSyncPoint = checkpointTs + // TODO implement async sink syncpoint + return s.syncpointStore.SinkSyncpoint(ctx, ctx.ChangefeedVars().ID, checkpointTs) +} + +func (s *asyncSinkImpl) Close() (err error) { + s.cancel() + err = s.sink.Close() + if s.syncpointStore != nil { + err = s.syncpointStore.Close() + } + s.wg.Wait() + return +} diff --git a/cdc/owner/async_sink_test.go b/cdc/owner/async_sink_test.go new file mode 100644 index 00000000000..5d20e3bef6c --- /dev/null +++ b/cdc/owner/async_sink_test.go @@ -0,0 +1,196 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package owner + +import ( + "context" + "sync" + "sync/atomic" + "time" + + "github.com/pingcap/check" + "github.com/pingcap/errors" + "github.com/pingcap/ticdc/cdc/model" + "github.com/pingcap/ticdc/cdc/sink" + "github.com/pingcap/ticdc/pkg/config" + cdcContext "github.com/pingcap/ticdc/pkg/context" + cerror "github.com/pingcap/ticdc/pkg/errors" + "github.com/pingcap/ticdc/pkg/retry" + "github.com/pingcap/ticdc/pkg/util/testleak" +) + +var _ = check.Suite(&asyncSinkSuite{}) + +type asyncSinkSuite struct { +} + +type mockSink struct { + sink.Sink + initTableInfo []*model.SimpleTableInfo + checkpointTs model.Ts + ddl *model.DDLEvent + ddlMu sync.Mutex + ddlError error +} + +func (m *mockSink) Initialize(ctx context.Context, tableInfo []*model.SimpleTableInfo) error { + m.initTableInfo = tableInfo + return nil +} + +func (m *mockSink) EmitCheckpointTs(ctx context.Context, ts uint64) error { + atomic.StoreUint64(&m.checkpointTs, ts) + return nil +} + +func (m *mockSink) EmitDDLEvent(ctx context.Context, ddl *model.DDLEvent) error { + m.ddlMu.Lock() + defer m.ddlMu.Unlock() + time.Sleep(1 * time.Second) + m.ddl = ddl + return m.ddlError +} + +func (m *mockSink) Close() error { + return nil +} + +func (m *mockSink) GetDDL() *model.DDLEvent { + m.ddlMu.Lock() + defer m.ddlMu.Unlock() + return m.ddl +} + +func newAsyncSink4Test(ctx cdcContext.Context, c *check.C) (cdcContext.Context, AsyncSink, *mockSink) { + ctx = cdcContext.WithChangefeedVars(ctx, &cdcContext.ChangefeedVars{ + ID: "test-changefeed", + Info: &model.ChangeFeedInfo{SinkURI: "blackhole://", Config: config.GetDefaultReplicaConfig()}, + }) + sink, err := newAsyncSink(ctx) + c.Assert(err, check.IsNil) + mockSink := &mockSink{} + sink.(*asyncSinkImpl).sink = mockSink + return ctx, sink, mockSink +} + +func (s *asyncSinkSuite) TestInitialize(c *check.C) { + defer testleak.AfterTest(c)() + ctx := cdcContext.NewBackendContext4Test(false) + ctx, sink, mockSink := newAsyncSink4Test(ctx, c) + defer sink.Close() + tableInfos := []*model.SimpleTableInfo{{Schema: "test"}} + err := sink.Initialize(ctx, tableInfos) + c.Assert(err, check.IsNil) + c.Assert(tableInfos, check.DeepEquals, mockSink.initTableInfo) +} + +func (s *asyncSinkSuite) TestCheckpoint(c *check.C) { + defer testleak.AfterTest(c)() + ctx := cdcContext.NewBackendContext4Test(false) + ctx, sink, mSink := newAsyncSink4Test(ctx, c) + defer sink.Close() + + waitCheckpointGrowingUp := func(m *mockSink, targetTs model.Ts) error { + return retry.Run(100*time.Millisecond, 30, func() error { + if targetTs != atomic.LoadUint64(&m.checkpointTs) { + return errors.New("targetTs!=checkpointTs") + } + return nil + }) + } + sink.EmitCheckpointTs(ctx, 1) + c.Assert(waitCheckpointGrowingUp(mSink, 1), check.IsNil) + sink.EmitCheckpointTs(ctx, 10) + c.Assert(waitCheckpointGrowingUp(mSink, 10), check.IsNil) +} + +func (s *asyncSinkSuite) TestExecDDL(c *check.C) { + defer testleak.AfterTest(c)() + ctx := cdcContext.NewBackendContext4Test(false) + ctx, sink, mSink := newAsyncSink4Test(ctx, c) + defer sink.Close() + ddl1 := &model.DDLEvent{CommitTs: 1} + for { + done, err := sink.EmitDDLEvent(ctx, ddl1) + c.Assert(err, check.IsNil) + if done { + c.Assert(mSink.GetDDL(), check.DeepEquals, ddl1) + break + } + } + ddl2 := &model.DDLEvent{CommitTs: 2} + ddl3 := &model.DDLEvent{CommitTs: 3} + _, err := sink.EmitDDLEvent(ctx, ddl2) + c.Assert(err, check.IsNil) + for { + done, err := sink.EmitDDLEvent(ctx, ddl2) + c.Assert(err, check.IsNil) + if done { + c.Assert(mSink.GetDDL(), check.DeepEquals, ddl2) + break + } + } + _, err = sink.EmitDDLEvent(ctx, ddl3) + c.Assert(err, check.IsNil) + for { + done, err := sink.EmitDDLEvent(ctx, ddl3) + c.Assert(err, check.IsNil) + if done { + c.Assert(mSink.GetDDL(), check.DeepEquals, ddl3) + break + } + } +} + +func (s *asyncSinkSuite) TestExecDDLError(c *check.C) { + defer testleak.AfterTest(c)() + ctx := cdcContext.NewBackendContext4Test(false) + var resultErr error + var resultErrMu sync.Mutex + getResultErr := func() error { + resultErrMu.Lock() + defer resultErrMu.Unlock() + return resultErr + } + ctx = cdcContext.WithErrorHandler(ctx, func(err error) error { + resultErrMu.Lock() + defer resultErrMu.Unlock() + resultErr = err + return nil + }) + ctx, sink, mSink := newAsyncSink4Test(ctx, c) + defer sink.Close() + mSink.ddlError = cerror.ErrDDLEventIgnored.GenWithStackByArgs() + ddl1 := &model.DDLEvent{CommitTs: 1} + for { + done, err := sink.EmitDDLEvent(ctx, ddl1) + c.Assert(err, check.IsNil) + if done { + c.Assert(mSink.GetDDL(), check.DeepEquals, ddl1) + break + } + } + c.Assert(getResultErr(), check.IsNil) + mSink.ddlError = cerror.ErrExecDDLFailed.GenWithStackByArgs() + ddl2 := &model.DDLEvent{CommitTs: 2} + for { + done, err := sink.EmitDDLEvent(ctx, ddl2) + c.Assert(err, check.IsNil) + if done || getResultErr() != nil { + c.Assert(mSink.GetDDL(), check.DeepEquals, ddl2) + break + } + } + c.Assert(cerror.ErrExecDDLFailed.Equal(errors.Cause(getResultErr())), check.IsTrue) +} diff --git a/cdc/owner/changefeed.go b/cdc/owner/changefeed.go new file mode 100644 index 00000000000..36b5fc91794 --- /dev/null +++ b/cdc/owner/changefeed.go @@ -0,0 +1,439 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package owner + +import ( + "context" + "sync" + "time" + + "github.com/pingcap/errors" + "github.com/pingcap/failpoint" + "github.com/pingcap/log" + timodel "github.com/pingcap/parser/model" + "github.com/pingcap/ticdc/cdc/model" + cdcContext "github.com/pingcap/ticdc/pkg/context" + cerror "github.com/pingcap/ticdc/pkg/errors" + "github.com/pingcap/ticdc/pkg/util" + "github.com/pingcap/tidb/sessionctx/binloginfo" + "github.com/pingcap/tidb/store/tikv/oracle" + "github.com/prometheus/client_golang/prometheus" + "go.uber.org/zap" +) + +type changefeed struct { + id model.ChangeFeedID + state *model.ChangefeedReactorState + + scheduler *scheduler + barriers *barriers + feedStateManager *feedStateManager + gcManager *gcManager + + schema *schemaWrap4Owner + sink AsyncSink + ddlPuller DDLPuller + initialized bool + + // only used for asyncExecDDL function + // ddlEventCache is not nil when the changefeed is executing a DDL event asynchronously + // After the DDL event has been executed, ddlEventCache will be set to nil. + ddlEventCache *model.DDLEvent + + errCh chan error + cancel context.CancelFunc + + // The changefeed will start some backend goroutines in the function `initialize`, + // such as DDLPuller, Sink, etc. + // `wg` is used to manage those backend goroutines. + // But it only manages the DDLPuller for now. + // TODO: manage the Sink and other backend goroutines. + wg sync.WaitGroup + + metricsChangefeedCheckpointTsGauge prometheus.Gauge + metricsChangefeedCheckpointTsLagGauge prometheus.Gauge + + newDDLPuller func(ctx cdcContext.Context, startTs uint64) (DDLPuller, error) + newSink func(ctx cdcContext.Context) (AsyncSink, error) +} + +func newChangefeed(id model.ChangeFeedID, gcManager *gcManager) *changefeed { + c := &changefeed{ + id: id, + scheduler: newScheduler(), + barriers: newBarriers(), + feedStateManager: new(feedStateManager), + gcManager: gcManager, + + errCh: make(chan error, defaultErrChSize), + cancel: func() {}, + + newDDLPuller: newDDLPuller, + } + c.newSink = newAsyncSink + return c +} + +func newChangefeed4Test( + id model.ChangeFeedID, gcManager *gcManager, + newDDLPuller func(ctx cdcContext.Context, startTs uint64) (DDLPuller, error), + newSink func(ctx cdcContext.Context) (AsyncSink, error), +) *changefeed { + c := newChangefeed(id, gcManager) + c.newDDLPuller = newDDLPuller + c.newSink = newSink + return c +} + +func (c *changefeed) Tick(ctx cdcContext.Context, state *model.ChangefeedReactorState, captures map[model.CaptureID]*model.CaptureInfo) { + ctx = cdcContext.WithErrorHandler(ctx, func(err error) error { + c.errCh <- errors.Trace(err) + return nil + }) + state.CheckCaptureAlive(ctx.GlobalVars().CaptureInfo.ID) + if err := c.tick(ctx, state, captures); err != nil { + log.Error("an error occurred in Owner", zap.String("changefeedID", c.state.ID), zap.Error(err)) + var code string + if rfcCode, ok := cerror.RFCCode(err); ok { + code = string(rfcCode) + } else { + code = string(cerror.ErrOwnerUnknown.RFCCode()) + } + c.feedStateManager.HandleError(&model.RunningError{ + Addr: util.CaptureAddrFromCtx(ctx), + Code: code, + Message: err.Error(), + }) + c.releaseResources() + } +} + +func (c *changefeed) tick(ctx cdcContext.Context, state *model.ChangefeedReactorState, captures map[model.CaptureID]*model.CaptureInfo) error { + c.state = state + c.feedStateManager.Tick(state) + checkpointTs := c.state.Info.GetCheckpointTs(c.state.Status) + switch c.state.Info.State { + case model.StateNormal, model.StateStopped, model.StateError: + if err := c.gcManager.CheckStaleCheckpointTs(ctx, checkpointTs); err != nil { + return errors.Trace(err) + } + } + if !c.feedStateManager.ShouldRunning() { + c.releaseResources() + return nil + } + if !c.preflightCheck(captures) { + return nil + } + if err := c.initialize(ctx); err != nil { + return errors.Trace(err) + } + + select { + case err := <-c.errCh: + return errors.Trace(err) + default: + } + + c.sink.EmitCheckpointTs(ctx, checkpointTs) + barrierTs, err := c.handleBarrier(ctx) + if err != nil { + return errors.Trace(err) + } + shouldUpdateState, err := c.scheduler.Tick(c.state, c.schema.AllPhysicalTables(), captures) + if err != nil { + return errors.Trace(err) + } + if shouldUpdateState { + c.updateStatus(barrierTs) + } + return nil +} + +func (c *changefeed) initialize(ctx cdcContext.Context) error { + if c.initialized { + return nil + } + // clean the errCh + // When the changefeed is resumed after being stopped, the changefeed instance will be reused, + // So we should make sure that the errCh is empty when the changefeed is restarting +LOOP: + for { + select { + case <-c.errCh: + default: + break LOOP + } + } + checkpointTs := c.state.Info.GetCheckpointTs(c.state.Status) + log.Info("initialize changefeed", zap.String("changefeed", c.state.ID), + zap.Stringer("info", c.state.Info), + zap.Uint64("checkpoint ts", checkpointTs)) + failpoint.Inject("NewChangefeedNoRetryError", func() { + failpoint.Return(cerror.ErrStartTsBeforeGC.GenWithStackByArgs(checkpointTs-300, checkpointTs)) + }) + + failpoint.Inject("NewChangefeedRetryError", func() { + failpoint.Return(errors.New("failpoint injected retriable error")) + }) + + if c.state.Info.Config.CheckGCSafePoint { + err := util.CheckSafetyOfStartTs(ctx, ctx.GlobalVars().PDClient, c.state.ID, checkpointTs) + if err != nil { + return errors.Trace(err) + } + } + if c.state.Info.SyncPointEnabled { + c.barriers.Update(syncPointBarrier, checkpointTs) + } + c.barriers.Update(ddlJobBarrier, checkpointTs) + c.barriers.Update(finishBarrier, c.state.Info.GetTargetTs()) + var err error + c.schema, err = newSchemaWrap4Owner(ctx.GlobalVars().KVStorage, checkpointTs, c.state.Info.Config) + if err != nil { + return errors.Trace(err) + } + cancelCtx, cancel := cdcContext.WithCancel(ctx) + c.cancel = cancel + c.sink, err = c.newSink(cancelCtx) + if err != nil { + return errors.Trace(err) + } + err = c.sink.Initialize(cancelCtx, c.schema.SinkTableInfos()) + if err != nil { + return errors.Trace(err) + } + c.ddlPuller, err = c.newDDLPuller(cancelCtx, checkpointTs) + if err != nil { + return errors.Trace(err) + } + c.wg.Add(1) + go func() { + defer c.wg.Done() + ctx.Throw(c.ddlPuller.Run(cancelCtx)) + }() + + // init metrics + c.metricsChangefeedCheckpointTsGauge = changefeedCheckpointTsGauge.WithLabelValues(c.id) + c.metricsChangefeedCheckpointTsLagGauge = changefeedCheckpointTsLagGauge.WithLabelValues(c.id) + c.initialized = true + return nil +} + +func (c *changefeed) releaseResources() { + if !c.initialized { + return + } + log.Info("close changefeed", zap.String("changefeed", c.state.ID), + zap.Stringer("info", c.state.Info)) + c.cancel() + c.cancel = func() {} + c.ddlPuller.Close() + c.schema = nil + if err := c.sink.Close(); err != nil { + log.Warn("Closing sink failed in Owner", zap.String("changefeedID", c.state.ID), zap.Error(err)) + } + c.wg.Wait() + changefeedCheckpointTsGauge.DeleteLabelValues(c.id) + changefeedCheckpointTsLagGauge.DeleteLabelValues(c.id) + c.metricsChangefeedCheckpointTsGauge = nil + c.metricsChangefeedCheckpointTsLagGauge = nil + c.initialized = false +} + +// preflightCheck makes sure that the metadata in Etcd is complete enough to run the tick. +// If the metadata is not complete, such as when the ChangeFeedStatus is nil, +// this function will reconstruct the lost metadata and skip this tick. +func (c *changefeed) preflightCheck(captures map[model.CaptureID]*model.CaptureInfo) (ok bool) { + ok = true + if c.state.Status == nil { + c.state.PatchStatus(func(status *model.ChangeFeedStatus) (*model.ChangeFeedStatus, bool, error) { + if status == nil { + status = &model.ChangeFeedStatus{ + // the changefeed status is nil when the changefeed is just created. + // the txn in start ts is not replicated at that time, + // so the checkpoint ts and resolved ts should less than start ts. + ResolvedTs: c.state.Info.StartTs - 1, + CheckpointTs: c.state.Info.StartTs - 1, + AdminJobType: model.AdminNone, + } + return status, true, nil + } + return status, false, nil + }) + ok = false + } + for captureID := range captures { + if _, exist := c.state.TaskStatuses[captureID]; !exist { + c.state.PatchTaskStatus(captureID, func(status *model.TaskStatus) (*model.TaskStatus, bool, error) { + if status == nil { + status = new(model.TaskStatus) + return status, true, nil + } + return status, false, nil + }) + ok = false + } + } + for captureID := range c.state.TaskStatuses { + if _, exist := captures[captureID]; !exist { + c.state.PatchTaskStatus(captureID, func(status *model.TaskStatus) (*model.TaskStatus, bool, error) { + return nil, status != nil, nil + }) + ok = false + } + } + + for captureID := range c.state.TaskPositions { + if _, exist := captures[captureID]; !exist { + c.state.PatchTaskPosition(captureID, func(position *model.TaskPosition) (*model.TaskPosition, bool, error) { + return nil, position != nil, nil + }) + ok = false + } + } + for captureID := range c.state.Workloads { + if _, exist := captures[captureID]; !exist { + c.state.PatchTaskWorkload(captureID, func(workload model.TaskWorkload) (model.TaskWorkload, bool, error) { + return nil, workload != nil, nil + }) + ok = false + } + } + return +} + +func (c *changefeed) handleBarrier(ctx cdcContext.Context) (uint64, error) { + barrierTp, barrierTs := c.barriers.Min() + blocked := (barrierTs == c.state.Status.CheckpointTs) && (barrierTs == c.state.Status.ResolvedTs) + switch barrierTp { + case ddlJobBarrier: + ddlResolvedTs, ddlJob := c.ddlPuller.FrontDDL() + if ddlJob == nil || ddlResolvedTs != barrierTs { + c.barriers.Update(ddlJobBarrier, ddlResolvedTs) + return barrierTs, nil + } + if !blocked { + return barrierTs, nil + } + done, err := c.asyncExecDDL(ctx, ddlJob) + if err != nil { + return 0, errors.Trace(err) + } + if !done { + return barrierTs, nil + } + c.ddlPuller.PopFrontDDL() + newDDLResolvedTs, _ := c.ddlPuller.FrontDDL() + c.barriers.Update(ddlJobBarrier, newDDLResolvedTs) + + case syncPointBarrier: + if !blocked { + return barrierTs, nil + } + nextSyncPointTs := oracle.ComposeTS(oracle.GetPhysical(oracle.GetTimeFromTS(barrierTs).Add(c.state.Info.SyncPointInterval)), 0) + if err := c.sink.SinkSyncpoint(ctx, barrierTs); err != nil { + return 0, errors.Trace(err) + } + c.barriers.Update(syncPointBarrier, nextSyncPointTs) + + case finishBarrier: + if !blocked { + return barrierTs, nil + } + c.feedStateManager.MarkFinished() + default: + log.Panic("Unknown barrier type", zap.Int("barrier type", int(barrierTp))) + } + return barrierTs, nil +} + +func (c *changefeed) asyncExecDDL(ctx cdcContext.Context, job *timodel.Job) (done bool, err error) { + if job.BinlogInfo == nil { + log.Warn("ignore the invalid DDL job", zap.Reflect("job", job)) + return true, nil + } + cyclicConfig := c.state.Info.Config.Cyclic + if cyclicConfig.IsEnabled() && !cyclicConfig.SyncDDL { + return true, nil + } + if c.ddlEventCache == nil || c.ddlEventCache.CommitTs != job.BinlogInfo.FinishedTS { + ddlEvent, err := c.schema.BuildDDLEvent(job) + if err != nil { + return false, errors.Trace(err) + } + err = c.schema.HandleDDL(job) + if err != nil { + return false, errors.Trace(err) + } + ddlEvent.Query = binloginfo.AddSpecialComment(ddlEvent.Query) + c.ddlEventCache = ddlEvent + } + if job.BinlogInfo.TableInfo != nil && c.schema.IsIneligibleTableID(job.BinlogInfo.TableInfo.ID) { + log.Warn("ignore the DDL job of ineligible table", zap.Reflect("job", job)) + return true, nil + } + done, err = c.sink.EmitDDLEvent(ctx, c.ddlEventCache) + if err != nil { + return false, err + } + if done { + c.ddlEventCache = nil + } + return done, nil +} + +func (c *changefeed) updateStatus(barrierTs model.Ts) { + resolvedTs := barrierTs + for _, position := range c.state.TaskPositions { + if resolvedTs > position.ResolvedTs { + resolvedTs = position.ResolvedTs + } + } + for _, taskStatus := range c.state.TaskStatuses { + for _, opt := range taskStatus.Operation { + if resolvedTs > opt.BoundaryTs { + resolvedTs = opt.BoundaryTs + } + } + } + checkpointTs := resolvedTs + for _, position := range c.state.TaskPositions { + if checkpointTs > position.CheckPointTs { + checkpointTs = position.CheckPointTs + } + } + c.state.PatchStatus(func(status *model.ChangeFeedStatus) (*model.ChangeFeedStatus, bool, error) { + changed := false + if status.ResolvedTs != resolvedTs { + status.ResolvedTs = resolvedTs + changed = true + } + if status.CheckpointTs != checkpointTs { + status.CheckpointTs = checkpointTs + changed = true + } + return status, changed, nil + }) + + phyTs := oracle.ExtractPhysical(checkpointTs) + c.metricsChangefeedCheckpointTsGauge.Set(float64(phyTs)) + // It is more accurate to get tso from PD, but in most cases since we have + // deployed NTP service, a little bias is acceptable here. + c.metricsChangefeedCheckpointTsLagGauge.Set(float64(oracle.GetPhysical(time.Now())-phyTs) / 1e3) +} + +func (c *changefeed) Close() { + c.releaseResources() +} diff --git a/cdc/owner/changefeed_test.go b/cdc/owner/changefeed_test.go new file mode 100644 index 00000000000..fa5e4075de6 --- /dev/null +++ b/cdc/owner/changefeed_test.go @@ -0,0 +1,307 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package owner + +import ( + "context" + "sync/atomic" + "time" + + "github.com/pingcap/check" + "github.com/pingcap/errors" + timodel "github.com/pingcap/parser/model" + "github.com/pingcap/ticdc/cdc/entry" + "github.com/pingcap/ticdc/cdc/model" + cdcContext "github.com/pingcap/ticdc/pkg/context" + "github.com/pingcap/ticdc/pkg/orchestrator" + "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tidb/store/tikv/oracle" +) + +type mockDDLPuller struct { + // DDLPuller + resolvedTs model.Ts + ddlQueue []*timodel.Job +} + +func (m *mockDDLPuller) FrontDDL() (uint64, *timodel.Job) { + if len(m.ddlQueue) > 0 { + return m.ddlQueue[0].BinlogInfo.FinishedTS, m.ddlQueue[0] + } + return m.resolvedTs, nil +} + +func (m *mockDDLPuller) PopFrontDDL() (uint64, *timodel.Job) { + if len(m.ddlQueue) > 0 { + job := m.ddlQueue[0] + m.ddlQueue = m.ddlQueue[1:] + return job.BinlogInfo.FinishedTS, job + } + return m.resolvedTs, nil +} + +func (m *mockDDLPuller) Close() {} + +func (m *mockDDLPuller) Run(ctx cdcContext.Context) error { + <-ctx.Done() + return nil +} + +type mockAsyncSink struct { + // AsyncSink + ddlExecuting *model.DDLEvent + ddlDone bool + checkpointTs model.Ts + syncPoint model.Ts + syncPointHis []model.Ts +} + +func (m *mockAsyncSink) EmitDDLEvent(ctx cdcContext.Context, ddl *model.DDLEvent) (bool, error) { + m.ddlExecuting = ddl + defer func() { m.ddlDone = false }() + return m.ddlDone, nil +} + +func (m *mockAsyncSink) SinkSyncpoint(ctx cdcContext.Context, checkpointTs uint64) error { + if checkpointTs == m.syncPoint { + return nil + } + m.syncPoint = checkpointTs + m.syncPointHis = append(m.syncPointHis, checkpointTs) + return nil +} + +func (m *mockAsyncSink) Initialize(ctx cdcContext.Context, tableInfo []*model.SimpleTableInfo) error { + return nil +} + +func (m *mockAsyncSink) EmitCheckpointTs(ctx cdcContext.Context, ts uint64) { + atomic.StoreUint64(&m.checkpointTs, ts) +} + +func (m *mockAsyncSink) Close() error { + return nil +} + +var _ = check.Suite(&changefeedSuite{}) + +type changefeedSuite struct { +} + +func createChangefeed4Test(ctx cdcContext.Context, c *check.C) (*changefeed, *model.ChangefeedReactorState, + map[model.CaptureID]*model.CaptureInfo, *orchestrator.ReactorStateTester) { + ctx.GlobalVars().PDClient = &mockPDClient{updateServiceGCSafePointFunc: func(ctx context.Context, serviceID string, ttl int64, safePoint uint64) (uint64, error) { + return safePoint, nil + }} + gcManager := newGCManager() + cf := newChangefeed4Test(ctx.ChangefeedVars().ID, gcManager, func(ctx cdcContext.Context, startTs uint64) (DDLPuller, error) { + return &mockDDLPuller{resolvedTs: startTs - 1}, nil + }, func(ctx cdcContext.Context) (AsyncSink, error) { + return &mockAsyncSink{}, nil + }) + state := model.NewChangefeedReactorState(ctx.ChangefeedVars().ID) + tester := orchestrator.NewReactorStateTester(c, state, nil) + state.PatchInfo(func(info *model.ChangeFeedInfo) (*model.ChangeFeedInfo, bool, error) { + c.Assert(info, check.IsNil) + info = ctx.ChangefeedVars().Info + return info, true, nil + }) + tester.MustUpdate("/tidb/cdc/capture/"+ctx.GlobalVars().CaptureInfo.ID, []byte(`{"id":"`+ctx.GlobalVars().CaptureInfo.ID+`","address":"127.0.0.1:8300"}`)) + tester.MustApplyPatches() + captures := map[model.CaptureID]*model.CaptureInfo{ctx.GlobalVars().CaptureInfo.ID: ctx.GlobalVars().CaptureInfo} + return cf, state, captures, tester +} + +func (s *changefeedSuite) TestPreCheck(c *check.C) { + defer testleak.AfterTest(c)() + ctx := cdcContext.NewBackendContext4Test(true) + cf, state, captures, tester := createChangefeed4Test(ctx, c) + cf.Tick(ctx, state, captures) + tester.MustApplyPatches() + c.Assert(state.Status, check.NotNil) + c.Assert(state.TaskStatuses, check.HasKey, ctx.GlobalVars().CaptureInfo.ID) + + // test clean the meta data of offline capture + offlineCaputreID := "offline-capture" + state.PatchTaskStatus(offlineCaputreID, func(status *model.TaskStatus) (*model.TaskStatus, bool, error) { + return new(model.TaskStatus), true, nil + }) + state.PatchTaskPosition(offlineCaputreID, func(position *model.TaskPosition) (*model.TaskPosition, bool, error) { + return new(model.TaskPosition), true, nil + }) + state.PatchTaskWorkload(offlineCaputreID, func(workload model.TaskWorkload) (model.TaskWorkload, bool, error) { + return make(model.TaskWorkload), true, nil + }) + tester.MustApplyPatches() + + cf.Tick(ctx, state, captures) + tester.MustApplyPatches() + c.Assert(state.Status, check.NotNil) + c.Assert(state.TaskStatuses, check.HasKey, ctx.GlobalVars().CaptureInfo.ID) + c.Assert(state.TaskStatuses, check.Not(check.HasKey), offlineCaputreID) + c.Assert(state.TaskPositions, check.Not(check.HasKey), offlineCaputreID) + c.Assert(state.Workloads, check.Not(check.HasKey), offlineCaputreID) +} + +func (s *changefeedSuite) TestInitialize(c *check.C) { + defer testleak.AfterTest(c)() + ctx := cdcContext.NewBackendContext4Test(true) + cf, state, captures, tester := createChangefeed4Test(ctx, c) + defer cf.Close() + // pre check + cf.Tick(ctx, state, captures) + tester.MustApplyPatches() + + // initialize + cf.Tick(ctx, state, captures) + tester.MustApplyPatches() + c.Assert(state.Status.CheckpointTs, check.Equals, ctx.ChangefeedVars().Info.StartTs-1) +} + +func (s *changefeedSuite) TestHandleError(c *check.C) { + defer testleak.AfterTest(c)() + ctx := cdcContext.NewBackendContext4Test(true) + cf, state, captures, tester := createChangefeed4Test(ctx, c) + defer cf.Close() + // pre check + cf.Tick(ctx, state, captures) + tester.MustApplyPatches() + + // initialize + cf.Tick(ctx, state, captures) + tester.MustApplyPatches() + + cf.errCh <- errors.New("fake error") + // handle error + cf.Tick(ctx, state, captures) + tester.MustApplyPatches() + c.Assert(state.Status.CheckpointTs, check.Equals, ctx.ChangefeedVars().Info.StartTs-1) + c.Assert(state.Info.Error.Message, check.Equals, "fake error") +} + +func (s *changefeedSuite) TestExecDDL(c *check.C) { + defer testleak.AfterTest(c)() + ctx := cdcContext.NewBackendContext4Test(true) + cf, state, captures, tester := createChangefeed4Test(ctx, c) + defer cf.Close() + helper := entry.NewSchemaTestHelper(c) + defer helper.Close() + tickThreeTime := func() { + cf.Tick(ctx, state, captures) + tester.MustApplyPatches() + cf.Tick(ctx, state, captures) + tester.MustApplyPatches() + cf.Tick(ctx, state, captures) + tester.MustApplyPatches() + } + // pre check and initialize + tickThreeTime() + + // ddl puller resolved ts grow uo + mockDDLPuller := cf.ddlPuller.(*mockDDLPuller) + mockDDLPuller.resolvedTs += 1000 + mockAsyncSink := cf.sink.(*mockAsyncSink) + // three tick to make sure all barriers set in initialize is handled + tickThreeTime() + c.Assert(state.Status.CheckpointTs, check.Equals, mockDDLPuller.resolvedTs) + + // handle create database + job := helper.DDL2Job("create database test1") + mockDDLPuller.resolvedTs += 1000 + job.BinlogInfo.FinishedTS = mockDDLPuller.resolvedTs + mockDDLPuller.ddlQueue = append(mockDDLPuller.ddlQueue, job) + tickThreeTime() + c.Assert(state.Status.CheckpointTs, check.Equals, mockDDLPuller.resolvedTs) + c.Assert(mockAsyncSink.ddlExecuting.Query, check.Equals, "create database test1") + + // executing the ddl finished + mockAsyncSink.ddlDone = true + mockDDLPuller.resolvedTs += 1000 + tickThreeTime() + c.Assert(state.Status.CheckpointTs, check.Equals, mockDDLPuller.resolvedTs) + + // handle create table + job = helper.DDL2Job("create table test1.test1(id int primary key)") + mockDDLPuller.resolvedTs += 1000 + job.BinlogInfo.FinishedTS = mockDDLPuller.resolvedTs + mockDDLPuller.ddlQueue = append(mockDDLPuller.ddlQueue, job) + tickThreeTime() + c.Assert(state.Status.CheckpointTs, check.Equals, mockDDLPuller.resolvedTs) + c.Assert(mockAsyncSink.ddlExecuting.Query, check.Equals, "create table test1.test1(id int primary key)") + + // executing the ddl finished + mockAsyncSink.ddlDone = true + mockDDLPuller.resolvedTs += 1000 + tickThreeTime() + c.Assert(state.TaskStatuses[ctx.GlobalVars().CaptureInfo.ID].Tables, check.HasKey, job.TableID) +} + +func (s *changefeedSuite) TestSyncPoint(c *check.C) { + defer testleak.AfterTest(c)() + ctx := cdcContext.NewBackendContext4Test(true) + ctx.ChangefeedVars().Info.SyncPointEnabled = true + ctx.ChangefeedVars().Info.SyncPointInterval = 1 * time.Second + cf, state, captures, tester := createChangefeed4Test(ctx, c) + defer cf.Close() + + // pre check + cf.Tick(ctx, state, captures) + tester.MustApplyPatches() + + // initialize + cf.Tick(ctx, state, captures) + tester.MustApplyPatches() + + mockDDLPuller := cf.ddlPuller.(*mockDDLPuller) + mockAsyncSink := cf.sink.(*mockAsyncSink) + // add 5s to resolvedTs + mockDDLPuller.resolvedTs = oracle.ComposeTS(oracle.GetPhysical(oracle.GetTimeFromTS(mockDDLPuller.resolvedTs).Add(5*time.Second)), 0) + // tick 20 times + for i := 0; i <= 20; i++ { + cf.Tick(ctx, state, captures) + tester.MustApplyPatches() + } + for i := 1; i < len(mockAsyncSink.syncPointHis); i++ { + // check the time interval between adjacent sync points is less or equal than one second + c.Assert(mockAsyncSink.syncPointHis[i]-mockAsyncSink.syncPointHis[i-1], check.LessEqual, uint64(1000<<18)) + } + c.Assert(len(mockAsyncSink.syncPointHis), check.GreaterEqual, 5) +} + +func (s *changefeedSuite) TestFinished(c *check.C) { + defer testleak.AfterTest(c)() + ctx := cdcContext.NewBackendContext4Test(true) + ctx.ChangefeedVars().Info.TargetTs = ctx.ChangefeedVars().Info.StartTs + 1000 + cf, state, captures, tester := createChangefeed4Test(ctx, c) + defer cf.Close() + + // pre check + cf.Tick(ctx, state, captures) + tester.MustApplyPatches() + + // initialize + cf.Tick(ctx, state, captures) + tester.MustApplyPatches() + + mockDDLPuller := cf.ddlPuller.(*mockDDLPuller) + mockDDLPuller.resolvedTs += 2000 + // tick many times to make sure the change feed is stopped + for i := 0; i <= 10; i++ { + cf.Tick(ctx, state, captures) + tester.MustApplyPatches() + } + + c.Assert(state.Status.CheckpointTs, check.Equals, state.Info.TargetTs) + c.Assert(state.Info.State, check.Equals, model.StateFinished) +} diff --git a/cdc/owner/ddl_puller.go b/cdc/owner/ddl_puller.go new file mode 100644 index 00000000000..dc39df706f7 --- /dev/null +++ b/cdc/owner/ddl_puller.go @@ -0,0 +1,166 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package owner + +import ( + "context" + "sync" + + "github.com/pingcap/errors" + "github.com/pingcap/log" + timodel "github.com/pingcap/parser/model" + "github.com/pingcap/ticdc/cdc/entry" + "github.com/pingcap/ticdc/cdc/model" + "github.com/pingcap/ticdc/cdc/puller" + "github.com/pingcap/ticdc/pkg/config" + cdcContext "github.com/pingcap/ticdc/pkg/context" + "github.com/pingcap/ticdc/pkg/filter" + "github.com/pingcap/ticdc/pkg/regionspan" + "github.com/pingcap/ticdc/pkg/util" + "go.uber.org/zap" + "golang.org/x/sync/errgroup" +) + +// DDLPuller is a wrapper of the Puller interface for the owner +// DDLPuller starts a puller, listens to the DDL range, adds the received DDLs into an internal queue +type DDLPuller interface { + // Run runs the DDLPuller + Run(ctx cdcContext.Context) error + // FrontDDL returns the first DDL job in the internal queue + FrontDDL() (uint64, *timodel.Job) + // PopFrontDDL returns and pops the first DDL job in the internal queue + PopFrontDDL() (uint64, *timodel.Job) + // Close closes the DDLPuller + Close() +} + +type ddlPullerImpl struct { + puller puller.Puller + filter *filter.Filter + + mu sync.Mutex + resolvedTS uint64 + pendingDDLJobs []*timodel.Job + cancel context.CancelFunc +} + +func newDDLPuller(ctx cdcContext.Context, startTs uint64) (DDLPuller, error) { + pdCli := ctx.GlobalVars().PDClient + conf := config.GetGlobalServerConfig() + f, err := filter.NewFilter(ctx.ChangefeedVars().Info.Config) + if err != nil { + return nil, errors.Trace(err) + } + var plr puller.Puller + kvStorage := ctx.GlobalVars().KVStorage + // kvStorage can be nil only in the test + if kvStorage != nil { + plr = puller.NewPuller(ctx, pdCli, conf.Security, kvStorage, startTs, + []regionspan.Span{regionspan.GetDDLSpan(), regionspan.GetAddIndexDDLSpan()}, + nil, false) + } + + return &ddlPullerImpl{ + puller: plr, + resolvedTS: startTs, + filter: f, + cancel: func() {}, + }, nil +} + +const ddlPullerName = "DDL_PULLER" + +func (h *ddlPullerImpl) Run(ctx cdcContext.Context) error { + ctx, cancel := cdcContext.WithCancel(ctx) + h.cancel = cancel + log.Debug("DDL puller started", zap.String("changefeed-id", ctx.ChangefeedVars().ID)) + stdCtx := util.PutTableInfoInCtx(ctx, -1, ddlPullerName) + errg, stdCtx := errgroup.WithContext(stdCtx) + ctx = cdcContext.WithStd(ctx, stdCtx) + + errg.Go(func() error { + return h.puller.Run(ctx) + }) + + rawDDLCh := puller.SortOutput(ctx, h.puller.Output()) + + receiveDDL := func(rawDDL *model.RawKVEntry) error { + if rawDDL == nil { + return nil + } + if rawDDL.OpType == model.OpTypeResolved { + h.mu.Lock() + defer h.mu.Unlock() + if rawDDL.CRTs > h.resolvedTS { + h.resolvedTS = rawDDL.CRTs + } + return nil + } + job, err := entry.UnmarshalDDL(rawDDL) + if err != nil { + return errors.Trace(err) + } + if job == nil { + return nil + } + if h.filter.ShouldDiscardDDL(job.Type) { + log.Info("discard the ddl job", zap.Int64("jobID", job.ID), zap.String("query", job.Query)) + return nil + } + h.mu.Lock() + defer h.mu.Unlock() + h.pendingDDLJobs = append(h.pendingDDLJobs, job) + return nil + } + + errg.Go(func() error { + for { + select { + case <-ctx.Done(): + return ctx.Err() + case e := <-rawDDLCh: + if err := receiveDDL(e); err != nil { + return errors.Trace(err) + } + } + } + }) + + return errg.Wait() +} + +func (h *ddlPullerImpl) FrontDDL() (uint64, *timodel.Job) { + h.mu.Lock() + defer h.mu.Unlock() + if len(h.pendingDDLJobs) == 0 { + return h.resolvedTS, nil + } + job := h.pendingDDLJobs[0] + return job.BinlogInfo.FinishedTS, job +} + +func (h *ddlPullerImpl) PopFrontDDL() (uint64, *timodel.Job) { + h.mu.Lock() + defer h.mu.Unlock() + if len(h.pendingDDLJobs) == 0 { + return h.resolvedTS, nil + } + job := h.pendingDDLJobs[0] + h.pendingDDLJobs = h.pendingDDLJobs[1:] + return job.BinlogInfo.FinishedTS, job +} + +func (h *ddlPullerImpl) Close() { + h.cancel() +} diff --git a/cdc/owner/ddl_puller_test.go b/cdc/owner/ddl_puller_test.go new file mode 100644 index 00000000000..371c0f5232a --- /dev/null +++ b/cdc/owner/ddl_puller_test.go @@ -0,0 +1,231 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package owner + +import ( + "context" + "encoding/json" + "sync" + "sync/atomic" + "time" + + "github.com/pingcap/check" + "github.com/pingcap/errors" + timodel "github.com/pingcap/parser/model" + "github.com/pingcap/ticdc/cdc/model" + cdcContext "github.com/pingcap/ticdc/pkg/context" + "github.com/pingcap/ticdc/pkg/retry" + "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tidb/util/codec" +) + +var _ = check.Suite(&ddlPullerSuite{}) + +type ddlPullerSuite struct { +} + +type mockPuller struct { + c *check.C + inCh chan *model.RawKVEntry + outCh chan *model.RawKVEntry + resolvedTs model.Ts +} + +func newMockPuller(c *check.C, startTs model.Ts) *mockPuller { + return &mockPuller{ + c: c, + inCh: make(chan *model.RawKVEntry), + outCh: make(chan *model.RawKVEntry), + resolvedTs: startTs - 1, + } +} + +func (m *mockPuller) Run(ctx context.Context) error { + for { + select { + case <-ctx.Done(): + return ctx.Err() + case e := <-m.inCh: + m.outCh <- e + atomic.StoreUint64(&m.resolvedTs, e.CRTs) + } + } +} + +func (m *mockPuller) GetResolvedTs() uint64 { + return atomic.LoadUint64(&m.resolvedTs) +} + +func (m *mockPuller) Output() <-chan *model.RawKVEntry { + return m.outCh +} + +func (m *mockPuller) IsInitialized() bool { + return true +} + +func (m *mockPuller) append(e *model.RawKVEntry) { + m.inCh <- e +} + +func (m *mockPuller) appendDDL(job *timodel.Job) { + b, err := json.Marshal(job) + m.c.Assert(err, check.IsNil) + ek := []byte("m") + ek = codec.EncodeBytes(ek, []byte("DDLJobList")) + ek = codec.EncodeUint(ek, uint64('l')) + ek = codec.EncodeInt(ek, 1) + m.append(&model.RawKVEntry{ + OpType: model.OpTypePut, + Key: ek, + Value: b, + StartTs: job.StartTS, + CRTs: job.BinlogInfo.FinishedTS, + }) +} + +func (m *mockPuller) appendResolvedTs(ts model.Ts) { + m.append(&model.RawKVEntry{ + OpType: model.OpTypeResolved, + CRTs: ts, + StartTs: ts, + }) +} + +func (s *ddlPullerSuite) TestPuller(c *check.C) { + defer testleak.AfterTest(c)() + startTs := uint64(10) + mockPuller := newMockPuller(c, startTs) + ctx := cdcContext.NewBackendContext4Test(true) + p, err := newDDLPuller(ctx, startTs) + c.Assert(err, check.IsNil) + p.(*ddlPullerImpl).puller = mockPuller + var wg sync.WaitGroup + wg.Add(1) + go func() { + defer wg.Done() + err := p.Run(ctx) + if errors.Cause(err) == context.Canceled { + err = nil + } + c.Assert(err, check.IsNil) + }() + defer wg.Wait() + defer p.Close() + + // test initialize state + resolvedTs, ddl := p.FrontDDL() + c.Assert(resolvedTs, check.Equals, startTs) + c.Assert(ddl, check.IsNil) + resolvedTs, ddl = p.PopFrontDDL() + c.Assert(resolvedTs, check.Equals, startTs) + c.Assert(ddl, check.IsNil) + + // test send resolvedTs + mockPuller.appendResolvedTs(15) + waitResolvedTsGrowing(c, p, 15) + + // test send ddl job out of order + mockPuller.appendDDL(&timodel.Job{ + ID: 2, + Type: timodel.ActionCreateTable, + StartTS: 5, + State: timodel.JobStateDone, + BinlogInfo: &timodel.HistoryInfo{FinishedTS: 18}, + }) + mockPuller.appendDDL(&timodel.Job{ + ID: 1, + Type: timodel.ActionCreateTable, + StartTS: 5, + State: timodel.JobStateDone, + BinlogInfo: &timodel.HistoryInfo{FinishedTS: 16}, + }) + resolvedTs, ddl = p.FrontDDL() + c.Assert(resolvedTs, check.Equals, uint64(15)) + c.Assert(ddl, check.IsNil) + mockPuller.appendResolvedTs(20) + waitResolvedTsGrowing(c, p, 16) + resolvedTs, ddl = p.FrontDDL() + c.Assert(resolvedTs, check.Equals, uint64(16)) + c.Assert(ddl.ID, check.Equals, int64(1)) + resolvedTs, ddl = p.PopFrontDDL() + c.Assert(resolvedTs, check.Equals, uint64(16)) + c.Assert(ddl.ID, check.Equals, int64(1)) + resolvedTs, ddl = p.PopFrontDDL() + c.Assert(resolvedTs, check.Equals, uint64(18)) + c.Assert(ddl.ID, check.Equals, int64(2)) + + // test add ddl job repeated + mockPuller.appendDDL(&timodel.Job{ + ID: 3, + Type: timodel.ActionCreateTable, + StartTS: 20, + State: timodel.JobStateDone, + BinlogInfo: &timodel.HistoryInfo{FinishedTS: 25}, + }) + mockPuller.appendDDL(&timodel.Job{ + ID: 3, + Type: timodel.ActionCreateTable, + StartTS: 20, + State: timodel.JobStateDone, + BinlogInfo: &timodel.HistoryInfo{FinishedTS: 25}, + }) + mockPuller.appendResolvedTs(30) + waitResolvedTsGrowing(c, p, 25) + + resolvedTs, ddl = p.PopFrontDDL() + c.Assert(resolvedTs, check.Equals, uint64(25)) + c.Assert(ddl.ID, check.Equals, int64(3)) + resolvedTs, ddl = p.PopFrontDDL() + c.Assert(resolvedTs, check.Equals, uint64(25)) + c.Assert(ddl.ID, check.Equals, int64(3)) + + waitResolvedTsGrowing(c, p, 30) + resolvedTs, ddl = p.PopFrontDDL() + c.Assert(resolvedTs, check.Equals, uint64(30)) + c.Assert(ddl, check.IsNil) + + // test add invalid ddl job + mockPuller.appendDDL(&timodel.Job{ + ID: 4, + Type: timodel.ActionLockTable, + StartTS: 20, + State: timodel.JobStateDone, + BinlogInfo: &timodel.HistoryInfo{FinishedTS: 35}, + }) + mockPuller.appendDDL(&timodel.Job{ + ID: 5, + Type: timodel.ActionCreateTable, + StartTS: 20, + State: timodel.JobStateCancelled, + BinlogInfo: &timodel.HistoryInfo{FinishedTS: 36}, + }) + mockPuller.appendResolvedTs(40) + waitResolvedTsGrowing(c, p, 40) + resolvedTs, ddl = p.PopFrontDDL() + // no ddl should be received + c.Assert(resolvedTs, check.Equals, uint64(40)) + c.Assert(ddl, check.IsNil) +} + +func waitResolvedTsGrowing(c *check.C, p DDLPuller, targetTs model.Ts) { + err := retry.Run(20*time.Millisecond, 100, func() error { + resolvedTs, _ := p.FrontDDL() + if resolvedTs < targetTs { + return errors.New("resolvedTs < targetTs") + } + return nil + }) + c.Assert(err, check.IsNil) +} diff --git a/cdc/owner/feed_state_manager.go b/cdc/owner/feed_state_manager.go new file mode 100644 index 00000000000..4527949a9a1 --- /dev/null +++ b/cdc/owner/feed_state_manager.go @@ -0,0 +1,285 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package owner + +import ( + "time" + + "github.com/pingcap/errors" + "github.com/pingcap/log" + "github.com/pingcap/ticdc/cdc/model" + cerrors "github.com/pingcap/ticdc/pkg/errors" + "go.uber.org/zap" +) + +// feedStateManager manages the feedState of a changefeed +// when the error, admin job happened, the feedStateManager is responsible for controlling the feedState +type feedStateManager struct { + state *model.ChangefeedReactorState + shouldBeRunning bool + + adminJobQueue []*model.AdminJob +} + +func (m *feedStateManager) Tick(state *model.ChangefeedReactorState) { + m.state = state + m.shouldBeRunning = true + defer func() { + if m.shouldBeRunning { + m.patchState(model.StateNormal) + } else { + m.cleanUpInfos() + } + }() + if m.handleAdminJob() { + // `handleAdminJob` returns true means that some admin jobs are pending + // skip to the next tick until all the admin jobs is handled + return + } + switch m.state.Info.State { + case model.StateStopped, model.StateFailed, model.StateRemoved, model.StateFinished: + m.shouldBeRunning = false + return + } + errs := m.errorsReportedByProcessors() + m.HandleError(errs...) +} + +func (m *feedStateManager) ShouldRunning() bool { + return m.shouldBeRunning +} + +func (m *feedStateManager) MarkFinished() { + if m.state == nil { + // when state is nil, it means that Tick has never been called + // skip this and wait for the next tick to finish the changefeed + return + } + m.pushAdminJob(&model.AdminJob{ + CfID: m.state.ID, + Type: model.AdminFinish, + }) +} + +func (m *feedStateManager) PushAdminJob(job *model.AdminJob) { + switch job.Type { + case model.AdminStop, model.AdminResume, model.AdminRemove: + default: + log.Panic("Can not handle this job", zap.String("changefeedID", m.state.ID), + zap.String("changefeedState", string(m.state.Info.State)), zap.Any("job", job)) + } + m.pushAdminJob(job) +} + +func (m *feedStateManager) handleAdminJob() (jobsPending bool) { + job := m.popAdminJob() + if job == nil || job.CfID != m.state.ID { + return false + } + log.Info("handle admin job", zap.String("changefeedID", m.state.ID), zap.Reflect("job", job)) + switch job.Type { + case model.AdminStop: + switch m.state.Info.State { + case model.StateNormal, model.StateError: + default: + log.Warn("can not pause the changefeed in the current state", zap.String("changefeedID", m.state.ID), + zap.String("changefeedState", string(m.state.Info.State)), zap.Any("job", job)) + return + } + m.shouldBeRunning = false + jobsPending = true + m.patchState(model.StateStopped) + case model.AdminRemove: + switch m.state.Info.State { + case model.StateNormal, model.StateError, model.StateFailed, + model.StateStopped, model.StateFinished, model.StateRemoved: + default: + log.Warn("can not remove the changefeed in the current state", zap.String("changefeedID", m.state.ID), + zap.String("changefeedState", string(m.state.Info.State)), zap.Any("job", job)) + return + } + m.shouldBeRunning = false + jobsPending = true + m.patchState(model.StateRemoved) + // remove changefeed info and state + m.state.PatchInfo(func(info *model.ChangeFeedInfo) (*model.ChangeFeedInfo, bool, error) { + return nil, true, nil + }) + m.state.PatchStatus(func(status *model.ChangeFeedStatus) (*model.ChangeFeedStatus, bool, error) { + return nil, true, nil + }) + checkpointTs := m.state.Info.GetCheckpointTs(m.state.Status) + log.Info("the changefeed removed", zap.String("changefeed-id", m.state.ID), zap.Uint64("checkpoint-ts", checkpointTs)) + + case model.AdminResume: + switch m.state.Info.State { + case model.StateFailed, model.StateError, model.StateStopped, model.StateFinished: + default: + log.Warn("can not resume the changefeed in the current state", zap.String("changefeedID", m.state.ID), + zap.String("changefeedState", string(m.state.Info.State)), zap.Any("job", job)) + return + } + m.shouldBeRunning = true + jobsPending = true + m.patchState(model.StateNormal) + // remove error history to make sure the changefeed can running in next tick + m.state.PatchInfo(func(info *model.ChangeFeedInfo) (*model.ChangeFeedInfo, bool, error) { + if info.Error != nil || len(info.ErrorHis) != 0 { + info.Error = nil + info.ErrorHis = nil + return info, true, nil + } + return info, false, nil + }) + case model.AdminFinish: + switch m.state.Info.State { + case model.StateNormal: + default: + log.Warn("can not finish the changefeed in the current state", zap.String("changefeedID", m.state.ID), + zap.String("changefeedState", string(m.state.Info.State)), zap.Any("job", job)) + return + } + m.shouldBeRunning = false + jobsPending = true + m.patchState(model.StateFinished) + default: + log.Warn("Unknown admin job", zap.Any("adminJob", job), zap.String("changefeed", m.state.ID)) + } + return +} + +func (m *feedStateManager) popAdminJob() *model.AdminJob { + if len(m.adminJobQueue) == 0 { + return nil + } + job := m.adminJobQueue[0] + m.adminJobQueue = m.adminJobQueue[1:] + return job +} + +func (m *feedStateManager) pushAdminJob(job *model.AdminJob) { + m.adminJobQueue = append(m.adminJobQueue, job) +} + +func (m *feedStateManager) patchState(feedState model.FeedState) { + var adminJobType model.AdminJobType + switch feedState { + case model.StateNormal: + adminJobType = model.AdminNone + case model.StateFinished: + adminJobType = model.AdminFinish + case model.StateError, model.StateStopped, model.StateFailed: + adminJobType = model.AdminStop + case model.StateRemoved: + adminJobType = model.AdminRemove + default: + log.Panic("Unreachable") + } + m.state.PatchStatus(func(status *model.ChangeFeedStatus) (*model.ChangeFeedStatus, bool, error) { + if status == nil { + return status, false, nil + } + if status.AdminJobType != adminJobType { + status.AdminJobType = adminJobType + return status, true, nil + } + return status, false, nil + }) + m.state.PatchInfo(func(info *model.ChangeFeedInfo) (*model.ChangeFeedInfo, bool, error) { + changed := false + if info.State != feedState { + info.State = feedState + changed = true + } + if info.AdminJobType != adminJobType { + info.AdminJobType = adminJobType + changed = true + } + return info, changed, nil + }) +} + +func (m *feedStateManager) cleanUpInfos() { + for captureID := range m.state.TaskStatuses { + m.state.PatchTaskStatus(captureID, func(status *model.TaskStatus) (*model.TaskStatus, bool, error) { + return nil, status != nil, nil + }) + } + for captureID := range m.state.TaskPositions { + m.state.PatchTaskPosition(captureID, func(position *model.TaskPosition) (*model.TaskPosition, bool, error) { + return nil, position != nil, nil + }) + } + for captureID := range m.state.Workloads { + m.state.PatchTaskWorkload(captureID, func(workload model.TaskWorkload) (model.TaskWorkload, bool, error) { + return nil, workload != nil, nil + }) + } +} + +func (m *feedStateManager) errorsReportedByProcessors() []*model.RunningError { + var runningErrors map[string]*model.RunningError + for captureID, position := range m.state.TaskPositions { + if position.Error != nil { + if runningErrors == nil { + runningErrors = make(map[string]*model.RunningError) + } + runningErrors[position.Error.Code] = position.Error + log.Error("processor report an error", zap.String("changefeedID", m.state.ID), zap.String("captureID", captureID), zap.Any("error", position.Error)) + m.state.PatchTaskPosition(captureID, func(position *model.TaskPosition) (*model.TaskPosition, bool, error) { + if position == nil { + return nil, false, nil + } + position.Error = nil + return position, true, nil + }) + } + } + if runningErrors == nil { + return nil + } + result := make([]*model.RunningError, 0, len(runningErrors)) + for _, err := range runningErrors { + result = append(result, err) + } + return result +} + +func (m *feedStateManager) HandleError(errs ...*model.RunningError) { + m.state.PatchInfo(func(info *model.ChangeFeedInfo) (*model.ChangeFeedInfo, bool, error) { + for _, err := range errs { + info.Error = err + info.ErrorHis = append(info.ErrorHis, time.Now().UnixNano()/1e6) + } + needSave := info.CleanUpOutdatedErrorHistory() + return info, needSave || len(errs) > 0, nil + }) + var err *model.RunningError + if len(errs) > 0 { + err = errs[len(errs)-1] + } + // if one of the error stored by changefeed state(error in the last tick) or the error specified by this function(error in the this tick) + // is a fast-fail error, the changefeed should be failed + if m.state.Info.HasFastFailError() || (err != nil && cerrors.ChangefeedFastFailErrorCode(errors.RFCErrorCode(err.Code))) { + m.shouldBeRunning = false + m.patchState(model.StateFailed) + return + } + // if the number of errors has reached the error threshold, stop the changefeed + if m.state.Info.ErrorsReachedThreshold() { + m.shouldBeRunning = false + m.patchState(model.StateError) + return + } +} diff --git a/cdc/owner/feed_state_manager_test.go b/cdc/owner/feed_state_manager_test.go new file mode 100644 index 00000000000..f16cf41594d --- /dev/null +++ b/cdc/owner/feed_state_manager_test.go @@ -0,0 +1,250 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package owner + +import ( + "github.com/pingcap/check" + "github.com/pingcap/ticdc/cdc/model" + "github.com/pingcap/ticdc/pkg/config" + cdcContext "github.com/pingcap/ticdc/pkg/context" + "github.com/pingcap/ticdc/pkg/orchestrator" + "github.com/pingcap/ticdc/pkg/util/testleak" +) + +var _ = check.Suite(&feedStateManagerSuite{}) + +type feedStateManagerSuite struct { +} + +func (s *feedStateManagerSuite) TestHandleJob(c *check.C) { + defer testleak.AfterTest(c)() + ctx := cdcContext.NewBackendContext4Test(true) + manager := new(feedStateManager) + state := model.NewChangefeedReactorState(ctx.ChangefeedVars().ID) + tester := orchestrator.NewReactorStateTester(c, state, nil) + state.PatchInfo(func(info *model.ChangeFeedInfo) (*model.ChangeFeedInfo, bool, error) { + c.Assert(info, check.IsNil) + return &model.ChangeFeedInfo{SinkURI: "123", Config: &config.ReplicaConfig{}}, true, nil + }) + state.PatchStatus(func(status *model.ChangeFeedStatus) (*model.ChangeFeedStatus, bool, error) { + c.Assert(status, check.IsNil) + return &model.ChangeFeedStatus{}, true, nil + }) + tester.MustApplyPatches() + manager.Tick(state) + tester.MustApplyPatches() + c.Assert(manager.ShouldRunning(), check.IsTrue) + + // an admin job which of changefeed is not match + manager.PushAdminJob(&model.AdminJob{ + CfID: "fake-changefeed-id", + Type: model.AdminStop, + }) + manager.Tick(state) + tester.MustApplyPatches() + c.Assert(manager.ShouldRunning(), check.IsTrue) + + // a running can not be resume + manager.PushAdminJob(&model.AdminJob{ + CfID: ctx.ChangefeedVars().ID, + Type: model.AdminResume, + }) + manager.Tick(state) + tester.MustApplyPatches() + c.Assert(manager.ShouldRunning(), check.IsTrue) + + // stop a changefeed + manager.PushAdminJob(&model.AdminJob{ + CfID: ctx.ChangefeedVars().ID, + Type: model.AdminStop, + }) + manager.Tick(state) + tester.MustApplyPatches() + c.Assert(manager.ShouldRunning(), check.IsFalse) + c.Assert(state.Info.State, check.Equals, model.StateStopped) + c.Assert(state.Info.AdminJobType, check.Equals, model.AdminStop) + c.Assert(state.Status.AdminJobType, check.Equals, model.AdminStop) + + // resume a changefeed + manager.PushAdminJob(&model.AdminJob{ + CfID: ctx.ChangefeedVars().ID, + Type: model.AdminResume, + }) + manager.Tick(state) + tester.MustApplyPatches() + c.Assert(manager.ShouldRunning(), check.IsTrue) + c.Assert(state.Info.State, check.Equals, model.StateNormal) + c.Assert(state.Info.AdminJobType, check.Equals, model.AdminNone) + c.Assert(state.Status.AdminJobType, check.Equals, model.AdminNone) + + // remove a changefeed + manager.PushAdminJob(&model.AdminJob{ + CfID: ctx.ChangefeedVars().ID, + Type: model.AdminRemove, + }) + manager.Tick(state) + tester.MustApplyPatches() + c.Assert(manager.ShouldRunning(), check.IsFalse) + c.Assert(state.Exist(), check.IsFalse) +} + +func (s *feedStateManagerSuite) TestMarkFinished(c *check.C) { + defer testleak.AfterTest(c)() + ctx := cdcContext.NewBackendContext4Test(true) + manager := new(feedStateManager) + state := model.NewChangefeedReactorState(ctx.ChangefeedVars().ID) + tester := orchestrator.NewReactorStateTester(c, state, nil) + state.PatchInfo(func(info *model.ChangeFeedInfo) (*model.ChangeFeedInfo, bool, error) { + c.Assert(info, check.IsNil) + return &model.ChangeFeedInfo{SinkURI: "123", Config: &config.ReplicaConfig{}}, true, nil + }) + state.PatchStatus(func(status *model.ChangeFeedStatus) (*model.ChangeFeedStatus, bool, error) { + c.Assert(status, check.IsNil) + return &model.ChangeFeedStatus{}, true, nil + }) + tester.MustApplyPatches() + manager.Tick(state) + tester.MustApplyPatches() + c.Assert(manager.ShouldRunning(), check.IsTrue) + + manager.MarkFinished() + manager.Tick(state) + tester.MustApplyPatches() + c.Assert(manager.ShouldRunning(), check.IsFalse) + c.Assert(state.Info.State, check.Equals, model.StateFinished) + c.Assert(state.Info.AdminJobType, check.Equals, model.AdminFinish) + c.Assert(state.Status.AdminJobType, check.Equals, model.AdminFinish) +} + +func (s *feedStateManagerSuite) TestCleanUpInfos(c *check.C) { + defer testleak.AfterTest(c)() + ctx := cdcContext.NewBackendContext4Test(true) + manager := new(feedStateManager) + state := model.NewChangefeedReactorState(ctx.ChangefeedVars().ID) + tester := orchestrator.NewReactorStateTester(c, state, nil) + state.PatchInfo(func(info *model.ChangeFeedInfo) (*model.ChangeFeedInfo, bool, error) { + c.Assert(info, check.IsNil) + return &model.ChangeFeedInfo{SinkURI: "123", Config: &config.ReplicaConfig{}}, true, nil + }) + state.PatchStatus(func(status *model.ChangeFeedStatus) (*model.ChangeFeedStatus, bool, error) { + c.Assert(status, check.IsNil) + return &model.ChangeFeedStatus{}, true, nil + }) + state.PatchTaskStatus(ctx.GlobalVars().CaptureInfo.ID, func(status *model.TaskStatus) (*model.TaskStatus, bool, error) { + return &model.TaskStatus{}, true, nil + }) + state.PatchTaskPosition(ctx.GlobalVars().CaptureInfo.ID, func(position *model.TaskPosition) (*model.TaskPosition, bool, error) { + return &model.TaskPosition{}, true, nil + }) + state.PatchTaskWorkload(ctx.GlobalVars().CaptureInfo.ID, func(workload model.TaskWorkload) (model.TaskWorkload, bool, error) { + return model.TaskWorkload{}, true, nil + }) + tester.MustApplyPatches() + c.Assert(state.TaskStatuses, check.HasKey, ctx.GlobalVars().CaptureInfo.ID) + c.Assert(state.TaskPositions, check.HasKey, ctx.GlobalVars().CaptureInfo.ID) + c.Assert(state.Workloads, check.HasKey, ctx.GlobalVars().CaptureInfo.ID) + manager.Tick(state) + tester.MustApplyPatches() + c.Assert(manager.ShouldRunning(), check.IsTrue) + + manager.MarkFinished() + manager.Tick(state) + tester.MustApplyPatches() + c.Assert(manager.ShouldRunning(), check.IsFalse) + c.Assert(state.Info.State, check.Equals, model.StateFinished) + c.Assert(state.Info.AdminJobType, check.Equals, model.AdminFinish) + c.Assert(state.Status.AdminJobType, check.Equals, model.AdminFinish) + c.Assert(state.TaskStatuses, check.Not(check.HasKey), ctx.GlobalVars().CaptureInfo.ID) + c.Assert(state.TaskPositions, check.Not(check.HasKey), ctx.GlobalVars().CaptureInfo.ID) + c.Assert(state.Workloads, check.Not(check.HasKey), ctx.GlobalVars().CaptureInfo.ID) +} + +func (s *feedStateManagerSuite) TestHandleError(c *check.C) { + defer testleak.AfterTest(c)() + ctx := cdcContext.NewBackendContext4Test(true) + manager := new(feedStateManager) + state := model.NewChangefeedReactorState(ctx.ChangefeedVars().ID) + tester := orchestrator.NewReactorStateTester(c, state, nil) + state.PatchInfo(func(info *model.ChangeFeedInfo) (*model.ChangeFeedInfo, bool, error) { + c.Assert(info, check.IsNil) + return &model.ChangeFeedInfo{SinkURI: "123", Config: &config.ReplicaConfig{}}, true, nil + }) + state.PatchStatus(func(status *model.ChangeFeedStatus) (*model.ChangeFeedStatus, bool, error) { + c.Assert(status, check.IsNil) + return &model.ChangeFeedStatus{}, true, nil + }) + state.PatchTaskStatus(ctx.GlobalVars().CaptureInfo.ID, func(status *model.TaskStatus) (*model.TaskStatus, bool, error) { + return &model.TaskStatus{}, true, nil + }) + state.PatchTaskPosition(ctx.GlobalVars().CaptureInfo.ID, func(position *model.TaskPosition) (*model.TaskPosition, bool, error) { + return &model.TaskPosition{Error: &model.RunningError{ + Addr: ctx.GlobalVars().CaptureInfo.AdvertiseAddr, + Code: "[CDC:ErrEtcdSessionDone]", + Message: "fake error for test", + }}, true, nil + }) + state.PatchTaskWorkload(ctx.GlobalVars().CaptureInfo.ID, func(workload model.TaskWorkload) (model.TaskWorkload, bool, error) { + return model.TaskWorkload{}, true, nil + }) + tester.MustApplyPatches() + manager.Tick(state) + tester.MustApplyPatches() + c.Assert(manager.ShouldRunning(), check.IsTrue) + // error reported by processor in task position should be cleaned + c.Assert(state.TaskPositions[ctx.GlobalVars().CaptureInfo.ID].Error, check.IsNil) + + // throw error more than history threshold to turn feed state into error + for i := 0; i < model.ErrorHistoryThreshold; i++ { + state.PatchTaskPosition(ctx.GlobalVars().CaptureInfo.ID, func(position *model.TaskPosition) (*model.TaskPosition, bool, error) { + return &model.TaskPosition{Error: &model.RunningError{ + Addr: ctx.GlobalVars().CaptureInfo.AdvertiseAddr, + Code: "[CDC:ErrEtcdSessionDone]", + Message: "fake error for test", + }}, true, nil + }) + tester.MustApplyPatches() + manager.Tick(state) + tester.MustApplyPatches() + } + c.Assert(manager.ShouldRunning(), check.IsFalse) + c.Assert(state.Info.State, check.Equals, model.StateError) + c.Assert(state.Info.AdminJobType, check.Equals, model.AdminStop) + c.Assert(state.Status.AdminJobType, check.Equals, model.AdminStop) +} + +func (s *feedStateManagerSuite) TestChangefeedStatusNotExist(c *check.C) { + defer testleak.AfterTest(c)() + ctx := cdcContext.NewBackendContext4Test(true) + manager := new(feedStateManager) + state := model.NewChangefeedReactorState(ctx.ChangefeedVars().ID) + tester := orchestrator.NewReactorStateTester(c, state, map[string]string{ + "/tidb/cdc/capture/d563bfc0-f406-4f34-bc7d-6dc2e35a44e5": `{"id":"d563bfc0-f406-4f34-bc7d-6dc2e35a44e5","address":"172.16.6.147:8300","version":"v5.0.0-master-dirty"}`, + "/tidb/cdc/changefeed/info/" + ctx.ChangefeedVars().ID: `{"sink-uri":"blackhole:///","opts":{},"create-time":"2021-06-05T00:44:15.065939487+08:00","start-ts":425381670108266496,"target-ts":0,"admin-job-type":1,"sort-engine":"unified","config":{"case-sensitive":true,"enable-old-value":true,"force-replicate":false,"check-gc-safe-point":true,"filter":{"rules":["*.*"],"ignore-txn-start-ts":null},"mounter":{"worker-num":16},"sink":{"dispatchers":null,"protocol":"default"},"cyclic-replication":{"enable":false,"replica-id":0,"filter-replica-ids":null,"id-buckets":0,"sync-ddl":false},"scheduler":{"type":"table-number","polling-time":-1}},"state":"failed","history":[],"error":{"addr":"172.16.6.147:8300","code":"CDC:ErrSnapshotLostByGC","message":"[CDC:ErrSnapshotLostByGC]fail to create or maintain changefeed due to snapshot loss caused by GC. checkpoint-ts 425381670108266496 is earlier than GC safepoint at 0"},"sync-point-enabled":false,"sync-point-interval":600000000000,"creator-version":"v5.0.0-master-dirty"}`, + "/tidb/cdc/owner/156579d017f84a68": "d563bfc0-f406-4f34-bc7d-6dc2e35a44e5", + }) + manager.Tick(state) + c.Assert(manager.ShouldRunning(), check.IsFalse) + tester.MustApplyPatches() + + manager.PushAdminJob(&model.AdminJob{ + CfID: ctx.ChangefeedVars().ID, + Type: model.AdminRemove, + Opts: &model.AdminJobOption{ForceRemove: true}, + }) + manager.Tick(state) + c.Assert(manager.ShouldRunning(), check.IsFalse) + tester.MustApplyPatches() + c.Assert(state.Info, check.IsNil) + c.Assert(state.Exist(), check.IsFalse) +} diff --git a/cdc/owner/gc_manager.go b/cdc/owner/gc_manager.go new file mode 100644 index 00000000000..4f265b9db3b --- /dev/null +++ b/cdc/owner/gc_manager.go @@ -0,0 +1,137 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package owner + +import ( + "math" + "time" + + "github.com/pingcap/errors" + "github.com/pingcap/failpoint" + "github.com/pingcap/log" + "github.com/pingcap/ticdc/cdc/model" + "github.com/pingcap/ticdc/pkg/config" + cdcContext "github.com/pingcap/ticdc/pkg/context" + cerror "github.com/pingcap/ticdc/pkg/errors" + "github.com/pingcap/tidb/store/tikv/oracle" + "go.uber.org/zap" +) + +const ( + // cdcServiceSafePointID is the ID of CDC service in pd.UpdateServiceGCSafePoint. + cdcServiceSafePointID = "ticdc" + pdTimeUpdateInterval = 10 * time.Minute +) + +// gcSafepointUpdateInterval is the minimual interval that CDC can update gc safepoint +var gcSafepointUpdateInterval = 1 * time.Minute + +type gcManager struct { + gcTTL int64 + + lastUpdatedTime time.Time + lastSucceededTime time.Time + lastSafePointTs uint64 + isTiCDCBlockGC bool + + pdPhysicalTimeCache time.Time + lastUpdatedPdTime time.Time +} + +func newGCManager() *gcManager { + serverConfig := config.GetGlobalServerConfig() + failpoint.Inject("InjectGcSafepointUpdateInterval", func(val failpoint.Value) { + gcSafepointUpdateInterval = time.Duration(val.(int) * int(time.Millisecond)) + }) + return &gcManager{ + lastSucceededTime: time.Now(), + gcTTL: serverConfig.GcTTL, + } +} + +func (m *gcManager) updateGCSafePoint(ctx cdcContext.Context, state *model.GlobalReactorState) error { + if time.Since(m.lastUpdatedTime) < gcSafepointUpdateInterval { + return nil + } + minCheckpointTs := uint64(math.MaxUint64) + for _, cfState := range state.Changefeeds { + if cfState.Info == nil { + continue + } + switch cfState.Info.State { + case model.StateNormal, model.StateStopped, model.StateError: + default: + continue + } + checkpointTs := cfState.Info.GetCheckpointTs(cfState.Status) + if minCheckpointTs > checkpointTs { + minCheckpointTs = checkpointTs + } + } + m.lastUpdatedTime = time.Now() + + actual, err := ctx.GlobalVars().PDClient.UpdateServiceGCSafePoint(ctx, cdcServiceSafePointID, m.gcTTL, minCheckpointTs) + if err != nil { + log.Warn("updateGCSafePoint failed", + zap.Uint64("safePointTs", minCheckpointTs), + zap.Error(err)) + if time.Since(m.lastSucceededTime) >= time.Second*time.Duration(m.gcTTL) { + return cerror.ErrUpdateServiceSafepointFailed.Wrap(err) + } + return nil + } + failpoint.Inject("InjectActualGCSafePoint", func(val failpoint.Value) { + actual = uint64(val.(int)) + }) + if actual > minCheckpointTs { + log.Warn("update gc safe point failed, the gc safe point is larger than checkpointTs", zap.Uint64("actual", actual), zap.Uint64("checkpointTs", minCheckpointTs)) + } + // if the min checkpoint ts is equal to the current gc safe point, + // it means that the service gc safe point set by TiCDC is the min service gc safe point + m.isTiCDCBlockGC = actual == minCheckpointTs + m.lastSafePointTs = actual + m.lastSucceededTime = time.Now() + return nil +} + +func (m *gcManager) currentTimeFromPDCached(ctx cdcContext.Context) (time.Time, error) { + if time.Since(m.lastUpdatedPdTime) <= pdTimeUpdateInterval { + return m.pdPhysicalTimeCache, nil + } + physical, logical, err := ctx.GlobalVars().PDClient.GetTS(ctx) + if err != nil { + return time.Now(), errors.Trace(err) + } + m.pdPhysicalTimeCache = oracle.GetTimeFromTS(oracle.ComposeTS(physical, logical)) + m.lastUpdatedPdTime = time.Now() + return m.pdPhysicalTimeCache, nil +} + +func (m *gcManager) CheckStaleCheckpointTs(ctx cdcContext.Context, checkpointTs model.Ts) error { + if m.isTiCDCBlockGC { + pdTime, err := m.currentTimeFromPDCached(ctx) + if err != nil { + return errors.Trace(err) + } + if pdTime.Sub(oracle.GetTimeFromTS(checkpointTs)) > time.Duration(m.gcTTL)*time.Second { + return cerror.ErrGCTTLExceeded.GenWithStackByArgs(checkpointTs, ctx.ChangefeedVars().ID) + } + } else { + // if `isTiCDCBlockGC` is false, it means there is another service gc point less than the min checkpoint ts. + if checkpointTs < m.lastSafePointTs { + return cerror.ErrSnapshotLostByGC.GenWithStackByArgs(checkpointTs, m.lastSafePointTs) + } + } + return nil +} diff --git a/cdc/owner/gc_manager_test.go b/cdc/owner/gc_manager_test.go new file mode 100644 index 00000000000..5ea2982ffff --- /dev/null +++ b/cdc/owner/gc_manager_test.go @@ -0,0 +1,171 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package owner + +import ( + "context" + "fmt" + "math" + "time" + + "github.com/pingcap/check" + "github.com/pingcap/errors" + "github.com/pingcap/ticdc/cdc/model" + cdcContext "github.com/pingcap/ticdc/pkg/context" + cerror "github.com/pingcap/ticdc/pkg/errors" + "github.com/pingcap/ticdc/pkg/orchestrator" + "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tidb/store/tikv/oracle" + pd "github.com/tikv/pd/client" +) + +var _ = check.Suite(&gcManagerSuite{}) + +type gcManagerSuite struct { +} + +type mockPDClient struct { + pd.Client + updateServiceGCSafePointFunc func(ctx context.Context, serviceID string, ttl int64, safePoint uint64) (uint64, error) +} + +func (m *mockPDClient) UpdateServiceGCSafePoint(ctx context.Context, serviceID string, ttl int64, safePoint uint64) (uint64, error) { + return m.updateServiceGCSafePointFunc(ctx, serviceID, ttl, safePoint) +} + +func (m *mockPDClient) GetTS(ctx context.Context) (int64, int64, error) { + return oracle.GetPhysical(time.Now()), 0, nil +} + +func (s *gcManagerSuite) TestUpdateGCSafePoint(c *check.C) { + defer testleak.AfterTest(c)() + gcManager := newGCManager() + ctx := cdcContext.NewBackendContext4Test(true) + mockPDClient := &mockPDClient{} + ctx.GlobalVars().PDClient = mockPDClient + state := model.NewGlobalState().(*model.GlobalReactorState) + tester := orchestrator.NewReactorStateTester(c, state, nil) + + // no changefeed, the gc safe point should be max uint64 + mockPDClient.updateServiceGCSafePointFunc = func(ctx context.Context, serviceID string, ttl int64, safePoint uint64) (uint64, error) { + c.Assert(safePoint, check.Equals, uint64(math.MaxUint64)) + return 0, nil + } + err := gcManager.updateGCSafePoint(ctx, state) + c.Assert(err, check.IsNil) + // add a stopped changefeed + changefeedID1 := "changefeed-test1" + changefeedID2 := "changefeed-test2" + tester.MustUpdate(fmt.Sprintf("/tidb/cdc/changefeed/info/%s", changefeedID1), []byte(`{"config":{"cyclic-replication":{}},"state":"failed"}`)) + tester.MustApplyPatches() + state.Changefeeds[changefeedID1].PatchStatus(func(status *model.ChangeFeedStatus) (*model.ChangeFeedStatus, bool, error) { + return &model.ChangeFeedStatus{CheckpointTs: 1}, true, nil + }) + tester.MustApplyPatches() + err = gcManager.updateGCSafePoint(ctx, state) + c.Assert(err, check.IsNil) + + // switch the state of changefeed to normal + state.Changefeeds[changefeedID1].PatchInfo(func(info *model.ChangeFeedInfo) (*model.ChangeFeedInfo, bool, error) { + info.State = model.StateNormal + return info, true, nil + }) + tester.MustApplyPatches() + // the gc safe point should be updated to 1(checkpoint Ts of changefeed-test1) + mockPDClient.updateServiceGCSafePointFunc = func(ctx context.Context, serviceID string, ttl int64, safePoint uint64) (uint64, error) { + c.Assert(serviceID, check.Equals, cdcServiceSafePointID) + c.Assert(ttl, check.Equals, gcManager.gcTTL) + c.Assert(safePoint, check.Equals, uint64(1)) + return 0, nil + } + err = gcManager.updateGCSafePoint(ctx, state) + c.Assert(err, check.IsNil) + + // add another changefeed + tester.MustUpdate(fmt.Sprintf("/tidb/cdc/changefeed/info/%s", changefeedID2), []byte(`{"config":{"cyclic-replication":{}},"state":"normal"}`)) + tester.MustApplyPatches() + state.Changefeeds[changefeedID1].PatchStatus(func(status *model.ChangeFeedStatus) (*model.ChangeFeedStatus, bool, error) { + return &model.ChangeFeedStatus{CheckpointTs: 20}, true, nil + }) + state.Changefeeds[changefeedID2].PatchStatus(func(status *model.ChangeFeedStatus) (*model.ChangeFeedStatus, bool, error) { + return &model.ChangeFeedStatus{CheckpointTs: 30}, true, nil + }) + tester.MustApplyPatches() + // the gc safe point should not be updated, because it was recently updated + mockPDClient.updateServiceGCSafePointFunc = func(ctx context.Context, serviceID string, ttl int64, safePoint uint64) (uint64, error) { + c.Errorf("should not update gc safe point") + return 0, nil + } + err = gcManager.updateGCSafePoint(ctx, state) + c.Assert(err, check.IsNil) + + // assume that the gc safe point updated one hour ago + gcManager.lastUpdatedTime = time.Now().Add(-time.Hour) + + // the gc safe point should be updated to 1(checkpoint Ts of changefeed-test1) + mockPDClient.updateServiceGCSafePointFunc = func(ctx context.Context, serviceID string, ttl int64, safePoint uint64) (uint64, error) { + c.Assert(serviceID, check.Equals, cdcServiceSafePointID) + c.Assert(ttl, check.Equals, gcManager.gcTTL) + c.Assert(safePoint, check.Equals, uint64(20)) + return 0, nil + } + err = gcManager.updateGCSafePoint(ctx, state) + c.Assert(err, check.IsNil) +} + +func (s *gcManagerSuite) TestTimeFromPD(c *check.C) { + defer testleak.AfterTest(c)() + gcManager := newGCManager() + ctx := cdcContext.NewBackendContext4Test(true) + mockPDClient := &mockPDClient{} + ctx.GlobalVars().PDClient = mockPDClient + t1, err := gcManager.currentTimeFromPDCached(ctx) + c.Assert(err, check.IsNil) + c.Assert(t1, check.Equals, gcManager.pdPhysicalTimeCache) + + time.Sleep(50 * time.Millisecond) + // should return cached time + t2, err := gcManager.currentTimeFromPDCached(ctx) + c.Assert(err, check.IsNil) + c.Assert(t2, check.Equals, gcManager.pdPhysicalTimeCache) + c.Assert(t2, check.Equals, t1) + + time.Sleep(50 * time.Millisecond) + // assume that the gc safe point updated one hour ago + gcManager.lastUpdatedPdTime = time.Now().Add(-time.Hour) + t3, err := gcManager.currentTimeFromPDCached(ctx) + c.Assert(err, check.IsNil) + c.Assert(t3, check.Equals, gcManager.pdPhysicalTimeCache) + // should return new time + c.Assert(t3, check.Not(check.Equals), t2) +} + +func (s *gcManagerSuite) TestCheckStaleCheckpointTs(c *check.C) { + defer testleak.AfterTest(c)() + gcManager := newGCManager() + gcManager.isTiCDCBlockGC = true + ctx := cdcContext.NewBackendContext4Test(true) + mockPDClient := &mockPDClient{} + ctx.GlobalVars().PDClient = mockPDClient + err := gcManager.CheckStaleCheckpointTs(ctx, 10) + c.Assert(cerror.ErrGCTTLExceeded.Equal(errors.Cause(err)), check.IsTrue) + + err = gcManager.CheckStaleCheckpointTs(ctx, oracle.ComposeTS(oracle.GetPhysical(time.Now()), 0)) + c.Assert(err, check.IsNil) + + gcManager.isTiCDCBlockGC = false + gcManager.lastSafePointTs = 20 + err = gcManager.CheckStaleCheckpointTs(ctx, 10) + c.Assert(cerror.ErrSnapshotLostByGC.Equal(errors.Cause(err)), check.IsTrue) +} diff --git a/cdc/owner/metrics.go b/cdc/owner/metrics.go new file mode 100644 index 00000000000..0b1ae01d765 --- /dev/null +++ b/cdc/owner/metrics.go @@ -0,0 +1,62 @@ +// 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 owner + +import "github.com/prometheus/client_golang/prometheus" + +var ( + changefeedCheckpointTsGauge = prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Namespace: "ticdc", + Subsystem: "owner", + Name: "checkpoint_ts", + Help: "checkpoint ts of changefeeds", + }, []string{"changefeed"}) + changefeedCheckpointTsLagGauge = prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Namespace: "ticdc", + Subsystem: "owner", + Name: "checkpoint_ts_lag", + Help: "checkpoint ts lag of changefeeds", + }, []string{"changefeed"}) + ownershipCounter = prometheus.NewCounter( + prometheus.CounterOpts{ + Namespace: "ticdc", + Subsystem: "owner", + Name: "ownership_counter", + Help: "The counter of ownership increases every 5 seconds on a owner capture", + }) + ownerMaintainTableNumGauge = prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Namespace: "ticdc", + Subsystem: "owner", + Name: "maintain_table_num", + Help: "number of replicated tables maintained in owner", + }, []string{"changefeed", "capture", "type"}) +) + +const ( + // total tables that have been dispatched to a single processor + maintainTableTypeTotal string = "total" + // tables that are dispatched to a processor and have not been finished yet + maintainTableTypeWip string = "wip" +) + +// InitMetrics registers all metrics used in owner +func InitMetrics(registry *prometheus.Registry) { + registry.MustRegister(changefeedCheckpointTsGauge) + registry.MustRegister(changefeedCheckpointTsLagGauge) + registry.MustRegister(ownershipCounter) + registry.MustRegister(ownerMaintainTableNumGauge) +} diff --git a/cdc/owner/owner.go b/cdc/owner/owner.go new file mode 100644 index 00000000000..18b2f51904a --- /dev/null +++ b/cdc/owner/owner.go @@ -0,0 +1,297 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package owner + +import ( + "context" + "fmt" + "io" + "sync" + "sync/atomic" + "time" + + "github.com/pingcap/errors" + "github.com/pingcap/failpoint" + "github.com/pingcap/log" + "github.com/pingcap/ticdc/cdc/model" + cdcContext "github.com/pingcap/ticdc/pkg/context" + cerror "github.com/pingcap/ticdc/pkg/errors" + "github.com/pingcap/ticdc/pkg/orchestrator" + "github.com/pingcap/ticdc/pkg/version" + "go.uber.org/zap" +) + +type ownerJobType int + +// All AdminJob types +const ( + ownerJobTypeRebalance ownerJobType = iota + ownerJobTypeManualSchedule + ownerJobTypeAdminJob + ownerJobTypeDebugInfo +) + +type ownerJob struct { + tp ownerJobType + changefeedID model.ChangeFeedID + + // for ManualSchedule only + targetCaptureID model.CaptureID + // for ManualSchedule only + tableID model.TableID + + // for Admin Job only + adminJob *model.AdminJob + + // for debug info only + debugInfoWriter io.Writer + + done chan struct{} +} + +// Owner manages many changefeeds +// All public functions are THREAD-SAFE, except for Tick, Tick is only used for etcd worker +type Owner struct { + changefeeds map[model.ChangeFeedID]*changefeed + + gcManager *gcManager + + ownerJobQueueMu sync.Mutex + ownerJobQueue []*ownerJob + + lastTickTime time.Time + + closed int32 + + newChangefeed func(id model.ChangeFeedID, gcManager *gcManager) *changefeed +} + +// NewOwner creates a new Owner +func NewOwner() *Owner { + return &Owner{ + changefeeds: make(map[model.ChangeFeedID]*changefeed), + gcManager: newGCManager(), + lastTickTime: time.Now(), + newChangefeed: newChangefeed, + } +} + +// NewOwner4Test creates a new Owner for test +func NewOwner4Test( + newDDLPuller func(ctx cdcContext.Context, startTs uint64) (DDLPuller, error), + newSink func(ctx cdcContext.Context) (AsyncSink, error)) *Owner { + o := NewOwner() + o.newChangefeed = func(id model.ChangeFeedID, gcManager *gcManager) *changefeed { + return newChangefeed4Test(id, gcManager, newDDLPuller, newSink) + } + return o +} + +// Tick implements the Reactor interface +func (o *Owner) Tick(stdCtx context.Context, rawState orchestrator.ReactorState) (nextState orchestrator.ReactorState, err error) { + failpoint.Inject("owner-run-with-error", func() { + failpoint.Return(nil, errors.New("owner run with injected error")) + }) + failpoint.Inject("sleep-in-owner-tick", nil) + ctx := stdCtx.(cdcContext.Context) + state := rawState.(*model.GlobalReactorState) + o.updateMetrics(state) + if !o.clusterVersionConsistent(state.Captures) { + // sleep one second to avoid printing too much log + time.Sleep(1 * time.Second) + return state, nil + } + err = o.gcManager.updateGCSafePoint(ctx, state) + if err != nil { + return nil, errors.Trace(err) + } + o.handleJobs() + for changefeedID, changefeedState := range state.Changefeeds { + if changefeedState.Info == nil { + o.cleanUpChangefeed(changefeedState) + continue + } + ctx = cdcContext.WithChangefeedVars(ctx, &cdcContext.ChangefeedVars{ + ID: changefeedID, + Info: changefeedState.Info, + }) + cfReactor, exist := o.changefeeds[changefeedID] + if !exist { + cfReactor = o.newChangefeed(changefeedID, o.gcManager) + o.changefeeds[changefeedID] = cfReactor + } + cfReactor.Tick(ctx, changefeedState, state.Captures) + } + if len(o.changefeeds) != len(state.Changefeeds) { + for changefeedID, cfReactor := range o.changefeeds { + if _, exist := state.Changefeeds[changefeedID]; exist { + continue + } + cfReactor.Close() + delete(o.changefeeds, changefeedID) + } + } + if atomic.LoadInt32(&o.closed) != 0 { + for _, cfReactor := range o.changefeeds { + cfReactor.Close() + } + return state, cerror.ErrReactorFinished.GenWithStackByArgs() + } + return state, nil +} + +// EnqueueJob enqueues a admin job into a internal queue, and the Owner will handle the job in the next tick +func (o *Owner) EnqueueJob(adminJob model.AdminJob) { + o.pushOwnerJob(&ownerJob{ + tp: ownerJobTypeAdminJob, + adminJob: &adminJob, + changefeedID: adminJob.CfID, + done: make(chan struct{}), + }) +} + +// TriggerRebalance triggers a rebalance for the specified changefeed +func (o *Owner) TriggerRebalance(cfID model.ChangeFeedID) { + o.pushOwnerJob(&ownerJob{ + tp: ownerJobTypeRebalance, + changefeedID: cfID, + done: make(chan struct{}), + }) +} + +// ManualSchedule moves a table from a capture to another capture +func (o *Owner) ManualSchedule(cfID model.ChangeFeedID, toCapture model.CaptureID, tableID model.TableID) { + o.pushOwnerJob(&ownerJob{ + tp: ownerJobTypeManualSchedule, + changefeedID: cfID, + targetCaptureID: toCapture, + tableID: tableID, + done: make(chan struct{}), + }) +} + +// WriteDebugInfo writes debug info into the specified http writer +func (o *Owner) WriteDebugInfo(w io.Writer) { + timeout := time.Second * 3 + done := make(chan struct{}) + o.pushOwnerJob(&ownerJob{ + tp: ownerJobTypeDebugInfo, + debugInfoWriter: w, + done: done, + }) + // wait the debug info printed + select { + case <-done: + case <-time.After(timeout): + fmt.Fprintf(w, "failed to print debug info for owner\n") + } +} + +// AsyncStop stops the owner asynchronously +func (o *Owner) AsyncStop() { + atomic.StoreInt32(&o.closed, 1) +} + +func (o *Owner) cleanUpChangefeed(state *model.ChangefeedReactorState) { + state.PatchInfo(func(info *model.ChangeFeedInfo) (*model.ChangeFeedInfo, bool, error) { + return nil, info != nil, nil + }) + state.PatchStatus(func(status *model.ChangeFeedStatus) (*model.ChangeFeedStatus, bool, error) { + return nil, status != nil, nil + }) + for captureID := range state.TaskStatuses { + state.PatchTaskStatus(captureID, func(status *model.TaskStatus) (*model.TaskStatus, bool, error) { + return nil, status != nil, nil + }) + } + for captureID := range state.TaskPositions { + state.PatchTaskPosition(captureID, func(position *model.TaskPosition) (*model.TaskPosition, bool, error) { + return nil, position != nil, nil + }) + } + for captureID := range state.Workloads { + state.PatchTaskWorkload(captureID, func(workload model.TaskWorkload) (model.TaskWorkload, bool, error) { + return nil, workload != nil, nil + }) + } +} + +func (o *Owner) updateMetrics(state *model.GlobalReactorState) { + // Keep the value of prometheus expression `rate(counter)` = 1 + // Please also change alert rule in ticdc.rules.yml when change the expression value. + now := time.Now() + ownershipCounter.Add(float64(now.Sub(o.lastTickTime)) / float64(time.Second)) + o.lastTickTime = now + + ownerMaintainTableNumGauge.Reset() + for changefeedID, changefeedState := range state.Changefeeds { + for captureID, captureInfo := range state.Captures { + taskStatus, exist := changefeedState.TaskStatuses[captureID] + if !exist { + continue + } + ownerMaintainTableNumGauge.WithLabelValues(changefeedID, captureInfo.AdvertiseAddr, maintainTableTypeTotal).Set(float64(len(taskStatus.Tables))) + ownerMaintainTableNumGauge.WithLabelValues(changefeedID, captureInfo.AdvertiseAddr, maintainTableTypeWip).Set(float64(len(taskStatus.Operation))) + } + } +} + +func (o *Owner) clusterVersionConsistent(captures map[model.CaptureID]*model.CaptureInfo) bool { + myVersion := version.ReleaseVersion + for _, capture := range captures { + if myVersion != capture.Version { + log.Warn("the capture version is different with the owner", zap.Reflect("capture", capture), zap.String("my-version", myVersion)) + return false + } + } + return true +} + +func (o *Owner) handleJobs() { + jobs := o.takeOnwerJobs() + for _, job := range jobs { + changefeedID := job.changefeedID + cfReactor, exist := o.changefeeds[changefeedID] + if !exist { + log.Warn("changefeed not found when handle a job", zap.Reflect("job", job)) + continue + } + switch job.tp { + case ownerJobTypeAdminJob: + cfReactor.feedStateManager.PushAdminJob(job.adminJob) + case ownerJobTypeManualSchedule: + cfReactor.scheduler.MoveTable(job.tableID, job.targetCaptureID) + case ownerJobTypeRebalance: + cfReactor.scheduler.Rebalance() + case ownerJobTypeDebugInfo: + // TODO: implement this function + } + close(job.done) + } +} + +func (o *Owner) takeOnwerJobs() []*ownerJob { + o.ownerJobQueueMu.Lock() + defer o.ownerJobQueueMu.Unlock() + + jobs := o.ownerJobQueue + o.ownerJobQueue = nil + return jobs +} + +func (o *Owner) pushOwnerJob(job *ownerJob) { + o.ownerJobQueueMu.Lock() + defer o.ownerJobQueueMu.Unlock() + o.ownerJobQueue = append(o.ownerJobQueue, job) +} diff --git a/cdc/owner/owner_test.go b/cdc/owner/owner_test.go new file mode 100644 index 00000000000..95536187a76 --- /dev/null +++ b/cdc/owner/owner_test.go @@ -0,0 +1,217 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package owner + +import ( + "bytes" + "context" + "time" + + "github.com/pingcap/check" + "github.com/pingcap/ticdc/cdc/model" + "github.com/pingcap/ticdc/pkg/config" + cdcContext "github.com/pingcap/ticdc/pkg/context" + "github.com/pingcap/ticdc/pkg/etcd" + "github.com/pingcap/ticdc/pkg/orchestrator" + "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tidb/store/tikv/oracle" +) + +var _ = check.Suite(&ownerSuite{}) + +type ownerSuite struct { +} + +func createOwner4Test(ctx cdcContext.Context, c *check.C) (*Owner, *model.GlobalReactorState, *orchestrator.ReactorStateTester) { + ctx.GlobalVars().PDClient = &mockPDClient{updateServiceGCSafePointFunc: func(ctx context.Context, serviceID string, ttl int64, safePoint uint64) (uint64, error) { + return safePoint, nil + }} + cf := NewOwner4Test(func(ctx cdcContext.Context, startTs uint64) (DDLPuller, error) { + return &mockDDLPuller{resolvedTs: startTs - 1}, nil + }, func(ctx cdcContext.Context) (AsyncSink, error) { + return &mockAsyncSink{}, nil + }) + state := model.NewGlobalState().(*model.GlobalReactorState) + tester := orchestrator.NewReactorStateTester(c, state, nil) + + // set captures + cdcKey := etcd.CDCKey{ + Tp: etcd.CDCKeyTypeCapture, + CaptureID: ctx.GlobalVars().CaptureInfo.ID, + } + captureBytes, err := ctx.GlobalVars().CaptureInfo.Marshal() + c.Assert(err, check.IsNil) + tester.MustUpdate(cdcKey.String(), captureBytes) + return cf, state, tester +} + +func (s *ownerSuite) TestCreateRemoveChangefeed(c *check.C) { + defer testleak.AfterTest(c)() + ctx := cdcContext.NewBackendContext4Test(false) + owner, state, tester := createOwner4Test(ctx, c) + changefeedID := "test-changefeed" + changefeedInfo := &model.ChangeFeedInfo{ + StartTs: oracle.ComposeTS(oracle.GetPhysical(time.Now()), 0), + Config: config.GetDefaultReplicaConfig(), + } + changefeedStr, err := changefeedInfo.Marshal() + c.Assert(err, check.IsNil) + cdcKey := etcd.CDCKey{ + Tp: etcd.CDCKeyTypeChangefeedInfo, + ChangefeedID: changefeedID, + } + tester.MustUpdate(cdcKey.String(), []byte(changefeedStr)) + _, err = owner.Tick(ctx, state) + tester.MustApplyPatches() + c.Assert(err, check.IsNil) + c.Assert(owner.changefeeds, check.HasKey, changefeedID) + + // delete changefeed info key to remove changefeed + tester.MustUpdate(cdcKey.String(), nil) + // this tick to clean the leak info fo the removed changefeed + _, err = owner.Tick(ctx, state) + c.Assert(err, check.IsNil) + // this tick to remove the changefeed state in memory + tester.MustApplyPatches() + _, err = owner.Tick(ctx, state) + c.Assert(err, check.IsNil) + tester.MustApplyPatches() + c.Assert(err, check.IsNil) + c.Assert(owner.changefeeds, check.Not(check.HasKey), changefeedID) + c.Assert(state.Changefeeds, check.Not(check.HasKey), changefeedID) +} + +func (s *ownerSuite) TestStopChangefeed(c *check.C) { + defer testleak.AfterTest(c)() + ctx := cdcContext.NewBackendContext4Test(false) + owner, state, tester := createOwner4Test(ctx, c) + changefeedID := "test-changefeed" + changefeedInfo := &model.ChangeFeedInfo{ + StartTs: oracle.ComposeTS(oracle.GetPhysical(time.Now()), 0), + Config: config.GetDefaultReplicaConfig(), + } + changefeedStr, err := changefeedInfo.Marshal() + c.Assert(err, check.IsNil) + cdcKey := etcd.CDCKey{ + Tp: etcd.CDCKeyTypeChangefeedInfo, + ChangefeedID: changefeedID, + } + tester.MustUpdate(cdcKey.String(), []byte(changefeedStr)) + _, err = owner.Tick(ctx, state) + tester.MustApplyPatches() + c.Assert(err, check.IsNil) + c.Assert(owner.changefeeds, check.HasKey, changefeedID) + + // remove changefeed forcibly + owner.EnqueueJob(model.AdminJob{ + CfID: changefeedID, + Type: model.AdminRemove, + Opts: &model.AdminJobOption{ + ForceRemove: true, + }, + }) + + // this tick to clean the leak info fo the removed changefeed + _, err = owner.Tick(ctx, state) + c.Assert(err, check.IsNil) + c.Assert(err, check.IsNil) + // this tick to remove the changefeed state in memory + tester.MustApplyPatches() + _, err = owner.Tick(ctx, state) + c.Assert(err, check.IsNil) + c.Assert(err, check.IsNil) + tester.MustApplyPatches() + c.Assert(err, check.IsNil) + c.Assert(owner.changefeeds, check.Not(check.HasKey), changefeedID) + c.Assert(state.Changefeeds, check.Not(check.HasKey), changefeedID) +} + +func (s *ownerSuite) TestCheckClusterVersion(c *check.C) { + defer testleak.AfterTest(c)() + ctx := cdcContext.NewBackendContext4Test(false) + owner, state, tester := createOwner4Test(ctx, c) + tester.MustUpdate("/tidb/cdc/capture/6bbc01c8-0605-4f86-a0f9-b3119109b225", []byte(`{"id":"6bbc01c8-0605-4f86-a0f9-b3119109b225","address":"127.0.0.1:8300","version":"v6.0.0"}`)) + + changefeedID := "test-changefeed" + changefeedInfo := &model.ChangeFeedInfo{ + StartTs: oracle.ComposeTS(oracle.GetPhysical(time.Now()), 0), + Config: config.GetDefaultReplicaConfig(), + } + changefeedStr, err := changefeedInfo.Marshal() + c.Assert(err, check.IsNil) + cdcKey := etcd.CDCKey{ + Tp: etcd.CDCKeyTypeChangefeedInfo, + ChangefeedID: changefeedID, + } + tester.MustUpdate(cdcKey.String(), []byte(changefeedStr)) + + // check the tick is skipped and the changefeed will not be handled + _, err = owner.Tick(ctx, state) + tester.MustApplyPatches() + c.Assert(err, check.IsNil) + c.Assert(owner.changefeeds, check.Not(check.HasKey), changefeedID) + + tester.MustUpdate("/tidb/cdc/capture/6bbc01c8-0605-4f86-a0f9-b3119109b225", + []byte(`{"id":"6bbc01c8-0605-4f86-a0f9-b3119109b225","address":"127.0.0.1:8300","version":"`+ctx.GlobalVars().CaptureInfo.Version+`"}`)) + + // check the tick is not skipped and the changefeed will be handled normally + _, err = owner.Tick(ctx, state) + tester.MustApplyPatches() + c.Assert(err, check.IsNil) + c.Assert(owner.changefeeds, check.HasKey, changefeedID) +} + +func (s *ownerSuite) TestAdminJob(c *check.C) { + defer testleak.AfterTest(c)() + ctx := cdcContext.NewBackendContext4Test(false) + owner, _, _ := createOwner4Test(ctx, c) + owner.EnqueueJob(model.AdminJob{ + CfID: "test-changefeed1", + Type: model.AdminResume, + }) + owner.TriggerRebalance("test-changefeed2") + owner.ManualSchedule("test-changefeed3", "test-caputre1", 10) + var buf bytes.Buffer + owner.WriteDebugInfo(&buf) + + // remove job.done, it's hard to check deep equals + jobs := owner.takeOnwerJobs() + for _, job := range jobs { + c.Assert(job.done, check.NotNil) + close(job.done) + job.done = nil + } + c.Assert(jobs, check.DeepEquals, []*ownerJob{ + { + tp: ownerJobTypeAdminJob, + adminJob: &model.AdminJob{ + CfID: "test-changefeed1", + Type: model.AdminResume, + }, + changefeedID: "test-changefeed1", + }, { + tp: ownerJobTypeRebalance, + changefeedID: "test-changefeed2", + }, { + tp: ownerJobTypeManualSchedule, + changefeedID: "test-changefeed3", + targetCaptureID: "test-caputre1", + tableID: 10, + }, { + tp: ownerJobTypeDebugInfo, + debugInfoWriter: &buf, + }, + }) + c.Assert(owner.takeOnwerJobs(), check.HasLen, 0) +} diff --git a/cdc/owner/scheduler.go b/cdc/owner/scheduler.go new file mode 100644 index 00000000000..53e3d596e36 --- /dev/null +++ b/cdc/owner/scheduler.go @@ -0,0 +1,387 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package owner + +import ( + "math" + + "github.com/pingcap/errors" + "github.com/pingcap/failpoint" + "github.com/pingcap/log" + "github.com/pingcap/ticdc/cdc/model" + cerror "github.com/pingcap/ticdc/pkg/errors" + "go.uber.org/zap" +) + +type schedulerJobType string + +const ( + schedulerJobTypeAddTable schedulerJobType = "ADD" + schedulerJobTypeRemoveTable schedulerJobType = "REMOVE" +) + +type schedulerJob struct { + Tp schedulerJobType + TableID model.TableID + // if the operation is a delete operation, boundaryTs is checkpoint ts + // if the operation is a add operation, boundaryTs is start ts + BoundaryTs uint64 + TargetCapture model.CaptureID +} + +type moveTableJob struct { + tableID model.TableID + target model.CaptureID +} + +type scheduler struct { + state *model.ChangefeedReactorState + currentTables []model.TableID + captures map[model.CaptureID]*model.CaptureInfo + + moveTableTargets map[model.TableID]model.CaptureID + moveTableJobQueue []*moveTableJob + needRebalanceNextTick bool + lastTickCaptureCount int +} + +func newScheduler() *scheduler { + return &scheduler{ + moveTableTargets: make(map[model.TableID]model.CaptureID), + } +} + +// Tick is the main function of scheduler. It dispatches tables to captures and handles move-table and rebalance events. +// Tick returns a bool representing whether the changefeed's state can be updated in this tick. +// The state can be updated only if all the tables which should be listened to have been dispatched to captures and no operations have been sent to captures in this tick. +func (s *scheduler) Tick(state *model.ChangefeedReactorState, currentTables []model.TableID, captures map[model.CaptureID]*model.CaptureInfo) (shouldUpdateState bool, err error) { + s.state = state + s.currentTables = currentTables + s.captures = captures + + s.cleanUpFinishedOperations() + pendingJob, err := s.syncTablesWithCurrentTables() + if err != nil { + return false, errors.Trace(err) + } + s.dispatchToTargetCaptures(pendingJob) + if len(pendingJob) != 0 { + log.Debug("scheduler:generated pending job to be executed", zap.Any("pendingJob", pendingJob)) + } + s.handleJobs(pendingJob) + + // only if the pending job list is empty and no table is being rebalanced or moved, + // can the global resolved ts and checkpoint ts be updated + shouldUpdateState = len(pendingJob) == 0 + shouldUpdateState = s.rebalance() && shouldUpdateState + shouldUpdateStateInMoveTable, err := s.handleMoveTableJob() + if err != nil { + return false, errors.Trace(err) + } + shouldUpdateState = shouldUpdateStateInMoveTable && shouldUpdateState + s.lastTickCaptureCount = len(captures) + return shouldUpdateState, nil +} + +func (s *scheduler) MoveTable(tableID model.TableID, target model.CaptureID) { + s.moveTableJobQueue = append(s.moveTableJobQueue, &moveTableJob{ + tableID: tableID, + target: target, + }) +} + +// handleMoveTableJob handles the move table job add be MoveTable function +func (s *scheduler) handleMoveTableJob() (shouldUpdateState bool, err error) { + shouldUpdateState = true + if len(s.moveTableJobQueue) == 0 { + return + } + table2CaptureIndex, err := s.table2CaptureIndex() + if err != nil { + return false, errors.Trace(err) + } + for _, job := range s.moveTableJobQueue { + source, exist := table2CaptureIndex[job.tableID] + if !exist { + return + } + s.moveTableTargets[job.tableID] = job.target + job := job + shouldUpdateState = false + // for all move table job, this just remove the table from the source capture. + // and the removed table by this function will be added to target function by syncTablesWithCurrentTables in the next tick. + s.state.PatchTaskStatus(source, func(status *model.TaskStatus) (*model.TaskStatus, bool, error) { + if status == nil { + // the capture may be down, just skip remove this table + return status, false, nil + } + if status.Operation != nil && status.Operation[job.tableID] != nil { + // skip removing this table to avoid the remove operation created by the rebalance function interfering with the operation created by another function + return status, false, nil + } + status.RemoveTable(job.tableID, s.state.Status.CheckpointTs, false) + return status, true, nil + }) + } + s.moveTableJobQueue = nil + return +} + +func (s *scheduler) Rebalance() { + s.needRebalanceNextTick = true +} + +func (s *scheduler) table2CaptureIndex() (map[model.TableID]model.CaptureID, error) { + table2CaptureIndex := make(map[model.TableID]model.CaptureID) + for captureID, taskStatus := range s.state.TaskStatuses { + for tableID := range taskStatus.Tables { + if preCaptureID, exist := table2CaptureIndex[tableID]; exist && preCaptureID != captureID { + return nil, cerror.ErrTableListenReplicated.GenWithStackByArgs(preCaptureID, captureID) + } + table2CaptureIndex[tableID] = captureID + } + for tableID := range taskStatus.Operation { + if preCaptureID, exist := table2CaptureIndex[tableID]; exist && preCaptureID != captureID { + return nil, cerror.ErrTableListenReplicated.GenWithStackByArgs(preCaptureID, captureID) + } + table2CaptureIndex[tableID] = captureID + } + } + return table2CaptureIndex, nil +} + +// dispatchToTargetCaptures sets the the TargetCapture of scheduler jobs +// If the TargetCapture of a job is not set, it chooses a capture with the minimum workload and sets the TargetCapture to the capture. +func (s *scheduler) dispatchToTargetCaptures(pendingJobs []*schedulerJob) { + workloads := make(map[model.CaptureID]uint64) + + for captureID := range s.captures { + workloads[captureID] = 0 + taskWorkload := s.state.Workloads[captureID] + if taskWorkload == nil { + continue + } + for _, workload := range taskWorkload { + workloads[captureID] += workload.Workload + } + } + + for _, pendingJob := range pendingJobs { + if pendingJob.TargetCapture == "" { + target, exist := s.moveTableTargets[pendingJob.TableID] + if !exist { + continue + } + pendingJob.TargetCapture = target + delete(s.moveTableTargets, pendingJob.TableID) + continue + } + switch pendingJob.Tp { + case schedulerJobTypeAddTable: + workloads[pendingJob.TargetCapture] += 1 + case schedulerJobTypeRemoveTable: + workloads[pendingJob.TargetCapture] -= 1 + default: + log.Panic("Unreachable, please report a bug", + zap.String("changefeed", s.state.ID), zap.Any("job", pendingJob)) + } + } + + getMinWorkloadCapture := func() model.CaptureID { + minCapture := "" + minWorkLoad := uint64(math.MaxUint64) + for captureID, workload := range workloads { + if workload < minWorkLoad { + minCapture = captureID + minWorkLoad = workload + } + } + + if minCapture == "" { + log.Panic("Unreachable, no capture is found") + } + return minCapture + } + + for _, pendingJob := range pendingJobs { + if pendingJob.TargetCapture != "" { + continue + } + minCapture := getMinWorkloadCapture() + pendingJob.TargetCapture = minCapture + workloads[minCapture] += 1 + } +} + +// syncTablesWithCurrentTables iterates all current tables and check whether all the table has been listened. +// if not, this function will return scheduler jobs to make sure all the table will be listened. +func (s *scheduler) syncTablesWithCurrentTables() ([]*schedulerJob, error) { + var pendingJob []*schedulerJob + allTableListeningNow, err := s.table2CaptureIndex() + if err != nil { + return nil, errors.Trace(err) + } + globalCheckpointTs := s.state.Status.CheckpointTs + for _, tableID := range s.currentTables { + if _, exist := allTableListeningNow[tableID]; exist { + delete(allTableListeningNow, tableID) + continue + } + // For each table which should be listened but is not, add an adding-table job to the pending job list + boundaryTs := globalCheckpointTs + pendingJob = append(pendingJob, &schedulerJob{ + Tp: schedulerJobTypeAddTable, + TableID: tableID, + BoundaryTs: boundaryTs, + }) + } + // The remaining tables are the tables which should be not listened + tablesThatShouldNotBeListened := allTableListeningNow + for tableID, captureID := range tablesThatShouldNotBeListened { + opts := s.state.TaskStatuses[captureID].Operation + if opts != nil && opts[tableID] != nil && opts[tableID].Delete { + // the table is being removed, skip + continue + } + pendingJob = append(pendingJob, &schedulerJob{ + Tp: schedulerJobTypeRemoveTable, + TableID: tableID, + BoundaryTs: globalCheckpointTs, + TargetCapture: captureID, + }) + } + return pendingJob, nil +} + +func (s *scheduler) handleJobs(jobs []*schedulerJob) { + for _, job := range jobs { + job := job + s.state.PatchTaskStatus(job.TargetCapture, func(status *model.TaskStatus) (*model.TaskStatus, bool, error) { + switch job.Tp { + case schedulerJobTypeAddTable: + if status == nil { + // if task status is not found, we can just skip adding the adding-table operation, since this table will be added in the next tick + log.Warn("task status of the capture is not found, may be the capture is already down. specify a new capture and redo the job", zap.Any("job", job)) + return status, false, nil + } + status.AddTable(job.TableID, &model.TableReplicaInfo{ + StartTs: job.BoundaryTs, + MarkTableID: 0, // mark table ID will be set in processors + }, job.BoundaryTs) + case schedulerJobTypeRemoveTable: + failpoint.Inject("OwnerRemoveTableError", func() { + // just skip removing this table + failpoint.Return(status, false, nil) + }) + if status == nil { + log.Warn("Task status of the capture is not found. Maybe the capture is already down. Specify a new capture and redo the job", zap.Any("job", job)) + return status, false, nil + } + status.RemoveTable(job.TableID, job.BoundaryTs, false) + default: + log.Panic("Unreachable, please report a bug", zap.Any("job", job)) + } + return status, true, nil + }) + } +} + +// cleanUpFinishedOperations clean up the finished operations. +func (s *scheduler) cleanUpFinishedOperations() { + for captureID := range s.state.TaskStatuses { + s.state.PatchTaskStatus(captureID, func(status *model.TaskStatus) (*model.TaskStatus, bool, error) { + changed := false + for tableID, operation := range status.Operation { + if operation.Status == model.OperFinished { + delete(status.Operation, tableID) + changed = true + } + } + return status, changed, nil + }) + } +} + +func (s *scheduler) rebalance() (shouldUpdateState bool) { + if !s.shouldRebalance() { + // if no table is rebalanced, we can update the resolved ts and checkpoint ts + return true + } + // we only support rebalance by table number for now + return s.rebalanceByTableNum() +} + +func (s *scheduler) shouldRebalance() bool { + if s.needRebalanceNextTick { + s.needRebalanceNextTick = false + return true + } + if s.lastTickCaptureCount != len(s.captures) { + // a new capture online and no table distributed to the capture + // or some captures offline + return true + } + // TODO periodic trigger rebalance + return false +} + +// rebalanceByTableNum removes tables from captures replicating an above-average number of tables. +// the removed table will be dispatched again by syncTablesWithCurrentTables function +func (s *scheduler) rebalanceByTableNum() (shouldUpdateState bool) { + totalTableNum := len(s.currentTables) + captureNum := len(s.captures) + upperLimitPerCapture := int(math.Ceil(float64(totalTableNum) / float64(captureNum))) + shouldUpdateState = true + + log.Info("Start rebalancing", + zap.String("changefeed", s.state.ID), + zap.Int("table-num", totalTableNum), + zap.Int("capture-num", captureNum), + zap.Int("target-limit", upperLimitPerCapture)) + + for captureID, taskStatus := range s.state.TaskStatuses { + tableNum2Remove := len(taskStatus.Tables) - upperLimitPerCapture + if tableNum2Remove <= 0 { + continue + } + + // here we pick `tableNum2Remove` tables to delete, + // and then the removed tables will be dispatched by `syncTablesWithCurrentTables` function in the next tick + for tableID := range taskStatus.Tables { + tableID := tableID + if tableNum2Remove <= 0 { + break + } + shouldUpdateState = false + s.state.PatchTaskStatus(captureID, func(status *model.TaskStatus) (*model.TaskStatus, bool, error) { + if status == nil { + // the capture may be down, just skip remove this table + return status, false, nil + } + if status.Operation != nil && status.Operation[tableID] != nil { + // skip remove this table to avoid the remove operation created by rebalance function to influence the operation created by other function + return status, false, nil + } + status.RemoveTable(tableID, s.state.Status.CheckpointTs, false) + log.Info("Rebalance: Move table", + zap.Int64("table-id", tableID), + zap.String("capture", captureID), + zap.String("changefeed-id", s.state.ID)) + return status, true, nil + }) + tableNum2Remove-- + } + } + return +} diff --git a/cdc/owner/scheduler_test.go b/cdc/owner/scheduler_test.go new file mode 100644 index 00000000000..712a89a31e6 --- /dev/null +++ b/cdc/owner/scheduler_test.go @@ -0,0 +1,342 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package owner + +import ( + "fmt" + "math/rand" + + "github.com/pingcap/check" + "github.com/pingcap/ticdc/cdc/model" + "github.com/pingcap/ticdc/pkg/orchestrator" + "github.com/pingcap/ticdc/pkg/util/testleak" +) + +var _ = check.Suite(&schedulerSuite{}) + +type schedulerSuite struct { + changefeedID model.ChangeFeedID + state *model.ChangefeedReactorState + tester *orchestrator.ReactorStateTester + captures map[model.CaptureID]*model.CaptureInfo + scheduler *scheduler +} + +func (s *schedulerSuite) reset(c *check.C) { + s.changefeedID = fmt.Sprintf("test-changefeed-%x", rand.Uint32()) + s.state = model.NewChangefeedReactorState("test-changefeed") + s.tester = orchestrator.NewReactorStateTester(c, s.state, nil) + s.scheduler = newScheduler() + s.captures = make(map[model.CaptureID]*model.CaptureInfo) + s.state.PatchStatus(func(status *model.ChangeFeedStatus) (*model.ChangeFeedStatus, bool, error) { + return &model.ChangeFeedStatus{}, true, nil + }) + s.tester.MustApplyPatches() +} + +func (s *schedulerSuite) addCapture(captureID model.CaptureID) { + captureInfo := &model.CaptureInfo{ + ID: captureID, + } + s.captures[captureID] = captureInfo + s.state.PatchTaskStatus(captureID, func(status *model.TaskStatus) (*model.TaskStatus, bool, error) { + return &model.TaskStatus{}, true, nil + }) + s.tester.MustApplyPatches() +} + +func (s *schedulerSuite) finishTableOperation(captureID model.CaptureID, tableIDs ...model.TableID) { + s.state.PatchTaskStatus(captureID, func(status *model.TaskStatus) (*model.TaskStatus, bool, error) { + for _, tableID := range tableIDs { + status.Operation[tableID].Done = true + status.Operation[tableID].Status = model.OperFinished + } + return status, true, nil + }) + s.state.PatchTaskWorkload(captureID, func(workload model.TaskWorkload) (model.TaskWorkload, bool, error) { + if workload == nil { + workload = make(model.TaskWorkload) + } + for _, tableID := range tableIDs { + if s.state.TaskStatuses[captureID].Operation[tableID].Delete { + delete(workload, tableID) + } else { + workload[tableID] = model.WorkloadInfo{ + Workload: 1, + } + } + } + return workload, true, nil + }) + s.tester.MustApplyPatches() +} + +func (s *schedulerSuite) TestScheduleOneCapture(c *check.C) { + defer testleak.AfterTest(c)() + s.reset(c) + captureID := "test-capture-1" + s.addCapture(captureID) + + // add three tables + shouldUpdateState, err := s.scheduler.Tick(s.state, []model.TableID{1, 2, 3, 4}, s.captures) + c.Assert(err, check.IsNil) + c.Assert(shouldUpdateState, check.IsFalse) + s.tester.MustApplyPatches() + c.Assert(s.state.TaskStatuses[captureID].Tables, check.DeepEquals, map[model.TableID]*model.TableReplicaInfo{ + 1: {StartTs: 0}, 2: {StartTs: 0}, 3: {StartTs: 0}, 4: {StartTs: 0}, + }) + c.Assert(s.state.TaskStatuses[captureID].Operation, check.DeepEquals, map[model.TableID]*model.TableOperation{ + 1: {Done: false, Delete: false, BoundaryTs: 0, Status: model.OperDispatched}, + 2: {Done: false, Delete: false, BoundaryTs: 0, Status: model.OperDispatched}, + 3: {Done: false, Delete: false, BoundaryTs: 0, Status: model.OperDispatched}, + 4: {Done: false, Delete: false, BoundaryTs: 0, Status: model.OperDispatched}, + }) + shouldUpdateState, err = s.scheduler.Tick(s.state, []model.TableID{1, 2, 3, 4}, s.captures) + c.Assert(err, check.IsNil) + c.Assert(shouldUpdateState, check.IsTrue) + s.tester.MustApplyPatches() + + // two tables finish adding operation + s.finishTableOperation(captureID, 2, 3) + + // remove table 1,2 and add table 4,5 + shouldUpdateState, err = s.scheduler.Tick(s.state, []model.TableID{3, 4, 5}, s.captures) + c.Assert(err, check.IsNil) + c.Assert(shouldUpdateState, check.IsFalse) + s.tester.MustApplyPatches() + c.Assert(s.state.TaskStatuses[captureID].Tables, check.DeepEquals, map[model.TableID]*model.TableReplicaInfo{ + 3: {StartTs: 0}, 4: {StartTs: 0}, 5: {StartTs: 0}, + }) + c.Assert(s.state.TaskStatuses[captureID].Operation, check.DeepEquals, map[model.TableID]*model.TableOperation{ + 1: {Done: false, Delete: true, BoundaryTs: 0, Status: model.OperDispatched}, + 2: {Done: false, Delete: true, BoundaryTs: 0, Status: model.OperDispatched}, + 4: {Done: false, Delete: false, BoundaryTs: 0, Status: model.OperDispatched}, + 5: {Done: false, Delete: false, BoundaryTs: 0, Status: model.OperDispatched}, + }) + + // move a non exist table to a non exist capture + s.scheduler.MoveTable(2, "fake-capture") + // move tables to a non exist capture + s.scheduler.MoveTable(3, "fake-capture") + s.scheduler.MoveTable(4, "fake-capture") + shouldUpdateState, err = s.scheduler.Tick(s.state, []model.TableID{3, 4, 5}, s.captures) + c.Assert(err, check.IsNil) + c.Assert(shouldUpdateState, check.IsFalse) + s.tester.MustApplyPatches() + c.Assert(s.state.TaskStatuses[captureID].Tables, check.DeepEquals, map[model.TableID]*model.TableReplicaInfo{ + 4: {StartTs: 0}, 5: {StartTs: 0}, + }) + c.Assert(s.state.TaskStatuses[captureID].Operation, check.DeepEquals, map[model.TableID]*model.TableOperation{ + 1: {Done: false, Delete: true, BoundaryTs: 0, Status: model.OperDispatched}, + 2: {Done: false, Delete: true, BoundaryTs: 0, Status: model.OperDispatched}, + 3: {Done: false, Delete: true, BoundaryTs: 0, Status: model.OperDispatched}, + 4: {Done: false, Delete: false, BoundaryTs: 0, Status: model.OperDispatched}, + 5: {Done: false, Delete: false, BoundaryTs: 0, Status: model.OperDispatched}, + }) + + // finish all operations + s.finishTableOperation(captureID, 1, 2, 3, 4, 5) + + shouldUpdateState, err = s.scheduler.Tick(s.state, []model.TableID{3, 4, 5}, s.captures) + c.Assert(err, check.IsNil) + c.Assert(shouldUpdateState, check.IsTrue) + s.tester.MustApplyPatches() + c.Assert(s.state.TaskStatuses[captureID].Tables, check.DeepEquals, map[model.TableID]*model.TableReplicaInfo{ + 4: {StartTs: 0}, 5: {StartTs: 0}, + }) + c.Assert(s.state.TaskStatuses[captureID].Operation, check.DeepEquals, map[model.TableID]*model.TableOperation{}) + + // table 3 is missing by expected, because the table was trying to move to a invalid capture + // and the move will failed, the table 3 will be add in next tick + shouldUpdateState, err = s.scheduler.Tick(s.state, []model.TableID{3, 4, 5}, s.captures) + c.Assert(err, check.IsNil) + c.Assert(shouldUpdateState, check.IsFalse) + s.tester.MustApplyPatches() + c.Assert(s.state.TaskStatuses[captureID].Tables, check.DeepEquals, map[model.TableID]*model.TableReplicaInfo{ + 4: {StartTs: 0}, 5: {StartTs: 0}, + }) + c.Assert(s.state.TaskStatuses[captureID].Operation, check.DeepEquals, map[model.TableID]*model.TableOperation{}) + + shouldUpdateState, err = s.scheduler.Tick(s.state, []model.TableID{3, 4, 5}, s.captures) + c.Assert(err, check.IsNil) + c.Assert(shouldUpdateState, check.IsFalse) + s.tester.MustApplyPatches() + c.Assert(s.state.TaskStatuses[captureID].Tables, check.DeepEquals, map[model.TableID]*model.TableReplicaInfo{ + 3: {StartTs: 0}, 4: {StartTs: 0}, 5: {StartTs: 0}, + }) + c.Assert(s.state.TaskStatuses[captureID].Operation, check.DeepEquals, map[model.TableID]*model.TableOperation{ + 3: {Done: false, Delete: false, BoundaryTs: 0, Status: model.OperDispatched}, + }) +} + +func (s *schedulerSuite) TestScheduleMoveTable(c *check.C) { + defer testleak.AfterTest(c)() + s.reset(c) + captureID1 := "test-capture-1" + captureID2 := "test-capture-2" + s.addCapture(captureID1) + + // add a table + shouldUpdateState, err := s.scheduler.Tick(s.state, []model.TableID{1}, s.captures) + c.Assert(err, check.IsNil) + c.Assert(shouldUpdateState, check.IsFalse) + s.tester.MustApplyPatches() + c.Assert(s.state.TaskStatuses[captureID1].Tables, check.DeepEquals, map[model.TableID]*model.TableReplicaInfo{ + 1: {StartTs: 0}, + }) + c.Assert(s.state.TaskStatuses[captureID1].Operation, check.DeepEquals, map[model.TableID]*model.TableOperation{ + 1: {Done: false, Delete: false, BoundaryTs: 0, Status: model.OperDispatched}, + }) + + s.finishTableOperation(captureID1, 1) + shouldUpdateState, err = s.scheduler.Tick(s.state, []model.TableID{1}, s.captures) + c.Assert(err, check.IsNil) + c.Assert(shouldUpdateState, check.IsTrue) + s.tester.MustApplyPatches() + + s.addCapture(captureID2) + + // add a table + shouldUpdateState, err = s.scheduler.Tick(s.state, []model.TableID{1, 2}, s.captures) + c.Assert(err, check.IsNil) + c.Assert(shouldUpdateState, check.IsFalse) + s.tester.MustApplyPatches() + c.Assert(s.state.TaskStatuses[captureID1].Tables, check.DeepEquals, map[model.TableID]*model.TableReplicaInfo{ + 1: {StartTs: 0}, + }) + c.Assert(s.state.TaskStatuses[captureID1].Operation, check.DeepEquals, map[model.TableID]*model.TableOperation{}) + c.Assert(s.state.TaskStatuses[captureID2].Tables, check.DeepEquals, map[model.TableID]*model.TableReplicaInfo{ + 2: {StartTs: 0}, + }) + c.Assert(s.state.TaskStatuses[captureID2].Operation, check.DeepEquals, map[model.TableID]*model.TableOperation{ + 2: {Done: false, Delete: false, BoundaryTs: 0, Status: model.OperDispatched}, + }) + + s.finishTableOperation(captureID2, 2) + + s.scheduler.MoveTable(2, captureID1) + shouldUpdateState, err = s.scheduler.Tick(s.state, []model.TableID{1, 2}, s.captures) + c.Assert(err, check.IsNil) + c.Assert(shouldUpdateState, check.IsFalse) + s.tester.MustApplyPatches() + c.Assert(s.state.TaskStatuses[captureID1].Tables, check.DeepEquals, map[model.TableID]*model.TableReplicaInfo{ + 1: {StartTs: 0}, + }) + c.Assert(s.state.TaskStatuses[captureID1].Operation, check.DeepEquals, map[model.TableID]*model.TableOperation{}) + c.Assert(s.state.TaskStatuses[captureID2].Tables, check.DeepEquals, map[model.TableID]*model.TableReplicaInfo{}) + c.Assert(s.state.TaskStatuses[captureID2].Operation, check.DeepEquals, map[model.TableID]*model.TableOperation{ + 2: {Done: false, Delete: true, BoundaryTs: 0, Status: model.OperDispatched}, + }) + + s.finishTableOperation(captureID2, 2) + + shouldUpdateState, err = s.scheduler.Tick(s.state, []model.TableID{1, 2}, s.captures) + c.Assert(err, check.IsNil) + c.Assert(shouldUpdateState, check.IsTrue) + s.tester.MustApplyPatches() + c.Assert(s.state.TaskStatuses[captureID1].Tables, check.DeepEquals, map[model.TableID]*model.TableReplicaInfo{ + 1: {StartTs: 0}, + }) + c.Assert(s.state.TaskStatuses[captureID1].Operation, check.DeepEquals, map[model.TableID]*model.TableOperation{}) + c.Assert(s.state.TaskStatuses[captureID2].Tables, check.DeepEquals, map[model.TableID]*model.TableReplicaInfo{}) + c.Assert(s.state.TaskStatuses[captureID2].Operation, check.DeepEquals, map[model.TableID]*model.TableOperation{}) + + shouldUpdateState, err = s.scheduler.Tick(s.state, []model.TableID{1, 2}, s.captures) + c.Assert(err, check.IsNil) + c.Assert(shouldUpdateState, check.IsFalse) + s.tester.MustApplyPatches() + c.Assert(s.state.TaskStatuses[captureID1].Tables, check.DeepEquals, map[model.TableID]*model.TableReplicaInfo{ + 1: {StartTs: 0}, 2: {StartTs: 0}, + }) + c.Assert(s.state.TaskStatuses[captureID1].Operation, check.DeepEquals, map[model.TableID]*model.TableOperation{ + 2: {Done: false, Delete: false, BoundaryTs: 0, Status: model.OperDispatched}, + }) + c.Assert(s.state.TaskStatuses[captureID2].Tables, check.DeepEquals, map[model.TableID]*model.TableReplicaInfo{}) + c.Assert(s.state.TaskStatuses[captureID2].Operation, check.DeepEquals, map[model.TableID]*model.TableOperation{}) +} + +func (s *schedulerSuite) TestScheduleRebalance(c *check.C) { + defer testleak.AfterTest(c)() + s.reset(c) + captureID1 := "test-capture-1" + captureID2 := "test-capture-2" + captureID3 := "test-capture-3" + s.addCapture(captureID1) + s.addCapture(captureID2) + s.addCapture(captureID3) + + s.state.PatchTaskStatus(captureID1, func(status *model.TaskStatus) (*model.TaskStatus, bool, error) { + status.Tables = make(map[model.TableID]*model.TableReplicaInfo) + status.Tables[1] = &model.TableReplicaInfo{StartTs: 1} + status.Tables[2] = &model.TableReplicaInfo{StartTs: 1} + status.Tables[3] = &model.TableReplicaInfo{StartTs: 1} + status.Tables[4] = &model.TableReplicaInfo{StartTs: 1} + status.Tables[5] = &model.TableReplicaInfo{StartTs: 1} + status.Tables[6] = &model.TableReplicaInfo{StartTs: 1} + return status, true, nil + }) + s.tester.MustApplyPatches() + + // rebalance table + shouldUpdateState, err := s.scheduler.Tick(s.state, []model.TableID{1, 2, 3, 4, 5, 6}, s.captures) + c.Assert(err, check.IsNil) + c.Assert(shouldUpdateState, check.IsFalse) + s.tester.MustApplyPatches() + // 4 tables remove in capture 1, this 4 tables will be added to another capture in next tick + c.Assert(s.state.TaskStatuses[captureID1].Tables, check.HasLen, 2) + c.Assert(s.state.TaskStatuses[captureID2].Tables, check.HasLen, 0) + c.Assert(s.state.TaskStatuses[captureID3].Tables, check.HasLen, 0) + + s.state.PatchTaskStatus(captureID1, func(status *model.TaskStatus) (*model.TaskStatus, bool, error) { + for _, opt := range status.Operation { + opt.Done = true + opt.Status = model.OperFinished + } + return status, true, nil + }) + s.state.PatchTaskWorkload(captureID1, func(workload model.TaskWorkload) (model.TaskWorkload, bool, error) { + c.Assert(workload, check.IsNil) + workload = make(model.TaskWorkload) + for tableID := range s.state.TaskStatuses[captureID1].Tables { + workload[tableID] = model.WorkloadInfo{Workload: 1} + } + return workload, true, nil + }) + s.tester.MustApplyPatches() + + // clean finished operation + shouldUpdateState, err = s.scheduler.Tick(s.state, []model.TableID{1, 2, 3, 4, 5, 6}, s.captures) + c.Assert(err, check.IsNil) + c.Assert(shouldUpdateState, check.IsTrue) + s.tester.MustApplyPatches() + // 4 tables add to another capture in this tick + c.Assert(s.state.TaskStatuses[captureID1].Operation, check.HasLen, 0) + + // rebalance table + shouldUpdateState, err = s.scheduler.Tick(s.state, []model.TableID{1, 2, 3, 4, 5, 6}, s.captures) + c.Assert(err, check.IsNil) + c.Assert(shouldUpdateState, check.IsFalse) + s.tester.MustApplyPatches() + // 4 tables add to another capture in this tick + c.Assert(s.state.TaskStatuses[captureID1].Tables, check.HasLen, 2) + c.Assert(s.state.TaskStatuses[captureID2].Tables, check.HasLen, 2) + c.Assert(s.state.TaskStatuses[captureID3].Tables, check.HasLen, 2) + tableIDs := make(map[model.TableID]struct{}) + for _, status := range s.state.TaskStatuses { + for tableID := range status.Tables { + tableIDs[tableID] = struct{}{} + } + } + c.Assert(tableIDs, check.DeepEquals, map[model.TableID]struct{}{1: {}, 2: {}, 3: {}, 4: {}, 5: {}, 6: {}}) +} diff --git a/cdc/owner/schema.go b/cdc/owner/schema.go new file mode 100644 index 00000000000..feb70898920 --- /dev/null +++ b/cdc/owner/schema.go @@ -0,0 +1,164 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package owner + +import ( + "github.com/pingcap/errors" + "github.com/pingcap/log" + timodel "github.com/pingcap/parser/model" + "github.com/pingcap/ticdc/cdc/entry" + "github.com/pingcap/ticdc/cdc/kv" + "github.com/pingcap/ticdc/cdc/model" + "github.com/pingcap/ticdc/pkg/config" + "github.com/pingcap/ticdc/pkg/cyclic/mark" + "github.com/pingcap/ticdc/pkg/filter" + tidbkv "github.com/pingcap/tidb/kv" + timeta "github.com/pingcap/tidb/meta" + "go.uber.org/zap" +) + +type schemaWrap4Owner struct { + schemaSnapshot *entry.SingleSchemaSnapshot + filter *filter.Filter + config *config.ReplicaConfig + + allPhysicalTablesCache []model.TableID + ddlHandledTs model.Ts +} + +func newSchemaWrap4Owner(kvStorage tidbkv.Storage, startTs model.Ts, config *config.ReplicaConfig) (*schemaWrap4Owner, error) { + var meta *timeta.Meta + if kvStorage != nil { + var err error + meta, err = kv.GetSnapshotMeta(kvStorage, startTs) + if err != nil { + return nil, errors.Trace(err) + } + } + schemaSnap, err := entry.NewSingleSchemaSnapshotFromMeta(meta, startTs-1, config.ForceReplicate) + if err != nil { + return nil, errors.Trace(err) + } + f, err := filter.NewFilter(config) + if err != nil { + return nil, errors.Trace(err) + } + return &schemaWrap4Owner{ + schemaSnapshot: schemaSnap, + filter: f, + config: config, + ddlHandledTs: startTs - 1, + }, nil +} + +// AllPhysicalTables returns the table IDs of all tables and partition tables. +func (s *schemaWrap4Owner) AllPhysicalTables() []model.TableID { + if s.allPhysicalTablesCache != nil { + return s.allPhysicalTablesCache + } + tables := s.schemaSnapshot.Tables() + s.allPhysicalTablesCache = make([]model.TableID, 0, len(tables)) + for _, tblInfo := range tables { + if s.shouldIgnoreTable(tblInfo) { + continue + } + + if pi := tblInfo.GetPartitionInfo(); pi != nil { + for _, partition := range pi.Definitions { + s.allPhysicalTablesCache = append(s.allPhysicalTablesCache, partition.ID) + } + } else { + s.allPhysicalTablesCache = append(s.allPhysicalTablesCache, tblInfo.ID) + } + } + return s.allPhysicalTablesCache +} + +func (s *schemaWrap4Owner) HandleDDL(job *timodel.Job) error { + if job.BinlogInfo.FinishedTS <= s.ddlHandledTs { + return nil + } + s.allPhysicalTablesCache = nil + err := s.schemaSnapshot.HandleDDL(job) + if err != nil { + return errors.Trace(err) + } + s.ddlHandledTs = job.BinlogInfo.FinishedTS + return nil +} + +func (s *schemaWrap4Owner) IsIneligibleTableID(tableID model.TableID) bool { + return s.schemaSnapshot.IsIneligibleTableID(tableID) +} + +func (s *schemaWrap4Owner) BuildDDLEvent(job *timodel.Job) (*model.DDLEvent, error) { + ddlEvent := new(model.DDLEvent) + preTableInfo, err := s.schemaSnapshot.PreTableInfo(job) + if err != nil { + return nil, errors.Trace(err) + } + err = s.schemaSnapshot.FillSchemaName(job) + if err != nil { + return nil, errors.Trace(err) + } + ddlEvent.FromJob(job, preTableInfo) + return ddlEvent, nil +} + +func (s *schemaWrap4Owner) SinkTableInfos() []*model.SimpleTableInfo { + var sinkTableInfos []*model.SimpleTableInfo + for tableID := range s.schemaSnapshot.CloneTables() { + tblInfo, ok := s.schemaSnapshot.TableByID(tableID) + if !ok { + log.Panic("table not found for table ID", zap.Int64("tid", tableID)) + } + if s.shouldIgnoreTable(tblInfo) { + continue + } + dbInfo, ok := s.schemaSnapshot.SchemaByTableID(tableID) + if !ok { + log.Panic("schema not found for table ID", zap.Int64("tid", tableID)) + } + + // TODO separate function for initializing SimpleTableInfo + sinkTableInfo := new(model.SimpleTableInfo) + sinkTableInfo.Schema = dbInfo.Name.O + sinkTableInfo.TableID = tableID + sinkTableInfo.Table = tblInfo.TableName.Table + sinkTableInfo.ColumnInfo = make([]*model.ColumnInfo, len(tblInfo.Cols())) + for i, colInfo := range tblInfo.Cols() { + sinkTableInfo.ColumnInfo[i] = new(model.ColumnInfo) + sinkTableInfo.ColumnInfo[i].FromTiColumnInfo(colInfo) + } + sinkTableInfos = append(sinkTableInfos, sinkTableInfo) + } + return sinkTableInfos +} + +func (s *schemaWrap4Owner) shouldIgnoreTable(tableInfo *model.TableInfo) bool { + schemaName := tableInfo.TableName.Schema + tableName := tableInfo.TableName.Table + if s.filter.ShouldIgnoreTable(schemaName, tableName) { + return true + } + if s.config.Cyclic.IsEnabled() && mark.IsMarkTable(schemaName, tableName) { + // skip the mark table if cyclic is enabled + return true + } + if !tableInfo.IsEligible(s.config.ForceReplicate) { + log.Warn("skip ineligible table", zap.Int64("tid", tableInfo.ID), zap.Stringer("table", tableInfo.TableName)) + return true + } + return false +} diff --git a/cdc/owner/schema_test.go b/cdc/owner/schema_test.go new file mode 100644 index 00000000000..04286d740b8 --- /dev/null +++ b/cdc/owner/schema_test.go @@ -0,0 +1,172 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package owner + +import ( + "sort" + + "github.com/pingcap/check" + timodel "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/ticdc/cdc/entry" + "github.com/pingcap/ticdc/cdc/model" + "github.com/pingcap/ticdc/pkg/config" + "github.com/pingcap/ticdc/pkg/util/testleak" +) + +var _ = check.Suite(&schemaSuite{}) + +type schemaSuite struct { +} + +func (s *schemaSuite) TestAllPhysicalTables(c *check.C) { + defer testleak.AfterTest(c)() + helper := entry.NewSchemaTestHelper(c) + defer helper.Close() + ver, err := helper.Storage().CurrentVersion() + c.Assert(err, check.IsNil) + schema, err := newSchemaWrap4Owner(helper.Storage(), ver.Ver, config.GetDefaultReplicaConfig()) + c.Assert(err, check.IsNil) + c.Assert(schema.AllPhysicalTables(), check.HasLen, 0) + // add normal table + job := helper.DDL2Job("create table test.t1(id int primary key)") + tableIDT1 := job.BinlogInfo.TableInfo.ID + c.Assert(schema.HandleDDL(job), check.IsNil) + c.Assert(schema.AllPhysicalTables(), check.DeepEquals, []model.TableID{tableIDT1}) + // add ineligible table + c.Assert(schema.HandleDDL(helper.DDL2Job("create table test.t2(id int)")), check.IsNil) + c.Assert(schema.AllPhysicalTables(), check.DeepEquals, []model.TableID{tableIDT1}) + // add partition table + job = helper.DDL2Job(`CREATE TABLE test.employees ( + id INT NOT NULL AUTO_INCREMENT PRIMARY KEY, + fname VARCHAR(25) NOT NULL, + lname VARCHAR(25) NOT NULL, + store_id INT NOT NULL, + department_id INT NOT NULL + ) + + PARTITION BY RANGE(id) ( + PARTITION p0 VALUES LESS THAN (5), + PARTITION p1 VALUES LESS THAN (10), + PARTITION p2 VALUES LESS THAN (15), + PARTITION p3 VALUES LESS THAN (20) + )`) + c.Assert(schema.HandleDDL(job), check.IsNil) + expectedTableIDs := []model.TableID{tableIDT1} + for _, p := range job.BinlogInfo.TableInfo.GetPartitionInfo().Definitions { + expectedTableIDs = append(expectedTableIDs, p.ID) + } + sortTableIDs := func(tableIDs []model.TableID) { + sort.Slice(tableIDs, func(i, j int) bool { + return tableIDs[i] < tableIDs[j] + }) + } + sortTableIDs(expectedTableIDs) + sortTableIDs(schema.AllPhysicalTables()) + c.Assert(schema.AllPhysicalTables(), check.DeepEquals, expectedTableIDs) +} + +func (s *schemaSuite) TestIsIneligibleTableID(c *check.C) { + defer testleak.AfterTest(c)() + helper := entry.NewSchemaTestHelper(c) + defer helper.Close() + ver, err := helper.Storage().CurrentVersion() + c.Assert(err, check.IsNil) + schema, err := newSchemaWrap4Owner(helper.Storage(), ver.Ver, config.GetDefaultReplicaConfig()) + c.Assert(err, check.IsNil) + // add normal table + job := helper.DDL2Job("create table test.t1(id int primary key)") + tableIDT1 := job.BinlogInfo.TableInfo.ID + c.Assert(schema.HandleDDL(job), check.IsNil) + // add ineligible table + job = helper.DDL2Job("create table test.t2(id int)") + tableIDT2 := job.BinlogInfo.TableInfo.ID + c.Assert(schema.HandleDDL(job), check.IsNil) + c.Assert(schema.IsIneligibleTableID(tableIDT1), check.IsFalse) + c.Assert(schema.IsIneligibleTableID(tableIDT2), check.IsTrue) +} + +func (s *schemaSuite) TestBuildDDLEvent(c *check.C) { + defer testleak.AfterTest(c)() + helper := entry.NewSchemaTestHelper(c) + defer helper.Close() + ver, err := helper.Storage().CurrentVersion() + c.Assert(err, check.IsNil) + schema, err := newSchemaWrap4Owner(helper.Storage(), ver.Ver, config.GetDefaultReplicaConfig()) + c.Assert(err, check.IsNil) + // add normal table + job := helper.DDL2Job("create table test.t1(id int primary key)") + event, err := schema.BuildDDLEvent(job) + c.Assert(err, check.IsNil) + c.Assert(event, check.DeepEquals, &model.DDLEvent{ + StartTs: job.StartTS, + CommitTs: job.BinlogInfo.FinishedTS, + Query: "create table test.t1(id int primary key)", + Type: timodel.ActionCreateTable, + TableInfo: &model.SimpleTableInfo{ + Schema: "test", + Table: "t1", + TableID: job.TableID, + ColumnInfo: []*model.ColumnInfo{{Name: "id", Type: mysql.TypeLong}}, + }, + PreTableInfo: nil, + }) + c.Assert(schema.HandleDDL(job), check.IsNil) + job = helper.DDL2Job("ALTER TABLE test.t1 ADD COLUMN c1 CHAR(16) NOT NULL") + event, err = schema.BuildDDLEvent(job) + c.Assert(err, check.IsNil) + c.Assert(event, check.DeepEquals, &model.DDLEvent{ + StartTs: job.StartTS, + CommitTs: job.BinlogInfo.FinishedTS, + Query: "ALTER TABLE test.t1 ADD COLUMN c1 CHAR(16) NOT NULL", + Type: timodel.ActionAddColumn, + TableInfo: &model.SimpleTableInfo{ + Schema: "test", + Table: "t1", + TableID: job.TableID, + ColumnInfo: []*model.ColumnInfo{{Name: "id", Type: mysql.TypeLong}, {Name: "c1", Type: mysql.TypeString}}, + }, + PreTableInfo: &model.SimpleTableInfo{ + Schema: "test", + Table: "t1", + TableID: job.TableID, + ColumnInfo: []*model.ColumnInfo{{Name: "id", Type: mysql.TypeLong}}, + }, + }) +} + +func (s *schemaSuite) TestSinkTableInfos(c *check.C) { + defer testleak.AfterTest(c)() + helper := entry.NewSchemaTestHelper(c) + defer helper.Close() + ver, err := helper.Storage().CurrentVersion() + c.Assert(err, check.IsNil) + schema, err := newSchemaWrap4Owner(helper.Storage(), ver.Ver, config.GetDefaultReplicaConfig()) + c.Assert(err, check.IsNil) + // add normal table + job := helper.DDL2Job("create table test.t1(id int primary key)") + tableIDT1 := job.BinlogInfo.TableInfo.ID + c.Assert(schema.HandleDDL(job), check.IsNil) + // add ineligible table + job = helper.DDL2Job("create table test.t2(id int)") + c.Assert(schema.HandleDDL(job), check.IsNil) + c.Assert(schema.SinkTableInfos(), check.DeepEquals, []*model.SimpleTableInfo{ + { + Schema: "test", + Table: "t1", + TableID: tableIDT1, + ColumnInfo: []*model.ColumnInfo{{Name: "id", Type: mysql.TypeLong}}, + }, + }) +} diff --git a/cdc/owner_test.go b/cdc/owner_test.go index c7d4b23940c..cd97634d4e7 100644 --- a/cdc/owner_test.go +++ b/cdc/owner_test.go @@ -49,6 +49,8 @@ import ( "golang.org/x/sync/errgroup" ) +const TiKVGCLifeTime = 10 * 60 * time.Second // 10 min + type ownerSuite struct { e *embed.Etcd clientURL *url.URL @@ -87,9 +89,10 @@ func (s *ownerSuite) TearDownTest(c *check.C) { type mockPDClient struct { pd.Client - invokeCounter int - mockSafePointLost bool - mockPDFailure bool + invokeCounter int + mockSafePointLost bool + mockPDFailure bool + mockTiKVGCLifeTime bool } func (m *mockPDClient) GetTS(ctx context.Context) (int64, int64, error) { @@ -111,13 +114,99 @@ func (m *mockPDClient) UpdateServiceGCSafePoint(ctx context.Context, serviceID s if m.mockPDFailure { return 0, errors.New("injected PD failure") } - + if m.mockTiKVGCLifeTime { + Ts := oracle.EncodeTSO(oracle.GetPhysical(time.Now().Add(-TiKVGCLifeTime))) + return Ts, nil + } return safePoint, nil } +type mockSink struct { + sink.Sink + checkpointTs model.Ts + + checkpointMu sync.Mutex + checkpointError error +} + +func (m *mockSink) EmitCheckpointTs(ctx context.Context, ts uint64) error { + m.checkpointMu.Lock() + defer m.checkpointMu.Unlock() + atomic.StoreUint64(&m.checkpointTs, ts) + return m.checkpointError +} + +func (m *mockSink) Close() error { + return nil +} + +// Test whether the owner can tolerate sink caused error, it won't be killed. +// also set the specific changefeed to stop +func (s *ownerSuite) TestOwnerCalcResolvedTs(c *check.C) { + defer testleak.AfterTest(c)() + mockPDCli := &mockPDClient{} + + sink := &mockSink{checkpointError: cerror.ErrKafkaSendMessage} + changeFeeds := map[model.ChangeFeedID]*changeFeed{ + "test_change_feed_1": { + info: &model.ChangeFeedInfo{State: model.StateNormal}, + etcdCli: s.client, + status: &model.ChangeFeedStatus{ + CheckpointTs: 0, + }, + targetTs: 2000, + ddlResolvedTs: 2000, + ddlState: model.ChangeFeedSyncDML, + taskStatus: model.ProcessorsInfos{ + "capture_1": {}, + "capture_2": {}, + }, + taskPositions: map[string]*model.TaskPosition{ + "capture_1": { + CheckPointTs: 2333, + ResolvedTs: 2333, + }, + "capture_2": { + CheckPointTs: 2333, + ResolvedTs: 2333, + }, + }, + sink: sink, + }, + } + + session, err := concurrency.NewSession(s.client.Client.Unwrap(), + concurrency.WithTTL(config.GetDefaultServerConfig().CaptureSessionTTL)) + c.Assert(err, check.IsNil) + mockOwner := Owner{ + session: session, + pdClient: mockPDCli, + etcdClient: s.client, + lastFlushChangefeeds: time.Now(), + flushChangefeedInterval: 1 * time.Hour, + changeFeeds: changeFeeds, + cfRWriter: s.client, + stoppedFeeds: make(map[model.ChangeFeedID]*model.ChangeFeedStatus), + minGCSafePointCache: minGCSafePointCacheEntry{}, + } + + err = mockOwner.calcResolvedTs(s.ctx) + c.Assert(err, check.IsNil) + + err = mockOwner.handleAdminJob(s.ctx) + c.Assert(err, check.IsNil) + c.Assert(mockOwner.stoppedFeeds["test_change_feed_1"], check.NotNil) + + err = mockOwner.flushChangeFeedInfos(s.ctx) + c.Assert(err, check.IsNil) + c.Assert(mockPDCli.invokeCounter, check.Equals, 1) + + s.TearDownTest(c) +} + func (s *ownerSuite) TestOwnerFlushChangeFeedInfos(c *check.C) { defer testleak.AfterTest(c)() - session, err := concurrency.NewSession(s.client.Client.Unwrap(), concurrency.WithTTL(defaultCaptureSessionTTL)) + session, err := concurrency.NewSession(s.client.Client.Unwrap(), concurrency.WithTTL(config.GetDefaultServerConfig().CaptureSessionTTL)) c.Assert(err, check.IsNil) mockPDCli := &mockPDClient{} mockOwner := Owner{ @@ -158,7 +247,7 @@ func (s *ownerSuite) TestOwnerFlushChangeFeedInfosFailed(c *check.C) { }, } - session, err := concurrency.NewSession(s.client.Client.Unwrap(), concurrency.WithTTL(defaultCaptureSessionTTL)) + session, err := concurrency.NewSession(s.client.Client.Unwrap(), concurrency.WithTTL(config.GetDefaultServerConfig().CaptureSessionTTL)) c.Assert(err, check.IsNil) mockOwner := Owner{ session: session, @@ -184,18 +273,19 @@ func (s *ownerSuite) TestOwnerFlushChangeFeedInfosFailed(c *check.C) { s.TearDownTest(c) } -// Test whether the owner handles the stagnant task correctly, so that it can't block the update of gcSafePoint. -// If a changefeed is put into the stop queue due to stagnation, it can no longer affect the update of gcSafePoint. -// So we just need to test whether the stagnant changefeed is put into the stop queue. -func (s *ownerSuite) TestOwnerHandleStaleChangeFeed(c *check.C) { - defer testleak.AfterTest(c)() +// Test whether it is possible to successfully create a changefeed +// with startTs less than currentTs - gcTTL when tikv_gc_life_time is greater than gc-ttl +func (s *ownerSuite) TestTiKVGCLifeTimeLargeThanGCTTL(c *check.C) { + defer testleak.AfterTest(c) mockPDCli := &mockPDClient{} + mockPDCli.mockTiKVGCLifeTime = true + changeFeeds := map[model.ChangeFeedID]*changeFeed{ "test_change_feed_1": { info: &model.ChangeFeedInfo{State: model.StateNormal}, etcdCli: s.client, status: &model.ChangeFeedStatus{ - CheckpointTs: 1000, + CheckpointTs: oracle.EncodeTSO(oracle.GetPhysical(time.Now().Add(-6 * time.Second))), }, targetTs: 2000, ddlState: model.ChangeFeedSyncDML, @@ -208,11 +298,58 @@ func (s *ownerSuite) TestOwnerHandleStaleChangeFeed(c *check.C) { "capture_2": {}, }, }, - "test_change_feed_2": { + } + + session, err := concurrency.NewSession(s.client.Client.Unwrap(), concurrency.WithTTL(config.GetDefaultServerConfig().CaptureSessionTTL)) + c.Assert(err, check.IsNil) + + mockOwner := Owner{ + session: session, + pdClient: mockPDCli, + etcdClient: s.client, + lastFlushChangefeeds: time.Now(), + flushChangefeedInterval: 1 * time.Hour, + // gcSafepointLastUpdate: time.Now(), + gcTTL: 6, // 6 seconds + changeFeeds: changeFeeds, + cfRWriter: s.client, + stoppedFeeds: make(map[model.ChangeFeedID]*model.ChangeFeedStatus), + minGCSafePointCache: minGCSafePointCacheEntry{}, + } + + err = mockOwner.flushChangeFeedInfos(s.ctx) + c.Assert(err, check.IsNil) + c.Assert(mockPDCli.invokeCounter, check.Equals, 1) + + err = mockOwner.handleAdminJob(s.ctx) + c.Assert(err, check.IsNil) + c.Assert(mockOwner.stoppedFeeds["test_change_feed_1"], check.IsNil) + c.Assert(mockOwner.changeFeeds["test_change_feed_1"].info.State, check.Equals, model.StateNormal) + + time.Sleep(7 * time.Second) // wait for gcTTL time pass + err = mockOwner.flushChangeFeedInfos(s.ctx) + c.Assert(err, check.IsNil) + c.Assert(mockPDCli.invokeCounter, check.Equals, 2) + + err = mockOwner.handleAdminJob(s.ctx) + c.Assert(err, check.IsNil) + c.Assert(mockOwner.stoppedFeeds["test_change_feed_1"], check.IsNil) + + s.TearDownTest(c) +} + +// Test whether the owner handles the stagnant task correctly, so that it can't block the update of gcSafePoint. +// If a changefeed is put into the stop queue due to stagnation, it can no longer affect the update of gcSafePoint. +// So we just need to test whether the stagnant changefeed is put into the stop queue. +func (s *ownerSuite) TestOwnerHandleStaleChangeFeed(c *check.C) { + defer testleak.AfterTest(c)() + mockPDCli := &mockPDClient{} + changeFeeds := map[model.ChangeFeedID]*changeFeed{ + "test_change_feed_1": { info: &model.ChangeFeedInfo{State: model.StateNormal}, etcdCli: s.client, status: &model.ChangeFeedStatus{ - CheckpointTs: oracle.EncodeTSO(oracle.GetPhysical(time.Now())), + CheckpointTs: 1000, }, targetTs: 2000, ddlState: model.ChangeFeedSyncDML, @@ -225,11 +362,11 @@ func (s *ownerSuite) TestOwnerHandleStaleChangeFeed(c *check.C) { "capture_2": {}, }, }, - "test_change_feed_3": { + "test_change_feed_2": { info: &model.ChangeFeedInfo{State: model.StateNormal}, etcdCli: s.client, status: &model.ChangeFeedStatus{ - CheckpointTs: 0, + CheckpointTs: oracle.EncodeTSO(oracle.GetPhysical(time.Now())), }, targetTs: 2000, ddlState: model.ChangeFeedSyncDML, @@ -245,7 +382,7 @@ func (s *ownerSuite) TestOwnerHandleStaleChangeFeed(c *check.C) { } session, err := concurrency.NewSession(s.client.Client.Unwrap(), - concurrency.WithTTL(defaultCaptureSessionTTL)) + concurrency.WithTTL(config.GetDefaultServerConfig().CaptureSessionTTL)) c.Assert(err, check.IsNil) mockOwner := Owner{ @@ -254,32 +391,41 @@ func (s *ownerSuite) TestOwnerHandleStaleChangeFeed(c *check.C) { etcdClient: s.client, lastFlushChangefeeds: time.Now(), flushChangefeedInterval: 1 * time.Hour, - // gcSafepointLastUpdate: time.Now(), - gcTTL: 6, // 6 seconds - changeFeeds: changeFeeds, - cfRWriter: s.client, - stoppedFeeds: make(map[model.ChangeFeedID]*model.ChangeFeedStatus), - minGCSafePointCache: minGCSafePointCacheEntry{}, + gcSafepointLastUpdate: time.Now().Add(-4 * time.Second), + gcTTL: 6, // 6 seconds + changeFeeds: changeFeeds, + cfRWriter: s.client, + stoppedFeeds: make(map[model.ChangeFeedID]*model.ChangeFeedStatus), + minGCSafePointCache: minGCSafePointCacheEntry{}, } - time.Sleep(3 * time.Second) err = mockOwner.flushChangeFeedInfos(s.ctx) c.Assert(err, check.IsNil) c.Assert(mockPDCli.invokeCounter, check.Equals, 1) - err = mockOwner.handleAdminJob(s.ctx) c.Assert(err, check.IsNil) + + time.Sleep(2 * time.Second) + err = mockOwner.flushChangeFeedInfos(s.ctx) + c.Assert(err, check.IsNil) + c.Assert(mockPDCli.invokeCounter, check.Equals, 2) err = mockOwner.handleAdminJob(s.ctx) c.Assert(err, check.IsNil) + c.Assert(mockOwner.stoppedFeeds["test_change_feed_1"], check.NotNil) - c.Assert(mockOwner.stoppedFeeds["test_change_feed_3"], check.NotNil) c.Assert(mockOwner.changeFeeds["test_change_feed_2"].info.State, check.Equals, model.StateNormal) time.Sleep(6 * time.Second) err = mockOwner.flushChangeFeedInfos(s.ctx) c.Assert(err, check.IsNil) - c.Assert(mockPDCli.invokeCounter, check.Equals, 2) + c.Assert(mockPDCli.invokeCounter, check.Equals, 3) + err = mockOwner.handleAdminJob(s.ctx) + c.Assert(err, check.IsNil) + time.Sleep(2 * time.Second) + err = mockOwner.flushChangeFeedInfos(s.ctx) + c.Assert(err, check.IsNil) + c.Assert(mockPDCli.invokeCounter, check.Equals, 4) err = mockOwner.handleAdminJob(s.ctx) c.Assert(err, check.IsNil) c.Assert(mockOwner.stoppedFeeds["test_change_feed_2"], check.NotNil) @@ -330,7 +476,7 @@ func (s *ownerSuite) TestOwnerUploadGCSafePointOutdated(c *check.C) { } session, err := concurrency.NewSession(s.client.Client.Unwrap(), - concurrency.WithTTL(defaultCaptureSessionTTL)) + concurrency.WithTTL(config.GetDefaultServerConfig().CaptureSessionTTL)) c.Assert(err, check.IsNil) mockOwner := Owner{ @@ -756,8 +902,7 @@ func (s *ownerSuite) TestHandleAdmin(c *check.C) { defer sink.Close() //nolint:errcheck sampleCF.sink = sink - capture, err := NewCapture(ctx, []string{s.clientURL.String()}, nil, - &security.Credential{}, "127.0.0.1:12034", &captureOpts{flushCheckpointInterval: time.Millisecond * 200}) + capture, err := NewCapture(ctx, []string{s.clientURL.String()}, nil, nil) c.Assert(err, check.IsNil) err = capture.Campaign(ctx) c.Assert(err, check.IsNil) @@ -1058,8 +1203,7 @@ func (s *ownerSuite) TestWatchCampaignKey(c *check.C) { defer s.TearDownTest(c) ctx, cancel := context.WithCancel(context.Background()) defer cancel() - capture, err := NewCapture(ctx, []string{s.clientURL.String()}, nil, - &security.Credential{}, "127.0.0.1:12034", &captureOpts{}) + capture, err := NewCapture(ctx, []string{s.clientURL.String()}, nil, nil) c.Assert(err, check.IsNil) err = capture.Campaign(ctx) c.Assert(err, check.IsNil) @@ -1116,8 +1260,7 @@ func (s *ownerSuite) TestCleanUpStaleTasks(c *check.C) { defer cancel() addr := "127.0.0.1:12034" ctx = util.PutCaptureAddrInCtx(ctx, addr) - capture, err := NewCapture(ctx, []string{s.clientURL.String()}, nil, - &security.Credential{}, addr, &captureOpts{}) + capture, err := NewCapture(ctx, []string{s.clientURL.String()}, nil, nil) c.Assert(err, check.IsNil) err = s.client.PutCaptureInfo(ctx, capture.info, capture.session.Lease()) c.Assert(err, check.IsNil) @@ -1203,8 +1346,7 @@ func (s *ownerSuite) TestWatchFeedChange(c *check.C) { defer cancel() addr := "127.0.0.1:12034" ctx = util.PutCaptureAddrInCtx(ctx, addr) - capture, err := NewCapture(ctx, []string{s.clientURL.String()}, nil, - &security.Credential{}, addr, &captureOpts{}) + capture, err := NewCapture(ctx, []string{s.clientURL.String()}, nil, nil) c.Assert(err, check.IsNil) owner, err := NewOwner(ctx, nil, &security.Credential{}, capture.session, cdcGCSafePointTTL4Test, time.Millisecond*200) diff --git a/cdc/processor/doc.go b/cdc/processor/doc.go index 456f2fc577c..89c304f152f 100644 --- a/cdc/processor/doc.go +++ b/cdc/processor/doc.go @@ -14,15 +14,15 @@ /* Package processor implements the processor logic based on ETCD worker(pkg/orchestrator). -There are three mainly module: Manager, Processor and TablePipeline(cdc/processor/pipeline). -The Manager's main responsibility is to maintain the Processor's life cycle, like create and destory the processor instances. +There are three main modules: Manager, Processor and TablePipeline(cdc/processor/pipeline). +The Manager's main responsibility is to maintain the Processor's life cycle, like create and destroy the processor instances. The Processor's main responsibility is to maintain the TablePipeline's life cycle according to the state stored by ETCD, and calculate the local resolved TS and local checkpoint Ts and put them into ETCD. -The TablePipeline listen the kv change logs of a specified table(with its mark table if exist), send logs to Sink After sorting and mounting. +The TablePipeline listens to the kv change logs of a specified table(with its mark table if it exists), and sends logs to Sink After sorting and mounting. The relationship between the three module is as follows: -One Capture(with processor role) -> Processr Manager -> Processor(changefeed1) -> TablePipeline(tableA) +One Capture(with processor role) -> Processor Manager -> Processor(changefeed1) -> TablePipeline(tableA) ╲ ╲ ╲ -> TablePipeline(tableB) ╲ diff --git a/cdc/processor/manager.go b/cdc/processor/manager.go index 27669b7b989..3d14e4c19c4 100644 --- a/cdc/processor/manager.go +++ b/cdc/processor/manager.go @@ -23,10 +23,10 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/log" "github.com/pingcap/ticdc/cdc/model" + tablepipeline "github.com/pingcap/ticdc/cdc/processor/pipeline" + cdcContext "github.com/pingcap/ticdc/pkg/context" cerrors "github.com/pingcap/ticdc/pkg/errors" "github.com/pingcap/ticdc/pkg/orchestrator" - "github.com/pingcap/ticdc/pkg/security" - pd "github.com/tikv/pd/client" "go.uber.org/zap" ) @@ -48,55 +48,64 @@ type command struct { type Manager struct { processors map[model.ChangeFeedID]*processor - pdCli pd.Client - credential *security.Credential - captureInfo *model.CaptureInfo - commandQueue chan *command - newProcessor func( - pdCli pd.Client, - changefeedID model.ChangeFeedID, - credential *security.Credential, - captureInfo *model.CaptureInfo, - ) *processor + newProcessor func(cdcContext.Context) *processor } // NewManager creates a new processor manager -func NewManager(pdCli pd.Client, credential *security.Credential, captureInfo *model.CaptureInfo) *Manager { +func NewManager() *Manager { return &Manager{ - processors: make(map[model.ChangeFeedID]*processor), - pdCli: pdCli, - credential: credential, - captureInfo: captureInfo, - + processors: make(map[model.ChangeFeedID]*processor), commandQueue: make(chan *command, 4), newProcessor: newProcessor, } } +// NewManager4Test creates a new processor manager for test +func NewManager4Test( + createTablePipeline func(ctx cdcContext.Context, tableID model.TableID, replicaInfo *model.TableReplicaInfo) (tablepipeline.TablePipeline, error), +) *Manager { + m := NewManager() + m.newProcessor = func(ctx cdcContext.Context) *processor { + return newProcessor4Test(ctx, createTablePipeline) + } + return m +} + // Tick implements the `orchestrator.State` interface // the `state` parameter is sent by the etcd worker, the `state` must be a snapshot of KVs in etcd // the Tick function of Manager create or remove processor instances according to the specified `state`, or pass the `state` to processor instances -func (m *Manager) Tick(ctx context.Context, state orchestrator.ReactorState) (nextState orchestrator.ReactorState, err error) { - globalState := state.(*globalState) +func (m *Manager) Tick(stdCtx context.Context, state orchestrator.ReactorState) (nextState orchestrator.ReactorState, err error) { + ctx := stdCtx.(cdcContext.Context) + globalState := state.(*model.GlobalReactorState) if err := m.handleCommand(); err != nil { return state, err } + captureID := ctx.GlobalVars().CaptureInfo.ID var inactiveChangefeedCount int for changefeedID, changefeedState := range globalState.Changefeeds { - if !changefeedState.Active() { + if !changefeedState.Active(captureID) { inactiveChangefeedCount++ m.closeProcessor(changefeedID) continue } + ctx := cdcContext.WithChangefeedVars(ctx, &cdcContext.ChangefeedVars{ + ID: changefeedID, + Info: changefeedState.Info, + }) processor, exist := m.processors[changefeedID] if !exist { - if changefeedState.TaskStatus.AdminJobType.IsStopState() { + if changefeedState.Status.AdminJobType.IsStopState() || changefeedState.TaskStatuses[captureID].AdminJobType.IsStopState() { + continue + } + // the processor should start after at least one table has been added to this capture + taskStatus := changefeedState.TaskStatuses[captureID] + if taskStatus == nil || (len(taskStatus.Tables) == 0 && len(taskStatus.Operation) == 0) { continue } failpoint.Inject("processorManagerHandleNewChangefeedDelay", nil) - processor = m.newProcessor(m.pdCli, changefeedID, m.credential, m.captureInfo) + processor = m.newProcessor(ctx) m.processors[changefeedID] = processor } if _, err := processor.Tick(ctx, changefeedState); err != nil { @@ -141,7 +150,7 @@ func (m *Manager) WriteDebugInfo(w io.Writer) { select { case <-done: case <-time.After(timeout): - fmt.Fprintf(w, "failed to print debug info\n") + fmt.Fprintf(w, "failed to print debug info for processor\n") } } diff --git a/cdc/processor/manager_test.go b/cdc/processor/manager_test.go index cb5d72c0de7..78bbc7df086 100644 --- a/cdc/processor/manager_test.go +++ b/cdc/processor/manager_test.go @@ -15,165 +15,196 @@ package processor import ( "bytes" - "context" + "fmt" "math" "time" "github.com/pingcap/check" "github.com/pingcap/errors" "github.com/pingcap/ticdc/cdc/model" + tablepipeline "github.com/pingcap/ticdc/cdc/processor/pipeline" "github.com/pingcap/ticdc/pkg/config" + cdcContext "github.com/pingcap/ticdc/pkg/context" cerrors "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/security" + "github.com/pingcap/ticdc/pkg/orchestrator" "github.com/pingcap/ticdc/pkg/util/testleak" - pd "github.com/tikv/pd/client" ) -type managerSuite struct{} +type managerSuite struct { + manager *Manager + state *model.GlobalReactorState + tester *orchestrator.ReactorStateTester +} var _ = check.Suite(&managerSuite{}) -func newManager4Test() *Manager { - m := NewManager(nil, nil, &model.CaptureInfo{ - ID: "test-captureID", - AdvertiseAddr: "127.0.0.1:0000", +func (s *managerSuite) resetSuit(ctx cdcContext.Context, c *check.C) { + s.manager = NewManager4Test(func(ctx cdcContext.Context, tableID model.TableID, replicaInfo *model.TableReplicaInfo) (tablepipeline.TablePipeline, error) { + return &mockTablePipeline{ + tableID: tableID, + name: fmt.Sprintf("`test`.`table%d`", tableID), + status: tablepipeline.TableStatusRunning, + resolvedTs: replicaInfo.StartTs, + checkpointTs: replicaInfo.StartTs, + }, nil + }) + s.state = model.NewGlobalState().(*model.GlobalReactorState) + captureInfoBytes, err := ctx.GlobalVars().CaptureInfo.Marshal() + c.Assert(err, check.IsNil) + s.tester = orchestrator.NewReactorStateTester(c, s.state, map[string]string{ + fmt.Sprintf("/tidb/cdc/capture/%s", ctx.GlobalVars().CaptureInfo.ID): string(captureInfoBytes), }) - m.newProcessor = func( - pdCli pd.Client, - changefeedID model.ChangeFeedID, - credential *security.Credential, - captureInfo *model.CaptureInfo, - ) *processor { - return newProcessor4Test() - } - return m } func (s *managerSuite) TestChangefeed(c *check.C) { defer testleak.AfterTest(c)() - ctx := context.Background() - m := newManager4Test() - state := &globalState{ - CaptureID: "test-captureID", - Changefeeds: make(map[model.ChangeFeedID]*changefeedState), - } + ctx := cdcContext.NewBackendContext4Test(false) + s.resetSuit(ctx, c) var err error // no changefeed - _, err = m.Tick(ctx, state) + _, err = s.manager.Tick(ctx, s.state) c.Assert(err, check.IsNil) // an inactive changefeed - state.Changefeeds["test-changefeed"] = newChangeFeedState("test-changefeed", state.CaptureID) - _, err = m.Tick(ctx, state) + s.state.Changefeeds["test-changefeed"] = model.NewChangefeedReactorState("test-changefeed") + _, err = s.manager.Tick(ctx, s.state) + s.tester.MustApplyPatches() c.Assert(err, check.IsNil) - c.Assert(m.processors, check.HasLen, 0) + c.Assert(s.manager.processors, check.HasLen, 0) // an active changefeed - state.Changefeeds["test-changefeed"].Info = &model.ChangeFeedInfo{ - SinkURI: "blackhole://", - CreateTime: time.Now(), - StartTs: 0, - TargetTs: math.MaxUint64, - Config: config.GetDefaultReplicaConfig(), - } - state.Changefeeds["test-changefeed"].Status = &model.ChangeFeedStatus{} - state.Changefeeds["test-changefeed"].TaskStatus = &model.TaskStatus{ - Tables: map[int64]*model.TableReplicaInfo{}, - } - _, err = m.Tick(ctx, state) + s.state.Changefeeds["test-changefeed"].PatchInfo(func(info *model.ChangeFeedInfo) (*model.ChangeFeedInfo, bool, error) { + return &model.ChangeFeedInfo{ + SinkURI: "blackhole://", + CreateTime: time.Now(), + StartTs: 0, + TargetTs: math.MaxUint64, + Config: config.GetDefaultReplicaConfig(), + }, true, nil + }) + s.state.Changefeeds["test-changefeed"].PatchStatus(func(status *model.ChangeFeedStatus) (*model.ChangeFeedStatus, bool, error) { + return &model.ChangeFeedStatus{}, true, nil + }) + s.state.Changefeeds["test-changefeed"].PatchTaskStatus(ctx.GlobalVars().CaptureInfo.ID, func(status *model.TaskStatus) (*model.TaskStatus, bool, error) { + return &model.TaskStatus{ + Tables: map[int64]*model.TableReplicaInfo{1: {}}, + }, true, nil + }) + s.tester.MustApplyPatches() + _, err = s.manager.Tick(ctx, s.state) + s.tester.MustApplyPatches() c.Assert(err, check.IsNil) - c.Assert(m.processors, check.HasLen, 1) + c.Assert(s.manager.processors, check.HasLen, 1) // processor return errors - state.Changefeeds["test-changefeed"].TaskStatus.AdminJobType = model.AdminStop - _, err = m.Tick(ctx, state) + s.state.Changefeeds["test-changefeed"].PatchStatus(func(status *model.ChangeFeedStatus) (*model.ChangeFeedStatus, bool, error) { + status.AdminJobType = model.AdminStop + return status, true, nil + }) + s.state.Changefeeds["test-changefeed"].PatchTaskStatus(ctx.GlobalVars().CaptureInfo.ID, func(status *model.TaskStatus) (*model.TaskStatus, bool, error) { + status.AdminJobType = model.AdminStop + return status, true, nil + }) + s.tester.MustApplyPatches() + _, err = s.manager.Tick(ctx, s.state) + s.tester.MustApplyPatches() c.Assert(err, check.IsNil) - c.Assert(m.processors, check.HasLen, 0) + c.Assert(s.manager.processors, check.HasLen, 0) } func (s *managerSuite) TestDebugInfo(c *check.C) { defer testleak.AfterTest(c)() - ctx := context.Background() - m := newManager4Test() - state := &globalState{ - CaptureID: "test-captureID", - Changefeeds: make(map[model.ChangeFeedID]*changefeedState), - } + ctx := cdcContext.NewBackendContext4Test(false) + s.resetSuit(ctx, c) var err error // no changefeed - _, err = m.Tick(ctx, state) + _, err = s.manager.Tick(ctx, s.state) c.Assert(err, check.IsNil) // an active changefeed - state.Changefeeds["test-changefeed"] = newChangeFeedState("test-changefeed", state.CaptureID) - state.Changefeeds["test-changefeed"].Info = &model.ChangeFeedInfo{ - SinkURI: "blackhole://", - CreateTime: time.Now(), - StartTs: 0, - TargetTs: math.MaxUint64, - Config: config.GetDefaultReplicaConfig(), - } - state.Changefeeds["test-changefeed"].Status = &model.ChangeFeedStatus{} - state.Changefeeds["test-changefeed"].TaskStatus = &model.TaskStatus{ - Tables: map[int64]*model.TableReplicaInfo{}, - } - _, err = m.Tick(ctx, state) + s.state.Changefeeds["test-changefeed"] = model.NewChangefeedReactorState("test-changefeed") + s.state.Changefeeds["test-changefeed"].PatchInfo(func(info *model.ChangeFeedInfo) (*model.ChangeFeedInfo, bool, error) { + return &model.ChangeFeedInfo{ + SinkURI: "blackhole://", + CreateTime: time.Now(), + StartTs: 0, + TargetTs: math.MaxUint64, + Config: config.GetDefaultReplicaConfig(), + }, true, nil + }) + s.state.Changefeeds["test-changefeed"].PatchStatus(func(status *model.ChangeFeedStatus) (*model.ChangeFeedStatus, bool, error) { + return &model.ChangeFeedStatus{}, true, nil + }) + s.state.Changefeeds["test-changefeed"].PatchTaskStatus(ctx.GlobalVars().CaptureInfo.ID, func(status *model.TaskStatus) (*model.TaskStatus, bool, error) { + return &model.TaskStatus{ + Tables: map[int64]*model.TableReplicaInfo{1: {}}, + }, true, nil + }) + s.tester.MustApplyPatches() + _, err = s.manager.Tick(ctx, s.state) c.Assert(err, check.IsNil) - c.Assert(m.processors, check.HasLen, 1) + s.tester.MustApplyPatches() + c.Assert(s.manager.processors, check.HasLen, 1) done := make(chan struct{}) go func() { defer close(done) for { - _, err := m.Tick(ctx, state) + _, err = s.manager.Tick(ctx, s.state) if err != nil { c.Assert(cerrors.ErrReactorFinished.Equal(errors.Cause(err)), check.IsTrue) return } c.Assert(err, check.IsNil) + s.tester.MustApplyPatches() } }() buf := bytes.NewBufferString("") - m.WriteDebugInfo(buf) + s.manager.WriteDebugInfo(buf) c.Assert(len(buf.String()), check.Greater, 0) - m.AsyncClose() + s.manager.AsyncClose() <-done } func (s *managerSuite) TestClose(c *check.C) { defer testleak.AfterTest(c)() - ctx := context.Background() - m := newManager4Test() - state := &globalState{ - CaptureID: "test-captureID", - Changefeeds: make(map[model.ChangeFeedID]*changefeedState), - } + ctx := cdcContext.NewBackendContext4Test(false) + s.resetSuit(ctx, c) var err error // no changefeed - _, err = m.Tick(ctx, state) + _, err = s.manager.Tick(ctx, s.state) c.Assert(err, check.IsNil) // an active changefeed - state.Changefeeds["test-changefeed"] = newChangeFeedState("test-changefeed", state.CaptureID) - state.Changefeeds["test-changefeed"].Info = &model.ChangeFeedInfo{ - SinkURI: "blackhole://", - CreateTime: time.Now(), - StartTs: 0, - TargetTs: math.MaxUint64, - Config: config.GetDefaultReplicaConfig(), - } - state.Changefeeds["test-changefeed"].Status = &model.ChangeFeedStatus{} - state.Changefeeds["test-changefeed"].TaskStatus = &model.TaskStatus{ - Tables: map[int64]*model.TableReplicaInfo{}, - } - _, err = m.Tick(ctx, state) + s.state.Changefeeds["test-changefeed"] = model.NewChangefeedReactorState("test-changefeed") + s.state.Changefeeds["test-changefeed"].PatchInfo(func(info *model.ChangeFeedInfo) (*model.ChangeFeedInfo, bool, error) { + return &model.ChangeFeedInfo{ + SinkURI: "blackhole://", + CreateTime: time.Now(), + StartTs: 0, + TargetTs: math.MaxUint64, + Config: config.GetDefaultReplicaConfig(), + }, true, nil + }) + s.state.Changefeeds["test-changefeed"].PatchStatus(func(status *model.ChangeFeedStatus) (*model.ChangeFeedStatus, bool, error) { + return &model.ChangeFeedStatus{}, true, nil + }) + s.state.Changefeeds["test-changefeed"].PatchTaskStatus(ctx.GlobalVars().CaptureInfo.ID, func(status *model.TaskStatus) (*model.TaskStatus, bool, error) { + return &model.TaskStatus{ + Tables: map[int64]*model.TableReplicaInfo{1: {}}, + }, true, nil + }) + s.tester.MustApplyPatches() + _, err = s.manager.Tick(ctx, s.state) c.Assert(err, check.IsNil) - c.Assert(m.processors, check.HasLen, 1) + s.tester.MustApplyPatches() + c.Assert(s.manager.processors, check.HasLen, 1) - m.AsyncClose() - _, err = m.Tick(ctx, state) + s.manager.AsyncClose() + _, err = s.manager.Tick(ctx, s.state) c.Assert(cerrors.ErrReactorFinished.Equal(errors.Cause(err)), check.IsTrue) - c.Assert(m.processors, check.HasLen, 0) + s.tester.MustApplyPatches() + c.Assert(s.manager.processors, check.HasLen, 0) } diff --git a/cdc/processor/pipeline/cyclic_mark.go b/cdc/processor/pipeline/cyclic_mark.go index 665714c5dca..b0fd2415037 100644 --- a/cdc/processor/pipeline/cyclic_mark.go +++ b/cdc/processor/pipeline/cyclic_mark.go @@ -46,8 +46,8 @@ func newCyclicMarkNode(markTableID model.TableID) pipeline.Node { } func (n *cyclicMarkNode) Init(ctx pipeline.NodeContext) error { - n.localReplicaID = ctx.Vars().Config.Cyclic.ReplicaID - filterReplicaID := ctx.Vars().Config.Cyclic.FilterReplicaID + n.localReplicaID = ctx.ChangefeedVars().Info.Config.Cyclic.ReplicaID + filterReplicaID := ctx.ChangefeedVars().Info.Config.Cyclic.FilterReplicaID n.filterReplicaID = make(map[uint64]struct{}) for _, rID := range filterReplicaID { n.filterReplicaID[rID] = struct{}{} @@ -110,7 +110,7 @@ func (n *cyclicMarkNode) appendMarkRow(ctx pipeline.NodeContext, event *model.Po if event.CRTs != n.currentCommitTs { log.Panic("the CommitTs of the received event is not equal to the currentCommitTs, please report a bug", zap.Reflect("event", event), zap.Uint64("currentCommitTs", n.currentCommitTs)) } - err := event.WaitPrepare(ctx.StdContext()) + err := event.WaitPrepare(ctx) if err != nil { return errors.Trace(err) } diff --git a/cdc/processor/pipeline/cyclic_mark_test.go b/cdc/processor/pipeline/cyclic_mark_test.go index f22b7bddce1..95e775960c2 100644 --- a/cdc/processor/pipeline/cyclic_mark_test.go +++ b/cdc/processor/pipeline/cyclic_mark_test.go @@ -14,7 +14,7 @@ package pipeline import ( - stdContext "context" + "context" "sort" "sync" @@ -22,7 +22,7 @@ import ( "github.com/pingcap/check" "github.com/pingcap/ticdc/cdc/model" "github.com/pingcap/ticdc/pkg/config" - "github.com/pingcap/ticdc/pkg/context" + cdcContext "github.com/pingcap/ticdc/pkg/context" "github.com/pingcap/ticdc/pkg/cyclic/mark" "github.com/pingcap/ticdc/pkg/pipeline" "github.com/pingcap/ticdc/pkg/util/testleak" @@ -131,12 +131,15 @@ func (s *markSuite) TestCyclicMarkNode(c *check.C) { } for _, tc := range testCases { - ctx := context.NewContext(stdContext.Background(), &context.Vars{ - Config: &config.ReplicaConfig{ - Cyclic: &config.CyclicConfig{ - Enable: true, - ReplicaID: tc.replicaID, - FilterReplicaID: tc.filterID, + ctx := cdcContext.NewContext(context.Background(), &cdcContext.GlobalVars{}) + ctx = cdcContext.WithChangefeedVars(ctx, &cdcContext.ChangefeedVars{ + Info: &model.ChangeFeedInfo{ + Config: &config.ReplicaConfig{ + Cyclic: &config.CyclicConfig{ + Enable: true, + ReplicaID: tc.replicaID, + FilterReplicaID: tc.filterID, + }, }, }, }) diff --git a/cdc/processor/pipeline/puller.go b/cdc/processor/pipeline/puller.go index 6a2a0b6da5a..db5d6a9b419 100644 --- a/cdc/processor/pipeline/puller.go +++ b/cdc/processor/pipeline/puller.go @@ -14,71 +14,62 @@ package pipeline import ( - stdContext "context" + "context" "github.com/pingcap/errors" "github.com/pingcap/ticdc/cdc/model" "github.com/pingcap/ticdc/cdc/puller" - "github.com/pingcap/ticdc/pkg/context" + "github.com/pingcap/ticdc/pkg/config" + cdcContext "github.com/pingcap/ticdc/pkg/context" "github.com/pingcap/ticdc/pkg/pipeline" "github.com/pingcap/ticdc/pkg/regionspan" - "github.com/pingcap/ticdc/pkg/security" "github.com/pingcap/ticdc/pkg/util" - tidbkv "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/store/tikv/oracle" "golang.org/x/sync/errgroup" ) type pullerNode struct { - credential *security.Credential - kvStorage tidbkv.Storage - limitter *puller.BlurResourceLimitter + limitter *puller.BlurResourceLimitter - changefeedID model.ChangeFeedID - tableName string // quoted schema and table, used in metircs only + tableName string // quoted schema and table, used in metircs only tableID model.TableID replicaInfo *model.TableReplicaInfo - cancel stdContext.CancelFunc + cancel context.CancelFunc wg errgroup.Group } func newPullerNode( - changefeedID model.ChangeFeedID, - credential *security.Credential, - kvStorage tidbkv.Storage, limitter *puller.BlurResourceLimitter, tableID model.TableID, replicaInfo *model.TableReplicaInfo, tableName string) pipeline.Node { return &pullerNode{ - credential: credential, - kvStorage: kvStorage, - limitter: limitter, - tableID: tableID, - replicaInfo: replicaInfo, - tableName: tableName, - changefeedID: changefeedID, + limitter: limitter, + tableID: tableID, + replicaInfo: replicaInfo, + tableName: tableName, } } -func (n *pullerNode) tableSpan(ctx context.Context) []regionspan.Span { +func (n *pullerNode) tableSpan(ctx cdcContext.Context) []regionspan.Span { // start table puller - enableOldValue := ctx.Vars().Config.EnableOldValue + config := ctx.ChangefeedVars().Info.Config spans := make([]regionspan.Span, 0, 4) - spans = append(spans, regionspan.GetTableSpan(n.tableID, enableOldValue)) + spans = append(spans, regionspan.GetTableSpan(n.tableID, config.EnableOldValue)) - if ctx.Vars().Config.Cyclic.IsEnabled() && n.replicaInfo.MarkTableID != 0 { - spans = append(spans, regionspan.GetTableSpan(n.replicaInfo.MarkTableID, enableOldValue)) + if config.Cyclic.IsEnabled() && n.replicaInfo.MarkTableID != 0 { + spans = append(spans, regionspan.GetTableSpan(n.replicaInfo.MarkTableID, config.EnableOldValue)) } return spans } func (n *pullerNode) Init(ctx pipeline.NodeContext) error { - metricTableResolvedTsGauge := tableResolvedTsGauge.WithLabelValues(n.changefeedID, ctx.Vars().CaptureAddr, n.tableName) - enableOldValue := ctx.Vars().Config.EnableOldValue - ctxC, cancel := stdContext.WithCancel(ctx.StdContext()) + metricTableResolvedTsGauge := tableResolvedTsGauge.WithLabelValues(ctx.ChangefeedVars().ID, ctx.GlobalVars().CaptureInfo.AdvertiseAddr, n.tableName) + globalConfig := config.GetGlobalServerConfig() + config := ctx.ChangefeedVars().Info.Config + ctxC, cancel := context.WithCancel(ctx) ctxC = util.PutTableInfoInCtx(ctxC, n.tableID, n.tableName) - plr := puller.NewPuller(ctxC, ctx.Vars().PDClient, n.credential, n.kvStorage, - n.replicaInfo.StartTs, n.tableSpan(ctx), n.limitter, enableOldValue) + plr := puller.NewPuller(ctxC, ctx.GlobalVars().PDClient, globalConfig.Security, ctx.GlobalVars().KVStorage, + n.replicaInfo.StartTs, n.tableSpan(ctx), n.limitter, config.EnableOldValue) n.wg.Go(func() error { ctx.Throw(errors.Trace(plr.Run(ctxC))) return nil @@ -112,7 +103,7 @@ func (n *pullerNode) Receive(ctx pipeline.NodeContext) error { } func (n *pullerNode) Destroy(ctx pipeline.NodeContext) error { - tableResolvedTsGauge.DeleteLabelValues(n.changefeedID, ctx.Vars().CaptureAddr, n.tableName) + tableResolvedTsGauge.DeleteLabelValues(ctx.ChangefeedVars().ID, ctx.GlobalVars().CaptureInfo.AdvertiseAddr, n.tableName) n.cancel() return n.wg.Wait() } diff --git a/cdc/processor/pipeline/sink.go b/cdc/processor/pipeline/sink.go index 02eb0bdba54..fa492118e20 100644 --- a/cdc/processor/pipeline/sink.go +++ b/cdc/processor/pipeline/sink.go @@ -53,11 +53,13 @@ func (s TableStatus) String() string { return "Unknown" } -func (s *TableStatus) load() TableStatus { +// Load TableStatus with THREAD-SAFE +func (s *TableStatus) Load() TableStatus { return TableStatus(atomic.LoadInt32((*int32)(s))) } -func (s *TableStatus) store(new TableStatus) { +// Store TableStatus with THREAD-SAFE +func (s *TableStatus) Store(new TableStatus) { atomic.StoreInt32((*int32)(s), int32(new)) } @@ -91,7 +93,7 @@ func newSinkNode(sink sink.Sink, startTs model.Ts, targetTs model.Ts, flowContro func (n *sinkNode) ResolvedTs() model.Ts { return atomic.LoadUint64(&n.resolvedTs) } func (n *sinkNode) CheckpointTs() model.Ts { return atomic.LoadUint64(&n.checkpointTs) } -func (n *sinkNode) Status() TableStatus { return n.status.load() } +func (n *sinkNode) Status() TableStatus { return n.status.Load() } func (n *sinkNode) Init(ctx pipeline.NodeContext) error { // do nothing @@ -101,11 +103,11 @@ func (n *sinkNode) Init(ctx pipeline.NodeContext) error { func (n *sinkNode) flushSink(ctx pipeline.NodeContext, resolvedTs model.Ts) (err error) { defer func() { if err != nil { - n.status.store(TableStatusStopped) + n.status.Store(TableStatusStopped) return } if n.checkpointTs >= n.targetTs { - n.status.store(TableStatusStopped) + n.status.Store(TableStatusStopped) err = n.sink.Close() if err != nil { err = errors.Trace(err) @@ -126,7 +128,7 @@ func (n *sinkNode) flushSink(ctx pipeline.NodeContext, resolvedTs model.Ts) (err if err := n.flushRow2Sink(ctx); err != nil { return errors.Trace(err) } - checkpointTs, err := n.sink.FlushRowChangedEvents(ctx.StdContext(), resolvedTs) + checkpointTs, err := n.sink.FlushRowChangedEvents(ctx, resolvedTs) if err != nil { return errors.Trace(err) } @@ -150,9 +152,8 @@ func (n *sinkNode) emitEvent(ctx pipeline.NodeContext, event *model.PolymorphicE } func (n *sinkNode) flushRow2Sink(ctx pipeline.NodeContext) error { - stdCtx := ctx.StdContext() for _, ev := range n.eventBuffer { - err := ev.WaitPrepare(stdCtx) + err := ev.WaitPrepare(ctx) if err != nil { return errors.Trace(err) } @@ -167,7 +168,7 @@ func (n *sinkNode) flushRow2Sink(ctx pipeline.NodeContext) error { time.Sleep(10 * time.Second) panic("ProcessorSyncResolvedPreEmit") }) - err := n.sink.EmitRowChangedEvents(stdCtx, n.rowBuffer...) + err := n.sink.EmitRowChangedEvents(ctx, n.rowBuffer...) if err != nil { return errors.Trace(err) } @@ -184,7 +185,7 @@ func (n *sinkNode) Receive(ctx pipeline.NodeContext) error { event := msg.PolymorphicEvent if event.RawKV.OpType == model.OpTypeResolved { if n.status == TableStatusInitializing { - n.status.store(TableStatusRunning) + n.status.Store(TableStatusRunning) } failpoint.Inject("ProcessorSyncResolvedError", func() { failpoint.Return(errors.New("processor sync resolved injected error")) @@ -221,7 +222,7 @@ func (n *sinkNode) Receive(ctx pipeline.NodeContext) error { } func (n *sinkNode) Destroy(ctx pipeline.NodeContext) error { - n.status.store(TableStatusStopped) + n.status.Store(TableStatusStopped) n.flowController.Abort() return n.sink.Close() } diff --git a/cdc/processor/pipeline/sink_test.go b/cdc/processor/pipeline/sink_test.go index 34b02440a8d..910ac91a3c0 100644 --- a/cdc/processor/pipeline/sink_test.go +++ b/cdc/processor/pipeline/sink_test.go @@ -14,12 +14,12 @@ package pipeline import ( - stdContext "context" + "context" "testing" "github.com/pingcap/check" "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/context" + cdcContext "github.com/pingcap/ticdc/pkg/context" cerrors "github.com/pingcap/ticdc/pkg/errors" "github.com/pingcap/ticdc/pkg/pipeline" "github.com/pingcap/ticdc/pkg/util/testleak" @@ -54,11 +54,11 @@ func (c *mockFlowController) GetConsumption() uint64 { return 0 } -func (s *mockSink) Initialize(ctx stdContext.Context, tableInfo []*model.SimpleTableInfo) error { +func (s *mockSink) Initialize(ctx context.Context, tableInfo []*model.SimpleTableInfo) error { return nil } -func (s *mockSink) EmitRowChangedEvents(ctx stdContext.Context, rows ...*model.RowChangedEvent) error { +func (s *mockSink) EmitRowChangedEvents(ctx context.Context, rows ...*model.RowChangedEvent) error { for _, row := range rows { s.received = append(s.received, struct { resolvedTs model.Ts @@ -68,11 +68,11 @@ func (s *mockSink) EmitRowChangedEvents(ctx stdContext.Context, rows ...*model.R return nil } -func (s *mockSink) EmitDDLEvent(ctx stdContext.Context, ddl *model.DDLEvent) error { +func (s *mockSink) EmitDDLEvent(ctx context.Context, ddl *model.DDLEvent) error { panic("unreachable") } -func (s *mockSink) FlushRowChangedEvents(ctx stdContext.Context, resolvedTs uint64) (uint64, error) { +func (s *mockSink) FlushRowChangedEvents(ctx context.Context, resolvedTs uint64) (uint64, error) { s.received = append(s.received, struct { resolvedTs model.Ts row *model.RowChangedEvent @@ -80,7 +80,7 @@ func (s *mockSink) FlushRowChangedEvents(ctx stdContext.Context, resolvedTs uint return resolvedTs, nil } -func (s *mockSink) EmitCheckpointTs(ctx stdContext.Context, ts uint64) error { +func (s *mockSink) EmitCheckpointTs(ctx context.Context, ts uint64) error { panic("unreachable") } @@ -105,7 +105,7 @@ var _ = check.Suite(&outputSuite{}) func (s *outputSuite) TestStatus(c *check.C) { defer testleak.AfterTest(c)() - ctx := context.NewContext(stdContext.Background(), &context.Vars{}) + ctx := cdcContext.NewContext(context.Background(), &cdcContext.GlobalVars{}) // test stop at targetTs node := newSinkNode(&mockSink{}, 0, 10, &mockFlowController{}) @@ -180,7 +180,7 @@ func (s *outputSuite) TestStatus(c *check.C) { func (s *outputSuite) TestManyTs(c *check.C) { defer testleak.AfterTest(c)() - ctx := context.NewContext(stdContext.Background(), &context.Vars{}) + ctx := cdcContext.NewContext(context.Background(), &cdcContext.GlobalVars{}) sink := &mockSink{} node := newSinkNode(sink, 0, 10, &mockFlowController{}) c.Assert(node.Init(pipeline.MockNodeContext4Test(ctx, nil, nil)), check.IsNil) diff --git a/cdc/processor/pipeline/sorter.go b/cdc/processor/pipeline/sorter.go index 591efcb6fca..47cae5781be 100644 --- a/cdc/processor/pipeline/sorter.go +++ b/cdc/processor/pipeline/sorter.go @@ -35,13 +35,10 @@ const ( ) type sorterNode struct { - sortEngine model.SortEngine - sortDir string - sorter puller.EventSorter + sorter puller.EventSorter - changeFeedID model.ChangeFeedID - tableID model.TableID - tableName string // quoted schema and table, used in metircs only + tableID model.TableID + tableName string // quoted schema and table, used in metircs only // for per-table flow control flowController tableFlowController @@ -50,36 +47,28 @@ type sorterNode struct { cancel context.CancelFunc } -func newSorterNode( - sortEngine model.SortEngine, - sortDir string, - changeFeedID model.ChangeFeedID, - tableName string, tableID model.TableID, - flowController tableFlowController) pipeline.Node { +func newSorterNode(tableName string, tableID model.TableID, flowController tableFlowController) pipeline.Node { return &sorterNode{ - sortEngine: sortEngine, - sortDir: sortDir, - - changeFeedID: changeFeedID, - tableID: tableID, - tableName: tableName, - + tableName: tableName, + tableID: tableID, flowController: flowController, } } func (n *sorterNode) Init(ctx pipeline.NodeContext) error { - stdCtx, cancel := context.WithCancel(ctx.StdContext()) + stdCtx, cancel := context.WithCancel(ctx) n.cancel = cancel var sorter puller.EventSorter - switch n.sortEngine { + sortEngine := ctx.ChangefeedVars().Info.Engine + switch sortEngine { case model.SortInMemory: sorter = puller.NewEntrySorter() case model.SortInFile: - err := util.IsDirAndWritable(n.sortDir) + sortDir := ctx.ChangefeedVars().Info.SortDir + err := util.IsDirAndWritable(sortDir) if err != nil { if os.IsNotExist(errors.Cause(err)) { - err = os.MkdirAll(n.sortDir, 0o755) + err = os.MkdirAll(sortDir, 0o755) if err != nil { return errors.Annotate(cerror.WrapError(cerror.ErrProcessorSortDir, err), "create dir") } @@ -88,18 +77,19 @@ func (n *sorterNode) Init(ctx pipeline.NodeContext) error { } } - sorter = puller.NewFileSorter(n.sortDir) + sorter = puller.NewFileSorter(sortDir) case model.SortUnified: - err := psorter.UnifiedSorterCheckDir(n.sortDir) + sortDir := ctx.ChangefeedVars().Info.SortDir + err := psorter.UnifiedSorterCheckDir(sortDir) if err != nil { return errors.Trace(err) } - sorter, err = psorter.NewUnifiedSorter(n.sortDir, n.changeFeedID, n.tableName, n.tableID, ctx.Vars().CaptureAddr) + sorter, err = psorter.NewUnifiedSorter(sortDir, ctx.ChangefeedVars().ID, n.tableName, n.tableID, ctx.GlobalVars().CaptureInfo.AdvertiseAddr) if err != nil { return errors.Trace(err) } default: - return cerror.ErrUnknownSortEngine.GenWithStackByArgs(n.sortEngine) + return cerror.ErrUnknownSortEngine.GenWithStackByArgs(sortEngine) } n.wg.Go(func() error { ctx.Throw(errors.Trace(sorter.Run(stdCtx))) @@ -118,7 +108,7 @@ func (n *sorterNode) Init(ctx pipeline.NodeContext) error { lastSendResolvedTsTime := time.Now() // the time at which we last sent a resolved-ts. lastCRTs := uint64(0) // the commit-ts of the last row changed we sent. - metricsTableMemoryGauge := tableMemoryGauge.WithLabelValues(n.changeFeedID, ctx.Vars().CaptureAddr, n.tableName) + metricsTableMemoryGauge := tableMemoryGauge.WithLabelValues(ctx.ChangefeedVars().ID, ctx.GlobalVars().CaptureInfo.AdvertiseAddr, n.tableName) metricsTicker := time.NewTicker(flushMemoryMetricsDuration) defer metricsTicker.Stop() @@ -197,7 +187,7 @@ func (n *sorterNode) Receive(ctx pipeline.NodeContext) error { msg := ctx.Message() switch msg.Tp { case pipeline.MessageTypePolymorphicEvent: - n.sorter.AddEntry(ctx.StdContext(), msg.PolymorphicEvent) + n.sorter.AddEntry(ctx, msg.PolymorphicEvent) default: ctx.SendToNextNode(msg) } @@ -205,7 +195,7 @@ func (n *sorterNode) Receive(ctx pipeline.NodeContext) error { } func (n *sorterNode) Destroy(ctx pipeline.NodeContext) error { - defer tableMemoryGauge.DeleteLabelValues(n.changeFeedID, ctx.Vars().CaptureAddr, n.tableName) + defer tableMemoryGauge.DeleteLabelValues(ctx.ChangefeedVars().ID, ctx.GlobalVars().CaptureInfo.AdvertiseAddr, n.tableName) n.cancel() return n.wg.Wait() } diff --git a/cdc/processor/pipeline/table.go b/cdc/processor/pipeline/table.go index 818c5203b31..b0c53180aed 100644 --- a/cdc/processor/pipeline/table.go +++ b/cdc/processor/pipeline/table.go @@ -14,7 +14,7 @@ package pipeline import ( - stdContext "context" + "context" "time" "github.com/pingcap/log" @@ -24,11 +24,9 @@ import ( "github.com/pingcap/ticdc/cdc/sink" "github.com/pingcap/ticdc/cdc/sink/common" serverConfig "github.com/pingcap/ticdc/pkg/config" - "github.com/pingcap/ticdc/pkg/context" + cdcContext "github.com/pingcap/ticdc/pkg/context" cerror "github.com/pingcap/ticdc/pkg/errors" "github.com/pingcap/ticdc/pkg/pipeline" - "github.com/pingcap/ticdc/pkg/security" - tidbkv "github.com/pingcap/tidb/kv" "go.uber.org/zap" ) @@ -70,7 +68,7 @@ type tablePipelineImpl struct { tableName string // quoted schema and table, used in metircs only sinkNode *sinkNode - cancel stdContext.CancelFunc + cancel context.CancelFunc } // TODO find a better name or avoid using an interface @@ -147,22 +145,16 @@ func (t *tablePipelineImpl) Wait() { } // NewTablePipeline creates a table pipeline -// TODO(leoppro): the parameters in this function are too much, try to move some parameters into ctx.Vars(). // TODO(leoppro): implement a mock kvclient to test the table pipeline -func NewTablePipeline(ctx context.Context, - changefeedID model.ChangeFeedID, - credential *security.Credential, - kvStorage tidbkv.Storage, +func NewTablePipeline(ctx cdcContext.Context, limitter *puller.BlurResourceLimitter, mounter entry.Mounter, - sortEngine model.SortEngine, - sortDir string, tableID model.TableID, tableName string, replicaInfo *model.TableReplicaInfo, sink sink.Sink, targetTs model.Ts) TablePipeline { - ctx, cancel := context.WithCancel(ctx) + ctx, cancel := cdcContext.WithCancel(ctx) tablePipeline := &tablePipelineImpl{ tableID: tableID, markTableID: replicaInfo.MarkTableID, @@ -172,16 +164,16 @@ func NewTablePipeline(ctx context.Context, perTableMemoryQuota := serverConfig.GetGlobalServerConfig().PerTableMemoryQuota log.Debug("creating table flow controller", - zap.String("changefeed-id", changefeedID), + zap.String("changefeed-id", ctx.ChangefeedVars().ID), zap.String("table-name", tableName), zap.Int64("table-id", tableID), zap.Uint64("quota", perTableMemoryQuota)) flowController := common.NewTableFlowController(perTableMemoryQuota) p := pipeline.NewPipeline(ctx, 500*time.Millisecond) - p.AppendNode(ctx, "puller", newPullerNode(changefeedID, credential, kvStorage, limitter, tableID, replicaInfo, tableName)) - p.AppendNode(ctx, "sorter", newSorterNode(sortEngine, sortDir, changefeedID, tableName, tableID, flowController)) + p.AppendNode(ctx, "puller", newPullerNode(limitter, tableID, replicaInfo, tableName)) + p.AppendNode(ctx, "sorter", newSorterNode(tableName, tableID, flowController)) p.AppendNode(ctx, "mounter", newMounterNode(mounter)) - config := ctx.Vars().Config + config := ctx.ChangefeedVars().Info.Config if config.Cyclic != nil && config.Cyclic.IsEnabled() { p.AppendNode(ctx, "cyclic", newCyclicMarkNode(replicaInfo.MarkTableID)) } diff --git a/cdc/processor/processor.go b/cdc/processor/processor.go index 7b81391b177..09342e402a2 100644 --- a/cdc/processor/processor.go +++ b/cdc/processor/processor.go @@ -17,6 +17,7 @@ import ( "context" "fmt" "io" + "math" "strconv" "sync" "time" @@ -30,18 +31,17 @@ import ( tablepipeline "github.com/pingcap/ticdc/cdc/processor/pipeline" "github.com/pingcap/ticdc/cdc/puller" "github.com/pingcap/ticdc/cdc/sink" - cdccontext "github.com/pingcap/ticdc/pkg/context" + "github.com/pingcap/ticdc/pkg/config" + cdcContext "github.com/pingcap/ticdc/pkg/context" "github.com/pingcap/ticdc/pkg/cyclic/mark" cerror "github.com/pingcap/ticdc/pkg/errors" "github.com/pingcap/ticdc/pkg/filter" "github.com/pingcap/ticdc/pkg/orchestrator" "github.com/pingcap/ticdc/pkg/regionspan" "github.com/pingcap/ticdc/pkg/retry" - "github.com/pingcap/ticdc/pkg/security" "github.com/pingcap/ticdc/pkg/util" "github.com/pingcap/tidb/store/tikv/oracle" "github.com/prometheus/client_golang/prometheus" - pd "github.com/tikv/pd/client" "go.uber.org/zap" ) @@ -53,26 +53,25 @@ const ( ) type processor struct { - changefeed *changefeedState + changefeedID model.ChangeFeedID + captureInfo *model.CaptureInfo + changefeed *model.ChangefeedReactorState tables map[model.TableID]tablepipeline.TablePipeline - pdCli pd.Client limitter *puller.BlurResourceLimitter - credential *security.Credential - captureInfo *model.CaptureInfo schemaStorage entry.SchemaStorage filter *filter.Filter mounter entry.Mounter sinkManager *sink.Manager - firstTick bool - errCh chan error - cancel context.CancelFunc - wg sync.WaitGroup + initialized bool + errCh chan error + cancel context.CancelFunc + wg sync.WaitGroup - lazyInit func(ctx context.Context) error - createTablePipeline func(ctx context.Context, tableID model.TableID, replicaInfo *model.TableReplicaInfo) (tablepipeline.TablePipeline, error) + lazyInit func(ctx cdcContext.Context) error + createTablePipeline func(ctx cdcContext.Context, tableID model.TableID, replicaInfo *model.TableReplicaInfo) (tablepipeline.TablePipeline, error) metricResolvedTsGauge prometheus.Gauge metricResolvedTsLagGauge prometheus.Gauge @@ -83,87 +82,97 @@ type processor struct { } // newProcessor creates a new processor -func newProcessor( - pdCli pd.Client, - changefeedID model.ChangeFeedID, - credential *security.Credential, - captureInfo *model.CaptureInfo, -) *processor { +func newProcessor(ctx cdcContext.Context) *processor { + changefeedID := ctx.ChangefeedVars().ID + advertiseAddr := ctx.GlobalVars().CaptureInfo.AdvertiseAddr p := &processor{ - pdCli: pdCli, - credential: credential, - captureInfo: captureInfo, - limitter: puller.NewBlurResourceLimmter(defaultMemBufferCapacity), - tables: make(map[model.TableID]tablepipeline.TablePipeline), - errCh: make(chan error, 1), - firstTick: true, - - metricResolvedTsGauge: resolvedTsGauge.WithLabelValues(changefeedID, captureInfo.AdvertiseAddr), - metricResolvedTsLagGauge: resolvedTsLagGauge.WithLabelValues(changefeedID, captureInfo.AdvertiseAddr), - metricCheckpointTsGauge: checkpointTsGauge.WithLabelValues(changefeedID, captureInfo.AdvertiseAddr), - metricCheckpointTsLagGauge: checkpointTsLagGauge.WithLabelValues(changefeedID, captureInfo.AdvertiseAddr), - metricSyncTableNumGauge: syncTableNumGauge.WithLabelValues(changefeedID, captureInfo.AdvertiseAddr), - metricProcessorErrorCounter: processorErrorCounter.WithLabelValues(changefeedID, captureInfo.AdvertiseAddr), + limitter: puller.NewBlurResourceLimmter(defaultMemBufferCapacity), + tables: make(map[model.TableID]tablepipeline.TablePipeline), + errCh: make(chan error, 1), + changefeedID: changefeedID, + captureInfo: ctx.GlobalVars().CaptureInfo, + cancel: func() {}, + + metricResolvedTsGauge: resolvedTsGauge.WithLabelValues(changefeedID, advertiseAddr), + metricResolvedTsLagGauge: resolvedTsLagGauge.WithLabelValues(changefeedID, advertiseAddr), + metricCheckpointTsGauge: checkpointTsGauge.WithLabelValues(changefeedID, advertiseAddr), + metricCheckpointTsLagGauge: checkpointTsLagGauge.WithLabelValues(changefeedID, advertiseAddr), + metricSyncTableNumGauge: syncTableNumGauge.WithLabelValues(changefeedID, advertiseAddr), + metricProcessorErrorCounter: processorErrorCounter.WithLabelValues(changefeedID, advertiseAddr), } p.createTablePipeline = p.createTablePipelineImpl p.lazyInit = p.lazyInitImpl return p } +func newProcessor4Test(ctx cdcContext.Context, + createTablePipeline func(ctx cdcContext.Context, tableID model.TableID, replicaInfo *model.TableReplicaInfo) (tablepipeline.TablePipeline, error), +) *processor { + p := newProcessor(ctx) + p.lazyInit = func(ctx cdcContext.Context) error { return nil } + p.createTablePipeline = createTablePipeline + return p +} + // Tick implements the `orchestrator.State` interface // the `state` parameter is sent by the etcd worker, the `state` must be a snapshot of KVs in etcd // The main logic of processor is in this function, including the calculation of many kinds of ts, maintain table pipeline, error handling, etc. -func (p *processor) Tick(ctx context.Context, state *changefeedState) (orchestrator.ReactorState, error) { +func (p *processor) Tick(ctx cdcContext.Context, state *model.ChangefeedReactorState) (orchestrator.ReactorState, error) { + p.changefeed = state + state.CheckCaptureAlive(ctx.GlobalVars().CaptureInfo.ID) + ctx = cdcContext.WithChangefeedVars(ctx, &cdcContext.ChangefeedVars{ + ID: state.ID, + Info: state.Info, + }) _, err := p.tick(ctx, state) - p.firstTick = false if err == nil { return state, nil } cause := errors.Cause(err) if cause == context.Canceled || cerror.ErrAdminStopProcessor.Equal(cause) || cerror.ErrReactorFinished.Equal(cause) { + log.Info("processor exited", cdcContext.ZapFieldCapture(ctx), cdcContext.ZapFieldChangefeed(ctx)) return state, cerror.ErrReactorFinished.GenWithStackByArgs() } p.metricProcessorErrorCounter.Inc() // record error information in etcd var code string - if terror, ok := err.(*errors.Error); ok { - code = string(terror.RFCCode()) + if rfcCode, ok := cerror.RFCCode(err); ok { + code = string(rfcCode) } else { code = string(cerror.ErrProcessorUnknown.RFCCode()) } - state.PatchTaskPosition(func(position *model.TaskPosition) (*model.TaskPosition, error) { + state.PatchTaskPosition(p.captureInfo.ID, func(position *model.TaskPosition) (*model.TaskPosition, bool, error) { if position == nil { position = &model.TaskPosition{} } position.Error = &model.RunningError{ - Addr: p.captureInfo.AdvertiseAddr, + Addr: ctx.GlobalVars().CaptureInfo.AdvertiseAddr, Code: code, Message: err.Error(), } - return position, nil + return position, true, nil }) log.Error("run processor failed", - zap.String("changefeed", p.changefeed.ID), - zap.String("capture-id", p.captureInfo.ID), - util.ZapFieldCapture(ctx), + cdcContext.ZapFieldChangefeed(ctx), + cdcContext.ZapFieldCapture(ctx), zap.Error(err)) return state, cerror.ErrReactorFinished.GenWithStackByArgs() } -func (p *processor) tick(ctx context.Context, state *changefeedState) (nextState orchestrator.ReactorState, err error) { +func (p *processor) tick(ctx cdcContext.Context, state *model.ChangefeedReactorState) (nextState orchestrator.ReactorState, err error) { p.changefeed = state + if !p.checkChangefeedNormal() { + return nil, cerror.ErrAdminStopProcessor.GenWithStackByArgs() + } + if skip := p.checkPosition(); skip { + return p.changefeed, nil + } if err := p.handleErrorCh(ctx); err != nil { return nil, errors.Trace(err) } - if p.changefeed.TaskStatus.AdminJobType.IsStopState() { - return nil, cerror.ErrAdminStopProcessor.GenWithStackByArgs() - } if err := p.lazyInit(ctx); err != nil { return nil, errors.Trace(err) } - if skip := p.checkPosition(); skip { - return p.changefeed, nil - } if err := p.handleTableOperation(ctx); err != nil { return nil, errors.Trace(err) } @@ -185,36 +194,46 @@ func (p *processor) tick(ctx context.Context, state *changefeedState) (nextState return p.changefeed, nil } +// checkChangefeedNormal checks if the changefeed is runnable. +func (p *processor) checkChangefeedNormal() bool { + // check the state in this tick, make sure that the admin job type of the changefeed is not stopped + if p.changefeed.Info.AdminJobType.IsStopState() || p.changefeed.Status.AdminJobType.IsStopState() { + return false + } + // add a patch to check the changefeed is runnable when applying the patches in the etcd worker. + p.changefeed.CheckChangefeedNormal() + return true +} + // checkPosition create a new task position, and put it into the etcd state. // task position maybe be not exist only when the processor is running first time. -func (p *processor) checkPosition() bool { - if p.changefeed.TaskPosition != nil { +func (p *processor) checkPosition() (skipThisTick bool) { + if p.changefeed.TaskPositions[p.captureInfo.ID] != nil { return false } - if !p.firstTick { + if p.initialized { log.Warn("position is nil, maybe position info is removed unexpected", zap.Any("state", p.changefeed)) } checkpointTs := p.changefeed.Info.GetCheckpointTs(p.changefeed.Status) - p.changefeed.PatchTaskPosition(func(position *model.TaskPosition) (*model.TaskPosition, error) { + p.changefeed.PatchTaskPosition(p.captureInfo.ID, func(position *model.TaskPosition) (*model.TaskPosition, bool, error) { if position == nil { return &model.TaskPosition{ CheckPointTs: checkpointTs, ResolvedTs: checkpointTs, - }, nil + }, true, nil } - return position, nil + return position, false, nil }) return true } // lazyInitImpl create Filter, SchemaStorage, Mounter instances at the first tick. -func (p *processor) lazyInitImpl(ctx context.Context) error { - if !p.firstTick { +func (p *processor) lazyInitImpl(ctx cdcContext.Context) error { + if p.initialized { return nil } - ctx, cancel := context.WithCancel(ctx) + ctx, cancel := cdcContext.WithCancel(ctx) p.cancel = cancel - ctx = util.PutChangefeedIDInCtx(ctx, p.changefeed.ID) errCh := make(chan error, 16) p.wg.Add(1) @@ -223,7 +242,7 @@ func (p *processor) lazyInitImpl(ctx context.Context) error { // there are some other objects need errCh, such as sink and sink manager // but we can't ensure that all the producer of errCh are non-blocking // It's very tricky that create a goroutine to receive the local errCh - // TODO(leoppro): we should using `pkg/context.Context` instead of standard context and handle error by `pkg/context.Context.Throw` + // TODO(leoppro): we should using `pkg/cdcContext.Context` instead of standard cdcContext and handle error by `pkg/cdcContext.Context.Throw` for { select { case <-ctx.Done(): @@ -249,11 +268,13 @@ func (p *processor) lazyInitImpl(ctx context.Context) error { return errors.Trace(err) } + stdCtx := util.PutChangefeedIDInCtx(ctx, p.changefeed.ID) + p.mounter = entry.NewMounter(p.schemaStorage, p.changefeed.Info.Config.Mounter.WorkerNum, p.changefeed.Info.Config.EnableOldValue) p.wg.Add(1) go func() { defer p.wg.Done() - p.sendError(p.mounter.Run(ctx)) + p.sendError(p.mounter.Run(stdCtx)) }() opts := make(map[string]string, len(p.changefeed.Info.Opts)+2) @@ -270,30 +291,20 @@ func (p *processor) lazyInitImpl(ctx context.Context) error { opts[mark.OptCyclicConfig] = string(cyclicCfg) } opts[sink.OptChangefeedID] = p.changefeed.ID - opts[sink.OptCaptureAddr] = p.captureInfo.AdvertiseAddr - s, err := sink.NewSink(ctx, p.changefeed.ID, p.changefeed.Info.SinkURI, p.filter, p.changefeed.Info.Config, opts, errCh) + opts[sink.OptCaptureAddr] = ctx.GlobalVars().CaptureInfo.AdvertiseAddr + s, err := sink.NewSink(stdCtx, p.changefeed.ID, p.changefeed.Info.SinkURI, p.filter, p.changefeed.Info.Config, opts, errCh) if err != nil { return errors.Trace(err) } checkpointTs := p.changefeed.Info.GetCheckpointTs(p.changefeed.Status) - p.sinkManager = sink.NewManager(ctx, s, errCh, checkpointTs) - - // Clean up possible residual error states - p.changefeed.PatchTaskPosition(func(position *model.TaskPosition) (*model.TaskPosition, error) { - if position != nil && position.Error != nil { - position.Error = nil - } - return position, nil - }) - - log.Info("run processor", - zap.String("capture-id", p.captureInfo.ID), util.ZapFieldCapture(ctx), - zap.String("changefeed-id", p.changefeed.ID)) + p.sinkManager = sink.NewManager(stdCtx, s, errCh, checkpointTs) + p.initialized = true + log.Info("run processor", cdcContext.ZapFieldCapture(ctx), cdcContext.ZapFieldChangefeed(ctx)) return nil } // handleErrorCh listen the error channel and throw the error if it is not expected. -func (p *processor) handleErrorCh(ctx context.Context) error { +func (p *processor) handleErrorCh(ctx cdcContext.Context) error { var err error select { case err = <-p.errCh: @@ -303,53 +314,36 @@ func (p *processor) handleErrorCh(ctx context.Context) error { cause := errors.Cause(err) if cause != nil && cause != context.Canceled && cerror.ErrAdminStopProcessor.NotEqual(cause) { log.Error("error on running processor", - util.ZapFieldCapture(ctx), - zap.String("changefeed", p.changefeed.ID), - zap.String("captureID", p.captureInfo.ID), - zap.String("captureAddr", p.captureInfo.AdvertiseAddr), + cdcContext.ZapFieldCapture(ctx), + cdcContext.ZapFieldChangefeed(ctx), zap.Error(err)) return err } - log.Info("processor exited", - util.ZapFieldCapture(ctx), - zap.String("changefeed", p.changefeed.ID), - zap.String("captureID", p.captureInfo.ID), - zap.String("captureAddr", p.captureInfo.AdvertiseAddr)) + log.Info("processor exited", cdcContext.ZapFieldCapture(ctx), cdcContext.ZapFieldChangefeed(ctx)) return cerror.ErrReactorFinished } // handleTableOperation handles the operation of `TaskStatus`(add table operation and remove table operation) -func (p *processor) handleTableOperation(ctx context.Context) error { +func (p *processor) handleTableOperation(ctx cdcContext.Context) error { patchOperation := func(tableID model.TableID, fn func(operation *model.TableOperation) error) { - p.changefeed.PatchTaskStatus(func(status *model.TaskStatus) (*model.TaskStatus, error) { + p.changefeed.PatchTaskStatus(p.captureInfo.ID, func(status *model.TaskStatus) (*model.TaskStatus, bool, error) { if status == nil || status.Operation == nil { log.Error("Operation not found, may be remove by other patch", zap.Int64("tableID", tableID), zap.Any("status", status)) - return nil, cerror.ErrTaskStatusNotExists.GenWithStackByArgs() + return nil, false, cerror.ErrTaskStatusNotExists.GenWithStackByArgs() } opt := status.Operation[tableID] if opt == nil { log.Error("Operation not found, may be remove by other patch", zap.Int64("tableID", tableID), zap.Any("status", status)) - return nil, cerror.ErrTaskStatusNotExists.GenWithStackByArgs() + return nil, false, cerror.ErrTaskStatusNotExists.GenWithStackByArgs() } if err := fn(opt); err != nil { - return nil, errors.Trace(err) - } - return status, nil - }) - } - // TODO: 👇👇 remove this six lines after the new owner is implemented, applied operation should be removed by owner - if !p.changefeed.TaskStatus.SomeOperationsUnapplied() && len(p.changefeed.TaskStatus.Operation) != 0 { - p.changefeed.PatchTaskStatus(func(status *model.TaskStatus) (*model.TaskStatus, error) { - if status == nil { - // for safety, status should never be nil - return nil, nil + return nil, false, errors.Trace(err) } - status.Operation = nil - return status, nil + return status, true, nil }) } - // 👆👆 remove this six lines - for tableID, opt := range p.changefeed.TaskStatus.Operation { + taskStatus := p.changefeed.TaskStatuses[p.captureInfo.ID] + for tableID, opt := range taskStatus.Operation { if opt.TableApplied() { continue } @@ -358,7 +352,7 @@ func (p *processor) handleTableOperation(ctx context.Context) error { table, exist := p.tables[tableID] if !exist { log.Warn("table which will be deleted is not found", - util.ZapFieldChangefeed(ctx), zap.Int64("tableID", tableID)) + cdcContext.ZapFieldChangefeed(ctx), zap.Int64("tableID", tableID)) patchOperation(tableID, func(operation *model.TableOperation) error { operation.Status = model.OperFinished operation.Done = true @@ -387,11 +381,11 @@ func (p *processor) handleTableOperation(ctx context.Context) error { operation.Done = true return nil }) - // TODO: check if the goroutines created by table pipeline is actually exited. (call tablepipeline.Wait()) table.Cancel() + table.Wait() delete(p.tables, tableID) log.Debug("Operation done signal received", - util.ZapFieldChangefeed(ctx), + cdcContext.ZapFieldChangefeed(ctx), zap.Int64("tableID", tableID), zap.Reflect("operation", opt)) default: @@ -400,13 +394,10 @@ func (p *processor) handleTableOperation(ctx context.Context) error { } else { switch opt.Status { case model.OperDispatched: - replicaInfo, exist := p.changefeed.TaskStatus.Tables[tableID] + replicaInfo, exist := taskStatus.Tables[tableID] if !exist { return cerror.ErrProcessorTableNotFound.GenWithStack("replicaInfo of table(%d)", tableID) } - if p.changefeed.Info.Config.Cyclic.IsEnabled() && replicaInfo.MarkTableID == 0 { - return cerror.ErrProcessorTableNotFound.GenWithStack("normal table(%d) and mark table not match ", tableID) - } if replicaInfo.StartTs != opt.BoundaryTs { log.Warn("the startTs and BoundaryTs of add table operation should be always equaled", zap.Any("replicaInfo", replicaInfo)) } @@ -421,10 +412,15 @@ func (p *processor) handleTableOperation(ctx context.Context) error { case model.OperProcessed: table, exist := p.tables[tableID] if !exist { - log.Panic("table which was added is not found", - util.ZapFieldChangefeed(ctx), zap.Int64("tableID", tableID)) + log.Warn("table which was added is not found", + cdcContext.ZapFieldChangefeed(ctx), zap.Int64("tableID", tableID)) + patchOperation(tableID, func(operation *model.TableOperation) error { + operation.Status = model.OperDispatched + return nil + }) + continue } - localResolvedTs := p.changefeed.TaskPosition.ResolvedTs + localResolvedTs := p.changefeed.TaskPositions[p.captureInfo.ID].ResolvedTs globalResolvedTs := p.changefeed.Status.ResolvedTs if table.ResolvedTs() >= localResolvedTs && localResolvedTs >= globalResolvedTs { patchOperation(tableID, func(operation *model.TableOperation) error { @@ -433,7 +429,7 @@ func (p *processor) handleTableOperation(ctx context.Context) error { return nil }) log.Debug("Operation done signal received", - util.ZapFieldChangefeed(ctx), + cdcContext.ZapFieldChangefeed(ctx), zap.Int64("tableID", tableID), zap.Reflect("operation", opt)) } @@ -445,11 +441,17 @@ func (p *processor) handleTableOperation(ctx context.Context) error { return nil } -func (p *processor) createAndDriveSchemaStorage(ctx context.Context) (entry.SchemaStorage, error) { - kvStorage := util.KVStorageFromCtx(ctx) +func (p *processor) createAndDriveSchemaStorage(ctx cdcContext.Context) (entry.SchemaStorage, error) { + kvStorage := ctx.GlobalVars().KVStorage ddlspans := []regionspan.Span{regionspan.GetDDLSpan(), regionspan.GetAddIndexDDLSpan()} checkpointTs := p.changefeed.Info.GetCheckpointTs(p.changefeed.Status) - ddlPuller := puller.NewPuller(ctx, p.pdCli, p.credential, kvStorage, checkpointTs, ddlspans, p.limitter, false) + conf := config.GetGlobalServerConfig() + ddlPuller := puller.NewPuller( + ctx, + ctx.GlobalVars().PDClient, + conf.Security, + ctx.GlobalVars().KVStorage, + checkpointTs, ddlspans, p.limitter, false) meta, err := kv.GetSnapshotMeta(kvStorage, checkpointTs) if err != nil { return nil, errors.Trace(err) @@ -471,7 +473,6 @@ func (p *processor) createAndDriveSchemaStorage(ctx context.Context) (entry.Sche for { select { case <-ctx.Done(): - p.sendError(ctx.Err()) return case ddlRawKV = <-ddlRawKVCh: } @@ -512,22 +513,25 @@ func (p *processor) sendError(err error) { // checkTablesNum if the number of table pipelines is equal to the number of TaskStatus in etcd state. // if the table number is not right, create or remove the odd tables. -func (p *processor) checkTablesNum(ctx context.Context) error { - if len(p.tables) == len(p.changefeed.TaskStatus.Tables) { +func (p *processor) checkTablesNum(ctx cdcContext.Context) error { + taskStatus := p.changefeed.TaskStatuses[p.captureInfo.ID] + if len(p.tables) == len(taskStatus.Tables) { return nil } // check if a table should be listen but not // this only could be happened in the first tick. - for tableID, replicaInfo := range p.changefeed.TaskStatus.Tables { + for tableID, replicaInfo := range taskStatus.Tables { if _, exist := p.tables[tableID]; exist { continue } - opt := p.changefeed.TaskStatus.Operation + opt := taskStatus.Operation + // TODO(leoppro): check if the operation is a undone add operation if opt != nil && opt[tableID] != nil { continue } - if !p.firstTick { - log.Warn("the table should be listen but not, already listen the table again, please report a bug", zap.Int64("tableID", tableID), zap.Any("replicaInfo", replicaInfo)) + log.Info("start to listen to the table immediately", zap.Int64("tableID", tableID), zap.Any("replicaInfo", replicaInfo)) + if replicaInfo.StartTs < p.changefeed.Status.CheckpointTs { + replicaInfo.StartTs = p.changefeed.Status.CheckpointTs } err := p.addTable(ctx, tableID, replicaInfo) if err != nil { @@ -537,24 +541,28 @@ func (p *processor) checkTablesNum(ctx context.Context) error { // check if a table should be removed but still exist // this shouldn't be happened in any time. for tableID, tablePipeline := range p.tables { - if _, exist := p.changefeed.TaskStatus.Tables[tableID]; exist { + if _, exist := taskStatus.Tables[tableID]; exist { continue } - opt := p.changefeed.TaskStatus.Operation + opt := taskStatus.Operation if opt != nil && opt[tableID] != nil && opt[tableID].Delete { // table will be removed by normal logic continue } tablePipeline.Cancel() + tablePipeline.Wait() delete(p.tables, tableID) - log.Warn("the table was forcibly deleted, this should not happen, please report a bug", zap.Int64("tableID", tableID), zap.Any("taskStatus", p.changefeed.TaskStatus)) + log.Warn("the table was forcibly deleted", zap.Int64("tableID", tableID), zap.Any("taskStatus", taskStatus)) } return nil } // handlePosition calculates the local resolved ts and local checkpoint ts func (p *processor) handlePosition() error { - minResolvedTs := p.schemaStorage.ResolvedTs() + minResolvedTs := uint64(math.MaxUint64) + if p.schemaStorage != nil { + minResolvedTs = p.schemaStorage.ResolvedTs() + } for _, table := range p.tables { ts := table.ResolvedTs() if ts < minResolvedTs { @@ -583,19 +591,19 @@ func (p *processor) handlePosition() error { p.metricCheckpointTsGauge.Set(float64(checkpointPhyTs)) // minResolvedTs and minCheckpointTs may less than global resolved ts and global checkpoint ts when a new table added, the startTs of the new table is less than global checkpoint ts. - if minResolvedTs != p.changefeed.TaskPosition.ResolvedTs || - minCheckpointTs != p.changefeed.TaskPosition.CheckPointTs { - p.changefeed.PatchTaskPosition(func(position *model.TaskPosition) (*model.TaskPosition, error) { + if minResolvedTs != p.changefeed.TaskPositions[p.captureInfo.ID].ResolvedTs || + minCheckpointTs != p.changefeed.TaskPositions[p.captureInfo.ID].CheckPointTs { + p.changefeed.PatchTaskPosition(p.captureInfo.ID, func(position *model.TaskPosition) (*model.TaskPosition, bool, error) { failpoint.Inject("ProcessorUpdatePositionDelaying", nil) if position == nil { // when the captureInfo is deleted, the old owner will delete task status, task position, task workload in non-atomic // so processor may see a intermediate state, for example the task status is exist but task position is deleted. log.Warn("task position is not exist, skip to update position", zap.String("changefeed", p.changefeed.ID)) - return nil, nil + return nil, false, nil } position.CheckPointTs = minCheckpointTs position.ResolvedTs = minResolvedTs - return position, nil + return position, true, nil }) } return nil @@ -603,12 +611,24 @@ func (p *processor) handlePosition() error { // handleWorkload calculates the workload of all tables func (p *processor) handleWorkload() error { - p.changefeed.PatchTaskWorkload(func(_ model.TaskWorkload) (model.TaskWorkload, error) { - workload := make(model.TaskWorkload, len(p.tables)) + p.changefeed.PatchTaskWorkload(p.captureInfo.ID, func(workloads model.TaskWorkload) (model.TaskWorkload, bool, error) { + changed := false + if workloads == nil { + workloads = make(model.TaskWorkload) + } + for tableID := range workloads { + if _, exist := p.tables[tableID]; !exist { + delete(workloads, tableID) + changed = true + } + } for tableID, table := range p.tables { - workload[tableID] = table.Workload() + if workloads[tableID] != table.Workload() { + workloads[tableID] = table.Workload() + changed = true + } } - return workload, nil + return workloads, changed, nil }) return nil } @@ -623,14 +643,15 @@ func (p *processor) pushResolvedTs2Table() error { } // addTable creates a new table pipeline and adds it to the `p.tables` -func (p *processor) addTable(ctx context.Context, tableID model.TableID, replicaInfo *model.TableReplicaInfo) error { +func (p *processor) addTable(ctx cdcContext.Context, tableID model.TableID, replicaInfo *model.TableReplicaInfo) error { if table, ok := p.tables[tableID]; ok { if table.Status() == tablepipeline.TableStatusStopped { - log.Warn("The same table exists but is stopped. Cancel it and continue.", util.ZapFieldChangefeed(ctx), zap.Int64("ID", tableID)) + log.Warn("The same table exists but is stopped. Cancel it and continue.", cdcContext.ZapFieldChangefeed(ctx), zap.Int64("ID", tableID)) table.Cancel() + table.Wait() delete(p.tables, tableID) } else { - log.Warn("Ignore existing table", util.ZapFieldChangefeed(ctx), zap.Int64("ID", tableID)) + log.Warn("Ignore existing table", cdcContext.ZapFieldChangefeed(ctx), zap.Int64("ID", tableID)) return nil } } @@ -639,7 +660,7 @@ func (p *processor) addTable(ctx context.Context, tableID model.TableID, replica if replicaInfo.StartTs < globalCheckpointTs { log.Warn("addTable: startTs < checkpoint", - util.ZapFieldChangefeed(ctx), + cdcContext.ZapFieldChangefeed(ctx), zap.Int64("tableID", tableID), zap.Uint64("checkpoint", globalCheckpointTs), zap.Uint64("startTs", replicaInfo.StartTs)) @@ -652,14 +673,8 @@ func (p *processor) addTable(ctx context.Context, tableID model.TableID, replica return nil } -func (p *processor) createTablePipelineImpl(ctx context.Context, tableID model.TableID, replicaInfo *model.TableReplicaInfo) (tablepipeline.TablePipeline, error) { - cdcCtx := cdccontext.NewContext(ctx, &cdccontext.Vars{ - CaptureAddr: p.captureInfo.AdvertiseAddr, - PDClient: p.pdCli, - SchemaStorage: p.schemaStorage, - Config: p.changefeed.Info.Config, - }) - cdcCtx = cdccontext.WithErrorHandler(cdcCtx, func(err error) error { +func (p *processor) createTablePipelineImpl(ctx cdcContext.Context, tableID model.TableID, replicaInfo *model.TableReplicaInfo) (tablepipeline.TablePipeline, error) { + ctx = cdcContext.WithErrorHandler(ctx, func(err error) error { if cerror.ErrTableProcessorStoppedSafely.Equal(err) || errors.Cause(errors.Cause(err)) == context.Canceled { return nil @@ -667,33 +682,53 @@ func (p *processor) createTablePipelineImpl(ctx context.Context, tableID model.T p.sendError(err) return nil }) - - kvStorage := util.KVStorageFromCtx(ctx) - var tableName string - err := retry.Run(time.Millisecond*5, 3, func() error { + var tableName *model.TableName + retry.Run(time.Millisecond*5, 3, func() error { //nolint:errcheck if name, ok := p.schemaStorage.GetLastSnapshot().GetTableNameByID(tableID); ok { - tableName = name.QuoteString() + tableName = &name return nil } return errors.Errorf("failed to get table name, fallback to use table id: %d", tableID) }) - if err != nil { - log.Warn("get table name for metric", zap.Error(err)) - tableName = strconv.Itoa(int(tableID)) + if p.changefeed.Info.Config.Cyclic.IsEnabled() { + // Retry to find mark table ID + var markTableID model.TableID + err := retry.Run(50*time.Millisecond, 20, func() error { + if tableName == nil { + name, exist := p.schemaStorage.GetLastSnapshot().GetTableNameByID(tableID) + if !exist { + return cerror.ErrProcessorTableNotFound.GenWithStack("normal table(%s)", tableID) + } + tableName = &name + } + markTableSchameName, markTableTableName := mark.GetMarkTableName(tableName.Schema, tableName.Table) + tableInfo, exist := p.schemaStorage.GetLastSnapshot().GetTableByName(markTableSchameName, markTableTableName) + if !exist { + return cerror.ErrProcessorTableNotFound.GenWithStack("normal table(%s) and mark table not match", tableName.String()) + } + markTableID = tableInfo.ID + return nil + }) + if err != nil { + return nil, errors.Trace(err) + } + replicaInfo.MarkTableID = markTableID + } + var tableNameStr string + if tableName == nil { + log.Warn("failed to get table name for metric") + tableNameStr = strconv.Itoa(int(tableID)) + } else { + tableNameStr = tableName.QuoteString() } - sink := p.sinkManager.CreateTableSink(tableID, replicaInfo.StartTs) + sink := p.sinkManager.CreateTableSink(tableID, replicaInfo.StartTs) table := tablepipeline.NewTablePipeline( - cdcCtx, - p.changefeed.ID, - p.credential, - kvStorage, + ctx, p.limitter, p.mounter, - p.changefeed.Info.Engine, - p.changefeed.Info.SortDir, tableID, - tableName, + tableNameStr, replicaInfo, sink, p.changefeed.Info.GetTargetTs(), @@ -705,13 +740,13 @@ func (p *processor) createTablePipelineImpl(ctx context.Context, tableID model.T p.wg.Done() p.metricSyncTableNumGauge.Dec() log.Debug("Table pipeline exited", zap.Int64("tableID", tableID), - util.ZapFieldChangefeed(ctx), + cdcContext.ZapFieldChangefeed(ctx), zap.String("name", table.Name()), zap.Any("replicaInfo", replicaInfo)) }() log.Debug("Add table pipeline", zap.Int64("tableID", tableID), - util.ZapFieldChangefeed(ctx), + cdcContext.ZapFieldChangefeed(ctx), zap.String("name", table.Name()), zap.Any("replicaInfo", replicaInfo), zap.Uint64("globalResolvedTs", p.changefeed.Status.ResolvedTs)) @@ -721,6 +756,10 @@ func (p *processor) createTablePipelineImpl(ctx context.Context, tableID model.T // doGCSchemaStorage trigger the schema storage GC func (p *processor) doGCSchemaStorage() error { + if p.schemaStorage == nil { + // schemaStorage is nil only in test + return nil + } // Delay GC to accommodate pullers starting from a startTs that's too small // TODO fix startTs problem and remove GC delay, or use other mechanism that prevents the problem deterministically gcTime := oracle.GetTimeFromTS(p.changefeed.Status.CheckpointTs).Add(-schemaStorageGCLag) @@ -730,36 +769,20 @@ func (p *processor) doGCSchemaStorage() error { } func (p *processor) Close() error { - log.Info("stop processor", zap.String("capture", p.captureInfo.AdvertiseAddr), zap.String("changefeed", p.changefeed.ID)) for _, tbl := range p.tables { tbl.Cancel() + tbl.Wait() } p.cancel() p.wg.Wait() - // mark tables share the same context with its original table, don't need to cancel + // mark tables share the same cdcContext with its original table, don't need to cancel failpoint.Inject("processorStopDelay", nil) - p.changefeed.PatchTaskPosition(func(position *model.TaskPosition) (*model.TaskPosition, error) { - if position == nil { - return nil, nil - } - if position.Error != nil { - return position, nil - } - return nil, nil - }) - p.changefeed.PatchTaskStatus(func(_ *model.TaskStatus) (*model.TaskStatus, error) { - return nil, nil - }) - p.changefeed.PatchTaskWorkload(func(_ model.TaskWorkload) (model.TaskWorkload, error) { - return nil, nil - }) - - resolvedTsGauge.DeleteLabelValues(p.changefeed.ID, p.captureInfo.AdvertiseAddr) - resolvedTsLagGauge.DeleteLabelValues(p.changefeed.ID, p.captureInfo.AdvertiseAddr) - checkpointTsGauge.DeleteLabelValues(p.changefeed.ID, p.captureInfo.AdvertiseAddr) - checkpointTsLagGauge.DeleteLabelValues(p.changefeed.ID, p.captureInfo.AdvertiseAddr) - syncTableNumGauge.DeleteLabelValues(p.changefeed.ID, p.captureInfo.AdvertiseAddr) - processorErrorCounter.DeleteLabelValues(p.changefeed.ID, p.captureInfo.AdvertiseAddr) + resolvedTsGauge.DeleteLabelValues(p.changefeedID, p.captureInfo.AdvertiseAddr) + resolvedTsLagGauge.DeleteLabelValues(p.changefeedID, p.captureInfo.AdvertiseAddr) + checkpointTsGauge.DeleteLabelValues(p.changefeedID, p.captureInfo.AdvertiseAddr) + checkpointTsLagGauge.DeleteLabelValues(p.changefeedID, p.captureInfo.AdvertiseAddr) + syncTableNumGauge.DeleteLabelValues(p.changefeedID, p.captureInfo.AdvertiseAddr) + processorErrorCounter.DeleteLabelValues(p.changefeedID, p.captureInfo.AdvertiseAddr) if p.sinkManager != nil { return p.sinkManager.Close() } diff --git a/cdc/processor/processor_test.go b/cdc/processor/processor_test.go index bff4f3eb0ec..33d9003a598 100644 --- a/cdc/processor/processor_test.go +++ b/cdc/processor/processor_test.go @@ -15,104 +15,43 @@ package processor import ( "context" - "encoding/json" "fmt" - "math" - "time" + "testing" "github.com/pingcap/check" "github.com/pingcap/errors" "github.com/pingcap/log" - timodel "github.com/pingcap/parser/model" - "github.com/pingcap/ticdc/cdc/entry" "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/cdc/processor/pipeline" tablepipeline "github.com/pingcap/ticdc/cdc/processor/pipeline" - "github.com/pingcap/ticdc/pkg/config" + cdcContext "github.com/pingcap/ticdc/pkg/context" cerror "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/etcd" + "github.com/pingcap/ticdc/pkg/orchestrator" "github.com/pingcap/ticdc/pkg/util/testleak" ) +func Test(t *testing.T) { check.TestingT(t) } + type processorSuite struct{} var _ = check.Suite(&processorSuite{}) -func newProcessor4Test() *processor { - changefeedID := "test-changefeed" - p := newProcessor(nil, "test-changefeed", nil, &model.CaptureInfo{ - ID: "test-captureID", - AdvertiseAddr: "127.0.0.1:0000", - }) - p.lazyInit = func(ctx context.Context) error { - if !p.firstTick { - return nil - } - p.schemaStorage = &mockSchemaStorage{} - return nil - } - p.createTablePipeline = func(ctx context.Context, tableID model.TableID, replicaInfo *model.TableReplicaInfo) (tablepipeline.TablePipeline, error) { +func initProcessor4Test(ctx cdcContext.Context, c *check.C) (*processor, *orchestrator.ReactorStateTester) { + p := newProcessor4Test(ctx, func(ctx cdcContext.Context, tableID model.TableID, replicaInfo *model.TableReplicaInfo) (tablepipeline.TablePipeline, error) { return &mockTablePipeline{ tableID: tableID, name: fmt.Sprintf("`test`.`table%d`", tableID), - status: pipeline.TableStatusRunning, + status: tablepipeline.TableStatusRunning, resolvedTs: replicaInfo.StartTs, checkpointTs: replicaInfo.StartTs, }, nil - } - p.changefeed = newChangeFeedState(changefeedID, p.captureInfo.ID) - p.changefeed.Info = &model.ChangeFeedInfo{ - SinkURI: "blackhole://", - CreateTime: time.Now(), - StartTs: 0, - TargetTs: math.MaxUint64, - Config: config.GetDefaultReplicaConfig(), - } - p.changefeed.Status = &model.ChangeFeedStatus{} - p.changefeed.TaskStatus = &model.TaskStatus{ - Tables: map[int64]*model.TableReplicaInfo{}, - } - p.cancel = func() {} - return p -} - -func applyPatches(c *check.C, state *changefeedState) { - for _, patch := range state.pendingPatches { - key := &etcd.CDCKey{} - c.Assert(key.Parse(patch.Key.String()), check.IsNil) - var value []byte - var err error - switch key.Tp { - case etcd.CDCKeyTypeTaskPosition: - if state.TaskPosition == nil { - value = nil - break - } - value, err = json.Marshal(state.TaskPosition) - c.Assert(err, check.IsNil) - case etcd.CDCKeyTypeTaskStatus: - if state.TaskStatus == nil { - value = nil - break - } - value, err = json.Marshal(state.TaskStatus) - c.Assert(err, check.IsNil) - case etcd.CDCKeyTypeTaskWorkload: - if state.Workload == nil { - value = nil - break - } - value, err = json.Marshal(state.Workload) - c.Assert(err, check.IsNil) - default: - c.Fatal("unexpected key type") - } - newValue, err := patch.Fun(value) - c.Assert(err, check.IsNil) - err = state.UpdateCDCKey(key, newValue) - c.Assert(err, check.IsNil) - } - state.pendingPatches = state.pendingPatches[:0] + }) + p.changefeed = model.NewChangefeedReactorState(ctx.ChangefeedVars().ID) + return p, orchestrator.NewReactorStateTester(c, p.changefeed, map[string]string{ + "/tidb/cdc/capture/" + ctx.GlobalVars().CaptureInfo.ID: `{"id":"` + ctx.GlobalVars().CaptureInfo.ID + `","address":"127.0.0.1:8300"}`, + "/tidb/cdc/changefeed/info/" + ctx.ChangefeedVars().ID: `{"sink-uri":"blackhole://","opts":{},"create-time":"2020-02-02T00:00:00.000000+00:00","start-ts":0,"target-ts":0,"admin-job-type":0,"sort-engine":"memory","sort-dir":".","config":{"case-sensitive":true,"enable-old-value":false,"force-replicate":false,"check-gc-safe-point":true,"filter":{"rules":["*.*"],"ignore-txn-start-ts":null,"ddl-allow-list":null},"mounter":{"worker-num":16},"sink":{"dispatchers":null,"protocol":"default"},"cyclic-replication":{"enable":false,"replica-id":0,"filter-replica-ids":null,"id-buckets":0,"sync-ddl":false},"scheduler":{"type":"table-number","polling-time":-1}},"state":"normal","history":null,"error":null,"sync-point-enabled":false,"sync-point-interval":600000000000}`, + "/tidb/cdc/job/" + ctx.ChangefeedVars().ID: `{"resolved-ts":0,"checkpoint-ts":0,"admin-job-type":0}`, + "/tidb/cdc/task/status/" + ctx.GlobalVars().CaptureInfo.ID + "/" + ctx.ChangefeedVars().ID: `{"tables":{},"operation":null,"admin-job-type":0}`, + }) } type mockTablePipeline struct { @@ -122,7 +61,7 @@ type mockTablePipeline struct { checkpointTs model.Ts barrierTs model.Ts stopTs model.Ts - status pipeline.TableStatus + status tablepipeline.TableStatus canceled bool } @@ -154,7 +93,7 @@ func (m *mockTablePipeline) Workload() model.WorkloadInfo { return model.WorkloadInfo{Workload: 1} } -func (m *mockTablePipeline) Status() pipeline.TableStatus { +func (m *mockTablePipeline) Status() tablepipeline.TableStatus { return m.status } @@ -166,47 +105,18 @@ func (m *mockTablePipeline) Cancel() { } func (m *mockTablePipeline) Wait() { - panic("not implemented") // TODO: Implement -} - -type mockSchemaStorage struct { - resolvedTs model.Ts - lastGcTs model.Ts -} - -func (m *mockSchemaStorage) GetSnapshot(ctx context.Context, ts uint64) (*entry.SingleSchemaSnapshot, error) { - panic("not implemented") // TODO: Implement -} - -func (m *mockSchemaStorage) GetLastSnapshot() *entry.SingleSchemaSnapshot { - panic("not implemented") // TODO: Implement -} - -func (m *mockSchemaStorage) HandleDDLJob(job *timodel.Job) error { - panic("not implemented") // TODO: Implement -} - -func (m *mockSchemaStorage) AdvanceResolvedTs(ts uint64) { - m.resolvedTs = ts -} - -func (m *mockSchemaStorage) ResolvedTs() uint64 { - return m.resolvedTs -} - -func (m *mockSchemaStorage) DoGC(ts uint64) { - m.lastGcTs = ts + // do nothing } func (s *processorSuite) TestCheckTablesNum(c *check.C) { defer testleak.AfterTest(c)() - p := newProcessor4Test() - ctx := context.Background() + ctx := cdcContext.NewBackendContext4Test(true) + p, tester := initProcessor4Test(ctx, c) var err error _, err = p.Tick(ctx, p.changefeed) c.Assert(err, check.IsNil) - applyPatches(c, p.changefeed) - c.Assert(p.changefeed.TaskPosition, check.DeepEquals, + tester.MustApplyPatches() + c.Assert(p.changefeed.TaskPositions[p.captureInfo.ID], check.DeepEquals, &model.TaskPosition{ CheckPointTs: 0, ResolvedTs: 0, @@ -214,13 +124,13 @@ func (s *processorSuite) TestCheckTablesNum(c *check.C) { Error: nil, }) - p = newProcessor4Test() + p, tester = initProcessor4Test(ctx, c) p.changefeed.Info.StartTs = 66 p.changefeed.Status.CheckpointTs = 88 _, err = p.Tick(ctx, p.changefeed) c.Assert(err, check.IsNil) - applyPatches(c, p.changefeed) - c.Assert(p.changefeed.TaskPosition, check.DeepEquals, + tester.MustApplyPatches() + c.Assert(p.changefeed.TaskPositions[p.captureInfo.ID], check.DeepEquals, &model.TaskPosition{ CheckPointTs: 88, ResolvedTs: 88, @@ -231,30 +141,40 @@ func (s *processorSuite) TestCheckTablesNum(c *check.C) { func (s *processorSuite) TestHandleTableOperation4SingleTable(c *check.C) { defer testleak.AfterTest(c)() - ctx := context.Background() - p := newProcessor4Test() + ctx := cdcContext.NewBackendContext4Test(true) + p, tester := initProcessor4Test(ctx, c) var err error // init tick _, err = p.Tick(ctx, p.changefeed) c.Assert(err, check.IsNil) - applyPatches(c, p.changefeed) - p.changefeed.Status.CheckpointTs = 90 - p.changefeed.Status.ResolvedTs = 90 - p.changefeed.TaskPosition.ResolvedTs = 100 - p.schemaStorage.AdvanceResolvedTs(200) + tester.MustApplyPatches() + p.changefeed.PatchStatus(func(status *model.ChangeFeedStatus) (*model.ChangeFeedStatus, bool, error) { + status.CheckpointTs = 90 + status.ResolvedTs = 100 + return status, true, nil + }) + p.changefeed.PatchTaskPosition(p.captureInfo.ID, func(position *model.TaskPosition) (*model.TaskPosition, bool, error) { + position.ResolvedTs = 100 + return position, true, nil + }) + tester.MustApplyPatches() // no operation _, err = p.Tick(ctx, p.changefeed) c.Assert(err, check.IsNil) - applyPatches(c, p.changefeed) + tester.MustApplyPatches() // add table, in processing // in current implementation of owner, the startTs and BoundaryTs of add table operation should be always equaled. - p.changefeed.TaskStatus.AddTable(66, &model.TableReplicaInfo{StartTs: 60}, 60) + p.changefeed.PatchTaskStatus(p.captureInfo.ID, func(status *model.TaskStatus) (*model.TaskStatus, bool, error) { + status.AddTable(66, &model.TableReplicaInfo{StartTs: 60}, 60) + return status, true, nil + }) + tester.MustApplyPatches() _, err = p.Tick(ctx, p.changefeed) c.Assert(err, check.IsNil) - applyPatches(c, p.changefeed) - c.Assert(p.changefeed.TaskStatus, check.DeepEquals, &model.TaskStatus{ + tester.MustApplyPatches() + c.Assert(p.changefeed.TaskStatuses[p.captureInfo.ID], check.DeepEquals, &model.TaskStatus{ Tables: map[int64]*model.TableReplicaInfo{ 66: {StartTs: 60}, }, @@ -266,8 +186,8 @@ func (s *processorSuite) TestHandleTableOperation4SingleTable(c *check.C) { // add table, not finished _, err = p.Tick(ctx, p.changefeed) c.Assert(err, check.IsNil) - applyPatches(c, p.changefeed) - c.Assert(p.changefeed.TaskStatus, check.DeepEquals, &model.TaskStatus{ + tester.MustApplyPatches() + c.Assert(p.changefeed.TaskStatuses[p.captureInfo.ID], check.DeepEquals, &model.TaskStatus{ Tables: map[int64]*model.TableReplicaInfo{ 66: {StartTs: 60}, }, @@ -281,8 +201,8 @@ func (s *processorSuite) TestHandleTableOperation4SingleTable(c *check.C) { table66.resolvedTs = 101 _, err = p.Tick(ctx, p.changefeed) c.Assert(err, check.IsNil) - applyPatches(c, p.changefeed) - c.Assert(p.changefeed.TaskStatus, check.DeepEquals, &model.TaskStatus{ + tester.MustApplyPatches() + c.Assert(p.changefeed.TaskStatuses[p.captureInfo.ID], check.DeepEquals, &model.TaskStatus{ Tables: map[int64]*model.TableReplicaInfo{ 66: {StartTs: 60}, }, @@ -290,13 +210,13 @@ func (s *processorSuite) TestHandleTableOperation4SingleTable(c *check.C) { 66: {Delete: false, BoundaryTs: 60, Done: false, Status: model.OperProcessed}, }, }) - c.Assert(p.changefeed.TaskPosition.ResolvedTs, check.Equals, uint64(101)) + c.Assert(p.changefeed.TaskPositions[p.captureInfo.ID].ResolvedTs, check.Equals, uint64(101)) // finish the operation _, err = p.Tick(ctx, p.changefeed) c.Assert(err, check.IsNil) - applyPatches(c, p.changefeed) - c.Assert(p.changefeed.TaskStatus, check.DeepEquals, &model.TaskStatus{ + tester.MustApplyPatches() + c.Assert(p.changefeed.TaskStatuses[p.captureInfo.ID], check.DeepEquals, &model.TaskStatus{ Tables: map[int64]*model.TableReplicaInfo{ 66: {StartTs: 60}, }, @@ -306,22 +226,18 @@ func (s *processorSuite) TestHandleTableOperation4SingleTable(c *check.C) { }) // clear finished operations - _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) - applyPatches(c, p.changefeed) - c.Assert(p.changefeed.TaskStatus, check.DeepEquals, &model.TaskStatus{ - Tables: map[int64]*model.TableReplicaInfo{ - 66: {StartTs: 60}, - }, - Operation: nil, - }) + cleanUpFinishedOpOperation(p.changefeed, p.captureInfo.ID, tester) // remove table, in processing - p.changefeed.TaskStatus.RemoveTable(66, 120, false) + p.changefeed.PatchTaskStatus(p.captureInfo.ID, func(status *model.TaskStatus) (*model.TaskStatus, bool, error) { + status.RemoveTable(66, 120, false) + return status, true, nil + }) + tester.MustApplyPatches() _, err = p.Tick(ctx, p.changefeed) c.Assert(err, check.IsNil) - applyPatches(c, p.changefeed) - c.Assert(p.changefeed.TaskStatus, check.DeepEquals, &model.TaskStatus{ + tester.MustApplyPatches() + c.Assert(p.changefeed.TaskStatuses[p.captureInfo.ID], check.DeepEquals, &model.TaskStatus{ Tables: map[int64]*model.TableReplicaInfo{}, Operation: map[int64]*model.TableOperation{ 66: {Delete: true, BoundaryTs: 120, Done: false, Status: model.OperProcessed}, @@ -332,8 +248,8 @@ func (s *processorSuite) TestHandleTableOperation4SingleTable(c *check.C) { // remove table, not finished _, err = p.Tick(ctx, p.changefeed) c.Assert(err, check.IsNil) - applyPatches(c, p.changefeed) - c.Assert(p.changefeed.TaskStatus, check.DeepEquals, &model.TaskStatus{ + tester.MustApplyPatches() + c.Assert(p.changefeed.TaskStatuses[p.captureInfo.ID], check.DeepEquals, &model.TaskStatus{ Tables: map[int64]*model.TableReplicaInfo{}, Operation: map[int64]*model.TableOperation{ 66: {Delete: true, BoundaryTs: 120, Done: false, Status: model.OperProcessed}, @@ -341,12 +257,12 @@ func (s *processorSuite) TestHandleTableOperation4SingleTable(c *check.C) { }) // remove table, finished - table66.status = pipeline.TableStatusStopped + table66.status = tablepipeline.TableStatusStopped table66.checkpointTs = 121 _, err = p.Tick(ctx, p.changefeed) c.Assert(err, check.IsNil) - applyPatches(c, p.changefeed) - c.Assert(p.changefeed.TaskStatus, check.DeepEquals, &model.TaskStatus{ + tester.MustApplyPatches() + c.Assert(p.changefeed.TaskStatuses[p.captureInfo.ID], check.DeepEquals, &model.TaskStatus{ Tables: map[int64]*model.TableReplicaInfo{}, Operation: map[int64]*model.TableOperation{ 66: {Delete: true, BoundaryTs: 121, Done: true, Status: model.OperFinished}, @@ -358,55 +274,44 @@ func (s *processorSuite) TestHandleTableOperation4SingleTable(c *check.C) { func (s *processorSuite) TestHandleTableOperation4MultiTable(c *check.C) { defer testleak.AfterTest(c)() - ctx := context.Background() - p := newProcessor4Test() + ctx := cdcContext.NewBackendContext4Test(true) + p, tester := initProcessor4Test(ctx, c) var err error // init tick _, err = p.Tick(ctx, p.changefeed) c.Assert(err, check.IsNil) - applyPatches(c, p.changefeed) - p.schemaStorage.AdvanceResolvedTs(200) - p.changefeed.Status.CheckpointTs = 90 - p.changefeed.Status.ResolvedTs = 90 - p.changefeed.TaskPosition.ResolvedTs = 100 - p.changefeed.TaskPosition.CheckPointTs = 90 + tester.MustApplyPatches() + p.changefeed.PatchStatus(func(status *model.ChangeFeedStatus) (*model.ChangeFeedStatus, bool, error) { + status.CheckpointTs = 20 + status.ResolvedTs = 20 + return status, true, nil + }) + p.changefeed.PatchTaskPosition(p.captureInfo.ID, func(position *model.TaskPosition) (*model.TaskPosition, bool, error) { + position.ResolvedTs = 100 + position.CheckPointTs = 90 + return position, true, nil + }) + tester.MustApplyPatches() // no operation _, err = p.Tick(ctx, p.changefeed) c.Assert(err, check.IsNil) - applyPatches(c, p.changefeed) + tester.MustApplyPatches() // add table, in processing // in current implementation of owner, the startTs and BoundaryTs of add table operation should be always equaled. - p.changefeed.TaskStatus.AddTable(1, &model.TableReplicaInfo{StartTs: 60}, 60) - p.changefeed.TaskStatus.AddTable(2, &model.TableReplicaInfo{StartTs: 50}, 50) - p.changefeed.TaskStatus.AddTable(3, &model.TableReplicaInfo{StartTs: 40}, 40) - p.changefeed.TaskStatus.Tables[4] = &model.TableReplicaInfo{StartTs: 30} - _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) - applyPatches(c, p.changefeed) - c.Assert(p.changefeed.TaskStatus, check.DeepEquals, &model.TaskStatus{ - Tables: map[int64]*model.TableReplicaInfo{ - 1: {StartTs: 60}, - 2: {StartTs: 50}, - 3: {StartTs: 40}, - 4: {StartTs: 30}, - }, - Operation: map[int64]*model.TableOperation{ - 1: {Delete: false, BoundaryTs: 60, Done: false, Status: model.OperProcessed}, - 2: {Delete: false, BoundaryTs: 50, Done: false, Status: model.OperProcessed}, - 3: {Delete: false, BoundaryTs: 40, Done: false, Status: model.OperProcessed}, - }, + p.changefeed.PatchTaskStatus(p.captureInfo.ID, func(status *model.TaskStatus) (*model.TaskStatus, bool, error) { + status.AddTable(1, &model.TableReplicaInfo{StartTs: 60}, 60) + status.AddTable(2, &model.TableReplicaInfo{StartTs: 50}, 50) + status.AddTable(3, &model.TableReplicaInfo{StartTs: 40}, 40) + status.Tables[4] = &model.TableReplicaInfo{StartTs: 30} + return status, true, nil }) - c.Assert(p.tables, check.HasLen, 4) - c.Assert(p.changefeed.TaskPosition.CheckPointTs, check.Equals, uint64(30)) - c.Assert(p.changefeed.TaskPosition.ResolvedTs, check.Equals, uint64(30)) - - // add table, not finished + tester.MustApplyPatches() _, err = p.Tick(ctx, p.changefeed) c.Assert(err, check.IsNil) - applyPatches(c, p.changefeed) - c.Assert(p.changefeed.TaskStatus, check.DeepEquals, &model.TaskStatus{ + tester.MustApplyPatches() + c.Assert(p.changefeed.TaskStatuses[p.captureInfo.ID], check.DeepEquals, &model.TaskStatus{ Tables: map[int64]*model.TableReplicaInfo{ 1: {StartTs: 60}, 2: {StartTs: 50}, @@ -420,8 +325,10 @@ func (s *processorSuite) TestHandleTableOperation4MultiTable(c *check.C) { }, }) c.Assert(p.tables, check.HasLen, 4) + c.Assert(p.changefeed.TaskPositions[p.captureInfo.ID].CheckPointTs, check.Equals, uint64(30)) + c.Assert(p.changefeed.TaskPositions[p.captureInfo.ID].ResolvedTs, check.Equals, uint64(30)) - // add table, push the resolvedTs + // add table, push the resolvedTs, finished add table table1 := p.tables[1].(*mockTablePipeline) table2 := p.tables[2].(*mockTablePipeline) table3 := p.tables[3].(*mockTablePipeline) @@ -431,34 +338,38 @@ func (s *processorSuite) TestHandleTableOperation4MultiTable(c *check.C) { table3.resolvedTs = 102 table4.resolvedTs = 103 // removed table 3 - p.changefeed.TaskStatus.RemoveTable(3, 60, false) + p.changefeed.PatchTaskStatus(p.captureInfo.ID, func(status *model.TaskStatus) (*model.TaskStatus, bool, error) { + status.RemoveTable(3, 60, false) + return status, true, nil + }) + tester.MustApplyPatches() _, err = p.Tick(ctx, p.changefeed) c.Assert(err, check.IsNil) - applyPatches(c, p.changefeed) - c.Assert(p.changefeed.TaskStatus, check.DeepEquals, &model.TaskStatus{ + tester.MustApplyPatches() + c.Assert(p.changefeed.TaskStatuses[p.captureInfo.ID], check.DeepEquals, &model.TaskStatus{ Tables: map[int64]*model.TableReplicaInfo{ 1: {StartTs: 60}, 2: {StartTs: 50}, 4: {StartTs: 30}, }, Operation: map[int64]*model.TableOperation{ - 1: {Delete: false, BoundaryTs: 60, Done: false, Status: model.OperProcessed}, - 2: {Delete: false, BoundaryTs: 50, Done: false, Status: model.OperProcessed}, + 1: {Delete: false, BoundaryTs: 60, Done: true, Status: model.OperFinished}, + 2: {Delete: false, BoundaryTs: 50, Done: true, Status: model.OperFinished}, 3: {Delete: true, BoundaryTs: 60, Done: false, Status: model.OperProcessed}, }, }) c.Assert(p.tables, check.HasLen, 4) c.Assert(table3.canceled, check.IsFalse) c.Assert(table3.stopTs, check.Equals, uint64(60)) - c.Assert(p.changefeed.TaskPosition.ResolvedTs, check.Equals, uint64(101)) + c.Assert(p.changefeed.TaskPositions[p.captureInfo.ID].ResolvedTs, check.Equals, uint64(101)) - // finish remove and add operations - table3.status = pipeline.TableStatusStopped + // finish remove operations + table3.status = tablepipeline.TableStatusStopped table3.checkpointTs = 65 _, err = p.Tick(ctx, p.changefeed) c.Assert(err, check.IsNil) - applyPatches(c, p.changefeed) - c.Assert(p.changefeed.TaskStatus, check.DeepEquals, &model.TaskStatus{ + tester.MustApplyPatches() + c.Assert(p.changefeed.TaskStatuses[p.captureInfo.ID], check.DeepEquals, &model.TaskStatus{ Tables: map[int64]*model.TableReplicaInfo{ 1: {StartTs: 60}, 2: {StartTs: 50}, @@ -474,27 +385,20 @@ func (s *processorSuite) TestHandleTableOperation4MultiTable(c *check.C) { c.Assert(table3.canceled, check.IsTrue) // clear finished operations - _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) - applyPatches(c, p.changefeed) - c.Assert(p.changefeed.TaskStatus, check.DeepEquals, &model.TaskStatus{ - Tables: map[int64]*model.TableReplicaInfo{ - 1: {StartTs: 60}, - 2: {StartTs: 50}, - 4: {StartTs: 30}, - }, - Operation: nil, - }) - c.Assert(p.tables, check.HasLen, 3) + cleanUpFinishedOpOperation(p.changefeed, p.captureInfo.ID, tester) // remove table, in processing - p.changefeed.TaskStatus.RemoveTable(1, 120, false) - p.changefeed.TaskStatus.RemoveTable(4, 120, false) - delete(p.changefeed.TaskStatus.Tables, 2) + p.changefeed.PatchTaskStatus(p.captureInfo.ID, func(status *model.TaskStatus) (*model.TaskStatus, bool, error) { + status.RemoveTable(1, 120, false) + status.RemoveTable(4, 120, false) + delete(status.Tables, 2) + return status, true, nil + }) + tester.MustApplyPatches() _, err = p.Tick(ctx, p.changefeed) c.Assert(err, check.IsNil) - applyPatches(c, p.changefeed) - c.Assert(p.changefeed.TaskStatus, check.DeepEquals, &model.TaskStatus{ + tester.MustApplyPatches() + c.Assert(p.changefeed.TaskStatuses[p.captureInfo.ID], check.DeepEquals, &model.TaskStatus{ Tables: map[int64]*model.TableReplicaInfo{}, Operation: map[int64]*model.TableOperation{ 1: {Delete: true, BoundaryTs: 120, Done: false, Status: model.OperProcessed}, @@ -509,8 +413,8 @@ func (s *processorSuite) TestHandleTableOperation4MultiTable(c *check.C) { // remove table, not finished _, err = p.Tick(ctx, p.changefeed) c.Assert(err, check.IsNil) - applyPatches(c, p.changefeed) - c.Assert(p.changefeed.TaskStatus, check.DeepEquals, &model.TaskStatus{ + tester.MustApplyPatches() + c.Assert(p.changefeed.TaskStatuses[p.captureInfo.ID], check.DeepEquals, &model.TaskStatus{ Tables: map[int64]*model.TableReplicaInfo{}, Operation: map[int64]*model.TableOperation{ 1: {Delete: true, BoundaryTs: 120, Done: false, Status: model.OperProcessed}, @@ -519,14 +423,14 @@ func (s *processorSuite) TestHandleTableOperation4MultiTable(c *check.C) { }) // remove table, finished - table1.status = pipeline.TableStatusStopped + table1.status = tablepipeline.TableStatusStopped table1.checkpointTs = 121 - table4.status = pipeline.TableStatusStopped + table4.status = tablepipeline.TableStatusStopped table4.checkpointTs = 122 _, err = p.Tick(ctx, p.changefeed) c.Assert(err, check.IsNil) - applyPatches(c, p.changefeed) - c.Assert(p.changefeed.TaskStatus, check.DeepEquals, &model.TaskStatus{ + tester.MustApplyPatches() + c.Assert(p.changefeed.TaskStatuses[p.captureInfo.ID], check.DeepEquals, &model.TaskStatus{ Tables: map[int64]*model.TableReplicaInfo{}, Operation: map[int64]*model.TableOperation{ 1: {Delete: true, BoundaryTs: 121, Done: true, Status: model.OperFinished}, @@ -540,190 +444,211 @@ func (s *processorSuite) TestHandleTableOperation4MultiTable(c *check.C) { func (s *processorSuite) TestInitTable(c *check.C) { defer testleak.AfterTest(c)() - p := newProcessor4Test() - ctx := context.Background() + ctx := cdcContext.NewBackendContext4Test(true) + p, tester := initProcessor4Test(ctx, c) var err error // init tick _, err = p.Tick(ctx, p.changefeed) c.Assert(err, check.IsNil) - applyPatches(c, p.changefeed) + tester.MustApplyPatches() - p.changefeed.TaskStatus.Tables[1] = &model.TableReplicaInfo{StartTs: 20} - p.changefeed.TaskStatus.Tables[2] = &model.TableReplicaInfo{StartTs: 30} + p.changefeed.PatchTaskStatus(p.captureInfo.ID, func(status *model.TaskStatus) (*model.TaskStatus, bool, error) { + status.Tables[1] = &model.TableReplicaInfo{StartTs: 20} + status.Tables[2] = &model.TableReplicaInfo{StartTs: 30} + return status, true, nil + }) + tester.MustApplyPatches() _, err = p.Tick(ctx, p.changefeed) c.Assert(err, check.IsNil) - applyPatches(c, p.changefeed) + tester.MustApplyPatches() c.Assert(p.tables[1], check.Not(check.IsNil)) c.Assert(p.tables[2], check.Not(check.IsNil)) } func (s *processorSuite) TestProcessorError(c *check.C) { defer testleak.AfterTest(c)() - p := newProcessor4Test() - ctx := context.Background() + ctx := cdcContext.NewBackendContext4Test(true) + p, tester := initProcessor4Test(ctx, c) var err error // init tick _, err = p.Tick(ctx, p.changefeed) c.Assert(err, check.IsNil) - applyPatches(c, p.changefeed) + tester.MustApplyPatches() // send a abnormal error p.sendError(cerror.ErrSinkURIInvalid) _, err = p.Tick(ctx, p.changefeed) - applyPatches(c, p.changefeed) + tester.MustApplyPatches() c.Assert(cerror.ErrReactorFinished.Equal(errors.Cause(err)), check.IsTrue) - c.Assert(p.changefeed.TaskPosition, check.DeepEquals, &model.TaskPosition{ + c.Assert(p.changefeed.TaskPositions[p.captureInfo.ID], check.DeepEquals, &model.TaskPosition{ Error: &model.RunningError{ Addr: "127.0.0.1:0000", - Code: "CDC:ErrProcessorUnknown", + Code: "CDC:ErrSinkURIInvalid", Message: "[CDC:ErrSinkURIInvalid]sink uri invalid", }, }) - p = newProcessor4Test() + p, tester = initProcessor4Test(ctx, c) // init tick _, err = p.Tick(ctx, p.changefeed) c.Assert(err, check.IsNil) - applyPatches(c, p.changefeed) + tester.MustApplyPatches() // send a normal error p.sendError(context.Canceled) _, err = p.Tick(ctx, p.changefeed) - applyPatches(c, p.changefeed) + tester.MustApplyPatches() c.Assert(cerror.ErrReactorFinished.Equal(errors.Cause(err)), check.IsTrue) - c.Assert(p.changefeed.TaskPosition, check.DeepEquals, &model.TaskPosition{ + c.Assert(p.changefeed.TaskPositions[p.captureInfo.ID], check.DeepEquals, &model.TaskPosition{ Error: nil, }) } func (s *processorSuite) TestProcessorExit(c *check.C) { defer testleak.AfterTest(c)() - p := newProcessor4Test() - ctx := context.Background() + ctx := cdcContext.NewBackendContext4Test(true) + p, tester := initProcessor4Test(ctx, c) var err error // init tick _, err = p.Tick(ctx, p.changefeed) c.Assert(err, check.IsNil) - applyPatches(c, p.changefeed) + tester.MustApplyPatches() // stop the changefeed - p.changefeed.TaskStatus.AdminJobType = model.AdminStop + p.changefeed.PatchStatus(func(status *model.ChangeFeedStatus) (*model.ChangeFeedStatus, bool, error) { + status.AdminJobType = model.AdminStop + return status, true, nil + }) + p.changefeed.PatchTaskStatus(ctx.GlobalVars().CaptureInfo.ID, func(status *model.TaskStatus) (*model.TaskStatus, bool, error) { + status.AdminJobType = model.AdminStop + return status, true, nil + }) + tester.MustApplyPatches() _, err = p.Tick(ctx, p.changefeed) c.Assert(cerror.ErrReactorFinished.Equal(errors.Cause(err)), check.IsTrue) - applyPatches(c, p.changefeed) - c.Assert(p.changefeed.TaskPosition, check.DeepEquals, &model.TaskPosition{ + tester.MustApplyPatches() + c.Assert(p.changefeed.TaskPositions[p.captureInfo.ID], check.DeepEquals, &model.TaskPosition{ Error: nil, }) } func (s *processorSuite) TestProcessorClose(c *check.C) { defer testleak.AfterTest(c)() - p := newProcessor4Test() - ctx := context.Background() + ctx := cdcContext.NewBackendContext4Test(true) + p, tester := initProcessor4Test(ctx, c) var err error // init tick _, err = p.Tick(ctx, p.changefeed) c.Assert(err, check.IsNil) - applyPatches(c, p.changefeed) + tester.MustApplyPatches() // add tables - p.changefeed.TaskStatus.Tables[1] = &model.TableReplicaInfo{StartTs: 20} - p.changefeed.TaskStatus.Tables[2] = &model.TableReplicaInfo{StartTs: 30} + p.changefeed.PatchTaskStatus(p.captureInfo.ID, func(status *model.TaskStatus) (*model.TaskStatus, bool, error) { + status.Tables[1] = &model.TableReplicaInfo{StartTs: 20} + status.Tables[2] = &model.TableReplicaInfo{StartTs: 30} + return status, true, nil + }) + tester.MustApplyPatches() _, err = p.Tick(ctx, p.changefeed) c.Assert(err, check.IsNil) - applyPatches(c, p.changefeed) + tester.MustApplyPatches() // push the resolvedTs and checkpointTs - p.schemaStorage.AdvanceResolvedTs(100) - p.changefeed.Status.ResolvedTs = 100 + p.changefeed.PatchStatus(func(status *model.ChangeFeedStatus) (*model.ChangeFeedStatus, bool, error) { + status.ResolvedTs = 100 + return status, true, nil + }) + tester.MustApplyPatches() p.tables[1].(*mockTablePipeline).resolvedTs = 110 p.tables[2].(*mockTablePipeline).resolvedTs = 90 p.tables[1].(*mockTablePipeline).checkpointTs = 90 p.tables[2].(*mockTablePipeline).checkpointTs = 95 _, err = p.Tick(ctx, p.changefeed) c.Assert(err, check.IsNil) - applyPatches(c, p.changefeed) - c.Assert(p.changefeed.TaskPosition, check.DeepEquals, &model.TaskPosition{ + tester.MustApplyPatches() + c.Assert(p.changefeed.TaskPositions[p.captureInfo.ID], check.DeepEquals, &model.TaskPosition{ CheckPointTs: 90, ResolvedTs: 90, Error: nil, }) - c.Assert(p.changefeed.TaskStatus, check.DeepEquals, &model.TaskStatus{ + c.Assert(p.changefeed.TaskStatuses[p.captureInfo.ID], check.DeepEquals, &model.TaskStatus{ Tables: map[int64]*model.TableReplicaInfo{1: {StartTs: 20}, 2: {StartTs: 30}}, }) - c.Assert(p.changefeed.Workload, check.DeepEquals, model.TaskWorkload{1: {Workload: 1}, 2: {Workload: 1}}) + c.Assert(p.changefeed.Workloads[p.captureInfo.ID], check.DeepEquals, model.TaskWorkload{1: {Workload: 1}, 2: {Workload: 1}}) c.Assert(p.Close(), check.IsNil) - applyPatches(c, p.changefeed) - c.Assert(p.changefeed.TaskPosition, check.IsNil) - c.Assert(p.changefeed.TaskStatus, check.IsNil) - c.Assert(p.changefeed.Workload, check.IsNil) + tester.MustApplyPatches() c.Assert(p.tables[1].(*mockTablePipeline).canceled, check.IsTrue) c.Assert(p.tables[2].(*mockTablePipeline).canceled, check.IsTrue) - p = newProcessor4Test() + p, tester = initProcessor4Test(ctx, c) // init tick _, err = p.Tick(ctx, p.changefeed) c.Assert(err, check.IsNil) - applyPatches(c, p.changefeed) + tester.MustApplyPatches() // add tables - p.changefeed.TaskStatus.Tables[1] = &model.TableReplicaInfo{StartTs: 20} - p.changefeed.TaskStatus.Tables[2] = &model.TableReplicaInfo{StartTs: 30} + p.changefeed.PatchTaskStatus(p.captureInfo.ID, func(status *model.TaskStatus) (*model.TaskStatus, bool, error) { + status.Tables[1] = &model.TableReplicaInfo{StartTs: 20} + status.Tables[2] = &model.TableReplicaInfo{StartTs: 30} + return status, true, nil + }) + tester.MustApplyPatches() _, err = p.Tick(ctx, p.changefeed) c.Assert(err, check.IsNil) - applyPatches(c, p.changefeed) + tester.MustApplyPatches() // send error p.sendError(cerror.ErrSinkURIInvalid) _, err = p.Tick(ctx, p.changefeed) c.Assert(cerror.ErrReactorFinished.Equal(errors.Cause(err)), check.IsTrue) - applyPatches(c, p.changefeed) + tester.MustApplyPatches() c.Assert(p.Close(), check.IsNil) - applyPatches(c, p.changefeed) - c.Assert(p.changefeed.TaskPosition, check.DeepEquals, &model.TaskPosition{ - Error: &model.RunningError{ - Addr: "127.0.0.1:0000", - Code: "CDC:ErrProcessorUnknown", - Message: "[CDC:ErrSinkURIInvalid]sink uri invalid", - }, + tester.MustApplyPatches() + c.Assert(p.changefeed.TaskPositions[p.captureInfo.ID].Error, check.DeepEquals, &model.RunningError{ + Addr: "127.0.0.1:0000", + Code: "CDC:ErrSinkURIInvalid", + Message: "[CDC:ErrSinkURIInvalid]sink uri invalid", }) - c.Assert(p.changefeed.TaskStatus, check.IsNil) - c.Assert(p.changefeed.Workload, check.IsNil) c.Assert(p.tables[1].(*mockTablePipeline).canceled, check.IsTrue) c.Assert(p.tables[2].(*mockTablePipeline).canceled, check.IsTrue) } func (s *processorSuite) TestPositionDeleted(c *check.C) { defer testleak.AfterTest(c)() - ctx := context.Background() - p := newProcessor4Test() - p.changefeed.TaskStatus.Tables[1] = &model.TableReplicaInfo{StartTs: 30} - p.changefeed.TaskStatus.Tables[2] = &model.TableReplicaInfo{StartTs: 40} + ctx := cdcContext.NewBackendContext4Test(true) + p, tester := initProcessor4Test(ctx, c) + p.changefeed.PatchTaskStatus(p.captureInfo.ID, func(status *model.TaskStatus) (*model.TaskStatus, bool, error) { + status.Tables[1] = &model.TableReplicaInfo{StartTs: 30} + status.Tables[2] = &model.TableReplicaInfo{StartTs: 40} + return status, true, nil + }) var err error // init tick _, err = p.Tick(ctx, p.changefeed) c.Assert(err, check.IsNil) - applyPatches(c, p.changefeed) - p.schemaStorage.AdvanceResolvedTs(200) + tester.MustApplyPatches() // cal position _, err = p.Tick(ctx, p.changefeed) c.Assert(err, check.IsNil) - applyPatches(c, p.changefeed) - c.Assert(p.changefeed.TaskPosition, check.DeepEquals, &model.TaskPosition{ + tester.MustApplyPatches() + c.Assert(p.changefeed.TaskPositions[p.captureInfo.ID], check.DeepEquals, &model.TaskPosition{ CheckPointTs: 30, ResolvedTs: 30, }) // some other delete the task position - p.changefeed.TaskPosition = nil + p.changefeed.PatchTaskPosition(p.captureInfo.ID, func(position *model.TaskPosition) (*model.TaskPosition, bool, error) { + return nil, true, nil + }) + tester.MustApplyPatches() // position created again _, err = p.Tick(ctx, p.changefeed) c.Assert(err, check.IsNil) - applyPatches(c, p.changefeed) - c.Assert(p.changefeed.TaskPosition, check.DeepEquals, &model.TaskPosition{ + tester.MustApplyPatches() + c.Assert(p.changefeed.TaskPositions[p.captureInfo.ID], check.DeepEquals, &model.TaskPosition{ CheckPointTs: 0, ResolvedTs: 0, }) @@ -731,9 +656,24 @@ func (s *processorSuite) TestPositionDeleted(c *check.C) { // cal position _, err = p.Tick(ctx, p.changefeed) c.Assert(err, check.IsNil) - applyPatches(c, p.changefeed) - c.Assert(p.changefeed.TaskPosition, check.DeepEquals, &model.TaskPosition{ + tester.MustApplyPatches() + c.Assert(p.changefeed.TaskPositions[p.captureInfo.ID], check.DeepEquals, &model.TaskPosition{ CheckPointTs: 30, ResolvedTs: 30, }) } + +func cleanUpFinishedOpOperation(state *model.ChangefeedReactorState, captureID model.CaptureID, tester *orchestrator.ReactorStateTester) { + state.PatchTaskStatus(captureID, func(status *model.TaskStatus) (*model.TaskStatus, bool, error) { + if status == nil || status.Operation == nil { + return status, false, nil + } + for tableID, opt := range status.Operation { + if opt.Done && opt.Status == model.OperFinished { + delete(status.Operation, tableID) + } + } + return status, true, nil + }) + tester.MustApplyPatches() +} diff --git a/cdc/processor/state.go b/cdc/processor/state.go deleted file mode 100644 index 0176a01d200..00000000000 --- a/cdc/processor/state.go +++ /dev/null @@ -1,266 +0,0 @@ -// Copyright 2021 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package processor - -import ( - "encoding/json" - "reflect" - - "github.com/pingcap/errors" - "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/etcd" - "github.com/pingcap/ticdc/pkg/orchestrator" - "github.com/pingcap/ticdc/pkg/orchestrator/util" - "go.uber.org/zap" -) - -type globalState struct { - CaptureID model.CaptureID - Changefeeds map[model.ChangeFeedID]*changefeedState -} - -// NewGlobalState creates a new global state for processor manager -func NewGlobalState(captureID model.CaptureID) orchestrator.ReactorState { - return &globalState{ - CaptureID: captureID, - Changefeeds: make(map[model.ChangeFeedID]*changefeedState), - } -} - -func (s *globalState) Update(key util.EtcdKey, value []byte, isInit bool) error { - k := new(etcd.CDCKey) - err := k.Parse(key.String()) - if err != nil { - return errors.Trace(err) - } - if k.Tp == etcd.CDCKeyTypeCapture || k.Tp == etcd.CDCKeyTypeOwner { - return nil - } - if len(k.CaptureID) != 0 && k.CaptureID != s.CaptureID { - return nil - } - changefeedState, exist := s.Changefeeds[k.ChangefeedID] - if !exist { - changefeedState = newChangeFeedState(k.ChangefeedID, s.CaptureID) - s.Changefeeds[k.ChangefeedID] = changefeedState - } - if err := changefeedState.UpdateCDCKey(k, value); err != nil { - return errors.Trace(err) - } - if value == nil && !changefeedState.Exist() { - delete(s.Changefeeds, k.ChangefeedID) - } - return nil -} - -func (s *globalState) GetPatches() []*orchestrator.DataPatch { - var pendingPatches []*orchestrator.DataPatch - for _, changefeedState := range s.Changefeeds { - pendingPatches = append(pendingPatches, changefeedState.GetPatches()...) - } - return pendingPatches -} - -type changefeedState struct { - ID model.ChangeFeedID - CaptureID model.CaptureID - Info *model.ChangeFeedInfo - Status *model.ChangeFeedStatus - TaskPosition *model.TaskPosition - TaskStatus *model.TaskStatus - Workload model.TaskWorkload - - pendingPatches []*orchestrator.DataPatch -} - -func newChangeFeedState(id model.ChangeFeedID, captureID model.CaptureID) *changefeedState { - return &changefeedState{ - ID: id, - CaptureID: captureID, - } -} - -func (s *changefeedState) Update(key util.EtcdKey, value []byte, isInit bool) error { - k := new(etcd.CDCKey) - err := k.Parse(key.String()) - if err != nil { - return errors.Trace(err) - } - if err := s.UpdateCDCKey(k, value); err != nil { - log.Error("failed to update status", zap.String("key", key.String()), zap.ByteString("value", value)) - return errors.Trace(err) - } - return nil -} - -func (s *changefeedState) UpdateCDCKey(key *etcd.CDCKey, value []byte) error { - var e interface{} - switch key.Tp { - case etcd.CDCKeyTypeChangefeedInfo: - if key.ChangefeedID != s.ID { - return nil - } - if value == nil { - s.Info = nil - return nil - } - s.Info = new(model.ChangeFeedInfo) - e = s.Info - case etcd.CDCKeyTypeChangeFeedStatus: - if key.ChangefeedID != s.ID { - return nil - } - if value == nil { - s.Status = nil - return nil - } - s.Status = new(model.ChangeFeedStatus) - e = s.Status - case etcd.CDCKeyTypeTaskPosition: - if key.ChangefeedID != s.ID || key.CaptureID != s.CaptureID { - return nil - } - if value == nil { - s.TaskPosition = nil - return nil - } - s.TaskPosition = new(model.TaskPosition) - e = s.TaskPosition - case etcd.CDCKeyTypeTaskStatus: - if key.ChangefeedID != s.ID || key.CaptureID != s.CaptureID { - return nil - } - if value == nil { - s.TaskStatus = nil - return nil - } - s.TaskStatus = new(model.TaskStatus) - e = s.TaskStatus - case etcd.CDCKeyTypeTaskWorkload: - if key.ChangefeedID != s.ID || key.CaptureID != s.CaptureID { - return nil - } - if value == nil { - s.Workload = nil - return nil - } - s.Workload = make(model.TaskWorkload) - e = &s.Workload - default: - return nil - } - err := json.Unmarshal(value, e) - if err != nil { - return errors.Trace(err) - } - if key.Tp == etcd.CDCKeyTypeChangefeedInfo { - err = s.Info.VerifyAndFix() - if err != nil { - return errors.Trace(err) - } - } - return nil -} - -func (s *changefeedState) Exist() bool { - return s.Info != nil || s.Status != nil || s.TaskPosition != nil || s.TaskStatus != nil || s.Workload != nil -} - -func (s *changefeedState) Active() bool { - return s.Info != nil && s.Status != nil && s.TaskStatus != nil -} - -func (s *changefeedState) GetPatches() []*orchestrator.DataPatch { - pendingPatches := s.pendingPatches - s.pendingPatches = nil - return pendingPatches -} - -var ( - taskPositionTPI *model.TaskPosition - taskStatusTPI *model.TaskStatus - taskWorkloadTPI *model.TaskWorkload -) - -func (s *changefeedState) PatchTaskPosition(fn func(*model.TaskPosition) (*model.TaskPosition, error)) { - key := &etcd.CDCKey{ - Tp: etcd.CDCKeyTypeTaskPosition, - CaptureID: s.CaptureID, - ChangefeedID: s.ID, - } - s.patchAny(key.String(), taskPositionTPI, func(e interface{}) (interface{}, error) { - // e == nil means that the key is not exist before this patch - if e == nil { - return fn(nil) - } - return fn(e.(*model.TaskPosition)) - }) -} - -func (s *changefeedState) PatchTaskStatus(fn func(*model.TaskStatus) (*model.TaskStatus, error)) { - key := &etcd.CDCKey{ - Tp: etcd.CDCKeyTypeTaskStatus, - CaptureID: s.CaptureID, - ChangefeedID: s.ID, - } - s.patchAny(key.String(), taskStatusTPI, func(e interface{}) (interface{}, error) { - // e == nil means that the key is not exist before this patch - if e == nil { - return fn(nil) - } - return fn(e.(*model.TaskStatus)) - }) -} - -func (s *changefeedState) PatchTaskWorkload(fn func(model.TaskWorkload) (model.TaskWorkload, error)) { - key := &etcd.CDCKey{ - Tp: etcd.CDCKeyTypeTaskWorkload, - CaptureID: s.CaptureID, - ChangefeedID: s.ID, - } - s.patchAny(key.String(), taskWorkloadTPI, func(e interface{}) (interface{}, error) { - // e == nil means that the key is not exist before this patch - if e == nil { - return fn(nil) - } - return fn(*e.(*model.TaskWorkload)) - }) -} - -func (s *changefeedState) patchAny(key string, tpi interface{}, fn func(interface{}) (interface{}, error)) { - patch := &orchestrator.DataPatch{ - Key: util.NewEtcdKey(key), - Fun: func(v []byte) ([]byte, error) { - var e interface{} - if v != nil { - tp := reflect.TypeOf(tpi) - e = reflect.New(tp.Elem()).Interface() - err := json.Unmarshal(v, e) - if err != nil { - return nil, errors.Trace(err) - } - } - ne, err := fn(e) - if err != nil { - return nil, errors.Trace(err) - } - if reflect.ValueOf(ne).IsNil() { - return nil, nil - } - return json.Marshal(ne) - }, - } - s.pendingPatches = append(s.pendingPatches, patch) -} diff --git a/cdc/processor/state_test.go b/cdc/processor/state_test.go deleted file mode 100644 index b8021958970..00000000000 --- a/cdc/processor/state_test.go +++ /dev/null @@ -1,443 +0,0 @@ -// Copyright 2021 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package processor - -import ( - "testing" - "time" - - "github.com/google/go-cmp/cmp" - "github.com/google/go-cmp/cmp/cmpopts" - "github.com/pingcap/check" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/config" - "github.com/pingcap/ticdc/pkg/orchestrator" - "github.com/pingcap/ticdc/pkg/orchestrator/util" - "github.com/pingcap/ticdc/pkg/util/testleak" -) - -func TestSuite(t *testing.T) { - check.TestingT(t) -} - -type stateSuite struct{} - -var _ = check.Suite(&stateSuite{}) - -type mockReactorStatePatcher struct { - state orchestrator.ReactorState - rawState map[util.EtcdKey][]byte - c *check.C -} - -func newMockReactorStatePatcher(c *check.C, state orchestrator.ReactorState) *mockReactorStatePatcher { - return &mockReactorStatePatcher{ - state: state, - rawState: make(map[util.EtcdKey][]byte), - c: c, - } -} - -func (m *mockReactorStatePatcher) applyPatches() { - patches := m.state.GetPatches() - m.c.Assert(m.state.GetPatches(), check.HasLen, 0) - for _, patch := range patches { - newValue, err := patch.Fun(m.rawState[patch.Key]) - m.c.Assert(err, check.IsNil) - err = m.state.Update(patch.Key, newValue, false) - m.c.Assert(err, check.IsNil) - m.rawState[patch.Key] = newValue - } -} - -func (s *stateSuite) TestChangefeedStateUpdate(c *check.C) { - defer testleak.AfterTest(c)() - createTime, err := time.Parse("2006-01-02", "2020-02-02") - c.Assert(err, check.IsNil) - testCases := []struct { - changefeedID string - captureID string - updateKey []string - updateValue []string - expected changefeedState - }{ - { // common case - changefeedID: "test1", - captureID: "6bbc01c8-0605-4f86-a0f9-b3119109b225", - updateKey: []string{ - "/tidb/cdc/changefeed/info/test1", - "/tidb/cdc/job/test1", - "/tidb/cdc/task/position/6bbc01c8-0605-4f86-a0f9-b3119109b225/test1", - "/tidb/cdc/task/status/6bbc01c8-0605-4f86-a0f9-b3119109b225/test1", - "/tidb/cdc/task/workload/6bbc01c8-0605-4f86-a0f9-b3119109b225/test1", - "/tidb/cdc/capture/6bbc01c8-0605-4f86-a0f9-b3119109b225", - }, - updateValue: []string{ - `{"sink-uri":"blackhole://","opts":{},"create-time":"2020-02-02T00:00:00.000000+00:00","start-ts":421980685886554116,"target-ts":0,"admin-job-type":0,"sort-engine":"memory","config":{"case-sensitive":true,"enable-old-value":false,"force-replicate":false,"check-gc-safe-point":true,"filter":{"rules":["*.*"],"ignore-txn-start-ts":null,"ddl-allow-list":null},"mounter":{"worker-num":16},"sink":{"dispatchers":null,"protocol":"default"},"cyclic-replication":{"enable":false,"replica-id":0,"filter-replica-ids":null,"id-buckets":0,"sync-ddl":false},"scheduler":{"type":"table-number","polling-time":-1}},"state":"normal","history":null,"error":null,"sync-point-enabled":false,"sync-point-interval":600000000000}`, - `{"resolved-ts":421980720003809281,"checkpoint-ts":421980719742451713,"admin-job-type":0}`, - `{"checkpoint-ts":421980720003809281,"resolved-ts":421980720003809281,"count":0,"error":null}`, - `{"tables":{"45":{"start-ts":421980685886554116,"mark-table-id":0}},"operation":null,"admin-job-type":0}`, - `{"45":{"workload":1}}`, - `{"id":"6bbc01c8-0605-4f86-a0f9-b3119109b225","address":"127.0.0.1:8300"}`, - }, - expected: changefeedState{ - ID: "test1", - CaptureID: "6bbc01c8-0605-4f86-a0f9-b3119109b225", - Info: &model.ChangeFeedInfo{ - SinkURI: "blackhole://", - Opts: map[string]string{}, - CreateTime: createTime, - StartTs: 421980685886554116, - Engine: model.SortInMemory, - State: "normal", - SyncPointInterval: time.Minute * 10, - Config: &config.ReplicaConfig{ - CaseSensitive: true, - CheckGCSafePoint: true, - Filter: &config.FilterConfig{Rules: []string{"*.*"}}, - Mounter: &config.MounterConfig{WorkerNum: 16}, - Sink: &config.SinkConfig{Protocol: "default"}, - Cyclic: &config.CyclicConfig{}, - Scheduler: &config.SchedulerConfig{Tp: "table-number", PollingTime: -1}, - }, - }, - Status: &model.ChangeFeedStatus{CheckpointTs: 421980719742451713, ResolvedTs: 421980720003809281}, - TaskStatus: &model.TaskStatus{ - Tables: map[int64]*model.TableReplicaInfo{45: {StartTs: 421980685886554116}}, - }, - TaskPosition: &model.TaskPosition{CheckPointTs: 421980720003809281, ResolvedTs: 421980720003809281}, - Workload: model.TaskWorkload{45: {Workload: 1}}, - }, - }, - { // testing captureID or changefeedID not match - changefeedID: "test1", - captureID: "6bbc01c8-0605-4f86-a0f9-b3119109b225", - updateKey: []string{ - "/tidb/cdc/changefeed/info/test1", - "/tidb/cdc/job/test1", - "/tidb/cdc/task/position/6bbc01c8-0605-4f86-a0f9-b3119109b225/test1", - "/tidb/cdc/task/status/6bbc01c8-0605-4f86-a0f9-b3119109b225/test1", - "/tidb/cdc/task/workload/6bbc01c8-0605-4f86-a0f9-b3119109b225/test1", - "/tidb/cdc/capture/6bbc01c8-0605-4f86-a0f9-b3119109b225", - "/tidb/cdc/changefeed/info/test-fake", - "/tidb/cdc/job/test-fake", - "/tidb/cdc/task/position/6bbc01c8-0605-4f86-a0f9-b3119109b225/test-fake", - "/tidb/cdc/task/status/6bbc01c8-0605-4f86-a0f9-b3119109b225/test-fake", - "/tidb/cdc/task/workload/6bbc01c8-0605-4f86-a0f9-b3119109b225/test-fake", - "/tidb/cdc/task/position/fake-capture-id/test1", - "/tidb/cdc/task/status/fake-capture-id/test1", - "/tidb/cdc/task/workload/fake-capture-id/test1", - }, - updateValue: []string{ - `{"sink-uri":"blackhole://","opts":{},"create-time":"2020-02-02T00:00:00.000000+00:00","start-ts":421980685886554116,"target-ts":0,"admin-job-type":0,"sort-engine":"memory","config":{"case-sensitive":true,"enable-old-value":false,"force-replicate":false,"check-gc-safe-point":true,"filter":{"rules":["*.*"],"ignore-txn-start-ts":null,"ddl-allow-list":null},"mounter":{"worker-num":16},"sink":{"dispatchers":null,"protocol":"default"},"cyclic-replication":{"enable":false,"replica-id":0,"filter-replica-ids":null,"id-buckets":0,"sync-ddl":false},"scheduler":{"type":"table-number","polling-time":-1}},"state":"normal","history":null,"error":null,"sync-point-enabled":false,"sync-point-interval":600000000000}`, - `{"resolved-ts":421980720003809281,"checkpoint-ts":421980719742451713,"admin-job-type":0}`, - `{"checkpoint-ts":421980720003809281,"resolved-ts":421980720003809281,"count":0,"error":null}`, - `{"tables":{"45":{"start-ts":421980685886554116,"mark-table-id":0}},"operation":null,"admin-job-type":0}`, - `{"45":{"workload":1}}`, - `{"id":"6bbc01c8-0605-4f86-a0f9-b3119109b225","address":"127.0.0.1:8300"}`, - `fake value`, - `fake value`, - `fake value`, - `fake value`, - `fake value`, - `fake value`, - `fake value`, - `fake value`, - }, - expected: changefeedState{ - ID: "test1", - CaptureID: "6bbc01c8-0605-4f86-a0f9-b3119109b225", - Info: &model.ChangeFeedInfo{ - SinkURI: "blackhole://", - Opts: map[string]string{}, - CreateTime: createTime, - StartTs: 421980685886554116, - Engine: model.SortInMemory, - State: "normal", - SyncPointInterval: time.Minute * 10, - Config: &config.ReplicaConfig{ - CaseSensitive: true, - CheckGCSafePoint: true, - Filter: &config.FilterConfig{Rules: []string{"*.*"}}, - Mounter: &config.MounterConfig{WorkerNum: 16}, - Sink: &config.SinkConfig{Protocol: "default"}, - Cyclic: &config.CyclicConfig{}, - Scheduler: &config.SchedulerConfig{Tp: "table-number", PollingTime: -1}, - }, - }, - Status: &model.ChangeFeedStatus{CheckpointTs: 421980719742451713, ResolvedTs: 421980720003809281}, - TaskStatus: &model.TaskStatus{ - Tables: map[int64]*model.TableReplicaInfo{45: {StartTs: 421980685886554116}}, - }, - TaskPosition: &model.TaskPosition{CheckPointTs: 421980720003809281, ResolvedTs: 421980720003809281}, - Workload: model.TaskWorkload{45: {Workload: 1}}, - }, - }, - { // testing value is nil - changefeedID: "test1", - captureID: "6bbc01c8-0605-4f86-a0f9-b3119109b225", - updateKey: []string{ - "/tidb/cdc/changefeed/info/test1", - "/tidb/cdc/job/test1", - "/tidb/cdc/task/position/6bbc01c8-0605-4f86-a0f9-b3119109b225/test1", - "/tidb/cdc/task/status/6bbc01c8-0605-4f86-a0f9-b3119109b225/test1", - "/tidb/cdc/task/workload/6bbc01c8-0605-4f86-a0f9-b3119109b225/test1", - "/tidb/cdc/capture/6bbc01c8-0605-4f86-a0f9-b3119109b225", - "/tidb/cdc/changefeed/info/test1", - "/tidb/cdc/job/test1", - "/tidb/cdc/task/position/6bbc01c8-0605-4f86-a0f9-b3119109b225/test1", - "/tidb/cdc/task/status/6bbc01c8-0605-4f86-a0f9-b3119109b225/test1", - "/tidb/cdc/task/workload/6bbc01c8-0605-4f86-a0f9-b3119109b225/test1", - "/tidb/cdc/capture/6bbc01c8-0605-4f86-a0f9-b3119109b225", - }, - updateValue: []string{ - `{"sink-uri":"blackhole://","opts":{},"create-time":"2020-02-02T00:00:00.000000+00:00","start-ts":421980685886554116,"target-ts":0,"admin-job-type":0,"sort-engine":"memory","config":{"case-sensitive":true,"enable-old-value":false,"force-replicate":false,"check-gc-safe-point":true,"filter":{"rules":["*.*"],"ignore-txn-start-ts":null,"ddl-allow-list":null},"mounter":{"worker-num":16},"sink":{"dispatchers":null,"protocol":"default"},"cyclic-replication":{"enable":false,"replica-id":0,"filter-replica-ids":null,"id-buckets":0,"sync-ddl":false},"scheduler":{"type":"table-number","polling-time":-1}},"state":"normal","history":null,"error":null,"sync-point-enabled":false,"sync-point-interval":600000000000}`, - `{"resolved-ts":421980720003809281,"checkpoint-ts":421980719742451713,"admin-job-type":0}`, - `{"checkpoint-ts":421980720003809281,"resolved-ts":421980720003809281,"count":0,"error":null}`, - `{"tables":{"45":{"start-ts":421980685886554116,"mark-table-id":0}},"operation":null,"admin-job-type":0}`, - `{"45":{"workload":1}}`, - `{"id":"6bbc01c8-0605-4f86-a0f9-b3119109b225","address":"127.0.0.1:8300"}`, - ``, - ``, - ``, - ``, - ``, - ``, - }, - expected: changefeedState{ - ID: "test1", - CaptureID: "6bbc01c8-0605-4f86-a0f9-b3119109b225", - Info: nil, - Status: nil, - TaskStatus: nil, - TaskPosition: nil, - Workload: nil, - }, - }, - { // testing the same key case - changefeedID: "test1", - captureID: "6bbc01c8-0605-4f86-a0f9-b3119109b225", - updateKey: []string{ - "/tidb/cdc/task/status/6bbc01c8-0605-4f86-a0f9-b3119109b225/test1", - "/tidb/cdc/task/status/6bbc01c8-0605-4f86-a0f9-b3119109b225/test1", - "/tidb/cdc/task/status/6bbc01c8-0605-4f86-a0f9-b3119109b225/test1", - }, - updateValue: []string{ - `{"tables":{"45":{"start-ts":421980685886554116,"mark-table-id":0}},"operation":null,"admin-job-type":0}`, - `{"tables":{"46":{"start-ts":421980685886554116,"mark-table-id":0}},"operation":null,"admin-job-type":0}`, - `{"tables":{"47":{"start-ts":421980685886554116,"mark-table-id":0}},"operation":null,"admin-job-type":0}`, - }, - expected: changefeedState{ - ID: "test1", - CaptureID: "6bbc01c8-0605-4f86-a0f9-b3119109b225", - TaskStatus: &model.TaskStatus{ - Tables: map[int64]*model.TableReplicaInfo{47: {StartTs: 421980685886554116}}, - }, - }, - }, - } - for _, tc := range testCases { - state := newChangeFeedState(tc.changefeedID, tc.captureID) - for i, k := range tc.updateKey { - value := []byte(tc.updateValue[i]) - if len(value) == 0 { - value = nil - } - err = state.Update(util.NewEtcdKey(k), value, false) - c.Assert(err, check.IsNil) - } - c.Assert(cmp.Equal(state, &tc.expected, cmpopts.IgnoreUnexported(changefeedState{})), check.IsTrue, - check.Commentf("%s", cmp.Diff(state, &tc.expected, cmpopts.IgnoreUnexported(changefeedState{})))) - } -} - -func (s *stateSuite) TestPatchTaskPosition(c *check.C) { - defer testleak.AfterTest(c)() - state := newChangeFeedState("test1", "caputre1") - patcher := newMockReactorStatePatcher(c, state) - state.PatchTaskPosition(func(position *model.TaskPosition) (*model.TaskPosition, error) { - c.Assert(position, check.IsNil) - return &model.TaskPosition{ - CheckPointTs: 1, - }, nil - }) - patcher.applyPatches() - c.Assert(state.TaskPosition, check.DeepEquals, &model.TaskPosition{ - CheckPointTs: 1, - }) - state.PatchTaskPosition(func(position *model.TaskPosition) (*model.TaskPosition, error) { - position.CheckPointTs = 3 - return position, nil - }) - state.PatchTaskPosition(func(position *model.TaskPosition) (*model.TaskPosition, error) { - position.ResolvedTs = 2 - return position, nil - }) - patcher.applyPatches() - c.Assert(state.TaskPosition, check.DeepEquals, &model.TaskPosition{ - CheckPointTs: 3, - ResolvedTs: 2, - }) - state.PatchTaskPosition(func(position *model.TaskPosition) (*model.TaskPosition, error) { - return nil, nil - }) - patcher.applyPatches() - c.Assert(state.TaskPosition, check.IsNil) -} - -func (s *stateSuite) TestPatchTaskStatus(c *check.C) { - defer testleak.AfterTest(c)() - state := newChangeFeedState("test1", "caputre1") - patcher := newMockReactorStatePatcher(c, state) - state.PatchTaskStatus(func(status *model.TaskStatus) (*model.TaskStatus, error) { - c.Assert(status, check.IsNil) - return &model.TaskStatus{ - Tables: map[model.TableID]*model.TableReplicaInfo{45: {StartTs: 1}}, - }, nil - }) - patcher.applyPatches() - c.Assert(state.TaskStatus, check.DeepEquals, &model.TaskStatus{ - Tables: map[model.TableID]*model.TableReplicaInfo{45: {StartTs: 1}}, - }) - state.PatchTaskStatus(func(status *model.TaskStatus) (*model.TaskStatus, error) { - status.Tables[46] = &model.TableReplicaInfo{StartTs: 2} - return status, nil - }) - state.PatchTaskStatus(func(status *model.TaskStatus) (*model.TaskStatus, error) { - status.Tables[45].StartTs++ - return status, nil - }) - patcher.applyPatches() - c.Assert(state.TaskStatus, check.DeepEquals, &model.TaskStatus{ - Tables: map[model.TableID]*model.TableReplicaInfo{45: {StartTs: 2}, 46: {StartTs: 2}}, - }) - state.PatchTaskStatus(func(status *model.TaskStatus) (*model.TaskStatus, error) { - return nil, nil - }) - patcher.applyPatches() - c.Assert(state.TaskStatus, check.IsNil) -} - -func (s *stateSuite) TestPatchTaskWorkload(c *check.C) { - defer testleak.AfterTest(c)() - state := newChangeFeedState("test1", "caputre1") - patcher := newMockReactorStatePatcher(c, state) - state.PatchTaskWorkload(func(workload model.TaskWorkload) (model.TaskWorkload, error) { - c.Assert(workload, check.IsNil) - return model.TaskWorkload{45: {Workload: 1}}, nil - }) - patcher.applyPatches() - c.Assert(state.Workload, check.DeepEquals, model.TaskWorkload{45: {Workload: 1}}) - state.PatchTaskWorkload(func(workload model.TaskWorkload) (model.TaskWorkload, error) { - workload[46] = model.WorkloadInfo{Workload: 2} - return workload, nil - }) - state.PatchTaskWorkload(func(workload model.TaskWorkload) (model.TaskWorkload, error) { - workload[45] = model.WorkloadInfo{Workload: 3} - return workload, nil - }) - patcher.applyPatches() - c.Assert(state.Workload, check.DeepEquals, model.TaskWorkload{45: {Workload: 3}, 46: {Workload: 2}}) - state.PatchTaskWorkload(func(workload model.TaskWorkload) (model.TaskWorkload, error) { - return nil, nil - }) - patcher.applyPatches() - c.Assert(state.Workload, check.IsNil) -} - -func (s *stateSuite) TestGlobalStateUpdate(c *check.C) { - defer testleak.AfterTest(c)() - testCases := []struct { - captureID string - updateKey []string - updateValue []string - expected globalState - }{ - { // common case - captureID: "6bbc01c8-0605-4f86-a0f9-b3119109b225", - updateKey: []string{ - "/tidb/cdc/task/position/6bbc01c8-0605-4f86-a0f9-b3119109b225/test1", - "/tidb/cdc/task/workload/6bbc01c8-0605-4f86-a0f9-b3119109b225/test2", - }, - updateValue: []string{ - `{"resolved-ts":421980720003809281,"checkpoint-ts":421980719742451713,"admin-job-type":0}`, - `{"45":{"workload":1}}`, - }, - expected: globalState{ - CaptureID: "6bbc01c8-0605-4f86-a0f9-b3119109b225", - Changefeeds: map[model.ChangeFeedID]*changefeedState{ - "test1": { - ID: "test1", - CaptureID: "6bbc01c8-0605-4f86-a0f9-b3119109b225", - TaskPosition: &model.TaskPosition{CheckPointTs: 421980719742451713, ResolvedTs: 421980720003809281}, - }, - "test2": { - ID: "test2", - CaptureID: "6bbc01c8-0605-4f86-a0f9-b3119109b225", - Workload: model.TaskWorkload{45: {Workload: 1}}, - }, - }, - }, - }, { // testing captureID not match - captureID: "6bbc01c8-0605-4f86-a0f9-b3119109b225", - updateKey: []string{ - "/tidb/cdc/task/position/6bbc01c8-0605-4f86-a0f9-b3119109b226/test1", - "/tidb/cdc/task/workload/6bbc01c8-0605-4f86-a0f9-b3119109b226/test2", - }, - updateValue: []string{ - `{"resolved-ts":421980720003809281,"checkpoint-ts":421980719742451713,"admin-job-type":0}`, - `{"45":{"workload":1}}`, - }, - expected: globalState{ - CaptureID: "6bbc01c8-0605-4f86-a0f9-b3119109b225", - Changefeeds: map[model.ChangeFeedID]*changefeedState{}, - }, - }, { // testing remove changefeed - captureID: "6bbc01c8-0605-4f86-a0f9-b3119109b225", - updateKey: []string{ - "/tidb/cdc/task/position/6bbc01c8-0605-4f86-a0f9-b3119109b225/test1", - "/tidb/cdc/task/workload/6bbc01c8-0605-4f86-a0f9-b3119109b225/test2", - "/tidb/cdc/task/position/6bbc01c8-0605-4f86-a0f9-b3119109b225/test1", - }, - updateValue: []string{ - `{"resolved-ts":421980720003809281,"checkpoint-ts":421980719742451713,"admin-job-type":0}`, - `{"45":{"workload":1}}`, - "", - }, - expected: globalState{ - CaptureID: "6bbc01c8-0605-4f86-a0f9-b3119109b225", - Changefeeds: map[model.ChangeFeedID]*changefeedState{ - "test2": { - ID: "test2", - CaptureID: "6bbc01c8-0605-4f86-a0f9-b3119109b225", - Workload: model.TaskWorkload{45: {Workload: 1}}, - }, - }, - }, - }, - } - for _, tc := range testCases { - state := NewGlobalState(tc.captureID) - for i, k := range tc.updateKey { - value := []byte(tc.updateValue[i]) - if len(value) == 0 { - value = nil - } - err := state.Update(util.NewEtcdKey(k), value, false) - c.Assert(err, check.IsNil) - } - c.Assert(cmp.Equal(state, &tc.expected, cmp.AllowUnexported(globalState{}), cmpopts.IgnoreUnexported(changefeedState{})), check.IsTrue, - check.Commentf("%s", cmp.Diff(state, &tc.expected, cmp.AllowUnexported(globalState{}), cmpopts.IgnoreUnexported(changefeedState{})))) - } -} diff --git a/cdc/puller/puller.go b/cdc/puller/puller.go index d284b08869a..93fd4f8ec71 100644 --- a/cdc/puller/puller.go +++ b/cdc/puller/puller.go @@ -160,12 +160,18 @@ func (p *pullerImpl) Run(ctx context.Context) error { lastResolvedTs := p.checkpointTs g.Go(func() error { output := func(raw *model.RawKVEntry) error { + // even after https://github.com/pingcap/ticdc/pull/2038, kv client + // could still miss region change notification, which leads to resolved + // ts update missing in puller, however resolved ts fallback here can + // be ignored since no late data is received and the guarantee of + // resolved ts is not broken. if raw.CRTs < p.resolvedTs || (raw.CRTs == p.resolvedTs && raw.OpType != model.OpTypeResolved) { - log.Panic("The CRTs must be greater than the resolvedTs", + log.Warn("The CRTs is fallen back in pulelr", zap.Reflect("row", raw), zap.Uint64("CRTs", raw.CRTs), zap.Uint64("resolvedTs", p.resolvedTs), zap.Int64("tableID", tableID)) + return nil } select { case <-ctx.Done(): diff --git a/cdc/puller/sorter/backend_pool.go b/cdc/puller/sorter/backend_pool.go index 61997752119..76509495be0 100644 --- a/cdc/puller/sorter/backend_pool.go +++ b/cdc/puller/sorter/backend_pool.go @@ -112,17 +112,27 @@ func newBackEndPool(dir string, captureAddr string) (*backEndPool, error) { // update memPressure m, err := memory.Get() + + failpoint.Inject("getMemoryPressureFails", func() { + m = nil + err = errors.New("injected get memory pressure failure") + }) + if err != nil { failpoint.Inject("sorterDebug", func() { log.Panic("unified sorter: getting system memory usage failed", zap.Error(err)) }) log.Warn("unified sorter: getting system memory usage failed", zap.Error(err)) + // Reports a 100% memory pressure, so that the backEndPool will allocate fileBackEnds. + // We default to fileBackEnds because they are unlikely to cause OOMs. If IO errors are + // encountered, we can fail gracefully. + atomic.StoreInt32(&ret.memPressure, 100) + } else { + memPressure := m.Used * 100 / m.Total + atomic.StoreInt32(&ret.memPressure, int32(memPressure)) } - memPressure := m.Used * 100 / m.Total - atomic.StoreInt32(&ret.memPressure, int32(memPressure)) - if memPressure := ret.memoryPressure(); memPressure > 50 { log.Debug("unified sorter: high memory pressure", zap.Int32("memPressure", memPressure), zap.Int64("usedBySorter", ret.sorterMemoryUsage())) diff --git a/cdc/puller/sorter/backend_pool_test.go b/cdc/puller/sorter/backend_pool_test.go index 8f4cec331bf..20852f2ad3e 100644 --- a/cdc/puller/sorter/backend_pool_test.go +++ b/cdc/puller/sorter/backend_pool_test.go @@ -107,7 +107,7 @@ func (s *backendPoolSuite) TestDirectoryBadPermission(c *check.C) { err := os.Chmod(dir, 0o311) // no permission to `ls` c.Assert(err, check.IsNil) - conf := config.GetDefaultServerConfig() + conf := config.GetGlobalServerConfig() conf.Sorter.MaxMemoryPressure = 0 // force using files backEndPool, err := newBackEndPool(dir, "") @@ -141,6 +141,7 @@ func (s *backendPoolSuite) TestCleanUpSelf(c *check.C) { err = failpoint.Enable("github.com/pingcap/ticdc/cdc/puller/sorter/memoryPressureInjectPoint", "return(100)") c.Assert(err, check.IsNil) + defer failpoint.Disable("github.com/pingcap/ticdc/cdc/puller/sorter/memoryPressureInjectPoint") //nolint:errcheck backEndPool, err := newBackEndPool("/tmp/sorter", "") c.Assert(err, check.IsNil) @@ -296,3 +297,33 @@ func (s *backendPoolSuite) TestCleanUpStaleLockNoPermission(c *check.C) { mockP.assertFilesExist(c) } + +// TestGetMemoryPressureFailure verifies that the backendPool can handle gracefully failures that happen when +// getting the current system memory pressure. Such a failure is usually caused by a lack of file descriptor quota +// set by the operating system. +func (s *backendPoolSuite) TestGetMemoryPressureFailure(c *check.C) { + defer testleak.AfterTest(c)() + + err := failpoint.Enable("github.com/pingcap/ticdc/cdc/puller/sorter/getMemoryPressureFails", "return(true)") + c.Assert(err, check.IsNil) + defer failpoint.Disable("github.com/pingcap/ticdc/cdc/puller/sorter/getMemoryPressureFails") //nolint:errcheck + + dir := c.MkDir() + backEndPool, err := newBackEndPool(dir, "") + c.Assert(err, check.IsNil) + c.Assert(backEndPool, check.NotNil) + defer backEndPool.terminate() + + after := time.After(time.Second * 20) + tick := time.Tick(time.Second * 1) + for { + select { + case <-after: + c.Fatal("TestGetMemoryPressureFailure timed out") + case <-tick: + if backEndPool.memoryPressure() == 100 { + return + } + } + } +} diff --git a/cdc/puller/sorter/file_backend.go b/cdc/puller/sorter/file_backend.go index f08deecfd3b..e56735f0e1a 100644 --- a/cdc/puller/sorter/file_backend.go +++ b/cdc/puller/sorter/file_backend.go @@ -28,7 +28,7 @@ import ( ) const ( - fileBufferSize = 1 * 1024 * 1024 // 1MB + fileBufferSize = 32 * 1024 // 32KB fileMagic = 0x12345678 numFileEntriesOffset = 4 blockMagic = 0xbeefbeef diff --git a/cdc/puller/sorter/heap_sorter.go b/cdc/puller/sorter/heap_sorter.go index a82c1cde67d..3612a7ecdfd 100644 --- a/cdc/puller/sorter/heap_sorter.go +++ b/cdc/puller/sorter/heap_sorter.go @@ -33,6 +33,8 @@ import ( const ( flushRateLimitPerSecond = 10 + sortHeapCapacity = 32 + sortHeapInputChSize = 1024 ) type flushTask struct { @@ -84,9 +86,9 @@ type heapSorter struct { func newHeapSorter(id int, out chan *flushTask) *heapSorter { return &heapSorter{ id: id, - inputCh: make(chan *model.PolymorphicEvent, 1024*1024), + inputCh: make(chan *model.PolymorphicEvent, sortHeapInputChSize), outputCh: out, - heap: make(sortHeap, 0, 65536), + heap: make(sortHeap, 0, sortHeapCapacity), canceller: new(asyncCanceller), } } @@ -159,7 +161,7 @@ func (h *heapSorter) flush(ctx context.Context, maxResolvedTs uint64) error { return nil } oldHeap = h.heap - h.heap = make(sortHeap, 0, 65536) + h.heap = make(sortHeap, 0, sortHeapCapacity) } else { task.dealloc = func() error { task.markDeallocated() diff --git a/cdc/puller/sorter/sorter_test.go b/cdc/puller/sorter/sorter_test.go index cdffb0f3f02..8a05d9136b7 100644 --- a/cdc/puller/sorter/sorter_test.go +++ b/cdc/puller/sorter/sorter_test.go @@ -354,7 +354,7 @@ func (s *sorterSuite) TestSorterIOError(c *check.C) { finishedCh := make(chan struct{}) go func() { - err := testSorter(ctx, c, sorter, 100000, true) + err := testSorter(ctx, c, sorter, 10000, true) c.Assert(err, check.ErrorMatches, ".*injected alloc error.*") close(finishedCh) }() @@ -366,6 +366,7 @@ func (s *sorterSuite) TestSorterIOError(c *check.C) { case <-finishedCh: } + UnifiedSorterCleanUp() _ = failpoint.Disable("github.com/pingcap/ticdc/cdc/puller/sorter/InjectErrorBackEndAlloc") // enable the failpoint to simulate backEnd write error (usually would happen when writing to a file) err = failpoint.Enable("github.com/pingcap/ticdc/cdc/puller/sorter/InjectErrorBackEndWrite", "return(true)") @@ -374,9 +375,13 @@ func (s *sorterSuite) TestSorterIOError(c *check.C) { _ = failpoint.Disable("github.com/pingcap/ticdc/cdc/puller/sorter/InjectErrorBackEndWrite") }() + // recreate the sorter + sorter, err = NewUnifiedSorter("/tmp/sorter", "test-cf", "test", 0, "0.0.0.0:0") + c.Assert(err, check.IsNil) + finishedCh = make(chan struct{}) go func() { - err := testSorter(ctx, c, sorter, 100000, true) + err := testSorter(ctx, c, sorter, 10000, true) c.Assert(err, check.ErrorMatches, ".*injected write error.*") close(finishedCh) }() @@ -429,7 +434,7 @@ func (s *sorterSuite) TestSorterErrorReportCorrect(c *check.C) { finishedCh := make(chan struct{}) go func() { - err := testSorter(ctx, c, sorter, 100000, true) + err := testSorter(ctx, c, sorter, 10000, true) c.Assert(err, check.ErrorMatches, ".*injected alloc error.*") close(finishedCh) }() diff --git a/cdc/puller/sorter/unified_sorter.go b/cdc/puller/sorter/unified_sorter.go index aa562cdca9a..e95f55b09e8 100644 --- a/cdc/puller/sorter/unified_sorter.go +++ b/cdc/puller/sorter/unified_sorter.go @@ -31,6 +31,17 @@ import ( "golang.org/x/sync/errgroup" ) +const ( + inputChSize = 128 + outputChSize = 128 + heapCollectChSize = 128 // this should be not be too small, to guarantee IO concurrency + // maxOpenHeapNum is the maximum number of allowed pending chunks in memory OR on-disk. + // This constant is a worst case upper limit, and setting a large number DOES NOT imply actually + // allocating these resources. This constant is PER TABLE. + // TODO refactor this out + maxOpenHeapNum = 1280000 +) + // UnifiedSorter provides both sorting in memory and in file. Memory pressure is used to determine which one to use. type UnifiedSorter struct { inputCh chan *model.PolymorphicEvent @@ -103,8 +114,8 @@ func NewUnifiedSorter( lazyInitWorkerPool() return &UnifiedSorter{ - inputCh: make(chan *model.PolymorphicEvent, 128000), - outputCh: make(chan *model.PolymorphicEvent, 128000), + inputCh: make(chan *model.PolymorphicEvent, inputChSize), + outputCh: make(chan *model.PolymorphicEvent, outputChSize), dir: dir, pool: pool, metricsInfo: &metricsInfo{ @@ -146,7 +157,7 @@ func (s *UnifiedSorter) Run(ctx context.Context) error { numConcurrentHeaps := sorterConfig.NumConcurrentWorker errg, subctx := errgroup.WithContext(ctx) - heapSorterCollectCh := make(chan *flushTask, 4096) + heapSorterCollectCh := make(chan *flushTask, heapCollectChSize) // mergerCleanUp will consumer the remaining elements in heapSorterCollectCh to prevent any FD leak. defer mergerCleanUp(heapSorterCollectCh) @@ -207,8 +218,7 @@ func (s *UnifiedSorter) Run(ctx context.Context) error { nextSorterID := 0 for { - // tentative value 1280000 - for atomic.LoadInt64(&mergerBufLen) > 1280000 { + for atomic.LoadInt64(&mergerBufLen) > maxOpenHeapNum { after := time.After(1 * time.Second) select { case <-subctx.Done(): diff --git a/cdc/server.go b/cdc/server.go index 70cc2c8e120..ce2be03ac9d 100644 --- a/cdc/server.go +++ b/cdc/server.go @@ -23,6 +23,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" + "github.com/pingcap/ticdc/cdc/capture" "github.com/pingcap/ticdc/cdc/kv" "github.com/pingcap/ticdc/cdc/puller/sorter" "github.com/pingcap/ticdc/pkg/config" @@ -30,10 +31,14 @@ import ( "github.com/pingcap/ticdc/pkg/httputil" "github.com/pingcap/ticdc/pkg/util" "github.com/pingcap/ticdc/pkg/version" + tidbkv "github.com/pingcap/tidb/kv" "github.com/prometheus/client_golang/prometheus" pd "github.com/tikv/pd/client" + "go.etcd.io/etcd/clientv3" "go.etcd.io/etcd/mvcc" + "go.etcd.io/etcd/pkg/logutil" "go.uber.org/zap" + "go.uber.org/zap/zapcore" "golang.org/x/sync/errgroup" "golang.org/x/time/rate" "google.golang.org/grpc" @@ -45,17 +50,19 @@ const ( // DefaultCDCGCSafePointTTL is the default value of cdc gc safe-point ttl, specified in seconds. DefaultCDCGCSafePointTTL = 24 * 60 * 60 - - defaultCaptureSessionTTL = 10 ) // Server is the capture server type Server struct { + captureV2 *capture.Capture + capture *Capture owner *Owner ownerLock sync.RWMutex statusServer *http.Server pdClient pd.Client + etcdClient *kv.CDCEtcdClient + kvStorage tidbkv.Storage pdEndpoints []string } @@ -100,7 +107,39 @@ func (s *Server) Run(ctx context.Context) error { return cerror.WrapError(cerror.ErrServerNewPDClient, err) } s.pdClient = pdClient - + if config.NewReplicaImpl { + tlsConfig, err := conf.Security.ToTLSConfig() + if err != nil { + return errors.Trace(err) + } + logConfig := logutil.DefaultZapLoggerConfig + logConfig.Level = zap.NewAtomicLevelAt(zapcore.ErrorLevel) + etcdCli, err := clientv3.New(clientv3.Config{ + Endpoints: s.pdEndpoints, + TLS: tlsConfig, + Context: ctx, + LogConfig: &logConfig, + DialTimeout: 5 * time.Second, + DialOptions: []grpc.DialOption{ + grpcTLSOption, + grpc.WithBlock(), + grpc.WithConnectParams(grpc.ConnectParams{ + Backoff: backoff.Config{ + BaseDelay: time.Second, + Multiplier: 1.1, + Jitter: 0.1, + MaxDelay: 3 * time.Second, + }, + MinConnectTimeout: 3 * time.Second, + }), + }, + }) + if err != nil { + return errors.Annotate(cerror.WrapError(cerror.ErrNewCaptureFailed, err), "new etcd client") + } + etcdClient := kv.NewCDCEtcdClient(ctx, etcdCli) + s.etcdClient = &etcdClient + } // To not block CDC server startup, we need to warn instead of error // when TiKV is incompatible. errorTiKVIncompatible := false @@ -123,7 +162,12 @@ func (s *Server) Run(ctx context.Context) error { log.Warn("kv store close failed", zap.Error(err)) } }() + s.kvStorage = kvStore ctx = util.PutKVStorageInCtx(ctx, kvStore) + if config.NewReplicaImpl { + s.captureV2 = capture.NewCapture(s.pdClient, s.kvStorage, s.etcdClient) + return s.run(ctx) + } // When a capture suicided, restart it for { if err := s.run(ctx); cerror.ErrCaptureSuicide.NotEqual(err) { @@ -187,7 +231,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)) @@ -240,36 +284,38 @@ func (s *Server) etcdHealthChecker(ctx context.Context) error { } func (s *Server) run(ctx context.Context) (err error) { - conf := config.GetGlobalServerConfig() - - opts := &captureOpts{ - flushCheckpointInterval: time.Duration(conf.ProcessorFlushInterval), - captureSessionTTL: defaultCaptureSessionTTL, - } - capture, err := NewCapture(ctx, s.pdEndpoints, s.pdClient, conf.Security, conf.AdvertiseAddr, opts) - if err != nil { - return err + if !config.NewReplicaImpl { + kvStorage := util.KVStorageFromCtx(ctx) + capture, err := NewCapture(ctx, s.pdEndpoints, s.pdClient, kvStorage) + if err != nil { + return err + } + s.capture = capture + s.etcdClient = &capture.etcdClient } - s.capture = capture ctx, cancel := context.WithCancel(ctx) defer cancel() wg, cctx := errgroup.WithContext(ctx) + if config.NewReplicaImpl { + wg.Go(func() error { + return s.captureV2.Run(cctx) + }) + } else { + wg.Go(func() error { + return s.campaignOwnerLoop(cctx) + }) - wg.Go(func() error { - return s.campaignOwnerLoop(cctx) - }) - - wg.Go(func() error { - return sorter.RunWorkerPool(cctx) - }) - + wg.Go(func() error { + return s.capture.Run(cctx) + }) + } wg.Go(func() error { return s.etcdHealthChecker(cctx) }) wg.Go(func() error { - return s.capture.Run(cctx) + return sorter.RunWorkerPool(cctx) }) return wg.Wait() @@ -288,6 +334,9 @@ func (s *Server) Close() { } closeCancel() } + if s.captureV2 != nil { + s.captureV2.AsyncClose() + } if s.statusServer != nil { err := s.statusServer.Close() if err != nil { diff --git a/cdc/sink/cdclog/s3.go b/cdc/sink/cdclog/s3.go index 923212803f5..5d42787f5e9 100644 --- a/cdc/sink/cdclog/s3.go +++ b/cdc/sink/cdclog/s3.go @@ -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 @@ -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{ diff --git a/cdc/sink/mq.go b/cdc/sink/mq.go index 07fd0cb9803..009667c08e4 100644 --- a/cdc/sink/mq.go +++ b/cdc/sink/mq.go @@ -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) diff --git a/cdc/sink/mysql.go b/cdc/sink/mysql.go index 10008810de7..177a4aa6136 100644 --- a/cdc/sink/mysql.go +++ b/cdc/sink/mysql.go @@ -637,6 +637,7 @@ func (s *mysqlSink) createSinkWorkers(ctx context.Context) error { select { case s.errCh <- err: default: + log.Info("mysql sink receives redundant error", zap.Error(err)) } } }() @@ -1339,7 +1340,7 @@ func (s *mysqlSyncpointStore) CreateSynctable(ctx context.Context) error { if err != nil { err2 := tx.Rollback() if err2 != nil { - log.Error(cerror.WrapError(cerror.ErrMySQLTxnError, err2).Error()) + log.Error("failed to create syncpoint table", zap.Error(cerror.WrapError(cerror.ErrMySQLTxnError, err2))) } return cerror.WrapError(cerror.ErrMySQLTxnError, err) } @@ -1347,7 +1348,7 @@ func (s *mysqlSyncpointStore) CreateSynctable(ctx context.Context) error { if err != nil { err2 := tx.Rollback() if err2 != nil { - log.Error(cerror.WrapError(cerror.ErrMySQLTxnError, err2).Error()) + log.Error("failed to create syncpoint table", zap.Error(cerror.WrapError(cerror.ErrMySQLTxnError, err2))) } return cerror.WrapError(cerror.ErrMySQLTxnError, err) } @@ -1355,7 +1356,7 @@ func (s *mysqlSyncpointStore) CreateSynctable(ctx context.Context) error { if err != nil { err2 := tx.Rollback() if err2 != nil { - log.Error(cerror.WrapError(cerror.ErrMySQLTxnError, err2).Error()) + log.Error("failed to create syncpoint table", zap.Error(cerror.WrapError(cerror.ErrMySQLTxnError, err2))) } return cerror.WrapError(cerror.ErrMySQLTxnError, err) } @@ -1376,7 +1377,7 @@ func (s *mysqlSyncpointStore) SinkSyncpoint(ctx context.Context, id string, chec log.Info("sync table: get tidb_current_ts err") err2 := tx.Rollback() if err2 != nil { - log.Error(cerror.WrapError(cerror.ErrMySQLTxnError, err2).Error()) + log.Error("failed to write syncpoint table", zap.Error(cerror.WrapError(cerror.ErrMySQLTxnError, err2))) } return cerror.WrapError(cerror.ErrMySQLTxnError, err) } @@ -1384,7 +1385,7 @@ func (s *mysqlSyncpointStore) SinkSyncpoint(ctx context.Context, id string, chec if err != nil { err2 := tx.Rollback() if err2 != nil { - log.Error(cerror.WrapError(cerror.ErrMySQLTxnError, err2).Error()) + log.Error("failed to write syncpoint table", zap.Error(cerror.WrapError(cerror.ErrMySQLTxnError, err2))) } return cerror.WrapError(cerror.ErrMySQLTxnError, err) } diff --git a/cdc/sink/producer/kafka/kafka.go b/cdc/sink/producer/kafka/kafka.go index f5c78b7c067..380960ebdbf 100644 --- a/cdc/sink/producer/kafka/kafka.go +++ b/cdc/sink/producer/kafka/kafka.go @@ -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") @@ -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 @@ -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(`[\?:,"]`) ) @@ -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 diff --git a/cdc/sink/producer/kafka/kafka_test.go b/cdc/sink/producer/kafka/kafka_test.go index 8c8c46bcfed..a93bfb029c5 100644 --- a/cdc/sink/producer/kafka/kafka_test.go +++ b/cdc/sink/producer/kafka/kafka_test.go @@ -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) diff --git a/cdc/sink/sink.go b/cdc/sink/sink.go index e5b30866c39..36327474b10 100644 --- a/cdc/sink/sink.go +++ b/cdc/sink/sink.go @@ -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 diff --git a/cmd/client_changefeed.go b/cmd/client_changefeed.go index 6c30830ab84..02936e4d64a 100644 --- a/cmd/client_changefeed.go +++ b/cmd/client_changefeed.go @@ -259,7 +259,7 @@ func verifyChangefeedParamers(ctx context.Context, cmd *cobra.Command, isCreate } startTs = oracle.ComposeTS(ts, logical) } - if err := verifyStartTs(ctx, startTs); err != nil { + if err := verifyStartTs(ctx, changefeedID, startTs); err != nil { return nil, err } if err := confirmLargeDataGap(ctx, cmd, startTs); err != nil { diff --git a/cmd/server.go b/cmd/server.go index 478c10ce06d..8c5038dacfb 100644 --- a/cmd/server.go +++ b/cmd/server.go @@ -19,6 +19,7 @@ import ( "time" "github.com/pingcap/errors" + "github.com/pingcap/failpoint" "github.com/pingcap/log" "github.com/pingcap/ticdc/cdc" "github.com/pingcap/ticdc/cdc/puller/sorter" @@ -105,6 +106,15 @@ func runEServer(cmd *cobra.Command, args []string) error { ctx = util.PutCaptureAddrInCtx(ctx, conf.AdvertiseAddr) version.LogVersionInfo() + if util.FailpointBuild { + for _, path := range failpoint.List() { + status, err := failpoint.Status(path) + if err != nil { + log.Error("fail to get failpoint status", zap.Error(err)) + } + log.Info("failpoint enabled", zap.String("path", path), zap.String("status", status)) + } + } logHTTPProxies() server, err := cdc.NewServer(strings.Split(serverPdAddr, ",")) diff --git a/cmd/server_test.go b/cmd/server_test.go index ef0bc440cc9..c79565af172 100644 --- a/cmd/server_test.go +++ b/cmd/server_test.go @@ -101,6 +101,7 @@ func (s *serverSuite) TestLoadAndVerifyServerConfig(c *check.C) { LogLevel: "debug", GcTTL: 10, TZ: "UTC", + CaptureSessionTTL: 10, OwnerFlushInterval: config.TomlDuration(150 * time.Millisecond), ProcessorFlushInterval: config.TomlDuration(150 * time.Millisecond), Sorter: &config.SorterConfig{ @@ -131,6 +132,7 @@ log-level = "warn" gc-ttl = 500 tz = "US" +capture-session-ttl = 10 owner-flush-interval = "600ms" processor-flush-interval = "600ms" @@ -157,6 +159,7 @@ sort-dir = "/tmp/just_a_test" LogLevel: "warn", GcTTL: 500, TZ: "US", + CaptureSessionTTL: 10, OwnerFlushInterval: config.TomlDuration(600 * time.Millisecond), ProcessorFlushInterval: config.TomlDuration(600 * time.Millisecond), Sorter: &config.SorterConfig{ @@ -206,6 +209,7 @@ cert-allowed-cn = ["dd","ee"] LogLevel: "debug", GcTTL: 10, TZ: "UTC", + CaptureSessionTTL: 10, OwnerFlushInterval: config.TomlDuration(150 * time.Millisecond), ProcessorFlushInterval: config.TomlDuration(150 * time.Millisecond), Sorter: &config.SorterConfig{ diff --git a/cmd/util.go b/cmd/util.go index d1b0cdb0e52..63d7581d462 100644 --- a/cmd/util.go +++ b/cmd/util.go @@ -217,11 +217,11 @@ func jsonPrint(cmd *cobra.Command, v interface{}) error { return nil } -func verifyStartTs(ctx context.Context, startTs uint64) error { +func verifyStartTs(ctx context.Context, changefeedID string, startTs uint64) error { if disableGCSafePointCheck { return nil } - return util.CheckSafetyOfStartTs(ctx, pdCli, startTs) + return util.CheckSafetyOfStartTs(ctx, pdCli, changefeedID, startTs) } func verifyTargetTs(ctx context.Context, startTs, targetTs uint64) error { diff --git a/errors.toml b/errors.toml index a980dc8e539..bf5a09d3779 100755 --- a/errors.toml +++ b/errors.toml @@ -11,7 +11,7 @@ error = ''' stop processor by admin command ''' -["CDC:ErrAsyncBroadcaseNotSupport"] +["CDC:ErrAsyncBroadcastNotSupport"] error = ''' Async broadcasts not supported ''' @@ -156,6 +156,11 @@ error = ''' decode row data to datum failed ''' +["CDC:ErrEmitCheckpointTsFailed"] +error = ''' +emit checkpoint ts failed +''' + ["CDC:ErrEncodeFailed"] error = ''' encode failed: %s @@ -256,6 +261,11 @@ error = ''' event is larger than the total memory quota, size: %d, quota: %d ''' +["CDC:ErrGCTTLExceeded"] +error = ''' +the checkpoint-ts(%d) lag of the changefeed(%s) %d has exceeded the GC TTL +''' + ["CDC:ErrGRPCDialFailed"] error = ''' grpc dial failed @@ -361,7 +371,7 @@ error = ''' kafka async send message failed ''' -["CDC:ErrKafkaFlushUnfished"] +["CDC:ErrKafkaFlushUnfinished"] error = ''' flush not finished before producer close ''' @@ -396,6 +406,11 @@ error = ''' kafka send message failed ''' +["CDC:ErrLeaseExpired"] +error = ''' +owner lease expired +''' + ["CDC:ErrLeaseTimeout"] error = ''' owner lease timeout @@ -491,6 +506,11 @@ error = ''' received event regionID %v, requestID %v from %v, but neither pending region nor running region was found ''' +["CDC:ErrNotOwner"] +error = ''' +this capture is not a owner +''' + ["CDC:ErrOldValueNotEnabled"] error = ''' old value is not enabled @@ -506,6 +526,11 @@ error = ''' owner campaign key deleted ''' +["CDC:ErrOwnerChangedUnexpectedly"] +error = ''' +owner changed unexpectedly +''' + ["CDC:ErrOwnerChangefeedNotFound"] error = ''' changefeed %s not found in owner cache @@ -526,6 +551,11 @@ error = ''' owner sort dir ''' +["CDC:ErrOwnerUnknown"] +error = ''' +owner running unknown error +''' + ["CDC:ErrPDBatchLoadRegions"] error = ''' pd batch load regions failed @@ -596,7 +626,7 @@ error = ''' resolve locks failed ''' -["CDC:ErrS3SinkInitialzie"] +["CDC:ErrS3SinkInitialize"] error = ''' new s3 sink ''' @@ -696,6 +726,11 @@ error = ''' this api supports POST method only ''' +["CDC:ErrTableListenReplicated"] +error = ''' +A table is being replicated by at least two processors(%s, %s), please report a bug +''' + ["CDC:ErrTableProcessorStoppedSafely"] error = ''' table processor stopped safely diff --git a/metrics/grafana/ticdc.json b/metrics/grafana/ticdc.json index fb46ee4dfee..5b5065ec7fc 100644 --- a/metrics/grafana/ticdc.json +++ b/metrics/grafana/ticdc.json @@ -2613,6 +2613,203 @@ "align": false, "alignLevel": null } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "fill": 1, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 54 + }, + "id": 149, + "legend": { + "avg": false, + "current": false, + "max": false, + "min": false, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(ticdc_processor_table_memory_consumption{changefeed =~ \"$changefeed\", capture =~ \"$capture\"}) by (capture)", + "format": "time_series", + "intervalFactor": 1, + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Processor Memory Consumption Per Capture", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "bytes", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "columns": [], + "datasource": "${DS_TEST-CLUSTER}", + "fontSize": "100%", + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 54 + }, + "id": 151, + "links": [], + "pageSize": null, + "scroll": true, + "showHeader": true, + "sort": { + "col": 0, + "desc": true + }, + "styles": [ + { + "alias": "", + "colorMode": null, + "colors": [ + "rgba(245, 54, 54, 0.9)", + "rgba(237, 129, 40, 0.89)", + "rgba(50, 172, 45, 0.97)" + ], + "decimals": 2, + "pattern": "changefeed", + "thresholds": [], + "type": "string", + "unit": "short" + }, + { + "alias": "", + "colorMode": null, + "colors": [ + "rgba(245, 54, 54, 0.9)", + "rgba(237, 129, 40, 0.89)", + "rgba(50, 172, 45, 0.97)" + ], + "dateFormat": "YYYY-MM-DD HH:mm:ss", + "decimals": 2, + "mappingType": 1, + "pattern": "table", + "thresholds": [], + "type": "string", + "unit": "short" + }, + { + "alias": "", + "colorMode": null, + "colors": [ + "rgba(245, 54, 54, 0.9)", + "rgba(237, 129, 40, 0.89)", + "rgba(50, 172, 45, 0.97)" + ], + "dateFormat": "YYYY-MM-DD HH:mm:ss", + "decimals": 2, + "mappingType": 1, + "pattern": "instance", + "thresholds": [], + "type": "string", + "unit": "short" + }, + { + "alias": "memory consumption", + "colorMode": null, + "colors": [ + "rgba(245, 54, 54, 0.9)", + "rgba(237, 129, 40, 0.89)", + "rgba(50, 172, 45, 0.97)" + ], + "dateFormat": "YYYY-MM-DD HH:mm:ss", + "decimals": 2, + "mappingType": 1, + "pattern": "Value", + "thresholds": [], + "type": "number", + "unit": "bytes" + }, + { + "alias": "", + "colorMode": null, + "colors": [ + "rgba(245, 54, 54, 0.9)", + "rgba(237, 129, 40, 0.89)", + "rgba(50, 172, 45, 0.97)" + ], + "dateFormat": "YYYY-MM-DD HH:mm:ss", + "decimals": 2, + "mappingType": 1, + "pattern": "/.*/", + "thresholds": [], + "type": "hidden", + "unit": "short" + } + ], + "targets": [ + { + "expr": "ticdc_processor_table_memory_consumption{changefeed =~ \"$changefeed\", capture =~ \"$capture\"}", + "format": "table", + "intervalFactor": 1, + "refId": "A" + } + ], + "timeFrom": null, + "timeShift": null, + "title": "Processor Memory Consumption Per Table", + "transform": "table", + "type": "table" } ], "title": "Changefeed", @@ -6704,7 +6901,7 @@ "multi": false, "name": "tidb_cluster", "options": [], - "query": "label_values(ticdc_processor_resolved_ts, tidb_cluster)", + "query": "label_values(go_goroutines, tidb_cluster)", "refresh": 2, "regex": "", "skipUrlSync": false, @@ -6868,5 +7065,5 @@ "timezone": "browser", "title": "Test-Cluster-TiCDC", "uid": "YiGL8hBZ1", - "version": 15 + "version": 16 } diff --git a/pkg/config/config.go b/pkg/config/config.go index ba7c7ad8f9a..e937519768c 100644 --- a/pkg/config/config.go +++ b/pkg/config/config.go @@ -33,6 +33,10 @@ import ( // new owner should be also switched on after it implemented const NewReplicaImpl = false +func init() { + StoreGlobalServerConfig(GetDefaultServerConfig()) +} + var defaultReplicaConfig = &ReplicaConfig{ CaseSensitive: true, EnableOldValue: false, @@ -139,19 +143,24 @@ func GetDefaultReplicaConfig() *ReplicaConfig { type SecurityConfig = security.Credential var defaultServerConfig = &ServerConfig{ - Addr: "127.0.0.1:8300", - AdvertiseAddr: "", - LogFile: "", - LogLevel: "info", - GcTTL: 24 * 60 * 60, // 24H - TZ: "System", + Addr: "127.0.0.1:8300", + AdvertiseAddr: "", + LogFile: "", + LogLevel: "info", + GcTTL: 24 * 60 * 60, // 24H + TZ: "System", + // The default election-timeout in PD is 3s and minimum session TTL is 5s, + // which is calculated by `math.Ceil(3 * election-timeout / 2)`, we choose + // default capture session ttl to 10s to increase robust to PD jitter, + // however it will decrease RTO when single TiCDC node error happens. + CaptureSessionTTL: 10, OwnerFlushInterval: TomlDuration(200 * time.Millisecond), ProcessorFlushInterval: TomlDuration(100 * time.Millisecond), Sorter: &SorterConfig{ NumConcurrentWorker: 4, - ChunkSizeLimit: 1024 * 1024 * 1024, // 1GB - MaxMemoryPressure: 80, - MaxMemoryConsumption: 8 * 1024 * 1024 * 1024, // 8GB + ChunkSizeLimit: 128 * 1024 * 1024, // 128MB + MaxMemoryPressure: 30, // 30% is safe on machines with memory capacity <= 16GB + MaxMemoryConsumption: 16 * 1024 * 1024 * 1024, // 16GB NumWorkerPoolGoroutine: 16, SortDir: "/tmp/cdc_sort", }, @@ -170,6 +179,8 @@ type ServerConfig struct { GcTTL int64 `toml:"gc-ttl" json:"gc-ttl"` TZ string `toml:"tz" json:"tz"` + CaptureSessionTTL int `toml:"capture-session-ttl" json:"capture-session-ttl"` + OwnerFlushInterval TomlDuration `toml:"owner-flush-interval" json:"owner-flush-interval"` ProcessorFlushInterval TomlDuration `toml:"processor-flush-interval" json:"processor-flush-interval"` @@ -241,6 +252,12 @@ func (c *ServerConfig) ValidateAndAdjust() error { return cerror.ErrInvalidServerOption.GenWithStack("empty GC TTL is not allowed") } + // 5s is minimum lease ttl in etcd(PD) + if c.CaptureSessionTTL < 5 { + log.Warn("capture session ttl too small, set to default value 10s") + c.CaptureSessionTTL = 10 + } + if c.Security != nil && c.Security.IsTLSEnabled() { var err error _, err = c.Security.ToTLSConfig() diff --git a/pkg/config/config_test.go b/pkg/config/config_test.go index 0c5869ea8ca..d429e825a8d 100644 --- a/pkg/config/config_test.go +++ b/pkg/config/config_test.go @@ -86,9 +86,9 @@ func (s *serverConfigSuite) TestMarshal(c *check.C) { conf.Sorter.ChunkSizeLimit = 999 b, err := conf.Marshal() c.Assert(err, check.IsNil) - c.Assert(b, check.Equals, `{"addr":"192.155.22.33:8887","advertise-addr":"","log-file":"","log-level":"info","gc-ttl":86400,"tz":"System","owner-flush-interval":200000000,"processor-flush-interval":100000000,"sorter":{"num-concurrent-worker":4,"chunk-size-limit":999,"max-memory-percentage":80,"max-memory-consumption":8589934592,"num-workerpool-goroutine":16,"sort-dir":"/tmp/cdc_sort"},"security":{"ca-path":"","cert-path":"","key-path":"","cert-allowed-cn":null},"per-table-memory-quota":20971520}`) + c.Assert(b, check.Equals, `{"addr":"192.155.22.33:8887","advertise-addr":"","log-file":"","log-level":"info","gc-ttl":86400,"tz":"System","capture-session-ttl":10,"owner-flush-interval":200000000,"processor-flush-interval":100000000,"sorter":{"num-concurrent-worker":4,"chunk-size-limit":999,"max-memory-percentage":30,"max-memory-consumption":17179869184,"num-workerpool-goroutine":16,"sort-dir":"/tmp/cdc_sort"},"security":{"ca-path":"","cert-path":"","key-path":"","cert-allowed-cn":null},"per-table-memory-quota":20971520}`) conf2 := new(ServerConfig) - err = conf2.Unmarshal([]byte(`{"addr":"192.155.22.33:8887","advertise-addr":"","log-file":"","log-level":"info","gc-ttl":86400,"tz":"System","owner-flush-interval":200000000,"processor-flush-interval":100000000,"sorter":{"num-concurrent-worker":4,"chunk-size-limit":999,"max-memory-percentage":80,"max-memory-consumption":8589934592,"num-workerpool-goroutine":16,"sort-dir":"/tmp/cdc_sort"},"security":{"ca-path":"","cert-path":"","key-path":"","cert-allowed-cn":null},"per-table-memory-quota":20971520}`)) + err = conf2.Unmarshal([]byte(`{"addr":"192.155.22.33:8887","advertise-addr":"","log-file":"","log-level":"info","gc-ttl":86400,"tz":"System","capture-session-ttl":10,"owner-flush-interval":200000000,"processor-flush-interval":100000000,"sorter":{"num-concurrent-worker":4,"chunk-size-limit":999,"max-memory-percentage":30,"max-memory-consumption":17179869184,"num-workerpool-goroutine":16,"sort-dir":"/tmp/cdc_sort"},"security":{"ca-path":"","cert-path":"","key-path":"","cert-allowed-cn":null},"per-table-memory-quota":20971520}`)) c.Assert(err, check.IsNil) c.Assert(conf2, check.DeepEquals, conf) } diff --git a/pkg/context/context.go b/pkg/context/context.go index f3b24cdd4bf..a64f0118854 100644 --- a/pkg/context/context.go +++ b/pkg/context/context.go @@ -16,63 +16,79 @@ package context import ( "context" "log" + "time" - "github.com/pingcap/ticdc/cdc/entry" + "github.com/pingcap/ticdc/pkg/version" + + "github.com/pingcap/ticdc/cdc/kv" + "github.com/pingcap/ticdc/cdc/model" "github.com/pingcap/ticdc/pkg/config" + tidbkv "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/store/tikv/oracle" pd "github.com/tikv/pd/client" "go.uber.org/zap" ) -// Vars contains some vars which can be used anywhere in a pipeline +// GlobalVars contains some vars which can be used anywhere in a pipeline +// the lifecycle of vars in the GlobalVars shoule be aligned with the ticdc server process. +// All field in Vars should be READ-ONLY and THREAD-SAFE +type GlobalVars struct { + PDClient pd.Client + KVStorage tidbkv.Storage + CaptureInfo *model.CaptureInfo + EtcdClient *kv.CDCEtcdClient +} + +// ChangefeedVars contains some vars which can be used anywhere in a pipeline +// the lifecycle of vars in the ChangefeedVars shoule be aligned with the changefeed. // All field in Vars should be READ-ONLY and THREAD-SAFE -type Vars struct { - // TODO add more vars - CaptureAddr string - PDClient pd.Client - SchemaStorage entry.SchemaStorage - Config *config.ReplicaConfig +type ChangefeedVars struct { + ID model.ChangeFeedID + Info *model.ChangeFeedInfo } // Context contains Vars(), Done(), Throw(error) and StdContext() context.Context // Context is used to instead of standard context type Context interface { + context.Context - // Vars return the `Vars` store by the root context created by `NewContext` - // Note that the `Vars` should be READ-ONLY and THREAD-SAFE - // The root node and all its children node share one pointer of `Vars` - // So any modification of `Vars` will cause all other family nodes to change. - Vars() *Vars + // GlobalVars return the `GlobalVars` store by the root context created by `NewContext` + // Note that the `GlobalVars` should be READ-ONLY and THREAD-SAFE + // The root node and all its children node share one pointer of `GlobalVars` + // So any modification of `GlobalVars` will cause all other family nodes to change. + GlobalVars() *GlobalVars - // Done return a channel which will be closed in the following cases: - // - the `cancel()` returned from `WithCancel` is called. - // - the `stdCtx` specified in `NewContext` is done. - Done() <-chan struct{} + // ChangefeedVars return the `ChangefeedVars` store by the context created by `WithChangefeedVars` + // Note that the `ChangefeedVars` should be READ-ONLY and THREAD-SAFE + // The root node and all its children node share one pointer of `ChangefeedVars` + // So any modification of `ChangefeedVars` will cause all other family nodes to change. + // ChangefeedVars could be return nil when the `ChangefeedVars` is not set by `WithChangefeedVars` + ChangefeedVars() *ChangefeedVars // Throw an error to parents nodes // we can using `WatchThrow` to listen the errors thrown by children nodes Throw(error) - - // StdContext return a simple struct implement the stdcontext.Context interface - // The Context in this package and the StdContext returned by this function have the same life cycle - // It means the `StdContext.Done()` will done when the `Context` is done. - StdContext() context.Context } type rootContext struct { Context - vars *Vars + globalVars *GlobalVars } // NewContext returns a new pipeline context -func NewContext(stdCtx context.Context, vars *Vars) Context { +func NewContext(stdCtx context.Context, globalVars *GlobalVars) Context { ctx := &rootContext{ - vars: vars, + globalVars: globalVars, } - return withStdCancel(ctx, stdCtx) + return WithStd(ctx, stdCtx) +} + +func (ctx *rootContext) GlobalVars() *GlobalVars { + return ctx.globalVars } -func (ctx *rootContext) Vars() *Vars { - return ctx.vars +func (ctx *rootContext) ChangefeedVars() *ChangefeedVars { + return nil } func (ctx *rootContext) Throw(err error) { @@ -83,31 +99,56 @@ func (ctx *rootContext) Throw(err error) { log.Panic("an error has escaped, please report a bug", zap.Error(err)) } +// WithChangefeedVars return a Context with the `ChangefeedVars` +func WithChangefeedVars(ctx Context, changefeedVars *ChangefeedVars) Context { + return &changefeedVarsContext{ + Context: ctx, + changefeedVars: changefeedVars, + } +} + +type changefeedVarsContext struct { + Context + changefeedVars *ChangefeedVars +} + +func (ctx *changefeedVarsContext) ChangefeedVars() *ChangefeedVars { + return ctx.changefeedVars +} + type stdContext struct { stdCtx context.Context Context } -func (ctx *stdContext) Done() <-chan struct{} { - return ctx.stdCtx.Done() +func (ctx *stdContext) Deadline() (deadline time.Time, ok bool) { + return ctx.stdCtx.Deadline() +} + +func (ctx *stdContext) Err() error { + return ctx.stdCtx.Err() } -func (ctx *stdContext) StdContext() context.Context { - return ctx.stdCtx +func (ctx *stdContext) Value(key interface{}) interface{} { + return ctx.stdCtx.Value(key) +} + +func (ctx *stdContext) Done() <-chan struct{} { + return ctx.stdCtx.Done() } -//revive:disable:context-as-argument -func withStdCancel(ctx Context, stdCtx context.Context) Context { +// WithStd returns a Context with the standard Context +func WithStd(ctx Context, stdCtx context.Context) Context { //revive:disable:context-as-argument return &stdContext{ stdCtx: stdCtx, Context: ctx, } } -// WithCancel return a Context with the cancel function +// WithCancel returns a Context with the cancel function func WithCancel(ctx Context) (Context, context.CancelFunc) { - stdCtx, cancel := context.WithCancel(ctx.StdContext()) - return withStdCancel(ctx, stdCtx), cancel + stdCtx, cancel := context.WithCancel(ctx) + return WithStd(ctx, stdCtx), cancel } type throwContext struct { @@ -133,3 +174,34 @@ func (ctx *throwContext) Throw(err error) { ctx.Context.Throw(err) } } + +// NewBackendContext4Test returns a new pipeline context for test +func NewBackendContext4Test(withChangefeedVars bool) Context { + ctx := NewContext(context.Background(), &GlobalVars{ + CaptureInfo: &model.CaptureInfo{ + ID: "capture-id-test", + AdvertiseAddr: "127.0.0.1:0000", + Version: version.ReleaseVersion, + }, + }) + if withChangefeedVars { + ctx = WithChangefeedVars(ctx, &ChangefeedVars{ + ID: "changefeed-id-test", + Info: &model.ChangeFeedInfo{ + StartTs: oracle.ComposeTS(oracle.GetPhysical(time.Now()), 0), + Config: config.GetDefaultReplicaConfig(), + }, + }) + } + return ctx +} + +// ZapFieldCapture returns a zap field containing capture address +func ZapFieldCapture(ctx Context) zap.Field { + return zap.String("capture", ctx.GlobalVars().CaptureInfo.AdvertiseAddr) +} + +// ZapFieldChangefeed returns a zap field containing changefeed id +func ZapFieldChangefeed(ctx Context) zap.Field { + return zap.String("changefeed", ctx.ChangefeedVars().ID) +} diff --git a/pkg/context/context_test.go b/pkg/context/context_test.go index 04ca9e14197..78186e1673d 100644 --- a/pkg/context/context_test.go +++ b/pkg/context/context_test.go @@ -20,6 +20,7 @@ import ( "github.com/pingcap/check" "github.com/pingcap/errors" + "github.com/pingcap/ticdc/cdc/model" "github.com/pingcap/ticdc/pkg/config" "github.com/pingcap/ticdc/pkg/util/testleak" ) @@ -35,29 +36,32 @@ func (s *contextSuite) TestVars(c *check.C) { stdCtx := context.Background() conf := config.GetDefaultReplicaConfig() conf.Filter.Rules = []string{"hello.world"} - ctx := NewContext(stdCtx, &Vars{ - Config: conf, + info := &model.ChangeFeedInfo{Config: conf} + ctx := NewContext(stdCtx, &GlobalVars{ + CaptureInfo: &model.CaptureInfo{ID: "capture1"}, }) - c.Assert(ctx.Vars().Config, check.DeepEquals, conf) + ctx = WithChangefeedVars(ctx, &ChangefeedVars{ + Info: info, + }) + c.Assert(ctx.ChangefeedVars().Info, check.DeepEquals, info) + c.Assert(ctx.GlobalVars().CaptureInfo.ID, check.Equals, "capture1") } func (s *contextSuite) TestStdCancel(c *check.C) { defer testleak.AfterTest(c)() stdCtx := context.Background() stdCtx, cancel := context.WithCancel(stdCtx) - ctx := NewContext(stdCtx, &Vars{}) + ctx := NewContext(stdCtx, &GlobalVars{}) cancel() - <-ctx.StdContext().Done() <-ctx.Done() } func (s *contextSuite) TestCancel(c *check.C) { defer testleak.AfterTest(c)() stdCtx := context.Background() - ctx := NewContext(stdCtx, &Vars{}) + ctx := NewContext(stdCtx, &GlobalVars{}) ctx, cancel := WithCancel(ctx) cancel() - <-ctx.StdContext().Done() <-ctx.Done() } @@ -65,15 +69,12 @@ func (s *contextSuite) TestCancelCascade(c *check.C) { defer testleak.AfterTest(c)() startTime := time.Now() stdCtx, cancel := context.WithDeadline(context.Background(), time.Now().Add(1*time.Second)) - ctx := NewContext(stdCtx, &Vars{}) + ctx := NewContext(stdCtx, &GlobalVars{}) ctx1, _ := WithCancel(ctx) ctx2, cancel2 := WithCancel(ctx) cancel2() - <-ctx2.StdContext().Done() <-ctx2.Done() c.Assert(time.Since(startTime), check.Less, time.Second) - <-ctx1.StdContext().Done() - c.Assert(time.Since(startTime), check.GreaterEqual, time.Second) <-ctx1.Done() c.Assert(time.Since(startTime), check.GreaterEqual, time.Second) cancel() @@ -82,7 +83,7 @@ func (s *contextSuite) TestCancelCascade(c *check.C) { func (s *contextSuite) TestThrow(c *check.C) { defer testleak.AfterTest(c)() stdCtx := context.Background() - ctx := NewContext(stdCtx, &Vars{}) + ctx := NewContext(stdCtx, &GlobalVars{}) ctx, cancel := WithCancel(ctx) ctx = WithErrorHandler(ctx, func(err error) error { c.Assert(err.Error(), check.Equals, "mock error") @@ -97,7 +98,7 @@ func (s *contextSuite) TestThrow(c *check.C) { func (s *contextSuite) TestThrowCascade(c *check.C) { defer testleak.AfterTest(c)() stdCtx := context.Background() - ctx := NewContext(stdCtx, &Vars{}) + ctx := NewContext(stdCtx, &GlobalVars{}) ctx, cancel := WithCancel(ctx) var errNum1, errNum2, errNum3 int ctx = WithErrorHandler(ctx, func(err error) error { @@ -138,7 +139,7 @@ func (s *contextSuite) TestThrowPanic(c *check.C) { c.Assert(panicMsg, check.Equals, "an error has escaped, please report a bug{error 26 0 mock error}") }() stdCtx := context.Background() - ctx := NewContext(stdCtx, &Vars{}) + ctx := NewContext(stdCtx, &GlobalVars{}) ctx.Throw(nil) ctx.Throw(errors.New("mock error")) } diff --git a/pkg/errors/errors.go b/pkg/errors/errors.go index 8efc5f26a93..6b107faf359 100644 --- a/pkg/errors/errors.go +++ b/pkg/errors/errors.go @@ -67,10 +67,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")) @@ -81,10 +82,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")) @@ -155,6 +156,7 @@ var ( ErrCaptureRegister = errors.Normalize("capture register to etcd failed", errors.RFCCodeText("CDC:ErrCaptureRegister")) ErrNewProcessorFailed = errors.Normalize("new processor failed", errors.RFCCodeText("CDC:ErrNewProcessorFailed")) ErrProcessorUnknown = errors.Normalize("processor running unknown error", errors.RFCCodeText("CDC:ErrProcessorUnknown")) + ErrOwnerUnknown = errors.Normalize("owner running unknown error", errors.RFCCodeText("CDC:ErrOwnerUnknown")) ErrProcessorTableNotFound = errors.Normalize("table not found in processor cache", errors.RFCCodeText("CDC:ErrProcessorTableNotFound")) ErrProcessorEtcdWatch = errors.Normalize("etcd watch returns error", errors.RFCCodeText("CDC:ErrProcessorEtcdWatch")) ErrProcessorSortDir = errors.Normalize("sort dir error", errors.RFCCodeText("CDC:ErrProcessorSortDir")) @@ -179,6 +181,9 @@ var ( ErrUpdateServiceSafepointFailed = errors.Normalize("updating service safepoint failed", errors.RFCCodeText("CDC:ErrUpdateServiceSafepointFailed")) ErrStartTsBeforeGC = errors.Normalize("fail to create changefeed because start-ts %d is earlier than GC safepoint at %d", errors.RFCCodeText("CDC:ErrStartTsBeforeGC")) ErrSnapshotLostByGC = errors.Normalize("fail to create or maintain changefeed due to snapshot loss caused by GC. checkpoint-ts %d is earlier than GC safepoint at %d", errors.RFCCodeText("CDC:ErrSnapshotLostByGC")) + ErrGCTTLExceeded = errors.Normalize("the checkpoint-ts(%d) lag of the changefeed(%s) %d has exceeded the GC TTL", errors.RFCCodeText("CDC:ErrGCTTLExceeded")) + ErrNotOwner = errors.Normalize("this capture is not a owner", errors.RFCCodeText("CDC:ErrNotOwner")) + ErrTableListenReplicated = errors.Normalize("A table is being replicated by at least two processors(%s, %s), please report a bug", errors.RFCCodeText("CDC:ErrTableListenReplicated")) // EtcdWorker related errors. Internal use only. // ErrEtcdTryAgain is used by a PatchFunc to force a transaction abort. ErrEtcdTryAgain = errors.Normalize("the etcd txn should be aborted and retried immediately", errors.RFCCodeText("CDC:ErrEtcdTryAgain")) @@ -189,6 +194,7 @@ var ( // ErrReactorFinished is used by reactor to signal a **normal** exit. ErrReactorFinished = errors.Normalize("the reactor has done its job and should no longer be executed", errors.RFCCodeText("CDC:ErrReactorFinished")) ErrLeaseTimeout = errors.Normalize("owner lease timeout", errors.RFCCodeText("CDC:ErrLeaseTimeout")) + ErrLeaseExpired = errors.Normalize("owner lease expired ", errors.RFCCodeText("CDC:ErrLeaseExpired")) // pipeline errors ErrSendToClosedPipeline = errors.Normalize("pipeline is closed, cannot send message", errors.RFCCodeText("CDC:ErrSendToClosedPipeline")) @@ -209,6 +215,8 @@ var ( // processor errors ErrTableProcessorStoppedSafely = errors.Normalize("table processor stopped safely", errors.RFCCodeText("CDC:ErrTableProcessorStoppedSafely")) + // owner errors + ErrOwnerChangedUnexpectedly = errors.Normalize("owner changed unexpectedly", errors.RFCCodeText("CDC:ErrOwnerChangedUnexpectedly")) // owner related errors ErrOwnerInconsistentStates = errors.Normalize("owner encountered inconsistent state. report a bug if this happens frequently. %s", errors.RFCCodeText("CDC:ErrOwnerInconsistentStates")) diff --git a/pkg/errors/helper.go b/pkg/errors/helper.go index dcf7f18e8af..2d50f52602a 100644 --- a/pkg/errors/helper.go +++ b/pkg/errors/helper.go @@ -27,3 +27,35 @@ func WrapError(rfcError *errors.Error, err error) error { } return rfcError.Wrap(err).GenWithStackByCause() } + +// ChangefeedFastFailError checks the error, returns true if it is meaningless +// to retry on this error +func ChangefeedFastFailError(err error) bool { + return ErrStartTsBeforeGC.Equal(errors.Cause(err)) || ErrSnapshotLostByGC.Equal(errors.Cause(err)) +} + +// ChangefeedFastFailErrorCode checks the error, returns true if it is meaningless +// to retry on this error +func ChangefeedFastFailErrorCode(errCode errors.RFCErrorCode) bool { + switch errCode { + case ErrStartTsBeforeGC.RFCCode(), ErrSnapshotLostByGC.RFCCode(): + return true + default: + return false + } +} + +// RFCCode returns a RFCCode from an error +func RFCCode(err error) (errors.RFCErrorCode, bool) { + type rfcCoder interface { + RFCCode() errors.RFCErrorCode + } + if terr, ok := err.(rfcCoder); ok { + return terr.RFCCode(), true + } + err = errors.Cause(err) + if terr, ok := err.(rfcCoder); ok { + return terr.RFCCode(), true + } + return "", false +} diff --git a/pkg/etcd/client.go b/pkg/etcd/client.go index 316aca5ad7b..3b09ac33957 100644 --- a/pkg/etcd/client.go +++ b/pkg/etcd/client.go @@ -17,6 +17,10 @@ import ( "context" "time" + "github.com/cenkalti/backoff" + "go.etcd.io/etcd/etcdserver/api/v3rpc/rpctypes" + "google.golang.org/grpc/codes" + "github.com/pingcap/errors" "github.com/pingcap/log" "github.com/pingcap/ticdc/pkg/retry" @@ -122,6 +126,12 @@ func (c *Client) Revoke(ctx context.Context, id clientv3.LeaseID) (resp *clientv err = retryRPC(EtcdRevoke, c.metrics[EtcdRevoke], func() error { var inErr error resp, inErr = c.cli.Revoke(ctx, id) + if inErr == nil { + return nil + } else if etcdErr := inErr.(rpctypes.EtcdError); etcdErr.Code() == codes.NotFound { + // it means the etcd lease is already expired or revoked + return backoff.Permanent(inErr) + } return inErr }) return diff --git a/pkg/etcd/etcd_test.go b/pkg/etcd/etcd_test.go index a99e378aafe..c414e3a6238 100644 --- a/pkg/etcd/etcd_test.go +++ b/pkg/etcd/etcd_test.go @@ -41,13 +41,19 @@ func (s *etcdSuite) SetUpTest(c *check.C) { c.Assert(err, check.IsNil) s.clientURL = curl s.etcd = e - go func() { - c.Log(<-e.Err()) - }() } func (s *etcdSuite) TearDownTest(c *check.C) { s.etcd.Close() +logEtcdError: + for { + select { + case err := <-s.etcd.Err(): + c.Logf("etcd server error: %v", err) + default: + break logEtcdError + } + } } func (s *etcdSuite) TestEmbedEtcd(c *check.C) { diff --git a/pkg/etcd/etcdkey.go b/pkg/etcd/etcdkey.go index 1f91636db15..51d4c91cf0a 100644 --- a/pkg/etcd/etcdkey.go +++ b/pkg/etcd/etcdkey.go @@ -17,7 +17,6 @@ import ( "log" "strings" - "github.com/pingcap/ticdc/cdc/model" cerror "github.com/pingcap/ticdc/pkg/errors" ) @@ -76,8 +75,8 @@ const ( */ type CDCKey struct { Tp CDCKeyType - ChangefeedID model.ChangeFeedID - CaptureID model.CaptureID + ChangefeedID string + CaptureID string OwnerLeaseID string } diff --git a/pkg/filter/errors.go b/pkg/filter/errors.go deleted file mode 100644 index 4cd65bfb67b..00000000000 --- a/pkg/filter/errors.go +++ /dev/null @@ -1,25 +0,0 @@ -// Copyright 2020 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package filter - -import ( - "github.com/pingcap/errors" - cerror "github.com/pingcap/ticdc/pkg/errors" -) - -// ChangefeedFastFailError checks the error, returns true if it is meaningless -// to retry on this error -func ChangefeedFastFailError(err error) bool { - return cerror.ErrStartTsBeforeGC.Equal(errors.Cause(err)) -} diff --git a/pkg/orchestrator/doc.go b/pkg/orchestrator/doc.go new file mode 100644 index 00000000000..ead0578d865 --- /dev/null +++ b/pkg/orchestrator/doc.go @@ -0,0 +1,46 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +/* +Package orchestrator mainly implements a ETCD worker. +A ETCD worker is used to read/write data from ETCD servers based on snapshot and data patches. +Here is a detailed description of how the ETCD worker works: + + ETCD Servers + | ^ + | | + 1. Watch | | 5. Txn + | | + v | + EtcdWorker + | ^ + | | + 2. Update| | 4. DataPatch + +--------+ +-------+ + | | + | | + v 3.Tick | + ReactorState ----------> Reactor + +1. EtcdWorker watches the txn modification log from ETCD servers +2. EtcdWorker updates the txn modification listened from ETCD servers by calling the Update function of ReactorState +3. EtcdWorker calls the Tick function of Reactor, and EtcdWorker make sure the state of ReactorState is a consistent snapshot of ETCD servers +4. Reactor is implemented by the upper layer application. Usually, Reactor will produce DataPatches when the Tick function called + EtcdWorker apply all the DataPatches produced by Reactor +5. EtcdWorker commits a txn to ETCD according to DataPatches + +The upper layer application which is a user of EtcdWorker only need to implement Reactor and ReactorState interface. +The ReactorState is used to maintenance status of ETCD, and the Reactor can produce DataPatches differently according to the ReactorState. +The EtcdWorker make sure any ReactorState which perceived by Reactor must be a consistent snapshot of ETCD servers. +*/ +package orchestrator diff --git a/pkg/orchestrator/etcd_worker.go b/pkg/orchestrator/etcd_worker.go index 22203c565ac..9e5308f8537 100644 --- a/pkg/orchestrator/etcd_worker.go +++ b/pkg/orchestrator/etcd_worker.go @@ -14,7 +14,6 @@ package orchestrator import ( - "bytes" "context" "time" @@ -86,9 +85,9 @@ func (worker *EtcdWorker) Run(ctx context.Context, session *concurrency.Session, ticker := time.NewTicker(timerInterval) defer ticker.Stop() - watchCh := worker.client.Watch(ctx1, worker.prefix.String(), clientv3.WithPrefix()) + watchCh := worker.client.Watch(ctx1, worker.prefix.String(), clientv3.WithPrefix(), clientv3.WithRev(worker.revision+1)) var ( - pendingPatches []*DataPatch + pendingPatches [][]DataPatch exiting bool sessionDone <-chan struct{} ) @@ -144,16 +143,13 @@ func (worker *EtcdWorker) Run(ctx context.Context, session *concurrency.Session, if len(pendingPatches) > 0 { // Here we have some patches yet to be uploaded to Etcd. - err := worker.applyPatches(ctx, pendingPatches) + pendingPatches, err = worker.applyPatchGroups(ctx, pendingPatches) if err != nil { if cerrors.ErrEtcdTryAgain.Equal(errors.Cause(err)) { continue } return errors.Trace(err) } - // If we are here, all patches have been successfully applied to Etcd. - // `applyPatches` is all-or-none, so in case of success, we should clear all the pendingPatches. - pendingPatches = pendingPatches[:0] } else { if exiting { // If exiting is true here, it means that the reactor returned `ErrReactorFinished` last tick, and all pending patches is applied. @@ -223,81 +219,64 @@ func (worker *EtcdWorker) syncRawState(ctx context.Context) error { return nil } -func mergePatch(patches []*DataPatch) []*DataPatch { - patchMap := make(map[util.EtcdKey][]*DataPatch) - for _, patch := range patches { - patchMap[patch.Key] = append(patchMap[patch.Key], patch) - } - result := make([]*DataPatch, 0, len(patchMap)) - for key, patches := range patchMap { - patches := patches - result = append(result, &DataPatch{ - Key: key, - Fun: func(old []byte) ([]byte, error) { - for _, patch := range patches { - newValue, err := patch.Fun(old) - if err != nil { - if cerrors.ErrEtcdIgnore.Equal(errors.Cause(err)) { - continue - } - return nil, err - } - old = newValue - } - return old, nil - }, - }) +func (worker *EtcdWorker) cloneRawState() map[util.EtcdKey][]byte { + ret := make(map[util.EtcdKey][]byte) + for k, v := range worker.rawState { + cloneV := make([]byte, len(v)) + copy(cloneV, v) + ret[util.NewEtcdKey(k.String())] = cloneV } - return result + return ret } -func etcdValueEqual(left, right []byte) bool { - if len(left) == 0 && len(right) == 0 { - return (left == nil && right == nil) || (left != nil && right != nil) +func (worker *EtcdWorker) applyPatchGroups(ctx context.Context, patchGroups [][]DataPatch) ([][]DataPatch, error) { + for len(patchGroups) > 0 { + patches := patchGroups[0] + err := worker.applyPatches(ctx, patches) + if err != nil { + return patchGroups, err + } + patchGroups = patchGroups[1:] } - return bytes.Equal(left, right) + return patchGroups, nil } -func (worker *EtcdWorker) applyPatches(ctx context.Context, patches []*DataPatch) error { - patches = mergePatch(patches) - cmps := make([]clientv3.Cmp, 0, len(patches)) - ops := make([]clientv3.Op, 0, len(patches)) - +func (worker *EtcdWorker) applyPatches(ctx context.Context, patches []DataPatch) error { + state := worker.cloneRawState() + changedSet := make(map[util.EtcdKey]struct{}) for _, patch := range patches { - old, ok := worker.rawState[patch.Key] - - value, err := patch.Fun(old) + err := patch.Patch(state, changedSet) if err != nil { if cerrors.ErrEtcdIgnore.Equal(errors.Cause(err)) { continue } return errors.Trace(err) } - + } + cmps := make([]clientv3.Cmp, 0, len(changedSet)) + ops := make([]clientv3.Op, 0, len(changedSet)) + for key := range changedSet { // make sure someone else has not updated the key after the last snapshot var cmp clientv3.Cmp - // if ok is false, it means that the key of this patch is not exist in a committed state - if ok { - cmp = clientv3.Compare(clientv3.ModRevision(patch.Key.String()), "<", worker.revision+1) + if _, ok := worker.rawState[key]; ok { + cmp = clientv3.Compare(clientv3.ModRevision(key.String()), "<", worker.revision+1) } else { + // if ok is false, it means that the key of this patch is not exist in a committed state // this compare is equivalent to `patch.Key` is not exist - cmp = clientv3.Compare(clientv3.ModRevision(patch.Key.String()), "=", 0) + cmp = clientv3.Compare(clientv3.ModRevision(key.String()), "=", 0) } cmps = append(cmps, cmp) - if etcdValueEqual(old, value) { - // Ignore patches that produce a new value that is the same as the old value. - continue - } - + value := state[key] var op clientv3.Op if value != nil { - op = clientv3.OpPut(patch.Key.String(), string(value)) + op = clientv3.OpPut(key.String(), string(value)) } else { - op = clientv3.OpDelete(patch.Key.String()) + op = clientv3.OpDelete(key.String()) } ops = append(ops, op) } + resp, err := worker.client.Txn(ctx).If(cmps...).Then(ops...).Commit() if err != nil { return errors.Trace(err) diff --git a/pkg/orchestrator/etcd_worker_bank_test.go b/pkg/orchestrator/etcd_worker_bank_test.go new file mode 100644 index 00000000000..6a7f3daec32 --- /dev/null +++ b/pkg/orchestrator/etcd_worker_bank_test.go @@ -0,0 +1,163 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package orchestrator + +import ( + "context" + "fmt" + "math/rand" + "strconv" + "strings" + "sync" + "time" + + "github.com/pingcap/check" + "github.com/pingcap/log" + cerror "github.com/pingcap/ticdc/pkg/errors" + "github.com/pingcap/ticdc/pkg/orchestrator/util" + "github.com/pingcap/ticdc/pkg/util/testleak" + "go.uber.org/zap" +) + +type bankReactorState struct { + c *check.C + account []int + pendingPatch [][]DataPatch + index int + notFirstTick bool +} + +const bankTestPrefix = "/ticdc/test/bank/" + +func (b *bankReactorState) Update(key util.EtcdKey, value []byte, isInit bool) error { + b.c.Assert(strings.HasPrefix(key.String(), bankTestPrefix), check.IsTrue) + indexStr := key.String()[len(bankTestPrefix):] + b.account[b.atoi(indexStr)] = b.atoi(string(value)) + return nil +} + +func (b *bankReactorState) GetPatches() [][]DataPatch { + pendingPatches := b.pendingPatch + b.pendingPatch = nil + return pendingPatches +} + +func (b *bankReactorState) Check() { + var sum int + for _, money := range b.account { + sum += money + } + if sum != 0 { + log.Info("show account", zap.Int("index", b.index), zap.Int("sum", sum), zap.Ints("account", b.account)) + } + b.c.Assert(sum, check.Equals, 0, check.Commentf("not ft:%t", b.notFirstTick)) +} + +func (b *bankReactorState) atoi(value string) int { + i, err := strconv.Atoi(value) + b.c.Assert(err, check.IsNil) + return i +} + +func (b *bankReactorState) patchAccount(index int, fn func(int) int) DataPatch { + return &SingleDataPatch{ + Key: util.NewEtcdKey(fmt.Sprintf("%s%d", bankTestPrefix, index)), + Func: func(old []byte) (newValue []byte, changed bool, err error) { + oldMoney := b.atoi(string(old)) + newMoney := fn(oldMoney) + if oldMoney == newMoney { + return old, false, nil + } + log.Debug("change money", zap.Int("account", index), zap.Int("from", oldMoney), zap.Int("to", newMoney)) + return []byte(strconv.Itoa(newMoney)), true, nil + }, + } +} + +func (b *bankReactorState) TransferRandomly(transferNumber int) { + for i := 0; i < transferNumber; i++ { + accountA := rand.Intn(len(b.account)) + accountB := rand.Intn(len(b.account)) + transferMoney := rand.Intn(100) + b.pendingPatch = append(b.pendingPatch, []DataPatch{ + b.patchAccount(accountA, func(money int) int { + return money - transferMoney + }), + b.patchAccount(accountB, func(money int) int { + return money + transferMoney + }), + }) + log.Debug("transfer money", zap.Int("accountA", accountA), zap.Int("accountB", accountB), zap.Int("money", transferMoney)) + } +} + +type bankReactor struct { + accountNumber int +} + +func (b *bankReactor) Tick(ctx context.Context, state ReactorState) (nextState ReactorState, err error) { + bankState := (state).(*bankReactorState) + bankState.Check() + // transfer 20% of account + bankState.TransferRandomly(rand.Intn(b.accountNumber/5 + 2)) + // there is a 20% chance of restarting etcd worker + if rand.Intn(10) < 2 { + err = cerror.ErrReactorFinished.GenWithStackByArgs() + } + bankState.notFirstTick = true + return state, err +} + +func (s *etcdWorkerSuite) TestEtcdBank(c *check.C) { + defer testleak.AfterTest(c)() + totalAccountNumber := 25 + workerNumber := 10 + var wg sync.WaitGroup + ctx, cancel := context.WithTimeout(context.Background(), time.Second*10) + defer cancel() + + newClient, closer := setUpTest(c) + defer closer() + + cli := newClient() + defer func() { + _ = cli.Unwrap().Close() + }() + + for i := 0; i < totalAccountNumber; i++ { + _, err := cli.Put(ctx, fmt.Sprintf("%s%d", bankTestPrefix, i), "0") + c.Assert(err, check.IsNil) + } + + for i := 0; i < workerNumber; i++ { + i := i + wg.Add(1) + go func() { + defer wg.Done() + for { + worker, err := NewEtcdWorker(cli, bankTestPrefix, &bankReactor{ + accountNumber: totalAccountNumber, + }, &bankReactorState{c: c, index: i, account: make([]int, totalAccountNumber)}) + c.Assert(err, check.IsNil) + err = worker.Run(ctx, nil, 100*time.Millisecond) + if err == nil || err.Error() == "etcdserver: request timed out" { + continue + } + c.Assert(err, check.ErrorMatches, ".*context deadline exceeded.*") + return + } + }() + } + wg.Wait() +} diff --git a/pkg/orchestrator/etcd_worker_test.go b/pkg/orchestrator/etcd_worker_test.go index 38554eb5099..0dcdf4b735c 100644 --- a/pkg/orchestrator/etcd_worker_test.go +++ b/pkg/orchestrator/etcd_worker_test.go @@ -108,7 +108,7 @@ type simpleReactorState struct { values [][]int sum int deltas []*delta - patches []*DataPatch + patches []DataPatch } var keyParseRegexp = regexp.MustCompile(regexp.QuoteMeta(testEtcdKeyPrefix) + `/(.+)`) @@ -118,17 +118,21 @@ func (s *simpleReactorState) Get(i1, i2 int) int { } func (s *simpleReactorState) Inc(i1, i2 int) { - patch := &DataPatch{ + patch := &SingleDataPatch{ Key: util.NewEtcdKey(testEtcdKeyPrefix + "/" + strconv.Itoa(i1)), - Fun: func(old []byte) ([]byte, error) { + Func: func(old []byte) ([]byte, bool, error) { var oldJSON []int err := json.Unmarshal(old, &oldJSON) if err != nil { - return nil, errors.Trace(err) + return nil, false, errors.Trace(err) } oldJSON[i2]++ - return json.Marshal(oldJSON) + newValue, err := json.Marshal(oldJSON) + if err != nil { + return nil, false, errors.Trace(err) + } + return newValue, true, nil }, } @@ -136,10 +140,10 @@ func (s *simpleReactorState) Inc(i1, i2 int) { } func (s *simpleReactorState) SetSum(sum int) { - patch := &DataPatch{ + patch := &SingleDataPatch{ Key: util.NewEtcdKey(testEtcdKeyPrefix + "/sum"), - Fun: func(_ []byte) ([]byte, error) { - return []byte(strconv.Itoa(sum)), nil + Func: func(_ []byte) ([]byte, bool, error) { + return []byte(strconv.Itoa(sum)), true, nil }, } @@ -187,10 +191,10 @@ func (s *simpleReactorState) Update(key util.EtcdKey, value []byte, isInit bool) return nil } -func (s *simpleReactorState) GetPatches() []*DataPatch { +func (s *simpleReactorState) GetPatches() [][]DataPatch { ret := s.patches s.patches = nil - return ret + return [][]DataPatch{ret} } func setUpTest(c *check.C) (func() *etcd.Client, func()) { @@ -289,8 +293,8 @@ func (s *intReactorState) Update(key util.EtcdKey, value []byte, isInit bool) er return nil } -func (s *intReactorState) GetPatches() []*DataPatch { - return []*DataPatch{} +func (s *intReactorState) GetPatches() [][]DataPatch { + return [][]DataPatch{} } type linearizabilityReactor struct { @@ -359,7 +363,7 @@ func (s *etcdWorkerSuite) TestLinearizability(c *check.C) { type commonReactorState struct { state map[string]string - pendingPatches []*DataPatch + pendingPatches []DataPatch } func (s *commonReactorState) Update(key util.EtcdKey, value []byte, isInit bool) error { @@ -367,17 +371,17 @@ func (s *commonReactorState) Update(key util.EtcdKey, value []byte, isInit bool) return nil } -func (s *commonReactorState) AppendPatch(key util.EtcdKey, fun PatchFunc) { - s.pendingPatches = append(s.pendingPatches, &DataPatch{ - Key: key, - Fun: fun, +func (s *commonReactorState) AppendPatch(key util.EtcdKey, fun func(old []byte) (newValue []byte, changed bool, err error)) { + s.pendingPatches = append(s.pendingPatches, &SingleDataPatch{ + Key: key, + Func: fun, }) } -func (s *commonReactorState) GetPatches() []*DataPatch { +func (s *commonReactorState) GetPatches() [][]DataPatch { pendingPatches := s.pendingPatches s.pendingPatches = nil - return pendingPatches + return [][]DataPatch{pendingPatches} } type finishedReactor struct { @@ -389,23 +393,20 @@ type finishedReactor struct { func (r *finishedReactor) Tick(ctx context.Context, state ReactorState) (nextState ReactorState, err error) { r.state = state.(*commonReactorState) if r.tickNum < 2 { - r.state.AppendPatch(util.NewEtcdKey(r.prefix+"/key1"), func(old []byte) (newValue []byte, err error) { - newValue = append(old, []byte("abc")...) - return + r.state.AppendPatch(util.NewEtcdKey(r.prefix+"/key1"), func(old []byte) (newValue []byte, changed bool, err error) { + return append(old, []byte("abc")...), true, nil }) - r.state.AppendPatch(util.NewEtcdKey(r.prefix+"/key2"), func(old []byte) (newValue []byte, err error) { - newValue = append(old, []byte("123")...) - return + r.state.AppendPatch(util.NewEtcdKey(r.prefix+"/key2"), func(old []byte) (newValue []byte, changed bool, err error) { + return append(old, []byte("123")...), true, nil }) r.tickNum++ return r.state, nil } - r.state.AppendPatch(util.NewEtcdKey(r.prefix+"/key1"), func(old []byte) (newValue []byte, err error) { - newValue = append(old, []byte("fin")...) - return + r.state.AppendPatch(util.NewEtcdKey(r.prefix+"/key1"), func(old []byte) (newValue []byte, changed bool, err error) { + return append(old, []byte("fin")...), true, nil }) - r.state.AppendPatch(util.NewEtcdKey(r.prefix+"/key2"), func(old []byte) (newValue []byte, err error) { - return nil, nil + r.state.AppendPatch(util.NewEtcdKey(r.prefix+"/key2"), func(old []byte) (newValue []byte, changed bool, err error) { + return nil, true, nil }) return r.state, cerrors.ErrReactorFinished } @@ -449,39 +450,32 @@ type coverReactor struct { func (r *coverReactor) Tick(ctx context.Context, state ReactorState) (nextState ReactorState, err error) { r.state = state.(*commonReactorState) if r.tickNum < 2 { - r.state.AppendPatch(util.NewEtcdKey(r.prefix+"/key1"), func(old []byte) (newValue []byte, err error) { - newValue = append(old, []byte("abc")...) - return + r.state.AppendPatch(util.NewEtcdKey(r.prefix+"/key1"), func(old []byte) (newValue []byte, changed bool, err error) { + return append(old, []byte("abc")...), true, nil }) - r.state.AppendPatch(util.NewEtcdKey(r.prefix+"/key2"), func(old []byte) (newValue []byte, err error) { - newValue = append(old, []byte("123")...) - return + r.state.AppendPatch(util.NewEtcdKey(r.prefix+"/key2"), func(old []byte) (newValue []byte, changed bool, err error) { + return append(old, []byte("123")...), true, nil }) - r.state.AppendPatch(util.NewEtcdKey(r.prefix+"/key1"), func(old []byte) (newValue []byte, err error) { - newValue = append(old, []byte("cba")...) - return + r.state.AppendPatch(util.NewEtcdKey(r.prefix+"/key1"), func(old []byte) (newValue []byte, changed bool, err error) { + return append(old, []byte("cba")...), true, nil }) - r.state.AppendPatch(util.NewEtcdKey(r.prefix+"/key2"), func(old []byte) (newValue []byte, err error) { - newValue = append(old, []byte("321")...) - return + r.state.AppendPatch(util.NewEtcdKey(r.prefix+"/key2"), func(old []byte) (newValue []byte, changed bool, err error) { + return append(old, []byte("321")...), true, nil }) r.tickNum++ return r.state, nil } - r.state.AppendPatch(util.NewEtcdKey(r.prefix+"/key1"), func(old []byte) (newValue []byte, err error) { - newValue = append(old, []byte("fin")...) - return + r.state.AppendPatch(util.NewEtcdKey(r.prefix+"/key1"), func(old []byte) (newValue []byte, changed bool, err error) { + return append(old, []byte("fin")...), true, nil }) - r.state.AppendPatch(util.NewEtcdKey(r.prefix+"/key1"), func(old []byte) (newValue []byte, err error) { - newValue = append(old, []byte("fin")...) - return + r.state.AppendPatch(util.NewEtcdKey(r.prefix+"/key1"), func(old []byte) (newValue []byte, changed bool, err error) { + return append(old, []byte("fin")...), true, nil }) - r.state.AppendPatch(util.NewEtcdKey(r.prefix+"/key2"), func(old []byte) (newValue []byte, err error) { - return nil, nil + r.state.AppendPatch(util.NewEtcdKey(r.prefix+"/key2"), func(old []byte) (newValue []byte, changed bool, err error) { + return nil, true, nil }) - r.state.AppendPatch(util.NewEtcdKey(r.prefix+"/key2"), func(old []byte) (newValue []byte, err error) { - newValue = append(old, []byte("fin")...) - return + r.state.AppendPatch(util.NewEtcdKey(r.prefix+"/key2"), func(old []byte) (newValue []byte, changed bool, err error) { + return append(old, []byte("fin")...), true, nil }) return r.state, cerrors.ErrReactorFinished } @@ -527,14 +521,14 @@ type emptyTxnReactor struct { func (r *emptyTxnReactor) Tick(ctx context.Context, state ReactorState) (nextState ReactorState, err error) { r.state = state.(*commonReactorState) if r.tickNum == 0 { - r.state.AppendPatch(util.NewEtcdKey(r.prefix+"/key1"), func(old []byte) (newValue []byte, err error) { - return []byte("abc"), nil + r.state.AppendPatch(util.NewEtcdKey(r.prefix+"/key1"), func(old []byte) (newValue []byte, changed bool, err error) { + return []byte("abc"), true, nil }) - r.state.AppendPatch(util.NewEtcdKey(r.prefix+"/key2"), func(old []byte) (newValue []byte, err error) { - return []byte("123"), nil + r.state.AppendPatch(util.NewEtcdKey(r.prefix+"/key2"), func(old []byte) (newValue []byte, changed bool, err error) { + return []byte("123"), true, nil }) - r.state.AppendPatch(util.NewEtcdKey(r.prefix+"/key1"), func(old []byte) (newValue []byte, err error) { - return nil, nil + r.state.AppendPatch(util.NewEtcdKey(r.prefix+"/key1"), func(old []byte) (newValue []byte, changed bool, err error) { + return nil, true, nil }) r.tickNum++ return r.state, nil @@ -546,20 +540,20 @@ func (r *emptyTxnReactor) Tick(ctx context.Context, state ReactorState) (nextSta return nil, errors.Trace(err) } - r.state.AppendPatch(util.NewEtcdKey(r.prefix+"/key2"), func(old []byte) (newValue []byte, err error) { - return []byte("123"), nil + r.state.AppendPatch(util.NewEtcdKey(r.prefix+"/key2"), func(old []byte) (newValue []byte, changed bool, err error) { + return []byte("123"), true, nil }) - r.state.AppendPatch(util.NewEtcdKey(r.prefix+"/key1"), func(old []byte) (newValue []byte, err error) { - return nil, nil + r.state.AppendPatch(util.NewEtcdKey(r.prefix+"/key1"), func(old []byte) (newValue []byte, changed bool, err error) { + return nil, true, nil }) r.tickNum++ return r.state, nil } - r.state.AppendPatch(util.NewEtcdKey(r.prefix+"/key1"), func(old []byte) (newValue []byte, err error) { - return nil, nil + r.state.AppendPatch(util.NewEtcdKey(r.prefix+"/key1"), func(old []byte) (newValue []byte, changed bool, err error) { + return nil, true, nil }) - r.state.AppendPatch(util.NewEtcdKey(r.prefix+"/key2"), func(old []byte) (newValue []byte, err error) { - return []byte("123"), nil + r.state.AppendPatch(util.NewEtcdKey(r.prefix+"/key2"), func(old []byte) (newValue []byte, changed bool, err error) { + return []byte("123"), true, nil }) return r.state, cerrors.ErrReactorFinished } @@ -604,30 +598,30 @@ type emptyOrNilReactor struct { func (r *emptyOrNilReactor) Tick(ctx context.Context, state ReactorState) (nextState ReactorState, err error) { r.state = state.(*commonReactorState) if r.tickNum == 0 { - r.state.AppendPatch(util.NewEtcdKey(r.prefix+"/key1"), func(old []byte) (newValue []byte, err error) { - return []byte(""), nil + r.state.AppendPatch(util.NewEtcdKey(r.prefix+"/key1"), func(old []byte) (newValue []byte, changed bool, err error) { + return []byte(""), true, nil }) - r.state.AppendPatch(util.NewEtcdKey(r.prefix+"/key2"), func(old []byte) (newValue []byte, err error) { - return nil, nil + r.state.AppendPatch(util.NewEtcdKey(r.prefix+"/key2"), func(old []byte) (newValue []byte, changed bool, err error) { + return nil, true, nil }) r.tickNum++ return r.state, nil } if r.tickNum == 1 { - r.state.AppendPatch(util.NewEtcdKey(r.prefix+"/key1"), func(old []byte) (newValue []byte, err error) { - return nil, nil + r.state.AppendPatch(util.NewEtcdKey(r.prefix+"/key1"), func(old []byte) (newValue []byte, changed bool, err error) { + return nil, true, nil }) - r.state.AppendPatch(util.NewEtcdKey(r.prefix+"/key2"), func(old []byte) (newValue []byte, err error) { - return []byte(""), nil + r.state.AppendPatch(util.NewEtcdKey(r.prefix+"/key2"), func(old []byte) (newValue []byte, changed bool, err error) { + return []byte(""), true, nil }) r.tickNum++ return r.state, nil } - r.state.AppendPatch(util.NewEtcdKey(r.prefix+"/key1"), func(old []byte) (newValue []byte, err error) { - return []byte(""), nil + r.state.AppendPatch(util.NewEtcdKey(r.prefix+"/key1"), func(old []byte) (newValue []byte, changed bool, err error) { + return []byte(""), true, nil }) - r.state.AppendPatch(util.NewEtcdKey(r.prefix+"/key2"), func(old []byte) (newValue []byte, err error) { - return nil, nil + r.state.AppendPatch(util.NewEtcdKey(r.prefix+"/key2"), func(old []byte) (newValue []byte, changed bool, err error) { + return nil, true, nil }) return r.state, cerrors.ErrReactorFinished } @@ -661,148 +655,3 @@ func (s *etcdWorkerSuite) TestEmptyOrNil(c *check.C) { err = cli.Unwrap().Close() c.Assert(err, check.IsNil) } - -func (s *etcdWorkerSuite) TestMergePatches(c *check.C) { - defer testleak.AfterTest(c)() - testCases := []struct { - state map[util.EtcdKey][]byte - patches []*DataPatch - }{ - { - state: map[util.EtcdKey][]byte{}, - patches: []*DataPatch{}, - }, - { - state: map[util.EtcdKey][]byte{ - util.NewEtcdKey("key1"): []byte("aa"), - }, - patches: []*DataPatch{ - { - Key: util.NewEtcdKey("key1"), - Fun: func(old []byte) (newValue []byte, err error) { - newValue = append(old, []byte("bb")...) - return - }, - }, - { - Key: util.NewEtcdKey("key1"), - Fun: func(old []byte) (newValue []byte, err error) { - newValue = append(old, []byte("cc")...) - return - }, - }, - { - Key: util.NewEtcdKey("key1"), - Fun: func(old []byte) (newValue []byte, err error) { - newValue = append(old, []byte("dd")...) - return - }, - }, - }, - }, - { - state: map[util.EtcdKey][]byte{ - util.NewEtcdKey("key1"): []byte("aa"), - }, - patches: []*DataPatch{ - { - Key: util.NewEtcdKey("key1"), - Fun: func(old []byte) (newValue []byte, err error) { - newValue = append(old, []byte("bb")...) - return - }, - }, - { - Key: util.NewEtcdKey("key2"), - Fun: func(old []byte) (newValue []byte, err error) { - newValue = append(old, []byte("cc")...) - return - }, - }, - { - Key: util.NewEtcdKey("key1"), - Fun: func(old []byte) (newValue []byte, err error) { - newValue = append(old, []byte("dd")...) - return - }, - }, - { - Key: util.NewEtcdKey("key2"), - Fun: func(old []byte) (newValue []byte, err error) { - newValue = append(old, []byte("ee")...) - return - }, - }, - }, - }, - { - state: map[util.EtcdKey][]byte{ - util.NewEtcdKey("key1"): []byte("aa"), - }, - patches: []*DataPatch{ - { - Key: util.NewEtcdKey("key1"), - Fun: func(old []byte) (newValue []byte, err error) { - newValue = append(old, []byte("bb")...) - err = cerrors.ErrEtcdIgnore - return - }, - }, - { - Key: util.NewEtcdKey("key2"), - Fun: func(old []byte) (newValue []byte, err error) { - newValue = append(old, []byte("cc")...) - return - }, - }, - { - Key: util.NewEtcdKey("key1"), - Fun: func(old []byte) (newValue []byte, err error) { - newValue = append(old, []byte("dd")...) - return - }, - }, - { - Key: util.NewEtcdKey("key2"), - Fun: func(old []byte) (newValue []byte, err error) { - newValue = append(old, []byte("ee")...) - err = cerrors.ErrEtcdIgnore - return - }, - }, - }, - }, - } - - applyPatches := func(state map[util.EtcdKey][]byte, patches []*DataPatch) map[util.EtcdKey][]byte { - // clone state map - clonedState := make(map[util.EtcdKey][]byte, len(state)) - for k, v := range state { - clonedState[k] = v - } - // apply patches - for _, p := range patches { - newValue, err := p.Fun(clonedState[p.Key]) - if cerrors.ErrEtcdIgnore.Equal(errors.Cause(err)) { - continue - } - c.Assert(err, check.IsNil) - clonedState[p.Key] = newValue - } - return clonedState - } - for _, tc := range testCases { - mergedPatches := mergePatch(tc.patches) - c.Assert(applyPatches(tc.state, mergedPatches), check.DeepEquals, applyPatches(tc.state, tc.patches)) - } -} - -func (s *etcdWorkerSuite) TestEtcdValueEqual(c *check.C) { - defer testleak.AfterTest(c)() - c.Assert(etcdValueEqual(nil, nil), check.IsTrue) - c.Assert(etcdValueEqual(nil, []byte{}), check.IsFalse) - c.Assert(etcdValueEqual([]byte{}, nil), check.IsFalse) - c.Assert(etcdValueEqual([]byte{}, []byte{}), check.IsTrue) - c.Assert(etcdValueEqual([]byte{11}, []byte{11}), check.IsTrue) - c.Assert(etcdValueEqual([]byte{11}, []byte{12}), check.IsFalse) -} diff --git a/pkg/orchestrator/interfaces.go b/pkg/orchestrator/interfaces.go index f0f5290e6f6..6bf0eaf7b6e 100644 --- a/pkg/orchestrator/interfaces.go +++ b/pkg/orchestrator/interfaces.go @@ -25,14 +25,9 @@ type Reactor interface { Tick(ctx context.Context, state ReactorState) (nextState ReactorState, err error) } -// PatchFunc should be a pure function that returns a new value given the old value. -// The function is called each time the EtcdWorker initiates an Etcd transaction. -type PatchFunc = func(old []byte) (newValue []byte, err error) - -// DataPatch represents an update to a given Etcd key -type DataPatch struct { - Key util.EtcdKey - Fun PatchFunc +// DataPatch represents an update of state +type DataPatch interface { + Patch(valueMap map[util.EtcdKey][]byte, changedSet map[util.EtcdKey]struct{}) error } // ReactorState models the Etcd state of a reactor @@ -40,7 +35,43 @@ type ReactorState interface { // Update is called by EtcdWorker to notify the Reactor of a latest change to the Etcd state. Update(key util.EtcdKey, value []byte, isInit bool) error - // GetPatches is called by EtcdWorker, and should return a slice of data patches that represents the changes + // GetPatches is called by EtcdWorker, and should return many slices of data patches that represents the changes // that a Reactor wants to apply to Etcd. - GetPatches() []*DataPatch + // a slice of DataPatch will be committed as one ETCD txn + GetPatches() [][]DataPatch +} + +// SingleDataPatch represents an update to a given Etcd key +type SingleDataPatch struct { + Key util.EtcdKey + // Func should be a pure function that returns a new value given the old value. + // The function is called each time the EtcdWorker initiates an Etcd transaction. + Func func(old []byte) (newValue []byte, changed bool, err error) +} + +// Patch implements the DataPatch interface +func (s *SingleDataPatch) Patch(valueMap map[util.EtcdKey][]byte, changedSet map[util.EtcdKey]struct{}) error { + value := valueMap[s.Key] + newValue, changed, err := s.Func(value) + if err != nil { + return err + } + if !changed { + return nil + } + changedSet[s.Key] = struct{}{} + if newValue == nil { + delete(valueMap, s.Key) + } else { + valueMap[s.Key] = newValue + } + return nil +} + +// MultiDatePatch represents an update to many keys +type MultiDatePatch func(valueMap map[util.EtcdKey][]byte, changedSet map[util.EtcdKey]struct{}) error + +// Patch implements the DataPatch interface +func (m MultiDatePatch) Patch(valueMap map[util.EtcdKey][]byte, changedSet map[util.EtcdKey]struct{}) error { + return m(valueMap, changedSet) } diff --git a/pkg/orchestrator/jsonstate/json_reactor_state.go b/pkg/orchestrator/jsonstate/json_reactor_state.go deleted file mode 100644 index d16620b72fc..00000000000 --- a/pkg/orchestrator/jsonstate/json_reactor_state.go +++ /dev/null @@ -1,139 +0,0 @@ -// Copyright 2020 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package jsonstate - -import ( - "encoding/json" - "reflect" - - "github.com/pingcap/errors" - cerrors "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/orchestrator" - "github.com/pingcap/ticdc/pkg/orchestrator/util" -) - -// JSONReactorState models a single key whose value is a json object. -type JSONReactorState struct { - // jsonData stores an object serializable to a valid `value` corresponding to `key`. - jsonData interface{} - // modifiedJSONData is the modified snapshot of jsonData that has not been uploaded to Etcd. - modifiedJSONData interface{} - key util.EtcdKey - isUpdatedByReactor bool - patches []JSONPatchFunc -} - -// JSONPatchFunc is a function that updates an object that is serializable to JSON. -// It is okay to modify the input and return the input itself. -// Use ErrEtcdTryAgain and ErrEtcdIgnore to trigger Etcd transaction retries and to give up this update. -type JSONPatchFunc = func(data interface{}) (newData interface{}, err error) - -// NewJSONReactorState returns a new JSONReactorState. -// `data` needs to be a pointer to an object serializable in JSON. -func NewJSONReactorState(key string, data interface{}) (*JSONReactorState, error) { - tp := reflect.TypeOf(data) - if tp.Kind() != reflect.Ptr { - return nil, errors.Errorf("expected pointer type, got %T", data) - } - - copied := reflect.New(tp.Elem()).Interface() - deepCopy(data, copied) - - return &JSONReactorState{ - jsonData: data, - modifiedJSONData: copied, - key: util.NewEtcdKey(key), - isUpdatedByReactor: false, - }, nil -} - -// Update implements the ReactorState interface. -func (s *JSONReactorState) Update(key util.EtcdKey, value []byte, isInit bool) error { - if key != s.key { - return nil - } - - err := json.Unmarshal(value, s.jsonData) - if err != nil { - return errors.Trace(err) - } - - deepCopy(s.jsonData, s.modifiedJSONData) - s.isUpdatedByReactor = true - return nil -} - -// GetPatches implements the ReactorState interface.[]*orchestrator.DataPatch -func (s *JSONReactorState) GetPatches() []*orchestrator.DataPatch { - if len(s.patches) == 0 { - return []*orchestrator.DataPatch{} - } - - // We need to let the PatchFunc capture the array of JSONPatchFunc's, - // and let the DataPatch be the sole object referring to those JSONPatchFunc's, - // so that JSONReactorState does not have to worry about when to clean them up. - subPatches := make([]JSONPatchFunc, len(s.patches)) - copy(subPatches, s.patches) - s.patches = s.patches[:0] - - dataPatch := &orchestrator.DataPatch{ - Key: s.key, - Fun: func(old []byte) ([]byte, error) { - tp := reflect.TypeOf(s.jsonData) - oldStruct := reflect.New(tp.Elem()).Interface() - err := json.Unmarshal(old, oldStruct) - if err != nil { - return nil, errors.Trace(err) - } - - for _, f := range subPatches { - newStruct, err := f(oldStruct) - if err != nil { - if cerrors.ErrEtcdIgnore.Equal(errors.Cause(err)) { - continue - } - return nil, errors.Trace(err) - } - oldStruct = newStruct - } - - newBytes, err := json.Marshal(oldStruct) - if err != nil { - return nil, errors.Trace(err) - } - - return newBytes, nil - }, - } - - return []*orchestrator.DataPatch{dataPatch} -} - -// Inner returns a copy of the snapshot of the state. -// DO NOT modify the returned object. The modified object will not be persisted. -func (s *JSONReactorState) Inner() interface{} { - return s.modifiedJSONData -} - -// AddUpdateFunc accepts a JSONPatchFunc that updates the managed JSON-serializable object. -// If multiple JSONPatchFunc's are added within a Tick, they are applied in the order in which AddUpdateFunc has been called. -func (s *JSONReactorState) AddUpdateFunc(f JSONPatchFunc) { - s.patches = append(s.patches, f) -} - -// TODO optimize for performance -func deepCopy(a, b interface{}) { - byt, _ := json.Marshal(a) - _ = json.Unmarshal(byt, b) -} diff --git a/pkg/orchestrator/jsonstate/json_reactor_state_test.go b/pkg/orchestrator/jsonstate/json_reactor_state_test.go deleted file mode 100644 index 1d9c72726d7..00000000000 --- a/pkg/orchestrator/jsonstate/json_reactor_state_test.go +++ /dev/null @@ -1,144 +0,0 @@ -// Copyright 2020 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package jsonstate - -import ( - "context" - "testing" - "time" - - "github.com/pingcap/check" - "github.com/pingcap/log" - cerrors "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/etcd" - "github.com/pingcap/ticdc/pkg/orchestrator" - "github.com/pingcap/ticdc/pkg/util/testleak" - "github.com/prometheus/client_golang/prometheus" - "go.etcd.io/etcd/clientv3" - "go.uber.org/zap" - "golang.org/x/sync/errgroup" -) - -const ( - testEtcdKeyPrefix = "/cdc_etcd_worker_test" -) - -func Test(t *testing.T) { check.TestingT(t) } - -var _ = check.Suite(&jsonReactorStateSuite{}) - -type jsonReactorStateSuite struct { -} - -type simpleJSONRecord struct { - A int `json:"a"` - B int `json:"b"` - C int `json:"c"` -} - -type simpleJSONReactor struct { - state *JSONReactorState - oldVal int - id int -} - -func (r *simpleJSONReactor) Tick(_ context.Context, state orchestrator.ReactorState) (nextState orchestrator.ReactorState, err error) { - if r.oldVal >= 100 { - return r.state, cerrors.ErrReactorFinished - } - newState := state.(*JSONReactorState) - r.state = newState - - snapshot := r.state.Inner().(*simpleJSONRecord) - oldVal := 0 - switch r.id { - case 0: - oldVal = snapshot.A - r.state.AddUpdateFunc(func(data interface{}) (newData interface{}, err error) { - data.(*simpleJSONRecord).A++ - return data, nil - }) - case 1: - oldVal = snapshot.B - r.state.AddUpdateFunc(func(data interface{}) (newData interface{}, err error) { - data.(*simpleJSONRecord).B++ - return data, nil - }) - case 2: - oldVal = snapshot.C - r.state.AddUpdateFunc(func(data interface{}) (newData interface{}, err error) { - data.(*simpleJSONRecord).C++ - return data, nil - }) - } - if r.oldVal != oldVal { - log.Panic("validation failed", zap.Int("id", r.id), zap.Int("expected", r.oldVal), zap.Int("actual", oldVal)) - } - r.oldVal++ - return r.state, nil -} - -func (s *jsonReactorStateSuite) TestSimpleJSONRecord(c *check.C) { - defer testleak.AfterTest(c)() - dir := c.MkDir() - url, etcdServer, err := etcd.SetupEmbedEtcd(dir) - c.Assert(err, check.IsNil) - defer etcdServer.Close() - - newClient := func() *etcd.Client { - rawCli, err := clientv3.NewFromURLs([]string{url.String()}) - c.Check(err, check.IsNil) - return etcd.Wrap(rawCli, map[string]prometheus.Counter{}) - } - - ctx, cancel := context.WithTimeout(context.Background(), time.Minute*1) - defer cancel() - - cli := newClient() - _, err = cli.Put(ctx, testEtcdKeyPrefix+"/json", `{"a": 0, "b": 0, "c": 0}`) - c.Assert(err, check.IsNil) - - errg, ctx := errgroup.WithContext(ctx) - for i := 0; i < 3; i++ { - reactor := &simpleJSONReactor{ - state: nil, - oldVal: 0, - id: i, - } - - initState, err := NewJSONReactorState(testEtcdKeyPrefix+"/json", &simpleJSONRecord{}) - c.Assert(err, check.IsNil) - - etcdWorker, err := orchestrator.NewEtcdWorker(newClient(), testEtcdKeyPrefix, reactor, initState) - c.Assert(err, check.IsNil) - - errg.Go(func() error { - err := etcdWorker.Run(ctx, nil, 10*time.Millisecond) - if err != nil { - log.Error("etcdWorker returned error", zap.Error(err)) - } - return err - }) - } - - err = errg.Wait() - c.Assert(err, check.IsNil) -} - -func (s *jsonReactorStateSuite) TestNotPointerError(c *check.C) { - defer testleak.AfterTest(c)() - - _, err := NewJSONReactorState("/json", simpleJSONRecord{}) - c.Assert(err, check.NotNil) -} diff --git a/pkg/orchestrator/reactor_state_tester.go b/pkg/orchestrator/reactor_state_tester.go index 7f2a92acc29..5149cd34097 100644 --- a/pkg/orchestrator/reactor_state_tester.go +++ b/pkg/orchestrator/reactor_state_tester.go @@ -14,6 +14,7 @@ package orchestrator import ( + "github.com/pingcap/check" "github.com/pingcap/errors" cerrors "github.com/pingcap/ticdc/pkg/errors" "github.com/pingcap/ticdc/pkg/orchestrator/util" @@ -21,71 +22,95 @@ import ( // ReactorStateTester is a helper struct for unit-testing an implementer of ReactorState type ReactorStateTester struct { + c *check.C state ReactorState kvEntries map[string]string } // NewReactorStateTester creates a new ReactorStateTester -func NewReactorStateTester(state ReactorState, initKVEntries map[string]string) *ReactorStateTester { +func NewReactorStateTester(c *check.C, state ReactorState, initKVEntries map[string]string) *ReactorStateTester { + if initKVEntries == nil { + initKVEntries = make(map[string]string) + } + for k, v := range initKVEntries { + err := state.Update(util.NewEtcdKey(k), []byte(v), true) + c.Assert(err, check.IsNil) + } return &ReactorStateTester{ + c: c, state: state, kvEntries: initKVEntries, } } -// UpdateKeys is used to update keys in the mocked kv-store. -func (t *ReactorStateTester) UpdateKeys(updatedKeys map[string][]byte) error { - for key, value := range updatedKeys { - k := util.NewEtcdKey(key) - err := t.state.Update(k, value, false) - if err != nil { - return errors.Trace(err) - } - - if value != nil { - t.kvEntries[key] = string(value) - } else { - delete(t.kvEntries, key) - } +// Update is used to update keys in the mocked kv-store. +func (t *ReactorStateTester) Update(key string, value []byte) error { + k := util.NewEtcdKey(key) + err := t.state.Update(k, value, false) + if err != nil { + return errors.Trace(err) + } + if value != nil { + t.kvEntries[key] = string(value) + } else { + delete(t.kvEntries, key) } - return nil } // ApplyPatches calls the GetPatches method on the ReactorState and apply the changes to the mocked kv-store. func (t *ReactorStateTester) ApplyPatches() error { - patches := t.state.GetPatches() - mergedPatches := mergePatch(patches) - - for _, patch := range mergedPatches { - old, ok := t.kvEntries[patch.Key.String()] - var ( - newBytes []byte - err error - ) - if ok { - newBytes, err = patch.Fun([]byte(old)) - } else { - newBytes, err = patch.Fun(nil) - } - if cerrors.ErrEtcdIgnore.Equal(errors.Cause(err)) { - continue - } + patchGroups := t.state.GetPatches() + for _, patches := range patchGroups { + err := t.applyPatches(patches) if err != nil { - return errors.Trace(err) + return err } - err = t.state.Update(patch.Key, newBytes, false) - if err != nil { - return errors.Trace(err) + } + return nil +} + +func (t *ReactorStateTester) applyPatches(patches []DataPatch) error { +RetryLoop: + for { + tmpKVEntries := make(map[util.EtcdKey][]byte) + for k, v := range t.kvEntries { + tmpKVEntries[util.NewEtcdKey(k)] = []byte(v) } - if newBytes == nil { - delete(t.kvEntries, patch.Key.String()) - continue + changedSet := make(map[util.EtcdKey]struct{}) + for _, patch := range patches { + err := patch.Patch(tmpKVEntries, changedSet) + if cerrors.ErrEtcdIgnore.Equal(errors.Cause(err)) { + continue + } else if cerrors.ErrEtcdTryAgain.Equal(errors.Cause(err)) { + continue RetryLoop + } else if err != nil { + return errors.Trace(err) + } } - t.kvEntries[patch.Key.String()] = string(newBytes) + for k := range changedSet { + err := t.state.Update(k, tmpKVEntries[k], false) + if err != nil { + return err + } + if value := tmpKVEntries[k]; value != nil { + t.kvEntries[k.String()] = string(value) + } else { + delete(t.kvEntries, k.String()) + } + } + return nil } +} - return nil +// MustApplyPatches calls ApplyPatches and must successfully +func (t *ReactorStateTester) MustApplyPatches() { + t.c.Assert(t.ApplyPatches(), check.IsNil) +} + +// MustUpdate calls Update and must successfully +func (t *ReactorStateTester) MustUpdate(key string, value []byte) { + t.c.Assert(t.Update(key, value), check.IsNil) } // KVEntries returns the contents of the mocked KV store. diff --git a/pkg/pipeline/pipeline_test.go b/pkg/pipeline/pipeline_test.go index a4ec9e638bc..7681e2adb25 100644 --- a/pkg/pipeline/pipeline_test.go +++ b/pkg/pipeline/pipeline_test.go @@ -101,7 +101,7 @@ func (n *checkNode) Destroy(ctx NodeContext) error { func (s *pipelineSuite) TestPipelineUsage(c *check.C) { defer testleak.AfterTest(c)() - ctx := context.NewContext(stdCtx.Background(), &context.Vars{}) + ctx := context.NewContext(stdCtx.Background(), &context.GlobalVars{}) ctx, cancel := context.WithCancel(ctx) ctx = context.WithErrorHandler(ctx, func(err error) error { c.Fatal(err) @@ -211,7 +211,7 @@ func (n *errorNode) Destroy(ctx NodeContext) error { func (s *pipelineSuite) TestPipelineError(c *check.C) { defer testleak.AfterTest(c)() - ctx := context.NewContext(stdCtx.Background(), &context.Vars{}) + ctx := context.NewContext(stdCtx.Background(), &context.GlobalVars{}) ctx, cancel := context.WithCancel(ctx) defer cancel() ctx = context.WithErrorHandler(ctx, func(err error) error { @@ -293,7 +293,7 @@ func (n *throwNode) Destroy(ctx NodeContext) error { func (s *pipelineSuite) TestPipelineThrow(c *check.C) { defer testleak.AfterTest(c)() - ctx := context.NewContext(stdCtx.Background(), &context.Vars{}) + ctx := context.NewContext(stdCtx.Background(), &context.GlobalVars{}) ctx, cancel := context.WithCancel(ctx) defer cancel() var errs []error @@ -345,7 +345,7 @@ func (s *pipelineSuite) TestPipelineThrow(c *check.C) { func (s *pipelineSuite) TestPipelineAppendNode(c *check.C) { defer testleak.AfterTest(c)() - ctx := context.NewContext(stdCtx.Background(), &context.Vars{}) + ctx := context.NewContext(stdCtx.Background(), &context.GlobalVars{}) ctx, cancel := context.WithCancel(ctx) ctx = context.WithErrorHandler(ctx, func(err error) error { c.Fatal(err) @@ -456,7 +456,7 @@ func (s *pipelineSuite) TestPipelinePanic(c *check.C) { panicInfo := recover().(string) c.Assert(panicInfo, check.Equals, "panic in panicNode") }() - ctx := context.NewContext(stdCtx.Background(), &context.Vars{}) + ctx := context.NewContext(stdCtx.Background(), &context.GlobalVars{}) ctx, cancel := context.WithCancel(ctx) defer cancel() ctx = context.WithErrorHandler(ctx, func(err error) error { diff --git a/pkg/util/gc_service.go b/pkg/util/gc_service.go index 24b6fb2cf5c..28f260b94de 100644 --- a/pkg/util/gc_service.go +++ b/pkg/util/gc_service.go @@ -23,15 +23,15 @@ import ( const ( // cdcChangefeedCreatingServiceGCSafePointID is service GC safe point ID - cdcChangefeedCreatingServiceGCSafePointID = "ticdc-changefeed-creating" + cdcChangefeedCreatingServiceGCSafePointID = "ticdc-creating-" // cdcChangefeedCreatingServiceGCSafePointTTL is service GC safe point TTL cdcChangefeedCreatingServiceGCSafePointTTL = 10 * 60 // 10 mins ) // CheckSafetyOfStartTs checks if the startTs less than the minimum of Service-GC-Ts // and this function will update the service GC to startTs -func CheckSafetyOfStartTs(ctx context.Context, pdCli pd.Client, startTs uint64) error { - minServiceGCTs, err := pdCli.UpdateServiceGCSafePoint(ctx, cdcChangefeedCreatingServiceGCSafePointID, +func CheckSafetyOfStartTs(ctx context.Context, pdCli pd.Client, changefeedID string, startTs uint64) error { + minServiceGCTs, err := pdCli.UpdateServiceGCSafePoint(ctx, cdcChangefeedCreatingServiceGCSafePointID+changefeedID, cdcChangefeedCreatingServiceGCSafePointTTL, startTs) if err != nil { return errors.Trace(err) diff --git a/pkg/util/gc_service_test.go b/pkg/util/gc_service_test.go index 322f1903eb5..84bbf3a82b6 100644 --- a/pkg/util/gc_service_test.go +++ b/pkg/util/gc_service_test.go @@ -34,13 +34,18 @@ func (s *gcServiceSuite) TestCheckSafetyOfStartTs(c *check.C) { defer testleak.AfterTest(c)() ctx := context.Background() s.pdCli.UpdateServiceGCSafePoint(ctx, "service1", 10, 60) //nolint:errcheck - err := CheckSafetyOfStartTs(ctx, s.pdCli, 50) + err := CheckSafetyOfStartTs(ctx, s.pdCli, "changefeed1", 50) c.Assert(err.Error(), check.Equals, "[CDC:ErrStartTsBeforeGC]fail to create changefeed because start-ts 50 is earlier than GC safepoint at 60") s.pdCli.UpdateServiceGCSafePoint(ctx, "service2", 10, 80) //nolint:errcheck s.pdCli.UpdateServiceGCSafePoint(ctx, "service3", 10, 70) //nolint:errcheck - err = CheckSafetyOfStartTs(ctx, s.pdCli, 65) + err = CheckSafetyOfStartTs(ctx, s.pdCli, "changefeed2", 65) c.Assert(err, check.IsNil) - c.Assert(s.pdCli.serviceSafePoint, check.DeepEquals, map[string]uint64{"service1": 60, "service2": 80, "service3": 70, "ticdc-changefeed-creating": 65}) + c.Assert(s.pdCli.serviceSafePoint, check.DeepEquals, map[string]uint64{ + "service1": 60, + "service2": 80, + "service3": 70, + "ticdc-creating-changefeed2": 65, + }) } type mockPdClientForServiceGCSafePoint struct { diff --git a/testing_utils/cdc_state_checker/state.go b/testing_utils/cdc_state_checker/state.go index da7b84f1ac5..ffc371a5c35 100644 --- a/testing_utils/cdc_state_checker/state.go +++ b/testing_utils/cdc_state_checker/state.go @@ -227,6 +227,6 @@ func (s *cdcReactorState) Update(key util.EtcdKey, value []byte, isInit bool) er return nil } -func (s *cdcReactorState) GetPatches() []*orchestrator.DataPatch { +func (s *cdcReactorState) GetPatches() [][]orchestrator.DataPatch { return nil } diff --git a/tests/_utils/start_tidb_cluster_impl b/tests/_utils/start_tidb_cluster_impl index 762d5015291..44c9ef8af06 100755 --- a/tests/_utils/start_tidb_cluster_impl +++ b/tests/_utils/start_tidb_cluster_impl @@ -233,6 +233,9 @@ sleep 2 run_sql "update mysql.tidb set variable_value='60m' where variable_name='tikv_gc_life_time';" ${UP_TIDB_HOST} ${UP_TIDB_PORT} run_sql "update mysql.tidb set variable_value='60m' where variable_name='tikv_gc_life_time';" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} +run_sql "CREATE user 'normal'@'%' identified by '123456';" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} +run_sql "GRANT select,insert,update,delete,index,create,drop,alter,create view ON *.* TO 'normal'@'%';" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} +run_sql "FLUSH privileges" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} cat - >"$OUT_DIR/tiflash-config.toml" <&1|tail -n2|head -n1|awk '{print $2}') run_sql "CREATE DATABASE capture_suicide_while_balance_table;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} diff --git a/tests/cdc/run.sh b/tests/cdc/run.sh index 64f8fb9ef93..46c24454080 100755 --- a/tests/cdc/run.sh +++ b/tests/cdc/run.sh @@ -24,7 +24,7 @@ function prepare() { TOPIC_NAME="ticdc-cdc-test-$RANDOM" case $SINK_TYPE in kafka) SINK_URI="kafka://127.0.0.1:9092/$TOPIC_NAME?partition-num=4&kafka-version=${KAFKA_VERSION}";; - *) SINK_URI="mysql://root@127.0.0.1:3306/";; + *) SINK_URI="mysql://normal:123456@127.0.0.1:3306/";; esac run_cdc_cli changefeed create --start-ts=$start_ts --sink-uri="$SINK_URI" if [ "$SINK_TYPE" == "kafka" ]; then diff --git a/tests/changefeed_auto_stop/run.sh b/tests/changefeed_auto_stop/run.sh index 7d1f71c45e4..7b83a6a17fe 100755 --- a/tests/changefeed_auto_stop/run.sh +++ b/tests/changefeed_auto_stop/run.sh @@ -45,14 +45,14 @@ function run() { TOPIC_NAME="ticdc-changefeed-auto-stop-test-$RANDOM" case $SINK_TYPE in kafka) SINK_URI="kafka://127.0.0.1:9092/$TOPIC_NAME?partition-num=4&kafka-version=${KAFKA_VERSION}";; - *) SINK_URI="mysql://root@127.0.0.1:3306/";; + *) SINK_URI="mysql://normal:123456@127.0.0.1:3306/";; esac changefeedid=$(cdc cli changefeed create --pd="http://${UP_PD_HOST_1}:${UP_PD_PORT_1}" --start-ts=$start_ts --sink-uri="$SINK_URI" 2>&1|tail -n2|head -n1|awk '{print $2}') if [ "$SINK_TYPE" == "kafka" ]; then run_kafka_consumer $WORK_DIR "kafka://127.0.0.1:9092/$TOPIC_NAME?partition-num=4&version=${KAFKA_VERSION}" fi - ensure 10 check_changefeed_state ${UP_PD_HOST_1}:${UP_PD_PORT_1} ${changefeedid} "stopped" + ensure 20 check_changefeed_state ${UP_PD_HOST_1}:${UP_PD_PORT_1} ${changefeedid} "stopped" cdc cli changefeed resume --changefeed-id=${changefeedid} --pd="http://${UP_PD_HOST_1}:${UP_PD_PORT_1}" for i in $(seq $DB_COUNT); do diff --git a/tests/changefeed_error/run.sh b/tests/changefeed_error/run.sh index 8aac6d082fc..25efe2be35a 100755 --- a/tests/changefeed_error/run.sh +++ b/tests/changefeed_error/run.sh @@ -121,7 +121,7 @@ function run() { TOPIC_NAME="ticdc-sink-retry-test-$RANDOM" case $SINK_TYPE in kafka) SINK_URI="kafka://127.0.0.1:9092/$TOPIC_NAME?partition-num=4&kafka-version=${KAFKA_VERSION}";; - *) SINK_URI="mysql://root@127.0.0.1:3306/?max-txn-row=1";; + *) SINK_URI="mysql://normal:123456@127.0.0.1:3306/?max-txn-row=1";; esac changefeedid=$(cdc cli changefeed create --start-ts=$start_ts --sink-uri="$SINK_URI" 2>&1|tail -n2|head -n1|awk '{print $2}') if [ "$SINK_TYPE" == "kafka" ]; then diff --git a/tests/changefeed_finish/run.sh b/tests/changefeed_finish/run.sh index 0bd535fc6c5..62bb11ff761 100755 --- a/tests/changefeed_finish/run.sh +++ b/tests/changefeed_finish/run.sh @@ -42,7 +42,7 @@ function run() { TOPIC_NAME="ticdc-changefeed-pause-resume-$RANDOM" case $SINK_TYPE in kafka) SINK_URI="kafka://127.0.0.1:9092/$TOPIC_NAME?partition-num=4&kafka-version=${KAFKA_VERSION}";; - *) SINK_URI="mysql://root@127.0.0.1:3306/?max-txn-row=1";; + *) SINK_URI="mysql://normal:123456@127.0.0.1:3306/?max-txn-row=1";; esac if [ "$SINK_TYPE" == "kafka" ]; then run_kafka_consumer $WORK_DIR "kafka://127.0.0.1:9092/$TOPIC_NAME?partition-num=4&version=${KAFKA_VERSION}" diff --git a/tests/changefeed_pause_resume/run.sh b/tests/changefeed_pause_resume/run.sh index f1b823e23f2..1d11a6a3a44 100755 --- a/tests/changefeed_pause_resume/run.sh +++ b/tests/changefeed_pause_resume/run.sh @@ -18,7 +18,7 @@ function run() { TOPIC_NAME="ticdc-changefeed-pause-resume-$RANDOM" case $SINK_TYPE in kafka) SINK_URI="kafka://127.0.0.1:9092/$TOPIC_NAME?partition-num=4&kafka-version=${KAFKA_VERSION}";; - *) SINK_URI="mysql://root@127.0.0.1:3306/?max-txn-row=1";; + *) SINK_URI="mysql://normal:123456@127.0.0.1:3306/?max-txn-row=1";; esac if [ "$SINK_TYPE" == "kafka" ]; then run_kafka_consumer $WORK_DIR "kafka://127.0.0.1:9092/$TOPIC_NAME?partition-num=4&version=${KAFKA_VERSION}" diff --git a/tests/changefeed_reconstruct/run.sh b/tests/changefeed_reconstruct/run.sh index 504c1c0e1a8..54c79c64b0a 100755 --- a/tests/changefeed_reconstruct/run.sh +++ b/tests/changefeed_reconstruct/run.sh @@ -41,7 +41,7 @@ function run() { TOPIC_NAME="ticdc-changefeed-reconstruct-$RANDOM" case $SINK_TYPE in kafka) SINK_URI="kafka://127.0.0.1:9092/$TOPIC_NAME?partition-num=4&kafka-version=${KAFKA_VERSION}";; - *) SINK_URI="mysql://root@127.0.0.1:3306/?max-txn-row=1";; + *) SINK_URI="mysql://normal:123456@127.0.0.1:3306/?max-txn-row=1";; esac if [ "$SINK_TYPE" == "kafka" ]; then run_kafka_consumer $WORK_DIR "kafka://127.0.0.1:9092/$TOPIC_NAME?partition-num=4&version=${KAFKA_VERSION}" diff --git a/tests/cli/run.sh b/tests/cli/run.sh index 034802ed6e9..8154e2ed33b 100644 --- a/tests/cli/run.sh +++ b/tests/cli/run.sh @@ -49,7 +49,7 @@ function run() { TOPIC_NAME="ticdc-cli-test-$RANDOM" case $SINK_TYPE in kafka) SINK_URI="kafka://127.0.0.1:9092/$TOPIC_NAME?partition-num=4&kafka-version=${KAFKA_VERSION}";; - *) SINK_URI="mysql://root@127.0.0.1:3306/";; + *) SINK_URI="mysql://normal:123456@127.0.0.1:3306/";; esac uuid="custom-changefeed-name" diff --git a/tests/common_1/run.sh b/tests/common_1/run.sh index 49058654931..db559e8441b 100644 --- a/tests/common_1/run.sh +++ b/tests/common_1/run.sh @@ -33,6 +33,8 @@ function run() { run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY + # this test contains `recover table`, which requires super privilege, so we + # can't use the normal user TOPIC_NAME="ticdc-common-1-test-$RANDOM" case $SINK_TYPE in kafka) SINK_URI="kafka://127.0.0.1:9092/$TOPIC_NAME?partition-num=4&kafka-version=${KAFKA_VERSION}";; diff --git a/tests/cyclic_abc/conf/changefeed.toml b/tests/cyclic_abc/conf/changefeed.toml new file mode 100644 index 00000000000..d3efede3139 --- /dev/null +++ b/tests/cyclic_abc/conf/changefeed.toml @@ -0,0 +1,2 @@ +[filter] +rules = ['test.*'] diff --git a/tests/ddl_puller_lag/run.sh b/tests/ddl_puller_lag/run.sh index fe54d9107bd..06cdd0e1733 100644 --- a/tests/ddl_puller_lag/run.sh +++ b/tests/ddl_puller_lag/run.sh @@ -27,7 +27,7 @@ function prepare() { TOPIC_NAME="ticdc-ddl-puller-lag-test-$RANDOM" case $SINK_TYPE in kafka) SINK_URI="kafka+ssl://127.0.0.1:9092/$TOPIC_NAME?partition-num=4&kafka-client-id=ddl_puller_lag&kafka-version=${KAFKA_VERSION}";; - *) SINK_URI="mysql+ssl://root@127.0.0.1:3306/";; + *) SINK_URI="mysql+ssl://normal:123456@127.0.0.1:3306/";; esac run_cdc_cli changefeed create --start-ts=$start_ts --sink-uri="$SINK_URI" if [ "$SINK_TYPE" == "kafka" ]; then diff --git a/tests/ddl_reentrant/run.sh b/tests/ddl_reentrant/run.sh index 32355702e91..74ff8ca3319 100644 --- a/tests/ddl_reentrant/run.sh +++ b/tests/ddl_reentrant/run.sh @@ -47,6 +47,8 @@ function complete_ddls() { } changefeedid="" +# this test contains `recover table`, which requires super privilege, so we +# can't use the normal user SINK_URI="mysql://root@127.0.0.1:3306/" function check_ts_forward() { diff --git a/tests/ddl_sequence/run.sh b/tests/ddl_sequence/run.sh index 13fb1be60c7..0be883c7651 100644 --- a/tests/ddl_sequence/run.sh +++ b/tests/ddl_sequence/run.sh @@ -23,7 +23,7 @@ function run() { TOPIC_NAME="ticdc-ddl-sequence-test-$RANDOM" case $SINK_TYPE in kafka) SINK_URI="kafka://127.0.0.1:9092/$TOPIC_NAME?partition-num=4&kafka-version=${KAFKA_VERSION}";; - *) SINK_URI="mysql://root@127.0.0.1:3306/";; + *) SINK_URI="mysql://normal:123456@127.0.0.1:3306/";; esac run_cdc_cli changefeed create --start-ts=$start_ts --sink-uri="$SINK_URI" if [ "$SINK_TYPE" == "kafka" ]; then diff --git a/tests/drop_many_tables/run.sh b/tests/drop_many_tables/run.sh index 93efa49d139..bfee5ab3877 100644 --- a/tests/drop_many_tables/run.sh +++ b/tests/drop_many_tables/run.sh @@ -23,7 +23,7 @@ function run() { TOPIC_NAME="ticdc-drop-tables-test-$RANDOM" case $SINK_TYPE in kafka) SINK_URI="kafka://127.0.0.1:9092/$TOPIC_NAME?partition-num=4&kafka-version=${KAFKA_VERSION}";; - *) SINK_URI="mysql://root@127.0.0.1:3306/";; + *) SINK_URI="mysql://normal:123456@127.0.0.1:3306/";; esac run_cdc_cli changefeed create --start-ts=$start_ts --sink-uri="$SINK_URI" if [ "$SINK_TYPE" == "kafka" ]; then diff --git a/tests/force_replicate_table/run.sh b/tests/force_replicate_table/run.sh index fee1fec5528..dd293582c82 100755 --- a/tests/force_replicate_table/run.sh +++ b/tests/force_replicate_table/run.sh @@ -60,7 +60,7 @@ function run() { TOPIC_NAME="ticdc-force_replicate_table-test-$RANDOM" case $SINK_TYPE in kafka) SINK_URI="kafka://127.0.0.1:9092/$TOPIC_NAME?partition-num=4&kafka-version=${KAFKA_VERSION}";; - *) SINK_URI="mysql://root@127.0.0.1:3306/?safe-mode=true";; + *) SINK_URI="mysql://normal:123456@127.0.0.1:3306/?safe-mode=true";; esac cdc cli changefeed create --start-ts=$start_ts --sink-uri="$SINK_URI" --config $CUR/conf/changefeed.toml if [ "$SINK_TYPE" == "kafka" ]; then diff --git a/tests/gc_safepoint/run.sh b/tests/gc_safepoint/run.sh index ba35c6d3e0c..e2443a315da 100755 --- a/tests/gc_safepoint/run.sh +++ b/tests/gc_safepoint/run.sh @@ -77,7 +77,7 @@ function run() { TOPIC_NAME="ticdc-gc-safepoint-$RANDOM" case $SINK_TYPE in kafka) SINK_URI="kafka://127.0.0.1:9092/$TOPIC_NAME?partition-num=4&kafka-version=${KAFKA_VERSION}";; - *) SINK_URI="mysql://root@127.0.0.1:3306/?max-txn-row=1";; + *) SINK_URI="mysql://normal:123456@127.0.0.1:3306/?max-txn-row=1";; esac if [ "$SINK_TYPE" == "kafka" ]; then run_kafka_consumer $WORK_DIR "kafka://127.0.0.1:9092/$TOPIC_NAME?partition-num=4&version=${KAFKA_VERSION}" diff --git a/tests/generate_column/run.sh b/tests/generate_column/run.sh index aa85babacea..4b0edfdead0 100644 --- a/tests/generate_column/run.sh +++ b/tests/generate_column/run.sh @@ -23,7 +23,7 @@ function run() { TOPIC_NAME="ticdc-generate-column-test-$RANDOM" case $SINK_TYPE in kafka) SINK_URI="kafka://127.0.0.1:9092/$TOPIC_NAME?partition-num=4&kafka-version=${KAFKA_VERSION}";; - *) SINK_URI="mysql://root@127.0.0.1:3306/";; + *) SINK_URI="mysql://normal:123456@127.0.0.1:3306/";; esac run_cdc_cli changefeed create --start-ts=$start_ts --sink-uri="$SINK_URI" if [ "$SINK_TYPE" == "kafka" ]; then diff --git a/tests/kill_owner_with_ddl/run.sh b/tests/kill_owner_with_ddl/run.sh index 5b982cf3e85..0913f5e555b 100755 --- a/tests/kill_owner_with_ddl/run.sh +++ b/tests/kill_owner_with_ddl/run.sh @@ -46,7 +46,7 @@ function run() { cd $WORK_DIR pd_addr="http://$UP_PD_HOST_1:$UP_PD_PORT_1" - SINK_URI="mysql://root@127.0.0.1:3306/?max-txn-row=1" + SINK_URI="mysql://normal:123456@127.0.0.1:3306/?max-txn-row=1" run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY --addr "127.0.0.1:8300" --pd $pd_addr cdc cli changefeed create --pd=$pd_addr --sink-uri="$SINK_URI" diff --git a/tests/kv_client_stream_reconnect/run.sh b/tests/kv_client_stream_reconnect/run.sh index c3bd400884f..7d2e014b4f9 100644 --- a/tests/kv_client_stream_reconnect/run.sh +++ b/tests/kv_client_stream_reconnect/run.sh @@ -20,7 +20,7 @@ function run() { TOPIC_NAME="kv-client-stream-reconnect-$RANDOM" case $SINK_TYPE in kafka) SINK_URI="kafka://127.0.0.1:9092/$TOPIC_NAME?partition-num=4&kafka-version=${KAFKA_VERSION}";; - *) SINK_URI="mysql://root@127.0.0.1:3306/?max-txn-row=1";; + *) SINK_URI="mysql://normal:123456@127.0.0.1:3306/?max-txn-row=1";; esac if [ "$SINK_TYPE" == "kafka" ]; then run_kafka_consumer $WORK_DIR "kafka://127.0.0.1:9092/$TOPIC_NAME?partition-num=4&version=${KAFKA_VERSION}" diff --git a/tests/many_pk_or_uk/run.sh b/tests/many_pk_or_uk/run.sh index 6fc0654bbf9..f097eb61dc7 100755 --- a/tests/many_pk_or_uk/run.sh +++ b/tests/many_pk_or_uk/run.sh @@ -24,7 +24,7 @@ function prepare() { TOPIC_NAME="ticdc-many-pk-or-uk-test-$RANDOM" case $SINK_TYPE in kafka) SINK_URI="kafka://127.0.0.1:9092/$TOPIC_NAME?partition-num=4&kafka-version=${KAFKA_VERSION}";; - *) SINK_URI="mysql://root@127.0.0.1:3306/";; + *) SINK_URI="mysql://normal:123456@127.0.0.1:3306/";; esac run_cdc_cli changefeed create --start-ts=$start_ts --sink-uri="$SINK_URI" if [ "$SINK_TYPE" == "kafka" ]; then diff --git a/tests/move_table/conf/workload b/tests/move_table/conf/workload index 5b9ca3189fc..128a6404b53 100644 --- a/tests/move_table/conf/workload +++ b/tests/move_table/conf/workload @@ -1,5 +1,5 @@ threadcount=10 -recordcount=60000 +recordcount=6000 operationcount=0 workload=core diff --git a/tests/move_table/run.sh b/tests/move_table/run.sh index 10784fab78d..aaf0ea7b307 100644 --- a/tests/move_table/run.sh +++ b/tests/move_table/run.sh @@ -24,7 +24,7 @@ function run() { TOPIC_NAME="ticdc-move-table-test-$RANDOM" case $SINK_TYPE in kafka) SINK_URI="kafka://127.0.0.1:9092/$TOPIC_NAME?partition-num=4&kafka-version=${KAFKA_VERSION}";; - *) SINK_URI="mysql://root@127.0.0.1:3306/?max-txn-row=1";; + *) SINK_URI="mysql://normal:123456@127.0.0.1:3306/?max-txn-row=1";; esac run_cdc_cli changefeed create --start-ts=$start_ts --sink-uri="$SINK_URI" diff --git a/tests/multi_capture/run.sh b/tests/multi_capture/run.sh index 5f20c92537d..13691a9bdf7 100755 --- a/tests/multi_capture/run.sh +++ b/tests/multi_capture/run.sh @@ -36,7 +36,7 @@ function run() { TOPIC_NAME="ticdc-multi-capture-test-$RANDOM" case $SINK_TYPE in kafka) SINK_URI="kafka://127.0.0.1:9092/$TOPIC_NAME?partition-num=4&kafka-version=${KAFKA_VERSION}";; - *) SINK_URI="mysql://root@127.0.0.1:3306/";; + *) SINK_URI="mysql://normal:123456@127.0.0.1:3306/";; esac run_cdc_cli changefeed create --start-ts=$start_ts --sink-uri="$SINK_URI" if [ "$SINK_TYPE" == "kafka" ]; then diff --git a/tests/multi_source/run.sh b/tests/multi_source/run.sh index 1e25c48ba18..5e5d28af876 100755 --- a/tests/multi_source/run.sh +++ b/tests/multi_source/run.sh @@ -24,7 +24,7 @@ function prepare() { TOPIC_NAME="ticdc-multi-source-test-$RANDOM" case $SINK_TYPE in kafka) SINK_URI="kafka://127.0.0.1:9092/$TOPIC_NAME?partition-num=4&kafka-version=${KAFKA_VERSION}";; - *) SINK_URI="mysql://root@127.0.0.1:3306/";; + *) SINK_URI="mysql://normal:123456@127.0.0.1:3306/";; esac run_cdc_cli changefeed create --start-ts=$start_ts --sink-uri="$SINK_URI" if [ "$SINK_TYPE" == "kafka" ]; then diff --git a/tests/new_ci_collation/run.sh b/tests/new_ci_collation/run.sh index 9283127f8c4..c8c9e3dee2a 100755 --- a/tests/new_ci_collation/run.sh +++ b/tests/new_ci_collation/run.sh @@ -23,7 +23,7 @@ function run() { TOPIC_NAME="ticdc-new_ci_collation-test-$RANDOM" case $SINK_TYPE in kafka) SINK_URI="kafka://127.0.0.1:9092/$TOPIC_NAME?partition-num=4&kafka-version=${KAFKA_VERSION}";; - *) SINK_URI="mysql://root@127.0.0.1:3306/?safe-mode=true";; + *) SINK_URI="mysql://normal:123456@127.0.0.1:3306/?safe-mode=true";; esac cdc cli changefeed create --start-ts=$start_ts --sink-uri="$SINK_URI" --config $CUR/conf/changefeed.toml if [ "$SINK_TYPE" == "kafka" ]; then diff --git a/tests/owner_remove_table_error/run.sh b/tests/owner_remove_table_error/run.sh index d78907c44e8..3f75d9df659 100644 --- a/tests/owner_remove_table_error/run.sh +++ b/tests/owner_remove_table_error/run.sh @@ -24,7 +24,7 @@ function run() { cd $WORK_DIR pd_addr="http://$UP_PD_HOST_1:$UP_PD_PORT_1" - SINK_URI="mysql://root@127.0.0.1:3306/?max-txn-row=1"; + SINK_URI="mysql://normal:123456@127.0.0.1:3306/?max-txn-row=1"; export GO_FAILPOINTS='github.com/pingcap/ticdc/cdc/OwnerRemoveTableError=1*return(true)' run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY --addr "127.0.0.1:8300" --pd $pd_addr diff --git a/tests/partition_table/run.sh b/tests/partition_table/run.sh index a124ab28719..ad413890caa 100644 --- a/tests/partition_table/run.sh +++ b/tests/partition_table/run.sh @@ -23,7 +23,7 @@ function run() { TOPIC_NAME="ticdc-partition-table-test-$RANDOM" case $SINK_TYPE in kafka) SINK_URI="kafka://127.0.0.1:9092/$TOPIC_NAME?partition-num=4&kafka-version=${KAFKA_VERSION}";; - *) SINK_URI="mysql://root@127.0.0.1:3306/";; + *) SINK_URI="mysql://normal:123456@127.0.0.1:3306/";; esac run_cdc_cli changefeed create --start-ts=$start_ts --sink-uri="$SINK_URI" if [ "$SINK_TYPE" == "kafka" ]; then diff --git a/tests/processor_err_chan/run.sh b/tests/processor_err_chan/run.sh index 6aa1d05464d..61f591aff5d 100644 --- a/tests/processor_err_chan/run.sh +++ b/tests/processor_err_chan/run.sh @@ -36,11 +36,11 @@ function run() { pd_addr="http://$UP_PD_HOST_1:$UP_PD_PORT_1" TOPIC_NAME="ticdc-processor-err-chan-$RANDOM" case $SINK_TYPE in - kafka) SINK_URI="kafka://127.0.0.1:9092/$TOPIC_NAME?partition-num=4";; - *) SINK_URI="mysql://root@127.0.0.1:3306/?max-txn-row=1";; + kafka) SINK_URI="kafka://127.0.0.1:9092/$TOPIC_NAME?partition-num=4&kafka-version=${KAFKA_VERSION}";; + *) SINK_URI="mysql://normal:123456@127.0.0.1:3306/?max-txn-row=1";; esac if [ "$SINK_TYPE" == "kafka" ]; then - run_kafka_consumer $WORK_DIR "kafka://127.0.0.1:9092/$TOPIC_NAME?partition-num=4" + run_kafka_consumer $WORK_DIR "kafka://127.0.0.1:9092/$TOPIC_NAME?partition-num=4&version=${KAFKA_VERSION}" fi run_sql "CREATE DATABASE processor_err_chan;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} diff --git a/tests/processor_panic/run.sh b/tests/processor_panic/run.sh index 202a223dbfe..44be40b51a4 100644 --- a/tests/processor_panic/run.sh +++ b/tests/processor_panic/run.sh @@ -29,7 +29,7 @@ function prepare() { TOPIC_NAME="ticdc-processor-panic-test-$RANDOM" case $SINK_TYPE in kafka) SINK_URI="kafka://127.0.0.1:9092/$TOPIC_NAME?partition-num=4&kafka-client-id=cdc_test_processor_panic&kafka-version=${KAFKA_VERSION}";; - *) SINK_URI="mysql://root@127.0.0.1:3306/";; + *) SINK_URI="mysql://normal:123456@127.0.0.1:3306/";; esac run_cdc_cli changefeed create --start-ts=$start_ts --sink-uri="$SINK_URI" if [ "$SINK_TYPE" == "kafka" ]; then diff --git a/tests/processor_resolved_ts_fallback/run.sh b/tests/processor_resolved_ts_fallback/run.sh index d61b92b0088..8d95edf7836 100755 --- a/tests/processor_resolved_ts_fallback/run.sh +++ b/tests/processor_resolved_ts_fallback/run.sh @@ -22,7 +22,7 @@ function run() { TOPIC_NAME="ticdc-processor-resolved-ts-fallback-test-$RANDOM" case $SINK_TYPE in kafka) SINK_URI="kafka://kafka01:9092/$TOPIC_NAME?partition-num=4&kafka-version=${KAFKA_VERSION}";; - *) SINK_URI="mysql://root@127.0.0.1:3306/";; + *) SINK_URI="mysql://normal:123456@127.0.0.1:3306/";; esac run_cdc_cli changefeed create --sink-uri="$SINK_URI" if [ "$SINK_TYPE" == "kafka" ]; then diff --git a/tests/processor_stop_delay/run.sh b/tests/processor_stop_delay/run.sh index 78be2277eab..6ced3fa5b18 100644 --- a/tests/processor_stop_delay/run.sh +++ b/tests/processor_stop_delay/run.sh @@ -18,7 +18,7 @@ function run() { TOPIC_NAME="ticdc-processor-stop-delay-$RANDOM" case $SINK_TYPE in kafka) SINK_URI="kafka://127.0.0.1:9092/$TOPIC_NAME?partition-num=4&kafka-version=${KAFKA_VERSION}";; - *) SINK_URI="mysql://root@127.0.0.1:3306/?max-txn-row=1";; + *) SINK_URI="mysql://normal:123456@127.0.0.1:3306/?max-txn-row=1";; esac if [ "$SINK_TYPE" == "kafka" ]; then run_kafka_consumer $WORK_DIR "kafka://127.0.0.1:9092/$TOPIC_NAME?partition-num=4&version=${KAFKA_VERSION}" diff --git a/tests/region_merge/run.sh b/tests/region_merge/run.sh index 863fe45f6ea..8e0ed9f85d8 100644 --- a/tests/region_merge/run.sh +++ b/tests/region_merge/run.sh @@ -38,7 +38,7 @@ function run() { TOPIC_NAME="ticdc-region-merge-test-$RANDOM" case $SINK_TYPE in kafka) SINK_URI="kafka://127.0.0.1:9092/$TOPIC_NAME?partition-num=4&kafka-version=${KAFKA_VERSION}";; - *) SINK_URI="mysql://root@127.0.0.1:3306/";; + *) SINK_URI="mysql://normal:123456@127.0.0.1:3306/";; esac run_cdc_cli changefeed create --sink-uri="$SINK_URI" if [ "$SINK_TYPE" == "kafka" ]; then diff --git a/tests/resolve_lock/run.sh b/tests/resolve_lock/run.sh index 953a73a1aae..f2b7e411123 100755 --- a/tests/resolve_lock/run.sh +++ b/tests/resolve_lock/run.sh @@ -24,7 +24,7 @@ function prepare() { TOPIC_NAME="ticdc-resolve-lock-test-$RANDOM" case $SINK_TYPE in kafka) SINK_URI="kafka://127.0.0.1:9092/$TOPIC_NAME?partition-num=4&kafka-version=${KAFKA_VERSION}";; - *) SINK_URI="mysql://root@127.0.0.1:3306/tidb-txn-mode=pessimistic";; + *) SINK_URI="mysql://normal:123456@127.0.0.1:3306/tidb-txn-mode=pessimistic";; esac run_cdc_cli changefeed create --start-ts=$start_ts --sink-uri="$SINK_URI" if [ "$SINK_TYPE" == "kafka" ]; then diff --git a/tests/row_format/run.sh b/tests/row_format/run.sh index a05e836676c..0b3bd078760 100644 --- a/tests/row_format/run.sh +++ b/tests/row_format/run.sh @@ -23,7 +23,7 @@ function run() { TOPIC_NAME="ticdc-row-format-test-$RANDOM" case $SINK_TYPE in kafka) SINK_URI="kafka://127.0.0.1:9092/$TOPIC_NAME?partition-num=4&kafka-version=${KAFKA_VERSION}";; - *) SINK_URI="mysql://root@127.0.0.1:3306/";; + *) SINK_URI="mysql://normal:123456@127.0.0.1:3306/";; esac run_cdc_cli changefeed create --start-ts=$start_ts --sink-uri="$SINK_URI" if [ "$SINK_TYPE" == "kafka" ]; then diff --git a/tests/run.sh b/tests/run.sh index 263d4e7a50f..4203cf3e55e 100755 --- a/tests/run.sh +++ b/tests/run.sh @@ -22,7 +22,7 @@ if [ "${1-}" = '--debug' ]; then cdc server --log-file $WORK_DIR/cdc.log --log-level debug --addr 127.0.0.1:8300 > $WORK_DIR/stdout.log 2>&1 & sleep 1 - cdc cli changefeed create --sink-uri="mysql://root@127.0.0.1:3306/" + cdc cli changefeed create --sink-uri="mysql://normal:123456@127.0.0.1:3306/" echo 'You may now debug from another terminal. Press [ENTER] to exit.' read line diff --git a/tests/simple/run.sh b/tests/simple/run.sh index 54d8f052661..cf3d2686142 100644 --- a/tests/simple/run.sh +++ b/tests/simple/run.sh @@ -26,7 +26,7 @@ function prepare() { TOPIC_NAME="ticdc-simple-test-$RANDOM" case $SINK_TYPE in kafka) SINK_URI="kafka+ssl://127.0.0.1:9092/$TOPIC_NAME?partition-num=4&kafka-client-id=cdc_test_simple&kafka-version=${KAFKA_VERSION}";; - *) SINK_URI="mysql+ssl://root@127.0.0.1:3306/";; + *) SINK_URI="mysql+ssl://normal:123456@127.0.0.1:3306/";; esac run_cdc_cli changefeed create --start-ts=$start_ts --sink-uri="$SINK_URI" if [ "$SINK_TYPE" == "kafka" ]; then diff --git a/tests/sink_hang/run.sh b/tests/sink_hang/run.sh index 7b4431daecb..c8f7b58dde6 100644 --- a/tests/sink_hang/run.sh +++ b/tests/sink_hang/run.sh @@ -39,7 +39,7 @@ function run() { TOPIC_NAME="ticdc-sink-hang-test-$RANDOM" case $SINK_TYPE in kafka) SINK_URI="kafka://127.0.0.1:9092/$TOPIC_NAME?partition-num=4&kafka-version=${KAFKA_VERSION}";; - *) SINK_URI="mysql://root@127.0.0.1:3306/?max-txn-row=1";; + *) SINK_URI="mysql://normal:123456@127.0.0.1:3306/?max-txn-row=1";; esac if [ "$SINK_TYPE" == "kafka" ]; then run_kafka_consumer $WORK_DIR "kafka://127.0.0.1:9092/$TOPIC_NAME?partition-num=4&version=${KAFKA_VERSION}" diff --git a/tests/sink_retry/run.sh b/tests/sink_retry/run.sh index f25c95fe519..97124b294d4 100755 --- a/tests/sink_retry/run.sh +++ b/tests/sink_retry/run.sh @@ -27,7 +27,7 @@ function run() { TOPIC_NAME="ticdc-sink-retry-test-$RANDOM" case $SINK_TYPE in kafka) SINK_URI="kafka://127.0.0.1:9092/$TOPIC_NAME?partition-num=4&kafka-version=${KAFKA_VERSION}";; - *) SINK_URI="mysql://root@127.0.0.1:3306/?max-txn-row=1";; + *) SINK_URI="mysql://normal:123456@127.0.0.1:3306/?max-txn-row=1";; esac run_cdc_cli changefeed create --start-ts=$start_ts --sink-uri="$SINK_URI" if [ "$SINK_TYPE" == "kafka" ]; then diff --git a/tests/split_region/run.sh b/tests/split_region/run.sh index 4fe9a9ad936..83ee343ef90 100755 --- a/tests/split_region/run.sh +++ b/tests/split_region/run.sh @@ -25,7 +25,7 @@ function run() { TOPIC_NAME="ticdc-split-region-test-$RANDOM" case $SINK_TYPE in kafka) SINK_URI="kafka://127.0.0.1:9092/$TOPIC_NAME?partition-num=4&kafka-version=${KAFKA_VERSION}";; - *) SINK_URI="mysql://root@127.0.0.1:3306/";; + *) SINK_URI="mysql://normal:123456@127.0.0.1:3306/";; esac run_cdc_cli changefeed create --start-ts=$start_ts --sink-uri="$SINK_URI" if [ "$SINK_TYPE" == "kafka" ]; then diff --git a/tests/syncpoint/run.sh b/tests/syncpoint/run.sh index 81d3fad9dad..8b2425a18c3 100755 --- a/tests/syncpoint/run.sh +++ b/tests/syncpoint/run.sh @@ -132,7 +132,7 @@ function run() { run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY - SINK_URI="mysql://root@127.0.0.1:3306/?max-txn-row=1" + SINK_URI="mysql://normal:123456@127.0.0.1:3306/?max-txn-row=1" run_cdc_cli changefeed create --start-ts=$start_ts --sink-uri="$SINK_URI" --sync-point --sync-interval=10s goSql diff --git a/tests/tiflash/run.sh b/tests/tiflash/run.sh index dc05858eac7..ddb19e6cf2e 100644 --- a/tests/tiflash/run.sh +++ b/tests/tiflash/run.sh @@ -23,7 +23,7 @@ function run() { TOPIC_NAME="ticdc-tiflash-test-$RANDOM" case $SINK_TYPE in kafka) SINK_URI="kafka://127.0.0.1:9092/$TOPIC_NAME?partition-num=4&kafka-version=${KAFKA_VERSION}";; - *) SINK_URI="mysql://root@127.0.0.1:3306/";; + *) SINK_URI="mysql://normal:123456@127.0.0.1:3306/";; esac run_cdc_cli changefeed create --start-ts=$start_ts --sink-uri="$SINK_URI" if [ "$SINK_TYPE" == "kafka" ]; then diff --git a/tests/unified_sorter/run.sh b/tests/unified_sorter/run.sh index 35f4abdde3f..ac07883af7b 100755 --- a/tests/unified_sorter/run.sh +++ b/tests/unified_sorter/run.sh @@ -27,7 +27,7 @@ function run() { TOPIC_NAME="ticdc-sink-retry-test-$RANDOM" case $SINK_TYPE in kafka) SINK_URI="kafka://127.0.0.1:9092/$TOPIC_NAME?partition-num=4&max-message-bytes=102400&kafka-version=${KAFKA_VERSION}";; - *) SINK_URI="mysql://root@127.0.0.1:3306/?max-txn-row=1";; + *) SINK_URI="mysql://normal:123456@127.0.0.1:3306/?max-txn-row=1";; esac sort_dir="$WORK_DIR/unified_sort_cache" mkdir $sort_dir diff --git a/tests/unified_sorter_sort_dir_conflict/run.sh b/tests/unified_sorter_sort_dir_conflict/run.sh index 3e1e29764ee..baea77efede 100644 --- a/tests/unified_sorter_sort_dir_conflict/run.sh +++ b/tests/unified_sorter_sort_dir_conflict/run.sh @@ -46,7 +46,7 @@ function prepare() { TOPIC_NAME="ticdc-simple-test-$RANDOM" case $SINK_TYPE in kafka) SINK_URI="kafka+ssl://127.0.0.1:9092/$TOPIC_NAME?partition-num=4&kafka-client-id=cdc_test_simple&kafka-version=${KAFKA_VERSION}";; - *) SINK_URI="mysql+ssl://root@127.0.0.1:3306/";; + *) SINK_URI="mysql+ssl://normal:123456@127.0.0.1:3306/";; esac changefeedid=$(cdc cli changefeed create --start-ts=$start_ts --sink-uri="$SINK_URI" 2>&1|tail -n2|head -n1|awk '{print $2}') if [ "$SINK_TYPE" == "kafka" ]; then