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