github.com/pingcap/ticdc@v0.0.0-20220526033649-485a10ef2652/cdc/owner/ddl_puller_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 owner 15 16 import ( 17 "context" 18 "encoding/json" 19 "sync" 20 "sync/atomic" 21 22 "github.com/pingcap/check" 23 "github.com/pingcap/errors" 24 timodel "github.com/pingcap/parser/model" 25 "github.com/pingcap/ticdc/cdc/model" 26 cdcContext "github.com/pingcap/ticdc/pkg/context" 27 "github.com/pingcap/ticdc/pkg/retry" 28 "github.com/pingcap/ticdc/pkg/util/testleak" 29 "github.com/pingcap/tidb/util/codec" 30 ) 31 32 var _ = check.Suite(&ddlPullerSuite{}) 33 34 type ddlPullerSuite struct { 35 } 36 37 type mockPuller struct { 38 c *check.C 39 inCh chan *model.RawKVEntry 40 outCh chan *model.RawKVEntry 41 resolvedTs model.Ts 42 } 43 44 func newMockPuller(c *check.C, startTs model.Ts) *mockPuller { 45 return &mockPuller{ 46 c: c, 47 inCh: make(chan *model.RawKVEntry), 48 outCh: make(chan *model.RawKVEntry), 49 resolvedTs: startTs - 1, 50 } 51 } 52 53 func (m *mockPuller) Run(ctx context.Context) error { 54 for { 55 select { 56 case <-ctx.Done(): 57 return ctx.Err() 58 case e := <-m.inCh: 59 m.outCh <- e 60 atomic.StoreUint64(&m.resolvedTs, e.CRTs) 61 } 62 } 63 } 64 65 func (m *mockPuller) GetResolvedTs() uint64 { 66 return atomic.LoadUint64(&m.resolvedTs) 67 } 68 69 func (m *mockPuller) Output() <-chan *model.RawKVEntry { 70 return m.outCh 71 } 72 73 func (m *mockPuller) IsInitialized() bool { 74 return true 75 } 76 77 func (m *mockPuller) append(e *model.RawKVEntry) { 78 m.inCh <- e 79 } 80 81 func (m *mockPuller) appendDDL(job *timodel.Job) { 82 b, err := json.Marshal(job) 83 m.c.Assert(err, check.IsNil) 84 ek := []byte("m") 85 ek = codec.EncodeBytes(ek, []byte("DDLJobList")) 86 ek = codec.EncodeUint(ek, uint64('l')) 87 ek = codec.EncodeInt(ek, 1) 88 m.append(&model.RawKVEntry{ 89 OpType: model.OpTypePut, 90 Key: ek, 91 Value: b, 92 StartTs: job.StartTS, 93 CRTs: job.BinlogInfo.FinishedTS, 94 }) 95 } 96 97 func (m *mockPuller) appendResolvedTs(ts model.Ts) { 98 m.append(&model.RawKVEntry{ 99 OpType: model.OpTypeResolved, 100 CRTs: ts, 101 StartTs: ts, 102 }) 103 } 104 105 func (s *ddlPullerSuite) TestPuller(c *check.C) { 106 defer testleak.AfterTest(c)() 107 startTs := uint64(10) 108 mockPuller := newMockPuller(c, startTs) 109 ctx := cdcContext.NewBackendContext4Test(true) 110 p, err := newDDLPuller(ctx, startTs) 111 c.Assert(err, check.IsNil) 112 p.(*ddlPullerImpl).puller = mockPuller 113 var wg sync.WaitGroup 114 wg.Add(1) 115 go func() { 116 defer wg.Done() 117 err := p.Run(ctx) 118 if errors.Cause(err) == context.Canceled { 119 err = nil 120 } 121 c.Assert(err, check.IsNil) 122 }() 123 defer wg.Wait() 124 defer p.Close() 125 126 // test initialize state 127 resolvedTs, ddl := p.FrontDDL() 128 c.Assert(resolvedTs, check.Equals, startTs) 129 c.Assert(ddl, check.IsNil) 130 resolvedTs, ddl = p.PopFrontDDL() 131 c.Assert(resolvedTs, check.Equals, startTs) 132 c.Assert(ddl, check.IsNil) 133 134 // test send resolvedTs 135 mockPuller.appendResolvedTs(15) 136 waitResolvedTsGrowing(c, p, 15) 137 138 // test send ddl job out of order 139 mockPuller.appendDDL(&timodel.Job{ 140 ID: 2, 141 Type: timodel.ActionCreateTable, 142 StartTS: 5, 143 State: timodel.JobStateDone, 144 BinlogInfo: &timodel.HistoryInfo{FinishedTS: 18}, 145 }) 146 mockPuller.appendDDL(&timodel.Job{ 147 ID: 1, 148 Type: timodel.ActionCreateTable, 149 StartTS: 5, 150 State: timodel.JobStateDone, 151 BinlogInfo: &timodel.HistoryInfo{FinishedTS: 16}, 152 }) 153 resolvedTs, ddl = p.FrontDDL() 154 c.Assert(resolvedTs, check.Equals, uint64(15)) 155 c.Assert(ddl, check.IsNil) 156 157 mockPuller.appendResolvedTs(20) 158 waitResolvedTsGrowing(c, p, 16) 159 resolvedTs, ddl = p.FrontDDL() 160 c.Assert(resolvedTs, check.Equals, uint64(16)) 161 c.Assert(ddl.ID, check.Equals, int64(1)) 162 resolvedTs, ddl = p.PopFrontDDL() 163 c.Assert(resolvedTs, check.Equals, uint64(16)) 164 c.Assert(ddl.ID, check.Equals, int64(1)) 165 166 // DDL could be processed with a delay, wait here for a pending DDL job is added 167 waitResolvedTsGrowing(c, p, 18) 168 resolvedTs, ddl = p.PopFrontDDL() 169 c.Assert(resolvedTs, check.Equals, uint64(18)) 170 c.Assert(ddl.ID, check.Equals, int64(2)) 171 172 // test add ddl job repeated 173 mockPuller.appendDDL(&timodel.Job{ 174 ID: 3, 175 Type: timodel.ActionCreateTable, 176 StartTS: 20, 177 State: timodel.JobStateDone, 178 BinlogInfo: &timodel.HistoryInfo{FinishedTS: 25}, 179 }) 180 mockPuller.appendDDL(&timodel.Job{ 181 ID: 3, 182 Type: timodel.ActionCreateTable, 183 StartTS: 20, 184 State: timodel.JobStateDone, 185 BinlogInfo: &timodel.HistoryInfo{FinishedTS: 25}, 186 }) 187 mockPuller.appendResolvedTs(30) 188 waitResolvedTsGrowing(c, p, 25) 189 190 resolvedTs, ddl = p.PopFrontDDL() 191 c.Assert(resolvedTs, check.Equals, uint64(25)) 192 c.Assert(ddl.ID, check.Equals, int64(3)) 193 _, ddl = p.PopFrontDDL() 194 c.Assert(ddl, check.IsNil) 195 196 waitResolvedTsGrowing(c, p, 30) 197 resolvedTs, ddl = p.PopFrontDDL() 198 c.Assert(resolvedTs, check.Equals, uint64(30)) 199 c.Assert(ddl, check.IsNil) 200 201 // test add invalid ddl job 202 mockPuller.appendDDL(&timodel.Job{ 203 ID: 4, 204 Type: timodel.ActionLockTable, 205 StartTS: 20, 206 State: timodel.JobStateDone, 207 BinlogInfo: &timodel.HistoryInfo{FinishedTS: 35}, 208 }) 209 mockPuller.appendDDL(&timodel.Job{ 210 ID: 5, 211 Type: timodel.ActionCreateTable, 212 StartTS: 20, 213 State: timodel.JobStateCancelled, 214 BinlogInfo: &timodel.HistoryInfo{FinishedTS: 36}, 215 }) 216 mockPuller.appendResolvedTs(40) 217 waitResolvedTsGrowing(c, p, 40) 218 resolvedTs, ddl = p.PopFrontDDL() 219 // no ddl should be received 220 c.Assert(resolvedTs, check.Equals, uint64(40)) 221 c.Assert(ddl, check.IsNil) 222 } 223 224 // waitResolvedTsGrowing can wait the first DDL reaches targetTs or if no pending 225 // DDL, DDL resolved ts reaches targetTs. 226 func waitResolvedTsGrowing(c *check.C, p DDLPuller, targetTs model.Ts) { 227 err := retry.Do(context.Background(), func() error { 228 resolvedTs, _ := p.FrontDDL() 229 if resolvedTs < targetTs { 230 return errors.New("resolvedTs < targetTs") 231 } 232 return nil 233 }, retry.WithBackoffBaseDelay(20), retry.WithMaxTries(100)) 234 c.Assert(err, check.IsNil) 235 }