async_producer.go 25 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676677678679680681682683684685686687688689690691692693694695696697698699700701702703704705706707708709710711712713714715716717718719720721722723724725726727728729730731732733734735736737738739740741742743744745746747748749750751752753754755756757758759760761762763764765766767768769770771772773774775776777778779780781782783784785786787788789790791792793794795796797798799800801802803804805806807808809810811812813814815816817818819820821822823824825826827828829830831832833834835836837838839840841842843844845846847848849850851852853854855856857858859860861862863864865866867868869870871872873874875876877878879880881882883884885886887888889890891892893
  1. package sarama
  2. import (
  3. "fmt"
  4. "sync"
  5. "time"
  6. "github.com/eapache/go-resiliency/breaker"
  7. "github.com/eapache/queue"
  8. )
  9. // AsyncProducer publishes Kafka messages using a non-blocking API. It routes messages
  10. // to the correct broker for the provided topic-partition, refreshing metadata as appropriate,
  11. // and parses responses for errors. You must read from the Errors() channel or the
  12. // producer will deadlock. You must call Close() or AsyncClose() on a producer to avoid
  13. // leaks: it will not be garbage-collected automatically when it passes out of
  14. // scope.
  15. type AsyncProducer interface {
  16. // AsyncClose triggers a shutdown of the producer, flushing any messages it may
  17. // have buffered. The shutdown has completed when both the Errors and Successes
  18. // channels have been closed. When calling AsyncClose, you *must* continue to
  19. // read from those channels in order to drain the results of any messages in
  20. // flight.
  21. AsyncClose()
  22. // Close shuts down the producer and flushes any messages it may have buffered.
  23. // You must call this function before a producer object passes out of scope, as
  24. // it may otherwise leak memory. You must call this before calling Close on the
  25. // underlying client.
  26. Close() error
  27. // Input is the input channel for the user to write messages to that they
  28. // wish to send.
  29. Input() chan<- *ProducerMessage
  30. // Successes is the success output channel back to the user when AckSuccesses is
  31. // enabled. If Return.Successes is true, you MUST read from this channel or the
  32. // Producer will deadlock. It is suggested that you send and read messages
  33. // together in a single select statement.
  34. Successes() <-chan *ProducerMessage
  35. // Errors is the error output channel back to the user. You MUST read from this
  36. // channel or the Producer will deadlock when the channel is full. Alternatively,
  37. // you can set Producer.Return.Errors in your config to false, which prevents
  38. // errors to be returned.
  39. Errors() <-chan *ProducerError
  40. }
  41. type asyncProducer struct {
  42. client Client
  43. conf *Config
  44. ownClient bool
  45. errors chan *ProducerError
  46. input, successes, retries chan *ProducerMessage
  47. inFlight sync.WaitGroup
  48. brokers map[*Broker]chan<- *ProducerMessage
  49. brokerRefs map[chan<- *ProducerMessage]int
  50. brokerLock sync.Mutex
  51. }
  52. // NewAsyncProducer creates a new AsyncProducer using the given broker addresses and configuration.
  53. func NewAsyncProducer(addrs []string, conf *Config) (AsyncProducer, error) {
  54. client, err := NewClient(addrs, conf)
  55. if err != nil {
  56. return nil, err
  57. }
  58. p, err := NewAsyncProducerFromClient(client)
  59. if err != nil {
  60. return nil, err
  61. }
  62. p.(*asyncProducer).ownClient = true
  63. return p, nil
  64. }
  65. // NewAsyncProducerFromClient creates a new Producer using the given client. It is still
  66. // necessary to call Close() on the underlying client when shutting down this producer.
  67. func NewAsyncProducerFromClient(client Client) (AsyncProducer, error) {
  68. // Check that we are not dealing with a closed Client before processing any other arguments
  69. if client.Closed() {
  70. return nil, ErrClosedClient
  71. }
  72. p := &asyncProducer{
  73. client: client,
  74. conf: client.Config(),
  75. errors: make(chan *ProducerError),
  76. input: make(chan *ProducerMessage),
  77. successes: make(chan *ProducerMessage),
  78. retries: make(chan *ProducerMessage),
  79. brokers: make(map[*Broker]chan<- *ProducerMessage),
  80. brokerRefs: make(map[chan<- *ProducerMessage]int),
  81. }
  82. // launch our singleton dispatchers
  83. go withRecover(p.dispatcher)
  84. go withRecover(p.retryHandler)
  85. return p, nil
  86. }
  87. type flagSet int8
  88. const (
  89. syn flagSet = 1 << iota // first message from partitionProducer to brokerProducer
  90. fin // final message from partitionProducer to brokerProducer and back
  91. shutdown // start the shutdown process
  92. )
  93. // ProducerMessage is the collection of elements passed to the Producer in order to send a message.
  94. type ProducerMessage struct {
  95. Topic string // The Kafka topic for this message.
  96. // The partitioning key for this message. Pre-existing Encoders include
  97. // StringEncoder and ByteEncoder.
  98. Key Encoder
  99. // The actual message to store in Kafka. Pre-existing Encoders include
  100. // StringEncoder and ByteEncoder.
  101. Value Encoder
  102. // This field is used to hold arbitrary data you wish to include so it
  103. // will be available when receiving on the Successes and Errors channels.
  104. // Sarama completely ignores this field and is only to be used for
  105. // pass-through data.
  106. Metadata interface{}
  107. // Below this point are filled in by the producer as the message is processed
  108. // Offset is the offset of the message stored on the broker. This is only
  109. // guaranteed to be defined if the message was successfully delivered and
  110. // RequiredAcks is not NoResponse.
  111. Offset int64
  112. // Partition is the partition that the message was sent to. This is only
  113. // guaranteed to be defined if the message was successfully delivered.
  114. Partition int32
  115. retries int
  116. flags flagSet
  117. }
  118. const producerMessageOverhead = 26 // the metadata overhead of CRC, flags, etc.
  119. func (m *ProducerMessage) byteSize() int {
  120. size := producerMessageOverhead
  121. if m.Key != nil {
  122. size += m.Key.Length()
  123. }
  124. if m.Value != nil {
  125. size += m.Value.Length()
  126. }
  127. return size
  128. }
  129. func (m *ProducerMessage) clear() {
  130. m.flags = 0
  131. m.retries = 0
  132. }
  133. // ProducerError is the type of error generated when the producer fails to deliver a message.
  134. // It contains the original ProducerMessage as well as the actual error value.
  135. type ProducerError struct {
  136. Msg *ProducerMessage
  137. Err error
  138. }
  139. func (pe ProducerError) Error() string {
  140. return fmt.Sprintf("kafka: Failed to produce message to topic %s: %s", pe.Msg.Topic, pe.Err)
  141. }
  142. // ProducerErrors is a type that wraps a batch of "ProducerError"s and implements the Error interface.
  143. // It can be returned from the Producer's Close method to avoid the need to manually drain the Errors channel
  144. // when closing a producer.
  145. type ProducerErrors []*ProducerError
  146. func (pe ProducerErrors) Error() string {
  147. return fmt.Sprintf("kafka: Failed to deliver %d messages.", len(pe))
  148. }
  149. func (p *asyncProducer) Errors() <-chan *ProducerError {
  150. return p.errors
  151. }
  152. func (p *asyncProducer) Successes() <-chan *ProducerMessage {
  153. return p.successes
  154. }
  155. func (p *asyncProducer) Input() chan<- *ProducerMessage {
  156. return p.input
  157. }
  158. func (p *asyncProducer) Close() error {
  159. p.AsyncClose()
  160. if p.conf.Producer.Return.Successes {
  161. go withRecover(func() {
  162. for _ = range p.successes {
  163. }
  164. })
  165. }
  166. var errors ProducerErrors
  167. if p.conf.Producer.Return.Errors {
  168. for event := range p.errors {
  169. errors = append(errors, event)
  170. }
  171. }
  172. if len(errors) > 0 {
  173. return errors
  174. }
  175. return nil
  176. }
  177. func (p *asyncProducer) AsyncClose() {
  178. go withRecover(p.shutdown)
  179. }
  180. // singleton
  181. // dispatches messages by topic
  182. func (p *asyncProducer) dispatcher() {
  183. handlers := make(map[string]chan<- *ProducerMessage)
  184. shuttingDown := false
  185. for msg := range p.input {
  186. if msg == nil {
  187. Logger.Println("Something tried to send a nil message, it was ignored.")
  188. continue
  189. }
  190. if msg.flags&shutdown != 0 {
  191. shuttingDown = true
  192. p.inFlight.Done()
  193. continue
  194. } else if msg.retries == 0 {
  195. if shuttingDown {
  196. // we can't just call returnError here because that decrements the wait group,
  197. // which hasn't been incremented yet for this message, and shouldn't be
  198. pErr := &ProducerError{Msg: msg, Err: ErrShuttingDown}
  199. if p.conf.Producer.Return.Errors {
  200. p.errors <- pErr
  201. } else {
  202. Logger.Println(pErr)
  203. }
  204. continue
  205. }
  206. p.inFlight.Add(1)
  207. }
  208. if msg.byteSize() > p.conf.Producer.MaxMessageBytes {
  209. p.returnError(msg, ErrMessageSizeTooLarge)
  210. continue
  211. }
  212. handler := handlers[msg.Topic]
  213. if handler == nil {
  214. handler = p.newTopicProducer(msg.Topic)
  215. handlers[msg.Topic] = handler
  216. }
  217. handler <- msg
  218. }
  219. for _, handler := range handlers {
  220. close(handler)
  221. }
  222. }
  223. // one per topic
  224. // partitions messages, then dispatches them by partition
  225. type topicProducer struct {
  226. parent *asyncProducer
  227. topic string
  228. input <-chan *ProducerMessage
  229. breaker *breaker.Breaker
  230. handlers map[int32]chan<- *ProducerMessage
  231. partitioner Partitioner
  232. }
  233. func (p *asyncProducer) newTopicProducer(topic string) chan<- *ProducerMessage {
  234. input := make(chan *ProducerMessage, p.conf.ChannelBufferSize)
  235. tp := &topicProducer{
  236. parent: p,
  237. topic: topic,
  238. input: input,
  239. breaker: breaker.New(3, 1, 10*time.Second),
  240. handlers: make(map[int32]chan<- *ProducerMessage),
  241. partitioner: p.conf.Producer.Partitioner(topic),
  242. }
  243. go withRecover(tp.dispatch)
  244. return input
  245. }
  246. func (tp *topicProducer) dispatch() {
  247. for msg := range tp.input {
  248. if msg.retries == 0 {
  249. if err := tp.partitionMessage(msg); err != nil {
  250. tp.parent.returnError(msg, err)
  251. continue
  252. }
  253. }
  254. handler := tp.handlers[msg.Partition]
  255. if handler == nil {
  256. handler = tp.parent.newPartitionProducer(msg.Topic, msg.Partition)
  257. tp.handlers[msg.Partition] = handler
  258. }
  259. handler <- msg
  260. }
  261. for _, handler := range tp.handlers {
  262. close(handler)
  263. }
  264. }
  265. func (tp *topicProducer) partitionMessage(msg *ProducerMessage) error {
  266. var partitions []int32
  267. err := tp.breaker.Run(func() (err error) {
  268. if tp.partitioner.RequiresConsistency() {
  269. partitions, err = tp.parent.client.Partitions(msg.Topic)
  270. } else {
  271. partitions, err = tp.parent.client.WritablePartitions(msg.Topic)
  272. }
  273. return
  274. })
  275. if err != nil {
  276. return err
  277. }
  278. numPartitions := int32(len(partitions))
  279. if numPartitions == 0 {
  280. return ErrLeaderNotAvailable
  281. }
  282. choice, err := tp.partitioner.Partition(msg, numPartitions)
  283. if err != nil {
  284. return err
  285. } else if choice < 0 || choice >= numPartitions {
  286. return ErrInvalidPartition
  287. }
  288. msg.Partition = partitions[choice]
  289. return nil
  290. }
  291. // one per partition per topic
  292. // dispatches messages to the appropriate broker
  293. // also responsible for maintaining message order during retries
  294. type partitionProducer struct {
  295. parent *asyncProducer
  296. topic string
  297. partition int32
  298. input <-chan *ProducerMessage
  299. leader *Broker
  300. breaker *breaker.Breaker
  301. output chan<- *ProducerMessage
  302. // highWatermark tracks the "current" retry level, which is the only one where we actually let messages through,
  303. // all other messages get buffered in retryState[msg.retries].buf to preserve ordering
  304. // retryState[msg.retries].expectChaser simply tracks whether we've seen a fin message for a given level (and
  305. // therefore whether our buffer is complete and safe to flush)
  306. highWatermark int
  307. retryState []partitionRetryState
  308. }
  309. type partitionRetryState struct {
  310. buf []*ProducerMessage
  311. expectChaser bool
  312. }
  313. func (p *asyncProducer) newPartitionProducer(topic string, partition int32) chan<- *ProducerMessage {
  314. input := make(chan *ProducerMessage, p.conf.ChannelBufferSize)
  315. pp := &partitionProducer{
  316. parent: p,
  317. topic: topic,
  318. partition: partition,
  319. input: input,
  320. breaker: breaker.New(3, 1, 10*time.Second),
  321. retryState: make([]partitionRetryState, p.conf.Producer.Retry.Max+1),
  322. }
  323. go withRecover(pp.dispatch)
  324. return input
  325. }
  326. func (pp *partitionProducer) dispatch() {
  327. // try to prefetch the leader; if this doesn't work, we'll do a proper call to `updateLeader`
  328. // on the first message
  329. pp.leader, _ = pp.parent.client.Leader(pp.topic, pp.partition)
  330. if pp.leader != nil {
  331. pp.output = pp.parent.getBrokerProducer(pp.leader)
  332. pp.parent.inFlight.Add(1) // we're generating a syn message; track it so we don't shut down while it's still inflight
  333. pp.output <- &ProducerMessage{Topic: pp.topic, Partition: pp.partition, flags: syn}
  334. }
  335. for msg := range pp.input {
  336. if msg.retries > pp.highWatermark {
  337. // a new, higher, retry level; handle it and then back off
  338. pp.newHighWatermark(msg.retries)
  339. time.Sleep(pp.parent.conf.Producer.Retry.Backoff)
  340. } else if pp.highWatermark > 0 {
  341. // we are retrying something (else highWatermark would be 0) but this message is not a *new* retry level
  342. if msg.retries < pp.highWatermark {
  343. // in fact this message is not even the current retry level, so buffer it for now (unless it's a just a fin)
  344. if msg.flags&fin == fin {
  345. pp.retryState[msg.retries].expectChaser = false
  346. pp.parent.inFlight.Done() // this fin is now handled and will be garbage collected
  347. } else {
  348. pp.retryState[msg.retries].buf = append(pp.retryState[msg.retries].buf, msg)
  349. }
  350. continue
  351. } else if msg.flags&fin == fin {
  352. // this message is of the current retry level (msg.retries == highWatermark) and the fin flag is set,
  353. // meaning this retry level is done and we can go down (at least) one level and flush that
  354. pp.retryState[pp.highWatermark].expectChaser = false
  355. pp.flushRetryBuffers()
  356. pp.parent.inFlight.Done() // this fin is now handled and will be garbage collected
  357. continue
  358. }
  359. }
  360. // if we made it this far then the current msg contains real data, and can be sent to the next goroutine
  361. // without breaking any of our ordering guarantees
  362. if pp.output == nil {
  363. if err := pp.updateLeader(); err != nil {
  364. pp.parent.returnError(msg, err)
  365. time.Sleep(pp.parent.conf.Producer.Retry.Backoff)
  366. continue
  367. }
  368. Logger.Printf("producer/leader/%s/%d selected broker %d\n", pp.topic, pp.partition, pp.leader.ID())
  369. }
  370. pp.output <- msg
  371. }
  372. if pp.output != nil {
  373. pp.parent.unrefBrokerProducer(pp.leader, pp.output)
  374. }
  375. }
  376. func (pp *partitionProducer) newHighWatermark(hwm int) {
  377. Logger.Printf("producer/leader/%s/%d state change to [retrying-%d]\n", pp.topic, pp.partition, hwm)
  378. pp.highWatermark = hwm
  379. // send off a fin so that we know when everything "in between" has made it
  380. // back to us and we can safely flush the backlog (otherwise we risk re-ordering messages)
  381. pp.retryState[pp.highWatermark].expectChaser = true
  382. pp.parent.inFlight.Add(1) // we're generating a fin message; track it so we don't shut down while it's still inflight
  383. pp.output <- &ProducerMessage{Topic: pp.topic, Partition: pp.partition, flags: fin, retries: pp.highWatermark - 1}
  384. // a new HWM means that our current broker selection is out of date
  385. Logger.Printf("producer/leader/%s/%d abandoning broker %d\n", pp.topic, pp.partition, pp.leader.ID())
  386. pp.parent.unrefBrokerProducer(pp.leader, pp.output)
  387. pp.output = nil
  388. }
  389. func (pp *partitionProducer) flushRetryBuffers() {
  390. Logger.Printf("producer/leader/%s/%d state change to [flushing-%d]\n", pp.topic, pp.partition, pp.highWatermark)
  391. for {
  392. pp.highWatermark--
  393. if pp.output == nil {
  394. if err := pp.updateLeader(); err != nil {
  395. pp.parent.returnErrors(pp.retryState[pp.highWatermark].buf, err)
  396. goto flushDone
  397. }
  398. Logger.Printf("producer/leader/%s/%d selected broker %d\n", pp.topic, pp.partition, pp.leader.ID())
  399. }
  400. for _, msg := range pp.retryState[pp.highWatermark].buf {
  401. pp.output <- msg
  402. }
  403. flushDone:
  404. pp.retryState[pp.highWatermark].buf = nil
  405. if pp.retryState[pp.highWatermark].expectChaser {
  406. Logger.Printf("producer/leader/%s/%d state change to [retrying-%d]\n", pp.topic, pp.partition, pp.highWatermark)
  407. break
  408. } else if pp.highWatermark == 0 {
  409. Logger.Printf("producer/leader/%s/%d state change to [normal]\n", pp.topic, pp.partition)
  410. break
  411. }
  412. }
  413. }
  414. func (pp *partitionProducer) updateLeader() error {
  415. return pp.breaker.Run(func() (err error) {
  416. if err = pp.parent.client.RefreshMetadata(pp.topic); err != nil {
  417. return err
  418. }
  419. if pp.leader, err = pp.parent.client.Leader(pp.topic, pp.partition); err != nil {
  420. return err
  421. }
  422. pp.output = pp.parent.getBrokerProducer(pp.leader)
  423. pp.parent.inFlight.Add(1) // we're generating a syn message; track it so we don't shut down while it's still inflight
  424. pp.output <- &ProducerMessage{Topic: pp.topic, Partition: pp.partition, flags: syn}
  425. return nil
  426. })
  427. }
  428. // one per broker; also constructs an associated flusher
  429. func (p *asyncProducer) newBrokerProducer(broker *Broker) chan<- *ProducerMessage {
  430. var (
  431. input = make(chan *ProducerMessage)
  432. bridge = make(chan *produceSet)
  433. responses = make(chan *brokerProducerResponse)
  434. )
  435. bp := &brokerProducer{
  436. parent: p,
  437. broker: broker,
  438. input: input,
  439. output: bridge,
  440. responses: responses,
  441. buffer: newProduceSet(p),
  442. currentRetries: make(map[string]map[int32]error),
  443. }
  444. go withRecover(bp.run)
  445. // minimal bridge to make the network response `select`able
  446. go withRecover(func() {
  447. for set := range bridge {
  448. request := set.buildRequest()
  449. response, err := broker.Produce(request)
  450. responses <- &brokerProducerResponse{
  451. set: set,
  452. err: err,
  453. res: response,
  454. }
  455. }
  456. close(responses)
  457. })
  458. return input
  459. }
  460. type brokerProducerResponse struct {
  461. set *produceSet
  462. err error
  463. res *ProduceResponse
  464. }
  465. // groups messages together into appropriately-sized batches for sending to the broker
  466. // handles state related to retries etc
  467. type brokerProducer struct {
  468. parent *asyncProducer
  469. broker *Broker
  470. input <-chan *ProducerMessage
  471. output chan<- *produceSet
  472. responses <-chan *brokerProducerResponse
  473. buffer *produceSet
  474. timer <-chan time.Time
  475. timerFired bool
  476. closing error
  477. currentRetries map[string]map[int32]error
  478. }
  479. func (bp *brokerProducer) run() {
  480. var output chan<- *produceSet
  481. Logger.Printf("producer/broker/%d starting up\n", bp.broker.ID())
  482. for {
  483. select {
  484. case msg := <-bp.input:
  485. if msg == nil {
  486. bp.shutdown()
  487. return
  488. }
  489. if msg.flags&syn == syn {
  490. Logger.Printf("producer/broker/%d state change to [open] on %s/%d\n",
  491. bp.broker.ID(), msg.Topic, msg.Partition)
  492. if bp.currentRetries[msg.Topic] == nil {
  493. bp.currentRetries[msg.Topic] = make(map[int32]error)
  494. }
  495. bp.currentRetries[msg.Topic][msg.Partition] = nil
  496. bp.parent.inFlight.Done()
  497. continue
  498. }
  499. if reason := bp.needsRetry(msg); reason != nil {
  500. bp.parent.retryMessage(msg, reason)
  501. if bp.closing == nil && msg.flags&fin == fin {
  502. // we were retrying this partition but we can start processing again
  503. delete(bp.currentRetries[msg.Topic], msg.Partition)
  504. Logger.Printf("producer/broker/%d state change to [closed] on %s/%d\n",
  505. bp.broker.ID(), msg.Topic, msg.Partition)
  506. }
  507. continue
  508. }
  509. if bp.buffer.wouldOverflow(msg) {
  510. if err := bp.waitForSpace(msg); err != nil {
  511. bp.parent.retryMessage(msg, err)
  512. continue
  513. }
  514. }
  515. if err := bp.buffer.add(msg); err != nil {
  516. bp.parent.returnError(msg, err)
  517. continue
  518. }
  519. if bp.parent.conf.Producer.Flush.Frequency > 0 && bp.timer == nil {
  520. bp.timer = time.After(bp.parent.conf.Producer.Flush.Frequency)
  521. }
  522. case <-bp.timer:
  523. bp.timerFired = true
  524. case output <- bp.buffer:
  525. bp.rollOver()
  526. case response := <-bp.responses:
  527. bp.handleResponse(response)
  528. }
  529. if bp.timerFired || bp.buffer.readyToFlush() {
  530. output = bp.output
  531. } else {
  532. output = nil
  533. }
  534. }
  535. }
  536. func (bp *brokerProducer) shutdown() {
  537. for !bp.buffer.empty() {
  538. select {
  539. case response := <-bp.responses:
  540. bp.handleResponse(response)
  541. case bp.output <- bp.buffer:
  542. bp.rollOver()
  543. }
  544. }
  545. close(bp.output)
  546. for response := range bp.responses {
  547. bp.handleResponse(response)
  548. }
  549. Logger.Printf("producer/broker/%d shut down\n", bp.broker.ID())
  550. }
  551. func (bp *brokerProducer) needsRetry(msg *ProducerMessage) error {
  552. if bp.closing != nil {
  553. return bp.closing
  554. }
  555. return bp.currentRetries[msg.Topic][msg.Partition]
  556. }
  557. func (bp *brokerProducer) waitForSpace(msg *ProducerMessage) error {
  558. Logger.Printf("producer/broker/%d maximum request accumulated, waiting for space\n", bp.broker.ID())
  559. for {
  560. select {
  561. case response := <-bp.responses:
  562. bp.handleResponse(response)
  563. // handling a response can change our state, so re-check some things
  564. if reason := bp.needsRetry(msg); reason != nil {
  565. return reason
  566. } else if !bp.buffer.wouldOverflow(msg) {
  567. return nil
  568. }
  569. case bp.output <- bp.buffer:
  570. bp.rollOver()
  571. return nil
  572. }
  573. }
  574. }
  575. func (bp *brokerProducer) rollOver() {
  576. bp.timer = nil
  577. bp.timerFired = false
  578. bp.buffer = newProduceSet(bp.parent)
  579. }
  580. func (bp *brokerProducer) handleResponse(response *brokerProducerResponse) {
  581. if response.err != nil {
  582. bp.handleError(response.set, response.err)
  583. } else {
  584. bp.handleSuccess(response.set, response.res)
  585. }
  586. if bp.buffer.empty() {
  587. bp.rollOver() // this can happen if the response invalidated our buffer
  588. }
  589. }
  590. func (bp *brokerProducer) handleSuccess(sent *produceSet, response *ProduceResponse) {
  591. // we iterate through the blocks in the request set, not the response, so that we notice
  592. // if the response is missing a block completely
  593. sent.eachPartition(func(topic string, partition int32, msgs []*ProducerMessage) {
  594. if response == nil {
  595. // this only happens when RequiredAcks is NoResponse, so we have to assume success
  596. bp.parent.returnSuccesses(msgs)
  597. return
  598. }
  599. block := response.GetBlock(topic, partition)
  600. if block == nil {
  601. bp.parent.returnErrors(msgs, ErrIncompleteResponse)
  602. return
  603. }
  604. switch block.Err {
  605. // Success
  606. case ErrNoError:
  607. for i, msg := range msgs {
  608. msg.Offset = block.Offset + int64(i)
  609. }
  610. bp.parent.returnSuccesses(msgs)
  611. // Retriable errors
  612. case ErrInvalidMessage, ErrUnknownTopicOrPartition, ErrLeaderNotAvailable, ErrNotLeaderForPartition,
  613. ErrRequestTimedOut, ErrNotEnoughReplicas, ErrNotEnoughReplicasAfterAppend:
  614. Logger.Printf("producer/broker/%d state change to [retrying] on %s/%d because %v\n",
  615. bp.broker.ID(), topic, partition, block.Err)
  616. bp.currentRetries[topic][partition] = block.Err
  617. bp.parent.retryMessages(msgs, block.Err)
  618. bp.parent.retryMessages(bp.buffer.dropPartition(topic, partition), block.Err)
  619. // Other non-retriable errors
  620. default:
  621. bp.parent.returnErrors(msgs, block.Err)
  622. }
  623. })
  624. }
  625. func (bp *brokerProducer) handleError(sent *produceSet, err error) {
  626. switch err.(type) {
  627. case PacketEncodingError:
  628. sent.eachPartition(func(topic string, partition int32, msgs []*ProducerMessage) {
  629. bp.parent.returnErrors(msgs, err)
  630. })
  631. default:
  632. Logger.Printf("producer/broker/%d state change to [closing] because %s\n", bp.broker.ID(), err)
  633. bp.parent.abandonBrokerConnection(bp.broker)
  634. _ = bp.broker.Close()
  635. bp.closing = err
  636. sent.eachPartition(func(topic string, partition int32, msgs []*ProducerMessage) {
  637. bp.parent.retryMessages(msgs, err)
  638. })
  639. bp.buffer.eachPartition(func(topic string, partition int32, msgs []*ProducerMessage) {
  640. bp.parent.retryMessages(msgs, err)
  641. })
  642. bp.rollOver()
  643. }
  644. }
  645. // singleton
  646. // effectively a "bridge" between the flushers and the dispatcher in order to avoid deadlock
  647. // based on https://godoc.org/github.com/eapache/channels#InfiniteChannel
  648. func (p *asyncProducer) retryHandler() {
  649. var msg *ProducerMessage
  650. buf := queue.New()
  651. for {
  652. if buf.Length() == 0 {
  653. msg = <-p.retries
  654. } else {
  655. select {
  656. case msg = <-p.retries:
  657. case p.input <- buf.Peek().(*ProducerMessage):
  658. buf.Remove()
  659. continue
  660. }
  661. }
  662. if msg == nil {
  663. return
  664. }
  665. buf.Add(msg)
  666. }
  667. }
  668. // utility functions
  669. func (p *asyncProducer) shutdown() {
  670. Logger.Println("Producer shutting down.")
  671. p.inFlight.Add(1)
  672. p.input <- &ProducerMessage{flags: shutdown}
  673. p.inFlight.Wait()
  674. if p.ownClient {
  675. err := p.client.Close()
  676. if err != nil {
  677. Logger.Println("producer/shutdown failed to close the embedded client:", err)
  678. }
  679. }
  680. close(p.input)
  681. close(p.retries)
  682. close(p.errors)
  683. close(p.successes)
  684. }
  685. func (p *asyncProducer) returnError(msg *ProducerMessage, err error) {
  686. msg.clear()
  687. pErr := &ProducerError{Msg: msg, Err: err}
  688. if p.conf.Producer.Return.Errors {
  689. p.errors <- pErr
  690. } else {
  691. Logger.Println(pErr)
  692. }
  693. p.inFlight.Done()
  694. }
  695. func (p *asyncProducer) returnErrors(batch []*ProducerMessage, err error) {
  696. for _, msg := range batch {
  697. p.returnError(msg, err)
  698. }
  699. }
  700. func (p *asyncProducer) returnSuccesses(batch []*ProducerMessage) {
  701. for _, msg := range batch {
  702. if p.conf.Producer.Return.Successes {
  703. msg.clear()
  704. p.successes <- msg
  705. }
  706. p.inFlight.Done()
  707. }
  708. }
  709. func (p *asyncProducer) retryMessage(msg *ProducerMessage, err error) {
  710. if msg.retries >= p.conf.Producer.Retry.Max {
  711. p.returnError(msg, err)
  712. } else {
  713. msg.retries++
  714. p.retries <- msg
  715. }
  716. }
  717. func (p *asyncProducer) retryMessages(batch []*ProducerMessage, err error) {
  718. for _, msg := range batch {
  719. p.retryMessage(msg, err)
  720. }
  721. }
  722. func (p *asyncProducer) getBrokerProducer(broker *Broker) chan<- *ProducerMessage {
  723. p.brokerLock.Lock()
  724. defer p.brokerLock.Unlock()
  725. bp := p.brokers[broker]
  726. if bp == nil {
  727. bp = p.newBrokerProducer(broker)
  728. p.brokers[broker] = bp
  729. p.brokerRefs[bp] = 0
  730. }
  731. p.brokerRefs[bp]++
  732. return bp
  733. }
  734. func (p *asyncProducer) unrefBrokerProducer(broker *Broker, bp chan<- *ProducerMessage) {
  735. p.brokerLock.Lock()
  736. defer p.brokerLock.Unlock()
  737. p.brokerRefs[bp]--
  738. if p.brokerRefs[bp] == 0 {
  739. close(bp)
  740. delete(p.brokerRefs, bp)
  741. if p.brokers[broker] == bp {
  742. delete(p.brokers, broker)
  743. }
  744. }
  745. }
  746. func (p *asyncProducer) abandonBrokerConnection(broker *Broker) {
  747. p.brokerLock.Lock()
  748. defer p.brokerLock.Unlock()
  749. delete(p.brokers, broker)
  750. }