Skip to content

Commit 4408887

Browse files
added exponential delay for poisoned message handlers (#175)
* added exponential delay for poisoned message handlers
1 parent b88bfe4 commit 4408887

File tree

9 files changed

+97
-43
lines changed

9 files changed

+97
-43
lines changed
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,19 @@
1+
package com.avast.clients.rabbitmq
2+
3+
import java.util.concurrent.TimeUnit
4+
import scala.concurrent.duration.{Duration, FiniteDuration}
5+
6+
class ExponentialDelay(val initialDelay: Duration, val period: Duration, val factor: Double, val maxLength: Duration) {
7+
private val maxMillis = maxLength.toMillis
8+
9+
def getExponentialDelay(attempt: Int): FiniteDuration = {
10+
if (attempt == 0) FiniteDuration(initialDelay._1, initialDelay._2)
11+
else {
12+
val millis = math.min(
13+
maxMillis,
14+
(period.toMillis * math.pow(factor, attempt - 1)).toLong
15+
)
16+
FiniteDuration(millis, TimeUnit.MILLISECONDS)
17+
}
18+
}
19+
}

core/src/main/scala/com/avast/clients/rabbitmq/PoisonedMessageHandler.scala

+34-18
Original file line numberDiff line numberDiff line change
@@ -1,7 +1,7 @@
11
package com.avast.clients.rabbitmq
22

33
import cats.Applicative
4-
import cats.effect.{Resource, Sync}
4+
import cats.effect.{Resource, Sync, Timer}
55
import cats.implicits.{catsSyntaxApplicativeError, catsSyntaxFlatMapOps, toFunctorOps}
66
import com.avast.bytes.Bytes
77
import com.avast.clients.rabbitmq.PoisonedMessageHandler.{defaultHandlePoisonedMessage, DiscardedTimeHeaderName}
@@ -19,7 +19,8 @@ sealed trait PoisonedMessageHandler[F[_], A] {
1919
implicit dctx: DeliveryContext): F[DeliveryResult]
2020
}
2121

22-
class LoggingPoisonedMessageHandler[F[_]: Sync, A](maxAttempts: Int) extends PoisonedMessageHandler[F, A] {
22+
class LoggingPoisonedMessageHandler[F[_]: Sync: Timer, A](maxAttempts: Int, republishDelay: Option[ExponentialDelay])
23+
extends PoisonedMessageHandler[F, A] {
2324
private val logger = ImplicitContextLogger.createLogger[F, LoggingPoisonedMessageHandler[F, A]]
2425

2526
override def interceptResult(delivery: Delivery[A], messageId: MessageId, rawBody: Bytes)(result: DeliveryResult)(
@@ -28,6 +29,7 @@ class LoggingPoisonedMessageHandler[F[_]: Sync, A](maxAttempts: Int) extends Poi
2829
messageId,
2930
maxAttempts,
3031
logger,
32+
republishDelay,
3133
(d: Delivery[A], _) => defaultHandlePoisonedMessage[F, A](maxAttempts, logger)(d))(result)
3234
}
3335
}
@@ -37,14 +39,19 @@ class NoOpPoisonedMessageHandler[F[_]: Sync, A] extends PoisonedMessageHandler[F
3739
implicit dctx: DeliveryContext): F[DeliveryResult] = Sync[F].pure(result)
3840
}
3941

