consumer_group.go 23 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676677678679680681682683684685686687688689690691692693694695696697698699700701702703704705706707708709710711712713714715716717718719720721722723724725726727728729730731732733734735736737738739740741742743744745746747748749750751752753754755756757758759760761762763764765766767768769770771772773774775776777778779780781782783784785786787788789790791792793794795796797798799800801802803804805806807808809810811812813814815816817818819
  1. package sarama
  2. import (
  3. "context"
  4. "errors"
  5. "fmt"
  6. "sort"
  7. "sync"
  8. "time"
  9. )
  10. // ErrClosedConsumerGroup is the error returned when a method is called on a consumer group that has been closed.
  11. var ErrClosedConsumerGroup = errors.New("kafka: tried to use a consumer group that was closed")
  12. // ConsumerGroup is responsible for dividing up processing of topics and partitions
  13. // over a collection of processes (the members of the consumer group).
  14. type ConsumerGroup interface {
  15. // Consume joins a cluster of consumers for a given list of topics and
  16. // starts a blocking ConsumerGroupSession through the ConsumerGroupHandler.
  17. //
  18. // The life-cycle of a session is represented by the following steps:
  19. //
  20. // 1. The consumers join the group (as explained in https://kafka.apache.org/documentation/#intro_consumers)
  21. // and is assigned their "fair share" of partitions, aka 'claims'.
  22. // 2. Before processing starts, the handler's Setup() hook is called to notify the user
  23. // of the claims and allow any necessary preparation or alteration of state.
  24. // 3. For each of the assigned claims the handler's ConsumeClaim() function is then called
  25. // in a separate goroutine which requires it to be thread-safe. Any state must be carefully protected
  26. // from concurrent reads/writes.
  27. // 4. The session will persist until one of the ConsumeClaim() functions exits. This can be either when the
  28. // parent context is cancelled or when a server-side rebalance cycle is initiated.
  29. // 5. Once all the ConsumeClaim() loops have exited, the handler's Cleanup() hook is called
  30. // to allow the user to perform any final tasks before a rebalance.
  31. // 6. Finally, marked offsets are committed one last time before claims are released.
  32. //
  33. // Please note, that once a rebalance is triggered, sessions must be completed within
  34. // Config.Consumer.Group.Rebalance.Timeout. This means that ConsumeClaim() functions must exit
  35. // as quickly as possible to allow time for Cleanup() and the final offset commit. If the timeout
  36. // is exceeded, the consumer will be removed from the group by Kafka, which will cause offset
  37. // commit failures.
  38. Consume(ctx context.Context, topics []string, handler ConsumerGroupHandler) error
  39. // Errors returns a read channel of errors that occurred during the consumer life-cycle.
  40. // By default, errors are logged and not returned over this channel.
  41. // If you want to implement any custom error handling, set your config's
  42. // Consumer.Return.Errors setting to true, and read from this channel.
  43. Errors() <-chan error
  44. // Close stops the ConsumerGroup and detaches any running sessions. It is required to call
  45. // this function before the object passes out of scope, as it will otherwise leak memory.
  46. Close() error
  47. }
  48. type consumerGroup struct {
  49. client Client
  50. config *Config
  51. consumer Consumer
  52. groupID string
  53. memberID string
  54. errors chan error
  55. lock sync.Mutex
  56. closed chan none
  57. closeOnce sync.Once
  58. userData []byte
  59. }
  60. // NewConsumerGroup creates a new consumer group the given broker addresses and configuration.
  61. func NewConsumerGroup(addrs []string, groupID string, config *Config) (ConsumerGroup, error) {
  62. client, err := NewClient(addrs, config)
  63. if err != nil {
  64. return nil, err
  65. }
  66. c, err := newConsumerGroup(groupID, client)
  67. if err != nil {
  68. _ = client.Close()
  69. }
  70. return c, err
  71. }
  72. // NewConsumerGroupFromClient creates a new consumer group using the given client. It is still
  73. // necessary to call Close() on the underlying client when shutting down this consumer.
  74. // PLEASE NOTE: consumer groups can only re-use but not share clients.
  75. func NewConsumerGroupFromClient(groupID string, client Client) (ConsumerGroup, error) {
  76. // For clients passed in by the client, ensure we don't
  77. // call Close() on it.
  78. cli := &nopCloserClient{client}
  79. return newConsumerGroup(groupID, cli)
  80. }
  81. func newConsumerGroup(groupID string, client Client) (ConsumerGroup, error) {
  82. config := client.Config()
  83. if !config.Version.IsAtLeast(V0_10_2_0) {
  84. return nil, ConfigurationError("consumer groups require Version to be >= V0_10_2_0")
  85. }
  86. consumer, err := NewConsumerFromClient(client)
  87. if err != nil {
  88. return nil, err
  89. }
  90. return &consumerGroup{
  91. client: client,
  92. consumer: consumer,
  93. config: config,
  94. groupID: groupID,
  95. errors: make(chan error, config.ChannelBufferSize),
  96. closed: make(chan none),
  97. }, nil
  98. }
  99. // Errors implements ConsumerGroup.
  100. func (c *consumerGroup) Errors() <-chan error { return c.errors }
  101. // Close implements ConsumerGroup.
  102. func (c *consumerGroup) Close() (err error) {
  103. c.closeOnce.Do(func() {
  104. close(c.closed)
  105. c.lock.Lock()
  106. defer c.lock.Unlock()
  107. // leave group
  108. if e := c.leave(); e != nil {
  109. err = e
  110. }
  111. // drain errors
  112. go func() {
  113. close(c.errors)
  114. }()
  115. for e := range c.errors {
  116. err = e
  117. }
  118. if e := c.client.Close(); e != nil {
  119. err = e
  120. }
  121. })
  122. return
  123. }
  124. // Consume implements ConsumerGroup.
  125. func (c *consumerGroup) Consume(ctx context.Context, topics []string, handler ConsumerGroupHandler) error {
  126. // Ensure group is not closed
  127. select {
  128. case <-c.closed:
  129. return ErrClosedConsumerGroup
  130. default:
  131. }
  132. c.lock.Lock()
  133. defer c.lock.Unlock()
  134. // Quick exit when no topics are provided
  135. if len(topics) == 0 {
  136. return fmt.Errorf("no topics provided")
  137. }
  138. // Refresh metadata for requested topics
  139. if err := c.client.RefreshMetadata(topics...); err != nil {
  140. return err
  141. }
  142. // Init session
  143. sess, err := c.newSession(ctx, topics, handler, c.config.Consumer.Group.Rebalance.Retry.Max)
  144. if err == ErrClosedClient {
  145. return ErrClosedConsumerGroup
  146. } else if err != nil {
  147. return err
  148. }
  149. // Wait for session exit signal
  150. <-sess.ctx.Done()
  151. // Gracefully release session claims
  152. return sess.release(true)
  153. }
  154. func (c *consumerGroup) retryNewSession(ctx context.Context, topics []string, handler ConsumerGroupHandler, retries int, refreshCoordinator bool) (*consumerGroupSession, error) {
  155. select {
  156. case <-c.closed:
  157. return nil, ErrClosedConsumerGroup
  158. case <-time.After(c.config.Consumer.Group.Rebalance.Retry.Backoff):
  159. }
  160. if refreshCoordinator {
  161. err := c.client.RefreshCoordinator(c.groupID)
  162. if err != nil {
  163. return c.retryNewSession(ctx, topics, handler, retries, true)
  164. }
  165. }
  166. return c.newSession(ctx, topics, handler, retries-1)
  167. }
  168. func (c *consumerGroup) newSession(ctx context.Context, topics []string, handler ConsumerGroupHandler, retries int) (*consumerGroupSession, error) {
  169. coordinator, err := c.client.Coordinator(c.groupID)
  170. if err != nil {
  171. if retries <= 0 {
  172. return nil, err
  173. }
  174. return c.retryNewSession(ctx, topics, handler, retries, true)
  175. }
  176. // Join consumer group
  177. join, err := c.joinGroupRequest(coordinator, topics)
  178. if err != nil {
  179. _ = coordinator.Close()
  180. return nil, err
  181. }
  182. switch join.Err {
  183. case ErrNoError:
  184. c.memberID = join.MemberId
  185. case ErrUnknownMemberId, ErrIllegalGeneration: // reset member ID and retry immediately
  186. c.memberID = ""
  187. return c.newSession(ctx, topics, handler, retries)
  188. case ErrNotCoordinatorForConsumer: // retry after backoff with coordinator refresh
  189. if retries <= 0 {
  190. return nil, join.Err
  191. }
  192. return c.retryNewSession(ctx, topics, handler, retries, true)
  193. case ErrRebalanceInProgress: // retry after backoff
  194. if retries <= 0 {
  195. return nil, join.Err
  196. }
  197. return c.retryNewSession(ctx, topics, handler, retries, false)
  198. default:
  199. return nil, join.Err
  200. }
  201. // Prepare distribution plan if we joined as the leader
  202. var plan BalanceStrategyPlan
  203. if join.LeaderId == join.MemberId {
  204. members, err := join.GetMembers()
  205. if err != nil {
  206. return nil, err
  207. }
  208. plan, err = c.balance(members)
  209. if err != nil {
  210. return nil, err
  211. }
  212. }
  213. // Sync consumer group
  214. sync, err := c.syncGroupRequest(coordinator, plan, join.GenerationId)
  215. if err != nil {
  216. _ = coordinator.Close()
  217. return nil, err
  218. }
  219. switch sync.Err {
  220. case ErrNoError:
  221. case ErrUnknownMemberId, ErrIllegalGeneration: // reset member ID and retry immediately
  222. c.memberID = ""
  223. return c.newSession(ctx, topics, handler, retries)
  224. case ErrNotCoordinatorForConsumer: // retry after backoff with coordinator refresh
  225. if retries <= 0 {
  226. return nil, sync.Err
  227. }
  228. return c.retryNewSession(ctx, topics, handler, retries, true)
  229. case ErrRebalanceInProgress: // retry after backoff
  230. if retries <= 0 {
  231. return nil, sync.Err
  232. }
  233. return c.retryNewSession(ctx, topics, handler, retries, false)
  234. default:
  235. return nil, sync.Err
  236. }
  237. // Retrieve and sort claims
  238. var claims map[string][]int32
  239. if len(sync.MemberAssignment) > 0 {
  240. members, err := sync.GetMemberAssignment()
  241. if err != nil {
  242. return nil, err
  243. }
  244. claims = members.Topics
  245. c.userData = members.UserData
  246. for _, partitions := range claims {
  247. sort.Sort(int32Slice(partitions))
  248. }
  249. }
  250. return newConsumerGroupSession(ctx, c, claims, join.MemberId, join.GenerationId, handler)
  251. }
  252. func (c *consumerGroup) joinGroupRequest(coordinator *Broker, topics []string) (*JoinGroupResponse, error) {
  253. req := &JoinGroupRequest{
  254. GroupId: c.groupID,
  255. MemberId: c.memberID,
  256. SessionTimeout: int32(c.config.Consumer.Group.Session.Timeout / time.Millisecond),
  257. ProtocolType: "consumer",
  258. }
  259. if c.config.Version.IsAtLeast(V0_10_1_0) {
  260. req.Version = 1
  261. req.RebalanceTimeout = int32(c.config.Consumer.Group.Rebalance.Timeout / time.Millisecond)
  262. }
  263. // use static user-data if configured, otherwise use consumer-group userdata from the last sync
  264. userData := c.config.Consumer.Group.Member.UserData
  265. if len(userData) == 0 {
  266. userData = c.userData
  267. }
  268. meta := &ConsumerGroupMemberMetadata{
  269. Topics: topics,
  270. UserData: userData,
  271. }
  272. strategy := c.config.Consumer.Group.Rebalance.Strategy
  273. if err := req.AddGroupProtocolMetadata(strategy.Name(), meta); err != nil {
  274. return nil, err
  275. }
  276. return coordinator.JoinGroup(req)
  277. }
  278. func (c *consumerGroup) syncGroupRequest(coordinator *Broker, plan BalanceStrategyPlan, generationID int32) (*SyncGroupResponse, error) {
  279. req := &SyncGroupRequest{
  280. GroupId: c.groupID,
  281. MemberId: c.memberID,
  282. GenerationId: generationID,
  283. }
  284. for memberID, topics := range plan {
  285. assignment := &ConsumerGroupMemberAssignment{Topics: topics}
  286. // Include topic assignments in group-assignment userdata for each consumer-group member
  287. if c.config.Consumer.Group.Rebalance.Strategy.Name() == StickyBalanceStrategyName {
  288. userDataBytes, err := encode(&StickyAssignorUserDataV1{
  289. Topics: topics,
  290. Generation: generationID,
  291. }, nil)
  292. if err != nil {
  293. return nil, err
  294. }
  295. assignment.UserData = userDataBytes
  296. }
  297. if err := req.AddGroupAssignmentMember(memberID, assignment); err != nil {
  298. return nil, err
  299. }
  300. }
  301. return coordinator.SyncGroup(req)
  302. }
  303. func (c *consumerGroup) heartbeatRequest(coordinator *Broker, memberID string, generationID int32) (*HeartbeatResponse, error) {
  304. req := &HeartbeatRequest{
  305. GroupId: c.groupID,
  306. MemberId: memberID,
  307. GenerationId: generationID,
  308. }
  309. return coordinator.Heartbeat(req)
  310. }
  311. func (c *consumerGroup) balance(members map[string]ConsumerGroupMemberMetadata) (BalanceStrategyPlan, error) {
  312. topics := make(map[string][]int32)
  313. for _, meta := range members {
  314. for _, topic := range meta.Topics {
  315. topics[topic] = nil
  316. }
  317. }
  318. for topic := range topics {
  319. partitions, err := c.client.Partitions(topic)
  320. if err != nil {
  321. return nil, err
  322. }
  323. topics[topic] = partitions
  324. }
  325. strategy := c.config.Consumer.Group.Rebalance.Strategy
  326. return strategy.Plan(members, topics)
  327. }
  328. // Leaves the cluster, called by Close, protected by lock.
  329. func (c *consumerGroup) leave() error {
  330. if c.memberID == "" {
  331. return nil
  332. }
  333. coordinator, err := c.client.Coordinator(c.groupID)
  334. if err != nil {
  335. return err
  336. }
  337. resp, err := coordinator.LeaveGroup(&LeaveGroupRequest{
  338. GroupId: c.groupID,
  339. MemberId: c.memberID,
  340. })
  341. if err != nil {
  342. _ = coordinator.Close()
  343. return err
  344. }
  345. // Unset memberID
  346. c.memberID = ""
  347. // Check response
  348. switch resp.Err {
  349. case ErrRebalanceInProgress, ErrUnknownMemberId, ErrNoError:
  350. return nil
  351. default:
  352. return resp.Err
  353. }
  354. }
  355. func (c *consumerGroup) handleError(err error, topic string, partition int32) {
  356. select {
  357. case <-c.closed:
  358. return
  359. default:
  360. }
  361. if _, ok := err.(*ConsumerError); !ok && topic != "" && partition > -1 {
  362. err = &ConsumerError{
  363. Topic: topic,
  364. Partition: partition,
  365. Err: err,
  366. }
  367. }
  368. if c.config.Consumer.Return.Errors {
  369. select {
  370. case c.errors <- err:
  371. default:
  372. }
  373. } else {
  374. Logger.Println(err)
  375. }
  376. }
  377. // --------------------------------------------------------------------
  378. // ConsumerGroupSession represents a consumer group member session.
  379. type ConsumerGroupSession interface {
  380. // Claims returns information about the claimed partitions by topic.
  381. Claims() map[string][]int32
  382. // MemberID returns the cluster member ID.
  383. MemberID() string
  384. // GenerationID returns the current generation ID.
  385. GenerationID() int32
  386. // MarkOffset marks the provided offset, alongside a metadata string
  387. // that represents the state of the partition consumer at that point in time. The
  388. // metadata string can be used by another consumer to restore that state, so it
  389. // can resume consumption.
  390. //
  391. // To follow upstream conventions, you are expected to mark the offset of the
  392. // next message to read, not the last message read. Thus, when calling `MarkOffset`
  393. // you should typically add one to the offset of the last consumed message.
  394. //
  395. // Note: calling MarkOffset does not necessarily commit the offset to the backend
  396. // store immediately for efficiency reasons, and it may never be committed if
  397. // your application crashes. This means that you may end up processing the same
  398. // message twice, and your processing should ideally be idempotent.
  399. MarkOffset(topic string, partition int32, offset int64, metadata string)
  400. // ResetOffset resets to the provided offset, alongside a metadata string that
  401. // represents the state of the partition consumer at that point in time. Reset
  402. // acts as a counterpart to MarkOffset, the difference being that it allows to
  403. // reset an offset to an earlier or smaller value, where MarkOffset only
  404. // allows incrementing the offset. cf MarkOffset for more details.
  405. ResetOffset(topic string, partition int32, offset int64, metadata string)
  406. // MarkMessage marks a message as consumed.
  407. MarkMessage(msg *ConsumerMessage, metadata string)
  408. // Context returns the session context.
  409. Context() context.Context
  410. }
  411. type consumerGroupSession struct {
  412. parent *consumerGroup
  413. memberID string
  414. generationID int32
  415. handler ConsumerGroupHandler
  416. claims map[string][]int32
  417. offsets *offsetManager
  418. ctx context.Context
  419. cancel func()
  420. waitGroup sync.WaitGroup
  421. releaseOnce sync.Once
  422. hbDying, hbDead chan none
  423. }
  424. func newConsumerGroupSession(ctx context.Context, parent *consumerGroup, claims map[string][]int32, memberID string, generationID int32, handler ConsumerGroupHandler) (*consumerGroupSession, error) {
  425. // init offset manager
  426. offsets, err := newOffsetManagerFromClient(parent.groupID, memberID, generationID, parent.client)
  427. if err != nil {
  428. return nil, err
  429. }
  430. // init context
  431. ctx, cancel := context.WithCancel(ctx)
  432. // init session
  433. sess := &consumerGroupSession{
  434. parent: parent,
  435. memberID: memberID,
  436. generationID: generationID,
  437. handler: handler,
  438. offsets: offsets,
  439. claims: claims,
  440. ctx: ctx,
  441. cancel: cancel,
  442. hbDying: make(chan none),
  443. hbDead: make(chan none),
  444. }
  445. // start heartbeat loop
  446. go sess.heartbeatLoop()
  447. // create a POM for each claim
  448. for topic, partitions := range claims {
  449. for _, partition := range partitions {
  450. pom, err := offsets.ManagePartition(topic, partition)
  451. if err != nil {
  452. _ = sess.release(false)
  453. return nil, err
  454. }
  455. // handle POM errors
  456. go func(topic string, partition int32) {
  457. for err := range pom.Errors() {
  458. sess.parent.handleError(err, topic, partition)
  459. }
  460. }(topic, partition)
  461. }
  462. }
  463. // perform setup
  464. if err := handler.Setup(sess); err != nil {
  465. _ = sess.release(true)
  466. return nil, err
  467. }
  468. // start consuming
  469. for topic, partitions := range claims {
  470. for _, partition := range partitions {
  471. sess.waitGroup.Add(1)
  472. go func(topic string, partition int32) {
  473. defer sess.waitGroup.Done()
  474. // cancel the as session as soon as the first
  475. // goroutine exits
  476. defer sess.cancel()
  477. // consume a single topic/partition, blocking
  478. sess.consume(topic, partition)
  479. }(topic, partition)
  480. }
  481. }
  482. return sess, nil
  483. }
  484. func (s *consumerGroupSession) Claims() map[string][]int32 { return s.claims }
  485. func (s *consumerGroupSession) MemberID() string { return s.memberID }
  486. func (s *consumerGroupSession) GenerationID() int32 { return s.generationID }
  487. func (s *consumerGroupSession) MarkOffset(topic string, partition int32, offset int64, metadata string) {
  488. if pom := s.offsets.findPOM(topic, partition); pom != nil {
  489. pom.MarkOffset(offset, metadata)
  490. }
  491. }
  492. func (s *consumerGroupSession) ResetOffset(topic string, partition int32, offset int64, metadata string) {
  493. if pom := s.offsets.findPOM(topic, partition); pom != nil {
  494. pom.ResetOffset(offset, metadata)
  495. }
  496. }
  497. func (s *consumerGroupSession) MarkMessage(msg *ConsumerMessage, metadata string) {
  498. s.MarkOffset(msg.Topic, msg.Partition, msg.Offset+1, metadata)
  499. }
  500. func (s *consumerGroupSession) Context() context.Context {
  501. return s.ctx
  502. }
  503. func (s *consumerGroupSession) consume(topic string, partition int32) {
  504. // quick exit if rebalance is due
  505. select {
  506. case <-s.ctx.Done():
  507. return
  508. case <-s.parent.closed:
  509. return
  510. default:
  511. }
  512. // get next offset
  513. offset := s.parent.config.Consumer.Offsets.Initial
  514. if pom := s.offsets.findPOM(topic, partition); pom != nil {
  515. offset, _ = pom.NextOffset()
  516. }
  517. // create new claim
  518. claim, err := newConsumerGroupClaim(s, topic, partition, offset)
  519. if err != nil {
  520. s.parent.handleError(err, topic, partition)
  521. return
  522. }
  523. // handle errors
  524. go func() {
  525. for err := range claim.Errors() {
  526. s.parent.handleError(err, topic, partition)
  527. }
  528. }()
  529. // trigger close when session is done
  530. go func() {
  531. select {
  532. case <-s.ctx.Done():
  533. case <-s.parent.closed:
  534. }
  535. claim.AsyncClose()
  536. }()
  537. // start processing
  538. if err := s.handler.ConsumeClaim(s, claim); err != nil {
  539. s.parent.handleError(err, topic, partition)
  540. }
  541. // ensure consumer is closed & drained
  542. claim.AsyncClose()
  543. for _, err := range claim.waitClosed() {
  544. s.parent.handleError(err, topic, partition)
  545. }
  546. }
  547. func (s *consumerGroupSession) release(withCleanup bool) (err error) {
  548. // signal release, stop heartbeat
  549. s.cancel()
  550. // wait for consumers to exit
  551. s.waitGroup.Wait()
  552. // perform release
  553. s.releaseOnce.Do(func() {
  554. if withCleanup {
  555. if e := s.handler.Cleanup(s); e != nil {
  556. s.parent.handleError(e, "", -1)
  557. err = e
  558. }
  559. }
  560. if e := s.offsets.Close(); e != nil {
  561. err = e
  562. }
  563. close(s.hbDying)
  564. <-s.hbDead
  565. })
  566. return
  567. }
  568. func (s *consumerGroupSession) heartbeatLoop() {
  569. defer close(s.hbDead)
  570. defer s.cancel() // trigger the end of the session on exit
  571. pause := time.NewTicker(s.parent.config.Consumer.Group.Heartbeat.Interval)
  572. defer pause.Stop()
  573. retries := s.parent.config.Metadata.Retry.Max
  574. for {
  575. coordinator, err := s.parent.client.Coordinator(s.parent.groupID)
  576. if err != nil {
  577. if retries <= 0 {
  578. s.parent.handleError(err, "", -1)
  579. return
  580. }
  581. select {
  582. case <-s.hbDying:
  583. return
  584. case <-time.After(s.parent.config.Metadata.Retry.Backoff):
  585. retries--
  586. }
  587. continue
  588. }
  589. resp, err := s.parent.heartbeatRequest(coordinator, s.memberID, s.generationID)
  590. if err != nil {
  591. _ = coordinator.Close()
  592. if retries <= 0 {
  593. s.parent.handleError(err, "", -1)
  594. return
  595. }
  596. retries--
  597. continue
  598. }
  599. switch resp.Err {
  600. case ErrNoError:
  601. retries = s.parent.config.Metadata.Retry.Max
  602. case ErrRebalanceInProgress, ErrUnknownMemberId, ErrIllegalGeneration:
  603. return
  604. default:
  605. s.parent.handleError(err, "", -1)
  606. return
  607. }
  608. select {
  609. case <-pause.C:
  610. case <-s.hbDying:
  611. return
  612. }
  613. }
  614. }
  615. // --------------------------------------------------------------------
  616. // ConsumerGroupHandler instances are used to handle individual topic/partition claims.
  617. // It also provides hooks for your consumer group session life-cycle and allow you to
  618. // trigger logic before or after the consume loop(s).
  619. //
  620. // PLEASE NOTE that handlers are likely be called from several goroutines concurrently,
  621. // ensure that all state is safely protected against race conditions.
  622. type ConsumerGroupHandler interface {
  623. // Setup is run at the beginning of a new session, before ConsumeClaim.
  624. Setup(ConsumerGroupSession) error
  625. // Cleanup is run at the end of a session, once all ConsumeClaim goroutines have exited
  626. // but before the offsets are committed for the very last time.
  627. Cleanup(ConsumerGroupSession) error
  628. // ConsumeClaim must start a consumer loop of ConsumerGroupClaim's Messages().
  629. // Once the Messages() channel is closed, the Handler must finish its processing
  630. // loop and exit.
  631. ConsumeClaim(ConsumerGroupSession, ConsumerGroupClaim) error
  632. }
  633. // ConsumerGroupClaim processes Kafka messages from a given topic and partition within a consumer group.
  634. type ConsumerGroupClaim interface {
  635. // Topic returns the consumed topic name.
  636. Topic() string
  637. // Partition returns the consumed partition.
  638. Partition() int32
  639. // InitialOffset returns the initial offset that was used as a starting point for this claim.
  640. InitialOffset() int64
  641. // HighWaterMarkOffset returns the high water mark offset of the partition,
  642. // i.e. the offset that will be used for the next message that will be produced.
  643. // You can use this to determine how far behind the processing is.
  644. HighWaterMarkOffset() int64
  645. // Messages returns the read channel for the messages that are returned by
  646. // the broker. The messages channel will be closed when a new rebalance cycle
  647. // is due. You must finish processing and mark offsets within
  648. // Config.Consumer.Group.Session.Timeout before the topic/partition is eventually
  649. // re-assigned to another group member.
  650. Messages() <-chan *ConsumerMessage
  651. }
  652. type consumerGroupClaim struct {
  653. topic string
  654. partition int32
  655. offset int64
  656. PartitionConsumer
  657. }
  658. func newConsumerGroupClaim(sess *consumerGroupSession, topic string, partition int32, offset int64) (*consumerGroupClaim, error) {
  659. pcm, err := sess.parent.consumer.ConsumePartition(topic, partition, offset)
  660. if err == ErrOffsetOutOfRange {
  661. offset = sess.parent.config.Consumer.Offsets.Initial
  662. pcm, err = sess.parent.consumer.ConsumePartition(topic, partition, offset)
  663. }
  664. if err != nil {
  665. return nil, err
  666. }
  667. go func() {
  668. for err := range pcm.Errors() {
  669. sess.parent.handleError(err, topic, partition)
  670. }
  671. }()
  672. return &consumerGroupClaim{
  673. topic: topic,
  674. partition: partition,
  675. offset: offset,
  676. PartitionConsumer: pcm,
  677. }, nil
  678. }
  679. func (c *consumerGroupClaim) Topic() string { return c.topic }
  680. func (c *consumerGroupClaim) Partition() int32 { return c.partition }
  681. func (c *consumerGroupClaim) InitialOffset() int64 { return c.offset }
  682. // Drains messages and errors, ensures the claim is fully closed.
  683. func (c *consumerGroupClaim) waitClosed() (errs ConsumerErrors) {
  684. go func() {
  685. for range c.Messages() {
  686. }
  687. }()
  688. for err := range c.Errors() {
  689. errs = append(errs, err)
  690. }
  691. return
  692. }