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

Improve ConsumerContext::commit_callback #198

Merged
merged 1 commit into from
Dec 16, 2019
Merged
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
7 changes: 7 additions & 0 deletions changelog.md
Original file line number Diff line number Diff line change
Expand Up @@ -22,10 +22,17 @@
Functions that return `future::Stream`s have had the analogous transformation
applied.

* Change the signature of `ConsumerContext::commit_callback` so that the
offsets are passed via a safe `TopicPartitionList` struct, and not a
raw `*mut rdkafka_sys::RDKafkaPartitionList` pointer. Thanks, [@scrogson]!
([#198]).

[#187]: https://github.com/fede1024/rust-rdkafka/pull/187
[#198]: https://github.com/fede1024/rust-rdkafka/pull/198

[@sd2k]: https://github.com/sd2k
[@dbcfd]: https://github.com/dbcfd
[@scrogson]: https://github.com/scrogson

<a name="0.22.0"></a>
## 0.22.0 (2019-12-01)
Expand Down
3 changes: 2 additions & 1 deletion examples/at_least_once.rs
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ use rdkafka::consumer::stream_consumer::StreamConsumer;
use rdkafka::consumer::{Consumer, ConsumerContext};
use rdkafka::error::KafkaResult;
use rdkafka::producer::{FutureProducer, FutureRecord};
use rdkafka::topic_partition_list::TopicPartitionList;
use rdkafka::util::get_rdkafka_version;
use rdkafka::Message;

Expand All @@ -39,7 +40,7 @@ impl ConsumerContext for LoggingConsumerContext {
fn commit_callback(
&self,
result: KafkaResult<()>,
_offsets: *mut rdkafka_sys::RDKafkaTopicPartitionList,
_offsets: &TopicPartitionList,
) {
match result {
Ok(_) => info!("Offsets committed successfully"),
Expand Down
3 changes: 2 additions & 1 deletion examples/simple_consumer.rs
Original file line number Diff line number Diff line change
Expand Up @@ -6,6 +6,7 @@ use rdkafka::client::ClientContext;
use rdkafka::config::{ClientConfig, RDKafkaLogLevel};
use rdkafka::consumer::stream_consumer::StreamConsumer;
use rdkafka::consumer::{CommitMode, Consumer, ConsumerContext, Rebalance};
use rdkafka::topic_partition_list::TopicPartitionList;
use rdkafka::error::KafkaResult;
use rdkafka::message::{Headers, Message};
use rdkafka::util::get_rdkafka_version;
Expand Down Expand Up @@ -33,7 +34,7 @@ impl ConsumerContext for CustomContext {
fn commit_callback(
&self,
result: KafkaResult<()>,
_offsets: *mut rdkafka_sys::RDKafkaTopicPartitionList,
_offsets: &TopicPartitionList,
) {
info!("Committing offsets: {:?}", result);
}
Expand Down
5 changes: 3 additions & 2 deletions src/consumer/base_consumer.rs
Original file line number Diff line number Diff line change
Expand Up @@ -27,15 +27,16 @@ pub(crate) unsafe extern "C" fn native_commit_cb<C: ConsumerContext>(
opaque_ptr: *mut c_void,
) {
let context = Box::from_raw(opaque_ptr as *mut C);

let commit_error = if err.is_error() {
Err(KafkaError::ConsumerCommit(err.into()))
} else {
Ok(())
};
(*context).commit_callback(commit_error, offsets);
let tpl = TopicPartitionList::from_ptr(offsets);
(*context).commit_callback(commit_error, &tpl);

mem::forget(context); // Do not free the context
tpl.leak() // Do not free offsets
}

/// Native rebalance callback. This callback will run on every rebalance, and it will call the
Expand Down
2 changes: 1 addition & 1 deletion src/consumer/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -90,7 +90,7 @@ pub trait ConsumerContext: ClientContext {
/// Post commit callback. This method will run after a group of offsets was committed to the
/// offset store.
#[allow(unused_variables)]
fn commit_callback(&self, result: KafkaResult<()>, offsets: *mut RDKafkaTopicPartitionList) {}
fn commit_callback(&self, result: KafkaResult<()>, offsets: &TopicPartitionList) {}

/// Returns the minimum interval at which to poll the main queue, which
/// services the logging, stats, and error callbacks.
Expand Down
2 changes: 1 addition & 1 deletion tests/test_consumers.rs
Original file line number Diff line number Diff line change
Expand Up @@ -35,7 +35,7 @@ impl ConsumerContext for TestContext {
fn commit_callback(
&self,
result: KafkaResult<()>,
_offsets: *mut rdkafka_sys::RDKafkaTopicPartitionList,
_offsets: &TopicPartitionList,
) {
println!("Committing offsets: {:?}", result);
}
Expand Down