Thanks!
On 6/2/26 11:23 AM, Aditya Kousik wrote:
Thanks for the feedback, Matthias.
My original intention was that the RebalanceConsumer would be a view with
limited operations, and so would be closed after each rebalance is completed to
prevent caching the object after exiting the callback. I thought making it
closeable would signal that.
And it would allows us to do something like
try (var view = createRebalanceConsumer(…)) {
listener.onPartitionsAssigned(partitions, view);
} and throw an IllegalStateExeption if someone invokes methods on consumer
later.
But I see that that just means users can invoke close() themselves
mid-rebalance which is not desirable.
I think we can achieve the same try-with-resources in the impl and not the
public interface.
Thanks for the catch. I’ve updated the KIP
-Aditya
On Jun 2, 2026, at 10:17, Matthias J. Sax <[email protected]> wrote:
Hey,
sorry for late reply, but I am wondering why the new `RebalanceConsumer`
interface would extend `AutoClosable`?
I don't think that `close()` would be a safe operation during a rebalance, and
the goal of the KIP is explicitly to only expose safe operations.
-Matthias
On 5/29/26 10:38 PM, Aditya Kousik wrote:
Thanks Chia-Ping.
chia_00: I’ve updated the JavaDoc (I realised it was lacking more detail that I
have in the actual PR). In this case, yes each invocation replaces the
previously set object. The final value before the subsequent poll() will be the
one to receive callbacks during rebalance events.
Thanks,
Aditya
On May 29, 2026, at 21:48, Chia-Ping Tsai <[email protected]> wrote:
hi Aditya
chia_00: What happens if a user calls setRebalanceListener(listener) multiple
times? I assume the previous listener will be replaced. If so, could you please
explicitly state this behavior in the Javadoc?
Best,
Chia-Ping
On 2026/04/01 15:16:36 Aditya Kousik wrote:
Hi all,
I'd like to start a discussion on KIP-1306: RebalanceHandler: Consumer-Aware
Rebalance Callback.
https://cwiki.apache.org/confluence/display/KAFKA/KIP-1306%3A+RebalanceHandler%3A+Consumer-Aware+Rebalance+Callback
Spring Kafka, SmallRye, and Micronaut all pass the consumer into rebalance
callbacks; the client doesn't. The standard workaround of constructor-injecting
a full Consumer reference allows dangerous operations like poll() and close()
inside a callback. This KIP proposes RebalanceHandler, with a
RebalanceConsumerView that exposes only safe operations, making misuse a
compile error.
Looking forward to your feedback.
Thanks