diff --git a/Makefile b/Makefile index 1f6315a121e..c6d56f1971e 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/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/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..db98a71688c --- /dev/null +++ b/cdc/entry/schema_test_helper.go @@ -0,0 +1,83 @@ +// 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/store/tikv/oracle" + "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.NewMockStore() + 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(oracle.GlobalTxnScope) + s.c.Assert(err, check.IsNil) + return timeta.NewSnapshotMeta(s.storage.GetSnapshot(ver)) +} + +// 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/etcd.go b/cdc/kv/etcd.go index 6b607a9ad80..580482382e0 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 d0d4c8c966a..10059635adc 100644 --- a/cdc/kv/etcd_test.go +++ b/cdc/kv/etcd_test.go @@ -524,6 +524,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/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 3b20d47c01d..1f610fd4a2c 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 @@ -217,13 +218,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 } @@ -232,6 +230,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 07918451ddf..c8954febcfe 100644 --- a/cdc/model/changefeed_test.go +++ b/cdc/model/changefeed_test.go @@ -224,11 +224,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) } @@ -236,7 +236,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 b9e930a27b8..f5a55232dcc 100644 --- a/cdc/owner.go +++ b/cdc/owner.go @@ -301,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) } @@ -654,7 +654,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 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/barrier.go b/cdc/owner/barrier.go new file mode 100644 index 00000000000..6eed7aba493 --- /dev/null +++ b/cdc/owner/barrier.go @@ -0,0 +1,85 @@ +// 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/log" + "github.com/pingcap/ticdc/cdc/model" +) + +type barrierType int + +const ( + // ddlJobBarrier denotes a replication barrier caused by a DDL. + ddlJobBarrier barrierType = iota + // syncPointBarrier denotes a barrier for snapshot replication. + syncPointBarrier + // finishBarrier denotes a barrier for changefeed finished. + finishBarrier +) + +// barriers stores some barrierType and barrierTs, and can calculate the min barrierTs +// barriers is NOT-THREAD-SAFE +type barriers struct { + inner map[barrierType]model.Ts + dirty bool + min barrierType +} + +func newBarriers() *barriers { + return &barriers{ + inner: make(map[barrierType]model.Ts), + dirty: true, + } +} + +func (b *barriers) Update(tp barrierType, barrierTs model.Ts) { + // the barriers structure was given the ability to handle a fallback barrierTs by design. + // but the barrierTs should never fallback in owner replication model + if !b.dirty && (tp == b.min || barrierTs <= b.inner[b.min]) { + b.dirty = true + } + b.inner[tp] = barrierTs +} + +func (b *barriers) Min() (tp barrierType, barrierTs model.Ts) { + if !b.dirty { + return b.min, b.inner[b.min] + } + tp, minTs := b.calcMin() + b.min = tp + b.dirty = false + return tp, minTs +} + +func (b *barriers) calcMin() (tp barrierType, barrierTs model.Ts) { + barrierTs = uint64(math.MaxUint64) + for br, ts := range b.inner { + if ts <= barrierTs { + tp = br + barrierTs = ts + } + } + if barrierTs == math.MaxUint64 { + log.Panic("the barriers is empty, please report a bug") + } + return +} + +func (b *barriers) Remove(tp barrierType) { + delete(b.inner, tp) + b.dirty = true +} diff --git a/cdc/owner/barrier_test.go b/cdc/owner/barrier_test.go new file mode 100644 index 00000000000..dcf2540d49a --- /dev/null +++ b/cdc/owner/barrier_test.go @@ -0,0 +1,97 @@ +// 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" + "math/rand" + "testing" + + "github.com/pingcap/check" + "github.com/pingcap/ticdc/cdc/model" + "github.com/pingcap/ticdc/pkg/util/testleak" +) + +func Test(t *testing.T) { check.TestingT(t) } + +var _ = check.Suite(&barrierSuite{}) + +type barrierSuite struct { +} + +func (s *barrierSuite) TestBarrier(c *check.C) { + defer testleak.AfterTest(c)() + b := newBarriers() + b.Update(ddlJobBarrier, 2) + b.Update(syncPointBarrier, 3) + b.Update(finishBarrier, 1) + tp, ts := b.Min() + c.Assert(tp, check.Equals, finishBarrier) + c.Assert(ts, check.Equals, uint64(1)) + + b.Update(finishBarrier, 4) + tp, ts = b.Min() + c.Assert(tp, check.Equals, ddlJobBarrier) + c.Assert(ts, check.Equals, uint64(2)) + + b.Remove(ddlJobBarrier) + tp, ts = b.Min() + c.Assert(tp, check.Equals, syncPointBarrier) + c.Assert(ts, check.Equals, uint64(3)) + + b.Update(finishBarrier, 1) + tp, ts = b.Min() + c.Assert(tp, check.Equals, finishBarrier) + c.Assert(ts, check.Equals, uint64(1)) + + b.Update(ddlJobBarrier, 5) + tp, ts = b.Min() + c.Assert(tp, check.Equals, finishBarrier) + c.Assert(ts, check.Equals, uint64(1)) +} + +func (s *barrierSuite) TestBarrierRandom(c *check.C) { + defer testleak.AfterTest(c)() + maxBarrierType := 50 + maxBarrierTs := 1000000 + b := newBarriers() + expectedBarriers := make(map[barrierType]model.Ts) + + // set a barrier which can not be removed to avoid the barrier map is empty + b.Update(barrierType(maxBarrierType), model.Ts(maxBarrierTs)) + expectedBarriers[barrierType(maxBarrierType)] = model.Ts(maxBarrierTs) + + for i := 0; i < 100000; i++ { + switch rand.Intn(2) { + case 0: + tp := barrierType(rand.Intn(maxBarrierType)) + ts := model.Ts(rand.Intn(maxBarrierTs)) + b.Update(tp, ts) + expectedBarriers[tp] = ts + case 1: + tp := barrierType(rand.Intn(maxBarrierType)) + b.Remove(tp) + delete(expectedBarriers, tp) + } + expectedMinTs := uint64(math.MaxUint64) + for _, ts := range expectedBarriers { + if ts < expectedMinTs { + expectedMinTs = ts + } + } + tp, ts := b.Min() + c.Assert(ts, check.Equals, expectedMinTs) + c.Assert(expectedBarriers[tp], check.Equals, expectedMinTs) + } +} diff --git a/cdc/owner/changefeed.go b/cdc/owner/changefeed.go new file mode 100644 index 00000000000..64ea4c40555 --- /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.GoTimeToTS(oracle.GetTimeFromTS(barrierTs).Add(c.state.Info.SyncPointInterval)) + 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_state.go b/cdc/owner/changefeed_state.go deleted file mode 100644 index 63d5a7f761d..00000000000 --- a/cdc/owner/changefeed_state.go +++ /dev/null @@ -1,223 +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 owner - -import ( - "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/model" - "go.uber.org/zap" -) - -// changeFeedState is part of the replication model that implements the control logic of a changeFeed -type changeFeedState struct { - TableTasks map[model.TableID]*tableTask - DDLResolvedTs uint64 - Barriers []*barrier - - scheduler scheduler -} - -type tableTask struct { - TableID model.TableID - CheckpointTs uint64 - ResolvedTs uint64 -} - -type barrierType = int - -const ( - // DDLBarrier denotes a replication barrier caused by a DDL. - DDLBarrier = barrierType(iota) - // SyncPointBarrier denotes a barrier for snapshot replication. - // SyncPointBarrier - // TODO support snapshot replication. -) - -type barrier struct { - BarrierType barrierType - BarrierTs uint64 -} - -type ddlResultAction = string - -const ( - // AddTableAction denotes a request to start replicating a table. - AddTableAction = ddlResultAction("add") - // DropTableAction denotes a request to stop replicating a table. - DropTableAction = ddlResultAction("drop") -) - -type ddlResult struct { - FinishTs uint64 - Actions []tableAction -} - -type tableAction struct { - Action ddlResultAction - tableID model.TableID -} - -func newChangeFeedState(initTableTasks map[model.TableID]*tableTask, ddlStartTs uint64, scheduler scheduler) *changeFeedState { - return &changeFeedState{ - TableTasks: initTableTasks, - DDLResolvedTs: ddlStartTs, - scheduler: scheduler, - } -} - -func (cf *changeFeedState) SyncTasks() { - if cf.scheduler.IsReady() { - cf.scheduler.PutTasks(cf.TableTasks) - } -} - -func (cf *changeFeedState) SetDDLResolvedTs(ddlResolvedTs uint64) { - cf.DDLResolvedTs = ddlResolvedTs -} - -func (cf *changeFeedState) AddDDLBarrier(barrierTs uint64) { - if len(cf.Barriers) > 0 && barrierTs < cf.Barriers[len(cf.Barriers)-1].BarrierTs { - log.Panic("changeFeedState: DDLBarrier too small", - zap.Uint64("last-barrier-ts", cf.Barriers[len(cf.Barriers)-1].BarrierTs), - zap.Uint64("new-barrier-ts", barrierTs)) - } - - if barrierTs < cf.DDLResolvedTs { - log.Panic("changeFeedState: DDLBarrier too small", - zap.Uint64("cur-ddl-resolved-ts", cf.DDLResolvedTs), - zap.Uint64("new-barrier-ts", barrierTs)) - } - - cf.Barriers = append(cf.Barriers, &barrier{ - BarrierType: DDLBarrier, - BarrierTs: barrierTs, - }) -} - -func (cf *changeFeedState) ShouldRunDDL() *barrier { - if len(cf.Barriers) > 0 { - if cf.Barriers[0].BarrierTs == cf.CheckpointTs()+1 && - cf.Barriers[0].BarrierType == DDLBarrier { - - return cf.Barriers[0] - } - } - - return nil -} - -func (cf *changeFeedState) MarkDDLDone(result ddlResult) { - if cf.CheckpointTs() != result.FinishTs-1 { - log.Panic("changeFeedState: Unexpected checkpoint when DDL is done", - zap.Uint64("cur-checkpoint-ts", cf.CheckpointTs()), - zap.Reflect("ddl-result", result)) - } - - if len(cf.Barriers) == 0 || - cf.Barriers[0].BarrierType != DDLBarrier || - cf.Barriers[0].BarrierTs != result.FinishTs { - - log.Panic("changeFeedState: no DDL barrier found", - zap.Reflect("barriers", cf.Barriers), - zap.Reflect("ddl-result", result)) - } - - cf.Barriers = cf.Barriers[1:] - - for _, tableAction := range result.Actions { - switch tableAction.Action { - case AddTableAction: - cf.TableTasks[tableAction.tableID] = &tableTask{ - TableID: tableAction.tableID, - CheckpointTs: cf.CheckpointTs(), - ResolvedTs: cf.CheckpointTs(), - } - case DropTableAction: - if _, ok := cf.TableTasks[tableAction.tableID]; !ok { - log.Panic("changeFeedState: Dropping unknown table", zap.Int64("table-id", tableAction.tableID)) - } - - delete(cf.TableTasks, tableAction.tableID) - default: - log.Panic("changeFeedState: unknown action", zap.String("action", tableAction.Action)) - } - } -} - -// TODO test-case: returned value is not zero -func (cf *changeFeedState) ResolvedTs() uint64 { - resolvedTs := cf.DDLResolvedTs - - for _, table := range cf.TableTasks { - if resolvedTs > table.ResolvedTs { - resolvedTs = table.ResolvedTs - } - } - - if len(cf.Barriers) > 0 && resolvedTs > cf.Barriers[0].BarrierTs-1 { - resolvedTs = cf.Barriers[0].BarrierTs - 1 - } - - if resolvedTs == 0 { - log.Panic("Unexpected resolvedTs") - } - return resolvedTs -} - -// TODO test-case: returned value is not zero -func (cf *changeFeedState) CheckpointTs() uint64 { - checkpointTs := cf.DDLResolvedTs - - for _, table := range cf.TableTasks { - if checkpointTs > table.CheckpointTs { - checkpointTs = table.CheckpointTs - } - } - - if len(cf.Barriers) > 0 && checkpointTs > cf.Barriers[0].BarrierTs-1 { - checkpointTs = cf.Barriers[0].BarrierTs - 1 - } - - if checkpointTs == 0 { - log.Panic("Unexpected checkpointTs", zap.Reflect("state", cf)) - } - return checkpointTs -} - -func (cf *changeFeedState) SetTableResolvedTs(tableID model.TableID, resolvedTs uint64) { - tableTask, ok := cf.TableTasks[tableID] - - if !ok { - return - } - - tableTask.ResolvedTs = resolvedTs -} - -func (cf *changeFeedState) SetTableCheckpointTs(tableID model.TableID, checkpointTs uint64) { - tableTask, ok := cf.TableTasks[tableID] - - if !ok { - return - } - - if tableTask.CheckpointTs > checkpointTs { - log.Panic("changeFeedState: table checkpoint regressed. Report a bug.", - zap.Int64("tableID", tableID), - zap.Uint64("oldCheckpointTs", tableTask.CheckpointTs), - zap.Uint64("checkpointTs", checkpointTs)) - } - - tableTask.CheckpointTs = checkpointTs -} diff --git a/cdc/owner/changefeed_state_test.go b/cdc/owner/changefeed_state_test.go deleted file mode 100644 index d124622ab38..00000000000 --- a/cdc/owner/changefeed_state_test.go +++ /dev/null @@ -1,205 +0,0 @@ -// Copyright 2021 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package owner - -import ( - "github.com/pingcap/check" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/util/testleak" - "github.com/stretchr/testify/mock" -) - -type changeFeedStateTestSuite struct { -} - -var _ = check.Suite(&changeFeedStateTestSuite{}) - -type mockScheduler struct { - mock.Mock -} - -func (m *mockScheduler) PutTasks(tables map[model.TableID]*tableTask) { - m.Called(tables) -} - -func (m *mockScheduler) SetAffinity(tableID model.TableID, captureID model.CaptureID, ttl int) { - m.Called(tableID, captureID, ttl) -} - -func (m *mockScheduler) IsReady() bool { - args := m.Called() - return args.Bool(0) -} - -func (s *changeFeedStateTestSuite) TestBasics(c *check.C) { - defer testleak.AfterTest(c)() - tasks := map[model.TableID]*tableTask{ - 1: { - TableID: 1, - CheckpointTs: 2000, - ResolvedTs: 2000, - }, - 2: { - TableID: 2, - CheckpointTs: 2000, - ResolvedTs: 2000, - }, - } - - scheduler := &mockScheduler{} - cfState := newChangeFeedState(tasks, 2000, scheduler) - - scheduler.On("IsReady").Return(false).Once() - cfState.SyncTasks() - scheduler.AssertNotCalled(c, "PutTasks") - - scheduler.On("IsReady").Return(true) - scheduler.On("PutTasks", mock.Anything).Return() - cfState.SyncTasks() - scheduler.AssertCalled(c, "PutTasks", tasks) - - cfState.AddDDLBarrier(2500) - cfState.SetDDLResolvedTs(3000) - c.Assert(cfState.ShouldRunDDL(), check.IsNil) - c.Assert(cfState.ResolvedTs(), check.Equals, uint64(2000)) - c.Assert(cfState.CheckpointTs(), check.Equals, uint64(2000)) - - cfState.SetTableResolvedTs(1, 2500) - cfState.SetTableCheckpointTs(1, 2500) - c.Assert(cfState.ShouldRunDDL(), check.IsNil) - c.Assert(cfState.ResolvedTs(), check.Equals, uint64(2000)) - c.Assert(cfState.CheckpointTs(), check.Equals, uint64(2000)) - - cfState.SetTableResolvedTs(2, 2500) - cfState.SetTableCheckpointTs(2, 2500) - c.Assert(cfState.ResolvedTs(), check.Equals, uint64(2499)) - c.Assert(cfState.CheckpointTs(), check.Equals, uint64(2499)) - c.Assert(cfState.ShouldRunDDL(), check.DeepEquals, &barrier{ - BarrierType: DDLBarrier, - BarrierTs: 2500, - }) - - cfState.MarkDDLDone(ddlResult{ - FinishTs: 2500, - Actions: []tableAction{{ - Action: AddTableAction, - tableID: 3, - }}, - }) - cfState.SyncTasks() - tasks[3] = &tableTask{ - TableID: 3, - CheckpointTs: 2499, - ResolvedTs: 2499, - } - scheduler.AssertCalled(c, "PutTasks", tasks) - c.Assert(cfState.CheckpointTs(), check.Equals, uint64(2499)) - c.Assert(cfState.ResolvedTs(), check.Equals, uint64(2499)) - - cfState.AddDDLBarrier(3500) - cfState.SetDDLResolvedTs(4000) - cfState.SetTableResolvedTs(1, 3499) - cfState.SetTableCheckpointTs(1, 3499) - cfState.SetTableResolvedTs(2, 3000) - cfState.SetTableCheckpointTs(2, 2600) - cfState.SetTableResolvedTs(3, 3000) - cfState.SetTableCheckpointTs(3, 2600) - c.Assert(cfState.CheckpointTs(), check.Equals, uint64(2600)) - c.Assert(cfState.ResolvedTs(), check.Equals, uint64(3000)) - - cfState.SetTableResolvedTs(1, 4000) - cfState.SetTableCheckpointTs(1, 3499) - cfState.SetTableResolvedTs(2, 4000) - cfState.SetTableCheckpointTs(2, 3499) - cfState.SetTableResolvedTs(3, 4000) - cfState.SetTableCheckpointTs(3, 3499) - c.Assert(cfState.ResolvedTs(), check.Equals, uint64(3499)) - c.Assert(cfState.CheckpointTs(), check.Equals, uint64(3499)) - c.Assert(cfState.ShouldRunDDL(), check.DeepEquals, &barrier{ - BarrierType: DDLBarrier, - BarrierTs: 3500, - }) - - cfState.MarkDDLDone(ddlResult{ - FinishTs: 3500, - Actions: []tableAction{{ - Action: DropTableAction, - tableID: 3, - }, { - Action: DropTableAction, - tableID: 2, - }}, - }) - - delete(tasks, 2) - delete(tasks, 3) - scheduler.AssertCalled(c, "PutTasks", tasks) - - cfState.SetTableCheckpointTs(2, 1000) // should be ok since table 2 does not exist - cfState.SetTableResolvedTs(2, 1000) // should be ok since table 2 does not exist -} - -func (s *changeFeedStateTestSuite) TestPanicCases(c *check.C) { - defer testleak.AfterTest(c)() - tasks := map[model.TableID]*tableTask{ - 1: { - TableID: 1, - CheckpointTs: 2000, - ResolvedTs: 2000, - }, - 2: { - TableID: 2, - CheckpointTs: 2000, - ResolvedTs: 2000, - }, - } - - scheduler := &mockScheduler{} - cfState := newChangeFeedState(tasks, 2000, scheduler) - - scheduler.On("IsReady").Return(true) - scheduler.On("PutTasks", mock.Anything).Return() - cfState.SyncTasks() - scheduler.AssertCalled(c, "PutTasks", tasks) - - c.Assert(func() { - cfState.SetTableCheckpointTs(1, 1000) - }, check.PanicMatches, ".*table checkpoint regressed.*") - - cfState.AddDDLBarrier(5000) - c.Assert(func() { - cfState.AddDDLBarrier(4000) - }, check.PanicMatches, ".*DDLBarrier too small.*") - - cfState.SetDDLResolvedTs(5500) - c.Assert(func() { - cfState.AddDDLBarrier(5400) - }, check.PanicMatches, ".*DDLBarrier too small.*") - - cfState.SetDDLResolvedTs(7000) - cfState.SetTableResolvedTs(1, 4999) - cfState.SetTableCheckpointTs(1, 4999) - cfState.SetTableResolvedTs(2, 4999) - cfState.SetTableCheckpointTs(2, 4999) - c.Assert(cfState.ShouldRunDDL(), check.DeepEquals, &barrier{ - BarrierType: DDLBarrier, - BarrierTs: 5000, - }) - c.Assert(func() { - cfState.MarkDDLDone(ddlResult{ - FinishTs: 3000, - Actions: nil, - }) - }, check.PanicMatches, ".*Unexpected checkpoint.*") -} diff --git a/cdc/owner/changefeed_test.go b/cdc/owner/changefeed_test.go new file mode 100644 index 00000000000..02819ab20c0 --- /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.GoTimeToTS(oracle.GetTimeFromTS(mockDDLPuller.resolvedTs).Add(5 * time.Second)) + // 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..ed712f61a16 --- /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.GoTimeToTS(time.Now())) + 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_state.go b/cdc/owner/owner_state.go deleted file mode 100644 index 69def09cd85..00000000000 --- a/cdc/owner/owner_state.go +++ /dev/null @@ -1,760 +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 owner - -import ( - "encoding/json" - "sort" - "time" - - "github.com/pingcap/errors" - "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/kv" - "github.com/pingcap/ticdc/cdc/model" - 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" -) - -// ownerReactorState is used by the Owner to manage its Etcd states. -type ownerReactorState struct { - // These fields represent the latest state of Etcd. - ChangeFeedInfos map[model.ChangeFeedID]*model.ChangeFeedInfo - Captures map[model.CaptureID]*model.CaptureInfo - ChangeFeedStatuses map[model.ChangeFeedID]*model.ChangeFeedStatus - TaskPositions map[model.ChangeFeedID]map[model.CaptureID]*model.TaskPosition - TaskStatuses map[model.ChangeFeedID]map[model.CaptureID]*model.TaskStatus - - patches []*orchestrator.DataPatch // patches yet to be uploaded to Etcd - tableToCaptureMapCache map[model.ChangeFeedID]map[model.TableID]model.CaptureID // table-to-capture mapping cached for frequent use - newCaptureHandler func(captureID model.CaptureID) // called when a new capture is added -} - -func newCDCReactorState() *ownerReactorState { - return &ownerReactorState{ - ChangeFeedInfos: make(map[model.ChangeFeedID]*model.ChangeFeedInfo), - Captures: make(map[model.CaptureID]*model.CaptureInfo), - ChangeFeedStatuses: make(map[model.ChangeFeedID]*model.ChangeFeedStatus), - TaskPositions: make(map[model.ChangeFeedID]map[model.CaptureID]*model.TaskPosition), - TaskStatuses: make(map[model.ChangeFeedID]map[model.CaptureID]*model.TaskStatus), - tableToCaptureMapCache: make(map[model.ChangeFeedID]map[model.TableID]model.CaptureID), - } -} - -// Update implements the Reactor interface. -func (s *ownerReactorState) 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.CDCKeyTypeCapture: - captureID := k.CaptureID - defer func() { - // invalidate cache because captures have changed. - s.invalidateTableToCaptureCache() - }() - - if value == nil { - log.Info("Capture deleted", - zap.String("captureID", captureID), - zap.Reflect("old-capture", s.Captures[captureID])) - - delete(s.Captures, captureID) - return nil - } - - var newCaptureInfo model.CaptureInfo - err := json.Unmarshal(value, &newCaptureInfo) - if err != nil { - return cerrors.ErrUnmarshalFailed.Wrap(err).GenWithStackByArgs() - } - - if oldCaptureInfo, ok := s.Captures[captureID]; ok { - log.Debug("Capture updated", - zap.String("captureID", captureID), - zap.Reflect("old-capture", oldCaptureInfo), - zap.Reflect("new-capture", newCaptureInfo)) - } else { - log.Info("Capture added", - zap.String("captureID", captureID), - zap.Reflect("new-capture", newCaptureInfo)) - - if s.newCaptureHandler != nil { - // Notify about the capture-added event - s.newCaptureHandler(captureID) - } - } - - s.Captures[captureID] = &newCaptureInfo - - return nil - case etcd.CDCKeyTypeChangeFeedStatus: - changefeedID := k.ChangefeedID - - if value == nil { - log.Info("Changefeed deleted", - zap.String("changefeedID", changefeedID), - zap.Reflect("old-changefeed", s.ChangeFeedStatuses)) - - delete(s.ChangeFeedStatuses, changefeedID) - return nil - } - - var newChangefeedStatus model.ChangeFeedStatus - err := json.Unmarshal(value, &newChangefeedStatus) - if err != nil { - return cerrors.ErrUnmarshalFailed.Wrap(err).GenWithStackByArgs() - } - - if oldChangefeedInfo, ok := s.ChangeFeedStatuses[changefeedID]; ok { - log.Debug("Changefeed updated", - zap.String("changefeedID", changefeedID), - zap.Reflect("old-changefeed", oldChangefeedInfo), - zap.Reflect("new-changefeed", newChangefeedStatus)) - } else { - log.Debug("Changefeed added", - zap.String("changefeedID", changefeedID), - zap.Reflect("new-changefeed", newChangefeedStatus)) - } - - s.ChangeFeedStatuses[changefeedID] = &newChangefeedStatus - return nil - case etcd.CDCKeyTypeTaskPosition: - captureID := k.CaptureID - changefeedID := k.ChangefeedID - - if value == nil { - if s.TaskPositions[changefeedID] == nil { - return nil - } - - log.Debug("Position deleted", - zap.String("captureID", captureID), - zap.String("changefeedID", changefeedID), - zap.Reflect("old-position", s.TaskPositions[changefeedID][captureID])) - - delete(s.TaskPositions[changefeedID], captureID) - if len(s.TaskPositions[changefeedID]) == 0 { - delete(s.TaskPositions, changefeedID) - } - - return nil - } - - var newTaskPosition model.TaskPosition - err := json.Unmarshal(value, &newTaskPosition) - if err != nil { - return cerrors.ErrUnmarshalFailed.Wrap(err).GenWithStackByArgs() - } - - if _, ok := s.TaskPositions[changefeedID]; !ok { - s.TaskPositions[changefeedID] = make(map[model.CaptureID]*model.TaskPosition) - } - - if position, ok := s.TaskPositions[changefeedID][captureID]; ok { - log.Debug("Position updated", - zap.String("captureID", captureID), - zap.String("changefeedID", changefeedID), - zap.Reflect("old-position", position), - zap.Reflect("new-position", newTaskPosition)) - } else { - log.Debug("Position created", - zap.String("captureID", captureID), - zap.String("changefeedID", changefeedID), - zap.Reflect("new-position", newTaskPosition)) - } - - s.TaskPositions[changefeedID][captureID] = &newTaskPosition - return nil - case etcd.CDCKeyTypeTaskStatus: - captureID := k.CaptureID - changefeedID := k.ChangefeedID - - if value == nil { - log.Debug("Status deleted", - zap.String("captureID", captureID), - zap.String("changefeedID", changefeedID), - zap.Reflect("old-status", s.TaskStatuses[changefeedID][captureID])) - - delete(s.TaskStatuses[changefeedID], captureID) - if len(s.TaskStatuses[changefeedID]) == 0 { - delete(s.TaskStatuses, changefeedID) - } - return nil - } - - var newTaskStatus model.TaskStatus - err := json.Unmarshal(value, &newTaskStatus) - if err != nil { - return cerrors.ErrUnmarshalFailed.Wrap(err).GenWithStackByArgs() - } - - if _, ok := s.TaskStatuses[changefeedID]; !ok { - s.TaskStatuses[changefeedID] = make(map[model.CaptureID]*model.TaskStatus) - } - - if status, ok := s.TaskStatuses[changefeedID][captureID]; ok { - log.Info("Status updated", - zap.String("captureID", captureID), - zap.String("changefeedID", changefeedID), - zap.Reflect("old-status", status), - zap.Reflect("new-status", newTaskStatus)) - } else { - log.Info("Status updated", - zap.String("captureID", captureID), - zap.String("changefeedID", changefeedID), - zap.Reflect("new-status", newTaskStatus)) - } - - s.TaskStatuses[changefeedID][captureID] = &newTaskStatus - s.updateTableToCaptureCache(changefeedID) - return nil - case etcd.CDCKeyTypeChangefeedInfo: - changeFeedID := k.ChangefeedID - - if value == nil { - log.Info("ChangeFeed deleted", zap.String("cfID", changeFeedID)) - delete(s.ChangeFeedInfos, changeFeedID) - return nil - } - - var changeFeedInfo model.ChangeFeedInfo - err := json.Unmarshal(value, &changeFeedInfo) - if err != nil { - return cerrors.ErrUnmarshalFailed.Wrap(err).GenWithStackByArgs() - } - - s.ChangeFeedInfos[changeFeedID] = &changeFeedInfo - return nil - default: - } - - return nil -} - -// GetPatches implements the Reactor interface. -func (s *ownerReactorState) GetPatches() []*orchestrator.DataPatch { - ret := s.patches - s.patches = nil - return ret -} - -// UpdateChangeFeedStatus updates the progress of the changefeed. -// NOTE: 0 is NOT a valid value for either resolvedTs or checkpointTs. -func (s *ownerReactorState) UpdateChangeFeedStatus(cfID model.ChangeFeedID, resolvedTs uint64, checkpointTs uint64) { - if resolvedTs == 0 || checkpointTs == 0 { - log.Panic("illegal changeFeedStatus", - zap.String("cfID", cfID), - zap.Uint64("resolvedTs", resolvedTs), - zap.Uint64("checkpointTs", checkpointTs)) - } - - patch := &orchestrator.DataPatch{ - Key: util.NewEtcdKey(kv.GetEtcdKeyChangeFeedStatus(cfID)), - Fun: func(old []byte) ([]byte, error) { - var changeFeedStatus model.ChangeFeedStatus - - if old != nil { - err := json.Unmarshal(old, &changeFeedStatus) - if err != nil { - return nil, cerrors.ErrUnmarshalFailed.Wrap(err) - } - - if changeFeedStatus.CheckpointTs > checkpointTs { - log.Panic("checkpointTs regressed", - zap.Reflect("changeFeedStatus", changeFeedStatus), - zap.Uint64("newCheckpointTs", checkpointTs)) - } - } - - changeFeedStatus.CheckpointTs = checkpointTs - changeFeedStatus.ResolvedTs = resolvedTs - - newBytes, err := json.Marshal(&changeFeedStatus) - if err != nil { - return nil, cerrors.ErrMarshalFailed.Wrap(err).GenWithStackByArgs() - } - - return newBytes, nil - }, - } - - s.patches = append(s.patches, patch) -} - -// DispatchTable dispatches table to a given capture. -func (s *ownerReactorState) DispatchTable(cfID model.ChangeFeedID, captureID model.CaptureID, tableID model.TableID, replicaInfo model.TableReplicaInfo) { - patch := &orchestrator.DataPatch{ - Key: util.NewEtcdKey(kv.GetEtcdKeyTaskStatus(cfID, captureID)), - Fun: func(old []byte) ([]byte, error) { - var taskStatus model.TaskStatus - if len(old) > 0 { - err := json.Unmarshal(old, &taskStatus) - if err != nil { - return nil, errors.Trace(err) - } - } - - if _, ok := taskStatus.Operation[tableID]; ok { - log.Debug("already dispatched, ignore", zap.Int("tableID", int(tableID))) - return nil, cerrors.ErrEtcdIgnore.GenWithStackByArgs() - } - - if taskStatus.Tables == nil { - taskStatus.Tables = make(map[model.TableID]*model.TableReplicaInfo) - } - - taskStatus.Tables[tableID] = &replicaInfo - operation := &model.TableOperation{ - Delete: false, - BoundaryTs: replicaInfo.StartTs, - Done: false, - Status: model.OperDispatched, - } - - if taskStatus.Operation == nil { - taskStatus.Operation = make(map[model.TableID]*model.TableOperation) - } - taskStatus.Operation[tableID] = operation - - newValue, err := json.Marshal(&taskStatus) - if err != nil { - return nil, cerrors.ErrMarshalFailed.Wrap(err).GenWithStackByArgs() - } - - return newValue, nil - }, - } - - s.patches = append(s.patches, patch) -} - -// StartDeletingTable initiates a delete operation. -// NOTE: callers need to guarantee that the table is running normally in the latest Etcd state snapshot. -// `running normally` is defined as (1) having a valid TaskStatus in the given Capture, (2) NOT having a pending deletion. -func (s *ownerReactorState) StartDeletingTable(cfID model.ChangeFeedID, captureID model.CaptureID, tableID model.TableID) { - captureTaskStatuses, ok := s.TaskStatuses[cfID] - if !ok { - log.Panic("owner bug: changeFeedState not found", zap.String("cfID", cfID)) - } - - _, ok = captureTaskStatuses[captureID] - if !ok { - log.Panic("owner bug: capture not found", zap.String("captureID", captureID)) - } - - patch := &orchestrator.DataPatch{ - Key: util.NewEtcdKey(kv.GetEtcdKeyTaskStatus(cfID, captureID)), - Fun: func(old []byte) ([]byte, error) { - if len(old) == 0 { - return nil, cerrors. - ErrOwnerInconsistentStates. - GenWithStackByArgs("TaskStatus deleted when owner tries to stop a table") - } - - var taskStatus model.TaskStatus - err := json.Unmarshal(old, &taskStatus) - if err != nil { - return nil, errors.Trace(err) - } - - if taskStatus.Operation == nil { - taskStatus.Operation = make(map[model.TableID]*model.TableOperation) - } - - if op, ok := taskStatus.Operation[tableID]; ok { - if op.Delete { - log.Panic("repeated deletion", - zap.String("cfID", cfID), - zap.String("captureID", captureID), - zap.Int("tableID", int(tableID))) - } - } - - boundaryTs := uint64(0) - // We may encounter the rare case when a table is being deleted when the ChangeFeedStatus has not been updated yet. - // TODO confirm whether this can happen. - if s.ChangeFeedStatuses[cfID] != nil { - boundaryTs = s.ChangeFeedStatuses[cfID].ResolvedTs - } - - taskStatus.Operation[tableID] = &model.TableOperation{ - Delete: true, - BoundaryTs: boundaryTs, - Done: false, - Status: model.OperDispatched, - } - - newValue, err := json.Marshal(&taskStatus) - if err != nil { - return nil, cerrors.ErrMarshalFailed.Wrap(err).GenWithStackByArgs() - } - - return newValue, nil - }, - } - - s.patches = append(s.patches, patch) -} - -// CleanOperation cleans up an operation in a capture. -// NOTE: callers need to guarantee that the capture has a valid TaskStatus. -func (s *ownerReactorState) CleanOperation(cfID model.ChangeFeedID, captureID model.CaptureID, tableID model.TableID) { - captureTaskStatuses, ok := s.TaskStatuses[cfID] - if !ok { - log.Panic("owner bug: changeFeedState not found", zap.String("cfID", cfID)) - } - - _, ok = captureTaskStatuses[captureID] - if !ok { - log.Panic("owner bug: capture not found", zap.String("captureID", captureID)) - } - - patch := &orchestrator.DataPatch{ - Key: util.NewEtcdKey(kv.GetEtcdKeyTaskStatus(cfID, captureID)), - Fun: func(old []byte) ([]byte, error) { - if len(old) == 0 { - return nil, cerrors. - ErrOwnerInconsistentStates. - GenWithStackByArgs("TaskStatus deleted when owner tries to clean up a table operation") - } - - var taskStatus model.TaskStatus - err := json.Unmarshal(old, &taskStatus) - if err != nil { - return nil, errors.Trace(err) - } - - // TODO remove this assertion - if taskStatus.Operation[tableID] != nil && taskStatus.Operation[tableID].Delete { - if _, ok := taskStatus.Tables[tableID]; ok { - log.Panic("processor bug: table not cleaned before marking done flag", - zap.Int("tableID", int(tableID)), - zap.String("captureID", captureID)) - } - } - - delete(taskStatus.Operation, tableID) - - newValue, err := json.Marshal(&taskStatus) - if err != nil { - return nil, cerrors.ErrMarshalFailed.Wrap(err).GenWithStackByArgs() - } - - return newValue, nil - }, - } - - s.patches = append(s.patches, patch) -} - -// AlterChangeFeedRuntimeState modifies a changefeed's runtime state. -func (s *ownerReactorState) AlterChangeFeedRuntimeState( - cfID model.ChangeFeedID, - adminJobType model.AdminJobType, - state model.FeedState, - cfErr *model.RunningError, - errTs int64) { - _, ok := s.ChangeFeedInfos[cfID] - if !ok { - log.Warn("owner bug: changeFeedInfo not found", zap.String("cfID", cfID)) - return - } - - patch := &orchestrator.DataPatch{ - Key: util.NewEtcdKey(kv.GetEtcdKeyChangeFeedInfo(cfID)), - Fun: func(old []byte) ([]byte, error) { - if old == nil { - log.Warn("AlterChangeFeedRuntimeState: changeFeedInfo forcibly removed", zap.String("cfID", cfID)) - return nil, cerrors.ErrEtcdIgnore - } - - var info model.ChangeFeedInfo - err := json.Unmarshal(old, &info) - if err != nil { - return nil, cerrors.ErrUnmarshalFailed.Wrap(err).GenWithStackByArgs() - } - - info.State = state - info.AdminJobType = adminJobType - - if cfErr != nil { - info.Error = cfErr - info.ErrorHis = append(info.ErrorHis, errTs) - } - - newBytes, err := json.Marshal(&info) - if err != nil { - return nil, cerrors.ErrMarshalFailed.Wrap(err).GenWithStackByArgs() - } - - return newBytes, nil - }, - } - - s.patches = append(s.patches, patch) -} - -// CleanUpTaskStatus removes the taskStatus of a changefeed for a given capture. -// NOTE: call this function only if the capture has gone. -func (s *ownerReactorState) CleanUpTaskStatus(cfID model.ChangeFeedID, captureID model.CaptureID) { - taskStatuses, ok := s.TaskStatuses[cfID] - if !ok { - log.Debug("CleanUpTaskStatus: task statuses for the given change-feed not found", - zap.String("cfID", cfID), - zap.String("captureID", captureID)) - return - } - - if _, ok := taskStatuses[captureID]; !ok { - return - } - - patch := &orchestrator.DataPatch{ - Key: util.NewEtcdKey(kv.GetEtcdKeyTaskStatus(cfID, captureID)), - Fun: func(old []byte) (newValue []byte, err error) { - if old == nil { - log.Debug("CleanUpTaskStatus: already removed", - zap.String("cfID", cfID), - zap.String("captureID", captureID)) - return nil, cerrors.ErrEtcdIgnore.GenWithStackByArgs() - } - - // remove the key - return nil, nil - }, - } - - s.patches = append(s.patches, patch) -} - -// CleanUpTaskPosition removes the taskPosition of a changefeed for a given capture. -func (s *ownerReactorState) CleanUpTaskPosition(cfID model.ChangeFeedID, captureID model.CaptureID) { - taskPositions, ok := s.TaskPositions[cfID] - if !ok { - log.Debug("CleanUpTaskPosition: task positions for the given change-feed not found", - zap.String("cfID", cfID), - zap.String("captureID", captureID)) - return - } - - if _, ok := taskPositions[captureID]; !ok { - log.Debug("CleanUpTaskPosition: task positions for the given capture not found", - zap.String("cfID", cfID), - zap.String("captureID", captureID)) - return - } - - patch := &orchestrator.DataPatch{ - Key: util.NewEtcdKey(kv.GetEtcdKeyTaskPosition(cfID, captureID)), - Fun: func(old []byte) (newValue []byte, err error) { - if old == nil { - log.Debug("CleanUpTaskPosition: already removed", - zap.String("cfID", cfID), - zap.String("captureID", captureID)) - return nil, cerrors.ErrEtcdIgnore.GenWithStackByArgs() - } - - // remove the key - return nil, nil - }, - } - - s.patches = append(s.patches, patch) -} - -// GetCaptureTables fetches all the tables that should be considered `running` on a certain capture. -func (s *ownerReactorState) GetCaptureTables(cfID model.ChangeFeedID, captureID model.CaptureID) []model.TableID { - var ret []model.TableID - - if !s.CaptureExists(captureID) { - log.Debug("GetCaptureTables: Capture is gone", zap.String("captureID", captureID)) - return ret - } - - taskStatuses, ok := s.TaskStatuses[cfID] - if !ok { - return ret - } - - taskStatus, ok := taskStatuses[captureID] - if !ok { - return ret - } - - tableIDSet := make(map[model.TableID]struct{}) - - for tableID := range taskStatus.Tables { - tableIDSet[tableID] = struct{}{} - } - - for tableID, op := range taskStatus.Operation { - // A table could be in the process of being added. - // We need to count this case. - if op.Status != model.OperFinished { - tableIDSet[tableID] = struct{}{} - } - } - - for tableID := range tableIDSet { - ret = append(ret, tableID) - } - - return ret -} - -// CleanUpChangeFeedErrorHistory does garbage collection to a changefeed's error history. -// NOTE: cfID must be a valid changefeed ID, or otherwise the function would panic. -func (s *ownerReactorState) CleanUpChangeFeedErrorHistory(cfID model.ChangeFeedID) { - _, ok := s.ChangeFeedInfos[cfID] - if !ok { - log.Panic("owner bug: changeFeedInfo not found", zap.String("cfID", cfID)) - } - - patch := &orchestrator.DataPatch{ - Key: util.NewEtcdKey(kv.GetEtcdKeyChangeFeedInfo(cfID)), - Fun: func(old []byte) ([]byte, error) { - if old == nil { - log.Warn("cleanUpChangeFeedErrorHistory: changeFeedInfo forcibly removed", zap.String("cfID", cfID)) - return nil, cerrors.ErrEtcdIgnore.GenWithStackByArgs() - } - - var info model.ChangeFeedInfo - err := json.Unmarshal(old, &info) - if err != nil { - return nil, cerrors.ErrUnmarshalFailed.Wrap(err) - } - - i := sort.Search(len(info.ErrorHis), func(i int) bool { - ts := info.ErrorHis[i] - // `ts` is in milliseconds. - return time.Since(time.Unix(ts/1e3, (ts%1e3)*1e6)) < model.ErrorHistoryGCInterval - }) - - if i == 0 { - // no need to clean anything. Returns ErrEtcdIgnore to ease the load on the Etcd server. - return nil, cerrors.ErrEtcdIgnore.GenWithStackByArgs() - } - - if i < len(info.ErrorHis) { - info.ErrorHis = info.ErrorHis[i:] - } - - newBytes, err := json.Marshal(&info) - if err != nil { - return nil, cerrors.ErrMarshalFailed.Wrap(err).GenWithStackByArgs() - } - - return newBytes, nil - }, - } - - s.patches = append(s.patches, patch) -} - -// GetTableToCaptureMap returns a map from tableIDs to captureIDs. -func (s *ownerReactorState) GetTableToCaptureMap(cfID model.ChangeFeedID) map[model.TableID]model.CaptureID { - tableToCaptureMap := make(map[model.TableID]model.CaptureID) - for captureID, taskStatus := range s.TaskStatuses[cfID] { - if !s.CaptureExists(captureID) { - continue - } - - for tableID := range taskStatus.Tables { - tableToCaptureMap[tableID] = captureID - } - - for tableID, op := range taskStatus.Operation { - if op.Status != model.OperFinished { - tableToCaptureMap[tableID] = captureID - } - } - } - - return tableToCaptureMap -} - -type tableProgress struct { - resolvedTs uint64 - checkpointTs uint64 -} - -// GetTableProgress returns the progress of a table. -// If the table is NOT active, or belongs to a capture that has not yet uploaded its task position, -// then GetTableProgress returns nil. -func (s *ownerReactorState) GetTableProgress(cfID model.ChangeFeedID, tableID model.TableID) *tableProgress { - if _, ok := s.tableToCaptureMapCache[cfID]; !ok { - s.updateTableToCaptureCache(cfID) - } - - m := s.tableToCaptureMapCache[cfID] - if captureID, ok := m[tableID]; ok { - position := s.TaskPositions[cfID][captureID] - if position == nil { - return nil - } - - return &tableProgress{ - resolvedTs: position.ResolvedTs, - checkpointTs: position.CheckPointTs, - } - } - - return nil -} - -// GetChangeFeedActiveTables returns all the active tables in a changeFeed. -// NOTE: Being `active` is defined as 1) appearing in a taskStatus, 2) not the target of any finished delete operation, -// and 3) belonging to a capture that holds a valid lease. -func (s *ownerReactorState) GetChangeFeedActiveTables(cfID model.ChangeFeedID) []model.TableID { - if _, ok := s.tableToCaptureMapCache[cfID]; !ok { - s.updateTableToCaptureCache(cfID) - } - - m := s.tableToCaptureMapCache[cfID] - - var tableIDs []model.TableID - for tableID := range m { - tableIDs = append(tableIDs, tableID) - } - - return tableIDs -} - -// CaptureExists returns whether a capture exists by checking the leased key in Etcd. -func (s *ownerReactorState) CaptureExists(captureID model.CaptureID) bool { - _, ok := s.Captures[captureID] - return ok -} - -// SetNewCaptureHandler is used to register a handler for capture-added events. -// This is normally used to trigger a table rebalance. -func (s *ownerReactorState) SetNewCaptureHandler(handler func(id model.CaptureID)) { - s.newCaptureHandler = handler -} - -// updateTableToCaptureCache updates the internal table-to-capture mapping cache for a given change-feed. -func (s *ownerReactorState) updateTableToCaptureCache(cfID model.ChangeFeedID) { - s.tableToCaptureMapCache[cfID] = s.GetTableToCaptureMap(cfID) -} - -// invalidateTableToCaptureCache used to clear all table-to-capture mapping cache. -func (s *ownerReactorState) invalidateTableToCaptureCache() { - s.tableToCaptureMapCache = make(map[model.ChangeFeedID]map[model.TableID]model.CaptureID) -} diff --git a/cdc/owner/owner_state_test.go b/cdc/owner/owner_state_test.go deleted file mode 100644 index 4083b62d475..00000000000 --- a/cdc/owner/owner_state_test.go +++ /dev/null @@ -1,1004 +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 owner - -import ( - "encoding/json" - "reflect" - "testing" - "time" - - "github.com/pingcap/check" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/orchestrator" - "github.com/pingcap/ticdc/pkg/util/testleak" -) - -func Test(t *testing.T) { check.TestingT(t) } - -type ownerStateTestSuite struct { -} - -var _ = check.Suite(&ownerStateTestSuite{}) - -func (s *ownerStateTestSuite) TestUpdateCapture(c *check.C) { - defer testleak.AfterTest(c)() - ownerState := newCDCReactorState() - tester := orchestrator.NewReactorStateTester(ownerState, map[string]string{}) - - // Test adding a capture - captureInfo := &model.CaptureInfo{ - ID: "capture-1", - AdvertiseAddr: "127.0.0.1:8083", - } - captureInfoJSON, err := json.Marshal(captureInfo) - c.Assert(err, check.IsNil) - - captureHandlerCalled := false - ownerState.SetNewCaptureHandler(func(id model.CaptureID) { - captureHandlerCalled = true - c.Assert(id, check.Equals, "capture-1") - }) - - err = tester.UpdateKeys(map[string][]byte{ - "/tidb/cdc/capture/capture-1": captureInfoJSON, - }) - c.Assert(err, check.IsNil) - c.Assert(ownerState.Captures["capture-1"], check.DeepEquals, captureInfo) - c.Assert(captureHandlerCalled, check.IsTrue) - - // Test updating a capture - captureInfo.AdvertiseAddr = "127.0.0.1:8084" - captureInfoJSON, err = json.Marshal(captureInfo) - c.Assert(err, check.IsNil) - - captureHandlerCalled = false - err = tester.UpdateKeys(map[string][]byte{ - "/tidb/cdc/capture/capture-1": captureInfoJSON, - }) - c.Assert(err, check.IsNil) - c.Assert(ownerState.Captures["capture-1"], check.DeepEquals, captureInfo) - c.Assert(captureHandlerCalled, check.IsFalse) - - // Test deleting a capture - err = tester.UpdateKeys(map[string][]byte{ - "/tidb/cdc/capture/capture-1": nil, - }) - c.Assert(err, check.IsNil) - c.Assert(ownerState.Captures, check.Not(check.HasKey), "capture-1") - c.Assert(captureHandlerCalled, check.IsFalse) -} - -func (s *ownerStateTestSuite) TestUpdateChangeFeedStatus(c *check.C) { - defer testleak.AfterTest(c)() - ownerState := newCDCReactorState() - tester := orchestrator.NewReactorStateTester(ownerState, map[string]string{}) - - // Test adding a changeFeed - changeFeedStatus := &model.ChangeFeedStatus{ - ResolvedTs: 1000, - CheckpointTs: 800, - AdminJobType: 0, - } - - changeFeedStatusJSON, err := json.Marshal(changeFeedStatus) - c.Assert(err, check.IsNil) - - err = tester.UpdateKeys(map[string][]byte{ - "/tidb/cdc/job/cf-1": changeFeedStatusJSON, - }) - c.Assert(err, check.IsNil) - c.Assert(ownerState.ChangeFeedStatuses["cf-1"], check.DeepEquals, changeFeedStatus) - - // Test updating a changeFeed - changeFeedStatus.ResolvedTs = 1200 - changeFeedStatusJSON, err = json.Marshal(changeFeedStatus) - c.Assert(err, check.IsNil) - - err = tester.UpdateKeys(map[string][]byte{ - "/tidb/cdc/job/cf-1": changeFeedStatusJSON, - }) - c.Assert(err, check.IsNil) - c.Assert(ownerState.ChangeFeedStatuses["cf-1"], check.DeepEquals, changeFeedStatus) - - // Test deleting a changeFeed - err = tester.UpdateKeys(map[string][]byte{ - "/tidb/cdc/job/cf-1": nil, - }) - c.Assert(err, check.IsNil) - c.Assert(ownerState.ChangeFeedStatuses, check.Not(check.HasKey), "cf-1") -} - -func (s *ownerStateTestSuite) TestUpdateTaskPosition(c *check.C) { - defer testleak.AfterTest(c)() - ownerState := newCDCReactorState() - tester := orchestrator.NewReactorStateTester(ownerState, map[string]string{}) - - // Test adding a position - taskPosition := &model.TaskPosition{ - CheckPointTs: 800, - ResolvedTs: 1000, - } - - taskPositionJSON, err := json.Marshal(taskPosition) - c.Assert(err, check.IsNil) - - err = tester.UpdateKeys(map[string][]byte{ - "/tidb/cdc/task/position/capture-1/cf-1": taskPositionJSON, - }) - c.Assert(err, check.IsNil) - c.Assert(ownerState.TaskPositions["cf-1"]["capture-1"], check.DeepEquals, taskPosition) - - // Test updating a position - taskPosition.ResolvedTs = 1200 - taskPositionJSON, err = json.Marshal(taskPosition) - c.Assert(err, check.IsNil) - - err = tester.UpdateKeys(map[string][]byte{ - "/tidb/cdc/task/position/capture-1/cf-1": taskPositionJSON, - }) - c.Assert(err, check.IsNil) - c.Assert(ownerState.TaskPositions["cf-1"]["capture-1"], check.DeepEquals, taskPosition) - - // Test deleting a position - err = tester.UpdateKeys(map[string][]byte{ - "/tidb/cdc/task/position/capture-1/cf-1": nil, - }) - c.Assert(err, check.IsNil) - c.Assert(ownerState.TaskPositions["cf-1"], check.Not(check.HasKey), "capture-1") -} - -func (s *ownerStateTestSuite) TestUpdateTaskStatus(c *check.C) { - defer testleak.AfterTest(c)() - ownerState := newCDCReactorState() - tester := orchestrator.NewReactorStateTester(ownerState, map[string]string{}) - - // Test adding a status - taskStatus := &model.TaskStatus{ - Tables: map[model.TableID]*model.TableReplicaInfo{1: { - StartTs: 100, - MarkTableID: 0, - }}, - Operation: nil, - } - - taskStatusJSON, err := json.Marshal(taskStatus) - c.Assert(err, check.IsNil) - - err = tester.UpdateKeys(map[string][]byte{ - "/tidb/cdc/task/status/capture-1/cf-1": taskStatusJSON, - }) - c.Assert(err, check.IsNil) - c.Assert(ownerState.TaskStatuses["cf-1"]["capture-1"], check.DeepEquals, taskStatus) - - // Test updating a status - taskStatus.Tables[2] = &model.TableReplicaInfo{StartTs: 200} - taskStatusJSON, err = json.Marshal(taskStatus) - c.Assert(err, check.IsNil) - - err = tester.UpdateKeys(map[string][]byte{ - "/tidb/cdc/task/status/capture-1/cf-1": taskStatusJSON, - }) - c.Assert(err, check.IsNil) - c.Assert(ownerState.TaskStatuses["cf-1"]["capture-1"], check.DeepEquals, taskStatus) - - // Test deleting a status - err = tester.UpdateKeys(map[string][]byte{ - "/tidb/cdc/task/status/capture-1/cf-1": nil, - }) - c.Assert(err, check.IsNil) - c.Assert(ownerState.TaskStatuses["cf-1"], check.Not(check.HasKey), "capture-1") -} - -func (s *ownerStateTestSuite) TestUpdateChangeFeedInfo(c *check.C) { - defer testleak.AfterTest(c)() - ownerState := newCDCReactorState() - tester := orchestrator.NewReactorStateTester(ownerState, map[string]string{}) - - // Test adding a changeFeed - changeFeedInfo := &model.ChangeFeedInfo{ - SinkURI: "blackhole:///", - } - - changeFeedInfoJSON, err := json.Marshal(changeFeedInfo) - c.Assert(err, check.IsNil) - - err = tester.UpdateKeys(map[string][]byte{ - "/tidb/cdc/changefeed/info/cf-1": changeFeedInfoJSON, - }) - c.Assert(err, check.IsNil) - c.Assert(ownerState.ChangeFeedInfos["cf-1"], check.DeepEquals, changeFeedInfo) - - // Test updating a changeFeed - changeFeedInfo.State = model.StateFailed - changeFeedInfoJSON, err = json.Marshal(changeFeedInfo) - c.Assert(err, check.IsNil) - - err = tester.UpdateKeys(map[string][]byte{ - "/tidb/cdc/changefeed/info/cf-1": changeFeedInfoJSON, - }) - c.Assert(err, check.IsNil) - c.Assert(ownerState.ChangeFeedInfos["cf-1"], check.DeepEquals, changeFeedInfo) - - // Test deleting a changeFeed - err = tester.UpdateKeys(map[string][]byte{ - "/tidb/cdc/changefeed/info/cf-1": nil, - }) - c.Assert(err, check.IsNil) - c.Assert(ownerState.ChangeFeedInfos, check.Not(check.HasKey), "cf-1") -} - -func (s *ownerStateTestSuite) TestPatchChangeFeedStatus(c *check.C) { - defer testleak.AfterTest(c)() - ownerState := newCDCReactorState() - changeFeedStatus := &model.ChangeFeedStatus{ - ResolvedTs: 800, - CheckpointTs: 1000, - AdminJobType: 0, - } - changeFeedStatusJSON, err := json.Marshal(changeFeedStatus) - c.Assert(err, check.IsNil) - tester := orchestrator.NewReactorStateTester(ownerState, map[string]string{}) - - // test the creation case - ownerState.UpdateChangeFeedStatus("cf-1", 800, 1000) - err = tester.ApplyPatches() - c.Assert(err, check.IsNil) - c.Assert(tester.KVEntries()["/tidb/cdc/job/cf-1"], check.Equals, string(changeFeedStatusJSON)) - - // test the normal case - ownerState.UpdateChangeFeedStatus("cf-1", 1200, 1000) - err = tester.ApplyPatches() - c.Assert(err, check.IsNil) - c.Assert(tester.KVEntries()["/tidb/cdc/job/cf-1"], check.NotNil) - - targetChangeFeedStatus := &model.ChangeFeedStatus{ - ResolvedTs: 1200, - CheckpointTs: 1000, - AdminJobType: 0, - } - targetChangeFeedStatusJSON, err := json.Marshal(targetChangeFeedStatus) - c.Assert(err, check.IsNil) - c.Assert(tester.KVEntries()["/tidb/cdc/job/cf-1"], check.Equals, string(targetChangeFeedStatusJSON)) - - // test the zero cases - c.Assert(func() { - ownerState.UpdateChangeFeedStatus("cf-1", 0, 1000) - }, check.PanicMatches, ".*illegal changeFeedStatus.*") - - c.Assert(func() { - ownerState.UpdateChangeFeedStatus("cf-1", 1200, 0) - }, check.PanicMatches, ".*illegal changeFeedStatus.*") - - // test the regression case - c.Assert(func() { - ownerState.UpdateChangeFeedStatus("cf-1", 1200, 900) - _ = tester.ApplyPatches() - }, check.PanicMatches, ".*checkpointTs regressed.*") -} - -func (s *ownerStateTestSuite) TestDispatchTable(c *check.C) { - defer testleak.AfterTest(c)() - ownerState := newCDCReactorState() - replicaInfo := model.TableReplicaInfo{ - StartTs: 1000, - MarkTableID: 0, - } - tester := orchestrator.NewReactorStateTester(ownerState, map[string]string{}) - - // test the creation case - ownerState.DispatchTable("cf-1", "capture-1", 1, replicaInfo) - err := tester.ApplyPatches() - c.Assert(err, check.IsNil) - c.Assert(tester.KVEntries()["/tidb/cdc/task/status/capture-1/cf-1"], check.NotNil) - - var status model.TaskStatus - err = json.Unmarshal([]byte(tester.KVEntries()["/tidb/cdc/task/status/capture-1/cf-1"]), &status) - c.Assert(err, check.IsNil) - c.Assert(status.Tables[1], check.DeepEquals, &replicaInfo) - c.Assert(status.Operation[1].Status, check.Equals, model.OperDispatched) - c.Assert(status.Operation[1].Delete, check.IsFalse) - c.Assert(status.Operation[1].BoundaryTs, check.Equals, uint64(1000)) - - // test the normal case - ownerState.DispatchTable("cf-1", "capture-1", 2, replicaInfo) - err = tester.ApplyPatches() - c.Assert(err, check.IsNil) - c.Assert(tester.KVEntries()["/tidb/cdc/task/status/capture-1/cf-1"], check.NotNil) - - err = json.Unmarshal([]byte(tester.KVEntries()["/tidb/cdc/task/status/capture-1/cf-1"]), &status) - c.Assert(err, check.IsNil) - c.Assert(status.Tables[2], check.DeepEquals, &replicaInfo) - c.Assert(status.Operation[2].Status, check.Equals, model.OperDispatched) - c.Assert(status.Operation[2].Delete, check.IsFalse) - c.Assert(status.Operation[2].BoundaryTs, check.Equals, uint64(1000)) - - c.Assert(status.Tables[1], check.NotNil) - c.Assert(status.Operation[2], check.NotNil) - - // test the duplication case - oldJSON := tester.KVEntries()["/tidb/cdc/task/status/capture-1/cf-1"] - ownerState.DispatchTable("cf-1", "capture-1", 2, replicaInfo) - err = tester.ApplyPatches() - c.Assert(err, check.IsNil) - c.Assert(tester.KVEntries()["/tidb/cdc/task/status/capture-1/cf-1"], check.Equals, oldJSON) -} - -func (s *ownerStateTestSuite) TestStartDeletingTableCase(c *check.C) { - defer testleak.AfterTest(c)() - ownerState := newCDCReactorState() - taskStatus := &model.TaskStatus{ - Tables: map[model.TableID]*model.TableReplicaInfo{ - 1: { - StartTs: 2000, - MarkTableID: 0, - }, - 2: { - StartTs: 3000, - MarkTableID: 0, - }, - }, - Operation: nil, - } - ownerState.ChangeFeedStatuses["cf-1"] = &model.ChangeFeedStatus{ - ResolvedTs: 5000, - CheckpointTs: 4500, - AdminJobType: 0, - } - ownerState.TaskStatuses["cf-1"] = map[model.CaptureID]*model.TaskStatus{ - "capture-1": taskStatus, - } - taskStatusJSON, err := json.Marshal(taskStatus) - c.Assert(err, check.IsNil) - tester := orchestrator.NewReactorStateTester(ownerState, map[string]string{ - "/tidb/cdc/task/status/capture-1/cf-1": string(taskStatusJSON), - }) - - // test the normal case - ownerState.StartDeletingTable("cf-1", "capture-1", 1) - err = tester.ApplyPatches() - c.Assert(err, check.IsNil) - - var newTaskStatus model.TaskStatus - err = json.Unmarshal([]byte(tester.KVEntries()["/tidb/cdc/task/status/capture-1/cf-1"]), &newTaskStatus) - c.Assert(err, check.IsNil) - c.Assert(newTaskStatus.Tables, check.DeepEquals, taskStatus.Tables) - c.Assert(newTaskStatus.Operation[1].Status, check.Equals, model.OperDispatched) - c.Assert(newTaskStatus.Operation[1].BoundaryTs, check.Equals, uint64(5000)) - c.Assert(newTaskStatus.Operation[1].Delete, check.IsTrue) - - // test the duplication case - ownerState.StartDeletingTable("cf-1", "capture-1", 1) - c.Assert(func() { - _ = tester.ApplyPatches() - }, check.PanicMatches, ".*repeated deletion.*") - - // test the inconsistent case - ownerState.StartDeletingTable("cf-1", "capture-1", 2) - // simulate a Etcd txn conflict - delete(tester.KVEntries(), "/tidb/cdc/task/status/capture-1/cf-1") - err = tester.ApplyPatches() - c.Assert(err, check.ErrorMatches, ".*TaskStatus deleted.*") - - // test the panic cases - delete(ownerState.TaskStatuses["cf-1"], "capture-1") - c.Assert(func() { - ownerState.StartDeletingTable("cf-1", "capture-1", 2) - }, check.PanicMatches, ".*capture not found.*") - - delete(ownerState.TaskStatuses, "cf-1") - c.Assert(func() { - ownerState.StartDeletingTable("cf-1", "capture-1", 2) - }, check.PanicMatches, ".*changeFeedState not found.*") -} - -func (s *ownerStateTestSuite) TestCleanOperation(c *check.C) { - defer testleak.AfterTest(c)() - ownerState := newCDCReactorState() - taskStatus := &model.TaskStatus{ - Tables: map[model.TableID]*model.TableReplicaInfo{ - 2: { - StartTs: 3000, - MarkTableID: 0, - }, - }, - Operation: map[model.TableID]*model.TableOperation{ - 1: { - Delete: false, - BoundaryTs: 4500, - Done: true, - Status: model.OperFinished, - }, - 2: { - Delete: false, - BoundaryTs: 4500, - Done: false, - Status: model.OperProcessed, - }, - }, - } - ownerState.ChangeFeedStatuses["cf-1"] = &model.ChangeFeedStatus{ - ResolvedTs: 5000, - CheckpointTs: 4500, - AdminJobType: 0, - } - ownerState.TaskStatuses["cf-1"] = map[model.CaptureID]*model.TaskStatus{ - "capture-1": taskStatus, - } - taskStatusJSON, err := json.Marshal(taskStatus) - c.Assert(err, check.IsNil) - tester := orchestrator.NewReactorStateTester(ownerState, map[string]string{ - "/tidb/cdc/task/status/capture-1/cf-1": string(taskStatusJSON), - }) - - // test the normal case - ownerState.CleanOperation("cf-1", "capture-1", 1) - err = tester.ApplyPatches() - c.Assert(err, check.IsNil) - - var newTaskStatus model.TaskStatus - err = json.Unmarshal([]byte(tester.KVEntries()["/tidb/cdc/task/status/capture-1/cf-1"]), &newTaskStatus) - c.Assert(err, check.IsNil) - c.Assert(newTaskStatus.Tables, check.DeepEquals, taskStatus.Tables) - c.Assert(newTaskStatus.Operation[2], check.DeepEquals, taskStatus.Operation[2]) - c.Assert(newTaskStatus.Operation, check.Not(check.HasKey), model.TableID(1)) - - // test the inconsistent case - ownerState.CleanOperation("cf-1", "capture-1", 1) - delete(tester.KVEntries(), "/tidb/cdc/task/status/capture-1/cf-1") - err = tester.ApplyPatches() - c.Assert(err, check.ErrorMatches, ".*TaskStatus deleted.*") - - // test the table-not-cleaned case - taskStatus.Tables[1] = &model.TableReplicaInfo{ - StartTs: 2000, - MarkTableID: 0, - } - taskStatus.Operation[1].Delete = true - taskStatusJSON, err = json.Marshal(taskStatus) - c.Assert(err, check.IsNil) - tester = orchestrator.NewReactorStateTester(ownerState, map[string]string{ - "/tidb/cdc/task/status/capture-1/cf-1": string(taskStatusJSON), - }) - ownerState.CleanOperation("cf-1", "capture-1", 1) - c.Assert(func() { - _ = tester.ApplyPatches() - }, check.PanicMatches, ".*table not cleaned.*") - - // test the panic cases - delete(ownerState.TaskStatuses["cf-1"], "capture-1") - c.Assert(func() { - ownerState.CleanOperation("cf-1", "capture-1", 2) - }, check.PanicMatches, ".*capture not found.*") - - delete(ownerState.TaskStatuses, "cf-1") - c.Assert(func() { - ownerState.CleanOperation("cf-1", "capture-1", 2) - }, check.PanicMatches, ".*changeFeedState not found.*") -} - -func (s *ownerStateTestSuite) TestAlterChangeFeedRuntimeState(c *check.C) { - defer testleak.AfterTest(c)() - ownerState := newCDCReactorState() - changeFeedInfo := &model.ChangeFeedInfo{ - SinkURI: "blackhole:///", - AdminJobType: model.AdminNone, - State: model.StateNormal, - ErrorHis: nil, - } - changeFeedInfoJSON, err := json.Marshal(changeFeedInfo) - c.Assert(err, check.IsNil) - - ownerState.ChangeFeedInfos["cf-1"] = changeFeedInfo - tester := orchestrator.NewReactorStateTester(ownerState, map[string]string{ - "/tidb/cdc/changefeed/info/cf-1": string(changeFeedInfoJSON), - }) - - // test the normal case - runningErr := &model.RunningError{ - Addr: "127.0.0.1:8083", - Code: "", - Message: "", - } - ts := time.Now().UnixNano() - ownerState.AlterChangeFeedRuntimeState("cf-1", model.AdminStop, model.StateFailed, runningErr, ts) - err = tester.ApplyPatches() - c.Assert(err, check.IsNil) - c.Assert(tester.KVEntries()["/tidb/cdc/changefeed/info/cf-1"], check.NotNil) - - var newChangeFeedInfo model.ChangeFeedInfo - err = json.Unmarshal([]byte(tester.KVEntries()["/tidb/cdc/changefeed/info/cf-1"]), &newChangeFeedInfo) - c.Assert(err, check.IsNil) - c.Assert(newChangeFeedInfo.Error, check.DeepEquals, runningErr) - c.Assert(newChangeFeedInfo.ErrorHis, check.DeepEquals, []int64{ts}) - c.Assert(newChangeFeedInfo.AdminJobType, check.Equals, model.AdminStop) - c.Assert(newChangeFeedInfo.State, check.Equals, model.StateFailed) - - // test that error is NOT cleared - // reset states - ownerState.ChangeFeedInfos["cf-1"] = &newChangeFeedInfo - ownerState.AlterChangeFeedRuntimeState("cf-1", model.AdminNone, model.StateNormal, nil, 0) - err = tester.ApplyPatches() - c.Assert(err, check.IsNil) - - err = json.Unmarshal([]byte(tester.KVEntries()["/tidb/cdc/changefeed/info/cf-1"]), &newChangeFeedInfo) - c.Assert(err, check.IsNil) - c.Assert(newChangeFeedInfo.Error, check.DeepEquals, runningErr) - c.Assert(newChangeFeedInfo.ErrorHis, check.DeepEquals, []int64{ts}) - c.Assert(newChangeFeedInfo.AdminJobType, check.Equals, model.AdminNone) - c.Assert(newChangeFeedInfo.State, check.Equals, model.StateNormal) - - // test changeFeedInfo-gone case: we should survive it - ownerState.AlterChangeFeedRuntimeState("cf-1", model.AdminNone, model.StateNormal, nil, 0) - delete(tester.KVEntries(), "/tidb/cdc/changefeed/info/cf-1") - err = tester.ApplyPatches() - c.Assert(err, check.IsNil) -} - -func (s *ownerStateTestSuite) TestCleanUpTaskStatus(c *check.C) { - defer testleak.AfterTest(c)() - ownerState := newCDCReactorState() - taskStatus := &model.TaskStatus{ - Tables: map[model.TableID]*model.TableReplicaInfo{ - 2: { - StartTs: 3000, - MarkTableID: 0, - }, - }, - Operation: map[model.TableID]*model.TableOperation{ - 1: { - Delete: false, - BoundaryTs: 4500, - Done: true, - Status: model.OperFinished, - }, - 2: { - Delete: false, - BoundaryTs: 4500, - Done: false, - Status: model.OperProcessed, - }, - }, - } - ownerState.TaskStatuses["cf-1"] = map[model.CaptureID]*model.TaskStatus{"capture-1": taskStatus} - - taskStatusJSON, err := json.Marshal(taskStatus) - c.Assert(err, check.IsNil) - tester := orchestrator.NewReactorStateTester(ownerState, map[string]string{ - "/tidb/cdc/task/status/capture-1/cf-1": string(taskStatusJSON), - }) - - // test the normal case - ownerState.CleanUpTaskStatus("cf-1", "capture-1") - err = tester.ApplyPatches() - c.Assert(err, check.IsNil) - c.Assert(tester.KVEntries(), check.Not(check.HasKey), "/tidb/cdc/task/status/capture-1/cf-1") - - // test the idempotent case - delete(ownerState.TaskStatuses["cf-1"], "capture-1") - ownerState.CleanUpTaskStatus("cf-1", "capture-1") - err = tester.ApplyPatches() - c.Assert(err, check.IsNil) - c.Assert(tester.KVEntries(), check.Not(check.HasKey), "/tidb/cdc/task/status/capture-1/cf-1") - - // test the no-changefeed case - delete(ownerState.TaskStatuses, "cf-1") - ownerState.CleanUpTaskStatus("cf-1", "capture-1") - err = tester.ApplyPatches() - c.Assert(err, check.IsNil) - - // test the conflict case - ownerState.TaskStatuses["cf-1"] = map[model.CaptureID]*model.TaskStatus{"capture-1": taskStatus} - tester.KVEntries()["/tidb/cdc/task/status/capture-1/cf-1"] = string(taskStatusJSON) - ownerState.CleanUpTaskStatus("cf-1", "capture-1") - delete(tester.KVEntries(), "/tidb/cdc/task/status/capture-1/cf-1") - err = tester.ApplyPatches() - c.Assert(err, check.IsNil) -} - -func (s *ownerStateTestSuite) TestCleanUpTaskPosition(c *check.C) { - defer testleak.AfterTest(c)() - ownerState := newCDCReactorState() - taskPosition := &model.TaskPosition{ - CheckPointTs: 1000, - ResolvedTs: 2000, - } - ownerState.TaskPositions["cf-1"] = map[model.CaptureID]*model.TaskPosition{"capture-1": taskPosition} - - taskPositionJSON, err := json.Marshal(taskPosition) - c.Assert(err, check.IsNil) - tester := orchestrator.NewReactorStateTester(ownerState, map[string]string{ - "/tidb/cdc/task/position/capture-1/cf-1": string(taskPositionJSON), - }) - - // test the normal case - ownerState.CleanUpTaskPosition("cf-1", "capture-1") - err = tester.ApplyPatches() - c.Assert(err, check.IsNil) - c.Assert(tester.KVEntries(), check.Not(check.HasKey), "/tidb/cdc/task/position/capture-1/cf-1") - - // test the idempotent case - delete(ownerState.TaskPositions["cf-1"], "capture-1") - ownerState.CleanUpTaskPosition("cf-1", "capture-1") - err = tester.ApplyPatches() - c.Assert(err, check.IsNil) - c.Assert(tester.KVEntries(), check.Not(check.HasKey), "/tidb/cdc/task/position/capture-1/cf-1") - - // test the no-changefeed case - delete(ownerState.TaskPositions, "cf-1") - ownerState.CleanUpTaskPosition("cf-1", "capture-1") - err = tester.ApplyPatches() - c.Assert(err, check.IsNil) - - // test the conflict case - ownerState.TaskPositions["cf-1"] = map[model.CaptureID]*model.TaskPosition{"capture-1": taskPosition} - tester.KVEntries()["/tidb/cdc/task/position/capture-1/cf-1"] = string(taskPositionJSON) - ownerState.CleanUpTaskPosition("cf-1", "capture-1") - delete(tester.KVEntries(), "/tidb/cdc/task/position/capture-1/cf-1") - err = tester.ApplyPatches() - c.Assert(err, check.IsNil) -} - -func (s *ownerStateTestSuite) TestGetCaptureTables(c *check.C) { - defer testleak.AfterTest(c)() - ownerState := newCDCReactorState() - taskStatus := &model.TaskStatus{ - Tables: map[model.TableID]*model.TableReplicaInfo{ - 1: { - StartTs: 2000, - MarkTableID: 0, - }, - 2: { - StartTs: 3000, - MarkTableID: 0, - }, - }, - Operation: map[model.TableID]*model.TableOperation{ - 2: { - Delete: false, - BoundaryTs: 4500, - Done: true, - Status: model.OperFinished, - }, - 3: { - Delete: false, - BoundaryTs: 4500, - Done: false, - Status: model.OperProcessed, - }, - }, - } - ownerState.TaskStatuses["cf-1"] = map[model.CaptureID]*model.TaskStatus{"capture-1": taskStatus} - ownerState.Captures["capture-1"] = &model.CaptureInfo{} - - // test normal case 1 - tables := ownerState.GetCaptureTables("cf-1", "capture-1") - if !tableIDListMatch(tables, []model.TableID{1, 2, 3}) { - c.Fatal(tables) - } - - // test normal case 2 - ownerState.TaskStatuses["cf-1"]["capture-1"].Operation[3].Delete = true - ownerState.TaskStatuses["cf-1"]["capture-1"].Operation[3].Status = model.OperFinished - tables = ownerState.GetCaptureTables("cf-1", "capture-1") - if !tableIDListMatch(tables, []model.TableID{1, 2}) { - c.Fatal(tables) - } - - // test normal case 3 - ownerState.TaskStatuses["cf-1"]["capture-1"].Tables[3] = &model.TableReplicaInfo{ - StartTs: 3000, - MarkTableID: 0, - } - - tables = ownerState.GetCaptureTables("cf-1", "capture-1") - if !tableIDListMatch(tables, []model.TableID{1, 2, 3}) { - c.Fatal(tables) - } - - // test no task status case 1 - delete(ownerState.TaskStatuses["cf-1"], "capture-1") - tables = ownerState.GetCaptureTables("cf-1", "capture-1") - c.Assert(tables, check.HasLen, 0) - - // test no task status case 2 - delete(ownerState.TaskStatuses, "cf-1") - tables = ownerState.GetCaptureTables("cf-1", "capture-1") - c.Assert(tables, check.HasLen, 0) - - // test capture gone case - delete(ownerState.Captures, "capture-1") - tables = ownerState.GetCaptureTables("cf-1", "capture-1") - c.Assert(tables, check.HasLen, 0) -} - -func tableIDListMatch(a, b []model.TableID) bool { - if len(a) != len(b) { - return false - } - - setA := make(map[model.TableID]struct{}) - for _, tableID := range a { - setA[tableID] = struct{}{} - } - - setB := make(map[model.TableID]struct{}) - for _, tableID := range b { - setB[tableID] = struct{}{} - } - - return reflect.DeepEqual(setA, setB) -} - -func (s *ownerStateTestSuite) TestCleanUpChangeFeedErrorHistory(c *check.C) { - defer testleak.AfterTest(c)() - now := time.Now() - ownerState := newCDCReactorState() - changeFeedInfo := &model.ChangeFeedInfo{ - SinkURI: "blackhole:///", - AdminJobType: model.AdminNone, - State: model.StateNormal, - ErrorHis: []int64{ - now.Add(-time.Hour).UnixNano() / 1e6, now.Add(-time.Minute*20).UnixNano() / 1e6, - now.Add(-time.Minute*5).UnixNano() / 1e6, now.Add(-time.Minute*3).UnixNano() / 1e6, - }, - } - changeFeedInfoJSON, err := json.Marshal(changeFeedInfo) - c.Assert(err, check.IsNil) - - ownerState.ChangeFeedInfos["cf-1"] = changeFeedInfo - tester := orchestrator.NewReactorStateTester(ownerState, map[string]string{ - "/tidb/cdc/changefeed/info/cf-1": string(changeFeedInfoJSON), - }) - - // test the normal case - ownerState.CleanUpChangeFeedErrorHistory("cf-1") - err = tester.ApplyPatches() - c.Assert(err, check.IsNil) - - var newChangeFeedInfo model.ChangeFeedInfo - err = json.Unmarshal([]byte(tester.KVEntries()["/tidb/cdc/changefeed/info/cf-1"]), &newChangeFeedInfo) - c.Assert(err, check.IsNil) - c.Assert(newChangeFeedInfo.ErrorHis, check.DeepEquals, - []int64{now.Add(-time.Minute*5).UnixNano() / 1e6, now.Add(-time.Minute*3).UnixNano() / 1e6}) - - // test the already cleaned case - oldJSON := tester.KVEntries()["/tidb/cdc/changefeed/info/cf-1"] - ownerState.CleanUpChangeFeedErrorHistory("cf-1") - err = tester.ApplyPatches() - c.Assert(err, check.IsNil) - c.Assert(tester.KVEntries()["/tidb/cdc/changefeed/info/cf-1"], check.Equals, oldJSON) - - // test changeFeedInfo gone case - ownerState.CleanUpChangeFeedErrorHistory("cf-1") - delete(tester.KVEntries(), "/tidb/cdc/changefeed/info/cf-1") - err = tester.ApplyPatches() - c.Assert(err, check.IsNil) -} - -func (s *ownerStateTestSuite) TestGetTableToCaptureMap(c *check.C) { - defer testleak.AfterTest(c)() - ownerState := newCDCReactorState() - taskStatus1 := &model.TaskStatus{ - Tables: map[model.TableID]*model.TableReplicaInfo{ - 1: { - StartTs: 2000, - MarkTableID: 0, - }, - 2: { - StartTs: 3000, - MarkTableID: 0, - }, - }, - } - taskStatus2 := &model.TaskStatus{ - Tables: map[model.TableID]*model.TableReplicaInfo{ - 3: { - StartTs: 2000, - MarkTableID: 0, - }, - 4: { - StartTs: 3000, - MarkTableID: 0, - }, - }, - } - ownerState.TaskStatuses["cf-1"] = map[model.CaptureID]*model.TaskStatus{ - "capture-1": taskStatus1, "capture-2": taskStatus2, - } - ownerState.Captures["capture-1"] = &model.CaptureInfo{} - ownerState.Captures["capture-2"] = &model.CaptureInfo{} - - // test the basic case - tableToCaptureMap := ownerState.GetTableToCaptureMap("cf-1") - c.Assert(tableToCaptureMap, check.DeepEquals, map[model.TableID]model.CaptureID{ - 1: "capture-1", - 2: "capture-1", - 3: "capture-2", - 4: "capture-2", - }) - - // test the operation case - ownerState.TaskStatuses["cf-1"]["capture-1"].Operation = make(map[model.TableID]*model.TableOperation) - ownerState.TaskStatuses["cf-1"]["capture-1"].Operation[5] = &model.TableOperation{ - Delete: false, - BoundaryTs: 5000, - Done: false, - Status: model.OperDispatched, - } - ownerState.TaskStatuses["cf-1"]["capture-2"].Operation = make(map[model.TableID]*model.TableOperation) - ownerState.TaskStatuses["cf-1"]["capture-2"].Operation[6] = &model.TableOperation{ - Delete: true, - BoundaryTs: 5000, - Done: true, - Status: model.OperFinished, - } - tableToCaptureMap = ownerState.GetTableToCaptureMap("cf-1") - c.Assert(tableToCaptureMap, check.DeepEquals, map[model.TableID]model.CaptureID{ - 1: "capture-1", - 2: "capture-1", - 3: "capture-2", - 4: "capture-2", - 5: "capture-1", - }) -} - -func (s *ownerStateTestSuite) TestGetTableProgressAndActiveTables(c *check.C) { - defer testleak.AfterTest(c)() - ownerState := newCDCReactorState() - taskStatus1 := &model.TaskStatus{ - Tables: map[model.TableID]*model.TableReplicaInfo{ - 1: { - StartTs: 2000, - MarkTableID: 0, - }, - 2: { - StartTs: 3000, - MarkTableID: 0, - }, - }, - } - taskStatus1JSON, err := json.Marshal(taskStatus1) - c.Assert(err, check.IsNil) - - taskStatus2 := &model.TaskStatus{ - Tables: map[model.TableID]*model.TableReplicaInfo{ - 3: { - StartTs: 2000, - MarkTableID: 0, - }, - 4: { - StartTs: 3000, - MarkTableID: 0, - }, - }, - } - taskStatus2JSON, err := json.Marshal(taskStatus2) - c.Assert(err, check.IsNil) - - taskPosition1 := &model.TaskPosition{ - CheckPointTs: 3500, - ResolvedTs: 4000, - } - taskPosition1JSON, err := json.Marshal(taskPosition1) - c.Assert(err, check.IsNil) - - taskPosition2 := &model.TaskPosition{ - CheckPointTs: 3600, - ResolvedTs: 3900, - } - taskPosition2JSON, err := json.Marshal(taskPosition2) - c.Assert(err, check.IsNil) - - captureInfo1 := &model.CaptureInfo{ - ID: "127.0.0.1:8081", - AdvertiseAddr: "127.0.0.1:8081", - } - captureInfo1JSON, err := json.Marshal(captureInfo1) - c.Assert(err, check.IsNil) - - captureInfo2 := &model.CaptureInfo{ - ID: "127.0.0.1:8082", - AdvertiseAddr: "127.0.0.1:8082", - } - captureInfo2JSON, err := json.Marshal(captureInfo2) - c.Assert(err, check.IsNil) - - tester := orchestrator.NewReactorStateTester(ownerState, map[string]string{}) - err = tester.UpdateKeys(map[string][]byte{ - "/tidb/cdc/task/status/capture-1/cf-1": taskStatus1JSON, - "/tidb/cdc/task/status/capture-2/cf-1": taskStatus2JSON, - "/tidb/cdc/task/position/capture-1/cf-1": taskPosition1JSON, - "/tidb/cdc/task/position/capture-2/cf-1": taskPosition2JSON, - "/tidb/cdc/capture/capture-1": captureInfo1JSON, - "/tidb/cdc/capture/capture-2": captureInfo2JSON, - }) - c.Assert(err, check.IsNil) - - // test the basic case - progress := ownerState.GetTableProgress("cf-1", 1) - c.Assert(progress, check.DeepEquals, &tableProgress{ - resolvedTs: 4000, - checkpointTs: 3500, - }) - - progress = ownerState.GetTableProgress("cf-1", 2) - c.Assert(progress, check.DeepEquals, &tableProgress{ - resolvedTs: 4000, - checkpointTs: 3500, - }) - - progress = ownerState.GetTableProgress("cf-1", 3) - c.Assert(progress, check.DeepEquals, &tableProgress{ - resolvedTs: 3900, - checkpointTs: 3600, - }) - - progress = ownerState.GetTableProgress("cf-1", 4) - c.Assert(progress, check.DeepEquals, &tableProgress{ - resolvedTs: 3900, - checkpointTs: 3600, - }) - - activeTables := ownerState.GetChangeFeedActiveTables("cf-1") - if !tableIDListMatch(activeTables, []model.TableID{1, 2, 3, 4}) { - c.Fatal(activeTables) - } - - // test stopping table - delete(taskStatus1.Tables, 1) - taskStatus1JSON, err = json.Marshal(taskStatus1) - c.Assert(err, check.IsNil) - - err = tester.UpdateKeys(map[string][]byte{ - "/tidb/cdc/task/status/capture-1/cf-1": taskStatus1JSON, - }) - c.Assert(err, check.IsNil) - - progress = ownerState.GetTableProgress("cf-1", 1) - c.Assert(progress, check.IsNil) - - activeTables = ownerState.GetChangeFeedActiveTables("cf-1") - if !tableIDListMatch(activeTables, []model.TableID{2, 3, 4}) { - c.Fatal(activeTables) - } - - // test capture gone case - err = tester.UpdateKeys(map[string][]byte{ - "/tidb/cdc/capture/capture-2": nil, - }) - c.Assert(err, check.IsNil) - - progress = ownerState.GetTableProgress("cf-1", 3) - c.Assert(progress, check.IsNil) - - activeTables = ownerState.GetChangeFeedActiveTables("cf-1") - if !tableIDListMatch(activeTables, []model.TableID{2}) { - c.Fatal(activeTables) - } - - // test position gone case - err = tester.UpdateKeys(map[string][]byte{ - "/tidb/cdc/task/position/capture-1/cf-1": nil, - }) - c.Assert(err, check.IsNil) - - progress = ownerState.GetTableProgress("cf-1", 2) - c.Assert(progress, check.IsNil) - - activeTables = ownerState.GetChangeFeedActiveTables("cf-1") - if !tableIDListMatch(activeTables, []model.TableID{2}) { - c.Fatal(activeTables) - } -} diff --git a/cdc/owner/owner_test.go b/cdc/owner/owner_test.go new file mode 100644 index 00000000000..8b178d1d25b --- /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.GoTimeToTS(time.Now()), + 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.GoTimeToTS(time.Now()), + 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.GoTimeToTS(time.Now()), + 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 index ee8c0273481..53e3d596e36 100644 --- a/cdc/owner/scheduler.go +++ b/cdc/owner/scheduler.go @@ -15,301 +15,373 @@ package owner import ( "math" - "math/rand" - "sync" - "time" + "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" ) -// scheduler is designed to abstract away the complexities associated with the Etcd data model. -type scheduler interface { - // PutTasks is used to pass ALL the tables that need replicating to the scheduler. - // It should be safe to be called repeatedly with the same argument, as long as the argument correctly - // represents the current tables. - // USE ONLY after IsReady() returns true. - PutTasks(tables map[model.TableID]*tableTask) - // SetAffinity sets a table's affinity to a capture. - // Affinities record a binding (often temporary) of tables to captures, and it is a useful mechanism to - // implement manual table migration. - SetAffinity(tableID model.TableID, captureID model.CaptureID, ttl int) - // IsReady returns true if and only if the scheduler is ready to perform operations. - // IsReady always returns the same value within a EtcdWorker tick. - IsReady() bool +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 schedulerImpl struct { - ownerState *ownerReactorState - cfID model.ChangeFeedID +type moveTableJob struct { + tableID model.TableID + target model.CaptureID +} - // the affinities mutex, guarding against concurrent access from the HTTP handle - mu sync.Mutex - affinities map[model.TableID]*affinity +type scheduler struct { + state *model.ChangefeedReactorState + currentTables []model.TableID + captures map[model.CaptureID]*model.CaptureInfo - // this flag is set by a capture-added event in the ownerState - needRebalance bool - captureWorkloadDeltas map[model.CaptureID]int + moveTableTargets map[model.TableID]model.CaptureID + moveTableJobQueue []*moveTableJob + needRebalanceNextTick bool + lastTickCaptureCount int } -// affinity is used to record a table-capture affinity setting. -type affinity struct { - targetCapture model.CaptureID - deadline time.Time +func newScheduler() *scheduler { + return &scheduler{ + moveTableTargets: make(map[model.TableID]model.CaptureID), + } } -func newScheduler(ownerState *ownerReactorState, cfID model.ChangeFeedID) *schedulerImpl { - ret := &schedulerImpl{ - ownerState: ownerState, - cfID: cfID, - affinities: make(map[model.TableID]*affinity), +// 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 +} - ownerState.SetNewCaptureHandler(func(captureID model.CaptureID) { - ret.onNewCapture(captureID) +func (s *scheduler) MoveTable(tableID model.TableID, target model.CaptureID) { + s.moveTableJobQueue = append(s.moveTableJobQueue, &moveTableJob{ + tableID: tableID, + target: target, }) +} - return ret +// 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 *schedulerImpl) PutTasks(tables map[model.TableID]*tableTask) { - s.captureWorkloadDeltas = make(map[model.CaptureID]int) +func (s *scheduler) Rebalance() { + s.needRebalanceNextTick = true +} - // We do NOT want to touch these tables because they are being deleted. - // We wait for the deletion(s) to finish before redispatching. - pendingList := s.cleanUpOperations() - pendingSet := make(map[model.TableID]struct{}) - for _, tableID := range pendingList { - pendingSet[tableID] = struct{}{} +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 +} - tableToCaptureMap := s.ownerState.GetTableToCaptureMap(s.cfID) +// 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) - // handle adding table - for tableID, task := range tables { - if _, ok := pendingSet[tableID]; ok { - // Table has a pending deletion. Skip. + 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 + } + } - if _, ok := tableToCaptureMap[tableID]; !ok { - // Table is not assigned to a capture. - target := s.getMinWorkloadCapture() - if target == "" { - log.Warn("no capture is active") - break - } - - if affCapture := s.lookUpAffinity(tableID); affCapture != "" { - log.Info("Dispatching table using affinity", - zap.String("cfID", s.cfID), - zap.Int("tableID", int(tableID)), - zap.String("target-capture", affCapture)) - target = affCapture + 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)) + } + } - replicaInfo := model.TableReplicaInfo{ - StartTs: task.CheckpointTs + 1, - MarkTableID: 0, // TODO support cyclic replication + getMinWorkloadCapture := func() model.CaptureID { + minCapture := "" + minWorkLoad := uint64(math.MaxUint64) + for captureID, workload := range workloads { + if workload < minWorkLoad { + minCapture = captureID + minWorkLoad = workload } + } - log.Info("Dispatching table", - zap.Int64("table-id", tableID), - zap.String("target-capture", target), - zap.String("changefeed-id", s.cfID)) - - s.ownerState.DispatchTable(s.cfID, target, tableID, replicaInfo) - s.captureWorkloadDeltas[target]++ + if minCapture == "" { + log.Panic("Unreachable, no capture is found") } + return minCapture } - // handle deleting table - for tableID, captureID := range tableToCaptureMap { - if _, ok := pendingSet[tableID]; ok { - // Table has a pending deletion. Skip. + for _, pendingJob := range pendingJobs { + if pendingJob.TargetCapture != "" { continue } + minCapture := getMinWorkloadCapture() + pendingJob.TargetCapture = minCapture + workloads[minCapture] += 1 + } +} - if _, ok := tables[tableID]; !ok { - // Table should be deleted from the capture - log.Info("Stopping table", - zap.Int64("table-id", tableID), - zap.String("capture", captureID), - zap.String("changefeed-id", s.cfID)) - - s.ownerState.StartDeletingTable(s.cfID, captureID, tableID) - s.captureWorkloadDeltas[captureID]-- +// 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, + }) } - - if s.needRebalance { - s.needRebalance = false - s.triggerRebalance() + // 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 } -// cleanUpOperations returns tablesIDs of tables that are NOT suitable for immediate redispatching. -func (s *schedulerImpl) cleanUpOperations() []model.TableID { - var pendingList []model.TableID - - for captureID, taskStatus := range s.ownerState.TaskStatuses[s.cfID] { - for tableID, operation := range taskStatus.Operation { - if operation.Status == model.OperFinished { - s.ownerState.CleanOperation(s.cfID, captureID, tableID) - } else { - // Only those tables that are being deleted are added to the pendingList, - // because while it is unsafe to try to dispatch a table in the process of deleting, - // it is safe to try to delete a table immediately after it being dispatched. - // In summary, being run by two capture is dangerous, but not being run at all is safe. - if operation.Delete { - pendingList = append(pendingList, tableID) +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 + }) } - - return pendingList } -// IsReady returns whether the scheduler is ready to process new requests. -func (s *schedulerImpl) IsReady() bool { - return !s.cleanUpStaleCaptureStatus() -} - -func (s *schedulerImpl) SetAffinity(tableID model.TableID, captureID model.CaptureID, ttl int) { - log.Info("Setting table affinity", - zap.String("cfID", s.cfID), - zap.String("captureID", captureID), - zap.Int("tableID", int(tableID)), - zap.Int("ttl", ttl)) - - s.mu.Lock() - defer s.mu.Unlock() - - s.affinities[tableID] = &affinity{ - targetCapture: captureID, - deadline: time.Now().Add(time.Duration(ttl) * time.Second), +// 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 *schedulerImpl) onNewCapture(_ model.CaptureID) { - s.needRebalance = true -} - -func (s *schedulerImpl) lookUpAffinity(tableID model.TableID) model.CaptureID { - s.mu.Lock() - defer s.mu.Unlock() - - s.cleanUpAffinities() - - af, ok := s.affinities[tableID] - if !ok { - return "" +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 } - - if !s.ownerState.CaptureExists(af.targetCapture) { - delete(s.affinities, tableID) - return "" - } - - return af.targetCapture + // we only support rebalance by table number for now + return s.rebalanceByTableNum() } -// cleanUpAffinities must be called with mu locked. -func (s *schedulerImpl) cleanUpAffinities() { - for tableID, af := range s.affinities { - if af.deadline.Before(time.Now()) { - delete(s.affinities, tableID) - } +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 } -func (s *schedulerImpl) triggerRebalance() { - tableToCaptureMap := s.ownerState.GetTableToCaptureMap(s.cfID) - totalTableNum := len(tableToCaptureMap) - captureNum := len(s.ownerState.Captures) - +// 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("cfID", s.cfID), + zap.String("changefeed", s.state.ID), zap.Int("table-num", totalTableNum), zap.Int("capture-num", captureNum), zap.Int("target-limit", upperLimitPerCapture)) - for captureID := range s.ownerState.Captures { - captureTables := s.ownerState.GetCaptureTables(s.cfID, captureID) - - // Use rand.Perm as a randomization source for choosing victims uniformly. - randPerm := rand.Perm(len(captureTables)) - - for i := 0; i < len(captureTables)-upperLimitPerCapture; i++ { - victimIdx := randPerm[i] - victimTableID := captureTables[victimIdx] - - log.Info("triggerRebalance: Stopping table", - zap.Int64("table-id", victimTableID), - zap.String("capture", captureID), - zap.String("changefeed-id", s.cfID)) - - s.ownerState.StartDeletingTable(s.cfID, captureID, victimTableID) + for captureID, taskStatus := range s.state.TaskStatuses { + tableNum2Remove := len(taskStatus.Tables) - upperLimitPerCapture + if tableNum2Remove <= 0 { + continue } - } -} - -func (s *schedulerImpl) getMinWorkloadCapture() model.CaptureID { - workloads := make(map[model.CaptureID]int) - for captureID := range s.ownerState.Captures { - workloads[captureID] = s.captureWorkloadDeltas[captureID] - } - - for _, captureStatuses := range s.ownerState.TaskStatuses { - for captureID, taskStatus := range captureStatuses { - if _, ok := workloads[captureID]; ok { - workloads[captureID] += len(taskStatus.Tables) + // 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-- } } - - minCapture := "" - minWorkLoad := math.MaxInt32 - for captureID, workload := range workloads { - if workload < 0 { - // TODO investigate and remove this log - log.Debug("negative workload, bug?", - zap.Reflect("workloads", workloads), - zap.Reflect("deltas", s.captureWorkloadDeltas)) - workload = 0 - } - - if workload < minWorkLoad { - minCapture = captureID - minWorkLoad = workload - } - } - - return minCapture -} - -// cleanUpStaleCaptureStatus cleans up TaskStatus and TaskPosition for captures that have just gone offline. -func (s *schedulerImpl) cleanUpStaleCaptureStatus() bool { - if _, ok := s.ownerState.TaskStatuses[s.cfID]; !ok { - return false - } - - hasPending := false - for captureID := range s.ownerState.TaskStatuses[s.cfID] { - if !s.ownerState.CaptureExists(captureID) { - log.Info("cleaning up stale capture", - zap.String("cfID", s.cfID), - zap.String("captureID", captureID)) - s.ownerState.CleanUpTaskStatus(s.cfID, captureID) - s.ownerState.CleanUpTaskPosition(s.cfID, captureID) - hasPending = true - } - } - - return hasPending + return } diff --git a/cdc/owner/scheduler_test.go b/cdc/owner/scheduler_test.go index 7262c8a754d..712a89a31e6 100644 --- a/cdc/owner/scheduler_test.go +++ b/cdc/owner/scheduler_test.go @@ -14,378 +14,329 @@ package owner import ( - "encoding/json" + "fmt" + "math/rand" "github.com/pingcap/check" - "github.com/pingcap/ticdc/cdc/kv" "github.com/pingcap/ticdc/cdc/model" "github.com/pingcap/ticdc/pkg/orchestrator" "github.com/pingcap/ticdc/pkg/util/testleak" ) -type schedulerTestSuite struct { -} +var _ = check.Suite(&schedulerSuite{}) -var _ = check.Suite(&schedulerTestSuite{}) +type schedulerSuite struct { + changefeedID model.ChangeFeedID + state *model.ChangefeedReactorState + tester *orchestrator.ReactorStateTester + captures map[model.CaptureID]*model.CaptureInfo + scheduler *scheduler +} -func setUp(c *check.C) (*schedulerImpl, *orchestrator.ReactorStateTester) { - ownerState := newCDCReactorState() - tester := orchestrator.NewReactorStateTester(ownerState, map[string]string{}) - return newScheduler(ownerState, "cf-1"), tester +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 addCapture(c *check.C, tester *orchestrator.ReactorStateTester, captureID model.CaptureID) { +func (s *schedulerSuite) addCapture(captureID model.CaptureID) { captureInfo := &model.CaptureInfo{ ID: captureID, } - captureInfoJSON, err := json.Marshal(captureInfo) - c.Assert(err, check.IsNil) - - err = tester.UpdateKeys(map[string][]byte{ - kv.GetEtcdKeyCaptureInfo(captureID): captureInfoJSON, + s.captures[captureID] = captureInfo + s.state.PatchTaskStatus(captureID, func(status *model.TaskStatus) (*model.TaskStatus, bool, error) { + return &model.TaskStatus{}, true, nil }) - c.Assert(err, check.IsNil) + s.tester.MustApplyPatches() } -func removeCapture(c *check.C, tester *orchestrator.ReactorStateTester, captureID model.CaptureID) { - err := tester.UpdateKeys(map[string][]byte{ - kv.GetEtcdKeyCaptureInfo(captureID): nil, +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 }) - c.Assert(err, check.IsNil) -} - -func mockProcessorTick(c *check.C, tester *orchestrator.ReactorStateTester, captureID model.CaptureID) { - var taskStatus model.TaskStatus - if jsonStr, ok := tester.KVEntries()[kv.GetEtcdKeyTaskStatus("cf-1", captureID)]; ok { - err := json.Unmarshal([]byte(jsonStr), &taskStatus) - c.Assert(err, check.IsNil) - } - - for tableID, op := range taskStatus.Operation { - if op.Delete { - delete(taskStatus.Tables, tableID) - op.Status = model.OperFinished - op.Done = true - } else { - if taskStatus.Tables == nil { - taskStatus.Tables = make(map[model.TableID]*model.TableReplicaInfo) - } - taskStatus.Tables[tableID] = &model.TableReplicaInfo{ - StartTs: op.BoundaryTs, + 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, + } } - op.Status = model.OperFinished - op.Done = true } - } - - newBytes, err := json.Marshal(&taskStatus) - c.Assert(err, check.IsNil) - - err = tester.UpdateKeys(map[string][]byte{ - kv.GetEtcdKeyTaskStatus("cf-1", captureID): newBytes, + return workload, true, nil }) - c.Assert(err, check.IsNil) + s.tester.MustApplyPatches() } -func readTaskStatus(c *check.C, tester *orchestrator.ReactorStateTester, captureID model.CaptureID) *model.TaskStatus { - var taskStatus model.TaskStatus - err := json.Unmarshal([]byte(tester.KVEntries()[kv.GetEtcdKeyTaskStatus("cf-1", captureID)]), &taskStatus) - c.Assert(err, check.IsNil) - return &taskStatus -} - -func (s *schedulerTestSuite) TestPutTaskAddRemove(c *check.C) { +func (s *schedulerSuite) TestScheduleOneCapture(c *check.C) { defer testleak.AfterTest(c)() - scheduler, tester := setUp(c) - addCapture(c, tester, "capture-1") - - tasks := map[model.TableID]*tableTask{ - 1: { - TableID: 1, - CheckpointTs: 1000, - ResolvedTs: 1200, - }, - } + s.reset(c) + captureID := "test-capture-1" + s.addCapture(captureID) - scheduler.PutTasks(tasks) - err := tester.ApplyPatches() + // add three tables + shouldUpdateState, err := s.scheduler.Tick(s.state, []model.TableID{1, 2, 3, 4}, s.captures) c.Assert(err, check.IsNil) - - taskStatus := readTaskStatus(c, tester, "capture-1") - c.Assert(taskStatus.Operation, check.HasKey, model.TableID(1)) - c.Assert(taskStatus.Operation[1], check.DeepEquals, &model.TableOperation{ - Delete: false, - BoundaryTs: 1001, - Done: false, - Status: model.OperDispatched, + 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() - mockProcessorTick(c, tester, "capture-1") - - tasks[2] = &tableTask{ - TableID: 2, - CheckpointTs: 1100, - ResolvedTs: 1000, - } + // two tables finish adding operation + s.finishTableOperation(captureID, 2, 3) - scheduler.PutTasks(tasks) - err = tester.ApplyPatches() + // 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}, + }) - // delay the mocked processor for one tick to test idempotency - scheduler.PutTasks(tasks) - err = tester.ApplyPatches() + // 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) - - taskStatus = readTaskStatus(c, tester, "capture-1") - c.Assert(taskStatus.Operation, check.HasKey, model.TableID(2)) - c.Assert(taskStatus.Operation[2], check.DeepEquals, &model.TableOperation{ - Delete: false, - BoundaryTs: 1101, - Done: false, - Status: model.OperDispatched, + 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}, }) - mockProcessorTick(c, tester, "capture-1") + // finish all operations + s.finishTableOperation(captureID, 1, 2, 3, 4, 5) - delete(tasks, 1) - scheduler.PutTasks(tasks) - err = tester.ApplyPatches() + shouldUpdateState, err = s.scheduler.Tick(s.state, []model.TableID{3, 4, 5}, s.captures) c.Assert(err, check.IsNil) - - taskStatus = readTaskStatus(c, tester, "capture-1") - c.Assert(taskStatus.Operation, check.HasKey, model.TableID(1)) - c.Assert(taskStatus.Operation[1], check.DeepEquals, &model.TableOperation{ - Delete: true, - BoundaryTs: 0, // we don't have a global resolved Ts in the mocked environment - Done: false, - Status: model.OperDispatched, + 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{}) - mockProcessorTick(c, tester, "capture-1") - - delete(tasks, 2) - scheduler.PutTasks(tasks) - err = tester.ApplyPatches() + // 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) - - taskStatus = readTaskStatus(c, tester, "capture-1") - c.Assert(taskStatus.Operation, check.HasKey, model.TableID(2)) - c.Assert(taskStatus.Operation[2], check.DeepEquals, &model.TableOperation{ - Delete: true, - BoundaryTs: 0, // we don't have a global resolved Ts in the mocked environment - Done: false, - Status: model.OperDispatched, + 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{}) - // delay the mocked processor for one tick to test idempotency - scheduler.PutTasks(tasks) - err = tester.ApplyPatches() + shouldUpdateState, err = s.scheduler.Tick(s.state, []model.TableID{3, 4, 5}, s.captures) c.Assert(err, check.IsNil) - - mockProcessorTick(c, tester, "capture-1") - taskStatus = readTaskStatus(c, tester, "capture-1") - c.Assert(taskStatus.Tables, check.HasLen, 0) + 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 *schedulerTestSuite) TestPutTaskRebalance(c *check.C) { +func (s *schedulerSuite) TestScheduleMoveTable(c *check.C) { defer testleak.AfterTest(c)() - scheduler, tester := setUp(c) - addCapture(c, tester, "capture-1") - - // Add two tables - tasks := map[model.TableID]*tableTask{ - 1: { - TableID: 1, - CheckpointTs: 1000, - ResolvedTs: 1200, - }, - 2: { - TableID: 2, - CheckpointTs: 1100, - ResolvedTs: 1210, - }, - } - scheduler.PutTasks(tasks) - err := tester.ApplyPatches() - c.Assert(err, check.IsNil) + s.reset(c) + captureID1 := "test-capture-1" + captureID2 := "test-capture-2" + s.addCapture(captureID1) - taskStatus := readTaskStatus(c, tester, "capture-1") - c.Assert(taskStatus.Operation, check.HasKey, model.TableID(1)) - c.Assert(taskStatus.Operation[1], check.DeepEquals, &model.TableOperation{ - Delete: false, - BoundaryTs: 1001, - Done: false, - Status: model.OperDispatched, + // 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(taskStatus.Operation, check.HasKey, model.TableID(2)) - c.Assert(taskStatus.Operation[2], check.DeepEquals, &model.TableOperation{ - Delete: false, - BoundaryTs: 1101, - Done: false, - Status: model.OperDispatched, + c.Assert(s.state.TaskStatuses[captureID1].Operation, check.DeepEquals, map[model.TableID]*model.TableOperation{ + 1: {Done: false, Delete: false, BoundaryTs: 0, Status: model.OperDispatched}, }) - mockProcessorTick(c, tester, "capture-1") - addCapture(c, tester, "capture-2") - scheduler.PutTasks(tasks) - // rebalance should have been triggered - err = tester.ApplyPatches() - c.Assert(err, check.IsNil) - - taskStatus1 := readTaskStatus(c, tester, "capture-1") - c.Assert(taskStatus1.Operation, check.HasLen, 1) - var victimID model.TableID - for tableID := range taskStatus1.Operation { - // record the victim ID since it is generated randomly - victimID = tableID - } - c.Assert(taskStatus1.Operation[victimID].Delete, check.IsTrue) - c.Assert(taskStatus1.Operation[victimID].Status, check.Equals, model.OperDispatched) - - mockProcessorTick(c, tester, "capture-1") - // capture-1 has processed the delete request - taskStatus1 = readTaskStatus(c, tester, "capture-1") - c.Assert(taskStatus1.Tables, check.HasLen, 1) - c.Assert(taskStatus1.Operation[victimID].Status, check.Equals, model.OperFinished) - - tasks[victimID].CheckpointTs = 1500 - // scheduler will redispatch the victim here - scheduler.PutTasks(tasks) - err = tester.ApplyPatches() + 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() - taskStatus1 = readTaskStatus(c, tester, "capture-1") - c.Assert(taskStatus1.Tables, check.HasLen, 1) - c.Assert(taskStatus1.Operation, check.HasLen, 0) + s.addCapture(captureID2) - taskStatus2 := readTaskStatus(c, tester, "capture-2") - c.Assert(taskStatus2.Tables, check.HasLen, 1) - c.Assert(taskStatus2.Operation, check.HasLen, 1) - c.Assert(taskStatus2.Operation[victimID].BoundaryTs, check.Equals, uint64(1501)) -} - -func (s *schedulerTestSuite) TestPutTaskAddAfterDelete(c *check.C) { - defer testleak.AfterTest(c)() - scheduler, tester := setUp(c) - addCapture(c, tester, "capture-1") - addCapture(c, tester, "capture-2") - - // Add two tables - tasks := map[model.TableID]*tableTask{ - 1: { - TableID: 1, - CheckpointTs: 1000, - ResolvedTs: 1200, - }, - 2: { - TableID: 2, - CheckpointTs: 1100, - ResolvedTs: 1210, - }, - } - scheduler.PutTasks(tasks) - err := tester.ApplyPatches() + // 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}, + }) - mockProcessorTick(c, tester, "capture-1") - mockProcessorTick(c, tester, "capture-2") + s.finishTableOperation(captureID2, 2) - // wait for one tick for the state to stabilize - scheduler.PutTasks(tasks) - err = tester.ApplyPatches() + 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}, + }) - taskStatus1 := readTaskStatus(c, tester, "capture-1") - c.Assert(taskStatus1.Tables, check.HasLen, 1) - c.Assert(taskStatus1.Operation, check.HasLen, 0) - - taskStatus2 := readTaskStatus(c, tester, "capture-2") - c.Assert(taskStatus2.Tables, check.HasLen, 1) - c.Assert(taskStatus2.Operation, check.HasLen, 0) - - // delete the two tables - scheduler.PutTasks(map[model.TableID]*tableTask{}) - err = tester.ApplyPatches() - c.Assert(err, check.IsNil) + s.finishTableOperation(captureID2, 2) - // add back the two tables - scheduler.PutTasks(tasks) - err = tester.ApplyPatches() + 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{}) - mockProcessorTick(c, tester, "capture-1") - mockProcessorTick(c, tester, "capture-2") - - // wait for one tick for the state to stabilize - scheduler.PutTasks(tasks) - err = tester.ApplyPatches() + shouldUpdateState, err = s.scheduler.Tick(s.state, []model.TableID{1, 2}, s.captures) c.Assert(err, check.IsNil) - - mockProcessorTick(c, tester, "capture-1") - mockProcessorTick(c, tester, "capture-2") - - taskStatus1 = readTaskStatus(c, tester, "capture-1") - c.Assert(taskStatus1.Tables, check.HasLen, 1) - c.Assert(taskStatus1.Operation, check.HasLen, 1) - - taskStatus2 = readTaskStatus(c, tester, "capture-2") - c.Assert(taskStatus2.Tables, check.HasLen, 1) - c.Assert(taskStatus2.Operation, check.HasLen, 1) + 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 *schedulerTestSuite) TestPutTaskWithAffinity(c *check.C) { +func (s *schedulerSuite) TestScheduleRebalance(c *check.C) { defer testleak.AfterTest(c)() - scheduler, tester := setUp(c) - addCapture(c, tester, "capture-1") - addCapture(c, tester, "capture-2") - - // Add two tables - tasks := map[model.TableID]*tableTask{ - 1: { - TableID: 1, - CheckpointTs: 1000, - ResolvedTs: 1200, - }, - 2: { - TableID: 2, - CheckpointTs: 1100, - ResolvedTs: 1210, - }, - } - - scheduler.SetAffinity(1, "capture-1", 10) - scheduler.SetAffinity(2, "capture-1", 10) + 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() - scheduler.PutTasks(tasks) - err := tester.ApplyPatches() + // 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() - mockProcessorTick(c, tester, "capture-1") - mockProcessorTick(c, tester, "capture-2") - - taskStatus1 := readTaskStatus(c, tester, "capture-1") - c.Assert(taskStatus1.Tables, check.HasLen, 2) - c.Assert(taskStatus1.Operation, check.HasLen, 2) - - taskStatus2 := readTaskStatus(c, tester, "capture-2") - c.Assert(taskStatus2.Tables, check.HasLen, 0) - c.Assert(taskStatus2.Operation, check.HasLen, 0) - - removeCapture(c, tester, "capture-1") - c.Assert(scheduler.IsReady(), check.IsFalse) - err = tester.ApplyPatches() + // 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(scheduler.IsReady(), check.IsTrue) + 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) - scheduler.PutTasks(tasks) - err = tester.ApplyPatches() + // rebalance table + shouldUpdateState, err = s.scheduler.Tick(s.state, []model.TableID{1, 2, 3, 4, 5, 6}, s.captures) c.Assert(err, check.IsNil) - - mockProcessorTick(c, tester, "capture-2") - taskStatus2 = readTaskStatus(c, tester, "capture-2") - c.Assert(taskStatus2.Tables, check.HasLen, 2) - c.Assert(taskStatus2.Operation, check.HasLen, 2) + 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: {}}) } - -// TODO add test cases for 1) affinity expiring 2) adding table when there is no capture. 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..ae9defcc87e --- /dev/null +++ b/cdc/owner/schema_test.go @@ -0,0 +1,173 @@ +// 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" + "github.com/pingcap/tidb/store/tikv/oracle" +) + +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(oracle.GlobalTxnScope) + 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(oracle.GlobalTxnScope) + 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(oracle.GlobalTxnScope) + 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(oracle.GlobalTxnScope) + 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 a674298f787..a697fa9755b 100644 --- a/cdc/owner_test.go +++ b/cdc/owner_test.go @@ -905,8 +905,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) @@ -1207,8 +1206,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) @@ -1265,8 +1263,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) @@ -1352,8 +1349,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/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 1bfafc47eab..0885f09512e 100644 --- a/cdc/processor/pipeline/sorter.go +++ b/cdc/processor/pipeline/sorter.go @@ -36,13 +36,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 @@ -51,36 +48,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") } @@ -89,18 +78,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) } failpoint.Inject("ProcessorAddTableError", func() { failpoint.Return(errors.New("processor add table injected error")) @@ -122,7 +112,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() @@ -201,7 +191,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) } @@ -209,7 +199,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 8c87203aa6a..375268bb206 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,17 +164,17 @@ 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 645ea7dbebc..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 nil, false, errors.Trace(err) } - return status, nil + return status, true, 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 - } - status.Operation = nil - return status, 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,14 +441,17 @@ func (p *processor) handleTableOperation(ctx context.Context) error { return nil } -func (p *processor) createAndDriveSchemaStorage(ctx context.Context) (entry.SchemaStorage, error) { - kvStorage, err := util.KVStorageFromCtx(ctx) - if err != nil { - return nil, errors.Trace(err) - } +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) @@ -474,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: } @@ -515,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 { @@ -540,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 { @@ -586,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 @@ -606,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 } @@ -626,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 } } @@ -642,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)) @@ -655,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 @@ -670,36 +682,53 @@ func (p *processor) createTablePipelineImpl(ctx context.Context, tableID model.T p.sendError(err) return nil }) - - kvStorage, err := util.KVStorageFromCtx(ctx) - if err != nil { - return nil, errors.Trace(err) - } - 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(), @@ -711,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)) @@ -727,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) @@ -736,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/server.go b/cdc/server.go index eb43e28a21d..f717508917b 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" @@ -46,11 +51,15 @@ const ( // 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 } @@ -95,7 +104,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 @@ -119,7 +160,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) { @@ -236,26 +282,35 @@ 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: conf.CaptureSessionTTL, - } - capture, err := NewCapture(ctx, s.pdEndpoints, s.pdClient, conf.Security, conf.AdvertiseAddr, opts) - if err != nil { - return err + if !config.NewReplicaImpl { + kvStorage, err := util.KVStorageFromCtx(ctx) + if err != nil { + return errors.Trace(err) + } + capture, err := NewCapture(ctx, s.pdEndpoints, s.pdClient, kvStorage) + if err != nil { + return err + } + s.capture = capture + s.etcdClient = &capture.etcdClient } - 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 s.capture.Run(cctx) + }) + } wg.Go(func() error { return s.etcdHealthChecker(cctx) }) @@ -268,10 +323,6 @@ func (s *Server) run(ctx context.Context) (err error) { return kv.RunWorkerPool(cctx) }) - wg.Go(func() error { - return s.capture.Run(cctx) - }) - return wg.Wait() } @@ -288,6 +339,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/cmd/client_changefeed.go b/cmd/client_changefeed.go index 8899fa802a0..d9a8c692d2b 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/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 7d744e426a3..0a6531ceb02 100755 --- a/errors.toml +++ b/errors.toml @@ -261,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 @@ -401,6 +406,11 @@ error = ''' kafka send message failed ''' +["CDC:ErrLeaseExpired"] +error = ''' +owner lease expired +''' + ["CDC:ErrLeaseTimeout"] error = ''' owner lease timeout @@ -496,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 @@ -511,6 +526,11 @@ error = ''' owner campaign key deleted ''' +["CDC:ErrOwnerChangedUnexpectedly"] +error = ''' +owner changed unexpectedly +''' + ["CDC:ErrOwnerChangefeedNotFound"] error = ''' changefeed %s not found in owner cache @@ -711,6 +731,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/pkg/config/config.go b/pkg/config/config.go index 7149e901b61..e05c4fccf1c 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: true, diff --git a/pkg/context/context.go b/pkg/context/context.go index f3b24cdd4bf..a968392f308 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.GoTimeToTS(time.Now()), + 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 7914381db23..1d6db2ef21a 100644 --- a/pkg/errors/errors.go +++ b/pkg/errors/errors.go @@ -182,6 +182,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")) @@ -192,6 +195,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")) @@ -212,6 +216,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 f0c03556308..2d50f52602a 100644 --- a/pkg/errors/helper.go +++ b/pkg/errors/helper.go @@ -28,6 +28,23 @@ 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 { 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/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/ctx.go b/pkg/util/ctx.go index dcde327206e..492b00f8b4e 100644 --- a/pkg/util/ctx.go +++ b/pkg/util/ctx.go @@ -18,7 +18,6 @@ import ( "time" "github.com/pingcap/errors" - "github.com/pingcap/tidb/kv" "go.uber.org/zap" ) 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/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.*']