github.com/QuangHoangHao/kafka-go@v0.4.36/consumergroup.go (about) 1 package kafka 2 3 import ( 4 "bufio" 5 "bytes" 6 "context" 7 "errors" 8 "fmt" 9 "io" 10 "math" 11 "net" 12 "strconv" 13 "sync" 14 "time" 15 ) 16 17 // ErrGroupClosed is returned by ConsumerGroup.Next when the group has already 18 // been closed. 19 var ErrGroupClosed = errors.New("consumer group is closed") 20 21 // ErrGenerationEnded is returned by the context.Context issued by the 22 // Generation's Start function when the context has been closed. 23 var ErrGenerationEnded = errors.New("consumer group generation has ended") 24 25 const ( 26 // defaultProtocolType holds the default protocol type documented in the 27 // kafka protocol 28 // 29 // See https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-GroupMembershipAPI 30 defaultProtocolType = "consumer" 31 32 // defaultHeartbeatInterval contains the default time between heartbeats. If 33 // the coordinator does not receive a heartbeat within the session timeout interval, 34 // the consumer will be considered dead and the coordinator will rebalance the 35 // group. 36 // 37 // As a rule, the heartbeat interval should be no greater than 1/3 the session timeout. 38 defaultHeartbeatInterval = 3 * time.Second 39 40 // defaultSessionTimeout contains the default interval the coordinator will wait 41 // for a heartbeat before marking a consumer as dead. 42 defaultSessionTimeout = 30 * time.Second 43 44 // defaultRebalanceTimeout contains the amount of time the coordinator will wait 45 // for consumers to issue a join group once a rebalance has been requested. 46 defaultRebalanceTimeout = 30 * time.Second 47 48 // defaultJoinGroupBackoff is the amount of time to wait after a failed 49 // consumer group generation before attempting to re-join. 50 defaultJoinGroupBackoff = 5 * time.Second 51 52 // defaultRetentionTime holds the length of time a the consumer group will be 53 // saved by kafka. This value tells the broker to use its configured value. 54 defaultRetentionTime = -1 * time.Millisecond 55 56 // defaultPartitionWatchTime contains the amount of time the kafka-go will wait to 57 // query the brokers looking for partition changes. 58 defaultPartitionWatchTime = 5 * time.Second 59 60 // defaultTimeout is the deadline to set when interacting with the 61 // consumer group coordinator. 62 defaultTimeout = 5 * time.Second 63 ) 64 65 // ConsumerGroupConfig is a configuration object used to create new instances of 66 // ConsumerGroup. 67 type ConsumerGroupConfig struct { 68 // ID is the consumer group ID. It must not be empty. 69 ID string 70 71 // The list of broker addresses used to connect to the kafka cluster. It 72 // must not be empty. 73 Brokers []string 74 75 // An dialer used to open connections to the kafka server. This field is 76 // optional, if nil, the default dialer is used instead. 77 Dialer *Dialer 78 79 // Topics is the list of topics that will be consumed by this group. It 80 // will usually have a single value, but it is permitted to have multiple 81 // for more complex use cases. 82 Topics []string 83 84 // GroupBalancers is the priority-ordered list of client-side consumer group 85 // balancing strategies that will be offered to the coordinator. The first 86 // strategy that all group members support will be chosen by the leader. 87 // 88 // Default: [Range, RoundRobin] 89 GroupBalancers []GroupBalancer 90 91 // HeartbeatInterval sets the optional frequency at which the reader sends the consumer 92 // group heartbeat update. 93 // 94 // Default: 3s 95 HeartbeatInterval time.Duration 96 97 // PartitionWatchInterval indicates how often a reader checks for partition changes. 98 // If a reader sees a partition change (such as a partition add) it will rebalance the group 99 // picking up new partitions. 100 // 101 // Default: 5s 102 PartitionWatchInterval time.Duration 103 104 // WatchForPartitionChanges is used to inform kafka-go that a consumer group should be 105 // polling the brokers and rebalancing if any partition changes happen to the topic. 106 WatchPartitionChanges bool 107 108 // SessionTimeout optionally sets the length of time that may pass without a heartbeat 109 // before the coordinator considers the consumer dead and initiates a rebalance. 110 // 111 // Default: 30s 112 SessionTimeout time.Duration 113 114 // RebalanceTimeout optionally sets the length of time the coordinator will wait 115 // for members to join as part of a rebalance. For kafka servers under higher 116 // load, it may be useful to set this value higher. 117 // 118 // Default: 30s 119 RebalanceTimeout time.Duration 120 121 // JoinGroupBackoff optionally sets the length of time to wait before re-joining 122 // the consumer group after an error. 123 // 124 // Default: 5s 125 JoinGroupBackoff time.Duration 126 127 // RetentionTime optionally sets the length of time the consumer group will 128 // be saved by the broker. -1 will disable the setting and leave the 129 // retention up to the broker's offsets.retention.minutes property. By 130 // default, that setting is 1 day for kafka < 2.0 and 7 days for kafka >= 131 // 2.0. 132 // 133 // Default: -1 134 RetentionTime time.Duration 135 136 // StartOffset determines from whence the consumer group should begin 137 // consuming when it finds a partition without a committed offset. If 138 // non-zero, it must be set to one of FirstOffset or LastOffset. 139 // 140 // Default: FirstOffset 141 StartOffset int64 142 143 // If not nil, specifies a logger used to report internal changes within the 144 // reader. 145 Logger Logger 146 147 // ErrorLogger is the logger used to report errors. If nil, the reader falls 148 // back to using Logger instead. 149 ErrorLogger Logger 150 151 // Timeout is the network timeout used when communicating with the consumer 152 // group coordinator. This value should not be too small since errors 153 // communicating with the broker will generally cause a consumer group 154 // rebalance, and it's undesirable that a transient network error intoduce 155 // that overhead. Similarly, it should not be too large or the consumer 156 // group may be slow to respond to the coordinator failing over to another 157 // broker. 158 // 159 // Default: 5s 160 Timeout time.Duration 161 162 // connect is a function for dialing the coordinator. This is provided for 163 // unit testing to mock broker connections. 164 connect func(dialer *Dialer, brokers ...string) (coordinator, error) 165 } 166 167 // Validate method validates ConsumerGroupConfig properties and sets relevant 168 // defaults. 169 func (config *ConsumerGroupConfig) Validate() error { 170 171 if len(config.Brokers) == 0 { 172 return errors.New("cannot create a consumer group with an empty list of broker addresses") 173 } 174 175 if len(config.Topics) == 0 { 176 return errors.New("cannot create a consumer group without a topic") 177 } 178 179 if config.ID == "" { 180 return errors.New("cannot create a consumer group without an ID") 181 } 182 183 if config.Dialer == nil { 184 config.Dialer = DefaultDialer 185 } 186 187 if len(config.GroupBalancers) == 0 { 188 config.GroupBalancers = []GroupBalancer{ 189 RangeGroupBalancer{}, 190 RoundRobinGroupBalancer{}, 191 } 192 } 193 194 if config.HeartbeatInterval == 0 { 195 config.HeartbeatInterval = defaultHeartbeatInterval 196 } 197 198 if config.SessionTimeout == 0 { 199 config.SessionTimeout = defaultSessionTimeout 200 } 201 202 if config.PartitionWatchInterval == 0 { 203 config.PartitionWatchInterval = defaultPartitionWatchTime 204 } 205 206 if config.RebalanceTimeout == 0 { 207 config.RebalanceTimeout = defaultRebalanceTimeout 208 } 209 210 if config.JoinGroupBackoff == 0 { 211 config.JoinGroupBackoff = defaultJoinGroupBackoff 212 } 213 214 if config.RetentionTime == 0 { 215 config.RetentionTime = defaultRetentionTime 216 } 217 218 if config.HeartbeatInterval < 0 || (config.HeartbeatInterval/time.Millisecond) >= math.MaxInt32 { 219 return fmt.Errorf("HeartbeatInterval out of bounds: %d", config.HeartbeatInterval) 220 } 221 222 if config.SessionTimeout < 0 || (config.SessionTimeout/time.Millisecond) >= math.MaxInt32 { 223 return fmt.Errorf("SessionTimeout out of bounds: %d", config.SessionTimeout) 224 } 225 226 if config.RebalanceTimeout < 0 || (config.RebalanceTimeout/time.Millisecond) >= math.MaxInt32 { 227 return fmt.Errorf("RebalanceTimeout out of bounds: %d", config.RebalanceTimeout) 228 } 229 230 if config.JoinGroupBackoff < 0 || (config.JoinGroupBackoff/time.Millisecond) >= math.MaxInt32 { 231 return fmt.Errorf("JoinGroupBackoff out of bounds: %d", config.JoinGroupBackoff) 232 } 233 234 if config.RetentionTime < 0 && config.RetentionTime != defaultRetentionTime { 235 return fmt.Errorf("RetentionTime out of bounds: %d", config.RetentionTime) 236 } 237 238 if config.PartitionWatchInterval < 0 || (config.PartitionWatchInterval/time.Millisecond) >= math.MaxInt32 { 239 return fmt.Errorf("PartitionWachInterval out of bounds %d", config.PartitionWatchInterval) 240 } 241 242 if config.StartOffset == 0 { 243 config.StartOffset = FirstOffset 244 } 245 246 if config.StartOffset != FirstOffset && config.StartOffset != LastOffset { 247 return fmt.Errorf("StartOffset is not valid %d", config.StartOffset) 248 } 249 250 if config.Timeout == 0 { 251 config.Timeout = defaultTimeout 252 } 253 254 if config.connect == nil { 255 config.connect = makeConnect(*config) 256 } 257 258 return nil 259 } 260 261 // PartitionAssignment represents the starting state of a partition that has 262 // been assigned to a consumer. 263 type PartitionAssignment struct { 264 // ID is the partition ID. 265 ID int 266 267 // Offset is the initial offset at which this assignment begins. It will 268 // either be an absolute offset if one has previously been committed for 269 // the consumer group or a relative offset such as FirstOffset when this 270 // is the first time the partition have been assigned to a member of the 271 // group. 272 Offset int64 273 } 274 275 // genCtx adapts the done channel of the generation to a context.Context. This 276 // is used by Generation.Start so that we can pass a context to go routines 277 // instead of passing around channels. 278 type genCtx struct { 279 gen *Generation 280 } 281 282 func (c genCtx) Done() <-chan struct{} { 283 return c.gen.done 284 } 285 286 func (c genCtx) Err() error { 287 select { 288 case <-c.gen.done: 289 return ErrGenerationEnded 290 default: 291 return nil 292 } 293 } 294 295 func (c genCtx) Deadline() (time.Time, bool) { 296 return time.Time{}, false 297 } 298 299 func (c genCtx) Value(interface{}) interface{} { 300 return nil 301 } 302 303 // Generation represents a single consumer group generation. The generation 304 // carries the topic+partition assignments for the given. It also provides 305 // facilities for committing offsets and for running functions whose lifecycles 306 // are bound to the generation. 307 type Generation struct { 308 // ID is the generation ID as assigned by the consumer group coordinator. 309 ID int32 310 311 // GroupID is the name of the consumer group. 312 GroupID string 313 314 // MemberID is the ID assigned to this consumer by the consumer group 315 // coordinator. 316 MemberID string 317 318 // Assignments is the initial state of this Generation. The partition 319 // assignments are grouped by topic. 320 Assignments map[string][]PartitionAssignment 321 322 conn coordinator 323 324 // the following fields are used for process accounting to synchronize 325 // between Start and close. lock protects all of them. done is closed 326 // when the generation is ending in order to signal that the generation 327 // should start self-desructing. closed protects against double-closing 328 // the done chan. routines is a count of running go routines that have been 329 // launched by Start. joined will be closed by the last go routine to exit. 330 lock sync.Mutex 331 done chan struct{} 332 closed bool 333 routines int 334 joined chan struct{} 335 336 retentionMillis int64 337 log func(func(Logger)) 338 logError func(func(Logger)) 339 } 340 341 // close stops the generation and waits for all functions launched via Start to 342 // terminate. 343 func (g *Generation) close() { 344 g.lock.Lock() 345 if !g.closed { 346 close(g.done) 347 g.closed = true 348 } 349 // determine whether any go routines are running that we need to wait for. 350 // waiting needs to happen outside of the critical section. 351 r := g.routines 352 g.lock.Unlock() 353 354 // NOTE: r will be zero if no go routines were ever launched. no need to 355 // wait in that case. 356 if r > 0 { 357 <-g.joined 358 } 359 } 360 361 // Start launches the provided function in a go routine and adds accounting such 362 // that when the function exits, it stops the current generation (if not 363 // already in the process of doing so). 364 // 365 // The provided function MUST support cancellation via the ctx argument and exit 366 // in a timely manner once the ctx is complete. When the context is closed, the 367 // context's Error() function will return ErrGenerationEnded. 368 // 369 // When closing out a generation, the consumer group will wait for all functions 370 // launched by Start to exit before the group can move on and join the next 371 // generation. If the function does not exit promptly, it will stop forward 372 // progress for this consumer and potentially cause consumer group membership 373 // churn. 374 func (g *Generation) Start(fn func(ctx context.Context)) { 375 g.lock.Lock() 376 defer g.lock.Unlock() 377 378 // this is an edge case: if the generation has already closed, then it's 379 // possible that the close func has already waited on outstanding go 380 // routines and exited. 381 // 382 // nonetheless, it's important to honor that the fn is invoked in case the 383 // calling function is waiting e.g. on a channel send or a WaitGroup. in 384 // such a case, fn should immediately exit because ctx.Err() will return 385 // ErrGenerationEnded. 386 if g.closed { 387 go fn(genCtx{g}) 388 return 389 } 390 391 // register that there is one more go routine that's part of this gen. 392 g.routines++ 393 394 go func() { 395 fn(genCtx{g}) 396 g.lock.Lock() 397 // shut down the generation as soon as one function exits. this is 398 // different from close() in that it doesn't wait for all go routines in 399 // the generation to exit. 400 if !g.closed { 401 close(g.done) 402 g.closed = true 403 } 404 g.routines-- 405 // if this was the last go routine in the generation, close the joined 406 // chan so that close() can exit if it's waiting. 407 if g.routines == 0 { 408 close(g.joined) 409 } 410 g.lock.Unlock() 411 }() 412 } 413 414 // CommitOffsets commits the provided topic+partition+offset combos to the 415 // consumer group coordinator. This can be used to reset the consumer to 416 // explicit offsets. 417 func (g *Generation) CommitOffsets(offsets map[string]map[int]int64) error { 418 if len(offsets) == 0 { 419 return nil 420 } 421 422 topics := make([]offsetCommitRequestV2Topic, 0, len(offsets)) 423 for topic, partitions := range offsets { 424 t := offsetCommitRequestV2Topic{Topic: topic} 425 for partition, offset := range partitions { 426 t.Partitions = append(t.Partitions, offsetCommitRequestV2Partition{ 427 Partition: int32(partition), 428 Offset: offset, 429 }) 430 } 431 topics = append(topics, t) 432 } 433 434 request := offsetCommitRequestV2{ 435 GroupID: g.GroupID, 436 GenerationID: g.ID, 437 MemberID: g.MemberID, 438 RetentionTime: g.retentionMillis, 439 Topics: topics, 440 } 441 442 _, err := g.conn.offsetCommit(request) 443 444 return err 445 } 446 447 // heartbeatLoop checks in with the consumer group coordinator at the provided 448 // interval. It exits if it ever encounters an error, which would signal the 449 // end of the generation. 450 func (g *Generation) heartbeatLoop(interval time.Duration) { 451 g.Start(func(ctx context.Context) { 452 g.log(func(l Logger) { 453 l.Printf("started heartbeat for group, %v [%v]", g.GroupID, interval) 454 }) 455 defer g.log(func(l Logger) { 456 l.Printf("stopped heartbeat for group %s\n", g.GroupID) 457 }) 458 459 ticker := time.NewTicker(interval) 460 defer ticker.Stop() 461 462 for { 463 select { 464 case <-ctx.Done(): 465 return 466 case <-ticker.C: 467 _, err := g.conn.heartbeat(heartbeatRequestV0{ 468 GroupID: g.GroupID, 469 GenerationID: g.ID, 470 MemberID: g.MemberID, 471 }) 472 if err != nil { 473 return 474 } 475 } 476 } 477 }) 478 } 479 480 // partitionWatcher queries kafka and watches for partition changes, triggering 481 // a rebalance if changes are found. Similar to heartbeat it's okay to return on 482 // error here as if you are unable to ask a broker for basic metadata you're in 483 // a bad spot and should rebalance. Commonly you will see an error here if there 484 // is a problem with the connection to the coordinator and a rebalance will 485 // establish a new connection to the coordinator. 486 func (g *Generation) partitionWatcher(interval time.Duration, topic string) { 487 g.Start(func(ctx context.Context) { 488 g.log(func(l Logger) { 489 l.Printf("started partition watcher for group, %v, topic %v [%v]", g.GroupID, topic, interval) 490 }) 491 defer g.log(func(l Logger) { 492 l.Printf("stopped partition watcher for group, %v, topic %v", g.GroupID, topic) 493 }) 494 495 ticker := time.NewTicker(interval) 496 defer ticker.Stop() 497 498 ops, err := g.conn.readPartitions(topic) 499 if err != nil { 500 g.logError(func(l Logger) { 501 l.Printf("Problem getting partitions during startup, %v\n, Returning and setting up nextGeneration", err) 502 }) 503 return 504 } 505 oParts := len(ops) 506 for { 507 select { 508 case <-ctx.Done(): 509 return 510 case <-ticker.C: 511 ops, err := g.conn.readPartitions(topic) 512 switch { 513 case err == nil, errors.Is(err, UnknownTopicOrPartition): 514 if len(ops) != oParts { 515 g.log(func(l Logger) { 516 l.Printf("Partition changes found, reblancing group: %v.", g.GroupID) 517 }) 518 return 519 } 520 521 default: 522 g.logError(func(l Logger) { 523 l.Printf("Problem getting partitions while checking for changes, %v", err) 524 }) 525 var kafkaError Error 526 if errors.As(err, &kafkaError) { 527 continue 528 } 529 // other errors imply that we lost the connection to the coordinator, so we 530 // should abort and reconnect. 531 return 532 } 533 } 534 } 535 }) 536 } 537 538 // coordinator is a subset of the functionality in Conn in order to facilitate 539 // testing the consumer group...especially for error conditions that are 540 // difficult to instigate with a live broker running in docker. 541 type coordinator interface { 542 io.Closer 543 findCoordinator(findCoordinatorRequestV0) (findCoordinatorResponseV0, error) 544 joinGroup(joinGroupRequestV1) (joinGroupResponseV1, error) 545 syncGroup(syncGroupRequestV0) (syncGroupResponseV0, error) 546 leaveGroup(leaveGroupRequestV0) (leaveGroupResponseV0, error) 547 heartbeat(heartbeatRequestV0) (heartbeatResponseV0, error) 548 offsetFetch(offsetFetchRequestV1) (offsetFetchResponseV1, error) 549 offsetCommit(offsetCommitRequestV2) (offsetCommitResponseV2, error) 550 readPartitions(...string) ([]Partition, error) 551 } 552 553 // timeoutCoordinator wraps the Conn to ensure that every operation has a 554 // deadline. Otherwise, it would be possible for requests to block indefinitely 555 // if the remote server never responds. There are many spots where the consumer 556 // group needs to interact with the broker, so it feels less error prone to 557 // factor all of the deadline management into this shared location as opposed to 558 // peppering it all through where the code actually interacts with the broker. 559 type timeoutCoordinator struct { 560 timeout time.Duration 561 sessionTimeout time.Duration 562 rebalanceTimeout time.Duration 563 conn *Conn 564 } 565 566 func (t *timeoutCoordinator) Close() error { 567 return t.conn.Close() 568 } 569 570 func (t *timeoutCoordinator) findCoordinator(req findCoordinatorRequestV0) (findCoordinatorResponseV0, error) { 571 if err := t.conn.SetDeadline(time.Now().Add(t.timeout)); err != nil { 572 return findCoordinatorResponseV0{}, err 573 } 574 return t.conn.findCoordinator(req) 575 } 576 577 func (t *timeoutCoordinator) joinGroup(req joinGroupRequestV1) (joinGroupResponseV1, error) { 578 // in the case of join group, the consumer group coordinator may wait up 579 // to rebalance timeout in order to wait for all members to join. 580 if err := t.conn.SetDeadline(time.Now().Add(t.timeout + t.rebalanceTimeout)); err != nil { 581 return joinGroupResponseV1{}, err 582 } 583 return t.conn.joinGroup(req) 584 } 585 586 func (t *timeoutCoordinator) syncGroup(req syncGroupRequestV0) (syncGroupResponseV0, error) { 587 // in the case of sync group, the consumer group leader is given up to 588 // the session timeout to respond before the coordinator will give up. 589 if err := t.conn.SetDeadline(time.Now().Add(t.timeout + t.sessionTimeout)); err != nil { 590 return syncGroupResponseV0{}, err 591 } 592 return t.conn.syncGroup(req) 593 } 594 595 func (t *timeoutCoordinator) leaveGroup(req leaveGroupRequestV0) (leaveGroupResponseV0, error) { 596 if err := t.conn.SetDeadline(time.Now().Add(t.timeout)); err != nil { 597 return leaveGroupResponseV0{}, err 598 } 599 return t.conn.leaveGroup(req) 600 } 601 602 func (t *timeoutCoordinator) heartbeat(req heartbeatRequestV0) (heartbeatResponseV0, error) { 603 if err := t.conn.SetDeadline(time.Now().Add(t.timeout)); err != nil { 604 return heartbeatResponseV0{}, err 605 } 606 return t.conn.heartbeat(req) 607 } 608 609 func (t *timeoutCoordinator) offsetFetch(req offsetFetchRequestV1) (offsetFetchResponseV1, error) { 610 if err := t.conn.SetDeadline(time.Now().Add(t.timeout)); err != nil { 611 return offsetFetchResponseV1{}, err 612 } 613 return t.conn.offsetFetch(req) 614 } 615 616 func (t *timeoutCoordinator) offsetCommit(req offsetCommitRequestV2) (offsetCommitResponseV2, error) { 617 if err := t.conn.SetDeadline(time.Now().Add(t.timeout)); err != nil { 618 return offsetCommitResponseV2{}, err 619 } 620 return t.conn.offsetCommit(req) 621 } 622 623 func (t *timeoutCoordinator) readPartitions(topics ...string) ([]Partition, error) { 624 if err := t.conn.SetDeadline(time.Now().Add(t.timeout)); err != nil { 625 return nil, err 626 } 627 return t.conn.ReadPartitions(topics...) 628 } 629 630 // NewConsumerGroup creates a new ConsumerGroup. It returns an error if the 631 // provided configuration is invalid. It does not attempt to connect to the 632 // Kafka cluster. That happens asynchronously, and any errors will be reported 633 // by Next. 634 func NewConsumerGroup(config ConsumerGroupConfig) (*ConsumerGroup, error) { 635 if err := config.Validate(); err != nil { 636 return nil, err 637 } 638 639 cg := &ConsumerGroup{ 640 config: config, 641 next: make(chan *Generation), 642 errs: make(chan error), 643 done: make(chan struct{}), 644 } 645 cg.wg.Add(1) 646 go func() { 647 cg.run() 648 cg.wg.Done() 649 }() 650 return cg, nil 651 } 652 653 // ConsumerGroup models a Kafka consumer group. A caller doesn't interact with 654 // the group directly. Rather, they interact with a Generation. Every time a 655 // member enters or exits the group, it results in a new Generation. The 656 // Generation is where partition assignments and offset management occur. 657 // Callers will use Next to get a handle to the Generation. 658 type ConsumerGroup struct { 659 config ConsumerGroupConfig 660 next chan *Generation 661 errs chan error 662 663 closeOnce sync.Once 664 wg sync.WaitGroup 665 done chan struct{} 666 } 667 668 // Close terminates the current generation by causing this member to leave and 669 // releases all local resources used to participate in the consumer group. 670 // Close will also end the current generation if it is still active. 671 func (cg *ConsumerGroup) Close() error { 672 cg.closeOnce.Do(func() { 673 close(cg.done) 674 }) 675 cg.wg.Wait() 676 return nil 677 } 678 679 // Next waits for the next consumer group generation. There will never be two 680 // active generations. Next will never return a new generation until the 681 // previous one has completed. 682 // 683 // If there are errors setting up the next generation, they will be surfaced 684 // here. 685 // 686 // If the ConsumerGroup has been closed, then Next will return ErrGroupClosed. 687 func (cg *ConsumerGroup) Next(ctx context.Context) (*Generation, error) { 688 select { 689 case <-ctx.Done(): 690 return nil, ctx.Err() 691 case <-cg.done: 692 return nil, ErrGroupClosed 693 case err := <-cg.errs: 694 return nil, err 695 case next := <-cg.next: 696 return next, nil 697 } 698 } 699 700 func (cg *ConsumerGroup) run() { 701 // the memberID is the only piece of information that is maintained across 702 // generations. it starts empty and will be assigned on the first nextGeneration 703 // when the joinGroup request is processed. it may change again later if 704 // the CG coordinator fails over or if the member is evicted. otherwise, it 705 // will be constant for the lifetime of this group. 706 var memberID string 707 var err error 708 for { 709 memberID, err = cg.nextGeneration(memberID) 710 711 // backoff will be set if this go routine should sleep before continuing 712 // to the next generation. it will be non-nil in the case of an error 713 // joining or syncing the group. 714 var backoff <-chan time.Time 715 716 switch { 717 case err == nil: 718 // no error...the previous generation finished normally. 719 continue 720 721 case errors.Is(err, ErrGroupClosed): 722 // the CG has been closed...leave the group and exit loop. 723 _ = cg.leaveGroup(memberID) 724 return 725 726 case errors.Is(err, RebalanceInProgress): 727 // in case of a RebalanceInProgress, don't leave the group or 728 // change the member ID, but report the error. the next attempt 729 // to join the group will then be subject to the rebalance 730 // timeout, so the broker will be responsible for throttling 731 // this loop. 732 733 default: 734 // leave the group and report the error if we had gotten far 735 // enough so as to have a member ID. also clear the member id 736 // so we don't attempt to use it again. in order to avoid 737 // a tight error loop, backoff before the next attempt to join 738 // the group. 739 _ = cg.leaveGroup(memberID) 740 memberID = "" 741 backoff = time.After(cg.config.JoinGroupBackoff) 742 } 743 // ensure that we exit cleanly in case the CG is done and no one is 744 // waiting to receive on the unbuffered error channel. 745 select { 746 case <-cg.done: 747 return 748 case cg.errs <- err: 749 } 750 // backoff if needed, being sure to exit cleanly if the CG is done. 751 if backoff != nil { 752 select { 753 case <-cg.done: 754 // exit cleanly if the group is closed. 755 return 756 case <-backoff: 757 } 758 } 759 } 760 } 761 762 func (cg *ConsumerGroup) nextGeneration(memberID string) (string, error) { 763 // get a new connection to the coordinator on each loop. the previous 764 // generation could have exited due to losing the connection, so this 765 // ensures that we always have a clean starting point. it means we will 766 // re-connect in certain cases, but that shouldn't be an issue given that 767 // rebalances are relatively infrequent under normal operating 768 // conditions. 769 conn, err := cg.coordinator() 770 if err != nil { 771 cg.withErrorLogger(func(log Logger) { 772 log.Printf("Unable to establish connection to consumer group coordinator for group %s: %v", cg.config.ID, err) 773 }) 774 return memberID, err // a prior memberID may still be valid, so don't return "" 775 } 776 defer conn.Close() 777 778 var generationID int32 779 var groupAssignments GroupMemberAssignments 780 var assignments map[string][]int32 781 782 // join group. this will join the group and prepare assignments if our 783 // consumer is elected leader. it may also change or assign the member ID. 784 memberID, generationID, groupAssignments, err = cg.joinGroup(conn, memberID) 785 if err != nil { 786 cg.withErrorLogger(func(log Logger) { 787 log.Printf("Failed to join group %s: %v", cg.config.ID, err) 788 }) 789 return memberID, err 790 } 791 cg.withLogger(func(log Logger) { 792 log.Printf("Joined group %s as member %s in generation %d", cg.config.ID, memberID, generationID) 793 }) 794 795 // sync group 796 assignments, err = cg.syncGroup(conn, memberID, generationID, groupAssignments) 797 if err != nil { 798 cg.withErrorLogger(func(log Logger) { 799 log.Printf("Failed to sync group %s: %v", cg.config.ID, err) 800 }) 801 return memberID, err 802 } 803 804 // fetch initial offsets. 805 var offsets map[string]map[int]int64 806 offsets, err = cg.fetchOffsets(conn, assignments) 807 if err != nil { 808 cg.withErrorLogger(func(log Logger) { 809 log.Printf("Failed to fetch offsets for group %s: %v", cg.config.ID, err) 810 }) 811 return memberID, err 812 } 813 814 // create the generation. 815 gen := Generation{ 816 ID: generationID, 817 GroupID: cg.config.ID, 818 MemberID: memberID, 819 Assignments: cg.makeAssignments(assignments, offsets), 820 conn: conn, 821 done: make(chan struct{}), 822 joined: make(chan struct{}), 823 retentionMillis: int64(cg.config.RetentionTime / time.Millisecond), 824 log: cg.withLogger, 825 logError: cg.withErrorLogger, 826 } 827 828 // spawn all of the go routines required to facilitate this generation. if 829 // any of these functions exit, then the generation is determined to be 830 // complete. 831 gen.heartbeatLoop(cg.config.HeartbeatInterval) 832 if cg.config.WatchPartitionChanges { 833 for _, topic := range cg.config.Topics { 834 gen.partitionWatcher(cg.config.PartitionWatchInterval, topic) 835 } 836 } 837 838 // make this generation available for retrieval. if the CG is closed before 839 // we can send it on the channel, exit. that case is required b/c the next 840 // channel is unbuffered. if the caller to Next has already bailed because 841 // it's own teardown logic has been invoked, this would deadlock otherwise. 842 select { 843 case <-cg.done: 844 gen.close() 845 return memberID, ErrGroupClosed // ErrGroupClosed will trigger leave logic. 846 case cg.next <- &gen: 847 } 848 849 // wait for generation to complete. if the CG is closed before the 850 // generation is finished, exit and leave the group. 851 select { 852 case <-cg.done: 853 gen.close() 854 return memberID, ErrGroupClosed // ErrGroupClosed will trigger leave logic. 855 case <-gen.done: 856 // time for next generation! make sure all the current go routines exit 857 // before continuing onward. 858 gen.close() 859 return memberID, nil 860 } 861 } 862 863 // connect returns a connection to ANY broker. 864 func makeConnect(config ConsumerGroupConfig) func(dialer *Dialer, brokers ...string) (coordinator, error) { 865 return func(dialer *Dialer, brokers ...string) (coordinator, error) { 866 var err error 867 for _, broker := range brokers { 868 var conn *Conn 869 if conn, err = dialer.Dial("tcp", broker); err == nil { 870 return &timeoutCoordinator{ 871 conn: conn, 872 timeout: config.Timeout, 873 sessionTimeout: config.SessionTimeout, 874 rebalanceTimeout: config.RebalanceTimeout, 875 }, nil 876 } 877 } 878 return nil, err // err will be non-nil 879 } 880 } 881 882 // coordinator establishes a connection to the coordinator for this consumer 883 // group. 884 func (cg *ConsumerGroup) coordinator() (coordinator, error) { 885 // NOTE : could try to cache the coordinator to avoid the double connect 886 // here. since consumer group balances happen infrequently and are 887 // an expensive operation, we're not currently optimizing that case 888 // in order to keep the code simpler. 889 conn, err := cg.config.connect(cg.config.Dialer, cg.config.Brokers...) 890 if err != nil { 891 return nil, err 892 } 893 defer conn.Close() 894 895 out, err := conn.findCoordinator(findCoordinatorRequestV0{ 896 CoordinatorKey: cg.config.ID, 897 }) 898 if err == nil && out.ErrorCode != 0 { 899 err = Error(out.ErrorCode) 900 } 901 if err != nil { 902 return nil, err 903 } 904 905 address := net.JoinHostPort(out.Coordinator.Host, strconv.Itoa(int(out.Coordinator.Port))) 906 return cg.config.connect(cg.config.Dialer, address) 907 } 908 909 // joinGroup attempts to join the reader to the consumer group. 910 // Returns GroupMemberAssignments is this Reader was selected as 911 // the leader. Otherwise, GroupMemberAssignments will be nil. 912 // 913 // Possible kafka error codes returned: 914 // * GroupLoadInProgress: 915 // * GroupCoordinatorNotAvailable: 916 // * NotCoordinatorForGroup: 917 // * InconsistentGroupProtocol: 918 // * InvalidSessionTimeout: 919 // * GroupAuthorizationFailed: 920 func (cg *ConsumerGroup) joinGroup(conn coordinator, memberID string) (string, int32, GroupMemberAssignments, error) { 921 request, err := cg.makeJoinGroupRequestV1(memberID) 922 if err != nil { 923 return "", 0, nil, err 924 } 925 926 response, err := conn.joinGroup(request) 927 if err == nil && response.ErrorCode != 0 { 928 err = Error(response.ErrorCode) 929 } 930 if err != nil { 931 return "", 0, nil, err 932 } 933 934 memberID = response.MemberID 935 generationID := response.GenerationID 936 937 cg.withLogger(func(l Logger) { 938 l.Printf("joined group %s as member %s in generation %d", cg.config.ID, memberID, generationID) 939 }) 940 941 var assignments GroupMemberAssignments 942 if iAmLeader := response.MemberID == response.LeaderID; iAmLeader { 943 v, err := cg.assignTopicPartitions(conn, response) 944 if err != nil { 945 return memberID, 0, nil, err 946 } 947 assignments = v 948 949 cg.withLogger(func(l Logger) { 950 for memberID, assignment := range assignments { 951 for topic, partitions := range assignment { 952 l.Printf("assigned member/topic/partitions %v/%v/%v", memberID, topic, partitions) 953 } 954 } 955 }) 956 } 957 958 cg.withLogger(func(l Logger) { 959 l.Printf("joinGroup succeeded for response, %v. generationID=%v, memberID=%v", cg.config.ID, response.GenerationID, response.MemberID) 960 }) 961 962 return memberID, generationID, assignments, nil 963 } 964 965 // makeJoinGroupRequestV1 handles the logic of constructing a joinGroup 966 // request. 967 func (cg *ConsumerGroup) makeJoinGroupRequestV1(memberID string) (joinGroupRequestV1, error) { 968 request := joinGroupRequestV1{ 969 GroupID: cg.config.ID, 970 MemberID: memberID, 971 SessionTimeout: int32(cg.config.SessionTimeout / time.Millisecond), 972 RebalanceTimeout: int32(cg.config.RebalanceTimeout / time.Millisecond), 973 ProtocolType: defaultProtocolType, 974 } 975 976 for _, balancer := range cg.config.GroupBalancers { 977 userData, err := balancer.UserData() 978 if err != nil { 979 return joinGroupRequestV1{}, fmt.Errorf("unable to construct protocol metadata for member, %v: %w", balancer.ProtocolName(), err) 980 } 981 request.GroupProtocols = append(request.GroupProtocols, joinGroupRequestGroupProtocolV1{ 982 ProtocolName: balancer.ProtocolName(), 983 ProtocolMetadata: groupMetadata{ 984 Version: 1, 985 Topics: cg.config.Topics, 986 UserData: userData, 987 }.bytes(), 988 }) 989 } 990 991 return request, nil 992 } 993 994 // assignTopicPartitions uses the selected GroupBalancer to assign members to 995 // their various partitions. 996 func (cg *ConsumerGroup) assignTopicPartitions(conn coordinator, group joinGroupResponseV1) (GroupMemberAssignments, error) { 997 cg.withLogger(func(l Logger) { 998 l.Printf("selected as leader for group, %s\n", cg.config.ID) 999 }) 1000 1001 balancer, ok := findGroupBalancer(group.GroupProtocol, cg.config.GroupBalancers) 1002 if !ok { 1003 // NOTE : this shouldn't happen in practice...the broker should not 1004 // return successfully from joinGroup unless all members support 1005 // at least one common protocol. 1006 return nil, fmt.Errorf("unable to find selected balancer, %v, for group, %v", group.GroupProtocol, cg.config.ID) 1007 } 1008 1009 members, err := cg.makeMemberProtocolMetadata(group.Members) 1010 if err != nil { 1011 return nil, err 1012 } 1013 1014 topics := extractTopics(members) 1015 partitions, err := conn.readPartitions(topics...) 1016 1017 // it's not a failure if the topic doesn't exist yet. it results in no 1018 // assignments for the topic. this matches the behavior of the official 1019 // clients: java, python, and librdkafka. 1020 // a topic watcher can trigger a rebalance when the topic comes into being. 1021 if err != nil && !errors.Is(err, UnknownTopicOrPartition) { 1022 return nil, err 1023 } 1024 1025 cg.withLogger(func(l Logger) { 1026 l.Printf("using '%v' balancer to assign group, %v", group.GroupProtocol, cg.config.ID) 1027 for _, member := range members { 1028 l.Printf("found member: %v/%#v", member.ID, member.UserData) 1029 } 1030 for _, partition := range partitions { 1031 l.Printf("found topic/partition: %v/%v", partition.Topic, partition.ID) 1032 } 1033 }) 1034 1035 return balancer.AssignGroups(members, partitions), nil 1036 } 1037 1038 // makeMemberProtocolMetadata maps encoded member metadata ([]byte) into []GroupMember. 1039 func (cg *ConsumerGroup) makeMemberProtocolMetadata(in []joinGroupResponseMemberV1) ([]GroupMember, error) { 1040 members := make([]GroupMember, 0, len(in)) 1041 for _, item := range in { 1042 metadata := groupMetadata{} 1043 reader := bufio.NewReader(bytes.NewReader(item.MemberMetadata)) 1044 if remain, err := (&metadata).readFrom(reader, len(item.MemberMetadata)); err != nil || remain != 0 { 1045 return nil, fmt.Errorf("unable to read metadata for member, %v: %w", item.MemberID, err) 1046 } 1047 1048 members = append(members, GroupMember{ 1049 ID: item.MemberID, 1050 Topics: metadata.Topics, 1051 UserData: metadata.UserData, 1052 }) 1053 } 1054 return members, nil 1055 } 1056 1057 // syncGroup completes the consumer group nextGeneration by accepting the 1058 // memberAssignments (if this Reader is the leader) and returning this 1059 // Readers subscriptions topic => partitions 1060 // 1061 // Possible kafka error codes returned: 1062 // * GroupCoordinatorNotAvailable: 1063 // * NotCoordinatorForGroup: 1064 // * IllegalGeneration: 1065 // * RebalanceInProgress: 1066 // * GroupAuthorizationFailed: 1067 func (cg *ConsumerGroup) syncGroup(conn coordinator, memberID string, generationID int32, memberAssignments GroupMemberAssignments) (map[string][]int32, error) { 1068 request := cg.makeSyncGroupRequestV0(memberID, generationID, memberAssignments) 1069 response, err := conn.syncGroup(request) 1070 if err == nil && response.ErrorCode != 0 { 1071 err = Error(response.ErrorCode) 1072 } 1073 if err != nil { 1074 return nil, err 1075 } 1076 1077 assignments := groupAssignment{} 1078 reader := bufio.NewReader(bytes.NewReader(response.MemberAssignments)) 1079 if _, err := (&assignments).readFrom(reader, len(response.MemberAssignments)); err != nil { 1080 return nil, err 1081 } 1082 1083 if len(assignments.Topics) == 0 { 1084 cg.withLogger(func(l Logger) { 1085 l.Printf("received empty assignments for group, %v as member %s for generation %d", cg.config.ID, memberID, generationID) 1086 }) 1087 } 1088 1089 cg.withLogger(func(l Logger) { 1090 l.Printf("sync group finished for group, %v", cg.config.ID) 1091 }) 1092 1093 return assignments.Topics, nil 1094 } 1095 1096 func (cg *ConsumerGroup) makeSyncGroupRequestV0(memberID string, generationID int32, memberAssignments GroupMemberAssignments) syncGroupRequestV0 { 1097 request := syncGroupRequestV0{ 1098 GroupID: cg.config.ID, 1099 GenerationID: generationID, 1100 MemberID: memberID, 1101 } 1102 1103 if memberAssignments != nil { 1104 request.GroupAssignments = make([]syncGroupRequestGroupAssignmentV0, 0, 1) 1105 1106 for memberID, topics := range memberAssignments { 1107 topics32 := make(map[string][]int32) 1108 for topic, partitions := range topics { 1109 partitions32 := make([]int32, len(partitions)) 1110 for i := range partitions { 1111 partitions32[i] = int32(partitions[i]) 1112 } 1113 topics32[topic] = partitions32 1114 } 1115 request.GroupAssignments = append(request.GroupAssignments, syncGroupRequestGroupAssignmentV0{ 1116 MemberID: memberID, 1117 MemberAssignments: groupAssignment{ 1118 Version: 1, 1119 Topics: topics32, 1120 }.bytes(), 1121 }) 1122 } 1123 1124 cg.withLogger(func(logger Logger) { 1125 logger.Printf("Syncing %d assignments for generation %d as member %s", len(request.GroupAssignments), generationID, memberID) 1126 }) 1127 } 1128 1129 return request 1130 } 1131 1132 func (cg *ConsumerGroup) fetchOffsets(conn coordinator, subs map[string][]int32) (map[string]map[int]int64, error) { 1133 req := offsetFetchRequestV1{ 1134 GroupID: cg.config.ID, 1135 Topics: make([]offsetFetchRequestV1Topic, 0, len(cg.config.Topics)), 1136 } 1137 for _, topic := range cg.config.Topics { 1138 req.Topics = append(req.Topics, offsetFetchRequestV1Topic{ 1139 Topic: topic, 1140 Partitions: subs[topic], 1141 }) 1142 } 1143 offsets, err := conn.offsetFetch(req) 1144 if err != nil { 1145 return nil, err 1146 } 1147 1148 offsetsByTopic := make(map[string]map[int]int64) 1149 for _, res := range offsets.Responses { 1150 offsetsByPartition := map[int]int64{} 1151 offsetsByTopic[res.Topic] = offsetsByPartition 1152 for _, pr := range res.PartitionResponses { 1153 for _, partition := range subs[res.Topic] { 1154 if partition == pr.Partition { 1155 offset := pr.Offset 1156 if offset < 0 { 1157 offset = cg.config.StartOffset 1158 } 1159 offsetsByPartition[int(partition)] = offset 1160 } 1161 } 1162 } 1163 } 1164 1165 return offsetsByTopic, nil 1166 } 1167 1168 func (cg *ConsumerGroup) makeAssignments(assignments map[string][]int32, offsets map[string]map[int]int64) map[string][]PartitionAssignment { 1169 topicAssignments := make(map[string][]PartitionAssignment) 1170 for _, topic := range cg.config.Topics { 1171 topicPartitions := assignments[topic] 1172 topicAssignments[topic] = make([]PartitionAssignment, 0, len(topicPartitions)) 1173 for _, partition := range topicPartitions { 1174 var offset int64 1175 partitionOffsets, ok := offsets[topic] 1176 if ok { 1177 offset, ok = partitionOffsets[int(partition)] 1178 } 1179 if !ok { 1180 offset = cg.config.StartOffset 1181 } 1182 topicAssignments[topic] = append(topicAssignments[topic], PartitionAssignment{ 1183 ID: int(partition), 1184 Offset: offset, 1185 }) 1186 } 1187 } 1188 return topicAssignments 1189 } 1190 1191 func (cg *ConsumerGroup) leaveGroup(memberID string) error { 1192 // don't attempt to leave the group if no memberID was ever assigned. 1193 if memberID == "" { 1194 return nil 1195 } 1196 1197 cg.withLogger(func(log Logger) { 1198 log.Printf("Leaving group %s, member %s", cg.config.ID, memberID) 1199 }) 1200 1201 // IMPORTANT : leaveGroup establishes its own connection to the coordinator 1202 // because it is often called after some other operation failed. 1203 // said failure could be the result of connection-level issues, 1204 // so we want to re-establish the connection to ensure that we 1205 // are able to process the cleanup step. 1206 coordinator, err := cg.coordinator() 1207 if err != nil { 1208 return err 1209 } 1210 1211 _, err = coordinator.leaveGroup(leaveGroupRequestV0{ 1212 GroupID: cg.config.ID, 1213 MemberID: memberID, 1214 }) 1215 if err != nil { 1216 cg.withErrorLogger(func(log Logger) { 1217 log.Printf("leave group failed for group, %v, and member, %v: %v", cg.config.ID, memberID, err) 1218 }) 1219 } 1220 1221 _ = coordinator.Close() 1222 1223 return err 1224 } 1225 1226 func (cg *ConsumerGroup) withLogger(do func(Logger)) { 1227 if cg.config.Logger != nil { 1228 do(cg.config.Logger) 1229 } 1230 } 1231 1232 func (cg *ConsumerGroup) withErrorLogger(do func(Logger)) { 1233 if cg.config.ErrorLogger != nil { 1234 do(cg.config.ErrorLogger) 1235 } else { 1236 cg.withLogger(do) 1237 } 1238 }