offset_manager.go 15 KB

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