async_producer.go 25 KB

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