offset_manager.go 15 KB

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