Skip to content

Commit d6ec6e1

Browse files
scanterogdavidblewett
authored andcommitted
Return back to the caller on rebalance events
This does not affect the StreamConsumer or any other wrapper consumer. It will only incur on an extra Poll call when there's a rebalance event. When using bindings built upon the rust-rdkafka ffi, the caller is responsible for initiating the rebalance calls (*assign). If a high timeout is specified, the rebalance handler will only be triggered once the timeout period has elapsed. This fixes it by always returning on rebalance events except when Timeout::Never. Poll calls with timeout::Never are expected to return a message.
1 parent 738590b commit d6ec6e1

File tree

1 file changed

+6
-0
lines changed

1 file changed

+6
-0
lines changed

src/consumer/base_consumer.rs

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -144,9 +144,15 @@ where
144144
}
145145
rdsys::RD_KAFKA_EVENT_REBALANCE => {
146146
self.handle_rebalance_event(event);
147+
if timeout != Timeout::Never {
148+
return None;
149+
}
147150
}
148151
rdsys::RD_KAFKA_EVENT_OFFSET_COMMIT => {
149152
self.handle_offset_commit_event(event);
153+
if timeout != Timeout::Never {
154+
return None;
155+
}
150156
}
151157
_ => {
152158
let buf = unsafe {

0 commit comments

Comments
 (0)