offset_manager.go 15 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560
  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. // ResetOffset resets to the provided offset, alongside a metadata string that
  125. // represents the state of the partition consumer at that point in time. Reset
  126. // acts as a counterpart to MarkOffset, the difference being that it allows to
  127. // reset an offset to an earlier or smaller value, where MarkOffset only
  128. // allows incrementing the offset. cf MarkOffset for more details.
  129. ResetOffset(offset int64, metadata string)
  130. // Errors returns a read channel of errors that occur during offset management, if
  131. // enabled. By default, errors are logged and not returned over this channel. If
  132. // you want to implement any custom error handling, set your config's
  133. // Consumer.Return.Errors setting to true, and read from this channel.
  134. Errors() <-chan *ConsumerError
  135. // AsyncClose initiates a shutdown of the PartitionOffsetManager. This method will
  136. // return immediately, after which you should wait until the 'errors' channel has
  137. // been drained and closed. It is required to call this function, or Close before
  138. // a consumer object passes out of scope, as it will otherwise leak memory. You
  139. // must call this before calling Close on the underlying client.
  140. AsyncClose()
  141. // Close stops the PartitionOffsetManager from managing offsets. It is required to
  142. // call this function (or AsyncClose) before a PartitionOffsetManager object
  143. // passes out of scope, as it will otherwise leak memory. You must call this
  144. // before calling Close on the underlying client.
  145. Close() error
  146. }
  147. type partitionOffsetManager struct {
  148. parent *offsetManager
  149. topic string
  150. partition int32
  151. lock sync.Mutex
  152. offset int64
  153. metadata string
  154. dirty bool
  155. clean sync.Cond
  156. broker *brokerOffsetManager
  157. errors chan *ConsumerError
  158. rebalance chan none
  159. dying chan none
  160. }
  161. func (om *offsetManager) newPartitionOffsetManager(topic string, partition int32) (*partitionOffsetManager, error) {
  162. pom := &partitionOffsetManager{
  163. parent: om,
  164. topic: topic,
  165. partition: partition,
  166. errors: make(chan *ConsumerError, om.conf.ChannelBufferSize),
  167. rebalance: make(chan none, 1),
  168. dying: make(chan none),
  169. }
  170. pom.clean.L = &pom.lock
  171. if err := pom.selectBroker(); err != nil {
  172. return nil, err
  173. }
  174. if err := pom.fetchInitialOffset(om.conf.Metadata.Retry.Max); err != nil {
  175. return nil, err
  176. }
  177. pom.broker.updateSubscriptions <- pom
  178. go withRecover(pom.mainLoop)
  179. return pom, nil
  180. }
  181. func (pom *partitionOffsetManager) mainLoop() {
  182. for {
  183. select {
  184. case <-pom.rebalance:
  185. if err := pom.selectBroker(); err != nil {
  186. pom.handleError(err)
  187. pom.rebalance <- none{}
  188. } else {
  189. pom.broker.updateSubscriptions <- pom
  190. }
  191. case <-pom.dying:
  192. if pom.broker != nil {
  193. select {
  194. case <-pom.rebalance:
  195. case pom.broker.updateSubscriptions <- pom:
  196. }
  197. pom.parent.unrefBrokerOffsetManager(pom.broker)
  198. }
  199. pom.parent.abandonPartitionOffsetManager(pom)
  200. close(pom.errors)
  201. return
  202. }
  203. }
  204. }
  205. func (pom *partitionOffsetManager) selectBroker() error {
  206. if pom.broker != nil {
  207. pom.parent.unrefBrokerOffsetManager(pom.broker)
  208. pom.broker = nil
  209. }
  210. var broker *Broker
  211. var err error
  212. if err = pom.parent.client.RefreshCoordinator(pom.parent.group); err != nil {
  213. return err
  214. }
  215. if broker, err = pom.parent.client.Coordinator(pom.parent.group); err != nil {
  216. return err
  217. }
  218. pom.broker = pom.parent.refBrokerOffsetManager(broker)
  219. return nil
  220. }
  221. func (pom *partitionOffsetManager) fetchInitialOffset(retries int) error {
  222. request := new(OffsetFetchRequest)
  223. request.Version = 1
  224. request.ConsumerGroup = pom.parent.group
  225. request.AddPartition(pom.topic, pom.partition)
  226. response, err := pom.broker.broker.FetchOffset(request)
  227. if err != nil {
  228. return err
  229. }
  230. block := response.GetBlock(pom.topic, pom.partition)
  231. if block == nil {
  232. return ErrIncompleteResponse
  233. }
  234. switch block.Err {
  235. case ErrNoError:
  236. pom.offset = block.Offset
  237. pom.metadata = block.Metadata
  238. return nil
  239. case ErrNotCoordinatorForConsumer:
  240. if retries <= 0 {
  241. return block.Err
  242. }
  243. if err := pom.selectBroker(); err != nil {
  244. return err
  245. }
  246. return pom.fetchInitialOffset(retries - 1)
  247. case ErrOffsetsLoadInProgress:
  248. if retries <= 0 {
  249. return block.Err
  250. }
  251. time.Sleep(pom.parent.conf.Metadata.Retry.Backoff)
  252. return pom.fetchInitialOffset(retries - 1)
  253. default:
  254. return block.Err
  255. }
  256. }
  257. func (pom *partitionOffsetManager) handleError(err error) {
  258. cErr := &ConsumerError{
  259. Topic: pom.topic,
  260. Partition: pom.partition,
  261. Err: err,
  262. }
  263. if pom.parent.conf.Consumer.Return.Errors {
  264. pom.errors <- cErr
  265. } else {
  266. Logger.Println(cErr)
  267. }
  268. }
  269. func (pom *partitionOffsetManager) Errors() <-chan *ConsumerError {
  270. return pom.errors
  271. }
  272. func (pom *partitionOffsetManager) MarkOffset(offset int64, metadata string) {
  273. pom.lock.Lock()
  274. defer pom.lock.Unlock()
  275. if offset > pom.offset {
  276. pom.offset = offset
  277. pom.metadata = metadata
  278. pom.dirty = true
  279. }
  280. }
  281. func (pom *partitionOffsetManager) ResetOffset(offset int64, metadata string) {
  282. pom.lock.Lock()
  283. defer pom.lock.Unlock()
  284. if offset <= pom.offset {
  285. pom.offset = offset
  286. pom.metadata = metadata
  287. pom.dirty = true
  288. }
  289. }
  290. func (pom *partitionOffsetManager) updateCommitted(offset int64, metadata string) {
  291. pom.lock.Lock()
  292. defer pom.lock.Unlock()
  293. if pom.offset == offset && pom.metadata == metadata {
  294. pom.dirty = false
  295. pom.clean.Signal()
  296. }
  297. }
  298. func (pom *partitionOffsetManager) NextOffset() (int64, string) {
  299. pom.lock.Lock()
  300. defer pom.lock.Unlock()
  301. if pom.offset >= 0 {
  302. return pom.offset, pom.metadata
  303. }
  304. return pom.parent.conf.Consumer.Offsets.Initial, ""
  305. }
  306. func (pom *partitionOffsetManager) AsyncClose() {
  307. go func() {
  308. pom.lock.Lock()
  309. defer pom.lock.Unlock()
  310. for pom.dirty {
  311. pom.clean.Wait()
  312. }
  313. close(pom.dying)
  314. }()
  315. }
  316. func (pom *partitionOffsetManager) Close() error {
  317. pom.AsyncClose()
  318. var errors ConsumerErrors
  319. for err := range pom.errors {
  320. errors = append(errors, err)
  321. }
  322. if len(errors) > 0 {
  323. return errors
  324. }
  325. return nil
  326. }
  327. // Broker Offset Manager
  328. type brokerOffsetManager struct {
  329. parent *offsetManager
  330. broker *Broker
  331. timer *time.Ticker
  332. updateSubscriptions chan *partitionOffsetManager
  333. subscriptions map[*partitionOffsetManager]none
  334. refs int
  335. }
  336. func (om *offsetManager) newBrokerOffsetManager(broker *Broker) *brokerOffsetManager {
  337. bom := &brokerOffsetManager{
  338. parent: om,
  339. broker: broker,
  340. timer: time.NewTicker(om.conf.Consumer.Offsets.CommitInterval),
  341. updateSubscriptions: make(chan *partitionOffsetManager),
  342. subscriptions: make(map[*partitionOffsetManager]none),
  343. }
  344. go withRecover(bom.mainLoop)
  345. return bom
  346. }
  347. func (bom *brokerOffsetManager) mainLoop() {
  348. for {
  349. select {
  350. case <-bom.timer.C:
  351. if len(bom.subscriptions) > 0 {
  352. bom.flushToBroker()
  353. }
  354. case s, ok := <-bom.updateSubscriptions:
  355. if !ok {
  356. bom.timer.Stop()
  357. return
  358. }
  359. if _, ok := bom.subscriptions[s]; ok {
  360. delete(bom.subscriptions, s)
  361. } else {
  362. bom.subscriptions[s] = none{}
  363. }
  364. }
  365. }
  366. }
  367. func (bom *brokerOffsetManager) flushToBroker() {
  368. request := bom.constructRequest()
  369. if request == nil {
  370. return
  371. }
  372. response, err := bom.broker.CommitOffset(request)
  373. if err != nil {
  374. bom.abort(err)
  375. return
  376. }
  377. for s := range bom.subscriptions {
  378. if request.blocks[s.topic] == nil || request.blocks[s.topic][s.partition] == nil {
  379. continue
  380. }
  381. var err KError
  382. var ok bool
  383. if response.Errors[s.topic] == nil {
  384. s.handleError(ErrIncompleteResponse)
  385. delete(bom.subscriptions, s)
  386. s.rebalance <- none{}
  387. continue
  388. }
  389. if err, ok = response.Errors[s.topic][s.partition]; !ok {
  390. s.handleError(ErrIncompleteResponse)
  391. delete(bom.subscriptions, s)
  392. s.rebalance <- none{}
  393. continue
  394. }
  395. switch err {
  396. case ErrNoError:
  397. block := request.blocks[s.topic][s.partition]
  398. s.updateCommitted(block.offset, block.metadata)
  399. case ErrNotLeaderForPartition, ErrLeaderNotAvailable,
  400. ErrConsumerCoordinatorNotAvailable, ErrNotCoordinatorForConsumer:
  401. // not a critical error, we just need to redispatch
  402. delete(bom.subscriptions, s)
  403. s.rebalance <- none{}
  404. case ErrOffsetMetadataTooLarge, ErrInvalidCommitOffsetSize:
  405. // nothing we can do about this, just tell the user and carry on
  406. s.handleError(err)
  407. case ErrOffsetsLoadInProgress:
  408. // nothing wrong but we didn't commit, we'll get it next time round
  409. break
  410. case ErrUnknownTopicOrPartition:
  411. // let the user know *and* try redispatching - if topic-auto-create is
  412. // enabled, redispatching should trigger a metadata request and create the
  413. // topic; if not then re-dispatching won't help, but we've let the user
  414. // know and it shouldn't hurt either (see https://github.com/Shopify/sarama/issues/706)
  415. fallthrough
  416. default:
  417. // dunno, tell the user and try redispatching
  418. s.handleError(err)
  419. delete(bom.subscriptions, s)
  420. s.rebalance <- none{}
  421. }
  422. }
  423. }
  424. func (bom *brokerOffsetManager) constructRequest() *OffsetCommitRequest {
  425. var r *OffsetCommitRequest
  426. var perPartitionTimestamp int64
  427. if bom.parent.conf.Consumer.Offsets.Retention == 0 {
  428. perPartitionTimestamp = ReceiveTime
  429. r = &OffsetCommitRequest{
  430. Version: 1,
  431. ConsumerGroup: bom.parent.group,
  432. ConsumerGroupGeneration: GroupGenerationUndefined,
  433. }
  434. } else {
  435. r = &OffsetCommitRequest{
  436. Version: 2,
  437. RetentionTime: int64(bom.parent.conf.Consumer.Offsets.Retention / time.Millisecond),
  438. ConsumerGroup: bom.parent.group,
  439. ConsumerGroupGeneration: GroupGenerationUndefined,
  440. }
  441. }
  442. for s := range bom.subscriptions {
  443. s.lock.Lock()
  444. if s.dirty {
  445. r.AddBlock(s.topic, s.partition, s.offset, perPartitionTimestamp, s.metadata)
  446. }
  447. s.lock.Unlock()
  448. }
  449. if len(r.blocks) > 0 {
  450. return r
  451. }
  452. return nil
  453. }
  454. func (bom *brokerOffsetManager) abort(err error) {
  455. _ = bom.broker.Close() // we don't care about the error this might return, we already have one
  456. bom.parent.abandonBroker(bom)
  457. for pom := range bom.subscriptions {
  458. pom.handleError(err)
  459. pom.rebalance <- none{}
  460. }
  461. for s := range bom.updateSubscriptions {
  462. if _, ok := bom.subscriptions[s]; !ok {
  463. s.handleError(err)
  464. s.rebalance <- none{}
  465. }
  466. }
  467. bom.subscriptions = make(map[*partitionOffsetManager]none)
  468. }