github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/cdc/kv/shared_stream.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 "sync" 19 "time" 20 21 "github.com/pingcap/errors" 22 "github.com/pingcap/kvproto/pkg/cdcpb" 23 "github.com/pingcap/log" 24 "github.com/pingcap/tiflow/cdc/kv/sharedconn" 25 "github.com/pingcap/tiflow/pkg/chann" 26 "github.com/pingcap/tiflow/pkg/util" 27 "github.com/pingcap/tiflow/pkg/version" 28 "go.uber.org/zap" 29 "golang.org/x/sync/errgroup" 30 grpcstatus "google.golang.org/grpc/status" 31 ) 32 33 type requestedStream struct { 34 streamID uint64 35 36 // To trigger a connect action lazily. 37 preFetchForConnecting *regionInfo 38 requests *chann.DrainableChann[regionInfo] 39 40 requestedRegions struct { 41 sync.RWMutex 42 // map[SubscriptionID]map[RegionID]*regionFeedState 43 m map[SubscriptionID]map[uint64]*regionFeedState 44 } 45 46 logRegionDetails func(msg string, fields ...zap.Field) 47 48 // multiplexing is for sharing one GRPC stream in many tables. 49 multiplexing *sharedconn.ConnAndClient 50 51 // tableExclusives means one GRPC stream is exclusive by one table. 52 tableExclusives chan tableExclusive 53 } 54 55 type tableExclusive struct { 56 subscriptionID SubscriptionID 57 cc *sharedconn.ConnAndClient 58 } 59 60 func newStream(ctx context.Context, c *SharedClient, g *errgroup.Group, r *requestedStore) *requestedStream { 61 stream := newRequestedStream(streamIDGen.Add(1)) 62 stream.logRegionDetails = c.logRegionDetails 63 stream.requests = chann.NewAutoDrainChann[regionInfo]() 64 65 waitForPreFetching := func() error { 66 if stream.preFetchForConnecting != nil { 67 log.Panic("preFetchForConnecting should be nil", 68 zap.String("namespace", c.changefeed.Namespace), 69 zap.String("changefeed", c.changefeed.ID), 70 zap.Uint64("streamID", stream.streamID), 71 zap.Uint64("storeID", r.storeID), 72 zap.String("addr", r.storeAddr)) 73 } 74 for { 75 select { 76 case <-ctx.Done(): 77 return ctx.Err() 78 case region := <-stream.requests.Out(): 79 if !region.isStoped() { 80 stream.preFetchForConnecting = new(regionInfo) 81 *stream.preFetchForConnecting = region 82 return nil 83 } 84 } 85 } 86 } 87 88 g.Go(func() error { 89 for { 90 if err := waitForPreFetching(); err != nil { 91 return err 92 } 93 if canceled := stream.run(ctx, c, r); canceled { 94 return nil 95 } 96 for _, m := range stream.clearStates() { 97 for _, state := range m { 98 state.markStopped(&sendRequestToStoreErr{}) 99 sfEvent := newEventItem(nil, state, stream) 100 slot := hashRegionID(state.region.verID.GetID(), len(c.workers)) 101 _ = c.workers[slot].sendEvent(ctx, sfEvent) 102 } 103 } 104 // Why we need to re-schedule pending regions? This because the store can 105 // fail forever, and all regions are scheduled to other stores. 106 for _, region := range stream.clearPendingRegions() { 107 if region.isStoped() { 108 // It means it's a special task for stopping the table. 109 continue 110 } 111 c.onRegionFail(newRegionErrorInfo(region, &sendRequestToStoreErr{})) 112 } 113 if err := util.Hang(ctx, time.Second); err != nil { 114 return err 115 } 116 } 117 }) 118 119 return stream 120 } 121 122 func newRequestedStream(streamID uint64) *requestedStream { 123 stream := &requestedStream{streamID: streamID} 124 stream.requestedRegions.m = make(map[SubscriptionID]map[uint64]*regionFeedState) 125 return stream 126 } 127 128 func (s *requestedStream) run(ctx context.Context, c *SharedClient, rs *requestedStore) (canceled bool) { 129 isCanceled := func() bool { 130 select { 131 case <-ctx.Done(): 132 return true 133 default: 134 return false 135 } 136 } 137 138 if err := version.CheckStoreVersion(ctx, c.pd, rs.storeID); err != nil { 139 log.Info("event feed check store version fails", 140 zap.String("namespace", c.changefeed.Namespace), 141 zap.String("changefeed", c.changefeed.ID), 142 zap.Uint64("streamID", s.streamID), 143 zap.Uint64("storeID", rs.storeID), 144 zap.String("addr", rs.storeAddr), 145 zap.Error(err)) 146 return isCanceled() 147 } 148 149 log.Info("event feed going to create grpc stream", 150 zap.String("namespace", c.changefeed.Namespace), 151 zap.String("changefeed", c.changefeed.ID), 152 zap.Uint64("streamID", s.streamID), 153 zap.Uint64("storeID", rs.storeID), 154 zap.String("addr", rs.storeAddr)) 155 156 defer func() { 157 log.Info("event feed grpc stream exits", 158 zap.String("namespace", c.changefeed.Namespace), 159 zap.String("changefeed", c.changefeed.ID), 160 zap.Uint64("streamID", s.streamID), 161 zap.Uint64("storeID", rs.storeID), 162 zap.String("addr", rs.storeAddr), 163 zap.Bool("canceled", canceled)) 164 if s.multiplexing != nil { 165 s.multiplexing = nil 166 } else if s.tableExclusives != nil { 167 close(s.tableExclusives) 168 s.tableExclusives = nil 169 } 170 }() 171 172 // grpc stream can be canceled by this context when any goroutine meet error, 173 // the underline established grpc connections is unaffected. 174 g, gctx := errgroup.WithContext(ctx) 175 cc, err := c.grpcPool.Connect(gctx, rs.storeAddr) 176 if err != nil { 177 log.Warn("event feed create grpc stream failed", 178 zap.String("namespace", c.changefeed.Namespace), 179 zap.String("changefeed", c.changefeed.ID), 180 zap.Uint64("streamID", s.streamID), 181 zap.Uint64("storeID", rs.storeID), 182 zap.String("addr", rs.storeAddr), 183 zap.Error(err)) 184 return isCanceled() 185 } 186 187 if cc.Multiplexing() { 188 s.multiplexing = cc 189 g.Go(func() error { return s.receive(gctx, c, rs, s.multiplexing, invalidSubscriptionID) }) 190 } else { 191 log.Info("event feed stream multiplexing is not supported, will fallback", 192 zap.String("namespace", c.changefeed.Namespace), 193 zap.String("changefeed", c.changefeed.ID), 194 zap.Uint64("streamID", s.streamID), 195 zap.Uint64("storeID", rs.storeID), 196 zap.String("addr", rs.storeAddr)) 197 cc.Release() 198 199 s.tableExclusives = make(chan tableExclusive, 8) 200 g.Go(func() error { 201 for { 202 select { 203 case <-gctx.Done(): 204 return gctx.Err() 205 case tableExclusive := <-s.tableExclusives: 206 subscriptionID := tableExclusive.subscriptionID 207 cc := tableExclusive.cc 208 g.Go(func() error { return s.receive(gctx, c, rs, cc, subscriptionID) }) 209 } 210 } 211 }) 212 } 213 g.Go(func() error { return s.send(gctx, c, rs) }) 214 _ = g.Wait() 215 return isCanceled() 216 } 217 218 func (s *requestedStream) receive( 219 ctx context.Context, 220 c *SharedClient, 221 rs *requestedStore, 222 cc *sharedconn.ConnAndClient, 223 subscriptionID SubscriptionID, 224 ) error { 225 client := cc.Client() 226 for { 227 cevent, err := client.Recv() 228 if err != nil { 229 s.logRegionDetails("event feed receive from grpc stream failed", 230 zap.String("namespace", c.changefeed.Namespace), 231 zap.String("changefeed", c.changefeed.ID), 232 zap.Uint64("streamID", s.streamID), 233 zap.Uint64("storeID", rs.storeID), 234 zap.String("addr", rs.storeAddr), 235 zap.String("code", grpcstatus.Code(err).String()), 236 zap.Error(err)) 237 if sharedconn.StatusIsEOF(grpcstatus.Convert(err)) { 238 return nil 239 } 240 return errors.Trace(err) 241 } 242 if len(cevent.Events) > 0 { 243 if err := s.sendRegionChangeEvents(ctx, c, cevent.Events, subscriptionID); err != nil { 244 return err 245 } 246 } 247 if cevent.ResolvedTs != nil { 248 c.metrics.batchResolvedSize.Observe(float64(len(cevent.ResolvedTs.Regions))) 249 if err := s.sendResolvedTs(ctx, c, cevent.ResolvedTs, subscriptionID); err != nil { 250 return err 251 } 252 } 253 } 254 } 255 256 func (s *requestedStream) send(ctx context.Context, c *SharedClient, rs *requestedStore) (err error) { 257 doSend := func(cc *sharedconn.ConnAndClient, req *cdcpb.ChangeDataRequest, subscriptionID SubscriptionID) error { 258 if err := cc.Client().Send(req); err != nil { 259 log.Warn("event feed send request to grpc stream failed", 260 zap.String("namespace", c.changefeed.Namespace), 261 zap.String("changefeed", c.changefeed.ID), 262 zap.Uint64("streamID", s.streamID), 263 zap.Any("subscriptionID", subscriptionID), 264 zap.Uint64("regionID", req.RegionId), 265 zap.Uint64("storeID", rs.storeID), 266 zap.String("addr", rs.storeAddr), 267 zap.Error(err)) 268 return errors.Trace(err) 269 } 270 log.Debug("event feed send request to grpc stream success", 271 zap.String("namespace", c.changefeed.Namespace), 272 zap.String("changefeed", c.changefeed.ID), 273 zap.Uint64("streamID", s.streamID), 274 zap.Any("subscriptionID", subscriptionID), 275 zap.Uint64("regionID", req.RegionId), 276 zap.Uint64("storeID", rs.storeID), 277 zap.String("addr", rs.storeAddr)) 278 return nil 279 } 280 281 fetchMoreReq := func() (regionInfo, error) { 282 waitReqTicker := time.NewTicker(60 * time.Second) 283 defer waitReqTicker.Stop() 284 for { 285 var region regionInfo 286 select { 287 case <-ctx.Done(): 288 return region, ctx.Err() 289 case region = <-s.requests.Out(): 290 return region, nil 291 case <-waitReqTicker.C: 292 // The stream is idle now, will be re-established when necessary. 293 if s.countStates() == 0 { 294 return region, errors.New("closed as idle") 295 } 296 } 297 } 298 } 299 300 tableExclusives := make(map[SubscriptionID]*sharedconn.ConnAndClient) 301 getTableExclusiveConn := func(subscriptionID SubscriptionID) (cc *sharedconn.ConnAndClient, err error) { 302 if cc = tableExclusives[subscriptionID]; cc == nil { 303 if cc, err = c.grpcPool.Connect(ctx, rs.storeAddr); err != nil { 304 return 305 } 306 if cc.Multiplexing() { 307 cc.Release() 308 cc, err = nil, errors.New("multiplexing is enabled, will re-establish the stream") 309 return 310 } 311 tableExclusives[subscriptionID] = cc 312 select { 313 case <-ctx.Done(): 314 case s.tableExclusives <- tableExclusive{subscriptionID, cc}: 315 } 316 } 317 return 318 } 319 defer func() { 320 if s.multiplexing != nil { 321 s.multiplexing.Release() 322 } 323 for _, cc := range tableExclusives { 324 cc.Release() 325 } 326 }() 327 328 region := *s.preFetchForConnecting 329 s.preFetchForConnecting = nil 330 for { 331 subscriptionID := region.subscribedTable.subscriptionID 332 log.Debug("event feed gets a singleRegionInfo", 333 zap.String("namespace", c.changefeed.Namespace), 334 zap.String("changefeed", c.changefeed.ID), 335 zap.Uint64("streamID", s.streamID), 336 zap.Any("subscriptionID", subscriptionID), 337 zap.Uint64("regionID", region.verID.GetID()), 338 zap.Uint64("storeID", rs.storeID), 339 zap.String("addr", rs.storeAddr)) 340 // It means it's a special task for stopping the table. 341 if region.isStoped() { 342 if s.multiplexing != nil { 343 req := &cdcpb.ChangeDataRequest{ 344 RequestId: uint64(subscriptionID), 345 Request: &cdcpb.ChangeDataRequest_Deregister_{}, 346 } 347 if err = doSend(s.multiplexing, req, subscriptionID); err != nil { 348 return err 349 } 350 } else if cc := tableExclusives[subscriptionID]; cc != nil { 351 delete(tableExclusives, subscriptionID) 352 cc.Release() 353 } 354 // NOTE: some principles to help understand deregistering a table: 355 // 1. after a Deregister(requestID) message is sent out, no more region requests 356 // with the same requestID will be sent out in the same GRPC stream; 357 // 2. so it's OK to clear all pending states in the GRPC stream; 358 // 3. is it possible that TiKV is keeping to send events belong to a removed state? 359 // I guess no because internal errors will cause the changefeed or table stopped, 360 // and then those regions from the bad requestID will be unsubscribed finally. 361 for _, state := range s.takeStates(subscriptionID) { 362 state.markStopped(&sendRequestToStoreErr{}) 363 sfEvent := newEventItem(nil, state, s) 364 slot := hashRegionID(state.region.verID.GetID(), len(c.workers)) 365 if err = c.workers[slot].sendEvent(ctx, sfEvent); err != nil { 366 return errors.Trace(err) 367 } 368 } 369 } else if region.subscribedTable.stopped.Load() { 370 // It can be skipped directly because there must be no pending states from 371 // the stopped subscribedTable, or the special singleRegionInfo for stopping 372 // the table will be handled later. 373 c.onRegionFail(newRegionErrorInfo(region, &sendRequestToStoreErr{})) 374 } else { 375 connectTime := time.Since(region.lockedRangeState.Created).Milliseconds() 376 c.metrics.regionConnectDuration.Observe(float64(connectTime)) 377 378 state := newRegionFeedState(region, uint64(subscriptionID)) 379 state.start() 380 s.setState(subscriptionID, region.verID.GetID(), state) 381 382 var cc *sharedconn.ConnAndClient 383 if s.multiplexing != nil { 384 cc = s.multiplexing 385 } else if cc, err = getTableExclusiveConn(subscriptionID); err != nil { 386 return err 387 } 388 if err = doSend(cc, c.createRegionRequest(region), subscriptionID); err != nil { 389 return err 390 } 391 } 392 393 if region, err = fetchMoreReq(); err != nil { 394 return err 395 } 396 } 397 } 398 399 func (s *requestedStream) countStates() (sum int) { 400 s.requestedRegions.Lock() 401 defer s.requestedRegions.Unlock() 402 for _, mm := range s.requestedRegions.m { 403 sum += len(mm) 404 } 405 return 406 } 407 408 func (s *requestedStream) setState(subscriptionID SubscriptionID, regionID uint64, state *regionFeedState) { 409 s.requestedRegions.Lock() 410 defer s.requestedRegions.Unlock() 411 var m map[uint64]*regionFeedState 412 if m = s.requestedRegions.m[subscriptionID]; m == nil { 413 m = make(map[uint64]*regionFeedState) 414 s.requestedRegions.m[subscriptionID] = m 415 } 416 m[regionID] = state 417 } 418 419 func (s *requestedStream) getState(subscriptionID SubscriptionID, regionID uint64) (state *regionFeedState) { 420 s.requestedRegions.RLock() 421 defer s.requestedRegions.RUnlock() 422 if m, ok := s.requestedRegions.m[subscriptionID]; ok { 423 state = m[regionID] 424 } 425 return state 426 } 427 428 func (s *requestedStream) takeState(subscriptionID SubscriptionID, regionID uint64) (state *regionFeedState) { 429 s.requestedRegions.Lock() 430 defer s.requestedRegions.Unlock() 431 if m, ok := s.requestedRegions.m[subscriptionID]; ok { 432 state = m[regionID] 433 delete(m, regionID) 434 if len(m) == 0 { 435 delete(s.requestedRegions.m, subscriptionID) 436 } 437 } 438 return 439 } 440 441 func (s *requestedStream) takeStates(subscriptionID SubscriptionID) (v map[uint64]*regionFeedState) { 442 s.requestedRegions.Lock() 443 defer s.requestedRegions.Unlock() 444 v = s.requestedRegions.m[subscriptionID] 445 delete(s.requestedRegions.m, subscriptionID) 446 return 447 } 448 449 func (s *requestedStream) clearStates() (v map[SubscriptionID]map[uint64]*regionFeedState) { 450 s.requestedRegions.Lock() 451 defer s.requestedRegions.Unlock() 452 v = s.requestedRegions.m 453 s.requestedRegions.m = make(map[SubscriptionID]map[uint64]*regionFeedState) 454 return 455 } 456 457 func (s *requestedStream) clearPendingRegions() []regionInfo { 458 regions := make([]regionInfo, 0, s.requests.Len()+1) 459 if s.preFetchForConnecting != nil { 460 region := *s.preFetchForConnecting 461 s.preFetchForConnecting = nil 462 regions = append(regions, region) 463 } 464 for i := 1; i < cap(regions); i++ { 465 regions = append(regions, <-s.requests.Out()) 466 } 467 return regions 468 } 469 470 func (s *requestedStream) sendRegionChangeEvents( 471 ctx context.Context, c *SharedClient, events []*cdcpb.Event, 472 tableSubID SubscriptionID, 473 ) error { 474 for _, event := range events { 475 regionID := event.RegionId 476 var subscriptionID SubscriptionID 477 if tableSubID == invalidSubscriptionID { 478 subscriptionID = SubscriptionID(event.RequestId) 479 } else { 480 subscriptionID = tableSubID 481 } 482 483 state := s.getState(subscriptionID, regionID) 484 switch x := event.Event.(type) { 485 case *cdcpb.Event_Error: 486 s.logRegionDetails("event feed receives a region error", 487 zap.String("namespace", c.changefeed.Namespace), 488 zap.String("changefeed", c.changefeed.ID), 489 zap.Uint64("streamID", s.streamID), 490 zap.Any("subscriptionID", subscriptionID), 491 zap.Uint64("regionID", event.RegionId), 492 zap.Bool("stateIsNil", state == nil), 493 zap.Any("error", x.Error)) 494 } 495 496 if state != nil { 497 sfEvent := newEventItem(event, state, s) 498 slot := hashRegionID(regionID, len(c.workers)) 499 if err := c.workers[slot].sendEvent(ctx, sfEvent); err != nil { 500 return errors.Trace(err) 501 } 502 } 503 } 504 return nil 505 } 506 507 func (s *requestedStream) sendResolvedTs( 508 ctx context.Context, c *SharedClient, resolvedTs *cdcpb.ResolvedTs, 509 tableSubID SubscriptionID, 510 ) error { 511 var subscriptionID SubscriptionID 512 if tableSubID == invalidSubscriptionID { 513 subscriptionID = SubscriptionID(resolvedTs.RequestId) 514 } else { 515 subscriptionID = tableSubID 516 } 517 sfEvents := make([]statefulEvent, len(c.workers)) 518 for _, regionID := range resolvedTs.Regions { 519 slot := hashRegionID(regionID, len(c.workers)) 520 if sfEvents[slot].stream == nil { 521 sfEvents[slot] = newResolvedTsBatch(resolvedTs.Ts, s) 522 } 523 x := &sfEvents[slot].resolvedTsBatch 524 if state := s.getState(subscriptionID, regionID); state != nil { 525 x.regions = append(x.regions, state) 526 } 527 } 528 529 for i, sfEvent := range sfEvents { 530 if len(sfEvent.resolvedTsBatch.regions) > 0 { 531 sfEvent.stream = s 532 if err := c.workers[i].sendEvent(ctx, sfEvent); err != nil { 533 return err 534 } 535 } 536 } 537 return nil 538 }