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