offset_manager.go 15 KB

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