offset_manager.go 15 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591
  1. package sarama
  2. import (
  3. "sync"
  4. "time"
  5. )
  6. // Offset Manager
  7. // OffsetManager uses Kafka to store and fetch consumed partition offsets.
  8. type OffsetManager interface {
  9. // ManagePartition creates a PartitionOffsetManager on the given topic/partition.
  10. // It will return an error if this OffsetManager is already managing the given
  11. // topic/partition.
  12. ManagePartition(topic string, partition int32) (PartitionOffsetManager, error)
  13. // Close stops the OffsetManager from managing offsets. It is required to call
  14. // this function before an OffsetManager object passes out of scope, as it
  15. // will otherwise leak memory. You must call this after all the
  16. // PartitionOffsetManagers are closed.
  17. Close() error
  18. // Commit commits the offsets. This method can be used if AutoCommit.Enable is
  19. // set to false.
  20. Commit()
  21. }
  22. type offsetManager struct {
  23. client Client
  24. conf *Config
  25. group string
  26. ticker *time.Ticker
  27. memberID string
  28. generation int32
  29. broker *Broker
  30. brokerLock sync.RWMutex
  31. poms map[string]map[int32]*partitionOffsetManager
  32. pomsLock sync.RWMutex
  33. closeOnce sync.Once
  34. closing chan none
  35. closed chan none
  36. }
  37. // NewOffsetManagerFromClient creates a new OffsetManager from the given client.
  38. // It is still necessary to call Close() on the underlying client when finished with the partition manager.
  39. func NewOffsetManagerFromClient(group string, client Client) (OffsetManager, error) {
  40. return newOffsetManagerFromClient(group, "", GroupGenerationUndefined, client)
  41. }
  42. func newOffsetManagerFromClient(group, memberID string, generation int32, client Client) (*offsetManager, error) {
  43. // Check that we are not dealing with a closed Client before processing any other arguments
  44. if client.Closed() {
  45. return nil, ErrClosedClient
  46. }
  47. conf := client.Config()
  48. om := &offsetManager{
  49. client: client,
  50. conf: conf,
  51. group: group,
  52. poms: make(map[string]map[int32]*partitionOffsetManager),
  53. memberID: memberID,
  54. generation: generation,
  55. closing: make(chan none),
  56. closed: make(chan none),
  57. }
  58. if conf.Consumer.Offsets.AutoCommit.Enable {
  59. om.ticker = time.NewTicker(conf.Consumer.Offsets.AutoCommit.Interval)
  60. go withRecover(om.mainLoop)
  61. }
  62. return om, nil
  63. }
  64. func (om *offsetManager) ManagePartition(topic string, partition int32) (PartitionOffsetManager, error) {
  65. pom, err := om.newPartitionOffsetManager(topic, partition)
  66. if err != nil {
  67. return nil, err
  68. }
  69. om.pomsLock.Lock()
  70. defer om.pomsLock.Unlock()
  71. topicManagers := om.poms[topic]
  72. if topicManagers == nil {
  73. topicManagers = make(map[int32]*partitionOffsetManager)
  74. om.poms[topic] = topicManagers
  75. }
  76. if topicManagers[partition] != nil {
  77. return nil, ConfigurationError("That topic/partition is already being managed")
  78. }
  79. topicManagers[partition] = pom
  80. return pom, nil
  81. }
  82. func (om *offsetManager) Close() error {
  83. om.closeOnce.Do(func() {
  84. // exit the mainLoop
  85. close(om.closing)
  86. if om.conf.Consumer.Offsets.AutoCommit.Enable {
  87. <-om.closed
  88. }
  89. // mark all POMs as closed
  90. om.asyncClosePOMs()
  91. // flush one last time
  92. for attempt := 0; attempt <= om.conf.Consumer.Offsets.Retry.Max; attempt++ {
  93. om.flushToBroker()
  94. if om.releasePOMs(false) == 0 {
  95. break
  96. }
  97. }
  98. om.releasePOMs(true)
  99. om.brokerLock.Lock()
  100. om.broker = nil
  101. om.brokerLock.Unlock()
  102. })
  103. return nil
  104. }
  105. func (om *offsetManager) computeBackoff(retries int) time.Duration {
  106. if om.conf.Metadata.Retry.BackoffFunc != nil {
  107. return om.conf.Metadata.Retry.BackoffFunc(retries, om.conf.Metadata.Retry.Max)
  108. } else {
  109. return om.conf.Metadata.Retry.Backoff
  110. }
  111. }
  112. func (om *offsetManager) fetchInitialOffset(topic string, partition int32, retries int) (int64, string, error) {
  113. broker, err := om.coordinator()
  114. if err != nil {
  115. if retries <= 0 {
  116. return 0, "", err
  117. }
  118. return om.fetchInitialOffset(topic, partition, retries-1)
  119. }
  120. req := new(OffsetFetchRequest)
  121. req.Version = 1
  122. req.ConsumerGroup = om.group
  123. req.AddPartition(topic, partition)
  124. resp, err := broker.FetchOffset(req)
  125. if err != nil {
  126. if retries <= 0 {
  127. return 0, "", err
  128. }
  129. om.releaseCoordinator(broker)
  130. return om.fetchInitialOffset(topic, partition, retries-1)
  131. }
  132. block := resp.GetBlock(topic, partition)
  133. if block == nil {
  134. return 0, "", ErrIncompleteResponse
  135. }
  136. switch block.Err {
  137. case ErrNoError:
  138. return block.Offset, block.Metadata, nil
  139. case ErrNotCoordinatorForConsumer:
  140. if retries <= 0 {
  141. return 0, "", block.Err
  142. }
  143. om.releaseCoordinator(broker)
  144. return om.fetchInitialOffset(topic, partition, retries-1)
  145. case ErrOffsetsLoadInProgress:
  146. if retries <= 0 {
  147. return 0, "", block.Err
  148. }
  149. backoff := om.computeBackoff(retries)
  150. select {
  151. case <-om.closing:
  152. return 0, "", block.Err
  153. case <-time.After(backoff):
  154. }
  155. return om.fetchInitialOffset(topic, partition, retries-1)
  156. default:
  157. return 0, "", block.Err
  158. }
  159. }
  160. func (om *offsetManager) coordinator() (*Broker, error) {
  161. om.brokerLock.RLock()
  162. broker := om.broker
  163. om.brokerLock.RUnlock()
  164. if broker != nil {
  165. return broker, nil
  166. }
  167. om.brokerLock.Lock()
  168. defer om.brokerLock.Unlock()
  169. if broker := om.broker; broker != nil {
  170. return broker, nil
  171. }
  172. if err := om.client.RefreshCoordinator(om.group); err != nil {
  173. return nil, err
  174. }
  175. broker, err := om.client.Coordinator(om.group)
  176. if err != nil {
  177. return nil, err
  178. }
  179. om.broker = broker
  180. return broker, nil
  181. }
  182. func (om *offsetManager) releaseCoordinator(b *Broker) {
  183. om.brokerLock.Lock()
  184. if om.broker == b {
  185. om.broker = nil
  186. }
  187. om.brokerLock.Unlock()
  188. }
  189. func (om *offsetManager) mainLoop() {
  190. defer om.ticker.Stop()
  191. defer close(om.closed)
  192. for {
  193. select {
  194. case <-om.ticker.C:
  195. om.Commit()
  196. case <-om.closing:
  197. return
  198. }
  199. }
  200. }
  201. func (om *offsetManager) Commit() {
  202. om.flushToBroker()
  203. om.releasePOMs(false)
  204. }
  205. func (om *offsetManager) flushToBroker() {
  206. req := om.constructRequest()
  207. if req == nil {
  208. return
  209. }
  210. broker, err := om.coordinator()
  211. if err != nil {
  212. om.handleError(err)
  213. return
  214. }
  215. resp, err := broker.CommitOffset(req)
  216. if err != nil {
  217. om.handleError(err)
  218. om.releaseCoordinator(broker)
  219. _ = broker.Close()
  220. return
  221. }
  222. om.handleResponse(broker, req, resp)
  223. }
  224. func (om *offsetManager) constructRequest() *OffsetCommitRequest {
  225. var r *OffsetCommitRequest
  226. var perPartitionTimestamp int64
  227. if om.conf.Consumer.Offsets.Retention == 0 {
  228. perPartitionTimestamp = ReceiveTime
  229. r = &OffsetCommitRequest{
  230. Version: 1,
  231. ConsumerGroup: om.group,
  232. ConsumerID: om.memberID,
  233. ConsumerGroupGeneration: om.generation,
  234. }
  235. } else {
  236. r = &OffsetCommitRequest{
  237. Version: 2,
  238. RetentionTime: int64(om.conf.Consumer.Offsets.Retention / time.Millisecond),
  239. ConsumerGroup: om.group,
  240. ConsumerID: om.memberID,
  241. ConsumerGroupGeneration: om.generation,
  242. }
  243. }
  244. om.pomsLock.RLock()
  245. defer om.pomsLock.RUnlock()
  246. for _, topicManagers := range om.poms {
  247. for _, pom := range topicManagers {
  248. pom.lock.Lock()
  249. if pom.dirty {
  250. r.AddBlock(pom.topic, pom.partition, pom.offset, perPartitionTimestamp, pom.metadata)
  251. }
  252. pom.lock.Unlock()
  253. }
  254. }
  255. if len(r.blocks) > 0 {
  256. return r
  257. }
  258. return nil
  259. }
  260. func (om *offsetManager) handleResponse(broker *Broker, req *OffsetCommitRequest, resp *OffsetCommitResponse) {
  261. om.pomsLock.RLock()
  262. defer om.pomsLock.RUnlock()
  263. for _, topicManagers := range om.poms {
  264. for _, pom := range topicManagers {
  265. if req.blocks[pom.topic] == nil || req.blocks[pom.topic][pom.partition] == nil {
  266. continue
  267. }
  268. var err KError
  269. var ok bool
  270. if resp.Errors[pom.topic] == nil {
  271. pom.handleError(ErrIncompleteResponse)
  272. continue
  273. }
  274. if err, ok = resp.Errors[pom.topic][pom.partition]; !ok {
  275. pom.handleError(ErrIncompleteResponse)
  276. continue
  277. }
  278. switch err {
  279. case ErrNoError:
  280. block := req.blocks[pom.topic][pom.partition]
  281. pom.updateCommitted(block.offset, block.metadata)
  282. case ErrNotLeaderForPartition, ErrLeaderNotAvailable,
  283. ErrConsumerCoordinatorNotAvailable, ErrNotCoordinatorForConsumer:
  284. // not a critical error, we just need to redispatch
  285. om.releaseCoordinator(broker)
  286. case ErrOffsetMetadataTooLarge, ErrInvalidCommitOffsetSize:
  287. // nothing we can do about this, just tell the user and carry on
  288. pom.handleError(err)
  289. case ErrOffsetsLoadInProgress:
  290. // nothing wrong but we didn't commit, we'll get it next time round
  291. case ErrUnknownTopicOrPartition:
  292. // let the user know *and* try redispatching - if topic-auto-create is
  293. // enabled, redispatching should trigger a metadata req and create the
  294. // topic; if not then re-dispatching won't help, but we've let the user
  295. // know and it shouldn't hurt either (see https://github.com/Shopify/sarama/issues/706)
  296. fallthrough
  297. default:
  298. // dunno, tell the user and try redispatching
  299. pom.handleError(err)
  300. om.releaseCoordinator(broker)
  301. }
  302. }
  303. }
  304. }
  305. func (om *offsetManager) handleError(err error) {
  306. om.pomsLock.RLock()
  307. defer om.pomsLock.RUnlock()
  308. for _, topicManagers := range om.poms {
  309. for _, pom := range topicManagers {
  310. pom.handleError(err)
  311. }
  312. }
  313. }
  314. func (om *offsetManager) asyncClosePOMs() {
  315. om.pomsLock.RLock()
  316. defer om.pomsLock.RUnlock()
  317. for _, topicManagers := range om.poms {
  318. for _, pom := range topicManagers {
  319. pom.AsyncClose()
  320. }
  321. }
  322. }
  323. // Releases/removes closed POMs once they are clean (or when forced)
  324. func (om *offsetManager) releasePOMs(force bool) (remaining int) {
  325. om.pomsLock.Lock()
  326. defer om.pomsLock.Unlock()
  327. for topic, topicManagers := range om.poms {
  328. for partition, pom := range topicManagers {
  329. pom.lock.Lock()
  330. releaseDue := pom.done && (force || !pom.dirty)
  331. pom.lock.Unlock()
  332. if releaseDue {
  333. pom.release()
  334. delete(om.poms[topic], partition)
  335. if len(om.poms[topic]) == 0 {
  336. delete(om.poms, topic)
  337. }
  338. }
  339. }
  340. remaining += len(om.poms[topic])
  341. }
  342. return
  343. }
  344. func (om *offsetManager) findPOM(topic string, partition int32) *partitionOffsetManager {
  345. om.pomsLock.RLock()
  346. defer om.pomsLock.RUnlock()
  347. if partitions, ok := om.poms[topic]; ok {
  348. if pom, ok := partitions[partition]; ok {
  349. return pom
  350. }
  351. }
  352. return nil
  353. }
  354. // Partition Offset Manager
  355. // PartitionOffsetManager uses Kafka to store and fetch consumed partition offsets. You MUST call Close()
  356. // on a partition offset manager to avoid leaks, it will not be garbage-collected automatically when it passes
  357. // out of scope.
  358. type PartitionOffsetManager interface {
  359. // NextOffset returns the next offset that should be consumed for the managed
  360. // partition, accompanied by metadata which can be used to reconstruct the state
  361. // of the partition consumer when it resumes. NextOffset() will return
  362. // `config.Consumer.Offsets.Initial` and an empty metadata string if no offset
  363. // was committed for this partition yet.
  364. NextOffset() (int64, string)
  365. // MarkOffset marks the provided offset, alongside a metadata string
  366. // that represents the state of the partition consumer at that point in time. The
  367. // metadata string can be used by another consumer to restore that state, so it
  368. // can resume consumption.
  369. //
  370. // To follow upstream conventions, you are expected to mark the offset of the
  371. // next message to read, not the last message read. Thus, when calling `MarkOffset`
  372. // you should typically add one to the offset of the last consumed message.
  373. //
  374. // Note: calling MarkOffset does not necessarily commit the offset to the backend
  375. // store immediately for efficiency reasons, and it may never be committed if
  376. // your application crashes. This means that you may end up processing the same
  377. // message twice, and your processing should ideally be idempotent.
  378. MarkOffset(offset int64, metadata string)
  379. // ResetOffset resets to the provided offset, alongside a metadata string that
  380. // represents the state of the partition consumer at that point in time. Reset
  381. // acts as a counterpart to MarkOffset, the difference being that it allows to
  382. // reset an offset to an earlier or smaller value, where MarkOffset only
  383. // allows incrementing the offset. cf MarkOffset for more details.
  384. ResetOffset(offset int64, metadata string)
  385. // Errors returns a read channel of errors that occur during offset management, if
  386. // enabled. By default, errors are logged and not returned over this channel. If
  387. // you want to implement any custom error handling, set your config's
  388. // Consumer.Return.Errors setting to true, and read from this channel.
  389. Errors() <-chan *ConsumerError
  390. // AsyncClose initiates a shutdown of the PartitionOffsetManager. This method will
  391. // return immediately, after which you should wait until the 'errors' channel has
  392. // been drained and closed. It is required to call this function, or Close before
  393. // a consumer object passes out of scope, as it will otherwise leak memory. You
  394. // must call this before calling Close on the underlying client.
  395. AsyncClose()
  396. // Close stops the PartitionOffsetManager from managing offsets. It is required to
  397. // call this function (or AsyncClose) before a PartitionOffsetManager object
  398. // passes out of scope, as it will otherwise leak memory. You must call this
  399. // before calling Close on the underlying client.
  400. Close() error
  401. }
  402. type partitionOffsetManager struct {
  403. parent *offsetManager
  404. topic string
  405. partition int32
  406. lock sync.Mutex
  407. offset int64
  408. metadata string
  409. dirty bool
  410. done bool
  411. releaseOnce sync.Once
  412. errors chan *ConsumerError
  413. }
  414. func (om *offsetManager) newPartitionOffsetManager(topic string, partition int32) (*partitionOffsetManager, error) {
  415. offset, metadata, err := om.fetchInitialOffset(topic, partition, om.conf.Metadata.Retry.Max)
  416. if err != nil {
  417. return nil, err
  418. }
  419. return &partitionOffsetManager{
  420. parent: om,
  421. topic: topic,
  422. partition: partition,
  423. errors: make(chan *ConsumerError, om.conf.ChannelBufferSize),
  424. offset: offset,
  425. metadata: metadata,
  426. }, nil
  427. }
  428. func (pom *partitionOffsetManager) Errors() <-chan *ConsumerError {
  429. return pom.errors
  430. }
  431. func (pom *partitionOffsetManager) MarkOffset(offset int64, metadata string) {
  432. pom.lock.Lock()
  433. defer pom.lock.Unlock()
  434. if offset > pom.offset {
  435. pom.offset = offset
  436. pom.metadata = metadata
  437. pom.dirty = true
  438. }
  439. }
  440. func (pom *partitionOffsetManager) ResetOffset(offset int64, metadata string) {
  441. pom.lock.Lock()
  442. defer pom.lock.Unlock()
  443. if offset <= pom.offset {
  444. pom.offset = offset
  445. pom.metadata = metadata
  446. pom.dirty = true
  447. }
  448. }
  449. func (pom *partitionOffsetManager) updateCommitted(offset int64, metadata string) {
  450. pom.lock.Lock()
  451. defer pom.lock.Unlock()
  452. if pom.offset == offset && pom.metadata == metadata {
  453. pom.dirty = false
  454. }
  455. }
  456. func (pom *partitionOffsetManager) NextOffset() (int64, string) {
  457. pom.lock.Lock()
  458. defer pom.lock.Unlock()
  459. if pom.offset >= 0 {
  460. return pom.offset, pom.metadata
  461. }
  462. return pom.parent.conf.Consumer.Offsets.Initial, ""
  463. }
  464. func (pom *partitionOffsetManager) AsyncClose() {
  465. pom.lock.Lock()
  466. pom.done = true
  467. pom.lock.Unlock()
  468. }
  469. func (pom *partitionOffsetManager) Close() error {
  470. pom.AsyncClose()
  471. var errors ConsumerErrors
  472. for err := range pom.errors {
  473. errors = append(errors, err)
  474. }
  475. if len(errors) > 0 {
  476. return errors
  477. }
  478. return nil
  479. }
  480. func (pom *partitionOffsetManager) handleError(err error) {
  481. cErr := &ConsumerError{
  482. Topic: pom.topic,
  483. Partition: pom.partition,
  484. Err: err,
  485. }
  486. if pom.parent.conf.Consumer.Return.Errors {
  487. pom.errors <- cErr
  488. } else {
  489. Logger.Println(cErr)
  490. }
  491. }
  492. func (pom *partitionOffsetManager) release() {
  493. pom.releaseOnce.Do(func() {
  494. close(pom.errors)
  495. })
  496. }