github.com/pingcap/ticdc@v0.0.0-20220526033649-485a10ef2652/cdc/processor/manager_test.go (about) 1 // Copyright 2021 PingCAP, Inc. 2 // 3 // Licensed under the Apache License, Version 2.0 (the "License"); 4 // you may not use this file except in compliance with the License. 5 // You may obtain a copy of the License at 6 // 7 // http://www.apache.org/licenses/LICENSE-2.0 8 // 9 // Unless required by applicable law or agreed to in writing, software 10 // distributed under the License is distributed on an "AS IS" BASIS, 11 // See the License for the specific language governing permissions and 12 // limitations under the License. 13 14 package processor 15 16 import ( 17 "bytes" 18 "fmt" 19 "math" 20 "time" 21 22 "github.com/pingcap/check" 23 "github.com/pingcap/errors" 24 "github.com/pingcap/ticdc/cdc/model" 25 tablepipeline "github.com/pingcap/ticdc/cdc/processor/pipeline" 26 "github.com/pingcap/ticdc/pkg/config" 27 cdcContext "github.com/pingcap/ticdc/pkg/context" 28 cerrors "github.com/pingcap/ticdc/pkg/errors" 29 "github.com/pingcap/ticdc/pkg/orchestrator" 30 "github.com/pingcap/ticdc/pkg/util/testleak" 31 ) 32 33 type managerSuite struct { 34 manager *Manager 35 state *model.GlobalReactorState 36 tester *orchestrator.ReactorStateTester 37 } 38 39 var _ = check.Suite(&managerSuite{}) 40 41 func (s *managerSuite) resetSuit(ctx cdcContext.Context, c *check.C) { 42 s.manager = NewManager4Test(func(ctx cdcContext.Context, tableID model.TableID, replicaInfo *model.TableReplicaInfo) (tablepipeline.TablePipeline, error) { 43 return &mockTablePipeline{ 44 tableID: tableID, 45 name: fmt.Sprintf("`test`.`table%d`", tableID), 46 status: tablepipeline.TableStatusRunning, 47 resolvedTs: replicaInfo.StartTs, 48 checkpointTs: replicaInfo.StartTs, 49 }, nil 50 }) 51 s.state = model.NewGlobalState().(*model.GlobalReactorState) 52 captureInfoBytes, err := ctx.GlobalVars().CaptureInfo.Marshal() 53 c.Assert(err, check.IsNil) 54 s.tester = orchestrator.NewReactorStateTester(c, s.state, map[string]string{ 55 fmt.Sprintf("/tidb/cdc/capture/%s", ctx.GlobalVars().CaptureInfo.ID): string(captureInfoBytes), 56 }) 57 } 58 59 func (s *managerSuite) TestChangefeed(c *check.C) { 60 defer testleak.AfterTest(c)() 61 ctx := cdcContext.NewBackendContext4Test(false) 62 s.resetSuit(ctx, c) 63 var err error 64 65 // no changefeed 66 _, err = s.manager.Tick(ctx, s.state) 67 c.Assert(err, check.IsNil) 68 69 // an inactive changefeed 70 s.state.Changefeeds["test-changefeed"] = model.NewChangefeedReactorState("test-changefeed") 71 _, err = s.manager.Tick(ctx, s.state) 72 s.tester.MustApplyPatches() 73 c.Assert(err, check.IsNil) 74 c.Assert(s.manager.processors, check.HasLen, 0) 75 76 // an active changefeed 77 s.state.Changefeeds["test-changefeed"].PatchInfo(func(info *model.ChangeFeedInfo) (*model.ChangeFeedInfo, bool, error) { 78 return &model.ChangeFeedInfo{ 79 SinkURI: "blackhole://", 80 CreateTime: time.Now(), 81 StartTs: 0, 82 TargetTs: math.MaxUint64, 83 Config: config.GetDefaultReplicaConfig(), 84 }, true, nil 85 }) 86 s.state.Changefeeds["test-changefeed"].PatchStatus(func(status *model.ChangeFeedStatus) (*model.ChangeFeedStatus, bool, error) { 87 return &model.ChangeFeedStatus{}, true, nil 88 }) 89 s.state.Changefeeds["test-changefeed"].PatchTaskStatus(ctx.GlobalVars().CaptureInfo.ID, func(status *model.TaskStatus) (*model.TaskStatus, bool, error) { 90 return &model.TaskStatus{ 91 Tables: map[int64]*model.TableReplicaInfo{1: {}}, 92 }, true, nil 93 }) 94 s.tester.MustApplyPatches() 95 _, err = s.manager.Tick(ctx, s.state) 96 s.tester.MustApplyPatches() 97 c.Assert(err, check.IsNil) 98 c.Assert(s.manager.processors, check.HasLen, 1) 99 100 // processor return errors 101 s.state.Changefeeds["test-changefeed"].PatchStatus(func(status *model.ChangeFeedStatus) (*model.ChangeFeedStatus, bool, error) { 102 status.AdminJobType = model.AdminStop 103 return status, true, nil 104 }) 105 s.state.Changefeeds["test-changefeed"].PatchTaskStatus(ctx.GlobalVars().CaptureInfo.ID, func(status *model.TaskStatus) (*model.TaskStatus, bool, error) { 106 status.AdminJobType = model.AdminStop 107 return status, true, nil 108 }) 109 s.tester.MustApplyPatches() 110 _, err = s.manager.Tick(ctx, s.state) 111 s.tester.MustApplyPatches() 112 c.Assert(err, check.IsNil) 113 c.Assert(s.manager.processors, check.HasLen, 0) 114 } 115 116 func (s *managerSuite) TestDebugInfo(c *check.C) { 117 defer testleak.AfterTest(c)() 118 ctx := cdcContext.NewBackendContext4Test(false) 119 s.resetSuit(ctx, c) 120 var err error 121 122 // no changefeed 123 _, err = s.manager.Tick(ctx, s.state) 124 c.Assert(err, check.IsNil) 125 126 // an active changefeed 127 s.state.Changefeeds["test-changefeed"] = model.NewChangefeedReactorState("test-changefeed") 128 s.state.Changefeeds["test-changefeed"].PatchInfo(func(info *model.ChangeFeedInfo) (*model.ChangeFeedInfo, bool, error) { 129 return &model.ChangeFeedInfo{ 130 SinkURI: "blackhole://", 131 CreateTime: time.Now(), 132 StartTs: 0, 133 TargetTs: math.MaxUint64, 134 Config: config.GetDefaultReplicaConfig(), 135 }, true, nil 136 }) 137 s.state.Changefeeds["test-changefeed"].PatchStatus(func(status *model.ChangeFeedStatus) (*model.ChangeFeedStatus, bool, error) { 138 return &model.ChangeFeedStatus{}, true, nil 139 }) 140 s.state.Changefeeds["test-changefeed"].PatchTaskStatus(ctx.GlobalVars().CaptureInfo.ID, func(status *model.TaskStatus) (*model.TaskStatus, bool, error) { 141 return &model.TaskStatus{ 142 Tables: map[int64]*model.TableReplicaInfo{1: {}}, 143 }, true, nil 144 }) 145 s.tester.MustApplyPatches() 146 _, err = s.manager.Tick(ctx, s.state) 147 c.Assert(err, check.IsNil) 148 s.tester.MustApplyPatches() 149 c.Assert(s.manager.processors, check.HasLen, 1) 150 done := make(chan struct{}) 151 go func() { 152 defer close(done) 153 for { 154 _, err = s.manager.Tick(ctx, s.state) 155 if err != nil { 156 c.Assert(cerrors.ErrReactorFinished.Equal(errors.Cause(err)), check.IsTrue) 157 return 158 } 159 c.Assert(err, check.IsNil) 160 s.tester.MustApplyPatches() 161 } 162 }() 163 buf := bytes.NewBufferString("") 164 s.manager.WriteDebugInfo(buf) 165 c.Assert(len(buf.String()), check.Greater, 0) 166 s.manager.AsyncClose() 167 <-done 168 } 169 170 func (s *managerSuite) TestClose(c *check.C) { 171 defer testleak.AfterTest(c)() 172 ctx := cdcContext.NewBackendContext4Test(false) 173 s.resetSuit(ctx, c) 174 var err error 175 176 // no changefeed 177 _, err = s.manager.Tick(ctx, s.state) 178 c.Assert(err, check.IsNil) 179 180 // an active changefeed 181 s.state.Changefeeds["test-changefeed"] = model.NewChangefeedReactorState("test-changefeed") 182 s.state.Changefeeds["test-changefeed"].PatchInfo(func(info *model.ChangeFeedInfo) (*model.ChangeFeedInfo, bool, error) { 183 return &model.ChangeFeedInfo{ 184 SinkURI: "blackhole://", 185 CreateTime: time.Now(), 186 StartTs: 0, 187 TargetTs: math.MaxUint64, 188 Config: config.GetDefaultReplicaConfig(), 189 }, true, nil 190 }) 191 s.state.Changefeeds["test-changefeed"].PatchStatus(func(status *model.ChangeFeedStatus) (*model.ChangeFeedStatus, bool, error) { 192 return &model.ChangeFeedStatus{}, true, nil 193 }) 194 s.state.Changefeeds["test-changefeed"].PatchTaskStatus(ctx.GlobalVars().CaptureInfo.ID, func(status *model.TaskStatus) (*model.TaskStatus, bool, error) { 195 return &model.TaskStatus{ 196 Tables: map[int64]*model.TableReplicaInfo{1: {}}, 197 }, true, nil 198 }) 199 s.tester.MustApplyPatches() 200 _, err = s.manager.Tick(ctx, s.state) 201 c.Assert(err, check.IsNil) 202 s.tester.MustApplyPatches() 203 c.Assert(s.manager.processors, check.HasLen, 1) 204 205 s.manager.AsyncClose() 206 _, err = s.manager.Tick(ctx, s.state) 207 c.Assert(cerrors.ErrReactorFinished.Equal(errors.Cause(err)), check.IsTrue) 208 s.tester.MustApplyPatches() 209 c.Assert(s.manager.processors, check.HasLen, 0) 210 }