Skip to content

Commit 02a1658

Browse files
authored
Merge branch 'fede1024:master' into mlowicki/optimise_ownedheaders_insert
2 parents a2bd0f3 + c6d9a65 commit 02a1658

File tree

9 files changed

+315
-82
lines changed

9 files changed

+315
-82
lines changed

README.md

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -184,7 +184,7 @@ re-exported as rdkafka features.
184184

185185
### Minimum supported Rust version (MSRV)
186186

187-
The current minimum supported Rust version (MSRV) is 1.61.0. Note that
187+
The current minimum supported Rust version (MSRV) is 1.70.0. Note that
188188
bumping the MSRV is not considered a breaking change. Any release of
189189
rust-rdkafka may bump the MSRV.
190190

rdkafka-sys/src/types.rs

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -78,6 +78,9 @@ pub type RDKafkaDeleteGroup = bindings::rd_kafka_DeleteGroup_t;
7878
/// Native rdkafka new partitions object.
7979
pub type RDKafkaNewPartitions = bindings::rd_kafka_NewPartitions_t;
8080

81+
/// Native rdkafka delete records object.
82+
pub type RDKafkaDeleteRecords = bindings::rd_kafka_DeleteRecords_t;
83+
8184
/// Native rdkafka config resource.
8285
pub type RDKafkaConfigResource = bindings::rd_kafka_ConfigResource_t;
8386

src/admin.rs

Lines changed: 85 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -26,6 +26,7 @@ use crate::config::{ClientConfig, FromClientConfig, FromClientConfigAndContext};
2626
use crate::error::{IsError, KafkaError, KafkaResult};
2727
use crate::log::{trace, warn};
2828
use crate::util::{cstr_to_owned, AsCArray, ErrBuf, IntoOpaque, KafkaDrop, NativePtr, Timeout};
29+
use crate::TopicPartitionList;
2930

3031
//
3132
// ********** ADMIN CLIENT **********
@@ -218,6 +219,53 @@ impl<C: ClientContext> AdminClient<C> {
218219
Ok(rx)
219220
}
220221

222+
/// Deletes records from a topic.
223+
///
224+
/// The provided `offsets` is a topic partition list specifying which
225+
/// records to delete from a list of topic partitions. For each entry in the
226+
/// list, the messages at offsets before the specified offsets (exclusive)
227+
/// in the specified partition will be deleted. Use offset [`Offset::End`]
228+
/// to delete all records in the partition.
229+
///
230+
/// Returns a topic partition list describing the result of the deletion. If
231+
/// the operation succeeded for a partition, the offset for that partition
232+
/// will be set to the post-deletion low-water mark for that partition. If
233+
/// the operation failed for a partition, there will be an error for that
234+
/// partition's entry in the list.
235+
pub fn delete_records(
236+
&self,
237+
offsets: &TopicPartitionList,
238+
opts: &AdminOptions,
239+
) -> impl Future<Output = KafkaResult<TopicPartitionList>> {
240+
match self.delete_records_inner(offsets, opts) {
241+
Ok(rx) => Either::Left(DeleteRecordsFuture { rx }),
242+
Err(err) => Either::Right(future::err(err)),
243+
}
244+
}
245+
246+
fn delete_records_inner(
247+
&self,
248+
offsets: &TopicPartitionList,
249+
opts: &AdminOptions,
250+
) -> KafkaResult<oneshot::Receiver<NativeEvent>> {
251+
let mut err_buf = ErrBuf::new();
252+
let delete_records = unsafe {
253+
NativeDeleteRecords::from_ptr(rdsys::rd_kafka_DeleteRecords_new(offsets.ptr()))
254+
}
255+
.ok_or_else(|| KafkaError::AdminOpCreation(err_buf.to_string()))?;
256+
let (native_opts, rx) = opts.to_native(self.client.native_ptr(), &mut err_buf)?;
257+
unsafe {
258+
rdsys::rd_kafka_DeleteRecords(
259+
self.client.native_ptr(),
260+
&mut delete_records.ptr(),
261+
1,
262+
native_opts.ptr(),
263+
self.queue.ptr(),
264+
);
265+
}
266+
Ok(rx)
267+
}
268+
221269
/// Retrieves the configuration parameters for the specified resources.
222270
///
223271
/// Note that while the API supports describing multiple configurations at
@@ -950,6 +998,43 @@ impl Future for CreatePartitionsFuture {
950998
}
951999
}
9521000

