async_producer.go 30 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676677678679680681682683684685686687688689690691692693694695696697698699700701702703704705706707708709710711712713714715716717718719720721722723724725726727728729730731732733734735736737738739740741742743744745746747748749750751752753754755756757758759760761762763764765766767768769770771772773774775776777778779780781782783784785786787788789790791792793794795796797798799800801802803804805806807808809810811812813814815816817818819820821822823824825826827828829830831832833834835836837838839840841842843844845846847848849850851852853854855856857858859860861862863864865866867868869870871872873874875876877878879880881882883884885886887888889890891892893894895896897898899900901902903904905906907908909910911912913914915916917918919920921922923924925926927928929930931932933934935936937938939940941942943944945946947948949950951952953954955956957958959960961962963964965966967968969970971972973974975976977978979980981982983984985986987988989990991992993994995996997998999100010011002100310041005100610071008100910101011101210131014101510161017101810191020102110221023102410251026102710281029103010311032103310341035103610371038103910401041104210431044104510461047104810491050105110521053105410551056105710581059106010611062106310641065106610671068106910701071107210731074107510761077107810791080
  1. package sarama
  2. import (
  3. "encoding/binary"
  4. "fmt"
  5. "sync"
  6. "time"
  7. "github.com/eapache/go-resiliency/breaker"
  8. "github.com/eapache/queue"
  9. )
  10. // AsyncProducer publishes Kafka messages using a non-blocking API. It routes messages
  11. // to the correct broker for the provided topic-partition, refreshing metadata as appropriate,
  12. // and parses responses for errors. You must read from the Errors() channel or the
  13. // producer will deadlock. You must call Close() or AsyncClose() on a producer to avoid
  14. // leaks: it will not be garbage-collected automatically when it passes out of
  15. // scope.
  16. type AsyncProducer interface {
  17. // AsyncClose triggers a shutdown of the producer. The shutdown has completed
  18. // when both the Errors and Successes channels have been closed. When calling
  19. // AsyncClose, you *must* continue to read from those channels in order to
  20. // drain the results of any messages in flight.
  21. AsyncClose()
  22. // Close shuts down the producer and waits for any buffered messages to be
  23. // flushed. You must call this function before a producer object passes out of
  24. // scope, as it may otherwise leak memory. You must call this before calling
  25. // Close on the underlying client.
  26. Close() error
  27. // Input is the input channel for the user to write messages to that they
  28. // wish to send.
  29. Input() chan<- *ProducerMessage
  30. // Successes is the success output channel back to the user when Return.Successes is
  31. // enabled. If Return.Successes is true, you MUST read from this channel or the
  32. // Producer will deadlock. It is suggested that you send and read messages
  33. // 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
  36. // channel or the Producer will deadlock when the channel is full. Alternatively,
  37. // you can set Producer.Return.Errors in your config to false, which prevents
  38. // errors to be returned.
  39. Errors() <-chan *ProducerError
  40. }
  41. // transactionManager keeps the state necessary to ensure idempotent production
  42. type transactionManager struct {
  43. producerID int64
  44. producerEpoch int16
  45. sequenceNumbers map[string]int32
  46. mutex sync.Mutex
  47. }
  48. const (
  49. noProducerID = -1
  50. noProducerEpoch = -1
  51. )
  52. func (t *transactionManager) getAndIncrementSequenceNumber(topic string, partition int32) int32 {
  53. key := fmt.Sprintf("%s-%d", topic, partition)
  54. t.mutex.Lock()
  55. defer t.mutex.Unlock()
  56. sequence := t.sequenceNumbers[key]
  57. t.sequenceNumbers[key] = sequence + 1
  58. return sequence
  59. }
  60. func newTransactionManager(conf *Config, client Client) (*transactionManager, error) {
  61. txnmgr := &transactionManager{
  62. producerID: noProducerID,
  63. producerEpoch: noProducerEpoch,
  64. }
  65. if conf.Producer.Idempotent {
  66. initProducerIDResponse, err := client.InitProducerID()
  67. if err != nil {
  68. return nil, err
  69. }
  70. txnmgr.producerID = initProducerIDResponse.ProducerID
  71. txnmgr.producerEpoch = initProducerIDResponse.ProducerEpoch
  72. txnmgr.sequenceNumbers = make(map[string]int32)
  73. txnmgr.mutex = sync.Mutex{}
  74. Logger.Printf("Obtained a ProducerId: %d and ProducerEpoch: %d\n", txnmgr.producerID, txnmgr.producerEpoch)
  75. }
  76. return txnmgr, nil
  77. }
  78. type asyncProducer struct {
  79. client Client
  80. conf *Config
  81. ownClient bool
  82. errors chan *ProducerError
  83. input, successes, retries chan *ProducerMessage
  84. inFlight sync.WaitGroup
  85. brokers map[*Broker]*brokerProducer
  86. brokerRefs map[*brokerProducer]int
  87. brokerLock sync.Mutex
  88. txnmgr *transactionManager
  89. }
  90. // NewAsyncProducer creates a new AsyncProducer using the given broker addresses and configuration.
  91. func NewAsyncProducer(addrs []string, conf *Config) (AsyncProducer, error) {
  92. client, err := NewClient(addrs, conf)
  93. if err != nil {
  94. return nil, err
  95. }
  96. p, err := NewAsyncProducerFromClient(client)
  97. if err != nil {
  98. return nil, err
  99. }
  100. p.(*asyncProducer).ownClient = true
  101. return p, nil
  102. }
  103. // NewAsyncProducerFromClient creates a new Producer using the given client. It is still
  104. // necessary to call Close() on the underlying client when shutting down this producer.
  105. func NewAsyncProducerFromClient(client Client) (AsyncProducer, error) {
  106. // Check that we are not dealing with a closed Client before processing any other arguments
  107. if client.Closed() {
  108. return nil, ErrClosedClient
  109. }
  110. txnmgr, err := newTransactionManager(client.Config(), client)
  111. if err != nil {
  112. return nil, err
  113. }
  114. p := &asyncProducer{
  115. client: client,
  116. conf: client.Config(),
  117. errors: make(chan *ProducerError),
  118. input: make(chan *ProducerMessage),
  119. successes: make(chan *ProducerMessage),
  120. retries: make(chan *ProducerMessage),
  121. brokers: make(map[*Broker]*brokerProducer),
  122. brokerRefs: make(map[*brokerProducer]int),
  123. txnmgr: txnmgr,
  124. }
  125. // launch our singleton dispatchers
  126. go withRecover(p.dispatcher)
  127. go withRecover(p.retryHandler)
  128. return p, nil
  129. }
  130. type flagSet int8
  131. const (
  132. syn flagSet = 1 << iota // first message from partitionProducer to brokerProducer
  133. fin // final message from partitionProducer to brokerProducer and back
  134. shutdown // start the shutdown process
  135. )
  136. // ProducerMessage is the collection of elements passed to the Producer in order to send a message.
  137. type ProducerMessage struct {
  138. Topic string // The Kafka topic for this message.
  139. // The partitioning key for this message. Pre-existing Encoders include
  140. // StringEncoder and ByteEncoder.
  141. Key Encoder
  142. // The actual message to store in Kafka. Pre-existing Encoders include
  143. // StringEncoder and ByteEncoder.
  144. Value Encoder
  145. // The headers are key-value pairs that are transparently passed
  146. // by Kafka between producers and consumers.
  147. Headers []RecordHeader
  148. // This field is used to hold arbitrary data you wish to include so it
  149. // will be available when receiving on the Successes and Errors channels.
  150. // Sarama completely ignores this field and is only to be used for
  151. // pass-through data.
  152. Metadata interface{}
  153. // Below this point are filled in by the producer as the message is processed
  154. // Offset is the offset of the message stored on the broker. This is only
  155. // guaranteed to be defined if the message was successfully delivered and
  156. // RequiredAcks is not NoResponse.
  157. Offset int64
  158. // Partition is the partition that the message was sent to. This is only
  159. // guaranteed to be defined if the message was successfully delivered.
  160. Partition int32
  161. // Timestamp is the timestamp assigned to the message by the broker. This
  162. // is only guaranteed to be defined if the message was successfully
  163. // delivered, RequiredAcks is not NoResponse, and the Kafka broker is at
  164. // least version 0.10.0.
  165. Timestamp time.Time
  166. retries int
  167. flags flagSet
  168. expectation chan *ProducerError
  169. sequenceNumber int32
  170. }
  171. const producerMessageOverhead = 26 // the metadata overhead of CRC, flags, etc.
  172. func (m *ProducerMessage) byteSize(version int) int {
  173. var size int
  174. if version >= 2 {
  175. size = maximumRecordOverhead
  176. for _, h := range m.Headers {
  177. size += len(h.Key) + len(h.Value) + 2*binary.MaxVarintLen32
  178. }
  179. } else {
  180. size = producerMessageOverhead
  181. }
  182. if m.Key != nil {
  183. size += m.Key.Length()
  184. }
  185. if m.Value != nil {
  186. size += m.Value.Length()
  187. }
  188. return size
  189. }
  190. func (m *ProducerMessage) clear() {
  191. m.flags = 0
  192. m.retries = 0
  193. }
  194. // ProducerError is the type of error generated when the producer fails to deliver a message.
  195. // It contains the original ProducerMessage as well as the actual error value.
  196. type ProducerError struct {
  197. Msg *ProducerMessage
  198. Err error
  199. }
  200. func (pe ProducerError) Error() string {
  201. return fmt.Sprintf("kafka: Failed to produce message to topic %s: %s", pe.Msg.Topic, pe.Err)
  202. }
  203. // ProducerErrors is a type that wraps a batch of "ProducerError"s and implements the Error interface.
  204. // It can be returned from the Producer's Close method to avoid the need to manually drain the Errors channel
  205. // when closing a producer.
  206. type ProducerErrors []*ProducerError
  207. func (pe ProducerErrors) Error() string {
  208. return fmt.Sprintf("kafka: Failed to deliver %d messages.", len(pe))
  209. }
  210. func (p *asyncProducer) Errors() <-chan *ProducerError {
  211. return p.errors
  212. }
  213. func (p *asyncProducer) Successes() <-chan *ProducerMessage {
  214. return p.successes
  215. }
  216. func (p *asyncProducer) Input() chan<- *ProducerMessage {
  217. return p.input
  218. }
  219. func (p *asyncProducer) Close() error {
  220. p.AsyncClose()
  221. if p.conf.Producer.Return.Successes {
  222. go withRecover(func() {
  223. for range p.successes {
  224. }
  225. })
  226. }
  227. var errors ProducerErrors
  228. if p.conf.Producer.Return.Errors {
  229. for event := range p.errors {
  230. errors = append(errors, event)
  231. }
  232. } else {
  233. <-p.errors
  234. }
  235. if len(errors) > 0 {
  236. return errors
  237. }
  238. return nil
  239. }
  240. func (p *asyncProducer) AsyncClose() {
  241. go withRecover(p.shutdown)
  242. }
  243. // singleton
  244. // dispatches messages by topic
  245. func (p *asyncProducer) dispatcher() {
  246. handlers := make(map[string]chan<- *ProducerMessage)
  247. shuttingDown := false
  248. for msg := range p.input {
  249. if msg == nil {
  250. Logger.Println("Something tried to send a nil message, it was ignored.")
  251. continue
  252. }
  253. if msg.flags&shutdown != 0 {
  254. shuttingDown = true
  255. p.inFlight.Done()
  256. continue
  257. } else if msg.retries == 0 {
  258. if shuttingDown {
  259. // we can't just call returnError here because that decrements the wait group,
  260. // which hasn't been incremented yet for this message, and shouldn't be
  261. pErr := &ProducerError{Msg: msg, Err: ErrShuttingDown}
  262. if p.conf.Producer.Return.Errors {
  263. p.errors <- pErr
  264. } else {
  265. Logger.Println(pErr)
  266. }
  267. continue
  268. }
  269. p.inFlight.Add(1)
  270. }
  271. version := 1
  272. if p.conf.Version.IsAtLeast(V0_11_0_0) {
  273. version = 2
  274. } else if msg.Headers != nil {
  275. p.returnError(msg, ConfigurationError("Producing headers requires Kafka at least v0.11"))
  276. continue
  277. }
  278. if msg.byteSize(version) > p.conf.Producer.MaxMessageBytes {
  279. p.returnError(msg, ErrMessageSizeTooLarge)
  280. continue
  281. }
  282. handler := handlers[msg.Topic]
  283. if handler == nil {
  284. handler = p.newTopicProducer(msg.Topic)
  285. handlers[msg.Topic] = handler
  286. }
  287. handler <- msg
  288. }
  289. for _, handler := range handlers {
  290. close(handler)
  291. }
  292. }
  293. // one per topic
  294. // partitions messages, then dispatches them by partition
  295. type topicProducer struct {
  296. parent *asyncProducer
  297. topic string
  298. input <-chan *ProducerMessage
  299. breaker *breaker.Breaker
  300. handlers map[int32]chan<- *ProducerMessage
  301. partitioner Partitioner
  302. }
  303. func (p *asyncProducer) newTopicProducer(topic string) chan<- *ProducerMessage {
  304. input := make(chan *ProducerMessage, p.conf.ChannelBufferSize)
  305. tp := &topicProducer{
  306. parent: p,
  307. topic: topic,
  308. input: input,
  309. breaker: breaker.New(3, 1, 10*time.Second),
  310. handlers: make(map[int32]chan<- *ProducerMessage),
  311. partitioner: p.conf.Producer.Partitioner(topic),
  312. }
  313. go withRecover(tp.dispatch)
  314. return input
  315. }
  316. func (tp *topicProducer) dispatch() {
  317. for msg := range tp.input {
  318. if msg.retries == 0 {
  319. if err := tp.partitionMessage(msg); err != nil {
  320. tp.parent.returnError(msg, err)
  321. continue
  322. }
  323. }
  324. // All messages being retried (sent or not) have already had their retry count updated
  325. if tp.parent.conf.Producer.Idempotent && msg.retries == 0 {
  326. msg.sequenceNumber = tp.parent.txnmgr.getAndIncrementSequenceNumber(msg.Topic, msg.Partition)
  327. }
  328. handler := tp.handlers[msg.Partition]
  329. if handler == nil {
  330. handler = tp.parent.newPartitionProducer(msg.Topic, msg.Partition)
  331. tp.handlers[msg.Partition] = handler
  332. }
  333. handler <- msg
  334. }
  335. for _, handler := range tp.handlers {
  336. close(handler)
  337. }
  338. }
  339. func (tp *topicProducer) partitionMessage(msg *ProducerMessage) error {
  340. var partitions []int32
  341. err := tp.breaker.Run(func() (err error) {
  342. var requiresConsistency = false
  343. if ep, ok := tp.partitioner.(DynamicConsistencyPartitioner); ok {
  344. requiresConsistency = ep.MessageRequiresConsistency(msg)
  345. } else {
  346. requiresConsistency = tp.partitioner.RequiresConsistency()
  347. }
  348. if requiresConsistency {
  349. partitions, err = tp.parent.client.Partitions(msg.Topic)
  350. } else {
  351. partitions, err = tp.parent.client.WritablePartitions(msg.Topic)
  352. }
  353. return
  354. })
  355. if err != nil {
  356. return err
  357. }
  358. numPartitions := int32(len(partitions))
  359. if numPartitions == 0 {
  360. return ErrLeaderNotAvailable
  361. }
  362. choice, err := tp.partitioner.Partition(msg, numPartitions)
  363. if err != nil {
  364. return err
  365. } else if choice < 0 || choice >= numPartitions {
  366. return ErrInvalidPartition
  367. }
  368. msg.Partition = partitions[choice]
  369. return nil
  370. }
  371. // one per partition per topic
  372. // dispatches messages to the appropriate broker
  373. // also responsible for maintaining message order during retries
  374. type partitionProducer struct {
  375. parent *asyncProducer
  376. topic string
  377. partition int32
  378. input <-chan *ProducerMessage
  379. leader *Broker
  380. breaker *breaker.Breaker
  381. brokerProducer *brokerProducer
  382. // highWatermark tracks the "current" retry level, which is the only one where we actually let messages through,
  383. // all other messages get buffered in retryState[msg.retries].buf to preserve ordering
  384. // retryState[msg.retries].expectChaser simply tracks whether we've seen a fin message for a given level (and
  385. // therefore whether our buffer is complete and safe to flush)
  386. highWatermark int
  387. retryState []partitionRetryState
  388. }
  389. type partitionRetryState struct {
  390. buf []*ProducerMessage
  391. expectChaser bool
  392. }
  393. func (p *asyncProducer) newPartitionProducer(topic string, partition int32) chan<- *ProducerMessage {
  394. input := make(chan *ProducerMessage, p.conf.ChannelBufferSize)
  395. pp := &partitionProducer{
  396. parent: p,
  397. topic: topic,
  398. partition: partition,
  399. input: input,
  400. breaker: breaker.New(3, 1, 10*time.Second),
  401. retryState: make([]partitionRetryState, p.conf.Producer.Retry.Max+1),
  402. }
  403. go withRecover(pp.dispatch)
  404. return input
  405. }
  406. func (pp *partitionProducer) dispatch() {
  407. // try to prefetch the leader; if this doesn't work, we'll do a proper call to `updateLeader`
  408. // on the first message
  409. pp.leader, _ = pp.parent.client.Leader(pp.topic, pp.partition)
  410. if pp.leader != nil {
  411. pp.brokerProducer = pp.parent.getBrokerProducer(pp.leader)
  412. pp.parent.inFlight.Add(1) // we're generating a syn message; track it so we don't shut down while it's still inflight
  413. pp.brokerProducer.input <- &ProducerMessage{Topic: pp.topic, Partition: pp.partition, flags: syn}
  414. }
  415. defer func() {
  416. if pp.brokerProducer != nil {
  417. pp.parent.unrefBrokerProducer(pp.leader, pp.brokerProducer)
  418. }
  419. }()
  420. for msg := range pp.input {
  421. if pp.brokerProducer != nil && pp.brokerProducer.abandoned != nil {
  422. select {
  423. case <-pp.brokerProducer.abandoned:
  424. // a message on the abandoned channel means that our current broker selection is out of date
  425. Logger.Printf("producer/leader/%s/%d abandoning broker %d\n", pp.topic, pp.partition, pp.leader.ID())
  426. pp.parent.unrefBrokerProducer(pp.leader, pp.brokerProducer)
  427. pp.brokerProducer = nil
  428. time.Sleep(pp.parent.conf.Producer.Retry.Backoff)
  429. default:
  430. // producer connection is still open.
  431. }
  432. }
  433. if msg.retries > pp.highWatermark {
  434. // a new, higher, retry level; handle it and then back off
  435. pp.newHighWatermark(msg.retries)
  436. time.Sleep(pp.parent.conf.Producer.Retry.Backoff)
  437. } else if pp.highWatermark > 0 {
  438. // we are retrying something (else highWatermark would be 0) but this message is not a *new* retry level
  439. if msg.retries < pp.highWatermark {
  440. // in fact this message is not even the current retry level, so buffer it for now (unless it's a just a fin)
  441. if msg.flags&fin == fin {
  442. pp.retryState[msg.retries].expectChaser = false
  443. pp.parent.inFlight.Done() // this fin is now handled and will be garbage collected
  444. } else {
  445. pp.retryState[msg.retries].buf = append(pp.retryState[msg.retries].buf, msg)
  446. }
  447. continue
  448. } else if msg.flags&fin == fin {
  449. // this message is of the current retry level (msg.retries == highWatermark) and the fin flag is set,
  450. // meaning this retry level is done and we can go down (at least) one level and flush that
  451. pp.retryState[pp.highWatermark].expectChaser = false
  452. pp.flushRetryBuffers()
  453. pp.parent.inFlight.Done() // this fin is now handled and will be garbage collected
  454. continue
  455. }
  456. }
  457. // if we made it this far then the current msg contains real data, and can be sent to the next goroutine
  458. // without breaking any of our ordering guarantees
  459. if pp.brokerProducer == nil {
  460. if err := pp.updateLeader(); err != nil {
  461. pp.parent.returnError(msg, err)
  462. time.Sleep(pp.parent.conf.Producer.Retry.Backoff)
  463. continue
  464. }
  465. Logger.Printf("producer/leader/%s/%d selected broker %d\n", pp.topic, pp.partition, pp.leader.ID())
  466. }
  467. pp.brokerProducer.input <- msg
  468. }
  469. }
  470. func (pp *partitionProducer) newHighWatermark(hwm int) {
  471. Logger.Printf("producer/leader/%s/%d state change to [retrying-%d]\n", pp.topic, pp.partition, hwm)
  472. pp.highWatermark = hwm
  473. // send off a fin so that we know when everything "in between" has made it
  474. // back to us and we can safely flush the backlog (otherwise we risk re-ordering messages)
  475. pp.retryState[pp.highWatermark].expectChaser = true
  476. pp.parent.inFlight.Add(1) // we're generating a fin message; track it so we don't shut down while it's still inflight
  477. pp.brokerProducer.input <- &ProducerMessage{Topic: pp.topic, Partition: pp.partition, flags: fin, retries: pp.highWatermark - 1}
  478. // a new HWM means that our current broker selection is out of date
  479. Logger.Printf("producer/leader/%s/%d abandoning broker %d\n", pp.topic, pp.partition, pp.leader.ID())
  480. pp.parent.unrefBrokerProducer(pp.leader, pp.brokerProducer)
  481. pp.brokerProducer = nil
  482. }
  483. func (pp *partitionProducer) flushRetryBuffers() {
  484. Logger.Printf("producer/leader/%s/%d state change to [flushing-%d]\n", pp.topic, pp.partition, pp.highWatermark)
  485. for {
  486. pp.highWatermark--
  487. if pp.brokerProducer == nil {
  488. if err := pp.updateLeader(); err != nil {
  489. pp.parent.returnErrors(pp.retryState[pp.highWatermark].buf, err)
  490. goto flushDone
  491. }
  492. Logger.Printf("producer/leader/%s/%d selected broker %d\n", pp.topic, pp.partition, pp.leader.ID())
  493. }
  494. for _, msg := range pp.retryState[pp.highWatermark].buf {
  495. pp.brokerProducer.input <- msg
  496. }
  497. flushDone:
  498. pp.retryState[pp.highWatermark].buf = nil
  499. if pp.retryState[pp.highWatermark].expectChaser {
  500. Logger.Printf("producer/leader/%s/%d state change to [retrying-%d]\n", pp.topic, pp.partition, pp.highWatermark)
  501. break
  502. } else if pp.highWatermark == 0 {
  503. Logger.Printf("producer/leader/%s/%d state change to [normal]\n", pp.topic, pp.partition)
  504. break
  505. }
  506. }
  507. }
  508. func (pp *partitionProducer) updateLeader() error {
  509. return pp.breaker.Run(func() (err error) {
  510. if err = pp.parent.client.RefreshMetadata(pp.topic); err != nil {
  511. return err
  512. }
  513. if pp.leader, err = pp.parent.client.Leader(pp.topic, pp.partition); err != nil {
  514. return err
  515. }
  516. pp.brokerProducer = pp.parent.getBrokerProducer(pp.leader)
  517. pp.parent.inFlight.Add(1) // we're generating a syn message; track it so we don't shut down while it's still inflight
  518. pp.brokerProducer.input <- &ProducerMessage{Topic: pp.topic, Partition: pp.partition, flags: syn}
  519. return nil
  520. })
  521. }
  522. // one per broker; also constructs an associated flusher
  523. func (p *asyncProducer) newBrokerProducer(broker *Broker) *brokerProducer {
  524. var (
  525. input = make(chan *ProducerMessage)
  526. bridge = make(chan *produceSet)
  527. responses = make(chan *brokerProducerResponse)
  528. )
  529. bp := &brokerProducer{
  530. parent: p,
  531. broker: broker,
  532. input: input,
  533. output: bridge,
  534. responses: responses,
  535. buffer: newProduceSet(p),
  536. currentRetries: make(map[string]map[int32]error),
  537. }
  538. go withRecover(bp.run)
  539. // minimal bridge to make the network response `select`able
  540. go withRecover(func() {
  541. for set := range bridge {
  542. request := set.buildRequest()
  543. response, err := broker.Produce(request)
  544. responses <- &brokerProducerResponse{
  545. set: set,
  546. err: err,
  547. res: response,
  548. }
  549. }
  550. close(responses)
  551. })
  552. if p.conf.Producer.Retry.Max <= 0 {
  553. bp.abandoned = make(chan struct{})
  554. }
  555. return bp
  556. }
  557. type brokerProducerResponse struct {
  558. set *produceSet
  559. err error
  560. res *ProduceResponse
  561. }
  562. // groups messages together into appropriately-sized batches for sending to the broker
  563. // handles state related to retries etc
  564. type brokerProducer struct {
  565. parent *asyncProducer
  566. broker *Broker
  567. input chan *ProducerMessage
  568. output chan<- *produceSet
  569. responses <-chan *brokerProducerResponse
  570. abandoned chan struct{}
  571. buffer *produceSet
  572. timer <-chan time.Time
  573. timerFired bool
  574. closing error
  575. currentRetries map[string]map[int32]error
  576. }
  577. func (bp *brokerProducer) run() {
  578. var output chan<- *produceSet
  579. Logger.Printf("producer/broker/%d starting up\n", bp.broker.ID())
  580. for {
  581. select {
  582. case msg := <-bp.input:
  583. if msg == nil {
  584. bp.shutdown()
  585. return
  586. }
  587. if msg.flags&syn == syn {
  588. Logger.Printf("producer/broker/%d state change to [open] on %s/%d\n",
  589. bp.broker.ID(), msg.Topic, msg.Partition)
  590. if bp.currentRetries[msg.Topic] == nil {
  591. bp.currentRetries[msg.Topic] = make(map[int32]error)
  592. }
  593. bp.currentRetries[msg.Topic][msg.Partition] = nil
  594. bp.parent.inFlight.Done()
  595. continue
  596. }
  597. if reason := bp.needsRetry(msg); reason != nil {
  598. bp.parent.retryMessage(msg, reason)
  599. if bp.closing == nil && msg.flags&fin == fin {
  600. // we were retrying this partition but we can start processing again
  601. delete(bp.currentRetries[msg.Topic], msg.Partition)
  602. Logger.Printf("producer/broker/%d state change to [closed] on %s/%d\n",
  603. bp.broker.ID(), msg.Topic, msg.Partition)
  604. }
  605. continue
  606. }
  607. if bp.buffer.wouldOverflow(msg) {
  608. if err := bp.waitForSpace(msg); err != nil {
  609. bp.parent.retryMessage(msg, err)
  610. continue
  611. }
  612. }
  613. if err := bp.buffer.add(msg); err != nil {
  614. bp.parent.returnError(msg, err)
  615. continue
  616. }
  617. if bp.parent.conf.Producer.Flush.Frequency > 0 && bp.timer == nil {
  618. bp.timer = time.After(bp.parent.conf.Producer.Flush.Frequency)
  619. }
  620. case <-bp.timer:
  621. bp.timerFired = true
  622. case output <- bp.buffer:
  623. bp.rollOver()
  624. case response := <-bp.responses:
  625. bp.handleResponse(response)
  626. }
  627. if bp.timerFired || bp.buffer.readyToFlush() {
  628. output = bp.output
  629. } else {
  630. output = nil
  631. }
  632. }
  633. }
  634. func (bp *brokerProducer) shutdown() {
  635. for !bp.buffer.empty() {
  636. select {
  637. case response := <-bp.responses:
  638. bp.handleResponse(response)
  639. case bp.output <- bp.buffer:
  640. bp.rollOver()
  641. }
  642. }
  643. close(bp.output)
  644. for response := range bp.responses {
  645. bp.handleResponse(response)
  646. }
  647. Logger.Printf("producer/broker/%d shut down\n", bp.broker.ID())
  648. }
  649. func (bp *brokerProducer) needsRetry(msg *ProducerMessage) error {
  650. if bp.closing != nil {
  651. return bp.closing
  652. }
  653. return bp.currentRetries[msg.Topic][msg.Partition]
  654. }
  655. func (bp *brokerProducer) waitForSpace(msg *ProducerMessage) error {
  656. Logger.Printf("producer/broker/%d maximum request accumulated, waiting for space\n", bp.broker.ID())
  657. for {
  658. select {
  659. case response := <-bp.responses:
  660. bp.handleResponse(response)
  661. // handling a response can change our state, so re-check some things
  662. if reason := bp.needsRetry(msg); reason != nil {
  663. return reason
  664. } else if !bp.buffer.wouldOverflow(msg) {
  665. return nil
  666. }
  667. case bp.output <- bp.buffer:
  668. bp.rollOver()
  669. return nil
  670. }
  671. }
  672. }
  673. func (bp *brokerProducer) rollOver() {
  674. bp.timer = nil
  675. bp.timerFired = false
  676. bp.buffer = newProduceSet(bp.parent)
  677. }
  678. func (bp *brokerProducer) handleResponse(response *brokerProducerResponse) {
  679. if response.err != nil {
  680. bp.handleError(response.set, response.err)
  681. } else {
  682. bp.handleSuccess(response.set, response.res)
  683. }
  684. if bp.buffer.empty() {
  685. bp.rollOver() // this can happen if the response invalidated our buffer
  686. }
  687. }
  688. func (bp *brokerProducer) handleSuccess(sent *produceSet, response *ProduceResponse) {
  689. // we iterate through the blocks in the request set, not the response, so that we notice
  690. // if the response is missing a block completely
  691. var retryTopics []string
  692. sent.eachPartition(func(topic string, partition int32, pSet *partitionSet) {
  693. if response == nil {
  694. // this only happens when RequiredAcks is NoResponse, so we have to assume success
  695. bp.parent.returnSuccesses(pSet.msgs)
  696. return
  697. }
  698. block := response.GetBlock(topic, partition)
  699. if block == nil {
  700. bp.parent.returnErrors(pSet.msgs, ErrIncompleteResponse)
  701. return
  702. }
  703. switch block.Err {
  704. // Success
  705. case ErrNoError:
  706. if bp.parent.conf.Version.IsAtLeast(V0_10_0_0) && !block.Timestamp.IsZero() {
  707. for _, msg := range pSet.msgs {
  708. msg.Timestamp = block.Timestamp
  709. }
  710. }
  711. for i, msg := range pSet.msgs {
  712. msg.Offset = block.Offset + int64(i)
  713. }
  714. bp.parent.returnSuccesses(pSet.msgs)
  715. // Duplicate
  716. case ErrDuplicateSequenceNumber:
  717. bp.parent.returnSuccesses(pSet.msgs)
  718. // Retriable errors
  719. case ErrInvalidMessage, ErrUnknownTopicOrPartition, ErrLeaderNotAvailable, ErrNotLeaderForPartition,
  720. ErrRequestTimedOut, ErrNotEnoughReplicas, ErrNotEnoughReplicasAfterAppend:
  721. if bp.parent.conf.Producer.Retry.Max <= 0 {
  722. bp.parent.abandonBrokerConnection(bp.broker)
  723. bp.parent.returnErrors(pSet.msgs, block.Err)
  724. } else {
  725. retryTopics = append(retryTopics, topic)
  726. }
  727. // Other non-retriable errors
  728. default:
  729. if bp.parent.conf.Producer.Retry.Max <= 0 {
  730. bp.parent.abandonBrokerConnection(bp.broker)
  731. }
  732. bp.parent.returnErrors(pSet.msgs, block.Err)
  733. }
  734. })
  735. if len(retryTopics) > 0 {
  736. err := bp.parent.client.RefreshMetadata(retryTopics...)
  737. if err != nil {
  738. Logger.Printf("Failed refreshing metadata because of %v\n", err)
  739. }
  740. sent.eachPartition(func(topic string, partition int32, pSet *partitionSet) {
  741. block := response.GetBlock(topic, partition)
  742. if block == nil {
  743. // handled in the previous "eachPartition" loop
  744. return
  745. }
  746. switch block.Err {
  747. case ErrInvalidMessage, ErrUnknownTopicOrPartition, ErrLeaderNotAvailable, ErrNotLeaderForPartition,
  748. ErrRequestTimedOut, ErrNotEnoughReplicas, ErrNotEnoughReplicasAfterAppend:
  749. Logger.Printf("producer/broker/%d state change to [retrying] on %s/%d because %v\n",
  750. bp.broker.ID(), topic, partition, block.Err)
  751. if bp.currentRetries[topic] == nil {
  752. bp.currentRetries[topic] = make(map[int32]error)
  753. }
  754. bp.currentRetries[topic][partition] = block.Err
  755. // dropping the following messages has the side effect of incrementing their retry count
  756. bp.parent.retryMessages(bp.buffer.dropPartition(topic, partition), block.Err)
  757. bp.parent.retryBatch(topic, partition, pSet, block.Err)
  758. }
  759. })
  760. }
  761. }
  762. func (p *asyncProducer) retryBatch(topic string, partition int32, pSet *partitionSet, kerr KError) {
  763. Logger.Printf("Retrying batch for %v-%d because of %s\n", topic, partition, kerr)
  764. produceSet := newProduceSet(p)
  765. produceSet.msgs[topic] = make(map[int32]*partitionSet)
  766. produceSet.msgs[topic][partition] = pSet
  767. produceSet.bufferBytes += pSet.bufferBytes
  768. produceSet.bufferCount += len(pSet.msgs)
  769. for _, msg := range pSet.msgs {
  770. if msg.retries >= p.conf.Producer.Retry.Max {
  771. p.returnError(msg, kerr)
  772. return
  773. }
  774. msg.retries++
  775. }
  776. // it's expected that a metadata refresh has been requested prior to calling retryBatch
  777. leader, err := p.client.Leader(topic, partition)
  778. if err != nil {
  779. Logger.Printf("Failed retrying batch for %v-%d because of %v while looking up for new leader\n", topic, partition, err)
  780. for _, msg := range pSet.msgs {
  781. p.returnError(msg, kerr)
  782. }
  783. return
  784. }
  785. bp := p.getBrokerProducer(leader)
  786. bp.output <- produceSet
  787. }
  788. func (bp *brokerProducer) handleError(sent *produceSet, err error) {
  789. switch err.(type) {
  790. case PacketEncodingError:
  791. sent.eachPartition(func(topic string, partition int32, pSet *partitionSet) {
  792. bp.parent.returnErrors(pSet.msgs, err)
  793. })
  794. default:
  795. Logger.Printf("producer/broker/%d state change to [closing] because %s\n", bp.broker.ID(), err)
  796. bp.parent.abandonBrokerConnection(bp.broker)
  797. _ = bp.broker.Close()
  798. bp.closing = err
  799. sent.eachPartition(func(topic string, partition int32, pSet *partitionSet) {
  800. bp.parent.retryMessages(pSet.msgs, err)
  801. })
  802. bp.buffer.eachPartition(func(topic string, partition int32, pSet *partitionSet) {
  803. bp.parent.retryMessages(pSet.msgs, err)
  804. })
  805. bp.rollOver()
  806. }
  807. }
  808. // singleton
  809. // effectively a "bridge" between the flushers and the dispatcher in order to avoid deadlock
  810. // based on https://godoc.org/github.com/eapache/channels#InfiniteChannel
  811. func (p *asyncProducer) retryHandler() {
  812. var msg *ProducerMessage
  813. buf := queue.New()
  814. for {
  815. if buf.Length() == 0 {
  816. msg = <-p.retries
  817. } else {
  818. select {
  819. case msg = <-p.retries:
  820. case p.input <- buf.Peek().(*ProducerMessage):
  821. buf.Remove()
  822. continue
  823. }
  824. }
  825. if msg == nil {
  826. return
  827. }
  828. buf.Add(msg)
  829. }
  830. }
  831. // utility functions
  832. func (p *asyncProducer) shutdown() {
  833. Logger.Println("Producer shutting down.")
  834. p.inFlight.Add(1)
  835. p.input <- &ProducerMessage{flags: shutdown}
  836. p.inFlight.Wait()
  837. if p.ownClient {
  838. err := p.client.Close()
  839. if err != nil {
  840. Logger.Println("producer/shutdown failed to close the embedded client:", err)
  841. }
  842. }
  843. close(p.input)
  844. close(p.retries)
  845. close(p.errors)
  846. close(p.successes)
  847. }
  848. func (p *asyncProducer) returnError(msg *ProducerMessage, err error) {
  849. msg.clear()
  850. pErr := &ProducerError{Msg: msg, Err: err}
  851. if p.conf.Producer.Return.Errors {
  852. p.errors <- pErr
  853. } else {
  854. Logger.Println(pErr)
  855. }
  856. p.inFlight.Done()
  857. }
  858. func (p *asyncProducer) returnErrors(batch []*ProducerMessage, err error) {
  859. for _, msg := range batch {
  860. p.returnError(msg, err)
  861. }
  862. }
  863. func (p *asyncProducer) returnSuccesses(batch []*ProducerMessage) {
  864. for _, msg := range batch {
  865. if p.conf.Producer.Return.Successes {
  866. msg.clear()
  867. p.successes <- msg
  868. }
  869. p.inFlight.Done()
  870. }
  871. }
  872. func (p *asyncProducer) retryMessage(msg *ProducerMessage, err error) {
  873. if msg.retries >= p.conf.Producer.Retry.Max {
  874. p.returnError(msg, err)
  875. } else {
  876. msg.retries++
  877. p.retries <- msg
  878. }
  879. }
  880. func (p *asyncProducer) retryMessages(batch []*ProducerMessage, err error) {
  881. for _, msg := range batch {
  882. p.retryMessage(msg, err)
  883. }
  884. }
  885. func (p *asyncProducer) getBrokerProducer(broker *Broker) *brokerProducer {
  886. p.brokerLock.Lock()
  887. defer p.brokerLock.Unlock()
  888. bp := p.brokers[broker]
  889. if bp == nil {
  890. bp = p.newBrokerProducer(broker)
  891. p.brokers[broker] = bp
  892. p.brokerRefs[bp] = 0
  893. }
  894. p.brokerRefs[bp]++
  895. return bp
  896. }
  897. func (p *asyncProducer) unrefBrokerProducer(broker *Broker, bp *brokerProducer) {
  898. p.brokerLock.Lock()
  899. defer p.brokerLock.Unlock()
  900. p.brokerRefs[bp]--
  901. if p.brokerRefs[bp] == 0 {
  902. close(bp.input)
  903. delete(p.brokerRefs, bp)
  904. if p.brokers[broker] == bp {
  905. delete(p.brokers, broker)
  906. }
  907. }
  908. }
  909. func (p *asyncProducer) abandonBrokerConnection(broker *Broker) {
  910. p.brokerLock.Lock()
  911. defer p.brokerLock.Unlock()
  912. bc, ok := p.brokers[broker]
  913. if ok && bc.abandoned != nil {
  914. close(bc.abandoned)
  915. }
  916. delete(p.brokers, broker)
  917. }