offset_manager.go 14 KB

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