1001+
//
1002+
// Delete records handling
1003+
//
1004+
1005+
type NativeDeleteRecords = NativePtr<RDKafkaDeleteRecords>;
1006+
1007+
unsafe impl KafkaDrop for RDKafkaDeleteRecords {
1008+
const TYPE: &'static str = "delete records";
1009+
const DROP: unsafe extern "C" fn(*mut Self) = rdsys::rd_kafka_DeleteRecords_destroy;
1010+
}
1011+
1012+
struct DeleteRecordsFuture {
1013+
rx: oneshot::Receiver<NativeEvent>,
1014+
}
1015+
1016+
impl Future for DeleteRecordsFuture {
1017+
type Output = KafkaResult<TopicPartitionList>;
1018+
1019+
fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Self::Output> {
1020+
let event = ready!(self.rx.poll_unpin(cx)).map_err(|_| KafkaError::Canceled)?;
1021+
event.check_error()?;
1022+
let res = unsafe { rdsys::rd_kafka_event_DeleteRecords_result(event.ptr()) };
1023+
if res.is_null() {
1024+
let typ = unsafe { rdsys::rd_kafka_event_type(event.ptr()) };
1025+
return Poll::Ready(Err(KafkaError::AdminOpCreation(format!(
1026+
"delete records request received response of incorrect type ({})",
1027+
typ
1028+
))));
1029+
}
1030+
let tpl = unsafe {
1031+
let tpl = rdsys::rd_kafka_DeleteRecords_result_offsets(res);
1032+
TopicPartitionList::from_ptr(rdsys::rd_kafka_topic_partition_list_copy(tpl))
1033+
};
1034+
Poll::Ready(Ok(tpl))
1035+
}
1036+
}
1037+
9531038
//
9541039
// Describe configs handling
9551040
//

src/client.rs

Lines changed: 32 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -198,6 +198,21 @@ impl NativeClient {
198198
}
199199
}
200200

201+
pub(crate) enum EventPollResult<T> {
202+
None,
203+
EventConsumed,
204+
Event(T),
205+
}
206+
207+
impl<T> From<EventPollResult<T>> for Option<T> {
208+
fn from(val: EventPollResult<T>) -> Self {
209+
match val {
210+
EventPollResult::None | EventPollResult::EventConsumed => None,
211+
EventPollResult::Event(evt) => Some(evt),
212+
}
213+
}
214+
}
215+
201216
/// A low-level rdkafka client.
202217
///
203218
/// This type is the basis of the consumers and producers in the [`consumer`]
@@ -278,31 +293,42 @@ impl<C: ClientContext> Client<C> {
278293
&self.context
279294
}
280295

