From de421ba8f0afba6722a83b7bc455d669e9ce7b37 Mon Sep 17 00:00:00 2001 From: Bruce Guenter Date: Wed, 29 Dec 2021 17:17:28 -0600 Subject: [PATCH] Fix handling of NULL pointer in `NativeClient::rebalance_protocol` 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. --- src/client.rs | 17 +++++++++++------ 1 file changed, 11 insertions(+), 6 deletions(-) diff --git a/src/client.rs b/src/client.rs index 20eb0cb16..7dff9a082 100644 --- a/src/client.rs +++ b/src/client.rs @@ -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!(), + } } } }