async_producer.go 24 KB

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