281-
pub(crate) fn poll_event(&self, queue: &NativeQueue, timeout: Timeout) -> Option<NativeEvent> {
296+
pub(crate) fn poll_event(
297+
&self,
298+
queue: &NativeQueue,
299+
timeout: Timeout,
300+
) -> EventPollResult<NativeEvent> {
282301
let event = unsafe { NativeEvent::from_ptr(queue.poll(timeout)) };
283302
if let Some(ev) = event {
284303
let evtype = unsafe { rdsys::rd_kafka_event_type(ev.ptr()) };
285304
match evtype {
286-
rdsys::RD_KAFKA_EVENT_LOG => self.handle_log_event(ev.ptr()),
287-
rdsys::RD_KAFKA_EVENT_STATS => self.handle_stats_event(ev.ptr()),
305+
rdsys::RD_KAFKA_EVENT_LOG => {
306+
self.handle_log_event(ev.ptr());
307+
return EventPollResult::EventConsumed;
308+
}
309+
rdsys::RD_KAFKA_EVENT_STATS => {
310+
self.handle_stats_event(ev.ptr());
311+
return EventPollResult::EventConsumed;
312+
}
288313
rdsys::RD_KAFKA_EVENT_ERROR => {
289314
// rdkafka reports consumer errors via RD_KAFKA_EVENT_ERROR but producer errors gets
290315
// embedded on the ack returned via RD_KAFKA_EVENT_DR. Hence we need to return this event
291316
// for the consumer case in order to return the error to the user.
292317
self.handle_error_event(ev.ptr());
293-
return Some(ev);
318+
return EventPollResult::Event(ev);
294319
}
295320
rdsys::RD_KAFKA_EVENT_OAUTHBEARER_TOKEN_REFRESH => {
296321
if C::ENABLE_REFRESH_OAUTH_TOKEN {
297322
self.handle_oauth_refresh_event(ev.ptr());
298323
}
324+
return EventPollResult::EventConsumed;
299325
}
300326
_ => {
301-
return Some(ev);
327+
return EventPollResult::Event(ev);
302328
}
303329
}
304330
}
305-
None
331+
EventPollResult::None
306332
}
307333

308334
fn handle_log_event(&self, event: *mut RDKafkaEvent) {

src/consumer/base_consumer.rs

Lines changed: 47 additions & 37 deletions
Original file line numberDiff line numberDiff line change
@@ -11,7 +11,7 @@ use log::{error, warn};
1111
use rdkafka_sys as rdsys;
1212
use rdkafka_sys::types::*;
1313

14-
use crate::client::{Client, NativeClient, NativeQueue};
14+
use crate::client::{Client, EventPollResult, NativeClient, NativeQueue};
1515
use crate::config::{
1616
ClientConfig, FromClientConfig, FromClientConfigAndContext, NativeClientConfig,
1717
};
@@ -117,59 +117,69 @@ where
117117
///
118118
/// The returned message lives in the memory of the consumer and cannot outlive it.
119119
pub fn poll<T: Into<Timeout>>(&self, timeout: T) -> Option<KafkaResult<BorrowedMessage<'_>>> {
120-
self.poll_queue(self.get_queue(), timeout)
120+
self.poll_queue(self.get_queue(), timeout).into()
121121
}
122122

123123
pub(crate) fn poll_queue<T: Into<Timeout>>(
124124
&self,
125125
queue: &NativeQueue,
126126
timeout: T,
127-
) -> Option<KafkaResult<BorrowedMessage<'_>>> {
127+
) -> EventPollResult<KafkaResult<BorrowedMessage<'_>>> {
128128
let now = Instant::now();
129-
let mut timeout = timeout.into();
129+
let initial_timeout = timeout.into();
130+
let mut timeout = initial_timeout;
130131
let min_poll_interval = self.context().main_queue_min_poll_interval();
131132
loop {
132133
let op_timeout = std::cmp::min(timeout, min_poll_interval);
133134
let maybe_event = self.client().poll_event(queue, op_timeout);
134-
if let Some(event) = maybe_event {
135-
let evtype = unsafe { rdsys::rd_kafka_event_type(event.ptr()) };
136-
match evtype {
137-
rdsys::RD_KAFKA_EVENT_FETCH => {
138-
if let Some(result) = self.handle_fetch_event(event) {
139-
return Some(result);
135+
match maybe_event {
136+
EventPollResult::Event(event) => {
137+
let evtype = unsafe { rdsys::rd_kafka_event_type(event.ptr()) };
138+
match evtype {
139+
rdsys::RD_KAFKA_EVENT_FETCH => {
140+
if let Some(result) = self.handle_fetch_event(event) {
141+
return EventPollResult::Event(result);
142+
}
140143
}
141-
}
142-
rdsys::RD_KAFKA_EVENT_ERROR => {
143-
if let Some(err) = self.handle_error_event(event) {
144-
return Some(Err(err));
144+
rdsys::RD_KAFKA_EVENT_ERROR => {
145+
if let Some(err) = self.handle_error_event(event) {
146+
return EventPollResult::Event(Err(err));
147+
}
145148
}
146-
}
147-
rdsys::RD_KAFKA_EVENT_REBALANCE => {
148-
self.handle_rebalance_event(event);
149-
if timeout != Timeout::Never {
150-
return None;
149+
rdsys::RD_KAFKA_EVENT_REBALANCE => {
150+
self.handle_rebalance_event(event);
151+
if timeout != Timeout::Never {
152+
return EventPollResult::EventConsumed;
153+
}
151154
}
152-
}
153-
rdsys::RD_KAFKA_EVENT_OFFSET_COMMIT => {
154-
self.handle_offset_commit_event(event);
155-
if timeout != Timeout::Never {
156-
return None;
155+
rdsys::RD_KAFKA_EVENT_OFFSET_COMMIT => {
156+
self.handle_offset_commit_event(event);
157+
if timeout != Timeout::Never {
158+
return EventPollResult::EventConsumed;
159+
}
160+
}
161+
_ => {
162+
let evname = unsafe {
163+
let evname = rdsys::rd_kafka_event_name(event.ptr());
164+
CStr::from_ptr(evname).to_string_lossy()
165+
};
166+
warn!("Ignored event '{evname}' on consumer poll");
157167
}
158168
}
159-
_ => {
160-
let evname = unsafe {
161-
let evname = rdsys::rd_kafka_event_name(event.ptr());
162-
CStr::from_ptr(evname).to_string_lossy()
163-
};
164-
warn!("Ignored event '{evname}' on consumer poll");
169+
}
170+
EventPollResult::None => {
171+
timeout = initial_timeout.saturating_sub(now.elapsed());
172+
if timeout.is_zero() {
173+
return EventPollResult::None;
165174
}
166175
}
167-
}
168-
169-
timeout = timeout.saturating_sub(now.elapsed());
170-
if timeout.is_zero() {
171-
return None;
172-
}
176+
EventPollResult::EventConsumed => {
177+
timeout = initial_timeout.saturating_sub(now.elapsed());
178+
if timeout.is_zero() {
179+
return EventPollResult::EventConsumed;
180+
}
181+
}
182+
};
173183
}
174184
}
175185

@@ -836,7 +846,7 @@ where
836846
/// associated consumer regularly, even if no messages are expected, to
837847
/// serve events.
838848
pub fn poll<T: Into<Timeout>>(&self, timeout: T) -> Option<KafkaResult<BorrowedMessage<'_>>> {
839-
self.consumer.poll_queue(&self.queue, timeout)
849+
self.consumer.poll_queue(&self.queue, timeout).into()
840850
}
841851

842852
/// Sets a callback that will be invoked whenever the queue becomes

src/consumer/stream_consumer.rs

Lines changed: 35 additions & 21 deletions
Original file line numberDiff line numberDiff line change
@@ -18,7 +18,7 @@ use slab::Slab;
1818
use rdkafka_sys as rdsys;
1919
use rdkafka_sys::types::*;
2020

21-
use crate::client::{Client, NativeQueue};
21+
use crate::client::{Client, EventPollResult, NativeQueue};
2222
use crate::config::{ClientConfig, FromClientConfig, FromClientConfigAndContext};
2323
use crate::consumer::base_consumer::{BaseConsumer, PartitionQueue};
2424
use crate::consumer::{
@@ -122,11 +122,12 @@ impl<'a, C: ConsumerContext> MessageStream<'a, C> {
122122
}
123123
}
124124

125-
fn poll(&self) -> Option<KafkaResult<BorrowedMessage<'a>>> {
125+
fn poll(&self) -> EventPollResult<KafkaResult<BorrowedMessage<'a>>> {
126126
if let Some(queue) = self.partition_queue {
127127
self.consumer.poll_queue(queue, Duration::ZERO)
128128
} else {
129-
self.consumer.poll(Duration::ZERO)
129+
self.consumer
130+
.poll_queue(self.consumer.get_queue(), Duration::ZERO)
130131
}
131132
}
132133
}
@@ -135,25 +136,38 @@ impl<'a, C: ConsumerContext> Stream for MessageStream<'a, C> {
135136
type Item = KafkaResult<BorrowedMessage<'a>>;
136137

137138
fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Option<Self::Item>> {
138-
// If there is a message ready, yield it immediately to avoid the
139-
// taking the lock in `self.set_waker`.
140-
if let Some(message) = self.poll() {
141-
return Poll::Ready(Some(message));
142-
}
143-
144-
// Otherwise, we need to wait for a message to become available. Store
145-
// the waker so that we are woken up if the queue flips from non-empty
146-
// to empty. We have to store the waker repatedly in case this future
147-
// migrates between tasks.
148-
self.wakers.set_waker(self.slot, cx.waker().clone());
149-
150-
// Check whether a new message became available after we installed the
151-
// waker. This avoids a race where `poll` returns None to indicate that
152-
// the queue is empty, but the queue becomes non-empty before we've
153-
// installed the waker.
154139
match self.poll() {
155-
None => Poll::Pending,
156-
Some(message) => Poll::Ready(Some(message)),
140+
EventPollResult::Event(message) => {
141+
// If there is a message ready, yield it immediately to avoid the
142+
// taking the lock in `self.set_waker`.
143+
Poll::Ready(Some(message))
144+
}
145+
EventPollResult::EventConsumed => {
146+
// Event was consumed, yield to runtime
147+
cx.waker().wake_by_ref();
148+
Poll::Pending
149+
}
150+
EventPollResult::None => {
151+
// Otherwise, we need to wait for a message to become available. Store
152+
// the waker so that we are woken up if the queue flips from non-empty
153+
// to empty. We have to store the waker repatedly in case this future
154+
// migrates between tasks.
155+
self.wakers.set_waker(self.slot, cx.waker().clone());
156+
157+
// Check whether a new message became available after we installed the
158+
// waker. This avoids a race where `poll` returns None to indicate that
159+
// the queue is empty, but the queue becomes non-empty before we've
160+
// installed the waker.
161+
match self.poll() {
162+
EventPollResult::Event(message) => Poll::Ready(Some(message)),
163+
EventPollResult::EventConsumed => {
164+
// Event was consumed, yield to runtime
165+
cx.waker().wake_by_ref();
166+
Poll::Pending
167+
}
168+
EventPollResult::None => Poll::Pending,
169+
}
170+
}
157171
}
158172
}
159173
}

0 commit comments

Comments
 (0)