github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/dm/syncer/causality_test.go (about) 1 // Copyright 2019 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 syncer 15 16 import ( 17 "math" 18 "testing" 19 "time" 20 21 "github.com/go-mysql-org/go-mysql/mysql" 22 . "github.com/pingcap/check" 23 cdcmodel "github.com/pingcap/tiflow/cdc/model" 24 "github.com/pingcap/tiflow/dm/config" 25 "github.com/pingcap/tiflow/dm/pkg/binlog" 26 tcontext "github.com/pingcap/tiflow/dm/pkg/context" 27 "github.com/pingcap/tiflow/dm/pkg/log" 28 "github.com/pingcap/tiflow/dm/pkg/utils" 29 "github.com/pingcap/tiflow/dm/syncer/metrics" 30 "github.com/pingcap/tiflow/pkg/sqlmodel" 31 "github.com/stretchr/testify/require" 32 ) 33 34 func (s *testSyncerSuite) TestDetectConflict(c *C) { 35 ca := &causality{ 36 relation: newCausalityRelation(), 37 } 38 caseData := []string{"test_1", "test_2", "test_3"} 39 excepted := map[string]string{ 40 "test_1": "test_1", 41 "test_2": "test_1", 42 "test_3": "test_1", 43 } 44 45 assertRelationsEq := func(expectMap map[string]string) { 46 c.Assert(ca.relation.len(), Equals, len(expectMap)) 47 for k, expV := range expectMap { 48 v, ok := ca.relation.get(k) 49 c.Assert(ok, IsTrue) 50 c.Assert(v, Equals, expV) 51 } 52 } 53 54 c.Assert(ca.detectConflict(caseData), IsFalse) 55 ca.add(caseData) 56 assertRelationsEq(excepted) 57 c.Assert(ca.detectConflict([]string{"test_4"}), IsFalse) 58 ca.add([]string{"test_4"}) 59 excepted["test_4"] = "test_4" 60 assertRelationsEq(excepted) 61 conflictData := []string{"test_4", "test_3"} 62 c.Assert(ca.detectConflict(conflictData), IsTrue) 63 ca.relation.clear() 64 c.Assert(ca.relation.len(), Equals, 0) 65 } 66 67 func TestCausality(t *testing.T) { 68 t.Parallel() 69 70 schemaStr := "create table tb(a int primary key, b int unique);" 71 ti := mockTableInfo(t, schemaStr) 72 73 jobCh := make(chan *job, 10) 74 syncer := &Syncer{ 75 cfg: &config.SubTaskConfig{ 76 SyncerConfig: config.SyncerConfig{ 77 QueueSize: 1024, 78 }, 79 Name: "task", 80 SourceID: "source", 81 }, 82 tctx: tcontext.Background().WithLogger(log.L()), 83 sessCtx: utils.NewSessionCtx(map[string]string{"time_zone": "UTC"}), 84 metricsProxies: &metrics.Proxies{}, 85 } 86 syncer.metricsProxies = metrics.DefaultMetricsProxies.CacheForOneTask("task", "worker", "source") 87 causalityCh := causalityWrap(jobCh, syncer) 88 testCases := []struct { 89 preVals []interface{} 90 postVals []interface{} 91 }{ 92 { 93 postVals: []interface{}{1, 2}, 94 }, 95 { 96 postVals: []interface{}{2, 3}, 97 }, 98 { 99 preVals: []interface{}{2, 3}, 100 postVals: []interface{}{3, 4}, 101 }, 102 { 103 preVals: []interface{}{1, 2}, 104 }, 105 { 106 postVals: []interface{}{1, 3}, 107 }, 108 } 109 results := []opType{dml, dml, dml, dml, conflict, dml} 110 table := &cdcmodel.TableName{Schema: "test", Table: "t1"} 111 location := binlog.MustZeroLocation(mysql.MySQLFlavor) 112 ec := &eventContext{startLocation: location, endLocation: location, lastLocation: location} 113 114 for _, tc := range testCases { 115 change := sqlmodel.NewRowChange(table, nil, tc.preVals, tc.postVals, ti, nil, nil) 116 job := newDMLJob(change, ec) 117 jobCh <- job 118 } 119 120 require.Eventually(t, func() bool { 121 return len(causalityCh) == len(results) 122 }, 3*time.Second, 100*time.Millisecond) 123 124 for _, op := range results { 125 job := <-causalityCh 126 require.Equal(t, op, job.tp) 127 } 128 } 129 130 func (s *testSyncerSuite) TestCasualityRelation(c *C) { 131 rm := newCausalityRelation() 132 c.Assert(rm.len(), Equals, 0) 133 c.Assert(len(rm.groups), Equals, 1) 134 135 testCases := []struct { 136 key string 137 val string 138 }{ 139 {key: "1.key", val: "1.val"}, 140 {key: "2.key", val: "2.val"}, 141 {key: "3.key", val: "3.val"}, 142 {key: "4.key", val: "4.val"}, 143 {key: "5.key", val: "5.val"}, 144 {key: "6.key", val: "6.val"}, 145 {key: "7.key", val: "7.val"}, 146 {key: "8.key", val: "8.val"}, 147 {key: "9.key", val: "9.val"}, 148 {key: "10.key", val: "10.val"}, 149 {key: "11.key", val: "11.val"}, 150 } 151 152 // test without rotate 153 for _, testcase := range testCases { 154 rm.set(testcase.key, testcase.val) 155 } 156 157 c.Assert(rm.len(), Equals, len(testCases)) 158 159 for _, testcase := range testCases { 160 val, ok := rm.get(testcase.key) 161 c.Assert(ok, Equals, true) 162 c.Assert(val, Equals, testcase.val) 163 } 164 165 rm.rotate(1) 166 rm.gc(1) 167 c.Assert(rm.len(), Equals, 0) 168 169 // test gc max 170 for _, testcase := range testCases { 171 rm.set(testcase.key, testcase.val) 172 } 173 174 rm.gc(math.MaxInt64) 175 c.Assert(rm.len(), Equals, 0) 176 177 // test with rotate 178 for index, testcase := range testCases { 179 rm.set(testcase.key, testcase.val) 180 rm.rotate(int64(index)) 181 } 182 183 c.Assert(rm.len(), Equals, len(testCases)) 184 185 for _, testcase := range testCases { 186 val, ok := rm.get(testcase.key) 187 c.Assert(ok, Equals, true) 188 c.Assert(val, Equals, testcase.val) 189 } 190 191 for index := range testCases { 192 rm.gc(int64(index)) 193 194 for _, rmMap := range rm.groups[1:] { 195 c.Assert(rmMap.prevFlushJobSeq, Not(Equals), int64(index)) 196 } 197 198 for ti := 0; ti < index; ti++ { 199 _, ok := rm.get(testCases[ti].key) 200 c.Assert(ok, Equals, false) 201 } 202 } 203 204 rm.clear() 205 c.Assert(rm.len(), Equals, 0) 206 }