async_producer.go 24 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676677678679680681682683684685686687688689690691692693694695696697698699700701702703704705706707708709710711712713714715716717718719720721722723724725726727728729730731732733734735736737738739740741742743744745746747748749750751752753754755756757758759760761762763764765766767768769770771772773774775776777778779780781782783784785786787788789790791792793794795796797798799800801802803804805
  1. package sarama
  2. import (
  3. "fmt"
  4. "sync"
  5. "time"
  6. "github.com/eapache/go-resiliency/breaker"
  7. )
  8. func forceFlushThreshold() int {
  9. return int(MaxRequestSize - (10 * 1024)) // 10KiB is safety room for misc. overhead, we might want to calculate this more precisely?
  10. }
  11. // AsyncProducer publishes Kafka messages using a non-blocking API. It routes messages
  12. // to the correct broker for the provided topic-partition, refreshing metadata as appropriate,
  13. // and parses responses for errors. You must read from the Errors() channel or the
  14. // producer will deadlock. You must call Close() or AsyncClose() on a producer to avoid
  15. // leaks: it will not be garbage-collected automatically when it passes out of
  16. // scope (this is in addition to calling Close on the underlying client, which
  17. // is still necessary).
  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 confured.
  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. brokers map[*Broker]*brokerProducer
  47. brokerLock sync.Mutex
  48. }
  49. // NewAsyncProducer creates a new AsyncProducer using the given broker addresses and configuration.
  50. func NewAsyncProducer(addrs []string, conf *Config) (AsyncProducer, error) {
  51. client, err := NewClient(addrs, conf)
  52. if err != nil {
  53. return nil, err
  54. }
  55. p, err := NewAsyncProducerFromClient(client)
  56. if err != nil {
  57. return nil, err
  58. }
  59. p.(*asyncProducer).ownClient = true
  60. return p, nil
  61. }
  62. // NewAsyncProducerFromClient creates a new Producer using the given client.
  63. func NewAsyncProducerFromClient(client Client) (AsyncProducer, error) {
  64. // Check that we are not dealing with a closed Client before processing any other arguments
  65. if client.Closed() {
  66. return nil, ErrClosedClient
  67. }
  68. p := &asyncProducer{
  69. client: client,
  70. conf: client.Config(),
  71. errors: make(chan *ProducerError),
  72. input: make(chan *ProducerMessage),
  73. successes: make(chan *ProducerMessage),
  74. retries: make(chan *ProducerMessage),
  75. brokers: make(map[*Broker]*brokerProducer),
  76. }
  77. // launch our singleton dispatchers
  78. go withRecover(p.topicDispatcher)
  79. go withRecover(p.retryHandler)
  80. return p, nil
  81. }
  82. type flagSet int8
  83. const (
  84. chaser flagSet = 1 << iota // message is last in a group that failed
  85. ref // add a reference to a singleton channel
  86. unref // remove a reference from a singleton channel
  87. shutdown // start the shutdown process
  88. )
  89. // ProducerMessage is the collection of elements passed to the Producer in order to send a message.
  90. type ProducerMessage struct {
  91. Topic string // The Kafka topic for this message.
  92. Key Encoder // The partitioning key for this message. It must implement the Encoder interface. Pre-existing Encoders include StringEncoder and ByteEncoder.
  93. Value Encoder // The actual message to store in Kafka. It must implement the Encoder interface. Pre-existing Encoders include StringEncoder and ByteEncoder.
  94. // These are filled in by the producer as the message is processed
  95. 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.
  96. 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.
  97. 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.
  98. retries int
  99. flags flagSet
  100. }
  101. func (m *ProducerMessage) byteSize() int {
  102. size := 26 // the metadata overhead of CRC, flags, etc.
  103. if m.Key != nil {
  104. size += m.Key.Length()
  105. }
  106. if m.Value != nil {
  107. size += m.Value.Length()
  108. }
  109. return size
  110. }
  111. // ProducerError is the type of error generated when the producer fails to deliver a message.
  112. // It contains the original ProducerMessage as well as the actual error value.
  113. type ProducerError struct {
  114. Msg *ProducerMessage
  115. Err error
  116. }
  117. func (pe ProducerError) Error() string {
  118. return fmt.Sprintf("kafka: Failed to produce message to topic %s: %s", pe.Msg.Topic, pe.Err)
  119. }
  120. // ProducerErrors is a type that wraps a batch of "ProducerError"s and implements the Error interface.
  121. // It can be returned from the Producer's Close method to avoid the need to manually drain the Errors channel
  122. // when closing a producer.
  123. type ProducerErrors []*ProducerError
  124. func (pe ProducerErrors) Error() string {
  125. return fmt.Sprintf("kafka: Failed to deliver %d messages.", len(pe))
  126. }
  127. func (p *asyncProducer) Errors() <-chan *ProducerError {
  128. return p.errors
  129. }
  130. func (p *asyncProducer) Successes() <-chan *ProducerMessage {
  131. return p.successes
  132. }
  133. func (p *asyncProducer) Input() chan<- *ProducerMessage {
  134. return p.input
  135. }
  136. func (p *asyncProducer) Close() error {
  137. p.AsyncClose()
  138. if p.conf.Producer.Return.Successes {
  139. go withRecover(func() {
  140. for _ = range p.successes {
  141. }
  142. })
  143. }
  144. var errors ProducerErrors
  145. if p.conf.Producer.Return.Errors {
  146. for event := range p.errors {
  147. errors = append(errors, event)
  148. }
  149. }
  150. if len(errors) > 0 {
  151. return errors
  152. }
  153. return nil
  154. }
  155. func (p *asyncProducer) AsyncClose() {
  156. go withRecover(func() {
  157. p.input <- &ProducerMessage{flags: shutdown}
  158. })
  159. }
  160. ///////////////////////////////////////////
  161. // In normal processing, a message flows through the following functions from top to bottom,
  162. // starting at topicDispatcher (which reads from Producer.input) and ending in flusher
  163. // (which sends the message to the broker). In cases where a message must be retried, it goes
  164. // through retryHandler before being returned to the top of the flow.
  165. ///////////////////////////////////////////
  166. // singleton
  167. // dispatches messages by topic
  168. func (p *asyncProducer) topicDispatcher() {
  169. handlers := make(map[string]chan *ProducerMessage)
  170. for msg := range p.input {
  171. if msg == nil {
  172. Logger.Println("Something tried to send a nil message, it was ignored.")
  173. continue
  174. }
  175. if msg.flags&shutdown != 0 {
  176. Logger.Println("Producer shutting down.")
  177. break
  178. }
  179. if (p.conf.Producer.Compression == CompressionNone && msg.Value != nil && msg.Value.Length() > p.conf.Producer.MaxMessageBytes) ||
  180. (msg.byteSize() > p.conf.Producer.MaxMessageBytes) {
  181. p.returnError(msg, ErrMessageSizeTooLarge)
  182. continue
  183. }
  184. handler := handlers[msg.Topic]
  185. if handler == nil {
  186. p.retries <- &ProducerMessage{flags: ref}
  187. newHandler := make(chan *ProducerMessage, p.conf.ChannelBufferSize)
  188. topic := msg.Topic // block local because go's closure semantics suck
  189. go withRecover(func() { p.partitionDispatcher(topic, newHandler) })
  190. handler = newHandler
  191. handlers[msg.Topic] = handler
  192. }
  193. handler <- msg
  194. }
  195. for _, handler := range handlers {
  196. close(handler)
  197. }
  198. p.retries <- &ProducerMessage{flags: shutdown}
  199. for msg := range p.input {
  200. p.returnError(msg, ErrShuttingDown)
  201. }
  202. if p.ownClient {
  203. err := p.client.Close()
  204. if err != nil {
  205. Logger.Println("producer/shutdown failed to close the embedded client:", err)
  206. }
  207. }
  208. close(p.errors)
  209. close(p.successes)
  210. }
  211. // one per topic
  212. // partitions messages, then dispatches them by partition
  213. func (p *asyncProducer) partitionDispatcher(topic string, input chan *ProducerMessage) {
  214. handlers := make(map[int32]chan *ProducerMessage)
  215. partitioner := p.conf.Producer.Partitioner()
  216. for msg := range input {
  217. if msg.retries == 0 {
  218. err := p.assignPartition(partitioner, msg)
  219. if err != nil {
  220. p.returnError(msg, err)
  221. continue
  222. }
  223. }
  224. handler := handlers[msg.Partition]
  225. if handler == nil {
  226. p.retries <- &ProducerMessage{flags: ref}
  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. p.retries <- &ProducerMessage{flags: unref}
  240. }
  241. // one per partition per topic
  242. // dispatches messages to the appropriate broker
  243. // also responsible for maintaining message order during retries
  244. func (p *asyncProducer) leaderDispatcher(topic string, partition int32, input chan *ProducerMessage) {
  245. var leader *Broker
  246. var output chan *ProducerMessage
  247. breaker := breaker.New(3, 1, 10*time.Second)
  248. doUpdate := func() (err error) {
  249. if err = p.client.RefreshMetadata(topic); err != nil {
  250. return err
  251. }
  252. if leader, err = p.client.Leader(topic, partition); err != nil {
  253. return err
  254. }
  255. output = p.getBrokerProducer(leader)
  256. return nil
  257. }
  258. // try to prefetch the leader; if this doesn't work, we'll do a proper breaker-protected refresh-and-fetch
  259. // on the first message
  260. leader, _ = p.client.Leader(topic, partition)
  261. if leader != nil {
  262. output = p.getBrokerProducer(leader)
  263. }
  264. // highWatermark tracks the "current" retry level, which is the only one where we actually let messages through,
  265. // all other messages get buffered in retryState[msg.retries].buf to preserve ordering
  266. // retryState[msg.retries].expectChaser simply tracks whether we've seen a chaser message for a given level (and
  267. // therefore whether our buffer is complete and safe to flush)
  268. highWatermark := 0
  269. retryState := make([]struct {
  270. buf []*ProducerMessage
  271. expectChaser bool
  272. }, p.conf.Producer.Retry.Max+1)
  273. for msg := range input {
  274. if msg.retries > highWatermark {
  275. // new, higher, retry level; send off a chaser so that we know when everything "in between" has made it
  276. // back to us and we can safely flush the backlog (otherwise we risk re-ordering messages)
  277. highWatermark = msg.retries
  278. Logger.Printf("producer/leader state change to [retrying-%d] on %s/%d\n", highWatermark, topic, partition)
  279. retryState[msg.retries].expectChaser = true
  280. output <- &ProducerMessage{Topic: topic, Partition: partition, flags: chaser, retries: msg.retries - 1}
  281. Logger.Printf("producer/leader abandoning broker %d on %s/%d\n", leader.ID(), topic, partition)
  282. p.unrefBrokerProducer(leader)
  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. } else {
  292. retryState[msg.retries].buf = append(retryState[msg.retries].buf, msg)
  293. }
  294. continue
  295. } else if msg.flags&chaser == chaser {
  296. // this message is of the current retry level (msg.retries == highWatermark) and the chaser flag is set,
  297. // meaning this retry level is done and we can go down (at least) one level and flush that
  298. retryState[highWatermark].expectChaser = false
  299. Logger.Printf("producer/leader state change to [normal-%d] on %s/%d\n", highWatermark, topic, partition)
  300. for {
  301. highWatermark--
  302. Logger.Printf("producer/leader state change to [flushing-%d] on %s/%d\n", highWatermark, topic, partition)
  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 selected broker %d on %s/%d\n", leader.ID(), topic, partition)
  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 state change to [retrying-%d] on %s/%d\n", highWatermark, topic, partition)
  317. break
  318. } else {
  319. Logger.Printf("producer/leader state change to [normal-%d] on %s/%d\n", highWatermark, topic, partition)
  320. if highWatermark == 0 {
  321. break
  322. }
  323. }
  324. }
  325. continue
  326. }
  327. }
  328. // if we made it this far then the current msg contains real data, and can be sent to the next goroutine
  329. // without breaking any of our ordering guarantees
  330. if output == nil {
  331. if err := breaker.Run(doUpdate); err != nil {
  332. p.returnError(msg, err)
  333. time.Sleep(p.conf.Producer.Retry.Backoff)
  334. continue
  335. }
  336. Logger.Printf("producer/leader selected broker %d on %s/%d\n", leader.ID(), topic, partition)
  337. }
  338. output <- msg
  339. }
  340. p.unrefBrokerProducer(leader)
  341. p.retries <- &ProducerMessage{flags: unref}
  342. }
  343. // one per broker
  344. // groups messages together into appropriately-sized batches for sending to the broker
  345. // based on https://godoc.org/github.com/eapache/channels#BatchingChannel
  346. func (p *asyncProducer) messageAggregator(broker *Broker, input chan *ProducerMessage) {
  347. var ticker *time.Ticker
  348. var timer <-chan time.Time
  349. if p.conf.Producer.Flush.Frequency > 0 {
  350. ticker = time.NewTicker(p.conf.Producer.Flush.Frequency)
  351. timer = ticker.C
  352. }
  353. var buffer []*ProducerMessage
  354. var doFlush chan []*ProducerMessage
  355. var bytesAccumulated int
  356. flusher := make(chan []*ProducerMessage)
  357. go withRecover(func() { p.flusher(broker, flusher) })
  358. for {
  359. select {
  360. case msg := <-input:
  361. if msg == nil {
  362. goto shutdown
  363. }
  364. if (bytesAccumulated+msg.byteSize() >= forceFlushThreshold()) ||
  365. (p.conf.Producer.Compression != CompressionNone && bytesAccumulated+msg.byteSize() >= p.conf.Producer.MaxMessageBytes) ||
  366. (p.conf.Producer.Flush.MaxMessages > 0 && len(buffer) >= p.conf.Producer.Flush.MaxMessages) {
  367. Logger.Println("producer/aggregator maximum request accumulated, forcing blocking flush")
  368. flusher <- buffer
  369. buffer = nil
  370. doFlush = nil
  371. bytesAccumulated = 0
  372. }
  373. buffer = append(buffer, msg)
  374. bytesAccumulated += msg.byteSize()
  375. if len(buffer) >= p.conf.Producer.Flush.Messages ||
  376. (p.conf.Producer.Flush.Bytes > 0 && bytesAccumulated >= p.conf.Producer.Flush.Bytes) {
  377. doFlush = flusher
  378. }
  379. case <-timer:
  380. doFlush = flusher
  381. case doFlush <- buffer:
  382. buffer = nil
  383. doFlush = nil
  384. bytesAccumulated = 0
  385. }
  386. }
  387. shutdown:
  388. if ticker != nil {
  389. ticker.Stop()
  390. }
  391. if len(buffer) > 0 {
  392. flusher <- buffer
  393. }
  394. close(flusher)
  395. }
  396. // one per broker
  397. // takes a batch at a time from the messageAggregator and sends to the broker
  398. func (p *asyncProducer) flusher(broker *Broker, input chan []*ProducerMessage) {
  399. var closing error
  400. currentRetries := make(map[string]map[int32]error)
  401. Logger.Printf("producer/flusher/%d starting up\n", broker.ID())
  402. for batch := range input {
  403. if closing != nil {
  404. p.retryMessages(batch, closing)
  405. continue
  406. }
  407. // group messages by topic/partition
  408. msgSets := make(map[string]map[int32][]*ProducerMessage)
  409. for i, msg := range batch {
  410. if currentRetries[msg.Topic] != nil && currentRetries[msg.Topic][msg.Partition] != nil {
  411. if msg.flags&chaser == chaser {
  412. // we can start processing this topic/partition again
  413. Logger.Printf("producer/flusher/%d state change to [normal] on %s/%d\n",
  414. broker.ID(), msg.Topic, msg.Partition)
  415. currentRetries[msg.Topic][msg.Partition] = nil
  416. }
  417. p.retryMessages([]*ProducerMessage{msg}, currentRetries[msg.Topic][msg.Partition])
  418. batch[i] = nil // to prevent it being returned/retried twice
  419. continue
  420. }
  421. partitionSet := msgSets[msg.Topic]
  422. if partitionSet == nil {
  423. partitionSet = make(map[int32][]*ProducerMessage)
  424. msgSets[msg.Topic] = partitionSet
  425. }
  426. partitionSet[msg.Partition] = append(partitionSet[msg.Partition], msg)
  427. }
  428. request := p.buildRequest(msgSets)
  429. if request == nil {
  430. continue
  431. }
  432. response, err := broker.Produce(request)
  433. switch err.(type) {
  434. case nil:
  435. break
  436. case PacketEncodingError:
  437. p.returnErrors(batch, err)
  438. continue
  439. default:
  440. Logger.Printf("producer/flusher/%d state change to [closing] because %s\n", broker.ID(), err)
  441. closing = err
  442. _ = broker.Close()
  443. p.retryMessages(batch, err)
  444. continue
  445. }
  446. if response == nil {
  447. // this only happens when RequiredAcks is NoResponse, so we have to assume success
  448. if p.conf.Producer.Return.Successes {
  449. p.returnSuccesses(batch)
  450. }
  451. continue
  452. }
  453. // we iterate through the blocks in the request, not the response, so that we notice
  454. // if the response is missing a block completely
  455. for topic, partitionSet := range msgSets {
  456. for partition, msgs := range partitionSet {
  457. block := response.GetBlock(topic, partition)
  458. if block == nil {
  459. p.returnErrors(msgs, ErrIncompleteResponse)
  460. continue
  461. }
  462. switch block.Err {
  463. case ErrNoError:
  464. // All the messages for this topic-partition were delivered successfully!
  465. if p.conf.Producer.Return.Successes {
  466. for i := range msgs {
  467. msgs[i].Offset = block.Offset + int64(i)
  468. }
  469. p.returnSuccesses(msgs)
  470. }
  471. case ErrUnknownTopicOrPartition, ErrNotLeaderForPartition, ErrLeaderNotAvailable,
  472. ErrRequestTimedOut, ErrNotEnoughReplicas, ErrNotEnoughReplicasAfterAppend:
  473. Logger.Printf("producer/flusher/%d state change to [retrying] on %s/%d because %v\n",
  474. broker.ID(), topic, partition, block.Err)
  475. if currentRetries[topic] == nil {
  476. currentRetries[topic] = make(map[int32]error)
  477. }
  478. currentRetries[topic][partition] = block.Err
  479. p.retryMessages(msgs, block.Err)
  480. default:
  481. p.returnErrors(msgs, block.Err)
  482. }
  483. }
  484. }
  485. }
  486. Logger.Printf("producer/flusher/%d shut down\n", broker.ID())
  487. p.retries <- &ProducerMessage{flags: unref}
  488. }
  489. // singleton
  490. // effectively a "bridge" between the flushers and the topicDispatcher in order to avoid deadlock
  491. // based on https://godoc.org/github.com/eapache/channels#InfiniteChannel
  492. func (p *asyncProducer) retryHandler() {
  493. var buf []*ProducerMessage
  494. var msg *ProducerMessage
  495. refs := 0
  496. shuttingDown := false
  497. for {
  498. if len(buf) == 0 {
  499. msg = <-p.retries
  500. } else {
  501. select {
  502. case msg = <-p.retries:
  503. case p.input <- buf[0]:
  504. buf = buf[1:]
  505. continue
  506. }
  507. }
  508. if msg.flags&ref != 0 {
  509. refs++
  510. } else if msg.flags&unref != 0 {
  511. refs--
  512. if refs == 0 && shuttingDown {
  513. break
  514. }
  515. } else if msg.flags&shutdown != 0 {
  516. shuttingDown = true
  517. if refs == 0 {
  518. break
  519. }
  520. } else {
  521. buf = append(buf, msg)
  522. }
  523. }
  524. close(p.retries)
  525. for i := range buf {
  526. p.input <- buf[i]
  527. }
  528. close(p.input)
  529. }
  530. ///////////////////////////////////////////
  531. ///////////////////////////////////////////
  532. // utility functions
  533. func (p *asyncProducer) assignPartition(partitioner Partitioner, msg *ProducerMessage) error {
  534. var partitions []int32
  535. var err error
  536. if partitioner.RequiresConsistency() {
  537. partitions, err = p.client.Partitions(msg.Topic)
  538. } else {
  539. partitions, err = p.client.WritablePartitions(msg.Topic)
  540. }
  541. if err != nil {
  542. return err
  543. }
  544. numPartitions := int32(len(partitions))
  545. if numPartitions == 0 {
  546. return ErrLeaderNotAvailable
  547. }
  548. choice, err := partitioner.Partition(msg, numPartitions)
  549. if err != nil {
  550. return err
  551. } else if choice < 0 || choice >= numPartitions {
  552. return ErrInvalidPartition
  553. }
  554. msg.Partition = partitions[choice]
  555. return nil
  556. }
  557. func (p *asyncProducer) buildRequest(batch map[string]map[int32][]*ProducerMessage) *ProduceRequest {
  558. req := &ProduceRequest{RequiredAcks: p.conf.Producer.RequiredAcks, Timeout: int32(p.conf.Producer.Timeout / time.Millisecond)}
  559. empty := true
  560. for topic, partitionSet := range batch {
  561. for partition, msgSet := range partitionSet {
  562. setToSend := new(MessageSet)
  563. setSize := 0
  564. for _, msg := range msgSet {
  565. var keyBytes, valBytes []byte
  566. var err error
  567. if msg.Key != nil {
  568. if keyBytes, err = msg.Key.Encode(); err != nil {
  569. p.returnError(msg, err)
  570. continue
  571. }
  572. }
  573. if msg.Value != nil {
  574. if valBytes, err = msg.Value.Encode(); err != nil {
  575. p.returnError(msg, err)
  576. continue
  577. }
  578. }
  579. if p.conf.Producer.Compression != CompressionNone && setSize+msg.byteSize() > p.conf.Producer.MaxMessageBytes {
  580. // compression causes message-sets to be wrapped as single messages, which have tighter
  581. // size requirements, so we have to respect those limits
  582. valBytes, err := encode(setToSend)
  583. if err != nil {
  584. Logger.Println(err) // if this happens, it's basically our fault.
  585. panic(err)
  586. }
  587. req.AddMessage(topic, partition, &Message{Codec: p.conf.Producer.Compression, Key: nil, Value: valBytes})
  588. setToSend = new(MessageSet)
  589. setSize = 0
  590. }
  591. setSize += msg.byteSize()
  592. setToSend.addMessage(&Message{Codec: CompressionNone, Key: keyBytes, Value: valBytes})
  593. empty = false
  594. }
  595. if p.conf.Producer.Compression == CompressionNone {
  596. req.AddSet(topic, partition, setToSend)
  597. } else {
  598. valBytes, err := encode(setToSend)
  599. if err != nil {
  600. Logger.Println(err) // if this happens, it's basically our fault.
  601. panic(err)
  602. }
  603. req.AddMessage(topic, partition, &Message{Codec: p.conf.Producer.Compression, Key: nil, Value: valBytes})
  604. }
  605. }
  606. }
  607. if empty {
  608. return nil
  609. }
  610. return req
  611. }
  612. func (p *asyncProducer) returnError(msg *ProducerMessage, err error) {
  613. msg.flags = 0
  614. msg.retries = 0
  615. pErr := &ProducerError{Msg: msg, Err: err}
  616. if p.conf.Producer.Return.Errors {
  617. p.errors <- pErr
  618. } else {
  619. Logger.Println(pErr)
  620. }
  621. }
  622. func (p *asyncProducer) returnErrors(batch []*ProducerMessage, err error) {
  623. for _, msg := range batch {
  624. if msg != nil {
  625. p.returnError(msg, err)
  626. }
  627. }
  628. }
  629. func (p *asyncProducer) returnSuccesses(batch []*ProducerMessage) {
  630. for _, msg := range batch {
  631. if msg != nil {
  632. msg.flags = 0
  633. p.successes <- msg
  634. }
  635. }
  636. }
  637. func (p *asyncProducer) retryMessages(batch []*ProducerMessage, err error) {
  638. for _, msg := range batch {
  639. if msg == nil {
  640. continue
  641. }
  642. if msg.retries >= p.conf.Producer.Retry.Max {
  643. p.returnError(msg, err)
  644. } else {
  645. msg.retries++
  646. p.retries <- msg
  647. }
  648. }
  649. }
  650. type brokerProducer struct {
  651. input chan *ProducerMessage
  652. refs int
  653. }
  654. func (p *asyncProducer) getBrokerProducer(broker *Broker) chan *ProducerMessage {
  655. p.brokerLock.Lock()
  656. defer p.brokerLock.Unlock()
  657. producer := p.brokers[broker]
  658. if producer == nil {
  659. p.retries <- &ProducerMessage{flags: ref}
  660. producer = &brokerProducer{
  661. refs: 1,
  662. input: make(chan *ProducerMessage),
  663. }
  664. p.brokers[broker] = producer
  665. go withRecover(func() { p.messageAggregator(broker, producer.input) })
  666. } else {
  667. producer.refs++
  668. }
  669. return producer.input
  670. }
  671. func (p *asyncProducer) unrefBrokerProducer(broker *Broker) {
  672. p.brokerLock.Lock()
  673. defer p.brokerLock.Unlock()
  674. producer := p.brokers[broker]
  675. if producer != nil {
  676. producer.refs--
  677. if producer.refs == 0 {
  678. close(producer.input)
  679. delete(p.brokers, broker)
  680. }
  681. }
  682. }