offset_manager.go 14 KB

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