Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Add support for get_watermark_offsets #648

Open
wants to merge 1 commit into
base: master
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
21 changes: 21 additions & 0 deletions src/consumer/base_consumer.rs
Original file line number Diff line number Diff line change
Expand Up @@ -656,6 +656,27 @@ where
}
}

fn get_watermark_offsets(&self, topic: &str, partition: i32) -> KafkaResult<(i64, i64)> {
let mut low = -1;
let mut high = -1;
let topic_c = CString::new(topic.to_string())?;
let result = unsafe {
rdsys::rd_kafka_get_watermark_offsets(
self.client.native_ptr(),
topic_c.as_ptr(),
partition,
&mut low as *mut i64,
&mut high as *mut i64,
)
};

if result.is_error() {
Err(KafkaError::MetadataFetch(result.into()))
} else {
Ok((low, high))
}
}

fn position(&self) -> KafkaResult<TopicPartitionList> {
let tpl = self.assignment()?;
let error = unsafe { rdsys::rd_kafka_position(self.client.native_ptr(), tpl.ptr()) };
Expand Down
9 changes: 9 additions & 0 deletions src/consumer/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -369,6 +369,15 @@ where
T: Into<Timeout>,
Self: Sized;

/// Get last known low (oldest/beginning) and high (newest/end) offsets for partition.
///
/// The low offset is updated periodically (if statistics.interval.ms is set) while the
/// high offset is updated on each fetched message set from the broker.
///
/// If there is no cached offset (either low or high, or both) then OFFSET_INVALID will
/// be returned for the respective offset.
fn get_watermark_offsets(&self, topic: &str, partition: i32) -> KafkaResult<(i64, i64)>;

/// Retrieve current positions (offsets) for topics and partitions.
fn position(&self) -> KafkaResult<TopicPartitionList>;

Expand Down
4 changes: 4 additions & 0 deletions src/consumer/stream_consumer.rs
Original file line number Diff line number Diff line change
Expand Up @@ -493,6 +493,10 @@ where
self.base.offsets_for_times(timestamps, timeout)
}

fn get_watermark_offsets(&self, topic: &str, partition: i32) -> KafkaResult<(i64, i64)> {
self.base.get_watermark_offsets(topic, partition)
}

fn position(&self) -> KafkaResult<TopicPartitionList> {
self.base.position()
}
Expand Down
26 changes: 26 additions & 0 deletions tests/test_high_consumers.rs
Original file line number Diff line number Diff line change
Expand Up @@ -314,6 +314,19 @@ async fn test_consumer_commit_message() {
(0, 12)
);

assert_eq!(
consumer.get_watermark_offsets(&topic_name, 0).unwrap(),
(0, 10)
);
assert_eq!(
consumer.get_watermark_offsets(&topic_name, 1).unwrap(),
(0, 11)
);
assert_eq!(
consumer.get_watermark_offsets(&topic_name, 2).unwrap(),
(0, 12)
);

let mut assignment = TopicPartitionList::new();
assignment
.add_partition_offset(&topic_name, 0, Offset::Stored)
Expand Down Expand Up @@ -399,6 +412,19 @@ async fn test_consumer_store_offset_commit() {
(0, 12)
);

assert_eq!(
consumer.get_watermark_offsets(&topic_name, 0).unwrap(),
(0, 10)
);
assert_eq!(
consumer.get_watermark_offsets(&topic_name, 1).unwrap(),
(0, 11)
);
assert_eq!(
consumer.get_watermark_offsets(&topic_name, 2).unwrap(),
(0, 12)
);

let mut assignment = TopicPartitionList::new();
assignment
.add_partition_offset(&topic_name, 0, Offset::Stored)
Expand Down