github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/cdc/kv/shared_client_test.go (about) 1 // Copyright 2020 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 "net" 19 "sync" 20 "testing" 21 "time" 22 23 "github.com/pingcap/kvproto/pkg/cdcpb" 24 "github.com/pingcap/log" 25 "github.com/pingcap/tidb/pkg/store/mockstore/mockcopr" 26 "github.com/pingcap/tiflow/cdc/kv/regionlock" 27 "github.com/pingcap/tiflow/cdc/kv/sharedconn" 28 "github.com/pingcap/tiflow/cdc/model" 29 "github.com/pingcap/tiflow/cdc/processor/tablepb" 30 "github.com/pingcap/tiflow/pkg/chann" 31 "github.com/pingcap/tiflow/pkg/config" 32 "github.com/pingcap/tiflow/pkg/errors" 33 "github.com/pingcap/tiflow/pkg/pdutil" 34 "github.com/pingcap/tiflow/pkg/security" 35 "github.com/pingcap/tiflow/pkg/txnutil" 36 "github.com/stretchr/testify/require" 37 "github.com/tikv/client-go/v2/oracle" 38 "github.com/tikv/client-go/v2/testutils" 39 "github.com/tikv/client-go/v2/tikv" 40 "google.golang.org/grpc" 41 "google.golang.org/grpc/keepalive" 42 ) 43 44 func newMockService( 45 ctx context.Context, 46 t *testing.T, 47 srv cdcpb.ChangeDataServer, 48 wg *sync.WaitGroup, 49 ) (grpcServer *grpc.Server, addr string) { 50 return newMockServiceSpecificAddr(ctx, t, srv, "127.0.0.1:0", wg) 51 } 52 53 func newMockServiceSpecificAddr( 54 ctx context.Context, 55 t *testing.T, 56 srv cdcpb.ChangeDataServer, 57 listenAddr string, 58 wg *sync.WaitGroup, 59 ) (grpcServer *grpc.Server, addr string) { 60 lc := &net.ListenConfig{} 61 lis, err := lc.Listen(ctx, "tcp", listenAddr) 62 require.Nil(t, err) 63 addr = lis.Addr().String() 64 kaep := keepalive.EnforcementPolicy{ 65 // force minimum ping interval 66 MinTime: 3 * time.Second, 67 PermitWithoutStream: true, 68 } 69 // Some tests rely on connect timeout and ping test, so we use a smaller num 70 kasp := keepalive.ServerParameters{ 71 MaxConnectionIdle: 10 * time.Second, // If a client is idle for 20 seconds, send a GOAWAY 72 MaxConnectionAge: 10 * time.Second, // If any connection is alive for more than 20 seconds, send a GOAWAY 73 MaxConnectionAgeGrace: 5 * time.Second, // Allow 5 seconds for pending RPCs to complete before forcibly closing connections 74 Time: 3 * time.Second, // Ping the client if it is idle for 5 seconds to ensure the connection is still active 75 Timeout: 1 * time.Second, // Wait 1 second for the ping ack before assuming the connection is dead 76 } 77 grpcServer = grpc.NewServer(grpc.KeepaliveEnforcementPolicy(kaep), grpc.KeepaliveParams(kasp)) 78 // grpcServer is the server, srv is the service 79 cdcpb.RegisterChangeDataServer(grpcServer, srv) 80 wg.Add(1) 81 go func() { 82 defer wg.Done() 83 err := grpcServer.Serve(lis) 84 require.Nil(t, err) 85 }() 86 return 87 } 88 89 func TestRequestedStreamRequestedRegions(t *testing.T) { 90 stream := newRequestedStream(100) 91 92 require.Nil(t, stream.getState(1, 2)) 93 require.Nil(t, stream.takeState(1, 2)) 94 95 stream.setState(1, 2, ®ionFeedState{}) 96 require.NotNil(t, stream.getState(1, 2)) 97 require.NotNil(t, stream.takeState(1, 2)) 98 require.Nil(t, stream.getState(1, 2)) 99 require.Equal(t, 0, len(stream.requestedRegions.m)) 100 101 stream.setState(1, 2, ®ionFeedState{}) 102 require.NotNil(t, stream.getState(1, 2)) 103 require.NotNil(t, stream.takeState(1, 2)) 104 require.Nil(t, stream.getState(1, 2)) 105 require.Equal(t, 0, len(stream.requestedRegions.m)) 106 } 107 108 func TestSubscribedTable(t *testing.T) { 109 s := &SharedClient{resolveLockTaskCh: chann.NewAutoDrainChann[resolveLockTask]()} 110 s.logRegionDetails = log.Info 111 span := tablepb.Span{TableID: 1, StartKey: []byte{'a'}, EndKey: []byte{'z'}} 112 table := s.newSubscribedTable(SubscriptionID(1), span, 100, nil) 113 s.totalSpans.v = make(map[SubscriptionID]*subscribedTable) 114 s.totalSpans.v[SubscriptionID(1)] = table 115 s.pdClock = pdutil.NewClock4Test() 116 117 // Lock a range, and then ResolveLock will trigger a task for it. 118 res := table.rangeLock.LockRange(context.Background(), []byte{'b'}, []byte{'c'}, 1, 100) 119 require.Equal(t, regionlock.LockRangeStatusSuccess, res.Status) 120 res.LockedRangeState.Initialzied.Store(true) 121 122 s.ResolveLock(SubscriptionID(1), 200) 123 select { 124 case <-s.resolveLockTaskCh.Out(): 125 case <-time.After(100 * time.Millisecond): 126 require.True(t, false, "must get a resolve lock task") 127 } 128 129 // Lock another range, no task will be triggered before initialized. 130 res = table.rangeLock.LockRange(context.Background(), []byte{'c'}, []byte{'d'}, 2, 100) 131 require.Equal(t, regionlock.LockRangeStatusSuccess, res.Status) 132 state := newRegionFeedState(regionInfo{lockedRangeState: res.LockedRangeState, subscribedTable: table}, 1) 133 select { 134 case <-s.resolveLockTaskCh.Out(): 135 require.True(t, false, "shouldn't get a resolve lock task") 136 case <-time.After(100 * time.Millisecond): 137 } 138 139 // Task will be triggered after initialized. 140 state.setInitialized() 141 state.updateResolvedTs(101) 142 select { 143 case <-s.resolveLockTaskCh.Out(): 144 case <-time.After(100 * time.Millisecond): 145 require.True(t, false, "must get a resolve lock task") 146 } 147 148 s.resolveLockTaskCh.CloseAndDrain() 149 } 150 151 func TestConnectToOfflineOrFailedTiKV(t *testing.T) { 152 ctx, cancel := context.WithCancel(context.Background()) 153 wg := &sync.WaitGroup{} 154 155 events1 := make(chan *cdcpb.ChangeDataEvent, 10) 156 events2 := make(chan *cdcpb.ChangeDataEvent, 10) 157 srv1 := newMockChangeDataServer(events1) 158 server1, addr1 := newMockService(ctx, t, srv1, wg) 159 srv2 := newMockChangeDataServer(events2) 160 server2, addr2 := newMockService(ctx, t, srv2, wg) 161 162 rpcClient, cluster, pdClient, _ := testutils.NewMockTiKV("", mockcopr.NewCoprRPCHandler()) 163 164 pdClient = &mockPDClient{Client: pdClient, versionGen: defaultVersionGen} 165 166 grpcPool := sharedconn.NewConnAndClientPool(&security.Credential{}, nil) 167 168 regionCache := tikv.NewRegionCache(pdClient) 169 170 pdClock := pdutil.NewClock4Test() 171 172 kvStorage, err := tikv.NewTestTiKVStore(rpcClient, pdClient, nil, nil, 0) 173 require.Nil(t, err) 174 lockResolver := txnutil.NewLockerResolver(kvStorage, model.ChangeFeedID{}) 175 176 invalidStore := "localhost:1" 177 cluster.AddStore(1, addr1) 178 cluster.AddStore(2, addr2) 179 cluster.AddStore(3, invalidStore) 180 cluster.Bootstrap(11, []uint64{1, 2, 3}, []uint64{4, 5, 6}, 6) 181 182 client := NewSharedClient( 183 model.ChangeFeedID{ID: "test"}, 184 &config.ServerConfig{ 185 KVClient: &config.KVClientConfig{ 186 WorkerConcurrent: 1, 187 GrpcStreamConcurrent: 1, 188 AdvanceIntervalInMs: 10, 189 }, 190 Debug: &config.DebugConfig{Puller: &config.PullerConfig{LogRegionDetails: false}}, 191 }, 192 false, pdClient, grpcPool, regionCache, pdClock, lockResolver, 193 ) 194 195 defer func() { 196 cancel() 197 client.Close() 198 _ = kvStorage.Close() 199 regionCache.Close() 200 pdClient.Close() 201 srv1.wg.Wait() 202 srv2.wg.Wait() 203 server1.Stop() 204 server2.Stop() 205 wg.Wait() 206 }() 207 208 wg.Add(1) 209 go func() { 210 defer wg.Done() 211 err := client.Run(ctx) 212 require.Equal(t, context.Canceled, errors.Cause(err)) 213 }() 214 215 subID := client.AllocSubscriptionID() 216 span := tablepb.Span{TableID: 1, StartKey: []byte("a"), EndKey: []byte("b")} 217 eventCh := make(chan MultiplexingEvent, 50) 218 client.Subscribe(subID, span, 1, eventCh) 219 220 makeTsEvent := func(regionID, ts, requestID uint64) *cdcpb.ChangeDataEvent { 221 return &cdcpb.ChangeDataEvent{ 222 Events: []*cdcpb.Event{ 223 { 224 RegionId: regionID, 225 RequestId: requestID, 226 Event: &cdcpb.Event_ResolvedTs{ResolvedTs: ts}, 227 }, 228 }, 229 } 230 } 231 232 checkTsEvent := func(event model.RegionFeedEvent, ts uint64) { 233 require.Equal(t, ts, event.Resolved.ResolvedTs) 234 } 235 236 events1 <- mockInitializedEvent(11, uint64(subID)) 237 ts := oracle.GoTimeToTS(pdClock.CurrentTime()) 238 events1 <- makeTsEvent(11, ts, uint64(subID)) 239 // After trying to receive something from the invalid store, 240 // it should auto switch to other stores and fetch events finally. 241 select { 242 case event := <-eventCh: 243 checkTsEvent(event.RegionFeedEvent, ts) 244 case <-time.After(5 * time.Second): 245 require.True(t, false, "reconnection not succeed in 5 second") 246 } 247 248 // Stop server1 and the client needs to handle it. 249 server1.Stop() 250 251 events2 <- mockInitializedEvent(11, uint64(subID)) 252 ts = oracle.GoTimeToTS(pdClock.CurrentTime()) 253 events2 <- makeTsEvent(11, ts, uint64(subID)) 254 // After trying to receive something from a failed store, 255 // it should auto switch to other stores and fetch events finally. 256 select { 257 case event := <-eventCh: 258 checkTsEvent(event.RegionFeedEvent, ts) 259 case <-time.After(5 * time.Second): 260 require.True(t, false, "reconnection not succeed in 5 second") 261 } 262 } 263 264 type mockChangeDataServer struct { 265 ch chan *cdcpb.ChangeDataEvent 266 wg sync.WaitGroup 267 } 268 269 func newMockChangeDataServer(ch chan *cdcpb.ChangeDataEvent) *mockChangeDataServer { 270 return &mockChangeDataServer{ch: ch} 271 } 272 273 func (m *mockChangeDataServer) EventFeed(s cdcpb.ChangeData_EventFeedServer) error { 274 closed := make(chan struct{}) 275 m.wg.Add(1) 276 go func() { 277 defer m.wg.Done() 278 defer close(closed) 279 for { 280 if _, err := s.Recv(); err != nil { 281 return 282 } 283 } 284 }() 285 m.wg.Add(1) 286 defer m.wg.Done() 287 ticker := time.NewTicker(20 * time.Millisecond) 288 defer ticker.Stop() 289 for { 290 select { 291 case <-closed: 292 return nil 293 case <-ticker.C: 294 } 295 select { 296 case event := <-m.ch: 297 if err := s.Send(event); err != nil { 298 return err 299 } 300 default: 301 } 302 } 303 } 304 305 func (m *mockChangeDataServer) EventFeedV2(s cdcpb.ChangeData_EventFeedV2Server) error { 306 return m.EventFeed(s) 307 }