Configurable retry (#854)

* no message

* Switching over usages of retry to the new package and making use of the DecodeConfig function

* Added decoding configuration using settings with a specific prefix

* Linter fixes

* Fixing linter error

* time.Sleep is the enemy

* Fix typo in comment

* Moving config to a pointer parameter so that the component can pass in the config with default values that make sense for that component.

* Renamed config struct

* Fix comment

Co-authored-by: Artur Souza <artursouza.ms@outlook.com>
This commit is contained in:
Phil Kedy 2021-05-06 12:43:57 -04:00 committed by GitHub
parent f69b9f5c34
commit 1b30e75e60
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
17 changed files with 591 additions and 93 deletions

View File

@ -19,10 +19,11 @@ import (
"time"
"github.com/cenkalti/backoff/v4"
"github.com/dapr/components-contrib/bindings"
"github.com/dapr/components-contrib/pubsub"
"github.com/dapr/kit/logger"
mqtt "github.com/eclipse/paho.mqtt.golang"
"github.com/dapr/components-contrib/bindings"
"github.com/dapr/components-contrib/internal/retry"
"github.com/dapr/kit/logger"
)
const (
@ -222,7 +223,7 @@ func (m *MQTT) Read(handler func(*bindings.ReadResponse) ([]byte, error)) error
if m.metadata.backOffMaxRetries >= 0 {
b = backoff.WithMaxRetries(m.backOff, uint64(m.metadata.backOffMaxRetries))
}
if err := pubsub.RetryNotifyRecover(func() error {
if err := retry.NotifyRecover(func() error {
m.logger.Debugf("Processing MQTT message %s/%d", mqttMsg.Topic(), mqttMsg.MessageID())
if _, err := handler(&msg); err != nil {
return err

View File

@ -33,7 +33,7 @@ type StringDecoder interface {
// Most of the heavy lifting is handled by the mapstructure library. A custom decoder is used to handle
// decoding string values to the supported primitives.
func Decode(input interface{}, output interface{}) error {
decoder, err := mapstructure.NewDecoder(&mapstructure.DecoderConfig{
decoder, err := mapstructure.NewDecoder(&mapstructure.DecoderConfig{ // nolint:exhaustivestruct
Result: output,
DecodeHook: decodeString,
})
@ -44,6 +44,7 @@ func Decode(input interface{}, output interface{}) error {
return decoder.Decode(input)
}
// nolint:cyclop
func decodeString(
f reflect.Type,
t reflect.Type,
@ -59,7 +60,10 @@ func decodeString(
return data, nil
}
dataString := data.(string)
dataString, ok := data.(string)
if !ok {
return nil, errors.Errorf("expected string: got %s", reflect.TypeOf(data))
}
var result interface{}
var decoder StringDecoder

View File

@ -3,22 +3,23 @@
// Licensed under the MIT License.
// ------------------------------------------------------------
package nameresolution
package config
import (
"fmt"
)
// ConvertConfig converts map[interface{}]interface{} to map[string]interface{} to normalize
// Normalize converts map[interface{}]interface{} to map[string]interface{} to normalize
// for JSON and usage in component initialization.
func ConvertConfig(i interface{}) (interface{}, error) {
// nolint:cyclop
func Normalize(i interface{}) (interface{}, error) {
var err error
switch x := i.(type) {
case map[interface{}]interface{}:
m2 := map[string]interface{}{}
for k, v := range x {
if strKey, ok := k.(string); ok {
if m2[strKey], err = ConvertConfig(v); err != nil {
if m2[strKey], err = Normalize(v); err != nil {
return nil, err
}
} else {
@ -30,7 +31,7 @@ func ConvertConfig(i interface{}) (interface{}, error) {
case map[string]interface{}:
m2 := map[string]interface{}{}
for k, v := range x {
if m2[k], err = ConvertConfig(v); err != nil {
if m2[k], err = Normalize(v); err != nil {
return nil, err
}
}
@ -38,7 +39,7 @@ func ConvertConfig(i interface{}) (interface{}, error) {
return m2, nil
case []interface{}:
for i, v := range x {
if x[i], err = ConvertConfig(v); err != nil {
if x[i], err = Normalize(v); err != nil {
return nil, err
}
}

View File

@ -3,17 +3,18 @@
// Licensed under the MIT License.
// ------------------------------------------------------------
package nameresolution_test
package config_test
import (
"testing"
"github.com/dapr/components-contrib/nameresolution"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
"github.com/dapr/components-contrib/internal/config"
)
func TestConfiguration(t *testing.T) {
func TestNormalize(t *testing.T) {
tests := map[string]struct {
input interface{}
expected interface{}
@ -84,7 +85,7 @@ func TestConfiguration(t *testing.T) {
for name, tc := range tests {
t.Run(name, func(t *testing.T) {
actual, err := nameresolution.ConvertConfig(tc.input)
actual, err := config.Normalize(tc.input)
if tc.err != "" {
require.Error(t, err)
assert.EqualError(t, err, tc.err)

53
internal/config/prefix.go Normal file
View File

@ -0,0 +1,53 @@
package config
import (
"strings"
"unicode"
)
func PrefixedBy(input interface{}, prefix string) (interface{}, error) {
normalized, err := Normalize(input)
if err != nil {
// The only error that can come from normalize is if
// input is a map[interface{}]interface{} and contains
// a key that is not a string.
return input, err
}
input = normalized
if inputMap, ok := input.(map[string]interface{}); ok {
converted := make(map[string]interface{}, len(inputMap))
for k, v := range inputMap {
if strings.HasPrefix(k, prefix) {
key := uncapitalize(strings.TrimPrefix(k, prefix))
converted[key] = v
}
}
return converted, nil
} else if inputMap, ok := input.(map[string]string); ok {
converted := make(map[string]string, len(inputMap))
for k, v := range inputMap {
if strings.HasPrefix(k, prefix) {
key := uncapitalize(strings.TrimPrefix(k, prefix))
converted[key] = v
}
}
return converted, nil
}
return input, nil
}
// uncapitalize initial capital letters in `str`.
func uncapitalize(str string) string {
if len(str) == 0 {
return str
}
vv := []rune(str) // Introduced later
vv[0] = unicode.ToLower(vv[0])
return string(vv)
}

View File

@ -0,0 +1,58 @@
package config_test
import (
"testing"
"github.com/stretchr/testify/assert"
"github.com/dapr/components-contrib/internal/config"
)
func TestPrefixedBy(t *testing.T) {
tests := map[string]struct {
prefix string
input interface{}
expected interface{}
err string
}{
"map of string to string": {
prefix: "test",
input: map[string]string{
"": "",
"ignore": "don't include me",
"testOne": "include me",
"testTwo": "and me",
},
expected: map[string]string{
"one": "include me",
"two": "and me",
},
},
"map of string to interface{}": {
prefix: "test",
input: map[string]interface{}{
"": "",
"ignore": "don't include me",
"testOne": "include me",
"testTwo": "and me",
},
expected: map[string]interface{}{
"one": "include me",
"two": "and me",
},
},
}
for name, tc := range tests {
t.Run(name, func(t *testing.T) {
actual, err := config.PrefixedBy(tc.input, tc.prefix)
if tc.err != "" {
if assert.Error(t, err) {
assert.Equal(t, tc.err, err.Error())
}
} else {
assert.Equal(t, tc.expected, actual, "unexpected output")
}
})
}
}

158
internal/retry/retry.go Normal file
View File

@ -0,0 +1,158 @@
package retry
import (
"context"
"strings"
"time"
"github.com/cenkalti/backoff/v4"
"github.com/pkg/errors"
"github.com/dapr/components-contrib/internal/config"
)
// PolicyType denotes if the back off delay should be constant or exponential.
type PolicyType int
const (
// PolicyConstant is a backoff policy that always returns the same backoff delay.
PolicyConstant PolicyType = iota
// PolicyExponential is a backoff implementation that increases the backoff period
// for each retry attempt using a randomization function that grows exponentially.
PolicyExponential
)
// Config encapsulates the back off policy configuration.
type Config struct {
Policy PolicyType `mapstructure:"policy"`
// Constant back off
Duration time.Duration `mapstructure:"duration"`
// Exponential back off
InitialInterval time.Duration `mapstructure:"initialInterval"`
RandomizationFactor float32 `mapstructure:"randomizationFactor"`
Multiplier float32 `mapstructure:"multiplier"`
MaxInterval time.Duration `mapstructure:"maxInterval"`
MaxElapsedTime time.Duration `mapstructure:"maxElapsedTime"`
// Additional options
MaxRetries int64 `mapstructure:"maxRetries"`
}
// DefaultConfig represents the default configuration for a
// `Config`.
func DefaultConfig() Config {
return Config{
Policy: PolicyConstant,
Duration: 5 * time.Second,
InitialInterval: backoff.DefaultInitialInterval,
RandomizationFactor: backoff.DefaultRandomizationFactor,
Multiplier: backoff.DefaultMultiplier,
MaxInterval: backoff.DefaultMaxInterval,
MaxElapsedTime: backoff.DefaultMaxElapsedTime,
MaxRetries: -1,
}
}
// DecodeConfig decodes a Go struct into a `Config`.
func DecodeConfig(c *Config, input interface{}) error {
// Use the deefault config if `c` is empty/zero value.
var emptyConfig Config
if *c == emptyConfig {
*c = DefaultConfig()
}
return config.Decode(input, c)
}
// DecodeConfigWithPrefix decodes a Go struct into a `Config`.
func DecodeConfigWithPrefix(c *Config, input interface{}, prefix string) error {
input, err := config.PrefixedBy(input, prefix)
if err != nil {
return err
}
return DecodeConfig(c, input)
}
// NewBackOff returns a BackOff instance for use with `NotifyRecover`
// or `backoff.RetryNotify` directly. The instance will not stop due to
// context cancellation. To support cancellation (recommended), use
// `NewBackOffWithContext`.
//
// Since the underlying backoff implementations are not always thread safe,
// `NewBackOff` or `NewBackOffWithContext` should be called each time
// `RetryNotifyRecover` or `backoff.RetryNotify` is used.
func (c *Config) NewBackOff() backoff.BackOff {
var b backoff.BackOff
switch c.Policy {
case PolicyConstant:
b = backoff.NewConstantBackOff(c.Duration)
case PolicyExponential:
eb := backoff.NewExponentialBackOff()
eb.InitialInterval = c.InitialInterval
eb.RandomizationFactor = float64(c.RandomizationFactor)
eb.Multiplier = float64(c.Multiplier)
eb.MaxInterval = c.MaxInterval
eb.MaxElapsedTime = c.MaxElapsedTime
b = eb
}
if c.MaxRetries >= 0 {
b = backoff.WithMaxRetries(b, uint64(c.MaxRetries))
}
return b
}
// NewBackOffWithContext returns a BackOff instance for use with `RetryNotifyRecover`
// or `backoff.RetryNotify` directly. The provided context is used to cancel retries
// if it is canceled.
//
// Since the underlying backoff implementations are not always thread safe,
// `NewBackOff` or `NewBackOffWithContext` should be called each time
// `RetryNotifyRecover` or `backoff.RetryNotify` is used.
func (c *Config) NewBackOffWithContext(ctx context.Context) backoff.BackOff {
b := c.NewBackOff()
return backoff.WithContext(b, ctx)
}
// NotifyRecover is a wrapper around backoff.RetryNotify that adds another callback for when an operation
// previously failed but has since recovered. The main purpose of this wrapper is to call `notify` only when
// the operations fails the first time and `recovered` when it finally succeeds. This can be helpful in limiting
// log messages to only the events that operators need to be alerted on.
func NotifyRecover(operation backoff.Operation, b backoff.BackOff, notify backoff.Notify, recovered func()) error {
var notified bool
return backoff.RetryNotify(func() error {
err := operation()
if err == nil && notified {
notified = false
recovered()
}
return err
}, b, func(err error, d time.Duration) {
if !notified {
notify(err, d)
notified = true
}
})
}
// DecodeString handles converting a string value to `p`.
func (p *PolicyType) DecodeString(value string) error {
switch strings.ToLower(value) {
case "constant":
*p = PolicyConstant
case "exponential":
*p = PolicyExponential
default:
return errors.Errorf("unexpected back off policy type: %s", value)
}
return nil
}

View File

@ -0,0 +1,228 @@
package retry_test
import (
"context"
"errors"
"testing"
"time"
"github.com/stretchr/testify/assert"
"github.com/dapr/components-contrib/internal/retry"
)
var errRetry = errors.New("Testing")
func TestDecode(t *testing.T) {
tests := map[string]struct {
config interface{}
overrides func(config *retry.Config)
err string
}{
"invalid policy type": {
config: map[string]interface{}{
"backOffPolicy": "invalid",
},
overrides: nil,
err: "1 error(s) decoding:\n\n* error decoding 'policy': invalid PolicyType \"invalid\": unexpected back off policy type: invalid",
},
"default": {
config: map[string]interface{}{},
overrides: nil,
err: "",
},
"constant default": {
config: map[string]interface{}{
"backOffPolicy": "constant",
},
overrides: nil,
err: "",
},
"constant with duraction": {
config: map[string]interface{}{
"backOffPolicy": "constant",
"backOffDuration": "10s",
},
overrides: func(config *retry.Config) {
config.Duration = 10 * time.Second
},
err: "",
},
"exponential default": {
config: map[string]interface{}{
"backOffPolicy": "exponential",
},
overrides: func(config *retry.Config) {
config.Policy = retry.PolicyExponential
},
err: "",
},
"exponential with string settings": {
config: map[string]interface{}{
"backOffPolicy": "exponential",
"backOffInitialInterval": "1000", // 1s
"backOffRandomizationFactor": "1.0",
"backOffMultiplier": "2.0",
"backOffMaxInterval": "120000", // 2m
"backOffMaxElapsedTime": "1800000", // 30m
},
overrides: func(config *retry.Config) {
config.Policy = retry.PolicyExponential
config.InitialInterval = 1 * time.Second
config.RandomizationFactor = 1.0
config.Multiplier = 2.0
config.MaxInterval = 2 * time.Minute
config.MaxElapsedTime = 30 * time.Minute
},
err: "",
},
"exponential with typed settings": {
config: map[string]interface{}{
"backOffPolicy": "exponential",
"backOffInitialInterval": "1000ms", // 1s
"backOffRandomizationFactor": 1.0,
"backOffMultiplier": 2.0,
"backOffMaxInterval": "120s", // 2m
"backOffMaxElapsedTime": "30m", // 30m
},
overrides: func(config *retry.Config) {
config.Policy = retry.PolicyExponential
config.InitialInterval = 1 * time.Second
config.RandomizationFactor = 1.0
config.Multiplier = 2.0
config.MaxInterval = 2 * time.Minute
config.MaxElapsedTime = 30 * time.Minute
},
err: "",
},
"map[string]string settings": {
config: map[string]string{
"backOffPolicy": "exponential",
"backOffInitialInterval": "1000ms", // 1s
"backOffRandomizationFactor": "1.0",
"backOffMultiplier": "2.0",
"backOffMaxInterval": "120s", // 2m
"backOffMaxElapsedTime": "30m", // 30m
},
overrides: func(config *retry.Config) {
config.Policy = retry.PolicyExponential
config.InitialInterval = 1 * time.Second
config.RandomizationFactor = 1.0
config.Multiplier = 2.0
config.MaxInterval = 2 * time.Minute
config.MaxElapsedTime = 30 * time.Minute
},
err: "",
},
}
for name, tc := range tests {
t.Run(name, func(t *testing.T) {
var actual retry.Config
err := retry.DecodeConfigWithPrefix(&actual, tc.config, "backOff")
if tc.err != "" {
if assert.Error(t, err) {
assert.Equal(t, tc.err, err.Error())
}
} else {
config := retry.DefaultConfig()
if tc.overrides != nil {
tc.overrides(&config)
}
assert.Equal(t, config, actual, "unexpected decoded configuration")
}
})
}
}
func TestRetryNotifyRecoverMaxRetries(t *testing.T) {
config := retry.DefaultConfig()
config.MaxRetries = 3
config.Duration = 1
var operationCalls, notifyCalls, recoveryCalls int
b := config.NewBackOff()
err := retry.NotifyRecover(func() error {
operationCalls++
return errRetry
}, b, func(err error, d time.Duration) {
notifyCalls++
}, func() {
recoveryCalls++
})
assert.Error(t, err)
assert.Equal(t, errRetry, err)
assert.Equal(t, 4, operationCalls)
assert.Equal(t, 1, notifyCalls)
assert.Equal(t, 0, recoveryCalls)
}
func TestRetryNotifyRecoverRecovery(t *testing.T) {
config := retry.DefaultConfig()
config.MaxRetries = 3
config.Duration = 1
var operationCalls, notifyCalls, recoveryCalls int
b := config.NewBackOff()
err := retry.NotifyRecover(func() error {
operationCalls++
if operationCalls >= 2 {
return nil
}
return errRetry
}, b, func(err error, d time.Duration) {
notifyCalls++
}, func() {
recoveryCalls++
})
assert.NoError(t, err)
assert.Equal(t, 2, operationCalls)
assert.Equal(t, 1, notifyCalls)
assert.Equal(t, 1, recoveryCalls)
}
func TestRetryNotifyRecoverCancel(t *testing.T) {
config := retry.DefaultConfig()
config.Policy = retry.PolicyExponential
config.InitialInterval = 10 * time.Millisecond
var notifyCalls, recoveryCalls int
ctx, cancel := context.WithCancel(context.Background())
b := config.NewBackOffWithContext(ctx)
errC := make(chan error, 1)
go func() {
errC <- retry.NotifyRecover(func() error {
return errRetry
}, b, func(err error, d time.Duration) {
notifyCalls++
}, func() {
recoveryCalls++
})
}()
time.Sleep(1 * time.Second)
cancel()
err := <-errC
assert.Error(t, err)
assert.True(t, errors.Is(err, context.Canceled))
assert.Equal(t, 1, notifyCalls)
assert.Equal(t, 0, recoveryCalls)
}
func TestCheckEmptyConfig(t *testing.T) {
var config retry.Config
err := retry.DecodeConfig(&config, map[string]interface{}{})
assert.NoError(t, err)
defaultConfig := retry.DefaultConfig()
assert.Equal(t, config, defaultConfig)
}

View File

@ -8,7 +8,7 @@ import (
consul "github.com/hashicorp/consul/api"
"github.com/dapr/components-contrib/nameresolution"
"github.com/dapr/components-contrib/internal/config"
)
// The intermediateConfig is based off of the consul api types. User configurations are
@ -37,9 +37,8 @@ type configSpec struct {
}
func parseConfig(rawConfig interface{}) (configSpec, error) {
result := configSpec{}
config := intermediateConfig{}
rawConfig, err := nameresolution.ConvertConfig(rawConfig)
var result configSpec
rawConfig, err := config.Normalize(rawConfig)
if err != nil {
return result, err
}
@ -52,11 +51,12 @@ func parseConfig(rawConfig interface{}) (configSpec, error) {
decoder := json.NewDecoder(bytes.NewReader(data))
decoder.DisallowUnknownFields()
if err := decoder.Decode(&config); err != nil {
var configuration intermediateConfig
if err := decoder.Decode(&configuration); err != nil {
return result, fmt.Errorf("error deserializing to configSpec: %w", err)
}
result = mapConfig(config)
result = mapConfig(configuration)
return result, nil
}

View File

@ -8,6 +8,7 @@ package kubernetes
import (
"fmt"
"github.com/dapr/components-contrib/internal/config"
"github.com/dapr/components-contrib/nameresolution"
"github.com/dapr/kit/logger"
)
@ -32,7 +33,7 @@ func NewResolver(logger logger.Logger) nameresolution.Resolver {
// Init initializes Kubernetes name resolver.
func (k *resolver) Init(metadata nameresolution.Metadata) error {
configInterface, err := nameresolution.ConvertConfig(metadata.Configuration)
configInterface, err := config.Normalize(metadata.Configuration)
if err != nil {
return err
}

View File

@ -17,6 +17,7 @@ import (
"github.com/cenkalti/backoff/v4"
azservicebus "github.com/Azure/azure-service-bus-go"
"github.com/dapr/components-contrib/internal/retry"
contrib_metadata "github.com/dapr/components-contrib/metadata"
"github.com/dapr/components-contrib/pubsub"
"github.com/dapr/kit/logger"
@ -310,7 +311,7 @@ func (a *azureServiceBus) doPublish(sender *azservicebus.Topic, msg *azservicebu
bo := backoff.WithMaxRetries(ebo, uint64(a.metadata.PublishMaxRetries))
bo = backoff.WithContext(bo, a.ctx)
return pubsub.RetryNotifyRecover(func() error {
return retry.NotifyRecover(func() error {
ctx, cancel := context.WithTimeout(a.ctx, time.Second*time.Duration(a.metadata.TimeoutInSec))
defer cancel()

View File

@ -12,6 +12,7 @@ import (
"github.com/hazelcast/hazelcast-go-client"
hazelcastCore "github.com/hazelcast/hazelcast-go-client/core"
"github.com/dapr/components-contrib/internal/retry"
"github.com/dapr/components-contrib/pubsub"
"github.com/dapr/kit/logger"
)
@ -151,7 +152,7 @@ func (l *hazelcastMessageListener) handleMessageObject(message []byte) error {
b = backoff.WithMaxRetries(b, uint64(l.p.metadata.backOffMaxRetries))
}
return pubsub.RetryNotifyRecover(func() error {
return retry.NotifyRecover(func() error {
l.p.logger.Debug("Processing Hazelcast message")
return l.pubsubHandler(l.p.ctx, &pubsubMsg)

View File

@ -17,7 +17,8 @@ import (
"time"
"github.com/Shopify/sarama"
"github.com/cenkalti/backoff/v4"
"github.com/dapr/components-contrib/internal/retry"
"github.com/dapr/components-contrib/pubsub"
"github.com/dapr/kit/logger"
)
@ -39,8 +40,9 @@ type Kafka struct {
topics map[string]bool
cancel context.CancelFunc
consumer consumer
backOff backoff.BackOff
config *sarama.Config
backOffConfig retry.Config
}
type kafkaMetadata struct {
@ -53,8 +55,7 @@ type kafkaMetadata struct {
}
type consumer struct {
logger logger.Logger
backOff backoff.BackOff
k *Kafka
ready chan bool
callback pubsub.Handler
once sync.Once
@ -65,24 +66,24 @@ func (consumer *consumer) ConsumeClaim(session sarama.ConsumerGroupSession, clai
return fmt.Errorf("nil consumer callback")
}
bo := backoff.WithContext(consumer.backOff, session.Context())
b := consumer.k.backOffConfig.NewBackOffWithContext(session.Context())
for message := range claim.Messages() {
msg := pubsub.NewMessage{
Topic: message.Topic,
Data: message.Value,
}
if err := pubsub.RetryNotifyRecover(func() error {
consumer.logger.Debugf("Processing Kafka message: %s/%d/%d [key=%s]", message.Topic, message.Partition, message.Offset, asBase64String(message.Key))
if err := retry.NotifyRecover(func() error {
consumer.k.logger.Debugf("Processing Kafka message: %s/%d/%d [key=%s]", message.Topic, message.Partition, message.Offset, asBase64String(message.Key))
err := consumer.callback(session.Context(), &msg)
if err == nil {
session.MarkMessage(message, "")
}
return err
}, bo, func(err error, d time.Duration) {
consumer.logger.Errorf("Error processing Kafka message: %s/%d/%d [key=%s]. Retrying...", message.Topic, message.Partition, message.Offset, asBase64String(message.Key))
}, b, func(err error, d time.Duration) {
consumer.k.logger.Errorf("Error processing Kafka message: %s/%d/%d [key=%s]. Retrying...", message.Topic, message.Partition, message.Offset, asBase64String(message.Key))
}, func() {
consumer.logger.Infof("Successfully processed Kafka message after it previously failed: %s/%d/%d [key=%s]", message.Topic, message.Partition, message.Offset, asBase64String(message.Key))
consumer.k.logger.Infof("Successfully processed Kafka message after it previously failed: %s/%d/%d [key=%s]", message.Topic, message.Partition, message.Offset, asBase64String(message.Key))
}); err != nil {
return err
}
@ -140,8 +141,14 @@ 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)
// Default retry configuration is used if no
// backOff properties are set.
if err := retry.DecodeConfigWithPrefix(
&k.backOffConfig,
metadata.Properties,
"backOff"); err != nil {
return err
}
k.logger.Debug("Kafka message bus initialization complete")
@ -227,8 +234,7 @@ func (k *Kafka) Subscribe(req pubsub.SubscribeRequest, handler pubsub.Handler) e
ready := make(chan bool)
k.consumer = consumer{
logger: k.logger,
backOff: k.backOff,
k: k,
ready: ready,
callback: handler,
}

View File

@ -18,6 +18,7 @@ import (
"github.com/cenkalti/backoff/v4"
mqtt "github.com/eclipse/paho.mqtt.golang"
"github.com/dapr/components-contrib/internal/retry"
"github.com/dapr/components-contrib/pubsub"
"github.com/dapr/kit/logger"
)
@ -216,7 +217,7 @@ func (m *mqttPubSub) Subscribe(req pubsub.SubscribeRequest, handler pubsub.Handl
if m.metadata.backOffMaxRetries >= 0 {
b = backoff.WithMaxRetries(m.backOff, uint64(m.metadata.backOffMaxRetries))
}
if err := pubsub.RetryNotifyRecover(func() error {
if err := retry.NotifyRecover(func() error {
m.logger.Debugf("Processing MQTT message %s/%d", mqttMsg.Topic(), mqttMsg.MessageID())
if err := handler(m.ctx, &msg); err != nil {
return err

View File

@ -16,11 +16,11 @@ import (
"strconv"
"time"
"github.com/cenkalti/backoff/v4"
nats "github.com/nats-io/nats.go"
stan "github.com/nats-io/stan.go"
"github.com/nats-io/stan.go/pb"
"github.com/dapr/components-contrib/internal/retry"
"github.com/dapr/components-contrib/pubsub"
"github.com/dapr/kit/logger"
)
@ -65,9 +65,9 @@ type natsStreamingPubSub struct {
logger logger.Logger
ctx context.Context
cancel context.CancelFunc
backOff backoff.BackOff
ctx context.Context
cancel context.CancelFunc
backOffConfig retry.Config
}
// NewNATSStreamingPubSub returns a new NATS Streaming pub-sub implementation
@ -193,13 +193,16 @@ func (n *natsStreamingPubSub) Init(metadata pubsub.Metadata) error {
}
n.logger.Debugf("connected to natsstreaming at %s", m.natsURL)
ctx, cancel := context.WithCancel(context.Background())
n.ctx = ctx
n.cancel = cancel
n.ctx, n.cancel = context.WithCancel(context.Background())
// TODO: Make the backoff configurable for constant or exponential
b := backoff.NewConstantBackOff(5 * time.Second)
n.backOff = backoff.WithContext(b, n.ctx)
// Default retry configuration is used if no
// backOff properties are set.
if err := retry.DecodeConfigWithPrefix(
&n.backOffConfig,
metadata.Properties,
"backOff"); err != nil {
return err
}
n.natStreamingConn = natStreamingConn
@ -226,7 +229,9 @@ func (n *natsStreamingPubSub) Subscribe(req pubsub.SubscribeRequest, handler pub
Topic: req.Topic,
Data: natsMsg.Data,
}
pubsub.RetryNotifyRecover(func() error {
b := n.backOffConfig.NewBackOffWithContext(n.ctx)
rerr := retry.NotifyRecover(func() error {
n.logger.Debugf("Processing NATS Streaming message %s/%d", natsMsg.Subject, natsMsg.Sequence)
herr := handler(n.ctx, &msg)
if herr == nil {
@ -235,11 +240,14 @@ func (n *natsStreamingPubSub) Subscribe(req pubsub.SubscribeRequest, handler pub
}
return herr
}, n.backOff, func(err error, d time.Duration) {
}, b, func(err error, d time.Duration) {
n.logger.Errorf("Error processing NATS Streaming message: %s/%d. Retrying...", natsMsg.Subject, natsMsg.Sequence)
}, func() {
n.logger.Infof("Successfully processed NATS Streaming message after it previously failed: %s/%d", natsMsg.Subject, natsMsg.Sequence)
})
if rerr != nil && !errors.Is(rerr, context.Canceled) {
n.logger.Errorf("Error processing message and retries are exhausted: %s/%d.", natsMsg.Subject, natsMsg.Sequence)
}
}
if n.metadata.subscriptionType == subscriptionTypeTopic {

View File

@ -8,9 +8,9 @@ import (
"time"
"github.com/apache/pulsar-client-go/pulsar"
"github.com/cenkalti/backoff/v4"
lru "github.com/hashicorp/golang-lru"
"github.com/dapr/components-contrib/internal/retry"
"github.com/dapr/components-contrib/pubsub"
"github.com/dapr/kit/logger"
)
@ -27,10 +27,10 @@ type Pulsar struct {
producer pulsar.Producer
metadata pulsarMetadata
ctx context.Context
cancel context.CancelFunc
backOff backoff.BackOff
cache *lru.Cache
ctx context.Context
cancel context.CancelFunc
backOffConfig retry.Config
cache *lru.Cache
}
func NewPulsar(l logger.Logger) pubsub.PubSub {
@ -92,9 +92,14 @@ func (p *Pulsar) Init(metadata pubsub.Metadata) error {
p.client = client
p.metadata = *m
// TODO: Make the backoff configurable for constant or exponential
b := backoff.NewConstantBackOff(5 * time.Second)
p.backOff = backoff.WithContext(b, p.ctx)
// Default retry configuration is used if no
// backOff properties are set.
if err := retry.DecodeConfigWithPrefix(
&p.backOffConfig,
metadata.Properties,
"backOff"); err != nil {
return err
}
return nil
}
@ -174,7 +179,9 @@ func (p *Pulsar) handleMessage(msg pulsar.ConsumerMessage, handler pubsub.Handle
Metadata: msg.Properties(),
}
return pubsub.RetryNotifyRecover(func() error {
b := p.backOffConfig.NewBackOffWithContext(p.ctx)
return retry.NotifyRecover(func() error {
p.logger.Debugf("Processing Pulsar message %s/%#v", msg.Topic(), msg.ID())
err := handler(p.ctx, &pubsubMsg)
if err == nil {
@ -182,7 +189,7 @@ func (p *Pulsar) handleMessage(msg pulsar.ConsumerMessage, handler pubsub.Handle
}
return err
}, p.backOff, func(err error, d time.Duration) {
}, b, func(err error, d time.Duration) {
p.logger.Errorf("Error processing Pulsar message: %s/%#v [key=%s]. Retrying...", msg.Topic(), msg.ID(), msg.Key())
}, func() {
p.logger.Infof("Successfully processed Pulsar message after it previously failed: %s/%#v [key=%s]", msg.Topic(), msg.ID(), msg.Key())

View File

@ -1,31 +0,0 @@
package pubsub
import (
"time"
"github.com/cenkalti/backoff/v4"
)
// RetryNotifyRecover is a wrapper around backoff.RetryNotify that adds another callback for when an operation
// previously failed but has since recovered. The main purpose of this wrapper is to call `notify` only when
// the operations fails the first time and `recovered` when it finally succeeds. This can be helpful in limiting
// log messages to only the events that operators need to be alerted on.
func RetryNotifyRecover(operation backoff.Operation, b backoff.BackOff, notify backoff.Notify, recovered func()) error {
var notified bool
return backoff.RetryNotify(func() error {
err := operation()
if err == nil && notified {
notified = false
recovered()
}
return err
}, b, func(err error, d time.Duration) {
if !notified {
notify(err, d)
notified = true
}
})
}