offset_manager.go 15 KB

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