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

breaking: Do not run user rebalance listener on same thread runtime #1205

Merged
merged 7 commits into from
Apr 3, 2024
Merged
Show file tree
Hide file tree
Changes from 2 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
Original file line number Diff line number Diff line change
Expand Up @@ -2,7 +2,8 @@ package zio.kafka.consumer

import org.apache.kafka.clients.consumer.ConsumerRebalanceListener
import org.apache.kafka.common.TopicPartition
import zio.{ Runtime, Task, Unsafe, ZIO }
import zio.{ Executor, Runtime, Task, Unsafe, ZIO }

import scala.jdk.CollectionConverters._

/**
Expand All @@ -27,6 +28,12 @@ final case class RebalanceListener(
(lost, consumer) => onLost(lost, consumer) *> that.onLost(lost, consumer)
)

def runOnExecutor(executor: Executor): RebalanceListener = RebalanceListener(
(assigned, consumer) => onAssigned(assigned, consumer).onExecutor(executor),
(revoked, consumer) => onRevoked(revoked, consumer).onExecutor(executor),
(lost, consumer) => onLost(lost, consumer).onExecutor(executor)
)

def toKafka(
runtime: Runtime[Any],
consumer: RebalanceConsumer
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ import scala.jdk.CollectionConverters._
//noinspection SimplifyWhenInspection,SimplifyUnlessInspection
private[consumer] final class Runloop private (
settings: ConsumerSettings,
topLevelExecutor: Executor,
sameThreadRuntime: Runtime[Any],
consumer: ConsumerAccess,
maxPollInterval: Duration,
Expand Down Expand Up @@ -74,7 +75,7 @@ private[consumer] final class Runloop private (
private[internal] def removeSubscription(subscription: Subscription): UIO[Unit] =
commandQueue.offer(RunloopCommand.RemoveSubscription(subscription)).unit

private val rebalanceListener: RebalanceListener = {
private def makeRebalanceListener(rc: RebalanceConsumer.Live): 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.
Expand All @@ -92,7 +93,8 @@ private[consumer] final class Runloop private (
else {
for {
_ <- ZIO.foreachDiscard(streamsToEnd)(_.end)
_ <- if (rebalanceSafeCommits) consumer.rebalanceListenerAccess(doAwaitStreamCommits(_, state, streamsToEnd))
_ <- if (rebalanceSafeCommits)
consumer.rebalanceListenerAccess(doAwaitStreamCommits(_, state, streamsToEnd))
else ZIO.unit
} yield ()
}
Expand Down Expand Up @@ -239,7 +241,8 @@ private[consumer] final class Runloop private (
} yield ()
)

recordRebalanceRebalancingListener ++ settings.rebalanceListener
(recordRebalanceRebalancingListener ++ settings.rebalanceListener.runOnExecutor(topLevelExecutor))
.toKafka(sameThreadRuntime, rc)
erikvanoosten marked this conversation as resolved.
Show resolved Hide resolved
erikvanoosten marked this conversation as resolved.
Show resolved Hide resolved
}

/** This is the implementation behind the user facing api `Offset.commit`. */
Expand Down Expand Up @@ -671,14 +674,14 @@ private[consumer] final class Runloop private (
.attempt(c.unsubscribe())
.as(Chunk.empty)
case SubscriptionState.Subscribed(_, Subscription.Pattern(pattern)) =>
val rc = RebalanceConsumer.Live(c)
val rebalanceListener = makeRebalanceListener(RebalanceConsumer.Live(c))
ZIO
.attempt(c.subscribe(pattern.pattern, rebalanceListener.toKafka(sameThreadRuntime, rc)))
.attempt(c.subscribe(pattern.pattern, rebalanceListener))
.as(Chunk.empty)
case SubscriptionState.Subscribed(_, Subscription.Topics(topics)) =>
val rc = RebalanceConsumer.Live(c)
val rebalanceListener = makeRebalanceListener(RebalanceConsumer.Live(c))
ZIO
.attempt(c.subscribe(topics.asJava, rebalanceListener.toKafka(sameThreadRuntime, rc)))
.attempt(c.subscribe(topics.asJava, rebalanceListener))
.as(Chunk.empty)
case SubscriptionState.Subscribed(_, Subscription.Manual(topicPartitions)) =>
// For manual subscriptions we have to do some manual work before starting the run loop
Expand Down Expand Up @@ -846,8 +849,10 @@ object Runloop {
currentStateRef <- Ref.make(initialState)
committedOffsetsRef <- Ref.make(CommitOffsets.empty)
sameThreadRuntime <- ZIO.runtime[Any].provideLayer(SameThreadRuntimeLayer)
executor <- ZIO.executor
runloop = new Runloop(
settings = settings,
topLevelExecutor = executor,
sameThreadRuntime = sameThreadRuntime,
consumer = consumer,
maxPollInterval = maxPollInterval,
Expand Down
Loading