Fix pubsub retry for Kafka (#688)
* Add conformance test for kafka pubsub + quickfix for Kafka error handling. * Adding backoff/retry logic to the Kafka pub/sub component * Add logic to return if error from backoff in kafka. * Addressing PR comments and lint. Co-authored-by: Phil Kedy <phil.kedy@gmail.com>
This commit is contained in:
parent
b10d346bde
commit
dd3e463b52
|
@ -32,6 +32,7 @@ jobs:
|
|||
- bindings.kafka
|
||||
- bindings.redis
|
||||
- pubsub.redis
|
||||
- pubsub.kafka
|
||||
- secretstores.localenv
|
||||
- secretstores.localfile
|
||||
- state.mongodb
|
||||
|
|
1
go.mod
1
go.mod
|
@ -25,6 +25,7 @@ require (
|
|||
github.com/apache/pulsar-client-go v0.1.0
|
||||
github.com/aws/aws-sdk-go v1.27.0
|
||||
github.com/bradfitz/gomemcache v0.0.0-20190913173617-a41fca850d0b
|
||||
github.com/cenkalti/backoff/v4 v4.1.0
|
||||
github.com/coreos/go-oidc v2.1.0+incompatible
|
||||
github.com/dancannon/gorethink v4.0.0+incompatible
|
||||
github.com/dapr/dapr v1.0.0-rc.2
|
||||
|
|
2
go.sum
2
go.sum
|
@ -245,6 +245,8 @@ github.com/cenkalti/backoff v2.0.0+incompatible/go.mod h1:90ReRw6GdpyfrHakVjL/QH
|
|||
github.com/cenkalti/backoff v2.1.1+incompatible/go.mod h1:90ReRw6GdpyfrHakVjL/QHaoyV4aDUVVkXQJJJ3NXXM=
|
||||
github.com/cenkalti/backoff v2.2.1+incompatible h1:tNowT99t7UNflLxfYYSlKYsBpXdEet03Pg2g16Swow4=
|
||||
github.com/cenkalti/backoff v2.2.1+incompatible/go.mod h1:90ReRw6GdpyfrHakVjL/QHaoyV4aDUVVkXQJJJ3NXXM=
|
||||
github.com/cenkalti/backoff/v4 v4.1.0 h1:c8LkOFQTzuO0WBM/ae5HdGQuZPfPxp7lqBRwQRm4fSc=
|
||||
github.com/cenkalti/backoff/v4 v4.1.0/go.mod h1:scbssz8iZGpm3xbr14ovlUdkxfGXNInqkPWOWmG2CLw=
|
||||
github.com/census-instrumentation/opencensus-proto v0.2.0/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU=
|
||||
github.com/census-instrumentation/opencensus-proto v0.2.1 h1:glEXhBS5PSLLv4IXzLA5yPRVX4bilULVyxxbrfOtDAk=
|
||||
github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU=
|
||||
|
|
|
@ -13,8 +13,10 @@ import (
|
|||
"strconv"
|
||||
"strings"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/Shopify/sarama"
|
||||
"github.com/cenkalti/backoff/v4"
|
||||
"github.com/dapr/components-contrib/pubsub"
|
||||
"github.com/dapr/dapr/pkg/logger"
|
||||
)
|
||||
|
@ -36,6 +38,7 @@ type Kafka struct {
|
|||
topics map[string]bool
|
||||
cancel context.CancelFunc
|
||||
consumer consumer
|
||||
backOff backoff.BackOff
|
||||
config *sarama.Config
|
||||
}
|
||||
|
||||
|
@ -49,21 +52,44 @@ type kafkaMetadata struct {
|
|||
}
|
||||
|
||||
type consumer struct {
|
||||
logger logger.Logger
|
||||
backOff backoff.BackOff
|
||||
ready chan bool
|
||||
callback func(msg *pubsub.NewMessage) error
|
||||
once sync.Once
|
||||
}
|
||||
|
||||
func (consumer *consumer) ConsumeClaim(session sarama.ConsumerGroupSession, claim sarama.ConsumerGroupClaim) error {
|
||||
bo := backoff.WithContext(consumer.backOff, session.Context())
|
||||
for message := range claim.Messages() {
|
||||
if consumer.callback != nil {
|
||||
if consumer.callback == nil {
|
||||
return fmt.Errorf("nil consumer callback")
|
||||
}
|
||||
|
||||
var warningLogged bool
|
||||
err := backoff.RetryNotify(func() error {
|
||||
err := consumer.callback(&pubsub.NewMessage{
|
||||
Topic: claim.Topic(),
|
||||
Data: message.Value,
|
||||
})
|
||||
if err == nil {
|
||||
session.MarkMessage(message, "")
|
||||
if warningLogged {
|
||||
consumer.logger.Infof("Kafka message processed successfully after previously failing: %s/%d [key=%s]", message.Topic, message.Partition, message.Key)
|
||||
warningLogged = false
|
||||
}
|
||||
}
|
||||
|
||||
return err
|
||||
}, bo, func(err error, d time.Duration) {
|
||||
if !warningLogged {
|
||||
consumer.logger.Warnf("Encountered error processing Kafka message: %s/%d [key=%s]. Retrying...", message.Topic, message.Partition, message.Key)
|
||||
warningLogged = true
|
||||
}
|
||||
})
|
||||
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -119,6 +145,9 @@ func (k *Kafka) Init(metadata pubsub.Metadata) error {
|
|||
|
||||
k.topics = make(map[string]bool)
|
||||
|
||||
// TODO: Make the backoff configurable for constant or exponential
|
||||
k.backOff = backoff.NewConstantBackOff(5 * time.Second)
|
||||
|
||||
k.logger.Debug("Kafka message bus initialization complete")
|
||||
|
||||
return nil
|
||||
|
@ -199,6 +228,8 @@ func (k *Kafka) Subscribe(req pubsub.SubscribeRequest, handler func(msg *pubsub.
|
|||
|
||||
ready := make(chan bool)
|
||||
k.consumer = consumer{
|
||||
logger: k.logger,
|
||||
backOff: k.backOff,
|
||||
ready: ready,
|
||||
callback: handler,
|
||||
}
|
||||
|
@ -215,6 +246,7 @@ func (k *Kafka) Subscribe(req pubsub.SubscribeRequest, handler func(msg *pubsub.
|
|||
k.logger.Debugf("Subscribed and listening to topics: %s", topics)
|
||||
|
||||
for {
|
||||
k.logger.Debugf("Starting loop to consume.")
|
||||
// Consume the requested topic
|
||||
innerError := k.cg.Consume(ctx, topics, &(k.consumer))
|
||||
if innerError != nil {
|
||||
|
@ -224,6 +256,8 @@ func (k *Kafka) Subscribe(req pubsub.SubscribeRequest, handler func(msg *pubsub.
|
|||
// If the context was cancelled, as is the case when handling SIGINT and SIGTERM below, then this pops
|
||||
// us out of the consume loop
|
||||
if ctx.Err() != nil {
|
||||
k.logger.Debugf("Context error, stopping consumer: %v", ctx.Err())
|
||||
|
||||
return
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,14 @@
|
|||
apiVersion: dapr.io/v1alpha1
|
||||
kind: Component
|
||||
metadata:
|
||||
name: pubsub
|
||||
spec:
|
||||
type: pubsub.kafka
|
||||
version: v1
|
||||
metadata:
|
||||
- name: brokers
|
||||
value: localhost:9092
|
||||
- name: consumerID
|
||||
value: pubsubgroup1
|
||||
- name: authRequired
|
||||
value: "false"
|
|
@ -5,7 +5,8 @@ components:
|
|||
config:
|
||||
pubsubName: azure-servicebus
|
||||
testTopicName: dapr-conf-test
|
||||
maxReadDuration: 1000
|
||||
- component: redis
|
||||
allOperations: true
|
||||
- component: kafka
|
||||
allOperations: true
|
||||
|
|
@ -25,6 +25,7 @@ import (
|
|||
b_redis "github.com/dapr/components-contrib/bindings/redis"
|
||||
"github.com/dapr/components-contrib/pubsub"
|
||||
p_servicebus "github.com/dapr/components-contrib/pubsub/azure/servicebus"
|
||||
p_kafka "github.com/dapr/components-contrib/pubsub/kafka"
|
||||
p_redis "github.com/dapr/components-contrib/pubsub/redis"
|
||||
"github.com/dapr/components-contrib/secretstores"
|
||||
ss_azure "github.com/dapr/components-contrib/secretstores/azure/keyvault"
|
||||
|
@ -50,6 +51,7 @@ import (
|
|||
|
||||
const (
|
||||
redis = "redis"
|
||||
kafka = "kafka"
|
||||
generateUUID = "$((uuid))"
|
||||
)
|
||||
|
||||
|
@ -261,6 +263,8 @@ func loadPubSub(tc TestComponent) pubsub.PubSub {
|
|||
pubsub = p_redis.NewRedisStreams(testLogger)
|
||||
case "azure.servicebus":
|
||||
pubsub = p_servicebus.NewAzureServiceBus(testLogger)
|
||||
case kafka:
|
||||
pubsub = p_kafka.NewKafka(testLogger)
|
||||
default:
|
||||
return nil
|
||||
}
|
||||
|
@ -314,7 +318,7 @@ func loadOutputBindings(tc TestComponent) bindings.OutputBinding {
|
|||
binding = b_azure_servicebusqueues.NewAzureServiceBusQueues(testLogger)
|
||||
case "azure.eventgrid":
|
||||
binding = b_azure_eventgrid.NewAzureEventGrid(testLogger)
|
||||
case "kafka":
|
||||
case kafka:
|
||||
binding = b_kafka.NewKafka(testLogger)
|
||||
case "http":
|
||||
binding = b_http.NewHTTP(testLogger)
|
||||
|
@ -335,7 +339,7 @@ func loadInputBindings(tc TestComponent) bindings.InputBinding {
|
|||
binding = b_azure_storagequeues.NewAzureStorageQueues(testLogger)
|
||||
case "azure.eventgrid":
|
||||
binding = b_azure_eventgrid.NewAzureEventGrid(testLogger)
|
||||
case "kafka":
|
||||
case kafka:
|
||||
binding = b_kafka.NewKafka(testLogger)
|
||||
default:
|
||||
return nil
|
||||
|
|
|
@ -13,25 +13,28 @@ import (
|
|||
|
||||
"github.com/dapr/components-contrib/pubsub"
|
||||
"github.com/dapr/components-contrib/tests/conformance/utils"
|
||||
"github.com/pkg/errors"
|
||||
"github.com/stretchr/testify/assert"
|
||||
"k8s.io/apimachinery/pkg/util/sets"
|
||||
)
|
||||
|
||||
const (
|
||||
defaultPubsubName = "pubusub"
|
||||
defaultTopicName = "testTopic"
|
||||
defaultMessageCount = 10
|
||||
defaultMaxReadDuration = 10 * time.Millisecond
|
||||
defaultPubsubName = "pubusub"
|
||||
defaultTopicName = "testTopic"
|
||||
defaultMessageCount = 10
|
||||
defaultMaxReadDuration = 20 * time.Second
|
||||
defaultWaitDurationToPublish = 5 * time.Second
|
||||
)
|
||||
|
||||
type TestConfig struct {
|
||||
utils.CommonConfig
|
||||
pubsubName string
|
||||
testTopicName string
|
||||
publishMetadata map[string]string
|
||||
subscribeMetadata map[string]string
|
||||
messageCount int
|
||||
maxReadDuration time.Duration
|
||||
pubsubName string
|
||||
testTopicName string
|
||||
publishMetadata map[string]string
|
||||
subscribeMetadata map[string]string
|
||||
messageCount int
|
||||
maxReadDuration time.Duration
|
||||
waitDurationToPublish time.Duration
|
||||
}
|
||||
|
||||
func NewTestConfig(componentName string, allOperations bool, operations []string, config map[string]string) TestConfig {
|
||||
|
@ -41,12 +44,13 @@ func NewTestConfig(componentName string, allOperations bool, operations []string
|
|||
ComponentName: componentName,
|
||||
AllOperations: allOperations,
|
||||
Operations: sets.NewString(operations...)},
|
||||
pubsubName: defaultPubsubName,
|
||||
testTopicName: defaultTopicName,
|
||||
messageCount: defaultMessageCount,
|
||||
maxReadDuration: defaultMaxReadDuration,
|
||||
publishMetadata: map[string]string{},
|
||||
subscribeMetadata: map[string]string{},
|
||||
pubsubName: defaultPubsubName,
|
||||
testTopicName: defaultTopicName,
|
||||
messageCount: defaultMessageCount,
|
||||
maxReadDuration: defaultMaxReadDuration,
|
||||
waitDurationToPublish: defaultWaitDurationToPublish,
|
||||
publishMetadata: map[string]string{},
|
||||
subscribeMetadata: map[string]string{},
|
||||
}
|
||||
for k, v := range config {
|
||||
if k == "pubsubName" {
|
||||
|
@ -67,6 +71,12 @@ func NewTestConfig(componentName string, allOperations bool, operations []string
|
|||
tc.maxReadDuration = time.Duration(val) * time.Millisecond
|
||||
}
|
||||
}
|
||||
if k == "waitDurationToPublish" {
|
||||
val, err := strconv.Atoi(v)
|
||||
if err == nil {
|
||||
tc.waitDurationToPublish = time.Duration(val) * time.Millisecond
|
||||
}
|
||||
}
|
||||
if strings.HasPrefix(k, "publish_") {
|
||||
tc.publishMetadata[strings.Replace(k, "publish_", "", 1)] = v
|
||||
}
|
||||
|
@ -92,6 +102,7 @@ func ConformanceTests(t *testing.T, props map[string]string, ps pubsub.PubSub, c
|
|||
assert.NoError(t, err, "expected no error on setting up pubsub")
|
||||
})
|
||||
|
||||
errorCount := 0
|
||||
// Subscribe
|
||||
if config.HasOperation("subscribe") {
|
||||
t.Run("subscribe", func(t *testing.T) {
|
||||
|
@ -99,6 +110,19 @@ func ConformanceTests(t *testing.T, props map[string]string, ps pubsub.PubSub, c
|
|||
Topic: config.testTopicName,
|
||||
Metadata: config.subscribeMetadata,
|
||||
}, func(_ *pubsub.NewMessage) error {
|
||||
// This behavior is standard to repro a failure of one message in a batch.
|
||||
if errorCount < 2 {
|
||||
// First message errors just to give time for more messages to pile up.
|
||||
// Second error is to force an error in a batch.
|
||||
errorCount++
|
||||
// Sleep to allow messages to pile up and be delivered as a batch.
|
||||
time.Sleep(2 * time.Second)
|
||||
t.Logf("Simulating subscriber error")
|
||||
|
||||
return errors.Errorf("conf test simulated error")
|
||||
}
|
||||
|
||||
t.Logf("Simulating subscriber success")
|
||||
actualReadCount++
|
||||
|
||||
return nil
|
||||
|
@ -109,6 +133,9 @@ func ConformanceTests(t *testing.T, props map[string]string, ps pubsub.PubSub, c
|
|||
|
||||
// Publish
|
||||
if config.HasOperation("publish") {
|
||||
// Some pubsub, like Kafka need to wait for Subscriber to be up before messages can be consumed.
|
||||
// So, wait for some time here.
|
||||
time.Sleep(config.waitDurationToPublish)
|
||||
t.Run("publish", func(t *testing.T) {
|
||||
for k := 0; k < config.messageCount; k++ {
|
||||
data := []byte("message-" + strconv.Itoa(k))
|
||||
|
|
Loading…
Reference in New Issue