github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/cdc/kv/shared_region_worker_test.go (about) 1 // Copyright 2023 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 kv 15 16 import ( 17 "context" 18 "testing" 19 "time" 20 21 "github.com/pingcap/kvproto/pkg/cdcpb" 22 "github.com/pingcap/tiflow/cdc/kv/regionlock" 23 "github.com/pingcap/tiflow/cdc/model" 24 "github.com/pingcap/tiflow/cdc/processor/tablepb" 25 "github.com/pingcap/tiflow/pkg/config" 26 "github.com/pingcap/tiflow/pkg/spanz" 27 "github.com/stretchr/testify/require" 28 "github.com/tikv/client-go/v2/tikv" 29 ) 30 31 func newSharedClientForTestSharedRegionWorker() *SharedClient { 32 // sharedRegionWorker only requires `SharedClient.onRegionFail`. 33 cfg := &config.ServerConfig{ 34 KVClient: &config.KVClientConfig{ 35 EnableMultiplexing: true, 36 WorkerConcurrent: 8, 37 GrpcStreamConcurrent: 2, 38 AdvanceIntervalInMs: 10, 39 }, 40 Debug: &config.DebugConfig{ 41 Puller: &config.PullerConfig{LogRegionDetails: false}, 42 }, 43 } 44 return NewSharedClient(model.ChangeFeedID{}, cfg, false, nil, nil, nil, nil, nil) 45 } 46 47 // For UPDATE SQL, its prewrite event has both value and old value. 48 // It is possible that TiDB prewrites multiple times for the same row when 49 // there are other transactions it conflicts with. For this case, 50 // if the value is not "short", only the first prewrite contains the value. 51 // 52 // TiKV may output events for the UPDATE SQL as following: 53 // 54 // TiDB: [Prwrite1] [Prewrite2] [Commit] 55 // 56 // v v v Time 57 // 58 // ----------------------------------------------------------------------------> 59 // 60 // ^ ^ ^ ^ ^ ^ ^ ^ ^ 61 // 62 // TiKV: [Scan Start] [Send Prewrite2] [Send Commit] [Send Prewrite1] [Send Init] 63 // TiCDC: [Recv Prewrite2] [Recv Commit] [Recv Prewrite1] [Recv Init] 64 func TestSharedRegionWokerHandleEventEntryEventOutOfOrder(t *testing.T) { 65 ctx, cancel := context.WithCancel(context.Background()) 66 defer cancel() 67 client := newSharedClientForTestSharedRegionWorker() 68 defer client.Close() 69 70 worker := newSharedRegionWorker(client) 71 eventCh := make(chan MultiplexingEvent, 2) 72 73 span := spanz.ToSpan([]byte{}, spanz.UpperBoundKey) 74 region := newRegionInfo( 75 tikv.RegionVerID{}, 76 span, 77 &tikv.RPCContext{}, 78 &subscribedTable{subscriptionID: SubscriptionID(1), eventCh: eventCh}, 79 ) 80 region.lockedRangeState = ®ionlock.LockedRangeState{} 81 state := newRegionFeedState(region, 1) 82 state.start() 83 84 // Receive prewrite2 with empty value. 85 events := &cdcpb.Event_Entries_{ 86 Entries: &cdcpb.Event_Entries{ 87 Entries: []*cdcpb.Event_Row{{ 88 StartTs: 1, 89 Type: cdcpb.Event_PREWRITE, 90 OpType: cdcpb.Event_Row_PUT, 91 Key: []byte("key"), 92 Value: nil, 93 OldValue: []byte("oldvalue"), 94 }}, 95 }, 96 } 97 err := worker.handleEventEntry(ctx, events, state) 98 require.Nil(t, err) 99 100 // Receive commit. 101 events = &cdcpb.Event_Entries_{ 102 Entries: &cdcpb.Event_Entries{ 103 Entries: []*cdcpb.Event_Row{{ 104 StartTs: 1, 105 CommitTs: 2, 106 Type: cdcpb.Event_COMMIT, 107 OpType: cdcpb.Event_Row_PUT, 108 Key: []byte("key"), 109 }}, 110 }, 111 } 112 err = worker.handleEventEntry(context.Background(), events, state) 113 require.Nil(t, err) 114 115 // Must not output event. 116 select { 117 case <-eventCh: 118 require.True(t, false, "shouldn't get an event") 119 case <-time.NewTimer(100 * time.Millisecond).C: 120 } 121 122 // Receive prewrite1 with actual value. 123 events = &cdcpb.Event_Entries_{ 124 Entries: &cdcpb.Event_Entries{ 125 Entries: []*cdcpb.Event_Row{{ 126 StartTs: 1, 127 Type: cdcpb.Event_PREWRITE, 128 OpType: cdcpb.Event_Row_PUT, 129 Key: []byte("key"), 130 Value: []byte("value"), 131 OldValue: []byte("oldvalue"), 132 }}, 133 }, 134 } 135 err = worker.handleEventEntry(ctx, events, state) 136 require.Nil(t, err) 137 138 // Must not output event. 139 select { 140 case <-eventCh: 141 require.True(t, false, "shouldn't get an event") 142 case <-time.NewTimer(100 * time.Millisecond).C: 143 } 144 145 // Receive initialized. 146 events = &cdcpb.Event_Entries_{ 147 Entries: &cdcpb.Event_Entries{ 148 Entries: []*cdcpb.Event_Row{ 149 { 150 Type: cdcpb.Event_INITIALIZED, 151 }, 152 }, 153 }, 154 } 155 err = worker.handleEventEntry(ctx, events, state) 156 require.Nil(t, err) 157 158 // Must output event. 159 select { 160 case event := <-eventCh: 161 require.Equal(t, uint64(2), event.Val.CRTs) 162 require.Equal(t, uint64(1), event.Val.StartTs) 163 require.Equal(t, "value", string(event.Val.Value)) 164 require.Equal(t, "oldvalue", string(event.Val.OldValue)) 165 case <-time.NewTimer(100 * time.Millisecond).C: 166 require.True(t, false, "must get an event") 167 } 168 } 169 170 func TestSharedRegionWorkerHandleResolvedTs(t *testing.T) { 171 ctx, cancel := context.WithCancel(context.Background()) 172 defer cancel() 173 client := newSharedClientForTestSharedRegionWorker() 174 defer client.Close() 175 176 worker := newSharedRegionWorker(client) 177 eventCh := make(chan MultiplexingEvent, 2) 178 179 s1 := newRegionFeedState(regionInfo{verID: tikv.NewRegionVerID(1, 1, 1)}, 1) 180 s1.region.subscribedTable = client.newSubscribedTable(1, tablepb.Span{}, 0, eventCh) 181 s1.region.lockedRangeState = ®ionlock.LockedRangeState{} 182 s1.setInitialized() 183 s1.updateResolvedTs(9) 184 185 s2 := newRegionFeedState(regionInfo{verID: tikv.NewRegionVerID(2, 2, 2)}, 2) 186 s2.region.subscribedTable = client.newSubscribedTable(2, tablepb.Span{}, 0, eventCh) 187 s2.region.lockedRangeState = ®ionlock.LockedRangeState{} 188 s2.setInitialized() 189 s2.updateResolvedTs(11) 190 191 s3 := newRegionFeedState(regionInfo{verID: tikv.NewRegionVerID(3, 3, 3)}, 3) 192 s3.region.subscribedTable = client.newSubscribedTable(3, tablepb.Span{}, 0, eventCh) 193 s3.region.lockedRangeState = ®ionlock.LockedRangeState{} 194 s3.updateResolvedTs(8) 195 196 worker.handleResolvedTs(ctx, resolvedTsBatch{ts: 10, regions: []*regionFeedState{s1, s2, s3}}) 197 require.Equal(t, uint64(10), s1.getLastResolvedTs()) 198 require.Equal(t, uint64(11), s2.getLastResolvedTs()) 199 require.Equal(t, uint64(8), s3.getLastResolvedTs()) 200 }