producer.go 28 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676677678679680681682683684685686687688689690691692693694695696697698699700701702703704705706707708709710711712713714715716717718719720721722723724725726727728729730731732733734735736737738739740741742743744745746747748749750751752753754755756757758759760761762763764765766767768769770771772773774775776777778779780781782783784785786787788789790791792793794795796797798799800801802803804805806807808809810811812813814815816817818819820821822823824825826827828829830831832833834835836837838839840841842843844845846847848849850851852853854855856857858859860861862863864865866867
  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. // ProducerConfig is used to pass multiple configuration options to NewProducer.
  12. //
  13. // Some of these configuration settings match settings with the JVM producer, but some of
  14. // these are implementation specific and have no equivalent in the JVM producer.
  15. type ProducerConfig struct {
  16. Partitioner PartitionerConstructor // Generates partitioners for choosing the partition to send messages to (defaults to hash). Similar to the `partitioner.class` setting for the JVM producer.
  17. RequiredAcks RequiredAcks // The level of acknowledgement reliability needed from the broker (defaults to WaitForLocal). Equivalent to the `request.required.acks` setting of the JVM producer.
  18. Timeout time.Duration // The maximum duration the broker will wait the receipt of the number of RequiredAcks (defaults to 10 seconds). This is only relevant when RequiredAcks is set to WaitForAll or a number > 1. Only supports millisecond resolution, nanoseconds will be truncated. Equivalent to the JVM producer's `request.timeout.ms` setting.
  19. Compression CompressionCodec // The type of compression to use on messages (defaults to no compression). Similar to `compression.codec` setting of the JVM producer.
  20. FlushMsgCount int // The number of messages needed to trigger a flush. This is a best effort; the number of messages may be more or less. Use `MaxMessagesPerReq` to set a hard upper limit.
  21. FlushFrequency time.Duration // If this amount of time elapses without a flush, one will be queued. The frequency is a best effort, and the actual frequency can be more or less. Equivalent to `queue.buffering.max.ms` setting of JVM producer.
  22. FlushByteCount int // If this many bytes of messages are accumulated, a flush will be triggered. This is a best effort; the number of bytes may be more or less. Use the gloabl `sarama.MaxRequestSize` to set a hard upper limit.
  23. AckSuccesses bool // If enabled, successfully delivered messages will be returned on the Successes channel.
  24. MaxMessageBytes int // The maximum permitted size of a message (defaults to 1000000). Equivalent to the broker's `message.max.bytes`.
  25. MaxMessagesPerReq int // The maximum number of messages the producer will send in a single broker request. Defaults to 0 for unlimited. The global setting MaxRequestSize still applies. Similar to `queue.buffering.max.messages` in the JVM producer.
  26. ChannelBufferSize int // The size of the buffers of the channels between the different goroutines (defaults to 256).
  27. RetryBackoff time.Duration // The amount of time to wait for the cluster to elect a new leader before processing retries (defaults to 100ms). Similar to the retry.backoff.ms setting of the JVM producer.
  28. MaxRetries int // The total number of times to retry sending a message (defaults to 3). Similar to the message.send.max.retries setting of the JVM producer.
  29. }
  30. // NewProducerConfig creates a new ProducerConfig instance with sensible defaults.
  31. func NewProducerConfig() *ProducerConfig {
  32. return &ProducerConfig{
  33. Partitioner: NewHashPartitioner,
  34. RequiredAcks: WaitForLocal,
  35. MaxMessageBytes: 1000000,
  36. ChannelBufferSize: 256,
  37. RetryBackoff: 100 * time.Millisecond,
  38. Timeout: 10 * time.Second,
  39. MaxRetries: 3,
  40. }
  41. }
  42. // Validate checks a ProducerConfig instance. It will return a
  43. // ConfigurationError if the specified value doesn't make sense.
  44. func (config *ProducerConfig) Validate() error {
  45. if config.RequiredAcks < -1 {
  46. return ConfigurationError("Invalid RequiredAcks")
  47. } else if config.RequiredAcks > 1 {
  48. Logger.Println("ProducerConfig.RequiredAcks > 1 is deprecated and will raise an exception with kafka >= 0.8.2.0.")
  49. }
  50. if config.Timeout < 0 {
  51. return ConfigurationError("Invalid Timeout")
  52. } else if config.Timeout%time.Millisecond != 0 {
  53. Logger.Println("ProducerConfig.Timeout only supports millisecond resolution; nanoseconds will be truncated.")
  54. }
  55. if config.RequiredAcks == WaitForAll && config.Timeout == 0 {
  56. return ConfigurationError("If you WaitForAll you must specify a non-zero timeout to wait.")
  57. }
  58. if config.FlushMsgCount < 0 {
  59. return ConfigurationError("Invalid FlushMsgCount")
  60. }
  61. if config.FlushByteCount < 0 {
  62. return ConfigurationError("Invalid FlushByteCount")
  63. } else if config.FlushByteCount >= forceFlushThreshold() {
  64. Logger.Println("ProducerConfig.FlushByteCount too close to MaxRequestSize; it will be ignored.")
  65. }
  66. if config.FlushFrequency < 0 {
  67. return ConfigurationError("Invalid FlushFrequency")
  68. }
  69. if config.Partitioner == nil {
  70. return ConfigurationError("No partitioner set")
  71. }
  72. if config.MaxMessageBytes <= 0 {
  73. return ConfigurationError("Invalid MaxMessageBytes")
  74. } else if config.MaxMessageBytes >= forceFlushThreshold() {
  75. Logger.Println("ProducerConfig.MaxMessageBytes too close to MaxRequestSize; it will be ignored.")
  76. }
  77. if config.MaxMessagesPerReq < 0 || (config.MaxMessagesPerReq > 0 && config.MaxMessagesPerReq < config.FlushMsgCount) {
  78. return ConfigurationError("Invalid MaxMessagesPerReq, must be non-negative and >= FlushMsgCount if set")
  79. }
  80. if config.RetryBackoff < 0 {
  81. return ConfigurationError("Invalid RetryBackoff")
  82. }
  83. if config.MaxRetries < 0 {
  84. return ConfigurationError("Invalid MaxRetries")
  85. }
  86. return nil
  87. }
  88. // Producer publishes Kafka messages. It routes messages to the correct broker
  89. // for the provided topic-partition, refreshing metadata as appropriate, and
  90. // parses responses for errors. You must read from the Errors() channel or the
  91. // producer will deadlock. You must call Close() or AsyncClose() on a producer to avoid
  92. // leaks: it will not be garbage-collected automatically when it passes out of
  93. // scope (this is in addition to calling Close on the underlying client, which
  94. // is still necessary).
  95. type Producer struct {
  96. client *Client
  97. config ProducerConfig
  98. errors chan *ProduceError
  99. input, successes, retries chan *MessageToSend
  100. brokers map[*Broker]*brokerWorker
  101. brokerLock sync.Mutex
  102. }
  103. // NewProducer creates a new Producer using the given client.
  104. func NewProducer(client *Client, config *ProducerConfig) (*Producer, error) {
  105. // Check that we are not dealing with a closed Client before processing
  106. // any other arguments
  107. if client.Closed() {
  108. return nil, ClosedClient
  109. }
  110. if config == nil {
  111. config = NewProducerConfig()
  112. }
  113. if err := config.Validate(); err != nil {
  114. return nil, err
  115. }
  116. p := &Producer{
  117. client: client,
  118. config: *config,
  119. errors: make(chan *ProduceError),
  120. input: make(chan *MessageToSend),
  121. successes: make(chan *MessageToSend),
  122. retries: make(chan *MessageToSend),
  123. brokers: make(map[*Broker]*brokerWorker),
  124. }
  125. // launch our singleton dispatchers
  126. go withRecover(p.topicDispatcher)
  127. go withRecover(p.retryHandler)
  128. return p, nil
  129. }
  130. type flagSet int8
  131. const (
  132. chaser flagSet = 1 << iota // message is last in a group that failed
  133. ref // add a reference to a singleton channel
  134. unref // remove a reference from a singleton channel
  135. shutdown // start the shutdown process
  136. )
  137. // MessageToSend is the collection of elements passed to the Producer in order to send a message.
  138. type MessageToSend struct {
  139. Topic string // The Kafka topic for this message.
  140. Key Encoder // The partitioning key for this message. It must implement the Encoder interface. Pre-existing Encoders include StringEncoder and ByteEncoder.
  141. Value Encoder // The actual message to store in Kafka. It must implement the Encoder interface. Pre-existing Encoders include StringEncoder and ByteEncoder.
  142. 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.
  143. // these are filled in by the producer as the message is processed
  144. offset int64
  145. partition int32
  146. retries int
  147. flags flagSet
  148. }
  149. // Offset is the offset of the message stored on the broker. This is only guaranteed to be defined if
  150. // the message was successfully delivered and RequiredAcks is not NoResponse.
  151. func (m *MessageToSend) Offset() int64 {
  152. return m.offset
  153. }
  154. // Partition is the partition that the message was sent to. This is only guaranteed to be defined if
  155. // the message was successfully delivered.
  156. func (m *MessageToSend) Partition() int32 {
  157. return m.partition
  158. }
  159. func (m *MessageToSend) byteSize() int {
  160. size := 26 // the metadata overhead of CRC, flags, etc.
  161. if m.Key != nil {
  162. size += m.Key.Length()
  163. }
  164. if m.Value != nil {
  165. size += m.Value.Length()
  166. }
  167. return size
  168. }
  169. // ProduceError is the type of error generated when the producer fails to deliver a message.
  170. // It contains the original MessageToSend as well as the actual error value.
  171. type ProduceError struct {
  172. Msg *MessageToSend
  173. Err error
  174. }
  175. // ProduceErrors is a type that wraps a batch of "ProduceError"s and implements the Error interface.
  176. // It can be returned from the Producer's Close method to avoid the need to manually drain the Errors channel
  177. // when closing a producer.
  178. type ProduceErrors []*ProduceError
  179. func (pe ProduceErrors) Error() string {
  180. return fmt.Sprintf("kafka: Failed to deliver %d messages.", len(pe))
  181. }
  182. // Errors is the error output channel back to the user. You MUST read from this channel or the Producer will deadlock.
  183. // It is suggested that you send messages and read errors together in a single select statement.
  184. func (p *Producer) Errors() <-chan *ProduceError {
  185. return p.errors
  186. }
  187. // Successes is the success output channel back to the user when AckSuccesses is configured.
  188. // If AckSuccesses is true, you MUST read from this channel or the Producer will deadlock.
  189. // It is suggested that you send and read messages together in a single select statement.
  190. func (p *Producer) Successes() <-chan *MessageToSend {
  191. return p.successes
  192. }
  193. // Input is the input channel for the user to write messages to that they wish to send.
  194. func (p *Producer) Input() chan<- *MessageToSend {
  195. return p.input
  196. }
  197. // Close shuts down the producer and flushes any messages it may have buffered.
  198. // You must call this function before a producer object passes out of scope, as
  199. // it may otherwise leak memory. You must call this before calling Close on the
  200. // underlying client.
  201. func (p *Producer) Close() error {
  202. p.AsyncClose()
  203. if p.config.AckSuccesses {
  204. go withRecover(func() {
  205. for _ = range p.successes {
  206. }
  207. })
  208. }
  209. var errors ProduceErrors
  210. for event := range p.errors {
  211. errors = append(errors, event)
  212. }
  213. if len(errors) > 0 {
  214. return errors
  215. }
  216. return nil
  217. }
  218. // AsyncClose triggers a shutdown of the producer, flushing any messages it may have
  219. // buffered. The shutdown has completed when both the Errors and Successes channels
  220. // have been closed. When calling AsyncClose, you *must* continue to read from those
  221. // channels in order to drain the results of any messages in flight.
  222. func (p *Producer) AsyncClose() {
  223. go withRecover(func() {
  224. p.input <- &MessageToSend{flags: shutdown}
  225. })
  226. }
  227. ///////////////////////////////////////////
  228. // In normal processing, a message flows through the following functions from top to bottom,
  229. // starting at topicDispatcher (which reads from Producer.input) and ending in flusher
  230. // (which sends the message to the broker). In cases where a message must be retried, it goes
  231. // through retryHandler before being returned to the top of the flow.
  232. ///////////////////////////////////////////
  233. // singleton
  234. // dispatches messages by topic
  235. func (p *Producer) topicDispatcher() {
  236. handlers := make(map[string]chan *MessageToSend)
  237. for msg := range p.input {
  238. if msg == nil {
  239. Logger.Println("Something tried to send a nil message, it was ignored.")
  240. continue
  241. }
  242. if msg.flags&shutdown != 0 {
  243. Logger.Println("Producer shutting down.")
  244. break
  245. }
  246. if (p.config.Compression == CompressionNone && msg.Value != nil && msg.Value.Length() > p.config.MaxMessageBytes) ||
  247. (msg.byteSize() > p.config.MaxMessageBytes) {
  248. p.returnError(msg, MessageSizeTooLarge)
  249. continue
  250. }
  251. handler := handlers[msg.Topic]
  252. if handler == nil {
  253. p.retries <- &MessageToSend{flags: ref}
  254. newHandler := make(chan *MessageToSend, p.config.ChannelBufferSize)
  255. topic := msg.Topic // block local because go's closure semantics suck
  256. go withRecover(func() { p.partitionDispatcher(topic, newHandler) })
  257. handler = newHandler
  258. handlers[msg.Topic] = handler
  259. }
  260. handler <- msg
  261. }
  262. for _, handler := range handlers {
  263. close(handler)
  264. }
  265. p.retries <- &MessageToSend{flags: shutdown}
  266. for msg := range p.input {
  267. p.returnError(msg, ShuttingDown)
  268. }
  269. close(p.errors)
  270. close(p.successes)
  271. }
  272. // one per topic
  273. // partitions messages, then dispatches them by partition
  274. func (p *Producer) partitionDispatcher(topic string, input chan *MessageToSend) {
  275. handlers := make(map[int32]chan *MessageToSend)
  276. partitioner := p.config.Partitioner()
  277. for msg := range input {
  278. if msg.retries == 0 {
  279. err := p.assignPartition(partitioner, msg)
  280. if err != nil {
  281. p.returnError(msg, err)
  282. continue
  283. }
  284. }
  285. handler := handlers[msg.partition]
  286. if handler == nil {
  287. p.retries <- &MessageToSend{flags: ref}
  288. newHandler := make(chan *MessageToSend, p.config.ChannelBufferSize)
  289. topic := msg.Topic // block local because go's closure semantics suck
  290. partition := msg.partition // block local because go's closure semantics suck
  291. go withRecover(func() { p.leaderDispatcher(topic, partition, newHandler) })
  292. handler = newHandler
  293. handlers[msg.partition] = handler
  294. }
  295. handler <- msg
  296. }
  297. for _, handler := range handlers {
  298. close(handler)
  299. }
  300. p.retries <- &MessageToSend{flags: unref}
  301. }
  302. // one per partition per topic
  303. // dispatches messages to the appropriate broker
  304. // also responsible for maintaining message order during retries
  305. func (p *Producer) leaderDispatcher(topic string, partition int32, input chan *MessageToSend) {
  306. var leader *Broker
  307. var output chan *MessageToSend
  308. breaker := breaker.New(3, 1, 10*time.Second)
  309. doUpdate := func() (err error) {
  310. if err = p.client.RefreshTopicMetadata(topic); err != nil {
  311. return err
  312. }
  313. if leader, err = p.client.Leader(topic, partition); err != nil {
  314. return err
  315. }
  316. output = p.getBrokerWorker(leader)
  317. return nil
  318. }
  319. // try to prefetch the leader; if this doesn't work, we'll do a proper breaker-protected refresh-and-fetch
  320. // on the first message
  321. leader, _ = p.client.Leader(topic, partition)
  322. if leader != nil {
  323. output = p.getBrokerWorker(leader)
  324. }
  325. // highWatermark tracks the "current" retry level, which is the only one where we actually let messages through,
  326. // all other messages get buffered in retryState[msg.retries].buf to preserve ordering
  327. // retryState[msg.retries].expectChaser simply tracks whether we've seen a chaser message for a given level (and
  328. // therefore whether our buffer is complete and safe to flush)
  329. highWatermark := 0
  330. retryState := make([]struct {
  331. buf []*MessageToSend
  332. expectChaser bool
  333. }, p.config.MaxRetries+1)
  334. for msg := range input {
  335. if msg.retries > highWatermark {
  336. // new, higher, retry level; send off a chaser so that we know when everything "in between" has made it
  337. // back to us and we can safely flush the backlog (otherwise we risk re-ordering messages)
  338. highWatermark = msg.retries
  339. Logger.Printf("producer/leader state change to [retrying-%d] on %s/%d\n", highWatermark, topic, partition)
  340. retryState[msg.retries].expectChaser = true
  341. output <- &MessageToSend{Topic: topic, partition: partition, flags: chaser, retries: msg.retries - 1}
  342. Logger.Printf("producer/leader abandoning broker %d on %s/%d\n", leader.ID(), topic, partition)
  343. p.unrefBrokerWorker(leader)
  344. output = nil
  345. time.Sleep(p.config.RetryBackoff)
  346. } else if highWatermark > 0 {
  347. // we are retrying something (else highWatermark would be 0) but this message is not a *new* retry level
  348. if msg.retries < highWatermark {
  349. // in fact this message is not even the current retry level, so buffer it for now (unless it's a just a chaser)
  350. if msg.flags&chaser == chaser {
  351. retryState[msg.retries].expectChaser = false
  352. } else {
  353. retryState[msg.retries].buf = append(retryState[msg.retries].buf, msg)
  354. }
  355. continue
  356. } else if msg.flags&chaser == chaser {
  357. // this message is of the current retry level (msg.retries == highWatermark) and the chaser flag is set,
  358. // meaning this retry level is done and we can go down (at least) one level and flush that
  359. retryState[highWatermark].expectChaser = false
  360. Logger.Printf("producer/leader state change to [normal-%d] on %s/%d\n", highWatermark, topic, partition)
  361. for {
  362. highWatermark--
  363. Logger.Printf("producer/leader state change to [flushing-%d] on %s/%d\n", highWatermark, topic, partition)
  364. if output == nil {
  365. if err := breaker.Run(doUpdate); err != nil {
  366. p.returnErrors(retryState[highWatermark].buf, err)
  367. goto flushDone
  368. }
  369. Logger.Printf("producer/leader selected broker %d on %s/%d\n", leader.ID(), topic, partition)
  370. }
  371. for _, msg := range retryState[highWatermark].buf {
  372. output <- msg
  373. }
  374. flushDone:
  375. retryState[highWatermark].buf = nil
  376. if retryState[highWatermark].expectChaser {
  377. Logger.Printf("producer/leader state change to [retrying-%d] on %s/%d\n", highWatermark, topic, partition)
  378. break
  379. } else {
  380. Logger.Printf("producer/leader state change to [normal-%d] on %s/%d\n", highWatermark, topic, partition)
  381. if highWatermark == 0 {
  382. break
  383. }
  384. }
  385. }
  386. continue
  387. }
  388. }
  389. // if we made it this far then the current msg contains real data, and can be sent to the next goroutine
  390. // without breaking any of our ordering guarantees
  391. if output == nil {
  392. if err := breaker.Run(doUpdate); err != nil {
  393. p.returnError(msg, err)
  394. time.Sleep(p.config.RetryBackoff)
  395. continue
  396. }
  397. Logger.Printf("producer/leader selected broker %d on %s/%d\n", leader.ID(), topic, partition)
  398. }
  399. output <- msg
  400. }
  401. p.unrefBrokerWorker(leader)
  402. p.retries <- &MessageToSend{flags: unref}
  403. }
  404. // one per broker
  405. // groups messages together into appropriately-sized batches for sending to the broker
  406. // based on https://godoc.org/github.com/eapache/channels#BatchingChannel
  407. func (p *Producer) messageAggregator(broker *Broker, input chan *MessageToSend) {
  408. var ticker *time.Ticker
  409. var timer <-chan time.Time
  410. if p.config.FlushFrequency > 0 {
  411. ticker = time.NewTicker(p.config.FlushFrequency)
  412. timer = ticker.C
  413. }
  414. var buffer []*MessageToSend
  415. var doFlush chan []*MessageToSend
  416. var bytesAccumulated int
  417. flusher := make(chan []*MessageToSend)
  418. go withRecover(func() { p.flusher(broker, flusher) })
  419. for {
  420. select {
  421. case msg := <-input:
  422. if msg == nil {
  423. goto shutdown
  424. }
  425. if (bytesAccumulated+msg.byteSize() >= forceFlushThreshold()) ||
  426. (p.config.Compression != CompressionNone && bytesAccumulated+msg.byteSize() >= p.config.MaxMessageBytes) ||
  427. (p.config.MaxMessagesPerReq > 0 && len(buffer) >= p.config.MaxMessagesPerReq) {
  428. Logger.Println("producer/aggregator maximum request accumulated, forcing blocking flush")
  429. flusher <- buffer
  430. buffer = nil
  431. doFlush = nil
  432. bytesAccumulated = 0
  433. }
  434. buffer = append(buffer, msg)
  435. bytesAccumulated += msg.byteSize()
  436. if len(buffer) >= p.config.FlushMsgCount ||
  437. (p.config.FlushByteCount > 0 && bytesAccumulated >= p.config.FlushByteCount) {
  438. doFlush = flusher
  439. }
  440. case <-timer:
  441. doFlush = flusher
  442. case doFlush <- buffer:
  443. buffer = nil
  444. doFlush = nil
  445. bytesAccumulated = 0
  446. }
  447. }
  448. shutdown:
  449. if ticker != nil {
  450. ticker.Stop()
  451. }
  452. if len(buffer) > 0 {
  453. flusher <- buffer
  454. }
  455. close(flusher)
  456. }
  457. // one per broker
  458. // takes a batch at a time from the messageAggregator and sends to the broker
  459. func (p *Producer) flusher(broker *Broker, input chan []*MessageToSend) {
  460. var closing error
  461. currentRetries := make(map[string]map[int32]error)
  462. Logger.Printf("producer/flusher/%d starting up\n", broker.ID())
  463. for batch := range input {
  464. if closing != nil {
  465. p.retryMessages(batch, closing)
  466. continue
  467. }
  468. // group messages by topic/partition
  469. msgSets := make(map[string]map[int32][]*MessageToSend)
  470. for i, msg := range batch {
  471. if currentRetries[msg.Topic] != nil && currentRetries[msg.Topic][msg.partition] != nil {
  472. if msg.flags&chaser == chaser {
  473. // we can start processing this topic/partition again
  474. Logger.Printf("producer/flusher/%d state change to [normal] on %s/%d\n",
  475. broker.ID(), msg.Topic, msg.partition)
  476. currentRetries[msg.Topic][msg.partition] = nil
  477. }
  478. p.retryMessages([]*MessageToSend{msg}, currentRetries[msg.Topic][msg.partition])
  479. batch[i] = nil // to prevent it being returned/retried twice
  480. continue
  481. }
  482. partitionSet := msgSets[msg.Topic]
  483. if partitionSet == nil {
  484. partitionSet = make(map[int32][]*MessageToSend)
  485. msgSets[msg.Topic] = partitionSet
  486. }
  487. partitionSet[msg.partition] = append(partitionSet[msg.partition], msg)
  488. }
  489. request := p.buildRequest(msgSets)
  490. if request == nil {
  491. continue
  492. }
  493. response, err := broker.Produce(p.client.id, request)
  494. switch err {
  495. case nil:
  496. break
  497. case EncodingError:
  498. p.returnErrors(batch, err)
  499. continue
  500. default:
  501. p.client.disconnectBroker(broker)
  502. Logger.Printf("producer/flusher/%d state change to [closing] because %s\n", broker.ID(), err)
  503. closing = err
  504. p.retryMessages(batch, err)
  505. continue
  506. }
  507. if response == nil {
  508. // this only happens when RequiredAcks is NoResponse, so we have to assume success
  509. if p.config.AckSuccesses {
  510. p.returnSuccesses(batch)
  511. }
  512. continue
  513. }
  514. // we iterate through the blocks in the request, not the response, so that we notice
  515. // if the response is missing a block completely
  516. for topic, partitionSet := range msgSets {
  517. for partition, msgs := range partitionSet {
  518. block := response.GetBlock(topic, partition)
  519. if block == nil {
  520. p.returnErrors(msgs, IncompleteResponse)
  521. continue
  522. }
  523. switch block.Err {
  524. case NoError:
  525. // All the messages for this topic-partition were delivered successfully!
  526. if p.config.AckSuccesses {
  527. for i := range msgs {
  528. msgs[i].offset = block.Offset + int64(i)
  529. }
  530. p.returnSuccesses(msgs)
  531. }
  532. case UnknownTopicOrPartition, NotLeaderForPartition, LeaderNotAvailable, RequestTimedOut:
  533. Logger.Printf("producer/flusher/%d state change to [retrying] on %s/%d because %v\n",
  534. broker.ID(), topic, partition, block.Err)
  535. if currentRetries[topic] == nil {
  536. currentRetries[topic] = make(map[int32]error)
  537. }
  538. currentRetries[topic][partition] = block.Err
  539. p.retryMessages(msgs, block.Err)
  540. default:
  541. p.returnErrors(msgs, block.Err)
  542. }
  543. }
  544. }
  545. }
  546. Logger.Printf("producer/flusher/%d shut down\n", broker.ID())
  547. p.retries <- &MessageToSend{flags: unref}
  548. }
  549. // singleton
  550. // effectively a "bridge" between the flushers and the topicDispatcher in order to avoid deadlock
  551. // based on https://godoc.org/github.com/eapache/channels#InfiniteChannel
  552. func (p *Producer) retryHandler() {
  553. var buf []*MessageToSend
  554. var msg *MessageToSend
  555. refs := 0
  556. shuttingDown := false
  557. for {
  558. if len(buf) == 0 {
  559. msg = <-p.retries
  560. } else {
  561. select {
  562. case msg = <-p.retries:
  563. case p.input <- buf[0]:
  564. buf = buf[1:]
  565. continue
  566. }
  567. }
  568. if msg.flags&ref != 0 {
  569. refs++
  570. } else if msg.flags&unref != 0 {
  571. refs--
  572. if refs == 0 && shuttingDown {
  573. break
  574. }
  575. } else if msg.flags&shutdown != 0 {
  576. shuttingDown = true
  577. if refs == 0 {
  578. break
  579. }
  580. } else {
  581. buf = append(buf, msg)
  582. }
  583. }
  584. close(p.retries)
  585. for i := range buf {
  586. p.input <- buf[i]
  587. }
  588. close(p.input)
  589. }
  590. ///////////////////////////////////////////
  591. ///////////////////////////////////////////
  592. // utility functions
  593. func (p *Producer) assignPartition(partitioner Partitioner, msg *MessageToSend) error {
  594. var partitions []int32
  595. var err error
  596. if partitioner.RequiresConsistency() {
  597. partitions, err = p.client.Partitions(msg.Topic)
  598. } else {
  599. partitions, err = p.client.WritablePartitions(msg.Topic)
  600. }
  601. if err != nil {
  602. return err
  603. }
  604. numPartitions := int32(len(partitions))
  605. if numPartitions == 0 {
  606. return LeaderNotAvailable
  607. }
  608. choice, err := partitioner.Partition(msg.Key, numPartitions)
  609. if err != nil {
  610. return err
  611. } else if choice < 0 || choice >= numPartitions {
  612. return InvalidPartition
  613. }
  614. msg.partition = partitions[choice]
  615. return nil
  616. }
  617. func (p *Producer) buildRequest(batch map[string]map[int32][]*MessageToSend) *ProduceRequest {
  618. req := &ProduceRequest{RequiredAcks: p.config.RequiredAcks, Timeout: int32(p.config.Timeout / time.Millisecond)}
  619. empty := true
  620. for topic, partitionSet := range batch {
  621. for partition, msgSet := range partitionSet {
  622. setToSend := new(MessageSet)
  623. setSize := 0
  624. for _, msg := range msgSet {
  625. var keyBytes, valBytes []byte
  626. var err error
  627. if msg.Key != nil {
  628. if keyBytes, err = msg.Key.Encode(); err != nil {
  629. p.returnError(msg, err)
  630. continue
  631. }
  632. }
  633. if msg.Value != nil {
  634. if valBytes, err = msg.Value.Encode(); err != nil {
  635. p.returnError(msg, err)
  636. continue
  637. }
  638. }
  639. if p.config.Compression != CompressionNone && setSize+msg.byteSize() > p.config.MaxMessageBytes {
  640. // compression causes message-sets to be wrapped as single messages, which have tighter
  641. // size requirements, so we have to respect those limits
  642. valBytes, err := encode(setToSend)
  643. if err != nil {
  644. Logger.Println(err) // if this happens, it's basically our fault.
  645. panic(err)
  646. }
  647. req.AddMessage(topic, partition, &Message{Codec: p.config.Compression, Key: nil, Value: valBytes})
  648. setToSend = new(MessageSet)
  649. setSize = 0
  650. }
  651. setSize += msg.byteSize()
  652. setToSend.addMessage(&Message{Codec: CompressionNone, Key: keyBytes, Value: valBytes})
  653. empty = false
  654. }
  655. if p.config.Compression == CompressionNone {
  656. req.AddSet(topic, partition, setToSend)
  657. } else {
  658. valBytes, err := encode(setToSend)
  659. if err != nil {
  660. Logger.Println(err) // if this happens, it's basically our fault.
  661. panic(err)
  662. }
  663. req.AddMessage(topic, partition, &Message{Codec: p.config.Compression, Key: nil, Value: valBytes})
  664. }
  665. }
  666. }
  667. if empty {
  668. return nil
  669. }
  670. return req
  671. }
  672. func (p *Producer) returnError(msg *MessageToSend, err error) {
  673. msg.flags = 0
  674. msg.retries = 0
  675. p.errors <- &ProduceError{Msg: msg, Err: err}
  676. }
  677. func (p *Producer) returnErrors(batch []*MessageToSend, err error) {
  678. for _, msg := range batch {
  679. if msg != nil {
  680. p.returnError(msg, err)
  681. }
  682. }
  683. }
  684. func (p *Producer) returnSuccesses(batch []*MessageToSend) {
  685. for _, msg := range batch {
  686. if msg != nil {
  687. msg.flags = 0
  688. p.successes <- msg
  689. }
  690. }
  691. }
  692. func (p *Producer) retryMessages(batch []*MessageToSend, err error) {
  693. for _, msg := range batch {
  694. if msg == nil {
  695. continue
  696. }
  697. if msg.retries >= p.config.MaxRetries {
  698. p.returnError(msg, err)
  699. } else {
  700. msg.retries++
  701. p.retries <- msg
  702. }
  703. }
  704. }
  705. type brokerWorker struct {
  706. input chan *MessageToSend
  707. refs int
  708. }
  709. func (p *Producer) getBrokerWorker(broker *Broker) chan *MessageToSend {
  710. p.brokerLock.Lock()
  711. defer p.brokerLock.Unlock()
  712. worker := p.brokers[broker]
  713. if worker == nil {
  714. p.retries <- &MessageToSend{flags: ref}
  715. worker = &brokerWorker{
  716. refs: 1,
  717. input: make(chan *MessageToSend),
  718. }
  719. p.brokers[broker] = worker
  720. go withRecover(func() { p.messageAggregator(broker, worker.input) })
  721. } else {
  722. worker.refs++
  723. }
  724. return worker.input
  725. }
  726. func (p *Producer) unrefBrokerWorker(broker *Broker) {
  727. p.brokerLock.Lock()
  728. defer p.brokerLock.Unlock()
  729. worker := p.brokers[broker]
  730. if worker != nil {
  731. worker.refs--
  732. if worker.refs == 0 {
  733. close(worker.input)
  734. delete(p.brokers, broker)
  735. }
  736. }
  737. }