diff --git a/src/topic_partition_list.rs b/src/topic_partition_list.rs index 16063bfbc..23d8c0254 100644 --- a/src/topic_partition_list.rs +++ b/src/topic_partition_list.rs @@ -317,6 +317,9 @@ impl TopicPartitionList { /// Sets all partitions in the list to the specified offset. pub fn set_all_offsets(&mut self, offset: Offset) -> Result<(), KafkaError> { + if self.count() == 0 { + return Ok(()); + } let slice = unsafe { slice::from_raw_parts_mut(self.ptr.elems, self.count()) }; for elem_ptr in slice { let mut elem = TopicPartitionListElem::from_ptr(self, &mut *elem_ptr); @@ -327,8 +330,11 @@ impl TopicPartitionList { /// Returns all the elements of the list. pub fn elements(&self) -> Vec> { + let mut vec = Vec::with_capacity(self.count()); + if self.count() == 0 { + return vec; + } let slice = unsafe { slice::from_raw_parts_mut(self.ptr.elems, self.count()) }; - let mut vec = Vec::with_capacity(slice.len()); for elem_ptr in slice { vec.push(TopicPartitionListElem::from_ptr(self, &mut *elem_ptr)); } @@ -337,8 +343,11 @@ impl TopicPartitionList { /// Returns all the elements of the list that belong to the specified topic. pub fn elements_for_topic<'a>(&'a self, topic: &str) -> Vec> { + let mut vec = Vec::with_capacity(self.count()); + if self.count() == 0 { + return vec; + } let slice = unsafe { slice::from_raw_parts_mut(self.ptr.elems, self.count()) }; - let mut vec = Vec::with_capacity(slice.len()); for elem_ptr in slice { let tp = TopicPartitionListElem::from_ptr(self, &mut *elem_ptr); if tp.topic() == topic {