40-
class DeadQueuePoisonedMessageHandler[F[_]: Sync, A](maxAttempts: Int)(moveToDeadQueue: (Delivery[A], Bytes, DeliveryContext) => F[Unit])
42+
class DeadQueuePoisonedMessageHandler[F[_]: Sync: Timer, A](maxAttempts: Int, republishDelay: Option[ExponentialDelay])(
43+
moveToDeadQueue: (Delivery[A], Bytes, DeliveryContext) => F[Unit])
4144
extends PoisonedMessageHandler[F, A] {
4245
private val logger = ImplicitContextLogger.createLogger[F, DeadQueuePoisonedMessageHandler[F, A]]
4346

4447
override def interceptResult(delivery: Delivery[A], messageId: MessageId, rawBody: Bytes)(result: DeliveryResult)(
4548
implicit dctx: DeliveryContext): F[DeliveryResult] = {
46-
PoisonedMessageHandler.handleResult(delivery, messageId, maxAttempts, logger, (d, _) => handlePoisonedMessage(d, messageId, rawBody))(
47-
result)
49+
PoisonedMessageHandler.handleResult(delivery,
50+
messageId,
51+
maxAttempts,
52+
logger,
53+
republishDelay,
54+
(d, _) => handlePoisonedMessage(d, messageId, rawBody))(result)
4855
}
4956

5057
private def handlePoisonedMessage(delivery: Delivery[A], messageId: MessageId, rawBody: Bytes)(
@@ -59,9 +66,9 @@ class DeadQueuePoisonedMessageHandler[F[_]: Sync, A](maxAttempts: Int)(moveToDea
5966
}
6067

6168
object DeadQueuePoisonedMessageHandler {
62-
def make[F[_]: Sync, A](c: DeadQueuePoisonedMessageHandling,
63-
connection: RabbitMQConnection[F],
64-
monitor: Monitor[F]): Resource[F, DeadQueuePoisonedMessageHandler[F, A]] = {
69+
def make[F[_]: Sync: Timer, A](c: DeadQueuePoisonedMessageHandling,
70+
connection: RabbitMQConnection[F],
71+
monitor: Monitor[F]): Resource[F, DeadQueuePoisonedMessageHandler[F, A]] = {
6572
val dqpc = c.deadQueueProducer
6673
val pc = ProducerConfig(
6774
name = dqpc.name,
@@ -73,7 +80,7 @@ object DeadQueuePoisonedMessageHandler {
7380
)
7481

7582
connection.newProducer[Bytes](pc, monitor.named("deadQueueProducer")).map { producer =>
76-
new DeadQueuePoisonedMessageHandler[F, A](c.maxAttempts)((d: Delivery[A], rawBody: Bytes, dctx: DeliveryContext) => {
83+
new DeadQueuePoisonedMessageHandler[F, A](c.maxAttempts, c.republishDelay)((d: Delivery[A], rawBody: Bytes, dctx: DeliveryContext) => {
7784
val cidStrategy = dctx.correlationId match {
7885
case Some(value) => CorrelationIdStrategy.Fixed(value.value)
7986
case None => CorrelationIdStrategy.RandomNew
@@ -93,11 +100,12 @@ object PoisonedMessageHandler {
93100
final val RepublishCountHeaderName: String = "X-Republish-Count"
94101
final val DiscardedTimeHeaderName: String = "X-Discarded-Time"
95102

96-
private[rabbitmq] def make[F[_]: Sync, A](config: Option[PoisonedMessageHandlingConfig],
97-
connection: RabbitMQConnection[F],
98-
monitor: Monitor[F]): Resource[F, PoisonedMessageHandler[F, A]] = {
103+
private[rabbitmq] def make[F[_]: Sync: Timer, A](config: Option[PoisonedMessageHandlingConfig],
104+
connection: RabbitMQConnection[F],
105+
monitor: Monitor[F]): Resource[F, PoisonedMessageHandler[F, A]] = {
99106
config match {
100-
case Some(LoggingPoisonedMessageHandling(maxAttempts)) => Resource.pure(new LoggingPoisonedMessageHandler[F, A](maxAttempts))
107+
case Some(LoggingPoisonedMessageHandling(maxAttempts, republishDelay)) =>
108+
Resource.pure(new LoggingPoisonedMessageHandler[F, A](maxAttempts, republishDelay))
101109
case Some(c: DeadQueuePoisonedMessageHandling) => DeadQueuePoisonedMessageHandler.make(c, connection, monitor)
102110
case Some(NoOpPoisonedMessageHandling) | None =>
103111
Resource.eval {
@@ -114,26 +122,30 @@ object PoisonedMessageHandler {
114122
logger.warn(s"Message failures reached the limit $maxAttempts attempts, throwing away: $delivery")
115123
}
116124

117-
private[rabbitmq] def handleResult[F[_]: Sync, A](
125+
private[rabbitmq] def handleResult[F[_]: Sync: Timer, A](
118126
delivery: Delivery[A],
119127
messageId: MessageId,
120128
maxAttempts: Int,
121129
logger: ImplicitContextLogger[F],
130+
republishDelay: Option[ExponentialDelay],
122131
handlePoisonedMessage: (Delivery[A], Int) => F[Unit])(r: DeliveryResult)(implicit dctx: DeliveryContext): F[DeliveryResult] = {
123132
r match {
124133
case Republish(isPoisoned, newHeaders) if isPoisoned =>
125-
adjustDeliveryResult(delivery, messageId, maxAttempts, newHeaders, logger, handlePoisonedMessage)
134+
adjustDeliveryResult(delivery, messageId, maxAttempts, newHeaders, logger, republishDelay, handlePoisonedMessage)
126135
case r => Applicative[F].pure(r) // keep other results as they are
127136
}
128137
}
129138

130-
private def adjustDeliveryResult[F[_]: Sync, A](
139+
private def adjustDeliveryResult[F[_]: Sync: Timer, A](
131140
delivery: Delivery[A],
132141
messageId: MessageId,
133142
maxAttempts: Int,
134143
newHeaders: Map[String, AnyRef],
135144
logger: ImplicitContextLogger[F],
145+
republishDelay: Option[ExponentialDelay],
136146
handlePoisonedMessage: (Delivery[A], Int) => F[Unit])(implicit dctx: DeliveryContext): F[DeliveryResult] = {
147+
import cats.syntax.traverse._
148+
137149
// get current attempt no. from passed headers with fallback to original (incoming) headers - the fallback will most likely happen
138150
// but we're giving the programmer chance to programmatically _pretend_ lower attempt number
139151
val attempt = (delivery.properties.headers ++ newHeaders)
@@ -143,8 +155,12 @@ object PoisonedMessageHandler {
143155

144156
logger.debug(s"Attempt $attempt/$maxAttempts for $messageId") >> {
145157
if (attempt < maxAttempts) {
146-
Applicative[F].pure(
147-
Republish(countAsPoisoned = true, newHeaders = newHeaders + (RepublishCountHeaderName -> attempt.asInstanceOf[AnyRef])))
158+
for {
159+
_ <- republishDelay.traverse { d =>
160+
val delay = d.getExponentialDelay(attempt)
161+
logger.debug(s"Will republish the message in $delay") >> Timer[F].sleep(delay)
162+
}
163+
} yield Republish(countAsPoisoned = true, newHeaders = newHeaders + (RepublishCountHeaderName -> attempt.asInstanceOf[AnyRef]))
148164
} else {
149165
val now = Instant.now()
150166

core/src/main/scala/com/avast/clients/rabbitmq/RecoveryDelayHandlers.scala

+6-14
Original file line numberDiff line numberDiff line change
@@ -13,21 +13,13 @@ object RecoveryDelayHandlers {
1313
}
1414
}
1515

16-
case class Exponential(initialDelay: Duration = 5.second,
17-
period: Duration = 5.seconds,
18-
factor: Double = 2.0,
19-
maxLength: Duration = 32.seconds)
20-
extends RecoveryDelayHandler {
21-
private val maxMillis = maxLength.toMillis
22-
16+
case class Exponential(override val initialDelay: Duration = 2.second,
17+
override val period: Duration = 2.seconds,
18+
override val factor: Double = 2.0,
19+
override val maxLength: Duration = 32.seconds)
20+
extends ExponentialDelay(initialDelay, period, factor, maxLength) with RecoveryDelayHandler {
2321
override def getDelay(recoveryAttempts: Int): Long = {
24-
if (recoveryAttempts == 0) initialDelay.toMillis
25-
else {
26-
math.min(
27-
maxMillis,
28-
(period.toMillis * math.pow(factor, recoveryAttempts - 1)).toLong
29-
)
30-
}
22+
getExponentialDelay(recoveryAttempts).toMillis
3123
}
3224
}
3325
}

core/src/main/scala/com/avast/clients/rabbitmq/configuration.scala

+2-2
Original file line numberDiff line numberDiff line change
@@ -125,8 +125,8 @@ final case class DeadQueueProducerConfig(name: String,
125125
properties: ProducerPropertiesConfig = ProducerPropertiesConfig())
126126

127127
case object NoOpPoisonedMessageHandling extends PoisonedMessageHandlingConfig
128-
final case class LoggingPoisonedMessageHandling(maxAttempts: Int) extends PoisonedMessageHandlingConfig
129-
final case class DeadQueuePoisonedMessageHandling(maxAttempts: Int, deadQueueProducer: DeadQueueProducerConfig)
128+
final case class LoggingPoisonedMessageHandling(maxAttempts: Int, republishDelay: Option[ExponentialDelay] = None) extends PoisonedMessageHandlingConfig
129+
final case class DeadQueuePoisonedMessageHandling(maxAttempts: Int, deadQueueProducer: DeadQueueProducerConfig, republishDelay: Option[ExponentialDelay] = None)
130130
extends PoisonedMessageHandlingConfig
131131

132132
sealed trait AddressResolverType

core/src/test/scala/com/avast/clients/rabbitmq/DefaultRabbitMQConsumerTest.scala

+1-1
Original file line numberDiff line numberDiff line change
@@ -466,5 +466,5 @@ class DefaultRabbitMQConsumerTest extends TestBase {
466466
)(userAction)
467467
}
468468

469-
object PMH extends LoggingPoisonedMessageHandler[Task, Bytes](3)
469+
object PMH extends LoggingPoisonedMessageHandler[Task, Bytes](3, None)
470470
}

core/src/test/scala/com/avast/clients/rabbitmq/DefaultRabbitMQPullConsumerTest.scala

+1-1
Original file line numberDiff line numberDiff line change
@@ -287,5 +287,5 @@ class DefaultRabbitMQPullConsumerTest extends TestBase {
287287
new DefaultRabbitMQPullConsumer[Task, A](base, channelOps)
288288
}
289289

290-
class PMH[A] extends LoggingPoisonedMessageHandler[Task, A](3)
290+
class PMH[A] extends LoggingPoisonedMessageHandler[Task, A](3, None)
291291
}

core/src/test/scala/com/avast/clients/rabbitmq/PoisonedMessageHandlerTest.scala

+32-6
Original file line numberDiff line numberDiff line change
@@ -25,7 +25,7 @@ class PoisonedMessageHandlerTest extends TestBase {
2525
val movedCount = new AtomicInteger(0)
2626

2727
PoisonedMessageHandler
28-
.handleResult[Task, Bytes](Delivery.Ok(Bytes.empty(), MessageProperties(), ""), MessageId("msg-id"), 1, ilogger, (_, _) => {
28+
.handleResult[Task, Bytes](Delivery.Ok(Bytes.empty(), MessageProperties(), ""), MessageId("msg-id"), 1, ilogger, None, (_, _) => {
2929
Task.delay { movedCount.incrementAndGet() }
3030
})(Republish(countAsPoisoned = false))
3131
.await
@@ -35,7 +35,7 @@ class PoisonedMessageHandlerTest extends TestBase {
3535
movedCount.set(0)
3636

3737
PoisonedMessageHandler
38-
.handleResult[Task, Bytes](Delivery.Ok(Bytes.empty(), MessageProperties(), ""), MessageId("msg-id"), 1, ilogger, (_, _) => {
38+
.handleResult[Task, Bytes](Delivery.Ok(Bytes.empty(), MessageProperties(), ""), MessageId("msg-id"), 1, ilogger, None, (_, _) => {
3939
Task.delay { movedCount.incrementAndGet() }
4040
})(Republish())
4141
.await
@@ -48,7 +48,7 @@ class PoisonedMessageHandlerTest extends TestBase {
4848
Task.now(Republish())
4949
}
5050

51-
val handler = new LoggingPoisonedMessageHandler[Task, Bytes](5)
51+
val handler = new LoggingPoisonedMessageHandler[Task, Bytes](5, None)
5252

5353
val properties = (1 to 4).foldLeft(MessageProperties.empty) {
5454
case (p, _) =>
@@ -65,6 +65,32 @@ class PoisonedMessageHandlerTest extends TestBase {
6565
assertResult(DeliveryResult.Reject)(run(handler, readAction, properties))
6666
}
6767

68+
test("LoggingPoisonedMessageHandler exponential delay") {
69+
import scala.concurrent.duration._
70+
71+
def readAction(d: Delivery[Bytes]): Task[DeliveryResult] = {
72+
Task.now(Republish())
73+
}
74+
75+
val handler = new LoggingPoisonedMessageHandler[Task, Bytes](5, Some(new ExponentialDelay(1.seconds, 1.seconds, 2, 2.seconds)))
76+
val timeBeforeExecution = Instant.now()
77+
val properties = (1 to 4).foldLeft(MessageProperties.empty) {
78+
case (p, _) =>
79+
run(handler, readAction, p) match {
80+
case Republish(_, h) => MessageProperties(headers = h)
81+
case _ => MessageProperties.empty
82+
}
83+
}
84+
85+
val now = Instant.now()
86+
assert(now.minusSeconds(7).isAfter(timeBeforeExecution) && now.minusSeconds(8).isBefore(timeBeforeExecution))
87+
// check it increases the header with count
88+
assertResult(MessageProperties(headers = Map(RepublishCountHeaderName -> 4.asInstanceOf[AnyRef])))(properties)
89+
90+
// check it will Reject the message on 5th attempt
91+
assertResult(DeliveryResult.Reject)(run(handler, readAction, properties))
92+
}
93+
6894
test("NoOpPoisonedMessageHandler basic") {
6995
def readAction(d: Delivery[Bytes]): Task[DeliveryResult] = {
7096
Task.now(Republish())
@@ -91,7 +117,7 @@ class PoisonedMessageHandlerTest extends TestBase {
91117

92118
val movedCount = new AtomicInteger(0)
93119

94-
val handler = new DeadQueuePoisonedMessageHandler[Task, Bytes](5)({ (_, _, _) =>
120+
val handler = new DeadQueuePoisonedMessageHandler[Task, Bytes](5, None)({ (_, _, _) =>
95121
Task.delay(movedCount.incrementAndGet())
96122
})
97123

@@ -119,7 +145,7 @@ class PoisonedMessageHandlerTest extends TestBase {
119145

120146
val movedCount = new AtomicInteger(0)
121147

122-
val handler = new DeadQueuePoisonedMessageHandler[Task, Bytes](3)({ (d, _, _) =>
148+
val handler = new DeadQueuePoisonedMessageHandler[Task, Bytes](3, None)({ (d, _, _) =>
123149
// test it's there and it can be parsed
124150
assert(Instant.parse(d.properties.headers(DiscardedTimeHeaderName).asInstanceOf[String]).toEpochMilli > 0)
125151

@@ -147,7 +173,7 @@ class PoisonedMessageHandlerTest extends TestBase {
147173

148174
val movedCount = new AtomicInteger(0)
149175

150-
val handler = new DeadQueuePoisonedMessageHandler[Task, Bytes](5)({ (_, _, _) =>
176+
val handler = new DeadQueuePoisonedMessageHandler[Task, Bytes](5, None)({ (_, _, _) =>
151177
Task.delay(movedCount.incrementAndGet())
152178
})
153179

core/src/test/scala/com/avast/clients/rabbitmq/RepublishStrategyTest.scala

+1-1
Original file line numberDiff line numberDiff line change
@@ -125,5 +125,5 @@ class RepublishStrategyTest extends TestBase {
125125
)(userAction)
126126
}
127127

128-
object PMH extends LoggingPoisonedMessageHandler[Task, Bytes](3)
128+
object PMH extends LoggingPoisonedMessageHandler[Task, Bytes](3, None)
129129
}

pureconfig/src/main/scala/com/avast/clients/rabbitmq/pureconfig/PureconfigImplicits.scala

+1
Original file line numberDiff line numberDiff line change
@@ -90,6 +90,7 @@ class PureconfigImplicits(implicit namingConvention: NamingConvention = CamelCas
9090
implicit val autoBindQueueConfigReader: ConfigReader[AutoBindQueueConfig] = deriveReader
9191
implicit val autoBindExchangeConfigReader: ConfigReader[AutoBindExchangeConfig] = deriveReader
9292
implicit val producerPropertiesConfigReader: ConfigReader[ProducerPropertiesConfig] = deriveReader
93+
implicit val exponentialDelayConfigReader: ConfigReader[ExponentialDelay] = deriveReader
9394

9495
implicit val logLevelReader: ConfigReader[Level] = ConfigReader.stringConfigReader.map(Level.valueOf)
9596
implicit val recoveryDelayHandlerReader: ConfigReader[RecoveryDelayHandler] = RecoveryDelayHandlerReader

0 commit comments

Comments
 (0)