| 123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676677678679680681682683684685686687688689690691692693694695696697698699700701702703704705706707708709710711712713714715716717718719720721722723724725726727728729730731732733734735736737738739740741742743744745746747748749750751752753754755756757758759760761762763764765766767768769770771772773774775776777778779780781782783784785786787788789790791792793794795796797798799800801802803804805 |
- package sarama
- import (
- "fmt"
- "sync"
- "time"
- "github.com/eapache/go-resiliency/breaker"
- )
- func forceFlushThreshold() int {
- return int(MaxRequestSize - (10 * 1024)) // 10KiB is safety room for misc. overhead, we might want to calculate this more precisely?
- }
- // AsyncProducer publishes Kafka messages using a non-blocking API. It routes messages
- // to the correct broker for the provided topic-partition, refreshing metadata as appropriate,
- // and parses responses for errors. You must read from the Errors() channel or the
- // producer will deadlock. You must call Close() or AsyncClose() on a producer to avoid
- // leaks: it will not be garbage-collected automatically when it passes out of
- // scope (this is in addition to calling Close on the underlying client, which
- // is still necessary).
- type AsyncProducer interface {
- // AsyncClose triggers a shutdown of the producer, flushing any messages it may have
- // buffered. The shutdown has completed when both the Errors and Successes channels
- // have been closed. When calling AsyncClose, you *must* continue to read from those
- // channels in order to drain the results of any messages in flight.
- AsyncClose()
- // Close shuts down the producer and flushes any messages it may have buffered.
- // You must call this function before a producer object passes out of scope, as
- // it may otherwise leak memory. You must call this before calling Close on the
- // underlying client.
- Close() error
- // Input is the input channel for the user to write messages to that they wish to send.
- Input() chan<- *ProducerMessage
- // Successes is the success output channel back to the user when AckSuccesses is confured.
- // If Return.Successes is true, you MUST read from this channel or the Producer will deadlock.
- // It is suggested that you send and read messages together in a single select statement.
- Successes() <-chan *ProducerMessage
- // Errors is the error output channel back to the user. You MUST read from this channel
- // or the Producer will deadlock when the channel is full. Alternatively, you can set
- // Producer.Return.Errors in your config to false, which prevents errors to be returned.
- Errors() <-chan *ProducerError
- }
- type asyncProducer struct {
- client Client
- conf *Config
- ownClient bool
- errors chan *ProducerError
- input, successes, retries chan *ProducerMessage
- brokers map[*Broker]*brokerProducer
- brokerLock sync.Mutex
- }
- // NewAsyncProducer creates a new AsyncProducer using the given broker addresses and configuration.
- func NewAsyncProducer(addrs []string, conf *Config) (AsyncProducer, error) {
- client, err := NewClient(addrs, conf)
- if err != nil {
- return nil, err
- }
- p, err := NewAsyncProducerFromClient(client)
- if err != nil {
- return nil, err
- }
- p.(*asyncProducer).ownClient = true
- return p, nil
- }
- // NewAsyncProducerFromClient creates a new Producer using the given client.
- func NewAsyncProducerFromClient(client Client) (AsyncProducer, error) {
- // Check that we are not dealing with a closed Client before processing any other arguments
- if client.Closed() {
- return nil, ErrClosedClient
- }
- p := &asyncProducer{
- client: client,
- conf: client.Config(),
- errors: make(chan *ProducerError),
- input: make(chan *ProducerMessage),
- successes: make(chan *ProducerMessage),
- retries: make(chan *ProducerMessage),
- brokers: make(map[*Broker]*brokerProducer),
- }
- // launch our singleton dispatchers
- go withRecover(p.topicDispatcher)
- go withRecover(p.retryHandler)
- return p, nil
- }
- type flagSet int8
- const (
- chaser flagSet = 1 << iota // message is last in a group that failed
- ref // add a reference to a singleton channel
- unref // remove a reference from a singleton channel
- shutdown // start the shutdown process
- )
- // ProducerMessage is the collection of elements passed to the Producer in order to send a message.
- type ProducerMessage struct {
- Topic string // The Kafka topic for this message.
- Key Encoder // The partitioning key for this message. It must implement the Encoder interface. Pre-existing Encoders include StringEncoder and ByteEncoder.
- Value Encoder // The actual message to store in Kafka. It must implement the Encoder interface. Pre-existing Encoders include StringEncoder and ByteEncoder.
- // These are filled in by the producer as the message is processed
- 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.
- 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.
- 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.
- retries int
- flags flagSet
- }
- func (m *ProducerMessage) byteSize() int {
- size := 26 // the metadata overhead of CRC, flags, etc.
- if m.Key != nil {
- size += m.Key.Length()
- }
- if m.Value != nil {
- size += m.Value.Length()
- }
- return size
- }
- // ProducerError is the type of error generated when the producer fails to deliver a message.
- // It contains the original ProducerMessage as well as the actual error value.
- type ProducerError struct {
- Msg *ProducerMessage
- Err error
- }
- func (pe ProducerError) Error() string {
- return fmt.Sprintf("kafka: Failed to produce message to topic %s: %s", pe.Msg.Topic, pe.Err)
- }
- // ProducerErrors is a type that wraps a batch of "ProducerError"s and implements the Error interface.
- // It can be returned from the Producer's Close method to avoid the need to manually drain the Errors channel
- // when closing a producer.
- type ProducerErrors []*ProducerError
- func (pe ProducerErrors) Error() string {
- return fmt.Sprintf("kafka: Failed to deliver %d messages.", len(pe))
- }
- func (p *asyncProducer) Errors() <-chan *ProducerError {
- return p.errors
- }
- func (p *asyncProducer) Successes() <-chan *ProducerMessage {
- return p.successes
- }
- func (p *asyncProducer) Input() chan<- *ProducerMessage {
- return p.input
- }
- func (p *asyncProducer) Close() error {
- p.AsyncClose()
- if p.conf.Producer.Return.Successes {
- go withRecover(func() {
- for _ = range p.successes {
- }
- })
- }
- var errors ProducerErrors
- if p.conf.Producer.Return.Errors {
- for event := range p.errors {
- errors = append(errors, event)
- }
- }
- if len(errors) > 0 {
- return errors
- }
- return nil
- }
- func (p *asyncProducer) AsyncClose() {
- go withRecover(func() {
- p.input <- &ProducerMessage{flags: shutdown}
- })
- }
- ///////////////////////////////////////////
- // In normal processing, a message flows through the following functions from top to bottom,
- // starting at topicDispatcher (which reads from Producer.input) and ending in flusher
- // (which sends the message to the broker). In cases where a message must be retried, it goes
- // through retryHandler before being returned to the top of the flow.
- ///////////////////////////////////////////
- // singleton
- // dispatches messages by topic
- func (p *asyncProducer) topicDispatcher() {
- handlers := make(map[string]chan *ProducerMessage)
- for msg := range p.input {
- if msg == nil {
- Logger.Println("Something tried to send a nil message, it was ignored.")
- continue
- }
- if msg.flags&shutdown != 0 {
- Logger.Println("Producer shutting down.")
- break
- }
- if (p.conf.Producer.Compression == CompressionNone && msg.Value != nil && msg.Value.Length() > p.conf.Producer.MaxMessageBytes) ||
- (msg.byteSize() > p.conf.Producer.MaxMessageBytes) {
- p.returnError(msg, ErrMessageSizeTooLarge)
- continue
- }
- handler := handlers[msg.Topic]
- if handler == nil {
- p.retries <- &ProducerMessage{flags: ref}
- newHandler := make(chan *ProducerMessage, p.conf.ChannelBufferSize)
- topic := msg.Topic // block local because go's closure semantics suck
- go withRecover(func() { p.partitionDispatcher(topic, newHandler) })
- handler = newHandler
- handlers[msg.Topic] = handler
- }
- handler <- msg
- }
- for _, handler := range handlers {
- close(handler)
- }
- p.retries <- &ProducerMessage{flags: shutdown}
- for msg := range p.input {
- p.returnError(msg, ErrShuttingDown)
- }
- if p.ownClient {
- err := p.client.Close()
- if err != nil {
- Logger.Println("producer/shutdown failed to close the embedded client:", err)
- }
- }
- close(p.errors)
- close(p.successes)
- }
- // one per topic
- // partitions messages, then dispatches them by partition
- func (p *asyncProducer) partitionDispatcher(topic string, input chan *ProducerMessage) {
- handlers := make(map[int32]chan *ProducerMessage)
- partitioner := p.conf.Producer.Partitioner()
- for msg := range input {
- if msg.retries == 0 {
- err := p.assignPartition(partitioner, msg)
- if err != nil {
- p.returnError(msg, err)
- continue
- }
- }
- handler := handlers[msg.Partition]
- if handler == nil {
- p.retries <- &ProducerMessage{flags: ref}
- newHandler := make(chan *ProducerMessage, p.conf.ChannelBufferSize)
- topic := msg.Topic // block local because go's closure semantics suck
- partition := msg.Partition // block local because go's closure semantics suck
- go withRecover(func() { p.leaderDispatcher(topic, partition, newHandler) })
- handler = newHandler
- handlers[msg.Partition] = handler
- }
- handler <- msg
- }
- for _, handler := range handlers {
- close(handler)
- }
- p.retries <- &ProducerMessage{flags: unref}
- }
- // one per partition per topic
- // dispatches messages to the appropriate broker
- // also responsible for maintaining message order during retries
- func (p *asyncProducer) leaderDispatcher(topic string, partition int32, input chan *ProducerMessage) {
- var leader *Broker
- var output chan *ProducerMessage
- breaker := breaker.New(3, 1, 10*time.Second)
- doUpdate := func() (err error) {
- if err = p.client.RefreshMetadata(topic); err != nil {
- return err
- }
- if leader, err = p.client.Leader(topic, partition); err != nil {
- return err
- }
- output = p.getBrokerProducer(leader)
- return nil
- }
- // try to prefetch the leader; if this doesn't work, we'll do a proper breaker-protected refresh-and-fetch
- // on the first message
- leader, _ = p.client.Leader(topic, partition)
- if leader != nil {
- output = p.getBrokerProducer(leader)
- }
- // highWatermark tracks the "current" retry level, which is the only one where we actually let messages through,
- // all other messages get buffered in retryState[msg.retries].buf to preserve ordering
- // retryState[msg.retries].expectChaser simply tracks whether we've seen a chaser message for a given level (and
- // therefore whether our buffer is complete and safe to flush)
- highWatermark := 0
- retryState := make([]struct {
- buf []*ProducerMessage
- expectChaser bool
- }, p.conf.Producer.Retry.Max+1)
- for msg := range input {
- if msg.retries > highWatermark {
- // new, higher, retry level; send off a chaser so that we know when everything "in between" has made it
- // back to us and we can safely flush the backlog (otherwise we risk re-ordering messages)
- highWatermark = msg.retries
- Logger.Printf("producer/leader state change to [retrying-%d] on %s/%d\n", highWatermark, topic, partition)
- retryState[msg.retries].expectChaser = true
- output <- &ProducerMessage{Topic: topic, Partition: partition, flags: chaser, retries: msg.retries - 1}
- Logger.Printf("producer/leader abandoning broker %d on %s/%d\n", leader.ID(), topic, partition)
- p.unrefBrokerProducer(leader)
- output = nil
- time.Sleep(p.conf.Producer.Retry.Backoff)
- } else if highWatermark > 0 {
- // we are retrying something (else highWatermark would be 0) but this message is not a *new* retry level
- if msg.retries < highWatermark {
- // in fact this message is not even the current retry level, so buffer it for now (unless it's a just a chaser)
- if msg.flags&chaser == chaser {
- retryState[msg.retries].expectChaser = false
- } else {
- retryState[msg.retries].buf = append(retryState[msg.retries].buf, msg)
- }
- continue
- } else if msg.flags&chaser == chaser {
- // this message is of the current retry level (msg.retries == highWatermark) and the chaser flag is set,
- // meaning this retry level is done and we can go down (at least) one level and flush that
- retryState[highWatermark].expectChaser = false
- Logger.Printf("producer/leader state change to [normal-%d] on %s/%d\n", highWatermark, topic, partition)
- for {
- highWatermark--
- Logger.Printf("producer/leader state change to [flushing-%d] on %s/%d\n", highWatermark, topic, partition)
- if output == nil {
- if err := breaker.Run(doUpdate); err != nil {
- p.returnErrors(retryState[highWatermark].buf, err)
- goto flushDone
- }
- Logger.Printf("producer/leader selected broker %d on %s/%d\n", leader.ID(), topic, partition)
- }
- for _, msg := range retryState[highWatermark].buf {
- output <- msg
- }
- flushDone:
- retryState[highWatermark].buf = nil
- if retryState[highWatermark].expectChaser {
- Logger.Printf("producer/leader state change to [retrying-%d] on %s/%d\n", highWatermark, topic, partition)
- break
- } else {
- Logger.Printf("producer/leader state change to [normal-%d] on %s/%d\n", highWatermark, topic, partition)
- if highWatermark == 0 {
- break
- }
- }
- }
- continue
- }
- }
- // if we made it this far then the current msg contains real data, and can be sent to the next goroutine
- // without breaking any of our ordering guarantees
- if output == nil {
- if err := breaker.Run(doUpdate); err != nil {
- p.returnError(msg, err)
- time.Sleep(p.conf.Producer.Retry.Backoff)
- continue
- }
- Logger.Printf("producer/leader selected broker %d on %s/%d\n", leader.ID(), topic, partition)
- }
- output <- msg
- }
- p.unrefBrokerProducer(leader)
- p.retries <- &ProducerMessage{flags: unref}
- }
- // one per broker
- // groups messages together into appropriately-sized batches for sending to the broker
- // based on https://godoc.org/github.com/eapache/channels#BatchingChannel
- func (p *asyncProducer) messageAggregator(broker *Broker, input chan *ProducerMessage) {
- var ticker *time.Ticker
- var timer <-chan time.Time
- if p.conf.Producer.Flush.Frequency > 0 {
- ticker = time.NewTicker(p.conf.Producer.Flush.Frequency)
- timer = ticker.C
- }
- var buffer []*ProducerMessage
- var doFlush chan []*ProducerMessage
- var bytesAccumulated int
- flusher := make(chan []*ProducerMessage)
- go withRecover(func() { p.flusher(broker, flusher) })
- for {
- select {
- case msg := <-input:
- if msg == nil {
- goto shutdown
- }
- if (bytesAccumulated+msg.byteSize() >= forceFlushThreshold()) ||
- (p.conf.Producer.Compression != CompressionNone && bytesAccumulated+msg.byteSize() >= p.conf.Producer.MaxMessageBytes) ||
- (p.conf.Producer.Flush.MaxMessages > 0 && len(buffer) >= p.conf.Producer.Flush.MaxMessages) {
- Logger.Println("producer/aggregator maximum request accumulated, forcing blocking flush")
- flusher <- buffer
- buffer = nil
- doFlush = nil
- bytesAccumulated = 0
- }
- buffer = append(buffer, msg)
- bytesAccumulated += msg.byteSize()
- if len(buffer) >= p.conf.Producer.Flush.Messages ||
- (p.conf.Producer.Flush.Bytes > 0 && bytesAccumulated >= p.conf.Producer.Flush.Bytes) {
- doFlush = flusher
- }
- case <-timer:
- doFlush = flusher
- case doFlush <- buffer:
- buffer = nil
- doFlush = nil
- bytesAccumulated = 0
- }
- }
- shutdown:
- if ticker != nil {
- ticker.Stop()
- }
- if len(buffer) > 0 {
- flusher <- buffer
- }
- close(flusher)
- }
- // one per broker
- // takes a batch at a time from the messageAggregator and sends to the broker
- func (p *asyncProducer) flusher(broker *Broker, input chan []*ProducerMessage) {
- var closing error
- currentRetries := make(map[string]map[int32]error)
- Logger.Printf("producer/flusher/%d starting up\n", broker.ID())
- for batch := range input {
- if closing != nil {
- p.retryMessages(batch, closing)
- continue
- }
- // group messages by topic/partition
- msgSets := make(map[string]map[int32][]*ProducerMessage)
- for i, msg := range batch {
- if currentRetries[msg.Topic] != nil && currentRetries[msg.Topic][msg.Partition] != nil {
- if msg.flags&chaser == chaser {
- // we can start processing this topic/partition again
- Logger.Printf("producer/flusher/%d state change to [normal] on %s/%d\n",
- broker.ID(), msg.Topic, msg.Partition)
- currentRetries[msg.Topic][msg.Partition] = nil
- }
- p.retryMessages([]*ProducerMessage{msg}, currentRetries[msg.Topic][msg.Partition])
- batch[i] = nil // to prevent it being returned/retried twice
- continue
- }
- partitionSet := msgSets[msg.Topic]
- if partitionSet == nil {
- partitionSet = make(map[int32][]*ProducerMessage)
- msgSets[msg.Topic] = partitionSet
- }
- partitionSet[msg.Partition] = append(partitionSet[msg.Partition], msg)
- }
- request := p.buildRequest(msgSets)
- if request == nil {
- continue
- }
- response, err := broker.Produce(request)
- switch err.(type) {
- case nil:
- break
- case PacketEncodingError:
- p.returnErrors(batch, err)
- continue
- default:
- Logger.Printf("producer/flusher/%d state change to [closing] because %s\n", broker.ID(), err)
- closing = err
- _ = broker.Close()
- p.retryMessages(batch, err)
- continue
- }
- if response == nil {
- // this only happens when RequiredAcks is NoResponse, so we have to assume success
- if p.conf.Producer.Return.Successes {
- p.returnSuccesses(batch)
- }
- continue
- }
- // we iterate through the blocks in the request, not the response, so that we notice
- // if the response is missing a block completely
- for topic, partitionSet := range msgSets {
- for partition, msgs := range partitionSet {
- block := response.GetBlock(topic, partition)
- if block == nil {
- p.returnErrors(msgs, ErrIncompleteResponse)
- continue
- }
- switch block.Err {
- case ErrNoError:
- // All the messages for this topic-partition were delivered successfully!
- if p.conf.Producer.Return.Successes {
- for i := range msgs {
- msgs[i].Offset = block.Offset + int64(i)
- }
- p.returnSuccesses(msgs)
- }
- case ErrUnknownTopicOrPartition, ErrNotLeaderForPartition, ErrLeaderNotAvailable,
- ErrRequestTimedOut, ErrNotEnoughReplicas, ErrNotEnoughReplicasAfterAppend:
- Logger.Printf("producer/flusher/%d state change to [retrying] on %s/%d because %v\n",
- broker.ID(), topic, partition, block.Err)
- if currentRetries[topic] == nil {
- currentRetries[topic] = make(map[int32]error)
- }
- currentRetries[topic][partition] = block.Err
- p.retryMessages(msgs, block.Err)
- default:
- p.returnErrors(msgs, block.Err)
- }
- }
- }
- }
- Logger.Printf("producer/flusher/%d shut down\n", broker.ID())
- p.retries <- &ProducerMessage{flags: unref}
- }
- // singleton
- // effectively a "bridge" between the flushers and the topicDispatcher in order to avoid deadlock
- // based on https://godoc.org/github.com/eapache/channels#InfiniteChannel
- func (p *asyncProducer) retryHandler() {
- var buf []*ProducerMessage
- var msg *ProducerMessage
- refs := 0
- shuttingDown := false
- for {
- if len(buf) == 0 {
- msg = <-p.retries
- } else {
- select {
- case msg = <-p.retries:
- case p.input <- buf[0]:
- buf = buf[1:]
- continue
- }
- }
- if msg.flags&ref != 0 {
- refs++
- } else if msg.flags&unref != 0 {
- refs--
- if refs == 0 && shuttingDown {
- break
- }
- } else if msg.flags&shutdown != 0 {
- shuttingDown = true
- if refs == 0 {
- break
- }
- } else {
- buf = append(buf, msg)
- }
- }
- close(p.retries)
- for i := range buf {
- p.input <- buf[i]
- }
- close(p.input)
- }
- ///////////////////////////////////////////
- ///////////////////////////////////////////
- // utility functions
- func (p *asyncProducer) assignPartition(partitioner Partitioner, msg *ProducerMessage) error {
- var partitions []int32
- var err error
- if partitioner.RequiresConsistency() {
- partitions, err = p.client.Partitions(msg.Topic)
- } else {
- partitions, err = p.client.WritablePartitions(msg.Topic)
- }
- if err != nil {
- return err
- }
- numPartitions := int32(len(partitions))
- if numPartitions == 0 {
- return ErrLeaderNotAvailable
- }
- choice, err := partitioner.Partition(msg, numPartitions)
- if err != nil {
- return err
- } else if choice < 0 || choice >= numPartitions {
- return ErrInvalidPartition
- }
- msg.Partition = partitions[choice]
- return nil
- }
- func (p *asyncProducer) buildRequest(batch map[string]map[int32][]*ProducerMessage) *ProduceRequest {
- req := &ProduceRequest{RequiredAcks: p.conf.Producer.RequiredAcks, Timeout: int32(p.conf.Producer.Timeout / time.Millisecond)}
- empty := true
- for topic, partitionSet := range batch {
- for partition, msgSet := range partitionSet {
- setToSend := new(MessageSet)
- setSize := 0
- for _, msg := range msgSet {
- var keyBytes, valBytes []byte
- var err error
- if msg.Key != nil {
- if keyBytes, err = msg.Key.Encode(); err != nil {
- p.returnError(msg, err)
- continue
- }
- }
- if msg.Value != nil {
- if valBytes, err = msg.Value.Encode(); err != nil {
- p.returnError(msg, err)
- continue
- }
- }
- if p.conf.Producer.Compression != CompressionNone && setSize+msg.byteSize() > p.conf.Producer.MaxMessageBytes {
- // compression causes message-sets to be wrapped as single messages, which have tighter
- // size requirements, so we have to respect those limits
- valBytes, err := encode(setToSend)
- if err != nil {
- Logger.Println(err) // if this happens, it's basically our fault.
- panic(err)
- }
- req.AddMessage(topic, partition, &Message{Codec: p.conf.Producer.Compression, Key: nil, Value: valBytes})
- setToSend = new(MessageSet)
- setSize = 0
- }
- setSize += msg.byteSize()
- setToSend.addMessage(&Message{Codec: CompressionNone, Key: keyBytes, Value: valBytes})
- empty = false
- }
- if p.conf.Producer.Compression == CompressionNone {
- req.AddSet(topic, partition, setToSend)
- } else {
- valBytes, err := encode(setToSend)
- if err != nil {
- Logger.Println(err) // if this happens, it's basically our fault.
- panic(err)
- }
- req.AddMessage(topic, partition, &Message{Codec: p.conf.Producer.Compression, Key: nil, Value: valBytes})
- }
- }
- }
- if empty {
- return nil
- }
- return req
- }
- func (p *asyncProducer) returnError(msg *ProducerMessage, err error) {
- msg.flags = 0
- msg.retries = 0
- pErr := &ProducerError{Msg: msg, Err: err}
- if p.conf.Producer.Return.Errors {
- p.errors <- pErr
- } else {
- Logger.Println(pErr)
- }
- }
- func (p *asyncProducer) returnErrors(batch []*ProducerMessage, err error) {
- for _, msg := range batch {
- if msg != nil {
- p.returnError(msg, err)
- }
- }
- }
- func (p *asyncProducer) returnSuccesses(batch []*ProducerMessage) {
- for _, msg := range batch {
- if msg != nil {
- msg.flags = 0
- p.successes <- msg
- }
- }
- }
- func (p *asyncProducer) retryMessages(batch []*ProducerMessage, err error) {
- for _, msg := range batch {
- if msg == nil {
- continue
- }
- if msg.retries >= p.conf.Producer.Retry.Max {
- p.returnError(msg, err)
- } else {
- msg.retries++
- p.retries <- msg
- }
- }
- }
- type brokerProducer struct {
- input chan *ProducerMessage
- refs int
- }
- func (p *asyncProducer) getBrokerProducer(broker *Broker) chan *ProducerMessage {
- p.brokerLock.Lock()
- defer p.brokerLock.Unlock()
- producer := p.brokers[broker]
- if producer == nil {
- p.retries <- &ProducerMessage{flags: ref}
- producer = &brokerProducer{
- refs: 1,
- input: make(chan *ProducerMessage),
- }
- p.brokers[broker] = producer
- go withRecover(func() { p.messageAggregator(broker, producer.input) })
- } else {
- producer.refs++
- }
- return producer.input
- }
- func (p *asyncProducer) unrefBrokerProducer(broker *Broker) {
- p.brokerLock.Lock()
- defer p.brokerLock.Unlock()
- producer := p.brokers[broker]
- if producer != nil {
- producer.refs--
- if producer.refs == 0 {
- close(producer.input)
- delete(p.brokers, broker)
- }
- }
- }
|