consumer.go 16 KB

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