consumer.go 36 KB

12345678910111213141516171819202122232425262728293031323334353637383940414243444546474849505152535455565758596061626364656667686970717273747576777879808182838485868788899091929394959697989910010110210310410510610710810911011111211311411511611711811912012112212312412512612712812913013113213313413513613713813914014114214314414514614714814915015115215315415515615715815916016116216316416516616716816917017117217317417517617717817918018118218318418518618718818919019119219319419519619719819920020120220320420520620720820921021121221321421521621721821922022122222322422522622722822923023123223323423523623723823924024124224324424524624724824925025125225325425525625725825926026126226326426526626726826927027127227327427527627727827928028128228328428528628728828929029129229329429529629729829930030130230330430530630730830931031131231331431531631731831932032132232332432532632732832933033133233333433533633733833934034134234334434534634734834935035135235335435535635735835936036136236336436536636736836937037137237337437537637737837938038138238338438538638738838939039139239339439539639739839940040140240340440540640740840941041141241341441541641741841942042142242342442542642742842943043143243343443543643743843944044144244344444544644744844945045145245345445545645745845946046146246346446546646746846947047147247347447547647747847948048148248348448548648748848949049149249349449549649749849950050150250350450550650750850951051151251351451551651751851952052152252352452552652752852953053153253353453553653753853954054154254354454554654754854955055155255355455555655755855956056156256356456556656756856957057157257357457557657757857958058158258358458558658758858959059159259359459559659759859960060160260360460560660760860961061161261361461561661761861962062162262362462562662762862963063163263363463563663763863964064164264364464564664764864965065165265365465565665765865966066166266366466566666766866967067167267367467567667767867968068168268368468568668768868969069169269369469569669769869970070170270370470570670770870971071171271371471571671771871972072172272372472572672772872973073173273373473573673773873974074174274374474574674774874975075175275375475575675775875976076176276376476576676776876977077177277377477577677777877978078178278378478578678778878979079179279379479579679779879980080180280380480580680780880981081181281381481581681781881982082182282382482582682782882983083183283383483583683783883984084184284384484584684784884985085185285385485585685785885986086186286386486586686786886987087187287387487587687787887988088188288388488588688788888989089189289389489589689789889990090190290390490590690790890991091191291391491591691791891992092192292392492592692792892993093193293393493593693793893994094194294394494594694794894995095195295395495595695795895996096196296396496596696796896997097197297397497597697797897998098198298398498598698798898999099199299399499599699799899910001001100210031004100510061007100810091010101110121013101410151016101710181019102010211022102310241025102610271028102910301031103210331034103510361037103810391040104110421043104410451046104710481049105010511052105310541055105610571058105910601061106210631064106510661067106810691070107110721073107410751076107710781079108010811082108310841085108610871088108910901091109210931094109510961097109810991100110111021103110411051106110711081109111011111112111311141115111611171118111911201121112211231124112511261127112811291130113111321133113411351136
  1. package sarama
  2. import (
  3. "errors"
  4. "fmt"
  5. "math"
  6. "sync"
  7. "sync/atomic"
  8. "time"
  9. "github.com/rcrowley/go-metrics"
  10. )
  11. // ConsumerMessage encapsulates a Kafka message returned by the consumer.
  12. type ConsumerMessage struct {
  13. Headers []*RecordHeader // only set if kafka is version 0.11+
  14. Timestamp time.Time // only set if kafka is version 0.10+, inner message timestamp
  15. BlockTimestamp time.Time // only set if kafka is version 0.10+, outer (compressed) block timestamp
  16. Key, Value []byte
  17. Topic string
  18. Partition int32
  19. Offset int64
  20. }
  21. // ConsumerError is what is provided to the user when an error occurs.
  22. // It wraps an error and includes the topic and partition.
  23. type ConsumerError struct {
  24. Topic string
  25. Partition int32
  26. Err error
  27. }
  28. func (ce ConsumerError) Error() string {
  29. return fmt.Sprintf("kafka: error while consuming %s/%d: %s", ce.Topic, ce.Partition, ce.Err)
  30. }
  31. func (ce ConsumerError) Unwrap() error {
  32. return ce.Err
  33. }
  34. // ConsumerErrors is a type that wraps a batch of errors and implements the Error interface.
  35. // It can be returned from the PartitionConsumer's Close methods to avoid the need to manually drain errors
  36. // when stopping.
  37. type ConsumerErrors []*ConsumerError
  38. func (ce ConsumerErrors) Error() string {
  39. return fmt.Sprintf("kafka: %d errors while consuming", len(ce))
  40. }
  41. // Consumer manages PartitionConsumers which process Kafka messages from brokers. You MUST call Close()
  42. // on a consumer to avoid leaks, it will not be garbage-collected automatically when it passes out of
  43. // scope.
  44. type Consumer interface {
  45. // Topics returns the set of available topics as retrieved from the cluster
  46. // metadata. This method is the same as Client.Topics(), and is provided for
  47. // convenience.
  48. Topics() ([]string, error)
  49. // Partitions returns the sorted list of all partition IDs for the given topic.
  50. // This method is the same as Client.Partitions(), and is provided for convenience.
  51. Partitions(topic string) ([]int32, error)
  52. // ConsumePartition creates a PartitionConsumer on the given topic/partition with
  53. // the given offset. It will return an error if this Consumer is already consuming
  54. // on the given topic/partition. Offset can be a literal offset, or OffsetNewest
  55. // or OffsetOldest
  56. ConsumePartition(topic string, partition int32, offset int64) (PartitionConsumer, error)
  57. // HighWaterMarks returns the current high water marks for each topic and partition.
  58. // Consistency between partitions is not guaranteed since high water marks are updated separately.
  59. HighWaterMarks() map[string]map[int32]int64
  60. // Close shuts down the consumer. It must be called after all child
  61. // PartitionConsumers have already been closed.
  62. Close() error
  63. // Pause suspends fetching from the requested partitions. Future calls to the broker will not return any
  64. // records from these partitions until they have been resumed using Resume()/ResumeAll().
  65. // Note that this method does not affect partition subscription.
  66. // In particular, it does not cause a group rebalance when automatic assignment is used.
  67. Pause(topicPartitions map[string][]int32)
  68. // Resume resumes specified partitions which have been paused with Pause()/PauseAll().
  69. // New calls to the broker will return records from these partitions if there are any to be fetched.
  70. Resume(topicPartitions map[string][]int32)
  71. // PauseAll suspends fetching from all partitions. Future calls to the broker will not return any
  72. // records from these partitions until they have been resumed using Resume()/ResumeAll().
  73. // Note that this method does not affect partition subscription.
  74. // In particular, it does not cause a group rebalance when automatic assignment is used.
  75. PauseAll()
  76. // ResumeAll resumes all partitions which have been paused with Pause()/PauseAll().
  77. // New calls to the broker will return records from these partitions if there are any to be fetched.
  78. ResumeAll()
  79. }
  80. // max time to wait for more partition subscriptions
  81. const partitionConsumersBatchTimeout = 100 * time.Millisecond
  82. type consumer struct {
  83. conf *Config
  84. children map[string]map[int32]*partitionConsumer
  85. brokerConsumers map[*Broker]*brokerConsumer
  86. client Client
  87. metricRegistry metrics.Registry
  88. lock sync.Mutex
  89. }
  90. // NewConsumer creates a new consumer using the given broker addresses and configuration.
  91. func NewConsumer(addrs []string, config *Config) (Consumer, error) {
  92. client, err := NewClient(addrs, config)
  93. if err != nil {
  94. return nil, err
  95. }
  96. return newConsumer(client)
  97. }
  98. // NewConsumerFromClient creates a new consumer using the given client. It is still
  99. // necessary to call Close() on the underlying client when shutting down this consumer.
  100. func NewConsumerFromClient(client Client) (Consumer, error) {
  101. // For clients passed in by the client, ensure we don't
  102. // call Close() on it.
  103. cli := &nopCloserClient{client}
  104. return newConsumer(cli)
  105. }
  106. func newConsumer(client Client) (Consumer, error) {
  107. // Check that we are not dealing with a closed Client before processing any other arguments
  108. if client.Closed() {
  109. return nil, ErrClosedClient
  110. }
  111. c := &consumer{
  112. client: client,
  113. conf: client.Config(),
  114. children: make(map[string]map[int32]*partitionConsumer),
  115. brokerConsumers: make(map[*Broker]*brokerConsumer),
  116. metricRegistry: newCleanupRegistry(client.Config().MetricRegistry),
  117. }
  118. return c, nil
  119. }
  120. func (c *consumer) Close() error {
  121. c.metricRegistry.UnregisterAll()
  122. return c.client.Close()
  123. }
  124. func (c *consumer) Topics() ([]string, error) {
  125. return c.client.Topics()
  126. }
  127. func (c *consumer) Partitions(topic string) ([]int32, error) {
  128. return c.client.Partitions(topic)
  129. }
  130. func (c *consumer) ConsumePartition(topic string, partition int32, offset int64) (PartitionConsumer, error) {
  131. child := &partitionConsumer{
  132. consumer: c,
  133. conf: c.conf,
  134. topic: topic,
  135. partition: partition,
  136. messages: make(chan *ConsumerMessage, c.conf.ChannelBufferSize),
  137. errors: make(chan *ConsumerError, c.conf.ChannelBufferSize),
  138. feeder: make(chan *FetchResponse, 1),
  139. leaderEpoch: invalidLeaderEpoch,
  140. preferredReadReplica: invalidPreferredReplicaID,
  141. trigger: make(chan none, 1),
  142. dying: make(chan none),
  143. fetchSize: c.conf.Consumer.Fetch.Default,
  144. }
  145. if err := child.chooseStartingOffset(offset); err != nil {
  146. return nil, err
  147. }
  148. leader, epoch, err := c.client.LeaderAndEpoch(child.topic, child.partition)
  149. if err != nil {
  150. return nil, err
  151. }
  152. if err := c.addChild(child); err != nil {
  153. return nil, err
  154. }
  155. go withRecover(child.dispatcher)
  156. go withRecover(child.responseFeeder)
  157. child.leaderEpoch = epoch
  158. child.broker = c.refBrokerConsumer(leader)
  159. child.broker.input <- child
  160. return child, nil
  161. }
  162. func (c *consumer) HighWaterMarks() map[string]map[int32]int64 {
  163. c.lock.Lock()
  164. defer c.lock.Unlock()
  165. hwms := make(map[string]map[int32]int64)
  166. for topic, p := range c.children {
  167. hwm := make(map[int32]int64, len(p))
  168. for partition, pc := range p {
  169. hwm[partition] = pc.HighWaterMarkOffset()
  170. }
  171. hwms[topic] = hwm
  172. }
  173. return hwms
  174. }
  175. func (c *consumer) addChild(child *partitionConsumer) error {
  176. c.lock.Lock()
  177. defer c.lock.Unlock()
  178. topicChildren := c.children[child.topic]
  179. if topicChildren == nil {
  180. topicChildren = make(map[int32]*partitionConsumer)
  181. c.children[child.topic] = topicChildren
  182. }
  183. if topicChildren[child.partition] != nil {
  184. return ConfigurationError("That topic/partition is already being consumed")
  185. }
  186. topicChildren[child.partition] = child
  187. return nil
  188. }
  189. func (c *consumer) removeChild(child *partitionConsumer) {
  190. c.lock.Lock()
  191. defer c.lock.Unlock()
  192. delete(c.children[child.topic], child.partition)
  193. }
  194. func (c *consumer) refBrokerConsumer(broker *Broker) *brokerConsumer {
  195. c.lock.Lock()
  196. defer c.lock.Unlock()
  197. bc := c.brokerConsumers[broker]
  198. if bc == nil {
  199. bc = c.newBrokerConsumer(broker)
  200. c.brokerConsumers[broker] = bc
  201. }
  202. bc.refs++
  203. return bc
  204. }
  205. func (c *consumer) unrefBrokerConsumer(brokerWorker *brokerConsumer) {
  206. c.lock.Lock()
  207. defer c.lock.Unlock()
  208. brokerWorker.refs--
  209. if brokerWorker.refs == 0 {
  210. close(brokerWorker.input)
  211. if c.brokerConsumers[brokerWorker.broker] == brokerWorker {
  212. delete(c.brokerConsumers, brokerWorker.broker)
  213. }
  214. }
  215. }
  216. func (c *consumer) abandonBrokerConsumer(brokerWorker *brokerConsumer) {
  217. c.lock.Lock()
  218. defer c.lock.Unlock()
  219. delete(c.brokerConsumers, brokerWorker.broker)
  220. }
  221. // Pause implements Consumer.
  222. func (c *consumer) Pause(topicPartitions map[string][]int32) {
  223. c.lock.Lock()
  224. defer c.lock.Unlock()
  225. for topic, partitions := range topicPartitions {
  226. for _, partition := range partitions {
  227. if topicConsumers, ok := c.children[topic]; ok {
  228. if partitionConsumer, ok := topicConsumers[partition]; ok {
  229. partitionConsumer.Pause()
  230. }
  231. }
  232. }
  233. }
  234. }
  235. // Resume implements Consumer.
  236. func (c *consumer) Resume(topicPartitions map[string][]int32) {
  237. c.lock.Lock()
  238. defer c.lock.Unlock()
  239. for topic, partitions := range topicPartitions {
  240. for _, partition := range partitions {
  241. if topicConsumers, ok := c.children[topic]; ok {
  242. if partitionConsumer, ok := topicConsumers[partition]; ok {
  243. partitionConsumer.Resume()
  244. }
  245. }
  246. }
  247. }
  248. }
  249. // PauseAll implements Consumer.
  250. func (c *consumer) PauseAll() {
  251. c.lock.Lock()
  252. defer c.lock.Unlock()
  253. for _, partitions := range c.children {
  254. for _, partitionConsumer := range partitions {
  255. partitionConsumer.Pause()
  256. }
  257. }
  258. }
  259. // ResumeAll implements Consumer.
  260. func (c *consumer) ResumeAll() {
  261. c.lock.Lock()
  262. defer c.lock.Unlock()
  263. for _, partitions := range c.children {
  264. for _, partitionConsumer := range partitions {
  265. partitionConsumer.Resume()
  266. }
  267. }
  268. }
  269. // PartitionConsumer
  270. // PartitionConsumer processes Kafka messages from a given topic and partition. You MUST call one of Close() or
  271. // AsyncClose() on a PartitionConsumer to avoid leaks; it will not be garbage-collected automatically when it passes out
  272. // of scope.
  273. //
  274. // The simplest way of using a PartitionConsumer is to loop over its Messages channel using a for/range
  275. // loop. The PartitionConsumer will only stop itself in one case: when the offset being consumed is reported
  276. // as out of range by the brokers. In this case you should decide what you want to do (try a different offset,
  277. // notify a human, etc) and handle it appropriately. For all other error cases, it will just keep retrying.
  278. // By default, it logs these errors to sarama.Logger; if you want to be notified directly of all errors, set
  279. // your config's Consumer.Return.Errors to true and read from the Errors channel, using a select statement
  280. // or a separate goroutine. Check out the Consumer examples to see implementations of these different approaches.
  281. //
  282. // To terminate such a for/range loop while the loop is executing, call AsyncClose. This will kick off the process of
  283. // consumer tear-down & return immediately. Continue to loop, servicing the Messages channel until the teardown process
  284. // AsyncClose initiated closes it (thus terminating the for/range loop). If you've already ceased reading Messages, call
  285. // Close; this will signal the PartitionConsumer's goroutines to begin shutting down (just like AsyncClose), but will
  286. // also drain the Messages channel, harvest all errors & return them once cleanup has completed.
  287. type PartitionConsumer interface {
  288. // AsyncClose initiates a shutdown of the PartitionConsumer. This method will return immediately, after which you
  289. // should continue to service the 'Messages' and 'Errors' channels until they are empty. It is required to call this
  290. // function, or Close before a consumer object passes out of scope, as it will otherwise leak memory. You must call
  291. // this before calling Close on the underlying client.
  292. AsyncClose()
  293. // Close stops the PartitionConsumer from fetching messages. It will initiate a shutdown just like AsyncClose, drain
  294. // the Messages channel, harvest any errors & return them to the caller. Note that if you are continuing to service
  295. // the Messages channel when this function is called, you will be competing with Close for messages; consider
  296. // calling AsyncClose, instead. It is required to call this function (or AsyncClose) before a consumer object passes
  297. // out of scope, as it will otherwise leak memory. You must call this before calling Close on the underlying client.
  298. Close() error
  299. // Messages returns the read channel for the messages that are returned by
  300. // the broker.
  301. Messages() <-chan *ConsumerMessage
  302. // Errors returns a read channel of errors that occurred during consuming, if
  303. // enabled. By default, errors are logged and not returned over this channel.
  304. // If you want to implement any custom error handling, set your config's
  305. // Consumer.Return.Errors setting to true, and read from this channel.
  306. Errors() <-chan *ConsumerError
  307. // HighWaterMarkOffset returns the high water mark offset of the partition,
  308. // i.e. the offset that will be used for the next message that will be produced.
  309. // You can use this to determine how far behind the processing is.
  310. HighWaterMarkOffset() int64
  311. // Pause suspends fetching from this partition. Future calls to the broker will not return
  312. // any records from these partition until it have been resumed using Resume().
  313. // Note that this method does not affect partition subscription.
  314. // In particular, it does not cause a group rebalance when automatic assignment is used.
  315. Pause()
  316. // Resume resumes this partition which have been paused with Pause().
  317. // New calls to the broker will return records from these partitions if there are any to be fetched.
  318. // If the partition was not previously paused, this method is a no-op.
  319. Resume()
  320. // IsPaused indicates if this partition consumer is paused or not
  321. IsPaused() bool
  322. }
  323. type partitionConsumer struct {
  324. highWaterMarkOffset int64 // must be at the top of the struct because https://golang.org/pkg/sync/atomic/#pkg-note-BUG
  325. consumer *consumer
  326. conf *Config
  327. broker *brokerConsumer
  328. messages chan *ConsumerMessage
  329. errors chan *ConsumerError
  330. feeder chan *FetchResponse
  331. leaderEpoch int32
  332. preferredReadReplica int32
  333. trigger, dying chan none
  334. closeOnce sync.Once
  335. topic string
  336. partition int32
  337. responseResult error
  338. fetchSize int32
  339. offset int64
  340. retries int32
  341. paused int32
  342. }
  343. var errTimedOut = errors.New("timed out feeding messages to the user") // not user-facing
  344. func (child *partitionConsumer) sendError(err error) {
  345. cErr := &ConsumerError{
  346. Topic: child.topic,
  347. Partition: child.partition,
  348. Err: err,
  349. }
  350. if child.conf.Consumer.Return.Errors {
  351. child.errors <- cErr
  352. } else {
  353. Logger.Println(cErr)
  354. }
  355. }
  356. func (child *partitionConsumer) computeBackoff() time.Duration {
  357. if child.conf.Consumer.Retry.BackoffFunc != nil {
  358. retries := atomic.AddInt32(&child.retries, 1)
  359. return child.conf.Consumer.Retry.BackoffFunc(int(retries))
  360. }
  361. return child.conf.Consumer.Retry.Backoff
  362. }
  363. func (child *partitionConsumer) dispatcher() {
  364. for range child.trigger {
  365. select {
  366. case <-child.dying:
  367. close(child.trigger)
  368. case <-time.After(child.computeBackoff()):
  369. if child.broker != nil {
  370. child.consumer.unrefBrokerConsumer(child.broker)
  371. child.broker = nil
  372. }
  373. if err := child.dispatch(); err != nil {
  374. child.sendError(err)
  375. child.trigger <- none{}
  376. }
  377. }
  378. }
  379. if child.broker != nil {
  380. child.consumer.unrefBrokerConsumer(child.broker)
  381. }
  382. child.consumer.removeChild(child)
  383. close(child.feeder)
  384. }
  385. func (child *partitionConsumer) preferredBroker() (*Broker, int32, error) {
  386. if child.preferredReadReplica >= 0 {
  387. broker, err := child.consumer.client.Broker(child.preferredReadReplica)
  388. if err == nil {
  389. return broker, child.leaderEpoch, nil
  390. }
  391. Logger.Printf(
  392. "consumer/%s/%d failed to find active broker for preferred read replica %d - will fallback to leader",
  393. child.topic, child.partition, child.preferredReadReplica)
  394. // if we couldn't find it, discard the replica preference and trigger a
  395. // metadata refresh whilst falling back to consuming from the leader again
  396. child.preferredReadReplica = invalidPreferredReplicaID
  397. _ = child.consumer.client.RefreshMetadata(child.topic)
  398. }
  399. // if preferred replica cannot be found fallback to leader
  400. return child.consumer.client.LeaderAndEpoch(child.topic, child.partition)
  401. }
  402. func (child *partitionConsumer) dispatch() error {
  403. if err := child.consumer.client.RefreshMetadata(child.topic); err != nil {
  404. return err
  405. }
  406. broker, epoch, err := child.preferredBroker()
  407. if err != nil {
  408. return err
  409. }
  410. child.leaderEpoch = epoch
  411. child.broker = child.consumer.refBrokerConsumer(broker)
  412. child.broker.input <- child
  413. return nil
  414. }
  415. func (child *partitionConsumer) chooseStartingOffset(offset int64) error {
  416. newestOffset, err := child.consumer.client.GetOffset(child.topic, child.partition, OffsetNewest)
  417. if err != nil {
  418. return err
  419. }
  420. child.highWaterMarkOffset = newestOffset
  421. oldestOffset, err := child.consumer.client.GetOffset(child.topic, child.partition, OffsetOldest)
  422. if err != nil {
  423. return err
  424. }
  425. switch {
  426. case offset == OffsetNewest:
  427. child.offset = newestOffset
  428. case offset == OffsetOldest:
  429. child.offset = oldestOffset
  430. case offset >= oldestOffset && offset <= newestOffset:
  431. child.offset = offset
  432. default:
  433. return ErrOffsetOutOfRange
  434. }
  435. return nil
  436. }
  437. func (child *partitionConsumer) Messages() <-chan *ConsumerMessage {
  438. return child.messages
  439. }
  440. func (child *partitionConsumer) Errors() <-chan *ConsumerError {
  441. return child.errors
  442. }
  443. func (child *partitionConsumer) AsyncClose() {
  444. // this triggers whatever broker owns this child to abandon it and close its trigger channel, which causes
  445. // the dispatcher to exit its loop, which removes it from the consumer then closes its 'messages' and
  446. // 'errors' channel (alternatively, if the child is already at the dispatcher for some reason, that will
  447. // also just close itself)
  448. child.closeOnce.Do(func() {
  449. close(child.dying)
  450. })
  451. }
  452. func (child *partitionConsumer) Close() error {
  453. child.AsyncClose()
  454. var consumerErrors ConsumerErrors
  455. for err := range child.errors {
  456. consumerErrors = append(consumerErrors, err)
  457. }
  458. if len(consumerErrors) > 0 {
  459. return consumerErrors
  460. }
  461. return nil
  462. }
  463. func (child *partitionConsumer) HighWaterMarkOffset() int64 {
  464. return atomic.LoadInt64(&child.highWaterMarkOffset)
  465. }
  466. func (child *partitionConsumer) responseFeeder() {
  467. var msgs []*ConsumerMessage
  468. expiryTicker := time.NewTicker(child.conf.Consumer.MaxProcessingTime)
  469. firstAttempt := true
  470. feederLoop:
  471. for response := range child.feeder {
  472. msgs, child.responseResult = child.parseResponse(response)
  473. if child.responseResult == nil {
  474. atomic.StoreInt32(&child.retries, 0)
  475. }
  476. for i, msg := range msgs {
  477. child.interceptors(msg)
  478. messageSelect:
  479. select {
  480. case <-child.dying:
  481. child.broker.acks.Done()
  482. continue feederLoop
  483. case child.messages <- msg:
  484. firstAttempt = true
  485. case <-expiryTicker.C:
  486. if !firstAttempt {
  487. child.responseResult = errTimedOut
  488. child.broker.acks.Done()
  489. remainingLoop:
  490. for _, msg = range msgs[i:] {
  491. child.interceptors(msg)
  492. select {
  493. case child.messages <- msg:
  494. case <-child.dying:
  495. break remainingLoop
  496. }
  497. }
  498. child.broker.input <- child
  499. continue feederLoop
  500. } else {
  501. // current message has not been sent, return to select
  502. // statement
  503. firstAttempt = false
  504. goto messageSelect
  505. }
  506. }
  507. }
  508. child.broker.acks.Done()
  509. }
  510. expiryTicker.Stop()
  511. close(child.messages)
  512. close(child.errors)
  513. }
  514. func (child *partitionConsumer) parseMessages(msgSet *MessageSet) ([]*ConsumerMessage, error) {
  515. var messages []*ConsumerMessage
  516. for _, msgBlock := range msgSet.Messages {
  517. for _, msg := range msgBlock.Messages() {
  518. offset := msg.Offset
  519. timestamp := msg.Msg.Timestamp
  520. if msg.Msg.Version >= 1 {
  521. baseOffset := msgBlock.Offset - msgBlock.Messages()[len(msgBlock.Messages())-1].Offset
  522. offset += baseOffset
  523. if msg.Msg.LogAppendTime {
  524. timestamp = msgBlock.Msg.Timestamp
  525. }
  526. }
  527. if offset < child.offset {
  528. continue
  529. }
  530. messages = append(messages, &ConsumerMessage{
  531. Topic: child.topic,
  532. Partition: child.partition,
  533. Key: msg.Msg.Key,
  534. Value: msg.Msg.Value,
  535. Offset: offset,
  536. Timestamp: timestamp,
  537. BlockTimestamp: msgBlock.Msg.Timestamp,
  538. })
  539. child.offset = offset + 1
  540. }
  541. }
  542. if len(messages) == 0 {
  543. child.offset++
  544. }
  545. return messages, nil
  546. }
  547. func (child *partitionConsumer) parseRecords(batch *RecordBatch) ([]*ConsumerMessage, error) {
  548. messages := make([]*ConsumerMessage, 0, len(batch.Records))
  549. for _, rec := range batch.Records {
  550. offset := batch.FirstOffset + rec.OffsetDelta
  551. if offset < child.offset {
  552. continue
  553. }
  554. timestamp := batch.FirstTimestamp.Add(rec.TimestampDelta)
  555. if batch.LogAppendTime {
  556. timestamp = batch.MaxTimestamp
  557. }
  558. messages = append(messages, &ConsumerMessage{
  559. Topic: child.topic,
  560. Partition: child.partition,
  561. Key: rec.Key,
  562. Value: rec.Value,
  563. Offset: offset,
  564. Timestamp: timestamp,
  565. Headers: rec.Headers,
  566. })
  567. child.offset = offset + 1
  568. }
  569. if len(messages) == 0 {
  570. child.offset++
  571. }
  572. return messages, nil
  573. }
  574. func (child *partitionConsumer) parseResponse(response *FetchResponse) ([]*ConsumerMessage, error) {
  575. var consumerBatchSizeMetric metrics.Histogram
  576. if child.consumer != nil && child.consumer.metricRegistry != nil {
  577. consumerBatchSizeMetric = getOrRegisterHistogram("consumer-batch-size", child.consumer.metricRegistry)
  578. }
  579. // If request was throttled and empty we log and return without error
  580. if response.ThrottleTime != time.Duration(0) && len(response.Blocks) == 0 {
  581. Logger.Printf(
  582. "consumer/broker/%d FetchResponse throttled %v\n",
  583. child.broker.broker.ID(), response.ThrottleTime)
  584. return nil, nil
  585. }
  586. block := response.GetBlock(child.topic, child.partition)
  587. if block == nil {
  588. return nil, ErrIncompleteResponse
  589. }
  590. if !errors.Is(block.Err, ErrNoError) {
  591. return nil, block.Err
  592. }
  593. nRecs, err := block.numRecords()
  594. if err != nil {
  595. return nil, err
  596. }
  597. if consumerBatchSizeMetric != nil {
  598. consumerBatchSizeMetric.Update(int64(nRecs))
  599. }
  600. if block.PreferredReadReplica != invalidPreferredReplicaID {
  601. child.preferredReadReplica = block.PreferredReadReplica
  602. }
  603. if nRecs == 0 {
  604. partialTrailingMessage, err := block.isPartial()
  605. if err != nil {
  606. return nil, err
  607. }
  608. // We got no messages. If we got a trailing one then we need to ask for more data.
  609. // Otherwise we just poll again and wait for one to be produced...
  610. if partialTrailingMessage {
  611. if child.conf.Consumer.Fetch.Max > 0 && child.fetchSize == child.conf.Consumer.Fetch.Max {
  612. // we can't ask for more data, we've hit the configured limit
  613. child.sendError(ErrMessageTooLarge)
  614. child.offset++ // skip this one so we can keep processing future messages
  615. } else {
  616. child.fetchSize *= 2
  617. // check int32 overflow
  618. if child.fetchSize < 0 {
  619. child.fetchSize = math.MaxInt32
  620. }
  621. if child.conf.Consumer.Fetch.Max > 0 && child.fetchSize > child.conf.Consumer.Fetch.Max {
  622. child.fetchSize = child.conf.Consumer.Fetch.Max
  623. }
  624. }
  625. } else if block.LastRecordsBatchOffset != nil && *block.LastRecordsBatchOffset < block.HighWaterMarkOffset {
  626. // check last record offset to avoid stuck if high watermark was not reached
  627. Logger.Printf("consumer/broker/%d received batch with zero records but high watermark was not reached, topic %s, partition %d, offset %d\n", child.broker.broker.ID(), child.topic, child.partition, *block.LastRecordsBatchOffset)
  628. child.offset = *block.LastRecordsBatchOffset + 1
  629. }
  630. return nil, nil
  631. }
  632. // we got messages, reset our fetch size in case it was increased for a previous request
  633. child.fetchSize = child.conf.Consumer.Fetch.Default
  634. atomic.StoreInt64(&child.highWaterMarkOffset, block.HighWaterMarkOffset)
  635. // abortedProducerIDs contains producerID which message should be ignored as uncommitted
  636. // - producerID are added when the partitionConsumer iterate over the offset at which an aborted transaction begins (abortedTransaction.FirstOffset)
  637. // - producerID are removed when partitionConsumer iterate over an aborted controlRecord, meaning the aborted transaction for this producer is over
  638. abortedProducerIDs := make(map[int64]struct{}, len(block.AbortedTransactions))
  639. abortedTransactions := block.getAbortedTransactions()
  640. var messages []*ConsumerMessage
  641. for _, records := range block.RecordsSet {
  642. switch records.recordsType {
  643. case legacyRecords:
  644. messageSetMessages, err := child.parseMessages(records.MsgSet)
  645. if err != nil {
  646. return nil, err
  647. }
  648. messages = append(messages, messageSetMessages...)
  649. case defaultRecords:
  650. // Consume remaining abortedTransaction up to last offset of current batch
  651. for _, txn := range abortedTransactions {
  652. if txn.FirstOffset > records.RecordBatch.LastOffset() {
  653. break
  654. }
  655. abortedProducerIDs[txn.ProducerID] = struct{}{}
  656. // Pop abortedTransactions so that we never add it again
  657. abortedTransactions = abortedTransactions[1:]
  658. }
  659. recordBatchMessages, err := child.parseRecords(records.RecordBatch)
  660. if err != nil {
  661. return nil, err
  662. }
  663. // Parse and commit offset but do not expose messages that are:
  664. // - control records
  665. // - part of an aborted transaction when set to `ReadCommitted`
  666. // control record
  667. isControl, err := records.isControl()
  668. if err != nil {
  669. // I don't know why there is this continue in case of error to begin with
  670. // Safe bet is to ignore control messages if ReadUncommitted
  671. // and block on them in case of error and ReadCommitted
  672. if child.conf.Consumer.IsolationLevel == ReadCommitted {
  673. return nil, err
  674. }
  675. continue
  676. }
  677. if isControl {
  678. controlRecord, err := records.getControlRecord()
  679. if err != nil {
  680. return nil, err
  681. }
  682. if controlRecord.Type == ControlRecordAbort {
  683. delete(abortedProducerIDs, records.RecordBatch.ProducerID)
  684. }
  685. continue
  686. }
  687. // filter aborted transactions
  688. if child.conf.Consumer.IsolationLevel == ReadCommitted {
  689. _, isAborted := abortedProducerIDs[records.RecordBatch.ProducerID]
  690. if records.RecordBatch.IsTransactional && isAborted {
  691. continue
  692. }
  693. }
  694. messages = append(messages, recordBatchMessages...)
  695. default:
  696. return nil, fmt.Errorf("unknown records type: %v", records.recordsType)
  697. }
  698. }
  699. return messages, nil
  700. }
  701. func (child *partitionConsumer) interceptors(msg *ConsumerMessage) {
  702. for _, interceptor := range child.conf.Consumer.Interceptors {
  703. msg.safelyApplyInterceptor(interceptor)
  704. }
  705. }
  706. // Pause implements PartitionConsumer.
  707. func (child *partitionConsumer) Pause() {
  708. atomic.StoreInt32(&child.paused, 1)
  709. }
  710. // Resume implements PartitionConsumer.
  711. func (child *partitionConsumer) Resume() {
  712. atomic.StoreInt32(&child.paused, 0)
  713. }
  714. // IsPaused implements PartitionConsumer.
  715. func (child *partitionConsumer) IsPaused() bool {
  716. return atomic.LoadInt32(&child.paused) == 1
  717. }
  718. type brokerConsumer struct {
  719. consumer *consumer
  720. broker *Broker
  721. input chan *partitionConsumer
  722. newSubscriptions chan []*partitionConsumer
  723. subscriptions map[*partitionConsumer]none
  724. acks sync.WaitGroup
  725. refs int
  726. }
  727. func (c *consumer) newBrokerConsumer(broker *Broker) *brokerConsumer {
  728. bc := &brokerConsumer{
  729. consumer: c,
  730. broker: broker,
  731. input: make(chan *partitionConsumer),
  732. newSubscriptions: make(chan []*partitionConsumer),
  733. subscriptions: make(map[*partitionConsumer]none),
  734. refs: 0,
  735. }
  736. go withRecover(bc.subscriptionManager)
  737. go withRecover(bc.subscriptionConsumer)
  738. return bc
  739. }
  740. // The subscriptionManager constantly accepts new subscriptions on `input` (even when the main subscriptionConsumer
  741. // goroutine is in the middle of a network request) and batches it up. The main worker goroutine picks
  742. // up a batch of new subscriptions between every network request by reading from `newSubscriptions`, so we give
  743. // it nil if no new subscriptions are available.
  744. func (bc *brokerConsumer) subscriptionManager() {
  745. defer close(bc.newSubscriptions)
  746. for {
  747. var partitionConsumers []*partitionConsumer
  748. // Check for any partition consumer asking to subscribe if there aren't
  749. // any, trigger the network request (to fetch Kafka messages) by sending "nil" to the
  750. // newSubscriptions channel
  751. select {
  752. case pc, ok := <-bc.input:
  753. if !ok {
  754. return
  755. }
  756. partitionConsumers = append(partitionConsumers, pc)
  757. case bc.newSubscriptions <- nil:
  758. continue
  759. }
  760. // drain input of any further incoming subscriptions
  761. timer := time.NewTimer(partitionConsumersBatchTimeout)
  762. for batchComplete := false; !batchComplete; {
  763. select {
  764. case pc := <-bc.input:
  765. partitionConsumers = append(partitionConsumers, pc)
  766. case <-timer.C:
  767. batchComplete = true
  768. }
  769. }
  770. timer.Stop()
  771. Logger.Printf(
  772. "consumer/broker/%d accumulated %d new subscriptions\n",
  773. bc.broker.ID(), len(partitionConsumers))
  774. bc.newSubscriptions <- partitionConsumers
  775. }
  776. }
  777. // subscriptionConsumer ensures we will get nil right away if no new subscriptions is available
  778. // this is the main loop that fetches Kafka messages
  779. func (bc *brokerConsumer) subscriptionConsumer() {
  780. for newSubscriptions := range bc.newSubscriptions {
  781. bc.updateSubscriptions(newSubscriptions)
  782. if len(bc.subscriptions) == 0 {
  783. // We're about to be shut down or we're about to receive more subscriptions.
  784. // Take a small nap to avoid burning the CPU.
  785. time.Sleep(partitionConsumersBatchTimeout)
  786. continue
  787. }
  788. response, err := bc.fetchNewMessages()
  789. if err != nil {
  790. Logger.Printf("consumer/broker/%d disconnecting due to error processing FetchRequest: %s\n", bc.broker.ID(), err)
  791. bc.abort(err)
  792. return
  793. }
  794. // if there isn't response, it means that not fetch was made
  795. // so we don't need to handle any response
  796. if response == nil {
  797. time.Sleep(partitionConsumersBatchTimeout)
  798. continue
  799. }
  800. bc.acks.Add(len(bc.subscriptions))
  801. for child := range bc.subscriptions {
  802. if _, ok := response.Blocks[child.topic]; !ok {
  803. bc.acks.Done()
  804. continue
  805. }
  806. if _, ok := response.Blocks[child.topic][child.partition]; !ok {
  807. bc.acks.Done()
  808. continue
  809. }
  810. child.feeder <- response
  811. }
  812. bc.acks.Wait()
  813. bc.handleResponses()
  814. }
  815. }
  816. func (bc *brokerConsumer) updateSubscriptions(newSubscriptions []*partitionConsumer) {
  817. for _, child := range newSubscriptions {
  818. bc.subscriptions[child] = none{}
  819. Logger.Printf("consumer/broker/%d added subscription to %s/%d\n", bc.broker.ID(), child.topic, child.partition)
  820. }
  821. for child := range bc.subscriptions {
  822. select {
  823. case <-child.dying:
  824. Logger.Printf("consumer/broker/%d closed dead subscription to %s/%d\n", bc.broker.ID(), child.topic, child.partition)
  825. close(child.trigger)
  826. delete(bc.subscriptions, child)
  827. default:
  828. // no-op
  829. }
  830. }
  831. }
  832. // handleResponses handles the response codes left for us by our subscriptions, and abandons ones that have been closed
  833. func (bc *brokerConsumer) handleResponses() {
  834. for child := range bc.subscriptions {
  835. result := child.responseResult
  836. child.responseResult = nil
  837. if result == nil {
  838. if preferredBroker, _, err := child.preferredBroker(); err == nil {
  839. if bc.broker.ID() != preferredBroker.ID() {
  840. // not an error but needs redispatching to consume from preferred replica
  841. Logger.Printf(
  842. "consumer/broker/%d abandoned in favor of preferred replica broker/%d\n",
  843. bc.broker.ID(), preferredBroker.ID())
  844. child.trigger <- none{}
  845. delete(bc.subscriptions, child)
  846. }
  847. }
  848. continue
  849. }
  850. // Discard any replica preference.
  851. child.preferredReadReplica = invalidPreferredReplicaID
  852. if errors.Is(result, errTimedOut) {
  853. Logger.Printf("consumer/broker/%d abandoned subscription to %s/%d because consuming was taking too long\n",
  854. bc.broker.ID(), child.topic, child.partition)
  855. delete(bc.subscriptions, child)
  856. } else if errors.Is(result, ErrOffsetOutOfRange) {
  857. // there's no point in retrying this it will just fail the same way again
  858. // shut it down and force the user to choose what to do
  859. child.sendError(result)
  860. Logger.Printf("consumer/%s/%d shutting down because %s\n", child.topic, child.partition, result)
  861. close(child.trigger)
  862. delete(bc.subscriptions, child)
  863. } else if errors.Is(result, ErrUnknownTopicOrPartition) ||
  864. errors.Is(result, ErrNotLeaderForPartition) ||
  865. errors.Is(result, ErrLeaderNotAvailable) ||
  866. errors.Is(result, ErrReplicaNotAvailable) ||
  867. errors.Is(result, ErrFencedLeaderEpoch) ||
  868. errors.Is(result, ErrUnknownLeaderEpoch) {
  869. // not an error, but does need redispatching
  870. Logger.Printf("consumer/broker/%d abandoned subscription to %s/%d because %s\n",
  871. bc.broker.ID(), child.topic, child.partition, result)
  872. child.trigger <- none{}
  873. delete(bc.subscriptions, child)
  874. } else {
  875. // dunno, tell the user and try redispatching
  876. child.sendError(result)
  877. Logger.Printf("consumer/broker/%d abandoned subscription to %s/%d because %s\n",
  878. bc.broker.ID(), child.topic, child.partition, result)
  879. child.trigger <- none{}
  880. delete(bc.subscriptions, child)
  881. }
  882. }
  883. }
  884. func (bc *brokerConsumer) abort(err error) {
  885. bc.consumer.abandonBrokerConsumer(bc)
  886. _ = bc.broker.Close() // we don't care about the error this might return, we already have one
  887. for child := range bc.subscriptions {
  888. child.sendError(err)
  889. child.trigger <- none{}
  890. }
  891. for newSubscriptions := range bc.newSubscriptions {
  892. if len(newSubscriptions) == 0 {
  893. // Take a small nap to avoid burning the CPU.
  894. time.Sleep(partitionConsumersBatchTimeout)
  895. continue
  896. }
  897. for _, child := range newSubscriptions {
  898. child.sendError(err)
  899. child.trigger <- none{}
  900. }
  901. }
  902. }
  903. // fetchResponse can be nil if no fetch is made, it can occur when
  904. // all partitions are paused
  905. func (bc *brokerConsumer) fetchNewMessages() (*FetchResponse, error) {
  906. request := &FetchRequest{
  907. MinBytes: bc.consumer.conf.Consumer.Fetch.Min,
  908. MaxWaitTime: int32(bc.consumer.conf.Consumer.MaxWaitTime / time.Millisecond),
  909. }
  910. // Version 1 is the same as version 0.
  911. if bc.consumer.conf.Version.IsAtLeast(V0_9_0_0) {
  912. request.Version = 1
  913. }
  914. // Starting in Version 2, the requestor must be able to handle Kafka Log
  915. // Message format version 1.
  916. if bc.consumer.conf.Version.IsAtLeast(V0_10_0_0) {
  917. request.Version = 2
  918. }
  919. // Version 3 adds MaxBytes. Starting in version 3, the partition ordering in
  920. // the request is now relevant. Partitions will be processed in the order
  921. // they appear in the request.
  922. if bc.consumer.conf.Version.IsAtLeast(V0_10_1_0) {
  923. request.Version = 3
  924. request.MaxBytes = MaxResponseSize
  925. }
  926. // Version 4 adds IsolationLevel. Starting in version 4, the reqestor must be
  927. // able to handle Kafka log message format version 2.
  928. // Version 5 adds LogStartOffset to indicate the earliest available offset of
  929. // partition data that can be consumed.
  930. if bc.consumer.conf.Version.IsAtLeast(V0_11_0_0) {
  931. request.Version = 5
  932. request.Isolation = bc.consumer.conf.Consumer.IsolationLevel
  933. }
  934. // Version 6 is the same as version 5.
  935. if bc.consumer.conf.Version.IsAtLeast(V1_0_0_0) {
  936. request.Version = 6
  937. }
  938. // Version 7 adds incremental fetch request support.
  939. if bc.consumer.conf.Version.IsAtLeast(V1_1_0_0) {
  940. request.Version = 7
  941. // We do not currently implement KIP-227 FetchSessions. Setting the id to 0
  942. // and the epoch to -1 tells the broker not to generate as session ID we're going
  943. // to just ignore anyway.
  944. request.SessionID = 0
  945. request.SessionEpoch = -1
  946. }
  947. // Version 8 is the same as version 7.
  948. if bc.consumer.conf.Version.IsAtLeast(V2_0_0_0) {
  949. request.Version = 8
  950. }
  951. // Version 9 adds CurrentLeaderEpoch, as described in KIP-320.
  952. // Version 10 indicates that we can use the ZStd compression algorithm, as
  953. // described in KIP-110.
  954. if bc.consumer.conf.Version.IsAtLeast(V2_1_0_0) {
  955. request.Version = 10
  956. }
  957. // Version 11 adds RackID for KIP-392 fetch from closest replica
  958. if bc.consumer.conf.Version.IsAtLeast(V2_3_0_0) {
  959. request.Version = 11
  960. request.RackID = bc.consumer.conf.RackID
  961. }
  962. for child := range bc.subscriptions {
  963. if !child.IsPaused() {
  964. request.AddBlock(child.topic, child.partition, child.offset, child.fetchSize, child.leaderEpoch)
  965. }
  966. }
  967. // avoid to fetch when there is no block
  968. if len(request.blocks) == 0 {
  969. return nil, nil
  970. }
  971. return bc.broker.Fetch(request)
  972. }