offset_manager.go 14 KB

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