offset_manager.go 14 KB

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