瀏覽代碼

Rename Producer to AsyncProducer

Willem van Bergen 10 年之前
父節點
當前提交
51060e6515
共有 7 個文件被更改,包括 69 次插入69 次删除
  1. 31 31
      async_producer.go
  2. 11 11
      async_producer_test.go
  3. 1 1
      errors.go
  4. 2 2
      functional_test.go
  5. 12 12
      mocks/async_producer.go
  6. 6 6
      mocks/async_producer_test.go
  7. 6 6
      sync_producer.go

+ 31 - 31
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?
 }
 
-// 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
 // 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 Producer interface {
+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
@@ -47,7 +47,7 @@ type Producer interface {
 	Errors() <-chan *ProducerError
 }
 
-type producer struct {
+type asyncProducer struct {
 	client    Client
 	conf      *Config
 	ownClient bool
@@ -59,29 +59,29 @@ type producer struct {
 	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)
 	if err != nil {
 		return nil, err
 	}
 
-	p, err := NewProducerFromClient(client)
+	p, err := NewAsyncProducerFromClient(client)
 	if err != nil {
 		return nil, err
 	}
-	p.(*producer).ownClient = true
+	p.(*asyncProducer).ownClient = true
 	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
 	if client.Closed() {
 		return nil, ErrClosedClient
 	}
 
-	p := &producer{
+	p := &asyncProducer{
 		client:    client,
 		conf:      client.Config(),
 		errors:    make(chan *ProducerError),
@@ -164,19 +164,19 @@ func (pe ProducerErrors) Error() string {
 	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
 }
 
-func (p *producer) Successes() <-chan *ProducerMessage {
+func (p *asyncProducer) Successes() <-chan *ProducerMessage {
 	return p.successes
 }
 
-func (p *producer) Input() chan<- *ProducerMessage {
+func (p *asyncProducer) Input() chan<- *ProducerMessage {
 	return p.input
 }
 
-func (p *producer) Close() error {
+func (p *asyncProducer) Close() error {
 	p.AsyncClose()
 
 	if p.conf.Producer.Return.Successes {
@@ -199,7 +199,7 @@ func (p *producer) Close() error {
 	return nil
 }
 
-func (p *producer) AsyncClose() {
+func (p *asyncProducer) AsyncClose() {
 	go withRecover(func() {
 		p.input <- &ProducerMessage{flags: shutdown}
 	})
@@ -214,7 +214,7 @@ func (p *producer) AsyncClose() {
 
 // singleton
 // dispatches messages by topic
-func (p *producer) topicDispatcher() {
+func (p *asyncProducer) topicDispatcher() {
 	handlers := make(map[string]chan *ProducerMessage)
 
 	for msg := range p.input {
@@ -270,7 +270,7 @@ func (p *producer) topicDispatcher() {
 
 // one per topic
 // 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)
 	partitioner := p.conf.Producer.Partitioner()
 
@@ -306,7 +306,7 @@ func (p *producer) partitionDispatcher(topic string, input chan *ProducerMessage
 // one per partition per topic
 // dispatches messages to the appropriate broker
 // 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 output chan *ProducerMessage
 
@@ -423,7 +423,7 @@ func (p *producer) leaderDispatcher(topic string, partition int32, input chan *P
 // 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 *producer) messageAggregator(broker *Broker, input chan *ProducerMessage) {
+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 {
@@ -483,7 +483,7 @@ shutdown:
 
 // one per 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
 	currentRetries := make(map[string]map[int32]error)
 	Logger.Printf("producer/flusher/%d starting up\n", broker.ID())
@@ -589,7 +589,7 @@ func (p *producer) flusher(broker *Broker, input chan []*ProducerMessage) {
 // 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 *producer) retryHandler() {
+func (p *asyncProducer) retryHandler() {
 	var buf []*ProducerMessage
 	var msg *ProducerMessage
 	refs := 0
@@ -636,7 +636,7 @@ func (p *producer) retryHandler() {
 
 // utility functions
 
-func (p *producer) assignPartition(partitioner Partitioner, msg *ProducerMessage) error {
+func (p *asyncProducer) assignPartition(partitioner Partitioner, msg *ProducerMessage) error {
 	var partitions []int32
 	var err error
 
@@ -669,7 +669,7 @@ func (p *producer) assignPartition(partitioner Partitioner, msg *ProducerMessage
 	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)}
 	empty := true
@@ -731,7 +731,7 @@ func (p *producer) buildRequest(batch map[string]map[int32][]*ProducerMessage) *
 	return req
 }
 
-func (p *producer) returnError(msg *ProducerMessage, err error) {
+func (p *asyncProducer) returnError(msg *ProducerMessage, err error) {
 	msg.flags = 0
 	msg.retries = 0
 	pErr := &ProducerError{Msg: msg, Err: err}
@@ -742,7 +742,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 {
 		if msg != nil {
 			p.returnError(msg, err)
@@ -750,7 +750,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 {
 		if msg != nil {
 			msg.flags = 0
@@ -759,7 +759,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 {
 		if msg == nil {
 			continue
@@ -778,7 +778,7 @@ type brokerProducer struct {
 	refs  int
 }
 
-func (p *producer) getBrokerProducer(broker *Broker) chan *ProducerMessage {
+func (p *asyncProducer) getBrokerProducer(broker *Broker) chan *ProducerMessage {
 	p.brokerLock.Lock()
 	defer p.brokerLock.Unlock()
 
@@ -799,7 +799,7 @@ func (p *producer) getBrokerProducer(broker *Broker) chan *ProducerMessage {
 	return producer.input
 }
 
-func (p *producer) unrefBrokerProducer(broker *Broker) {
+func (p *asyncProducer) unrefBrokerProducer(broker *Broker) {
 	p.brokerLock.Lock()
 	defer p.brokerLock.Unlock()
 

+ 11 - 11
producer_test.go → async_producer_test.go

@@ -10,7 +10,7 @@ import (
 
 const TestMessage = "ABC THE MESSAGE"
 
-func closeProducer(t *testing.T, p Producer) {
+func closeProducer(t *testing.T, p AsyncProducer) {
 	var wg sync.WaitGroup
 	p.AsyncClose()
 
@@ -126,7 +126,7 @@ func TestProducer(t *testing.T) {
 	config := NewConfig()
 	config.Producer.Flush.Messages = 10
 	config.Producer.Return.Successes = true
-	producer, err := NewProducer([]string{seedBroker.Addr()}, config)
+	producer, err := NewAsyncProducer([]string{seedBroker.Addr()}, config)
 	if err != nil {
 		t.Fatal(err)
 	}
@@ -174,7 +174,7 @@ func TestProducerMultipleFlushes(t *testing.T) {
 	config := NewConfig()
 	config.Producer.Flush.Messages = 5
 	config.Producer.Return.Successes = true
-	producer, err := NewProducer([]string{seedBroker.Addr()}, config)
+	producer, err := NewAsyncProducer([]string{seedBroker.Addr()}, config)
 	if err != nil {
 		t.Fatal(err)
 	}
@@ -227,7 +227,7 @@ func TestProducerMultipleBrokers(t *testing.T) {
 	config.Producer.Flush.Messages = 5
 	config.Producer.Return.Successes = true
 	config.Producer.Partitioner = NewRoundRobinPartitioner
-	producer, err := NewProducer([]string{seedBroker.Addr()}, config)
+	producer, err := NewAsyncProducer([]string{seedBroker.Addr()}, config)
 	if err != nil {
 		t.Fatal(err)
 	}
@@ -269,7 +269,7 @@ func TestProducerFailureRetry(t *testing.T) {
 	config.Producer.Flush.Messages = 10
 	config.Producer.Return.Successes = true
 	config.Producer.Retry.Backoff = 0
-	producer, err := NewProducer([]string{seedBroker.Addr()}, config)
+	producer, err := NewAsyncProducer([]string{seedBroker.Addr()}, config)
 	if err != nil {
 		t.Fatal(err)
 	}
@@ -341,7 +341,7 @@ func TestProducerBrokerBounce(t *testing.T) {
 	config.Producer.Flush.Messages = 10
 	config.Producer.Return.Successes = true
 	config.Producer.Retry.Backoff = 0
-	producer, err := NewProducer([]string{seedBroker.Addr()}, config)
+	producer, err := NewAsyncProducer([]string{seedBroker.Addr()}, config)
 	if err != nil {
 		t.Fatal(err)
 	}
@@ -390,7 +390,7 @@ func TestProducerBrokerBounceWithStaleMetadata(t *testing.T) {
 	config.Producer.Return.Successes = true
 	config.Producer.Retry.Max = 3
 	config.Producer.Retry.Backoff = 0
-	producer, err := NewProducer([]string{seedBroker.Addr()}, config)
+	producer, err := NewAsyncProducer([]string{seedBroker.Addr()}, config)
 	if err != nil {
 		t.Fatal(err)
 	}
@@ -445,7 +445,7 @@ func TestProducerMultipleRetries(t *testing.T) {
 	config.Producer.Return.Successes = true
 	config.Producer.Retry.Max = 4
 	config.Producer.Retry.Backoff = 0
-	producer, err := NewProducer([]string{seedBroker.Addr()}, config)
+	producer, err := NewAsyncProducer([]string{seedBroker.Addr()}, config)
 	if err != nil {
 		t.Fatal(err)
 	}
@@ -525,7 +525,7 @@ func TestProducerOutOfRetries(t *testing.T) {
 	config.Producer.Return.Successes = true
 	config.Producer.Retry.Backoff = 0
 	config.Producer.Retry.Max = 0
-	producer, err := NewProducer([]string{seedBroker.Addr()}, config)
+	producer, err := NewAsyncProducer([]string{seedBroker.Addr()}, config)
 	if err != nil {
 		t.Fatal(err)
 	}
@@ -575,7 +575,7 @@ func TestProducerOutOfRetries(t *testing.T) {
 // This example shows how to use the producer while simultaneously
 // reading the Errors channel to know about any failures.
 func ExampleProducer_select() {
-	producer, err := NewProducer([]string{"localhost:9092"}, nil)
+	producer, err := NewAsyncProducer([]string{"localhost:9092"}, nil)
 	if err != nil {
 		panic(err)
 	}
@@ -614,7 +614,7 @@ ProducerLoop:
 func ExampleProducer_goroutines() {
 	config := NewConfig()
 	config.Producer.Return.Successes = true
-	producer, err := NewProducer([]string{"localhost:9092"}, config)
+	producer, err := NewAsyncProducer([]string{"localhost:9092"}, config)
 	if err != nil {
 		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)
 }
 
-// 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.
 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.Messages = 200
 	config.Producer.Return.Successes = true
-	producer, err := NewProducer([]string{kafkaAddr}, config)
+	producer, err := NewAsyncProducer([]string{kafkaAddr}, config)
 	if err != nil {
 		t.Fatal(err)
 	}
@@ -139,7 +139,7 @@ func testProducingMessages(t *testing.T, config *Config) {
 		t.Fatal(err)
 	}
 
-	producer, err := NewProducerFromClient(client)
+	producer, err := NewAsyncProducerFromClient(client)
 	if err != nil {
 		t.Fatal(err)
 	}

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

@@ -6,11 +6,11 @@ import (
 	"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
 // so it knows how to handle the input. This way you can easily test success and
 // failure scenarios.
-type Producer struct {
+type AsyncProducer struct {
 	l            sync.Mutex
 	t            ErrorReporter
 	expectations []*producerExpectation
@@ -20,15 +20,15 @@ type Producer struct {
 	errors       chan *sarama.ProducerError
 }
 
-// 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
 // an expectation is violated. The config argument is used to determine whether it
 // 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 {
 		config = sarama.NewConfig()
 	}
-	mp := &Producer{
+	mp := &AsyncProducer{
 		t:            t,
 		closed:       make(chan struct{}, 0),
 		expectations: make([]*producerExpectation, 0),
@@ -83,14 +83,14 @@ func NewProducer(t ErrorReporter, config *sarama.Config) *Producer {
 // 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
 // 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 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
 // 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.closed
 	return nil
@@ -101,17 +101,17 @@ func (mp *Producer) Close() error {
 // 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
 // state object.
-func (mp *Producer) Input() chan<- *sarama.ProducerMessage {
+func (mp *AsyncProducer) Input() chan<- *sarama.ProducerMessage {
 	return mp.input
 }
 
 // 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
 }
 
 // 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
 }
 
@@ -123,7 +123,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,
 // i.e. it will make it available on the Successes channel if the Producer.Return.Successes setting
 // is set to true.
-func (mp *Producer) ExpectInputAndSucceed() {
+func (mp *AsyncProducer) ExpectInputAndSucceed() {
 	mp.l.Lock()
 	defer mp.l.Unlock()
 	mp.expectations = append(mp.expectations, &producerExpectation{Result: errProduceSuccess})
@@ -132,7 +132,7 @@ func (mp *Producer) ExpectInputAndSucceed() {
 // 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
 // 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()
 	defer mp.l.Unlock()
 	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...))
 }
 
-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.")
 	}
 }
@@ -29,7 +29,7 @@ func TestMockProducerImplementsProducerInterface(t *testing.T) {
 func TestProducerReturnsExpectationsToChannels(t *testing.T) {
 	config := sarama.NewConfig()
 	config.Producer.Return.Successes = true
-	mp := NewProducer(t, config)
+	mp := NewAsyncProducer(t, config)
 
 	mp.ExpectInputAndSucceed()
 	mp.ExpectInputAndSucceed()
@@ -62,7 +62,7 @@ func TestProducerReturnsExpectationsToChannels(t *testing.T) {
 
 func TestProducerWithTooFewExpectations(t *testing.T) {
 	trm := newTestReporterMock()
-	mp := NewProducer(trm, nil)
+	mp := NewAsyncProducer(trm, nil)
 	mp.ExpectInputAndSucceed()
 
 	mp.Input() <- &sarama.ProducerMessage{Topic: "test"}
@@ -79,7 +79,7 @@ func TestProducerWithTooFewExpectations(t *testing.T) {
 
 func TestProducerWithTooManyExpectations(t *testing.T) {
 	trm := newTestReporterMock()
-	mp := NewProducer(trm, nil)
+	mp := NewAsyncProducer(trm, nil)
 	mp.ExpectInputAndSucceed()
 	mp.ExpectInputAndFail(sarama.ErrOutOfBrokers)
 

+ 6 - 6
sync_producer.go

@@ -17,29 +17,29 @@ type SyncProducer interface {
 }
 
 type syncProducer struct {
-	producer *producer
+	producer *asyncProducer
 	wg       sync.WaitGroup
 }
 
 // NewSyncProducer creates a new SyncProducer using the given broker addresses and configuration.
 func NewSyncProducer(addrs []string, config *Config) (SyncProducer, error) {
-	p, err := NewProducer(addrs, config)
+	p, err := NewAsyncProducer(addrs, config)
 	if err != nil {
 		return nil, err
 	}
-	return newSyncProducerFromProducer(p.(*producer)), nil
+	return newSyncProducerFromAsyncProducer(p.(*asyncProducer)), nil
 }
 
 // NewSyncProducerFromClient creates a new SyncProducer using the given client.
 func NewSyncProducerFromClient(client Client) (SyncProducer, error) {
-	p, err := NewProducerFromClient(client)
+	p, err := NewAsyncProducerFromClient(client)
 	if err != nil {
 		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.Errors = true
 	sp := &syncProducer{producer: p}