From 9a1476264a50047ea7547c4358b72b83b7be3fa2 Mon Sep 17 00:00:00 2001 From: Joe Wreschnig Date: Tue, 6 Sep 2022 11:38:46 +0200 Subject: [PATCH] Validate the `Config` when creating a mock producer/consumer Normally this happens during creation of the `Client`, but for mock interfaces there is no `Client`. --- mocks/async_producer.go | 7 +++++-- mocks/async_producer_test.go | 15 +++++++++++++++ mocks/consumer.go | 6 +++++- mocks/consumer_test.go | 17 +++++++++++++++++ mocks/sync_producer.go | 6 +++++- mocks/sync_producer_test.go | 16 ++++++++++++++++ 6 files changed, 63 insertions(+), 4 deletions(-) diff --git a/mocks/async_producer.go b/mocks/async_producer.go index cdef5bfe7..4381f04ee 100644 --- a/mocks/async_producer.go +++ b/mocks/async_producer.go @@ -30,12 +30,15 @@ type AsyncProducer struct { // 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 and to handle partitioning. +// an expectation is violated. The config argument is validated and used to determine +// whether it should ack successes on the Successes channel and handle partitioning. func NewAsyncProducer(t ErrorReporter, config *sarama.Config) *AsyncProducer { if config == nil { config = sarama.NewConfig() } + if err := config.Validate(); err != nil { + t.Errorf("Invalid mock configuration provided: %s", err.Error()) + } mp := &AsyncProducer{ t: t, closed: make(chan struct{}), diff --git a/mocks/async_producer_test.go b/mocks/async_producer_test.go index 69eedf2a1..83e6d93ae 100644 --- a/mocks/async_producer_test.go +++ b/mocks/async_producer_test.go @@ -247,3 +247,18 @@ func (brokePartitioner) Partition(msg *sarama.ProducerMessage, n int32) (int32, } func (brokePartitioner) RequiresConsistency() bool { return false } + +func TestProducerWithInvalidConfiguration(t *testing.T) { + trm := newTestReporterMock() + config := NewTestConfig() + config.ClientID = "not a valid client ID" + mp := NewAsyncProducer(trm, config) + if err := mp.Close(); err != nil { + t.Error(err) + } + if len(trm.errors) != 1 { + t.Error("Expected to report a single error") + } else if !strings.Contains(trm.errors[0], "ClientID is invalid") { + t.Errorf("Unexpected error: %s", trm.errors[0]) + } +} diff --git a/mocks/consumer.go b/mocks/consumer.go index d8a212221..37a07cf38 100644 --- a/mocks/consumer.go +++ b/mocks/consumer.go @@ -20,11 +20,15 @@ type Consumer struct { // NewConsumer returns a new mock Consumer instance. 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 can be set to nil. +// an expectation is violated. The config argument can be set to nil; if it is +// non-nil it is validated. func NewConsumer(t ErrorReporter, config *sarama.Config) *Consumer { if config == nil { config = sarama.NewConfig() } + if err := config.Validate(); err != nil { + t.Errorf("Invalid mock configuration provided: %s", err.Error()) + } c := &Consumer{ t: t, diff --git a/mocks/consumer_test.go b/mocks/consumer_test.go index 9a19b1596..2aa961464 100644 --- a/mocks/consumer_test.go +++ b/mocks/consumer_test.go @@ -3,6 +3,7 @@ package mocks import ( "errors" "sort" + "strings" "testing" "github.com/Shopify/sarama" @@ -395,3 +396,19 @@ func TestConsumerOffsetsAreManagedCorrectlyWithSpecifiedOffset(t *testing.T) { t.Errorf("Expected to not report any errors, found: %v", trm.errors) } } + +func TestConsumerInvalidConfiguration(t *testing.T) { + trm := newTestReporterMock() + config := NewTestConfig() + config.ClientID = "not a valid client ID" + consumer := NewConsumer(trm, config) + if err := consumer.Close(); err != nil { + t.Error(err) + } + + if len(trm.errors) != 1 { + t.Error("Expected to report a single error") + } else if !strings.Contains(trm.errors[0], "ClientID is invalid") { + t.Errorf("Unexpected error: %s", trm.errors[0]) + } +} diff --git a/mocks/sync_producer.go b/mocks/sync_producer.go index 0acf10a6e..668abda7a 100644 --- a/mocks/sync_producer.go +++ b/mocks/sync_producer.go @@ -28,11 +28,15 @@ type SyncProducer struct { // NewSyncProducer instantiates a new SyncProducer 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 handle partitioning. +// an expectation is violated. The config argument is validated and used to handle +// partitioning. func NewSyncProducer(t ErrorReporter, config *sarama.Config) *SyncProducer { if config == nil { config = sarama.NewConfig() } + if err := config.Validate(); err != nil { + t.Errorf("Invalid mock configuration provided: %s", err.Error()) + } return &SyncProducer{ t: t, expectations: make([]*producerExpectation, 0), diff --git a/mocks/sync_producer_test.go b/mocks/sync_producer_test.go index 3003711f9..4544f1c1e 100644 --- a/mocks/sync_producer_test.go +++ b/mocks/sync_producer_test.go @@ -352,3 +352,19 @@ func (f faultyEncoder) Encode() ([]byte, error) { func (f faultyEncoder) Length() int { return len(f) } + +func TestSyncProducerInvalidConfiguration(t *testing.T) { + trm := newTestReporterMock() + config := NewTestConfig() + config.ClientID = "not a valid client ID" + mp := NewSyncProducer(trm, config) + if err := mp.Close(); err != nil { + t.Error(err) + } + + if len(trm.errors) != 1 { + t.Error("Expected to report a single error") + } else if !strings.Contains(trm.errors[0], "ClientID is invalid") { + t.Errorf("Unexpected error: %s", trm.errors[0]) + } +}