Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

async #917

Closed
wants to merge 1 commit into from
Closed

async #917

Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
42 changes: 33 additions & 9 deletions zio-kafka/src/main/scala/zio/kafka/consumer/internal/Runloop.scala
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,7 @@ import zio.kafka.consumer.internal.ConsumerAccess.ByteArrayKafkaConsumer
import zio.kafka.consumer.internal.RunloopAccess.PartitionAssignment
import zio.stream._

import java.util
import scala.jdk.CollectionConverters._

//noinspection SimplifyWhenInspection
Expand Down Expand Up @@ -117,16 +118,39 @@ private[consumer] final class Runloop private (
private def doCommit(cmd: RunloopCommand.Commit): UIO[Unit] = {
val offsets = cmd.offsets.map { case (tp, offset) => tp -> new OffsetAndMetadata(offset + 1) }

//noinspection ConvertExpressionToSAM
def callback(cb: UIO[Unit] => Unit): OffsetCommitCallback =
new OffsetCommitCallback {
override def onComplete(map: util.Map[TopicPartition, OffsetAndMetadata], e: Exception): Unit =
if (e == null)
cb(
ZIO.logDebug(s"Done for: $offsets") *> cmd.succeed <*
diagnostics.emit(DiagnosticEvent.Commit.Success(offsets))
)
else
cb(
ZIO.logDebugCause(s"Failed for: $offsets", Cause.fail(e)) *>
(e match {
case _: RebalanceInProgressException =>
ZIO.logDebug(s"Rebalance in progress, retrying commit for offsets $offsets") *>
commandQueue.offer(cmd).unit
case err =>
ZIO.logDebug(s"TATA") *> cmd.fail(err) <*
diagnostics.emit(DiagnosticEvent.Commit.Failure(offsets, err))
})
)
}

consumer
.runloopAccess(c => ZIO.attempt(c.commitSync(offsets.asJava)))
.foldZIO(
{
case _: RebalanceInProgressException =>
ZIO.logDebug(s"Rebalance in progress, retrying commit for offsets $offsets") *>
commandQueue.offer(cmd).unit
case err => cmd.fail(err) <* diagnostics.emit(DiagnosticEvent.Commit.Failure(offsets, err))
},
_ => cmd.succeed <* diagnostics.emit(DiagnosticEvent.Commit.Success(offsets))
.runloopAccess(c =>
for {
fiberId <- ZIO.fiberId
_ <- ZIO.logDebug(s"fiberId: $fiberId")
_ <- ZIO.async(
register = (cb: UIO[Unit] => Unit) => c.commitAsync(offsets.asJava, callback(cb)),
blockingOn = fiberId
)
} yield ()
)
}

Expand Down