Skip to content

Commit

Permalink
Fix handling of NULL pointer in NativeClient::rebalance_protocol
Browse files Browse the repository at this point in the history
The underlying function call `rdsys::rd_kafka_rebalance_protocol` can
return a NULL pointer in the case of an error. The callers within
librdkafka treat this similarly to it returning "NONE", so we will
return `RebalanceProtocol::None` to emulate this behavior.
  • Loading branch information
bruceg committed Dec 30, 2021
1 parent bbd3417 commit de421ba
Showing 1 changed file with 11 additions and 6 deletions.
17 changes: 11 additions & 6 deletions src/client.rs
Original file line number Diff line number Diff line change
Expand Up @@ -155,12 +155,17 @@ impl NativeClient {
}

pub(crate) fn rebalance_protocol(&self) -> RebalanceProtocol {
let protocol = unsafe { CStr::from_ptr(rdsys::rd_kafka_rebalance_protocol(self.ptr())) };
match protocol.to_bytes() {
b"NONE" => RebalanceProtocol::None,
b"EAGER" => RebalanceProtocol::Eager,
b"COOPERATIVE" => RebalanceProtocol::Cooperative,
_ => unreachable!(),
let protocol = unsafe { rdsys::rd_kafka_rebalance_protocol(self.ptr()) };
if protocol.is_null() {
RebalanceProtocol::None
} else {
let protocol = unsafe { CStr::from_ptr(protocol) };
match protocol.to_bytes() {
b"NONE" => RebalanceProtocol::None,
b"EAGER" => RebalanceProtocol::Eager,
b"COOPERATIVE" => RebalanceProtocol::Cooperative,
_ => unreachable!(),
}
}
}
}
Expand Down

0 comments on commit de421ba

Please sign in to comment.