consumer_group.go 34 KB

12345678910111213141516171819202122232425262728293031323334353637383940414243444546474849505152535455565758596061626364656667686970717273747576777879808182838485868788899091929394959697989910010110210310410510610710810911011111211311411511611711811912012112212312412512612712812913013113213313413513613713813914014114214314414514614714814915015115215315415515615715815916016116216316416516616716816917017117217317417517617717817918018118218318418518618718818919019119219319419519619719819920020120220320420520620720820921021121221321421521621721821922022122222322422522622722822923023123223323423523623723823924024124224324424524624724824925025125225325425525625725825926026126226326426526626726826927027127227327427527627727827928028128228328428528628728828929029129229329429529629729829930030130230330430530630730830931031131231331431531631731831932032132232332432532632732832933033133233333433533633733833934034134234334434534634734834935035135235335435535635735835936036136236336436536636736836937037137237337437537637737837938038138238338438538638738838939039139239339439539639739839940040140240340440540640740840941041141241341441541641741841942042142242342442542642742842943043143243343443543643743843944044144244344444544644744844945045145245345445545645745845946046146246346446546646746846947047147247347447547647747847948048148248348448548648748848949049149249349449549649749849950050150250350450550650750850951051151251351451551651751851952052152252352452552652752852953053153253353453553653753853954054154254354454554654754854955055155255355455555655755855956056156256356456556656756856957057157257357457557657757857958058158258358458558658758858959059159259359459559659759859960060160260360460560660760860961061161261361461561661761861962062162262362462562662762862963063163263363463563663763863964064164264364464564664764864965065165265365465565665765865966066166266366466566666766866967067167267367467567667767867968068168268368468568668768868969069169269369469569669769869970070170270370470570670770870971071171271371471571671771871972072172272372472572672772872973073173273373473573673773873974074174274374474574674774874975075175275375475575675775875976076176276376476576676776876977077177277377477577677777877978078178278378478578678778878979079179279379479579679779879980080180280380480580680780880981081181281381481581681781881982082182282382482582682782882983083183283383483583683783883984084184284384484584684784884985085185285385485585685785885986086186286386486586686786886987087187287387487587687787887988088188288388488588688788888989089189289389489589689789889990090190290390490590690790890991091191291391491591691791891992092192292392492592692792892993093193293393493593693793893994094194294394494594694794894995095195295395495595695795895996096196296396496596696796896997097197297397497597697797897998098198298398498598698798898999099199299399499599699799899910001001100210031004100510061007100810091010101110121013101410151016101710181019102010211022102310241025102610271028102910301031103210331034103510361037103810391040104110421043104410451046104710481049105010511052105310541055105610571058105910601061106210631064106510661067106810691070107110721073107410751076107710781079108010811082108310841085108610871088108910901091109210931094109510961097109810991100110111021103110411051106110711081109111011111112111311141115111611171118111911201121112211231124112511261127112811291130113111321133113411351136113711381139114011411142114311441145114611471148114911501151115211531154115511561157115811591160
  1. package sarama
  2. import (
  3. "context"
  4. "errors"
  5. "fmt"
  6. "sort"
  7. "sync"
  8. "time"
  9. "github.com/rcrowley/go-metrics"
  10. )
  11. // ErrClosedConsumerGroup is the error returned when a method is called on a consumer group that has been closed.
  12. var ErrClosedConsumerGroup = errors.New("kafka: tried to use a consumer group that was closed")
  13. // ConsumerGroup is responsible for dividing up processing of topics and partitions
  14. // over a collection of processes (the members of the consumer group).
  15. type ConsumerGroup interface {
  16. // Consume joins a cluster of consumers for a given list of topics and
  17. // starts a blocking ConsumerGroupSession through the ConsumerGroupHandler.
  18. //
  19. // The life-cycle of a session is represented by the following steps:
  20. //
  21. // 1. The consumers join the group (as explained in https://kafka.apache.org/documentation/#intro_consumers)
  22. // and is assigned their "fair share" of partitions, aka 'claims'.
  23. // 2. Before processing starts, the handler's Setup() hook is called to notify the user
  24. // of the claims and allow any necessary preparation or alteration of state.
  25. // 3. For each of the assigned claims the handler's ConsumeClaim() function is then called
  26. // in a separate goroutine which requires it to be thread-safe. Any state must be carefully protected
  27. // from concurrent reads/writes.
  28. // 4. The session will persist until one of the ConsumeClaim() functions exits. This can be either when the
  29. // parent context is canceled or when a server-side rebalance cycle is initiated.
  30. // 5. Once all the ConsumeClaim() loops have exited, the handler's Cleanup() hook is called
  31. // to allow the user to perform any final tasks before a rebalance.
  32. // 6. Finally, marked offsets are committed one last time before claims are released.
  33. //
  34. // Please note, that once a rebalance is triggered, sessions must be completed within
  35. // Config.Consumer.Group.Rebalance.Timeout. This means that ConsumeClaim() functions must exit
  36. // as quickly as possible to allow time for Cleanup() and the final offset commit. If the timeout
  37. // is exceeded, the consumer will be removed from the group by Kafka, which will cause offset
  38. // commit failures.
  39. // This method should be called inside an infinite loop, when a
  40. // server-side rebalance happens, the consumer session will need to be
  41. // recreated to get the new claims.
  42. Consume(ctx context.Context, topics []string, handler ConsumerGroupHandler) error
  43. // Errors returns a read channel of errors that occurred during the consumer life-cycle.
  44. // By default, errors are logged and not returned over this channel.
  45. // If you want to implement any custom error handling, set your config's
  46. // Consumer.Return.Errors setting to true, and read from this channel.
  47. Errors() <-chan error
  48. // Close stops the ConsumerGroup and detaches any running sessions. It is required to call
  49. // this function before the object passes out of scope, as it will otherwise leak memory.
  50. Close() error
  51. // Pause suspends fetching from the requested partitions. Future calls to the broker will not return any
  52. // records from these partitions until they have been resumed using Resume()/ResumeAll().
  53. // Note that this method does not affect partition subscription.
  54. // In particular, it does not cause a group rebalance when automatic assignment is used.
  55. Pause(partitions map[string][]int32)
  56. // Resume resumes specified partitions which have been paused with Pause()/PauseAll().
  57. // New calls to the broker will return records from these partitions if there are any to be fetched.
  58. Resume(partitions map[string][]int32)
  59. // Pause suspends fetching from all partitions. Future calls to the broker will not return any
  60. // records from these partitions until they have been resumed using Resume()/ResumeAll().
  61. // Note that this method does not affect partition subscription.
  62. // In particular, it does not cause a group rebalance when automatic assignment is used.
  63. PauseAll()
  64. // Resume resumes all partitions which have been paused with Pause()/PauseAll().
  65. // New calls to the broker will return records from these partitions if there are any to be fetched.
  66. ResumeAll()
  67. }
  68. type consumerGroup struct {
  69. client Client
  70. config *Config
  71. consumer Consumer
  72. groupID string
  73. groupInstanceId *string
  74. memberID string
  75. errors chan error
  76. lock sync.Mutex
  77. errorsLock sync.RWMutex
  78. closed chan none
  79. closeOnce sync.Once
  80. userData []byte
  81. metricRegistry metrics.Registry
  82. }
  83. // NewConsumerGroup creates a new consumer group the given broker addresses and configuration.
  84. func NewConsumerGroup(addrs []string, groupID string, config *Config) (ConsumerGroup, error) {
  85. client, err := NewClient(addrs, config)
  86. if err != nil {
  87. return nil, err
  88. }
  89. c, err := newConsumerGroup(groupID, client)
  90. if err != nil {
  91. _ = client.Close()
  92. }
  93. return c, err
  94. }
  95. // NewConsumerGroupFromClient creates a new consumer group using the given client. It is still
  96. // necessary to call Close() on the underlying client when shutting down this consumer.
  97. // PLEASE NOTE: consumer groups can only re-use but not share clients.
  98. func NewConsumerGroupFromClient(groupID string, client Client) (ConsumerGroup, error) {
  99. if client == nil {
  100. return nil, ConfigurationError("client must not be nil")
  101. }
  102. // For clients passed in by the client, ensure we don't
  103. // call Close() on it.
  104. cli := &nopCloserClient{client}
  105. return newConsumerGroup(groupID, cli)
  106. }
  107. func newConsumerGroup(groupID string, client Client) (ConsumerGroup, error) {
  108. config := client.Config()
  109. if !config.Version.IsAtLeast(V0_10_2_0) {
  110. return nil, ConfigurationError("consumer groups require Version to be >= V0_10_2_0")
  111. }
  112. consumer, err := newConsumer(client)
  113. if err != nil {
  114. return nil, err
  115. }
  116. cg := &consumerGroup{
  117. client: client,
  118. consumer: consumer,
  119. config: config,
  120. groupID: groupID,
  121. errors: make(chan error, config.ChannelBufferSize),
  122. closed: make(chan none),
  123. userData: config.Consumer.Group.Member.UserData,
  124. metricRegistry: newCleanupRegistry(config.MetricRegistry),
  125. }
  126. if config.Consumer.Group.InstanceId != "" && config.Version.IsAtLeast(V2_3_0_0) {
  127. cg.groupInstanceId = &config.Consumer.Group.InstanceId
  128. }
  129. return cg, nil
  130. }
  131. // Errors implements ConsumerGroup.
  132. func (c *consumerGroup) Errors() <-chan error { return c.errors }
  133. // Close implements ConsumerGroup.
  134. func (c *consumerGroup) Close() (err error) {
  135. c.closeOnce.Do(func() {
  136. close(c.closed)
  137. // leave group
  138. if e := c.leave(); e != nil {
  139. err = e
  140. }
  141. go func() {
  142. c.errorsLock.Lock()
  143. defer c.errorsLock.Unlock()
  144. close(c.errors)
  145. }()
  146. // drain errors
  147. for e := range c.errors {
  148. err = e
  149. }
  150. if e := c.client.Close(); e != nil {
  151. err = e
  152. }
  153. c.metricRegistry.UnregisterAll()
  154. })
  155. return
  156. }
  157. // Consume implements ConsumerGroup.
  158. func (c *consumerGroup) Consume(ctx context.Context, topics []string, handler ConsumerGroupHandler) error {
  159. // Ensure group is not closed
  160. select {
  161. case <-c.closed:
  162. return ErrClosedConsumerGroup
  163. default:
  164. }
  165. c.lock.Lock()
  166. defer c.lock.Unlock()
  167. // Quick exit when no topics are provided
  168. if len(topics) == 0 {
  169. return fmt.Errorf("no topics provided")
  170. }
  171. // Refresh metadata for requested topics
  172. if err := c.client.RefreshMetadata(topics...); err != nil {
  173. return err
  174. }
  175. // Init session
  176. sess, err := c.newSession(ctx, topics, handler, c.config.Consumer.Group.Rebalance.Retry.Max)
  177. if errors.Is(err, ErrClosedClient) {
  178. return ErrClosedConsumerGroup
  179. } else if err != nil {
  180. return err
  181. }
  182. // Wait for session exit signal or Close() call
  183. select {
  184. case <-c.closed:
  185. case <-sess.ctx.Done():
  186. }
  187. // Gracefully release session claims
  188. return sess.release(true)
  189. }
  190. // Pause implements ConsumerGroup.
  191. func (c *consumerGroup) Pause(partitions map[string][]int32) {
  192. c.consumer.Pause(partitions)
  193. }
  194. // Resume implements ConsumerGroup.
  195. func (c *consumerGroup) Resume(partitions map[string][]int32) {
  196. c.consumer.Resume(partitions)
  197. }
  198. // PauseAll implements ConsumerGroup.
  199. func (c *consumerGroup) PauseAll() {
  200. c.consumer.PauseAll()
  201. }
  202. // ResumeAll implements ConsumerGroup.
  203. func (c *consumerGroup) ResumeAll() {
  204. c.consumer.ResumeAll()
  205. }
  206. func (c *consumerGroup) retryNewSession(ctx context.Context, topics []string, handler ConsumerGroupHandler, retries int, refreshCoordinator bool) (*consumerGroupSession, error) {
  207. select {
  208. case <-ctx.Done():
  209. return nil, ctx.Err()
  210. case <-c.closed:
  211. return nil, ErrClosedConsumerGroup
  212. case <-time.After(c.config.Consumer.Group.Rebalance.Retry.Backoff):
  213. }
  214. if refreshCoordinator {
  215. err := c.client.RefreshCoordinator(c.groupID)
  216. if err != nil {
  217. if retries <= 0 {
  218. return nil, err
  219. }
  220. return c.retryNewSession(ctx, topics, handler, retries-1, true)
  221. }
  222. }
  223. return c.newSession(ctx, topics, handler, retries-1)
  224. }
  225. func (c *consumerGroup) newSession(ctx context.Context, topics []string, handler ConsumerGroupHandler, retries int) (*consumerGroupSession, error) {
  226. if ctx.Err() != nil {
  227. return nil, ctx.Err()
  228. }
  229. coordinator, err := c.client.Coordinator(c.groupID)
  230. if err != nil {
  231. if retries <= 0 {
  232. return nil, err
  233. }
  234. return c.retryNewSession(ctx, topics, handler, retries, true)
  235. }
  236. var (
  237. metricRegistry = c.metricRegistry
  238. consumerGroupJoinTotal metrics.Counter
  239. consumerGroupJoinFailed metrics.Counter
  240. consumerGroupSyncTotal metrics.Counter
  241. consumerGroupSyncFailed metrics.Counter
  242. )
  243. if metricRegistry != nil {
  244. consumerGroupJoinTotal = metrics.GetOrRegisterCounter(fmt.Sprintf("consumer-group-join-total-%s", c.groupID), metricRegistry)
  245. consumerGroupJoinFailed = metrics.GetOrRegisterCounter(fmt.Sprintf("consumer-group-join-failed-%s", c.groupID), metricRegistry)
  246. consumerGroupSyncTotal = metrics.GetOrRegisterCounter(fmt.Sprintf("consumer-group-sync-total-%s", c.groupID), metricRegistry)
  247. consumerGroupSyncFailed = metrics.GetOrRegisterCounter(fmt.Sprintf("consumer-group-sync-failed-%s", c.groupID), metricRegistry)
  248. }
  249. // Join consumer group
  250. join, err := c.joinGroupRequest(coordinator, topics)
  251. if consumerGroupJoinTotal != nil {
  252. consumerGroupJoinTotal.Inc(1)
  253. }
  254. if err != nil {
  255. _ = coordinator.Close()
  256. if consumerGroupJoinFailed != nil {
  257. consumerGroupJoinFailed.Inc(1)
  258. }
  259. return nil, err
  260. }
  261. if !errors.Is(join.Err, ErrNoError) {
  262. if consumerGroupJoinFailed != nil {
  263. consumerGroupJoinFailed.Inc(1)
  264. }
  265. }
  266. switch join.Err {
  267. case ErrNoError:
  268. c.memberID = join.MemberId
  269. case ErrUnknownMemberId, ErrIllegalGeneration:
  270. // reset member ID and retry immediately
  271. c.memberID = ""
  272. return c.newSession(ctx, topics, handler, retries)
  273. case ErrNotCoordinatorForConsumer, ErrRebalanceInProgress, ErrOffsetsLoadInProgress:
  274. // retry after backoff
  275. if retries <= 0 {
  276. return nil, join.Err
  277. }
  278. return c.retryNewSession(ctx, topics, handler, retries, true)
  279. case ErrMemberIdRequired:
  280. // from JoinGroupRequest v4 onwards (due to KIP-394) if the client starts
  281. // with an empty member id, it needs to get the assigned id from the
  282. // response and send another join request with that id to actually join the
  283. // group
  284. c.memberID = join.MemberId
  285. return c.newSession(ctx, topics, handler, retries)
  286. case ErrFencedInstancedId:
  287. if c.groupInstanceId != nil {
  288. Logger.Printf("JoinGroup failed: group instance id %s has been fenced\n", *c.groupInstanceId)
  289. }
  290. return nil, join.Err
  291. default:
  292. return nil, join.Err
  293. }
  294. var strategy BalanceStrategy
  295. var ok bool
  296. if strategy = c.config.Consumer.Group.Rebalance.Strategy; strategy == nil {
  297. strategy, ok = c.findStrategy(join.GroupProtocol, c.config.Consumer.Group.Rebalance.GroupStrategies)
  298. if !ok {
  299. // this case shouldn't happen in practice, since the leader will choose the protocol
  300. // that all the members support
  301. return nil, fmt.Errorf("unable to find selected strategy: %s", join.GroupProtocol)
  302. }
  303. }
  304. // Prepare distribution plan if we joined as the leader
  305. var plan BalanceStrategyPlan
  306. var members map[string]ConsumerGroupMemberMetadata
  307. var allSubscribedTopicPartitions map[string][]int32
  308. var allSubscribedTopics []string
  309. if join.LeaderId == join.MemberId {
  310. members, err = join.GetMembers()
  311. if err != nil {
  312. return nil, err
  313. }
  314. allSubscribedTopicPartitions, allSubscribedTopics, plan, err = c.balance(strategy, members)
  315. if err != nil {
  316. return nil, err
  317. }
  318. }
  319. // Sync consumer group
  320. syncGroupResponse, err := c.syncGroupRequest(coordinator, members, plan, join.GenerationId, strategy)
  321. if consumerGroupSyncTotal != nil {
  322. consumerGroupSyncTotal.Inc(1)
  323. }
  324. if err != nil {
  325. _ = coordinator.Close()
  326. if consumerGroupSyncFailed != nil {
  327. consumerGroupSyncFailed.Inc(1)
  328. }
  329. return nil, err
  330. }
  331. if !errors.Is(syncGroupResponse.Err, ErrNoError) {
  332. if consumerGroupSyncFailed != nil {
  333. consumerGroupSyncFailed.Inc(1)
  334. }
  335. }
  336. switch syncGroupResponse.Err {
  337. case ErrNoError:
  338. case ErrUnknownMemberId, ErrIllegalGeneration:
  339. // reset member ID and retry immediately
  340. c.memberID = ""
  341. return c.newSession(ctx, topics, handler, retries)
  342. case ErrNotCoordinatorForConsumer, ErrRebalanceInProgress, ErrOffsetsLoadInProgress:
  343. // retry after backoff
  344. if retries <= 0 {
  345. return nil, syncGroupResponse.Err
  346. }
  347. return c.retryNewSession(ctx, topics, handler, retries, true)
  348. case ErrFencedInstancedId:
  349. if c.groupInstanceId != nil {
  350. Logger.Printf("JoinGroup failed: group instance id %s has been fenced\n", *c.groupInstanceId)
  351. }
  352. return nil, syncGroupResponse.Err
  353. default:
  354. return nil, syncGroupResponse.Err
  355. }
  356. // Retrieve and sort claims
  357. var claims map[string][]int32
  358. if len(syncGroupResponse.MemberAssignment) > 0 {
  359. members, err := syncGroupResponse.GetMemberAssignment()
  360. if err != nil {
  361. return nil, err
  362. }
  363. claims = members.Topics
  364. // in the case of stateful balance strategies, hold on to the returned
  365. // assignment metadata, otherwise, reset the statically defined consumer
  366. // group metadata
  367. if members.UserData != nil {
  368. c.userData = members.UserData
  369. } else {
  370. c.userData = c.config.Consumer.Group.Member.UserData
  371. }
  372. for _, partitions := range claims {
  373. sort.Sort(int32Slice(partitions))
  374. }
  375. }
  376. session, err := newConsumerGroupSession(ctx, c, claims, join.MemberId, join.GenerationId, handler)
  377. if err != nil {
  378. return nil, err
  379. }
  380. // only the leader needs to check whether there are newly-added partitions in order to trigger a rebalance
  381. if join.LeaderId == join.MemberId {
  382. go c.loopCheckPartitionNumbers(allSubscribedTopicPartitions, allSubscribedTopics, session)
  383. }
  384. return session, err
  385. }
  386. func (c *consumerGroup) joinGroupRequest(coordinator *Broker, topics []string) (*JoinGroupResponse, error) {
  387. req := &JoinGroupRequest{
  388. GroupId: c.groupID,
  389. MemberId: c.memberID,
  390. SessionTimeout: int32(c.config.Consumer.Group.Session.Timeout / time.Millisecond),
  391. ProtocolType: "consumer",
  392. }
  393. if c.config.Version.IsAtLeast(V0_10_1_0) {
  394. req.Version = 1
  395. req.RebalanceTimeout = int32(c.config.Consumer.Group.Rebalance.Timeout / time.Millisecond)
  396. }
  397. if c.config.Version.IsAtLeast(V0_11_0_0) {
  398. req.Version = 2
  399. }
  400. if c.config.Version.IsAtLeast(V0_11_0_0) {
  401. req.Version = 2
  402. }
  403. if c.config.Version.IsAtLeast(V2_0_0_0) {
  404. req.Version = 3
  405. }
  406. // from JoinGroupRequest v4 onwards (due to KIP-394) the client will actually
  407. // send two JoinGroupRequests, once with the empty member id, and then again
  408. // with the assigned id from the first response. This is handled via the
  409. // ErrMemberIdRequired case.
  410. if c.config.Version.IsAtLeast(V2_2_0_0) {
  411. req.Version = 4
  412. }
  413. if c.config.Version.IsAtLeast(V2_3_0_0) {
  414. req.Version = 5
  415. req.GroupInstanceId = c.groupInstanceId
  416. }
  417. meta := &ConsumerGroupMemberMetadata{
  418. Topics: topics,
  419. UserData: c.userData,
  420. }
  421. var strategy BalanceStrategy
  422. if strategy = c.config.Consumer.Group.Rebalance.Strategy; strategy != nil {
  423. if err := req.AddGroupProtocolMetadata(strategy.Name(), meta); err != nil {
  424. return nil, err
  425. }
  426. } else {
  427. for _, strategy = range c.config.Consumer.Group.Rebalance.GroupStrategies {
  428. if err := req.AddGroupProtocolMetadata(strategy.Name(), meta); err != nil {
  429. return nil, err
  430. }
  431. }
  432. }
  433. return coordinator.JoinGroup(req)
  434. }
  435. // findStrategy returns the BalanceStrategy with the specified protocolName
  436. // from the slice provided.
  437. func (c *consumerGroup) findStrategy(name string, groupStrategies []BalanceStrategy) (BalanceStrategy, bool) {
  438. for _, strategy := range groupStrategies {
  439. if strategy.Name() == name {
  440. return strategy, true
  441. }
  442. }
  443. return nil, false
  444. }
  445. func (c *consumerGroup) syncGroupRequest(
  446. coordinator *Broker,
  447. members map[string]ConsumerGroupMemberMetadata,
  448. plan BalanceStrategyPlan,
  449. generationID int32,
  450. strategy BalanceStrategy,
  451. ) (*SyncGroupResponse, error) {
  452. req := &SyncGroupRequest{
  453. GroupId: c.groupID,
  454. MemberId: c.memberID,
  455. GenerationId: generationID,
  456. }
  457. // Versions 1 and 2 are the same as version 0.
  458. if c.config.Version.IsAtLeast(V0_11_0_0) {
  459. req.Version = 1
  460. }
  461. if c.config.Version.IsAtLeast(V2_0_0_0) {
  462. req.Version = 2
  463. }
  464. // Starting from version 3, we add a new field called groupInstanceId to indicate member identity across restarts.
  465. if c.config.Version.IsAtLeast(V2_3_0_0) {
  466. req.Version = 3
  467. req.GroupInstanceId = c.groupInstanceId
  468. }
  469. for memberID, topics := range plan {
  470. assignment := &ConsumerGroupMemberAssignment{Topics: topics}
  471. userDataBytes, err := strategy.AssignmentData(memberID, topics, generationID)
  472. if err != nil {
  473. return nil, err
  474. }
  475. assignment.UserData = userDataBytes
  476. if err := req.AddGroupAssignmentMember(memberID, assignment); err != nil {
  477. return nil, err
  478. }
  479. delete(members, memberID)
  480. }
  481. // add empty assignments for any remaining members
  482. for memberID := range members {
  483. if err := req.AddGroupAssignmentMember(memberID, &ConsumerGroupMemberAssignment{}); err != nil {
  484. return nil, err
  485. }
  486. }
  487. return coordinator.SyncGroup(req)
  488. }
  489. func (c *consumerGroup) heartbeatRequest(coordinator *Broker, memberID string, generationID int32) (*HeartbeatResponse, error) {
  490. req := &HeartbeatRequest{
  491. GroupId: c.groupID,
  492. MemberId: memberID,
  493. GenerationId: generationID,
  494. }
  495. // Version 1 and version 2 are the same as version 0.
  496. if c.config.Version.IsAtLeast(V0_11_0_0) {
  497. req.Version = 1
  498. }
  499. if c.config.Version.IsAtLeast(V2_0_0_0) {
  500. req.Version = 2
  501. }
  502. // Starting from version 3, we add a new field called groupInstanceId to indicate member identity across restarts.
  503. if c.config.Version.IsAtLeast(V2_3_0_0) {
  504. req.Version = 3
  505. req.GroupInstanceId = c.groupInstanceId
  506. }
  507. return coordinator.Heartbeat(req)
  508. }
  509. func (c *consumerGroup) balance(strategy BalanceStrategy, members map[string]ConsumerGroupMemberMetadata) (map[string][]int32, []string, BalanceStrategyPlan, error) {
  510. topicPartitions := make(map[string][]int32)
  511. for _, meta := range members {
  512. for _, topic := range meta.Topics {
  513. topicPartitions[topic] = nil
  514. }
  515. }
  516. allSubscribedTopics := make([]string, 0, len(topicPartitions))
  517. for topic := range topicPartitions {
  518. allSubscribedTopics = append(allSubscribedTopics, topic)
  519. }
  520. // refresh metadata for all the subscribed topics in the consumer group
  521. // to avoid using stale metadata to assigning partitions
  522. err := c.client.RefreshMetadata(allSubscribedTopics...)
  523. if err != nil {
  524. return nil, nil, nil, err
  525. }
  526. for topic := range topicPartitions {
  527. partitions, err := c.client.Partitions(topic)
  528. if err != nil {
  529. return nil, nil, nil, err
  530. }
  531. topicPartitions[topic] = partitions
  532. }
  533. plan, err := strategy.Plan(members, topicPartitions)
  534. return topicPartitions, allSubscribedTopics, plan, err
  535. }
  536. // Leaves the cluster, called by Close.
  537. func (c *consumerGroup) leave() error {
  538. c.lock.Lock()
  539. defer c.lock.Unlock()
  540. if c.memberID == "" {
  541. return nil
  542. }
  543. coordinator, err := c.client.Coordinator(c.groupID)
  544. if err != nil {
  545. return err
  546. }
  547. // as per KIP-345 if groupInstanceId is set, i.e. static membership is in action, then do not leave group when consumer closed, just clear memberID
  548. if c.groupInstanceId != nil {
  549. c.memberID = ""
  550. return nil
  551. }
  552. req := &LeaveGroupRequest{
  553. GroupId: c.groupID,
  554. MemberId: c.memberID,
  555. }
  556. if c.config.Version.IsAtLeast(V0_11_0_0) {
  557. req.Version = 1
  558. }
  559. if c.config.Version.IsAtLeast(V2_0_0_0) {
  560. req.Version = 2
  561. }
  562. if c.config.Version.IsAtLeast(V2_4_0_0) {
  563. req.Version = 3
  564. req.Members = append(req.Members, MemberIdentity{
  565. MemberId: c.memberID,
  566. })
  567. }
  568. resp, err := coordinator.LeaveGroup(req)
  569. if err != nil {
  570. _ = coordinator.Close()
  571. return err
  572. }
  573. // clear the memberID
  574. c.memberID = ""
  575. switch resp.Err {
  576. case ErrRebalanceInProgress, ErrUnknownMemberId, ErrNoError:
  577. return nil
  578. default:
  579. return resp.Err
  580. }
  581. }
  582. func (c *consumerGroup) handleError(err error, topic string, partition int32) {
  583. var consumerError *ConsumerError
  584. if ok := errors.As(err, &consumerError); !ok && topic != "" && partition > -1 {
  585. err = &ConsumerError{
  586. Topic: topic,
  587. Partition: partition,
  588. Err: err,
  589. }
  590. }
  591. if !c.config.Consumer.Return.Errors {
  592. Logger.Println(err)
  593. return
  594. }
  595. c.errorsLock.RLock()
  596. defer c.errorsLock.RUnlock()
  597. select {
  598. case <-c.closed:
  599. // consumer is closed
  600. return
  601. default:
  602. }
  603. select {
  604. case c.errors <- err:
  605. default:
  606. // no error listener
  607. }
  608. }
  609. func (c *consumerGroup) loopCheckPartitionNumbers(allSubscribedTopicPartitions map[string][]int32, topics []string, session *consumerGroupSession) {
  610. if c.config.Metadata.RefreshFrequency == time.Duration(0) {
  611. return
  612. }
  613. defer session.cancel()
  614. oldTopicToPartitionNum := make(map[string]int, len(allSubscribedTopicPartitions))
  615. for topic, partitions := range allSubscribedTopicPartitions {
  616. oldTopicToPartitionNum[topic] = len(partitions)
  617. }
  618. pause := time.NewTicker(c.config.Metadata.RefreshFrequency)
  619. defer pause.Stop()
  620. for {
  621. if newTopicToPartitionNum, err := c.topicToPartitionNumbers(topics); err != nil {
  622. return
  623. } else {
  624. for topic, num := range oldTopicToPartitionNum {
  625. if newTopicToPartitionNum[topic] != num {
  626. Logger.Printf(
  627. "consumergroup/%s loop check partition number goroutine find partitions in topics %s changed from %d to %d\n",
  628. c.groupID, topics, num, newTopicToPartitionNum[topic])
  629. return // trigger the end of the session on exit
  630. }
  631. }
  632. }
  633. select {
  634. case <-pause.C:
  635. case <-session.ctx.Done():
  636. Logger.Printf(
  637. "consumergroup/%s loop check partition number goroutine will exit, topics %s\n",
  638. c.groupID, topics)
  639. // if session closed by other, should be exited
  640. return
  641. case <-c.closed:
  642. return
  643. }
  644. }
  645. }
  646. func (c *consumerGroup) topicToPartitionNumbers(topics []string) (map[string]int, error) {
  647. topicToPartitionNum := make(map[string]int, len(topics))
  648. for _, topic := range topics {
  649. if partitionNum, err := c.client.Partitions(topic); err != nil {
  650. Logger.Printf(
  651. "consumergroup/%s topic %s get partition number failed due to '%v'\n",
  652. c.groupID, topic, err)
  653. return nil, err
  654. } else {
  655. topicToPartitionNum[topic] = len(partitionNum)
  656. }
  657. }
  658. return topicToPartitionNum, nil
  659. }
  660. // --------------------------------------------------------------------
  661. // ConsumerGroupSession represents a consumer group member session.
  662. type ConsumerGroupSession interface {
  663. // Claims returns information about the claimed partitions by topic.
  664. Claims() map[string][]int32
  665. // MemberID returns the cluster member ID.
  666. MemberID() string
  667. // GenerationID returns the current generation ID.
  668. GenerationID() int32
  669. // MarkOffset marks the provided offset, alongside a metadata string
  670. // that represents the state of the partition consumer at that point in time. The
  671. // metadata string can be used by another consumer to restore that state, so it
  672. // can resume consumption.
  673. //
  674. // To follow upstream conventions, you are expected to mark the offset of the
  675. // next message to read, not the last message read. Thus, when calling `MarkOffset`
  676. // you should typically add one to the offset of the last consumed message.
  677. //
  678. // Note: calling MarkOffset does not necessarily commit the offset to the backend
  679. // store immediately for efficiency reasons, and it may never be committed if
  680. // your application crashes. This means that you may end up processing the same
  681. // message twice, and your processing should ideally be idempotent.
  682. MarkOffset(topic string, partition int32, offset int64, metadata string)
  683. // Commit the offset to the backend
  684. //
  685. // Note: calling Commit performs a blocking synchronous operation.
  686. Commit()
  687. // ResetOffset resets to the provided offset, alongside a metadata string that
  688. // represents the state of the partition consumer at that point in time. Reset
  689. // acts as a counterpart to MarkOffset, the difference being that it allows to
  690. // reset an offset to an earlier or smaller value, where MarkOffset only
  691. // allows incrementing the offset. cf MarkOffset for more details.
  692. ResetOffset(topic string, partition int32, offset int64, metadata string)
  693. // MarkMessage marks a message as consumed.
  694. MarkMessage(msg *ConsumerMessage, metadata string)
  695. // Context returns the session context.
  696. Context() context.Context
  697. }
  698. type consumerGroupSession struct {
  699. parent *consumerGroup
  700. memberID string
  701. generationID int32
  702. handler ConsumerGroupHandler
  703. claims map[string][]int32
  704. offsets *offsetManager
  705. ctx context.Context
  706. cancel func()
  707. waitGroup sync.WaitGroup
  708. releaseOnce sync.Once
  709. hbDying, hbDead chan none
  710. }
  711. func newConsumerGroupSession(ctx context.Context, parent *consumerGroup, claims map[string][]int32, memberID string, generationID int32, handler ConsumerGroupHandler) (*consumerGroupSession, error) {
  712. // init context
  713. ctx, cancel := context.WithCancel(ctx)
  714. // init offset manager
  715. offsets, err := newOffsetManagerFromClient(parent.groupID, memberID, generationID, parent.client, cancel)
  716. if err != nil {
  717. return nil, err
  718. }
  719. // init session
  720. sess := &consumerGroupSession{
  721. parent: parent,
  722. memberID: memberID,
  723. generationID: generationID,
  724. handler: handler,
  725. offsets: offsets,
  726. claims: claims,
  727. ctx: ctx,
  728. cancel: cancel,
  729. hbDying: make(chan none),
  730. hbDead: make(chan none),
  731. }
  732. // start heartbeat loop
  733. go sess.heartbeatLoop()
  734. // create a POM for each claim
  735. for topic, partitions := range claims {
  736. for _, partition := range partitions {
  737. pom, err := offsets.ManagePartition(topic, partition)
  738. if err != nil {
  739. _ = sess.release(false)
  740. return nil, err
  741. }
  742. // handle POM errors
  743. go func(topic string, partition int32) {
  744. for err := range pom.Errors() {
  745. sess.parent.handleError(err, topic, partition)
  746. }
  747. }(topic, partition)
  748. }
  749. }
  750. // perform setup
  751. if err := handler.Setup(sess); err != nil {
  752. _ = sess.release(true)
  753. return nil, err
  754. }
  755. // start consuming
  756. for topic, partitions := range claims {
  757. for _, partition := range partitions {
  758. sess.waitGroup.Add(1)
  759. go func(topic string, partition int32) {
  760. defer sess.waitGroup.Done()
  761. // cancel the as session as soon as the first
  762. // goroutine exits
  763. defer sess.cancel()
  764. // consume a single topic/partition, blocking
  765. sess.consume(topic, partition)
  766. }(topic, partition)
  767. }
  768. }
  769. return sess, nil
  770. }
  771. func (s *consumerGroupSession) Claims() map[string][]int32 { return s.claims }
  772. func (s *consumerGroupSession) MemberID() string { return s.memberID }
  773. func (s *consumerGroupSession) GenerationID() int32 { return s.generationID }
  774. func (s *consumerGroupSession) MarkOffset(topic string, partition int32, offset int64, metadata string) {
  775. if pom := s.offsets.findPOM(topic, partition); pom != nil {
  776. pom.MarkOffset(offset, metadata)
  777. }
  778. }
  779. func (s *consumerGroupSession) Commit() {
  780. s.offsets.Commit()
  781. }
  782. func (s *consumerGroupSession) ResetOffset(topic string, partition int32, offset int64, metadata string) {
  783. if pom := s.offsets.findPOM(topic, partition); pom != nil {
  784. pom.ResetOffset(offset, metadata)
  785. }
  786. }
  787. func (s *consumerGroupSession) MarkMessage(msg *ConsumerMessage, metadata string) {
  788. s.MarkOffset(msg.Topic, msg.Partition, msg.Offset+1, metadata)
  789. }
  790. func (s *consumerGroupSession) Context() context.Context {
  791. return s.ctx
  792. }
  793. func (s *consumerGroupSession) consume(topic string, partition int32) {
  794. // quick exit if rebalance is due
  795. select {
  796. case <-s.ctx.Done():
  797. return
  798. case <-s.parent.closed:
  799. return
  800. default:
  801. }
  802. // get next offset
  803. offset := s.parent.config.Consumer.Offsets.Initial
  804. if pom := s.offsets.findPOM(topic, partition); pom != nil {
  805. offset, _ = pom.NextOffset()
  806. }
  807. // create new claim
  808. claim, err := newConsumerGroupClaim(s, topic, partition, offset)
  809. if err != nil {
  810. s.parent.handleError(err, topic, partition)
  811. return
  812. }
  813. // handle errors
  814. go func() {
  815. for err := range claim.Errors() {
  816. s.parent.handleError(err, topic, partition)
  817. }
  818. }()
  819. // trigger close when session is done
  820. go func() {
  821. select {
  822. case <-s.ctx.Done():
  823. case <-s.parent.closed:
  824. }
  825. claim.AsyncClose()
  826. }()
  827. // start processing
  828. if err := s.handler.ConsumeClaim(s, claim); err != nil {
  829. s.parent.handleError(err, topic, partition)
  830. }
  831. // ensure consumer is closed & drained
  832. claim.AsyncClose()
  833. for _, err := range claim.waitClosed() {
  834. s.parent.handleError(err, topic, partition)
  835. }
  836. }
  837. func (s *consumerGroupSession) release(withCleanup bool) (err error) {
  838. // signal release, stop heartbeat
  839. s.cancel()
  840. // wait for consumers to exit
  841. s.waitGroup.Wait()
  842. // perform release
  843. s.releaseOnce.Do(func() {
  844. if withCleanup {
  845. if e := s.handler.Cleanup(s); e != nil {
  846. s.parent.handleError(e, "", -1)
  847. err = e
  848. }
  849. }
  850. if e := s.offsets.Close(); e != nil {
  851. err = e
  852. }
  853. close(s.hbDying)
  854. <-s.hbDead
  855. })
  856. Logger.Printf(
  857. "consumergroup/session/%s/%d released\n",
  858. s.MemberID(), s.GenerationID())
  859. return
  860. }
  861. func (s *consumerGroupSession) heartbeatLoop() {
  862. defer close(s.hbDead)
  863. defer s.cancel() // trigger the end of the session on exit
  864. defer func() {
  865. Logger.Printf(
  866. "consumergroup/session/%s/%d heartbeat loop stopped\n",
  867. s.MemberID(), s.GenerationID())
  868. }()
  869. pause := time.NewTicker(s.parent.config.Consumer.Group.Heartbeat.Interval)
  870. defer pause.Stop()
  871. retryBackoff := time.NewTimer(s.parent.config.Metadata.Retry.Backoff)
  872. defer retryBackoff.Stop()
  873. retries := s.parent.config.Metadata.Retry.Max
  874. for {
  875. coordinator, err := s.parent.client.Coordinator(s.parent.groupID)
  876. if err != nil {
  877. if retries <= 0 {
  878. s.parent.handleError(err, "", -1)
  879. return
  880. }
  881. retryBackoff.Reset(s.parent.config.Metadata.Retry.Backoff)
  882. select {
  883. case <-s.hbDying:
  884. return
  885. case <-retryBackoff.C:
  886. retries--
  887. }
  888. continue
  889. }
  890. resp, err := s.parent.heartbeatRequest(coordinator, s.memberID, s.generationID)
  891. if err != nil {
  892. _ = coordinator.Close()
  893. if retries <= 0 {
  894. s.parent.handleError(err, "", -1)
  895. return
  896. }
  897. retries--
  898. continue
  899. }
  900. switch resp.Err {
  901. case ErrNoError:
  902. retries = s.parent.config.Metadata.Retry.Max
  903. case ErrRebalanceInProgress:
  904. retries = s.parent.config.Metadata.Retry.Max
  905. s.cancel()
  906. case ErrUnknownMemberId, ErrIllegalGeneration:
  907. return
  908. case ErrFencedInstancedId:
  909. if s.parent.groupInstanceId != nil {
  910. Logger.Printf("JoinGroup failed: group instance id %s has been fenced\n", *s.parent.groupInstanceId)
  911. }
  912. s.parent.handleError(resp.Err, "", -1)
  913. return
  914. default:
  915. s.parent.handleError(resp.Err, "", -1)
  916. return
  917. }
  918. select {
  919. case <-pause.C:
  920. case <-s.hbDying:
  921. return
  922. }
  923. }
  924. }
  925. // --------------------------------------------------------------------
  926. // ConsumerGroupHandler instances are used to handle individual topic/partition claims.
  927. // It also provides hooks for your consumer group session life-cycle and allow you to
  928. // trigger logic before or after the consume loop(s).
  929. //
  930. // PLEASE NOTE that handlers are likely be called from several goroutines concurrently,
  931. // ensure that all state is safely protected against race conditions.
  932. type ConsumerGroupHandler interface {
  933. // Setup is run at the beginning of a new session, before ConsumeClaim.
  934. Setup(ConsumerGroupSession) error
  935. // Cleanup is run at the end of a session, once all ConsumeClaim goroutines have exited
  936. // but before the offsets are committed for the very last time.
  937. Cleanup(ConsumerGroupSession) error
  938. // ConsumeClaim must start a consumer loop of ConsumerGroupClaim's Messages().
  939. // Once the Messages() channel is closed, the Handler must finish its processing
  940. // loop and exit.
  941. ConsumeClaim(ConsumerGroupSession, ConsumerGroupClaim) error
  942. }
  943. // ConsumerGroupClaim processes Kafka messages from a given topic and partition within a consumer group.
  944. type ConsumerGroupClaim interface {
  945. // Topic returns the consumed topic name.
  946. Topic() string
  947. // Partition returns the consumed partition.
  948. Partition() int32
  949. // InitialOffset returns the initial offset that was used as a starting point for this claim.
  950. InitialOffset() int64
  951. // HighWaterMarkOffset returns the high watermark offset of the partition,
  952. // i.e. the offset that will be used for the next message that will be produced.
  953. // You can use this to determine how far behind the processing is.
  954. HighWaterMarkOffset() int64
  955. // Messages returns the read channel for the messages that are returned by
  956. // the broker. The messages channel will be closed when a new rebalance cycle
  957. // is due. You must finish processing and mark offsets within
  958. // Config.Consumer.Group.Session.Timeout before the topic/partition is eventually
  959. // re-assigned to another group member.
  960. Messages() <-chan *ConsumerMessage
  961. }
  962. type consumerGroupClaim struct {
  963. topic string
  964. partition int32
  965. offset int64
  966. PartitionConsumer
  967. }
  968. func newConsumerGroupClaim(sess *consumerGroupSession, topic string, partition int32, offset int64) (*consumerGroupClaim, error) {
  969. pcm, err := sess.parent.consumer.ConsumePartition(topic, partition, offset)
  970. if errors.Is(err, ErrOffsetOutOfRange) && sess.parent.config.Consumer.Group.ResetInvalidOffsets {
  971. offset = sess.parent.config.Consumer.Offsets.Initial
  972. pcm, err = sess.parent.consumer.ConsumePartition(topic, partition, offset)
  973. }
  974. if err != nil {
  975. return nil, err
  976. }
  977. go func() {
  978. for err := range pcm.Errors() {
  979. sess.parent.handleError(err, topic, partition)
  980. }
  981. }()
  982. return &consumerGroupClaim{
  983. topic: topic,
  984. partition: partition,
  985. offset: offset,
  986. PartitionConsumer: pcm,
  987. }, nil
  988. }
  989. func (c *consumerGroupClaim) Topic() string { return c.topic }
  990. func (c *consumerGroupClaim) Partition() int32 { return c.partition }
  991. func (c *consumerGroupClaim) InitialOffset() int64 { return c.offset }
  992. // Drains messages and errors, ensures the claim is fully closed.
  993. func (c *consumerGroupClaim) waitClosed() (errs ConsumerErrors) {
  994. go func() {
  995. for range c.Messages() {
  996. }
  997. }()
  998. for err := range c.Errors() {
  999. errs = append(errs, err)
  1000. }
  1001. return
  1002. }