Ver código fonte

Merge pull request #350 from Shopify/async_producer

Async Producer API changes
Willem van Bergen 10 anos atrás
pai
commit
7884212fc1
10 arquivos alterados com 135 adições e 104 exclusões
  1. 51 61
      async_producer.go
  2. 13 13
      async_producer_test.go
  3. 1 1
      errors.go
  4. 2 2
      functional_test.go
  5. 15 12
      mocks/async_producer.go
  6. 6 6
      mocks/async_producer_test.go
  7. 16 0
      partitioner.go
  8. 22 0
      partitioner_test.go
  9. 1 1
      sarama.go
  10. 8 8
      sync_producer.go

+ 51 - 61
producer.go → async_producer.go

@@ -12,14 +12,14 @@ func forceFlushThreshold() int {
 	return int(MaxRequestSize - (10 * 1024)) // 10KiB is safety room for misc. overhead, we might want to calculate this more precisely?
 	return int(MaxRequestSize - (10 * 1024)) // 10KiB is safety room for misc. overhead, we might want to calculate this more precisely?
 }
 }
 
 
-// Producer publishes Kafka messages. 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
+// 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
 // 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
 // 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
 // scope (this is in addition to calling Close on the underlying client, which
 // is still necessary).
 // is still necessary).
-type Producer interface {
+type AsyncProducer interface {
 
 
 	// AsyncClose triggers a shutdown of the producer, flushing any messages it may have
 	// 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
 	// buffered. The shutdown has completed when both the Errors and Successes channels
@@ -47,7 +47,7 @@ type Producer interface {
 	Errors() <-chan *ProducerError
 	Errors() <-chan *ProducerError
 }
 }
 
 
-type producer struct {
+type asyncProducer struct {
 	client    Client
 	client    Client
 	conf      *Config
 	conf      *Config
 	ownClient bool
 	ownClient bool
@@ -59,29 +59,29 @@ type producer struct {
 	brokerLock sync.Mutex
 	brokerLock sync.Mutex
 }
 }
 
 
-// NewProducer creates a new Producer using the given broker addresses and configuration.
-func NewProducer(addrs []string, conf *Config) (Producer, error) {
+// 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)
 	client, err := NewClient(addrs, conf)
 	if err != nil {
 	if err != nil {
 		return nil, err
 		return nil, err
 	}
 	}
 
 
-	p, err := NewProducerFromClient(client)
+	p, err := NewAsyncProducerFromClient(client)
 	if err != nil {
 	if err != nil {
 		return nil, err
 		return nil, err
 	}
 	}
-	p.(*producer).ownClient = true
+	p.(*asyncProducer).ownClient = true
 	return p, nil
 	return p, nil
 }
 }
 
 
-// NewProducerFromClient creates a new Producer using the given client.
-func NewProducerFromClient(client Client) (Producer, error) {
+// 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
 	// Check that we are not dealing with a closed Client before processing any other arguments
 	if client.Closed() {
 	if client.Closed() {
 		return nil, ErrClosedClient
 		return nil, ErrClosedClient
 	}
 	}
 
 
-	p := &producer{
+	p := &asyncProducer{
 		client:    client,
 		client:    client,
 		conf:      client.Config(),
 		conf:      client.Config(),
 		errors:    make(chan *ProducerError),
 		errors:    make(chan *ProducerError),
@@ -109,28 +109,18 @@ const (
 
 
 // ProducerMessage is the collection of elements passed to the Producer in order to send a message.
 // ProducerMessage is the collection of elements passed to the Producer in order to send a message.
 type ProducerMessage struct {
 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.
-	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.
+	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
-	partition int32
-	retries   int
-	flags     flagSet
-}
+	// 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.
 
 
-// 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.
-func (m *ProducerMessage) Offset() int64 {
-	return m.offset
-}
+	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.
 
 
-// Partition is the partition that the message was sent to. This is only guaranteed to be defined if
-// the message was successfully delivered.
-func (m *ProducerMessage) Partition() int32 {
-	return m.partition
+	retries int
+	flags   flagSet
 }
 }
 
 
 func (m *ProducerMessage) byteSize() int {
 func (m *ProducerMessage) byteSize() int {
@@ -164,19 +154,19 @@ func (pe ProducerErrors) Error() string {
 	return fmt.Sprintf("kafka: Failed to deliver %d messages.", len(pe))
 	return fmt.Sprintf("kafka: Failed to deliver %d messages.", len(pe))
 }
 }
 
 
-func (p *producer) Errors() <-chan *ProducerError {
+func (p *asyncProducer) Errors() <-chan *ProducerError {
 	return p.errors
 	return p.errors
 }
 }
 
 
-func (p *producer) Successes() <-chan *ProducerMessage {
+func (p *asyncProducer) Successes() <-chan *ProducerMessage {
 	return p.successes
 	return p.successes
 }
 }
 
 
-func (p *producer) Input() chan<- *ProducerMessage {
+func (p *asyncProducer) Input() chan<- *ProducerMessage {
 	return p.input
 	return p.input
 }
 }
 
 
-func (p *producer) Close() error {
+func (p *asyncProducer) Close() error {
 	p.AsyncClose()
 	p.AsyncClose()
 
 
 	if p.conf.Producer.Return.Successes {
 	if p.conf.Producer.Return.Successes {
@@ -199,7 +189,7 @@ func (p *producer) Close() error {
 	return nil
 	return nil
 }
 }
 
 
-func (p *producer) AsyncClose() {
+func (p *asyncProducer) AsyncClose() {
 	go withRecover(func() {
 	go withRecover(func() {
 		p.input <- &ProducerMessage{flags: shutdown}
 		p.input <- &ProducerMessage{flags: shutdown}
 	})
 	})
@@ -214,7 +204,7 @@ func (p *producer) AsyncClose() {
 
 
 // singleton
 // singleton
 // dispatches messages by topic
 // dispatches messages by topic
-func (p *producer) topicDispatcher() {
+func (p *asyncProducer) topicDispatcher() {
 	handlers := make(map[string]chan *ProducerMessage)
 	handlers := make(map[string]chan *ProducerMessage)
 
 
 	for msg := range p.input {
 	for msg := range p.input {
@@ -270,7 +260,7 @@ func (p *producer) topicDispatcher() {
 
 
 // one per topic
 // one per topic
 // partitions messages, then dispatches them by partition
 // partitions messages, then dispatches them by partition
-func (p *producer) partitionDispatcher(topic string, input chan *ProducerMessage) {
+func (p *asyncProducer) partitionDispatcher(topic string, input chan *ProducerMessage) {
 	handlers := make(map[int32]chan *ProducerMessage)
 	handlers := make(map[int32]chan *ProducerMessage)
 	partitioner := p.conf.Producer.Partitioner()
 	partitioner := p.conf.Producer.Partitioner()
 
 
@@ -283,15 +273,15 @@ func (p *producer) partitionDispatcher(topic string, input chan *ProducerMessage
 			}
 			}
 		}
 		}
 
 
-		handler := handlers[msg.partition]
+		handler := handlers[msg.Partition]
 		if handler == nil {
 		if handler == nil {
 			p.retries <- &ProducerMessage{flags: ref}
 			p.retries <- &ProducerMessage{flags: ref}
 			newHandler := make(chan *ProducerMessage, p.conf.ChannelBufferSize)
 			newHandler := make(chan *ProducerMessage, p.conf.ChannelBufferSize)
 			topic := msg.Topic         // block local because go's closure semantics suck
 			topic := msg.Topic         // block local because go's closure semantics suck
-			partition := msg.partition // 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) })
 			go withRecover(func() { p.leaderDispatcher(topic, partition, newHandler) })
 			handler = newHandler
 			handler = newHandler
-			handlers[msg.partition] = handler
+			handlers[msg.Partition] = handler
 		}
 		}
 
 
 		handler <- msg
 		handler <- msg
@@ -306,7 +296,7 @@ func (p *producer) partitionDispatcher(topic string, input chan *ProducerMessage
 // one per partition per topic
 // one per partition per topic
 // dispatches messages to the appropriate broker
 // dispatches messages to the appropriate broker
 // also responsible for maintaining message order during retries
 // also responsible for maintaining message order during retries
-func (p *producer) leaderDispatcher(topic string, partition int32, input chan *ProducerMessage) {
+func (p *asyncProducer) leaderDispatcher(topic string, partition int32, input chan *ProducerMessage) {
 	var leader *Broker
 	var leader *Broker
 	var output chan *ProducerMessage
 	var output chan *ProducerMessage
 
 
@@ -348,7 +338,7 @@ func (p *producer) leaderDispatcher(topic string, partition int32, input chan *P
 			highWatermark = msg.retries
 			highWatermark = msg.retries
 			Logger.Printf("producer/leader state change to [retrying-%d] on %s/%d\n", highWatermark, topic, partition)
 			Logger.Printf("producer/leader state change to [retrying-%d] on %s/%d\n", highWatermark, topic, partition)
 			retryState[msg.retries].expectChaser = true
 			retryState[msg.retries].expectChaser = true
-			output <- &ProducerMessage{Topic: topic, partition: partition, flags: chaser, retries: msg.retries - 1}
+			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)
 			Logger.Printf("producer/leader abandoning broker %d on %s/%d\n", leader.ID(), topic, partition)
 			p.unrefBrokerProducer(leader)
 			p.unrefBrokerProducer(leader)
 			output = nil
 			output = nil
@@ -423,7 +413,7 @@ func (p *producer) leaderDispatcher(topic string, partition int32, input chan *P
 // one per broker
 // one per broker
 // groups messages together into appropriately-sized batches for sending to the broker
 // groups messages together into appropriately-sized batches for sending to the broker
 // based on https://godoc.org/github.com/eapache/channels#BatchingChannel
 // based on https://godoc.org/github.com/eapache/channels#BatchingChannel
-func (p *producer) messageAggregator(broker *Broker, input chan *ProducerMessage) {
+func (p *asyncProducer) messageAggregator(broker *Broker, input chan *ProducerMessage) {
 	var ticker *time.Ticker
 	var ticker *time.Ticker
 	var timer <-chan time.Time
 	var timer <-chan time.Time
 	if p.conf.Producer.Flush.Frequency > 0 {
 	if p.conf.Producer.Flush.Frequency > 0 {
@@ -483,7 +473,7 @@ shutdown:
 
 
 // one per broker
 // one per broker
 // takes a batch at a time from the messageAggregator and sends to the broker
 // takes a batch at a time from the messageAggregator and sends to the broker
-func (p *producer) flusher(broker *Broker, input chan []*ProducerMessage) {
+func (p *asyncProducer) flusher(broker *Broker, input chan []*ProducerMessage) {
 	var closing error
 	var closing error
 	currentRetries := make(map[string]map[int32]error)
 	currentRetries := make(map[string]map[int32]error)
 	Logger.Printf("producer/flusher/%d starting up\n", broker.ID())
 	Logger.Printf("producer/flusher/%d starting up\n", broker.ID())
@@ -497,14 +487,14 @@ func (p *producer) flusher(broker *Broker, input chan []*ProducerMessage) {
 		// group messages by topic/partition
 		// group messages by topic/partition
 		msgSets := make(map[string]map[int32][]*ProducerMessage)
 		msgSets := make(map[string]map[int32][]*ProducerMessage)
 		for i, msg := range batch {
 		for i, msg := range batch {
-			if currentRetries[msg.Topic] != nil && currentRetries[msg.Topic][msg.partition] != nil {
+			if currentRetries[msg.Topic] != nil && currentRetries[msg.Topic][msg.Partition] != nil {
 				if msg.flags&chaser == chaser {
 				if msg.flags&chaser == chaser {
 					// we can start processing this topic/partition again
 					// we can start processing this topic/partition again
 					Logger.Printf("producer/flusher/%d state change to [normal] on %s/%d\n",
 					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
+						broker.ID(), msg.Topic, msg.Partition)
+					currentRetries[msg.Topic][msg.Partition] = nil
 				}
 				}
-				p.retryMessages([]*ProducerMessage{msg}, currentRetries[msg.Topic][msg.partition])
+				p.retryMessages([]*ProducerMessage{msg}, currentRetries[msg.Topic][msg.Partition])
 				batch[i] = nil // to prevent it being returned/retried twice
 				batch[i] = nil // to prevent it being returned/retried twice
 				continue
 				continue
 			}
 			}
@@ -515,7 +505,7 @@ func (p *producer) flusher(broker *Broker, input chan []*ProducerMessage) {
 				msgSets[msg.Topic] = partitionSet
 				msgSets[msg.Topic] = partitionSet
 			}
 			}
 
 
-			partitionSet[msg.partition] = append(partitionSet[msg.partition], msg)
+			partitionSet[msg.Partition] = append(partitionSet[msg.Partition], msg)
 		}
 		}
 
 
 		request := p.buildRequest(msgSets)
 		request := p.buildRequest(msgSets)
@@ -563,7 +553,7 @@ func (p *producer) flusher(broker *Broker, input chan []*ProducerMessage) {
 					// All the messages for this topic-partition were delivered successfully!
 					// All the messages for this topic-partition were delivered successfully!
 					if p.conf.Producer.Return.Successes {
 					if p.conf.Producer.Return.Successes {
 						for i := range msgs {
 						for i := range msgs {
-							msgs[i].offset = block.Offset + int64(i)
+							msgs[i].Offset = block.Offset + int64(i)
 						}
 						}
 						p.returnSuccesses(msgs)
 						p.returnSuccesses(msgs)
 					}
 					}
@@ -589,7 +579,7 @@ func (p *producer) flusher(broker *Broker, input chan []*ProducerMessage) {
 // singleton
 // singleton
 // effectively a "bridge" between the flushers and the topicDispatcher in order to avoid deadlock
 // effectively a "bridge" between the flushers and the topicDispatcher in order to avoid deadlock
 // based on https://godoc.org/github.com/eapache/channels#InfiniteChannel
 // based on https://godoc.org/github.com/eapache/channels#InfiniteChannel
-func (p *producer) retryHandler() {
+func (p *asyncProducer) retryHandler() {
 	var buf []*ProducerMessage
 	var buf []*ProducerMessage
 	var msg *ProducerMessage
 	var msg *ProducerMessage
 	refs := 0
 	refs := 0
@@ -636,7 +626,7 @@ func (p *producer) retryHandler() {
 
 
 // utility functions
 // utility functions
 
 
-func (p *producer) assignPartition(partitioner Partitioner, msg *ProducerMessage) error {
+func (p *asyncProducer) assignPartition(partitioner Partitioner, msg *ProducerMessage) error {
 	var partitions []int32
 	var partitions []int32
 	var err error
 	var err error
 
 
@@ -664,12 +654,12 @@ func (p *producer) assignPartition(partitioner Partitioner, msg *ProducerMessage
 		return ErrInvalidPartition
 		return ErrInvalidPartition
 	}
 	}
 
 
-	msg.partition = partitions[choice]
+	msg.Partition = partitions[choice]
 
 
 	return nil
 	return nil
 }
 }
 
 
-func (p *producer) buildRequest(batch map[string]map[int32][]*ProducerMessage) *ProduceRequest {
+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)}
 	req := &ProduceRequest{RequiredAcks: p.conf.Producer.RequiredAcks, Timeout: int32(p.conf.Producer.Timeout / time.Millisecond)}
 	empty := true
 	empty := true
@@ -731,7 +721,7 @@ func (p *producer) buildRequest(batch map[string]map[int32][]*ProducerMessage) *
 	return req
 	return req
 }
 }
 
 
-func (p *producer) returnError(msg *ProducerMessage, err error) {
+func (p *asyncProducer) returnError(msg *ProducerMessage, err error) {
 	msg.flags = 0
 	msg.flags = 0
 	msg.retries = 0
 	msg.retries = 0
 	pErr := &ProducerError{Msg: msg, Err: err}
 	pErr := &ProducerError{Msg: msg, Err: err}
@@ -742,7 +732,7 @@ func (p *producer) returnError(msg *ProducerMessage, err error) {
 	}
 	}
 }
 }
 
 
-func (p *producer) returnErrors(batch []*ProducerMessage, err error) {
+func (p *asyncProducer) returnErrors(batch []*ProducerMessage, err error) {
 	for _, msg := range batch {
 	for _, msg := range batch {
 		if msg != nil {
 		if msg != nil {
 			p.returnError(msg, err)
 			p.returnError(msg, err)
@@ -750,7 +740,7 @@ func (p *producer) returnErrors(batch []*ProducerMessage, err error) {
 	}
 	}
 }
 }
 
 
-func (p *producer) returnSuccesses(batch []*ProducerMessage) {
+func (p *asyncProducer) returnSuccesses(batch []*ProducerMessage) {
 	for _, msg := range batch {
 	for _, msg := range batch {
 		if msg != nil {
 		if msg != nil {
 			msg.flags = 0
 			msg.flags = 0
@@ -759,7 +749,7 @@ func (p *producer) returnSuccesses(batch []*ProducerMessage) {
 	}
 	}
 }
 }
 
 
-func (p *producer) retryMessages(batch []*ProducerMessage, err error) {
+func (p *asyncProducer) retryMessages(batch []*ProducerMessage, err error) {
 	for _, msg := range batch {
 	for _, msg := range batch {
 		if msg == nil {
 		if msg == nil {
 			continue
 			continue
@@ -778,7 +768,7 @@ type brokerProducer struct {
 	refs  int
 	refs  int
 }
 }
 
 
-func (p *producer) getBrokerProducer(broker *Broker) chan *ProducerMessage {
+func (p *asyncProducer) getBrokerProducer(broker *Broker) chan *ProducerMessage {
 	p.brokerLock.Lock()
 	p.brokerLock.Lock()
 	defer p.brokerLock.Unlock()
 	defer p.brokerLock.Unlock()
 
 
@@ -799,7 +789,7 @@ func (p *producer) getBrokerProducer(broker *Broker) chan *ProducerMessage {
 	return producer.input
 	return producer.input
 }
 }
 
 
-func (p *producer) unrefBrokerProducer(broker *Broker) {
+func (p *asyncProducer) unrefBrokerProducer(broker *Broker) {
 	p.brokerLock.Lock()
 	p.brokerLock.Lock()
 	defer p.brokerLock.Unlock()
 	defer p.brokerLock.Unlock()
 
 

+ 13 - 13
producer_test.go → async_producer_test.go

@@ -10,7 +10,7 @@ import (
 
 
 const TestMessage = "ABC THE MESSAGE"
 const TestMessage = "ABC THE MESSAGE"
 
 
-func closeProducer(t *testing.T, p Producer) {
+func closeProducer(t *testing.T, p AsyncProducer) {
 	var wg sync.WaitGroup
 	var wg sync.WaitGroup
 	p.AsyncClose()
 	p.AsyncClose()
 
 
@@ -126,7 +126,7 @@ func TestProducer(t *testing.T) {
 	config := NewConfig()
 	config := NewConfig()
 	config.Producer.Flush.Messages = 10
 	config.Producer.Flush.Messages = 10
 	config.Producer.Return.Successes = true
 	config.Producer.Return.Successes = true
-	producer, err := NewProducer([]string{seedBroker.Addr()}, config)
+	producer, err := NewAsyncProducer([]string{seedBroker.Addr()}, config)
 	if err != nil {
 	if err != nil {
 		t.Fatal(err)
 		t.Fatal(err)
 	}
 	}
@@ -174,7 +174,7 @@ func TestProducerMultipleFlushes(t *testing.T) {
 	config := NewConfig()
 	config := NewConfig()
 	config.Producer.Flush.Messages = 5
 	config.Producer.Flush.Messages = 5
 	config.Producer.Return.Successes = true
 	config.Producer.Return.Successes = true
-	producer, err := NewProducer([]string{seedBroker.Addr()}, config)
+	producer, err := NewAsyncProducer([]string{seedBroker.Addr()}, config)
 	if err != nil {
 	if err != nil {
 		t.Fatal(err)
 		t.Fatal(err)
 	}
 	}
@@ -227,7 +227,7 @@ func TestProducerMultipleBrokers(t *testing.T) {
 	config.Producer.Flush.Messages = 5
 	config.Producer.Flush.Messages = 5
 	config.Producer.Return.Successes = true
 	config.Producer.Return.Successes = true
 	config.Producer.Partitioner = NewRoundRobinPartitioner
 	config.Producer.Partitioner = NewRoundRobinPartitioner
-	producer, err := NewProducer([]string{seedBroker.Addr()}, config)
+	producer, err := NewAsyncProducer([]string{seedBroker.Addr()}, config)
 	if err != nil {
 	if err != nil {
 		t.Fatal(err)
 		t.Fatal(err)
 	}
 	}
@@ -269,7 +269,7 @@ func TestProducerFailureRetry(t *testing.T) {
 	config.Producer.Flush.Messages = 10
 	config.Producer.Flush.Messages = 10
 	config.Producer.Return.Successes = true
 	config.Producer.Return.Successes = true
 	config.Producer.Retry.Backoff = 0
 	config.Producer.Retry.Backoff = 0
-	producer, err := NewProducer([]string{seedBroker.Addr()}, config)
+	producer, err := NewAsyncProducer([]string{seedBroker.Addr()}, config)
 	if err != nil {
 	if err != nil {
 		t.Fatal(err)
 		t.Fatal(err)
 	}
 	}
@@ -341,7 +341,7 @@ func TestProducerBrokerBounce(t *testing.T) {
 	config.Producer.Flush.Messages = 10
 	config.Producer.Flush.Messages = 10
 	config.Producer.Return.Successes = true
 	config.Producer.Return.Successes = true
 	config.Producer.Retry.Backoff = 0
 	config.Producer.Retry.Backoff = 0
-	producer, err := NewProducer([]string{seedBroker.Addr()}, config)
+	producer, err := NewAsyncProducer([]string{seedBroker.Addr()}, config)
 	if err != nil {
 	if err != nil {
 		t.Fatal(err)
 		t.Fatal(err)
 	}
 	}
@@ -390,7 +390,7 @@ func TestProducerBrokerBounceWithStaleMetadata(t *testing.T) {
 	config.Producer.Return.Successes = true
 	config.Producer.Return.Successes = true
 	config.Producer.Retry.Max = 3
 	config.Producer.Retry.Max = 3
 	config.Producer.Retry.Backoff = 0
 	config.Producer.Retry.Backoff = 0
-	producer, err := NewProducer([]string{seedBroker.Addr()}, config)
+	producer, err := NewAsyncProducer([]string{seedBroker.Addr()}, config)
 	if err != nil {
 	if err != nil {
 		t.Fatal(err)
 		t.Fatal(err)
 	}
 	}
@@ -445,7 +445,7 @@ func TestProducerMultipleRetries(t *testing.T) {
 	config.Producer.Return.Successes = true
 	config.Producer.Return.Successes = true
 	config.Producer.Retry.Max = 4
 	config.Producer.Retry.Max = 4
 	config.Producer.Retry.Backoff = 0
 	config.Producer.Retry.Backoff = 0
-	producer, err := NewProducer([]string{seedBroker.Addr()}, config)
+	producer, err := NewAsyncProducer([]string{seedBroker.Addr()}, config)
 	if err != nil {
 	if err != nil {
 		t.Fatal(err)
 		t.Fatal(err)
 	}
 	}
@@ -525,7 +525,7 @@ func TestProducerOutOfRetries(t *testing.T) {
 	config.Producer.Return.Successes = true
 	config.Producer.Return.Successes = true
 	config.Producer.Retry.Backoff = 0
 	config.Producer.Retry.Backoff = 0
 	config.Producer.Retry.Max = 0
 	config.Producer.Retry.Max = 0
-	producer, err := NewProducer([]string{seedBroker.Addr()}, config)
+	producer, err := NewAsyncProducer([]string{seedBroker.Addr()}, config)
 	if err != nil {
 	if err != nil {
 		t.Fatal(err)
 		t.Fatal(err)
 	}
 	}
@@ -574,8 +574,8 @@ func TestProducerOutOfRetries(t *testing.T) {
 
 
 // This example shows how to use the producer while simultaneously
 // This example shows how to use the producer while simultaneously
 // reading the Errors channel to know about any failures.
 // reading the Errors channel to know about any failures.
-func ExampleProducer_select() {
-	producer, err := NewProducer([]string{"localhost:9092"}, nil)
+func ExampleAsyncProducer_select() {
+	producer, err := NewAsyncProducer([]string{"localhost:9092"}, nil)
 	if err != nil {
 	if err != nil {
 		panic(err)
 		panic(err)
 	}
 	}
@@ -611,10 +611,10 @@ ProducerLoop:
 // reading from the Successes and Errors channels. Note that in order
 // reading from the Successes and Errors channels. Note that in order
 // for the Successes channel to be populated, you have to set
 // for the Successes channel to be populated, you have to set
 // config.Producer.Return.Successes to true.
 // config.Producer.Return.Successes to true.
-func ExampleProducer_goroutines() {
+func ExampleAsyncProducer_goroutines() {
 	config := NewConfig()
 	config := NewConfig()
 	config.Producer.Return.Successes = true
 	config.Producer.Return.Successes = true
-	producer, err := NewProducer([]string{"localhost:9092"}, config)
+	producer, err := NewAsyncProducer([]string{"localhost:9092"}, config)
 	if err != nil {
 	if err != nil {
 		panic(err)
 		panic(err)
 	}
 	}

+ 1 - 1
errors.go

@@ -58,7 +58,7 @@ func (err PacketDecodingError) Error() string {
 	return fmt.Sprintf("kafka: Error while decoding packet: %s", err.Info)
 	return fmt.Sprintf("kafka: Error while decoding packet: %s", err.Info)
 }
 }
 
 
-// ConfigurationError is the type of error returned from NewClient, NewProducer or NewConsumer when the specified
+// ConfigurationError is the type of error returned from NewClient, NewAsyncProducer or NewConsumer when the specified
 // configuration is invalid.
 // configuration is invalid.
 type ConfigurationError string
 type ConfigurationError string
 
 

+ 2 - 2
functional_test.go

@@ -91,7 +91,7 @@ func TestFuncMultiPartitionProduce(t *testing.T) {
 	config.Producer.Flush.Frequency = 50 * time.Millisecond
 	config.Producer.Flush.Frequency = 50 * time.Millisecond
 	config.Producer.Flush.Messages = 200
 	config.Producer.Flush.Messages = 200
 	config.Producer.Return.Successes = true
 	config.Producer.Return.Successes = true
-	producer, err := NewProducer([]string{kafkaAddr}, config)
+	producer, err := NewAsyncProducer([]string{kafkaAddr}, config)
 	if err != nil {
 	if err != nil {
 		t.Fatal(err)
 		t.Fatal(err)
 	}
 	}
@@ -139,7 +139,7 @@ func testProducingMessages(t *testing.T, config *Config) {
 		t.Fatal(err)
 		t.Fatal(err)
 	}
 	}
 
 
-	producer, err := NewProducerFromClient(client)
+	producer, err := NewAsyncProducerFromClient(client)
 	if err != nil {
 	if err != nil {
 		t.Fatal(err)
 		t.Fatal(err)
 	}
 	}

+ 15 - 12
mocks/producer.go → mocks/async_producer.go

@@ -6,11 +6,11 @@ import (
 	"github.com/Shopify/sarama"
 	"github.com/Shopify/sarama"
 )
 )
 
 
-// Producer implements sarama's Producer interface for testing purposes.
+// AsyncProducer implements sarama's Producer interface for testing purposes.
 // Before you can send messages to it's Input channel, you have to set expectations
 // Before you can send messages to it's Input channel, you have to set expectations
 // so it knows how to handle the input. This way you can easily test success and
 // so it knows how to handle the input. This way you can easily test success and
 // failure scenarios.
 // failure scenarios.
-type Producer struct {
+type AsyncProducer struct {
 	l            sync.Mutex
 	l            sync.Mutex
 	t            ErrorReporter
 	t            ErrorReporter
 	expectations []*producerExpectation
 	expectations []*producerExpectation
@@ -18,17 +18,18 @@ type Producer struct {
 	input        chan *sarama.ProducerMessage
 	input        chan *sarama.ProducerMessage
 	successes    chan *sarama.ProducerMessage
 	successes    chan *sarama.ProducerMessage
 	errors       chan *sarama.ProducerError
 	errors       chan *sarama.ProducerError
+	lastOffset   int64
 }
 }
 
 
-// NewProducer instantiates a new Producer mock. The t argument should
+// NewAsyncProducer instantiates a new Producer mock. The t argument should
 // be the *testing.T instance of your test method. An error will be written to it if
 // be the *testing.T instance of your test method. An error will be written to it if
 // an expectation is violated. The config argument is used to determine whether it
 // an expectation is violated. The config argument is used to determine whether it
 // should ack successes on the Successes channel.
 // should ack successes on the Successes channel.
-func NewProducer(t ErrorReporter, config *sarama.Config) *Producer {
+func NewAsyncProducer(t ErrorReporter, config *sarama.Config) *AsyncProducer {
 	if config == nil {
 	if config == nil {
 		config = sarama.NewConfig()
 		config = sarama.NewConfig()
 	}
 	}
-	mp := &Producer{
+	mp := &AsyncProducer{
 		t:            t,
 		t:            t,
 		closed:       make(chan struct{}, 0),
 		closed:       make(chan struct{}, 0),
 		expectations: make([]*producerExpectation, 0),
 		expectations: make([]*producerExpectation, 0),
@@ -52,7 +53,9 @@ func NewProducer(t ErrorReporter, config *sarama.Config) *Producer {
 				expectation := mp.expectations[0]
 				expectation := mp.expectations[0]
 				mp.expectations = mp.expectations[1:]
 				mp.expectations = mp.expectations[1:]
 				if expectation.Result == errProduceSuccess {
 				if expectation.Result == errProduceSuccess {
+					mp.lastOffset++
 					if config.Producer.Return.Successes {
 					if config.Producer.Return.Successes {
+						msg.Offset = mp.lastOffset
 						mp.successes <- msg
 						mp.successes <- msg
 					}
 					}
 				} else {
 				} else {
@@ -83,14 +86,14 @@ func NewProducer(t ErrorReporter, config *sarama.Config) *Producer {
 // AsyncClose corresponds with the AsyncClose method of sarama's Producer implementation.
 // AsyncClose corresponds with the AsyncClose method of sarama's Producer implementation.
 // By closing a mock producer, you also tell it that no more input will be provided, so it will
 // By closing a mock producer, you also tell it that no more input will be provided, so it will
 // write an error to the test state if there's any remaining expectations.
 // write an error to the test state if there's any remaining expectations.
-func (mp *Producer) AsyncClose() {
+func (mp *AsyncProducer) AsyncClose() {
 	close(mp.input)
 	close(mp.input)
 }
 }
 
 
 // Close corresponds with the Close method of sarama's Producer implementation.
 // Close corresponds with the Close method of sarama's Producer implementation.
 // By closing a mock producer, you also tell it that no more input will be provided, so it will
 // By closing a mock producer, you also tell it that no more input will be provided, so it will
 // write an error to the test state if there's any remaining expectations.
 // write an error to the test state if there's any remaining expectations.
-func (mp *Producer) Close() error {
+func (mp *AsyncProducer) Close() error {
 	mp.AsyncClose()
 	mp.AsyncClose()
 	<-mp.closed
 	<-mp.closed
 	return nil
 	return nil
@@ -101,17 +104,17 @@ func (mp *Producer) Close() error {
 // channel, so it knows how to handle them. If there is no more remaining expectations and
 // channel, so it knows how to handle them. If there is no more remaining expectations and
 // a messages is written to the Input channel, the mock producer will write an error to the test
 // a messages is written to the Input channel, the mock producer will write an error to the test
 // state object.
 // state object.
-func (mp *Producer) Input() chan<- *sarama.ProducerMessage {
+func (mp *AsyncProducer) Input() chan<- *sarama.ProducerMessage {
 	return mp.input
 	return mp.input
 }
 }
 
 
 // Successes corresponds with the Successes method of sarama's Producer implementation.
 // Successes corresponds with the Successes method of sarama's Producer implementation.
-func (mp *Producer) Successes() <-chan *sarama.ProducerMessage {
+func (mp *AsyncProducer) Successes() <-chan *sarama.ProducerMessage {
 	return mp.successes
 	return mp.successes
 }
 }
 
 
 // Errors corresponds with the Errors method of sarama's Producer implementation.
 // Errors corresponds with the Errors method of sarama's Producer implementation.
-func (mp *Producer) Errors() <-chan *sarama.ProducerError {
+func (mp *AsyncProducer) Errors() <-chan *sarama.ProducerError {
 	return mp.errors
 	return mp.errors
 }
 }
 
 
@@ -123,7 +126,7 @@ func (mp *Producer) Errors() <-chan *sarama.ProducerError {
 // on the input channel. The mock producer will handle the message as if it is produced successfully,
 // on the input channel. The mock producer will handle the message as if it is produced successfully,
 // i.e. it will make it available on the Successes channel if the Producer.Return.Successes setting
 // i.e. it will make it available on the Successes channel if the Producer.Return.Successes setting
 // is set to true.
 // is set to true.
-func (mp *Producer) ExpectInputAndSucceed() {
+func (mp *AsyncProducer) ExpectInputAndSucceed() {
 	mp.l.Lock()
 	mp.l.Lock()
 	defer mp.l.Unlock()
 	defer mp.l.Unlock()
 	mp.expectations = append(mp.expectations, &producerExpectation{Result: errProduceSuccess})
 	mp.expectations = append(mp.expectations, &producerExpectation{Result: errProduceSuccess})
@@ -132,7 +135,7 @@ func (mp *Producer) ExpectInputAndSucceed() {
 // ExpectInputAndFail sets an expectation on the mock producer that a message will be provided
 // ExpectInputAndFail sets an expectation on the mock producer that a message will be provided
 // on the input channel. The mock producer will handle the message as if it failed to produce
 // on the input channel. The mock producer will handle the message as if it failed to produce
 // successfully. This means it will make a ProducerError available on the Errors channel.
 // successfully. This means it will make a ProducerError available on the Errors channel.
-func (mp *Producer) ExpectInputAndFail(err error) {
+func (mp *AsyncProducer) ExpectInputAndFail(err error) {
 	mp.l.Lock()
 	mp.l.Lock()
 	defer mp.l.Unlock()
 	defer mp.l.Unlock()
 	mp.expectations = append(mp.expectations, &producerExpectation{Result: err})
 	mp.expectations = append(mp.expectations, &producerExpectation{Result: err})

+ 6 - 6
mocks/producer_test.go → mocks/async_producer_test.go

@@ -19,9 +19,9 @@ func (trm *testReporterMock) Errorf(format string, args ...interface{}) {
 	trm.errors = append(trm.errors, fmt.Sprintf(format, args...))
 	trm.errors = append(trm.errors, fmt.Sprintf(format, args...))
 }
 }
 
 
-func TestMockProducerImplementsProducerInterface(t *testing.T) {
-	var mp interface{} = &Producer{}
-	if _, ok := mp.(sarama.Producer); !ok {
+func TestMockAsyncProducerImplementsAsyncProducerInterface(t *testing.T) {
+	var mp interface{} = &AsyncProducer{}
+	if _, ok := mp.(sarama.AsyncProducer); !ok {
 		t.Error("The mock producer should implement the sarama.Producer interface.")
 		t.Error("The mock producer should implement the sarama.Producer interface.")
 	}
 	}
 }
 }
@@ -29,7 +29,7 @@ func TestMockProducerImplementsProducerInterface(t *testing.T) {
 func TestProducerReturnsExpectationsToChannels(t *testing.T) {
 func TestProducerReturnsExpectationsToChannels(t *testing.T) {
 	config := sarama.NewConfig()
 	config := sarama.NewConfig()
 	config.Producer.Return.Successes = true
 	config.Producer.Return.Successes = true
-	mp := NewProducer(t, config)
+	mp := NewAsyncProducer(t, config)
 
 
 	mp.ExpectInputAndSucceed()
 	mp.ExpectInputAndSucceed()
 	mp.ExpectInputAndSucceed()
 	mp.ExpectInputAndSucceed()
@@ -62,7 +62,7 @@ func TestProducerReturnsExpectationsToChannels(t *testing.T) {
 
 
 func TestProducerWithTooFewExpectations(t *testing.T) {
 func TestProducerWithTooFewExpectations(t *testing.T) {
 	trm := newTestReporterMock()
 	trm := newTestReporterMock()
-	mp := NewProducer(trm, nil)
+	mp := NewAsyncProducer(trm, nil)
 	mp.ExpectInputAndSucceed()
 	mp.ExpectInputAndSucceed()
 
 
 	mp.Input() <- &sarama.ProducerMessage{Topic: "test"}
 	mp.Input() <- &sarama.ProducerMessage{Topic: "test"}
@@ -79,7 +79,7 @@ func TestProducerWithTooFewExpectations(t *testing.T) {
 
 
 func TestProducerWithTooManyExpectations(t *testing.T) {
 func TestProducerWithTooManyExpectations(t *testing.T) {
 	trm := newTestReporterMock()
 	trm := newTestReporterMock()
-	mp := NewProducer(trm, nil)
+	mp := NewAsyncProducer(trm, nil)
 	mp.ExpectInputAndSucceed()
 	mp.ExpectInputAndSucceed()
 	mp.ExpectInputAndFail(sarama.ErrOutOfBrokers)
 	mp.ExpectInputAndFail(sarama.ErrOutOfBrokers)
 
 

+ 16 - 0
partitioner.go

@@ -22,6 +22,22 @@ type Partitioner interface {
 // PartitionerConstructor is the type for a function capable of constructing new Partitioners.
 // PartitionerConstructor is the type for a function capable of constructing new Partitioners.
 type PartitionerConstructor func() Partitioner
 type PartitionerConstructor func() Partitioner
 
 
+type manualPartitioner struct{}
+
+// NewManualPartitioner returns a Partitioner which uses the partition manually set in the provided
+// ProducerMessage's Partition field as the partition to produce to.
+func NewManualPartitioner() Partitioner {
+	return new(manualPartitioner)
+}
+
+func (p *manualPartitioner) Partition(message *ProducerMessage, numPartitions int32) (int32, error) {
+	return message.Partition, nil
+}
+
+func (p *manualPartitioner) RequiresConsistency() bool {
+	return true
+}
+
 type randomPartitioner struct {
 type randomPartitioner struct {
 	generator *rand.Rand
 	generator *rand.Rand
 }
 }

+ 22 - 0
partitioner_test.go

@@ -98,3 +98,25 @@ func TestHashPartitioner(t *testing.T) {
 		assertPartitioningConsistent(t, partitioner, &ProducerMessage{Key: ByteEncoder(buf)}, 50)
 		assertPartitioningConsistent(t, partitioner, &ProducerMessage{Key: ByteEncoder(buf)}, 50)
 	}
 	}
 }
 }
+
+func TestManualPartitioner(t *testing.T) {
+	partitioner := NewManualPartitioner()
+
+	choice, err := partitioner.Partition(&ProducerMessage{}, 1)
+	if err != nil {
+		t.Error(partitioner, err)
+	}
+	if choice != 0 {
+		t.Error("Returned non-zero partition when only one available.")
+	}
+
+	for i := int32(1); i < 50; i++ {
+		choice, err := partitioner.Partition(&ProducerMessage{Partition: i}, 50)
+		if err != nil {
+			t.Error(partitioner, err)
+		}
+		if choice != i {
+			t.Error("Returned partition not the same as the input partition")
+		}
+	}
+}

+ 1 - 1
sarama.go

@@ -1,5 +1,5 @@
 /*
 /*
-Package sarama provides client libraries for the Kafka 0.8 protocol. The Producer object is the high-level
+Package sarama provides client libraries for the Kafka 0.8 protocol. The AsyncProducer object is the high-level
 API for producing messages asynchronously; the SyncProducer provides a blocking API for the same purpose.
 API for producing messages asynchronously; the SyncProducer provides a blocking API for the same purpose.
 The Consumer object is the high-level API for consuming messages. The Client object provides metadata
 The Consumer object is the high-level API for consuming messages. The Client object provides metadata
 management functionality that is shared between the higher-level objects.
 management functionality that is shared between the higher-level objects.

+ 8 - 8
sync_producer.go

@@ -17,29 +17,29 @@ type SyncProducer interface {
 }
 }
 
 
 type syncProducer struct {
 type syncProducer struct {
-	producer *producer
+	producer *asyncProducer
 	wg       sync.WaitGroup
 	wg       sync.WaitGroup
 }
 }
 
 
 // NewSyncProducer creates a new SyncProducer using the given broker addresses and configuration.
 // NewSyncProducer creates a new SyncProducer using the given broker addresses and configuration.
 func NewSyncProducer(addrs []string, config *Config) (SyncProducer, error) {
 func NewSyncProducer(addrs []string, config *Config) (SyncProducer, error) {
-	p, err := NewProducer(addrs, config)
+	p, err := NewAsyncProducer(addrs, config)
 	if err != nil {
 	if err != nil {
 		return nil, err
 		return nil, err
 	}
 	}
-	return newSyncProducerFromProducer(p.(*producer)), nil
+	return newSyncProducerFromAsyncProducer(p.(*asyncProducer)), nil
 }
 }
 
 
 // NewSyncProducerFromClient creates a new SyncProducer using the given client.
 // NewSyncProducerFromClient creates a new SyncProducer using the given client.
 func NewSyncProducerFromClient(client Client) (SyncProducer, error) {
 func NewSyncProducerFromClient(client Client) (SyncProducer, error) {
-	p, err := NewProducerFromClient(client)
+	p, err := NewAsyncProducerFromClient(client)
 	if err != nil {
 	if err != nil {
 		return nil, err
 		return nil, err
 	}
 	}
-	return newSyncProducerFromProducer(p.(*producer)), nil
+	return newSyncProducerFromAsyncProducer(p.(*asyncProducer)), nil
 }
 }
 
 
-func newSyncProducerFromProducer(p *producer) *syncProducer {
+func newSyncProducerFromAsyncProducer(p *asyncProducer) *syncProducer {
 	p.conf.Producer.Return.Successes = true
 	p.conf.Producer.Return.Successes = true
 	p.conf.Producer.Return.Errors = true
 	p.conf.Producer.Return.Errors = true
 	sp := &syncProducer{producer: p}
 	sp := &syncProducer{producer: p}
@@ -56,8 +56,8 @@ func (sp *syncProducer) SendMessage(topic string, key, value Encoder) (partition
 	msg := &ProducerMessage{Topic: topic, Key: key, Value: value, Metadata: expectation}
 	msg := &ProducerMessage{Topic: topic, Key: key, Value: value, Metadata: expectation}
 	sp.producer.Input() <- msg
 	sp.producer.Input() <- msg
 	err = <-expectation
 	err = <-expectation
-	partition = msg.Partition()
-	offset = msg.Offset()
+	partition = msg.Partition
+	offset = msg.Offset
 	return
 	return
 }
 }