consumer.go 16 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579
  1. package sarama
  2. import (
  3. "fmt"
  4. "sync"
  5. "time"
  6. )
  7. // ConsumerMessage encapsulates a Kafka message returned by the consumer.
  8. type ConsumerMessage struct {
  9. Key, Value []byte
  10. Topic string
  11. Partition int32
  12. Offset int64
  13. }
  14. // ConsumerError is what is provided to the user when an error occurs.
  15. // It wraps an error and includes the topic and partition.
  16. type ConsumerError struct {
  17. Topic string
  18. Partition int32
  19. Err error
  20. }
  21. func (ce ConsumerError) Error() string {
  22. return fmt.Sprintf("kafka: error while consuming %s/%d: %s", ce.Topic, ce.Partition, ce.Err)
  23. }
  24. // ConsumerErrors is a type that wraps a batch of errors and implements the Error interface.
  25. // It can be returned from the PartitionConsumer's Close methods to avoid the need to manually drain errors
  26. // when stopping.
  27. type ConsumerErrors []*ConsumerError
  28. func (ce ConsumerErrors) Error() string {
  29. return fmt.Sprintf("kafka: %d errors while consuming", len(ce))
  30. }
  31. // Consumer manages PartitionConsumers which process Kafka messages from brokers. You MUST call Close()
  32. // on a consumer to avoid leaks, it will not be garbage-collected automatically when it passes out of
  33. // scope.
  34. type Consumer interface {
  35. // ConsumePartition creates a PartitionConsumer on the given topic/partition with the given offset. It will
  36. // return an error if this Consumer is already consuming on the given topic/partition. Offset can be a
  37. // literal offset, or OffsetNewest or OffsetOldest
  38. ConsumePartition(topic string, partition int32, offset int64) (PartitionConsumer, error)
  39. // Close shuts down the consumer. It must be called after all child PartitionConsumers have already been closed.
  40. Close() error
  41. }
  42. type consumer struct {
  43. client Client
  44. conf *Config
  45. ownClient bool
  46. lock sync.Mutex
  47. children map[string]map[int32]*partitionConsumer
  48. brokerConsumers map[*Broker]*brokerConsumer
  49. }
  50. // NewConsumer creates a new consumer using the given broker addresses and configuration.
  51. func NewConsumer(addrs []string, config *Config) (Consumer, error) {
  52. client, err := NewClient(addrs, config)
  53. if err != nil {
  54. return nil, err
  55. }
  56. c, err := NewConsumerFromClient(client)
  57. if err != nil {
  58. return nil, err
  59. }
  60. c.(*consumer).ownClient = true
  61. return c, nil
  62. }
  63. // NewConsumerFromClient creates a new consumer using the given client. It is still
  64. // necessary to call Close() on the underlying client when shutting down this consumer.
  65. func NewConsumerFromClient(client Client) (Consumer, error) {
  66. // Check that we are not dealing with a closed Client before processing any other arguments
  67. if client.Closed() {
  68. return nil, ErrClosedClient
  69. }
  70. c := &consumer{
  71. client: client,
  72. conf: client.Config(),
  73. children: make(map[string]map[int32]*partitionConsumer),
  74. brokerConsumers: make(map[*Broker]*brokerConsumer),
  75. }
  76. return c, nil
  77. }
  78. func (c *consumer) Close() error {
  79. if c.ownClient {
  80. return c.client.Close()
  81. }
  82. return nil
  83. }
  84. func (c *consumer) ConsumePartition(topic string, partition int32, offset int64) (PartitionConsumer, error) {
  85. child := &partitionConsumer{
  86. consumer: c,
  87. conf: c.conf,
  88. topic: topic,
  89. partition: partition,
  90. messages: make(chan *ConsumerMessage, c.conf.ChannelBufferSize),
  91. errors: make(chan *ConsumerError, c.conf.ChannelBufferSize),
  92. trigger: make(chan none, 1),
  93. dying: make(chan none),
  94. fetchSize: c.conf.Consumer.Fetch.Default,
  95. }
  96. if err := child.chooseStartingOffset(offset); err != nil {
  97. return nil, err
  98. }
  99. var leader *Broker
  100. var err error
  101. if leader, err = c.client.Leader(child.topic, child.partition); err != nil {
  102. return nil, err
  103. }
  104. if err := c.addChild(child); err != nil {
  105. return nil, err
  106. }
  107. go withRecover(child.dispatcher)
  108. child.broker = c.refBrokerConsumer(leader)
  109. child.broker.input <- child
  110. return child, nil
  111. }
  112. func (c *consumer) addChild(child *partitionConsumer) error {
  113. c.lock.Lock()
  114. defer c.lock.Unlock()
  115. topicChildren := c.children[child.topic]
  116. if topicChildren == nil {
  117. topicChildren = make(map[int32]*partitionConsumer)
  118. c.children[child.topic] = topicChildren
  119. }
  120. if topicChildren[child.partition] != nil {
  121. return ConfigurationError("That topic/partition is already being consumed")
  122. }
  123. topicChildren[child.partition] = child
  124. return nil
  125. }
  126. func (c *consumer) removeChild(child *partitionConsumer) {
  127. c.lock.Lock()
  128. defer c.lock.Unlock()
  129. delete(c.children[child.topic], child.partition)
  130. }
  131. func (c *consumer) refBrokerConsumer(broker *Broker) *brokerConsumer {
  132. c.lock.Lock()
  133. defer c.lock.Unlock()
  134. brokerWorker := c.brokerConsumers[broker]
  135. if brokerWorker == nil {
  136. brokerWorker = &brokerConsumer{
  137. consumer: c,
  138. broker: broker,
  139. input: make(chan *partitionConsumer),
  140. newSubscriptions: make(chan []*partitionConsumer),
  141. wait: make(chan none),
  142. subscriptions: make(map[*partitionConsumer]none),
  143. refs: 0,
  144. }
  145. go withRecover(brokerWorker.subscriptionManager)
  146. go withRecover(brokerWorker.subscriptionConsumer)
  147. c.brokerConsumers[broker] = brokerWorker
  148. }
  149. brokerWorker.refs++
  150. return brokerWorker
  151. }
  152. func (c *consumer) unrefBrokerConsumer(brokerWorker *brokerConsumer) {
  153. c.lock.Lock()
  154. defer c.lock.Unlock()
  155. brokerWorker.refs--
  156. if brokerWorker.refs == 0 {
  157. close(brokerWorker.input)
  158. if c.brokerConsumers[brokerWorker.broker] == brokerWorker {
  159. delete(c.brokerConsumers, brokerWorker.broker)
  160. }
  161. }
  162. }
  163. func (c *consumer) abandonBrokerConsumer(brokerWorker *brokerConsumer) {
  164. c.lock.Lock()
  165. defer c.lock.Unlock()
  166. delete(c.brokerConsumers, brokerWorker.broker)
  167. }
  168. // PartitionConsumer
  169. // PartitionConsumer processes Kafka messages from a given topic and partition. You MUST call Close()
  170. // or AsyncClose() on a PartitionConsumer to avoid leaks, it will not be garbage-collected automatically
  171. // when it passes out of scope.
  172. //
  173. // The simplest way of using a PartitionConsumer is to loop over its Messages channel using a for/range
  174. // loop. The PartitionConsumer will under no circumstances stop by itself once it is started, it will
  175. // just keep retrying if it encounters errors. By default, it logs these errors to sarama.Logger;
  176. // if you want to handle errors yourself, set your config's Consumer.Return.Errors to true, and read
  177. // from the Errors channel as well, using a select statement or in a separate goroutine. Check out
  178. // the examples of Consumer to see examples of these different approaches.
  179. type PartitionConsumer interface {
  180. // AsyncClose initiates a shutdown of the PartitionConsumer. This method will return immediately,
  181. // after which you should wait until the 'messages' and 'errors' channel are drained.
  182. // It is required to call this function, or Close before a consumer object passes out of scope,
  183. // as it will otherwise leak memory. You must call this before calling Close on the underlying
  184. // client.
  185. AsyncClose()
  186. // Close stops the PartitionConsumer from fetching messages. It is required to call this function
  187. // (or AsyncClose) before a consumer object passes out of scope, as it will otherwise leak memory. You must
  188. // call this before calling Close on the underlying client.
  189. Close() error
  190. // Messages returns the read channel for the messages that are returned by the broker.
  191. Messages() <-chan *ConsumerMessage
  192. // Errors returns a read channel of errors that occured during consuming, if enabled. By default,
  193. // errors are logged and not returned over this channel. If you want to implement any custom errpr
  194. // handling, set your config's Consumer.Return.Errors setting to true, and read from this channel.
  195. Errors() <-chan *ConsumerError
  196. }
  197. type partitionConsumer struct {
  198. consumer *consumer
  199. conf *Config
  200. topic string
  201. partition int32
  202. broker *brokerConsumer
  203. messages chan *ConsumerMessage
  204. errors chan *ConsumerError
  205. trigger, dying chan none
  206. fetchSize int32
  207. offset int64
  208. }
  209. func (child *partitionConsumer) sendError(err error) {
  210. cErr := &ConsumerError{
  211. Topic: child.topic,
  212. Partition: child.partition,
  213. Err: err,
  214. }
  215. if child.conf.Consumer.Return.Errors {
  216. child.errors <- cErr
  217. } else {
  218. Logger.Println(cErr)
  219. }
  220. }
  221. func (child *partitionConsumer) dispatcher() {
  222. for _ = range child.trigger {
  223. select {
  224. case <-child.dying:
  225. close(child.trigger)
  226. case <-time.After(child.conf.Consumer.Retry.Backoff):
  227. if child.broker != nil {
  228. child.consumer.unrefBrokerConsumer(child.broker)
  229. child.broker = nil
  230. }
  231. Logger.Printf("consumer/%s/%d finding new broker\n", child.topic, child.partition)
  232. if err := child.dispatch(); err != nil {
  233. child.sendError(err)
  234. child.trigger <- none{}
  235. }
  236. }
  237. }
  238. if child.broker != nil {
  239. child.consumer.unrefBrokerConsumer(child.broker)
  240. }
  241. child.consumer.removeChild(child)
  242. close(child.messages)
  243. close(child.errors)
  244. }
  245. func (child *partitionConsumer) dispatch() error {
  246. if err := child.consumer.client.RefreshMetadata(child.topic); err != nil {
  247. return err
  248. }
  249. var leader *Broker
  250. var err error
  251. if leader, err = child.consumer.client.Leader(child.topic, child.partition); err != nil {
  252. return err
  253. }
  254. child.broker = child.consumer.refBrokerConsumer(leader)
  255. child.broker.input <- child
  256. return nil
  257. }
  258. func (child *partitionConsumer) chooseStartingOffset(offset int64) error {
  259. newestOffset, err := child.consumer.client.GetOffset(child.topic, child.partition, OffsetNewest)
  260. if err != nil {
  261. return err
  262. }
  263. oldestOffset, err := child.consumer.client.GetOffset(child.topic, child.partition, OffsetOldest)
  264. if err != nil {
  265. return err
  266. }
  267. switch {
  268. case offset == OffsetNewest:
  269. child.offset = newestOffset
  270. case offset == OffsetOldest:
  271. child.offset = oldestOffset
  272. case offset >= oldestOffset && offset <= newestOffset:
  273. child.offset = offset
  274. default:
  275. return ErrOffsetOutOfRange
  276. }
  277. return nil
  278. }
  279. func (child *partitionConsumer) Messages() <-chan *ConsumerMessage {
  280. return child.messages
  281. }
  282. func (child *partitionConsumer) Errors() <-chan *ConsumerError {
  283. return child.errors
  284. }
  285. func (child *partitionConsumer) AsyncClose() {
  286. // this triggers whatever worker owns this child to abandon it and close its trigger channel, which causes
  287. // the dispatcher to exit its loop, which removes it from the consumer then closes its 'messages' and
  288. // 'errors' channel (alternatively, if the child is already at the dispatcher for some reason, that will
  289. // also just close itself)
  290. close(child.dying)
  291. }
  292. func (child *partitionConsumer) Close() error {
  293. child.AsyncClose()
  294. go withRecover(func() {
  295. for _ = range child.messages {
  296. // drain
  297. }
  298. })
  299. var errors ConsumerErrors
  300. for err := range child.errors {
  301. errors = append(errors, err)
  302. }
  303. if len(errors) > 0 {
  304. return errors
  305. }
  306. return nil
  307. }
  308. func (child *partitionConsumer) handleResponse(response *FetchResponse) error {
  309. block := response.GetBlock(child.topic, child.partition)
  310. if block == nil {
  311. return ErrIncompleteResponse
  312. }
  313. if block.Err != ErrNoError {
  314. return block.Err
  315. }
  316. if len(block.MsgSet.Messages) == 0 {
  317. // We got no messages. If we got a trailing one then we need to ask for more data.
  318. // Otherwise we just poll again and wait for one to be produced...
  319. if block.MsgSet.PartialTrailingMessage {
  320. if child.conf.Consumer.Fetch.Max > 0 && child.fetchSize == child.conf.Consumer.Fetch.Max {
  321. // we can't ask for more data, we've hit the configured limit
  322. child.sendError(ErrMessageTooLarge)
  323. child.offset++ // skip this one so we can keep processing future messages
  324. } else {
  325. child.fetchSize *= 2
  326. if child.conf.Consumer.Fetch.Max > 0 && child.fetchSize > child.conf.Consumer.Fetch.Max {
  327. child.fetchSize = child.conf.Consumer.Fetch.Max
  328. }
  329. }
  330. }
  331. return nil
  332. }
  333. // we got messages, reset our fetch size in case it was increased for a previous request
  334. child.fetchSize = child.conf.Consumer.Fetch.Default
  335. incomplete := false
  336. atLeastOne := false
  337. prelude := true
  338. for _, msgBlock := range block.MsgSet.Messages {
  339. for _, msg := range msgBlock.Messages() {
  340. if prelude && msg.Offset < child.offset {
  341. continue
  342. }
  343. prelude = false
  344. if msg.Offset >= child.offset {
  345. atLeastOne = true
  346. child.messages <- &ConsumerMessage{
  347. Topic: child.topic,
  348. Partition: child.partition,
  349. Key: msg.Msg.Key,
  350. Value: msg.Msg.Value,
  351. Offset: msg.Offset,
  352. }
  353. child.offset = msg.Offset + 1
  354. } else {
  355. incomplete = true
  356. }
  357. }
  358. }
  359. if incomplete || !atLeastOne {
  360. return ErrIncompleteResponse
  361. }
  362. return nil
  363. }
  364. // brokerConsumer
  365. type brokerConsumer struct {
  366. consumer *consumer
  367. broker *Broker
  368. input chan *partitionConsumer
  369. newSubscriptions chan []*partitionConsumer
  370. wait chan none
  371. subscriptions map[*partitionConsumer]none
  372. refs int
  373. }
  374. func (w *brokerConsumer) subscriptionManager() {
  375. var buffer []*partitionConsumer
  376. // The subscriptionManager constantly accepts new subscriptions on `input` (even when the main subscriptionConsumer
  377. // goroutine is in the middle of a network request) and batches it up. The main worker goroutine picks
  378. // up a batch of new subscriptions between every network request by reading from `newSubscriptions`, so we give
  379. // it nil if no new subscriptions are available. We also write to `wait` only when new subscriptions is available,
  380. // so the main goroutine can block waiting for work if it has none.
  381. for {
  382. if len(buffer) > 0 {
  383. select {
  384. case event, ok := <-w.input:
  385. if !ok {
  386. goto done
  387. }
  388. buffer = append(buffer, event)
  389. case w.newSubscriptions <- buffer:
  390. buffer = nil
  391. case w.wait <- none{}:
  392. }
  393. } else {
  394. select {
  395. case event, ok := <-w.input:
  396. if !ok {
  397. goto done
  398. }
  399. buffer = append(buffer, event)
  400. case w.newSubscriptions <- nil:
  401. }
  402. }
  403. }
  404. done:
  405. close(w.wait)
  406. if len(buffer) > 0 {
  407. w.newSubscriptions <- buffer
  408. }
  409. close(w.newSubscriptions)
  410. }
  411. func (w *brokerConsumer) subscriptionConsumer() {
  412. <-w.wait // wait for our first piece of work
  413. // the subscriptionConsumer ensures we will get nil right away if no new subscriptions is available
  414. for newSubscriptions := range w.newSubscriptions {
  415. w.updateSubscriptionCache(newSubscriptions)
  416. if len(w.subscriptions) == 0 {
  417. // We're about to be shut down or we're about to receive more subscriptions.
  418. // Either way, the signal just hasn't propagated to our goroutine yet.
  419. <-w.wait
  420. continue
  421. }
  422. response, err := w.fetchNewMessages()
  423. if err != nil {
  424. Logger.Printf("consumer/broker/%d disconnecting due to error processing FetchRequest: %s\n", w.broker.ID(), err)
  425. w.abort(err)
  426. return
  427. }
  428. for child := range w.subscriptions {
  429. if err := child.handleResponse(response); err != nil {
  430. switch err {
  431. default:
  432. child.sendError(err)
  433. fallthrough
  434. case ErrUnknownTopicOrPartition, ErrNotLeaderForPartition, ErrLeaderNotAvailable:
  435. // these three are not fatal errors, but do require redispatching
  436. child.trigger <- none{}
  437. delete(w.subscriptions, child)
  438. Logger.Printf("consumer/broker/%d abandoned subscription to %s/%d because %s\n", w.broker.ID(), child.topic, child.partition, err)
  439. }
  440. }
  441. }
  442. }
  443. }
  444. func (w *brokerConsumer) updateSubscriptionCache(newSubscriptions []*partitionConsumer) {
  445. // take new subscriptions, and abandon subscriptions that have been closed
  446. for _, child := range newSubscriptions {
  447. w.subscriptions[child] = none{}
  448. Logger.Printf("consumer/broker/%d added subscription to %s/%d\n", w.broker.ID(), child.topic, child.partition)
  449. }
  450. for child := range w.subscriptions {
  451. select {
  452. case <-child.dying:
  453. close(child.trigger)
  454. delete(w.subscriptions, child)
  455. Logger.Printf("consumer/broker/%d closed dead subscription to %s/%d\n", w.broker.ID(), child.topic, child.partition)
  456. default:
  457. }
  458. }
  459. }
  460. func (w *brokerConsumer) abort(err error) {
  461. w.consumer.abandonBrokerConsumer(w)
  462. _ = w.broker.Close() // we don't care about the error this might return, we already have one
  463. for child := range w.subscriptions {
  464. child.sendError(err)
  465. child.trigger <- none{}
  466. }
  467. for newSubscription := range w.newSubscriptions {
  468. for _, child := range newSubscription {
  469. child.sendError(err)
  470. child.trigger <- none{}
  471. }
  472. }
  473. }
  474. func (w *brokerConsumer) fetchNewMessages() (*FetchResponse, error) {
  475. request := &FetchRequest{
  476. MinBytes: w.consumer.conf.Consumer.Fetch.Min,
  477. MaxWaitTime: int32(w.consumer.conf.Consumer.MaxWaitTime / time.Millisecond),
  478. }
  479. for child := range w.subscriptions {
  480. request.AddBlock(child.topic, child.partition, child.offset, child.fetchSize)
  481. }
  482. return w.broker.Fetch(request)
  483. }