diff --git a/core/src/it/scala/com/wixpress/dst/greyhound/core/AdminClientIT.scala b/core/src/it/scala/com/wixpress/dst/greyhound/core/AdminClientIT.scala index 95d2353a..dec5377c 100644 --- a/core/src/it/scala/com/wixpress/dst/greyhound/core/AdminClientIT.scala +++ b/core/src/it/scala/com/wixpress/dst/greyhound/core/AdminClientIT.scala @@ -10,7 +10,7 @@ import com.wixpress.dst.greyhound.core.producer.ProducerRecord import com.wixpress.dst.greyhound.core.testkit.{BaseTestWithSharedEnv, TestMetrics} import com.wixpress.dst.greyhound.core.zioutils.CountDownLatch import com.wixpress.dst.greyhound.testenv.ITEnv -import com.wixpress.dst.greyhound.testenv.ITEnv.{Env, TestResources, testResources} +import com.wixpress.dst.greyhound.testenv.ITEnv.{testResources, Env, TestResources} import org.apache.kafka.common.config.TopicConfig.{DELETE_RETENTION_MS_CONFIG, MAX_MESSAGE_BYTES_CONFIG, RETENTION_MS_CONFIG} import org.apache.kafka.common.errors.InvalidTopicException import org.specs2.specification.core.Fragments @@ -83,7 +83,7 @@ class AdminClientIT extends BaseTestWithSharedEnv[Env, TestResources] { } } - //todo uncomment this after https://github.com/wix-private/core-server-build-tools/pull/13043 is merged + // todo uncomment this after https://github.com/wix-private/core-server-build-tools/pull/13043 is merged // "reflect errors" in { // val topic1 = aTopicConfig() // val topic2 = aTopicConfig("x" * 250) @@ -104,7 +104,7 @@ class AdminClientIT extends BaseTestWithSharedEnv[Env, TestResources] { // created === Map(badTopic.name -> None) // } // } - //todo uncomment this after https://github.com/wix-private/core-server-build-tools/pull/13043 is merged + // todo uncomment this after https://github.com/wix-private/core-server-build-tools/pull/13043 is merged // ================================================================================================================================= "ignore TopicExistsException by default" in { val topic = aTopicConfig() diff --git a/core/src/it/scala/com/wixpress/dst/greyhound/core/BUILD.bazel b/core/src/it/scala/com/wixpress/dst/greyhound/core/BUILD.bazel index 28d79c08..9735d210 100644 --- a/core/src/it/scala/com/wixpress/dst/greyhound/core/BUILD.bazel +++ b/core/src/it/scala/com/wixpress/dst/greyhound/core/BUILD.bazel @@ -24,13 +24,8 @@ specs2_ite2e_test( "//core/src/main/scala/com/wixpress/dst/greyhound/core/producer", "//core/src/main/scala/com/wixpress/dst/greyhound/core/zioutils", "//core/src/test/resources", - #"//core/src/test/scala/com/wixpress/dst/greyhound/core/consumer", - #"//core/src/test/scala/com/wixpress/dst/greyhound/core/testkit", - "@ch_qos_logback_logback_classic", # "@dev_zio_izumi_reflect_2_12", "@dev_zio_zio_2_12", - "@dev_zio_zio_test_2_12", - "@org_apache_kafka_kafka_2_12", "@org_apache_kafka_kafka_clients", "//core/src/test/scala/com/wixpress/dst/greyhound/core/testkit", ], diff --git a/core/src/it/scala/com/wixpress/dst/greyhound/core/offset/BUILD.bazel b/core/src/it/scala/com/wixpress/dst/greyhound/core/offset/BUILD.bazel index ce9f7faf..a5c08ae5 100644 --- a/core/src/it/scala/com/wixpress/dst/greyhound/core/offset/BUILD.bazel +++ b/core/src/it/scala/com/wixpress/dst/greyhound/core/offset/BUILD.bazel @@ -13,7 +13,6 @@ specs2_ite2e_test( "@dev_zio_izumi_reflect_2_12", "@dev_zio_zio_managed_2_12", "//core/src/it/resources", - "//core/src/it/scala/com/wixpress/dst/greyhound/core", "//core/src/it/scala/com/wixpress/dst/greyhound/testenv", "//core/src/it/scala/com/wixpress/dst/greyhound/testkit", "//core/src/main/scala/com/wixpress/dst/greyhound/core", @@ -23,9 +22,7 @@ specs2_ite2e_test( "//core/src/main/scala/com/wixpress/dst/greyhound/core/metrics", "//core/src/main/scala/com/wixpress/dst/greyhound/core/producer", "//core/src/main/scala/com/wixpress/dst/greyhound/core/zioutils", - "//core/src/test/scala/com/wixpress/dst/greyhound/core/consumer", "//core/src/test/scala/com/wixpress/dst/greyhound/core/testkit", - "@ch_qos_logback_logback_classic", # "@dev_zio_izumi_reflect_2_12", "@dev_zio_zio_2_12", "@org_apache_kafka_kafka_clients", diff --git a/core/src/it/scala/com/wixpress/dst/greyhound/core/rabalance/BUILD.bazel b/core/src/it/scala/com/wixpress/dst/greyhound/core/rabalance/BUILD.bazel index 3da73eba..c763308a 100644 --- a/core/src/it/scala/com/wixpress/dst/greyhound/core/rabalance/BUILD.bazel +++ b/core/src/it/scala/com/wixpress/dst/greyhound/core/rabalance/BUILD.bazel @@ -13,7 +13,6 @@ specs2_ite2e_test( "@dev_zio_izumi_reflect_2_12", "@dev_zio_zio_managed_2_12", "//core/src/it/resources", - "//core/src/it/scala/com/wixpress/dst/greyhound/core", "//core/src/it/scala/com/wixpress/dst/greyhound/testenv", "//core/src/it/scala/com/wixpress/dst/greyhound/testkit", "//core/src/main/scala/com/wixpress/dst/greyhound/core", @@ -23,9 +22,7 @@ specs2_ite2e_test( "//core/src/main/scala/com/wixpress/dst/greyhound/core/metrics", "//core/src/main/scala/com/wixpress/dst/greyhound/core/producer", "//core/src/main/scala/com/wixpress/dst/greyhound/core/zioutils", - "//core/src/test/scala/com/wixpress/dst/greyhound/core/consumer", "//core/src/test/scala/com/wixpress/dst/greyhound/core/testkit", - "@ch_qos_logback_logback_classic", # "@dev_zio_izumi_reflect_2_12", "@dev_zio_zio_2_12", "@org_apache_kafka_kafka_clients", diff --git a/core/src/it/scala/com/wixpress/dst/greyhound/core/retry/BUILD.bazel b/core/src/it/scala/com/wixpress/dst/greyhound/core/retry/BUILD.bazel index 3f83ff43..cb001169 100644 --- a/core/src/it/scala/com/wixpress/dst/greyhound/core/retry/BUILD.bazel +++ b/core/src/it/scala/com/wixpress/dst/greyhound/core/retry/BUILD.bazel @@ -12,7 +12,6 @@ specs2_ite2e_test( deps = [ "@dev_zio_izumi_reflect_2_12", "@dev_zio_zio_managed_2_12", - "//core/src/it/scala/com/wixpress/dst/greyhound/core", "//core/src/it/scala/com/wixpress/dst/greyhound/testenv", "//core/src/it/scala/com/wixpress/dst/greyhound/testkit", "//core/src/main/scala/com/wixpress/dst/greyhound/core", @@ -21,11 +20,8 @@ specs2_ite2e_test( "//core/src/main/scala/com/wixpress/dst/greyhound/core/consumer/retry", "//core/src/main/scala/com/wixpress/dst/greyhound/core/metrics", "//core/src/main/scala/com/wixpress/dst/greyhound/core/producer", - "//core/src/main/scala/com/wixpress/dst/greyhound/core/zioutils", "//core/src/test/resources", - "//core/src/test/scala/com/wixpress/dst/greyhound/core/consumer", "//core/src/test/scala/com/wixpress/dst/greyhound/core/testkit", - "@ch_qos_logback_logback_classic", # "@dev_zio_izumi_reflect_2_12", "@dev_zio_zio_2_12", "@org_apache_kafka_kafka_clients", diff --git a/core/src/it/scala/com/wixpress/dst/greyhound/testenv/BUILD.bazel b/core/src/it/scala/com/wixpress/dst/greyhound/testenv/BUILD.bazel index 35ab06f9..6b5f15d9 100644 --- a/core/src/it/scala/com/wixpress/dst/greyhound/testenv/BUILD.bazel +++ b/core/src/it/scala/com/wixpress/dst/greyhound/testenv/BUILD.bazel @@ -13,15 +13,11 @@ scala_library( "@dev_zio_zio_managed_2_12", "//core/src/it/scala/com/wixpress/dst/greyhound/testkit", "//core/src/main/scala/com/wixpress/dst/greyhound/core", - "//core/src/main/scala/com/wixpress/dst/greyhound/core/admin", "//core/src/main/scala/com/wixpress/dst/greyhound/core/metrics", "//core/src/main/scala/com/wixpress/dst/greyhound/core/producer", "//core/src/test/scala/com/wixpress/dst/greyhound/core/testkit", # "@dev_zio_izumi_reflect_2_12", "@dev_zio_zio_2_12", "@dev_zio_zio_test_2_12", - "@org_apache_curator_curator_test", - "@org_apache_kafka_kafka_2_12", - "@org_apache_kafka_kafka_clients", ], ) diff --git a/core/src/it/scala/com/wixpress/dst/greyhound/testkit/BUILD.bazel b/core/src/it/scala/com/wixpress/dst/greyhound/testkit/BUILD.bazel index fb990677..f3345b95 100644 --- a/core/src/it/scala/com/wixpress/dst/greyhound/testkit/BUILD.bazel +++ b/core/src/it/scala/com/wixpress/dst/greyhound/testkit/BUILD.bazel @@ -16,7 +16,6 @@ scala_library( "//core/src/main/scala/com/wixpress/dst/greyhound/core/metrics", # "//core/src/main/scala/com/wixpress/dst/greyhound/core/producer", "@dev_zio_zio_2_12", - "@dev_zio_zio_test_2_12", "@org_apache_curator_curator_test", "@org_apache_kafka_kafka_2_12", "@org_apache_kafka_kafka_clients", diff --git a/core/src/main/scala/com/wixpress/dst/greyhound/core/consumer/Consumer.scala b/core/src/main/scala/com/wixpress/dst/greyhound/core/consumer/Consumer.scala index 3a53c551..7592362e 100644 --- a/core/src/main/scala/com/wixpress/dst/greyhound/core/consumer/Consumer.scala +++ b/core/src/main/scala/com/wixpress/dst/greyhound/core/consumer/Consumer.scala @@ -382,7 +382,10 @@ object UnsafeOffsetOperations { } override def offsetsForTimes(partitions: Set[TopicPartition], timeEpoch: Long, timeout: Duration): Map[TopicPartition, Option[Long]] = - consumer.offsetsForTimes(partitions.map(_.asKafka).map(tp => (tp, new lang.Long(timeEpoch))).toMap.asJava, timeout) - .asScala.toMap.map { case (tp, of) => TopicPartition(tp) -> (Option(of).map(_.offset())) } + consumer + .offsetsForTimes(partitions.map(_.asKafka).map(tp => (tp, new lang.Long(timeEpoch))).toMap.asJava, timeout) + .asScala + .toMap + .map { case (tp, of) => TopicPartition(tp) -> (Option(of).map(_.offset())) } } } diff --git a/core/src/main/scala/com/wixpress/dst/greyhound/core/consumer/EventLoop.scala b/core/src/main/scala/com/wixpress/dst/greyhound/core/consumer/EventLoop.scala index 6901b5c9..5985f80d 100644 --- a/core/src/main/scala/com/wixpress/dst/greyhound/core/consumer/EventLoop.scala +++ b/core/src/main/scala/com/wixpress/dst/greyhound/core/consumer/EventLoop.scala @@ -56,14 +56,14 @@ object EventLoop { partitionsAssigned <- Promise.make[Nothing, Unit] // TODO how to handle errors in subscribe? rebalanceListener = listener(pausedPartitionsRef, config, dispatcher, partitionsAssigned, group, consumer, clientId, offsets) - _ <- report(SubscribingToInitialSubAndRebalanceListener(clientId, group, consumerAttributes)) + _ <- report(SubscribingToInitialSubAndRebalanceListener(clientId, group, consumerAttributes)) _ <- subscribe(initialSubscription, rebalanceListener)(consumer) running <- Ref.make[EventLoopState](Running) - _ <- report(CreatingPollOnceFiber(clientId, group, consumerAttributes)) + _ <- report(CreatingPollOnceFiber(clientId, group, consumerAttributes)) fiber <- pollOnce(running, consumer, dispatcher, pausedPartitionsRef, positionsRef, offsets, config, clientId, group) .repeatWhile(_ == true) .forkDaemon - _ <- report(AwaitingPartitionsAssignment(clientId, group, consumerAttributes)) + _ <- report(AwaitingPartitionsAssignment(clientId, group, consumerAttributes)) _ <- partitionsAssigned.await env <- ZIO.environment[Env] } yield (dispatcher, fiber, offsets, positionsRef, running, rebalanceListener.provideEnvironment(env)) @@ -303,9 +303,11 @@ object EventLoopMetric { case class FailedToUpdatePositions(t: Throwable, clientId: ClientId, attributes: Map[String, String] = Map.empty) extends EventLoopMetric - case class CreatingDispatcher(clientId: ClientId, group: Group, attributes: Map[String, String], startPaused: Boolean) extends EventLoopMetric + case class CreatingDispatcher(clientId: ClientId, group: Group, attributes: Map[String, String], startPaused: Boolean) + extends EventLoopMetric - case class SubscribingToInitialSubAndRebalanceListener(clientId: ClientId, group: Group, attributes: Map[String, String]) extends EventLoopMetric + case class SubscribingToInitialSubAndRebalanceListener(clientId: ClientId, group: Group, attributes: Map[String, String]) + extends EventLoopMetric case class CreatingPollOnceFiber(clientId: ClientId, group: Group, attributes: Map[String, String]) extends EventLoopMetric diff --git a/core/src/main/scala/com/wixpress/dst/greyhound/core/consumer/OffsetsAndGaps.scala b/core/src/main/scala/com/wixpress/dst/greyhound/core/consumer/OffsetsAndGaps.scala new file mode 100644 index 00000000..f5718555 --- /dev/null +++ b/core/src/main/scala/com/wixpress/dst/greyhound/core/consumer/OffsetsAndGaps.scala @@ -0,0 +1,92 @@ +package com.wixpress.dst.greyhound.core.consumer + +import com.wixpress.dst.greyhound.core.{Offset, TopicPartition} +import zio._ + +trait OffsetsAndGaps { + def getCommittableAndClear: UIO[Map[TopicPartition, OffsetAndGaps]] + + def gapsForPartition(partition: TopicPartition): UIO[Seq[Gap]] + + def update(partition: TopicPartition, batch: Seq[Offset]): UIO[Unit] + + def contains(partition: TopicPartition, offset: Offset): UIO[Boolean] +} + +object OffsetsAndGaps { + def make: UIO[OffsetsAndGaps] = + Ref.make(Map.empty[TopicPartition, OffsetAndGaps]).map { ref => + new OffsetsAndGaps { + override def getCommittableAndClear: UIO[Map[TopicPartition, OffsetAndGaps]] = + ref.modify(offsetsAndGaps => { + val committable = offsetsAndGaps.filter(_._2.committable) + val updated = offsetsAndGaps.mapValues(_.markCommitted) + (committable, updated) + }) + + override def gapsForPartition(partition: TopicPartition): UIO[Seq[Gap]] = + ref.get.map(_.get(partition).fold(Seq.empty[Gap])(_.gaps.sortBy(_.start))) + + override def update(partition: TopicPartition, batch: Seq[Offset]): UIO[Unit] = + ref.update { offsetsAndGaps => + val sortedBatch = batch.sorted + val maxBatchOffset = sortedBatch.last + val maybeOffsetAndGaps = offsetsAndGaps.get(partition) + val prevOffset = maybeOffsetAndGaps.fold(-1L)(_.offset) + val partitionOffsetAndGaps = maybeOffsetAndGaps.fold(OffsetAndGaps(maxBatchOffset))(identity) + + val newGaps = gapsInBatch(sortedBatch, prevOffset) + + val updatedGaps = updateGapsByOffsets( + partitionOffsetAndGaps.gaps ++ newGaps, + sortedBatch + ) + + offsetsAndGaps + (partition -> OffsetAndGaps(maxBatchOffset max prevOffset, updatedGaps)) + }.unit + + override def contains(partition: TopicPartition, offset: Offset): UIO[Boolean] = + ref.get.map(_.get(partition).fold(false)(_.contains(offset))) + + private def gapsInBatch(batch: Seq[Offset], prevLastOffset: Offset): Seq[Gap] = + batch.sorted + .foldLeft(Seq.empty[Gap], prevLastOffset) { + case ((gaps, lastOffset), offset) => + if (offset <= lastOffset) (gaps, lastOffset) + else if (offset == lastOffset + 1) (gaps, offset) + else { + val newGap = Gap(lastOffset + 1, offset - 1) + (newGap +: gaps, offset) + } + } + ._1 + .reverse + + private def updateGapsByOffsets(gaps: Seq[Gap], offsets: Seq[Offset]): Seq[Gap] = { + val gapsToOffsets = gaps.map(gap => gap -> offsets.filter(o => o >= gap.start && o <= gap.end)).toMap + gapsToOffsets.flatMap { + case (gap, offsets) => + if (offsets.isEmpty) Seq(gap) + else if (offsets.size == (gap.size)) Seq.empty[Gap] + else gapsInBatch(offsets ++ Seq(gap.start - 1, gap.end + 1), gap.start - 2) + }.toSeq + } + } + } +} + +case class Gap(start: Offset, end: Offset) { + def contains(offset: Offset): Boolean = start <= offset && offset <= end + + def size: Long = end - start + 1 +} + +case class OffsetAndGaps(offset: Offset, gaps: Seq[Gap], committable: Boolean = true) { + def contains(offset: Offset): Boolean = gaps.exists(_.contains(offset)) + + def markCommitted: OffsetAndGaps = copy(committable = false) +} + +object OffsetAndGaps { + def apply(offset: Offset): OffsetAndGaps = OffsetAndGaps(offset, Seq.empty[Gap]) +} diff --git a/core/src/main/scala/com/wixpress/dst/greyhound/core/consumer/OffsetsInitializer.scala b/core/src/main/scala/com/wixpress/dst/greyhound/core/consumer/OffsetsInitializer.scala index 97f1b341..e684eef7 100644 --- a/core/src/main/scala/com/wixpress/dst/greyhound/core/consumer/OffsetsInitializer.scala +++ b/core/src/main/scala/com/wixpress/dst/greyhound/core/consumer/OffsetsInitializer.scala @@ -40,8 +40,10 @@ class OffsetsInitializer( offsetOperations.pause(toPause) val rewindUncommittedOffsets = if (offsetResetIsEarliest || notCommitted.isEmpty || rewindUncommittedOffsetsBy.isZero) Map.empty - else offsetOperations.offsetsForTimes(notCommitted, clock.millis() - rewindUncommittedOffsetsBy.toMillis, effectiveTimeout) - .map{case (tp, maybeRewindedOffset) => (tp, maybeRewindedOffset.orElse(endOffsets.get(tp)).getOrElse(0L))} + else + offsetOperations + .offsetsForTimes(notCommitted, clock.millis() - rewindUncommittedOffsetsBy.toMillis, effectiveTimeout) + .map { case (tp, maybeRewindedOffset) => (tp, maybeRewindedOffset.orElse(endOffsets.get(tp)).getOrElse(0L)) } val positions = notCommitted.map(tp => tp -> offsetOperations.position(tp, effectiveTimeout)).toMap ++ toOffsets ++ rewindUncommittedOffsets diff --git a/core/src/main/scala/com/wixpress/dst/greyhound/core/consumer/RecordConsumer.scala b/core/src/main/scala/com/wixpress/dst/greyhound/core/consumer/RecordConsumer.scala index 883f17a8..1b426d60 100644 --- a/core/src/main/scala/com/wixpress/dst/greyhound/core/consumer/RecordConsumer.scala +++ b/core/src/main/scala/com/wixpress/dst/greyhound/core/consumer/RecordConsumer.scala @@ -79,7 +79,7 @@ object RecordConsumer { (initialSubscription, topicsToCreate) = config.retryConfig.fold((config.initialSubscription, Set.empty[Topic]))(policy => maybeAddRetryTopics(policy, config, nonBlockingRetryHelper) ) - _ <- AdminClient + _ <- ZIO.when(config.createRetryTopics)(AdminClient .make(AdminClientConfig(config.bootstrapServers, config.kafkaAuthProperties), config.consumerAttributes) .tap(client => client.createTopics( @@ -87,7 +87,7 @@ object RecordConsumer { TopicConfig(topic, partitions = 1, replicationFactor = 1, cleanupPolicy = CleanupPolicy.Delete(86400000L)) ) ) - ) + )) blockingState <- Ref.make[Map[BlockingTarget, BlockingState]](Map.empty) blockingStateResolver = BlockingStateResolver(blockingState) workersShutdownRef <- Ref.make[Map[TopicPartition, ShutdownPromise]](Map.empty) @@ -321,7 +321,8 @@ case class RecordConsumerConfig( decryptor: Decryptor[Any, Throwable, Chunk[Byte], Chunk[Byte]] = new NoOpDecryptor, retryProducerAttributes: Map[String, String] = Map.empty, commitMetadataString: Metadata = OffsetAndMetadata.NO_METADATA, - rewindUncommittedOffsetsBy: Duration = 0.millis + rewindUncommittedOffsetsBy: Duration = 0.millis, + createRetryTopics: Boolean = true ) extends CommonGreyhoundConfig { override def kafkaProps: Map[String, String] = extraProperties diff --git a/core/src/main/scala/com/wixpress/dst/greyhound/core/consumer/ReportingConsumer.scala b/core/src/main/scala/com/wixpress/dst/greyhound/core/consumer/ReportingConsumer.scala index 96a929f8..d0f93a00 100644 --- a/core/src/main/scala/com/wixpress/dst/greyhound/core/consumer/ReportingConsumer.scala +++ b/core/src/main/scala/com/wixpress/dst/greyhound/core/consumer/ReportingConsumer.scala @@ -41,10 +41,14 @@ case class ReportingConsumer(clientId: ClientId, group: Group, internal: Consume implicit trace: Trace ): UIO[DelayedRebalanceEffect] = (report(PartitionsRevoked(clientId, group, partitions, config.consumerAttributes)) *> - rebalanceListener.onPartitionsRevoked(consumer, partitions) - .timed.tap { case (duration, _) => report(PartitionsRevokedComplete(clientId, group, partitions, config.consumerAttributes, duration.toMillis)) } - .map(_._2) - ).provideEnvironment(r) + rebalanceListener + .onPartitionsRevoked(consumer, partitions) + .timed + .tap { + case (duration, _) => + report(PartitionsRevokedComplete(clientId, group, partitions, config.consumerAttributes, duration.toMillis)) + } + .map(_._2)).provideEnvironment(r) override def onPartitionsAssigned(consumer: Consumer, partitions: Set[TopicPartition])(implicit trace: Trace): UIO[Any] = (report(PartitionsAssigned(clientId, group, partitions, config.consumerAttributes)) *> @@ -241,11 +245,13 @@ object ConsumerMetric { attributes: Map[String, String] = Map.empty ) extends ConsumerMetric - case class PartitionsRevokedComplete(clientId: ClientId, - group: Group, - partitions: Set[TopicPartition], - attributes: Map[String, String] = Map.empty, - durationMs: Long) extends ConsumerMetric + case class PartitionsRevokedComplete( + clientId: ClientId, + group: Group, + partitions: Set[TopicPartition], + attributes: Map[String, String] = Map.empty, + durationMs: Long + ) extends ConsumerMetric case class SubscribeFailed(clientId: ClientId, group: Group, error: Throwable, attributes: Map[String, String] = Map.empty) extends ConsumerMetric diff --git a/core/src/main/scala/com/wixpress/dst/greyhound/core/consumer/retry/BlockingRetryRecordHandler.scala b/core/src/main/scala/com/wixpress/dst/greyhound/core/consumer/retry/BlockingRetryRecordHandler.scala index 502e31e1..2f5d59a9 100644 --- a/core/src/main/scala/com/wixpress/dst/greyhound/core/consumer/retry/BlockingRetryRecordHandler.scala +++ b/core/src/main/scala/com/wixpress/dst/greyhound/core/consumer/retry/BlockingRetryRecordHandler.scala @@ -3,9 +3,8 @@ package com.wixpress.dst.greyhound.core.consumer.retry import java.util.concurrent.TimeUnit import com.wixpress.dst.greyhound.core.{Group, TopicPartition} import com.wixpress.dst.greyhound.core.consumer.domain.{ConsumerRecord, RecordHandler} -import com.wixpress.dst.greyhound.core.consumer.retry.BlockingState.{Blocked, Blocking => InternalBlocking, IgnoringOnce} +import com.wixpress.dst.greyhound.core.consumer.retry.BlockingState.{Blocked, IgnoringOnce, Blocking => InternalBlocking} import com.wixpress.dst.greyhound.core.consumer.retry.RetryRecordHandlerMetric.{BlockingRetryHandlerInvocationFailed, DoneBlockingBeforeRetry, NoRetryOnNonRetryableFailure} -import com.wixpress.dst.greyhound.core.consumer.retry.ZIOHelper.foreachWhile import com.wixpress.dst.greyhound.core.metrics.GreyhoundMetrics import com.wixpress.dst.greyhound.core.metrics.GreyhoundMetrics.report import com.wixpress.dst.greyhound.core.zioutils.AwaitShutdown @@ -31,7 +30,7 @@ private[retry] object BlockingRetryRecordHandler { override def handle(record: ConsumerRecord[K, V])(implicit trace: Trace): ZIO[GreyhoundMetrics with R, Nothing, LastHandleResult] = { val topicPartition = TopicPartition(record.topic, record.partition) - def pollBlockingStateWithSuspensions(interval: Duration, start: Long): URIO[GreyhoundMetrics, PollResult] = { + def pollBlockingStateWithSuspensions(record: ConsumerRecord[K, V], interval: Duration, start: Long): URIO[GreyhoundMetrics, PollResult] = { for { shouldBlock <- blockingStateResolver.resolve(record) shouldPollAgain <- @@ -43,14 +42,14 @@ private[retry] object BlockingRetryRecordHandler { } yield shouldPollAgain } - def blockOnErrorFor(interval: Duration) = { + def blockOnErrorFor(record: ConsumerRecord[K, V], interval: Duration) = { for { start <- currentTime(TimeUnit.MILLISECONDS) continueBlocking <- if (interval.toMillis > 100L) { awaitShutdown(record.topicPartition).flatMap( _.interruptOnShutdown( - pollBlockingStateWithSuspensions(interval, start).repeatWhile(result => result.pollAgain).map(_.blockHandling) + pollBlockingStateWithSuspensions(record, interval, start).repeatWhile(result => result.pollAgain).map(_.blockHandling) ).reporting(r => DoneBlockingBeforeRetry(record.topic, record.partition, record.offset, r.duration, r.failed)) ) } else { @@ -63,6 +62,7 @@ private[retry] object BlockingRetryRecordHandler { } def handleAndMaybeBlockOnErrorFor( + record: ConsumerRecord[K, V], interval: Option[Duration] ): ZIO[R with GreyhoundMetrics, Nothing, LastHandleResult] = { handler.handle(record).map(_ => LastHandleResult(lastHandleSucceeded = true, shouldContinue = false)).catchAll { @@ -73,7 +73,7 @@ private[retry] object BlockingRetryRecordHandler { case error => interval .map { interval => - report(BlockingRetryHandlerInvocationFailed(topicPartition, record.offset, error.toString)) *> blockOnErrorFor(interval) + report(BlockingRetryHandlerInvocationFailed(topicPartition, record.offset, error.toString)) *> blockOnErrorFor(record, interval) } .getOrElse(ZIO.succeed(LastHandleResult(lastHandleSucceeded = false, shouldContinue = false))) } @@ -96,13 +96,44 @@ private[retry] object BlockingRetryRecordHandler { } else { val durationsIncludingForInvocationWithNoErrorHandling = retryConfig.blockingBackoffs(record.topic)().map(Some(_)) :+ None for { - result <- foreachWhile(durationsIncludingForInvocationWithNoErrorHandling) { interval => handleAndMaybeBlockOnErrorFor(interval) } - _ <- maybeBackToStateBlocking + result <- retryEvery(record, durationsIncludingForInvocationWithNoErrorHandling) { (rec, interval) => + handleAndMaybeBlockOnErrorFor(rec, interval) + } + _ <- maybeBackToStateBlocking } yield result } } } + private def retryEvery[K, V, R, E](record: ConsumerRecord[K, V], as: Iterable[Option[Duration]])( + f: (ConsumerRecord[K, V], Option[Duration]) => ZIO[R, E, LastHandleResult] + )(implicit trace: Trace): ZIO[R, E, LastHandleResult] = { + ZIO.succeed(as.iterator).flatMap { i => + def loop(retryAttempt: Option[RetryAttempt]): ZIO[R, E, LastHandleResult] = + if (i.hasNext) { + val nextDelay = i.next + val recordWithAttempt = retryAttempt.fold(record) { attempt => + record.copy(headers = record.headers ++ RetryAttempt.toHeaders(attempt)) + } + f(recordWithAttempt, nextDelay).flatMap { result => + if (result.shouldContinue) Clock.instant.flatMap { now => + val nextAttempt = RetryAttempt( + originalTopic = record.topic, + attempt = retryAttempt.fold(0)(_.attempt + 1), + submittedAt = now, + backoff = nextDelay getOrElse Duration.Zero + ) + loop(Some(nextAttempt)) + } + else ZIO.succeed(result) + } + } + else ZIO.succeed(LastHandleResult(lastHandleSucceeded = false, shouldContinue = false)) + + loop(None) + } + } + private def handleNonRetriable[K, V, E, R](record: ConsumerRecord[K, V], topicPartition: TopicPartition, cause: Exception) = report(NoRetryOnNonRetryableFailure(topicPartition, record.offset, cause)) .as(LastHandleResult(lastHandleSucceeded = false, shouldContinue = false)) diff --git a/core/src/main/scala/com/wixpress/dst/greyhound/core/consumer/retry/NonBlockingRetryHelper.scala b/core/src/main/scala/com/wixpress/dst/greyhound/core/consumer/retry/NonBlockingRetryHelper.scala index c8dd3edd..1d1cd24c 100644 --- a/core/src/main/scala/com/wixpress/dst/greyhound/core/consumer/retry/NonBlockingRetryHelper.scala +++ b/core/src/main/scala/com/wixpress/dst/greyhound/core/consumer/retry/NonBlockingRetryHelper.scala @@ -1,40 +1,31 @@ package com.wixpress.dst.greyhound.core.consumer.retry -import java.time.{Instant, Duration => JavaDuration} -import java.util.concurrent.TimeUnit.MILLISECONDS -import java.util.regex.Pattern -import com.wixpress.dst.greyhound.core.Serdes.StringSerde import com.wixpress.dst.greyhound.core.consumer.domain.ConsumerSubscription.{TopicPattern, Topics} -import com.wixpress.dst.greyhound.core.consumer.retry.RetryAttempt.{RetryAttemptNumber, currentTime} -import com.wixpress.dst.greyhound.core.consumer.retry.RetryDecision.{NoMoreRetries, RetryWith} import com.wixpress.dst.greyhound.core.consumer.domain.{ConsumerRecord, ConsumerSubscription} +import com.wixpress.dst.greyhound.core.consumer.retry.RetryDecision.{NoMoreRetries, RetryWith} import com.wixpress.dst.greyhound.core.consumer.retry.RetryRecordHandlerMetric.WaitingForRetry import com.wixpress.dst.greyhound.core.metrics.GreyhoundMetrics import com.wixpress.dst.greyhound.core.metrics.GreyhoundMetrics.report import com.wixpress.dst.greyhound.core.producer.ProducerRecord -import com.wixpress.dst.greyhound.core.{Group, Headers, Topic, durationDeserializer, instantDeserializer} -import zio.Clock -import zio.Duration +import com.wixpress.dst.greyhound.core.{Group, Topic} import zio.Schedule.spaced -import zio.{Chunk, UIO, URIO, _} +import zio.{Chunk, Clock, Duration, URIO, _} +import java.time.Instant +import java.util.regex.Pattern import scala.util.Try trait NonBlockingRetryHelper { def retryTopicsFor(originalTopic: Topic): Set[Topic] - def retryAttempt(topic: Topic, headers: Headers, subscription: ConsumerSubscription)( - implicit trace: Trace - ): UIO[Option[RetryAttempt]] - def retryDecision[E]( - retryAttempt: Option[RetryAttempt], - record: ConsumerRecord[Chunk[Byte], Chunk[Byte]], - error: E, - subscription: ConsumerSubscription + retryAttempt: Option[RetryAttempt], + record: ConsumerRecord[Chunk[Byte], Chunk[Byte]], + error: E, + subscription: ConsumerSubscription )(implicit trace: Trace): URIO[Any, RetryDecision] - def retrySteps = retryTopicsFor("").size + def retrySteps: Int = retryTopicsFor("").size } object NonBlockingRetryHelper { @@ -47,95 +38,81 @@ object NonBlockingRetryHelper { .getOrElse(NonBlockingBackoffPolicy.empty) override def retryTopicsFor(topic: Topic): Set[Topic] = - policy(topic).intervals.indices.foldLeft(Set.empty[String])((acc, attempt) => - acc + s"$topic-$group-retry-$attempt" - ) - - override def retryAttempt(topic: Topic, headers: Headers, subscription: ConsumerSubscription)( - implicit trace: Trace - ): UIO[Option[RetryAttempt]] = { - (for { - submitted <- headers.get(RetryHeader.Submitted, instantDeserializer) - backoff <- headers.get(RetryHeader.Backoff, durationDeserializer) - originalTopic <- headers.get[String](RetryHeader.OriginalTopic, StringSerde) - } yield for { - ta <- topicAttempt(subscription, topic, originalTopic) - TopicAttempt(originalTopic, attempt) = ta - s <- submitted - b <- backoff - } yield RetryAttempt(originalTopic, attempt, s, b)) - .catchAll(_ => ZIO.none) - } - - private def topicAttempt( - subscription: ConsumerSubscription, - topic: Topic, - originalTopicHeader: Option[String] - ) = - subscription match { - case _: Topics => extractTopicAttempt(group, topic) - case _: TopicPattern => - extractTopicAttemptFromPatternRetryTopic(group, topic, originalTopicHeader) - } + policy(topic).intervals.indices.foldLeft(Set.empty[String])((acc, attempt) => acc + s"$topic-$group-retry-$attempt") override def retryDecision[E]( - retryAttempt: Option[RetryAttempt], - record: ConsumerRecord[Chunk[Byte], Chunk[Byte]], - error: E, - subscription: ConsumerSubscription - )(implicit trace: Trace): URIO[Any, RetryDecision] = currentTime.map(now => { - val nextRetryAttempt = retryAttempt.fold(0)(_.attempt + 1) + retryAttempt: Option[RetryAttempt], + record: ConsumerRecord[Chunk[Byte], Chunk[Byte]], + error: E, + subscription: ConsumerSubscription + )(implicit trace: Trace): URIO[Any, RetryDecision] = Clock.instant.map(now => { + val blockingRetriesBefore = RetryAttempt.maxBlockingAttempts( + NonBlockingRetryHelper.originalTopic(record.topic, group), + retryConfig + ).getOrElse(0) + + // attempt if present contains full number of retries + val nextNonBlockingAttempt = retryAttempt.fold(0)(_.attempt + 1 - blockingRetriesBefore) + val nextRetryAttempt = nextNonBlockingAttempt + blockingRetriesBefore val originalTopic = retryAttempt.fold(record.topic)(_.originalTopic) - val retryTopic = subscription match { - case _: TopicPattern => patternRetryTopic(group, nextRetryAttempt) - case _: Topics => fixedRetryTopic(originalTopic, group, nextRetryAttempt) + val retryTopic = subscription match { + case _: TopicPattern => patternRetryTopic(group, nextNonBlockingAttempt) + case _: Topics => fixedRetryTopic(originalTopic, group, nextNonBlockingAttempt) } val topicRetryPolicy = policy(record.topic) topicRetryPolicy.intervals - .lift(nextRetryAttempt) + .lift(nextNonBlockingAttempt) .map { backoff => + val attempt = RetryAttempt( + attempt = nextRetryAttempt, + originalTopic = originalTopic, + submittedAt = now, + backoff = backoff + ) topicRetryPolicy.recordMutate( ProducerRecord( topic = retryTopic, value = record.value, key = record.key, partition = None, - headers = record.headers + - (RetryHeader.Submitted -> toChunk(now.toEpochMilli)) + - (RetryHeader.Backoff -> toChunk(backoff.toMillis)) + - (RetryHeader.OriginalTopic -> toChunk(originalTopic)) + - (RetryHeader.RetryAttempt -> toChunk(nextRetryAttempt)) + headers = record.headers ++ RetryAttempt.toHeaders(attempt) ) ) } .fold[RetryDecision](NoMoreRetries)(RetryWith) }) + } - private def toChunk(long: Long): Chunk[Byte] = - Chunk.fromArray(long.toString.getBytes) - - private def toChunk(str: String): Chunk[Byte] = - Chunk.fromArray(str.getBytes) + private[retry] def attemptNumberFromTopic( + subscription: ConsumerSubscription, + topic: Topic, + originalTopicHeader: Option[String], + group: Group + ) = + subscription match { + case _: Topics => extractTopicAttempt(group, topic) + case _: TopicPattern => + extractTopicAttemptFromPatternRetryTopic(group, topic, originalTopicHeader) } - private def extractTopicAttempt[E](group: Group, inputTopic: Topic) = + private def extractTopicAttempt(group: Group, inputTopic: Topic) = inputTopic.split(s"-$group-retry-").toSeq match { case Seq(topic, attempt) if Try(attempt.toInt).isSuccess => Some(TopicAttempt(topic, attempt.toInt)) case _ => None } - private def extractTopicAttemptFromPatternRetryTopic[E]( - group: Group, - inputTopic: Topic, - originalTopicHeader: Option[String] + private def extractTopicAttemptFromPatternRetryTopic( + group: Group, + inputTopic: Topic, + originalTopicHeader: Option[String] ) = { originalTopicHeader.flatMap(originalTopic => { inputTopic.split(s"__gh_pattern-retry-$group-attempt-").toSeq match { case Seq(_, attempt) if Try(attempt.toInt).isSuccess => Some(TopicAttempt(originalTopic, attempt.toInt)) - case _ => None + case _ => None } }) } @@ -168,49 +145,27 @@ object DelayHeaders { val Backoff = "backOffTimeMs" } -object RetryHeader { - val Submitted = "submitTimestamp" - val Backoff = DelayHeaders.Backoff - val OriginalTopic = "GH_OriginalTopic" - val RetryAttempt = "GH_RetryAttempt" -} - -case class RetryAttempt( - originalTopic: Topic, - attempt: RetryAttemptNumber, - submittedAt: Instant, - backoff: Duration -) { - - def sleep(implicit trace: Trace): URIO[GreyhoundMetrics, Unit] = - (RetryUtil.sleep(submittedAt, backoff) race reportWaitingInIntervals(every = 60.seconds)) - - private def reportWaitingInIntervals(every: Duration) = - report(WaitingForRetry(originalTopic, attempt, submittedAt.toEpochMilli, backoff.toMillis)) - .repeat(spaced(every)) - .unit -} - object RetryUtil { + def sleep(attempt: RetryAttempt)(implicit trace: Trace): URIO[GreyhoundMetrics, Unit] = + sleep(attempt.submittedAt, attempt.backoff) race + report(WaitingForRetry(attempt.originalTopic, attempt.attempt, attempt.submittedAt.toEpochMilli, attempt.backoff.toMillis)) + .repeat(spaced(60.seconds)) + .unit + def sleep(submittedAt: Instant, backoff: Duration)(implicit trace: Trace): URIO[Any, Unit] = { val expiresAt = submittedAt.plus(backoff.asJava) - currentTime + Clock.instant .map(_.isAfter(expiresAt)) .flatMap(expired => if (expired) ZIO.unit else - ZIO.sleep(1.seconds).repeatUntilZIO(_ => currentTime.map(_.isAfter(expiresAt))).unit + ZIO.sleep(1.second).repeatUntilZIO(_ => Clock.instant.map(_.isAfter(expiresAt))).unit ) } } private case class TopicAttempt(originalTopic: Topic, attempt: Int) -object RetryAttempt { - type RetryAttemptNumber = Int - val currentTime = Clock.currentTime(MILLISECONDS).map(Instant.ofEpochMilli) -} - sealed trait RetryDecision object RetryDecision { diff --git a/core/src/main/scala/com/wixpress/dst/greyhound/core/consumer/retry/NonBlockingRetryRecordHandler.scala b/core/src/main/scala/com/wixpress/dst/greyhound/core/consumer/retry/NonBlockingRetryRecordHandler.scala index b6e410d3..a6ff6560 100644 --- a/core/src/main/scala/com/wixpress/dst/greyhound/core/consumer/retry/NonBlockingRetryRecordHandler.scala +++ b/core/src/main/scala/com/wixpress/dst/greyhound/core/consumer/retry/NonBlockingRetryRecordHandler.scala @@ -31,11 +31,12 @@ private[retry] object NonBlockingRetryRecordHandler { retryConfig: RetryConfig, subscription: ConsumerSubscription, nonBlockingRetryHelper: NonBlockingRetryHelper, + groupId: Group, awaitShutdown: TopicPartition => UIO[AwaitShutdown] )(implicit evK: K <:< Chunk[Byte], evV: V <:< Chunk[Byte]): NonBlockingRetryRecordHandler[V, K, R] = new NonBlockingRetryRecordHandler[V, K, R] { override def handle(record: ConsumerRecord[K, V]): ZIO[GreyhoundMetrics with R, Nothing, Any] = { - nonBlockingRetryHelper.retryAttempt(record.topic, record.headers, subscription).flatMap { retryAttempt => + RetryAttempt.extract(record.headers, record.topic, groupId, subscription, Some(retryConfig)).flatMap { retryAttempt => maybeDelayRetry(record, retryAttempt) *> handler.handle(record).catchAll { case Right(_: NonRetriableException) => ZIO.unit @@ -49,14 +50,17 @@ private[retry] object NonBlockingRetryRecordHandler { } private def delayRetry(record: ConsumerRecord[_, _], awaitShutdown: TopicPartition => UIO[AwaitShutdown])( - retryAttempt: RetryAttempt) = + retryAttempt: RetryAttempt + ) = zio.Random.nextInt.flatMap(correlationId => report( WaitingBeforeRetry(record.topic, retryAttempt, record.partition, record.offset, correlationId) ) *> awaitShutdown(record.topicPartition) - .flatMap(_.interruptOnShutdown(retryAttempt.sleep)) - .reporting(r => DoneWaitingBeforeRetry(record.topic, record.partition, record.offset, retryAttempt, r.duration, r.failed, correlationId)) + .flatMap(_.interruptOnShutdown(RetryUtil.sleep(retryAttempt))) + .reporting(r => + DoneWaitingBeforeRetry(record.topic, record.partition, record.offset, retryAttempt, r.duration, r.failed, correlationId) + ) ) override def isHandlingRetryTopicMessage(group: Group, record: ConsumerRecord[K, V]): Boolean = { @@ -71,7 +75,7 @@ private[retry] object NonBlockingRetryRecordHandler { override def handleAfterBlockingFailed( record: ConsumerRecord[K, V] ): ZIO[GreyhoundMetrics with R, Nothing, Any] = { - nonBlockingRetryHelper.retryAttempt(record.topic, record.headers, subscription).flatMap { retryAttempt => + RetryAttempt.extract(record.headers, record.topic, groupId, subscription, Some(retryConfig)).flatMap { retryAttempt => maybeRetry(retryAttempt, BlockingHandlerFailed, record) } } diff --git a/core/src/main/scala/com/wixpress/dst/greyhound/core/consumer/retry/RetryAttempt.scala b/core/src/main/scala/com/wixpress/dst/greyhound/core/consumer/retry/RetryAttempt.scala new file mode 100644 index 00000000..3fa3cba8 --- /dev/null +++ b/core/src/main/scala/com/wixpress/dst/greyhound/core/consumer/retry/RetryAttempt.scala @@ -0,0 +1,98 @@ +package com.wixpress.dst.greyhound.core.consumer.retry + +import com.wixpress.dst.greyhound.core.Serdes.StringSerde +import com.wixpress.dst.greyhound.core._ +import com.wixpress.dst.greyhound.core.consumer.domain.ConsumerSubscription +import com.wixpress.dst.greyhound.core.consumer.retry.NonBlockingRetryHelper.attemptNumberFromTopic +import com.wixpress.dst.greyhound.core.consumer.retry.RetryAttempt.RetryAttemptNumber +import zio._ + +import java.time.Instant + +/** + * Description of a retry attempt + * @param attempt contains which attempt is it, starting from 0 including blocking and non-blocking attempts + */ +case class RetryAttempt( + originalTopic: Topic, + attempt: RetryAttemptNumber, + submittedAt: Instant, + backoff: Duration +) + +object RetryHeader { + val Submitted = "submitTimestamp" + val Backoff = DelayHeaders.Backoff + val OriginalTopic = "GH_OriginalTopic" + val RetryAttempt = "GH_RetryAttempt" +} + +object RetryAttempt { + type RetryAttemptNumber = Int + + private def toChunk(str: String): Chunk[Byte] = Chunk.fromArray(str.getBytes) + + def toHeaders(attempt: RetryAttempt): Headers = Headers( + RetryHeader.Submitted -> toChunk(attempt.submittedAt.toEpochMilli.toString), + RetryHeader.Backoff -> toChunk(attempt.backoff.toMillis.toString), + RetryHeader.OriginalTopic -> toChunk(attempt.originalTopic), + RetryHeader.RetryAttempt -> toChunk(attempt.attempt.toString), + ) + + private case class RetryAttemptHeaders( + originalTopic: Option[Topic], + attempt: Option[RetryAttemptNumber], + submittedAt: Option[Instant], + backoff: Option[Duration] + ) + + private def fromHeaders(headers: Headers): Task[RetryAttemptHeaders] = + for { + submitted <- headers.get(RetryHeader.Submitted, instantDeserializer) + backoff <- headers.get(RetryHeader.Backoff, durationDeserializer) + topic <- headers.get[String](RetryHeader.OriginalTopic, StringSerde) + attempt <- headers.get(RetryHeader.RetryAttempt, longDeserializer) + } yield RetryAttemptHeaders(topic, attempt.map(_.toInt), submitted, backoff) + + /** @return None on infinite blocking retries */ + def maxBlockingAttempts(topic: Topic, retryConfig: Option[RetryConfig]): Option[Int] = + retryConfig.map(_.blockingBackoffs(topic)()).fold(Option(0)) { + case finite if finite.hasDefiniteSize => Some(finite.size) + case _ => None + } + + /** @return None on infinite retries */ + def maxOverallAttempts(topic: Topic, retryConfig: Option[RetryConfig]): Option[Int] = + maxBlockingAttempts(topic, retryConfig).map { + _ + retryConfig.fold(0)(_.nonBlockingBackoffs(topic).length) + } + + def extract( + headers: Headers, + topic: Topic, + group: Group, + subscription: ConsumerSubscription, + retryConfig: Option[RetryConfig], + )(implicit trace: Trace): UIO[Option[RetryAttempt]] = { + + def maybeNonBlockingAttempt(hs: RetryAttemptHeaders): Option[RetryAttempt] = + for { + submitted <- hs.submittedAt + backoff <- hs.backoff + TopicAttempt(originalTopic, attempt) <- attemptNumberFromTopic(subscription, topic, hs.originalTopic, group) + blockingRetries = maxBlockingAttempts(originalTopic, retryConfig).getOrElse(0) + } yield RetryAttempt(originalTopic, blockingRetries + attempt, submitted, backoff) + + def maybeBlockingAttempt(hs: RetryAttemptHeaders): Option[RetryAttempt] = + for { + submitted <- hs.submittedAt + backoff <- hs.backoff + originalTopic <- hs.originalTopic if originalTopic == topic + attempt <- hs.attempt + } yield RetryAttempt(originalTopic, attempt, submitted, backoff) + + fromHeaders(headers).map { hs => + maybeNonBlockingAttempt(hs) orElse maybeBlockingAttempt(hs) + } + }.catchAll(_ => ZIO.none) +} diff --git a/core/src/main/scala/com/wixpress/dst/greyhound/core/consumer/retry/RetryRecordHandler.scala b/core/src/main/scala/com/wixpress/dst/greyhound/core/consumer/retry/RetryRecordHandler.scala index af0749c6..8ee8ab9f 100644 --- a/core/src/main/scala/com/wixpress/dst/greyhound/core/consumer/retry/RetryRecordHandler.scala +++ b/core/src/main/scala/com/wixpress/dst/greyhound/core/consumer/retry/RetryRecordHandler.scala @@ -33,7 +33,7 @@ object RetryRecordHandler { ): RecordHandler[R with R2 with GreyhoundMetrics, Nothing, K, V] = { val nonBlockingHandler = - NonBlockingRetryRecordHandler(handler, producer, retryConfig, subscription, nonBlockingRetryHelper, awaitShutdown) + NonBlockingRetryRecordHandler(handler, producer, retryConfig, subscription, nonBlockingRetryHelper, groupId, awaitShutdown) val blockingHandler = BlockingRetryRecordHandler(groupId, handler, retryConfig, blockingState, nonBlockingHandler, awaitShutdown) val blockingAndNonBlockingHandler = BlockingAndNonBlockingRetryRecordHandler(groupId, blockingHandler, nonBlockingHandler) @@ -55,15 +55,4 @@ object RetryRecordHandler { record.headers.get[String](key, StringSerde).catchAll(_ => ZIO.none) } -object ZIOHelper { - def foreachWhile[R, E, A](as: Iterable[A])(f: A => ZIO[R, E, LastHandleResult])(implicit trace: Trace): ZIO[R, E, LastHandleResult] = - ZIO.succeed(as.iterator).flatMap { i => - def loop: ZIO[R, E, LastHandleResult] = - if (i.hasNext) f(i.next).flatMap(result => if (result.shouldContinue) loop else ZIO.succeed(result)) - else ZIO.succeed(LastHandleResult(lastHandleSucceeded = false, shouldContinue = false)) - - loop - } -} - case class LastHandleResult(lastHandleSucceeded: Boolean, shouldContinue: Boolean) diff --git a/core/src/main/scala/com/wixpress/dst/greyhound/core/consumer/retry/RetryRecordHandlerMetric.scala b/core/src/main/scala/com/wixpress/dst/greyhound/core/consumer/retry/RetryRecordHandlerMetric.scala index 980a71f7..b1580b16 100644 --- a/core/src/main/scala/com/wixpress/dst/greyhound/core/consumer/retry/RetryRecordHandlerMetric.scala +++ b/core/src/main/scala/com/wixpress/dst/greyhound/core/consumer/retry/RetryRecordHandlerMetric.scala @@ -20,7 +20,8 @@ object RetryRecordHandlerMetric { case class NoRetryOnNonRetryableFailure(partition: TopicPartition, offset: Long, cause: Exception) extends RetryRecordHandlerMetric case object Silent extends RetryRecordHandlerMetric - case class WaitingBeforeRetry(retryTopic: Topic, retryAttempt: RetryAttempt, partition: Int, offset:Long, correlationId: Int) extends RetryRecordHandlerMetric + case class WaitingBeforeRetry(retryTopic: Topic, retryAttempt: RetryAttempt, partition: Int, offset: Long, correlationId: Int) + extends RetryRecordHandlerMetric case class DoneWaitingBeforeRetry( retryTopic: Topic, diff --git a/core/src/test/scala/com/wixpress/dst/greyhound/core/consumer/BUILD.bazel b/core/src/test/scala/com/wixpress/dst/greyhound/core/consumer/BUILD.bazel index 90ed4056..3e104e18 100644 --- a/core/src/test/scala/com/wixpress/dst/greyhound/core/consumer/BUILD.bazel +++ b/core/src/test/scala/com/wixpress/dst/greyhound/core/consumer/BUILD.bazel @@ -11,17 +11,14 @@ specs2_unit_test( deps = [ "@dev_zio_zio_managed_2_12", "@dev_zio_zio_stacktracer_2_12", - "//core/src/it/scala/com/wixpress/dst/greyhound/testkit", "//core/src/main/scala/com/wixpress/dst/greyhound/core", "//core/src/main/scala/com/wixpress/dst/greyhound/core/consumer", "//core/src/main/scala/com/wixpress/dst/greyhound/core/consumer/domain", "//core/src/main/scala/com/wixpress/dst/greyhound/core/consumer/retry", "//core/src/main/scala/com/wixpress/dst/greyhound/core/metrics", - "//core/src/main/scala/com/wixpress/dst/greyhound/core/producer", "//core/src/main/scala/com/wixpress/dst/greyhound/core/zioutils", "//core/src/test/resources", "//core/src/test/scala/com/wixpress/dst/greyhound/core/testkit", - "@ch_qos_logback_logback_classic", # "@dev_zio_izumi_reflect_2_12", "@dev_zio_zio_2_12", "@dev_zio_zio_streams_2_12", diff --git a/core/src/test/scala/com/wixpress/dst/greyhound/core/consumer/OffsetsAndGapsTest.scala b/core/src/test/scala/com/wixpress/dst/greyhound/core/consumer/OffsetsAndGapsTest.scala new file mode 100644 index 00000000..df765ce5 --- /dev/null +++ b/core/src/test/scala/com/wixpress/dst/greyhound/core/consumer/OffsetsAndGapsTest.scala @@ -0,0 +1,62 @@ +package com.wixpress.dst.greyhound.core.consumer + +import com.wixpress.dst.greyhound.core.TopicPartition +import com.wixpress.dst.greyhound.core.consumer.OffsetGapsTest._ +import com.wixpress.dst.greyhound.core.testkit.BaseTestNoEnv + +class OffsetsAndGapsTestGapsTest extends BaseTestNoEnv { + + "calculate gaps created by handled batch" in { + for { + offsetGaps <- OffsetsAndGaps.make + _ <- offsetGaps.update(topicPartition, Seq(1L, 3L, 7L)) + currentGaps <- offsetGaps.gapsForPartition(topicPartition) + } yield currentGaps must beEqualTo(Seq(Gap(0L, 0L), Gap(2L, 2L), Gap(4L, 6L))) + } + + "update offset and gaps according to handled batch" in { + for { + offsetGaps <- OffsetsAndGaps.make + _ <- offsetGaps.update(topicPartition, Seq(1L, 3L, 7L)) + _ <- offsetGaps.update(topicPartition, Seq(2L, 5L)) + getCommittableAndClear <- offsetGaps.getCommittableAndClear + } yield getCommittableAndClear must havePair(topicPartition -> OffsetAndGaps(7L, Seq(Gap(0L, 0L), Gap(4L, 4L), Gap(6L, 6L)))) + } + + "clear committable offsets" in { + for { + offsetGaps <- OffsetsAndGaps.make + _ <- offsetGaps.update(topicPartition, Seq(1L, 3L, 7L)) + _ <- offsetGaps.getCommittableAndClear + getCommittableAndClear <- offsetGaps.getCommittableAndClear + } yield getCommittableAndClear must beEmpty + } + + "do not clear gaps on retrieving current" in { + for { + offsetGaps <- OffsetsAndGaps.make + _ <- offsetGaps.update(topicPartition, Seq(1L, 3L, 7L)) + _ <- offsetGaps.gapsForPartition(topicPartition) + currentGaps <- offsetGaps.gapsForPartition(topicPartition) + } yield currentGaps must beEqualTo(Seq(Gap(0L, 0L), Gap(2L, 2L), Gap(4L, 6L))) + } + + "update with larger offset" in { + val partition0 = TopicPartition(topic, 0) + val partition1 = TopicPartition(topic, 1) + + for { + offsetGaps <- OffsetsAndGaps.make + _ <- offsetGaps.update(partition0, Seq(1L)) + _ <- offsetGaps.update(partition0, Seq(0L)) + _ <- offsetGaps.update(partition1, Seq(0L)) + current <- offsetGaps.getCommittableAndClear + } yield current must havePairs(partition0 -> OffsetAndGaps(1L, Seq()), partition1 -> OffsetAndGaps(0L, Seq())) + } + +} + +object OffsetGapsTest { + val topic = "some-topic" + val topicPartition = TopicPartition(topic, 0) +} diff --git a/core/src/test/scala/com/wixpress/dst/greyhound/core/consumer/OffsetsInitializerTest.scala b/core/src/test/scala/com/wixpress/dst/greyhound/core/consumer/OffsetsInitializerTest.scala index d8c0c28e..5e708452 100644 --- a/core/src/test/scala/com/wixpress/dst/greyhound/core/consumer/OffsetsInitializerTest.scala +++ b/core/src/test/scala/com/wixpress/dst/greyhound/core/consumer/OffsetsInitializerTest.scala @@ -16,7 +16,7 @@ class OffsetsInitializerTest extends SpecificationWithJUnit with Mockito { private val Seq(p1, p2, p3) = Seq("t1" -> 1, "t2" -> 2, "t3" -> 3).map(tp => TopicPartition(tp._1, tp._2)) private val partitions = Set(p1, p2, p3) private val p1Pos, p2Pos, p3Pos = randomInt.toLong - val epochTimeToRewind = 1000L + val epochTimeToRewind = 1000L "do nothing if no missing offsets" in new ctx { @@ -124,71 +124,73 @@ class OffsetsInitializerTest extends SpecificationWithJUnit with Mockito { reported must contain(CommittedMissingOffsetsFailed(clientId, group, partitions, Map.empty, elapsed = Duration.ZERO, e)) } - "rewind uncommitted offsets" in new ctx { - givenCommittedOffsets(partitions)(Map(p2 -> randomInt)) - givenPositions(p2 -> p2Pos, p3 -> p3Pos) - givenOffsetsForTimes(epochTimeToRewind, p1 -> 0L, p2 -> 1L) - - committer.initializeOffsets(partitions) + "rewind uncommitted offsets" in + new ctx { + givenCommittedOffsets(partitions)(Map(p2 -> randomInt)) + givenPositions(p2 -> p2Pos, p3 -> p3Pos) + givenOffsetsForTimes(epochTimeToRewind, p1 -> 0L, p2 -> 1L) - val missingOffsets = Map( - p1 -> p1Pos, - p3 -> p3Pos - ) + committer.initializeOffsets(partitions) - val rewindedOffsets = Map( - p1 -> 0L, - ) + val missingOffsets = Map( + p1 -> p1Pos, + p3 -> p3Pos + ) - there was - one(offsetOps).commit( - missingOffsets ++ rewindedOffsets, - timeout + val rewindedOffsets = Map( + p1 -> 0L ) - } - "rewind to endOffsets for uncommitted partitions when offsetsForTimes return null offsets " in new ctx { - givenCommittedOffsets(partitions)(Map(p2 -> randomInt, p3 -> randomInt)) - givenPositions(p3 -> p3Pos) - givenEndOffsets(partitions, timeout)(Map(p1 -> p1Pos)) - givenOffsetsForTimes(Set(p1))(p1 -> None /*kafka SDK returned null*/) + there was + one(offsetOps).commit( + missingOffsets ++ rewindedOffsets, + timeout + ) + } - committer.initializeOffsets(partitions) + "rewind to endOffsets for uncommitted partitions when offsetsForTimes return null offsets " in + new ctx { + givenCommittedOffsets(partitions)(Map(p2 -> randomInt, p3 -> randomInt)) + givenPositions(p3 -> p3Pos) + givenEndOffsets(partitions, timeout)(Map(p1 -> p1Pos)) + givenOffsetsForTimes(Set(p1))(p1 -> None /*kafka SDK returned null*/ ) - val committedOffsets = Map( - p1 -> p1Pos, - ) + committer.initializeOffsets(partitions) - there was - one(offsetOps).commit( - committedOffsets, - timeout + val committedOffsets = Map( + p1 -> p1Pos ) - } - "not rewind uncommitted offsets when offset reset is earliest" in new ctx(offsetReset = OffsetReset.Earliest) { - givenCommittedOffsets(partitions)(Map(p2 -> randomInt)) - givenPositions(p2 -> p2Pos, p3 -> p3Pos) - givenOffsetsForTimes(epochTimeToRewind, p1 -> 0L, p2 -> 1L) + there was + one(offsetOps).commit( + committedOffsets, + timeout + ) + } - committer.initializeOffsets(partitions) + "not rewind uncommitted offsets when offset reset is earliest" in + new ctx(offsetReset = OffsetReset.Earliest) { + givenCommittedOffsets(partitions)(Map(p2 -> randomInt)) + givenPositions(p2 -> p2Pos, p3 -> p3Pos) + givenOffsetsForTimes(epochTimeToRewind, p1 -> 0L, p2 -> 1L) - val missingOffsets = Map( - p1 -> p1Pos, - p3 -> p3Pos - ) + committer.initializeOffsets(partitions) - val rewindedOffsets = Map( - p1 -> 0L, - ) + val missingOffsets = Map( + p1 -> p1Pos, + p3 -> p3Pos + ) - there was - one(offsetOps).commit( - missingOffsets ++ rewindedOffsets, - timeout + val rewindedOffsets = Map( + p1 -> 0L ) - } + there was + one(offsetOps).commit( + missingOffsets ++ rewindedOffsets, + timeout + ) + } class ctx(val seekTo: Map[TopicPartition, SeekTo] = Map.empty, offsetReset: OffsetReset = OffsetReset.Latest) extends Scope { private val metricsLogRef = new AtomicReference(Seq.empty[GreyhoundMetric]) @@ -256,4 +258,4 @@ class OffsetsInitializerTest extends SpecificationWithJUnit with Mockito { private def randomStr = Random.alphanumeric.take(5).mkString private def randomInt = Random.nextInt(200) private def randomPartition = TopicPartition(randomStr, randomInt) -} \ No newline at end of file +} diff --git a/core/src/test/scala/com/wixpress/dst/greyhound/core/consumer/retry/BUILD.bazel b/core/src/test/scala/com/wixpress/dst/greyhound/core/consumer/retry/BUILD.bazel index b02990a0..dea7a061 100644 --- a/core/src/test/scala/com/wixpress/dst/greyhound/core/consumer/retry/BUILD.bazel +++ b/core/src/test/scala/com/wixpress/dst/greyhound/core/consumer/retry/BUILD.bazel @@ -17,7 +17,6 @@ specs2_unit_test( "//core/src/main/scala/com/wixpress/dst/greyhound/core/metrics", "//core/src/main/scala/com/wixpress/dst/greyhound/core/producer", "//core/src/main/scala/com/wixpress/dst/greyhound/core/zioutils", - "//core/src/test/scala/com/wixpress/dst/greyhound/core/consumer", "//core/src/test/scala/com/wixpress/dst/greyhound/core/testkit", # "@dev_zio_izumi_reflect_2_12", "@dev_zio_zio_2_12", diff --git a/core/src/test/scala/com/wixpress/dst/greyhound/core/consumer/retry/RetryAttemptTest.scala b/core/src/test/scala/com/wixpress/dst/greyhound/core/consumer/retry/RetryAttemptTest.scala new file mode 100644 index 00000000..9e27c10b --- /dev/null +++ b/core/src/test/scala/com/wixpress/dst/greyhound/core/consumer/retry/RetryAttemptTest.scala @@ -0,0 +1,94 @@ +package com.wixpress.dst.greyhound.core.consumer.retry + +import com.wixpress.dst.greyhound.core.consumer.domain.ConsumerSubscription +import com.wixpress.dst.greyhound.core.testkit.BaseTest +import zio.test.TestEnvironment + +import java.time.{Duration, Instant} +import scala.util.Random +import scala.concurrent.duration._ + +class RetryAttemptTest extends BaseTest[TestEnvironment] { + + "RetryAttempt.extract" should { + "deserialize attempt from headers for blocking retries" in { + val attempt = randomRetryAttempt + val headers = RetryAttempt.toHeaders(attempt) + val subscription = ConsumerSubscription.Topics(Set(attempt.originalTopic)) + for (result <- RetryAttempt.extract(headers, attempt.originalTopic, randomStr, subscription, None)) + yield result must beSome(attempt) + } + "deserialize attempt from headers and topic for non-blocking retries" in { + val attempt = randomRetryAttempt + // topic and attempt must be extracted from retryTopic + val headers = RetryAttempt.toHeaders(attempt.copy(originalTopic = "", attempt = -1)) + val subscription = ConsumerSubscription.Topics(Set(attempt.originalTopic)) + val group = randomStr + val retryTopic = NonBlockingRetryHelper.fixedRetryTopic(attempt.originalTopic, group, attempt.attempt) + for (result <- RetryAttempt.extract(headers, retryTopic, group, subscription, None)) + yield result must beSome(attempt) + } + "deserialize attempt for non-blocking retry after blocking retries" in { + val attempt = randomRetryAttempt + val headers = RetryAttempt.toHeaders(attempt) + val subscription = ConsumerSubscription.Topics(Set(attempt.originalTopic)) + val group = randomStr + val retries = RetryConfig.blockingFollowedByNonBlockingRetry( + blockingBackoffs = 1.milli :: 1.second :: Nil, + nonBlockingBackoffs = 5.minutes :: Nil, + ) + val retryTopic = NonBlockingRetryHelper.fixedRetryTopic(attempt.originalTopic, group, attempt.attempt) + for (result <- RetryAttempt.extract(headers, retryTopic, group, subscription, Some(retries))) + yield result must beSome(attempt.copy(attempt = attempt.attempt + 2)) // with 2 blocking retries before + } + "In case incorrect originalTopic header propagated from a different consumer group, ignore it," + + "do NOT consider it as if it's a non-blocking retry" in { + val attempt = propagatedRetryAttempt + val headers = RetryAttempt.toHeaders(attempt) + val currentTopic = "relevant-topic" + val subscription = ConsumerSubscription.Topics(Set(currentTopic)) + for (result <- RetryAttempt.extract(headers, currentTopic, randomStr, subscription, None)) + yield result must beNone + } + } + + "RetryAttempt.maxOverallAttempts" should { + "return 0 if no retries configured" in { + RetryAttempt.maxOverallAttempts(randomStr, None) must beSome(0) + } + "return max attempts for blocking retries" in { + val config = RetryConfig.finiteBlockingRetry(1.milli, 1.second) + RetryAttempt.maxOverallAttempts(randomStr, Some(config)) must beSome(2) + } + "return max attempts for non-blocking retries" in { + val config = RetryConfig.nonBlockingRetry(1.milli, 1.second, 5.minutes) + RetryAttempt.maxOverallAttempts(randomStr, Some(config)) must beSome(3) + } + "return max attempts for blocking retries followed by non-blocking" in { + val config = RetryConfig.blockingFollowedByNonBlockingRetry(1.milli :: 2.seconds :: Nil, 1.minute :: Nil) + RetryAttempt.maxOverallAttempts(randomStr, Some(config)) must beSome(3) + } + "return None for infinite blocking retries" in { + val config = RetryConfig.infiniteBlockingRetry(1.milli) + RetryAttempt.maxOverallAttempts(randomStr, Some(config)) must beNone + } + } + + override def env = testEnvironment + + private def randomStr = Random.alphanumeric.take(10).mkString + + private def randomRetryAttempt = RetryAttempt( + originalTopic = randomStr, + attempt = Random.nextInt(1000), + submittedAt = Instant.ofEpochMilli(math.abs(Random.nextLong())), + backoff = Duration.ofMillis(Random.nextInt(100000)) + ) + + private def propagatedRetryAttempt = RetryAttempt( + originalTopic = "some-other-topic", + attempt = Random.nextInt(1000), + submittedAt = Instant.ofEpochMilli(math.abs(Random.nextLong())), + backoff = Duration.ofMillis(Random.nextInt(100000)) + ) +} diff --git a/core/src/test/scala/com/wixpress/dst/greyhound/core/consumer/retry/RetryConsumerRecordHandlerTest.scala b/core/src/test/scala/com/wixpress/dst/greyhound/core/consumer/retry/RetryConsumerRecordHandlerTest.scala index c713fb12..0797ba25 100644 --- a/core/src/test/scala/com/wixpress/dst/greyhound/core/consumer/retry/RetryConsumerRecordHandlerTest.scala +++ b/core/src/test/scala/com/wixpress/dst/greyhound/core/consumer/retry/RetryConsumerRecordHandlerTest.scala @@ -4,7 +4,7 @@ import java.time.Instant import com.wixpress.dst.greyhound.core.Serdes._ import com.wixpress.dst.greyhound.core._ import com.wixpress.dst.greyhound.core.consumer.domain.ConsumerSubscription.Topics -import com.wixpress.dst.greyhound.core.consumer.domain.{ConsumerRecord, ConsumerSubscription, RecordHandler} +import com.wixpress.dst.greyhound.core.consumer.domain.{ConsumerRecord, RecordHandler} import com.wixpress.dst.greyhound.core.consumer.retry.BlockingState.{Blocked, Blocking => InternalBlocking, IgnoringAll, IgnoringOnce} import com.wixpress.dst.greyhound.core.consumer.retry.RetryConsumerRecordHandlerTest.{offset, partition, _} import com.wixpress.dst.greyhound.core.consumer.retry.RetryRecordHandlerMetric.{BlockingIgnoredForAllFor, BlockingIgnoredOnceFor, BlockingRetryHandlerInvocationFailed, NoRetryOnNonRetryableFailure} @@ -21,8 +21,6 @@ import zio.Random.{nextBytes, nextIntBounded} import zio.managed.UManaged import zio.test.TestClock -import scala.concurrent.TimeoutException - class RetryConsumerRecordHandlerTest extends BaseTest[TestClock with TestMetrics] { override def env: UManaged[ZEnvironment[TestClock with TestMetrics]] = @@ -52,9 +50,6 @@ class RetryConsumerRecordHandlerTest extends BaseTest[TestClock with TestMetrics _ <- retryHandler.handle(ConsumerRecord(topic, partition, offset, Headers.Empty, Some(key), value, 0L, 0L, 0L)) record <- producer.records.take now <- currentTime - retryAttempt <- IntSerde.serialize(retryTopic, 0) - submittedAt <- InstantSerde.serialize(retryTopic, now) - backoff <- DurationSerde.serialize(retryTopic, 1.second) } yield { record === ProducerRecord( @@ -62,7 +57,7 @@ class RetryConsumerRecordHandlerTest extends BaseTest[TestClock with TestMetrics value, Some(key), partition = None, - headers = Headers("retry-attempt" -> retryAttempt, "retry-submitted-at" -> submittedAt, "retry-backoff" -> backoff) + headers = RetryAttempt.toHeaders(RetryAttempt(topic, 0, now, 1.second)) ) } } @@ -71,7 +66,8 @@ class RetryConsumerRecordHandlerTest extends BaseTest[TestClock with TestMetrics for { producer <- FakeProducer.make topic <- randomTopicName - retryTopic = s"$topic-retry" + attempt = 0 + retryTopic = NonBlockingRetryHelper.fixedRetryTopic(topic, group, attempt) executionTime <- Promise.make[Nothing, Instant] handler = RecordHandler[Clock, HandlerError, Chunk[Byte], Chunk[Byte]] { _ => currentTime.flatMap(executionTime.succeed) } blockingState <- Ref.make[Map[BlockingTarget, BlockingState]](Map.empty) @@ -86,10 +82,7 @@ class RetryConsumerRecordHandlerTest extends BaseTest[TestClock with TestMetrics ) value <- bytes begin <- currentTime - retryAttempt <- IntSerde.serialize(retryTopic, 0) - submittedAt <- InstantSerde.serialize(retryTopic, begin) - backoff <- DurationSerde.serialize(retryTopic, 1.second) - headers = Headers("retry-attempt" -> retryAttempt, "retry-submitted-at" -> submittedAt, "retry-backoff" -> backoff) + headers = RetryAttempt.toHeaders(RetryAttempt(topic, attempt, begin, 1.second)) _ <- retryHandler.handle(ConsumerRecord(retryTopic, partition, offset, headers, None, value, 0L, 0L, 0L)).fork _ <- TestClock.adjust(1.second).repeat(Schedule.once) end <- executionTime.await.disconnect.timeoutFail(TimeoutWaitingForAssertion)(5.seconds) @@ -404,7 +397,8 @@ class RetryConsumerRecordHandlerTest extends BaseTest[TestClock with TestMetrics producer <- FakeProducer.make topic <- randomTopicName blockingState <- Ref.make[Map[BlockingTarget, BlockingState]](Map.empty) - retryHelper = alwaysBackOffRetryHelper(3.seconds) + retryHelper = FakeRetryHelper(topic) + now <- Clock.instant handling <- AwaitShutdown.makeManaged.flatMap { awaitShutdown => val retryHandler = RetryRecordHandler.withRetries( group, @@ -416,11 +410,12 @@ class RetryConsumerRecordHandlerTest extends BaseTest[TestClock with TestMetrics retryHelper, awaitShutdown = _ => ZIO.succeed(awaitShutdown) ) + val headers = RetryAttempt.toHeaders(RetryAttempt(topic, 0, now, 3.seconds)) for { key <- bytes value <- bytes handling <- retryHandler - .handle(ConsumerRecord(topic, partition, offset, Headers.Empty, Some(key), value, 0L, 0L, 0L)) + .handle(ConsumerRecord(topic, partition, offset, headers, Some(key), value, 0L, 0L, 0L)) .forkDaemon } yield handling } @@ -534,18 +529,6 @@ object RetryConsumerRecordHandlerTest { def randomTopicName = randomStr.map(suffix => s"some-topic-$suffix") val cause = new RuntimeException("cause") - - def alwaysBackOffRetryHelper(backoff: Duration) = { - new FakeNonBlockingRetryHelper { - override val topic: Topic = "" - - override def retryAttempt(topic: Topic, headers: Headers, subscription: ConsumerSubscription)( - implicit trace: Trace - ): UIO[Option[RetryAttempt]] = ZIO.succeed( - Some(RetryAttempt(topic, 1, Instant.now, backoff)) - ) - } - } } object TimeoutWaitingForAssertion extends RuntimeException diff --git a/core/src/test/scala/com/wixpress/dst/greyhound/core/producer/BUILD.bazel b/core/src/test/scala/com/wixpress/dst/greyhound/core/producer/BUILD.bazel index 27f12381..46310b8b 100644 --- a/core/src/test/scala/com/wixpress/dst/greyhound/core/producer/BUILD.bazel +++ b/core/src/test/scala/com/wixpress/dst/greyhound/core/producer/BUILD.bazel @@ -16,7 +16,6 @@ specs2_unit_test( "//core/src/main/scala/com/wixpress/dst/greyhound/core/producer", "//core/src/test/resources", "//core/src/test/scala/com/wixpress/dst/greyhound/core/testkit", - "@ch_qos_logback_logback_classic", # "@dev_zio_izumi_reflect_2_12", "@dev_zio_zio_2_12", "@dev_zio_zio_test_2_12", diff --git a/core/src/test/scala/com/wixpress/dst/greyhound/core/testkit/BUILD.bazel b/core/src/test/scala/com/wixpress/dst/greyhound/core/testkit/BUILD.bazel index 17c68cef..75f65d20 100644 --- a/core/src/test/scala/com/wixpress/dst/greyhound/core/testkit/BUILD.bazel +++ b/core/src/test/scala/com/wixpress/dst/greyhound/core/testkit/BUILD.bazel @@ -15,7 +15,6 @@ scala_library( "//core/src/main/scala/com/wixpress/dst/greyhound/core/consumer/retry", "//core/src/main/scala/com/wixpress/dst/greyhound/core/metrics", "//core/src/main/scala/com/wixpress/dst/greyhound/core/producer", - "//core/src/main/scala/com/wixpress/dst/greyhound/core/zioutils", # "@dev_zio_izumi_reflect_2_12", # "@dev_zio_izumi_reflect_thirdparty_boopickle_shaded_2_12", "@dev_zio_zio_2_12", diff --git a/core/src/test/scala/com/wixpress/dst/greyhound/core/testkit/FakeRetryHelper.scala b/core/src/test/scala/com/wixpress/dst/greyhound/core/testkit/FakeRetryHelper.scala index cee13948..619ada36 100644 --- a/core/src/test/scala/com/wixpress/dst/greyhound/core/testkit/FakeRetryHelper.scala +++ b/core/src/test/scala/com/wixpress/dst/greyhound/core/testkit/FakeRetryHelper.scala @@ -1,35 +1,21 @@ package com.wixpress.dst.greyhound.core.testkit import java.time.Instant -import java.util.concurrent.TimeUnit.MILLISECONDS - -import com.wixpress.dst.greyhound.core.Serdes._ import com.wixpress.dst.greyhound.core._ import com.wixpress.dst.greyhound.core.consumer.domain.{ConsumerRecord, ConsumerSubscription} import com.wixpress.dst.greyhound.core.consumer.retry.RetryDecision.{NoMoreRetries, RetryWith} -import com.wixpress.dst.greyhound.core.consumer.retry.{BlockingHandlerFailed, NonBlockingRetryHelper, RetryAttempt, RetryDecision} +import com.wixpress.dst.greyhound.core.consumer.retry.{BlockingHandlerFailed, NonBlockingRetryHelper, RetryAttempt, RetryDecision, RetryHeader} import com.wixpress.dst.greyhound.core.producer.ProducerRecord import com.wixpress.dst.greyhound.core.testkit.FakeRetryHelper._ import zio._ import zio.Clock -import zio.Clock - trait FakeNonBlockingRetryHelper extends NonBlockingRetryHelper { val topic: Topic override def retryTopicsFor(originalTopic: Topic): Set[Topic] = Set(s"$originalTopic-retry") - override def retryAttempt(topic: Topic, headers: Headers, subscription: ConsumerSubscription)( - implicit trace: Trace - ): UIO[Option[RetryAttempt]] = - (for { - attempt <- headers.get(Header.Attempt, IntSerde) - submittedAt <- headers.get(Header.SubmittedAt, InstantSerde) - backoff <- headers.get(Header.Backoff, DurationSerde) - } yield retryAttemptInternal(topic, attempt, submittedAt, backoff)).orElse(ZIO.none) - override def retryDecision[E]( retryAttempt: Option[RetryAttempt], record: ConsumerRecord[Chunk[Byte], Chunk[Byte]], @@ -38,35 +24,23 @@ trait FakeNonBlockingRetryHelper extends NonBlockingRetryHelper { )(implicit trace: Trace): URIO[Any, RetryDecision] = error match { case RetriableError | BlockingHandlerFailed => - currentTime.flatMap(now => - recordFrom(now, retryAttempt, record) - .fold(_ => NoMoreRetries, RetryWith) + currentTime.map(now => + RetryWith(recordFrom(now, retryAttempt, record)) ) case NonRetriableError => ZIO.succeed(NoMoreRetries) } - private def retryAttemptInternal(topic: Topic, attempt: Option[Int], submittedAt: Option[Instant], backoff: Option[Duration]) = - for { - a <- attempt - s <- submittedAt - b <- backoff - } yield RetryAttempt(topic, a, s, b) - private def recordFrom(now: Instant, retryAttempt: Option[RetryAttempt], record: ConsumerRecord[Chunk[Byte], Chunk[Byte]])( implicit trace: Trace ) = { val nextRetryAttempt = retryAttempt.fold(0)(_.attempt + 1) - for { - retryAttempt <- IntSerde.serialize(topic, nextRetryAttempt) - submittedAt <- InstantSerde.serialize(topic, now) - backoff <- DurationSerde.serialize(topic, 1.second) - } yield ProducerRecord( + ProducerRecord( topic = s"$topic-retry", value = record.value, key = record.key, partition = None, - headers = Headers(Header.Attempt -> retryAttempt, Header.SubmittedAt -> submittedAt, Header.Backoff -> backoff) + headers = RetryAttempt.toHeaders(RetryAttempt(topic, nextRetryAttempt, now, 1.second)) ) } } @@ -75,13 +49,8 @@ case class FakeRetryHelper(topic: Topic) extends FakeNonBlockingRetryHelper object FakeRetryHelper { implicit private val trace = Trace.empty - object Header { - val Attempt = "retry-attempt" - val SubmittedAt = "retry-submitted-at" - val Backoff = "retry-backoff" - } - val currentTime = Clock.currentTime(MILLISECONDS).map(Instant.ofEpochMilli) + val currentTime: UIO[Instant] = Clock.instant } sealed trait HandlerError diff --git a/future-interop/src/it/scala/com/wixpress/dst/greyhound/future/BUILD.bazel b/future-interop/src/it/scala/com/wixpress/dst/greyhound/future/BUILD.bazel index 45e8069e..af2ef67c 100644 --- a/future-interop/src/it/scala/com/wixpress/dst/greyhound/future/BUILD.bazel +++ b/future-interop/src/it/scala/com/wixpress/dst/greyhound/future/BUILD.bazel @@ -21,11 +21,8 @@ specs2_ite2e_test( "//core/src/test/resources", "//core/src/test/scala/com/wixpress/dst/greyhound/core/testkit", "//future-interop/src/main/scala/com/wixpress/dst/greyhound/future", - "//java-interop/src/main/java/com/wixpress/dst/greyhound/java", - "@ch_qos_logback_logback_classic", "@dev_zio_izumi_reflect_2_12", "@dev_zio_zio_2_12", "@dev_zio_zio_stacktracer_2_12", - "@org_apache_kafka_kafka_clients", ], )