async_producer.go 24 KB

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