async_producer.go 24 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676677678679680681682683684685686687688689690691692693694695696697698699700701702703704705706707708709710711712713714715716717718719720721722723724725726727728729730731732733734735736737738739740741742743744745746747748749750751752753754755756757758759760761762763764765766767768769770771772773774775776777778779780781782783784785786787788789790791792793794795796797798799800801802803804805806807808809
  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.topicDispatcher)
  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. ///////////////////////////////////////////
  165. // In normal processing, a message flows through the following functions from top to bottom,
  166. // starting at topicDispatcher (which reads from Producer.input) and ending in flusher
  167. // (which sends the message to the broker). In cases where a message must be retried, it goes
  168. // through retryHandler before being returned to the top of the flow.
  169. ///////////////////////////////////////////
  170. // singleton
  171. // dispatches messages by topic
  172. func (p *asyncProducer) topicDispatcher() {
  173. handlers := make(map[string]chan *ProducerMessage)
  174. shuttingDown := false
  175. for msg := range p.input {
  176. if msg == nil {
  177. Logger.Println("Something tried to send a nil message, it was ignored.")
  178. continue
  179. }
  180. if msg.flags&shutdown != 0 {
  181. shuttingDown = true
  182. continue
  183. } else if msg.retries == 0 {
  184. p.inFlight.Add(1)
  185. if shuttingDown {
  186. p.returnError(msg, ErrShuttingDown)
  187. continue
  188. }
  189. }
  190. if (p.conf.Producer.Compression == CompressionNone && msg.Value != nil && msg.Value.Length() > p.conf.Producer.MaxMessageBytes) ||
  191. (msg.byteSize() > p.conf.Producer.MaxMessageBytes) {
  192. p.returnError(msg, ErrMessageSizeTooLarge)
  193. continue
  194. }
  195. handler := handlers[msg.Topic]
  196. if handler == nil {
  197. newHandler := make(chan *ProducerMessage, p.conf.ChannelBufferSize)
  198. topic := msg.Topic // block local because go's closure semantics suck
  199. go withRecover(func() { p.partitionDispatcher(topic, newHandler) })
  200. handler = newHandler
  201. handlers[msg.Topic] = handler
  202. }
  203. handler <- msg
  204. }
  205. for _, handler := range handlers {
  206. close(handler)
  207. }
  208. }
  209. // one per topic
  210. // partitions messages, then dispatches them by partition
  211. func (p *asyncProducer) partitionDispatcher(topic string, input chan *ProducerMessage) {
  212. handlers := make(map[int32]chan *ProducerMessage)
  213. partitioner := p.conf.Producer.Partitioner(topic)
  214. breaker := breaker.New(3, 1, 10*time.Second)
  215. for msg := range input {
  216. if msg.retries == 0 {
  217. err := breaker.Run(func() error {
  218. return p.assignPartition(partitioner, msg)
  219. })
  220. if err != nil {
  221. p.returnError(msg, err)
  222. continue
  223. }
  224. }
  225. handler := handlers[msg.Partition]
  226. if handler == nil {
  227. newHandler := make(chan *ProducerMessage, p.conf.ChannelBufferSize)
  228. topic := msg.Topic // block local because go's closure semantics suck
  229. partition := msg.Partition // block local because go's closure semantics suck
  230. go withRecover(func() { p.leaderDispatcher(topic, partition, newHandler) })
  231. handler = newHandler
  232. handlers[msg.Partition] = handler
  233. }
  234. handler <- msg
  235. }
  236. for _, handler := range handlers {
  237. close(handler)
  238. }
  239. }
  240. // one per partition per topic
  241. // dispatches messages to the appropriate broker
  242. // also responsible for maintaining message order during retries
  243. func (p *asyncProducer) leaderDispatcher(topic string, partition int32, input chan *ProducerMessage) {
  244. var leader *Broker
  245. var output chan *ProducerMessage
  246. breaker := breaker.New(3, 1, 10*time.Second)
  247. doUpdate := func() (err error) {
  248. if err = p.client.RefreshMetadata(topic); err != nil {
  249. return err
  250. }
  251. if leader, err = p.client.Leader(topic, partition); err != nil {
  252. return err
  253. }
  254. output = p.getBrokerProducer(leader)
  255. return nil
  256. }
  257. // try to prefetch the leader; if this doesn't work, we'll do a proper breaker-protected refresh-and-fetch
  258. // on the first message
  259. leader, _ = p.client.Leader(topic, partition)
  260. if leader != nil {
  261. output = p.getBrokerProducer(leader)
  262. }
  263. // highWatermark tracks the "current" retry level, which is the only one where we actually let messages through,
  264. // all other messages get buffered in retryState[msg.retries].buf to preserve ordering
  265. // retryState[msg.retries].expectChaser simply tracks whether we've seen a chaser message for a given level (and
  266. // therefore whether our buffer is complete and safe to flush)
  267. highWatermark := 0
  268. retryState := make([]struct {
  269. buf []*ProducerMessage
  270. expectChaser bool
  271. }, p.conf.Producer.Retry.Max+1)
  272. for msg := range input {
  273. if msg.retries > highWatermark {
  274. // new, higher, retry level; send off a chaser so that we know when everything "in between" has made it
  275. // back to us and we can safely flush the backlog (otherwise we risk re-ordering messages)
  276. highWatermark = msg.retries
  277. Logger.Printf("producer/leader/%s/%d state change to [retrying-%d]\n", topic, partition, highWatermark)
  278. retryState[msg.retries].expectChaser = true
  279. p.inFlight.Add(1) // we're generating a chaser message; track it so we don't shut down while it's still inflight
  280. output <- &ProducerMessage{Topic: topic, Partition: partition, flags: chaser, retries: msg.retries - 1}
  281. Logger.Printf("producer/leader/%s/%d abandoning broker %d\n", topic, partition, leader.ID())
  282. p.unrefBrokerProducer(leader, output)
  283. output = nil
  284. time.Sleep(p.conf.Producer.Retry.Backoff)
  285. } else if highWatermark > 0 {
  286. // we are retrying something (else highWatermark would be 0) but this message is not a *new* retry level
  287. if msg.retries < highWatermark {
  288. // in fact this message is not even the current retry level, so buffer it for now (unless it's a just a chaser)
  289. if msg.flags&chaser == chaser {
  290. retryState[msg.retries].expectChaser = false
  291. p.inFlight.Done() // this chaser is now handled and will be garbage collected
  292. } else {
  293. retryState[msg.retries].buf = append(retryState[msg.retries].buf, msg)
  294. }
  295. continue
  296. } else if msg.flags&chaser == chaser {
  297. // this message is of the current retry level (msg.retries == highWatermark) and the chaser flag is set,
  298. // meaning this retry level is done and we can go down (at least) one level and flush that
  299. retryState[highWatermark].expectChaser = false
  300. Logger.Printf("producer/leader/%s/%d state change to [flushing-%d]\n", topic, partition, highWatermark)
  301. for {
  302. highWatermark--
  303. if output == nil {
  304. if err := breaker.Run(doUpdate); err != nil {
  305. p.returnErrors(retryState[highWatermark].buf, err)
  306. goto flushDone
  307. }
  308. Logger.Printf("producer/leader/%s/%d selected broker %d\n", topic, partition, leader.ID())
  309. }
  310. for _, msg := range retryState[highWatermark].buf {
  311. output <- msg
  312. }
  313. flushDone:
  314. retryState[highWatermark].buf = nil
  315. if retryState[highWatermark].expectChaser {
  316. Logger.Printf("producer/leader/%s/%d state change to [retrying-%d]\n", topic, partition, highWatermark)
  317. break
  318. } else {
  319. if highWatermark == 0 {
  320. Logger.Printf("producer/leader/%s/%d state change to [normal]\n", topic, partition)
  321. break
  322. }
  323. }
  324. }
  325. p.inFlight.Done() // this chaser is now handled and will be garbage collected
  326. continue
  327. }
  328. }
  329. // if we made it this far then the current msg contains real data, and can be sent to the next goroutine
  330. // without breaking any of our ordering guarantees
  331. if output == nil {
  332. if err := breaker.Run(doUpdate); err != nil {
  333. p.returnError(msg, err)
  334. time.Sleep(p.conf.Producer.Retry.Backoff)
  335. continue
  336. }
  337. Logger.Printf("producer/leader/%s/%d selected broker %d\n", topic, partition, leader.ID())
  338. }
  339. output <- msg
  340. }
  341. if output != nil {
  342. p.unrefBrokerProducer(leader, output)
  343. }
  344. }
  345. // one per broker
  346. // groups messages together into appropriately-sized batches for sending to the broker
  347. // based on https://godoc.org/github.com/eapache/channels#BatchingChannel
  348. func (p *asyncProducer) messageAggregator(broker *Broker, input chan *ProducerMessage) {
  349. var (
  350. timer <-chan time.Time
  351. buffer []*ProducerMessage
  352. flushTriggered chan []*ProducerMessage
  353. bytesAccumulated int
  354. defaultFlush bool
  355. )
  356. if p.conf.Producer.Flush.Frequency == 0 && p.conf.Producer.Flush.Bytes == 0 && p.conf.Producer.Flush.Messages == 0 {
  357. defaultFlush = true
  358. }
  359. output := make(chan []*ProducerMessage)
  360. go withRecover(func() { p.flusher(broker, output) })
  361. for {
  362. select {
  363. case msg := <-input:
  364. if msg == nil {
  365. goto shutdown
  366. }
  367. if (bytesAccumulated+msg.byteSize() >= forceFlushThreshold()) ||
  368. (p.conf.Producer.Compression != CompressionNone && bytesAccumulated+msg.byteSize() >= p.conf.Producer.MaxMessageBytes) ||
  369. (p.conf.Producer.Flush.MaxMessages > 0 && len(buffer) >= p.conf.Producer.Flush.MaxMessages) {
  370. Logger.Printf("producer/aggregator/%d maximum request accumulated, forcing blocking flush\n", broker.ID())
  371. output <- buffer
  372. timer = nil
  373. buffer = nil
  374. flushTriggered = nil
  375. bytesAccumulated = 0
  376. }
  377. buffer = append(buffer, msg)
  378. bytesAccumulated += msg.byteSize()
  379. if defaultFlush ||
  380. msg.flags&chaser == chaser ||
  381. (p.conf.Producer.Flush.Messages > 0 && len(buffer) >= p.conf.Producer.Flush.Messages) ||
  382. (p.conf.Producer.Flush.Bytes > 0 && bytesAccumulated >= p.conf.Producer.Flush.Bytes) {
  383. flushTriggered = output
  384. } else if p.conf.Producer.Flush.Frequency > 0 && timer == nil {
  385. timer = time.After(p.conf.Producer.Flush.Frequency)
  386. }
  387. case <-timer:
  388. flushTriggered = output
  389. case flushTriggered <- buffer:
  390. timer = nil
  391. buffer = nil
  392. flushTriggered = nil
  393. bytesAccumulated = 0
  394. }
  395. }
  396. shutdown:
  397. if len(buffer) > 0 {
  398. output <- buffer
  399. }
  400. close(output)
  401. }
  402. // one per broker
  403. // takes a batch at a time from the messageAggregator and sends to the broker
  404. func (p *asyncProducer) flusher(broker *Broker, input chan []*ProducerMessage) {
  405. var closing error
  406. currentRetries := make(map[string]map[int32]error)
  407. Logger.Printf("producer/flusher/%d starting up\n", broker.ID())
  408. for batch := range input {
  409. if closing != nil {
  410. p.retryMessages(batch, closing)
  411. continue
  412. }
  413. // group messages by topic/partition
  414. msgSets := make(map[string]map[int32][]*ProducerMessage)
  415. for i, msg := range batch {
  416. if currentRetries[msg.Topic] != nil && currentRetries[msg.Topic][msg.Partition] != nil {
  417. if msg.flags&chaser == chaser {
  418. // we can start processing this topic/partition again
  419. Logger.Printf("producer/flusher/%d state change to [normal] on %s/%d\n",
  420. broker.ID(), msg.Topic, msg.Partition)
  421. currentRetries[msg.Topic][msg.Partition] = nil
  422. }
  423. p.retryMessages([]*ProducerMessage{msg}, currentRetries[msg.Topic][msg.Partition])
  424. batch[i] = nil // to prevent it being returned/retried twice
  425. continue
  426. }
  427. partitionSet := msgSets[msg.Topic]
  428. if partitionSet == nil {
  429. partitionSet = make(map[int32][]*ProducerMessage)
  430. msgSets[msg.Topic] = partitionSet
  431. }
  432. partitionSet[msg.Partition] = append(partitionSet[msg.Partition], msg)
  433. }
  434. request := p.buildRequest(msgSets)
  435. if request == nil {
  436. continue
  437. }
  438. response, err := broker.Produce(request)
  439. switch err.(type) {
  440. case nil:
  441. break
  442. case PacketEncodingError:
  443. p.returnErrors(batch, err)
  444. continue
  445. default:
  446. Logger.Printf("producer/flusher/%d state change to [closing] because %s\n", broker.ID(), err)
  447. p.abandonBrokerConnection(broker)
  448. p.retryMessages(batch, err)
  449. _ = broker.Close()
  450. closing = err
  451. continue
  452. }
  453. if response == nil {
  454. // this only happens when RequiredAcks is NoResponse, so we have to assume success
  455. p.returnSuccesses(batch)
  456. continue
  457. }
  458. // we iterate through the blocks in the request, not the response, so that we notice
  459. // if the response is missing a block completely
  460. for topic, partitionSet := range msgSets {
  461. for partition, msgs := range partitionSet {
  462. block := response.GetBlock(topic, partition)
  463. if block == nil {
  464. p.returnErrors(msgs, ErrIncompleteResponse)
  465. continue
  466. }
  467. switch block.Err {
  468. case ErrNoError:
  469. // All the messages for this topic-partition were delivered successfully!
  470. for i := range msgs {
  471. msgs[i].Offset = block.Offset + int64(i)
  472. }
  473. p.returnSuccesses(msgs)
  474. case ErrUnknownTopicOrPartition, ErrNotLeaderForPartition, ErrLeaderNotAvailable,
  475. ErrRequestTimedOut, ErrNotEnoughReplicas, ErrNotEnoughReplicasAfterAppend:
  476. Logger.Printf("producer/flusher/%d state change to [retrying] on %s/%d because %v\n",
  477. broker.ID(), topic, partition, block.Err)
  478. if currentRetries[topic] == nil {
  479. currentRetries[topic] = make(map[int32]error)
  480. }
  481. currentRetries[topic][partition] = block.Err
  482. p.retryMessages(msgs, block.Err)
  483. default:
  484. p.returnErrors(msgs, block.Err)
  485. }
  486. }
  487. }
  488. }
  489. Logger.Printf("producer/flusher/%d shut down\n", broker.ID())
  490. }
  491. // singleton
  492. // effectively a "bridge" between the flushers and the topicDispatcher in order to avoid deadlock
  493. // based on https://godoc.org/github.com/eapache/channels#InfiniteChannel
  494. func (p *asyncProducer) retryHandler() {
  495. var msg *ProducerMessage
  496. buf := queue.New()
  497. for {
  498. if buf.Length() == 0 {
  499. msg = <-p.retries
  500. } else {
  501. select {
  502. case msg = <-p.retries:
  503. case p.input <- buf.Peek().(*ProducerMessage):
  504. buf.Remove()
  505. continue
  506. }
  507. }
  508. if msg == nil {
  509. return
  510. }
  511. buf.Add(msg)
  512. }
  513. }
  514. ///////////////////////////////////////////
  515. ///////////////////////////////////////////
  516. // utility functions
  517. func (p *asyncProducer) shutdown() {
  518. Logger.Println("Producer shutting down.")
  519. p.input <- &ProducerMessage{flags: shutdown}
  520. p.inFlight.Wait()
  521. if p.ownClient {
  522. err := p.client.Close()
  523. if err != nil {
  524. Logger.Println("producer/shutdown failed to close the embedded client:", err)
  525. }
  526. }
  527. close(p.input)
  528. close(p.retries)
  529. close(p.errors)
  530. close(p.successes)
  531. }
  532. func (p *asyncProducer) assignPartition(partitioner Partitioner, msg *ProducerMessage) error {
  533. var partitions []int32
  534. var err error
  535. if partitioner.RequiresConsistency() {
  536. partitions, err = p.client.Partitions(msg.Topic)
  537. } else {
  538. partitions, err = p.client.WritablePartitions(msg.Topic)
  539. }
  540. if err != nil {
  541. return err
  542. }
  543. numPartitions := int32(len(partitions))
  544. if numPartitions == 0 {
  545. return ErrLeaderNotAvailable
  546. }
  547. choice, err := partitioner.Partition(msg, numPartitions)
  548. if err != nil {
  549. return err
  550. } else if choice < 0 || choice >= numPartitions {
  551. return ErrInvalidPartition
  552. }
  553. msg.Partition = partitions[choice]
  554. return nil
  555. }
  556. func (p *asyncProducer) buildRequest(batch map[string]map[int32][]*ProducerMessage) *ProduceRequest {
  557. req := &ProduceRequest{RequiredAcks: p.conf.Producer.RequiredAcks, Timeout: int32(p.conf.Producer.Timeout / time.Millisecond)}
  558. empty := true
  559. for topic, partitionSet := range batch {
  560. for partition, msgSet := range partitionSet {
  561. setToSend := new(MessageSet)
  562. setSize := 0
  563. for _, msg := range msgSet {
  564. var keyBytes, valBytes []byte
  565. var err error
  566. if msg.Key != nil {
  567. if keyBytes, err = msg.Key.Encode(); err != nil {
  568. p.returnError(msg, err)
  569. continue
  570. }
  571. }
  572. if msg.Value != nil {
  573. if valBytes, err = msg.Value.Encode(); err != nil {
  574. p.returnError(msg, err)
  575. continue
  576. }
  577. }
  578. if p.conf.Producer.Compression != CompressionNone && setSize+msg.byteSize() > p.conf.Producer.MaxMessageBytes {
  579. // compression causes message-sets to be wrapped as single messages, which have tighter
  580. // size requirements, so we have to respect those limits
  581. valBytes, err := encode(setToSend)
  582. if err != nil {
  583. Logger.Println(err) // if this happens, it's basically our fault.
  584. panic(err)
  585. }
  586. req.AddMessage(topic, partition, &Message{Codec: p.conf.Producer.Compression, Key: nil, Value: valBytes})
  587. setToSend = new(MessageSet)
  588. setSize = 0
  589. }
  590. setSize += msg.byteSize()
  591. setToSend.addMessage(&Message{Codec: CompressionNone, Key: keyBytes, Value: valBytes})
  592. empty = false
  593. }
  594. if p.conf.Producer.Compression == CompressionNone {
  595. req.AddSet(topic, partition, setToSend)
  596. } else {
  597. valBytes, err := encode(setToSend)
  598. if err != nil {
  599. Logger.Println(err) // if this happens, it's basically our fault.
  600. panic(err)
  601. }
  602. req.AddMessage(topic, partition, &Message{Codec: p.conf.Producer.Compression, Key: nil, Value: valBytes})
  603. }
  604. }
  605. }
  606. if empty {
  607. return nil
  608. }
  609. return req
  610. }
  611. func (p *asyncProducer) returnError(msg *ProducerMessage, err error) {
  612. msg.clear()
  613. pErr := &ProducerError{Msg: msg, Err: err}
  614. if p.conf.Producer.Return.Errors {
  615. p.errors <- pErr
  616. } else {
  617. Logger.Println(pErr)
  618. }
  619. p.inFlight.Done()
  620. }
  621. func (p *asyncProducer) returnErrors(batch []*ProducerMessage, err error) {
  622. for _, msg := range batch {
  623. if msg != nil {
  624. p.returnError(msg, err)
  625. }
  626. }
  627. }
  628. func (p *asyncProducer) returnSuccesses(batch []*ProducerMessage) {
  629. for _, msg := range batch {
  630. if msg == nil {
  631. continue
  632. }
  633. if p.conf.Producer.Return.Successes {
  634. msg.clear()
  635. p.successes <- msg
  636. }
  637. p.inFlight.Done()
  638. }
  639. }
  640. func (p *asyncProducer) retryMessages(batch []*ProducerMessage, err error) {
  641. for _, msg := range batch {
  642. if msg == nil {
  643. continue
  644. }
  645. if msg.retries >= p.conf.Producer.Retry.Max {
  646. p.returnError(msg, err)
  647. } else {
  648. msg.retries++
  649. p.retries <- msg
  650. }
  651. }
  652. }
  653. func (p *asyncProducer) getBrokerProducer(broker *Broker) chan *ProducerMessage {
  654. p.brokerLock.Lock()
  655. defer p.brokerLock.Unlock()
  656. bp := p.brokers[broker]
  657. if bp == nil {
  658. bp = make(chan *ProducerMessage)
  659. p.brokers[broker] = bp
  660. p.brokerRefs[bp] = 0
  661. go withRecover(func() { p.messageAggregator(broker, bp) })
  662. }
  663. p.brokerRefs[bp]++
  664. return bp
  665. }
  666. func (p *asyncProducer) unrefBrokerProducer(broker *Broker, bp chan *ProducerMessage) {
  667. p.brokerLock.Lock()
  668. defer p.brokerLock.Unlock()
  669. p.brokerRefs[bp]--
  670. if p.brokerRefs[bp] == 0 {
  671. close(bp)
  672. delete(p.brokerRefs, bp)
  673. if p.brokers[broker] == bp {
  674. delete(p.brokers, broker)
  675. }
  676. }
  677. }
  678. func (p *asyncProducer) abandonBrokerConnection(broker *Broker) {
  679. p.brokerLock.Lock()
  680. defer p.brokerLock.Unlock()
  681. delete(p.brokers, broker)
  682. }