From fd0c49df371e8011f1ef621a5b2cc8840b41251f Mon Sep 17 00:00:00 2001 From: Steven Vroonland Date: Sun, 3 Nov 2024 10:35:00 +0100 Subject: [PATCH 01/57] Refactor stream completion status in preparation of additional logging --- .../zio/kafka/consumer/internal/Runloop.scala | 56 ++++++++++++------- 1 file changed, 35 insertions(+), 21 deletions(-) diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala index 35cf60fa5..6caed5076 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala @@ -125,35 +125,49 @@ private[consumer] final class Runloop private ( ZIO.attempt(consumer.commitAsync(java.util.Collections.emptyMap(), null)).orDie } - def endingStreamsCompletedAndCommitsExist(newCommits: Chunk[Commit]): Task[Boolean] = + sealed trait EndOffsetCommitStatus + case object EndOffsetNotCommitted extends EndOffsetCommitStatus + case object EndOffsetCommitPending extends EndOffsetCommitStatus + case object EndOffsetCommitted extends EndOffsetCommitStatus + + case class StreamCompletionStatus( + tp: TopicPartition, + isDone: Boolean, + lastPulledOffset: Option[Long], + endOffsetCommitStatus: EndOffsetCommitStatus + ) + + def getStreamCompletionStatuses(newCommits: Chunk[Commit]): ZIO[Any, Nothing, Chunk[StreamCompletionStatus]] = for { + committedOffsets <- committedOffsetsRef.get + allPendingCommitOffsets = (previousPendingCommits ++ commitsOfEndingStreams(newCommits)).flatMap(_.offsets) streamResults <- ZIO.foreach(streamsToEnd) { stream => for { isDone <- stream.completedPromise.isDone lastPulledOffset <- stream.lastPulledOffset endOffset <- if (isDone) stream.completedPromise.await else ZIO.none - } yield (isDone || lastPulledOffset.isEmpty, endOffset) - } - committedOffsets <- committedOffsetsRef.get - } yield { - val allStreamsCompleted = streamResults.forall(_._1) - allStreamsCompleted && { - val endOffsets: Chunk[Offset] = streamResults.flatMap(_._2) - val allPendingCommits = previousPendingCommits ++ commitsOfEndingStreams(newCommits) - endOffsets.forall { endOffset => - val tp = endOffset.topicPartition - val offset = endOffset.offset - def endOffsetWasCommitted = committedOffsets.contains(tp, offset) - def endOffsetCommitIsPending = allPendingCommits.exists { pendingCommit => - pendingCommit.offsets.get(tp).exists { pendingOffset => - pendingOffset.offset() >= offset - } - } - endOffsetWasCommitted || endOffsetCommitIsPending + + endOffsetCommitStatus = endOffset match { + case Some(endOffset) + if committedOffsets.contains(stream.tp, endOffset.offset) => + EndOffsetCommitted + case Some(endOffset) if allPendingCommitOffsets.exists { case (tp, offset) => + tp == stream.tp && offset.offset() >= endOffset.offset + } => + EndOffsetCommitPending + case _ => EndOffsetNotCommitted + } + } yield StreamCompletionStatus(stream.tp, isDone, lastPulledOffset.map(_.offset), endOffsetCommitStatus) } - } - } + } yield streamResults + + def endingStreamsCompletedAndCommitsExist(newCommits: Chunk[Commit]): Task[Boolean] = + for { + streamResults <- getStreamCompletionStatuses(newCommits) + } yield streamResults.forall(status => + (status.isDone || status.lastPulledOffset.isEmpty) && (status.endOffsetCommitStatus != EndOffsetNotCommitted) + ) def commitSync: Task[Unit] = ZIO.attempt(consumer.commitSync(java.util.Collections.emptyMap(), commitTimeout)) From 0d6258be16af517072e6a2e63071fd46fe28ac8e Mon Sep 17 00:00:00 2001 From: Steven Vroonland Date: Sun, 3 Nov 2024 11:01:24 +0100 Subject: [PATCH 02/57] Add logging + fix condition --- .../zio/kafka/consumer/ConsumerSpec.scala | 5 ++-- .../zio/kafka/consumer/internal/Runloop.scala | 29 ++++++++++++++++--- 2 files changed, 28 insertions(+), 6 deletions(-) diff --git a/zio-kafka-test/src/test/scala/zio/kafka/consumer/ConsumerSpec.scala b/zio-kafka-test/src/test/scala/zio/kafka/consumer/ConsumerSpec.scala index 4fae127a0..e491a2dfd 100644 --- a/zio-kafka-test/src/test/scala/zio/kafka/consumer/ConsumerSpec.scala +++ b/zio-kafka-test/src/test/scala/zio/kafka/consumer/ConsumerSpec.scala @@ -769,7 +769,8 @@ object ConsumerSpec extends ZIOSpecDefaultSlf4j with KafkaRandom { clientId = clientId, groupId = Some(groupId), `max.poll.records` = 1, - rebalanceSafeCommits = rebalanceSafeCommits + rebalanceSafeCommits = rebalanceSafeCommits, + maxRebalanceDuration = 30.seconds ) consumer <- Consumer.make(settings) } yield consumer @@ -863,7 +864,7 @@ object ConsumerSpec extends ZIOSpecDefaultSlf4j with KafkaRandom { testForPartitionAssignmentStrategy[RangeAssignor], testForPartitionAssignmentStrategy[CooperativeStickyAssignor] ) - }: _*), + }: _*) @@ TestAspect.sequential, test("partitions for topic doesn't fail if doesn't exist") { for { topic <- randomTopic diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala index 6caed5076..a7378de73 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala @@ -135,7 +135,10 @@ private[consumer] final class Runloop private ( isDone: Boolean, lastPulledOffset: Option[Long], endOffsetCommitStatus: EndOffsetCommitStatus - ) + ) { + override def toString: String = + s"${tp}: isDone=${isDone}, lastPulledOffset=${lastPulledOffset.getOrElse("none")}, endOffsetCommitStatus: ${endOffsetCommitStatus}" + } def getStreamCompletionStatuses(newCommits: Chunk[Commit]): ZIO[Any, Nothing, Chunk[StreamCompletionStatus]] = for { @@ -162,11 +165,27 @@ private[consumer] final class Runloop private ( } } yield streamResults + def logInitialStreamCompletionStatuses: ZIO[Any, Nothing, Unit] = + getStreamCompletionStatuses(newCommits = Chunk.empty).flatMap { completionStatuses => + val statusStrings = completionStatuses.map(_.toString) + ZIO.logInfo(s"Waiting for ${streamsToEnd.size} streams to end: ${statusStrings.mkString("; ")}") + } + + def logFinalStreamCompletionStatuses(newCommits: Chunk[Commit]): ZIO[Any, Nothing, Unit] = + getStreamCompletionStatuses(newCommits).flatMap { completionStatuses => + val statusStrings = completionStatuses.map(_.toString) + ZIO + .logWarning( + s"Exceeded deadline waiting for streams to end, will continue with rebalance: ${statusStrings.mkString("; ")}" + ) + .when(java.lang.System.nanoTime() >= deadline) + }.unit + def endingStreamsCompletedAndCommitsExist(newCommits: Chunk[Commit]): Task[Boolean] = for { streamResults <- getStreamCompletionStatuses(newCommits) } yield streamResults.forall(status => - (status.isDone || status.lastPulledOffset.isEmpty) && (status.endOffsetCommitStatus != EndOffsetNotCommitted) + status.lastPulledOffset.isEmpty || (status.isDone && status.endOffsetCommitStatus != EndOffsetNotCommitted) ) def commitSync: Task[Unit] = @@ -188,7 +207,7 @@ private[consumer] final class Runloop private ( // // Note, we cannot use ZStream.fromQueue because that will emit nothing when the queue is empty. // Instead, we poll the queue in a loop. - ZIO.logDebug(s"Waiting for ${streamsToEnd.size} streams to end") *> + logInitialStreamCompletionStatuses *> ZStream .fromZIO(blockingSleep(commitQueuePollInterval) *> commitQueue.takeAll) .tap(commitAsync) @@ -196,7 +215,9 @@ private[consumer] final class Runloop private ( .takeWhile(_ => java.lang.System.nanoTime() <= deadline) .scan(Chunk.empty[Runloop.Commit])(_ ++ _) .takeUntilZIO(endingStreamsCompletedAndCommitsExist) - .runDrain *> + .runCollect + .map(_.flatten) + .flatMap(logFinalStreamCompletionStatuses) *> commitSync *> ZIO.logDebug(s"Done waiting for ${streamsToEnd.size} streams to end") } From 399dcac6ce1bb4c6ce28ac47d85ac4b056329275 Mon Sep 17 00:00:00 2001 From: Steven Vroonland Date: Sun, 3 Nov 2024 11:05:48 +0100 Subject: [PATCH 03/57] Also some debug logging while waiting --- .../main/scala/zio/kafka/consumer/internal/Runloop.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala index a7378de73..00108220c 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala @@ -183,8 +183,10 @@ private[consumer] final class Runloop private ( def endingStreamsCompletedAndCommitsExist(newCommits: Chunk[Commit]): Task[Boolean] = for { - streamResults <- getStreamCompletionStatuses(newCommits) - } yield streamResults.forall(status => + completionStatuses <- getStreamCompletionStatuses(newCommits) + statusStrings = completionStatuses.map(_.toString) + _ <- ZIO.logDebug(s"Waiting for ${streamsToEnd.size} streams to end: ${statusStrings.mkString("; ")}") + } yield completionStatuses.forall(status => status.lastPulledOffset.isEmpty || (status.isDone && status.endOffsetCommitStatus != EndOffsetNotCommitted) ) From c56dc69228284cb20a8d0353fb78f7cbdb64e60b Mon Sep 17 00:00:00 2001 From: Steven Vroonland Date: Sun, 3 Nov 2024 11:16:50 +0100 Subject: [PATCH 04/57] Fix lint --- .../src/main/scala/zio/kafka/consumer/internal/Runloop.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala index 00108220c..a9fae0f48 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala @@ -130,7 +130,7 @@ private[consumer] final class Runloop private ( case object EndOffsetCommitPending extends EndOffsetCommitStatus case object EndOffsetCommitted extends EndOffsetCommitStatus - case class StreamCompletionStatus( + final case class StreamCompletionStatus( tp: TopicPartition, isDone: Boolean, lastPulledOffset: Option[Long], From 11b3238556124d2b5e0654e6b7e1f4f8071a8171 Mon Sep 17 00:00:00 2001 From: Steven Vroonland Date: Sun, 3 Nov 2024 14:37:04 +0100 Subject: [PATCH 05/57] Increase timeout --- .../src/test/scala/zio/kafka/consumer/ConsumerSpec.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/zio-kafka-test/src/test/scala/zio/kafka/consumer/ConsumerSpec.scala b/zio-kafka-test/src/test/scala/zio/kafka/consumer/ConsumerSpec.scala index e491a2dfd..fcc2d6102 100644 --- a/zio-kafka-test/src/test/scala/zio/kafka/consumer/ConsumerSpec.scala +++ b/zio-kafka-test/src/test/scala/zio/kafka/consumer/ConsumerSpec.scala @@ -770,7 +770,7 @@ object ConsumerSpec extends ZIOSpecDefaultSlf4j with KafkaRandom { groupId = Some(groupId), `max.poll.records` = 1, rebalanceSafeCommits = rebalanceSafeCommits, - maxRebalanceDuration = 30.seconds + maxRebalanceDuration = 60.seconds ) consumer <- Consumer.make(settings) } yield consumer From dc38739810615ff8d2a52633e7ebe30117620780 Mon Sep 17 00:00:00 2001 From: Steven Vroonland Date: Tue, 5 Nov 2024 20:50:27 +0100 Subject: [PATCH 06/57] Correct race condition --- .../zio/kafka/consumer/internal/Runloop.scala | 21 ++++++++++--------- 1 file changed, 11 insertions(+), 10 deletions(-) diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala index a9fae0f48..25ce14b49 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala @@ -171,17 +171,17 @@ private[consumer] final class Runloop private ( ZIO.logInfo(s"Waiting for ${streamsToEnd.size} streams to end: ${statusStrings.mkString("; ")}") } - def logFinalStreamCompletionStatuses(newCommits: Chunk[Commit]): ZIO[Any, Nothing, Unit] = + def logFinalStreamCompletionStatuses(completed: Boolean, newCommits: Chunk[Commit]): ZIO[Any, Nothing, Unit] = getStreamCompletionStatuses(newCommits).flatMap { completionStatuses => - val statusStrings = completionStatuses.map(_.toString) ZIO .logWarning( - s"Exceeded deadline waiting for streams to end, will continue with rebalance: ${statusStrings.mkString("; ")}" + s"Exceeded deadline waiting for streams to end, will continue with rebalance: ${completionStatuses.map(_.toString).mkString("; ")}" ) - .when(java.lang.System.nanoTime() >= deadline) - }.unit + } + .unless(completed) + .unit - def endingStreamsCompletedAndCommitsExist(newCommits: Chunk[Commit]): Task[Boolean] = + def endingStreamsCompletedAndCommitsExist(newCommits: Chunk[Commit]): UIO[Boolean] = for { completionStatuses <- getStreamCompletionStatuses(newCommits) statusStrings = completionStatuses.map(_.toString) @@ -216,10 +216,11 @@ private[consumer] final class Runloop private ( .forever .takeWhile(_ => java.lang.System.nanoTime() <= deadline) .scan(Chunk.empty[Runloop.Commit])(_ ++ _) - .takeUntilZIO(endingStreamsCompletedAndCommitsExist) - .runCollect - .map(_.flatten) - .flatMap(logFinalStreamCompletionStatuses) *> + .mapZIO(commits => endingStreamsCompletedAndCommitsExist(commits).map((_, commits))) + .takeUntil { case (completed, commits @ _) => completed } + .runLast + .map(_.getOrElse((false, Chunk.empty))) + .flatMap { case (completed, commits) => logFinalStreamCompletionStatuses(completed, commits) } *> commitSync *> ZIO.logDebug(s"Done waiting for ${streamsToEnd.size} streams to end") } From 2ee212ccbd48cc3b5c41b0a4ffcc84e48b0da89a Mon Sep 17 00:00:00 2001 From: Steven Vroonland Date: Tue, 5 Nov 2024 20:50:33 +0100 Subject: [PATCH 07/57] Remove sequential --- .../src/test/scala/zio/kafka/consumer/ConsumerSpec.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/zio-kafka-test/src/test/scala/zio/kafka/consumer/ConsumerSpec.scala b/zio-kafka-test/src/test/scala/zio/kafka/consumer/ConsumerSpec.scala index fcc2d6102..679a0d582 100644 --- a/zio-kafka-test/src/test/scala/zio/kafka/consumer/ConsumerSpec.scala +++ b/zio-kafka-test/src/test/scala/zio/kafka/consumer/ConsumerSpec.scala @@ -864,7 +864,7 @@ object ConsumerSpec extends ZIOSpecDefaultSlf4j with KafkaRandom { testForPartitionAssignmentStrategy[RangeAssignor], testForPartitionAssignmentStrategy[CooperativeStickyAssignor] ) - }: _*) @@ TestAspect.sequential, + }: _*), test("partitions for topic doesn't fail if doesn't exist") { for { topic <- randomTopic From 9caee508e8f4dcce90df2bfb7e9843822f4d3b5f Mon Sep 17 00:00:00 2001 From: svroonland Date: Wed, 6 Nov 2024 09:32:55 +0100 Subject: [PATCH 08/57] Update zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala Co-authored-by: Erik van Oosten --- .../src/main/scala/zio/kafka/consumer/internal/Runloop.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala index 25ce14b49..dae5331b3 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala @@ -217,7 +217,7 @@ private[consumer] final class Runloop private ( .takeWhile(_ => java.lang.System.nanoTime() <= deadline) .scan(Chunk.empty[Runloop.Commit])(_ ++ _) .mapZIO(commits => endingStreamsCompletedAndCommitsExist(commits).map((_, commits))) - .takeUntil { case (completed, commits @ _) => completed } + .takeUntil { case (completed, _) => completed } .runLast .map(_.getOrElse((false, Chunk.empty))) .flatMap { case (completed, commits) => logFinalStreamCompletionStatuses(completed, commits) } *> From e034b778d2438dc1ae3c43d4e0f50289182d8915 Mon Sep 17 00:00:00 2001 From: svroonland Date: Wed, 6 Nov 2024 09:33:26 +0100 Subject: [PATCH 09/57] Update zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala Co-authored-by: Erik van Oosten --- .../src/main/scala/zio/kafka/consumer/internal/Runloop.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala index dae5331b3..f5cea5d45 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala @@ -175,7 +175,7 @@ private[consumer] final class Runloop private ( getStreamCompletionStatuses(newCommits).flatMap { completionStatuses => ZIO .logWarning( - s"Exceeded deadline waiting for streams to end, will continue with rebalance: ${completionStatuses.map(_.toString).mkString("; ")}" + s"Exceeded deadline waiting for streams to commit the offsets of the records they consumed; the rebalance will continue. This might cause another consumer to process some records again. ${completionStatuses.map(_.toString).mkString("; ")}" ) } .unless(completed) From 221b2839e43ea86a2e667e89495a67a269574326 Mon Sep 17 00:00:00 2001 From: svroonland Date: Wed, 6 Nov 2024 09:43:07 +0100 Subject: [PATCH 10/57] Update zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala Co-authored-by: Erik van Oosten --- .../src/main/scala/zio/kafka/consumer/internal/Runloop.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala index f5cea5d45..e6267439f 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala @@ -186,9 +186,10 @@ private[consumer] final class Runloop private ( completionStatuses <- getStreamCompletionStatuses(newCommits) statusStrings = completionStatuses.map(_.toString) _ <- ZIO.logDebug(s"Waiting for ${streamsToEnd.size} streams to end: ${statusStrings.mkString("; ")}") - } yield completionStatuses.forall(status => + } yield completionStatuses.forall { status => + // A stream is complete when it never got any records, or when it committed the offset of the last consumed record status.lastPulledOffset.isEmpty || (status.isDone && status.endOffsetCommitStatus != EndOffsetNotCommitted) - ) + } def commitSync: Task[Unit] = ZIO.attempt(consumer.commitSync(java.util.Collections.emptyMap(), commitTimeout)) From bad1defee846331dd374e2e9fc5a7cc940fc44a3 Mon Sep 17 00:00:00 2001 From: Steven Vroonland Date: Sat, 9 Nov 2024 09:55:33 +0100 Subject: [PATCH 11/57] Stricter comparison of pending commit offset to last pulled offset --- .../scala/zio/kafka/consumer/internal/Runloop.scala | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala index e6267439f..f1b7984b4 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala @@ -143,7 +143,10 @@ private[consumer] final class Runloop private ( def getStreamCompletionStatuses(newCommits: Chunk[Commit]): ZIO[Any, Nothing, Chunk[StreamCompletionStatus]] = for { committedOffsets <- committedOffsetsRef.get - allPendingCommitOffsets = (previousPendingCommits ++ commitsOfEndingStreams(newCommits)).flatMap(_.offsets) + allPendingCommitOffsets = + (previousPendingCommits ++ commitsOfEndingStreams(newCommits)).flatMap(_.offsets).map { + case (tp, offsetAndMetadata) => (tp, offsetAndMetadata.offset()) + } streamResults <- ZIO.foreach(streamsToEnd) { stream => for { @@ -155,9 +158,8 @@ private[consumer] final class Runloop private ( case Some(endOffset) if committedOffsets.contains(stream.tp, endOffset.offset) => EndOffsetCommitted - case Some(endOffset) if allPendingCommitOffsets.exists { case (tp, offset) => - tp == stream.tp && offset.offset() >= endOffset.offset - } => + case Some(endOffset) + if allPendingCommitOffsets.contains((stream.tp, endOffset.offset)) => EndOffsetCommitPending case _ => EndOffsetNotCommitted } From cc377dd838b1070814786a8056b3403f9d4fe2fd Mon Sep 17 00:00:00 2001 From: Erik van Oosten Date: Sun, 10 Nov 2024 09:10:23 +0100 Subject: [PATCH 12/57] More rebalance safe commits logging alt (#1367) --- .../zio/kafka/consumer/internal/Runloop.scala | 115 +++++++++++------- 1 file changed, 69 insertions(+), 46 deletions(-) diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala index f1b7984b4..ff31673a7 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala @@ -106,6 +106,8 @@ private[consumer] final class Runloop private ( ): Task[Unit] = { val deadline = java.lang.System.nanoTime() + maxRebalanceDuration.toNanos - commitTimeoutNanos + def timeToDeadlineMillis(): Long = (java.lang.System.nanoTime() - deadline) / 1000000L + val endingTps = streamsToEnd.map(_.tp).toSet def commitsOfEndingStreams(commits: Chunk[Runloop.Commit]): Chunk[Runloop.Commit] = @@ -126,21 +128,27 @@ private[consumer] final class Runloop private ( } sealed trait EndOffsetCommitStatus - case object EndOffsetNotCommitted extends EndOffsetCommitStatus - case object EndOffsetCommitPending extends EndOffsetCommitStatus - case object EndOffsetCommitted extends EndOffsetCommitStatus + case object EndOffsetNotCommitted extends EndOffsetCommitStatus { override def toString = "not committed" } + case object EndOffsetCommitPending extends EndOffsetCommitStatus { override def toString = "commit pending" } + case object EndOffsetCommitted extends EndOffsetCommitStatus { override def toString = "committed" } final case class StreamCompletionStatus( tp: TopicPartition, - isDone: Boolean, + streamEnded: Boolean, lastPulledOffset: Option[Long], endOffsetCommitStatus: EndOffsetCommitStatus ) { override def toString: String = - s"${tp}: isDone=${isDone}, lastPulledOffset=${lastPulledOffset.getOrElse("none")}, endOffsetCommitStatus: ${endOffsetCommitStatus}" + s"${tp}: " + + s"${if (streamEnded) "stream ended" else "stream is running"}, " + + s"last pulled offset=${lastPulledOffset.getOrElse("none")}, " + + endOffsetCommitStatus } - def getStreamCompletionStatuses(newCommits: Chunk[Commit]): ZIO[Any, Nothing, Chunk[StreamCompletionStatus]] = + def completionStatusesAsString(completionStatuses: Chunk[StreamCompletionStatus]): String = + "Revoked partitions: " + completionStatuses.map(_.toString).mkString("; ") + + def getStreamCompletionStatuses(newCommits: Chunk[Commit]): UIO[Chunk[StreamCompletionStatus]] = for { committedOffsets <- committedOffsetsRef.get allPendingCommitOffsets = @@ -154,45 +162,57 @@ private[consumer] final class Runloop private ( lastPulledOffset <- stream.lastPulledOffset endOffset <- if (isDone) stream.completedPromise.await else ZIO.none - endOffsetCommitStatus = endOffset match { - case Some(endOffset) - if committedOffsets.contains(stream.tp, endOffset.offset) => - EndOffsetCommitted - case Some(endOffset) - if allPendingCommitOffsets.contains((stream.tp, endOffset.offset)) => - EndOffsetCommitPending - case _ => EndOffsetNotCommitted - } + endOffsetCommitStatus = + endOffset match { + case Some(endOffset) if committedOffsets.contains(stream.tp, endOffset.offset) => + EndOffsetCommitted + case Some(endOffset) if allPendingCommitOffsets.contains((stream.tp, endOffset.offset)) => + EndOffsetCommitPending + case _ => EndOffsetNotCommitted + } } yield StreamCompletionStatus(stream.tp, isDone, lastPulledOffset.map(_.offset), endOffsetCommitStatus) } } yield streamResults - def logInitialStreamCompletionStatuses: ZIO[Any, Nothing, Unit] = - getStreamCompletionStatuses(newCommits = Chunk.empty).flatMap { completionStatuses => - val statusStrings = completionStatuses.map(_.toString) - ZIO.logInfo(s"Waiting for ${streamsToEnd.size} streams to end: ${statusStrings.mkString("; ")}") - } + @inline + def logStreamCompletionStatuses(completionStatuses: Chunk[StreamCompletionStatus]): UIO[Unit] = { + val statusStrings = completionStatusesAsString(completionStatuses) + ZIO.logInfo( + s"Delaying rebalance until ${streamsToEnd.size} streams (of revoked partitions) have committed " + + s"the offsets of the records they consumed. Deadline in ${timeToDeadlineMillis()}ms. $statusStrings" + ) + } - def logFinalStreamCompletionStatuses(completed: Boolean, newCommits: Chunk[Commit]): ZIO[Any, Nothing, Unit] = - getStreamCompletionStatuses(newCommits).flatMap { completionStatuses => - ZIO - .logWarning( - s"Exceeded deadline waiting for streams to commit the offsets of the records they consumed; the rebalance will continue. This might cause another consumer to process some records again. ${completionStatuses.map(_.toString).mkString("; ")}" - ) - } - .unless(completed) - .unit + def logInitialStreamCompletionStatuses: UIO[Unit] = + for { + completionStatuses <- getStreamCompletionStatuses(newCommits = Chunk.empty) + _ <- logStreamCompletionStatuses(completionStatuses) + } yield () def endingStreamsCompletedAndCommitsExist(newCommits: Chunk[Commit]): UIO[Boolean] = for { completionStatuses <- getStreamCompletionStatuses(newCommits) - statusStrings = completionStatuses.map(_.toString) - _ <- ZIO.logDebug(s"Waiting for ${streamsToEnd.size} streams to end: ${statusStrings.mkString("; ")}") + _ <- logStreamCompletionStatuses(completionStatuses) } yield completionStatuses.forall { status => // A stream is complete when it never got any records, or when it committed the offset of the last consumed record - status.lastPulledOffset.isEmpty || (status.isDone && status.endOffsetCommitStatus != EndOffsetNotCommitted) + status.lastPulledOffset.isEmpty || (status.streamEnded && status.endOffsetCommitStatus != EndOffsetNotCommitted) } + def logFinalStreamCompletionStatuses(completed: Boolean, newCommits: Chunk[Commit]): UIO[Unit] = + if (completed) + ZIO.logInfo("Continuing rebalance, all offsets of consumed records in the revoked partitions were committed.") + else + for { + completionStatuses <- getStreamCompletionStatuses(newCommits) + statusStrings = completionStatusesAsString(completionStatuses) + _ <- + ZIO.logWarning( + s"Exceeded deadline waiting for streams (of revoked partitions) to commit the offsets of " + + s"the records they consumed; the rebalance will continue. " + + s"This might cause another consumer to process some records again. $statusStrings" + ) + } yield () + def commitSync: Task[Unit] = ZIO.attempt(consumer.commitSync(java.util.Collections.emptyMap(), commitTimeout)) @@ -212,20 +232,23 @@ private[consumer] final class Runloop private ( // // Note, we cannot use ZStream.fromQueue because that will emit nothing when the queue is empty. // Instead, we poll the queue in a loop. - logInitialStreamCompletionStatuses *> - ZStream - .fromZIO(blockingSleep(commitQueuePollInterval) *> commitQueue.takeAll) - .tap(commitAsync) - .forever - .takeWhile(_ => java.lang.System.nanoTime() <= deadline) - .scan(Chunk.empty[Runloop.Commit])(_ ++ _) - .mapZIO(commits => endingStreamsCompletedAndCommitsExist(commits).map((_, commits))) - .takeUntil { case (completed, _) => completed } - .runLast - .map(_.getOrElse((false, Chunk.empty))) - .flatMap { case (completed, commits) => logFinalStreamCompletionStatuses(completed, commits) } *> - commitSync *> - ZIO.logDebug(s"Done waiting for ${streamsToEnd.size} streams to end") + for { + _ <- logInitialStreamCompletionStatuses + (completed, commits) <- + ZStream + .fromZIO(blockingSleep(commitQueuePollInterval) *> commitQueue.takeAll) + .tap(commitAsync) + .forever + .takeWhile(_ => java.lang.System.nanoTime() <= deadline) + .scan(Chunk.empty[Runloop.Commit])(_ ++ _) + .mapZIO(commits => endingStreamsCompletedAndCommitsExist(commits).map((_, commits))) + .takeUntil { case (completed, _) => completed } + .runLast + .map(_.getOrElse((false, Chunk.empty))) + _ <- logFinalStreamCompletionStatuses(completed, commits) + _ <- commitSync + _ <- ZIO.logDebug(s"Done waiting for ${streamsToEnd.size} streams to end") + } yield () } // During a poll, the java kafka client might call each method of the rebalance listener 0 or 1 times. From 7994954a9291b74899292ed812560252e1fae82f Mon Sep 17 00:00:00 2001 From: Steven Vroonland Date: Sun, 10 Nov 2024 09:21:04 +0100 Subject: [PATCH 13/57] Remove withFilter usage --- .../src/main/scala/zio/kafka/consumer/internal/Runloop.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala index ff31673a7..686ef768f 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala @@ -234,7 +234,7 @@ private[consumer] final class Runloop private ( // Instead, we poll the queue in a loop. for { _ <- logInitialStreamCompletionStatuses - (completed, commits) <- + completedAndCommits <- ZStream .fromZIO(blockingSleep(commitQueuePollInterval) *> commitQueue.takeAll) .tap(commitAsync) @@ -245,7 +245,7 @@ private[consumer] final class Runloop private ( .takeUntil { case (completed, _) => completed } .runLast .map(_.getOrElse((false, Chunk.empty))) - _ <- logFinalStreamCompletionStatuses(completed, commits) + _ <- logFinalStreamCompletionStatuses(completedAndCommits._1, completedAndCommits._2) _ <- commitSync _ <- ZIO.logDebug(s"Done waiting for ${streamsToEnd.size} streams to end") } yield () From d3e0270f67f19fed7374c77474028ec6bae5fa5a Mon Sep 17 00:00:00 2001 From: Steven Vroonland Date: Sun, 10 Nov 2024 17:37:01 +0100 Subject: [PATCH 14/57] Fix timeToDeadlineMillis --- .../src/main/scala/zio/kafka/consumer/internal/Runloop.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala index 686ef768f..83948be52 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala @@ -106,7 +106,7 @@ private[consumer] final class Runloop private ( ): Task[Unit] = { val deadline = java.lang.System.nanoTime() + maxRebalanceDuration.toNanos - commitTimeoutNanos - def timeToDeadlineMillis(): Long = (java.lang.System.nanoTime() - deadline) / 1000000L + def timeToDeadlineMillis(): Long = (deadline - java.lang.System.nanoTime()) / 1000000L val endingTps = streamsToEnd.map(_.tp).toSet From 54494f3a0d0690dcb73b55cd8f6252640fd38b69 Mon Sep 17 00:00:00 2001 From: Steven Vroonland Date: Sun, 10 Nov 2024 17:02:40 +0100 Subject: [PATCH 15/57] Extract a Committer and RunloopRebalanceListener from the Runloop --- ...setsSpec.scala => CommitOffsetsSpec.scala} | 33 +- .../kafka/consumer/internal/Committer.scala | 207 +++++++ .../consumer/internal/ConsumerMetrics.scala | 16 +- .../zio/kafka/consumer/internal/Runloop.scala | 511 ++---------------- .../internal/RunloopRebalanceListener.scala | 289 ++++++++++ 5 files changed, 579 insertions(+), 477 deletions(-) rename zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/{RunloopCommitOffsetsSpec.scala => CommitOffsetsSpec.scala} (75%) create mode 100644 zio-kafka/src/main/scala/zio/kafka/consumer/internal/Committer.scala create mode 100644 zio-kafka/src/main/scala/zio/kafka/consumer/internal/RunloopRebalanceListener.scala diff --git a/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/RunloopCommitOffsetsSpec.scala b/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/CommitOffsetsSpec.scala similarity index 75% rename from zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/RunloopCommitOffsetsSpec.scala rename to zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/CommitOffsetsSpec.scala index 77dbc77b0..ab9a2aca8 100644 --- a/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/RunloopCommitOffsetsSpec.scala +++ b/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/CommitOffsetsSpec.scala @@ -1,11 +1,12 @@ package zio.kafka.consumer.internal +import org.apache.kafka.clients.consumer.OffsetAndMetadata import org.apache.kafka.common.TopicPartition import zio._ -import org.apache.kafka.clients.consumer.OffsetAndMetadata +import zio.kafka.consumer.internal.Committer.{ Commit, CommitOffsets } import zio.test._ -object RunloopCommitOffsetsSpec extends ZIOSpecDefault { +object CommitOffsetsSpec extends ZIOSpecDefault { private val tp10 = new TopicPartition("t1", 0) private val tp11 = new TopicPartition("t1", 1) @@ -14,9 +15,9 @@ object RunloopCommitOffsetsSpec extends ZIOSpecDefault { private val tp22 = new TopicPartition("t2", 2) override def spec: Spec[TestEnvironment with Scope, Any] = - suite("Runloop.CommitOffsets spec")( + suite("CommitOffsets spec")( test("addCommits adds to empty CommitOffsets") { - val s1 = Runloop.CommitOffsets(Map.empty) + val s1 = CommitOffsets(Map.empty) val (inc, s2) = s1.addCommits(Chunk(makeCommit(Map(tp10 -> 10)))) assertTrue( inc == 0, @@ -24,7 +25,7 @@ object RunloopCommitOffsetsSpec extends ZIOSpecDefault { ) }, test("addCommits updates offset when it is higher") { - val s1 = Runloop.CommitOffsets(Map(tp10 -> 4L)) + val s1 = CommitOffsets(Map(tp10 -> 4L)) val (inc, s2) = s1.addCommits(Chunk(makeCommit(Map(tp10 -> 10)))) assertTrue( inc == 10 - 4, @@ -32,7 +33,7 @@ object RunloopCommitOffsetsSpec extends ZIOSpecDefault { ) }, test("addCommits ignores an offset when it is lower") { - val s1 = Runloop.CommitOffsets(Map(tp10 -> 10L)) + val s1 = CommitOffsets(Map(tp10 -> 10L)) val (inc, s2) = s1.addCommits(Chunk(makeCommit(Map(tp10 -> 5)))) assertTrue( inc == 0, @@ -40,7 +41,7 @@ object RunloopCommitOffsetsSpec extends ZIOSpecDefault { ) }, test("addCommits keeps unrelated partitions") { - val s1 = Runloop.CommitOffsets(Map(tp10 -> 10L)) + val s1 = CommitOffsets(Map(tp10 -> 10L)) val (inc, s2) = s1.addCommits(Chunk(makeCommit(Map(tp11 -> 11)))) assertTrue( inc == 0, @@ -48,7 +49,7 @@ object RunloopCommitOffsetsSpec extends ZIOSpecDefault { ) }, test("addCommits does it all at once") { - val s1 = Runloop.CommitOffsets(Map(tp10 -> 10L, tp20 -> 205L, tp21 -> 210L, tp22 -> 220L)) + val s1 = CommitOffsets(Map(tp10 -> 10L, tp20 -> 205L, tp21 -> 210L, tp22 -> 220L)) val (inc, s2) = s1.addCommits(Chunk(makeCommit(Map(tp11 -> 11, tp20 -> 206L, tp21 -> 209L, tp22 -> 220L)))) assertTrue( inc == /* tp10 */ 0 + /* tp11 */ 0 + /* tp20 */ 1 + /* tp21 */ 0 + /* tp22 */ 0, @@ -56,7 +57,7 @@ object RunloopCommitOffsetsSpec extends ZIOSpecDefault { ) }, test("addCommits adds multiple commits") { - val s1 = Runloop.CommitOffsets(Map(tp10 -> 10L, tp20 -> 200L, tp21 -> 210L, tp22 -> 220L)) + val s1 = CommitOffsets(Map(tp10 -> 10L, tp20 -> 200L, tp21 -> 210L, tp22 -> 220L)) val (inc, s2) = s1.addCommits( Chunk( makeCommit(Map(tp11 -> 11, tp20 -> 199L, tp21 -> 211L, tp22 -> 219L)), @@ -69,35 +70,35 @@ object RunloopCommitOffsetsSpec extends ZIOSpecDefault { ) }, test("keepPartitions removes some partitions") { - val s1 = Runloop.CommitOffsets(Map(tp10 -> 10L, tp20 -> 20L)) + val s1 = CommitOffsets(Map(tp10 -> 10L, tp20 -> 20L)) val s2 = s1.keepPartitions(Set(tp10)) assertTrue(s2.offsets == Map(tp10 -> 10L)) }, test("does not 'contain' offset when tp is not present") { - val s1 = Runloop.CommitOffsets(Map(tp10 -> 10L)) + val s1 = CommitOffsets(Map(tp10 -> 10L)) val result = s1.contains(tp20, 10) assertTrue(!result) }, test("does not 'contain' a higher offset") { - val s1 = Runloop.CommitOffsets(Map(tp10 -> 10L, tp20 -> 20L)) + val s1 = CommitOffsets(Map(tp10 -> 10L, tp20 -> 20L)) val result = s1.contains(tp10, 11) assertTrue(!result) }, test("does 'contain' equal offset") { - val s1 = Runloop.CommitOffsets(Map(tp10 -> 10L, tp20 -> 20L)) + val s1 = CommitOffsets(Map(tp10 -> 10L, tp20 -> 20L)) val result = s1.contains(tp10, 10) assertTrue(result) }, test("does 'contain' lower offset") { - val s1 = Runloop.CommitOffsets(Map(tp10 -> 10L, tp20 -> 20L)) + val s1 = CommitOffsets(Map(tp10 -> 10L, tp20 -> 20L)) val result = s1.contains(tp20, 19) assertTrue(result) } ) - private def makeCommit(offsets: Map[TopicPartition, Long]): Runloop.Commit = { + private def makeCommit(offsets: Map[TopicPartition, Long]): Commit = { val o = offsets.map { case (tp, offset) => tp -> new OffsetAndMetadata(offset) } val p = Unsafe.unsafe(implicit unsafe => Promise.unsafe.make[Throwable, Unit](FiberId.None)) - Runloop.Commit(0L, o, p) + Commit(0L, o, p) } } diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Committer.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Committer.scala new file mode 100644 index 000000000..dc12c5782 --- /dev/null +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Committer.scala @@ -0,0 +1,207 @@ +package zio.kafka.consumer.internal +import org.apache.kafka.clients.consumer.{ OffsetAndMetadata, OffsetCommitCallback } +import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.errors.RebalanceInProgressException +import zio.kafka.consumer.Consumer.CommitTimeout +import zio.kafka.consumer.ConsumerSettings +import zio.kafka.consumer.diagnostics.{ DiagnosticEvent, Diagnostics } +import zio.kafka.consumer.internal.Committer.{ Commit, CommitOffsets } +import zio.kafka.consumer.internal.ConsumerAccess.ByteArrayKafkaConsumer +import zio.{ durationLong, Chunk, Exit, Promise, Queue, Ref, Runtime, Scope, Task, UIO, Unsafe, ZIO } + +import java.lang.Math.max +import java.util +import java.util.{ Map => JavaMap } +import scala.collection.mutable +import scala.jdk.CollectionConverters._ + +private[consumer] final class Committer( + commitQueue: Queue[Commit], + settings: ConsumerSettings, + diagnostics: Diagnostics, + consumerMetrics: ConsumerMetrics, + onCommitAvailable: UIO[Unit], + committedOffsetsRef: Ref[CommitOffsets], + sameThreadRuntime: Runtime[Any], + pendingCommits: Ref[Chunk[Commit]] // TODO make Commit internal +) { + private val commitTimeout = settings.commitTimeout + + /** This is the implementation behind the user facing api `Offset.commit`. */ + val commit: Map[TopicPartition, OffsetAndMetadata] => Task[Unit] = + offsets => + for { + p <- Promise.make[Throwable, Unit] + startTime = java.lang.System.nanoTime() + _ <- commitQueue.offer(Commit(java.lang.System.nanoTime(), offsets, p)) + _ <- onCommitAvailable + _ <- diagnostics.emit(DiagnosticEvent.Commit.Started(offsets)) + _ <- p.await.timeoutFail(CommitTimeout)(commitTimeout) + endTime = java.lang.System.nanoTime() + latency = (endTime - startTime).nanoseconds + _ <- consumerMetrics.observeCommit(latency) + } yield () + + /** + * Takes commits from the queue, commits them and adds them to pending commits + * + * If the queue is empty, nothing happens, unless executeOnEmpty is true. + * + * @param consumer + * Consumer with exclusive access + */ + def handleNewCommits(consumer: ByteArrayKafkaConsumer, executeOnEmpty: Boolean = false): Task[Unit] = for { + commits <- commitQueue.takeAll + _ <- ZIO.logDebug(s"Processing ${commits.size} commits") + _ <- ZIO.unless(commits.isEmpty && !executeOnEmpty) { + val (offsets, callback, onFailure) = asyncCommitParameters(commits) + pendingCommits.update(_ ++ commits) *> + // We don't wait for the completion of the commit here, because it + // will only complete once we poll again. + ZIO + .attempt(consumer.commitAsync(offsets, callback)) + .catchAll(onFailure) + } + } yield () + + /** Merge commits and prepare parameters for calling `consumer.commitAsync`. */ + private def asyncCommitParameters( + commits: Chunk[Commit] + ): (JavaMap[TopicPartition, OffsetAndMetadata], OffsetCommitCallback, Throwable => UIO[Unit]) = { + val offsets = commits + .foldLeft(mutable.Map.empty[TopicPartition, OffsetAndMetadata]) { case (acc, commit) => + commit.offsets.foreach { case (tp, offset) => + acc += (tp -> acc + .get(tp) + .map(current => if (current.offset() > offset.offset()) current else offset) + .getOrElse(offset)) + } + acc + } + .toMap + val offsetsWithMetaData = offsets.map { case (tp, offset) => + tp -> new OffsetAndMetadata(offset.offset + 1, offset.leaderEpoch, offset.metadata) + } + val cont = (e: Exit[Throwable, Unit]) => ZIO.foreachDiscard(commits)(_.cont.done(e)) + // We assume the commit is started immediately after returning from this method. + val startTime = java.lang.System.nanoTime() + val onSuccess = { + val endTime = java.lang.System.nanoTime() + val latency = (endTime - startTime).nanoseconds + for { + offsetIncrease <- committedOffsetsRef.modify(_.addCommits(commits)) + _ <- consumerMetrics.observeAggregatedCommit(latency, offsetIncrease).when(commits.nonEmpty) + result <- cont(Exit.unit) + _ <- diagnostics.emit(DiagnosticEvent.Commit.Success(offsetsWithMetaData)) + } yield result + } + val onFailure: Throwable => UIO[Unit] = { + case _: RebalanceInProgressException => + for { + _ <- ZIO.logDebug(s"Rebalance in progress, commit for offsets $offsets will be retried") + _ <- commitQueue.offerAll(commits) + _ <- onCommitAvailable // + } yield () + case err: Throwable => + cont(Exit.fail(err)) <* diagnostics.emit(DiagnosticEvent.Commit.Failure(offsetsWithMetaData, err)) + } + val callback = + new OffsetCommitCallback { + override def onComplete(offsets: util.Map[TopicPartition, OffsetAndMetadata], exception: Exception): Unit = + Unsafe.unsafe { implicit u => + sameThreadRuntime.unsafe.run { + if (exception eq null) onSuccess else onFailure(exception) + } + .getOrThrowFiberFailure() + } + } + (offsetsWithMetaData.asJava, callback, onFailure) + } + + def pendingCommitCount: UIO[Int] = pendingCommits.get.map(_.size) + + def queueSize: UIO[Int] = commitQueue.size + + def updatePendingCommitsAfterPoll: UIO[Unit] = + pendingCommits.get.flatMap(ZIO.filter(_)(_.isPending)).flatMap(pendingCommits.set) + + def pruneCommittedOffsets(assignedPartitions: Set[TopicPartition]): UIO[Unit] = + committedOffsetsRef.update(_.keepPartitions(assignedPartitions)) + + def getCommittedOffsets: UIO[CommitOffsets] = committedOffsetsRef.get + + def getPendingCommits: UIO[CommitOffsets] = + pendingCommits.get.map(CommitOffsets.empty.addCommits(_)._2) +} + +private[internal] object Committer { + def make( + settings: ConsumerSettings, + diagnostics: Diagnostics, + consumerMetrics: ConsumerMetrics, + onCommitAvailable: UIO[Unit], + sameThreadRuntime: Runtime[Any] + ): ZIO[Any with Scope, Nothing, Committer] = for { + pendingCommits <- Ref.make(Chunk.empty[Commit]) + commitQueue <- ZIO.acquireRelease(Queue.unbounded[Commit])(_.shutdown) + committedOffsetsRef <- Ref.make(CommitOffsets.empty) + } yield new Committer( + commitQueue, + settings, + diagnostics, + consumerMetrics, + onCommitAvailable, + committedOffsetsRef, + sameThreadRuntime, + pendingCommits + ) + + private[internal] final case class Commit( + createdAt: NanoTime, + offsets: Map[TopicPartition, OffsetAndMetadata], + cont: Promise[Throwable, Unit] + ) { + @inline def isDone: UIO[Boolean] = cont.isDone + @inline def isPending: UIO[Boolean] = isDone.negate + } + + // package private for unit testing + private[internal] final case class CommitOffsets(offsets: Map[TopicPartition, Long]) { + + /** Returns an estimate of the total offset increase, and a new `CommitOffsets` with the given offsets added. */ + def addCommits(c: Chunk[Commit]): (Long, CommitOffsets) = { + val updatedOffsets = mutable.Map.empty[TopicPartition, Long] + updatedOffsets.sizeHint(offsets.size) + updatedOffsets ++= offsets + var offsetIncrease = 0L + c.foreach { commit => + commit.offsets.foreach { case (tp, offsetAndMeta) => + val offset = offsetAndMeta.offset() + val maxOffset = updatedOffsets.get(tp) match { + case Some(existingOffset) => + offsetIncrease += max(0L, offset - existingOffset) + max(existingOffset, offset) + case None => + // This partition was not committed to from this consumer yet. Therefore we do not know the offset + // increase. A good estimate would be the poll size for this consumer, another okayish estimate is 0. + // Lets go with the simplest for now: ```offsetIncrease += 0``` + offset + } + updatedOffsets += tp -> maxOffset + } + } + (offsetIncrease, CommitOffsets(offsets = updatedOffsets.toMap)) + } + + def keepPartitions(tps: Set[TopicPartition]): CommitOffsets = + CommitOffsets(offsets.filter { case (tp, _) => tps.contains(tp) }) + + def contains(tp: TopicPartition, offset: Long): Boolean = + offsets.get(tp).exists(_ >= offset) + } + + private[internal] object CommitOffsets { + val empty: CommitOffsets = CommitOffsets(Map.empty) + } + +} diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/ConsumerMetrics.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/ConsumerMetrics.scala index f43329a4f..aff76f7cd 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/ConsumerMetrics.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/ConsumerMetrics.scala @@ -16,7 +16,12 @@ private[internal] trait ConsumerMetrics { def observeCommit(latency: Duration): UIO[Unit] def observeAggregatedCommit(latency: Duration, commitSize: Long): UIO[Unit] def observeRebalance(currentlyAssignedCount: Int, assignedCount: Int, revokedCount: Int, lostCount: Int): UIO[Unit] - def observeRunloopMetrics(state: Runloop.State, commandQueueSize: Int, commitQueueSize: Int): UIO[Unit] + def observeRunloopMetrics( + state: Runloop.State, + commandQueueSize: Int, + commitQueueSize: Int, + pendingCommits: Int + ): UIO[Unit] def observePollAuthError(): UIO[Unit] } @@ -330,14 +335,19 @@ private[internal] class ZioConsumerMetrics(metricLabels: Set[MetricLabel]) exten .contramap[Int](_.toDouble) .tagged(metricLabels) - override def observeRunloopMetrics(state: Runloop.State, commandQueueSize: Int, commitQueueSize: Int): UIO[Unit] = + override def observeRunloopMetrics( + state: Runloop.State, + commandQueueSize: Int, + commitQueueSize: Int, + pendingCommits: Int + ): UIO[Unit] = for { _ <- ZIO.foreachDiscard(state.assignedStreams)(_.outstandingPolls @@ queuePollsHistogram) queueSizes <- ZIO.foreach(state.assignedStreams)(_.queueSize) _ <- ZIO.foreachDiscard(queueSizes)(qs => queueSizeHistogram.update(qs)) _ <- allQueueSizeHistogram.update(queueSizes.sum) _ <- pendingRequestsHistogram.update(state.pendingRequests.size) - _ <- pendingCommitsHistogram.update(state.pendingCommits.size) + _ <- pendingCommitsHistogram.update(pendingCommits) _ <- subscriptionStateGauge.update(state.subscriptionState) _ <- commandQueueSizeHistogram.update(commandQueueSize) _ <- commitQueueSizeHistogram.update(commitQueueSize) diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala index 83948be52..8901628af 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala @@ -2,21 +2,18 @@ package zio.kafka.consumer.internal import org.apache.kafka.clients.consumer._ import org.apache.kafka.common.TopicPartition -import org.apache.kafka.common.errors.{ AuthenticationException, AuthorizationException, RebalanceInProgressException } +import org.apache.kafka.common.errors.{ AuthenticationException, AuthorizationException } import zio._ -import zio.kafka.consumer.Consumer.{ CommitTimeout, OffsetRetrieval } +import zio.kafka.consumer.Consumer.OffsetRetrieval import zio.kafka.consumer._ import zio.kafka.consumer.diagnostics.DiagnosticEvent.{ Finalization, Rebalance } import zio.kafka.consumer.diagnostics.{ DiagnosticEvent, Diagnostics } import zio.kafka.consumer.internal.ConsumerAccess.ByteArrayKafkaConsumer import zio.kafka.consumer.internal.Runloop._ import zio.kafka.consumer.internal.RunloopAccess.PartitionAssignment +import zio.kafka.consumer.internal.RunloopRebalanceListener.RebalanceEvent import zio.stream._ -import java.lang.Math.max -import java.util -import java.util.{ Map => JavaMap } -import scala.collection.mutable import scala.jdk.CollectionConverters._ //noinspection SimplifyWhenInspection,SimplifyUnlessInspection @@ -26,23 +23,15 @@ private[consumer] final class Runloop private ( sameThreadRuntime: Runtime[Any], consumer: ConsumerAccess, maxPollInterval: Duration, - commitQueue: Queue[Commit], commandQueue: Queue[RunloopCommand], - lastRebalanceEvent: Ref.Synchronized[Runloop.RebalanceEvent], + lastRebalanceEvent: Ref.Synchronized[RebalanceEvent], partitionsHub: Hub[Take[Throwable, PartitionAssignment]], diagnostics: Diagnostics, - maxRebalanceDuration: Duration, currentStateRef: Ref[State], - committedOffsetsRef: Ref[CommitOffsets] + runloopRebalanceListener: RunloopRebalanceListener, + consumerMetrics: ConsumerMetrics, + committer: Committer ) { - private val commitTimeout = settings.commitTimeout - private val commitTimeoutNanos = settings.commitTimeout.toNanos - - private val restartStreamsOnRebalancing = settings.restartStreamOnRebalancing - private val rebalanceSafeCommits = settings.rebalanceSafeCommits - - private val consumerMetrics = new ZioConsumerMetrics(settings.metricLabels) - private def newPartitionStream(tp: TopicPartition): UIO[PartitionStreamControl] = PartitionStreamControl.newPartitionStream(tp, commandQueue, diagnostics, maxPollInterval) @@ -76,236 +65,6 @@ private[consumer] final class Runloop private ( commandQueue.offer(RunloopCommand.RemoveSubscription(subscription)).unit private def makeRebalanceListener: ConsumerRebalanceListener = { - // All code in this block is called from the rebalance listener and therefore runs on the same-thread-runtime. This - // is because the Java kafka client requires us to invoke the consumer from the same thread that invoked the - // rebalance listener. - // Unfortunately the same-thread-runtime does not work for all ZIO operations. For example, `ZIO.timeout`, - // `ZStream.repeat`, `Promise.await` on non-completed promises, and any other ZIO operation that shifts the work to - // another thread cannot be used. - - // Time between polling the commit queue from the rebalance listener when `rebalanceSafeCommits` is enabled. - val commitQueuePollInterval = 100.millis - - // End streams from the rebalance listener. - // When `rebalanceSafeCommits` is enabled, wait for consumed offsets to be committed. - def endStreams(state: State, streamsToEnd: Chunk[PartitionStreamControl]): Task[Unit] = - if (streamsToEnd.isEmpty) ZIO.unit - else { - for { - _ <- ZIO.foreachDiscard(streamsToEnd)(_.end) - _ <- if (rebalanceSafeCommits) - consumer.rebalanceListenerAccess(doAwaitStreamCommits(_, state, streamsToEnd)) - else ZIO.unit - } yield () - } - - def doAwaitStreamCommits( - consumer: ByteArrayKafkaConsumer, - state: State, - streamsToEnd: Chunk[PartitionStreamControl] - ): Task[Unit] = { - val deadline = java.lang.System.nanoTime() + maxRebalanceDuration.toNanos - commitTimeoutNanos - - def timeToDeadlineMillis(): Long = (deadline - java.lang.System.nanoTime()) / 1000000L - - val endingTps = streamsToEnd.map(_.tp).toSet - - def commitsOfEndingStreams(commits: Chunk[Runloop.Commit]): Chunk[Runloop.Commit] = - commits.filter(commit => (commit.offsets.keySet intersect endingTps).nonEmpty) - - lazy val previousPendingCommits: Chunk[Commit] = - commitsOfEndingStreams(state.pendingCommits) - - def commitAsync(commits: Chunk[Commit]): UIO[Unit] = - if (commits.nonEmpty) { - val (offsets, callback, onFailure) = asyncCommitParameters(commits) - ZIO.logDebug(s"Async commit of ${offsets.size} offsets for ${commits.size} commits") *> - ZIO.attempt(consumer.commitAsync(offsets, callback)).catchAll(onFailure) - } else { - // Continue to drive communication with the broker so that commits can complete and the streams can - // make progress. - ZIO.attempt(consumer.commitAsync(java.util.Collections.emptyMap(), null)).orDie - } - - sealed trait EndOffsetCommitStatus - case object EndOffsetNotCommitted extends EndOffsetCommitStatus { override def toString = "not committed" } - case object EndOffsetCommitPending extends EndOffsetCommitStatus { override def toString = "commit pending" } - case object EndOffsetCommitted extends EndOffsetCommitStatus { override def toString = "committed" } - - final case class StreamCompletionStatus( - tp: TopicPartition, - streamEnded: Boolean, - lastPulledOffset: Option[Long], - endOffsetCommitStatus: EndOffsetCommitStatus - ) { - override def toString: String = - s"${tp}: " + - s"${if (streamEnded) "stream ended" else "stream is running"}, " + - s"last pulled offset=${lastPulledOffset.getOrElse("none")}, " + - endOffsetCommitStatus - } - - def completionStatusesAsString(completionStatuses: Chunk[StreamCompletionStatus]): String = - "Revoked partitions: " + completionStatuses.map(_.toString).mkString("; ") - - def getStreamCompletionStatuses(newCommits: Chunk[Commit]): UIO[Chunk[StreamCompletionStatus]] = - for { - committedOffsets <- committedOffsetsRef.get - allPendingCommitOffsets = - (previousPendingCommits ++ commitsOfEndingStreams(newCommits)).flatMap(_.offsets).map { - case (tp, offsetAndMetadata) => (tp, offsetAndMetadata.offset()) - } - streamResults <- - ZIO.foreach(streamsToEnd) { stream => - for { - isDone <- stream.completedPromise.isDone - lastPulledOffset <- stream.lastPulledOffset - endOffset <- if (isDone) stream.completedPromise.await else ZIO.none - - endOffsetCommitStatus = - endOffset match { - case Some(endOffset) if committedOffsets.contains(stream.tp, endOffset.offset) => - EndOffsetCommitted - case Some(endOffset) if allPendingCommitOffsets.contains((stream.tp, endOffset.offset)) => - EndOffsetCommitPending - case _ => EndOffsetNotCommitted - } - } yield StreamCompletionStatus(stream.tp, isDone, lastPulledOffset.map(_.offset), endOffsetCommitStatus) - } - } yield streamResults - - @inline - def logStreamCompletionStatuses(completionStatuses: Chunk[StreamCompletionStatus]): UIO[Unit] = { - val statusStrings = completionStatusesAsString(completionStatuses) - ZIO.logInfo( - s"Delaying rebalance until ${streamsToEnd.size} streams (of revoked partitions) have committed " + - s"the offsets of the records they consumed. Deadline in ${timeToDeadlineMillis()}ms. $statusStrings" - ) - } - - def logInitialStreamCompletionStatuses: UIO[Unit] = - for { - completionStatuses <- getStreamCompletionStatuses(newCommits = Chunk.empty) - _ <- logStreamCompletionStatuses(completionStatuses) - } yield () - - def endingStreamsCompletedAndCommitsExist(newCommits: Chunk[Commit]): UIO[Boolean] = - for { - completionStatuses <- getStreamCompletionStatuses(newCommits) - _ <- logStreamCompletionStatuses(completionStatuses) - } yield completionStatuses.forall { status => - // A stream is complete when it never got any records, or when it committed the offset of the last consumed record - status.lastPulledOffset.isEmpty || (status.streamEnded && status.endOffsetCommitStatus != EndOffsetNotCommitted) - } - - def logFinalStreamCompletionStatuses(completed: Boolean, newCommits: Chunk[Commit]): UIO[Unit] = - if (completed) - ZIO.logInfo("Continuing rebalance, all offsets of consumed records in the revoked partitions were committed.") - else - for { - completionStatuses <- getStreamCompletionStatuses(newCommits) - statusStrings = completionStatusesAsString(completionStatuses) - _ <- - ZIO.logWarning( - s"Exceeded deadline waiting for streams (of revoked partitions) to commit the offsets of " + - s"the records they consumed; the rebalance will continue. " + - s"This might cause another consumer to process some records again. $statusStrings" - ) - } yield () - - def commitSync: Task[Unit] = - ZIO.attempt(consumer.commitSync(java.util.Collections.emptyMap(), commitTimeout)) - - // Outline: - // - Every `commitQueuePollInterval` until the deadline has been reached: - // - Get all commits from the commit queue. - // - Start an async commit for these commits. - // - Collect all these new (pending) commits. - // - repeat the above until: - // - All streams that were ended have completed their work, and - // - we have seen a completed or pending commit for all end-offsets. - // An end-offset of a stream is the offset of the last record given to that stream. - // - Do a single sync commit without any offsets, this has the side-effect of blocking until all - // preceding async commits are complete (this requires kafka-client 3.6.0 or later). - // Because all commits created here (including those from non-ending streams) are now complete, we do not - // have to add them to the pending commits of the runloop state. - // - // Note, we cannot use ZStream.fromQueue because that will emit nothing when the queue is empty. - // Instead, we poll the queue in a loop. - for { - _ <- logInitialStreamCompletionStatuses - completedAndCommits <- - ZStream - .fromZIO(blockingSleep(commitQueuePollInterval) *> commitQueue.takeAll) - .tap(commitAsync) - .forever - .takeWhile(_ => java.lang.System.nanoTime() <= deadline) - .scan(Chunk.empty[Runloop.Commit])(_ ++ _) - .mapZIO(commits => endingStreamsCompletedAndCommitsExist(commits).map((_, commits))) - .takeUntil { case (completed, _) => completed } - .runLast - .map(_.getOrElse((false, Chunk.empty))) - _ <- logFinalStreamCompletionStatuses(completedAndCommits._1, completedAndCommits._2) - _ <- commitSync - _ <- ZIO.logDebug(s"Done waiting for ${streamsToEnd.size} streams to end") - } yield () - } - - // During a poll, the java kafka client might call each method of the rebalance listener 0 or 1 times. - // We do not know the order in which the call-back methods are invoked. - // - // Ref `lastRebalanceEvent` is used to track what happens during the poll. Just before the poll the - // `RebalanceEvent.None` is stored. Then during the poll, inside each method of the rebalance listener, - // the ref is updated. - // - // Each method: - // - emits a diagnostic event - // - determines if this is the first method invoked during this poll (`rebalanceEvent.wasInvoked`) to - // make sure that the `restartStreamsOnRebalancing` feature is applied only once per poll - // - ends streams that need to be ended - // - updates `lastRebalanceEvent` - // - val recordRebalanceRebalancingListener = RebalanceListener( - onAssigned = assignedTps => - for { - rebalanceEvent <- lastRebalanceEvent.get - _ <- ZIO.logDebug { - val sameRebalance = if (rebalanceEvent.wasInvoked) " in same rebalance" else "" - s"${assignedTps.size} partitions are assigned$sameRebalance" - } - state <- currentStateRef.get - streamsToEnd = if (restartStreamsOnRebalancing && !rebalanceEvent.wasInvoked) state.assignedStreams - else Chunk.empty - _ <- endStreams(state, streamsToEnd) - _ <- lastRebalanceEvent.set(rebalanceEvent.onAssigned(assignedTps, endedStreams = streamsToEnd)) - _ <- ZIO.logTrace("onAssigned done") - } yield (), - onRevoked = revokedTps => - for { - rebalanceEvent <- lastRebalanceEvent.get - _ <- ZIO.logDebug { - val sameRebalance = if (rebalanceEvent.wasInvoked) " in same rebalance" else "" - s"${revokedTps.size} partitions are revoked$sameRebalance" - } - state <- currentStateRef.get - streamsToEnd = if (restartStreamsOnRebalancing && !rebalanceEvent.wasInvoked) state.assignedStreams - else state.assignedStreams.filter(control => revokedTps.contains(control.tp)) - _ <- endStreams(state, streamsToEnd) - _ <- lastRebalanceEvent.set(rebalanceEvent.onRevoked(revokedTps, endedStreams = streamsToEnd)) - _ <- ZIO.logTrace("onRevoked done") - } yield (), - onLost = lostTps => - for { - _ <- ZIO.logDebug(s"${lostTps.size} partitions are lost") - rebalanceEvent <- lastRebalanceEvent.get - state <- currentStateRef.get - lostStreams = state.assignedStreams.filter(control => lostTps.contains(control.tp)) - _ <- ZIO.foreachDiscard(lostStreams)(_.lost) - _ <- lastRebalanceEvent.set(rebalanceEvent.onLost(lostTps, lostStreams)) - _ <- ZIO.logTrace(s"onLost done") - } yield () - ) - // Here we just want to avoid any executor shift if the user provided listener is the noop listener. val userRebalanceListener = settings.rebalanceListener match { @@ -313,93 +72,9 @@ private[consumer] final class Runloop private ( case _ => settings.rebalanceListener.runOnExecutor(topLevelExecutor) } - RebalanceListener.toKafka(recordRebalanceRebalancingListener ++ userRebalanceListener, sameThreadRuntime) + RebalanceListener.toKafka(runloopRebalanceListener.toRebalanceListener ++ userRebalanceListener, sameThreadRuntime) } - /** This is the implementation behind the user facing api `Offset.commit`. */ - private val commit: Map[TopicPartition, OffsetAndMetadata] => Task[Unit] = - offsets => - for { - p <- Promise.make[Throwable, Unit] - startTime = java.lang.System.nanoTime() - _ <- commitQueue.offer(Runloop.Commit(java.lang.System.nanoTime(), offsets, p)) - _ <- commandQueue.offer(RunloopCommand.CommitAvailable) - _ <- diagnostics.emit(DiagnosticEvent.Commit.Started(offsets)) - _ <- p.await.timeoutFail(CommitTimeout)(commitTimeout) - endTime = java.lang.System.nanoTime() - latency = (endTime - startTime).nanoseconds - _ <- consumerMetrics.observeCommit(latency) - } yield () - - /** Merge commits and prepare parameters for calling `consumer.commitAsync`. */ - private def asyncCommitParameters( - commits: Chunk[Runloop.Commit] - ): (JavaMap[TopicPartition, OffsetAndMetadata], OffsetCommitCallback, Throwable => UIO[Unit]) = { - val offsets = commits - .foldLeft(mutable.Map.empty[TopicPartition, OffsetAndMetadata]) { case (acc, commit) => - commit.offsets.foreach { case (tp, offset) => - acc += (tp -> acc - .get(tp) - .map(current => if (current.offset() > offset.offset()) current else offset) - .getOrElse(offset)) - } - acc - } - .toMap - val offsetsWithMetaData = offsets.map { case (tp, offset) => - tp -> new OffsetAndMetadata(offset.offset + 1, offset.leaderEpoch, offset.metadata) - } - val cont = (e: Exit[Throwable, Unit]) => ZIO.foreachDiscard(commits)(_.cont.done(e)) - // We assume the commit is started immediately after returning from this method. - val startTime = java.lang.System.nanoTime() - val onSuccess = { - val endTime = java.lang.System.nanoTime() - val latency = (endTime - startTime).nanoseconds - for { - offsetIncrease <- committedOffsetsRef.modify(_.addCommits(commits)) - _ <- consumerMetrics.observeAggregatedCommit(latency, offsetIncrease).when(commits.nonEmpty) - result <- cont(Exit.unit) - _ <- diagnostics.emit(DiagnosticEvent.Commit.Success(offsetsWithMetaData)) - } yield result - } - val onFailure: Throwable => UIO[Unit] = { - case _: RebalanceInProgressException => - for { - _ <- ZIO.logDebug(s"Rebalance in progress, commit for offsets $offsets will be retried") - _ <- commitQueue.offerAll(commits) - _ <- commandQueue.offer(RunloopCommand.CommitAvailable) - } yield () - case err: Throwable => - cont(Exit.fail(err)) <* diagnostics.emit(DiagnosticEvent.Commit.Failure(offsetsWithMetaData, err)) - } - val callback = - new OffsetCommitCallback { - override def onComplete(offsets: util.Map[TopicPartition, OffsetAndMetadata], exception: Exception): Unit = - Unsafe.unsafe { implicit u => - sameThreadRuntime.unsafe.run { - if (exception eq null) onSuccess else onFailure(exception) - } - .getOrThrowFiberFailure() - } - } - (offsetsWithMetaData.asJava, callback, onFailure) - } - - private def handleCommits(state: State, commits: Chunk[Runloop.Commit]): UIO[State] = - if (commits.isEmpty) { - ZIO.succeed(state) - } else { - val (offsets, callback, onFailure) = asyncCommitParameters(commits) - val newState = state.addPendingCommits(commits) - consumer.runloopAccess { c => - // We don't wait for the completion of the commit here, because it - // will only complete once we poll again. - ZIO.attempt(c.commitAsync(offsets, callback)) - } - .catchAll(onFailure) - .as(newState) - } - /** * Does all needed to end revoked partitions: * 1. Complete the revoked assigned streams 2. Remove from the list of pending requests @@ -463,7 +138,7 @@ private[consumer] final class Runloop private ( builder += CommittableRecord[Array[Byte], Array[Byte]]( record = consumerRecord, - commitHandle = commit, + commitHandle = committer.commit, consumerGroupMetadata = consumerGroupMetadata ) } @@ -530,7 +205,7 @@ private[consumer] final class Runloop private ( partitionsToFetch <- settings.fetchStrategy.selectPartitionsToFetch(state.assignedStreams) _ <- ZIO.logDebug( s"Starting poll with ${state.pendingRequests.size} pending requests and" + - s" ${state.pendingCommits.size} pending commits," + + s" ${committer.pendingCommitCount} pending commits," + s" resuming $partitionsToFetch partitions" ) _ <- currentStateRef.set(state) @@ -607,9 +282,7 @@ private[consumer] final class Runloop private ( // Remove committed offsets for partitions that are no longer assigned: // NOTE: the type annotation is needed to keep the IntelliJ compiler happy. - _ <- - committedOffsetsRef - .update(_.keepPartitions(updatedAssignedStreams.map(_.tp).toSet)): Task[Unit] + _ <- committer.pruneCommittedOffsets(updatedAssignedStreams.map(_.tp).toSet): Task[Unit] _ <- consumerMetrics.observeRebalance( currentAssigned.size, @@ -651,11 +324,10 @@ private[consumer] final class Runloop private ( pollResult.ignoreRecordsForTps, pollResult.records ) - updatedPendingCommits <- ZIO.filter(state.pendingCommits)(_.isPending) - _ <- checkStreamPollInterval(pollResult.assignedStreams) + _ <- committer.updatePendingCommitsAfterPoll + _ <- checkStreamPollInterval(pollResult.assignedStreams) } yield state.copy( pendingRequests = fulfillResult.pendingRequests, - pendingCommits = updatedPendingCommits, assignedStreams = pollResult.assignedStreams ) } @@ -805,19 +477,17 @@ private[consumer] final class Runloop private ( .takeWhile(_ != RunloopCommand.StopRunloop) .runFoldChunksDiscardZIO(initialState) { (state, commands) => for { - commitCommands <- commitQueue.takeAll - _ <- ZIO.logDebug( - s"Processing ${commitCommands.size} commits," + - s" ${commands.size} commands: ${commands.mkString(",")}" - ) - stateAfterCommits <- handleCommits(state, commitCommands) + _ <- ZIO.logDebug(s"Processing ${commands.size} commands: ${commands.mkString(",")}") + _ <- consumer.runloopAccess(committer.handleNewCommits(_)) streamCommands = commands.collect { case cmd: RunloopCommand.StreamCommand => cmd } - stateAfterCommands <- ZIO.foldLeft(streamCommands)(stateAfterCommits)(handleCommand) + stateAfterCommands <- ZIO.foldLeft(streamCommands)(state)(handleCommand) - updatedStateAfterPoll <- if (stateAfterCommands.shouldPoll) handlePoll(stateAfterCommands) - else ZIO.succeed(stateAfterCommands) + updatedStateAfterPoll <- shouldPoll(stateAfterCommands).flatMap { + case true => handlePoll(stateAfterCommands) + case false => ZIO.succeed(stateAfterCommands) + } // Immediately poll again, after processing all new queued commands - _ <- if (updatedStateAfterPoll.shouldPoll) commandQueue.offer(RunloopCommand.Poll) else ZIO.unit + _ <- ZIO.whenZIO(shouldPoll(updatedStateAfterPoll))(commandQueue.offer(RunloopCommand.Poll)) // Save the current state for other parts of Runloop (read-only, for metrics only) _ <- currentStateRef.set(updatedStateAfterPoll) } yield updatedStateAfterPoll @@ -826,13 +496,19 @@ private[consumer] final class Runloop private ( .onError(cause => partitionsHub.offer(Take.failCause(cause))) } + def shouldPoll(state: State): UIO[Boolean] = + committer.pendingCommitCount.map { pendingCommitCount => + state.subscriptionState.isSubscribed && (state.pendingRequests.nonEmpty || pendingCommitCount > 0 || state.assignedStreams.isEmpty) + } + private def observeRunloopMetrics(runloopMetricsSchedule: Schedule[Any, Unit, Long]): ZIO[Any, Nothing, Unit] = { val observe = for { currentState <- currentStateRef.get commandQueueSize <- commandQueue.size - commitQueueSize <- commitQueue.size + commitQueueSize <- committer.queueSize + pendingCommits <- committer.pendingCommitCount _ <- consumerMetrics - .observeRunloopMetrics(currentState, commandQueueSize, commitQueueSize) + .observeRunloopMetrics(currentState, commandQueueSize, commitQueueSize, pendingCommits) } yield () observe @@ -878,57 +554,6 @@ object Runloop { pendingRequests: Chunk[RunloopCommand.Request] ) - private final case class RebalanceEvent( - wasInvoked: Boolean, - assignedTps: Set[TopicPartition], - revokedTps: Set[TopicPartition], - lostTps: Set[TopicPartition], - endedStreams: Chunk[PartitionStreamControl] - ) { - def onAssigned( - assigned: Set[TopicPartition], - endedStreams: Chunk[PartitionStreamControl] - ): RebalanceEvent = - copy( - wasInvoked = true, - assignedTps = assignedTps ++ assigned, - endedStreams = this.endedStreams ++ endedStreams - ) - - def onRevoked( - revoked: Set[TopicPartition], - endedStreams: Chunk[PartitionStreamControl] - ): RebalanceEvent = - copy( - wasInvoked = true, - assignedTps = assignedTps -- revoked, - revokedTps = revokedTps ++ revoked, - endedStreams = this.endedStreams ++ endedStreams - ) - - def onLost(lost: Set[TopicPartition], endedStreams: Chunk[PartitionStreamControl]): RebalanceEvent = - copy( - wasInvoked = true, - assignedTps = assignedTps -- lost, - lostTps = lostTps ++ lost, - endedStreams = this.endedStreams ++ endedStreams - ) - } - - private object RebalanceEvent { - val None: RebalanceEvent = - RebalanceEvent(wasInvoked = false, Set.empty, Set.empty, Set.empty, Chunk.empty) - } - - private[internal] final case class Commit( - createdAt: NanoTime, - offsets: Map[TopicPartition, OffsetAndMetadata], - cont: Promise[Throwable, Unit] - ) { - @inline def isDone: UIO[Boolean] = cont.isDone - @inline def isPending: UIO[Boolean] = isDone.negate - } - private[consumer] def make( settings: ConsumerSettings, maxPollInterval: Duration, @@ -939,28 +564,42 @@ object Runloop { ): URIO[Scope, Runloop] = for { _ <- ZIO.addFinalizer(diagnostics.emit(Finalization.RunloopFinalized)) - commitQueue <- ZIO.acquireRelease(Queue.unbounded[Runloop.Commit])(_.shutdown) commandQueue <- ZIO.acquireRelease(Queue.unbounded[RunloopCommand])(_.shutdown) - lastRebalanceEvent <- Ref.Synchronized.make[Runloop.RebalanceEvent](Runloop.RebalanceEvent.None) + lastRebalanceEvent <- Ref.Synchronized.make[RebalanceEvent](RebalanceEvent.None) initialState = State.initial - currentStateRef <- Ref.make(initialState) - committedOffsetsRef <- Ref.make(CommitOffsets.empty) - sameThreadRuntime <- ZIO.runtime[Any].provideLayer(SameThreadRuntimeLayer) - executor <- ZIO.executor + currentStateRef <- Ref.make(initialState) + sameThreadRuntime <- ZIO.runtime[Any].provideLayer(SameThreadRuntimeLayer) + executor <- ZIO.executor + metrics = new ZioConsumerMetrics(settings.metricLabels) + committer <- Committer.make( + settings, + diagnostics, + metrics, + commandQueue.offer(RunloopCommand.CommitAvailable).unit, + sameThreadRuntime + ) + rebalanceListener = new RunloopRebalanceListener( + lastRebalanceEvent, + settings, + consumer, + maxRebalanceDuration, + currentStateRef.get.map(_.assignedStreams), + committer + ) runloop = new Runloop( settings = settings, topLevelExecutor = executor, sameThreadRuntime = sameThreadRuntime, consumer = consumer, maxPollInterval = maxPollInterval, - commitQueue = commitQueue, commandQueue = commandQueue, lastRebalanceEvent = lastRebalanceEvent, partitionsHub = partitionsHub, diagnostics = diagnostics, - maxRebalanceDuration = maxRebalanceDuration, currentStateRef = currentStateRef, - committedOffsetsRef = committedOffsetsRef + runloopRebalanceListener = rebalanceListener, + consumerMetrics = metrics, + committer = committer ) _ <- ZIO.logDebug("Starting Runloop") @@ -981,62 +620,18 @@ object Runloop { private[internal] final case class State( pendingRequests: Chunk[RunloopCommand.Request], - pendingCommits: Chunk[Runloop.Commit], assignedStreams: Chunk[PartitionStreamControl], subscriptionState: SubscriptionState ) { - def addPendingCommits(c: Chunk[Runloop.Commit]): State = copy(pendingCommits = pendingCommits ++ c) - def addRequest(r: RunloopCommand.Request): State = copy(pendingRequests = pendingRequests :+ r) - - def shouldPoll: Boolean = - subscriptionState.isSubscribed && (pendingRequests.nonEmpty || pendingCommits.nonEmpty || assignedStreams.isEmpty) + def addRequest(r: RunloopCommand.Request): State = copy(pendingRequests = pendingRequests :+ r) } private object State { val initial: State = State( pendingRequests = Chunk.empty, - pendingCommits = Chunk.empty, assignedStreams = Chunk.empty, subscriptionState = SubscriptionState.NotSubscribed ) } - // package private for unit testing - private[internal] final case class CommitOffsets(offsets: Map[TopicPartition, Long]) { - - /** Returns an estimate of the total offset increase, and a new `CommitOffsets` with the given offsets added. */ - def addCommits(c: Chunk[Runloop.Commit]): (Long, CommitOffsets) = { - val updatedOffsets = mutable.Map.empty[TopicPartition, Long] - updatedOffsets.sizeHint(offsets.size) - updatedOffsets ++= offsets - var offsetIncrease = 0L - c.foreach { commit => - commit.offsets.foreach { case (tp, offsetAndMeta) => - val offset = offsetAndMeta.offset() - val maxOffset = updatedOffsets.get(tp) match { - case Some(existingOffset) => - offsetIncrease += max(0L, offset - existingOffset) - max(existingOffset, offset) - case None => - // This partition was not committed to from this consumer yet. Therefore we do not know the offset - // increase. A good estimate would be the poll size for this consumer, another okayish estimate is 0. - // Lets go with the simplest for now: ```offsetIncrease += 0``` - offset - } - updatedOffsets += tp -> maxOffset - } - } - (offsetIncrease, CommitOffsets(offsets = updatedOffsets.toMap)) - } - - def keepPartitions(tps: Set[TopicPartition]): CommitOffsets = - CommitOffsets(offsets.filter { case (tp, _) => tps.contains(tp) }) - - def contains(tp: TopicPartition, offset: Long): Boolean = - offsets.get(tp).exists(_ >= offset) - } - - private[internal] object CommitOffsets { - val empty: CommitOffsets = CommitOffsets(Map.empty) - } } diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/RunloopRebalanceListener.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/RunloopRebalanceListener.scala new file mode 100644 index 000000000..15c354597 --- /dev/null +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/RunloopRebalanceListener.scala @@ -0,0 +1,289 @@ +package zio.kafka.consumer.internal +import org.apache.kafka.common.TopicPartition +import zio.kafka.consumer.internal.ConsumerAccess.ByteArrayKafkaConsumer +import zio.kafka.consumer.internal.RunloopRebalanceListener.{ + EndOffsetCommitPending, + EndOffsetCommitted, + EndOffsetNotCommitted, + RebalanceEvent, + StreamCompletionStatus +} +import zio.kafka.consumer.{ ConsumerSettings, RebalanceListener } +import zio.stream.ZStream +import zio.{ durationInt, Chunk, Duration, Ref, Task, UIO, ZIO } + +/** + * The Runloop's RebalanceListener gets notified of partitions that are assigned, revoked and lost + * + * Because this happens during the call to `poll()`, we communicate any results to the Runloop via a `Ref` + * + * When rebalanceSafeCommits is enabled, we await completion of all revoked partitions' streams and their commits before + * continuing. + */ +private[internal] class RunloopRebalanceListener( + lastRebalanceEvent: Ref.Synchronized[RebalanceEvent], + settings: ConsumerSettings, + consumer: ConsumerAccess, + maxRebalanceDuration: Duration, + getCurrentAssignedStreams: UIO[Chunk[PartitionStreamControl]], + committer: Committer +) { + private val commitTimeoutNanos = settings.commitTimeout.toNanos + + private val restartStreamsOnRebalancing = settings.restartStreamOnRebalancing + private val rebalanceSafeCommits = settings.rebalanceSafeCommits + private val commitTimeout = settings.commitTimeout + + // All code in this block is called from the rebalance listener and therefore runs on the same-thread-runtime. This + // is because the Java kafka client requires us to invoke the consumer from the same thread that invoked the + // rebalance listener. + // Unfortunately the same-thread-runtime does not work for all ZIO operations. For example, `ZIO.timeout`, + // `ZStream.repeat`, `Promise.await` on non-completed promises, and any other ZIO operation that shifts the work to + // another thread cannot be used. + + // Time between polling the commit queue from the rebalance listener when `rebalanceSafeCommits` is enabled. + private val commitQueuePollInterval = 100.millis + + // End streams from the rebalance listener. + // When `rebalanceSafeCommits` is enabled, wait for consumed offsets to be committed. + private def endStreams(streamsToEnd: Chunk[PartitionStreamControl]): Task[Any] = + ZIO.unless(streamsToEnd.isEmpty) { + for { + _ <- ZIO.foreachDiscard(streamsToEnd)(_.end) + _ <- consumer.rebalanceListenerAccess(doAwaitStreamCommits(_, streamsToEnd)).when(rebalanceSafeCommits) + } yield () + } + + private def doAwaitStreamCommits( + consumer: ByteArrayKafkaConsumer, + streamsToEnd: Chunk[PartitionStreamControl] + ): Task[Unit] = { + val deadline = java.lang.System.nanoTime() + maxRebalanceDuration.toNanos - commitTimeoutNanos + + def timeToDeadlineMillis(): Long = (java.lang.System.nanoTime() - deadline) / 1000000L + + def completionStatusesAsString(completionStatuses: Chunk[StreamCompletionStatus]): String = + "Revoked partitions: " + completionStatuses.map(_.toString).mkString("; ") + + def getStreamCompletionStatuses: UIO[Chunk[StreamCompletionStatus]] = + for { + committedOffsets <- committer.getCommittedOffsets + allPendingCommitOffsets <- committer.getPendingCommits.map(_.offsets.toSeq) // TODO toSeq efficient enough? + streamResults <- + ZIO.foreach(streamsToEnd) { stream => + for { + isDone <- stream.completedPromise.isDone + lastPulledOffset <- stream.lastPulledOffset + endOffset <- if (isDone) stream.completedPromise.await else ZIO.none + + endOffsetCommitStatus = + endOffset match { + case Some(endOffset) if committedOffsets.contains(stream.tp, endOffset.offset) => + EndOffsetCommitted + case Some(endOffset) if allPendingCommitOffsets.contains((stream.tp, endOffset.offset)) => + EndOffsetCommitPending + case _ => EndOffsetNotCommitted + } + } yield StreamCompletionStatus(stream.tp, isDone, lastPulledOffset.map(_.offset), endOffsetCommitStatus) + } + } yield streamResults + + @inline + def logStreamCompletionStatuses(completionStatuses: Chunk[StreamCompletionStatus]): UIO[Unit] = { + val statusStrings = completionStatusesAsString(completionStatuses) + ZIO.logInfo( + s"Delaying rebalance until ${streamsToEnd.size} streams (of revoked partitions) have committed " + + s"the offsets of the records they consumed. Deadline in ${timeToDeadlineMillis()}ms. $statusStrings" + ) + } + + def logInitialStreamCompletionStatuses: UIO[Unit] = + for { + completionStatuses <- getStreamCompletionStatuses + _ <- logStreamCompletionStatuses(completionStatuses) + } yield () + + def endingStreamsCompletedAndCommitsExist: UIO[Boolean] = + for { + completionStatuses <- getStreamCompletionStatuses + _ <- logStreamCompletionStatuses(completionStatuses) + } yield completionStatuses.forall { status => + // A stream is complete when it never got any records, or when it committed the offset of the last consumed record + status.lastPulledOffset.isEmpty || (status.streamEnded && status.endOffsetCommitStatus != EndOffsetNotCommitted) + } + + def logFinalStreamCompletionStatuses(completed: Boolean): UIO[Unit] = + if (completed) + ZIO.logInfo("Continuing rebalance, all offsets of consumed records in the revoked partitions were committed.") + else + for { + completionStatuses <- getStreamCompletionStatuses + statusStrings = completionStatusesAsString(completionStatuses) + _ <- + ZIO.logWarning( + s"Exceeded deadline waiting for streams (of revoked partitions) to commit the offsets of " + + s"the records they consumed; the rebalance will continue. " + + s"This might cause another consumer to process some records again. $statusStrings" + ) + } yield () + + def commitSync: Task[Unit] = + ZIO.attempt(consumer.commitSync(java.util.Collections.emptyMap(), commitTimeout)) + + // Outline: + // - Every `commitQueuePollInterval` until the deadline has been reached: + // - Get all commits from the commit queue. + // - Start an async commit for these commits. + // - Collect all these new (pending) commits. + // - repeat the above until: + // - All streams that were ended have completed their work, and + // (TODO is the pending part still true? Code did not reflect that) + // - we have seen a completed or pending commit for all end-offsets. + // An end-offset of a stream is the offset of the last record given to that stream. + // - Do a single sync commit without any offsets, this has the side-effect of blocking until all + // preceding async commits are complete (this requires kafka-client 3.6.0 or later). + // Because all commits created here (including those from non-ending streams) are now complete, we do not + // have to add them to the pending commits of the runloop state. + // + // Note, we cannot use ZStream.fromQueue because that will emit nothing when the queue is empty. + // Instead, we poll the queue in a loop. + for { + _ <- logInitialStreamCompletionStatuses + completed <- + ZStream + .fromZIO(blockingSleep(commitQueuePollInterval)) + .forever + // Even if there is nothing to commit, continue to drive communication with the broker + // so that commits can complete and the streams can make progress, by setting + // executeOnEmpty = true + .tap(_ => committer.handleNewCommits(consumer, executeOnEmpty = true)) + .takeWhile(_ => java.lang.System.nanoTime() <= deadline) + .mapZIO(_ => endingStreamsCompletedAndCommitsExist) + .takeUntil(completed => completed) + .runLast + .map(_.getOrElse(false)) + _ <- logFinalStreamCompletionStatuses(completed) + _ <- commitSync + _ <- ZIO.logDebug(s"Done waiting for ${streamsToEnd.size} streams to end") + } yield () + } + + // During a poll, the java kafka client might call each method of the rebalance listener 0 or 1 times. + // We do not know the order in which the call-back methods are invoked. + // + // Ref `lastRebalanceEvent` is used to track what happens during the poll. Just before the poll the + // `RebalanceEvent.None` is stored. Then during the poll, inside each method of the rebalance listener, + // the ref is updated. + // + // Each method: + // - emits a diagnostic event + // - determines if this is the first method invoked during this poll (`rebalanceEvent.wasInvoked`) to + // make sure that the `restartStreamsOnRebalancing` feature is applied only once per poll + // - ends streams that need to be ended + // - updates `lastRebalanceEvent` + // + + def toRebalanceListener: RebalanceListener = RebalanceListener( + onAssigned = assignedTps => + for { + rebalanceEvent <- lastRebalanceEvent.get + _ <- ZIO.logDebug { + val sameRebalance = if (rebalanceEvent.wasInvoked) " in same rebalance" else "" + s"${assignedTps.size} partitions are assigned$sameRebalance" + } + assignedStreams <- getCurrentAssignedStreams + streamsToEnd = if (restartStreamsOnRebalancing && !rebalanceEvent.wasInvoked) assignedStreams + else Chunk.empty + _ <- endStreams(streamsToEnd) + _ <- lastRebalanceEvent.set(rebalanceEvent.onAssigned(assignedTps, endedStreams = streamsToEnd)) + _ <- ZIO.logTrace("onAssigned done") + } yield (), + onRevoked = revokedTps => + for { + rebalanceEvent <- lastRebalanceEvent.get + _ <- ZIO.logDebug { + val sameRebalance = if (rebalanceEvent.wasInvoked) " in same rebalance" else "" + s"${revokedTps.size} partitions are revoked$sameRebalance" + } + assignedStreams <- getCurrentAssignedStreams + streamsToEnd = if (restartStreamsOnRebalancing && !rebalanceEvent.wasInvoked) assignedStreams + else assignedStreams.filter(control => revokedTps.contains(control.tp)) + _ <- endStreams(streamsToEnd) + _ <- lastRebalanceEvent.set(rebalanceEvent.onRevoked(revokedTps, endedStreams = streamsToEnd)) + _ <- ZIO.logTrace("onRevoked done") + } yield (), + onLost = lostTps => + for { + _ <- ZIO.logDebug(s"${lostTps.size} partitions are lost") + rebalanceEvent <- lastRebalanceEvent.get + assignedStreams <- getCurrentAssignedStreams + lostStreams = assignedStreams.filter(control => lostTps.contains(control.tp)) + _ <- ZIO.foreachDiscard(lostStreams)(_.lost) + _ <- lastRebalanceEvent.set(rebalanceEvent.onLost(lostTps, lostStreams)) + _ <- ZIO.logTrace(s"onLost done") + } yield () + ) +} + +private[internal] object RunloopRebalanceListener { + + sealed trait EndOffsetCommitStatus + case object EndOffsetNotCommitted extends EndOffsetCommitStatus { override def toString = "not committed" } + case object EndOffsetCommitPending extends EndOffsetCommitStatus { override def toString = "commit pending" } + case object EndOffsetCommitted extends EndOffsetCommitStatus { override def toString = "committed" } + + final case class StreamCompletionStatus( + tp: TopicPartition, + streamEnded: Boolean, + lastPulledOffset: Option[Long], + endOffsetCommitStatus: EndOffsetCommitStatus + ) { + override def toString: String = + s"$tp: " + + s"${if (streamEnded) "stream ended" else "stream is running"}, " + + s"last pulled offset=${lastPulledOffset.getOrElse("none")}, " + + endOffsetCommitStatus + } + + final case class RebalanceEvent( + wasInvoked: Boolean, + assignedTps: Set[TopicPartition], + revokedTps: Set[TopicPartition], + lostTps: Set[TopicPartition], + endedStreams: Chunk[PartitionStreamControl] + ) { + def onAssigned( + assigned: Set[TopicPartition], + endedStreams: Chunk[PartitionStreamControl] + ): RebalanceEvent = + copy( + wasInvoked = true, + assignedTps = assignedTps ++ assigned, + endedStreams = this.endedStreams ++ endedStreams + ) + + def onRevoked( + revoked: Set[TopicPartition], + endedStreams: Chunk[PartitionStreamControl] + ): RebalanceEvent = + copy( + wasInvoked = true, + assignedTps = assignedTps -- revoked, + revokedTps = revokedTps ++ revoked, + endedStreams = this.endedStreams ++ endedStreams + ) + + def onLost(lost: Set[TopicPartition], endedStreams: Chunk[PartitionStreamControl]): RebalanceEvent = + copy( + wasInvoked = true, + assignedTps = assignedTps -- lost, + lostTps = lostTps ++ lost, + endedStreams = this.endedStreams ++ endedStreams + ) + } + + object RebalanceEvent { + val None: RebalanceEvent = + RebalanceEvent(wasInvoked = false, Set.empty, Set.empty, Set.empty, Chunk.empty) + } +} From 1e47de2aaf5af1a2c6a36d1512aee89f34f3a629 Mon Sep 17 00:00:00 2001 From: Steven Vroonland Date: Sun, 10 Nov 2024 17:12:36 +0100 Subject: [PATCH 16/57] Fix unstable test, seems unrelated --- .../test/scala/zio/kafka/consumer/ConsumerSpec.scala | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/zio-kafka-test/src/test/scala/zio/kafka/consumer/ConsumerSpec.scala b/zio-kafka-test/src/test/scala/zio/kafka/consumer/ConsumerSpec.scala index 679a0d582..80d20d54c 100644 --- a/zio-kafka-test/src/test/scala/zio/kafka/consumer/ConsumerSpec.scala +++ b/zio-kafka-test/src/test/scala/zio/kafka/consumer/ConsumerSpec.scala @@ -28,6 +28,7 @@ import zio.test.Assertion._ import zio.test.TestAspect._ import zio.test._ +import java.util.concurrent.ExecutionException import scala.reflect.ClassTag //noinspection SimplifyAssertInspection @@ -1377,9 +1378,13 @@ object ConsumerSpec extends ZIOSpecDefaultSlf4j with KafkaRandom { }: _*) @@ TestAspect.nonFlaky(2), test("running streams don't stall after a poll timeout") { for { - topic <- randomTopic - clientId <- randomClient - _ <- ZIO.fromTry(EmbeddedKafka.createCustomTopic(topic)) + topic <- randomTopic + clientId <- randomClient + _ <- ZIO.attempt(EmbeddedKafka.createCustomTopic(topic)).flatMap(ZIO.fromTry(_)).retryN(3).catchSome { + case e: ExecutionException + if e.getCause.isInstanceOf[org.apache.kafka.common.errors.TopicExistsException] => + ZIO.unit + } settings <- consumerSettings(clientId) consumer <- Consumer.make(settings.withPollTimeout(50.millis)) recordsOut <- Queue.unbounded[Unit] From ac4e97823b427fa3819f5485815e0956407f115f Mon Sep 17 00:00:00 2001 From: Steven Vroonland Date: Sun, 10 Nov 2024 17:34:54 +0100 Subject: [PATCH 17/57] Fix timeToDeadlineMillis --- .../kafka/consumer/internal/RunloopRebalanceListener.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/RunloopRebalanceListener.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/RunloopRebalanceListener.scala index 15c354597..59d08e604 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/RunloopRebalanceListener.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/RunloopRebalanceListener.scala @@ -60,7 +60,7 @@ private[internal] class RunloopRebalanceListener( ): Task[Unit] = { val deadline = java.lang.System.nanoTime() + maxRebalanceDuration.toNanos - commitTimeoutNanos - def timeToDeadlineMillis(): Long = (java.lang.System.nanoTime() - deadline) / 1000000L + def timeToDeadlineMillis(): Long = (deadline - java.lang.System.nanoTime()) / 1000000L def completionStatusesAsString(completionStatuses: Chunk[StreamCompletionStatus]): String = "Revoked partitions: " + completionStatuses.map(_.toString).mkString("; ") @@ -91,7 +91,7 @@ private[internal] class RunloopRebalanceListener( @inline def logStreamCompletionStatuses(completionStatuses: Chunk[StreamCompletionStatus]): UIO[Unit] = { val statusStrings = completionStatusesAsString(completionStatuses) - ZIO.logInfo( + ZIO.logDebug( s"Delaying rebalance until ${streamsToEnd.size} streams (of revoked partitions) have committed " + s"the offsets of the records they consumed. Deadline in ${timeToDeadlineMillis()}ms. $statusStrings" ) From 5917c1d96df55ca1d30e3d8c341cdc1f6d502f17 Mon Sep 17 00:00:00 2001 From: Steven Vroonland Date: Sun, 10 Nov 2024 17:48:45 +0100 Subject: [PATCH 18/57] Small convenience method --- .../internal/RunloopRebalanceListener.scala | 74 ++++++++++--------- 1 file changed, 38 insertions(+), 36 deletions(-) diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/RunloopRebalanceListener.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/RunloopRebalanceListener.scala index 59d08e604..4d29ab18d 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/RunloopRebalanceListener.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/RunloopRebalanceListener.scala @@ -182,47 +182,49 @@ private[internal] class RunloopRebalanceListener( // - ends streams that need to be ended // - updates `lastRebalanceEvent` // - def toRebalanceListener: RebalanceListener = RebalanceListener( onAssigned = assignedTps => - for { - rebalanceEvent <- lastRebalanceEvent.get - _ <- ZIO.logDebug { - val sameRebalance = if (rebalanceEvent.wasInvoked) " in same rebalance" else "" - s"${assignedTps.size} partitions are assigned$sameRebalance" - } - assignedStreams <- getCurrentAssignedStreams - streamsToEnd = if (restartStreamsOnRebalancing && !rebalanceEvent.wasInvoked) assignedStreams - else Chunk.empty - _ <- endStreams(streamsToEnd) - _ <- lastRebalanceEvent.set(rebalanceEvent.onAssigned(assignedTps, endedStreams = streamsToEnd)) - _ <- ZIO.logTrace("onAssigned done") - } yield (), + withLastRebalanceEvent { rebalanceEvent => + for { + _ <- ZIO.logDebug { + val sameRebalance = if (rebalanceEvent.wasInvoked) " in same rebalance" else "" + s"${assignedTps.size} partitions are assigned$sameRebalance" + } + assignedStreams <- getCurrentAssignedStreams + streamsToEnd = if (restartStreamsOnRebalancing && !rebalanceEvent.wasInvoked) assignedStreams + else Chunk.empty + _ <- endStreams(streamsToEnd) + _ <- ZIO.logTrace("onAssigned done") + } yield rebalanceEvent.onAssigned(assignedTps, endedStreams = streamsToEnd) + }, onRevoked = revokedTps => - for { - rebalanceEvent <- lastRebalanceEvent.get - _ <- ZIO.logDebug { - val sameRebalance = if (rebalanceEvent.wasInvoked) " in same rebalance" else "" - s"${revokedTps.size} partitions are revoked$sameRebalance" - } - assignedStreams <- getCurrentAssignedStreams - streamsToEnd = if (restartStreamsOnRebalancing && !rebalanceEvent.wasInvoked) assignedStreams - else assignedStreams.filter(control => revokedTps.contains(control.tp)) - _ <- endStreams(streamsToEnd) - _ <- lastRebalanceEvent.set(rebalanceEvent.onRevoked(revokedTps, endedStreams = streamsToEnd)) - _ <- ZIO.logTrace("onRevoked done") - } yield (), + withLastRebalanceEvent { rebalanceEvent => + for { + _ <- ZIO.logDebug { + val sameRebalance = if (rebalanceEvent.wasInvoked) " in same rebalance" else "" + s"${revokedTps.size} partitions are revoked$sameRebalance" + } + assignedStreams <- getCurrentAssignedStreams + streamsToEnd = if (restartStreamsOnRebalancing && !rebalanceEvent.wasInvoked) assignedStreams + else assignedStreams.filter(control => revokedTps.contains(control.tp)) + _ <- endStreams(streamsToEnd) + _ <- ZIO.logTrace("onRevoked done") + } yield rebalanceEvent.onRevoked(revokedTps, endedStreams = streamsToEnd) + }, onLost = lostTps => - for { - _ <- ZIO.logDebug(s"${lostTps.size} partitions are lost") - rebalanceEvent <- lastRebalanceEvent.get - assignedStreams <- getCurrentAssignedStreams - lostStreams = assignedStreams.filter(control => lostTps.contains(control.tp)) - _ <- ZIO.foreachDiscard(lostStreams)(_.lost) - _ <- lastRebalanceEvent.set(rebalanceEvent.onLost(lostTps, lostStreams)) - _ <- ZIO.logTrace(s"onLost done") - } yield () + withLastRebalanceEvent { rebalanceEvent => + for { + _ <- ZIO.logDebug(s"${lostTps.size} partitions are lost") + assignedStreams <- getCurrentAssignedStreams + lostStreams = assignedStreams.filter(control => lostTps.contains(control.tp)) + _ <- ZIO.foreachDiscard(lostStreams)(_.lost) + _ <- ZIO.logTrace(s"onLost done") + } yield rebalanceEvent.onLost(lostTps, lostStreams) + } ) + + private def withLastRebalanceEvent(f: RebalanceEvent => Task[RebalanceEvent]): Task[Unit] = + lastRebalanceEvent.get.flatMap(f).flatMap(lastRebalanceEvent.set) } private[internal] object RunloopRebalanceListener { From dc614daf9b05ac075a3da8c4fb9348780c84e6ef Mon Sep 17 00:00:00 2001 From: Steven Vroonland Date: Sun, 10 Nov 2024 18:44:57 +0100 Subject: [PATCH 19/57] Adjust unrepresentative test With rebalanceSafeCommits, we expect all pulled records to be processed and committed. --- .../zio/kafka/consumer/ConsumerSpec.scala | 20 +++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/zio-kafka-test/src/test/scala/zio/kafka/consumer/ConsumerSpec.scala b/zio-kafka-test/src/test/scala/zio/kafka/consumer/ConsumerSpec.scala index 80d20d54c..a798abd41 100644 --- a/zio-kafka-test/src/test/scala/zio/kafka/consumer/ConsumerSpec.scala +++ b/zio-kafka-test/src/test/scala/zio/kafka/consumer/ConsumerSpec.scala @@ -324,21 +324,20 @@ object ConsumerSpec extends ZIOSpecDefaultSlf4j with KafkaRandom { } yield assert(offset.map(_.offset))(isSome(equalTo(9L))) }, test("process outstanding commits after a graceful shutdown with aggregateAsync using `maxRebalanceDuration`") { - val kvs = (1 to 100).toList.map(i => (s"key$i", s"msg$i")) for { - topic <- randomTopic - group <- randomGroup - client <- randomClient - _ <- produceMany(topic, kvs) - messagesReceived <- Ref.make[Int](0) + topic <- randomTopic + group <- randomGroup + client <- randomClient + _ <- scheduledProduce(topic, Schedule.fixed(50.millis).jittered).runDrain.forkScoped + lastProcessedOffset <- Ref.make(0L) offset <- ( Consumer .plainStream(Subscription.topics(topic), Serde.string, Serde.string) - .mapConcatZIO { record => + .mapZIO { record => for { - nr <- messagesReceived.updateAndGet(_ + 1) + nr <- lastProcessedOffset.updateAndGet(_ + 1) _ <- Consumer.stopConsumption.when(nr == 10) - } yield if (nr < 10) Seq(record.offset) else Seq.empty + } yield record.offset } .aggregateAsync(Consumer.offsetBatches) .mapZIO(_.commit) @@ -354,7 +353,8 @@ object ConsumerSpec extends ZIOSpecDefaultSlf4j with KafkaRandom { maxRebalanceDuration = 6.seconds ) ) - } yield assertTrue(offset.map(_.offset).contains(9L)) + lastOffset <- lastProcessedOffset.get + } yield assertTrue(offset.map(_.offset).contains(lastOffset)) } @@ TestAspect.nonFlaky(2), test("a consumer timeout interrupts the stream and shuts down the consumer") { // Setup of this test: From 02532da29aab41d16ff67dde2c14a0d89f632768 Mon Sep 17 00:00:00 2001 From: Steven Vroonland Date: Sun, 10 Nov 2024 18:45:03 +0100 Subject: [PATCH 20/57] Document parameter --- .../main/scala/zio/kafka/consumer/internal/Committer.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Committer.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Committer.scala index dc12c5782..e4fb56fa5 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Committer.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Committer.scala @@ -49,6 +49,8 @@ private[consumer] final class Committer( * * @param consumer * Consumer with exclusive access + * @param executeOnEmpty + * Execute commitAsync() even if there are no commits */ def handleNewCommits(consumer: ByteArrayKafkaConsumer, executeOnEmpty: Boolean = false): Task[Unit] = for { commits <- commitQueue.takeAll @@ -100,7 +102,7 @@ private[consumer] final class Committer( for { _ <- ZIO.logDebug(s"Rebalance in progress, commit for offsets $offsets will be retried") _ <- commitQueue.offerAll(commits) - _ <- onCommitAvailable // + _ <- onCommitAvailable } yield () case err: Throwable => cont(Exit.fail(err)) <* diagnostics.emit(DiagnosticEvent.Commit.Failure(offsetsWithMetaData, err)) From 6ad6eaf722a83795508dd6abe6397d3b5eae404e Mon Sep 17 00:00:00 2001 From: Steven Vroonland Date: Sun, 10 Nov 2024 18:52:06 +0100 Subject: [PATCH 21/57] Inline RebalanceEvent modifications Helps understanding the logic here --- .../internal/RunloopRebalanceListener.scala | 51 +++++++------------ 1 file changed, 18 insertions(+), 33 deletions(-) diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/RunloopRebalanceListener.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/RunloopRebalanceListener.scala index 4d29ab18d..d370bf1c1 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/RunloopRebalanceListener.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/RunloopRebalanceListener.scala @@ -195,7 +195,11 @@ private[internal] class RunloopRebalanceListener( else Chunk.empty _ <- endStreams(streamsToEnd) _ <- ZIO.logTrace("onAssigned done") - } yield rebalanceEvent.onAssigned(assignedTps, endedStreams = streamsToEnd) + } yield rebalanceEvent.copy( + wasInvoked = true, + assignedTps = rebalanceEvent.assignedTps ++ assignedTps, + endedStreams = rebalanceEvent.endedStreams ++ streamsToEnd + ) }, onRevoked = revokedTps => withLastRebalanceEvent { rebalanceEvent => @@ -209,7 +213,12 @@ private[internal] class RunloopRebalanceListener( else assignedStreams.filter(control => revokedTps.contains(control.tp)) _ <- endStreams(streamsToEnd) _ <- ZIO.logTrace("onRevoked done") - } yield rebalanceEvent.onRevoked(revokedTps, endedStreams = streamsToEnd) + } yield rebalanceEvent.copy( + wasInvoked = true, + assignedTps = rebalanceEvent.assignedTps -- revokedTps, + revokedTps = rebalanceEvent.revokedTps ++ revokedTps, + endedStreams = rebalanceEvent.endedStreams ++ streamsToEnd + ) }, onLost = lostTps => withLastRebalanceEvent { rebalanceEvent => @@ -219,7 +228,12 @@ private[internal] class RunloopRebalanceListener( lostStreams = assignedStreams.filter(control => lostTps.contains(control.tp)) _ <- ZIO.foreachDiscard(lostStreams)(_.lost) _ <- ZIO.logTrace(s"onLost done") - } yield rebalanceEvent.onLost(lostTps, lostStreams) + } yield rebalanceEvent.copy( + wasInvoked = true, + assignedTps = rebalanceEvent.assignedTps -- lostTps, + lostTps = rebalanceEvent.lostTps ++ lostTps, + endedStreams = rebalanceEvent.endedStreams ++ lostStreams + ) } ) @@ -253,36 +267,7 @@ private[internal] object RunloopRebalanceListener { revokedTps: Set[TopicPartition], lostTps: Set[TopicPartition], endedStreams: Chunk[PartitionStreamControl] - ) { - def onAssigned( - assigned: Set[TopicPartition], - endedStreams: Chunk[PartitionStreamControl] - ): RebalanceEvent = - copy( - wasInvoked = true, - assignedTps = assignedTps ++ assigned, - endedStreams = this.endedStreams ++ endedStreams - ) - - def onRevoked( - revoked: Set[TopicPartition], - endedStreams: Chunk[PartitionStreamControl] - ): RebalanceEvent = - copy( - wasInvoked = true, - assignedTps = assignedTps -- revoked, - revokedTps = revokedTps ++ revoked, - endedStreams = this.endedStreams ++ endedStreams - ) - - def onLost(lost: Set[TopicPartition], endedStreams: Chunk[PartitionStreamControl]): RebalanceEvent = - copy( - wasInvoked = true, - assignedTps = assignedTps -- lost, - lostTps = lostTps ++ lost, - endedStreams = this.endedStreams ++ endedStreams - ) - } + ) object RebalanceEvent { val None: RebalanceEvent = From 9c36a5decdc8a1f15ebaac6c2700e09b2005f57b Mon Sep 17 00:00:00 2001 From: Steven Vroonland Date: Sun, 10 Nov 2024 18:57:50 +0100 Subject: [PATCH 22/57] Use startTime in Commit --- .../src/main/scala/zio/kafka/consumer/internal/Committer.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Committer.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Committer.scala index e4fb56fa5..8e869c1de 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Committer.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Committer.scala @@ -33,7 +33,7 @@ private[consumer] final class Committer( for { p <- Promise.make[Throwable, Unit] startTime = java.lang.System.nanoTime() - _ <- commitQueue.offer(Commit(java.lang.System.nanoTime(), offsets, p)) + _ <- commitQueue.offer(Commit(startTime, offsets, p)) _ <- onCommitAvailable _ <- diagnostics.emit(DiagnosticEvent.Commit.Started(offsets)) _ <- p.await.timeoutFail(CommitTimeout)(commitTimeout) From d1957d861bf0f36cdf263abf3453cb4c2623e210 Mon Sep 17 00:00:00 2001 From: Steven Vroonland Date: Tue, 12 Nov 2024 18:16:13 +0100 Subject: [PATCH 23/57] Do not depend on entire ConsumerSettings in Committer --- .../kafka/consumer/internal/Committer.scala | 18 ++++++++---------- .../zio/kafka/consumer/internal/Runloop.scala | 2 +- 2 files changed, 9 insertions(+), 11 deletions(-) diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Committer.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Committer.scala index 8e869c1de..91ff0523a 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Committer.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Committer.scala @@ -1,23 +1,22 @@ package zio.kafka.consumer.internal -import org.apache.kafka.clients.consumer.{ OffsetAndMetadata, OffsetCommitCallback } +import org.apache.kafka.clients.consumer.{OffsetAndMetadata, OffsetCommitCallback} import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.errors.RebalanceInProgressException import zio.kafka.consumer.Consumer.CommitTimeout -import zio.kafka.consumer.ConsumerSettings -import zio.kafka.consumer.diagnostics.{ DiagnosticEvent, Diagnostics } -import zio.kafka.consumer.internal.Committer.{ Commit, CommitOffsets } +import zio.kafka.consumer.diagnostics.{DiagnosticEvent, Diagnostics} +import zio.kafka.consumer.internal.Committer.{Commit, CommitOffsets} import zio.kafka.consumer.internal.ConsumerAccess.ByteArrayKafkaConsumer -import zio.{ durationLong, Chunk, Exit, Promise, Queue, Ref, Runtime, Scope, Task, UIO, Unsafe, ZIO } +import zio.{Chunk, Duration, Exit, Promise, Queue, Ref, Runtime, Scope, Task, UIO, Unsafe, ZIO, durationLong} import java.lang.Math.max import java.util -import java.util.{ Map => JavaMap } +import java.util.{Map => JavaMap} import scala.collection.mutable import scala.jdk.CollectionConverters._ private[consumer] final class Committer( commitQueue: Queue[Commit], - settings: ConsumerSettings, + commitTimeout: Duration, diagnostics: Diagnostics, consumerMetrics: ConsumerMetrics, onCommitAvailable: UIO[Unit], @@ -25,7 +24,6 @@ private[consumer] final class Committer( sameThreadRuntime: Runtime[Any], pendingCommits: Ref[Chunk[Commit]] // TODO make Commit internal ) { - private val commitTimeout = settings.commitTimeout /** This is the implementation behind the user facing api `Offset.commit`. */ val commit: Map[TopicPartition, OffsetAndMetadata] => Task[Unit] = @@ -138,7 +136,7 @@ private[consumer] final class Committer( private[internal] object Committer { def make( - settings: ConsumerSettings, + commitTimeout: Duration, diagnostics: Diagnostics, consumerMetrics: ConsumerMetrics, onCommitAvailable: UIO[Unit], @@ -149,7 +147,7 @@ private[internal] object Committer { committedOffsetsRef <- Ref.make(CommitOffsets.empty) } yield new Committer( commitQueue, - settings, + commitTimeout, diagnostics, consumerMetrics, onCommitAvailable, diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala index 8bdd91aac..758114d5e 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala @@ -577,7 +577,7 @@ object Runloop { executor <- ZIO.executor metrics = new ZioConsumerMetrics(settings.metricLabels) committer <- Committer.make( - settings, + settings.commitTimeout, diagnostics, metrics, commandQueue.offer(RunloopCommand.CommitAvailable).unit, From 807fdf9516ddb95192e54bf6d0dc571bb7fff9cb Mon Sep 17 00:00:00 2001 From: Steven Vroonland Date: Tue, 12 Nov 2024 18:38:51 +0100 Subject: [PATCH 24/57] Make Committer better testable --- .../kafka/consumer/internal/Committer.scala | 29 ++++++++++--------- .../zio/kafka/consumer/internal/Runloop.scala | 4 ++- .../internal/RunloopRebalanceListener.scala | 7 ++++- 3 files changed, 24 insertions(+), 16 deletions(-) diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Committer.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Committer.scala index 91ff0523a..a8e7a5a7f 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Committer.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Committer.scala @@ -1,16 +1,15 @@ package zio.kafka.consumer.internal -import org.apache.kafka.clients.consumer.{OffsetAndMetadata, OffsetCommitCallback} +import org.apache.kafka.clients.consumer.{ OffsetAndMetadata, OffsetCommitCallback } import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.errors.RebalanceInProgressException import zio.kafka.consumer.Consumer.CommitTimeout -import zio.kafka.consumer.diagnostics.{DiagnosticEvent, Diagnostics} -import zio.kafka.consumer.internal.Committer.{Commit, CommitOffsets} -import zio.kafka.consumer.internal.ConsumerAccess.ByteArrayKafkaConsumer -import zio.{Chunk, Duration, Exit, Promise, Queue, Ref, Runtime, Scope, Task, UIO, Unsafe, ZIO, durationLong} +import zio.kafka.consumer.diagnostics.{ DiagnosticEvent, Diagnostics } +import zio.kafka.consumer.internal.Committer.{ Commit, CommitOffsets } +import zio.{ durationLong, Chunk, Duration, Exit, Promise, Queue, Ref, Runtime, Scope, Task, UIO, Unsafe, ZIO } import java.lang.Math.max import java.util -import java.util.{Map => JavaMap} +import java.util.{ Map => JavaMap } import scala.collection.mutable import scala.jdk.CollectionConverters._ @@ -22,7 +21,7 @@ private[consumer] final class Committer( onCommitAvailable: UIO[Unit], committedOffsetsRef: Ref[CommitOffsets], sameThreadRuntime: Runtime[Any], - pendingCommits: Ref[Chunk[Commit]] // TODO make Commit internal + pendingCommits: Ref[Chunk[Commit]] ) { /** This is the implementation behind the user facing api `Offset.commit`. */ @@ -50,7 +49,10 @@ private[consumer] final class Committer( * @param executeOnEmpty * Execute commitAsync() even if there are no commits */ - def handleNewCommits(consumer: ByteArrayKafkaConsumer, executeOnEmpty: Boolean = false): Task[Unit] = for { + def handleNewCommits( + commitAsync: (JavaMap[TopicPartition, OffsetAndMetadata], OffsetCommitCallback) => Task[Unit], + executeOnEmpty: Boolean = false + ): Task[Unit] = for { commits <- commitQueue.takeAll _ <- ZIO.logDebug(s"Processing ${commits.size} commits") _ <- ZIO.unless(commits.isEmpty && !executeOnEmpty) { @@ -58,8 +60,7 @@ private[consumer] final class Committer( pendingCommits.update(_ ++ commits) *> // We don't wait for the completion of the commit here, because it // will only complete once we poll again. - ZIO - .attempt(consumer.commitAsync(offsets, callback)) + commitAsync(offsets, callback) .catchAll(onFailure) } } yield () @@ -118,9 +119,12 @@ private[consumer] final class Committer( (offsetsWithMetaData.asJava, callback, onFailure) } + def queueSize: UIO[Int] = commitQueue.size + def pendingCommitCount: UIO[Int] = pendingCommits.get.map(_.size) - def queueSize: UIO[Int] = commitQueue.size + def getPendingCommits: UIO[CommitOffsets] = + pendingCommits.get.map(CommitOffsets.empty.addCommits(_)._2) def updatePendingCommitsAfterPoll: UIO[Unit] = pendingCommits.get.flatMap(ZIO.filter(_)(_.isPending)).flatMap(pendingCommits.set) @@ -129,9 +133,6 @@ private[consumer] final class Committer( committedOffsetsRef.update(_.keepPartitions(assignedPartitions)) def getCommittedOffsets: UIO[CommitOffsets] = committedOffsetsRef.get - - def getPendingCommits: UIO[CommitOffsets] = - pendingCommits.get.map(CommitOffsets.empty.addCommits(_)._2) } private[internal] object Committer { diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala index 758114d5e..25230e0a2 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala @@ -483,7 +483,9 @@ private[consumer] final class Runloop private ( .runFoldChunksDiscardZIO(initialState) { (state, commands) => for { _ <- ZIO.logDebug(s"Processing ${commands.size} commands: ${commands.mkString(",")}") - _ <- consumer.runloopAccess(committer.handleNewCommits(_)) + _ <- consumer.runloopAccess { consumer => + committer.handleNewCommits((offsets, callback) => ZIO.attempt(consumer.commitAsync(offsets, callback))) + } streamCommands = commands.collect { case cmd: RunloopCommand.StreamCommand => cmd } stateAfterCommands <- ZIO.foldLeft(streamCommands)(state)(handleCommand) diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/RunloopRebalanceListener.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/RunloopRebalanceListener.scala index d370bf1c1..a67fcb3ea 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/RunloopRebalanceListener.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/RunloopRebalanceListener.scala @@ -156,7 +156,12 @@ private[internal] class RunloopRebalanceListener( // Even if there is nothing to commit, continue to drive communication with the broker // so that commits can complete and the streams can make progress, by setting // executeOnEmpty = true - .tap(_ => committer.handleNewCommits(consumer, executeOnEmpty = true)) + .tap(_ => + committer.handleNewCommits( + (offsets, callback) => ZIO.attempt(consumer.commitAsync(offsets, callback)), + executeOnEmpty = true + ) + ) .takeWhile(_ => java.lang.System.nanoTime() <= deadline) .mapZIO(_ => endingStreamsCompletedAndCommitsExist) .takeUntil(completed => completed) From 98a10317b3d8bcbb32630dd193da3967330eb787 Mon Sep 17 00:00:00 2001 From: Steven Vroonland Date: Tue, 12 Nov 2024 20:57:43 +0100 Subject: [PATCH 25/57] Tests for Committer --- .../consumer/internal/CommitterSpec.scala | 233 ++++++++++++++++++ 1 file changed, 233 insertions(+) create mode 100644 zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/CommitterSpec.scala diff --git a/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/CommitterSpec.scala b/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/CommitterSpec.scala new file mode 100644 index 000000000..8d6293b4d --- /dev/null +++ b/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/CommitterSpec.scala @@ -0,0 +1,233 @@ +package zio.kafka.consumer.internal + +import org.apache.kafka.clients.consumer.OffsetAndMetadata +import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.errors.RebalanceInProgressException +import zio.kafka.consumer.diagnostics.Diagnostics +import zio.test._ +import zio.{ durationInt, Promise, UIO, ZIO } + +import java.util.{ Map => JavaMap } +import scala.jdk.CollectionConverters.MapHasAsJava + +/** + * Committer should: + * - execute a successful commit (+ 1) + * - communicate back a failed commit + * - automatically retry a commit when rebalancing + * - batch commits from multiple partitions and multiple offsets and confirm each one of them + * - keep track of pending commits (clear before and after) + * - keep track of committed offsets + * - clean committed offsets of no-longer assigned partitions + * - publish commit metrics + */ +object CommitterSpec extends ZIOSpecDefault { + val mockMetrics = new ConsumerMetrics { + override def observePoll(resumedCount: Int, pausedCount: Int, latency: zio.Duration, pollSize: Int): UIO[Unit] = + ZIO.unit + + override def observeCommit(latency: zio.Duration): UIO[Unit] = ZIO.unit + override def observeAggregatedCommit(latency: zio.Duration, commitSize: NanoTime): UIO[Unit] = ZIO.unit + override def observeRebalance( + currentlyAssignedCount: Int, + assignedCount: Int, + revokedCount: Int, + lostCount: Int + ): UIO[Unit] = ZIO.unit + override def observeRunloopMetrics( + state: Runloop.State, + commandQueueSize: Int, + commitQueueSize: Int, + pendingCommits: Int + ): UIO[Unit] = ZIO.unit + override def observePollAuthError(): UIO[Unit] = ZIO.unit + } + + override def spec = suite("Committer")( + test("signals that a new commit is available") { + for { + runtime <- ZIO.runtime[Any] + commitAvailable <- Promise.make[Nothing, Unit] + committer <- Committer.make( + 10.seconds, + Diagnostics.NoOp, + mockMetrics, + onCommitAvailable = commitAvailable.succeed(()).unit, + sameThreadRuntime = runtime + ) + tp = new TopicPartition("topic", 0) + _ <- committer.commit(Map(tp -> new OffsetAndMetadata(0))).forkScoped + _ <- commitAvailable.await + } yield assertCompletes + }, + test("handles a successful commit by completing the commit effect") { + for { + runtime <- ZIO.runtime[Any] + commitAvailable <- Promise.make[Nothing, Unit] + committer <- Committer.make( + 10.seconds, + Diagnostics.NoOp, + mockMetrics, + onCommitAvailable = commitAvailable.succeed(()).unit, + sameThreadRuntime = runtime + ) + tp = new TopicPartition("topic", 0) + commitFiber <- committer.commit(Map(tp -> new OffsetAndMetadata(0))).forkScoped + _ <- commitAvailable.await + _ <- committer.handleNewCommits((offsets, callback) => ZIO.attempt(callback.onComplete(offsets, null))) + _ <- commitFiber.join + } yield assertCompletes + }, + test("handles a failed commit by completing the commit effect with a failure") { + for { + runtime <- ZIO.runtime[Any] + commitAvailable <- Promise.make[Nothing, Unit] + committer <- Committer.make( + 10.seconds, + Diagnostics.NoOp, + mockMetrics, + onCommitAvailable = commitAvailable.succeed(()).unit, + sameThreadRuntime = runtime + ) + tp = new TopicPartition("topic", 0) + commitFiber <- committer.commit(Map(tp -> new OffsetAndMetadata(0))).forkScoped + _ <- commitAvailable.await + _ <- committer.handleNewCommits((offsets, callback) => + ZIO.attempt(callback.onComplete(offsets, new RuntimeException("Commit failed"))) + ) + result <- commitFiber.await + } yield assertTrue(result.isFailure) + }, + test("retries when rebalancing") { + for { + runtime <- ZIO.runtime[Any] + commitAvailable <- Promise.make[Nothing, Unit] + committer <- Committer.make( + 10.seconds, + Diagnostics.NoOp, + mockMetrics, + onCommitAvailable = commitAvailable.succeed(()).unit, + sameThreadRuntime = runtime + ) + tp = new TopicPartition("topic", 0) + commitFiber <- committer.commit(Map(tp -> new OffsetAndMetadata(0))).forkScoped + _ <- commitAvailable.await + _ <- committer.handleNewCommits((offsets, callback) => + ZIO.attempt(callback.onComplete(offsets, new RebalanceInProgressException("Rebalance in progress"))) + ) + _ <- committer.handleNewCommits((offsets, callback) => ZIO.attempt(callback.onComplete(offsets, null))) + result <- commitFiber.await + } yield assertTrue(result.isSuccess) + }, + test("adds 1 to the committed last offset") { + for { + runtime <- ZIO.runtime[Any] + commitAvailable <- Promise.make[Nothing, Unit] + committer <- Committer.make( + 10.seconds, + Diagnostics.NoOp, + mockMetrics, + onCommitAvailable = commitAvailable.succeed(()).unit, + sameThreadRuntime = runtime + ) + tp = new TopicPartition("topic", 0) + _ <- committer.commit(Map(tp -> new OffsetAndMetadata(1))).forkScoped + _ <- commitAvailable.await + committedOffsets <- Promise.make[Nothing, JavaMap[TopicPartition, OffsetAndMetadata]] + _ <- committer.handleNewCommits((offsets, callback) => + committedOffsets.succeed(offsets) *> ZIO.attempt(callback.onComplete(offsets, null)) + ) + offsetsCommitted <- committedOffsets.await + } yield assertTrue( + offsetsCommitted == Map(tp -> new OffsetAndMetadata(2)).asJava + ) + }, + test("batches commits from multiple partitions and offsets") { + for { + runtime <- ZIO.runtime[Any] + commitAvailable <- Promise.make[Nothing, Unit] + committer <- Committer.make( + 10.seconds, + Diagnostics.NoOp, + mockMetrics, + onCommitAvailable = commitAvailable.succeed(()).unit, + sameThreadRuntime = runtime + ) + tp = new TopicPartition("topic", 0) + tp2 = new TopicPartition("topic", 1) + commitFiber1 <- committer.commit(Map(tp -> new OffsetAndMetadata(1))).forkScoped + commitFiber2 <- committer.commit(Map(tp -> new OffsetAndMetadata(2))).forkScoped + commitFiber3 <- committer.commit(Map(tp2 -> new OffsetAndMetadata(3))).forkScoped + _ <- commitAvailable.await + committedOffsets <- Promise.make[Nothing, JavaMap[TopicPartition, OffsetAndMetadata]] + _ <- committer.handleNewCommits((offsets, callback) => + committedOffsets.succeed(offsets) *> ZIO.attempt(callback.onComplete(offsets, null)) + ) + _ <- commitFiber1.join zip commitFiber2.join zip commitFiber3.join + offsetsCommitted <- committedOffsets.await + } yield assertTrue( + offsetsCommitted == Map(tp -> new OffsetAndMetadata(3), tp2 -> new OffsetAndMetadata(4)).asJava + ) + }, + test("keeps track of pending commits") { + for { + runtime <- ZIO.runtime[Any] + commitAvailable <- Promise.make[Nothing, Unit] + committer <- Committer.make( + 10.seconds, + Diagnostics.NoOp, + mockMetrics, + onCommitAvailable = commitAvailable.succeed(()).unit, + sameThreadRuntime = runtime + ) + tp = new TopicPartition("topic", 0) + commitFiber <- committer.commit(Map(tp -> new OffsetAndMetadata(0))).forkScoped + _ <- commitAvailable.await + _ <- committer.handleNewCommits((offsets, callback) => ZIO.attempt(callback.onComplete(offsets, null))) + pendingCommitsDuringCommit <- committer.pendingCommitCount + _ <- committer.updatePendingCommitsAfterPoll + pendingCommitsAfterCommit <- committer.pendingCommitCount + _ <- commitFiber.join + } yield assertTrue(pendingCommitsDuringCommit == 1 && pendingCommitsAfterCommit == 0) + }, + test("keep track of committed offsets") { + for { + runtime <- ZIO.runtime[Any] + commitAvailable <- Promise.make[Nothing, Unit] + committer <- Committer.make( + 10.seconds, + Diagnostics.NoOp, + mockMetrics, + onCommitAvailable = commitAvailable.succeed(()).unit, + sameThreadRuntime = runtime + ) + tp = new TopicPartition("topic", 0) + commitFiber <- committer.commit(Map(tp -> new OffsetAndMetadata(0))).forkScoped + _ <- commitAvailable.await + _ <- committer.handleNewCommits((offsets, callback) => ZIO.attempt(callback.onComplete(offsets, null))) + committedOffsets <- committer.getCommittedOffsets + _ <- commitFiber.join + } yield assertTrue(committedOffsets.offsets == Map(tp -> 0L)) + }, + test("clean committed offsets of no-longer assigned partitions") { + for { + runtime <- ZIO.runtime[Any] + commitAvailable <- Promise.make[Nothing, Unit] + committer <- Committer.make( + 10.seconds, + Diagnostics.NoOp, + mockMetrics, + onCommitAvailable = commitAvailable.succeed(()).unit, + sameThreadRuntime = runtime + ) + tp = new TopicPartition("topic", 0) + commitFiber <- committer.commit(Map(tp -> new OffsetAndMetadata(0))).forkScoped + _ <- commitAvailable.await + _ <- committer.handleNewCommits((offsets, callback) => ZIO.attempt(callback.onComplete(offsets, null))) + _ <- committer.pruneCommittedOffsets(Set.empty) + committedOffsets <- committer.getCommittedOffsets + _ <- commitFiber.join + } yield assertTrue(committedOffsets.offsets.isEmpty) + } + ) @@ TestAspect.withLiveClock +} From 810362b63d0555ed9c826e946a5f61974025b882 Mon Sep 17 00:00:00 2001 From: Steven Vroonland Date: Tue, 12 Nov 2024 21:28:00 +0100 Subject: [PATCH 26/57] Let RunloopRebalanceListener own its last event --- .../main/scala/zio/kafka/consumer/internal/Runloop.scala | 6 ++---- .../kafka/consumer/internal/RunloopRebalanceListener.scala | 5 ++++- 2 files changed, 6 insertions(+), 5 deletions(-) diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala index 25230e0a2..7a8738f08 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala @@ -24,7 +24,6 @@ private[consumer] final class Runloop private ( consumer: ConsumerAccess, maxPollInterval: Duration, commandQueue: Queue[RunloopCommand], - lastRebalanceEvent: Ref.Synchronized[RebalanceEvent], partitionsHub: Hub[Take[Throwable, PartitionAssignment]], diagnostics: Diagnostics, currentStateRef: Ref[State], @@ -234,7 +233,7 @@ private[consumer] final class Runloop private ( tpWithoutData = requestedPartitions -- providedTps ) } - pollresult <- lastRebalanceEvent.getAndSet(RebalanceEvent.None).flatMap { + pollresult <- runloopRebalanceListener.getAndResetLastEvent.flatMap { case RebalanceEvent(false, _, _, _, _) => // The fast track, rebalance listener was not invoked: // no assignment changes, no new commits, only new records. @@ -572,7 +571,7 @@ object Runloop { for { _ <- ZIO.addFinalizer(diagnostics.emit(Finalization.RunloopFinalized)) commandQueue <- ZIO.acquireRelease(Queue.unbounded[RunloopCommand])(_.shutdown) - lastRebalanceEvent <- Ref.Synchronized.make[RebalanceEvent](RebalanceEvent.None) + lastRebalanceEvent <- Ref.make[RebalanceEvent](RebalanceEvent.None) initialState = State.initial currentStateRef <- Ref.make(initialState) sameThreadRuntime <- ZIO.runtime[Any].provideLayer(SameThreadRuntimeLayer) @@ -600,7 +599,6 @@ object Runloop { consumer = consumer, maxPollInterval = maxPollInterval, commandQueue = commandQueue, - lastRebalanceEvent = lastRebalanceEvent, partitionsHub = partitionsHub, diagnostics = diagnostics, currentStateRef = currentStateRef, diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/RunloopRebalanceListener.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/RunloopRebalanceListener.scala index a67fcb3ea..5c9384441 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/RunloopRebalanceListener.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/RunloopRebalanceListener.scala @@ -21,7 +21,7 @@ import zio.{ durationInt, Chunk, Duration, Ref, Task, UIO, ZIO } * continuing. */ private[internal] class RunloopRebalanceListener( - lastRebalanceEvent: Ref.Synchronized[RebalanceEvent], + lastRebalanceEvent: Ref[RebalanceEvent], settings: ConsumerSettings, consumer: ConsumerAccess, maxRebalanceDuration: Duration, @@ -44,6 +44,9 @@ private[internal] class RunloopRebalanceListener( // Time between polling the commit queue from the rebalance listener when `rebalanceSafeCommits` is enabled. private val commitQueuePollInterval = 100.millis + def getAndResetLastEvent: UIO[RebalanceEvent] = + lastRebalanceEvent.getAndSet(RebalanceEvent.None) + // End streams from the rebalance listener. // When `rebalanceSafeCommits` is enabled, wait for consumed offsets to be committed. private def endStreams(streamsToEnd: Chunk[PartitionStreamControl]): Task[Any] = From 2310ea1a29264abb3adf1056f46e4d8596a9c516 Mon Sep 17 00:00:00 2001 From: Steven Vroonland Date: Wed, 13 Nov 2024 09:44:20 +0100 Subject: [PATCH 27/57] Extract Committer trait --- .../consumer/internal/CommitOffsetsSpec.scala | 2 +- .../consumer/internal/CommitterSpec.scala | 49 +++-- .../kafka/consumer/internal/Committer.scala | 195 ++-------------- .../consumer/internal/LiveCommitter.scala | 208 ++++++++++++++++++ .../zio/kafka/consumer/internal/Runloop.scala | 17 +- .../internal/RunloopRebalanceListener.scala | 2 +- 6 files changed, 257 insertions(+), 216 deletions(-) create mode 100644 zio-kafka/src/main/scala/zio/kafka/consumer/internal/LiveCommitter.scala diff --git a/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/CommitOffsetsSpec.scala b/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/CommitOffsetsSpec.scala index ab9a2aca8..b1e6ee9bb 100644 --- a/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/CommitOffsetsSpec.scala +++ b/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/CommitOffsetsSpec.scala @@ -3,7 +3,7 @@ package zio.kafka.consumer.internal import org.apache.kafka.clients.consumer.OffsetAndMetadata import org.apache.kafka.common.TopicPartition import zio._ -import zio.kafka.consumer.internal.Committer.{ Commit, CommitOffsets } +import zio.kafka.consumer.internal.LiveCommitter.{ Commit, CommitOffsets } import zio.test._ object CommitOffsetsSpec extends ZIOSpecDefault { diff --git a/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/CommitterSpec.scala b/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/CommitterSpec.scala index 8d6293b4d..df24126cc 100644 --- a/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/CommitterSpec.scala +++ b/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/CommitterSpec.scala @@ -48,13 +48,14 @@ object CommitterSpec extends ZIOSpecDefault { for { runtime <- ZIO.runtime[Any] commitAvailable <- Promise.make[Nothing, Unit] - committer <- Committer.make( - 10.seconds, - Diagnostics.NoOp, - mockMetrics, - onCommitAvailable = commitAvailable.succeed(()).unit, - sameThreadRuntime = runtime - ) + committer <- LiveCommitter + .make( + 10.seconds, + Diagnostics.NoOp, + mockMetrics, + onCommitAvailable = commitAvailable.succeed(()).unit, + sameThreadRuntime = runtime + ) tp = new TopicPartition("topic", 0) _ <- committer.commit(Map(tp -> new OffsetAndMetadata(0))).forkScoped _ <- commitAvailable.await @@ -64,7 +65,7 @@ object CommitterSpec extends ZIOSpecDefault { for { runtime <- ZIO.runtime[Any] commitAvailable <- Promise.make[Nothing, Unit] - committer <- Committer.make( + committer <- LiveCommitter.make( 10.seconds, Diagnostics.NoOp, mockMetrics, @@ -74,7 +75,7 @@ object CommitterSpec extends ZIOSpecDefault { tp = new TopicPartition("topic", 0) commitFiber <- committer.commit(Map(tp -> new OffsetAndMetadata(0))).forkScoped _ <- commitAvailable.await - _ <- committer.handleNewCommits((offsets, callback) => ZIO.attempt(callback.onComplete(offsets, null))) + _ <- committer.processQueuedCommits((offsets, callback) => ZIO.attempt(callback.onComplete(offsets, null))) _ <- commitFiber.join } yield assertCompletes }, @@ -82,7 +83,7 @@ object CommitterSpec extends ZIOSpecDefault { for { runtime <- ZIO.runtime[Any] commitAvailable <- Promise.make[Nothing, Unit] - committer <- Committer.make( + committer <- LiveCommitter.make( 10.seconds, Diagnostics.NoOp, mockMetrics, @@ -92,7 +93,7 @@ object CommitterSpec extends ZIOSpecDefault { tp = new TopicPartition("topic", 0) commitFiber <- committer.commit(Map(tp -> new OffsetAndMetadata(0))).forkScoped _ <- commitAvailable.await - _ <- committer.handleNewCommits((offsets, callback) => + _ <- committer.processQueuedCommits((offsets, callback) => ZIO.attempt(callback.onComplete(offsets, new RuntimeException("Commit failed"))) ) result <- commitFiber.await @@ -102,7 +103,7 @@ object CommitterSpec extends ZIOSpecDefault { for { runtime <- ZIO.runtime[Any] commitAvailable <- Promise.make[Nothing, Unit] - committer <- Committer.make( + committer <- LiveCommitter.make( 10.seconds, Diagnostics.NoOp, mockMetrics, @@ -112,10 +113,10 @@ object CommitterSpec extends ZIOSpecDefault { tp = new TopicPartition("topic", 0) commitFiber <- committer.commit(Map(tp -> new OffsetAndMetadata(0))).forkScoped _ <- commitAvailable.await - _ <- committer.handleNewCommits((offsets, callback) => + _ <- committer.processQueuedCommits((offsets, callback) => ZIO.attempt(callback.onComplete(offsets, new RebalanceInProgressException("Rebalance in progress"))) ) - _ <- committer.handleNewCommits((offsets, callback) => ZIO.attempt(callback.onComplete(offsets, null))) + _ <- committer.processQueuedCommits((offsets, callback) => ZIO.attempt(callback.onComplete(offsets, null))) result <- commitFiber.await } yield assertTrue(result.isSuccess) }, @@ -123,7 +124,7 @@ object CommitterSpec extends ZIOSpecDefault { for { runtime <- ZIO.runtime[Any] commitAvailable <- Promise.make[Nothing, Unit] - committer <- Committer.make( + committer <- LiveCommitter.make( 10.seconds, Diagnostics.NoOp, mockMetrics, @@ -134,7 +135,7 @@ object CommitterSpec extends ZIOSpecDefault { _ <- committer.commit(Map(tp -> new OffsetAndMetadata(1))).forkScoped _ <- commitAvailable.await committedOffsets <- Promise.make[Nothing, JavaMap[TopicPartition, OffsetAndMetadata]] - _ <- committer.handleNewCommits((offsets, callback) => + _ <- committer.processQueuedCommits((offsets, callback) => committedOffsets.succeed(offsets) *> ZIO.attempt(callback.onComplete(offsets, null)) ) offsetsCommitted <- committedOffsets.await @@ -146,7 +147,7 @@ object CommitterSpec extends ZIOSpecDefault { for { runtime <- ZIO.runtime[Any] commitAvailable <- Promise.make[Nothing, Unit] - committer <- Committer.make( + committer <- LiveCommitter.make( 10.seconds, Diagnostics.NoOp, mockMetrics, @@ -160,7 +161,7 @@ object CommitterSpec extends ZIOSpecDefault { commitFiber3 <- committer.commit(Map(tp2 -> new OffsetAndMetadata(3))).forkScoped _ <- commitAvailable.await committedOffsets <- Promise.make[Nothing, JavaMap[TopicPartition, OffsetAndMetadata]] - _ <- committer.handleNewCommits((offsets, callback) => + _ <- committer.processQueuedCommits((offsets, callback) => committedOffsets.succeed(offsets) *> ZIO.attempt(callback.onComplete(offsets, null)) ) _ <- commitFiber1.join zip commitFiber2.join zip commitFiber3.join @@ -173,7 +174,7 @@ object CommitterSpec extends ZIOSpecDefault { for { runtime <- ZIO.runtime[Any] commitAvailable <- Promise.make[Nothing, Unit] - committer <- Committer.make( + committer <- LiveCommitter.make( 10.seconds, Diagnostics.NoOp, mockMetrics, @@ -183,7 +184,7 @@ object CommitterSpec extends ZIOSpecDefault { tp = new TopicPartition("topic", 0) commitFiber <- committer.commit(Map(tp -> new OffsetAndMetadata(0))).forkScoped _ <- commitAvailable.await - _ <- committer.handleNewCommits((offsets, callback) => ZIO.attempt(callback.onComplete(offsets, null))) + _ <- committer.processQueuedCommits((offsets, callback) => ZIO.attempt(callback.onComplete(offsets, null))) pendingCommitsDuringCommit <- committer.pendingCommitCount _ <- committer.updatePendingCommitsAfterPoll pendingCommitsAfterCommit <- committer.pendingCommitCount @@ -194,7 +195,7 @@ object CommitterSpec extends ZIOSpecDefault { for { runtime <- ZIO.runtime[Any] commitAvailable <- Promise.make[Nothing, Unit] - committer <- Committer.make( + committer <- LiveCommitter.make( 10.seconds, Diagnostics.NoOp, mockMetrics, @@ -204,7 +205,7 @@ object CommitterSpec extends ZIOSpecDefault { tp = new TopicPartition("topic", 0) commitFiber <- committer.commit(Map(tp -> new OffsetAndMetadata(0))).forkScoped _ <- commitAvailable.await - _ <- committer.handleNewCommits((offsets, callback) => ZIO.attempt(callback.onComplete(offsets, null))) + _ <- committer.processQueuedCommits((offsets, callback) => ZIO.attempt(callback.onComplete(offsets, null))) committedOffsets <- committer.getCommittedOffsets _ <- commitFiber.join } yield assertTrue(committedOffsets.offsets == Map(tp -> 0L)) @@ -213,7 +214,7 @@ object CommitterSpec extends ZIOSpecDefault { for { runtime <- ZIO.runtime[Any] commitAvailable <- Promise.make[Nothing, Unit] - committer <- Committer.make( + committer <- LiveCommitter.make( 10.seconds, Diagnostics.NoOp, mockMetrics, @@ -223,7 +224,7 @@ object CommitterSpec extends ZIOSpecDefault { tp = new TopicPartition("topic", 0) commitFiber <- committer.commit(Map(tp -> new OffsetAndMetadata(0))).forkScoped _ <- commitAvailable.await - _ <- committer.handleNewCommits((offsets, callback) => ZIO.attempt(callback.onComplete(offsets, null))) + _ <- committer.processQueuedCommits((offsets, callback) => ZIO.attempt(callback.onComplete(offsets, null))) _ <- committer.pruneCommittedOffsets(Set.empty) committedOffsets <- committer.getCommittedOffsets _ <- commitFiber.join diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Committer.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Committer.scala index a8e7a5a7f..22263f5ae 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Committer.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Committer.scala @@ -1,43 +1,14 @@ package zio.kafka.consumer.internal + import org.apache.kafka.clients.consumer.{ OffsetAndMetadata, OffsetCommitCallback } import org.apache.kafka.common.TopicPartition -import org.apache.kafka.common.errors.RebalanceInProgressException -import zio.kafka.consumer.Consumer.CommitTimeout -import zio.kafka.consumer.diagnostics.{ DiagnosticEvent, Diagnostics } -import zio.kafka.consumer.internal.Committer.{ Commit, CommitOffsets } -import zio.{ durationLong, Chunk, Duration, Exit, Promise, Queue, Ref, Runtime, Scope, Task, UIO, Unsafe, ZIO } +import zio.kafka.consumer.internal.LiveCommitter.CommitOffsets +import zio.{ Task, UIO } -import java.lang.Math.max -import java.util import java.util.{ Map => JavaMap } -import scala.collection.mutable -import scala.jdk.CollectionConverters._ - -private[consumer] final class Committer( - commitQueue: Queue[Commit], - commitTimeout: Duration, - diagnostics: Diagnostics, - consumerMetrics: ConsumerMetrics, - onCommitAvailable: UIO[Unit], - committedOffsetsRef: Ref[CommitOffsets], - sameThreadRuntime: Runtime[Any], - pendingCommits: Ref[Chunk[Commit]] -) { - /** This is the implementation behind the user facing api `Offset.commit`. */ - val commit: Map[TopicPartition, OffsetAndMetadata] => Task[Unit] = - offsets => - for { - p <- Promise.make[Throwable, Unit] - startTime = java.lang.System.nanoTime() - _ <- commitQueue.offer(Commit(startTime, offsets, p)) - _ <- onCommitAvailable - _ <- diagnostics.emit(DiagnosticEvent.Commit.Started(offsets)) - _ <- p.await.timeoutFail(CommitTimeout)(commitTimeout) - endTime = java.lang.System.nanoTime() - latency = (endTime - startTime).nanoseconds - _ <- consumerMetrics.observeCommit(latency) - } yield () +trait Committer { + val commit: Map[TopicPartition, OffsetAndMetadata] => Task[Unit] /** * Takes commits from the queue, commits them and adds them to pending commits @@ -49,160 +20,20 @@ private[consumer] final class Committer( * @param executeOnEmpty * Execute commitAsync() even if there are no commits */ - def handleNewCommits( + def processQueuedCommits( commitAsync: (JavaMap[TopicPartition, OffsetAndMetadata], OffsetCommitCallback) => Task[Unit], executeOnEmpty: Boolean = false - ): Task[Unit] = for { - commits <- commitQueue.takeAll - _ <- ZIO.logDebug(s"Processing ${commits.size} commits") - _ <- ZIO.unless(commits.isEmpty && !executeOnEmpty) { - val (offsets, callback, onFailure) = asyncCommitParameters(commits) - pendingCommits.update(_ ++ commits) *> - // We don't wait for the completion of the commit here, because it - // will only complete once we poll again. - commitAsync(offsets, callback) - .catchAll(onFailure) - } - } yield () - - /** Merge commits and prepare parameters for calling `consumer.commitAsync`. */ - private def asyncCommitParameters( - commits: Chunk[Commit] - ): (JavaMap[TopicPartition, OffsetAndMetadata], OffsetCommitCallback, Throwable => UIO[Unit]) = { - val offsets = commits - .foldLeft(mutable.Map.empty[TopicPartition, OffsetAndMetadata]) { case (acc, commit) => - commit.offsets.foreach { case (tp, offset) => - acc += (tp -> acc - .get(tp) - .map(current => if (current.offset() > offset.offset()) current else offset) - .getOrElse(offset)) - } - acc - } - .toMap - val offsetsWithMetaData = offsets.map { case (tp, offset) => - tp -> new OffsetAndMetadata(offset.offset + 1, offset.leaderEpoch, offset.metadata) - } - val cont = (e: Exit[Throwable, Unit]) => ZIO.foreachDiscard(commits)(_.cont.done(e)) - // We assume the commit is started immediately after returning from this method. - val startTime = java.lang.System.nanoTime() - val onSuccess = { - val endTime = java.lang.System.nanoTime() - val latency = (endTime - startTime).nanoseconds - for { - offsetIncrease <- committedOffsetsRef.modify(_.addCommits(commits)) - _ <- consumerMetrics.observeAggregatedCommit(latency, offsetIncrease).when(commits.nonEmpty) - result <- cont(Exit.unit) - _ <- diagnostics.emit(DiagnosticEvent.Commit.Success(offsetsWithMetaData)) - } yield result - } - val onFailure: Throwable => UIO[Unit] = { - case _: RebalanceInProgressException => - for { - _ <- ZIO.logDebug(s"Rebalance in progress, commit for offsets $offsets will be retried") - _ <- commitQueue.offerAll(commits) - _ <- onCommitAvailable - } yield () - case err: Throwable => - cont(Exit.fail(err)) <* diagnostics.emit(DiagnosticEvent.Commit.Failure(offsetsWithMetaData, err)) - } - val callback = - new OffsetCommitCallback { - override def onComplete(offsets: util.Map[TopicPartition, OffsetAndMetadata], exception: Exception): Unit = - Unsafe.unsafe { implicit u => - sameThreadRuntime.unsafe.run { - if (exception eq null) onSuccess else onFailure(exception) - } - .getOrThrowFiberFailure() - } - } - (offsetsWithMetaData.asJava, callback, onFailure) - } - - def queueSize: UIO[Int] = commitQueue.size - - def pendingCommitCount: UIO[Int] = pendingCommits.get.map(_.size) - - def getPendingCommits: UIO[CommitOffsets] = - pendingCommits.get.map(CommitOffsets.empty.addCommits(_)._2) - - def updatePendingCommitsAfterPoll: UIO[Unit] = - pendingCommits.get.flatMap(ZIO.filter(_)(_.isPending)).flatMap(pendingCommits.set) - - def pruneCommittedOffsets(assignedPartitions: Set[TopicPartition]): UIO[Unit] = - committedOffsetsRef.update(_.keepPartitions(assignedPartitions)) - - def getCommittedOffsets: UIO[CommitOffsets] = committedOffsetsRef.get -} - -private[internal] object Committer { - def make( - commitTimeout: Duration, - diagnostics: Diagnostics, - consumerMetrics: ConsumerMetrics, - onCommitAvailable: UIO[Unit], - sameThreadRuntime: Runtime[Any] - ): ZIO[Any with Scope, Nothing, Committer] = for { - pendingCommits <- Ref.make(Chunk.empty[Commit]) - commitQueue <- ZIO.acquireRelease(Queue.unbounded[Commit])(_.shutdown) - committedOffsetsRef <- Ref.make(CommitOffsets.empty) - } yield new Committer( - commitQueue, - commitTimeout, - diagnostics, - consumerMetrics, - onCommitAvailable, - committedOffsetsRef, - sameThreadRuntime, - pendingCommits - ) - - private[internal] final case class Commit( - createdAt: NanoTime, - offsets: Map[TopicPartition, OffsetAndMetadata], - cont: Promise[Throwable, Unit] - ) { - @inline def isDone: UIO[Boolean] = cont.isDone - @inline def isPending: UIO[Boolean] = isDone.negate - } + ): Task[Unit] - // package private for unit testing - private[internal] final case class CommitOffsets(offsets: Map[TopicPartition, Long]) { + def queueSize: UIO[Int] - /** Returns an estimate of the total offset increase, and a new `CommitOffsets` with the given offsets added. */ - def addCommits(c: Chunk[Commit]): (Long, CommitOffsets) = { - val updatedOffsets = mutable.Map.empty[TopicPartition, Long] - updatedOffsets.sizeHint(offsets.size) - updatedOffsets ++= offsets - var offsetIncrease = 0L - c.foreach { commit => - commit.offsets.foreach { case (tp, offsetAndMeta) => - val offset = offsetAndMeta.offset() - val maxOffset = updatedOffsets.get(tp) match { - case Some(existingOffset) => - offsetIncrease += max(0L, offset - existingOffset) - max(existingOffset, offset) - case None => - // This partition was not committed to from this consumer yet. Therefore we do not know the offset - // increase. A good estimate would be the poll size for this consumer, another okayish estimate is 0. - // Lets go with the simplest for now: ```offsetIncrease += 0``` - offset - } - updatedOffsets += tp -> maxOffset - } - } - (offsetIncrease, CommitOffsets(offsets = updatedOffsets.toMap)) - } + def pendingCommitCount: UIO[Int] - def keepPartitions(tps: Set[TopicPartition]): CommitOffsets = - CommitOffsets(offsets.filter { case (tp, _) => tps.contains(tp) }) + def getPendingCommits: UIO[CommitOffsets] - def contains(tp: TopicPartition, offset: Long): Boolean = - offsets.get(tp).exists(_ >= offset) - } + def updatePendingCommitsAfterPoll: UIO[Unit] - private[internal] object CommitOffsets { - val empty: CommitOffsets = CommitOffsets(Map.empty) - } + def pruneCommittedOffsets(assignedPartitions: Set[TopicPartition]): UIO[Unit] + def getCommittedOffsets: UIO[CommitOffsets] } diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/LiveCommitter.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/LiveCommitter.scala new file mode 100644 index 000000000..70f3ea0c2 --- /dev/null +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/LiveCommitter.scala @@ -0,0 +1,208 @@ +package zio.kafka.consumer.internal +import org.apache.kafka.clients.consumer.{ OffsetAndMetadata, OffsetCommitCallback } +import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.errors.RebalanceInProgressException +import zio.kafka.consumer.Consumer.CommitTimeout +import zio.kafka.consumer.diagnostics.{ DiagnosticEvent, Diagnostics } +import zio.kafka.consumer.internal.LiveCommitter.{ Commit, CommitOffsets } +import zio.{ durationLong, Chunk, Duration, Exit, Promise, Queue, Ref, Runtime, Scope, Task, UIO, Unsafe, ZIO } + +import java.lang.Math.max +import java.util +import java.util.{ Map => JavaMap } +import scala.collection.mutable +import scala.jdk.CollectionConverters._ + +private[consumer] final class LiveCommitter( + commitQueue: Queue[Commit], + commitTimeout: Duration, + diagnostics: Diagnostics, + consumerMetrics: ConsumerMetrics, + onCommitAvailable: UIO[Unit], + committedOffsetsRef: Ref[CommitOffsets], + sameThreadRuntime: Runtime[Any], + pendingCommits: Ref[Chunk[Commit]] +) extends Committer { + + /** This is the implementation behind the user facing api `Offset.commit`. */ + override val commit: Map[TopicPartition, OffsetAndMetadata] => Task[Unit] = + offsets => + for { + p <- Promise.make[Throwable, Unit] + startTime = java.lang.System.nanoTime() + _ <- commitQueue.offer(Commit(startTime, offsets, p)) + _ <- onCommitAvailable + _ <- diagnostics.emit(DiagnosticEvent.Commit.Started(offsets)) + _ <- p.await.timeoutFail(CommitTimeout)(commitTimeout) + endTime = java.lang.System.nanoTime() + latency = (endTime - startTime).nanoseconds + _ <- consumerMetrics.observeCommit(latency) + } yield () + + /** + * Takes commits from the queue, commits them and adds them to pending commits + * + * If the queue is empty, nothing happens, unless executeOnEmpty is true. + * + * @param consumer + * Consumer with exclusive access + * @param executeOnEmpty + * Execute commitAsync() even if there are no commits + */ + override def processQueuedCommits( + commitAsync: (JavaMap[TopicPartition, OffsetAndMetadata], OffsetCommitCallback) => Task[Unit], + executeOnEmpty: Boolean = false + ): Task[Unit] = for { + commits <- commitQueue.takeAll + _ <- ZIO.logDebug(s"Processing ${commits.size} commits") + _ <- ZIO.unless(commits.isEmpty && !executeOnEmpty) { + val (offsets, callback, onFailure) = asyncCommitParameters(commits) + pendingCommits.update(_ ++ commits) *> + // We don't wait for the completion of the commit here, because it + // will only complete once we poll again. + commitAsync(offsets, callback) + .catchAll(onFailure) + } + } yield () + + /** Merge commits and prepare parameters for calling `consumer.commitAsync`. */ + private def asyncCommitParameters( + commits: Chunk[Commit] + ): (JavaMap[TopicPartition, OffsetAndMetadata], OffsetCommitCallback, Throwable => UIO[Unit]) = { + val offsets = commits + .foldLeft(mutable.Map.empty[TopicPartition, OffsetAndMetadata]) { case (acc, commit) => + commit.offsets.foreach { case (tp, offset) => + acc += (tp -> acc + .get(tp) + .map(current => if (current.offset() > offset.offset()) current else offset) + .getOrElse(offset)) + } + acc + } + .toMap + val offsetsWithMetaData = offsets.map { case (tp, offset) => + tp -> new OffsetAndMetadata(offset.offset + 1, offset.leaderEpoch, offset.metadata) + } + val cont = (e: Exit[Throwable, Unit]) => ZIO.foreachDiscard(commits)(_.cont.done(e)) + // We assume the commit is started immediately after returning from this method. + val startTime = java.lang.System.nanoTime() + val onSuccess = { + val endTime = java.lang.System.nanoTime() + val latency = (endTime - startTime).nanoseconds + for { + offsetIncrease <- committedOffsetsRef.modify(_.addCommits(commits)) + _ <- consumerMetrics.observeAggregatedCommit(latency, offsetIncrease).when(commits.nonEmpty) + result <- cont(Exit.unit) + _ <- diagnostics.emit(DiagnosticEvent.Commit.Success(offsetsWithMetaData)) + } yield result + } + val onFailure: Throwable => UIO[Unit] = { + case _: RebalanceInProgressException => + for { + _ <- ZIO.logDebug(s"Rebalance in progress, commit for offsets $offsets will be retried") + _ <- commitQueue.offerAll(commits) + _ <- onCommitAvailable + } yield () + case err: Throwable => + cont(Exit.fail(err)) <* diagnostics.emit(DiagnosticEvent.Commit.Failure(offsetsWithMetaData, err)) + } + val callback = + new OffsetCommitCallback { + override def onComplete(offsets: util.Map[TopicPartition, OffsetAndMetadata], exception: Exception): Unit = + Unsafe.unsafe { implicit u => + sameThreadRuntime.unsafe.run { + if (exception eq null) onSuccess else onFailure(exception) + } + .getOrThrowFiberFailure() + } + } + (offsetsWithMetaData.asJava, callback, onFailure) + } + + override def queueSize: UIO[Int] = commitQueue.size + + override def pendingCommitCount: UIO[Int] = pendingCommits.get.map(_.size) + + override def getPendingCommits: UIO[CommitOffsets] = + pendingCommits.get.map(CommitOffsets.empty.addCommits(_)._2) + + override def updatePendingCommitsAfterPoll: UIO[Unit] = + pendingCommits.get.flatMap(ZIO.filter(_)(_.isPending)).flatMap(pendingCommits.set) + + override def pruneCommittedOffsets(assignedPartitions: Set[TopicPartition]): UIO[Unit] = + committedOffsetsRef.update(_.keepPartitions(assignedPartitions)) + + override def getCommittedOffsets: UIO[CommitOffsets] = committedOffsetsRef.get +} + +private[internal] object LiveCommitter { + def make( + commitTimeout: Duration, + diagnostics: Diagnostics, + consumerMetrics: ConsumerMetrics, + onCommitAvailable: UIO[Unit], + sameThreadRuntime: Runtime[Any] + ): ZIO[Scope, Nothing, LiveCommitter] = for { + pendingCommits <- Ref.make(Chunk.empty[Commit]) + commitQueue <- ZIO.acquireRelease(Queue.unbounded[Commit])(_.shutdown) + committedOffsetsRef <- Ref.make(CommitOffsets.empty) + } yield new LiveCommitter( + commitQueue, + commitTimeout, + diagnostics, + consumerMetrics, + onCommitAvailable, + committedOffsetsRef, + sameThreadRuntime, + pendingCommits + ) + + private[internal] final case class Commit( + createdAt: NanoTime, + offsets: Map[TopicPartition, OffsetAndMetadata], + cont: Promise[Throwable, Unit] + ) { + @inline def isDone: UIO[Boolean] = cont.isDone + @inline def isPending: UIO[Boolean] = isDone.negate + } + + // package private for unit testing + private[internal] final case class CommitOffsets(offsets: Map[TopicPartition, Long]) { + + /** Returns an estimate of the total offset increase, and a new `CommitOffsets` with the given offsets added. */ + def addCommits(c: Chunk[Commit]): (Long, CommitOffsets) = { + val updatedOffsets = mutable.Map.empty[TopicPartition, Long] + updatedOffsets.sizeHint(offsets.size) + updatedOffsets ++= offsets + var offsetIncrease = 0L + c.foreach { commit => + commit.offsets.foreach { case (tp, offsetAndMeta) => + val offset = offsetAndMeta.offset() + val maxOffset = updatedOffsets.get(tp) match { + case Some(existingOffset) => + offsetIncrease += max(0L, offset - existingOffset) + max(existingOffset, offset) + case None => + // This partition was not committed to from this consumer yet. Therefore we do not know the offset + // increase. A good estimate would be the poll size for this consumer, another okayish estimate is 0. + // Lets go with the simplest for now: ```offsetIncrease += 0``` + offset + } + updatedOffsets += tp -> maxOffset + } + } + (offsetIncrease, CommitOffsets(offsets = updatedOffsets.toMap)) + } + + def keepPartitions(tps: Set[TopicPartition]): CommitOffsets = + CommitOffsets(offsets.filter { case (tp, _) => tps.contains(tp) }) + + def contains(tp: TopicPartition, offset: Long): Boolean = + offsets.get(tp).exists(_ >= offset) + } + + private[internal] object CommitOffsets { + val empty: CommitOffsets = CommitOffsets(Map.empty) + } + +} diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala index 7a8738f08..2beb5490d 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala @@ -483,7 +483,7 @@ private[consumer] final class Runloop private ( for { _ <- ZIO.logDebug(s"Processing ${commands.size} commands: ${commands.mkString(",")}") _ <- consumer.runloopAccess { consumer => - committer.handleNewCommits((offsets, callback) => ZIO.attempt(consumer.commitAsync(offsets, callback))) + committer.processQueuedCommits((offsets, callback) => ZIO.attempt(consumer.commitAsync(offsets, callback))) } streamCommands = commands.collect { case cmd: RunloopCommand.StreamCommand => cmd } stateAfterCommands <- ZIO.foldLeft(streamCommands)(state)(handleCommand) @@ -577,13 +577,14 @@ object Runloop { sameThreadRuntime <- ZIO.runtime[Any].provideLayer(SameThreadRuntimeLayer) executor <- ZIO.executor metrics = new ZioConsumerMetrics(settings.metricLabels) - committer <- Committer.make( - settings.commitTimeout, - diagnostics, - metrics, - commandQueue.offer(RunloopCommand.CommitAvailable).unit, - sameThreadRuntime - ) + committer <- LiveCommitter + .make( + settings.commitTimeout, + diagnostics, + metrics, + commandQueue.offer(RunloopCommand.CommitAvailable).unit, + sameThreadRuntime + ) rebalanceListener = new RunloopRebalanceListener( lastRebalanceEvent, settings, diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/RunloopRebalanceListener.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/RunloopRebalanceListener.scala index 5c9384441..98ef49613 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/RunloopRebalanceListener.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/RunloopRebalanceListener.scala @@ -160,7 +160,7 @@ private[internal] class RunloopRebalanceListener( // so that commits can complete and the streams can make progress, by setting // executeOnEmpty = true .tap(_ => - committer.handleNewCommits( + committer.processQueuedCommits( (offsets, callback) => ZIO.attempt(consumer.commitAsync(offsets, callback)), executeOnEmpty = true ) From 51d3a0699c80fd6f42e7265c4d58d82d3f8808ff Mon Sep 17 00:00:00 2001 From: Steven Vroonland Date: Wed, 13 Nov 2024 11:28:45 +0100 Subject: [PATCH 28/57] Add last committed offset to stream completion status --- .../zio/kafka/consumer/internal/LiveCommitter.scala | 2 ++ .../consumer/internal/RunloopRebalanceListener.scala | 11 ++++++++++- 2 files changed, 12 insertions(+), 1 deletion(-) diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/LiveCommitter.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/LiveCommitter.scala index 70f3ea0c2..8df5fcc36 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/LiveCommitter.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/LiveCommitter.scala @@ -199,6 +199,8 @@ private[internal] object LiveCommitter { def contains(tp: TopicPartition, offset: Long): Boolean = offsets.get(tp).exists(_ >= offset) + + def get(tp: TopicPartition): Option[Long] = offsets.get(tp) } private[internal] object CommitOffsets { diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/RunloopRebalanceListener.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/RunloopRebalanceListener.scala index 98ef49613..4cdd20424 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/RunloopRebalanceListener.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/RunloopRebalanceListener.scala @@ -87,7 +87,13 @@ private[internal] class RunloopRebalanceListener( EndOffsetCommitPending case _ => EndOffsetNotCommitted } - } yield StreamCompletionStatus(stream.tp, isDone, lastPulledOffset.map(_.offset), endOffsetCommitStatus) + } yield StreamCompletionStatus( + stream.tp, + isDone, + lastPulledOffset.map(_.offset), + committedOffsets.get(stream.tp), + endOffsetCommitStatus + ) } } yield streamResults @@ -247,6 +253,7 @@ private[internal] class RunloopRebalanceListener( private def withLastRebalanceEvent(f: RebalanceEvent => Task[RebalanceEvent]): Task[Unit] = lastRebalanceEvent.get.flatMap(f).flatMap(lastRebalanceEvent.set) + } private[internal] object RunloopRebalanceListener { @@ -260,12 +267,14 @@ private[internal] object RunloopRebalanceListener { tp: TopicPartition, streamEnded: Boolean, lastPulledOffset: Option[Long], + lastCommittedOffset: Option[Long], endOffsetCommitStatus: EndOffsetCommitStatus ) { override def toString: String = s"$tp: " + s"${if (streamEnded) "stream ended" else "stream is running"}, " + s"last pulled offset=${lastPulledOffset.getOrElse("none")}, " + + s"last committed offset=${lastCommittedOffset.getOrElse("none")}, " + endOffsetCommitStatus } From 3439e102e6e0ae968123cff051641a5948c51ff2 Mon Sep 17 00:00:00 2001 From: Steven Vroonland Date: Wed, 13 Nov 2024 11:32:54 +0100 Subject: [PATCH 29/57] Some tests for the rebalance listener --- .../RunloopRebalanceListenerSpec.scala | 164 ++++++++++++++++++ 1 file changed, 164 insertions(+) create mode 100644 zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/RunloopRebalanceListenerSpec.scala diff --git a/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/RunloopRebalanceListenerSpec.scala b/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/RunloopRebalanceListenerSpec.scala new file mode 100644 index 000000000..4ab60c8ff --- /dev/null +++ b/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/RunloopRebalanceListenerSpec.scala @@ -0,0 +1,164 @@ +package zio.kafka.consumer.internal + +import org.apache.kafka.clients.consumer._ +import org.apache.kafka.common.TopicPartition +import zio.kafka.ZIOSpecDefaultSlf4j +import zio.kafka.consumer.diagnostics.Diagnostics +import zio.kafka.consumer.internal.LiveCommitter.{ Commit, CommitOffsets } +import zio.kafka.consumer.internal.Runloop.ByteArrayCommittableRecord +import zio.kafka.consumer.{ CommittableRecord, ConsumerSettings } +import zio.test._ +import zio.{ durationInt, Chunk, Promise, Ref, Task, UIO, ZIO } + +/** + * Runloop should: + * - Track the partitions that were assigned, revoked and lost in multiple invocations + * - rebalanceSafeCommits mode + * - Await stream completion and end offset commit + * - End streams when partitions are assigned + */ + +object RunloopRebalanceListenerSpec extends ZIOSpecDefaultSlf4j { + type BinaryMockConsumer = MockConsumer[Array[Byte], Array[Byte]] + + def spec = suite("RunloopRebalanceListener")( + test("should track assigned, revoked and lost partitions") { + for { + lastEvent <- Ref.make(RunloopRebalanceListener.RebalanceEvent.None) + consumer = new BinaryMockConsumer(OffsetResetStrategy.LATEST) {} + tp = new TopicPartition("topic", 0) + tp2 = new TopicPartition("topic", 1) + tp3 = new TopicPartition("topic", 2) + tp4 = new TopicPartition("topic", 3) + listener <- makeListener(lastEvent, consumer) + _ <- listener.toRebalanceListener.onAssigned(Set(tp)) + _ <- listener.toRebalanceListener.onAssigned(Set(tp4)) + _ <- listener.toRebalanceListener.onRevoked(Set(tp2)) + _ <- listener.toRebalanceListener.onLost(Set(tp3)) + event <- lastEvent.get + } yield assertTrue( + event.wasInvoked && event.assignedTps == Set(tp, tp4) && event.revokedTps == Set(tp2) && event.lostTps == Set( + tp3 + ) + ) + }, + test("should end streams for revoked and lost partitions") { + for { + lastEvent <- Ref.make(RunloopRebalanceListener.RebalanceEvent.None) + consumer = new BinaryMockConsumer(OffsetResetStrategy.LATEST) {} + tp = new TopicPartition("topic", 0) + tp2 = new TopicPartition("topic", 1) + tp3 = new TopicPartition("topic", 2) + assignedStreams <- ZIO.foreach(Chunk(tp, tp2, tp3))(makeStreamControl) + listener <- makeListener(lastEvent, consumer, assignedStreams = assignedStreams) + _ <- listener.toRebalanceListener.onAssigned(Set(tp)) + _ <- listener.toRebalanceListener.onRevoked(Set(tp2)) + _ <- listener.toRebalanceListener.onLost(Set(tp3)) + event <- lastEvent.get + // Lost and end partition's stream should be ended + _ <- assignedStreams(1).stream.runDrain + _ <- assignedStreams(2).stream.runDrain + } yield assertTrue(event.endedStreams.map(_.tp).toSet == Set(tp2, tp3)) + }, + suite("rebalanceSafeCommits")( + test("should wait for the last pulled offset to commit") { + for { + lastEvent <- Ref.make(RunloopRebalanceListener.RebalanceEvent.None) + consumer = new BinaryMockConsumer(OffsetResetStrategy.LATEST) {} + tp = new TopicPartition("topic", 0) + streamControl <- makeStreamControl(tp) + records = createTestRecords(3) + recordsPulled <- Promise.make[Nothing, Unit] + _ <- streamControl.offerRecords(records) + committedOffsets <- Ref.make(CommitOffsets.empty) + done <- Promise.make[Throwable, Unit] + committer = new MockCommitter { + override val commit = + offsets => + committedOffsets + .update(_.addCommits(Chunk(Commit(0L, offsets, done)))._2) + override def getCommittedOffsets = committedOffsets.get + } + streamDrain <- + streamControl.stream + .tap(_ => recordsPulled.succeed(())) + .tap(record => + committer.commit( + Map( + new TopicPartition("topic", record.partition) -> new OffsetAndMetadata(record.offset.offset, null) + ) + ) + ) + .runDrain + .forkScoped + listener <- + makeListener( + lastEvent, + consumer, + assignedStreams = Chunk(streamControl), + rebalanceSafeCommits = true, + committer = committer + ) + _ <- listener.toRebalanceListener.onRevoked(Set(tp)) + _ <- streamDrain.join + } yield assertCompletes + } +// TODO test("should wait until timeout") + // TODO something with driving commits during waiting + ) + ) @@ TestAspect.withLiveClock + + private def makeStreamControl(tp: TopicPartition): UIO[PartitionStreamControl] = + PartitionStreamControl.newPartitionStream(tp, ZIO.unit, Diagnostics.NoOp, 30.seconds) + + private def makeListener( + lastEvent: Ref[RunloopRebalanceListener.RebalanceEvent], + mockConsumer: BinaryMockConsumer, + assignedStreams: Chunk[PartitionStreamControl] = Chunk.empty, + committer: Committer = new MockCommitter {}, + settings: ConsumerSettings = ConsumerSettings(List("")), + rebalanceSafeCommits: Boolean = false + ) = + ConsumerAccess.make(mockConsumer).map { consumerAccess => + new RunloopRebalanceListener( + lastEvent, + settings.withRebalanceSafeCommits(rebalanceSafeCommits), + consumerAccess, + 60.seconds, + ZIO.succeed(assignedStreams), + committer + ) + } + + private def createTestRecords(count: Int): Chunk[ByteArrayCommittableRecord] = + Chunk.fromIterable( + (1 to count).map(i => + CommittableRecord( + record = new ConsumerRecord[Array[Byte], Array[Byte]]( + "test-topic", + 0, + i.toLong, + Array[Byte](), + Array[Byte]() + ), + commitHandle = _ => ZIO.unit, + consumerGroupMetadata = None + ) + ) + ) +} + +abstract class MockCommitter extends Committer { + override val commit: Map[TopicPartition, OffsetAndMetadata] => Task[Unit] = _ => ZIO.unit + + override def processQueuedCommits( + commitAsync: (java.util.Map[TopicPartition, OffsetAndMetadata], OffsetCommitCallback) => zio.Task[Unit], + executeOnEmpty: Boolean + ): zio.Task[Unit] = ZIO.unit + override def queueSize: UIO[Int] = ZIO.succeed(0) + override def pendingCommitCount: UIO[Int] = ZIO.succeed(0) + override def getPendingCommits: UIO[CommitOffsets] = ZIO.succeed(CommitOffsets.empty) + override def updatePendingCommitsAfterPoll: UIO[Unit] = ZIO.unit + override def pruneCommittedOffsets(assignedPartitions: Set[TopicPartition]): UIO[Unit] = ZIO.unit + override def getCommittedOffsets: UIO[CommitOffsets] = ZIO.succeed(CommitOffsets.empty) +} From 8fb3622c3b4b2405be421f3a59f847ee3b297a6e Mon Sep 17 00:00:00 2001 From: Steven Vroonland Date: Wed, 13 Nov 2024 11:37:32 +0100 Subject: [PATCH 30/57] Renames --- ...c.scala => RebalanceCoordinatorSpec.scala} | 25 ++++++----- ...tener.scala => RebalanceCoordinator.scala} | 6 +-- .../zio/kafka/consumer/internal/Runloop.scala | 44 ++++++++++--------- 3 files changed, 39 insertions(+), 36 deletions(-) rename zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/{RunloopRebalanceListenerSpec.scala => RebalanceCoordinatorSpec.scala} (89%) rename zio-kafka/src/main/scala/zio/kafka/consumer/internal/{RunloopRebalanceListener.scala => RebalanceCoordinator.scala} (98%) diff --git a/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/RunloopRebalanceListenerSpec.scala b/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/RebalanceCoordinatorSpec.scala similarity index 89% rename from zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/RunloopRebalanceListenerSpec.scala rename to zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/RebalanceCoordinatorSpec.scala index 4ab60c8ff..b33dc30d8 100644 --- a/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/RunloopRebalanceListenerSpec.scala +++ b/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/RebalanceCoordinatorSpec.scala @@ -5,10 +5,11 @@ import org.apache.kafka.common.TopicPartition import zio.kafka.ZIOSpecDefaultSlf4j import zio.kafka.consumer.diagnostics.Diagnostics import zio.kafka.consumer.internal.LiveCommitter.{ Commit, CommitOffsets } +import zio.kafka.consumer.internal.RebalanceCoordinator.RebalanceEvent import zio.kafka.consumer.internal.Runloop.ByteArrayCommittableRecord import zio.kafka.consumer.{ CommittableRecord, ConsumerSettings } import zio.test._ -import zio.{ durationInt, Chunk, Promise, Ref, Task, UIO, ZIO } +import zio.{ durationInt, Chunk, Promise, Ref, Scope, Task, UIO, ZIO } /** * Runloop should: @@ -18,19 +19,19 @@ import zio.{ durationInt, Chunk, Promise, Ref, Task, UIO, ZIO } * - End streams when partitions are assigned */ -object RunloopRebalanceListenerSpec extends ZIOSpecDefaultSlf4j { +object RebalanceCoordinatorSpec extends ZIOSpecDefaultSlf4j { type BinaryMockConsumer = MockConsumer[Array[Byte], Array[Byte]] def spec = suite("RunloopRebalanceListener")( test("should track assigned, revoked and lost partitions") { for { - lastEvent <- Ref.make(RunloopRebalanceListener.RebalanceEvent.None) + lastEvent <- Ref.make(RebalanceCoordinator.RebalanceEvent.None) consumer = new BinaryMockConsumer(OffsetResetStrategy.LATEST) {} tp = new TopicPartition("topic", 0) tp2 = new TopicPartition("topic", 1) tp3 = new TopicPartition("topic", 2) tp4 = new TopicPartition("topic", 3) - listener <- makeListener(lastEvent, consumer) + listener <- makeCoordinator(lastEvent, consumer) _ <- listener.toRebalanceListener.onAssigned(Set(tp)) _ <- listener.toRebalanceListener.onAssigned(Set(tp4)) _ <- listener.toRebalanceListener.onRevoked(Set(tp2)) @@ -44,13 +45,13 @@ object RunloopRebalanceListenerSpec extends ZIOSpecDefaultSlf4j { }, test("should end streams for revoked and lost partitions") { for { - lastEvent <- Ref.make(RunloopRebalanceListener.RebalanceEvent.None) + lastEvent <- Ref.make(RebalanceCoordinator.RebalanceEvent.None) consumer = new BinaryMockConsumer(OffsetResetStrategy.LATEST) {} tp = new TopicPartition("topic", 0) tp2 = new TopicPartition("topic", 1) tp3 = new TopicPartition("topic", 2) assignedStreams <- ZIO.foreach(Chunk(tp, tp2, tp3))(makeStreamControl) - listener <- makeListener(lastEvent, consumer, assignedStreams = assignedStreams) + listener <- makeCoordinator(lastEvent, consumer, assignedStreams = assignedStreams) _ <- listener.toRebalanceListener.onAssigned(Set(tp)) _ <- listener.toRebalanceListener.onRevoked(Set(tp2)) _ <- listener.toRebalanceListener.onLost(Set(tp3)) @@ -63,7 +64,7 @@ object RunloopRebalanceListenerSpec extends ZIOSpecDefaultSlf4j { suite("rebalanceSafeCommits")( test("should wait for the last pulled offset to commit") { for { - lastEvent <- Ref.make(RunloopRebalanceListener.RebalanceEvent.None) + lastEvent <- Ref.make(RebalanceCoordinator.RebalanceEvent.None) consumer = new BinaryMockConsumer(OffsetResetStrategy.LATEST) {} tp = new TopicPartition("topic", 0) streamControl <- makeStreamControl(tp) @@ -92,7 +93,7 @@ object RunloopRebalanceListenerSpec extends ZIOSpecDefaultSlf4j { .runDrain .forkScoped listener <- - makeListener( + makeCoordinator( lastEvent, consumer, assignedStreams = Chunk(streamControl), @@ -111,16 +112,16 @@ object RunloopRebalanceListenerSpec extends ZIOSpecDefaultSlf4j { private def makeStreamControl(tp: TopicPartition): UIO[PartitionStreamControl] = PartitionStreamControl.newPartitionStream(tp, ZIO.unit, Diagnostics.NoOp, 30.seconds) - private def makeListener( - lastEvent: Ref[RunloopRebalanceListener.RebalanceEvent], + private def makeCoordinator( + lastEvent: Ref[RebalanceEvent], mockConsumer: BinaryMockConsumer, assignedStreams: Chunk[PartitionStreamControl] = Chunk.empty, committer: Committer = new MockCommitter {}, settings: ConsumerSettings = ConsumerSettings(List("")), rebalanceSafeCommits: Boolean = false - ) = + ): ZIO[Scope, Throwable, RebalanceCoordinator] = ConsumerAccess.make(mockConsumer).map { consumerAccess => - new RunloopRebalanceListener( + new RebalanceCoordinator( lastEvent, settings.withRebalanceSafeCommits(rebalanceSafeCommits), consumerAccess, diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/RunloopRebalanceListener.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/RebalanceCoordinator.scala similarity index 98% rename from zio-kafka/src/main/scala/zio/kafka/consumer/internal/RunloopRebalanceListener.scala rename to zio-kafka/src/main/scala/zio/kafka/consumer/internal/RebalanceCoordinator.scala index 4cdd20424..934cb0bcd 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/RunloopRebalanceListener.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/RebalanceCoordinator.scala @@ -1,7 +1,7 @@ package zio.kafka.consumer.internal import org.apache.kafka.common.TopicPartition import zio.kafka.consumer.internal.ConsumerAccess.ByteArrayKafkaConsumer -import zio.kafka.consumer.internal.RunloopRebalanceListener.{ +import zio.kafka.consumer.internal.RebalanceCoordinator.{ EndOffsetCommitPending, EndOffsetCommitted, EndOffsetNotCommitted, @@ -20,7 +20,7 @@ import zio.{ durationInt, Chunk, Duration, Ref, Task, UIO, ZIO } * When rebalanceSafeCommits is enabled, we await completion of all revoked partitions' streams and their commits before * continuing. */ -private[internal] class RunloopRebalanceListener( +private[internal] class RebalanceCoordinator( lastRebalanceEvent: Ref[RebalanceEvent], settings: ConsumerSettings, consumer: ConsumerAccess, @@ -256,7 +256,7 @@ private[internal] class RunloopRebalanceListener( } -private[internal] object RunloopRebalanceListener { +private[internal] object RebalanceCoordinator { sealed trait EndOffsetCommitStatus case object EndOffsetNotCommitted extends EndOffsetCommitStatus { override def toString = "not committed" } diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala index 2beb5490d..2c4ecab17 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala @@ -11,7 +11,7 @@ import zio.kafka.consumer.diagnostics.{ DiagnosticEvent, Diagnostics } import zio.kafka.consumer.internal.ConsumerAccess.ByteArrayKafkaConsumer import zio.kafka.consumer.internal.Runloop._ import zio.kafka.consumer.internal.RunloopAccess.PartitionAssignment -import zio.kafka.consumer.internal.RunloopRebalanceListener.RebalanceEvent +import zio.kafka.consumer.internal.RebalanceCoordinator.RebalanceEvent import zio.stream._ import scala.jdk.CollectionConverters._ @@ -27,7 +27,7 @@ private[consumer] final class Runloop private ( partitionsHub: Hub[Take[Throwable, PartitionAssignment]], diagnostics: Diagnostics, currentStateRef: Ref[State], - runloopRebalanceListener: RunloopRebalanceListener, + rebalanceCoordinator: RebalanceCoordinator, consumerMetrics: ConsumerMetrics, committer: Committer ) { @@ -76,7 +76,7 @@ private[consumer] final class Runloop private ( case _ => settings.rebalanceListener.runOnExecutor(topLevelExecutor) } - RebalanceListener.toKafka(runloopRebalanceListener.toRebalanceListener ++ userRebalanceListener, sameThreadRuntime) + RebalanceListener.toKafka(rebalanceCoordinator.toRebalanceListener ++ userRebalanceListener, sameThreadRuntime) } /** @@ -233,7 +233,7 @@ private[consumer] final class Runloop private ( tpWithoutData = requestedPartitions -- providedTps ) } - pollresult <- runloopRebalanceListener.getAndResetLastEvent.flatMap { + pollresult <- rebalanceCoordinator.getAndResetLastEvent.flatMap { case RebalanceEvent(false, _, _, _, _) => // The fast track, rebalance listener was not invoked: // no assignment changes, no new commits, only new records. @@ -483,7 +483,9 @@ private[consumer] final class Runloop private ( for { _ <- ZIO.logDebug(s"Processing ${commands.size} commands: ${commands.mkString(",")}") _ <- consumer.runloopAccess { consumer => - committer.processQueuedCommits((offsets, callback) => ZIO.attempt(consumer.commitAsync(offsets, callback))) + committer.processQueuedCommits((offsets, callback) => + ZIO.attempt(consumer.commitAsync(offsets, callback)) + ) } streamCommands = commands.collect { case cmd: RunloopCommand.StreamCommand => cmd } stateAfterCommands <- ZIO.foldLeft(streamCommands)(state)(handleCommand) @@ -578,21 +580,21 @@ object Runloop { executor <- ZIO.executor metrics = new ZioConsumerMetrics(settings.metricLabels) committer <- LiveCommitter - .make( - settings.commitTimeout, - diagnostics, - metrics, - commandQueue.offer(RunloopCommand.CommitAvailable).unit, - sameThreadRuntime - ) - rebalanceListener = new RunloopRebalanceListener( - lastRebalanceEvent, - settings, - consumer, - maxRebalanceDuration, - currentStateRef.get.map(_.assignedStreams), - committer - ) + .make( + settings.commitTimeout, + diagnostics, + metrics, + commandQueue.offer(RunloopCommand.CommitAvailable).unit, + sameThreadRuntime + ) + rebalanceCoordinator = new RebalanceCoordinator( + lastRebalanceEvent, + settings, + consumer, + maxRebalanceDuration, + currentStateRef.get.map(_.assignedStreams), + committer + ) runloop = new Runloop( settings = settings, topLevelExecutor = executor, @@ -603,8 +605,8 @@ object Runloop { partitionsHub = partitionsHub, diagnostics = diagnostics, currentStateRef = currentStateRef, - runloopRebalanceListener = rebalanceListener, consumerMetrics = metrics, + rebalanceCoordinator = rebalanceCoordinator, committer = committer ) _ <- ZIO.logDebug("Starting Runloop") From b60bd8e6449708fee59c81133cfbdc15eba83485 Mon Sep 17 00:00:00 2001 From: Steven Vroonland Date: Wed, 13 Nov 2024 13:07:52 +0100 Subject: [PATCH 31/57] More tests + cleanup --- .../internal/RebalanceCoordinatorSpec.scala | 101 ++++++++++++++---- .../internal/RebalanceCoordinator.scala | 7 +- 2 files changed, 86 insertions(+), 22 deletions(-) diff --git a/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/RebalanceCoordinatorSpec.scala b/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/RebalanceCoordinatorSpec.scala index b33dc30d8..43c42a3ce 100644 --- a/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/RebalanceCoordinatorSpec.scala +++ b/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/RebalanceCoordinatorSpec.scala @@ -11,17 +11,32 @@ import zio.kafka.consumer.{ CommittableRecord, ConsumerSettings } import zio.test._ import zio.{ durationInt, Chunk, Promise, Ref, Scope, Task, UIO, ZIO } -/** - * Runloop should: - * - Track the partitions that were assigned, revoked and lost in multiple invocations - * - rebalanceSafeCommits mode - * - Await stream completion and end offset commit - * - End streams when partitions are assigned - */ +import java.util object RebalanceCoordinatorSpec extends ZIOSpecDefaultSlf4j { type BinaryMockConsumer = MockConsumer[Array[Byte], Array[Byte]] + private val mockMetrics = new ConsumerMetrics { + override def observePoll(resumedCount: Int, pausedCount: Int, latency: zio.Duration, pollSize: Int): UIO[Unit] = + ZIO.unit + + override def observeCommit(latency: zio.Duration): UIO[Unit] = ZIO.unit + override def observeAggregatedCommit(latency: zio.Duration, commitSize: NanoTime): UIO[Unit] = ZIO.unit + override def observeRebalance( + currentlyAssignedCount: Int, + assignedCount: Int, + revokedCount: Int, + lostCount: Int + ): UIO[Unit] = ZIO.unit + override def observeRunloopMetrics( + state: Runloop.State, + commandQueueSize: Int, + commitQueueSize: Int, + pendingCommits: Int + ): UIO[Unit] = ZIO.unit + override def observePollAuthError(): UIO[Unit] = ZIO.unit + } + def spec = suite("RunloopRebalanceListener")( test("should track assigned, revoked and lost partitions") { for { @@ -63,6 +78,64 @@ object RebalanceCoordinatorSpec extends ZIOSpecDefaultSlf4j { }, suite("rebalanceSafeCommits")( test("should wait for the last pulled offset to commit") { + for { + lastEvent <- Ref.make(RebalanceCoordinator.RebalanceEvent.None) + // Mock consumer that does not complete commit callbacks until commitSync is called + consumer = new BinaryMockConsumer(OffsetResetStrategy.LATEST) { + var callback: OffsetCommitCallback = null + var offsets: util.Map[TopicPartition, OffsetAndMetadata] = null + + override def commitAsync( + offsets: util.Map[TopicPartition, OffsetAndMetadata], + callback: OffsetCommitCallback + ): Unit = { + // Do nothing during rebalancing + if (callback != null) callback.onComplete(offsets, null) + if (!offsets.isEmpty) { + this.callback = callback + this.offsets = offsets + } + } + + override def commitSync(): Unit = + callback.onComplete(offsets, null) + } + tp = new TopicPartition("topic", 0) + streamControl <- makeStreamControl(tp) + records = createTestRecords(3) + recordsPulled <- Promise.make[Nothing, Unit] + _ <- streamControl.offerRecords(records) + runtime <- ZIO.runtime[Any] + committer <- LiveCommitter.make(10.seconds, Diagnostics.NoOp, mockMetrics, ZIO.unit, runtime) + + streamDrain <- + streamControl.stream + .tap(_ => recordsPulled.succeed(())) + .tap(record => + committer + .commit( + Map( + new TopicPartition("topic", record.partition) -> new OffsetAndMetadata(record.offset.offset, null) + ) + ) + .debug(s"Done commit for ${record.offset.offset}") + ) + .runDrain + .forkScoped + listener <- + makeCoordinator( + lastEvent, + consumer, + assignedStreams = Chunk(streamControl), + rebalanceSafeCommits = true, + committer = committer + ) + _ <- listener.toRebalanceListener.onRevoked(Set(tp)) + _ <- streamDrain.join + } yield assertCompletes + }, + // TODO something with driving commits during waiting + test("should continue if waiting for the stream to continue has timed out") { for { lastEvent <- Ref.make(RebalanceCoordinator.RebalanceEvent.None) consumer = new BinaryMockConsumer(OffsetResetStrategy.LATEST) {} @@ -83,13 +156,6 @@ object RebalanceCoordinatorSpec extends ZIOSpecDefaultSlf4j { streamDrain <- streamControl.stream .tap(_ => recordsPulled.succeed(())) - .tap(record => - committer.commit( - Map( - new TopicPartition("topic", record.partition) -> new OffsetAndMetadata(record.offset.offset, null) - ) - ) - ) .runDrain .forkScoped listener <- @@ -104,9 +170,8 @@ object RebalanceCoordinatorSpec extends ZIOSpecDefaultSlf4j { _ <- streamDrain.join } yield assertCompletes } -// TODO test("should wait until timeout") - // TODO something with driving commits during waiting ) + // TODO something with driving commits during waiting ) @@ TestAspect.withLiveClock private def makeStreamControl(tp: TopicPartition): UIO[PartitionStreamControl] = @@ -117,7 +182,7 @@ object RebalanceCoordinatorSpec extends ZIOSpecDefaultSlf4j { mockConsumer: BinaryMockConsumer, assignedStreams: Chunk[PartitionStreamControl] = Chunk.empty, committer: Committer = new MockCommitter {}, - settings: ConsumerSettings = ConsumerSettings(List("")), + settings: ConsumerSettings = ConsumerSettings(List("")).withCommitTimeout(1.second), rebalanceSafeCommits: Boolean = false ): ZIO[Scope, Throwable, RebalanceCoordinator] = ConsumerAccess.make(mockConsumer).map { consumerAccess => @@ -125,7 +190,7 @@ object RebalanceCoordinatorSpec extends ZIOSpecDefaultSlf4j { lastEvent, settings.withRebalanceSafeCommits(rebalanceSafeCommits), consumerAccess, - 60.seconds, + 5.seconds, ZIO.succeed(assignedStreams), committer ) diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/RebalanceCoordinator.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/RebalanceCoordinator.scala index 934cb0bcd..066bfdf15 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/RebalanceCoordinator.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/RebalanceCoordinator.scala @@ -70,8 +70,8 @@ private[internal] class RebalanceCoordinator( def getStreamCompletionStatuses: UIO[Chunk[StreamCompletionStatus]] = for { - committedOffsets <- committer.getCommittedOffsets - allPendingCommitOffsets <- committer.getPendingCommits.map(_.offsets.toSeq) // TODO toSeq efficient enough? + committedOffsets <- committer.getCommittedOffsets + latestPendingCommitOffsets <- committer.getPendingCommits.map(_.offsets) streamResults <- ZIO.foreach(streamsToEnd) { stream => for { @@ -83,7 +83,7 @@ private[internal] class RebalanceCoordinator( endOffset match { case Some(endOffset) if committedOffsets.contains(stream.tp, endOffset.offset) => EndOffsetCommitted - case Some(endOffset) if allPendingCommitOffsets.contains((stream.tp, endOffset.offset)) => + case Some(endOffset) if latestPendingCommitOffsets.get(stream.tp).contains(endOffset.offset) => EndOffsetCommitPending case _ => EndOffsetNotCommitted } @@ -146,7 +146,6 @@ private[internal] class RebalanceCoordinator( // - Collect all these new (pending) commits. // - repeat the above until: // - All streams that were ended have completed their work, and - // (TODO is the pending part still true? Code did not reflect that) // - we have seen a completed or pending commit for all end-offsets. // An end-offset of a stream is the offset of the last record given to that stream. // - Do a single sync commit without any offsets, this has the side-effect of blocking until all From 4acefdf504d7c748adc4baa0577f8db017009925 Mon Sep 17 00:00:00 2001 From: Steven Vroonland Date: Wed, 13 Nov 2024 13:18:39 +0100 Subject: [PATCH 32/57] Move CommittOffsets to Committer as it is part of its interface --- .../consumer/internal/CommitOffsetsSpec.scala | 3 +- .../internal/RebalanceCoordinatorSpec.scala | 3 +- .../kafka/consumer/internal/Committer.scala | 54 +++++++++++++++++-- .../consumer/internal/LiveCommitter.scala | 53 +++--------------- 4 files changed, 61 insertions(+), 52 deletions(-) diff --git a/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/CommitOffsetsSpec.scala b/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/CommitOffsetsSpec.scala index b1e6ee9bb..ffc79101e 100644 --- a/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/CommitOffsetsSpec.scala +++ b/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/CommitOffsetsSpec.scala @@ -3,7 +3,8 @@ package zio.kafka.consumer.internal import org.apache.kafka.clients.consumer.OffsetAndMetadata import org.apache.kafka.common.TopicPartition import zio._ -import zio.kafka.consumer.internal.LiveCommitter.{ Commit, CommitOffsets } +import zio.kafka.consumer.internal.Committer.CommitOffsets +import zio.kafka.consumer.internal.LiveCommitter.Commit import zio.test._ object CommitOffsetsSpec extends ZIOSpecDefault { diff --git a/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/RebalanceCoordinatorSpec.scala b/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/RebalanceCoordinatorSpec.scala index add83ecd5..b04e8f61f 100644 --- a/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/RebalanceCoordinatorSpec.scala +++ b/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/RebalanceCoordinatorSpec.scala @@ -4,7 +4,8 @@ import org.apache.kafka.clients.consumer._ import org.apache.kafka.common.TopicPartition import zio.kafka.ZIOSpecDefaultSlf4j import zio.kafka.consumer.diagnostics.Diagnostics -import zio.kafka.consumer.internal.LiveCommitter.{ Commit, CommitOffsets } +import zio.kafka.consumer.internal.Committer.CommitOffsets +import zio.kafka.consumer.internal.LiveCommitter.Commit import zio.kafka.consumer.internal.RebalanceCoordinator.RebalanceEvent import zio.kafka.consumer.internal.Runloop.ByteArrayCommittableRecord import zio.kafka.consumer.{ CommittableRecord, ConsumerSettings } diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Committer.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Committer.scala index 22263f5ae..331ed2d48 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Committer.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Committer.scala @@ -2,12 +2,15 @@ package zio.kafka.consumer.internal import org.apache.kafka.clients.consumer.{ OffsetAndMetadata, OffsetCommitCallback } import org.apache.kafka.common.TopicPartition -import zio.kafka.consumer.internal.LiveCommitter.CommitOffsets -import zio.{ Task, UIO } +import zio.kafka.consumer.internal.Committer.CommitOffsets +import zio.kafka.consumer.internal.LiveCommitter.Commit +import zio.{ Chunk, Task, UIO } +import java.lang.Math.max import java.util.{ Map => JavaMap } +import scala.collection.mutable -trait Committer { +private[internal] trait Committer { val commit: Map[TopicPartition, OffsetAndMetadata] => Task[Unit] /** @@ -37,3 +40,48 @@ trait Committer { def getCommittedOffsets: UIO[CommitOffsets] } + +object Committer { + + // package private for unit testing + private[internal] final case class CommitOffsets(offsets: Map[TopicPartition, Long]) { + + /** Returns an estimate of the total offset increase, and a new `CommitOffsets` with the given offsets added. */ + def addCommits(c: Chunk[Commit]): (Long, CommitOffsets) = { + val updatedOffsets = mutable.Map.empty[TopicPartition, Long] + updatedOffsets.sizeHint(offsets.size) + updatedOffsets ++= offsets + var offsetIncrease = 0L + c.foreach { commit => + commit.offsets.foreach { case (tp, offsetAndMeta) => + val offset = offsetAndMeta.offset() + val maxOffset = updatedOffsets.get(tp) match { + case Some(existingOffset) => + offsetIncrease += max(0L, offset - existingOffset) + max(existingOffset, offset) + case None => + // This partition was not committed to from this consumer yet. Therefore we do not know the offset + // increase. A good estimate would be the poll size for this consumer, another okayish estimate is 0. + // Lets go with the simplest for now: ```offsetIncrease += 0``` + offset + } + updatedOffsets += tp -> maxOffset + } + } + (offsetIncrease, CommitOffsets(offsets = updatedOffsets.toMap)) + } + + def keepPartitions(tps: Set[TopicPartition]): CommitOffsets = + CommitOffsets(offsets.filter { case (tp, _) => tps.contains(tp) }) + + def contains(tp: TopicPartition, offset: Long): Boolean = + offsets.get(tp).exists(_ >= offset) + + def get(tp: TopicPartition): Option[Long] = offsets.get(tp) + } + + private[internal] object CommitOffsets { + val empty: CommitOffsets = CommitOffsets(Map.empty) + } + +} diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/LiveCommitter.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/LiveCommitter.scala index 8df5fcc36..04726e3c9 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/LiveCommitter.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/LiveCommitter.scala @@ -1,15 +1,15 @@ package zio.kafka.consumer.internal -import org.apache.kafka.clients.consumer.{ OffsetAndMetadata, OffsetCommitCallback } +import org.apache.kafka.clients.consumer.{OffsetAndMetadata, OffsetCommitCallback} import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.errors.RebalanceInProgressException import zio.kafka.consumer.Consumer.CommitTimeout -import zio.kafka.consumer.diagnostics.{ DiagnosticEvent, Diagnostics } -import zio.kafka.consumer.internal.LiveCommitter.{ Commit, CommitOffsets } -import zio.{ durationLong, Chunk, Duration, Exit, Promise, Queue, Ref, Runtime, Scope, Task, UIO, Unsafe, ZIO } +import zio.kafka.consumer.diagnostics.{DiagnosticEvent, Diagnostics} +import zio.kafka.consumer.internal.Committer.CommitOffsets +import zio.kafka.consumer.internal.LiveCommitter.Commit +import zio.{Chunk, Duration, Exit, Promise, Queue, Ref, Runtime, Scope, Task, UIO, Unsafe, ZIO, durationLong} -import java.lang.Math.max import java.util -import java.util.{ Map => JavaMap } +import java.util.{Map => JavaMap} import scala.collection.mutable import scala.jdk.CollectionConverters._ @@ -166,45 +166,4 @@ private[internal] object LiveCommitter { @inline def isPending: UIO[Boolean] = isDone.negate } - // package private for unit testing - private[internal] final case class CommitOffsets(offsets: Map[TopicPartition, Long]) { - - /** Returns an estimate of the total offset increase, and a new `CommitOffsets` with the given offsets added. */ - def addCommits(c: Chunk[Commit]): (Long, CommitOffsets) = { - val updatedOffsets = mutable.Map.empty[TopicPartition, Long] - updatedOffsets.sizeHint(offsets.size) - updatedOffsets ++= offsets - var offsetIncrease = 0L - c.foreach { commit => - commit.offsets.foreach { case (tp, offsetAndMeta) => - val offset = offsetAndMeta.offset() - val maxOffset = updatedOffsets.get(tp) match { - case Some(existingOffset) => - offsetIncrease += max(0L, offset - existingOffset) - max(existingOffset, offset) - case None => - // This partition was not committed to from this consumer yet. Therefore we do not know the offset - // increase. A good estimate would be the poll size for this consumer, another okayish estimate is 0. - // Lets go with the simplest for now: ```offsetIncrease += 0``` - offset - } - updatedOffsets += tp -> maxOffset - } - } - (offsetIncrease, CommitOffsets(offsets = updatedOffsets.toMap)) - } - - def keepPartitions(tps: Set[TopicPartition]): CommitOffsets = - CommitOffsets(offsets.filter { case (tp, _) => tps.contains(tp) }) - - def contains(tp: TopicPartition, offset: Long): Boolean = - offsets.get(tp).exists(_ >= offset) - - def get(tp: TopicPartition): Option[Long] = offsets.get(tp) - } - - private[internal] object CommitOffsets { - val empty: CommitOffsets = CommitOffsets(Map.empty) - } - } From 0fd4114e131b6c78926c31f08af190f60908fa7a Mon Sep 17 00:00:00 2001 From: Steven Vroonland Date: Wed, 13 Nov 2024 13:26:04 +0100 Subject: [PATCH 33/57] Cleanup --- .../kafka/consumer/internal/CommitterSpec.scala | 13 +------------ .../internal/RebalanceCoordinatorSpec.scala | 17 ++--------------- .../zio/kafka/consumer/internal/Committer.scala | 6 ++---- 3 files changed, 5 insertions(+), 31 deletions(-) diff --git a/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/CommitterSpec.scala b/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/CommitterSpec.scala index df24126cc..b855febb8 100644 --- a/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/CommitterSpec.scala +++ b/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/CommitterSpec.scala @@ -10,19 +10,8 @@ import zio.{ durationInt, Promise, UIO, ZIO } import java.util.{ Map => JavaMap } import scala.jdk.CollectionConverters.MapHasAsJava -/** - * Committer should: - * - execute a successful commit (+ 1) - * - communicate back a failed commit - * - automatically retry a commit when rebalancing - * - batch commits from multiple partitions and multiple offsets and confirm each one of them - * - keep track of pending commits (clear before and after) - * - keep track of committed offsets - * - clean committed offsets of no-longer assigned partitions - * - publish commit metrics - */ object CommitterSpec extends ZIOSpecDefault { - val mockMetrics = new ConsumerMetrics { + private val mockMetrics = new ConsumerMetrics { override def observePoll(resumedCount: Int, pausedCount: Int, latency: zio.Duration, pollSize: Int): UIO[Unit] = ZIO.unit diff --git a/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/RebalanceCoordinatorSpec.scala b/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/RebalanceCoordinatorSpec.scala index b04e8f61f..78da09a45 100644 --- a/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/RebalanceCoordinatorSpec.scala +++ b/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/RebalanceCoordinatorSpec.scala @@ -81,25 +81,15 @@ object RebalanceCoordinatorSpec extends ZIOSpecDefaultSlf4j { test("should wait for the last pulled offset to commit") { for { lastEvent <- Ref.make(RebalanceCoordinator.RebalanceEvent.None) - // Mock consumer that does not complete commit callbacks until commitSync is called consumer = new BinaryMockConsumer(OffsetResetStrategy.LATEST) { - var callback: OffsetCommitCallback = null - var offsets: util.Map[TopicPartition, OffsetAndMetadata] = null - override def commitAsync( offsets: util.Map[TopicPartition, OffsetAndMetadata], callback: OffsetCommitCallback - ): Unit = { + ): Unit = // Do nothing during rebalancing if (callback != null) callback.onComplete(offsets, null) - if (!offsets.isEmpty) { - this.callback = callback - this.offsets = offsets - } - } - override def commitSync(): Unit = - callback.onComplete(offsets, null) + override def commitSync(): Unit = () } tp = new TopicPartition("topic", 0) streamControl <- makeStreamControl(tp) @@ -119,7 +109,6 @@ object RebalanceCoordinatorSpec extends ZIOSpecDefaultSlf4j { new TopicPartition("topic", record.partition) -> new OffsetAndMetadata(record.offset.offset, null) ) ) - .debug(s"Done commit for ${record.offset.offset}") ) .runDrain .forkScoped @@ -135,7 +124,6 @@ object RebalanceCoordinatorSpec extends ZIOSpecDefaultSlf4j { _ <- streamDrain.join } yield assertCompletes }, - // TODO something with driving commits during waiting test("should continue if waiting for the stream to continue has timed out") { for { lastEvent <- Ref.make(RebalanceCoordinator.RebalanceEvent.None) @@ -172,7 +160,6 @@ object RebalanceCoordinatorSpec extends ZIOSpecDefaultSlf4j { } yield assertCompletes } ) - // TODO something with driving commits during waiting ) @@ TestAspect.withLiveClock private def makeStreamControl(tp: TopicPartition): UIO[PartitionStreamControl] = diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Committer.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Committer.scala index 331ed2d48..f57f969d3 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Committer.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Committer.scala @@ -41,10 +41,8 @@ private[internal] trait Committer { def getCommittedOffsets: UIO[CommitOffsets] } -object Committer { - - // package private for unit testing - private[internal] final case class CommitOffsets(offsets: Map[TopicPartition, Long]) { +private[internal] object Committer { + final case class CommitOffsets(offsets: Map[TopicPartition, Long]) { /** Returns an estimate of the total offset increase, and a new `CommitOffsets` with the given offsets added. */ def addCommits(c: Chunk[Commit]): (Long, CommitOffsets) = { From 3915b08792da0c5cf72ecfc905e460a4c96cb2a2 Mon Sep 17 00:00:00 2001 From: Steven Vroonland Date: Wed, 13 Nov 2024 13:39:11 +0100 Subject: [PATCH 34/57] cFix doc --- .../kafka/consumer/internal/Committer.scala | 5 +++-- .../consumer/internal/LiveCommitter.scala | 18 ++++-------------- 2 files changed, 7 insertions(+), 16 deletions(-) diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Committer.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Committer.scala index f57f969d3..dc2864fde 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Committer.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Committer.scala @@ -18,8 +18,9 @@ private[internal] trait Committer { * * If the queue is empty, nothing happens, unless executeOnEmpty is true. * - * @param consumer - * Consumer with exclusive access + * @param commitAsync + * Function 'commitAsync' on the KafkaConsumer. This is isolated from the whole KafkaConsumer for testing purposes. + * The caller should ensure exclusive access to the KafkaConsumer. * @param executeOnEmpty * Execute commitAsync() even if there are no commits */ diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/LiveCommitter.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/LiveCommitter.scala index 04726e3c9..9b4ddc53c 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/LiveCommitter.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/LiveCommitter.scala @@ -1,15 +1,15 @@ package zio.kafka.consumer.internal -import org.apache.kafka.clients.consumer.{OffsetAndMetadata, OffsetCommitCallback} +import org.apache.kafka.clients.consumer.{ OffsetAndMetadata, OffsetCommitCallback } import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.errors.RebalanceInProgressException import zio.kafka.consumer.Consumer.CommitTimeout -import zio.kafka.consumer.diagnostics.{DiagnosticEvent, Diagnostics} +import zio.kafka.consumer.diagnostics.{ DiagnosticEvent, Diagnostics } import zio.kafka.consumer.internal.Committer.CommitOffsets import zio.kafka.consumer.internal.LiveCommitter.Commit -import zio.{Chunk, Duration, Exit, Promise, Queue, Ref, Runtime, Scope, Task, UIO, Unsafe, ZIO, durationLong} +import zio.{ durationLong, Chunk, Duration, Exit, Promise, Queue, Ref, Runtime, Scope, Task, UIO, Unsafe, ZIO } import java.util -import java.util.{Map => JavaMap} +import java.util.{ Map => JavaMap } import scala.collection.mutable import scala.jdk.CollectionConverters._ @@ -39,16 +39,6 @@ private[consumer] final class LiveCommitter( _ <- consumerMetrics.observeCommit(latency) } yield () - /** - * Takes commits from the queue, commits them and adds them to pending commits - * - * If the queue is empty, nothing happens, unless executeOnEmpty is true. - * - * @param consumer - * Consumer with exclusive access - * @param executeOnEmpty - * Execute commitAsync() even if there are no commits - */ override def processQueuedCommits( commitAsync: (JavaMap[TopicPartition, OffsetAndMetadata], OffsetCommitCallback) => Task[Unit], executeOnEmpty: Boolean = false From e86d757375479ac1fe7d4ff1689e3adfcbc90193 Mon Sep 17 00:00:00 2001 From: Steven Vroonland Date: Wed, 13 Nov 2024 13:44:25 +0100 Subject: [PATCH 35/57] Formatting --- .../src/main/scala/zio/kafka/bench/ConsumerBenchmark.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/zio-kafka-bench/src/main/scala/zio/kafka/bench/ConsumerBenchmark.scala b/zio-kafka-bench/src/main/scala/zio/kafka/bench/ConsumerBenchmark.scala index a652fc3d5..2695b05cb 100644 --- a/zio-kafka-bench/src/main/scala/zio/kafka/bench/ConsumerBenchmark.scala +++ b/zio-kafka-bench/src/main/scala/zio/kafka/bench/ConsumerBenchmark.scala @@ -35,7 +35,7 @@ class ConsumerBenchmark extends ZioBenchmark[Kafka with Producer] { consumerSettings( randomThing("client"), Some(randomThing("group")), - `max.poll.records` = 1000, + `max.poll.records` = 1000 ).map(_.withPartitionPreFetchBufferLimit(8192)) ) ++ ZLayer.succeed(Diagnostics.NoOp)) >>> Consumer.live From 77e54269b0246f1afd55fb6aa99f53ea3c5099fe Mon Sep 17 00:00:00 2001 From: Steven Vroonland Date: Sun, 24 Nov 2024 13:53:58 +0100 Subject: [PATCH 36/57] Fix log message --- .../src/main/scala/zio/kafka/consumer/internal/Runloop.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala index 192d1b651..1530dbb16 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala @@ -206,10 +206,11 @@ private[consumer] final class Runloop private ( private def handlePoll(state: State): Task[State] = { for { - partitionsToFetch <- settings.fetchStrategy.selectPartitionsToFetch(state.assignedStreams) + partitionsToFetch <- settings.fetchStrategy.selectPartitionsToFetch(state.assignedStreams) + pendingCommitCount <- committer.pendingCommitCount _ <- ZIO.logDebug( s"Starting poll with ${state.pendingRequests.size} pending requests and" + - s" ${committer.pendingCommitCount} pending commits," + + s" ${pendingCommitCount} pending commits," + s" resuming $partitionsToFetch partitions" ) _ <- currentStateRef.set(state) From b0e0aae164d9c3d926e91e98759a502c2505a6db Mon Sep 17 00:00:00 2001 From: Steven Vroonland Date: Sun, 24 Nov 2024 16:28:56 +0100 Subject: [PATCH 37/57] WIP --- .../consumer/internal/CommitterSpec.scala | 94 ++++++--------- .../internal/RebalanceCoordinatorSpec.scala | 32 ++--- .../kafka/consumer/internal/Committer.scala | 5 +- .../consumer/internal/LiveCommitter.scala | 110 +++++++----------- .../internal/RebalanceCoordinator.scala | 26 ++++- .../zio/kafka/consumer/internal/Runloop.scala | 29 ++++- 6 files changed, 143 insertions(+), 153 deletions(-) diff --git a/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/CommitterSpec.scala b/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/CommitterSpec.scala index b855febb8..284da6af2 100644 --- a/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/CommitterSpec.scala +++ b/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/CommitterSpec.scala @@ -5,7 +5,7 @@ import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.errors.RebalanceInProgressException import zio.kafka.consumer.diagnostics.Diagnostics import zio.test._ -import zio.{ durationInt, Promise, UIO, ZIO } +import zio.{ durationInt, Promise, Queue, UIO, ZIO } import java.util.{ Map => JavaMap } import scala.jdk.CollectionConverters.MapHasAsJava @@ -35,15 +35,13 @@ object CommitterSpec extends ZIOSpecDefault { override def spec = suite("Committer")( test("signals that a new commit is available") { for { - runtime <- ZIO.runtime[Any] commitAvailable <- Promise.make[Nothing, Unit] committer <- LiveCommitter .make( 10.seconds, Diagnostics.NoOp, mockMetrics, - onCommitAvailable = commitAvailable.succeed(()).unit, - sameThreadRuntime = runtime + onCommitAvailable = commitAvailable.succeed(()).unit ) tp = new TopicPartition("topic", 0) _ <- committer.commit(Map(tp -> new OffsetAndMetadata(0))).forkScoped @@ -52,81 +50,69 @@ object CommitterSpec extends ZIOSpecDefault { }, test("handles a successful commit by completing the commit effect") { for { - runtime <- ZIO.runtime[Any] commitAvailable <- Promise.make[Nothing, Unit] committer <- LiveCommitter.make( 10.seconds, Diagnostics.NoOp, mockMetrics, - onCommitAvailable = commitAvailable.succeed(()).unit, - sameThreadRuntime = runtime + onCommitAvailable = commitAvailable.succeed(()).unit ) tp = new TopicPartition("topic", 0) commitFiber <- committer.commit(Map(tp -> new OffsetAndMetadata(0))).forkScoped _ <- commitAvailable.await - _ <- committer.processQueuedCommits((offsets, callback) => ZIO.attempt(callback.onComplete(offsets, null))) - _ <- commitFiber.join + _ <- committer.processQueuedCommits(offsets => ZIO.succeed(offsets)) + _ <- commitFiber.join } yield assertCompletes }, test("handles a failed commit by completing the commit effect with a failure") { for { - runtime <- ZIO.runtime[Any] commitAvailable <- Promise.make[Nothing, Unit] committer <- LiveCommitter.make( 10.seconds, Diagnostics.NoOp, mockMetrics, - onCommitAvailable = commitAvailable.succeed(()).unit, - sameThreadRuntime = runtime + onCommitAvailable = commitAvailable.succeed(()).unit ) tp = new TopicPartition("topic", 0) commitFiber <- committer.commit(Map(tp -> new OffsetAndMetadata(0))).forkScoped _ <- commitAvailable.await - _ <- committer.processQueuedCommits((offsets, callback) => - ZIO.attempt(callback.onComplete(offsets, new RuntimeException("Commit failed"))) - ) - result <- commitFiber.await + _ <- committer.processQueuedCommits(_ => ZIO.fail(new RuntimeException("Commit failed"))) + result <- commitFiber.await } yield assertTrue(result.isFailure) }, test("retries when rebalancing") { for { - runtime <- ZIO.runtime[Any] - commitAvailable <- Promise.make[Nothing, Unit] + commitAvailable <- Queue.bounded[Unit](1) committer <- LiveCommitter.make( 10.seconds, Diagnostics.NoOp, mockMetrics, - onCommitAvailable = commitAvailable.succeed(()).unit, - sameThreadRuntime = runtime + onCommitAvailable = commitAvailable.offer(()).unit ) tp = new TopicPartition("topic", 0) commitFiber <- committer.commit(Map(tp -> new OffsetAndMetadata(0))).forkScoped - _ <- commitAvailable.await - _ <- committer.processQueuedCommits((offsets, callback) => - ZIO.attempt(callback.onComplete(offsets, new RebalanceInProgressException("Rebalance in progress"))) - ) - _ <- committer.processQueuedCommits((offsets, callback) => ZIO.attempt(callback.onComplete(offsets, null))) - result <- commitFiber.await - } yield assertTrue(result.isSuccess) + _ <- commitAvailable.take + _ <- + committer.processQueuedCommits(_ => ZIO.fail(new RebalanceInProgressException("Rebalance in progress"))) + _ <- commitAvailable.take + _ <- committer.processQueuedCommits(offsets => ZIO.succeed(offsets)) + _ <- commitFiber.join + } yield assertCompletes }, test("adds 1 to the committed last offset") { for { - runtime <- ZIO.runtime[Any] commitAvailable <- Promise.make[Nothing, Unit] committer <- LiveCommitter.make( 10.seconds, Diagnostics.NoOp, mockMetrics, - onCommitAvailable = commitAvailable.succeed(()).unit, - sameThreadRuntime = runtime + onCommitAvailable = commitAvailable.succeed(()).unit ) tp = new TopicPartition("topic", 0) _ <- committer.commit(Map(tp -> new OffsetAndMetadata(1))).forkScoped _ <- commitAvailable.await committedOffsets <- Promise.make[Nothing, JavaMap[TopicPartition, OffsetAndMetadata]] - _ <- committer.processQueuedCommits((offsets, callback) => - committedOffsets.succeed(offsets) *> ZIO.attempt(callback.onComplete(offsets, null)) - ) + _ <- committer.processQueuedCommits(offsets => committedOffsets.succeed(offsets.asJava).as(offsets)) offsetsCommitted <- committedOffsets.await } yield assertTrue( offsetsCommitted == Map(tp -> new OffsetAndMetadata(2)).asJava @@ -134,14 +120,12 @@ object CommitterSpec extends ZIOSpecDefault { }, test("batches commits from multiple partitions and offsets") { for { - runtime <- ZIO.runtime[Any] commitAvailable <- Promise.make[Nothing, Unit] committer <- LiveCommitter.make( 10.seconds, Diagnostics.NoOp, mockMetrics, - onCommitAvailable = commitAvailable.succeed(()).unit, - sameThreadRuntime = runtime + onCommitAvailable = commitAvailable.succeed(()).unit ) tp = new TopicPartition("topic", 0) tp2 = new TopicPartition("topic", 1) @@ -150,10 +134,8 @@ object CommitterSpec extends ZIOSpecDefault { commitFiber3 <- committer.commit(Map(tp2 -> new OffsetAndMetadata(3))).forkScoped _ <- commitAvailable.await committedOffsets <- Promise.make[Nothing, JavaMap[TopicPartition, OffsetAndMetadata]] - _ <- committer.processQueuedCommits((offsets, callback) => - committedOffsets.succeed(offsets) *> ZIO.attempt(callback.onComplete(offsets, null)) - ) - _ <- commitFiber1.join zip commitFiber2.join zip commitFiber3.join + _ <- committer.processQueuedCommits(offsets => committedOffsets.succeed(offsets.asJava).as(offsets)) + _ <- commitFiber1.join zip commitFiber2.join zip commitFiber3.join offsetsCommitted <- committedOffsets.await } yield assertTrue( offsetsCommitted == Map(tp -> new OffsetAndMetadata(3), tp2 -> new OffsetAndMetadata(4)).asJava @@ -161,19 +143,17 @@ object CommitterSpec extends ZIOSpecDefault { }, test("keeps track of pending commits") { for { - runtime <- ZIO.runtime[Any] commitAvailable <- Promise.make[Nothing, Unit] committer <- LiveCommitter.make( 10.seconds, Diagnostics.NoOp, mockMetrics, - onCommitAvailable = commitAvailable.succeed(()).unit, - sameThreadRuntime = runtime + onCommitAvailable = commitAvailable.succeed(()).unit ) tp = new TopicPartition("topic", 0) - commitFiber <- committer.commit(Map(tp -> new OffsetAndMetadata(0))).forkScoped - _ <- commitAvailable.await - _ <- committer.processQueuedCommits((offsets, callback) => ZIO.attempt(callback.onComplete(offsets, null))) + commitFiber <- committer.commit(Map(tp -> new OffsetAndMetadata(0))).forkScoped + _ <- commitAvailable.await + _ <- committer.processQueuedCommits(ZIO.succeed(_)) pendingCommitsDuringCommit <- committer.pendingCommitCount _ <- committer.updatePendingCommitsAfterPoll pendingCommitsAfterCommit <- committer.pendingCommitCount @@ -182,39 +162,35 @@ object CommitterSpec extends ZIOSpecDefault { }, test("keep track of committed offsets") { for { - runtime <- ZIO.runtime[Any] commitAvailable <- Promise.make[Nothing, Unit] committer <- LiveCommitter.make( 10.seconds, Diagnostics.NoOp, mockMetrics, - onCommitAvailable = commitAvailable.succeed(()).unit, - sameThreadRuntime = runtime + onCommitAvailable = commitAvailable.succeed(()).unit ) tp = new TopicPartition("topic", 0) - commitFiber <- committer.commit(Map(tp -> new OffsetAndMetadata(0))).forkScoped - _ <- commitAvailable.await - _ <- committer.processQueuedCommits((offsets, callback) => ZIO.attempt(callback.onComplete(offsets, null))) + commitFiber <- committer.commit(Map(tp -> new OffsetAndMetadata(0))).forkScoped + _ <- commitAvailable.await + _ <- committer.processQueuedCommits(ZIO.succeed(_)) committedOffsets <- committer.getCommittedOffsets _ <- commitFiber.join } yield assertTrue(committedOffsets.offsets == Map(tp -> 0L)) }, test("clean committed offsets of no-longer assigned partitions") { for { - runtime <- ZIO.runtime[Any] commitAvailable <- Promise.make[Nothing, Unit] committer <- LiveCommitter.make( 10.seconds, Diagnostics.NoOp, mockMetrics, - onCommitAvailable = commitAvailable.succeed(()).unit, - sameThreadRuntime = runtime + onCommitAvailable = commitAvailable.succeed(()).unit ) tp = new TopicPartition("topic", 0) - commitFiber <- committer.commit(Map(tp -> new OffsetAndMetadata(0))).forkScoped - _ <- commitAvailable.await - _ <- committer.processQueuedCommits((offsets, callback) => ZIO.attempt(callback.onComplete(offsets, null))) - _ <- committer.pruneCommittedOffsets(Set.empty) + commitFiber <- committer.commit(Map(tp -> new OffsetAndMetadata(0))).forkScoped + _ <- commitAvailable.await + _ <- committer.processQueuedCommits(ZIO.succeed(_)) + _ <- committer.pruneCommittedOffsets(Set.empty) committedOffsets <- committer.getCommittedOffsets _ <- commitFiber.join } yield assertTrue(committedOffsets.offsets.isEmpty) diff --git a/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/RebalanceCoordinatorSpec.scala b/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/RebalanceCoordinatorSpec.scala index 78da09a45..db312fdd4 100644 --- a/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/RebalanceCoordinatorSpec.scala +++ b/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/RebalanceCoordinatorSpec.scala @@ -96,8 +96,7 @@ object RebalanceCoordinatorSpec extends ZIOSpecDefaultSlf4j { records = createTestRecords(3) recordsPulled <- Promise.make[Nothing, Unit] _ <- streamControl.offerRecords(records) - runtime <- ZIO.runtime[Any] - committer <- LiveCommitter.make(10.seconds, Diagnostics.NoOp, mockMetrics, ZIO.unit, runtime) + committer <- LiveCommitter.make(10.seconds, Diagnostics.NoOp, mockMetrics, ZIO.unit) streamDrain <- streamControl.stream @@ -173,16 +172,23 @@ object RebalanceCoordinatorSpec extends ZIOSpecDefaultSlf4j { settings: ConsumerSettings = ConsumerSettings(List("")).withCommitTimeout(1.second), rebalanceSafeCommits: Boolean = false ): ZIO[Scope, Throwable, RebalanceCoordinator] = - Semaphore.make(1).map(new ConsumerAccess(mockConsumer, _)).map { consumerAccess => - new RebalanceCoordinator( - lastEvent, - settings.withRebalanceSafeCommits(rebalanceSafeCommits), - consumerAccess, - 5.seconds, - ZIO.succeed(assignedStreams), - committer - ) - } + Semaphore + .make(1) + .map(new ConsumerAccess(mockConsumer, _)) + .flatMap { consumerAccess => + ZIO.runtime[Any].map { sameThreadRuntime => + new RebalanceCoordinator( + lastEvent, + settings.withRebalanceSafeCommits(rebalanceSafeCommits), + consumerAccess, + 5.seconds, + ZIO.succeed(assignedStreams), + committer, + sameThreadRuntime = sameThreadRuntime + ) + } + } + .provideLayer(SameThreadRuntimeLayer) private def createTestRecords(count: Int): Chunk[ByteArrayCommittableRecord] = Chunk.fromIterable( @@ -206,7 +212,7 @@ abstract class MockCommitter extends Committer { override val commit: Map[TopicPartition, OffsetAndMetadata] => Task[Unit] = _ => ZIO.unit override def processQueuedCommits( - commitAsync: (java.util.Map[TopicPartition, OffsetAndMetadata], OffsetCommitCallback) => zio.Task[Unit], + commitAsync: Map[TopicPartition, OffsetAndMetadata] => Task[Map[TopicPartition, OffsetAndMetadata]], executeOnEmpty: Boolean ): zio.Task[Unit] = ZIO.unit override def queueSize: UIO[Int] = ZIO.succeed(0) diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Committer.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Committer.scala index dc2864fde..84324db29 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Committer.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Committer.scala @@ -1,13 +1,12 @@ package zio.kafka.consumer.internal -import org.apache.kafka.clients.consumer.{ OffsetAndMetadata, OffsetCommitCallback } +import org.apache.kafka.clients.consumer.OffsetAndMetadata import org.apache.kafka.common.TopicPartition import zio.kafka.consumer.internal.Committer.CommitOffsets import zio.kafka.consumer.internal.LiveCommitter.Commit import zio.{ Chunk, Task, UIO } import java.lang.Math.max -import java.util.{ Map => JavaMap } import scala.collection.mutable private[internal] trait Committer { @@ -25,7 +24,7 @@ private[internal] trait Committer { * Execute commitAsync() even if there are no commits */ def processQueuedCommits( - commitAsync: (JavaMap[TopicPartition, OffsetAndMetadata], OffsetCommitCallback) => Task[Unit], + commitAsync: Map[TopicPartition, OffsetAndMetadata] => Task[Map[TopicPartition, OffsetAndMetadata]], executeOnEmpty: Boolean = false ): Task[Unit] diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/LiveCommitter.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/LiveCommitter.scala index 9b4ddc53c..da2970156 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/LiveCommitter.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/LiveCommitter.scala @@ -1,17 +1,14 @@ package zio.kafka.consumer.internal -import org.apache.kafka.clients.consumer.{ OffsetAndMetadata, OffsetCommitCallback } +import org.apache.kafka.clients.consumer.OffsetAndMetadata import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.errors.RebalanceInProgressException import zio.kafka.consumer.Consumer.CommitTimeout import zio.kafka.consumer.diagnostics.{ DiagnosticEvent, Diagnostics } import zio.kafka.consumer.internal.Committer.CommitOffsets import zio.kafka.consumer.internal.LiveCommitter.Commit -import zio.{ durationLong, Chunk, Duration, Exit, Promise, Queue, Ref, Runtime, Scope, Task, UIO, Unsafe, ZIO } +import zio.{ durationLong, Cause, Chunk, Duration, Exit, Promise, Queue, Ref, Scope, Task, UIO, ZIO } -import java.util -import java.util.{ Map => JavaMap } import scala.collection.mutable -import scala.jdk.CollectionConverters._ private[consumer] final class LiveCommitter( commitQueue: Queue[Commit], @@ -20,7 +17,6 @@ private[consumer] final class LiveCommitter( consumerMetrics: ConsumerMetrics, onCommitAvailable: UIO[Unit], committedOffsetsRef: Ref[CommitOffsets], - sameThreadRuntime: Runtime[Any], pendingCommits: Ref[Chunk[Commit]] ) extends Committer { @@ -40,75 +36,53 @@ private[consumer] final class LiveCommitter( } yield () override def processQueuedCommits( - commitAsync: (JavaMap[TopicPartition, OffsetAndMetadata], OffsetCommitCallback) => Task[Unit], + commitAsyncZIO: Map[TopicPartition, OffsetAndMetadata] => Task[Map[TopicPartition, OffsetAndMetadata]], executeOnEmpty: Boolean = false ): Task[Unit] = for { commits <- commitQueue.takeAll _ <- ZIO.logDebug(s"Processing ${commits.size} commits") _ <- ZIO.unless(commits.isEmpty && !executeOnEmpty) { - val (offsets, callback, onFailure) = asyncCommitParameters(commits) + val offsets = commits + .foldLeft(mutable.Map.empty[TopicPartition, OffsetAndMetadata]) { case (acc, commit) => + commit.offsets.foreach { case (tp, offset) => + acc += (tp -> acc + .get(tp) + .map(current => if (current.offset() > offset.offset()) current else offset) + .getOrElse(offset)) + } + acc + } + .toMap + val offsetsWithMetaData = offsets.map { case (tp, offset) => + tp -> new OffsetAndMetadata(offset.offset + 1, offset.leaderEpoch, offset.metadata) + } + pendingCommits.update(_ ++ commits) *> - // We don't wait for the completion of the commit here, because it - // will only complete once we poll again. - commitAsync(offsets, callback) - .catchAll(onFailure) + ZIO.debug("Here!") *> + commitAsyncZIO(offsetsWithMetaData).timed.debug.flatMap { case (latency, offsetsWithMetaData) => + for { + offsetIncrease <- committedOffsetsRef.modify(_.addCommits(commits)) + _ <- consumerMetrics.observeAggregatedCommit(latency, offsetIncrease).when(commits.nonEmpty) + result <- ZIO.foreachDiscard(commits)(_.cont.done(Exit.unit)) + _ <- diagnostics.emit(DiagnosticEvent.Commit.Success(offsetsWithMetaData)) + } yield result + }.catchAllCause { + case Cause.Fail(_: RebalanceInProgressException, _) => + for { + _ <- ZIO.logDebug(s"Rebalance in progress, commit for offsets $offsets will be retried") + _ <- commitQueue.offerAll(commits) + _ <- onCommitAvailable + } yield () + case c => + ZIO.foreachDiscard(commits)(_.cont.done(Exit.fail(c.squash))) <* diagnostics.emit( + DiagnosticEvent.Commit.Failure(offsets, c.squash) + ) + }.debug + // We don't wait for the completion of the commit here, because it + // will only complete once we poll again. } } yield () - /** Merge commits and prepare parameters for calling `consumer.commitAsync`. */ - private def asyncCommitParameters( - commits: Chunk[Commit] - ): (JavaMap[TopicPartition, OffsetAndMetadata], OffsetCommitCallback, Throwable => UIO[Unit]) = { - val offsets = commits - .foldLeft(mutable.Map.empty[TopicPartition, OffsetAndMetadata]) { case (acc, commit) => - commit.offsets.foreach { case (tp, offset) => - acc += (tp -> acc - .get(tp) - .map(current => if (current.offset() > offset.offset()) current else offset) - .getOrElse(offset)) - } - acc - } - .toMap - val offsetsWithMetaData = offsets.map { case (tp, offset) => - tp -> new OffsetAndMetadata(offset.offset + 1, offset.leaderEpoch, offset.metadata) - } - val cont = (e: Exit[Throwable, Unit]) => ZIO.foreachDiscard(commits)(_.cont.done(e)) - // We assume the commit is started immediately after returning from this method. - val startTime = java.lang.System.nanoTime() - val onSuccess = { - val endTime = java.lang.System.nanoTime() - val latency = (endTime - startTime).nanoseconds - for { - offsetIncrease <- committedOffsetsRef.modify(_.addCommits(commits)) - _ <- consumerMetrics.observeAggregatedCommit(latency, offsetIncrease).when(commits.nonEmpty) - result <- cont(Exit.unit) - _ <- diagnostics.emit(DiagnosticEvent.Commit.Success(offsetsWithMetaData)) - } yield result - } - val onFailure: Throwable => UIO[Unit] = { - case _: RebalanceInProgressException => - for { - _ <- ZIO.logDebug(s"Rebalance in progress, commit for offsets $offsets will be retried") - _ <- commitQueue.offerAll(commits) - _ <- onCommitAvailable - } yield () - case err: Throwable => - cont(Exit.fail(err)) <* diagnostics.emit(DiagnosticEvent.Commit.Failure(offsetsWithMetaData, err)) - } - val callback = - new OffsetCommitCallback { - override def onComplete(offsets: util.Map[TopicPartition, OffsetAndMetadata], exception: Exception): Unit = - Unsafe.unsafe { implicit u => - sameThreadRuntime.unsafe.run { - if (exception eq null) onSuccess else onFailure(exception) - } - .getOrThrowFiberFailure() - } - } - (offsetsWithMetaData.asJava, callback, onFailure) - } - override def queueSize: UIO[Int] = commitQueue.size override def pendingCommitCount: UIO[Int] = pendingCommits.get.map(_.size) @@ -130,8 +104,7 @@ private[internal] object LiveCommitter { commitTimeout: Duration, diagnostics: Diagnostics, consumerMetrics: ConsumerMetrics, - onCommitAvailable: UIO[Unit], - sameThreadRuntime: Runtime[Any] + onCommitAvailable: UIO[Unit] ): ZIO[Scope, Nothing, LiveCommitter] = for { pendingCommits <- Ref.make(Chunk.empty[Commit]) commitQueue <- ZIO.acquireRelease(Queue.unbounded[Commit])(_.shutdown) @@ -143,7 +116,6 @@ private[internal] object LiveCommitter { consumerMetrics, onCommitAvailable, committedOffsetsRef, - sameThreadRuntime, pendingCommits ) diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/RebalanceCoordinator.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/RebalanceCoordinator.scala index 066bfdf15..cb43aadad 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/RebalanceCoordinator.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/RebalanceCoordinator.scala @@ -1,4 +1,5 @@ package zio.kafka.consumer.internal +import org.apache.kafka.clients.consumer.OffsetAndMetadata import org.apache.kafka.common.TopicPartition import zio.kafka.consumer.internal.ConsumerAccess.ByteArrayKafkaConsumer import zio.kafka.consumer.internal.RebalanceCoordinator.{ @@ -10,7 +11,10 @@ import zio.kafka.consumer.internal.RebalanceCoordinator.{ } import zio.kafka.consumer.{ ConsumerSettings, RebalanceListener } import zio.stream.ZStream -import zio.{ durationInt, Chunk, Duration, Ref, Task, UIO, ZIO } +import zio.{ durationInt, Chunk, Duration, Ref, Runtime, Task, UIO, ZIO } + +import java.util.{ Map => JavaMap } +import scala.jdk.CollectionConverters._ /** * The Runloop's RebalanceListener gets notified of partitions that are assigned, revoked and lost @@ -26,7 +30,8 @@ private[internal] class RebalanceCoordinator( consumer: ConsumerAccess, maxRebalanceDuration: Duration, getCurrentAssignedStreams: UIO[Chunk[PartitionStreamControl]], - committer: Committer + committer: Committer, + sameThreadRuntime: Runtime[Any] ) { private val commitTimeoutNanos = settings.commitTimeout.toNanos @@ -166,7 +171,7 @@ private[internal] class RebalanceCoordinator( // executeOnEmpty = true .tap(_ => committer.processQueuedCommits( - (offsets, callback) => ZIO.attempt(consumer.commitAsync(offsets, callback)), + offsets => commitAsyncZIO(consumer, offsets), executeOnEmpty = true ) ) @@ -181,6 +186,21 @@ private[internal] class RebalanceCoordinator( } yield () } + private def commitAsyncZIO( + consumer: ByteArrayKafkaConsumer, + offsets: Map[TopicPartition, OffsetAndMetadata] + ): Task[Map[TopicPartition, OffsetAndMetadata]] = + sameThreadRuntime.run { + ZIO.async[Any, Throwable, Map[TopicPartition, OffsetAndMetadata]]({ onDone => + consumer.commitAsync( + offsets.asJava, + (offsets: JavaMap[TopicPartition, OffsetAndMetadata], exception: Exception) => + if (exception == null) onDone(ZIO.succeed(offsets.asScala.toMap)) + else onDone(ZIO.fail(exception)) + ) + }) + } + // During a poll, the java kafka client might call each method of the rebalance listener 0 or 1 times. // We do not know the order in which the call-back methods are invoked. // diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala index 1530dbb16..f44c3fba6 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala @@ -14,6 +14,7 @@ import zio.kafka.consumer.internal.RunloopAccess.PartitionAssignment import zio.kafka.consumer.internal.RebalanceCoordinator.RebalanceEvent import zio.stream._ +import java.util.{ Map => JavaMap } import scala.jdk.CollectionConverters._ //noinspection SimplifyWhenInspection,SimplifyUnlessInspection @@ -493,9 +494,7 @@ private[consumer] final class Runloop private ( for { _ <- ZIO.logDebug(s"Processing ${commands.size} commands: ${commands.mkString(",")}") _ <- consumer.runloopAccess { consumer => - committer.processQueuedCommits((offsets, callback) => - ZIO.attempt(consumer.commitAsync(offsets, callback)) - ) + committer.processQueuedCommits(commitAsyncZIO(consumer, _)) } streamCommands = commands.collect { case cmd: RunloopCommand.StreamCommand => cmd } stateAfterCommands <- ZIO.foldLeft(streamCommands)(state)(handleCommand) @@ -514,6 +513,24 @@ private[consumer] final class Runloop private ( .onError(cause => partitionsHub.offer(Take.failCause(cause))) } + private def commitAsyncZIO( + consumer: ByteArrayKafkaConsumer, + offsets: Map[TopicPartition, OffsetAndMetadata] + ): Task[Map[TopicPartition, OffsetAndMetadata]] = + ZIO + .async[Any, Throwable, Map[TopicPartition, OffsetAndMetadata]]({ onDone => + print("Starting commitAsync") + consumer.commitAsync( + offsets.asJava, + (offsets: JavaMap[TopicPartition, OffsetAndMetadata], exception: Exception) => { + print("Ending commitAsync") + if (exception == null) onDone(ZIO.succeed(offsets.asScala.toMap)) + else onDone(ZIO.fail(exception)) + } + ) + }) + // TODO the async doesn't continue, probably because it's on the blocking runtime..? + def shouldPoll(state: State): UIO[Boolean] = committer.pendingCommitCount.map { pendingCommitCount => state.subscriptionState.isSubscribed && (state.pendingRequests.nonEmpty || pendingCommitCount > 0 || state.assignedStreams.isEmpty) @@ -594,8 +611,7 @@ object Runloop { settings.commitTimeout, diagnostics, metrics, - commandQueue.offer(RunloopCommand.CommitAvailable).unit, - sameThreadRuntime + commandQueue.offer(RunloopCommand.CommitAvailable).unit ) rebalanceCoordinator = new RebalanceCoordinator( lastRebalanceEvent, @@ -603,7 +619,8 @@ object Runloop { consumer, maxRebalanceDuration, currentStateRef.get.map(_.assignedStreams), - committer + committer, + sameThreadRuntime ) runloop = new Runloop( settings = settings, From 7cd3f523e5689ba74b173cdcbd1ee3b3523fb220 Mon Sep 17 00:00:00 2001 From: Steven Vroonland Date: Sun, 24 Nov 2024 16:41:23 +0100 Subject: [PATCH 38/57] Process review comments --- .../consumer/internal/CommitterSpec.scala | 4 ++-- .../internal/RebalanceCoordinatorSpec.scala | 20 ++++++++-------- .../kafka/consumer/internal/Committer.scala | 10 +++++--- .../consumer/internal/LiveCommitter.scala | 24 ++++++++++++------- .../internal/RebalanceCoordinator.scala | 12 ++++------ .../zio/kafka/consumer/internal/Runloop.scala | 17 ++++++------- 6 files changed, 48 insertions(+), 39 deletions(-) diff --git a/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/CommitterSpec.scala b/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/CommitterSpec.scala index b855febb8..7e9c5f0f2 100644 --- a/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/CommitterSpec.scala +++ b/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/CommitterSpec.scala @@ -175,7 +175,7 @@ object CommitterSpec extends ZIOSpecDefault { _ <- commitAvailable.await _ <- committer.processQueuedCommits((offsets, callback) => ZIO.attempt(callback.onComplete(offsets, null))) pendingCommitsDuringCommit <- committer.pendingCommitCount - _ <- committer.updatePendingCommitsAfterPoll + _ <- committer.cleanupPendingCommits pendingCommitsAfterCommit <- committer.pendingCommitCount _ <- commitFiber.join } yield assertTrue(pendingCommitsDuringCommit == 1 && pendingCommitsAfterCommit == 0) @@ -214,7 +214,7 @@ object CommitterSpec extends ZIOSpecDefault { commitFiber <- committer.commit(Map(tp -> new OffsetAndMetadata(0))).forkScoped _ <- commitAvailable.await _ <- committer.processQueuedCommits((offsets, callback) => ZIO.attempt(callback.onComplete(offsets, null))) - _ <- committer.pruneCommittedOffsets(Set.empty) + _ <- committer.keepCommitsForPartitions(Set.empty) committedOffsets <- committer.getCommittedOffsets _ <- commitFiber.join } yield assertTrue(committedOffsets.offsets.isEmpty) diff --git a/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/RebalanceCoordinatorSpec.scala b/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/RebalanceCoordinatorSpec.scala index 78da09a45..562db3f2c 100644 --- a/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/RebalanceCoordinatorSpec.scala +++ b/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/RebalanceCoordinatorSpec.scala @@ -41,7 +41,7 @@ object RebalanceCoordinatorSpec extends ZIOSpecDefaultSlf4j { def spec = suite("RunloopRebalanceListener")( test("should track assigned, revoked and lost partitions") { for { - lastEvent <- Ref.make(RebalanceCoordinator.RebalanceEvent.None) + lastEvent <- Ref.Synchronized.make(RebalanceCoordinator.RebalanceEvent.None) consumer = new BinaryMockConsumer(OffsetResetStrategy.LATEST) {} tp = new TopicPartition("topic", 0) tp2 = new TopicPartition("topic", 1) @@ -61,7 +61,7 @@ object RebalanceCoordinatorSpec extends ZIOSpecDefaultSlf4j { }, test("should end streams for revoked and lost partitions") { for { - lastEvent <- Ref.make(RebalanceCoordinator.RebalanceEvent.None) + lastEvent <- Ref.Synchronized.make(RebalanceCoordinator.RebalanceEvent.None) consumer = new BinaryMockConsumer(OffsetResetStrategy.LATEST) {} tp = new TopicPartition("topic", 0) tp2 = new TopicPartition("topic", 1) @@ -80,7 +80,7 @@ object RebalanceCoordinatorSpec extends ZIOSpecDefaultSlf4j { suite("rebalanceSafeCommits")( test("should wait for the last pulled offset to commit") { for { - lastEvent <- Ref.make(RebalanceCoordinator.RebalanceEvent.None) + lastEvent <- Ref.Synchronized.make(RebalanceCoordinator.RebalanceEvent.None) consumer = new BinaryMockConsumer(OffsetResetStrategy.LATEST) { override def commitAsync( offsets: util.Map[TopicPartition, OffsetAndMetadata], @@ -126,7 +126,7 @@ object RebalanceCoordinatorSpec extends ZIOSpecDefaultSlf4j { }, test("should continue if waiting for the stream to continue has timed out") { for { - lastEvent <- Ref.make(RebalanceCoordinator.RebalanceEvent.None) + lastEvent <- Ref.Synchronized.make(RebalanceCoordinator.RebalanceEvent.None) consumer = new BinaryMockConsumer(OffsetResetStrategy.LATEST) {} tp = new TopicPartition("topic", 0) streamControl <- makeStreamControl(tp) @@ -166,7 +166,7 @@ object RebalanceCoordinatorSpec extends ZIOSpecDefaultSlf4j { PartitionStreamControl.newPartitionStream(tp, ZIO.unit, Diagnostics.NoOp, 30.seconds) private def makeCoordinator( - lastEvent: Ref[RebalanceEvent], + lastEvent: Ref.Synchronized[RebalanceEvent], mockConsumer: BinaryMockConsumer, assignedStreams: Chunk[PartitionStreamControl] = Chunk.empty, committer: Committer = new MockCommitter {}, @@ -209,10 +209,10 @@ abstract class MockCommitter extends Committer { commitAsync: (java.util.Map[TopicPartition, OffsetAndMetadata], OffsetCommitCallback) => zio.Task[Unit], executeOnEmpty: Boolean ): zio.Task[Unit] = ZIO.unit - override def queueSize: UIO[Int] = ZIO.succeed(0) - override def pendingCommitCount: UIO[Int] = ZIO.succeed(0) - override def getPendingCommits: UIO[CommitOffsets] = ZIO.succeed(CommitOffsets.empty) - override def updatePendingCommitsAfterPoll: UIO[Unit] = ZIO.unit - override def pruneCommittedOffsets(assignedPartitions: Set[TopicPartition]): UIO[Unit] = ZIO.unit + override def queueSize: UIO[Int] = ZIO.succeed(0) + override def pendingCommitCount: UIO[Int] = ZIO.succeed(0) + override def getPendingCommits: UIO[CommitOffsets] = ZIO.succeed(CommitOffsets.empty) + override def cleanupPendingCommits: UIO[Unit] = ZIO.unit + override def keepCommitsForPartitions(assignedPartitions: Set[TopicPartition]): UIO[Unit] = ZIO.unit override def getCommittedOffsets: UIO[CommitOffsets] = ZIO.succeed(CommitOffsets.empty) } diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Committer.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Committer.scala index dc2864fde..ab8b98ac5 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Committer.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Committer.scala @@ -18,6 +18,9 @@ private[internal] trait Committer { * * If the queue is empty, nothing happens, unless executeOnEmpty is true. * + * WARNING: this method is used during a rebalance from the same-thread-runtime. This restricts what ZIO operations + * may be used. Please see [[RebalanceCoordinator]] for more information. + * * @param commitAsync * Function 'commitAsync' on the KafkaConsumer. This is isolated from the whole KafkaConsumer for testing purposes. * The caller should ensure exclusive access to the KafkaConsumer. @@ -35,9 +38,10 @@ private[internal] trait Committer { def getPendingCommits: UIO[CommitOffsets] - def updatePendingCommitsAfterPoll: UIO[Unit] + /** Removes all completed commits from `pendingCommits`. */ + def cleanupPendingCommits: UIO[Unit] - def pruneCommittedOffsets(assignedPartitions: Set[TopicPartition]): UIO[Unit] + def keepCommitsForPartitions(assignedPartitions: Set[TopicPartition]): UIO[Unit] def getCommittedOffsets: UIO[CommitOffsets] } @@ -61,7 +65,7 @@ private[internal] object Committer { case None => // This partition was not committed to from this consumer yet. Therefore we do not know the offset // increase. A good estimate would be the poll size for this consumer, another okayish estimate is 0. - // Lets go with the simplest for now: ```offsetIncrease += 0``` + // Let's go with the simplest for now: ```offsetIncrease += 0``` offset } updatedOffsets += tp -> maxOffset diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/LiveCommitter.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/LiveCommitter.scala index 9b4ddc53c..f976c1f9e 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/LiveCommitter.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/LiveCommitter.scala @@ -21,7 +21,7 @@ private[consumer] final class LiveCommitter( onCommitAvailable: UIO[Unit], committedOffsetsRef: Ref[CommitOffsets], sameThreadRuntime: Runtime[Any], - pendingCommits: Ref[Chunk[Commit]] + pendingCommits: Ref.Synchronized[Chunk[Commit]] ) extends Committer { /** This is the implementation behind the user facing api `Offset.commit`. */ @@ -39,6 +39,10 @@ private[consumer] final class LiveCommitter( _ <- consumerMetrics.observeCommit(latency) } yield () + /** + * WARNING: this method is used during a rebalance from the same-thread-runtime. This restricts what ZIO operations + * may be used. Please see [[RebalanceCoordinator]] for more information. + */ override def processQueuedCommits( commitAsync: (JavaMap[TopicPartition, OffsetAndMetadata], OffsetCommitCallback) => Task[Unit], executeOnEmpty: Boolean = false @@ -55,7 +59,12 @@ private[consumer] final class LiveCommitter( } } yield () - /** Merge commits and prepare parameters for calling `consumer.commitAsync`. */ + /** + * Merge commits and prepare parameters for calling `consumer.commitAsync`. + * + * WARNING: this method is used during a rebalance from the same-thread-runtime. This restricts what ZIO operations + * may be used. Please see [[RebalanceCoordinator]] for more information. + */ private def asyncCommitParameters( commits: Chunk[Commit] ): (JavaMap[TopicPartition, OffsetAndMetadata], OffsetCommitCallback, Throwable => UIO[Unit]) = { @@ -116,10 +125,10 @@ private[consumer] final class LiveCommitter( override def getPendingCommits: UIO[CommitOffsets] = pendingCommits.get.map(CommitOffsets.empty.addCommits(_)._2) - override def updatePendingCommitsAfterPoll: UIO[Unit] = - pendingCommits.get.flatMap(ZIO.filter(_)(_.isPending)).flatMap(pendingCommits.set) + override def cleanupPendingCommits: UIO[Unit] = + pendingCommits.updateZIO(_.filterZIO(_.isPending)) - override def pruneCommittedOffsets(assignedPartitions: Set[TopicPartition]): UIO[Unit] = + override def keepCommitsForPartitions(assignedPartitions: Set[TopicPartition]): UIO[Unit] = committedOffsetsRef.update(_.keepPartitions(assignedPartitions)) override def getCommittedOffsets: UIO[CommitOffsets] = committedOffsetsRef.get @@ -133,7 +142,7 @@ private[internal] object LiveCommitter { onCommitAvailable: UIO[Unit], sameThreadRuntime: Runtime[Any] ): ZIO[Scope, Nothing, LiveCommitter] = for { - pendingCommits <- Ref.make(Chunk.empty[Commit]) + pendingCommits <- Ref.Synchronized.make(Chunk.empty[Commit]) commitQueue <- ZIO.acquireRelease(Queue.unbounded[Commit])(_.shutdown) committedOffsetsRef <- Ref.make(CommitOffsets.empty) } yield new LiveCommitter( @@ -152,8 +161,7 @@ private[internal] object LiveCommitter { offsets: Map[TopicPartition, OffsetAndMetadata], cont: Promise[Throwable, Unit] ) { - @inline def isDone: UIO[Boolean] = cont.isDone - @inline def isPending: UIO[Boolean] = isDone.negate + @inline def isPending: UIO[Boolean] = cont.isDone.negate } } diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/RebalanceCoordinator.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/RebalanceCoordinator.scala index 066bfdf15..06126e9b9 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/RebalanceCoordinator.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/RebalanceCoordinator.scala @@ -21,7 +21,7 @@ import zio.{ durationInt, Chunk, Duration, Ref, Task, UIO, ZIO } * continuing. */ private[internal] class RebalanceCoordinator( - lastRebalanceEvent: Ref[RebalanceEvent], + lastRebalanceEvent: Ref.Synchronized[RebalanceEvent], settings: ConsumerSettings, consumer: ConsumerAccess, maxRebalanceDuration: Duration, @@ -197,7 +197,7 @@ private[internal] class RebalanceCoordinator( // def toRebalanceListener: RebalanceListener = RebalanceListener( onAssigned = assignedTps => - withLastRebalanceEvent { rebalanceEvent => + lastRebalanceEvent.updateZIO { rebalanceEvent => for { _ <- ZIO.logDebug { val sameRebalance = if (rebalanceEvent.wasInvoked) " in same rebalance" else "" @@ -215,7 +215,7 @@ private[internal] class RebalanceCoordinator( ) }, onRevoked = revokedTps => - withLastRebalanceEvent { rebalanceEvent => + lastRebalanceEvent.updateZIO { rebalanceEvent => for { _ <- ZIO.logDebug { val sameRebalance = if (rebalanceEvent.wasInvoked) " in same rebalance" else "" @@ -234,7 +234,7 @@ private[internal] class RebalanceCoordinator( ) }, onLost = lostTps => - withLastRebalanceEvent { rebalanceEvent => + lastRebalanceEvent.updateZIO { rebalanceEvent => for { _ <- ZIO.logDebug(s"${lostTps.size} partitions are lost") assignedStreams <- getCurrentAssignedStreams @@ -249,10 +249,6 @@ private[internal] class RebalanceCoordinator( ) } ) - - private def withLastRebalanceEvent(f: RebalanceEvent => Task[RebalanceEvent]): Task[Unit] = - lastRebalanceEvent.get.flatMap(f).flatMap(lastRebalanceEvent.set) - } private[internal] object RebalanceCoordinator { diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala index 192d1b651..b4ceddbb2 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala @@ -286,7 +286,8 @@ private[consumer] final class Runloop private ( // Remove committed offsets for partitions that are no longer assigned: // NOTE: the type annotation is needed to keep the IntelliJ compiler happy. - _ <- committer.pruneCommittedOffsets(updatedAssignedStreams.map(_.tp).toSet): Task[Unit] + _ <- + committer.keepCommitsForPartitions(updatedAssignedStreams.map(_.tp).toSet): Task[Unit] _ <- consumerMetrics.observeRebalance( currentAssigned.size, @@ -328,7 +329,7 @@ private[consumer] final class Runloop private ( pollResult.ignoreRecordsForTps, pollResult.records ) - _ <- committer.updatePendingCommitsAfterPoll + _ <- committer.cleanupPendingCommits _ <- checkStreamPullInterval(pollResult.assignedStreams) } yield state.copy( pendingRequests = fulfillResult.pendingRequests, @@ -520,12 +521,12 @@ private[consumer] final class Runloop private ( private def observeRunloopMetrics(runloopMetricsSchedule: Schedule[Any, Unit, Long]): ZIO[Any, Nothing, Unit] = { val observe = for { - currentState <- currentStateRef.get - commandQueueSize <- commandQueue.size - commitQueueSize <- committer.queueSize - pendingCommits <- committer.pendingCommitCount + currentState <- currentStateRef.get + commandQueueSize <- commandQueue.size + commitQueueSize <- committer.queueSize + pendingCommitCount <- committer.pendingCommitCount _ <- consumerMetrics - .observeRunloopMetrics(currentState, commandQueueSize, commitQueueSize, pendingCommits) + .observeRunloopMetrics(currentState, commandQueueSize, commitQueueSize, pendingCommitCount) } yield () observe @@ -582,7 +583,7 @@ object Runloop { for { _ <- ZIO.addFinalizer(diagnostics.emit(Finalization.RunloopFinalized)) commandQueue <- ZIO.acquireRelease(Queue.unbounded[RunloopCommand])(_.shutdown) - lastRebalanceEvent <- Ref.make[RebalanceEvent](RebalanceEvent.None) + lastRebalanceEvent <- Ref.Synchronized.make[RebalanceEvent](RebalanceEvent.None) initialState = State.initial currentStateRef <- Ref.make(initialState) sameThreadRuntime <- ZIO.runtime[Any].provideLayer(SameThreadRuntimeLayer) From 7e21ebb45310cae4fb6db1bf9cfdbcdcf2ce3b68 Mon Sep 17 00:00:00 2001 From: Steven Vroonland Date: Sun, 24 Nov 2024 16:44:30 +0100 Subject: [PATCH 39/57] Separate file for mocked metrics --- .../consumer/internal/CommitterSpec.scala | 41 +++++-------------- .../consumer/internal/DummyMetrics.scala | 23 +++++++++++ 2 files changed, 33 insertions(+), 31 deletions(-) create mode 100644 zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/DummyMetrics.scala diff --git a/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/CommitterSpec.scala b/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/CommitterSpec.scala index 7e9c5f0f2..cf51f1fe1 100644 --- a/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/CommitterSpec.scala +++ b/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/CommitterSpec.scala @@ -5,33 +5,12 @@ import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.errors.RebalanceInProgressException import zio.kafka.consumer.diagnostics.Diagnostics import zio.test._ -import zio.{ durationInt, Promise, UIO, ZIO } +import zio.{ durationInt, Promise, ZIO } import java.util.{ Map => JavaMap } import scala.jdk.CollectionConverters.MapHasAsJava object CommitterSpec extends ZIOSpecDefault { - private val mockMetrics = new ConsumerMetrics { - override def observePoll(resumedCount: Int, pausedCount: Int, latency: zio.Duration, pollSize: Int): UIO[Unit] = - ZIO.unit - - override def observeCommit(latency: zio.Duration): UIO[Unit] = ZIO.unit - override def observeAggregatedCommit(latency: zio.Duration, commitSize: NanoTime): UIO[Unit] = ZIO.unit - override def observeRebalance( - currentlyAssignedCount: Int, - assignedCount: Int, - revokedCount: Int, - lostCount: Int - ): UIO[Unit] = ZIO.unit - override def observeRunloopMetrics( - state: Runloop.State, - commandQueueSize: Int, - commitQueueSize: Int, - pendingCommits: Int - ): UIO[Unit] = ZIO.unit - override def observePollAuthError(): UIO[Unit] = ZIO.unit - } - override def spec = suite("Committer")( test("signals that a new commit is available") { for { @@ -41,7 +20,7 @@ object CommitterSpec extends ZIOSpecDefault { .make( 10.seconds, Diagnostics.NoOp, - mockMetrics, + new DummyMetrics, onCommitAvailable = commitAvailable.succeed(()).unit, sameThreadRuntime = runtime ) @@ -57,7 +36,7 @@ object CommitterSpec extends ZIOSpecDefault { committer <- LiveCommitter.make( 10.seconds, Diagnostics.NoOp, - mockMetrics, + new DummyMetrics, onCommitAvailable = commitAvailable.succeed(()).unit, sameThreadRuntime = runtime ) @@ -75,7 +54,7 @@ object CommitterSpec extends ZIOSpecDefault { committer <- LiveCommitter.make( 10.seconds, Diagnostics.NoOp, - mockMetrics, + new DummyMetrics, onCommitAvailable = commitAvailable.succeed(()).unit, sameThreadRuntime = runtime ) @@ -95,7 +74,7 @@ object CommitterSpec extends ZIOSpecDefault { committer <- LiveCommitter.make( 10.seconds, Diagnostics.NoOp, - mockMetrics, + new DummyMetrics, onCommitAvailable = commitAvailable.succeed(()).unit, sameThreadRuntime = runtime ) @@ -116,7 +95,7 @@ object CommitterSpec extends ZIOSpecDefault { committer <- LiveCommitter.make( 10.seconds, Diagnostics.NoOp, - mockMetrics, + new DummyMetrics, onCommitAvailable = commitAvailable.succeed(()).unit, sameThreadRuntime = runtime ) @@ -139,7 +118,7 @@ object CommitterSpec extends ZIOSpecDefault { committer <- LiveCommitter.make( 10.seconds, Diagnostics.NoOp, - mockMetrics, + new DummyMetrics, onCommitAvailable = commitAvailable.succeed(()).unit, sameThreadRuntime = runtime ) @@ -166,7 +145,7 @@ object CommitterSpec extends ZIOSpecDefault { committer <- LiveCommitter.make( 10.seconds, Diagnostics.NoOp, - mockMetrics, + new DummyMetrics, onCommitAvailable = commitAvailable.succeed(()).unit, sameThreadRuntime = runtime ) @@ -187,7 +166,7 @@ object CommitterSpec extends ZIOSpecDefault { committer <- LiveCommitter.make( 10.seconds, Diagnostics.NoOp, - mockMetrics, + new DummyMetrics, onCommitAvailable = commitAvailable.succeed(()).unit, sameThreadRuntime = runtime ) @@ -206,7 +185,7 @@ object CommitterSpec extends ZIOSpecDefault { committer <- LiveCommitter.make( 10.seconds, Diagnostics.NoOp, - mockMetrics, + new DummyMetrics, onCommitAvailable = commitAvailable.succeed(()).unit, sameThreadRuntime = runtime ) diff --git a/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/DummyMetrics.scala b/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/DummyMetrics.scala new file mode 100644 index 000000000..a1bb7d90a --- /dev/null +++ b/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/DummyMetrics.scala @@ -0,0 +1,23 @@ +package zio.kafka.consumer.internal +import zio.{ UIO, ZIO } + +class DummyMetrics extends ConsumerMetrics { + override def observePoll(resumedCount: Int, pausedCount: Int, latency: zio.Duration, pollSize: Int): UIO[Unit] = + ZIO.unit + + override def observeCommit(latency: zio.Duration): UIO[Unit] = ZIO.unit + override def observeAggregatedCommit(latency: zio.Duration, commitSize: NanoTime): UIO[Unit] = ZIO.unit + override def observeRebalance( + currentlyAssignedCount: Int, + assignedCount: Int, + revokedCount: Int, + lostCount: Int + ): UIO[Unit] = ZIO.unit + override def observeRunloopMetrics( + state: Runloop.State, + commandQueueSize: Int, + commitQueueSize: Int, + pendingCommits: Int + ): UIO[Unit] = ZIO.unit + override def observePollAuthError(): UIO[Unit] = ZIO.unit +} From c517f46293766d7f24fa3d319d9a995c229c9286 Mon Sep 17 00:00:00 2001 From: Steven Vroonland Date: Sun, 24 Nov 2024 16:50:07 +0100 Subject: [PATCH 40/57] Fix --- .../test/scala/zio/kafka/consumer/internal/DummyMetrics.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/DummyMetrics.scala b/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/DummyMetrics.scala index a1bb7d90a..5879a1a1c 100644 --- a/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/DummyMetrics.scala +++ b/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/DummyMetrics.scala @@ -1,7 +1,7 @@ package zio.kafka.consumer.internal import zio.{ UIO, ZIO } -class DummyMetrics extends ConsumerMetrics { +private[internal] class DummyMetrics extends ConsumerMetrics { override def observePoll(resumedCount: Int, pausedCount: Int, latency: zio.Duration, pollSize: Int): UIO[Unit] = ZIO.unit From 4889cd120965eda181e1cc3b5a7a775fd29b1eb4 Mon Sep 17 00:00:00 2001 From: Steven Vroonland Date: Sun, 24 Nov 2024 18:53:47 +0100 Subject: [PATCH 41/57] Make it work --- .../zio/kafka/consumer/ConsumerSpec.scala | 2 +- .../consumer/internal/CommitterSpec.scala | 24 ++++--- .../internal/RebalanceCoordinatorSpec.scala | 2 +- .../kafka/consumer/internal/Committer.scala | 5 +- .../consumer/internal/LiveCommitter.scala | 51 ++++++++------- .../internal/RebalanceCoordinator.scala | 36 ++++++---- .../zio/kafka/consumer/internal/Runloop.scala | 65 ++++++++++++------- 7 files changed, 114 insertions(+), 71 deletions(-) diff --git a/zio-kafka-test/src/test/scala/zio/kafka/consumer/ConsumerSpec.scala b/zio-kafka-test/src/test/scala/zio/kafka/consumer/ConsumerSpec.scala index 13d51c789..ce7cbae65 100644 --- a/zio-kafka-test/src/test/scala/zio/kafka/consumer/ConsumerSpec.scala +++ b/zio-kafka-test/src/test/scala/zio/kafka/consumer/ConsumerSpec.scala @@ -1591,6 +1591,6 @@ object ConsumerSpec extends ZIOSpecDefaultSlf4j with KafkaRandom { .provideSome[Scope & Kafka](producer) .provideSomeShared[Scope]( Kafka.embedded - ) @@ withLiveClock @@ timeout(2.minutes) + ) @@ withLiveClock @@ timeout(2.minutes) @@ TestAspect.timed } diff --git a/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/CommitterSpec.scala b/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/CommitterSpec.scala index bc32f9300..60b4dc549 100644 --- a/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/CommitterSpec.scala +++ b/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/CommitterSpec.scala @@ -39,7 +39,7 @@ object CommitterSpec extends ZIOSpecDefault { tp = new TopicPartition("topic", 0) commitFiber <- committer.commit(Map(tp -> new OffsetAndMetadata(0))).forkScoped _ <- commitAvailable.await - _ <- committer.processQueuedCommits(offsets => ZIO.succeed(offsets)) + _ <- committer.processQueuedCommits(offsets => ZIO.succeed(ZIO.succeed(offsets))) _ <- commitFiber.join } yield assertCompletes }, @@ -55,7 +55,7 @@ object CommitterSpec extends ZIOSpecDefault { tp = new TopicPartition("topic", 0) commitFiber <- committer.commit(Map(tp -> new OffsetAndMetadata(0))).forkScoped _ <- commitAvailable.await - _ <- committer.processQueuedCommits(_ => ZIO.fail(new RuntimeException("Commit failed"))) + _ <- committer.processQueuedCommits(_ => ZIO.succeed(ZIO.fail(new RuntimeException("Commit failed")))) result <- commitFiber.await } yield assertTrue(result.isFailure) }, @@ -72,9 +72,11 @@ object CommitterSpec extends ZIOSpecDefault { commitFiber <- committer.commit(Map(tp -> new OffsetAndMetadata(0))).forkScoped _ <- commitAvailable.take _ <- - committer.processQueuedCommits(_ => ZIO.fail(new RebalanceInProgressException("Rebalance in progress"))) + committer.processQueuedCommits(_ => + ZIO.succeed(ZIO.fail(new RebalanceInProgressException("Rebalance in progress"))) + ) _ <- commitAvailable.take - _ <- committer.processQueuedCommits(offsets => ZIO.succeed(offsets)) + _ <- committer.processQueuedCommits(offsets => ZIO.succeed(ZIO.succeed(offsets))) _ <- commitFiber.join } yield assertCompletes }, @@ -91,7 +93,8 @@ object CommitterSpec extends ZIOSpecDefault { _ <- committer.commit(Map(tp -> new OffsetAndMetadata(1))).forkScoped _ <- commitAvailable.await committedOffsets <- Promise.make[Nothing, JavaMap[TopicPartition, OffsetAndMetadata]] - _ <- committer.processQueuedCommits(offsets => committedOffsets.succeed(offsets.asJava).as(offsets)) + _ <- + committer.processQueuedCommits(offsets => ZIO.succeed(committedOffsets.succeed(offsets.asJava).as(offsets))) offsetsCommitted <- committedOffsets.await } yield assertTrue( offsetsCommitted == Map(tp -> new OffsetAndMetadata(2)).asJava @@ -113,8 +116,9 @@ object CommitterSpec extends ZIOSpecDefault { commitFiber3 <- committer.commit(Map(tp2 -> new OffsetAndMetadata(3))).forkScoped _ <- commitAvailable.await committedOffsets <- Promise.make[Nothing, JavaMap[TopicPartition, OffsetAndMetadata]] - _ <- committer.processQueuedCommits(offsets => committedOffsets.succeed(offsets.asJava).as(offsets)) - _ <- commitFiber1.join zip commitFiber2.join zip commitFiber3.join + _ <- + committer.processQueuedCommits(offsets => ZIO.succeed(committedOffsets.succeed(offsets.asJava).as(offsets))) + _ <- commitFiber1.join zip commitFiber2.join zip commitFiber3.join offsetsCommitted <- committedOffsets.await } yield assertTrue( offsetsCommitted == Map(tp -> new OffsetAndMetadata(3), tp2 -> new OffsetAndMetadata(4)).asJava @@ -132,7 +136,7 @@ object CommitterSpec extends ZIOSpecDefault { tp = new TopicPartition("topic", 0) commitFiber <- committer.commit(Map(tp -> new OffsetAndMetadata(0))).forkScoped _ <- commitAvailable.await - _ <- committer.processQueuedCommits(ZIO.succeed(_)) + _ <- committer.processQueuedCommits(offsets => ZIO.succeed(ZIO.succeed(offsets))) pendingCommitsDuringCommit <- committer.pendingCommitCount _ <- committer.cleanupPendingCommits pendingCommitsAfterCommit <- committer.pendingCommitCount @@ -151,7 +155,7 @@ object CommitterSpec extends ZIOSpecDefault { tp = new TopicPartition("topic", 0) commitFiber <- committer.commit(Map(tp -> new OffsetAndMetadata(0))).forkScoped _ <- commitAvailable.await - _ <- committer.processQueuedCommits(ZIO.succeed(_)) + _ <- committer.processQueuedCommits(offsets => ZIO.succeed(ZIO.succeed(offsets))) committedOffsets <- committer.getCommittedOffsets _ <- commitFiber.join } yield assertTrue(committedOffsets.offsets == Map(tp -> 0L)) @@ -168,7 +172,7 @@ object CommitterSpec extends ZIOSpecDefault { tp = new TopicPartition("topic", 0) commitFiber <- committer.commit(Map(tp -> new OffsetAndMetadata(0))).forkScoped _ <- commitAvailable.await - _ <- committer.processQueuedCommits(ZIO.succeed(_)) + _ <- committer.processQueuedCommits(offsets => ZIO.succeed(ZIO.succeed(offsets))) _ <- committer.keepCommitsForPartitions(Set.empty) committedOffsets <- committer.getCommittedOffsets _ <- commitFiber.join diff --git a/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/RebalanceCoordinatorSpec.scala b/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/RebalanceCoordinatorSpec.scala index dae83d335..712167c64 100644 --- a/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/RebalanceCoordinatorSpec.scala +++ b/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/RebalanceCoordinatorSpec.scala @@ -212,7 +212,7 @@ abstract class MockCommitter extends Committer { override val commit: Map[TopicPartition, OffsetAndMetadata] => Task[Unit] = _ => ZIO.unit override def processQueuedCommits( - commitAsync: Map[TopicPartition, OffsetAndMetadata] => Task[Map[TopicPartition, OffsetAndMetadata]], + commitAsync: Map[TopicPartition, OffsetAndMetadata] => Task[Task[Map[TopicPartition, OffsetAndMetadata]]], executeOnEmpty: Boolean ): zio.Task[Unit] = ZIO.unit override def queueSize: UIO[Int] = ZIO.succeed(0) diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Committer.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Committer.scala index 413b2892b..6f784b262 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Committer.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Committer.scala @@ -22,12 +22,13 @@ private[internal] trait Committer { * * @param commitAsync * Function 'commitAsync' on the KafkaConsumer. This is isolated from the whole KafkaConsumer for testing purposes. - * The caller should ensure exclusive access to the KafkaConsumer. + * The caller should ensure exclusive access to the KafkaConsumer. The outer task represents the finishing of the + * commitAsync call, the inner task represents the callback results. * @param executeOnEmpty * Execute commitAsync() even if there are no commits */ def processQueuedCommits( - commitAsync: Map[TopicPartition, OffsetAndMetadata] => Task[Map[TopicPartition, OffsetAndMetadata]], + commitAsync: Map[TopicPartition, OffsetAndMetadata] => Task[Task[Map[TopicPartition, OffsetAndMetadata]]], executeOnEmpty: Boolean = false ): Task[Unit] diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/LiveCommitter.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/LiveCommitter.scala index 22b2880f0..733f8cae7 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/LiveCommitter.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/LiveCommitter.scala @@ -40,7 +40,7 @@ private[consumer] final class LiveCommitter( * may be used. Please see [[RebalanceCoordinator]] for more information. */ override def processQueuedCommits( - commitAsyncZIO: Map[TopicPartition, OffsetAndMetadata] => Task[Map[TopicPartition, OffsetAndMetadata]], + commitAsync: Map[TopicPartition, OffsetAndMetadata] => Task[Task[Map[TopicPartition, OffsetAndMetadata]]], executeOnEmpty: Boolean = false ): Task[Unit] = for { commits <- commitQueue.takeAll @@ -61,27 +61,34 @@ private[consumer] final class LiveCommitter( tp -> new OffsetAndMetadata(offset.offset + 1, offset.leaderEpoch, offset.metadata) } - pendingCommits.update(_ ++ commits) *> - ZIO.debug("Here!") *> - commitAsyncZIO(offsetsWithMetaData).timed.debug.flatMap { case (latency, offsetsWithMetaData) => - for { - offsetIncrease <- committedOffsetsRef.modify(_.addCommits(commits)) - _ <- consumerMetrics.observeAggregatedCommit(latency, offsetIncrease).when(commits.nonEmpty) - result <- ZIO.foreachDiscard(commits)(_.cont.done(Exit.unit)) - _ <- diagnostics.emit(DiagnosticEvent.Commit.Success(offsetsWithMetaData)) - } yield result - }.catchAllCause { - case Cause.Fail(_: RebalanceInProgressException, _) => - for { - _ <- ZIO.logDebug(s"Rebalance in progress, commit for offsets $offsets will be retried") - _ <- commitQueue.offerAll(commits) - _ <- onCommitAvailable - } yield () - case c => - ZIO.foreachDiscard(commits)(_.cont.done(Exit.fail(c.squash))) <* diagnostics.emit( - DiagnosticEvent.Commit.Failure(offsets, c.squash) - ) - }.debug + for { + _ <- pendingCommits.update(_ ++ commits) + startTime <- ZIO.clockWith(_.nanoTime) + getCommitResults <- commitAsync(offsetsWithMetaData) + _ <- getCommitResults.flatMap { offsetsWithMetaData => + for { + endTime <- ZIO.clockWith(_.nanoTime) + latency = (endTime - startTime).nanoseconds + offsetIncrease <- committedOffsetsRef.modify(_.addCommits(commits)) + _ <- consumerMetrics.observeAggregatedCommit(latency, offsetIncrease).when(commits.nonEmpty) + result <- ZIO.foreachDiscard(commits)(_.cont.done(Exit.unit)) + _ <- diagnostics.emit(DiagnosticEvent.Commit.Success(offsetsWithMetaData)) + } yield result + }.catchAllCause { + case Cause.Fail(_: RebalanceInProgressException, _) => + for { + _ <- ZIO.logDebug(s"Rebalance in progress, commit for offsets $offsets will be retried") + _ <- commitQueue.offerAll(commits) + _ <- onCommitAvailable + } yield () + case c => + ZIO.foreachDiscard(commits)(_.cont.done(Exit.fail(c.squash))) <* diagnostics.emit( + DiagnosticEvent.Commit.Failure(offsets, c.squash) + ) + }.forkDaemon + + } yield () + // We don't wait for the completion of the commit here, because it // will only complete once we poll again. } diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/RebalanceCoordinator.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/RebalanceCoordinator.scala index f9db0ff30..db463349d 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/RebalanceCoordinator.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/RebalanceCoordinator.scala @@ -1,5 +1,5 @@ package zio.kafka.consumer.internal -import org.apache.kafka.clients.consumer.OffsetAndMetadata +import org.apache.kafka.clients.consumer.{ OffsetAndMetadata, OffsetCommitCallback } import org.apache.kafka.common.TopicPartition import zio.kafka.consumer.internal.ConsumerAccess.ByteArrayKafkaConsumer import zio.kafka.consumer.internal.RebalanceCoordinator.{ @@ -11,7 +11,7 @@ import zio.kafka.consumer.internal.RebalanceCoordinator.{ } import zio.kafka.consumer.{ ConsumerSettings, RebalanceListener } import zio.stream.ZStream -import zio.{ durationInt, Chunk, Duration, Ref, Runtime, Task, UIO, ZIO } +import zio.{ durationInt, Chunk, Duration, Promise, Ref, Runtime, Task, UIO, Unsafe, ZIO } import java.util.{ Map => JavaMap } import scala.jdk.CollectionConverters._ @@ -189,17 +189,27 @@ private[internal] class RebalanceCoordinator( private def commitAsyncZIO( consumer: ByteArrayKafkaConsumer, offsets: Map[TopicPartition, OffsetAndMetadata] - ): Task[Map[TopicPartition, OffsetAndMetadata]] = - sameThreadRuntime.run { - ZIO.async[Any, Throwable, Map[TopicPartition, OffsetAndMetadata]]({ onDone => - consumer.commitAsync( - offsets.asJava, - (offsets: JavaMap[TopicPartition, OffsetAndMetadata], exception: Exception) => - if (exception == null) onDone(ZIO.succeed(offsets.asScala.toMap)) - else onDone(ZIO.fail(exception)) - ) - }) - } + ): Task[Task[Map[TopicPartition, OffsetAndMetadata]]] = + for { + result <- Promise.make[Throwable, Map[TopicPartition, OffsetAndMetadata]] + _ <- ZIO.attempt { + consumer.commitAsync( + offsets.asJava, + new OffsetCommitCallback { + override def onComplete( + offsets: JavaMap[TopicPartition, OffsetAndMetadata], + exception: Exception + ): Unit = + Unsafe.unsafe { implicit unsafe => + sameThreadRuntime.unsafe.run { + if (exception == null) result.succeed(offsets.asScala.toMap) + else result.fail(exception) + }: Unit + } + } + ) + } + } yield result.await // During a poll, the java kafka client might call each method of the rebalance listener 0 or 1 times. // We do not know the order in which the call-back methods are invoked. diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala index 04e574d80..3ce187b97 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala @@ -9,9 +9,9 @@ import zio.kafka.consumer._ import zio.kafka.consumer.diagnostics.DiagnosticEvent.{ Finalization, Rebalance } import zio.kafka.consumer.diagnostics.{ DiagnosticEvent, Diagnostics } import zio.kafka.consumer.internal.ConsumerAccess.ByteArrayKafkaConsumer +import zio.kafka.consumer.internal.RebalanceCoordinator.RebalanceEvent import zio.kafka.consumer.internal.Runloop._ import zio.kafka.consumer.internal.RunloopAccess.PartitionAssignment -import zio.kafka.consumer.internal.RebalanceCoordinator.RebalanceEvent import zio.stream._ import java.util.{ Map => JavaMap } @@ -485,7 +485,7 @@ private[consumer] final class Runloop private ( * - Poll periodically when we are subscribed but do not have assigned streams yet. This happens after * initialization and rebalancing */ - private def run(initialState: State): ZIO[Scope, Throwable, Any] = { + private def run(initialState: State, commitExecutor: Executor): ZIO[Scope, Throwable, Any] = { import Runloop.StreamOps ZStream @@ -494,9 +494,9 @@ private[consumer] final class Runloop private ( .runFoldChunksDiscardZIO(initialState) { (state, commands) => for { _ <- ZIO.logDebug(s"Processing ${commands.size} commands: ${commands.mkString(",")}") - _ <- consumer.runloopAccess { consumer => - committer.processQueuedCommits(commitAsyncZIO(consumer, _)) - } + _ <- committer + .processQueuedCommits(commitAsyncZIO) + .onExecutor(commitExecutor) streamCommands = commands.collect { case cmd: RunloopCommand.StreamCommand => cmd } stateAfterCommands <- ZIO.foldLeft(streamCommands)(state)(handleCommand) @@ -514,23 +514,42 @@ private[consumer] final class Runloop private ( .onError(cause => partitionsHub.offer(Take.failCause(cause))) } + /** + * Wrapper that converts KafkaConsumer#commitAsync to ZIO + * + * @param offsets + * Offsets to commit + * @return + * Task whose completion indicates completion of the commitAsync call. The inner task completes when the callback is + * executed and represents the callback results. + */ private def commitAsyncZIO( - consumer: ByteArrayKafkaConsumer, offsets: Map[TopicPartition, OffsetAndMetadata] - ): Task[Map[TopicPartition, OffsetAndMetadata]] = - ZIO - .async[Any, Throwable, Map[TopicPartition, OffsetAndMetadata]]({ onDone => - print("Starting commitAsync") - consumer.commitAsync( - offsets.asJava, - (offsets: JavaMap[TopicPartition, OffsetAndMetadata], exception: Exception) => { - print("Ending commitAsync") - if (exception == null) onDone(ZIO.succeed(offsets.asScala.toMap)) - else onDone(ZIO.fail(exception)) - } - ) - }) - // TODO the async doesn't continue, probably because it's on the blocking runtime..? + ): Task[Task[Map[TopicPartition, OffsetAndMetadata]]] = + for { + runtime <- ZIO.runtime[Any] + result <- Promise.make[Throwable, Map[TopicPartition, OffsetAndMetadata]] + // commitAsync does not need to be called from a special thread here, as long as we have exclusive access + _ <- consumer.runloopAccess { consumer => + ZIO.attempt { + consumer.commitAsync( + offsets.asJava, + new OffsetCommitCallback { + override def onComplete( + offsets: JavaMap[TopicPartition, OffsetAndMetadata], + exception: Exception + ): Unit = + Unsafe.unsafe { implicit unsafe => + runtime.unsafe.run { + if (exception == null) result.succeed(offsets.asScala.toMap) + else result.fail(exception) + }: Unit + } + } + ) + } + } + } yield result.await def shouldPoll(state: State): UIO[Boolean] = committer.pendingCommitCount.map { pendingCommitCount => @@ -642,8 +661,10 @@ object Runloop { _ <- runloop.observeRunloopMetrics(settings.runloopMetricsSchedule).forkScoped // Run the entire loop on a dedicated thread to avoid executor shifts - executor <- RunloopExecutor.newInstance - fiber <- ZIO.onExecutor(executor)(runloop.run(initialState)).forkScoped + + executor <- RunloopExecutor.newInstance + commitExecutor <- ZIO.executor + fiber <- ZIO.onExecutor(executor)(runloop.run(initialState, commitExecutor)).forkScoped waitForRunloopStop = fiber.join.orDie _ <- ZIO.addFinalizer( From 094258a142a176f4b929a457f301a4b11cfc7fa0 Mon Sep 17 00:00:00 2001 From: Steven Vroonland Date: Sun, 24 Nov 2024 19:19:59 +0100 Subject: [PATCH 42/57] Touchups --- .../internal/RebalanceCoordinatorSpec.scala | 22 ++++++++----------- .../consumer/internal/LiveCommitter.scala | 8 +++---- .../internal/RebalanceCoordinator.scala | 18 ++++++--------- .../zio/kafka/consumer/internal/Runloop.scala | 14 +++++++----- 4 files changed, 29 insertions(+), 33 deletions(-) diff --git a/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/RebalanceCoordinatorSpec.scala b/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/RebalanceCoordinatorSpec.scala index 712167c64..e0331ba7a 100644 --- a/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/RebalanceCoordinatorSpec.scala +++ b/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/RebalanceCoordinatorSpec.scala @@ -175,20 +175,16 @@ object RebalanceCoordinatorSpec extends ZIOSpecDefaultSlf4j { Semaphore .make(1) .map(new ConsumerAccess(mockConsumer, _)) - .flatMap { consumerAccess => - ZIO.runtime[Any].map { sameThreadRuntime => - new RebalanceCoordinator( - lastEvent, - settings.withRebalanceSafeCommits(rebalanceSafeCommits), - consumerAccess, - 5.seconds, - ZIO.succeed(assignedStreams), - committer, - sameThreadRuntime = sameThreadRuntime - ) - } + .map { consumerAccess => + new RebalanceCoordinator( + lastEvent, + settings.withRebalanceSafeCommits(rebalanceSafeCommits), + consumerAccess, + 5.seconds, + ZIO.succeed(assignedStreams), + committer + ) } - .provideLayer(SameThreadRuntimeLayer) private def createTestRecords(count: Int): Chunk[ByteArrayCommittableRecord] = Chunk.fromIterable( diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/LiveCommitter.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/LiveCommitter.scala index 733f8cae7..7a4cb1bc3 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/LiveCommitter.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/LiveCommitter.scala @@ -70,10 +70,10 @@ private[consumer] final class LiveCommitter( endTime <- ZIO.clockWith(_.nanoTime) latency = (endTime - startTime).nanoseconds offsetIncrease <- committedOffsetsRef.modify(_.addCommits(commits)) - _ <- consumerMetrics.observeAggregatedCommit(latency, offsetIncrease).when(commits.nonEmpty) - result <- ZIO.foreachDiscard(commits)(_.cont.done(Exit.unit)) - _ <- diagnostics.emit(DiagnosticEvent.Commit.Success(offsetsWithMetaData)) - } yield result + _ <- consumerMetrics.observeAggregatedCommit(latency, offsetIncrease).when(commits.nonEmpty) + _ <- ZIO.foreachDiscard(commits)(_.cont.done(Exit.unit)) + _ <- diagnostics.emit(DiagnosticEvent.Commit.Success(offsetsWithMetaData)) + } yield () }.catchAllCause { case Cause.Fail(_: RebalanceInProgressException, _) => for { diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/RebalanceCoordinator.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/RebalanceCoordinator.scala index db463349d..098ca40c0 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/RebalanceCoordinator.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/RebalanceCoordinator.scala @@ -2,16 +2,10 @@ package zio.kafka.consumer.internal import org.apache.kafka.clients.consumer.{ OffsetAndMetadata, OffsetCommitCallback } import org.apache.kafka.common.TopicPartition import zio.kafka.consumer.internal.ConsumerAccess.ByteArrayKafkaConsumer -import zio.kafka.consumer.internal.RebalanceCoordinator.{ - EndOffsetCommitPending, - EndOffsetCommitted, - EndOffsetNotCommitted, - RebalanceEvent, - StreamCompletionStatus -} +import zio.kafka.consumer.internal.RebalanceCoordinator._ import zio.kafka.consumer.{ ConsumerSettings, RebalanceListener } import zio.stream.ZStream -import zio.{ durationInt, Chunk, Duration, Promise, Ref, Runtime, Task, UIO, Unsafe, ZIO } +import zio.{ durationInt, Chunk, Duration, Promise, Ref, Task, UIO, Unsafe, ZIO } import java.util.{ Map => JavaMap } import scala.jdk.CollectionConverters._ @@ -30,8 +24,7 @@ private[internal] class RebalanceCoordinator( consumer: ConsumerAccess, maxRebalanceDuration: Duration, getCurrentAssignedStreams: UIO[Chunk[PartitionStreamControl]], - committer: Committer, - sameThreadRuntime: Runtime[Any] + committer: Committer ) { private val commitTimeoutNanos = settings.commitTimeout.toNanos @@ -192,6 +185,9 @@ private[internal] class RebalanceCoordinator( ): Task[Task[Map[TopicPartition, OffsetAndMetadata]]] = for { result <- Promise.make[Throwable, Map[TopicPartition, OffsetAndMetadata]] + runtime <- ZIO.runtime[ + Any + ] // RebalanceListener callbacks are executed on the same-thread runtime, see RebalanceListener.toKafka _ <- ZIO.attempt { consumer.commitAsync( offsets.asJava, @@ -201,7 +197,7 @@ private[internal] class RebalanceCoordinator( exception: Exception ): Unit = Unsafe.unsafe { implicit unsafe => - sameThreadRuntime.unsafe.run { + runtime.unsafe.run { if (exception == null) result.succeed(offsets.asScala.toMap) else result.fail(exception) }: Unit diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala index 3ce187b97..e048d7717 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala @@ -484,6 +484,8 @@ private[consumer] final class Runloop private ( * - Poll continuously when there are (still) unfulfilled requests or pending commits * - Poll periodically when we are subscribed but do not have assigned streams yet. This happens after * initialization and rebalancing + * + * Note that this method is executed on a dedicated single-thread blocking exector */ private def run(initialState: State, commitExecutor: Executor): ZIO[Scope, Throwable, Any] = { import Runloop.StreamOps @@ -494,9 +496,12 @@ private[consumer] final class Runloop private ( .runFoldChunksDiscardZIO(initialState) { (state, commands) => for { _ <- ZIO.logDebug(s"Processing ${commands.size} commands: ${commands.mkString(",")}") - _ <- committer - .processQueuedCommits(commitAsyncZIO) - .onExecutor(commitExecutor) + _ <- + committer + .processQueuedCommits(commitAsyncZIO) + .onExecutor( + commitExecutor + ) // processQueuedCommits does a fork to await the commit callback, which is not supported by the single-thread executor streamCommands = commands.collect { case cmd: RunloopCommand.StreamCommand => cmd } stateAfterCommands <- ZIO.foldLeft(streamCommands)(state)(handleCommand) @@ -639,8 +644,7 @@ object Runloop { consumer, maxRebalanceDuration, currentStateRef.get.map(_.assignedStreams), - committer, - sameThreadRuntime + committer ) runloop = new Runloop( settings = settings, From 564e4987fb9c6dd8c8b73dfbe9d1a4c16512cdac Mon Sep 17 00:00:00 2001 From: Steven Vroonland Date: Sun, 24 Nov 2024 19:34:20 +0100 Subject: [PATCH 43/57] WIP --- .../scala/zio/kafka/consumer/internal/LiveCommitter.scala | 2 +- .../zio/kafka/consumer/internal/RebalanceCoordinator.scala | 5 ++--- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/LiveCommitter.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/LiveCommitter.scala index 7a4cb1bc3..844d093fa 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/LiveCommitter.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/LiveCommitter.scala @@ -85,7 +85,7 @@ private[consumer] final class LiveCommitter( ZIO.foreachDiscard(commits)(_.cont.done(Exit.fail(c.squash))) <* diagnostics.emit( DiagnosticEvent.Commit.Failure(offsets, c.squash) ) - }.forkDaemon + }.forkDaemon // We don't await the completion of commits } yield () diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/RebalanceCoordinator.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/RebalanceCoordinator.scala index 098ca40c0..c928ed6b8 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/RebalanceCoordinator.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/RebalanceCoordinator.scala @@ -185,9 +185,8 @@ private[internal] class RebalanceCoordinator( ): Task[Task[Map[TopicPartition, OffsetAndMetadata]]] = for { result <- Promise.make[Throwable, Map[TopicPartition, OffsetAndMetadata]] - runtime <- ZIO.runtime[ - Any - ] // RebalanceListener callbacks are executed on the same-thread runtime, see RebalanceListener.toKafka + // RebalanceListener callbacks are executed on the same-thread runtime, see RebalanceListener.toKafka + runtime <- ZIO.runtime[Any] _ <- ZIO.attempt { consumer.commitAsync( offsets.asJava, From b17024376e35a4b9bed51304ab61f3f5cf6e7473 Mon Sep 17 00:00:00 2001 From: Steven Vroonland Date: Sun, 24 Nov 2024 20:25:57 +0100 Subject: [PATCH 44/57] Styling --- .../zio/kafka/consumer/internal/RebalanceCoordinator.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/RebalanceCoordinator.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/RebalanceCoordinator.scala index 06126e9b9..43be57980 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/RebalanceCoordinator.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/RebalanceCoordinator.scala @@ -164,12 +164,12 @@ private[internal] class RebalanceCoordinator( // Even if there is nothing to commit, continue to drive communication with the broker // so that commits can complete and the streams can make progress, by setting // executeOnEmpty = true - .tap(_ => + .tap { _ => committer.processQueuedCommits( (offsets, callback) => ZIO.attempt(consumer.commitAsync(offsets, callback)), executeOnEmpty = true ) - ) + } .takeWhile(_ => java.lang.System.nanoTime() <= deadline) .mapZIO(_ => endingStreamsCompletedAndCommitsExist) .takeUntil(completed => completed) From 0e9812ffea35bc94c718605ab049acea914a5406 Mon Sep 17 00:00:00 2001 From: Steven Vroonland Date: Sun, 24 Nov 2024 20:46:40 +0100 Subject: [PATCH 45/57] Fix flaky test --- .../kafka/consumer/internal/CommitterSpec.scala | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/CommitterSpec.scala b/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/CommitterSpec.scala index cf51f1fe1..9e0832201 100644 --- a/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/CommitterSpec.scala +++ b/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/CommitterSpec.scala @@ -5,7 +5,7 @@ import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.errors.RebalanceInProgressException import zio.kafka.consumer.diagnostics.Diagnostics import zio.test._ -import zio.{ durationInt, Promise, ZIO } +import zio.{ durationInt, Promise, Ref, ZIO } import java.util.{ Map => JavaMap } import scala.jdk.CollectionConverters.MapHasAsJava @@ -113,13 +113,15 @@ object CommitterSpec extends ZIOSpecDefault { }, test("batches commits from multiple partitions and offsets") { for { - runtime <- ZIO.runtime[Any] - commitAvailable <- Promise.make[Nothing, Unit] + runtime <- ZIO.runtime[Any] + commitsAvailable <- Promise.make[Nothing, Unit] + nrCommitsDone <- Ref.make(0) committer <- LiveCommitter.make( 10.seconds, Diagnostics.NoOp, new DummyMetrics, - onCommitAvailable = commitAvailable.succeed(()).unit, + onCommitAvailable = + ZIO.whenZIO(nrCommitsDone.updateAndGet(_ + 1).map(_ == 3))(commitsAvailable.succeed(())).unit, sameThreadRuntime = runtime ) tp = new TopicPartition("topic", 0) @@ -127,7 +129,7 @@ object CommitterSpec extends ZIOSpecDefault { commitFiber1 <- committer.commit(Map(tp -> new OffsetAndMetadata(1))).forkScoped commitFiber2 <- committer.commit(Map(tp -> new OffsetAndMetadata(2))).forkScoped commitFiber3 <- committer.commit(Map(tp2 -> new OffsetAndMetadata(3))).forkScoped - _ <- commitAvailable.await + _ <- commitsAvailable.await committedOffsets <- Promise.make[Nothing, JavaMap[TopicPartition, OffsetAndMetadata]] _ <- committer.processQueuedCommits((offsets, callback) => committedOffsets.succeed(offsets) *> ZIO.attempt(callback.onComplete(offsets, null)) @@ -198,5 +200,5 @@ object CommitterSpec extends ZIOSpecDefault { _ <- commitFiber.join } yield assertTrue(committedOffsets.offsets.isEmpty) } - ) @@ TestAspect.withLiveClock + ) @@ TestAspect.withLiveClock @@ TestAspect.nonFlaky(100) } From 2dfb74414bb098a6265e2c4d1512c3173515d273 Mon Sep 17 00:00:00 2001 From: Steven Vroonland Date: Sun, 24 Nov 2024 21:50:53 +0100 Subject: [PATCH 46/57] Fix discarded unit warning --- .../zio/kafka/consumer/internal/RebalanceCoordinator.scala | 3 ++- .../src/main/scala/zio/kafka/consumer/internal/Runloop.scala | 3 ++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/RebalanceCoordinator.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/RebalanceCoordinator.scala index dfb6f72f6..8b2d832cd 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/RebalanceCoordinator.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/RebalanceCoordinator.scala @@ -199,7 +199,8 @@ private[internal] class RebalanceCoordinator( runtime.unsafe.run { if (exception == null) result.succeed(offsets.asScala.toMap) else result.fail(exception) - }: Unit + } + () } } ) diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala index e048d7717..c86d67f3c 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala @@ -548,7 +548,8 @@ private[consumer] final class Runloop private ( runtime.unsafe.run { if (exception == null) result.succeed(offsets.asScala.toMap) else result.fail(exception) - }: Unit + } + () } } ) From 585d7418c602c3cc9bf6fa8739b18bf68671ab75 Mon Sep 17 00:00:00 2001 From: Steven Vroonland Date: Sun, 24 Nov 2024 22:00:47 +0100 Subject: [PATCH 47/57] Fixup --- .../zio/kafka/consumer/internal/RebalanceCoordinator.scala | 7 +++---- .../main/scala/zio/kafka/consumer/internal/Runloop.scala | 7 +++---- 2 files changed, 6 insertions(+), 8 deletions(-) diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/RebalanceCoordinator.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/RebalanceCoordinator.scala index 8b2d832cd..8f4dff5de 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/RebalanceCoordinator.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/RebalanceCoordinator.scala @@ -197,10 +197,9 @@ private[internal] class RebalanceCoordinator( ): Unit = Unsafe.unsafe { implicit unsafe => runtime.unsafe.run { - if (exception == null) result.succeed(offsets.asScala.toMap) - else result.fail(exception) - } - () + if (exception == null) result.succeed(offsets.asScala.toMap).unit + else result.fail(exception).unit + }.getOrThrowFiberFailure() } } ) diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala index c86d67f3c..27a8a707e 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala @@ -546,10 +546,9 @@ private[consumer] final class Runloop private ( ): Unit = Unsafe.unsafe { implicit unsafe => runtime.unsafe.run { - if (exception == null) result.succeed(offsets.asScala.toMap) - else result.fail(exception) - } - () + if (exception == null) result.succeed(offsets.asScala.toMap).unit + else result.fail(exception).unit + }.getOrThrowFiberFailure() } } ) From 7859e4dbd5b73847e7300392493b6bb2afd3ced6 Mon Sep 17 00:00:00 2001 From: Steven Vroonland Date: Sun, 24 Nov 2024 22:09:08 +0100 Subject: [PATCH 48/57] Fix flaky tests --- .../scala/zio/kafka/consumer/internal/CommitterSpec.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/CommitterSpec.scala b/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/CommitterSpec.scala index 7954ba0be..4d95c4811 100644 --- a/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/CommitterSpec.scala +++ b/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/CommitterSpec.scala @@ -140,9 +140,9 @@ object CommitterSpec extends ZIOSpecDefault { _ <- commitAvailable.await _ <- committer.processQueuedCommits(offsets => ZIO.succeed(ZIO.succeed(offsets))) pendingCommitsDuringCommit <- committer.pendingCommitCount + _ <- commitFiber.join _ <- committer.cleanupPendingCommits pendingCommitsAfterCommit <- committer.pendingCommitCount - _ <- commitFiber.join } yield assertTrue(pendingCommitsDuringCommit == 1 && pendingCommitsAfterCommit == 0) }, test("keep track of committed offsets") { @@ -158,8 +158,8 @@ object CommitterSpec extends ZIOSpecDefault { commitFiber <- committer.commit(Map(tp -> new OffsetAndMetadata(0))).forkScoped _ <- commitAvailable.await _ <- committer.processQueuedCommits(offsets => ZIO.succeed(ZIO.succeed(offsets))) - committedOffsets <- committer.getCommittedOffsets _ <- commitFiber.join + committedOffsets <- committer.getCommittedOffsets } yield assertTrue(committedOffsets.offsets == Map(tp -> 0L)) }, test("clean committed offsets of no-longer assigned partitions") { From ca199ac16416d9a3db660fb79efdf2bbe56de292 Mon Sep 17 00:00:00 2001 From: Steven Vroonland Date: Mon, 25 Nov 2024 08:30:44 +0100 Subject: [PATCH 49/57] Improve processQueuedCommits --- .../consumer/internal/LiveCommitter.scala | 72 ++++++++++--------- 1 file changed, 38 insertions(+), 34 deletions(-) diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/LiveCommitter.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/LiveCommitter.scala index 844d093fa..c6363ca6d 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/LiveCommitter.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/LiveCommitter.scala @@ -46,17 +46,7 @@ private[consumer] final class LiveCommitter( commits <- commitQueue.takeAll _ <- ZIO.logDebug(s"Processing ${commits.size} commits") _ <- ZIO.unless(commits.isEmpty && !executeOnEmpty) { - val offsets = commits - .foldLeft(mutable.Map.empty[TopicPartition, OffsetAndMetadata]) { case (acc, commit) => - commit.offsets.foreach { case (tp, offset) => - acc += (tp -> acc - .get(tp) - .map(current => if (current.offset() > offset.offset()) current else offset) - .getOrElse(offset)) - } - acc - } - .toMap + val offsets = mergeCommitOffsets(commits) val offsetsWithMetaData = offsets.map { case (tp, offset) => tp -> new OffsetAndMetadata(offset.offset + 1, offset.leaderEpoch, offset.metadata) } @@ -65,35 +55,49 @@ private[consumer] final class LiveCommitter( _ <- pendingCommits.update(_ ++ commits) startTime <- ZIO.clockWith(_.nanoTime) getCommitResults <- commitAsync(offsetsWithMetaData) - _ <- getCommitResults.flatMap { offsetsWithMetaData => - for { - endTime <- ZIO.clockWith(_.nanoTime) - latency = (endTime - startTime).nanoseconds - offsetIncrease <- committedOffsetsRef.modify(_.addCommits(commits)) - _ <- consumerMetrics.observeAggregatedCommit(latency, offsetIncrease).when(commits.nonEmpty) - _ <- ZIO.foreachDiscard(commits)(_.cont.done(Exit.unit)) - _ <- diagnostics.emit(DiagnosticEvent.Commit.Success(offsetsWithMetaData)) - } yield () - }.catchAllCause { - case Cause.Fail(_: RebalanceInProgressException, _) => + _ <- getCommitResults + .zipLeft(ZIO.foreachDiscard(commits)(_.cont.done(Exit.unit))) + .zipLeft( for { - _ <- ZIO.logDebug(s"Rebalance in progress, commit for offsets $offsets will be retried") - _ <- commitQueue.offerAll(commits) - _ <- onCommitAvailable + endTime <- ZIO.clockWith(_.nanoTime) + latency = (endTime - startTime).nanoseconds + offsetIncrease <- committedOffsetsRef.modify(_.addCommits(commits)) + _ <- consumerMetrics.observeAggregatedCommit(latency, offsetIncrease).when(commits.nonEmpty) } yield () - case c => - ZIO.foreachDiscard(commits)(_.cont.done(Exit.fail(c.squash))) <* diagnostics.emit( - DiagnosticEvent.Commit.Failure(offsets, c.squash) - ) - }.forkDaemon // We don't await the completion of commits - + ) + .tap(offsetsWithMetaData => diagnostics.emit(DiagnosticEvent.Commit.Success(offsetsWithMetaData))) + .catchAllCause { + case Cause.Fail(_: RebalanceInProgressException, _) => + for { + _ <- ZIO.logDebug(s"Rebalance in progress, commit for offsets $offsets will be retried") + _ <- commitQueue.offerAll(commits) + _ <- onCommitAvailable + } yield () + case c => + ZIO.foreachDiscard(commits)(_.cont.done(Exit.fail(c.squash))) <* diagnostics.emit( + DiagnosticEvent.Commit.Failure(offsets, c.squash) + ) + } + // We don't wait for the completion of the commit here, because it + // will only complete once we poll again. + .forkDaemon } yield () - - // We don't wait for the completion of the commit here, because it - // will only complete once we poll again. } } yield () + private def mergeCommitOffsets(commits: Chunk[Commit]): Map[TopicPartition, OffsetAndMetadata] = + commits + .foldLeft(mutable.Map.empty[TopicPartition, OffsetAndMetadata]) { case (acc, commit) => + commit.offsets.foreach { case (tp, offset) => + acc += (tp -> acc + .get(tp) + .map(current => if (current.offset() > offset.offset()) current else offset) + .getOrElse(offset)) + } + acc + } + .toMap + override def queueSize: UIO[Int] = commitQueue.size override def pendingCommitCount: UIO[Int] = pendingCommits.get.map(_.size) From 9a519a04c7b7fc2fc7e7a1f12b9d8c79df241a4b Mon Sep 17 00:00:00 2001 From: Steven Vroonland Date: Mon, 25 Nov 2024 09:01:16 +0100 Subject: [PATCH 50/57] Fix flaky test --- .../main/scala/zio/kafka/consumer/internal/LiveCommitter.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/LiveCommitter.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/LiveCommitter.scala index c6363ca6d..39c1adb88 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/LiveCommitter.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/LiveCommitter.scala @@ -56,7 +56,6 @@ private[consumer] final class LiveCommitter( startTime <- ZIO.clockWith(_.nanoTime) getCommitResults <- commitAsync(offsetsWithMetaData) _ <- getCommitResults - .zipLeft(ZIO.foreachDiscard(commits)(_.cont.done(Exit.unit))) .zipLeft( for { endTime <- ZIO.clockWith(_.nanoTime) @@ -65,6 +64,7 @@ private[consumer] final class LiveCommitter( _ <- consumerMetrics.observeAggregatedCommit(latency, offsetIncrease).when(commits.nonEmpty) } yield () ) + .zipLeft(ZIO.foreachDiscard(commits)(_.cont.done(Exit.unit))) .tap(offsetsWithMetaData => diagnostics.emit(DiagnosticEvent.Commit.Success(offsetsWithMetaData))) .catchAllCause { case Cause.Fail(_: RebalanceInProgressException, _) => From 60a67ea13accb6c1aefc88e842aa1a2a2fb9b18e Mon Sep 17 00:00:00 2001 From: Steven Vroonland Date: Tue, 26 Nov 2024 19:40:02 +0100 Subject: [PATCH 51/57] Only one commitAsyncZio. Callers are ensuring locking and execution thread --- .../consumer/internal/CommitterSpec.scala | 72 ++++++++++++++----- .../internal/RebalanceCoordinatorSpec.scala | 7 +- .../kafka/consumer/internal/Committer.scala | 9 ++- .../consumer/internal/LiveCommitter.scala | 44 ++++++++++-- .../internal/RebalanceCoordinator.scala | 40 +---------- .../zio/kafka/consumer/internal/Runloop.scala | 50 ++----------- 6 files changed, 111 insertions(+), 111 deletions(-) diff --git a/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/CommitterSpec.scala b/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/CommitterSpec.scala index 4d95c4811..1f901c89c 100644 --- a/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/CommitterSpec.scala +++ b/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/CommitterSpec.scala @@ -1,14 +1,14 @@ package zio.kafka.consumer.internal -import org.apache.kafka.clients.consumer.OffsetAndMetadata +import org.apache.kafka.clients.consumer.{ MockConsumer, OffsetAndMetadata, OffsetCommitCallback, OffsetResetStrategy } import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.errors.RebalanceInProgressException import zio.kafka.consumer.diagnostics.Diagnostics import zio.test._ -import zio.{ durationInt, Promise, Queue, Ref, ZIO } +import zio.{ durationInt, Promise, Queue, Ref, Task, Unsafe, ZIO } import java.util.{ Map => JavaMap } -import scala.jdk.CollectionConverters.MapHasAsJava +import scala.jdk.CollectionConverters.{ MapHasAsJava, MapHasAsScala } object CommitterSpec extends ZIOSpecDefault { override def spec = suite("Committer")( @@ -39,7 +39,8 @@ object CommitterSpec extends ZIOSpecDefault { tp = new TopicPartition("topic", 0) commitFiber <- committer.commit(Map(tp -> new OffsetAndMetadata(0))).forkScoped _ <- commitAvailable.await - _ <- committer.processQueuedCommits(offsets => ZIO.succeed(ZIO.succeed(offsets))) + consumer <- createMockConsumer(offsets => ZIO.succeed(offsets)) + _ <- committer.processQueuedCommits(consumer) _ <- commitFiber.join } yield assertCompletes }, @@ -55,7 +56,8 @@ object CommitterSpec extends ZIOSpecDefault { tp = new TopicPartition("topic", 0) commitFiber <- committer.commit(Map(tp -> new OffsetAndMetadata(0))).forkScoped _ <- commitAvailable.await - _ <- committer.processQueuedCommits(_ => ZIO.succeed(ZIO.fail(new RuntimeException("Commit failed")))) + consumer <- createMockConsumer(_ => ZIO.fail(new RuntimeException("Commit failed"))) + _ <- committer.processQueuedCommits(consumer) result <- commitFiber.await } yield assertTrue(result.isFailure) }, @@ -71,12 +73,20 @@ object CommitterSpec extends ZIOSpecDefault { tp = new TopicPartition("topic", 0) commitFiber <- committer.commit(Map(tp -> new OffsetAndMetadata(0))).forkScoped _ <- commitAvailable.take - _ <- - committer.processQueuedCommits(_ => - ZIO.succeed(ZIO.fail(new RebalanceInProgressException("Rebalance in progress"))) - ) + callCount <- Ref.make(0) + consumer <- + createMockConsumer { offsets => + callCount.updateAndGet(_ + 1).flatMap { count => + if (count == 1) { + ZIO.fail(new RebalanceInProgressException("Rebalance in progress")) + } else { + ZIO.succeed(offsets) + } + } + } + _ <- committer.processQueuedCommits(consumer) _ <- commitAvailable.take - _ <- committer.processQueuedCommits(offsets => ZIO.succeed(ZIO.succeed(offsets))) + _ <- committer.processQueuedCommits(consumer) _ <- commitFiber.join } yield assertCompletes }, @@ -93,8 +103,8 @@ object CommitterSpec extends ZIOSpecDefault { _ <- committer.commit(Map(tp -> new OffsetAndMetadata(1))).forkScoped _ <- commitAvailable.await committedOffsets <- Promise.make[Nothing, JavaMap[TopicPartition, OffsetAndMetadata]] - _ <- - committer.processQueuedCommits(offsets => ZIO.succeed(committedOffsets.succeed(offsets.asJava).as(offsets))) + consumer <- createMockConsumer(offsets => committedOffsets.succeed(offsets.asJava).as(offsets)) + _ <- committer.processQueuedCommits(consumer) offsetsCommitted <- committedOffsets.await } yield assertTrue( offsetsCommitted == Map(tp -> new OffsetAndMetadata(2)).asJava @@ -118,8 +128,8 @@ object CommitterSpec extends ZIOSpecDefault { commitFiber3 <- committer.commit(Map(tp2 -> new OffsetAndMetadata(3))).forkScoped _ <- commitsAvailable.await committedOffsets <- Promise.make[Nothing, JavaMap[TopicPartition, OffsetAndMetadata]] - _ <- - committer.processQueuedCommits(offsets => ZIO.succeed(committedOffsets.succeed(offsets.asJava).as(offsets))) + consumer <- createMockConsumer(offsets => committedOffsets.succeed(offsets.asJava).as(offsets)) + _ <- committer.processQueuedCommits(consumer) _ <- commitFiber1.join zip commitFiber2.join zip commitFiber3.join offsetsCommitted <- committedOffsets.await } yield assertTrue( @@ -138,7 +148,8 @@ object CommitterSpec extends ZIOSpecDefault { tp = new TopicPartition("topic", 0) commitFiber <- committer.commit(Map(tp -> new OffsetAndMetadata(0))).forkScoped _ <- commitAvailable.await - _ <- committer.processQueuedCommits(offsets => ZIO.succeed(ZIO.succeed(offsets))) + consumer <- createMockConsumer(offsets => ZIO.succeed(offsets)) + _ <- committer.processQueuedCommits(consumer) pendingCommitsDuringCommit <- committer.pendingCommitCount _ <- commitFiber.join _ <- committer.cleanupPendingCommits @@ -157,7 +168,8 @@ object CommitterSpec extends ZIOSpecDefault { tp = new TopicPartition("topic", 0) commitFiber <- committer.commit(Map(tp -> new OffsetAndMetadata(0))).forkScoped _ <- commitAvailable.await - _ <- committer.processQueuedCommits(offsets => ZIO.succeed(ZIO.succeed(offsets))) + consumer <- createMockConsumer(offsets => ZIO.succeed(offsets)) + _ <- committer.processQueuedCommits(consumer) _ <- commitFiber.join committedOffsets <- committer.getCommittedOffsets } yield assertTrue(committedOffsets.offsets == Map(tp -> 0L)) @@ -174,11 +186,37 @@ object CommitterSpec extends ZIOSpecDefault { tp = new TopicPartition("topic", 0) commitFiber <- committer.commit(Map(tp -> new OffsetAndMetadata(0))).forkScoped _ <- commitAvailable.await - _ <- committer.processQueuedCommits(offsets => ZIO.succeed(ZIO.succeed(offsets))) + consumer <- createMockConsumer(offsets => ZIO.succeed(offsets)) + _ <- committer.processQueuedCommits(consumer) _ <- committer.keepCommitsForPartitions(Set.empty) committedOffsets <- committer.getCommittedOffsets _ <- commitFiber.join } yield assertTrue(committedOffsets.offsets.isEmpty) } ) @@ TestAspect.withLiveClock @@ TestAspect.nonFlaky(100) + + private def createMockConsumer( + onCommitAsync: Map[TopicPartition, OffsetAndMetadata] => Task[Map[TopicPartition, OffsetAndMetadata]] + ): ZIO[Any, Nothing, MockConsumer[Array[Byte], Array[Byte]]] = + ZIO.runtime[Any].map { runtime => + new MockConsumer[Array[Byte], Array[Byte]](OffsetResetStrategy.LATEST) { + override def commitAsync( + offsets: JavaMap[TopicPartition, OffsetAndMetadata], + callback: OffsetCommitCallback + ): Unit = + Unsafe.unsafe { implicit unsafe => + runtime.unsafe + .run( + onCommitAsync(offsets.asScala.toMap) + .tapBoth( + e => ZIO.attempt(callback.onComplete(offsets, e.asInstanceOf[Exception])), + offsets => ZIO.attempt(callback.onComplete(offsets.asJava, null)) + ) + .ignore + ) + .getOrThrowFiberFailure() + } + + } + } } diff --git a/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/RebalanceCoordinatorSpec.scala b/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/RebalanceCoordinatorSpec.scala index e0331ba7a..514063acf 100644 --- a/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/RebalanceCoordinatorSpec.scala +++ b/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/RebalanceCoordinatorSpec.scala @@ -5,6 +5,7 @@ import org.apache.kafka.common.TopicPartition import zio.kafka.ZIOSpecDefaultSlf4j import zio.kafka.consumer.diagnostics.Diagnostics import zio.kafka.consumer.internal.Committer.CommitOffsets +import zio.kafka.consumer.internal.ConsumerAccess.ByteArrayKafkaConsumer import zio.kafka.consumer.internal.LiveCommitter.Commit import zio.kafka.consumer.internal.RebalanceCoordinator.RebalanceEvent import zio.kafka.consumer.internal.Runloop.ByteArrayCommittableRecord @@ -207,10 +208,8 @@ object RebalanceCoordinatorSpec extends ZIOSpecDefaultSlf4j { abstract class MockCommitter extends Committer { override val commit: Map[TopicPartition, OffsetAndMetadata] => Task[Unit] = _ => ZIO.unit - override def processQueuedCommits( - commitAsync: Map[TopicPartition, OffsetAndMetadata] => Task[Task[Map[TopicPartition, OffsetAndMetadata]]], - executeOnEmpty: Boolean - ): zio.Task[Unit] = ZIO.unit + override def processQueuedCommits(consumer: ByteArrayKafkaConsumer, executeOnEmpty: Boolean): Task[Unit] = ZIO.unit + override def queueSize: UIO[Int] = ZIO.succeed(0) override def pendingCommitCount: UIO[Int] = ZIO.succeed(0) override def getPendingCommits: UIO[CommitOffsets] = ZIO.succeed(CommitOffsets.empty) diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Committer.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Committer.scala index 6f784b262..a40427dcc 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Committer.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Committer.scala @@ -3,6 +3,7 @@ package zio.kafka.consumer.internal import org.apache.kafka.clients.consumer.OffsetAndMetadata import org.apache.kafka.common.TopicPartition import zio.kafka.consumer.internal.Committer.CommitOffsets +import zio.kafka.consumer.internal.ConsumerAccess.ByteArrayKafkaConsumer import zio.kafka.consumer.internal.LiveCommitter.Commit import zio.{ Chunk, Task, UIO } @@ -20,15 +21,13 @@ private[internal] trait Committer { * WARNING: this method is used during a rebalance from the same-thread-runtime. This restricts what ZIO operations * may be used. Please see [[RebalanceCoordinator]] for more information. * - * @param commitAsync - * Function 'commitAsync' on the KafkaConsumer. This is isolated from the whole KafkaConsumer for testing purposes. - * The caller should ensure exclusive access to the KafkaConsumer. The outer task represents the finishing of the - * commitAsync call, the inner task represents the callback results. + * @param consumer + * KafkaConsumer to use. The caller is responsible or guaranteeing exclusive access. * @param executeOnEmpty * Execute commitAsync() even if there are no commits */ def processQueuedCommits( - commitAsync: Map[TopicPartition, OffsetAndMetadata] => Task[Task[Map[TopicPartition, OffsetAndMetadata]]], + consumer: ByteArrayKafkaConsumer, executeOnEmpty: Boolean = false ): Task[Unit] diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/LiveCommitter.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/LiveCommitter.scala index 39c1adb88..9e2af07ca 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/LiveCommitter.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/LiveCommitter.scala @@ -1,14 +1,17 @@ package zio.kafka.consumer.internal -import org.apache.kafka.clients.consumer.OffsetAndMetadata +import org.apache.kafka.clients.consumer.{ OffsetAndMetadata, OffsetCommitCallback } import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.errors.RebalanceInProgressException import zio.kafka.consumer.Consumer.CommitTimeout import zio.kafka.consumer.diagnostics.{ DiagnosticEvent, Diagnostics } import zio.kafka.consumer.internal.Committer.CommitOffsets +import zio.kafka.consumer.internal.ConsumerAccess.ByteArrayKafkaConsumer import zio.kafka.consumer.internal.LiveCommitter.Commit -import zio.{ durationLong, Cause, Chunk, Duration, Exit, Promise, Queue, Ref, Scope, Task, UIO, ZIO } +import zio.{ durationLong, Cause, Chunk, Duration, Exit, Promise, Queue, Ref, Scope, Task, UIO, Unsafe, ZIO } +import java.util.{ Map => JavaMap } import scala.collection.mutable +import scala.jdk.CollectionConverters._ private[consumer] final class LiveCommitter( commitQueue: Queue[Commit], @@ -40,7 +43,7 @@ private[consumer] final class LiveCommitter( * may be used. Please see [[RebalanceCoordinator]] for more information. */ override def processQueuedCommits( - commitAsync: Map[TopicPartition, OffsetAndMetadata] => Task[Task[Map[TopicPartition, OffsetAndMetadata]]], + consumer: ByteArrayKafkaConsumer, executeOnEmpty: Boolean = false ): Task[Unit] = for { commits <- commitQueue.takeAll @@ -54,7 +57,7 @@ private[consumer] final class LiveCommitter( for { _ <- pendingCommits.update(_ ++ commits) startTime <- ZIO.clockWith(_.nanoTime) - getCommitResults <- commitAsync(offsetsWithMetaData) + getCommitResults <- commitAsyncZIO(consumer, offsetsWithMetaData) _ <- getCommitResults .zipLeft( for { @@ -98,6 +101,39 @@ private[consumer] final class LiveCommitter( } .toMap + /** + * Wrapper that converts KafkaConsumer#commitAsync to ZIO + * + * @return + * Task whose completion indicates completion of the commitAsync call. The inner task completes when the callback is + * executed and represents the callback results. + */ + private def commitAsyncZIO( + consumer: ByteArrayKafkaConsumer, + offsets: Map[TopicPartition, OffsetAndMetadata] + ): Task[Task[Map[TopicPartition, OffsetAndMetadata]]] = + for { + runtime <- ZIO.runtime[Any] + result <- Promise.make[Throwable, Map[TopicPartition, OffsetAndMetadata]] + _ <- ZIO.attempt { + consumer.commitAsync( + offsets.asJava, + new OffsetCommitCallback { + override def onComplete( + offsets: JavaMap[TopicPartition, OffsetAndMetadata], + exception: Exception + ): Unit = + Unsafe.unsafe { implicit unsafe => + runtime.unsafe.run { + if (exception == null) result.succeed(offsets.asScala.toMap).unit + else result.fail(exception).unit + }.getOrThrowFiberFailure() + } + } + ) + } + } yield result.await + override def queueSize: UIO[Int] = commitQueue.size override def pendingCommitCount: UIO[Int] = pendingCommits.get.map(_.size) diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/RebalanceCoordinator.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/RebalanceCoordinator.scala index 8f4dff5de..3aa87af3b 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/RebalanceCoordinator.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/RebalanceCoordinator.scala @@ -1,14 +1,10 @@ package zio.kafka.consumer.internal -import org.apache.kafka.clients.consumer.{ OffsetAndMetadata, OffsetCommitCallback } import org.apache.kafka.common.TopicPartition import zio.kafka.consumer.internal.ConsumerAccess.ByteArrayKafkaConsumer import zio.kafka.consumer.internal.RebalanceCoordinator._ import zio.kafka.consumer.{ ConsumerSettings, RebalanceListener } import zio.stream.ZStream -import zio.{ durationInt, Chunk, Duration, Promise, Ref, Task, UIO, Unsafe, ZIO } - -import java.util.{ Map => JavaMap } -import scala.jdk.CollectionConverters._ +import zio.{ durationInt, Chunk, Duration, Ref, Task, UIO, ZIO } /** * The Runloop's RebalanceListener gets notified of partitions that are assigned, revoked and lost @@ -162,12 +158,7 @@ private[internal] class RebalanceCoordinator( // Even if there is nothing to commit, continue to drive communication with the broker // so that commits can complete and the streams can make progress, by setting // executeOnEmpty = true - .tap { _ => - committer.processQueuedCommits( - offsets => commitAsyncZIO(consumer, offsets), - executeOnEmpty = true - ) - } + .tap(_ => committer.processQueuedCommits(consumer, executeOnEmpty = true)) .takeWhile(_ => java.lang.System.nanoTime() <= deadline) .mapZIO(_ => endingStreamsCompletedAndCommitsExist) .takeUntil(completed => completed) @@ -179,33 +170,6 @@ private[internal] class RebalanceCoordinator( } yield () } - private def commitAsyncZIO( - consumer: ByteArrayKafkaConsumer, - offsets: Map[TopicPartition, OffsetAndMetadata] - ): Task[Task[Map[TopicPartition, OffsetAndMetadata]]] = - for { - result <- Promise.make[Throwable, Map[TopicPartition, OffsetAndMetadata]] - // RebalanceListener callbacks are executed on the same-thread runtime, see RebalanceListener.toKafka - runtime <- ZIO.runtime[Any] - _ <- ZIO.attempt { - consumer.commitAsync( - offsets.asJava, - new OffsetCommitCallback { - override def onComplete( - offsets: JavaMap[TopicPartition, OffsetAndMetadata], - exception: Exception - ): Unit = - Unsafe.unsafe { implicit unsafe => - runtime.unsafe.run { - if (exception == null) result.succeed(offsets.asScala.toMap).unit - else result.fail(exception).unit - }.getOrThrowFiberFailure() - } - } - ) - } - } yield result.await - // During a poll, the java kafka client might call each method of the rebalance listener 0 or 1 times. // We do not know the order in which the call-back methods are invoked. // diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala index 27a8a707e..e275d65e9 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala @@ -14,7 +14,6 @@ import zio.kafka.consumer.internal.Runloop._ import zio.kafka.consumer.internal.RunloopAccess.PartitionAssignment import zio.stream._ -import java.util.{ Map => JavaMap } import scala.jdk.CollectionConverters._ //noinspection SimplifyWhenInspection,SimplifyUnlessInspection @@ -497,11 +496,13 @@ private[consumer] final class Runloop private ( for { _ <- ZIO.logDebug(s"Processing ${commands.size} commands: ${commands.mkString(",")}") _ <- - committer - .processQueuedCommits(commitAsyncZIO) - .onExecutor( - commitExecutor - ) // processQueuedCommits does a fork to await the commit callback, which is not supported by the single-thread executor + consumer.runloopAccess { consumer => + committer + .processQueuedCommits(consumer) + .onExecutor( + commitExecutor + ) // processQueuedCommits does a fork to await the commit callback, which is not supported by the single-thread executor + } streamCommands = commands.collect { case cmd: RunloopCommand.StreamCommand => cmd } stateAfterCommands <- ZIO.foldLeft(streamCommands)(state)(handleCommand) @@ -519,43 +520,6 @@ private[consumer] final class Runloop private ( .onError(cause => partitionsHub.offer(Take.failCause(cause))) } - /** - * Wrapper that converts KafkaConsumer#commitAsync to ZIO - * - * @param offsets - * Offsets to commit - * @return - * Task whose completion indicates completion of the commitAsync call. The inner task completes when the callback is - * executed and represents the callback results. - */ - private def commitAsyncZIO( - offsets: Map[TopicPartition, OffsetAndMetadata] - ): Task[Task[Map[TopicPartition, OffsetAndMetadata]]] = - for { - runtime <- ZIO.runtime[Any] - result <- Promise.make[Throwable, Map[TopicPartition, OffsetAndMetadata]] - // commitAsync does not need to be called from a special thread here, as long as we have exclusive access - _ <- consumer.runloopAccess { consumer => - ZIO.attempt { - consumer.commitAsync( - offsets.asJava, - new OffsetCommitCallback { - override def onComplete( - offsets: JavaMap[TopicPartition, OffsetAndMetadata], - exception: Exception - ): Unit = - Unsafe.unsafe { implicit unsafe => - runtime.unsafe.run { - if (exception == null) result.succeed(offsets.asScala.toMap).unit - else result.fail(exception).unit - }.getOrThrowFiberFailure() - } - } - ) - } - } - } yield result.await - def shouldPoll(state: State): UIO[Boolean] = committer.pendingCommitCount.map { pendingCommitCount => state.subscriptionState.isSubscribed && (state.pendingRequests.nonEmpty || pendingCommitCount > 0 || state.assignedStreams.isEmpty) From 7050accf6fff2819c8684ff250c4ab311dac456c Mon Sep 17 00:00:00 2001 From: Steven Vroonland Date: Tue, 26 Nov 2024 19:55:37 +0100 Subject: [PATCH 52/57] Refactor commit completion --- .../consumer/internal/CommitterSpec.scala | 2 +- .../consumer/internal/LiveCommitter.scala | 58 +++++++++++-------- .../zio/kafka/consumer/internal/Runloop.scala | 2 +- 3 files changed, 37 insertions(+), 25 deletions(-) diff --git a/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/CommitterSpec.scala b/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/CommitterSpec.scala index 1f901c89c..c505dc237 100644 --- a/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/CommitterSpec.scala +++ b/zio-kafka-test/src/test/scala/zio/kafka/consumer/internal/CommitterSpec.scala @@ -188,9 +188,9 @@ object CommitterSpec extends ZIOSpecDefault { _ <- commitAvailable.await consumer <- createMockConsumer(offsets => ZIO.succeed(offsets)) _ <- committer.processQueuedCommits(consumer) + _ <- commitFiber.join _ <- committer.keepCommitsForPartitions(Set.empty) committedOffsets <- committer.getCommittedOffsets - _ <- commitFiber.join } yield assertTrue(committedOffsets.offsets.isEmpty) } ) @@ TestAspect.withLiveClock @@ TestAspect.nonFlaky(100) diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/LiveCommitter.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/LiveCommitter.scala index 9e2af07ca..f80041ad0 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/LiveCommitter.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/LiveCommitter.scala @@ -58,29 +58,8 @@ private[consumer] final class LiveCommitter( _ <- pendingCommits.update(_ ++ commits) startTime <- ZIO.clockWith(_.nanoTime) getCommitResults <- commitAsyncZIO(consumer, offsetsWithMetaData) - _ <- getCommitResults - .zipLeft( - for { - endTime <- ZIO.clockWith(_.nanoTime) - latency = (endTime - startTime).nanoseconds - offsetIncrease <- committedOffsetsRef.modify(_.addCommits(commits)) - _ <- consumerMetrics.observeAggregatedCommit(latency, offsetIncrease).when(commits.nonEmpty) - } yield () - ) - .zipLeft(ZIO.foreachDiscard(commits)(_.cont.done(Exit.unit))) - .tap(offsetsWithMetaData => diagnostics.emit(DiagnosticEvent.Commit.Success(offsetsWithMetaData))) - .catchAllCause { - case Cause.Fail(_: RebalanceInProgressException, _) => - for { - _ <- ZIO.logDebug(s"Rebalance in progress, commit for offsets $offsets will be retried") - _ <- commitQueue.offerAll(commits) - _ <- onCommitAvailable - } yield () - case c => - ZIO.foreachDiscard(commits)(_.cont.done(Exit.fail(c.squash))) <* diagnostics.emit( - DiagnosticEvent.Commit.Failure(offsets, c.squash) - ) - } + _ <- getCommitResults.exit + .flatMap(handleCommitCompletion(commits, offsetsWithMetaData, startTime, _)) // We don't wait for the completion of the commit here, because it // will only complete once we poll again. .forkDaemon @@ -88,6 +67,39 @@ private[consumer] final class LiveCommitter( } } yield () + private def handleCommitCompletion( + commits: Chunk[Commit], + offsets: Map[TopicPartition, OffsetAndMetadata], + startTime: NanoTime, + commitResults: Exit[Throwable, Map[TopicPartition, OffsetAndMetadata]] + ): Task[Unit] = + ZIO + .from(commitResults) + .unexit + .zipLeft( + for { + endTime <- ZIO.clockWith(_.nanoTime) + latency = (endTime - startTime).nanoseconds + offsetIncrease <- committedOffsetsRef.modify(_.addCommits(commits)) + _ <- consumerMetrics.observeAggregatedCommit(latency, offsetIncrease).when(commits.nonEmpty) + } yield () + ) + .zipLeft(ZIO.foreachDiscard(commits)(_.cont.done(Exit.unit))) + .tap(offsetsWithMetaData => diagnostics.emit(DiagnosticEvent.Commit.Success(offsetsWithMetaData))) + .catchAllCause { + case Cause.Fail(_: RebalanceInProgressException, _) => + for { + _ <- ZIO.logDebug(s"Rebalance in progress, commit for offsets $offsets will be retried") + _ <- commitQueue.offerAll(commits) + _ <- onCommitAvailable + } yield () + case c => + ZIO.foreachDiscard(commits)(_.cont.done(Exit.fail(c.squash))) <* diagnostics.emit( + DiagnosticEvent.Commit.Failure(offsets, c.squash) + ) + } + .unit + private def mergeCommitOffsets(commits: Chunk[Commit]): Map[TopicPartition, OffsetAndMetadata] = commits .foldLeft(mutable.Map.empty[TopicPartition, OffsetAndMetadata]) { case (acc, commit) => diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala index e275d65e9..704c116ee 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala @@ -484,7 +484,7 @@ private[consumer] final class Runloop private ( * - Poll periodically when we are subscribed but do not have assigned streams yet. This happens after * initialization and rebalancing * - * Note that this method is executed on a dedicated single-thread blocking exector + * Note that this method is executed on a dedicated single-thread Executor */ private def run(initialState: State, commitExecutor: Executor): ZIO[Scope, Throwable, Any] = { import Runloop.StreamOps From 208cd46ab7926a2c201ebd5d532e8ff9a1807801 Mon Sep 17 00:00:00 2001 From: Steven Vroonland Date: Tue, 26 Nov 2024 20:44:15 +0100 Subject: [PATCH 53/57] Reduce diff --- .../consumer/internal/LiveCommitter.scala | 26 +++++++++---------- 1 file changed, 13 insertions(+), 13 deletions(-) diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/LiveCommitter.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/LiveCommitter.scala index f80041ad0..554e287cc 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/LiveCommitter.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/LiveCommitter.scala @@ -67,6 +67,19 @@ private[consumer] final class LiveCommitter( } } yield () + private def mergeCommitOffsets(commits: Chunk[Commit]): Map[TopicPartition, OffsetAndMetadata] = + commits + .foldLeft(mutable.Map.empty[TopicPartition, OffsetAndMetadata]) { case (acc, commit) => + commit.offsets.foreach { case (tp, offset) => + acc += (tp -> acc + .get(tp) + .map(current => if (current.offset() > offset.offset()) current else offset) + .getOrElse(offset)) + } + acc + } + .toMap + private def handleCommitCompletion( commits: Chunk[Commit], offsets: Map[TopicPartition, OffsetAndMetadata], @@ -100,19 +113,6 @@ private[consumer] final class LiveCommitter( } .unit - private def mergeCommitOffsets(commits: Chunk[Commit]): Map[TopicPartition, OffsetAndMetadata] = - commits - .foldLeft(mutable.Map.empty[TopicPartition, OffsetAndMetadata]) { case (acc, commit) => - commit.offsets.foreach { case (tp, offset) => - acc += (tp -> acc - .get(tp) - .map(current => if (current.offset() > offset.offset()) current else offset) - .getOrElse(offset)) - } - acc - } - .toMap - /** * Wrapper that converts KafkaConsumer#commitAsync to ZIO * From 11494580881e57f4fd28693c8b6de16f50ffcf5b Mon Sep 17 00:00:00 2001 From: svroonland Date: Wed, 27 Nov 2024 17:31:44 +0100 Subject: [PATCH 54/57] Update zio-kafka/src/main/scala/zio/kafka/consumer/internal/LiveCommitter.scala Co-authored-by: Erik van Oosten --- .../main/scala/zio/kafka/consumer/internal/LiveCommitter.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/LiveCommitter.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/LiveCommitter.scala index 554e287cc..2d7ebdc79 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/LiveCommitter.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/LiveCommitter.scala @@ -48,7 +48,7 @@ private[consumer] final class LiveCommitter( ): Task[Unit] = for { commits <- commitQueue.takeAll _ <- ZIO.logDebug(s"Processing ${commits.size} commits") - _ <- ZIO.unless(commits.isEmpty && !executeOnEmpty) { + _ <- ZIO.when(commits.nonEmpty || executeOnEmpty) { val offsets = mergeCommitOffsets(commits) val offsetsWithMetaData = offsets.map { case (tp, offset) => tp -> new OffsetAndMetadata(offset.offset + 1, offset.leaderEpoch, offset.metadata) From 63af5a7228c66ffc5f02293378e1e7a94433a125 Mon Sep 17 00:00:00 2001 From: svroonland Date: Wed, 27 Nov 2024 17:31:58 +0100 Subject: [PATCH 55/57] Update zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala Co-authored-by: Erik van Oosten --- .../src/main/scala/zio/kafka/consumer/internal/Runloop.scala | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala index 704c116ee..299a430b2 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala @@ -497,11 +497,10 @@ private[consumer] final class Runloop private ( _ <- ZIO.logDebug(s"Processing ${commands.size} commands: ${commands.mkString(",")}") _ <- consumer.runloopAccess { consumer => + // processQueuedCommits does forks to await commit callbacks, move those to another executor to keep the runloop executor clear for the runloop committer .processQueuedCommits(consumer) - .onExecutor( - commitExecutor - ) // processQueuedCommits does a fork to await the commit callback, which is not supported by the single-thread executor + .onExecutor(commitExecutor) } streamCommands = commands.collect { case cmd: RunloopCommand.StreamCommand => cmd } stateAfterCommands <- ZIO.foldLeft(streamCommands)(state)(handleCommand) From efa2441f9a92dc2d83b6d3c1f251af3f51e20833 Mon Sep 17 00:00:00 2001 From: Steven Vroonland Date: Wed, 27 Nov 2024 20:48:30 +0100 Subject: [PATCH 56/57] Handle commit completion on the kafka thread --- .../consumer/internal/LiveCommitter.scala | 36 +++++++++---------- 1 file changed, 17 insertions(+), 19 deletions(-) diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/LiveCommitter.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/LiveCommitter.scala index 2d7ebdc79..d780f7497 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/LiveCommitter.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/LiveCommitter.scala @@ -55,14 +55,14 @@ private[consumer] final class LiveCommitter( } for { - _ <- pendingCommits.update(_ ++ commits) - startTime <- ZIO.clockWith(_.nanoTime) - getCommitResults <- commitAsyncZIO(consumer, offsetsWithMetaData) - _ <- getCommitResults.exit - .flatMap(handleCommitCompletion(commits, offsetsWithMetaData, startTime, _)) - // We don't wait for the completion of the commit here, because it - // will only complete once we poll again. - .forkDaemon + _ <- pendingCommits.update(_ ++ commits) + startTime <- ZIO.clockWith(_.nanoTime) + _ <- commitAsyncZIO( + consumer, + offsetsWithMetaData, + doOnComplete = handleCommitCompletion(commits, offsetsWithMetaData, startTime, _) + ) + // We don't wait for the completion of the commit here, because it will only complete once we poll again. } yield () } } yield () @@ -84,11 +84,10 @@ private[consumer] final class LiveCommitter( commits: Chunk[Commit], offsets: Map[TopicPartition, OffsetAndMetadata], startTime: NanoTime, - commitResults: Exit[Throwable, Map[TopicPartition, OffsetAndMetadata]] - ): Task[Unit] = + commitResults: Either[Exception, Map[TopicPartition, OffsetAndMetadata]] + ): UIO[Unit] = ZIO .from(commitResults) - .unexit .zipLeft( for { endTime <- ZIO.clockWith(_.nanoTime) @@ -111,7 +110,7 @@ private[consumer] final class LiveCommitter( DiagnosticEvent.Commit.Failure(offsets, c.squash) ) } - .unit + .ignore /** * Wrapper that converts KafkaConsumer#commitAsync to ZIO @@ -122,11 +121,11 @@ private[consumer] final class LiveCommitter( */ private def commitAsyncZIO( consumer: ByteArrayKafkaConsumer, - offsets: Map[TopicPartition, OffsetAndMetadata] - ): Task[Task[Map[TopicPartition, OffsetAndMetadata]]] = + offsets: Map[TopicPartition, OffsetAndMetadata], + doOnComplete: Either[Exception, Map[TopicPartition, OffsetAndMetadata]] => UIO[Unit] + ): Task[Unit] = for { runtime <- ZIO.runtime[Any] - result <- Promise.make[Throwable, Map[TopicPartition, OffsetAndMetadata]] _ <- ZIO.attempt { consumer.commitAsync( offsets.asJava, @@ -137,14 +136,14 @@ private[consumer] final class LiveCommitter( ): Unit = Unsafe.unsafe { implicit unsafe => runtime.unsafe.run { - if (exception == null) result.succeed(offsets.asScala.toMap).unit - else result.fail(exception).unit + if (exception == null) doOnComplete(Right(offsets.asScala.toMap)) + else doOnComplete(Left(exception)) }.getOrThrowFiberFailure() } } ) } - } yield result.await + } yield () override def queueSize: UIO[Int] = commitQueue.size @@ -189,5 +188,4 @@ private[internal] object LiveCommitter { ) { @inline def isPending: UIO[Boolean] = cont.isDone.negate } - } From be7a333810c357b5b21e8d2b70fa1e680a4d65e9 Mon Sep 17 00:00:00 2001 From: Steven Vroonland Date: Thu, 28 Nov 2024 17:56:15 +0100 Subject: [PATCH 57/57] Don't need this anymore --- .../kafka/consumer/internal/LiveCommitter.scala | 3 +-- .../zio/kafka/consumer/internal/Runloop.scala | 15 ++++----------- 2 files changed, 5 insertions(+), 13 deletions(-) diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/LiveCommitter.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/LiveCommitter.scala index d780f7497..6ababd40f 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/LiveCommitter.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/LiveCommitter.scala @@ -116,8 +116,7 @@ private[consumer] final class LiveCommitter( * Wrapper that converts KafkaConsumer#commitAsync to ZIO * * @return - * Task whose completion indicates completion of the commitAsync call. The inner task completes when the callback is - * executed and represents the callback results. + * Task whose completion indicates completion of the commitAsync call. */ private def commitAsyncZIO( consumer: ByteArrayKafkaConsumer, diff --git a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala index 299a430b2..3c277d8c0 100644 --- a/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala +++ b/zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala @@ -486,7 +486,7 @@ private[consumer] final class Runloop private ( * * Note that this method is executed on a dedicated single-thread Executor */ - private def run(initialState: State, commitExecutor: Executor): ZIO[Scope, Throwable, Any] = { + private def run(initialState: State): ZIO[Scope, Throwable, Any] = { import Runloop.StreamOps ZStream @@ -495,13 +495,7 @@ private[consumer] final class Runloop private ( .runFoldChunksDiscardZIO(initialState) { (state, commands) => for { _ <- ZIO.logDebug(s"Processing ${commands.size} commands: ${commands.mkString(",")}") - _ <- - consumer.runloopAccess { consumer => - // processQueuedCommits does forks to await commit callbacks, move those to another executor to keep the runloop executor clear for the runloop - committer - .processQueuedCommits(consumer) - .onExecutor(commitExecutor) - } + _ <- consumer.runloopAccess(committer.processQueuedCommits(_)) streamCommands = commands.collect { case cmd: RunloopCommand.StreamCommand => cmd } stateAfterCommands <- ZIO.foldLeft(streamCommands)(state)(handleCommand) @@ -629,9 +623,8 @@ object Runloop { // Run the entire loop on a dedicated thread to avoid executor shifts - executor <- RunloopExecutor.newInstance - commitExecutor <- ZIO.executor - fiber <- ZIO.onExecutor(executor)(runloop.run(initialState, commitExecutor)).forkScoped + executor <- RunloopExecutor.newInstance + fiber <- ZIO.onExecutor(executor)(runloop.run(initialState)).forkScoped waitForRunloopStop = fiber.join.orDie _ <- ZIO.addFinalizer(