From 46550f7eae011db040d6b959db9a49085e0e361d Mon Sep 17 00:00:00 2001 From: Joel Wachsler <125314669+JoelWachsler@users.noreply.github.com> Date: Thu, 3 Aug 2023 19:11:24 +0200 Subject: [PATCH 01/59] Allow users to set make when using cmake --- rdkafka-sys/build.rs | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/rdkafka-sys/build.rs b/rdkafka-sys/build.rs index 73933279d..d615744f3 100644 --- a/rdkafka-sys/build.rs +++ b/rdkafka-sys/build.rs @@ -287,6 +287,10 @@ fn build_librdkafka() { config.define("CMAKE_SYSTEM_NAME", system_name); } + if let Ok(make_program) = env::var("CMAKE_MAKE_PROGRAM") { + config.define("CMAKE_MAKE_PROGRAM", make_program); + } + if !cmake_library_paths.is_empty() { env::set_var("CMAKE_LIBRARY_PATH", cmake_library_paths.join(";")); } From eaee5528cdbe5c8b65fdd1bc4c1b325ab6cf9c6d Mon Sep 17 00:00:00 2001 From: David Blewett Date: Fri, 3 Nov 2023 15:57:11 -0400 Subject: [PATCH 02/59] Bump librdkafka to 2.3.0. --- changelog.md | 2 + rdkafka-sys/Cargo.toml | 2 +- rdkafka-sys/changelog.md | 2 + rdkafka-sys/librdkafka | 2 +- rdkafka-sys/src/bindings.rs | 312 +++++++++++++++++++++++++++++++++--- 5 files changed, 298 insertions(+), 22 deletions(-) diff --git a/changelog.md b/changelog.md index b44ccb13b..c3ae2884c 100644 --- a/changelog.md +++ b/changelog.md @@ -4,6 +4,8 @@ See also the [rdkafka-sys changelog](rdkafka-sys/changelog.md). ## Unreleased +* Update bundled librdkafka to 2.3.0. + ## 0.34.0 (2023-08-25) * Update bundled librdkafka to 2.2.0. diff --git a/rdkafka-sys/Cargo.toml b/rdkafka-sys/Cargo.toml index 4939f2373..9ea117031 100644 --- a/rdkafka-sys/Cargo.toml +++ b/rdkafka-sys/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "rdkafka-sys" -version = "4.6.0+2.2.0" +version = "4.6.0+2.3.0" authors = ["Federico Giraud "] build = "build.rs" links = "rdkafka" diff --git a/rdkafka-sys/changelog.md b/rdkafka-sys/changelog.md index 614c98a50..a56299ec5 100644 --- a/rdkafka-sys/changelog.md +++ b/rdkafka-sys/changelog.md @@ -2,6 +2,8 @@ ## Unreleased +* Upgrade to librdkafka v2.3.0. + ## v4.6.0+2.2.0 (2023-08-25) * Upgrade to librdkafka v2.2.0. diff --git a/rdkafka-sys/librdkafka b/rdkafka-sys/librdkafka index e75de5be1..95a542c87 160000 --- a/rdkafka-sys/librdkafka +++ b/rdkafka-sys/librdkafka @@ -1 +1 @@ -Subproject commit e75de5be191b6b8e9602efc969f4af64071550de +Subproject commit 95a542c87c61d2c45b445f91c73dd5442eb04f3c diff --git a/rdkafka-sys/src/bindings.rs b/rdkafka-sys/src/bindings.rs index 322f56493..78ba16b0c 100644 --- a/rdkafka-sys/src/bindings.rs +++ b/rdkafka-sys/src/bindings.rs @@ -3,7 +3,7 @@ use libc::{c_char, c_int, c_void, sockaddr, FILE}; use num_enum::TryFromPrimitive; -pub const RD_KAFKA_VERSION: i32 = 33685759; +pub const RD_KAFKA_VERSION: i32 = 33751295; pub const RD_KAFKA_DEBUG_CONTEXTS : & [u8 ; 138] = b"all,generic,broker,topic,metadata,feature,queue,msg,protocol,cgrp,security,fetch,interceptor,plugin,consumer,admin,eos,mock,assignor,conf\0" ; pub const RD_KAFKA_DESTROY_F_NO_CONSUMER_CLOSE: i32 = 8; pub const RD_KAFKA_OFFSET_BEGINNING: i32 = -2; @@ -46,6 +46,9 @@ pub const RD_KAFKA_EVENT_ALTERCONSUMERGROUPOFFSETS_RESULT: i32 = 65536; pub const RD_KAFKA_EVENT_INCREMENTALALTERCONFIGS_RESULT: i32 = 131072; pub const RD_KAFKA_EVENT_DESCRIBEUSERSCRAMCREDENTIALS_RESULT: i32 = 262144; pub const RD_KAFKA_EVENT_ALTERUSERSCRAMCREDENTIALS_RESULT: i32 = 524288; +pub const RD_KAFKA_EVENT_DESCRIBETOPICS_RESULT: i32 = 1048576; +pub const RD_KAFKA_EVENT_DESCRIBECLUSTER_RESULT: i32 = 2097152; +pub const RD_KAFKA_EVENT_LISTOFFSETS_RESULT: i32 = 4194304; extern "C" { pub fn rd_kafka_version() -> c_int; } @@ -140,6 +143,12 @@ pub struct rd_kafka_acl_result_s { _unused: [u8; 0], } pub type rd_kafka_acl_result_t = rd_kafka_acl_result_s; +#[repr(C)] +#[derive(Debug, Copy, Clone)] +pub struct rd_kafka_Uuid_s { + _unused: [u8; 0], +} +pub type rd_kafka_Uuid_t = rd_kafka_Uuid_s; #[repr(i32)] #[derive(Debug, Copy, Clone, Hash, PartialEq, Eq, TryFromPrimitive)] pub enum rd_kafka_resp_err_t { @@ -641,6 +650,27 @@ extern "C" { extern "C" { pub fn rd_kafka_message_leader_epoch(rkmessage: *const rd_kafka_message_t) -> i32; } +extern "C" { + pub fn rd_kafka_Uuid_base64str(uuid: *const rd_kafka_Uuid_t) -> *const c_char; +} +extern "C" { + pub fn rd_kafka_Uuid_least_significant_bits(uuid: *const rd_kafka_Uuid_t) -> i64; +} +extern "C" { + pub fn rd_kafka_Uuid_most_significant_bits(uuid: *const rd_kafka_Uuid_t) -> i64; +} +extern "C" { + pub fn rd_kafka_Uuid_new( + most_significant_bits: i64, + least_significant_bits: i64, + ) -> *mut rd_kafka_Uuid_t; +} +extern "C" { + pub fn rd_kafka_Uuid_copy(uuid: *const rd_kafka_Uuid_t) -> *mut rd_kafka_Uuid_t; +} +extern "C" { + pub fn rd_kafka_Uuid_destroy(uuid: *mut rd_kafka_Uuid_t); +} #[repr(i32)] #[derive(Debug, Copy, Clone, Hash, PartialEq, Eq)] pub enum rd_kafka_conf_res_t { @@ -1591,6 +1621,9 @@ extern "C" { extern "C" { pub fn rd_kafka_Node_port(node: *const rd_kafka_Node_t) -> u16; } +extern "C" { + pub fn rd_kafka_Node_rack(node: *const rd_kafka_Node_t) -> *const c_char; +} #[repr(C)] #[derive(Debug, Copy, Clone)] pub struct rd_kafka_group_member_info { @@ -1794,8 +1827,11 @@ pub type rd_kafka_DeleteGroups_result_t = rd_kafka_event_t; pub type rd_kafka_DeleteConsumerGroupOffsets_result_t = rd_kafka_event_t; pub type rd_kafka_AlterConsumerGroupOffsets_result_t = rd_kafka_event_t; pub type rd_kafka_ListConsumerGroupOffsets_result_t = rd_kafka_event_t; +pub type rd_kafka_DescribeTopics_result_t = rd_kafka_event_t; +pub type rd_kafka_DescribeCluster_result_t = rd_kafka_event_t; pub type rd_kafka_DescribeUserScramCredentials_result_t = rd_kafka_event_t; pub type rd_kafka_AlterUserScramCredentials_result_t = rd_kafka_event_t; +pub type rd_kafka_ListOffsets_result_t = rd_kafka_event_t; extern "C" { pub fn rd_kafka_event_CreateTopics_result( rkev: *mut rd_kafka_event_t, @@ -1841,6 +1877,16 @@ extern "C" { rkev: *mut rd_kafka_event_t, ) -> *const rd_kafka_DescribeConsumerGroups_result_t; } +extern "C" { + pub fn rd_kafka_event_DescribeTopics_result( + rkev: *mut rd_kafka_event_t, + ) -> *const rd_kafka_DescribeTopics_result_t; +} +extern "C" { + pub fn rd_kafka_event_DescribeCluster_result( + rkev: *mut rd_kafka_event_t, + ) -> *const rd_kafka_DescribeCluster_result_t; +} extern "C" { pub fn rd_kafka_event_DeleteGroups_result( rkev: *mut rd_kafka_event_t, @@ -1876,6 +1922,11 @@ extern "C" { rkev: *mut rd_kafka_event_t, ) -> *const rd_kafka_AlterConsumerGroupOffsets_result_t; } +extern "C" { + pub fn rd_kafka_event_ListOffsets_result( + rkev: *mut rd_kafka_event_t, + ) -> *const rd_kafka_ListOffsets_result_t; +} extern "C" { pub fn rd_kafka_event_DescribeUserScramCredentials_result( rkev: *mut rd_kafka_event_t, @@ -2180,7 +2231,10 @@ pub enum rd_kafka_admin_op_t { RD_KAFKA_ADMIN_OP_INCREMENTALALTERCONFIGS = 16, RD_KAFKA_ADMIN_OP_DESCRIBEUSERSCRAMCREDENTIALS = 17, RD_KAFKA_ADMIN_OP_ALTERUSERSCRAMCREDENTIALS = 18, - RD_KAFKA_ADMIN_OP__CNT = 19, + RD_KAFKA_ADMIN_OP_DESCRIBETOPICS = 19, + RD_KAFKA_ADMIN_OP_DESCRIBECLUSTER = 20, + RD_KAFKA_ADMIN_OP_LISTOFFSETS = 21, + RD_KAFKA_ADMIN_OP__CNT = 22, } #[repr(C)] #[derive(Debug, Copy, Clone)] @@ -2188,6 +2242,12 @@ pub struct rd_kafka_AdminOptions_s { _unused: [u8; 0], } pub type rd_kafka_AdminOptions_t = rd_kafka_AdminOptions_s; +#[repr(u32)] +#[derive(Debug, Copy, Clone, Hash, PartialEq, Eq)] +pub enum rd_kafka_IsolationLevel_t { + RD_KAFKA_ISOLATION_LEVEL_READ_UNCOMMITTED = 0, + RD_KAFKA_ISOLATION_LEVEL_READ_COMMITTED = 1, +} extern "C" { pub fn rd_kafka_AdminOptions_new( rk: *mut rd_kafka_t, @@ -2235,6 +2295,12 @@ extern "C" { true_or_false: c_int, ) -> *mut rd_kafka_error_t; } +extern "C" { + pub fn rd_kafka_AdminOptions_set_include_authorized_operations( + options: *mut rd_kafka_AdminOptions_t, + true_or_false: c_int, + ) -> *mut rd_kafka_error_t; +} extern "C" { pub fn rd_kafka_AdminOptions_set_match_consumer_group_states( options: *mut rd_kafka_AdminOptions_t, @@ -2242,12 +2308,36 @@ extern "C" { consumer_group_states_cnt: usize, ) -> *mut rd_kafka_error_t; } +extern "C" { + pub fn rd_kafka_AdminOptions_set_isolation_level( + options: *mut rd_kafka_AdminOptions_t, + value: rd_kafka_IsolationLevel_t, + ) -> *mut rd_kafka_error_t; +} extern "C" { pub fn rd_kafka_AdminOptions_set_opaque( options: *mut rd_kafka_AdminOptions_t, ev_opaque: *mut c_void, ); } +#[repr(u32)] +#[derive(Debug, Copy, Clone, Hash, PartialEq, Eq)] +pub enum rd_kafka_AclOperation_t { + RD_KAFKA_ACL_OPERATION_UNKNOWN = 0, + RD_KAFKA_ACL_OPERATION_ANY = 1, + RD_KAFKA_ACL_OPERATION_ALL = 2, + RD_KAFKA_ACL_OPERATION_READ = 3, + RD_KAFKA_ACL_OPERATION_WRITE = 4, + RD_KAFKA_ACL_OPERATION_CREATE = 5, + RD_KAFKA_ACL_OPERATION_DELETE = 6, + RD_KAFKA_ACL_OPERATION_ALTER = 7, + RD_KAFKA_ACL_OPERATION_DESCRIBE = 8, + RD_KAFKA_ACL_OPERATION_CLUSTER_ACTION = 9, + RD_KAFKA_ACL_OPERATION_DESCRIBE_CONFIGS = 10, + RD_KAFKA_ACL_OPERATION_ALTER_CONFIGS = 11, + RD_KAFKA_ACL_OPERATION_IDEMPOTENT_WRITE = 12, + RD_KAFKA_ACL_OPERATION__CNT = 13, +} #[repr(C)] #[derive(Debug, Copy, Clone)] pub struct rd_kafka_NewTopic_s { @@ -2612,6 +2702,130 @@ extern "C" { } #[repr(C)] #[derive(Debug, Copy, Clone)] +pub struct rd_kafka_TopicCollection_s { + _unused: [u8; 0], +} +pub type rd_kafka_TopicCollection_t = rd_kafka_TopicCollection_s; +#[repr(C)] +#[derive(Debug, Copy, Clone)] +pub struct rd_kafka_TopicPartitionInfo_s { + _unused: [u8; 0], +} +pub type rd_kafka_TopicPartitionInfo_t = rd_kafka_TopicPartitionInfo_s; +#[repr(C)] +#[derive(Debug, Copy, Clone)] +pub struct rd_kafka_TopicDescription_s { + _unused: [u8; 0], +} +pub type rd_kafka_TopicDescription_t = rd_kafka_TopicDescription_s; +extern "C" { + pub fn rd_kafka_TopicCollection_of_topic_names( + topics: *mut *const c_char, + topics_cnt: usize, + ) -> *mut rd_kafka_TopicCollection_t; +} +extern "C" { + pub fn rd_kafka_TopicCollection_destroy(topics: *mut rd_kafka_TopicCollection_t); +} +extern "C" { + pub fn rd_kafka_DescribeTopics( + rk: *mut rd_kafka_t, + topics: *const rd_kafka_TopicCollection_t, + options: *const rd_kafka_AdminOptions_t, + rkqu: *mut rd_kafka_queue_t, + ); +} +extern "C" { + pub fn rd_kafka_DescribeTopics_result_topics( + result: *const rd_kafka_DescribeTopics_result_t, + cntp: *mut usize, + ) -> *mut *const rd_kafka_TopicDescription_t; +} +extern "C" { + pub fn rd_kafka_TopicDescription_partitions( + topicdesc: *const rd_kafka_TopicDescription_t, + cntp: *mut usize, + ) -> *mut *const rd_kafka_TopicPartitionInfo_t; +} +extern "C" { + pub fn rd_kafka_TopicPartitionInfo_partition( + partition: *const rd_kafka_TopicPartitionInfo_t, + ) -> c_int; +} +extern "C" { + pub fn rd_kafka_TopicPartitionInfo_leader( + partition: *const rd_kafka_TopicPartitionInfo_t, + ) -> *const rd_kafka_Node_t; +} +extern "C" { + pub fn rd_kafka_TopicPartitionInfo_isr( + partition: *const rd_kafka_TopicPartitionInfo_t, + cntp: *mut usize, + ) -> *mut *const rd_kafka_Node_t; +} +extern "C" { + pub fn rd_kafka_TopicPartitionInfo_replicas( + partition: *const rd_kafka_TopicPartitionInfo_t, + cntp: *mut usize, + ) -> *mut *const rd_kafka_Node_t; +} +extern "C" { + pub fn rd_kafka_TopicDescription_authorized_operations( + topicdesc: *const rd_kafka_TopicDescription_t, + cntp: *mut usize, + ) -> *const rd_kafka_AclOperation_t; +} +extern "C" { + pub fn rd_kafka_TopicDescription_name( + topicdesc: *const rd_kafka_TopicDescription_t, + ) -> *const c_char; +} +extern "C" { + pub fn rd_kafka_TopicDescription_topic_id( + topicdesc: *const rd_kafka_TopicDescription_t, + ) -> *const rd_kafka_Uuid_t; +} +extern "C" { + pub fn rd_kafka_TopicDescription_is_internal( + topicdesc: *const rd_kafka_TopicDescription_t, + ) -> c_int; +} +extern "C" { + pub fn rd_kafka_TopicDescription_error( + topicdesc: *const rd_kafka_TopicDescription_t, + ) -> *const rd_kafka_error_t; +} +extern "C" { + pub fn rd_kafka_DescribeCluster( + rk: *mut rd_kafka_t, + options: *const rd_kafka_AdminOptions_t, + rkqu: *mut rd_kafka_queue_t, + ); +} +extern "C" { + pub fn rd_kafka_DescribeCluster_result_nodes( + result: *const rd_kafka_DescribeCluster_result_t, + cntp: *mut usize, + ) -> *mut *const rd_kafka_Node_t; +} +extern "C" { + pub fn rd_kafka_DescribeCluster_result_authorized_operations( + result: *const rd_kafka_DescribeCluster_result_t, + cntp: *mut usize, + ) -> *const rd_kafka_AclOperation_t; +} +extern "C" { + pub fn rd_kafka_DescribeCluster_result_controller( + result: *const rd_kafka_DescribeCluster_result_t, + ) -> *const rd_kafka_Node_t; +} +extern "C" { + pub fn rd_kafka_DescribeCluster_result_cluster_id( + result: *const rd_kafka_DescribeCluster_result_t, + ) -> *const c_char; +} +#[repr(C)] +#[derive(Debug, Copy, Clone)] pub struct rd_kafka_ConsumerGroupListing_s { _unused: [u8; 0], } @@ -2709,6 +2923,12 @@ extern "C" { grpdesc: *const rd_kafka_ConsumerGroupDescription_t, ) -> *const c_char; } +extern "C" { + pub fn rd_kafka_ConsumerGroupDescription_authorized_operations( + grpdesc: *const rd_kafka_ConsumerGroupDescription_t, + cntp: *mut usize, + ) -> *const rd_kafka_AclOperation_t; +} extern "C" { pub fn rd_kafka_ConsumerGroupDescription_state( grpdesc: *const rd_kafka_ConsumerGroupDescription_t, @@ -2907,6 +3127,43 @@ extern "C" { cntp: *mut usize, ) -> *mut *const rd_kafka_group_result_t; } +#[repr(i32)] +#[derive(Debug, Copy, Clone, Hash, PartialEq, Eq)] +pub enum rd_kafka_OffsetSpec_t { + RD_KAFKA_OFFSET_SPEC_MAX_TIMESTAMP = -3, + RD_KAFKA_OFFSET_SPEC_EARLIEST = -2, + RD_KAFKA_OFFSET_SPEC_LATEST = -1, +} +#[repr(C)] +#[derive(Debug, Copy, Clone)] +pub struct rd_kafka_ListOffsetsResultInfo_s { + _unused: [u8; 0], +} +pub type rd_kafka_ListOffsetsResultInfo_t = rd_kafka_ListOffsetsResultInfo_s; +extern "C" { + pub fn rd_kafka_ListOffsetsResultInfo_topic_partition( + result_info: *const rd_kafka_ListOffsetsResultInfo_t, + ) -> *const rd_kafka_topic_partition_t; +} +extern "C" { + pub fn rd_kafka_ListOffsetsResultInfo_timestamp( + result_info: *const rd_kafka_ListOffsetsResultInfo_t, + ) -> i64; +} +extern "C" { + pub fn rd_kafka_ListOffsets_result_infos( + result: *const rd_kafka_ListOffsets_result_t, + cntp: *mut usize, + ) -> *mut *const rd_kafka_ListOffsetsResultInfo_t; +} +extern "C" { + pub fn rd_kafka_ListOffsets( + rk: *mut rd_kafka_t, + topic_partitions: *mut rd_kafka_topic_partition_list_t, + options: *const rd_kafka_AdminOptions_t, + rkqu: *mut rd_kafka_queue_t, + ); +} #[repr(u32)] #[derive(Debug, Copy, Clone, Hash, PartialEq, Eq)] pub enum rd_kafka_ScramMechanism_t { @@ -3051,24 +3308,6 @@ extern "C" { aclres: *const rd_kafka_acl_result_t, ) -> *const rd_kafka_error_t; } -#[repr(u32)] -#[derive(Debug, Copy, Clone, Hash, PartialEq, Eq)] -pub enum rd_kafka_AclOperation_t { - RD_KAFKA_ACL_OPERATION_UNKNOWN = 0, - RD_KAFKA_ACL_OPERATION_ANY = 1, - RD_KAFKA_ACL_OPERATION_ALL = 2, - RD_KAFKA_ACL_OPERATION_READ = 3, - RD_KAFKA_ACL_OPERATION_WRITE = 4, - RD_KAFKA_ACL_OPERATION_CREATE = 5, - RD_KAFKA_ACL_OPERATION_DELETE = 6, - RD_KAFKA_ACL_OPERATION_ALTER = 7, - RD_KAFKA_ACL_OPERATION_DESCRIBE = 8, - RD_KAFKA_ACL_OPERATION_CLUSTER_ACTION = 9, - RD_KAFKA_ACL_OPERATION_DESCRIBE_CONFIGS = 10, - RD_KAFKA_ACL_OPERATION_ALTER_CONFIGS = 11, - RD_KAFKA_ACL_OPERATION_IDEMPOTENT_WRITE = 12, - RD_KAFKA_ACL_OPERATION__CNT = 13, -} extern "C" { pub fn rd_kafka_AclOperation_name(acl_operation: rd_kafka_AclOperation_t) -> *const c_char; } @@ -3407,3 +3646,36 @@ extern "C" { MaxVersion: i16, ) -> rd_kafka_resp_err_t; } +extern "C" { + pub fn rd_kafka_mock_start_request_tracking(mcluster: *mut rd_kafka_mock_cluster_t); +} +extern "C" { + pub fn rd_kafka_mock_stop_request_tracking(mcluster: *mut rd_kafka_mock_cluster_t); +} +#[repr(C)] +#[derive(Debug, Copy, Clone)] +pub struct rd_kafka_mock_request_s { + _unused: [u8; 0], +} +pub type rd_kafka_mock_request_t = rd_kafka_mock_request_s; +extern "C" { + pub fn rd_kafka_mock_request_destroy(mreq: *mut rd_kafka_mock_request_t); +} +extern "C" { + pub fn rd_kafka_mock_request_id(mreq: *mut rd_kafka_mock_request_t) -> i32; +} +extern "C" { + pub fn rd_kafka_mock_request_api_key(mreq: *mut rd_kafka_mock_request_t) -> i16; +} +extern "C" { + pub fn rd_kafka_mock_request_timestamp(mreq: *mut rd_kafka_mock_request_t) -> i64; +} +extern "C" { + pub fn rd_kafka_mock_get_requests( + mcluster: *mut rd_kafka_mock_cluster_t, + cntp: *mut usize, + ) -> *mut *mut rd_kafka_mock_request_t; +} +extern "C" { + pub fn rd_kafka_mock_clear_requests(mcluster: *mut rd_kafka_mock_cluster_t); +} From 2990bc1cd6c96882332d13ae15d2d01557e11066 Mon Sep 17 00:00:00 2001 From: David Blewett Date: Fri, 3 Nov 2023 16:37:00 -0400 Subject: [PATCH 03/59] Add rust-version, so that our MSRV is enforced by cargo: https://doc.rust-lang.org/cargo/reference/manifest.html#the-rust-version-field --- Cargo.toml | 1 + changelog.md | 1 + rdkafka-sys/Cargo.toml | 1 + rdkafka-sys/changelog.md | 1 + 4 files changed, 4 insertions(+) diff --git a/Cargo.toml b/Cargo.toml index 7544d41fd..e71a7b0bb 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -10,6 +10,7 @@ keywords = ["kafka", "rdkafka"] categories = ["api-bindings"] edition = "2018" exclude = ["Cargo.lock"] +rust-version = "1.61" [workspace] members = ["rdkafka-sys"] diff --git a/changelog.md b/changelog.md index c3ae2884c..f14092dd3 100644 --- a/changelog.md +++ b/changelog.md @@ -5,6 +5,7 @@ See also the [rdkafka-sys changelog](rdkafka-sys/changelog.md). ## Unreleased * Update bundled librdkafka to 2.3.0. +* Add cargo enforcement of MSRV of 1.61. ## 0.34.0 (2023-08-25) diff --git a/rdkafka-sys/Cargo.toml b/rdkafka-sys/Cargo.toml index 9ea117031..9d4f61118 100644 --- a/rdkafka-sys/Cargo.toml +++ b/rdkafka-sys/Cargo.toml @@ -10,6 +10,7 @@ description = "Native bindings to the librdkafka library" keywords = ["kafka", "rdkafka"] categories = ["external-ffi-bindings"] edition = "2018" +rust-version = "1.61" [dependencies] num_enum = "0.5.0" diff --git a/rdkafka-sys/changelog.md b/rdkafka-sys/changelog.md index a56299ec5..9eefa812b 100644 --- a/rdkafka-sys/changelog.md +++ b/rdkafka-sys/changelog.md @@ -3,6 +3,7 @@ ## Unreleased * Upgrade to librdkafka v2.3.0. +* Add cargo enforcement of MSRV of 1.61. ## v4.6.0+2.2.0 (2023-08-25) From 4753bf6db9722983aa41a8bf473ac676423501b5 Mon Sep 17 00:00:00 2001 From: Devin Date: Tue, 7 Nov 2023 10:25:19 -0800 Subject: [PATCH 04/59] Derives `serde::Serialize` on `Statistics` (#616) I realize that there is a `stats_raw` trait method, but it would be better to not have to re-type out the statistics fields myself. This came about because I would like to act on many of the individual fields locally via some aggregations, but then output them back out (along with other fields in a larger status struct) as a JSON-encoded string, e.g.: ```rs #[derive(Serialize)] pub struct Status { pub sample_field: u64, pub sample_field_2: boolean, pub statistics: Statistics, } ``` --- src/statistics.rs | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/src/statistics.rs b/src/statistics.rs index 99a424aef..8c3fc4c45 100644 --- a/src/statistics.rs +++ b/src/statistics.rs @@ -12,10 +12,10 @@ use std::collections::HashMap; -use serde::Deserialize; +use serde::{Deserialize, Serialize}; /// Overall statistics. -#[derive(Deserialize, Debug, Default, Clone)] +#[derive(Serialize, Deserialize, Debug, Default, Clone)] pub struct Statistics { /// The name of the librdkafka handle. pub name: String, @@ -73,7 +73,7 @@ pub struct Statistics { } /// Per-broker statistics. -#[derive(Deserialize, Debug, Default, Clone)] +#[derive(Serialize, Deserialize, Debug, Default, Clone)] pub struct Broker { /// The broker hostname, port, and ID, in the form `HOSTNAME:PORT/ID`. pub name: String, @@ -168,7 +168,7 @@ pub struct Broker { /// /// These values are not exact; they are sampled estimates maintained by an /// HDR histogram in librdkafka. -#[derive(Deserialize, Debug, Default, Clone)] +#[derive(Serialize, Deserialize, Debug, Default, Clone)] pub struct Window { /// The smallest value. pub min: i64, @@ -202,7 +202,7 @@ pub struct Window { } /// A topic and partition specifier. -#[derive(Deserialize, Debug, Default, Clone)] +#[derive(Serialize, Deserialize, Debug, Default, Clone)] pub struct TopicPartition { /// The name of the topic. pub topic: String, @@ -211,7 +211,7 @@ pub struct TopicPartition { } /// Per-topic statistics. -#[derive(Deserialize, Debug, Default, Clone)] +#[derive(Serialize, Deserialize, Debug, Default, Clone)] pub struct Topic { /// The name of the topic. pub topic: String, @@ -226,7 +226,7 @@ pub struct Topic { } /// Per-partition statistics. -#[derive(Deserialize, Debug, Default, Clone)] +#[derive(Serialize, Deserialize, Debug, Default, Clone)] pub struct Partition { /// The partition ID. pub partition: i32, @@ -299,7 +299,7 @@ pub struct Partition { } /// Consumer group manager statistics. -#[derive(Deserialize, Debug, Default, Clone)] +#[derive(Serialize, Deserialize, Debug, Default, Clone)] pub struct ConsumerGroup { /// The local consumer group handler's state. pub state: String, @@ -321,7 +321,7 @@ pub struct ConsumerGroup { } /// Exactly-once semantics statistics. -#[derive(Deserialize, Debug, Default, Clone)] +#[derive(Serialize, Deserialize, Debug, Default, Clone)] pub struct ExactlyOnceSemantics { /// The current idempotent producer state. pub idemp_state: String, From e439b6e0f1943ddc2b8e01a96fa7ed91a68b94c4 Mon Sep 17 00:00:00 2001 From: Samuel Cantero Date: Tue, 7 Nov 2023 15:33:02 -0300 Subject: [PATCH 05/59] Release rdkafka-sys v4.7.0+2.3.0 --- Cargo.lock | 2 +- rdkafka-sys/Cargo.toml | 2 +- rdkafka-sys/changelog.md | 2 ++ 3 files changed, 4 insertions(+), 2 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index b18df341d..33449bc72 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1099,7 +1099,7 @@ dependencies = [ [[package]] name = "rdkafka-sys" -version = "4.6.0+2.2.0" +version = "4.7.0+2.3.0" dependencies = [ "cmake", "curl-sys", diff --git a/rdkafka-sys/Cargo.toml b/rdkafka-sys/Cargo.toml index 9d4f61118..4870e15b3 100644 --- a/rdkafka-sys/Cargo.toml +++ b/rdkafka-sys/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "rdkafka-sys" -version = "4.6.0+2.3.0" +version = "4.7.0+2.3.0" authors = ["Federico Giraud "] build = "build.rs" links = "rdkafka" diff --git a/rdkafka-sys/changelog.md b/rdkafka-sys/changelog.md index 9eefa812b..44451c738 100644 --- a/rdkafka-sys/changelog.md +++ b/rdkafka-sys/changelog.md @@ -2,6 +2,8 @@ ## Unreleased +## v4.7.0+2.2.0 (2023-11-07) + * Upgrade to librdkafka v2.3.0. * Add cargo enforcement of MSRV of 1.61. From 05de3f79a16711b1680da324351fff74a297acc6 Mon Sep 17 00:00:00 2001 From: Samuel Cantero Date: Tue, 7 Nov 2023 15:38:49 -0300 Subject: [PATCH 06/59] Release v0.35.0 --- Cargo.lock | 2 +- Cargo.toml | 4 ++-- changelog.md | 3 +++ 3 files changed, 6 insertions(+), 3 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 33449bc72..b719eeca7 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1070,7 +1070,7 @@ dependencies = [ [[package]] name = "rdkafka" -version = "0.34.0" +version = "0.35.0" dependencies = [ "async-std", "backoff", diff --git a/Cargo.toml b/Cargo.toml index e71a7b0bb..a05b1a30f 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "rdkafka" -version = "0.34.0" +version = "0.35.0" authors = ["Federico Giraud "] repository = "https://github.com/fede1024/rust-rdkafka" readme = "README.md" @@ -16,7 +16,7 @@ rust-version = "1.61" members = ["rdkafka-sys"] [dependencies] -rdkafka-sys = { path = "rdkafka-sys", version = "4.6.0", default-features = false } +rdkafka-sys = { path = "rdkafka-sys", version = "4.7.0", default-features = false } futures-channel = "0.3.0" futures-executor = { version = "0.3.0", optional = true } futures-util = { version = "0.3.0", default-features = false } diff --git a/changelog.md b/changelog.md index f14092dd3..e954bcebc 100644 --- a/changelog.md +++ b/changelog.md @@ -4,8 +4,11 @@ See also the [rdkafka-sys changelog](rdkafka-sys/changelog.md). ## Unreleased +## 0.35.0 (2023-11-07) + * Update bundled librdkafka to 2.3.0. * Add cargo enforcement of MSRV of 1.61. +* Derives serde::Serialize on Statistics ## 0.34.0 (2023-08-25) From 5fbed45531affda6982e53fbefefd4af076bd8fc Mon Sep 17 00:00:00 2001 From: Samuel Cantero Date: Tue, 7 Nov 2023 15:47:51 -0300 Subject: [PATCH 07/59] Update release notes --- CONTRIBUTING.md | 9 +++++---- rdkafka-sys/CONTRIBUTING.md | 6 ++++-- 2 files changed, 9 insertions(+), 6 deletions(-) diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md index eba19b4ca..987ad1ce2 100644 --- a/CONTRIBUTING.md +++ b/CONTRIBUTING.md @@ -50,11 +50,12 @@ autocreation in order for the tests to succeed. ## Releasing -* Ensure `rdkafka-sys` has no unreleased changes. -* Ensure the changelog is up to date. -* Ensure Cargo.toml is up to date. +* Checkout into master and pull the latest changes. +* Ensure `rdkafka-sys` has no unreleased changes. If it does, release `rdkafka-sys` first. +* Ensure the changelog is up to date (i.e not Unreleased changes). * Run `./generate_readme.py > README.md`. +* Bump the version in Cargo.toml and commit locally. +* Run `cargo publish`. * Run `git tag -am $VERSION $VERSION`. * Run `git push`. * Run `git push origin $VERSION`. -* Run `cargo publish`. diff --git a/rdkafka-sys/CONTRIBUTING.md b/rdkafka-sys/CONTRIBUTING.md index e53414e1b..e2f140dcd 100644 --- a/rdkafka-sys/CONTRIBUTING.md +++ b/rdkafka-sys/CONTRIBUTING.md @@ -19,7 +19,9 @@ Then: ## Releasing -* Ensure the changelog is up to date. -* Ensure Cargo.toml is up to date. +* Checkout into master and pull the latest changes. +* Ensure the changelog is up to date (i.e not Unreleased changes). * Run `cd rdkafka-sys && ../generate_readme.py > README.md`. +* Bump the version in Cargo.toml and commit locally. * Run `cargo publish`. +* Push the commit. From e57447f9e2e82f2a86d2be2a26a8b1de5a01bf05 Mon Sep 17 00:00:00 2001 From: Samuel Cantero Date: Fri, 11 Aug 2023 17:00:15 -0400 Subject: [PATCH 08/59] Move to Event-based API --- src/admin.rs | 2 +- src/client.rs | 242 +++++++++++++++++--------------- src/consumer/base_consumer.rs | 211 ++++++++++++++++++---------- src/consumer/stream_consumer.rs | 5 - src/message.rs | 48 ++++++- src/producer/base_producer.rs | 93 +++++++----- 6 files changed, 362 insertions(+), 239 deletions(-) diff --git a/src/admin.rs b/src/admin.rs index bc642a30d..69dba537b 100644 --- a/src/admin.rs +++ b/src/admin.rs @@ -403,7 +403,7 @@ fn start_poll_thread(queue: Arc, should_stop: Arc) -> J .expect("Failed to start polling thread") } -type NativeEvent = NativePtr; +pub(crate) type NativeEvent = NativePtr; unsafe impl KafkaDrop for RDKafkaEvent { const TYPE: &'static str = "event"; diff --git a/src/client.rs b/src/client.rs index c8c31ea3b..635dcb451 100644 --- a/src/client.rs +++ b/src/client.rs @@ -11,19 +11,19 @@ //! [`consumer`]: crate::consumer //! [`producer`]: crate::producer -use std::convert::TryFrom; use std::error::Error; use std::ffi::{CStr, CString}; use std::mem::ManuallyDrop; -use std::os::raw::{c_char, c_void}; +use std::os::raw::c_char; use std::ptr; -use std::slice; use std::string::ToString; use std::sync::Arc; +use libc::c_void; use rdkafka_sys as rdsys; use rdkafka_sys::types::*; +use crate::admin::NativeEvent; use crate::config::{ClientConfig, NativeClientConfig, RDKafkaLogLevel}; use crate::consumer::RebalanceProtocol; use crate::error::{IsError, KafkaError, KafkaResult}; @@ -239,21 +239,6 @@ impl Client { Arc::as_ptr(&context) as *mut c_void, ) }; - unsafe { rdsys::rd_kafka_conf_set_log_cb(native_config.ptr(), Some(native_log_cb::)) }; - unsafe { - rdsys::rd_kafka_conf_set_stats_cb(native_config.ptr(), Some(native_stats_cb::)) - }; - unsafe { - rdsys::rd_kafka_conf_set_error_cb(native_config.ptr(), Some(native_error_cb::)) - }; - if C::ENABLE_REFRESH_OAUTH_TOKEN { - unsafe { - rdsys::rd_kafka_conf_set_oauthbearer_token_refresh_cb( - native_config.ptr(), - Some(native_oauth_refresh_cb::), - ) - }; - } let client_ptr = unsafe { let native_config = ManuallyDrop::new(native_config); @@ -293,6 +278,126 @@ impl Client { &self.context } + pub(crate) fn poll_event( + &self, + queue: Arc, + timeout: Timeout, + ) -> Option { + let event = unsafe { NativeEvent::from_ptr(queue.poll(timeout)) }; + if let Some(ev) = event { + let evtype = unsafe { rdsys::rd_kafka_event_type(ev.ptr()) }; + match evtype { + rdsys::RD_KAFKA_EVENT_LOG => self.handle_log_event(ev.ptr()), + rdsys::RD_KAFKA_EVENT_STATS => self.handle_stats_event(ev.ptr()), + rdsys::RD_KAFKA_EVENT_ERROR => self.handle_error_event(ev.ptr()), + rdsys::RD_KAFKA_EVENT_OAUTHBEARER_TOKEN_REFRESH => { + if C::ENABLE_REFRESH_OAUTH_TOKEN { + self.handle_oauth_refresh_event(ev.ptr()); + } + } + _ => { + return Some(ev); + } + } + } + None + } + + fn handle_log_event(&self, event: *mut RDKafkaEvent) { + let mut fac: *const c_char = std::ptr::null(); + let mut str_: *const c_char = std::ptr::null(); + let mut level: i32 = 0; + let result = unsafe { rdsys::rd_kafka_event_log(event, &mut fac, &mut str_, &mut level) }; + if result == 0 { + let fac = unsafe { CStr::from_ptr(fac).to_string_lossy() }; + let log_message = unsafe { CStr::from_ptr(str_).to_string_lossy() }; + self.context().log( + RDKafkaLogLevel::from_int(level), + fac.trim(), + log_message.trim(), + ); + } + } + + fn handle_stats_event(&self, event: *mut RDKafkaEvent) { + let json = unsafe { CStr::from_ptr(rdsys::rd_kafka_event_stats(event)) }; + self.context().stats_raw(json.to_bytes()); + } + + fn handle_error_event(&self, event: *mut RDKafkaEvent) { + let rdkafka_err = unsafe { rdsys::rd_kafka_event_error(event) }; + let error = KafkaError::Global(rdkafka_err.into()); + let reason = + unsafe { CStr::from_ptr(rdsys::rd_kafka_event_error_string(event)).to_string_lossy() }; + self.context().error(error, reason.trim()); + } + + fn handle_oauth_refresh_event(&self, event: *mut RDKafkaEvent) { + let oauthbearer_config = unsafe { rdsys::rd_kafka_event_config_string(event) }; + let res: Result<_, Box> = (|| { + let oauthbearer_config = match oauthbearer_config.is_null() { + true => None, + false => unsafe { Some(util::cstr_to_owned(oauthbearer_config)) }, + }; + let token_info = self + .context() + .generate_oauth_token(oauthbearer_config.as_deref())?; + let token = CString::new(token_info.token)?; + let principal_name = CString::new(token_info.principal_name)?; + Ok((token, principal_name, token_info.lifetime_ms)) + })(); + match res { + Ok((token, principal_name, lifetime_ms)) => { + let mut err_buf = ErrBuf::new(); + let code = unsafe { + rdkafka_sys::rd_kafka_oauthbearer_set_token( + self.native_ptr(), + token.as_ptr(), + lifetime_ms, + principal_name.as_ptr(), + ptr::null_mut(), + 0, + err_buf.as_mut_ptr(), + err_buf.capacity(), + ) + }; + if code == RDKafkaRespErr::RD_KAFKA_RESP_ERR_NO_ERROR { + debug!("successfully set refreshed OAuth token"); + } else { + debug!( + "failed to set refreshed OAuth token (code {:?}): {}", + code, err_buf + ); + unsafe { + rdkafka_sys::rd_kafka_oauthbearer_set_token_failure( + self.native_ptr(), + err_buf.as_mut_ptr(), + ) + }; + } + } + Err(e) => { + debug!("failed to refresh OAuth token: {}", e); + let message = match CString::new(e.to_string()) { + Ok(message) => message, + Err(e) => { + error!("error message generated while refreshing OAuth token has embedded null character: {}", e); + CString::new( + "error while refreshing OAuth token has embedded null character", + ) + .expect("known to be a valid CString") + } + }; + unsafe { + rdkafka_sys::rd_kafka_oauthbearer_set_token_failure( + self.native_ptr(), + message.as_ptr(), + ) + }; + } + } + } + /// Returns the metadata information for the specified topic, or for all topics in the cluster /// if no topic is specified. pub fn fetch_metadata>( @@ -442,6 +547,11 @@ impl Client { pub(crate) fn consumer_queue(&self) -> Option { unsafe { NativeQueue::from_ptr(rdsys::rd_kafka_queue_get_consumer(self.native_ptr())) } } + + /// Returns a NativeQueue for the main librdkafka event queue from the current client. + pub(crate) fn main_queue(&self) -> NativeQueue { + unsafe { NativeQueue::from_ptr(rdsys::rd_kafka_queue_get_main(self.native_ptr())).unwrap() } + } } pub(crate) type NativeTopic = NativePtr; @@ -471,48 +581,6 @@ impl NativeQueue { } } -pub(crate) unsafe extern "C" fn native_log_cb( - client: *const RDKafka, - level: i32, - fac: *const c_char, - buf: *const c_char, -) { - let fac = CStr::from_ptr(fac).to_string_lossy(); - let log_message = CStr::from_ptr(buf).to_string_lossy(); - - let context = &mut *(rdsys::rd_kafka_opaque(client) as *mut C); - context.log( - RDKafkaLogLevel::from_int(level), - fac.trim(), - log_message.trim(), - ); -} - -pub(crate) unsafe extern "C" fn native_stats_cb( - _conf: *mut RDKafka, - json: *mut c_char, - json_len: usize, - opaque: *mut c_void, -) -> i32 { - let context = &mut *(opaque as *mut C); - context.stats_raw(slice::from_raw_parts(json as *mut u8, json_len)); - 0 // librdkafka will free the json buffer -} - -pub(crate) unsafe extern "C" fn native_error_cb( - _client: *mut RDKafka, - err: i32, - reason: *const c_char, - opaque: *mut c_void, -) { - let err = RDKafkaRespErr::try_from(err).expect("global error not an rd_kafka_resp_err_t"); - let error = KafkaError::Global(err.into()); - let reason = CStr::from_ptr(reason).to_string_lossy(); - - let context = &mut *(opaque as *mut C); - context.error(error, reason.trim()); -} - /// A generated OAuth token and its associated metadata. /// /// When using the `OAUTHBEARER` SASL authentication method, this type is @@ -529,60 +597,6 @@ pub struct OAuthToken { pub lifetime_ms: i64, } -pub(crate) unsafe extern "C" fn native_oauth_refresh_cb( - client: *mut RDKafka, - oauthbearer_config: *const c_char, - opaque: *mut c_void, -) { - let res: Result<_, Box> = (|| { - let context = &mut *(opaque as *mut C); - let oauthbearer_config = match oauthbearer_config.is_null() { - true => None, - false => Some(util::cstr_to_owned(oauthbearer_config)), - }; - let token_info = context.generate_oauth_token(oauthbearer_config.as_deref())?; - let token = CString::new(token_info.token)?; - let principal_name = CString::new(token_info.principal_name)?; - Ok((token, principal_name, token_info.lifetime_ms)) - })(); - match res { - Ok((token, principal_name, lifetime_ms)) => { - let mut err_buf = ErrBuf::new(); - let code = rdkafka_sys::rd_kafka_oauthbearer_set_token( - client, - token.as_ptr(), - lifetime_ms, - principal_name.as_ptr(), - ptr::null_mut(), - 0, - err_buf.as_mut_ptr(), - err_buf.capacity(), - ); - if code == RDKafkaRespErr::RD_KAFKA_RESP_ERR_NO_ERROR { - debug!("successfully set refreshed OAuth token"); - } else { - debug!( - "failed to set refreshed OAuth token (code {:?}): {}", - code, err_buf - ); - rdkafka_sys::rd_kafka_oauthbearer_set_token_failure(client, err_buf.as_mut_ptr()); - } - } - Err(e) => { - debug!("failed to refresh OAuth token: {}", e); - let message = match CString::new(e.to_string()) { - Ok(message) => message, - Err(e) => { - error!("error message generated while refreshing OAuth token has embedded null character: {}", e); - CString::new("error while refreshing OAuth token has embedded null character") - .expect("known to be a valid CString") - } - }; - rdkafka_sys::rd_kafka_oauthbearer_set_token_failure(client, message.as_ptr()); - } - } -} - #[cfg(test)] mod tests { // Just call everything to test there no panics by default, behavior diff --git a/src/consumer/base_consumer.rs b/src/consumer/base_consumer.rs index ee03b906b..1b3bcdfe9 100644 --- a/src/consumer/base_consumer.rs +++ b/src/consumer/base_consumer.rs @@ -1,16 +1,17 @@ //! Low-level consumers. -use std::cmp; -use std::ffi::CString; +use std::ffi::{CStr, CString}; use std::mem::ManuallyDrop; use std::os::raw::c_void; use std::ptr; use std::sync::Arc; +use std::time::Duration; +use log::warn; use rdkafka_sys as rdsys; use rdkafka_sys::types::*; -use crate::client::{Client, NativeClient, NativeQueue}; +use crate::client::{Client, NativeQueue}; use crate::config::{ ClientConfig, FromClientConfig, FromClientConfigAndContext, NativeClientConfig, }; @@ -26,41 +27,6 @@ use crate::metadata::Metadata; use crate::topic_partition_list::{Offset, TopicPartitionList}; use crate::util::{cstr_to_owned, NativePtr, Timeout}; -pub(crate) unsafe extern "C" fn native_commit_cb( - _conf: *mut RDKafka, - err: RDKafkaRespErr, - offsets: *mut RDKafkaTopicPartitionList, - opaque_ptr: *mut c_void, -) { - let context = &mut *(opaque_ptr as *mut C); - let commit_error = if err.is_error() { - Err(KafkaError::ConsumerCommit(err.into())) - } else { - Ok(()) - }; - if offsets.is_null() { - let tpl = TopicPartitionList::new(); - context.commit_callback(commit_error, &tpl); - } else { - let tpl = ManuallyDrop::new(TopicPartitionList::from_ptr(offsets)); - context.commit_callback(commit_error, &tpl); - } -} - -/// Native rebalance callback. This callback will run on every rebalance, and it will call the -/// rebalance method defined in the current `Context`. -unsafe extern "C" fn native_rebalance_cb( - rk: *mut RDKafka, - err: RDKafkaRespErr, - native_tpl: *mut RDKafkaTopicPartitionList, - opaque_ptr: *mut c_void, -) { - let context = &mut *(opaque_ptr as *mut C); - let native_client = ManuallyDrop::new(NativeClient::from_ptr(rk)); - let mut tpl = ManuallyDrop::new(TopicPartitionList::from_ptr(native_tpl)); - context.rebalance(&native_client, err, &mut tpl); -} - /// A low-level consumer that requires manual polling. /// /// This consumer must be periodically polled to make progress on rebalancing, @@ -70,7 +36,8 @@ where C: ConsumerContext, { client: Client, - main_queue_min_poll_interval: Timeout, + queue: Arc, + static_member: bool, } impl FromClientConfig for BaseConsumer { @@ -95,58 +62,50 @@ where native_config: NativeClientConfig, context: C, ) -> KafkaResult> { + let mut static_member = false; + if let Some(group_instance_id) = config.get("group.instance.id") { + if !group_instance_id.is_empty() { + static_member = true; + } + } unsafe { - rdsys::rd_kafka_conf_set_rebalance_cb( - native_config.ptr(), - Some(native_rebalance_cb::), - ); - rdsys::rd_kafka_conf_set_offset_commit_cb( + rdsys::rd_kafka_conf_set_events( native_config.ptr(), - Some(native_commit_cb::), - ); - } - let main_queue_min_poll_interval = context.main_queue_min_poll_interval(); + rdsys::RD_KAFKA_EVENT_REBALANCE + | rdsys::RD_KAFKA_EVENT_OFFSET_COMMIT + | rdsys::RD_KAFKA_EVENT_STATS + | rdsys::RD_KAFKA_EVENT_ERROR + | rdsys::RD_KAFKA_EVENT_OAUTHBEARER_TOKEN_REFRESH, + ) + }; let client = Client::new( config, native_config, RDKafkaType::RD_KAFKA_CONSUMER, context, )?; + + // Redirect rdkafka's main queue to the consumer queue so that we only + // need to listen to the consumer queue to observe events like + // rebalancings and stats. + unsafe { rdsys::rd_kafka_poll_set_consumer(client.native_ptr()) }; + let queue = Arc::new(client.consumer_queue().ok_or(KafkaError::ClientCreation( + "rdkafka consumer queue not available".to_string(), + ))?); Ok(BaseConsumer { client, - main_queue_min_poll_interval, + queue, + static_member, }) } - /// Polls the consumer for messages and returns a pointer to the native rdkafka-sys struct. - /// This method is for internal use only. Use poll instead. - pub(crate) fn poll_raw(&self, mut timeout: Timeout) -> Option> { - loop { - unsafe { rdsys::rd_kafka_poll(self.client.native_ptr(), 0) }; - let op_timeout = cmp::min(timeout, self.main_queue_min_poll_interval); - let message_ptr = unsafe { - NativePtr::from_ptr(rdsys::rd_kafka_consumer_poll( - self.client.native_ptr(), - op_timeout.as_millis(), - )) - }; - if let Some(message_ptr) = message_ptr { - break Some(message_ptr); - } - if op_timeout >= timeout { - break None; - } - timeout -= op_timeout; - } - } - /// Polls the consumer for new messages. /// /// It won't block for more than the specified timeout. Use zero `Duration` for non-blocking /// call. With no timeout it blocks until an event is received. /// /// This method should be called at regular intervals, even if no message is expected, - /// to serve any queued callbacks waiting to be called. This is especially important for + /// to serve any queued events waiting to be handled. This is especially important for /// automatic consumer rebalance, as the rebalance function will be executed by the thread /// calling the poll() function. /// @@ -154,8 +113,84 @@ where /// /// The returned message lives in the memory of the consumer and cannot outlive it. pub fn poll>(&self, timeout: T) -> Option>> { - self.poll_raw(timeout.into()) - .map(|ptr| unsafe { BorrowedMessage::from_consumer(ptr, self) }) + let event = Arc::new( + self.client() + .poll_event(self.queue.clone(), timeout.into())?, + ); + let evtype = unsafe { rdsys::rd_kafka_event_type(event.ptr()) }; + match evtype { + rdsys::RD_KAFKA_EVENT_FETCH => self.handle_fetch_event(event.clone()), + rdsys::RD_KAFKA_EVENT_REBALANCE => { + self.handle_rebalance_event(event.clone()); + None + } + rdsys::RD_KAFKA_EVENT_OFFSET_COMMIT => { + self.handle_offset_commit_event(event.clone()); + None + } + _ => { + let buf = unsafe { + let evname = rdsys::rd_kafka_event_name(event.ptr()); + CStr::from_ptr(evname).to_bytes() + }; + let evname = String::from_utf8(buf.to_vec()).unwrap(); + warn!("Ignored event '{}' on consumer poll", evname); + None + } + } + } + + fn handle_fetch_event( + &self, + event: Arc>, + ) -> Option>> { + unsafe { + NativePtr::from_ptr(rdsys::rd_kafka_event_message_next(event.ptr()) as *mut _) + .map(|ptr| BorrowedMessage::from_fetch_event(ptr, event.clone())) + } + } + + fn handle_rebalance_event(&self, event: Arc>) { + let err = unsafe { rdsys::rd_kafka_event_error(event.ptr()) }; + match err { + rdsys::rd_kafka_resp_err_t::RD_KAFKA_RESP_ERR__ASSIGN_PARTITIONS + | rdsys::rd_kafka_resp_err_t::RD_KAFKA_RESP_ERR__REVOKE_PARTITIONS => { + let tpl = unsafe { + let native_tpl = rdsys::rd_kafka_event_topic_partition_list(event.ptr()); + TopicPartitionList::from_ptr(native_tpl) + }; + let mut tpl = ManuallyDrop::new(tpl); + self.context() + .rebalance(self.client.native_client(), err, &mut tpl); + } + _ => { + let buf = unsafe { + let err_name = + rdsys::rd_kafka_err2name(rdsys::rd_kafka_event_error(event.ptr())); + CStr::from_ptr(err_name).to_bytes() + }; + let err = String::from_utf8(buf.to_vec()).unwrap(); + warn!("invalid rebalance event: {:?}", err); + } + } + } + + fn handle_offset_commit_event(&self, event: Arc>) { + let err = unsafe { rdsys::rd_kafka_event_error(event.ptr()) }; + let commit_error = if err.is_error() { + Err(KafkaError::ConsumerCommit(err.into())) + } else { + Ok(()) + }; + + let offsets = unsafe { rdsys::rd_kafka_event_topic_partition_list(event.ptr()) }; + if offsets.is_null() { + let tpl = TopicPartitionList::new(); + self.context().commit_callback(commit_error, &tpl); + } else { + let tpl = ManuallyDrop::new(unsafe { TopicPartitionList::from_ptr(offsets) }); + self.context().commit_callback(commit_error, &tpl); + } } /// Returns an iterator over the available messages. @@ -607,7 +642,31 @@ where C: ConsumerContext, { fn drop(&mut self) { - trace!("Destroying consumer: {:?}", self.client.native_ptr()); // TODO: fix me (multiple executions ?) + trace!("Destroying consumer: {:?}", self.client.native_ptr()); + // If this consumer is configured for static membership, do not explicitly unsubscribe from + // the group. Note that static members will *not* receive a final revoke event when they + // shutdown. + if !self.static_member { + // We use the Event API rather than the Callback API. + // As we don't register a rebalance_cb, rd_kafka_consumer_close() + // will shortcut the rebalance_cb and do a forced unassign. + // This is undesired as the application might need the final + // revoke events before shutting down. Hence, we trigger + // an Unsubscribe() first, wait for that to propagate, and then + // close the consumer. + unsafe { rdsys::rd_kafka_unsubscribe(self.client.native_ptr()) }; + + // Poll for rebalance events + loop { + self.poll(Duration::from_secs(5)); + let qlen = unsafe { rdsys::rd_kafka_queue_length(self.queue.ptr()) }; + if qlen == 0 { + break; + } + } + } + + // TODO(sam): do we need to destroy the queue before calling close? unsafe { rdsys::rd_kafka_consumer_close(self.client.native_ptr()) }; trace!("Consumer destroyed: {:?}", self.client.native_ptr()); } @@ -677,7 +736,7 @@ where /// /// Remember that you must also call [`BaseConsumer::poll`] on the /// associated consumer regularly, even if no messages are expected, to - /// serve callbacks. + /// serve events. pub fn poll>(&self, timeout: T) -> Option>> { unsafe { NativePtr::from_ptr(rdsys::rd_kafka_consume_queue( diff --git a/src/consumer/stream_consumer.rs b/src/consumer/stream_consumer.rs index 0c959f329..fd59fd138 100644 --- a/src/consumer/stream_consumer.rs +++ b/src/consumer/stream_consumer.rs @@ -200,11 +200,6 @@ where let base = BaseConsumer::new(config, native_config, context)?; let native_ptr = base.client().native_ptr() as usize; - // Redirect rdkafka's main queue to the consumer queue so that we only - // need to listen to the consumer queue to observe events like - // rebalancings and stats. - unsafe { rdsys::rd_kafka_poll_set_consumer(base.client().native_ptr()) }; - let queue = base.client().consumer_queue().ok_or_else(|| { KafkaError::ClientCreation("librdkafka failed to create consumer queue".into()) })?; diff --git a/src/message.rs b/src/message.rs index 0f47baebe..08733903c 100644 --- a/src/message.rs +++ b/src/message.rs @@ -6,11 +6,13 @@ use std::marker::PhantomData; use std::os::raw::c_void; use std::ptr; use std::str; +use std::sync::Arc; use std::time::SystemTime; use rdkafka_sys as rdsys; use rdkafka_sys::types::*; +use crate::admin::NativeEvent; use crate::error::{IsError, KafkaError, KafkaResult}; use crate::util::{self, millis_to_epoch, KafkaDrop, NativePtr}; @@ -306,12 +308,15 @@ impl Headers for BorrowedHeaders { /// [`detach`](BorrowedMessage::detach) method. pub struct BorrowedMessage<'a> { ptr: NativePtr, + _event: Option>, _owner: PhantomData<&'a u8>, } +unsafe extern "C" fn no_op(_: *mut RDKafkaMessage) {} + unsafe impl KafkaDrop for RDKafkaMessage { const TYPE: &'static str = "message"; - const DROP: unsafe extern "C" fn(*mut Self) = rdsys::rd_kafka_message_destroy; + const DROP: unsafe extern "C" fn(*mut Self) = no_op; } impl<'a> fmt::Debug for BorrowedMessage<'a> { @@ -342,22 +347,51 @@ impl<'a> BorrowedMessage<'a> { } else { Ok(BorrowedMessage { ptr, + _event: None, + _owner: PhantomData, + }) + } + } + + /// Creates a new `BorrowedMessage` that wraps the native Kafka message + /// pointer returned by a consumer. The lifetime of the message will be + /// bound to the lifetime of the event passed as parameter. If the message + /// contains an error, only the error is returned and the message structure + /// is freed. + pub(crate) unsafe fn from_fetch_event( + ptr: NativePtr, + event: Arc, + ) -> KafkaResult> { + if ptr.err.is_error() { + let err = match ptr.err { + rdsys::rd_kafka_resp_err_t::RD_KAFKA_RESP_ERR__PARTITION_EOF => { + KafkaError::PartitionEOF((*ptr).partition) + } + e => KafkaError::MessageConsumption(e.into()), + }; + Err(err) + } else { + Ok(BorrowedMessage { + ptr, + _event: Some(event), + // TODO(sam): what does it mean this when the event holds the ownership? _owner: PhantomData, }) } } /// Creates a new `BorrowedMessage` that wraps the native Kafka message - /// pointer returned by the delivery callback of a producer. The lifetime of - /// the message will be bound to the lifetime of the reference passed as - /// parameter. This method should only be used with messages coming from the - /// delivery callback. The message will not be freed in any circumstance. - pub(crate) unsafe fn from_dr_callback( + /// pointer returned via the delivery report event. The lifetime of + /// the message will be bound to the lifetime of the event passed as + /// parameter. The message will not be freed in any circumstance. + pub(crate) unsafe fn from_dr_event( ptr: *mut RDKafkaMessage, - _owner: &'a O, + event: Arc, ) -> DeliveryResult<'a> { let borrowed_message = BorrowedMessage { ptr: NativePtr::from_ptr(ptr).unwrap(), + _event: Some(event), + // TODO(sam): what does it mean this when the event holds the ownership? _owner: PhantomData, }; if (*ptr).err.is_error() { diff --git a/src/producer/base_producer.rs b/src/producer/base_producer.rs index 48acd925d..1c6a3ab38 100644 --- a/src/producer/base_producer.rs +++ b/src/producer/base_producer.rs @@ -57,7 +57,7 @@ use rdkafka_sys as rdsys; use rdkafka_sys::rd_kafka_vtype_t::*; use rdkafka_sys::types::*; -use crate::client::Client; +use crate::client::{Client, NativeQueue}; use crate::config::{ClientConfig, FromClientConfig, FromClientConfigAndContext}; use crate::consumer::ConsumerGroupMetadata; use crate::error::{IsError, KafkaError, KafkaResult, RDKafkaError}; @@ -67,33 +67,12 @@ use crate::producer::{ DefaultProducerContext, Partitioner, Producer, ProducerContext, PurgeConfig, }; use crate::topic_partition_list::TopicPartitionList; -use crate::util::{IntoOpaque, Timeout}; +use crate::util::{IntoOpaque, NativePtr, Timeout}; pub use crate::message::DeliveryResult; use super::NoCustomPartitioner; -/// Callback that gets called from librdkafka every time a message succeeds or fails to be -/// delivered. -unsafe extern "C" fn delivery_cb>( - _client: *mut RDKafka, - msg: *const RDKafkaMessage, - opaque: *mut c_void, -) { - let producer_context = &mut *(opaque as *mut C); - let delivery_opaque = C::DeliveryOpaque::from_ptr((*msg)._private); - let owner = 42u8; - // Wrap the message pointer into a BorrowedMessage that will only live for the body of this - // function. - let delivery_result = BorrowedMessage::from_dr_callback(msg as *mut RDKafkaMessage, &owner); - trace!("Delivery event received: {:?}", delivery_result); - producer_context.delivery(&delivery_result, delivery_opaque); - match delivery_result { - // Do not free the message, librdkafka will do it for us - Ok(message) | Err((_, message)) => mem::forget(message), - } -} - // // ********** BASE PRODUCER ********** // @@ -294,7 +273,13 @@ where } unsafe { - rdsys::rd_kafka_conf_set_dr_msg_cb(native_config.ptr(), Some(delivery_cb::)) + rdsys::rd_kafka_conf_set_events( + native_config.ptr(), + rdsys::RD_KAFKA_EVENT_DR + | rdsys::RD_KAFKA_EVENT_STATS + | rdsys::RD_KAFKA_EVENT_ERROR + | rdsys::RD_KAFKA_EVENT_OAUTHBEARER_TOKEN_REFRESH, + ) }; let client = Client::new_context_arc( config, @@ -351,6 +336,7 @@ where C: ProducerContext, { client: Client, + queue: Arc, _partitioner: PhantomData, } @@ -361,18 +347,57 @@ where { /// Creates a base producer starting from a Client. fn from_client(client: Client) -> BaseProducer { + let queue = Arc::new(client.main_queue()); BaseProducer { client, + queue, _partitioner: PhantomData, } } - /// Polls the producer, returning the number of events served. + /// Polls the producer /// /// Regular calls to `poll` are required to process the events and execute /// the message delivery callbacks. - pub fn poll>(&self, timeout: T) -> i32 { - unsafe { rdsys::rd_kafka_poll(self.native_ptr(), timeout.into().as_millis()) } + pub fn poll>(&self, timeout: T) { + let event = self.client().poll_event(self.queue.clone(), timeout.into()); + if let Some(ev) = event { + let ev = Arc::new(ev); + let evtype = unsafe { rdsys::rd_kafka_event_type(ev.ptr()) }; + match evtype { + rdsys::RD_KAFKA_EVENT_DR => self.handle_delivery_report_event(ev), + _ => { + let buf = unsafe { + let evname = rdsys::rd_kafka_event_name(ev.ptr()); + CStr::from_ptr(evname).to_bytes() + }; + let evname = String::from_utf8(buf.to_vec()).unwrap(); + warn!("Ignored event '{}' on base producer poll", evname); + } + } + } + } + + fn handle_delivery_report_event(&self, event: Arc>) { + let max_messages = unsafe { rdsys::rd_kafka_event_message_count(event.ptr()) }; + let messages: Vec<*const RDKafkaMessage> = Vec::with_capacity(max_messages); + + let mut messages = mem::ManuallyDrop::new(messages); + let messages = unsafe { + let msgs_cnt = rdsys::rd_kafka_event_message_array( + event.ptr(), + messages.as_mut_ptr(), + max_messages, + ); + Vec::from_raw_parts(messages.as_mut_ptr(), msgs_cnt, max_messages) + }; + + for msg in messages { + let delivery_result = + unsafe { BorrowedMessage::from_dr_event(msg as *mut _, event.clone()) }; + let delivery_opaque = unsafe { C::DeliveryOpaque::from_ptr((*msg)._private) }; + self.context().delivery(&delivery_result, delivery_opaque); + } } /// Returns a pointer to the native Kafka client. @@ -618,15 +643,11 @@ where .spawn(move || { trace!("Polling thread loop started"); loop { - let n = producer.poll(Duration::from_millis(100)); - if n == 0 { - if should_stop.load(Ordering::Relaxed) { - // We received nothing and the thread should - // stop, so break the loop. - break; - } - } else { - trace!("Received {} events", n); + producer.poll(Duration::from_millis(100)); + if should_stop.load(Ordering::Relaxed) { + // We received nothing and the thread should + // stop, so break the loop. + break; } } trace!("Polling thread loop terminated"); From 75ff5d81680e4cdc9452f2138dc4eebcc78f5dd0 Mon Sep 17 00:00:00 2001 From: Samuel Cantero Date: Tue, 17 Oct 2023 10:54:17 +0200 Subject: [PATCH 09/59] Adapt the StreamConsumer to poll the underlying BaseConsumer --- src/consumer/base_consumer.rs | 4 ++++ src/consumer/stream_consumer.rs | 40 ++++++++++++--------------------- 2 files changed, 18 insertions(+), 26 deletions(-) diff --git a/src/consumer/base_consumer.rs b/src/consumer/base_consumer.rs index 1b3bcdfe9..f4c56aa30 100644 --- a/src/consumer/base_consumer.rs +++ b/src/consumer/base_consumer.rs @@ -237,6 +237,10 @@ where Iter(self) } + pub(crate) fn get_queue(&self) -> Arc { + self.queue.clone() + } + /// Splits messages for the specified partition into their own queue. /// /// If the `topic` or `partition` is invalid, returns `None`. diff --git a/src/consumer/stream_consumer.rs b/src/consumer/stream_consumer.rs index fd59fd138..df01e56f5 100644 --- a/src/consumer/stream_consumer.rs +++ b/src/consumer/stream_consumer.rs @@ -38,7 +38,7 @@ unsafe extern "C" fn native_message_queue_nonempty_cb(_: *mut RDKafka, opaque_pt wakers.wake_all(); } -unsafe fn enable_nonempty_callback(queue: &NativeQueue, wakers: &Arc) { +unsafe fn enable_nonempty_callback(queue: &Arc, wakers: &Arc) { rdsys::rd_kafka_queue_cb_event_enable( queue.ptr(), Some(native_message_queue_nonempty_cb), @@ -89,31 +89,24 @@ impl WakerSlab { /// A stream of messages from a [`StreamConsumer`]. /// /// See the documentation of [`StreamConsumer::stream`] for details. -pub struct MessageStream<'a> { +pub struct MessageStream<'a, C: ConsumerContext> { wakers: &'a WakerSlab, - queue: &'a NativeQueue, + base: Arc>, slot: usize, } -impl<'a> MessageStream<'a> { - fn new(wakers: &'a WakerSlab, queue: &'a NativeQueue) -> MessageStream<'a> { +impl<'a, C: ConsumerContext> MessageStream<'a, C> { + fn new(wakers: &'a WakerSlab, base: Arc>) -> MessageStream<'a, C> { let slot = wakers.register(); - MessageStream { - wakers, - queue, - slot, - } + MessageStream { wakers, base, slot } } fn poll(&self) -> Option>> { - unsafe { - NativePtr::from_ptr(rdsys::rd_kafka_consume_queue(self.queue.ptr(), 0)) - .map(|p| BorrowedMessage::from_consumer(p, self.queue)) - } + self.base.poll(Duration::ZERO) } } -impl<'a> Stream for MessageStream<'a> { +impl<'a, C: ConsumerContext> Stream for MessageStream<'a, C> { type Item = KafkaResult>; fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { @@ -140,7 +133,7 @@ impl<'a> Stream for MessageStream<'a> { } } -impl<'a> Drop for MessageStream<'a> { +impl<'a, C: ConsumerContext> Drop for MessageStream<'a, C> { fn drop(&mut self) { self.wakers.unregister(self.slot); } @@ -165,8 +158,7 @@ pub struct StreamConsumer where C: ConsumerContext, { - queue: NativeQueue, // queue must be dropped before the base to avoid deadlock - base: BaseConsumer, + base: Arc>, wakers: Arc, _shutdown_trigger: oneshot::Sender<()>, _runtime: PhantomData, @@ -197,14 +189,11 @@ where Duration::from_millis(millis) }; - let base = BaseConsumer::new(config, native_config, context)?; + let base = Arc::new(BaseConsumer::new(config, native_config, context)?); let native_ptr = base.client().native_ptr() as usize; - let queue = base.client().consumer_queue().ok_or_else(|| { - KafkaError::ClientCreation("librdkafka failed to create consumer queue".into()) - })?; let wakers = Arc::new(WakerSlab::new()); - unsafe { enable_nonempty_callback(&queue, &wakers) } + unsafe { enable_nonempty_callback(&base.get_queue(), &wakers) } // We need to make sure we poll the consumer at least once every max // poll interval, *unless* the processing task has wedged. To accomplish @@ -236,7 +225,6 @@ where Ok(StreamConsumer { base, wakers, - queue, _shutdown_trigger: shutdown_trigger, _runtime: PhantomData, }) @@ -259,8 +247,8 @@ where /// /// If you want multiple independent views of a Kafka topic, create multiple /// consumers, not multiple message streams. - pub fn stream(&self) -> MessageStream<'_> { - MessageStream::new(&self.wakers, &self.queue) + pub fn stream(&self) -> MessageStream<'_, C> { + MessageStream::new(&self.wakers, self.base.clone()) } /// Receives the next message from the stream. From 8a42919a2ed70f181415412d20bf695e994f99e4 Mon Sep 17 00:00:00 2001 From: David Blewett Date: Wed, 18 Oct 2023 11:03:57 +0200 Subject: [PATCH 10/59] Pass arc by value rather than reference and fix generic type. --- src/consumer/stream_consumer.rs | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/consumer/stream_consumer.rs b/src/consumer/stream_consumer.rs index df01e56f5..14326a853 100644 --- a/src/consumer/stream_consumer.rs +++ b/src/consumer/stream_consumer.rs @@ -38,7 +38,7 @@ unsafe extern "C" fn native_message_queue_nonempty_cb(_: *mut RDKafka, opaque_pt wakers.wake_all(); } -unsafe fn enable_nonempty_callback(queue: &Arc, wakers: &Arc) { +unsafe fn enable_nonempty_callback(queue: Arc, wakers: &Arc) { rdsys::rd_kafka_queue_cb_event_enable( queue.ptr(), Some(native_message_queue_nonempty_cb), @@ -193,7 +193,7 @@ where let native_ptr = base.client().native_ptr() as usize; let wakers = Arc::new(WakerSlab::new()); - unsafe { enable_nonempty_callback(&base.get_queue(), &wakers) } + unsafe { enable_nonempty_callback(base.get_queue(), &wakers) } // We need to make sure we poll the consumer at least once every max // poll interval, *unless* the processing task has wedged. To accomplish @@ -332,7 +332,7 @@ where let wakers = Arc::new(WakerSlab::new()); unsafe { rdsys::rd_kafka_queue_forward(queue.ptr(), ptr::null_mut()); - enable_nonempty_callback(&queue, &wakers); + enable_nonempty_callback(Arc::new(queue), &wakers); } StreamPartitionQueue { queue, @@ -555,7 +555,7 @@ where /// /// If you want multiple independent views of a Kafka partition, create /// multiple consumers, not multiple partition streams. - pub fn stream(&self) -> MessageStream<'_> { + pub fn stream(&self) -> MessageStream<'_, C> { MessageStream::new(&self.wakers, &self.queue) } From 2d82d4077ae5096a395db48e0af387093803e155 Mon Sep 17 00:00:00 2001 From: David Blewett Date: Fri, 20 Oct 2023 12:58:59 +0200 Subject: [PATCH 11/59] Refactor to use references and lifetimes rather than Arc. --- src/client.rs | 2 +- src/consumer/base_consumer.rs | 33 +++++++------- src/message.rs | 86 +++++++++++++---------------------- src/producer/base_producer.rs | 14 +++--- 4 files changed, 55 insertions(+), 80 deletions(-) diff --git a/src/client.rs b/src/client.rs index 635dcb451..e71482e99 100644 --- a/src/client.rs +++ b/src/client.rs @@ -280,7 +280,7 @@ impl Client { pub(crate) fn poll_event( &self, - queue: Arc, + queue: &NativeQueue, timeout: Timeout, ) -> Option { let event = unsafe { NativeEvent::from_ptr(queue.poll(timeout)) }; diff --git a/src/consumer/base_consumer.rs b/src/consumer/base_consumer.rs index f4c56aa30..dd2c1db95 100644 --- a/src/consumer/base_consumer.rs +++ b/src/consumer/base_consumer.rs @@ -36,7 +36,7 @@ where C: ConsumerContext, { client: Client, - queue: Arc, + queue: NativeQueue, static_member: bool, } @@ -89,9 +89,9 @@ where // need to listen to the consumer queue to observe events like // rebalancings and stats. unsafe { rdsys::rd_kafka_poll_set_consumer(client.native_ptr()) }; - let queue = Arc::new(client.consumer_queue().ok_or(KafkaError::ClientCreation( + let queue = client.consumer_queue().ok_or(KafkaError::ClientCreation( "rdkafka consumer queue not available".to_string(), - ))?); + ))?; Ok(BaseConsumer { client, queue, @@ -113,19 +113,18 @@ where /// /// The returned message lives in the memory of the consumer and cannot outlive it. pub fn poll>(&self, timeout: T) -> Option>> { - let event = Arc::new( - self.client() - .poll_event(self.queue.clone(), timeout.into())?, - ); + let event = self + .client() + .poll_event(self.get_queue(), timeout.into())?; let evtype = unsafe { rdsys::rd_kafka_event_type(event.ptr()) }; match evtype { - rdsys::RD_KAFKA_EVENT_FETCH => self.handle_fetch_event(event.clone()), + rdsys::RD_KAFKA_EVENT_FETCH => self.handle_fetch_event(event), rdsys::RD_KAFKA_EVENT_REBALANCE => { - self.handle_rebalance_event(event.clone()); + self.handle_rebalance_event(event); None } rdsys::RD_KAFKA_EVENT_OFFSET_COMMIT => { - self.handle_offset_commit_event(event.clone()); + self.handle_offset_commit_event(event); None } _ => { @@ -142,15 +141,15 @@ where fn handle_fetch_event( &self, - event: Arc>, + event: NativePtr, ) -> Option>> { unsafe { NativePtr::from_ptr(rdsys::rd_kafka_event_message_next(event.ptr()) as *mut _) - .map(|ptr| BorrowedMessage::from_fetch_event(ptr, event.clone())) + .map(|ptr| BorrowedMessage::from_client(ptr, event, self.client())) } } - fn handle_rebalance_event(&self, event: Arc>) { + fn handle_rebalance_event(&self, event: NativePtr) { let err = unsafe { rdsys::rd_kafka_event_error(event.ptr()) }; match err { rdsys::rd_kafka_resp_err_t::RD_KAFKA_RESP_ERR__ASSIGN_PARTITIONS @@ -175,7 +174,7 @@ where } } - fn handle_offset_commit_event(&self, event: Arc>) { + fn handle_offset_commit_event(&self, event: NativePtr) { let err = unsafe { rdsys::rd_kafka_event_error(event.ptr()) }; let commit_error = if err.is_error() { Err(KafkaError::ConsumerCommit(err.into())) @@ -237,8 +236,8 @@ where Iter(self) } - pub(crate) fn get_queue(&self) -> Arc { - self.queue.clone() + pub(crate) fn get_queue(&self) -> &NativeQueue { + &self.queue } /// Splits messages for the specified partition into their own queue. @@ -717,7 +716,7 @@ where C: ConsumerContext, { consumer: Arc>, - queue: NativeQueue, + pub(crate) queue: NativeQueue, nonempty_callback: Option>>, } diff --git a/src/message.rs b/src/message.rs index 08733903c..e83971382 100644 --- a/src/message.rs +++ b/src/message.rs @@ -6,7 +6,6 @@ use std::marker::PhantomData; use std::os::raw::c_void; use std::ptr; use std::str; -use std::sync::Arc; use std::time::SystemTime; use rdkafka_sys as rdsys; @@ -308,7 +307,7 @@ impl Headers for BorrowedHeaders { /// [`detach`](BorrowedMessage::detach) method. pub struct BorrowedMessage<'a> { ptr: NativePtr, - _event: Option>, + _event: NativeEvent, _owner: PhantomData<&'a u8>, } @@ -332,9 +331,10 @@ impl<'a> BorrowedMessage<'a> { /// should only be used with messages coming from consumers. If the message /// contains an error, only the error is returned and the message structure /// is freed. - pub(crate) unsafe fn from_consumer( + pub(crate) unsafe fn from_client( ptr: NativePtr, - _consumer: &'a C, + event: NativeEvent, + _client: &'a C, ) -> KafkaResult> { if ptr.err.is_error() { let err = match ptr.err { @@ -347,62 +347,38 @@ impl<'a> BorrowedMessage<'a> { } else { Ok(BorrowedMessage { ptr, - _event: None, + _event: event, _owner: PhantomData, }) } } - /// Creates a new `BorrowedMessage` that wraps the native Kafka message - /// pointer returned by a consumer. The lifetime of the message will be - /// bound to the lifetime of the event passed as parameter. If the message - /// contains an error, only the error is returned and the message structure - /// is freed. - pub(crate) unsafe fn from_fetch_event( - ptr: NativePtr, - event: Arc, - ) -> KafkaResult> { - if ptr.err.is_error() { - let err = match ptr.err { - rdsys::rd_kafka_resp_err_t::RD_KAFKA_RESP_ERR__PARTITION_EOF => { - KafkaError::PartitionEOF((*ptr).partition) - } - e => KafkaError::MessageConsumption(e.into()), - }; - Err(err) - } else { - Ok(BorrowedMessage { - ptr, - _event: Some(event), - // TODO(sam): what does it mean this when the event holds the ownership? - _owner: PhantomData, - }) - } - } - - /// Creates a new `BorrowedMessage` that wraps the native Kafka message - /// pointer returned via the delivery report event. The lifetime of - /// the message will be bound to the lifetime of the event passed as - /// parameter. The message will not be freed in any circumstance. - pub(crate) unsafe fn from_dr_event( - ptr: *mut RDKafkaMessage, - event: Arc, - ) -> DeliveryResult<'a> { - let borrowed_message = BorrowedMessage { - ptr: NativePtr::from_ptr(ptr).unwrap(), - _event: Some(event), - // TODO(sam): what does it mean this when the event holds the ownership? - _owner: PhantomData, - }; - if (*ptr).err.is_error() { - Err(( - KafkaError::MessageProduction((*ptr).err.into()), - borrowed_message, - )) - } else { - Ok(borrowed_message) - } - } + /// Creates a new `BorrowedMessage` that wraps the native Kafka message + /// pointer returned via the delivery report event. The lifetime of + /// the message will be bound to the lifetime of the event passed as + /// parameter. The message will not be freed in any circumstance. + pub(crate) unsafe fn from_dr_event( + ptr: *mut RDKafkaMessage, + //ptr: NativePtr, + event: *mut RDKafkaEvent, + _client: &'a C, + ) -> DeliveryResult<'a> { + let borrowed_message = BorrowedMessage { + ptr: NativePtr::from_ptr(ptr).unwrap(), + //ptr, + _event: NativePtr::from_ptr(event).unwrap(), + // TODO(sam): what does it mean this when the event holds the ownership? + _owner: PhantomData, + }; + if (*ptr).err.is_error() { + Err(( + KafkaError::MessageProduction((*ptr).err.into()), + borrowed_message, + )) + } else { + Ok(borrowed_message) + } + } /// Returns a pointer to the [`RDKafkaMessage`]. pub fn ptr(&self) -> *mut RDKafkaMessage { diff --git a/src/producer/base_producer.rs b/src/producer/base_producer.rs index 1c6a3ab38..d1de5052c 100644 --- a/src/producer/base_producer.rs +++ b/src/producer/base_producer.rs @@ -336,7 +336,7 @@ where C: ProducerContext, { client: Client, - queue: Arc, + queue: NativeQueue, _partitioner: PhantomData, } @@ -347,7 +347,7 @@ where { /// Creates a base producer starting from a Client. fn from_client(client: Client) -> BaseProducer { - let queue = Arc::new(client.main_queue()); + let queue = client.main_queue(); BaseProducer { client, queue, @@ -360,9 +360,8 @@ where /// Regular calls to `poll` are required to process the events and execute /// the message delivery callbacks. pub fn poll>(&self, timeout: T) { - let event = self.client().poll_event(self.queue.clone(), timeout.into()); + let event = self.client().poll_event(&self.queue, timeout.into()); if let Some(ev) = event { - let ev = Arc::new(ev); let evtype = unsafe { rdsys::rd_kafka_event_type(ev.ptr()) }; match evtype { rdsys::RD_KAFKA_EVENT_DR => self.handle_delivery_report_event(ev), @@ -378,7 +377,7 @@ where } } - fn handle_delivery_report_event(&self, event: Arc>) { + fn handle_delivery_report_event(&self, event: NativePtr) { let max_messages = unsafe { rdsys::rd_kafka_event_message_count(event.ptr()) }; let messages: Vec<*const RDKafkaMessage> = Vec::with_capacity(max_messages); @@ -393,8 +392,9 @@ where }; for msg in messages { - let delivery_result = - unsafe { BorrowedMessage::from_dr_event(msg as *mut _, event.clone()) }; + let delivery_result = unsafe { + BorrowedMessage::from_dr_event(msg as *mut _, event.ptr(), self.client()) + }; let delivery_opaque = unsafe { C::DeliveryOpaque::from_ptr((*msg)._private) }; self.context().delivery(&delivery_result, delivery_opaque); } From abc01a857c57e4366dbf4a725db6808c933a381a Mon Sep 17 00:00:00 2001 From: David Blewett Date: Sun, 22 Oct 2023 23:56:41 +0200 Subject: [PATCH 12/59] Work on supporting StreamConsumer via lifetimes instead of Arc. The `Arc>` to `MessageStream` isn't necessary anymore, and the changes to `split_partition_queue` can be reverted as well I think. --- src/consumer/base_consumer.rs | 26 ++++++--- src/consumer/stream_consumer.rs | 95 +++++++++++++++++++-------------- 2 files changed, 74 insertions(+), 47 deletions(-) diff --git a/src/consumer/base_consumer.rs b/src/consumer/base_consumer.rs index dd2c1db95..1f33644d5 100644 --- a/src/consumer/base_consumer.rs +++ b/src/consumer/base_consumer.rs @@ -741,13 +741,27 @@ where /// associated consumer regularly, even if no messages are expected, to /// serve events. pub fn poll>(&self, timeout: T) -> Option>> { - unsafe { - NativePtr::from_ptr(rdsys::rd_kafka_consume_queue( - self.queue.ptr(), - timeout.into().as_millis(), - )) + let event = self.consumer + .client() + .poll_event(&self.queue, timeout.into())?; + let evtype = unsafe { rdsys::rd_kafka_event_type(event.ptr()) }; + match evtype { + rdsys::RD_KAFKA_EVENT_FETCH => { + unsafe { + NativePtr::from_ptr(rdsys::rd_kafka_event_message_next(event.ptr()) as *mut _) + .map(|ptr| BorrowedMessage::from_client(ptr, event, self)) + } + } + _ => { + let buf = unsafe { + let evname = rdsys::rd_kafka_event_name(event.ptr()); + CStr::from_ptr(evname).to_bytes() + }; + let evname = String::from_utf8(buf.to_vec()).unwrap(); + warn!("Ignored event '{}' on consumer poll", evname); + None + } } - .map(|ptr| unsafe { BorrowedMessage::from_consumer(ptr, &self.consumer) }) } /// Sets a callback that will be invoked whenever the queue becomes diff --git a/src/consumer/stream_consumer.rs b/src/consumer/stream_consumer.rs index 14326a853..09ca27694 100644 --- a/src/consumer/stream_consumer.rs +++ b/src/consumer/stream_consumer.rs @@ -1,6 +1,6 @@ //! High-level consumers with a [`Stream`](futures_util::Stream) interface. -use std::ffi::CString; +use std::ffi::CStr; use std::marker::PhantomData; use std::os::raw::c_void; use std::pin::Pin; @@ -14,14 +14,16 @@ use futures_channel::oneshot; use futures_util::future::{self, Either, FutureExt}; use futures_util::pin_mut; use futures_util::stream::{Stream, StreamExt}; +use log::warn; use slab::Slab; use rdkafka_sys as rdsys; use rdkafka_sys::types::*; +use crate::admin::NativeEvent; use crate::client::{Client, NativeQueue}; use crate::config::{ClientConfig, FromClientConfig, FromClientConfigAndContext}; -use crate::consumer::base_consumer::BaseConsumer; +use crate::consumer::base_consumer::{BaseConsumer, PartitionQueue}; use crate::consumer::{ CommitMode, Consumer, ConsumerContext, ConsumerGroupMetadata, DefaultConsumerContext, RebalanceProtocol, @@ -38,7 +40,7 @@ unsafe extern "C" fn native_message_queue_nonempty_cb(_: *mut RDKafka, opaque_pt wakers.wake_all(); } -unsafe fn enable_nonempty_callback(queue: Arc, wakers: &Arc) { +unsafe fn enable_nonempty_callback(queue: &NativeQueue, wakers: &Arc) { rdsys::rd_kafka_queue_cb_event_enable( queue.ptr(), Some(native_message_queue_nonempty_cb), @@ -89,24 +91,47 @@ impl WakerSlab { /// A stream of messages from a [`StreamConsumer`]. /// /// See the documentation of [`StreamConsumer::stream`] for details. -pub struct MessageStream<'a, C: ConsumerContext> { +pub struct MessageStream<'a> { wakers: &'a WakerSlab, - base: Arc>, + queue: &'a NativeQueue, slot: usize, } -impl<'a, C: ConsumerContext> MessageStream<'a, C> { - fn new(wakers: &'a WakerSlab, base: Arc>) -> MessageStream<'a, C> { +impl<'a> MessageStream<'a> { + fn new(wakers: &'a WakerSlab, queue: &'a NativeQueue) -> MessageStream<'a> { let slot = wakers.register(); - MessageStream { wakers, base, slot } + MessageStream { wakers, queue, slot } } fn poll(&self) -> Option>> { - self.base.poll(Duration::ZERO) + let timeout: Timeout = Duration::ZERO.into(); + let event = unsafe { NativeEvent::from_ptr(self.queue.poll(timeout)) }; + if let Some(ev) = event { + let evtype = unsafe { rdsys::rd_kafka_event_type(ev.ptr()) }; + match evtype { + rdsys::RD_KAFKA_EVENT_FETCH => { + unsafe { + NativePtr::from_ptr(rdsys::rd_kafka_event_message_next(ev.ptr()) as *mut _) + .map(|ptr| BorrowedMessage::from_client(ptr, ev, self.queue)) + } + } + _ => { + let buf = unsafe { + let evname = rdsys::rd_kafka_event_name(ev.ptr()); + CStr::from_ptr(evname).to_bytes() + }; + let evname = String::from_utf8(buf.to_vec()).unwrap(); + warn!("Ignored event '{}' on consumer poll", evname); + None + } + } + } else { + None + } } } -impl<'a, C: ConsumerContext> Stream for MessageStream<'a, C> { +impl<'a> Stream for MessageStream<'a> { type Item = KafkaResult>; fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { @@ -133,7 +158,7 @@ impl<'a, C: ConsumerContext> Stream for MessageStream<'a, C> { } } -impl<'a, C: ConsumerContext> Drop for MessageStream<'a, C> { +impl<'a> Drop for MessageStream<'a> { fn drop(&mut self) { self.wakers.unregister(self.slot); } @@ -159,6 +184,7 @@ where C: ConsumerContext, { base: Arc>, + queue: NativeQueue, wakers: Arc, _shutdown_trigger: oneshot::Sender<()>, _runtime: PhantomData, @@ -190,7 +216,9 @@ where }; let base = Arc::new(BaseConsumer::new(config, native_config, context)?); - let native_ptr = base.client().native_ptr() as usize; + let client = base.client(); + let native_ptr = client.native_ptr() as usize; + let queue = client.main_queue(); let wakers = Arc::new(WakerSlab::new()); unsafe { enable_nonempty_callback(base.get_queue(), &wakers) } @@ -225,6 +253,7 @@ where Ok(StreamConsumer { base, wakers, + queue, _shutdown_trigger: shutdown_trigger, _runtime: PhantomData, }) @@ -247,8 +276,8 @@ where /// /// If you want multiple independent views of a Kafka topic, create multiple /// consumers, not multiple message streams. - pub fn stream(&self) -> MessageStream<'_, C> { - MessageStream::new(&self.wakers, self.base.clone()) + pub fn stream(&self) -> MessageStream<'_> { + MessageStream::new(&self.wakers, &self.queue) } /// Receives the next message from the stream. @@ -317,29 +346,13 @@ where topic: &str, partition: i32, ) -> Option> { - let topic = match CString::new(topic) { - Ok(topic) => topic, - Err(_) => return None, - }; - let queue = unsafe { - NativeQueue::from_ptr(rdsys::rd_kafka_queue_get_partition( - self.base.client().native_ptr(), - topic.as_ptr(), - partition, - )) - }; - queue.map(|queue| { - let wakers = Arc::new(WakerSlab::new()); - unsafe { - rdsys::rd_kafka_queue_forward(queue.ptr(), ptr::null_mut()); - enable_nonempty_callback(Arc::new(queue), &wakers); - } - StreamPartitionQueue { - queue, - wakers, - _consumer: self.clone(), - } - }) + self.base + .split_partition_queue(topic, partition) + .map(|queue| { + let wakers = Arc::new(WakerSlab::new()); + StreamPartitionQueue { queue, wakers, _consumer: self.clone() } + }) + } } @@ -534,7 +547,7 @@ pub struct StreamPartitionQueue where C: ConsumerContext, { - queue: NativeQueue, + queue: PartitionQueue, wakers: Arc, _consumer: Arc>, } @@ -555,8 +568,8 @@ where /// /// If you want multiple independent views of a Kafka partition, create /// multiple consumers, not multiple partition streams. - pub fn stream(&self) -> MessageStream<'_, C> { - MessageStream::new(&self.wakers, &self.queue) + pub fn stream(&self) -> MessageStream<'_> { + MessageStream::new(&self.wakers, &self.queue.queue) } /// Receives the next message from the stream. @@ -595,6 +608,6 @@ where C: ConsumerContext, { fn drop(&mut self) { - unsafe { disable_nonempty_callback(&self.queue) } + unsafe { disable_nonempty_callback(&self.queue.queue) } } } From 374855b0a88574c3bbef0741cbf246a47e698546 Mon Sep 17 00:00:00 2001 From: Samuel Cantero Date: Tue, 24 Oct 2023 18:08:29 -0300 Subject: [PATCH 13/59] Use Arc for events in BorrowMessage This is required as multiple write acks are tied to a single event. --- src/consumer/base_consumer.rs | 19 +++++----- src/consumer/stream_consumer.rs | 23 ++++++++----- src/message.rs | 61 +++++++++++++++++---------------- src/producer/base_producer.rs | 6 ++-- 4 files changed, 57 insertions(+), 52 deletions(-) diff --git a/src/consumer/base_consumer.rs b/src/consumer/base_consumer.rs index 1f33644d5..ae6993bbf 100644 --- a/src/consumer/base_consumer.rs +++ b/src/consumer/base_consumer.rs @@ -113,9 +113,7 @@ where /// /// The returned message lives in the memory of the consumer and cannot outlive it. pub fn poll>(&self, timeout: T) -> Option>> { - let event = self - .client() - .poll_event(self.get_queue(), timeout.into())?; + let event = self.client().poll_event(self.get_queue(), timeout.into())?; let evtype = unsafe { rdsys::rd_kafka_event_type(event.ptr()) }; match evtype { rdsys::RD_KAFKA_EVENT_FETCH => self.handle_fetch_event(event), @@ -145,7 +143,7 @@ where ) -> Option>> { unsafe { NativePtr::from_ptr(rdsys::rd_kafka_event_message_next(event.ptr()) as *mut _) - .map(|ptr| BorrowedMessage::from_client(ptr, event, self.client())) + .map(|ptr| BorrowedMessage::from_client(ptr, Arc::new(event), self.client())) } } @@ -741,17 +739,16 @@ where /// associated consumer regularly, even if no messages are expected, to /// serve events. pub fn poll>(&self, timeout: T) -> Option>> { - let event = self.consumer + let event = self + .consumer .client() .poll_event(&self.queue, timeout.into())?; let evtype = unsafe { rdsys::rd_kafka_event_type(event.ptr()) }; match evtype { - rdsys::RD_KAFKA_EVENT_FETCH => { - unsafe { - NativePtr::from_ptr(rdsys::rd_kafka_event_message_next(event.ptr()) as *mut _) - .map(|ptr| BorrowedMessage::from_client(ptr, event, self)) - } - } + rdsys::RD_KAFKA_EVENT_FETCH => unsafe { + NativePtr::from_ptr(rdsys::rd_kafka_event_message_next(event.ptr()) as *mut _) + .map(|ptr| BorrowedMessage::from_client(ptr, Arc::new(event), self)) + }, _ => { let buf = unsafe { let evname = rdsys::rd_kafka_event_name(event.ptr()); diff --git a/src/consumer/stream_consumer.rs b/src/consumer/stream_consumer.rs index 09ca27694..28d672b22 100644 --- a/src/consumer/stream_consumer.rs +++ b/src/consumer/stream_consumer.rs @@ -100,7 +100,11 @@ pub struct MessageStream<'a> { impl<'a> MessageStream<'a> { fn new(wakers: &'a WakerSlab, queue: &'a NativeQueue) -> MessageStream<'a> { let slot = wakers.register(); - MessageStream { wakers, queue, slot } + MessageStream { + wakers, + queue, + slot, + } } fn poll(&self) -> Option>> { @@ -109,12 +113,10 @@ impl<'a> MessageStream<'a> { if let Some(ev) = event { let evtype = unsafe { rdsys::rd_kafka_event_type(ev.ptr()) }; match evtype { - rdsys::RD_KAFKA_EVENT_FETCH => { - unsafe { - NativePtr::from_ptr(rdsys::rd_kafka_event_message_next(ev.ptr()) as *mut _) - .map(|ptr| BorrowedMessage::from_client(ptr, ev, self.queue)) - } - } + rdsys::RD_KAFKA_EVENT_FETCH => unsafe { + NativePtr::from_ptr(rdsys::rd_kafka_event_message_next(ev.ptr()) as *mut _) + .map(|ptr| BorrowedMessage::from_client(ptr, Arc::new(ev), self.queue)) + }, _ => { let buf = unsafe { let evname = rdsys::rd_kafka_event_name(ev.ptr()); @@ -350,9 +352,12 @@ where .split_partition_queue(topic, partition) .map(|queue| { let wakers = Arc::new(WakerSlab::new()); - StreamPartitionQueue { queue, wakers, _consumer: self.clone() } + StreamPartitionQueue { + queue, + wakers, + _consumer: self.clone(), + } }) - } } diff --git a/src/message.rs b/src/message.rs index e83971382..1915d22b6 100644 --- a/src/message.rs +++ b/src/message.rs @@ -6,6 +6,7 @@ use std::marker::PhantomData; use std::os::raw::c_void; use std::ptr; use std::str; +use std::sync::Arc; use std::time::SystemTime; use rdkafka_sys as rdsys; @@ -307,7 +308,7 @@ impl Headers for BorrowedHeaders { /// [`detach`](BorrowedMessage::detach) method. pub struct BorrowedMessage<'a> { ptr: NativePtr, - _event: NativeEvent, + _event: Arc, _owner: PhantomData<&'a u8>, } @@ -320,7 +321,12 @@ unsafe impl KafkaDrop for RDKafkaMessage { impl<'a> fmt::Debug for BorrowedMessage<'a> { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - write!(f, "Message {{ ptr: {:?} }}", self.ptr()) + write!( + f, + "Message {{ ptr: {:?}, event_ptr: {:?} }}", + self.ptr(), + self._event.ptr() + ) } } @@ -333,7 +339,7 @@ impl<'a> BorrowedMessage<'a> { /// is freed. pub(crate) unsafe fn from_client( ptr: NativePtr, - event: NativeEvent, + event: Arc, _client: &'a C, ) -> KafkaResult> { if ptr.err.is_error() { @@ -353,32 +359,29 @@ impl<'a> BorrowedMessage<'a> { } } - /// Creates a new `BorrowedMessage` that wraps the native Kafka message - /// pointer returned via the delivery report event. The lifetime of - /// the message will be bound to the lifetime of the event passed as - /// parameter. The message will not be freed in any circumstance. - pub(crate) unsafe fn from_dr_event( - ptr: *mut RDKafkaMessage, - //ptr: NativePtr, - event: *mut RDKafkaEvent, - _client: &'a C, - ) -> DeliveryResult<'a> { - let borrowed_message = BorrowedMessage { - ptr: NativePtr::from_ptr(ptr).unwrap(), - //ptr, - _event: NativePtr::from_ptr(event).unwrap(), - // TODO(sam): what does it mean this when the event holds the ownership? - _owner: PhantomData, - }; - if (*ptr).err.is_error() { - Err(( - KafkaError::MessageProduction((*ptr).err.into()), - borrowed_message, - )) - } else { - Ok(borrowed_message) - } - } + /// Creates a new `BorrowedMessage` that wraps the native Kafka message + /// pointer returned via the delivery report event. The lifetime of + /// the message will be bound to the lifetime of the event passed as + /// parameter. The message will not be freed in any circumstance. + pub(crate) unsafe fn from_dr_event( + ptr: *mut RDKafkaMessage, + event: Arc, + _client: &'a C, + ) -> DeliveryResult<'a> { + let borrowed_message = BorrowedMessage { + ptr: NativePtr::from_ptr(ptr).unwrap(), + _event: event, + _owner: PhantomData, + }; + if (*ptr).err.is_error() { + Err(( + KafkaError::MessageProduction((*ptr).err.into()), + borrowed_message, + )) + } else { + Ok(borrowed_message) + } + } /// Returns a pointer to the [`RDKafkaMessage`]. pub fn ptr(&self) -> *mut RDKafkaMessage { diff --git a/src/producer/base_producer.rs b/src/producer/base_producer.rs index d1de5052c..c24089185 100644 --- a/src/producer/base_producer.rs +++ b/src/producer/base_producer.rs @@ -391,10 +391,10 @@ where Vec::from_raw_parts(messages.as_mut_ptr(), msgs_cnt, max_messages) }; + let ev = Arc::new(event); for msg in messages { - let delivery_result = unsafe { - BorrowedMessage::from_dr_event(msg as *mut _, event.ptr(), self.client()) - }; + let delivery_result = + unsafe { BorrowedMessage::from_dr_event(msg as *mut _, ev.clone(), self.client()) }; let delivery_opaque = unsafe { C::DeliveryOpaque::from_ptr((*msg)._private) }; self.context().delivery(&delivery_result, delivery_opaque); } From d49db5b0e27c97fec14117f1896c7cc86d93def6 Mon Sep 17 00:00:00 2001 From: Samuel Cantero Date: Tue, 24 Oct 2023 18:47:10 -0300 Subject: [PATCH 14/59] Adapt producer Flush to the Event API semantics --- src/producer/base_producer.rs | 28 +++++++++++++++++++++++----- 1 file changed, 23 insertions(+), 5 deletions(-) diff --git a/src/producer/base_producer.rs b/src/producer/base_producer.rs index c24089185..b23bb79af 100644 --- a/src/producer/base_producer.rs +++ b/src/producer/base_producer.rs @@ -338,6 +338,7 @@ where client: Client, queue: NativeQueue, _partitioner: PhantomData, + min_poll_interval: Timeout, } impl BaseProducer @@ -352,6 +353,7 @@ where client, queue, _partitioner: PhantomData, + min_poll_interval: Timeout::After(Duration::from_millis(100)), } } @@ -489,12 +491,28 @@ where &self.client } + // As this library uses the rdkafka Event API, flush will not call rd_kafka_poll() but instead wait for + // the librdkafka-handled message count to reach zero. Runs until value reaches zero or timeout. fn flush>(&self, timeout: T) -> KafkaResult<()> { - let ret = unsafe { rdsys::rd_kafka_flush(self.native_ptr(), timeout.into().as_millis()) }; - if ret.is_error() { - Err(KafkaError::Flush(ret.into())) - } else { - Ok(()) + let mut timeout = timeout.into(); + loop { + let op_timeout = std::cmp::min(timeout, self.min_poll_interval); + if self.in_flight_count() > 0 { + unsafe { rdsys::rd_kafka_flush(self.native_ptr(), 0) }; + self.poll(op_timeout); + } else { + return Ok(()); + } + + if op_timeout >= timeout { + let ret = unsafe { rdsys::rd_kafka_flush(self.native_ptr(), 0) }; + if ret.is_error() { + return Err(KafkaError::Flush(ret.into())); + } else { + return Ok(()); + } + } + timeout -= op_timeout; } } From 53cebf9ed9fd26fdd06e630a13ba751c7c963bff Mon Sep 17 00:00:00 2001 From: Samuel Cantero Date: Wed, 25 Oct 2023 13:00:35 -0300 Subject: [PATCH 15/59] Explain why the TPL need to be manuallyDrop on the consumer events handler --- src/consumer/base_consumer.rs | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/consumer/base_consumer.rs b/src/consumer/base_consumer.rs index ae6993bbf..2c3813301 100644 --- a/src/consumer/base_consumer.rs +++ b/src/consumer/base_consumer.rs @@ -156,6 +156,8 @@ where let native_tpl = rdsys::rd_kafka_event_topic_partition_list(event.ptr()); TopicPartitionList::from_ptr(native_tpl) }; + // The TPL is owned by the Event and will be destroyed when the event is destroyed. + // Dropping it here will lead to double free. let mut tpl = ManuallyDrop::new(tpl); self.context() .rebalance(self.client.native_client(), err, &mut tpl); @@ -185,6 +187,8 @@ where let tpl = TopicPartitionList::new(); self.context().commit_callback(commit_error, &tpl); } else { + // The TPL is owned by the Event and will be destroyed when the event is destroyed. + // Dropping it here will lead to double free. let tpl = ManuallyDrop::new(unsafe { TopicPartitionList::from_ptr(offsets) }); self.context().commit_callback(commit_error, &tpl); } From d5d2bc0ca5b1d8e74b6fb17b8f2d111974a063cc Mon Sep 17 00:00:00 2001 From: Samuel Cantero Date: Wed, 25 Oct 2023 13:31:11 -0300 Subject: [PATCH 16/59] Add comment for no-op method used on RDKafkaMessage impl of the KafkaDrop trait --- src/message.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/src/message.rs b/src/message.rs index 1915d22b6..2ebda9071 100644 --- a/src/message.rs +++ b/src/message.rs @@ -312,6 +312,7 @@ pub struct BorrowedMessage<'a> { _owner: PhantomData<&'a u8>, } +// When using the Event API, messages must not be freed with rd_kafka_message_destroy unsafe extern "C" fn no_op(_: *mut RDKafkaMessage) {} unsafe impl KafkaDrop for RDKafkaMessage { From ae6e652fcd5a74bf5d3bb357f6cba05610ebaef3 Mon Sep 17 00:00:00 2001 From: Samuel Cantero Date: Wed, 25 Oct 2023 13:35:36 -0300 Subject: [PATCH 17/59] Update doc comment for BorrowedMessage::from_dr_event --- src/message.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/message.rs b/src/message.rs index 2ebda9071..696a50100 100644 --- a/src/message.rs +++ b/src/message.rs @@ -362,8 +362,8 @@ impl<'a> BorrowedMessage<'a> { /// Creates a new `BorrowedMessage` that wraps the native Kafka message /// pointer returned via the delivery report event. The lifetime of - /// the message will be bound to the lifetime of the event passed as - /// parameter. The message will not be freed in any circumstance. + /// the message will be bound to the lifetime of the client passed as + /// parameter. pub(crate) unsafe fn from_dr_event( ptr: *mut RDKafkaMessage, event: Arc, From 1e5285c456553c8b93cf79f9b0b74c0b12667372 Mon Sep 17 00:00:00 2001 From: Samuel Cantero Date: Wed, 25 Oct 2023 14:44:50 -0300 Subject: [PATCH 18/59] Replace poll with flush on baseProducer drop One poll call might not be enough to serve the delivery report callbacks of the purged messages. The current flush impl will call poll multiple times until the queue is empty or timeout. --- src/producer/base_producer.rs | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/src/producer/base_producer.rs b/src/producer/base_producer.rs index b23bb79af..73fafc94d 100644 --- a/src/producer/base_producer.rs +++ b/src/producer/base_producer.rs @@ -611,8 +611,13 @@ where { fn drop(&mut self) { self.purge(PurgeConfig::default().queue().inflight()); - // Still have to poll after purging to get the results that have been made ready by the purge - self.poll(Timeout::After(Duration::ZERO)); + // Still have to flush after purging to get the results that have been made ready by the purge + if let Err(err) = self.flush(Timeout::After(Duration::from_millis(500))) { + warn!( + "Failed to flush outstanding messages while dropping the producer: {:?}", + err + ); + } } } From 3e5e24bbaf90075972a5189ebb15f929cf7c165c Mon Sep 17 00:00:00 2001 From: Samuel Cantero Date: Thu, 26 Oct 2023 13:21:23 -0300 Subject: [PATCH 19/59] StreamConsumer Stream impl fixes for the event API --- src/client.rs | 6 +-- src/consumer/base_consumer.rs | 31 ++++--------- src/consumer/stream_consumer.rs | 81 ++++++++++++++++----------------- 3 files changed, 51 insertions(+), 67 deletions(-) diff --git a/src/client.rs b/src/client.rs index e71482e99..7475f287c 100644 --- a/src/client.rs +++ b/src/client.rs @@ -278,11 +278,7 @@ impl Client { &self.context } - pub(crate) fn poll_event( - &self, - queue: &NativeQueue, - timeout: Timeout, - ) -> Option { + pub(crate) fn poll_event(&self, queue: &NativeQueue, timeout: Timeout) -> Option { let event = unsafe { NativeEvent::from_ptr(queue.poll(timeout)) }; if let Some(ev) = event { let evtype = unsafe { rdsys::rd_kafka_event_type(ev.ptr()) }; diff --git a/src/consumer/base_consumer.rs b/src/consumer/base_consumer.rs index 2c3813301..4f7d0e399 100644 --- a/src/consumer/base_consumer.rs +++ b/src/consumer/base_consumer.rs @@ -113,7 +113,15 @@ where /// /// The returned message lives in the memory of the consumer and cannot outlive it. pub fn poll>(&self, timeout: T) -> Option>> { - let event = self.client().poll_event(self.get_queue(), timeout.into())?; + self.poll_queue(self.get_queue(), timeout) + } + + pub(crate) fn poll_queue>( + &self, + queue: &NativeQueue, + timeout: T, + ) -> Option>> { + let event = self.client().poll_event(queue, timeout.into())?; let evtype = unsafe { rdsys::rd_kafka_event_type(event.ptr()) }; match evtype { rdsys::RD_KAFKA_EVENT_FETCH => self.handle_fetch_event(event), @@ -743,26 +751,7 @@ where /// associated consumer regularly, even if no messages are expected, to /// serve events. pub fn poll>(&self, timeout: T) -> Option>> { - let event = self - .consumer - .client() - .poll_event(&self.queue, timeout.into())?; - let evtype = unsafe { rdsys::rd_kafka_event_type(event.ptr()) }; - match evtype { - rdsys::RD_KAFKA_EVENT_FETCH => unsafe { - NativePtr::from_ptr(rdsys::rd_kafka_event_message_next(event.ptr()) as *mut _) - .map(|ptr| BorrowedMessage::from_client(ptr, Arc::new(event), self)) - }, - _ => { - let buf = unsafe { - let evname = rdsys::rd_kafka_event_name(event.ptr()); - CStr::from_ptr(evname).to_bytes() - }; - let evname = String::from_utf8(buf.to_vec()).unwrap(); - warn!("Ignored event '{}' on consumer poll", evname); - None - } - } + self.consumer.poll_queue(&self.queue, timeout) } /// Sets a callback that will be invoked whenever the queue becomes diff --git a/src/consumer/stream_consumer.rs b/src/consumer/stream_consumer.rs index 28d672b22..5a7f60552 100644 --- a/src/consumer/stream_consumer.rs +++ b/src/consumer/stream_consumer.rs @@ -1,6 +1,5 @@ //! High-level consumers with a [`Stream`](futures_util::Stream) interface. -use std::ffi::CStr; use std::marker::PhantomData; use std::os::raw::c_void; use std::pin::Pin; @@ -14,13 +13,11 @@ use futures_channel::oneshot; use futures_util::future::{self, Either, FutureExt}; use futures_util::pin_mut; use futures_util::stream::{Stream, StreamExt}; -use log::warn; use slab::Slab; use rdkafka_sys as rdsys; use rdkafka_sys::types::*; -use crate::admin::NativeEvent; use crate::client::{Client, NativeQueue}; use crate::config::{ClientConfig, FromClientConfig, FromClientConfigAndContext}; use crate::consumer::base_consumer::{BaseConsumer, PartitionQueue}; @@ -33,7 +30,7 @@ use crate::groups::GroupList; use crate::message::BorrowedMessage; use crate::metadata::Metadata; use crate::topic_partition_list::{Offset, TopicPartitionList}; -use crate::util::{AsyncRuntime, DefaultRuntime, NativePtr, Timeout}; +use crate::util::{AsyncRuntime, DefaultRuntime, Timeout}; unsafe extern "C" fn native_message_queue_nonempty_cb(_: *mut RDKafka, opaque_ptr: *mut c_void) { let wakers = &*(opaque_ptr as *const WakerSlab); @@ -91,49 +88,50 @@ impl WakerSlab { /// A stream of messages from a [`StreamConsumer`]. /// /// See the documentation of [`StreamConsumer::stream`] for details. -pub struct MessageStream<'a> { +pub struct MessageStream<'a, C: ConsumerContext> { wakers: &'a WakerSlab, - queue: &'a NativeQueue, + consumer: &'a BaseConsumer, + partition_queue: Option<&'a NativeQueue>, slot: usize, } -impl<'a> MessageStream<'a> { - fn new(wakers: &'a WakerSlab, queue: &'a NativeQueue) -> MessageStream<'a> { +impl<'a, C: ConsumerContext> MessageStream<'a, C> { + fn new(wakers: &'a WakerSlab, consumer: &'a BaseConsumer) -> MessageStream<'a, C> { + Self::new_with_optional_partition_queue(wakers, consumer, None) + } + + fn new_with_partition_queue( + wakers: &'a WakerSlab, + consumer: &'a BaseConsumer, + partition_queue: &'a NativeQueue, + ) -> MessageStream<'a, C> { + Self::new_with_optional_partition_queue(wakers, consumer, Some(partition_queue)) + } + + fn new_with_optional_partition_queue( + wakers: &'a WakerSlab, + consumer: &'a BaseConsumer, + partition_queue: Option<&'a NativeQueue>, + ) -> MessageStream<'a, C> { let slot = wakers.register(); MessageStream { wakers, - queue, + consumer, + partition_queue, slot, } } fn poll(&self) -> Option>> { - let timeout: Timeout = Duration::ZERO.into(); - let event = unsafe { NativeEvent::from_ptr(self.queue.poll(timeout)) }; - if let Some(ev) = event { - let evtype = unsafe { rdsys::rd_kafka_event_type(ev.ptr()) }; - match evtype { - rdsys::RD_KAFKA_EVENT_FETCH => unsafe { - NativePtr::from_ptr(rdsys::rd_kafka_event_message_next(ev.ptr()) as *mut _) - .map(|ptr| BorrowedMessage::from_client(ptr, Arc::new(ev), self.queue)) - }, - _ => { - let buf = unsafe { - let evname = rdsys::rd_kafka_event_name(ev.ptr()); - CStr::from_ptr(evname).to_bytes() - }; - let evname = String::from_utf8(buf.to_vec()).unwrap(); - warn!("Ignored event '{}' on consumer poll", evname); - None - } - } + if let Some(queue) = self.partition_queue { + self.consumer.poll_queue(queue, Duration::ZERO) } else { - None + self.consumer.poll(Duration::ZERO) } } } -impl<'a> Stream for MessageStream<'a> { +impl<'a, C: ConsumerContext> Stream for MessageStream<'a, C> { type Item = KafkaResult>; fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { @@ -160,7 +158,7 @@ impl<'a> Stream for MessageStream<'a> { } } -impl<'a> Drop for MessageStream<'a> { +impl<'a, C: ConsumerContext> Drop for MessageStream<'a, C> { fn drop(&mut self) { self.wakers.unregister(self.slot); } @@ -186,7 +184,6 @@ where C: ConsumerContext, { base: Arc>, - queue: NativeQueue, wakers: Arc, _shutdown_trigger: oneshot::Sender<()>, _runtime: PhantomData, @@ -218,9 +215,7 @@ where }; let base = Arc::new(BaseConsumer::new(config, native_config, context)?); - let client = base.client(); - let native_ptr = client.native_ptr() as usize; - let queue = client.main_queue(); + let native_ptr = base.client().native_ptr() as usize; let wakers = Arc::new(WakerSlab::new()); unsafe { enable_nonempty_callback(base.get_queue(), &wakers) } @@ -255,7 +250,6 @@ where Ok(StreamConsumer { base, wakers, - queue, _shutdown_trigger: shutdown_trigger, _runtime: PhantomData, }) @@ -278,8 +272,8 @@ where /// /// If you want multiple independent views of a Kafka topic, create multiple /// consumers, not multiple message streams. - pub fn stream(&self) -> MessageStream<'_> { - MessageStream::new(&self.wakers, &self.queue) + pub fn stream(&self) -> MessageStream<'_, C> { + MessageStream::new(&self.wakers, &self.base) } /// Receives the next message from the stream. @@ -322,7 +316,7 @@ where /// `StreamConsumer::recv`. /// /// You must periodically await `StreamConsumer::recv`, even if no messages - /// are expected, to serve callbacks. Consider using a background task like: + /// are expected, to serve events. Consider using a background task like: /// /// ``` /// # use rdkafka::consumer::StreamConsumer; @@ -352,6 +346,7 @@ where .split_partition_queue(topic, partition) .map(|queue| { let wakers = Arc::new(WakerSlab::new()); + unsafe { enable_nonempty_callback(&queue.queue, &wakers) }; StreamPartitionQueue { queue, wakers, @@ -573,8 +568,12 @@ where /// /// If you want multiple independent views of a Kafka partition, create /// multiple consumers, not multiple partition streams. - pub fn stream(&self) -> MessageStream<'_> { - MessageStream::new(&self.wakers, &self.queue.queue) + pub fn stream(&self) -> MessageStream<'_, C> { + MessageStream::new_with_partition_queue( + &self.wakers, + &self._consumer.base, + &self.queue.queue, + ) } /// Receives the next message from the stream. From 32554704f4427788bc2f3e25af2bae503294655d Mon Sep 17 00:00:00 2001 From: Samuel Cantero Date: Thu, 26 Oct 2023 15:52:17 -0300 Subject: [PATCH 20/59] Consumer needs to read from earliest otherwise consumer will never read anything --- tests/test_metadata.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/test_metadata.rs b/tests/test_metadata.rs index 3b2667a9c..e62bee556 100644 --- a/tests/test_metadata.rs +++ b/tests/test_metadata.rs @@ -22,6 +22,7 @@ fn create_consumer(group_id: &str) -> StreamConsumer { .set("session.timeout.ms", "6000") .set("api.version.request", "true") .set("debug", "all") + .set("auto.offset.reset", "earliest") .create() .expect("Failed to create StreamConsumer") } From 9a2d7d59d680b43070419cb2dc27bc717caa95f2 Mon Sep 17 00:00:00 2001 From: Samuel Cantero Date: Thu, 26 Oct 2023 18:10:32 -0300 Subject: [PATCH 21/59] Poll should not return None if timeout has not been reached If timeout::Never is used, poll should eventually return a Message or Error rather than None when handling other events like stats, rebalances, etc. --- src/consumer/base_consumer.rs | 51 ++++++++++++++++++++++------------- src/consumer/mod.rs | 4 +-- 2 files changed, 34 insertions(+), 21 deletions(-) diff --git a/src/consumer/base_consumer.rs b/src/consumer/base_consumer.rs index 4f7d0e399..870f981fc 100644 --- a/src/consumer/base_consumer.rs +++ b/src/consumer/base_consumer.rs @@ -121,27 +121,40 @@ where queue: &NativeQueue, timeout: T, ) -> Option>> { - let event = self.client().poll_event(queue, timeout.into())?; - let evtype = unsafe { rdsys::rd_kafka_event_type(event.ptr()) }; - match evtype { - rdsys::RD_KAFKA_EVENT_FETCH => self.handle_fetch_event(event), - rdsys::RD_KAFKA_EVENT_REBALANCE => { - self.handle_rebalance_event(event); - None - } - rdsys::RD_KAFKA_EVENT_OFFSET_COMMIT => { - self.handle_offset_commit_event(event); - None + let mut timeout = timeout.into(); + let min_poll_interval = self.context().main_queue_min_poll_interval(); + loop { + let op_timeout = std::cmp::min(timeout, min_poll_interval); + let maybe_event = self.client().poll_event(queue, op_timeout); + if let Some(event) = maybe_event { + let evtype = unsafe { rdsys::rd_kafka_event_type(event.ptr()) }; + match evtype { + rdsys::RD_KAFKA_EVENT_FETCH => { + if let Some(result) = self.handle_fetch_event(event) { + return Some(result); + } + } + rdsys::RD_KAFKA_EVENT_REBALANCE => { + self.handle_rebalance_event(event); + } + rdsys::RD_KAFKA_EVENT_OFFSET_COMMIT => { + self.handle_offset_commit_event(event); + } + _ => { + let buf = unsafe { + let evname = rdsys::rd_kafka_event_name(event.ptr()); + CStr::from_ptr(evname).to_bytes() + }; + let evname = String::from_utf8(buf.to_vec()).unwrap(); + warn!("Ignored event '{}' on consumer poll", evname); + } + } } - _ => { - let buf = unsafe { - let evname = rdsys::rd_kafka_event_name(event.ptr()); - CStr::from_ptr(evname).to_bytes() - }; - let evname = String::from_utf8(buf.to_vec()).unwrap(); - warn!("Ignored event '{}' on consumer poll", evname); - None + + if op_timeout >= timeout { + return None; } + timeout -= op_timeout; } } diff --git a/src/consumer/mod.rs b/src/consumer/mod.rs index 65bbd215a..fb3ff0460 100644 --- a/src/consumer/mod.rs +++ b/src/consumer/mod.rs @@ -113,12 +113,12 @@ pub trait ConsumerContext: ClientContext { 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. + /// services the logging, stats, and error events. /// /// The main queue is polled once whenever [`BaseConsumer::poll`] is called. /// If `poll` is called with a timeout that is larger than this interval, /// then the main queue will be polled at that interval while the consumer - /// queue is blocked. + /// queue is blocked. This allows serving events while there are no messages. /// /// For example, if the main queue's minimum poll interval is 200ms and /// `poll` is called with a timeout of 1s, then `poll` may block for up to From 7b20a5d0a5fff41af319eea99f825fd929839850 Mon Sep 17 00:00:00 2001 From: Samuel Cantero Date: Thu, 26 Oct 2023 18:56:16 -0300 Subject: [PATCH 22/59] Cargo clippy --- src/consumer/base_consumer.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/consumer/base_consumer.rs b/src/consumer/base_consumer.rs index 870f981fc..6eb14b8fc 100644 --- a/src/consumer/base_consumer.rs +++ b/src/consumer/base_consumer.rs @@ -89,9 +89,9 @@ where // need to listen to the consumer queue to observe events like // rebalancings and stats. unsafe { rdsys::rd_kafka_poll_set_consumer(client.native_ptr()) }; - let queue = client.consumer_queue().ok_or(KafkaError::ClientCreation( - "rdkafka consumer queue not available".to_string(), - ))?; + let queue = client.consumer_queue().ok_or_else(|| { + KafkaError::ClientCreation("rdkafka consumer queue not available".to_string()) + })?; Ok(BaseConsumer { client, queue, From bf3914f1bbb96573b2cf3fd5af456a6685369744 Mon Sep 17 00:00:00 2001 From: Samuel Cantero Date: Fri, 27 Oct 2023 13:27:12 -0300 Subject: [PATCH 23/59] Propagate errors for the consumer rdkafka reports consumer errors via RD_KAFKA_EVENT_ERROR but producer errors gets embedded on the ack returned via RD_KAFKA_EVENT_DR. Hence we need to return this event for the consumer case in order to return the error to the user. --- src/client.rs | 8 +++++++- src/consumer/base_consumer.rs | 23 +++++++++++++++++++++++ src/message.rs | 2 +- 3 files changed, 31 insertions(+), 2 deletions(-) diff --git a/src/client.rs b/src/client.rs index 7475f287c..1b9f6bd1c 100644 --- a/src/client.rs +++ b/src/client.rs @@ -285,7 +285,13 @@ impl Client { match evtype { rdsys::RD_KAFKA_EVENT_LOG => self.handle_log_event(ev.ptr()), rdsys::RD_KAFKA_EVENT_STATS => self.handle_stats_event(ev.ptr()), - rdsys::RD_KAFKA_EVENT_ERROR => self.handle_error_event(ev.ptr()), + rdsys::RD_KAFKA_EVENT_ERROR => { + // rdkafka reports consumer errors via RD_KAFKA_EVENT_ERROR but producer errors gets + // embedded on the ack returned via RD_KAFKA_EVENT_DR. Hence we need to return this event + // for the consumer case in order to return the error to the user. + self.handle_error_event(ev.ptr()); + return Some(ev); + } rdsys::RD_KAFKA_EVENT_OAUTHBEARER_TOKEN_REFRESH => { if C::ENABLE_REFRESH_OAUTH_TOKEN { self.handle_oauth_refresh_event(ev.ptr()); diff --git a/src/consumer/base_consumer.rs b/src/consumer/base_consumer.rs index 6eb14b8fc..4379500c1 100644 --- a/src/consumer/base_consumer.rs +++ b/src/consumer/base_consumer.rs @@ -134,6 +134,11 @@ where return Some(result); } } + rdsys::RD_KAFKA_EVENT_ERROR => { + if let Some(err) = self.handle_error_event(event) { + return Some(Err(err)); + } + } rdsys::RD_KAFKA_EVENT_REBALANCE => { self.handle_rebalance_event(event); } @@ -215,6 +220,24 @@ where } } + fn handle_error_event(&self, event: NativePtr) -> Option { + let rdkafka_err = unsafe { rdsys::rd_kafka_event_error(event.ptr()) }; + if rdkafka_err.is_error() { + let err = match rdkafka_err { + rdsys::rd_kafka_resp_err_t::RD_KAFKA_RESP_ERR__PARTITION_EOF => { + let tp_ptr = unsafe { rdsys::rd_kafka_event_topic_partition(event.ptr()) }; + let partition = unsafe { (*tp_ptr).partition }; + unsafe { rdsys::rd_kafka_topic_partition_destroy(tp_ptr) }; + KafkaError::PartitionEOF(partition) + } + e => KafkaError::MessageConsumption(e.into()), + }; + Some(err) + } else { + None + } + } + /// Returns an iterator over the available messages. /// /// It repeatedly calls [`poll`](#method.poll) with no timeout. diff --git a/src/message.rs b/src/message.rs index 696a50100..76bac9c39 100644 --- a/src/message.rs +++ b/src/message.rs @@ -346,7 +346,7 @@ impl<'a> BorrowedMessage<'a> { if ptr.err.is_error() { let err = match ptr.err { rdsys::rd_kafka_resp_err_t::RD_KAFKA_RESP_ERR__PARTITION_EOF => { - KafkaError::PartitionEOF((*ptr).partition) + KafkaError::PartitionEOF(ptr.partition) } e => KafkaError::MessageConsumption(e.into()), }; From 990a6b265067ba3aa65fd008f1056963a3ed5d4b Mon Sep 17 00:00:00 2001 From: Samuel Cantero Date: Fri, 27 Oct 2023 13:37:06 -0300 Subject: [PATCH 24/59] Adapt commit_transaction to the event api --- src/producer/base_producer.rs | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/src/producer/base_producer.rs b/src/producer/base_producer.rs index 73fafc94d..886d5bdee 100644 --- a/src/producer/base_producer.rs +++ b/src/producer/base_producer.rs @@ -577,10 +577,17 @@ where } fn commit_transaction>(&self, timeout: T) -> KafkaResult<()> { + // rd_kafka_commit_transaction will call flush but the user must call poll in order to + // server the event queue. In order to avoid blocking here forever on the base producer, + // we call Flush that will flush the outstanding messages and serve the event queue. + // https://github.com/confluentinc/librdkafka/blob/95a542c87c61d2c45b445f91c73dd5442eb04f3c/src/rdkafka.h#L10231 + // The recommended timeout here is -1 (never, i.e, infinite). + let timeout = timeout.into(); + self.flush(timeout)?; let ret = unsafe { RDKafkaError::from_ptr(rdsys::rd_kafka_commit_transaction( self.native_ptr(), - timeout.into().as_millis(), + timeout.as_millis(), )) }; if ret.is_error() { From 8ed896b8712b0da5c5c6bcc74f73865ffa1fe651 Mon Sep 17 00:00:00 2001 From: Samuel Cantero Date: Mon, 30 Oct 2023 14:48:50 -0300 Subject: [PATCH 25/59] Adapt consumer close to the event api --- src/consumer/base_consumer.rs | 46 ++++++++--------------------------- 1 file changed, 10 insertions(+), 36 deletions(-) diff --git a/src/consumer/base_consumer.rs b/src/consumer/base_consumer.rs index 4379500c1..fb2188774 100644 --- a/src/consumer/base_consumer.rs +++ b/src/consumer/base_consumer.rs @@ -7,7 +7,7 @@ use std::ptr; use std::sync::Arc; use std::time::Duration; -use log::warn; +use log::{error, warn}; use rdkafka_sys as rdsys; use rdkafka_sys::types::*; @@ -37,7 +37,6 @@ where { client: Client, queue: NativeQueue, - static_member: bool, } impl FromClientConfig for BaseConsumer { @@ -62,12 +61,6 @@ where native_config: NativeClientConfig, context: C, ) -> KafkaResult> { - let mut static_member = false; - if let Some(group_instance_id) = config.get("group.instance.id") { - if !group_instance_id.is_empty() { - static_member = true; - } - } unsafe { rdsys::rd_kafka_conf_set_events( native_config.ptr(), @@ -92,11 +85,7 @@ where let queue = client.consumer_queue().ok_or_else(|| { KafkaError::ClientCreation("rdkafka consumer queue not available".to_string()) })?; - Ok(BaseConsumer { - client, - queue, - static_member, - }) + Ok(BaseConsumer { client, queue }) } /// Polls the consumer for new messages. @@ -692,31 +681,16 @@ where { fn drop(&mut self) { trace!("Destroying consumer: {:?}", self.client.native_ptr()); - // If this consumer is configured for static membership, do not explicitly unsubscribe from - // the group. Note that static members will *not* receive a final revoke event when they - // shutdown. - if !self.static_member { - // We use the Event API rather than the Callback API. - // As we don't register a rebalance_cb, rd_kafka_consumer_close() - // will shortcut the rebalance_cb and do a forced unassign. - // This is undesired as the application might need the final - // revoke events before shutting down. Hence, we trigger - // an Unsubscribe() first, wait for that to propagate, and then - // close the consumer. - unsafe { rdsys::rd_kafka_unsubscribe(self.client.native_ptr()) }; - - // Poll for rebalance events - loop { - self.poll(Duration::from_secs(5)); - let qlen = unsafe { rdsys::rd_kafka_queue_length(self.queue.ptr()) }; - if qlen == 0 { - break; - } - } + let err = unsafe { + rdsys::rd_kafka_consumer_close_queue(self.client.native_ptr(), self.queue.ptr()) + }; + if !err.is_null() { + error!("Failed to close the consumer queue on drop"); } - // TODO(sam): do we need to destroy the queue before calling close? - unsafe { rdsys::rd_kafka_consumer_close(self.client.native_ptr()) }; + while unsafe { rdsys::rd_kafka_consumer_closed(self.client.native_ptr()) } != 1 { + self.poll(Duration::from_millis(100)); + } trace!("Consumer destroyed: {:?}", self.client.native_ptr()); } } From 207dd3856d9b019597dd5dd249bd1a768974d5a0 Mon Sep 17 00:00:00 2001 From: Samuel Cantero Date: Mon, 30 Oct 2023 14:58:07 -0300 Subject: [PATCH 26/59] Allow creating a consumer without group.id Currently if a group.id is not specified we allow the use of the consumer for fetching metadata and watermarks. Keeping this behaviour. --- src/consumer/base_consumer.rs | 47 +++++++++++++++++++++++------------ 1 file changed, 31 insertions(+), 16 deletions(-) diff --git a/src/consumer/base_consumer.rs b/src/consumer/base_consumer.rs index fb2188774..01f1af462 100644 --- a/src/consumer/base_consumer.rs +++ b/src/consumer/base_consumer.rs @@ -37,6 +37,7 @@ where { client: Client, queue: NativeQueue, + group_id: Option, } impl FromClientConfig for BaseConsumer { @@ -78,14 +79,26 @@ where context, )?; - // Redirect rdkafka's main queue to the consumer queue so that we only - // need to listen to the consumer queue to observe events like - // rebalancings and stats. - unsafe { rdsys::rd_kafka_poll_set_consumer(client.native_ptr()) }; - let queue = client.consumer_queue().ok_or_else(|| { - KafkaError::ClientCreation("rdkafka consumer queue not available".to_string()) - })?; - Ok(BaseConsumer { client, queue }) + let group_id = config.get("group.id").map(|s| s.to_string()); + // If a group.id is not specified, we won't redirect the main queue to the consumer queue, + // allowing continued use of the consumer for fetching metadata and watermarks without the + // need to specify a group.id + let queue = if group_id.is_some() { + // Redirect rdkafka's main queue to the consumer queue so that we only need to listen + // to the consumer queue to observe events like rebalancings and stats. + unsafe { rdsys::rd_kafka_poll_set_consumer(client.native_ptr()) }; + client.consumer_queue().ok_or_else(|| { + KafkaError::ClientCreation("rdkafka consumer queue not available".to_string()) + })? + } else { + client.main_queue() + }; + + Ok(BaseConsumer { + client, + queue, + group_id, + }) } /// Polls the consumer for new messages. @@ -681,15 +694,17 @@ where { fn drop(&mut self) { trace!("Destroying consumer: {:?}", self.client.native_ptr()); - let err = unsafe { - rdsys::rd_kafka_consumer_close_queue(self.client.native_ptr(), self.queue.ptr()) - }; - if !err.is_null() { - error!("Failed to close the consumer queue on drop"); - } + if self.group_id.is_some() { + let err = unsafe { + rdsys::rd_kafka_consumer_close_queue(self.client.native_ptr(), self.queue.ptr()) + }; + if !err.is_null() { + error!("Failed to close the consumer queue on drop"); + } - while unsafe { rdsys::rd_kafka_consumer_closed(self.client.native_ptr()) } != 1 { - self.poll(Duration::from_millis(100)); + while unsafe { rdsys::rd_kafka_consumer_closed(self.client.native_ptr()) } != 1 { + self.poll(Duration::from_millis(100)); + } } trace!("Consumer destroyed: {:?}", self.client.native_ptr()); } From 0bae0d94c658948695a54ef2514f30fa1d9cc973 Mon Sep 17 00:00:00 2001 From: Samuel Cantero Date: Tue, 31 Oct 2023 16:36:04 -0300 Subject: [PATCH 27/59] Do not panic on transient errors on test_consume_partition_order --- tests/test_high_consumers.rs | 33 +++++++++++++++++++++++++++++++-- tests/test_low_consumers.rs | 32 ++++++++++++++++++++++++++++++-- 2 files changed, 61 insertions(+), 4 deletions(-) diff --git a/tests/test_high_consumers.rs b/tests/test_high_consumers.rs index d139127b0..97ca4f5a0 100644 --- a/tests/test_high_consumers.rs +++ b/tests/test_high_consumers.rs @@ -7,6 +7,7 @@ use std::sync::Arc; use futures::future::{self, FutureExt}; use futures::stream::StreamExt; use maplit::hashmap; +use rdkafka_sys::RDKafkaErrorCode; use tokio::time::{self, Duration}; use rdkafka::consumer::{CommitMode, Consumer, ConsumerContext, StreamConsumer}; @@ -546,13 +547,41 @@ async fn test_consume_partition_order() { let mut i = 0; while i < 12 { if let Some(m) = consumer.recv().now_or_never() { - let partition = m.unwrap().partition(); + // retry on transient errors until we get a message + let m = match m { + Err(KafkaError::MessageConsumption( + RDKafkaErrorCode::BrokerTransportFailure, + )) + | Err(KafkaError::MessageConsumption(RDKafkaErrorCode::AllBrokersDown)) + | Err(KafkaError::MessageConsumption(RDKafkaErrorCode::OperationTimedOut)) => { + continue + } + Err(err) => { + panic!("Unexpected error receiving message: {:?}", err); + } + Ok(m) => m, + }; + let partition: i32 = m.partition(); assert!(partition == 0 || partition == 2); i += 1; } if let Some(m) = partition1.recv().now_or_never() { - assert_eq!(m.unwrap().partition(), 1); + // retry on transient errors until we get a message + let m = match m { + Err(KafkaError::MessageConsumption( + RDKafkaErrorCode::BrokerTransportFailure, + )) + | Err(KafkaError::MessageConsumption(RDKafkaErrorCode::AllBrokersDown)) + | Err(KafkaError::MessageConsumption(RDKafkaErrorCode::OperationTimedOut)) => { + continue + } + Err(err) => { + panic!("Unexpected error receiving message: {:?}", err); + } + Ok(m) => m, + }; + assert_eq!(m.partition(), 1); i += 1; } } diff --git a/tests/test_low_consumers.rs b/tests/test_low_consumers.rs index e1ce16bdf..c4aa305f7 100644 --- a/tests/test_low_consumers.rs +++ b/tests/test_low_consumers.rs @@ -288,13 +288,41 @@ async fn test_consume_partition_order() { let mut i = 0; while i < 12 { if let Some(m) = consumer.poll(Timeout::After(Duration::from_secs(0))) { - let partition = m.unwrap().partition(); + // retry on transient errors until we get a message + let m = match m { + Err(KafkaError::MessageConsumption( + RDKafkaErrorCode::BrokerTransportFailure, + )) + | Err(KafkaError::MessageConsumption(RDKafkaErrorCode::AllBrokersDown)) + | Err(KafkaError::MessageConsumption(RDKafkaErrorCode::OperationTimedOut)) => { + continue + } + Err(err) => { + panic!("Unexpected error receiving message: {:?}", err); + } + Ok(m) => m, + }; + let partition = m.partition(); assert!(partition == 0 || partition == 2); i += 1; } if let Some(m) = partition1.poll(Timeout::After(Duration::from_secs(0))) { - assert_eq!(m.unwrap().partition(), 1); + // retry on transient errors until we get a message + let m = match m { + Err(KafkaError::MessageConsumption( + RDKafkaErrorCode::BrokerTransportFailure, + )) + | Err(KafkaError::MessageConsumption(RDKafkaErrorCode::AllBrokersDown)) + | Err(KafkaError::MessageConsumption(RDKafkaErrorCode::OperationTimedOut)) => { + continue + } + Err(err) => { + panic!("Unexpected error receiving message: {:?}", err); + } + Ok(m) => m, + }; + assert_eq!(m.partition(), 1); i += 1; } } From 2570ae2036eacf3f5062695002770c8fa299a067 Mon Sep 17 00:00:00 2001 From: Samuel Cantero Date: Mon, 30 Oct 2023 15:47:12 -0300 Subject: [PATCH 28/59] Expose a close_queue and closed methods If you have a consumer wrapping this one (FFI cases), the outer consumer must close the queue and serve the events via Poll. Otherwise it will hang forever as prior to calling close there's a rebalance & rdkafka awaits a response before continuing. --- src/consumer/base_consumer.rs | 21 +++++++++++++++++++++ src/error.rs | 8 ++++++++ 2 files changed, 29 insertions(+) diff --git a/src/consumer/base_consumer.rs b/src/consumer/base_consumer.rs index 01f1af462..b2518b73f 100644 --- a/src/consumer/base_consumer.rs +++ b/src/consumer/base_consumer.rs @@ -331,6 +331,27 @@ where PartitionQueue::new(self.clone(), queue) }) } + + /// Close the queue used by a consumer. + /// Only exposed for advanced usage of this API and should not be used under normal circumstances. + pub fn close_queue(&self) -> KafkaResult<()> { + let err = unsafe { + RDKafkaError::from_ptr(rdsys::rd_kafka_consumer_close_queue( + self.client.native_ptr(), + self.queue.ptr(), + )) + }; + if err.is_error() { + Err(KafkaError::ConsumerQueueClose(err.code())) + } else { + Ok(()) + } + } + + /// Returns true if the consumer is closed, else false. + pub fn closed(&self) -> bool { + unsafe { rdsys::rd_kafka_consumer_closed(self.client.native_ptr()) == 1 } + } } impl Consumer for BaseConsumer diff --git a/src/error.rs b/src/error.rs index 72e364479..46a40be54 100644 --- a/src/error.rs +++ b/src/error.rs @@ -147,6 +147,8 @@ pub enum KafkaError { ClientCreation(String), /// Consumer commit failed. ConsumerCommit(RDKafkaErrorCode), + /// Consumer queue close failed. + ConsumerQueueClose(RDKafkaErrorCode), /// Flushing failed Flush(RDKafkaErrorCode), /// Global error. @@ -204,6 +206,9 @@ impl fmt::Debug for KafkaError { KafkaError::ConsumerCommit(err) => { write!(f, "KafkaError (Consumer commit error: {})", err) } + KafkaError::ConsumerQueueClose(err) => { + write!(f, "KafkaError (Consumer queue close error: {})", err) + } KafkaError::Flush(err) => write!(f, "KafkaError (Flush error: {})", err), KafkaError::Global(err) => write!(f, "KafkaError (Global error: {})", err), KafkaError::GroupListFetch(err) => { @@ -255,6 +260,7 @@ impl fmt::Display for KafkaError { } KafkaError::ClientCreation(ref err) => write!(f, "Client creation error: {}", err), KafkaError::ConsumerCommit(err) => write!(f, "Consumer commit error: {}", err), + KafkaError::ConsumerQueueClose(err) => write!(f, "Consumer queue close error: {}", err), KafkaError::Flush(err) => write!(f, "Flush error: {}", err), KafkaError::Global(err) => write!(f, "Global error: {}", err), KafkaError::GroupListFetch(err) => write!(f, "Group list fetch error: {}", err), @@ -288,6 +294,7 @@ impl Error for KafkaError { KafkaError::ClientConfig(..) => None, KafkaError::ClientCreation(_) => None, KafkaError::ConsumerCommit(err) => Some(err), + KafkaError::ConsumerQueueClose(err) => Some(err), KafkaError::Flush(err) => Some(err), KafkaError::Global(err) => Some(err), KafkaError::GroupListFetch(err) => Some(err), @@ -327,6 +334,7 @@ impl KafkaError { KafkaError::ClientConfig(..) => None, KafkaError::ClientCreation(_) => None, KafkaError::ConsumerCommit(err) => Some(*err), + KafkaError::ConsumerQueueClose(err) => Some(*err), KafkaError::Flush(err) => Some(*err), KafkaError::Global(err) => Some(*err), KafkaError::GroupListFetch(err) => Some(*err), From 004ad40bb6966319c8274436a98fe4d8a13a9c40 Mon Sep 17 00:00:00 2001 From: Samuel Cantero Date: Mon, 6 Nov 2023 10:12:12 -0300 Subject: [PATCH 29/59] Use closed and close_queue methods on drop --- src/consumer/base_consumer.rs | 15 ++++++--------- 1 file changed, 6 insertions(+), 9 deletions(-) diff --git a/src/consumer/base_consumer.rs b/src/consumer/base_consumer.rs index b2518b73f..a6d1e39a1 100644 --- a/src/consumer/base_consumer.rs +++ b/src/consumer/base_consumer.rs @@ -716,15 +716,12 @@ where fn drop(&mut self) { trace!("Destroying consumer: {:?}", self.client.native_ptr()); if self.group_id.is_some() { - let err = unsafe { - rdsys::rd_kafka_consumer_close_queue(self.client.native_ptr(), self.queue.ptr()) - }; - if !err.is_null() { - error!("Failed to close the consumer queue on drop"); - } - - while unsafe { rdsys::rd_kafka_consumer_closed(self.client.native_ptr()) } != 1 { - self.poll(Duration::from_millis(100)); + if let Err(err) = self.close_queue() { + error!("Failed to close consumer queue on drop: {}", err); + } else { + while !self.closed() { + self.poll(Duration::from_millis(100)); + } } } trace!("Consumer destroyed: {:?}", self.client.native_ptr()); From 51b24cdb5ec0e184e4e51caf667947b287b8d34a Mon Sep 17 00:00:00 2001 From: Samuel Cantero Date: Mon, 6 Nov 2023 10:12:31 -0300 Subject: [PATCH 30/59] Propagate fatal errors With the Event API we propagate generic client instance-level errors, such as broker connection failures, authentication issues, etc. However, fatal errors are also propagated via the Event API. These indicates that the particular instance of the client (producer/consumer) becomes non-functional. --- src/consumer/base_consumer.rs | 20 ++++++++++---------- src/error.rs | 10 ++++++++++ 2 files changed, 20 insertions(+), 10 deletions(-) diff --git a/src/consumer/base_consumer.rs b/src/consumer/base_consumer.rs index a6d1e39a1..5fcc42bc5 100644 --- a/src/consumer/base_consumer.rs +++ b/src/consumer/base_consumer.rs @@ -225,16 +225,16 @@ where fn handle_error_event(&self, event: NativePtr) -> Option { let rdkafka_err = unsafe { rdsys::rd_kafka_event_error(event.ptr()) }; if rdkafka_err.is_error() { - let err = match rdkafka_err { - rdsys::rd_kafka_resp_err_t::RD_KAFKA_RESP_ERR__PARTITION_EOF => { - let tp_ptr = unsafe { rdsys::rd_kafka_event_topic_partition(event.ptr()) }; - let partition = unsafe { (*tp_ptr).partition }; - unsafe { rdsys::rd_kafka_topic_partition_destroy(tp_ptr) }; - KafkaError::PartitionEOF(partition) - } - e => KafkaError::MessageConsumption(e.into()), - }; - Some(err) + if rdkafka_err == rdsys::rd_kafka_resp_err_t::RD_KAFKA_RESP_ERR__PARTITION_EOF { + let tp_ptr = unsafe { rdsys::rd_kafka_event_topic_partition(event.ptr()) }; + let partition = unsafe { (*tp_ptr).partition }; + unsafe { rdsys::rd_kafka_topic_partition_destroy(tp_ptr) }; + Some(KafkaError::PartitionEOF(partition)) + } else if unsafe { rdsys::rd_kafka_event_error_is_fatal(event.ptr()) } != 0 { + Some(KafkaError::MessageConsumptionFatal(rdkafka_err.into())) + } else { + Some(KafkaError::MessageConsumption(rdkafka_err.into())) + } } else { None } diff --git a/src/error.rs b/src/error.rs index 46a40be54..312a6bb65 100644 --- a/src/error.rs +++ b/src/error.rs @@ -157,6 +157,8 @@ pub enum KafkaError { GroupListFetch(RDKafkaErrorCode), /// Message consumption failed. MessageConsumption(RDKafkaErrorCode), + /// Message consumption failed with fatal error. + MessageConsumptionFatal(RDKafkaErrorCode), /// Message production error. MessageProduction(RDKafkaErrorCode), /// Metadata fetch error. @@ -217,6 +219,9 @@ impl fmt::Debug for KafkaError { KafkaError::MessageConsumption(err) => { write!(f, "KafkaError (Message consumption error: {})", err) } + KafkaError::MessageConsumptionFatal(err) => { + write!(f, "(Fatal) KafkaError (Message consumption error: {})", err) + } KafkaError::MessageProduction(err) => { write!(f, "KafkaError (Message production error: {})", err) } @@ -265,6 +270,9 @@ impl fmt::Display for KafkaError { KafkaError::Global(err) => write!(f, "Global error: {}", err), KafkaError::GroupListFetch(err) => write!(f, "Group list fetch error: {}", err), KafkaError::MessageConsumption(err) => write!(f, "Message consumption error: {}", err), + KafkaError::MessageConsumptionFatal(err) => { + write!(f, "(Fatal) Message consumption error: {}", err) + } KafkaError::MessageProduction(err) => write!(f, "Message production error: {}", err), KafkaError::MetadataFetch(err) => write!(f, "Meta data fetch error: {}", err), KafkaError::NoMessageReceived => { @@ -299,6 +307,7 @@ impl Error for KafkaError { KafkaError::Global(err) => Some(err), KafkaError::GroupListFetch(err) => Some(err), KafkaError::MessageConsumption(err) => Some(err), + KafkaError::MessageConsumptionFatal(err) => Some(err), KafkaError::MessageProduction(err) => Some(err), KafkaError::MetadataFetch(err) => Some(err), KafkaError::NoMessageReceived => None, @@ -339,6 +348,7 @@ impl KafkaError { KafkaError::Global(err) => Some(*err), KafkaError::GroupListFetch(err) => Some(*err), KafkaError::MessageConsumption(err) => Some(*err), + KafkaError::MessageConsumptionFatal(err) => Some(*err), KafkaError::MessageProduction(err) => Some(*err), KafkaError::MetadataFetch(err) => Some(*err), KafkaError::NoMessageReceived => None, From 8437073fdae621d7b6bdf639f330e16a31b23d72 Mon Sep 17 00:00:00 2001 From: Samuel Cantero Date: Tue, 7 Nov 2023 14:39:04 -0300 Subject: [PATCH 31/59] Fix op timeout computation logic on poll_queue --- src/consumer/base_consumer.rs | 7 ++++--- src/util.rs | 16 ++++++++++++++++ 2 files changed, 20 insertions(+), 3 deletions(-) diff --git a/src/consumer/base_consumer.rs b/src/consumer/base_consumer.rs index 5fcc42bc5..08ec51b78 100644 --- a/src/consumer/base_consumer.rs +++ b/src/consumer/base_consumer.rs @@ -5,7 +5,7 @@ use std::mem::ManuallyDrop; use std::os::raw::c_void; use std::ptr; use std::sync::Arc; -use std::time::Duration; +use std::time::{Duration, Instant}; use log::{error, warn}; use rdkafka_sys as rdsys; @@ -123,6 +123,7 @@ where queue: &NativeQueue, timeout: T, ) -> Option>> { + let now = Instant::now(); let mut timeout = timeout.into(); let min_poll_interval = self.context().main_queue_min_poll_interval(); loop { @@ -158,10 +159,10 @@ where } } - if op_timeout >= timeout { + timeout = timeout.saturating_sub(now.elapsed()); + if timeout.is_zero() { return None; } - timeout -= op_timeout; } } diff --git a/src/util.rs b/src/util.rs index 16b146f58..543481d3f 100644 --- a/src/util.rs +++ b/src/util.rs @@ -48,6 +48,22 @@ impl Timeout { Timeout::Never => -1, } } + + /// Saturating `Duration` subtraction to Timeout. + pub(crate) fn saturating_sub(&self, rhs: Duration) -> Timeout { + match (self, rhs) { + (Timeout::After(lhs), rhs) => Timeout::After(lhs.saturating_sub(rhs)), + (Timeout::Never, _) => Timeout::Never, + } + } + + /// Returns `true` if the timeout is zero. + pub(crate) fn is_zero(&self) -> bool { + match self { + Timeout::After(d) => d.is_zero(), + Timeout::Never => false, + } + } } impl std::ops::SubAssign for Timeout { From c356f05713149f9aa388a0825954afa18e14c69c Mon Sep 17 00:00:00 2001 From: David Blewett Date: Wed, 8 Nov 2023 11:06:56 -0500 Subject: [PATCH 32/59] Release v0.36.0. --- Cargo.lock | 2 +- Cargo.toml | 2 +- changelog.md | 5 +++++ 3 files changed, 7 insertions(+), 2 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index b719eeca7..1d3ab07a5 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1070,7 +1070,7 @@ dependencies = [ [[package]] name = "rdkafka" -version = "0.35.0" +version = "0.36.0" dependencies = [ "async-std", "backoff", diff --git a/Cargo.toml b/Cargo.toml index a05b1a30f..2571e6303 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "rdkafka" -version = "0.35.0" +version = "0.36.0" authors = ["Federico Giraud "] repository = "https://github.com/fede1024/rust-rdkafka" readme = "README.md" diff --git a/changelog.md b/changelog.md index e954bcebc..5c866864e 100644 --- a/changelog.md +++ b/changelog.md @@ -4,6 +4,11 @@ See also the [rdkafka-sys changelog](rdkafka-sys/changelog.md). ## Unreleased +## 0.36.0 (2023-11-08) + +* Pivot the library from using librdkafka's callback interface to using + the event interface. The public API of the crate does not change. + ## 0.35.0 (2023-11-07) * Update bundled librdkafka to 2.3.0. From ecbc83a8d507fc5d94a2adb50ee4cfc2e0e821d7 Mon Sep 17 00:00:00 2001 From: David Blewett Date: Thu, 11 Jan 2024 14:57:47 -0500 Subject: [PATCH 33/59] Use a short timeout instead of now_or_never. --- tests/test_high_consumers.rs | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/tests/test_high_consumers.rs b/tests/test_high_consumers.rs index 97ca4f5a0..548ccb94d 100644 --- a/tests/test_high_consumers.rs +++ b/tests/test_high_consumers.rs @@ -4,7 +4,7 @@ use std::collections::HashMap; use std::error::Error; use std::sync::Arc; -use futures::future::{self, FutureExt}; +use futures::future; use futures::stream::StreamExt; use maplit::hashmap; use rdkafka_sys::RDKafkaErrorCode; @@ -491,7 +491,7 @@ async fn test_consumer_commit_metadata() -> Result<(), Box> { Ok(()) } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] async fn test_consume_partition_order() { let _r = env_logger::try_init(); @@ -545,8 +545,8 @@ async fn test_consume_partition_order() { let partition1 = consumer.split_partition_queue(&topic_name, 1).unwrap(); let mut i = 0; - while i < 12 { - if let Some(m) = consumer.recv().now_or_never() { + while i < 5 { + if let Ok(m) = time::timeout(Duration::from_millis(1000), consumer.recv()).await { // retry on transient errors until we get a message let m = match m { Err(KafkaError::MessageConsumption( @@ -564,9 +564,11 @@ async fn test_consume_partition_order() { let partition: i32 = m.partition(); assert!(partition == 0 || partition == 2); i += 1; + } else { + panic!("Timeout receiving message"); } - if let Some(m) = partition1.recv().now_or_never() { + if let Ok(m) = time::timeout(Duration::from_millis(1000), partition1.recv()).await { // retry on transient errors until we get a message let m = match m { Err(KafkaError::MessageConsumption( @@ -583,6 +585,8 @@ async fn test_consume_partition_order() { }; assert_eq!(m.partition(), 1); i += 1; + } else { + panic!("Timeout receiving message"); } } } From f360f576df7cf0fa1a11d2da4b75dbb08127f487 Mon Sep 17 00:00:00 2001 From: David Blewett Date: Thu, 11 Jan 2024 14:58:07 -0500 Subject: [PATCH 34/59] Disable valgrind for now, and start up kafka/zk before starting building --- test_suite.sh | 50 ++++++++++++++++++++++++++------------------------ 1 file changed, 26 insertions(+), 24 deletions(-) diff --git a/test_suite.sh b/test_suite.sh index 3d875bad9..900abd507 100755 --- a/test_suite.sh +++ b/test_suite.sh @@ -29,34 +29,36 @@ run_with_valgrind() { # Initialize. git submodule update --init -cargo test --no-run docker-compose up -d +cargo test # Run unit tests. -echo_good "*** Run unit tests ***" -for test_file in target/debug/deps/rdkafka-* -do - if [[ -x "$test_file" ]] - then - echo_good "Executing "$test_file"" - run_with_valgrind "$test_file" - fi -done -echo_good "*** Unit tests succeeded ***" - -# Run integration tests. - -echo_good "*** Run unit tests ***" -for test_file in target/debug/deps/test_* -do - if [[ -x "$test_file" ]] - then - echo_good "Executing "$test_file"" - run_with_valgrind "$test_file" - fi -done -echo_good "*** Integration tests succeeded ***" +#echo_good "*** Run unit tests ***" +#for test_file in target/debug/deps/rdkafka-* +#do +# if [[ -x "$test_file" ]] +# then +# echo_good "Executing "$test_file"" +# run_with_valgrind "$test_file" +# fi +#done +#echo_good "*** Unit tests succeeded ***" +# +## Run integration tests. +# +#echo_good "*** Run integration tests ***" +#for test_file in target/debug/deps/test_* +#do +# if [[ -x "$test_file" ]] +# then +# #echo_good "*** Restarting kafka/zk ***" +# #docker-compose restart --timeout 30 +# echo_good "Executing "$test_file"" +# run_with_valgrind "$test_file" +# fi +#done +#echo_good "*** Integration tests succeeded ***" # Run smol runtime example. From ddfc3b43b7b608dc0b5c0f5ef2d994f94a95e249 Mon Sep 17 00:00:00 2001 From: David Blewett Date: Thu, 11 Jan 2024 15:30:10 -0500 Subject: [PATCH 35/59] Avoid topic pollution by prefixing with test name. --- tests/test_admin.rs | 14 +++++++------- tests/test_high_consumers.rs | 20 ++++++++++---------- tests/test_high_producers.rs | 4 ++-- tests/test_low_consumers.rs | 12 ++++++------ tests/test_low_producers.rs | 22 ++++++++++++++-------- tests/test_metadata.rs | 6 +++--- tests/test_transactions.rs | 8 ++++---- tests/utils.rs | 6 +++--- 8 files changed, 49 insertions(+), 43 deletions(-) diff --git a/tests/test_admin.rs b/tests/test_admin.rs index 79c20ac81..846a96c2a 100644 --- a/tests/test_admin.rs +++ b/tests/test_admin.rs @@ -32,7 +32,7 @@ fn create_admin_client() -> AdminClient { async fn create_consumer_group(consumer_group_name: &str) { let admin_client = create_admin_client(); - let topic_name = &rand_test_topic(); + let topic_name = &rand_test_topic(consumer_group_name); let consumer: BaseConsumer = create_config() .set("group.id", consumer_group_name.clone()) .create() @@ -124,8 +124,8 @@ async fn test_topics() { // Verify that topics are created as specified, and that they can later // be deleted. { - let name1 = rand_test_topic(); - let name2 = rand_test_topic(); + let name1 = rand_test_topic("test_topics"); + let name2 = rand_test_topic("test_topics"); // Test both the builder API and the literal construction. let topic1 = @@ -254,7 +254,7 @@ async fn test_topics() { // Verify that incorrect replication configurations are ignored when // creating partitions. { - let name = rand_test_topic(); + let name = rand_test_topic("test_topics"); let topic = NewTopic::new(&name, 1, TopicReplication::Fixed(1)); let res = admin_client @@ -291,7 +291,7 @@ async fn test_topics() { // Verify that deleting a non-existent topic fails. { - let name = rand_test_topic(); + let name = rand_test_topic("test_topics"); let res = admin_client .delete_topics(&[&name], &opts) .await @@ -305,8 +305,8 @@ async fn test_topics() { // Verify that mixed-success operations properly report the successful and // failing operators. { - let name1 = rand_test_topic(); - let name2 = rand_test_topic(); + let name1 = rand_test_topic("test_topics"); + let name2 = rand_test_topic("test_topics"); let topic1 = NewTopic::new(&name1, 1, TopicReplication::Fixed(1)); let topic2 = NewTopic::new(&name2, 1, TopicReplication::Fixed(1)); diff --git a/tests/test_high_consumers.rs b/tests/test_high_consumers.rs index 548ccb94d..4aec62595 100644 --- a/tests/test_high_consumers.rs +++ b/tests/test_high_consumers.rs @@ -70,7 +70,7 @@ async fn test_produce_consume_base() { let _r = env_logger::try_init(); let start_time = current_time_millis(); - let topic_name = rand_test_topic(); + let topic_name = rand_test_topic("test_produce_consume_base"); let message_map = populate_topic(&topic_name, 100, &value_fn, &key_fn, None, None).await; let consumer = create_stream_consumer(&rand_test_group(), None); consumer.subscribe(&[topic_name.as_str()]).unwrap(); @@ -105,7 +105,7 @@ async fn test_produce_consume_base() { async fn test_produce_consume_base_concurrent() { let _r = env_logger::try_init(); - let topic_name = rand_test_topic(); + let topic_name = rand_test_topic("test_produce_consume_base_concurrent"); populate_topic(&topic_name, 100, &value_fn, &key_fn, None, None).await; let consumer = Arc::new(create_stream_consumer(&rand_test_group(), None)); @@ -135,7 +135,7 @@ async fn test_produce_consume_base_concurrent() { async fn test_produce_consume_base_assign() { let _r = env_logger::try_init(); - let topic_name = rand_test_topic(); + let topic_name = rand_test_topic("test_produce_consume_base_assign"); populate_topic(&topic_name, 10, &value_fn, &key_fn, Some(0), None).await; populate_topic(&topic_name, 10, &value_fn, &key_fn, Some(1), None).await; populate_topic(&topic_name, 10, &value_fn, &key_fn, Some(2), None).await; @@ -170,7 +170,7 @@ async fn test_produce_consume_base_assign() { async fn test_produce_consume_base_unassign() { let _r = env_logger::try_init(); - let topic_name = rand_test_topic(); + let topic_name = rand_test_topic("test_produce_consume_base_unassign"); populate_topic(&topic_name, 10, &value_fn, &key_fn, Some(0), None).await; populate_topic(&topic_name, 10, &value_fn, &key_fn, Some(1), None).await; populate_topic(&topic_name, 10, &value_fn, &key_fn, Some(2), None).await; @@ -195,7 +195,7 @@ async fn test_produce_consume_base_unassign() { async fn test_produce_consume_base_incremental_assign_and_unassign() { let _r = env_logger::try_init(); - let topic_name = rand_test_topic(); + let topic_name = rand_test_topic("test_produce_consume_base_incremental_assign_and_unassign"); populate_topic(&topic_name, 10, &value_fn, &key_fn, Some(0), None).await; populate_topic(&topic_name, 10, &value_fn, &key_fn, Some(1), None).await; populate_topic(&topic_name, 10, &value_fn, &key_fn, Some(2), None).await; @@ -236,7 +236,7 @@ async fn test_produce_consume_base_incremental_assign_and_unassign() { async fn test_produce_consume_with_timestamp() { let _r = env_logger::try_init(); - let topic_name = rand_test_topic(); + let topic_name = rand_test_topic("test_produce_consume_with_timestamp"); let message_map = populate_topic(&topic_name, 100, &value_fn, &key_fn, Some(0), Some(1111)).await; let consumer = create_stream_consumer(&rand_test_group(), None); @@ -277,7 +277,7 @@ async fn test_produce_consume_with_timestamp() { async fn test_consumer_commit_message() { let _r = env_logger::try_init(); - let topic_name = rand_test_topic(); + let topic_name = rand_test_topic("test_consumer_commit_message"); populate_topic(&topic_name, 10, &value_fn, &key_fn, Some(0), None).await; populate_topic(&topic_name, 11, &value_fn, &key_fn, Some(1), None).await; populate_topic(&topic_name, 12, &value_fn, &key_fn, Some(2), None).await; @@ -355,7 +355,7 @@ async fn test_consumer_commit_message() { async fn test_consumer_store_offset_commit() { let _r = env_logger::try_init(); - let topic_name = rand_test_topic(); + let topic_name = rand_test_topic("test_consumer_store_offset_commit"); populate_topic(&topic_name, 10, &value_fn, &key_fn, Some(0), None).await; populate_topic(&topic_name, 11, &value_fn, &key_fn, Some(1), None).await; populate_topic(&topic_name, 12, &value_fn, &key_fn, Some(2), None).await; @@ -440,7 +440,7 @@ async fn test_consumer_store_offset_commit() { async fn test_consumer_commit_metadata() -> Result<(), Box> { let _ = env_logger::try_init(); - let topic_name = rand_test_topic(); + let topic_name = rand_test_topic("test_consumer_commit_metadata"); let group_name = rand_test_group(); populate_topic(&topic_name, 10, &value_fn, &key_fn, None, None).await; @@ -495,7 +495,7 @@ async fn test_consumer_commit_metadata() -> Result<(), Box> { async fn test_consume_partition_order() { let _r = env_logger::try_init(); - let topic_name = rand_test_topic(); + let topic_name = rand_test_topic("test_consume_partition_order"); populate_topic(&topic_name, 4, &value_fn, &key_fn, Some(0), None).await; populate_topic(&topic_name, 4, &value_fn, &key_fn, Some(1), None).await; populate_topic(&topic_name, 4, &value_fn, &key_fn, Some(2), None).await; diff --git a/tests/test_high_producers.rs b/tests/test_high_producers.rs index 85b8784e0..bddc1beae 100644 --- a/tests/test_high_producers.rs +++ b/tests/test_high_producers.rs @@ -30,7 +30,7 @@ fn future_producer(config_overrides: HashMap<&str, &str>) -> FutureProducer = (0..10) .map(|_| { @@ -60,7 +60,7 @@ async fn test_future_producer_send_full() { config.insert("message.timeout.ms", "5000"); config.insert("queue.buffering.max.messages", "1"); let producer = &future_producer(config); - let topic_name = &rand_test_topic(); + let topic_name = &rand_test_topic("test_future_producer_send_full"); // Fill up the queue. producer diff --git a/tests/test_low_consumers.rs b/tests/test_low_consumers.rs index c4aa305f7..e6642b688 100644 --- a/tests/test_low_consumers.rs +++ b/tests/test_low_consumers.rs @@ -31,7 +31,7 @@ fn create_base_consumer( async fn test_produce_consume_seek() { let _r = env_logger::try_init(); - let topic_name = rand_test_topic(); + let topic_name = rand_test_topic("test_produce_consume_seek"); populate_topic(&topic_name, 5, &value_fn, &key_fn, Some(0), None).await; let consumer = create_base_consumer(&rand_test_group(), None); consumer.subscribe(&[topic_name.as_str()]).unwrap(); @@ -96,7 +96,7 @@ async fn test_produce_consume_seek() { async fn test_produce_consume_seek_partitions() { let _r = env_logger::try_init(); - let topic_name = rand_test_topic(); + let topic_name = rand_test_topic("test_produce_consume_seek_partitions"); populate_topic(&topic_name, 30, &value_fn, &key_fn, None, None).await; let consumer = create_base_consumer(&rand_test_group(), None); @@ -158,7 +158,7 @@ async fn test_produce_consume_iter() { let _r = env_logger::try_init(); let start_time = current_time_millis(); - let topic_name = rand_test_topic(); + let topic_name = rand_test_topic("test_produce_consume_iter"); let message_map = populate_topic(&topic_name, 100, &value_fn, &key_fn, None, None).await; let consumer = create_base_consumer(&rand_test_group(), None); consumer.subscribe(&[topic_name.as_str()]).unwrap(); @@ -196,7 +196,7 @@ async fn test_pause_resume_consumer_iter() { let _r = env_logger::try_init(); - let topic_name = rand_test_topic(); + let topic_name = rand_test_topic("test_pause_resume_consumer_iter"); populate_topic( &topic_name, MESSAGE_COUNT, @@ -237,7 +237,7 @@ async fn test_pause_resume_consumer_iter() { async fn test_consume_partition_order() { let _r = env_logger::try_init(); - let topic_name = rand_test_topic(); + let topic_name = rand_test_topic("test_consume_partition_order"); populate_topic(&topic_name, 4, &value_fn, &key_fn, Some(0), None).await; populate_topic(&topic_name, 4, &value_fn, &key_fn, Some(1), None).await; populate_topic(&topic_name, 4, &value_fn, &key_fn, Some(2), None).await; @@ -357,7 +357,7 @@ async fn test_consume_partition_order() { async fn test_produce_consume_message_queue_nonempty_callback() { let _r = env_logger::try_init(); - let topic_name = rand_test_topic(); + let topic_name = rand_test_topic("test_produce_consume_message_queue_nonempty_callback"); create_topic(&topic_name, 1).await; diff --git a/tests/test_low_producers.rs b/tests/test_low_producers.rs index bf5c9ef2a..493642617 100644 --- a/tests/test_low_producers.rs +++ b/tests/test_low_producers.rs @@ -191,7 +191,7 @@ where #[test] fn test_base_producer_queue_full() { let producer = base_producer(hashmap! { "queue.buffering.max.messages" => "10" }); - let topic_name = rand_test_topic(); + let topic_name = rand_test_topic("test_base_producer_queue_full"); let results = (0..30) .map(|id| { @@ -235,7 +235,7 @@ fn test_base_producer_timeout() { "bootstrap.servers" => "1.2.3.4" }, ); - let topic_name = rand_test_topic(); + let topic_name = rand_test_topic("test_base_producer_timeout"); let results_count = (0..10) .map(|id| { @@ -346,7 +346,7 @@ fn test_base_producer_headers() { ids: ids_set.clone(), }; let producer = base_producer_with_context(context, HashMap::new()); - let topic_name = rand_test_topic(); + let topic_name = rand_test_topic("test_base_producer_headers"); let results_count = (0..10) .map(|id| { @@ -387,7 +387,7 @@ fn test_base_producer_headers() { fn test_threaded_producer_send() { let context = CollectingContext::new(); let producer = threaded_producer_with_context(context.clone(), HashMap::new()); - let topic_name = rand_test_topic(); + let topic_name = rand_test_topic("test_threaded_producer_send"); let results_count = (0..10) .map(|id| { @@ -431,7 +431,7 @@ fn test_base_producer_opaque_arc() -> Result<(), Box> { let shared_count = Arc::new(Mutex::new(0)); let context = OpaqueArcContext {}; let producer = base_producer_with_context(context, HashMap::new()); - let topic_name = rand_test_topic(); + let topic_name = rand_test_topic("test_base_producer_opaque_arc"); let results_count = (0..10) .map(|_| { @@ -482,7 +482,13 @@ fn test_register_custom_partitioner_linger_non_zero_key_null() { let producer = base_producer_with_context(context.clone(), config_overrides); producer - .send(BaseRecord::<(), str, usize>::with_opaque_to(&rand_test_topic(), 0).payload("")) + .send( + BaseRecord::<(), str, usize>::with_opaque_to( + &rand_test_topic("test_register_custom_partitioner_linger_non_zero_key_null"), + 0, + ) + .payload(""), + ) .unwrap(); producer.flush(Duration::from_secs(10)).unwrap(); @@ -499,7 +505,7 @@ fn test_register_custom_partitioner_linger_non_zero_key_null() { fn test_custom_partitioner_base_producer() { let context = CollectingContext::new_with_custom_partitioner(FixedPartitioner::new(2)); let producer = base_producer_with_context(context.clone(), HashMap::new()); - let topic_name = rand_test_topic(); + let topic_name = rand_test_topic("test_custom_partitioner_base_producer"); let results_count = (0..10) .map(|id| { @@ -527,7 +533,7 @@ fn test_custom_partitioner_base_producer() { fn test_custom_partitioner_threaded_producer() { let context = CollectingContext::new_with_custom_partitioner(FixedPartitioner::new(2)); let producer = threaded_producer_with_context(context.clone(), HashMap::new()); - let topic_name = rand_test_topic(); + let topic_name = rand_test_topic("test_custom_partitioner_threaded_producer"); let results_count = (0..10) .map(|id| { diff --git a/tests/test_metadata.rs b/tests/test_metadata.rs index e62bee556..ba4d4f2ca 100644 --- a/tests/test_metadata.rs +++ b/tests/test_metadata.rs @@ -31,7 +31,7 @@ fn create_consumer(group_id: &str) -> StreamConsumer { async fn test_metadata() { let _r = env_logger::try_init(); - let topic_name = rand_test_topic(); + let topic_name = rand_test_topic("test_metadata"); populate_topic(&topic_name, 1, &value_fn, &key_fn, Some(0), None).await; populate_topic(&topic_name, 1, &value_fn, &key_fn, Some(1), None).await; populate_topic(&topic_name, 1, &value_fn, &key_fn, Some(2), None).await; @@ -92,7 +92,7 @@ async fn test_metadata() { async fn test_subscription() { let _r = env_logger::try_init(); - let topic_name = rand_test_topic(); + let topic_name = rand_test_topic("test_subscription"); populate_topic(&topic_name, 10, &value_fn, &key_fn, None, None).await; let consumer = create_consumer(&rand_test_group()); consumer.subscribe(&[topic_name.as_str()]).unwrap(); @@ -109,7 +109,7 @@ async fn test_subscription() { async fn test_group_membership() { let _r = env_logger::try_init(); - let topic_name = rand_test_topic(); + let topic_name = rand_test_topic("test_group_membership"); let group_name = rand_test_group(); populate_topic(&topic_name, 1, &value_fn, &key_fn, Some(0), None).await; populate_topic(&topic_name, 1, &value_fn, &key_fn, Some(1), None).await; diff --git a/tests/test_transactions.rs b/tests/test_transactions.rs index 5d6e47160..1fe84a98b 100644 --- a/tests/test_transactions.rs +++ b/tests/test_transactions.rs @@ -64,8 +64,8 @@ fn count_records(topic: &str, iso: IsolationLevel) -> Result #[tokio::test] async fn test_transaction_abort() -> Result<(), Box> { - let consume_topic = rand_test_topic(); - let produce_topic = rand_test_topic(); + let consume_topic = rand_test_topic("test_transaction_abort"); + let produce_topic = rand_test_topic("test_transaction_abort"); populate_topic(&consume_topic, 30, &value_fn, &key_fn, Some(0), None).await; @@ -132,8 +132,8 @@ async fn test_transaction_abort() -> Result<(), Box> { #[tokio::test] async fn test_transaction_commit() -> Result<(), Box> { - let consume_topic = rand_test_topic(); - let produce_topic = rand_test_topic(); + let consume_topic = rand_test_topic("test_transaction_commit"); + let produce_topic = rand_test_topic("test_transaction_commit"); populate_topic(&consume_topic, 30, &value_fn, &key_fn, Some(0), None).await; diff --git a/tests/utils.rs b/tests/utils.rs index 6730f9747..447213672 100644 --- a/tests/utils.rs +++ b/tests/utils.rs @@ -17,12 +17,12 @@ use rdkafka::producer::{FutureProducer, FutureRecord}; use rdkafka::statistics::Statistics; use rdkafka::TopicPartitionList; -pub fn rand_test_topic() -> String { +pub fn rand_test_topic(test_name: &str) -> String { let id = rand::thread_rng() .gen_ascii_chars() .take(10) .collect::(); - format!("__test_{}", id) + format!("__{}_{}", test_name, id) } pub fn rand_test_group() -> String { @@ -170,7 +170,7 @@ mod tests { #[tokio::test] async fn test_populate_topic() { - let topic_name = rand_test_topic(); + let topic_name = rand_test_topic("test_populate_topic"); let message_map = populate_topic(&topic_name, 100, &value_fn, &key_fn, Some(0), None).await; let total_messages = message_map From f572dcb844ba43c98255729ab8c9e8202e059991 Mon Sep 17 00:00:00 2001 From: Samuel Cantero Date: Thu, 4 Jan 2024 16:01:22 -0300 Subject: [PATCH 36/59] Return back to the caller on rebalance events This does not affect the StreamConsumer or any other wrapper consumer. It will only incur on an extra Poll call when there's a rebalance event. When using bindings built upon the rust-rdkafka ffi, the caller is responsible for initiating the rebalance calls (*assign). If a high timeout is specified, the rebalance handler will only be triggered once the timeout period has elapsed. This fixes it by always returning on rebalance events except when Timeout::Never. Poll calls with timeout::Never are expected to return a message. --- src/consumer/base_consumer.rs | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/consumer/base_consumer.rs b/src/consumer/base_consumer.rs index 08ec51b78..2a45306c5 100644 --- a/src/consumer/base_consumer.rs +++ b/src/consumer/base_consumer.rs @@ -144,9 +144,15 @@ where } rdsys::RD_KAFKA_EVENT_REBALANCE => { self.handle_rebalance_event(event); + if timeout != Timeout::Never { + return None; + } } rdsys::RD_KAFKA_EVENT_OFFSET_COMMIT => { self.handle_offset_commit_event(event); + if timeout != Timeout::Never { + return None; + } } _ => { let buf = unsafe { From e301b5fdaf3abb81cef9a31355094cf8ced24e84 Mon Sep 17 00:00:00 2001 From: David Blewett Date: Thu, 11 Jan 2024 17:30:59 -0500 Subject: [PATCH 37/59] Release v0.36.1. --- Cargo.toml | 2 +- changelog.md | 5 +++++ 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/Cargo.toml b/Cargo.toml index 2571e6303..f76cc0b0a 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "rdkafka" -version = "0.36.0" +version = "0.36.1" authors = ["Federico Giraud "] repository = "https://github.com/fede1024/rust-rdkafka" readme = "README.md" diff --git a/changelog.md b/changelog.md index 5c866864e..bb2a80be2 100644 --- a/changelog.md +++ b/changelog.md @@ -4,6 +4,11 @@ See also the [rdkafka-sys changelog](rdkafka-sys/changelog.md). ## Unreleased +## 0.36.1 (2024-01-11) + +* Update `BaseConsumer::poll` to return `None` when handling rebalance + or offset commit events. + ## 0.36.0 (2023-11-08) * Pivot the library from using librdkafka's callback interface to using From f0543b6bb9d5d3adde55274b1bf1af535638a79b Mon Sep 17 00:00:00 2001 From: David Blewett Date: Thu, 11 Jan 2024 17:51:01 -0500 Subject: [PATCH 38/59] Add Kafka 3.6 to the integration test matrix. --- .github/workflows/ci.yml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index c92519585..96708961d 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -69,6 +69,8 @@ jobs: fail-fast: false matrix: include: + - confluent-version: 7.5.1 + kafka-version: 3.6 - confluent-version: 7.5.1 kafka-version: 3.5 - confluent-version: 5.3.1 From 12d05dfe1748137718ab8e095393670b5303bf25 Mon Sep 17 00:00:00 2001 From: David Blewett Date: Tue, 16 Jan 2024 16:55:30 -0500 Subject: [PATCH 39/59] Release v0.36.2. --- Cargo.lock | 2 +- Cargo.toml | 2 +- changelog.md | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 1d3ab07a5..708bf88b1 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1070,7 +1070,7 @@ dependencies = [ [[package]] name = "rdkafka" -version = "0.36.0" +version = "0.36.2" dependencies = [ "async-std", "backoff", diff --git a/Cargo.toml b/Cargo.toml index f76cc0b0a..e190bd18e 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "rdkafka" -version = "0.36.1" +version = "0.36.2" authors = ["Federico Giraud "] repository = "https://github.com/fede1024/rust-rdkafka" readme = "README.md" diff --git a/changelog.md b/changelog.md index bb2a80be2..1806d5807 100644 --- a/changelog.md +++ b/changelog.md @@ -4,7 +4,7 @@ See also the [rdkafka-sys changelog](rdkafka-sys/changelog.md). ## Unreleased -## 0.36.1 (2024-01-11) +## 0.36.2 (2024-01-16) * Update `BaseConsumer::poll` to return `None` when handling rebalance or offset commit events. From 7834a256c82e51ddfa026cf5e5fb735a140a4928 Mon Sep 17 00:00:00 2001 From: Arpad Borsos Date: Tue, 19 Dec 2023 12:30:11 +0100 Subject: [PATCH 40/59] Use `CStr::to_string_lossy` in Base{Consumer,Producer} In some error cases, the `Base{Consumer,Producer}` were eagerly copying strings, and `unwrap`ing utf8 validation, just to print an error message. This will avoid the allocation in the common case, and be panic-safe in the presumably unreachable case of invalid utf-8. --- src/consumer/base_consumer.rs | 14 ++++++-------- src/mocking.rs | 2 +- src/producer/base_producer.rs | 5 ++--- 3 files changed, 9 insertions(+), 12 deletions(-) diff --git a/src/consumer/base_consumer.rs b/src/consumer/base_consumer.rs index 2a45306c5..58d81078d 100644 --- a/src/consumer/base_consumer.rs +++ b/src/consumer/base_consumer.rs @@ -155,12 +155,11 @@ where } } _ => { - let buf = unsafe { + let evname = unsafe { let evname = rdsys::rd_kafka_event_name(event.ptr()); - CStr::from_ptr(evname).to_bytes() + CStr::from_ptr(evname).to_string_lossy() }; - let evname = String::from_utf8(buf.to_vec()).unwrap(); - warn!("Ignored event '{}' on consumer poll", evname); + warn!("Ignored event '{evname}' on consumer poll"); } } } @@ -198,13 +197,12 @@ where .rebalance(self.client.native_client(), err, &mut tpl); } _ => { - let buf = unsafe { + let err = unsafe { let err_name = rdsys::rd_kafka_err2name(rdsys::rd_kafka_event_error(event.ptr())); - CStr::from_ptr(err_name).to_bytes() + CStr::from_ptr(err_name).to_string_lossy() }; - let err = String::from_utf8(buf.to_vec()).unwrap(); - warn!("invalid rebalance event: {:?}", err); + warn!("invalid rebalance event: {err}"); } } } diff --git a/src/mocking.rs b/src/mocking.rs index 474b36f02..b07b05230 100644 --- a/src/mocking.rs +++ b/src/mocking.rs @@ -137,7 +137,7 @@ where pub fn bootstrap_servers(&self) -> String { let bootstrap = unsafe { CStr::from_ptr(rdsys::rd_kafka_mock_cluster_bootstraps(self.mock_cluster)) }; - bootstrap.to_string_lossy().to_string() + bootstrap.to_string_lossy().into_owned() } /// Clear the cluster's error state for the given ApiKey. diff --git a/src/producer/base_producer.rs b/src/producer/base_producer.rs index 886d5bdee..1cc6e05ce 100644 --- a/src/producer/base_producer.rs +++ b/src/producer/base_producer.rs @@ -368,11 +368,10 @@ where match evtype { rdsys::RD_KAFKA_EVENT_DR => self.handle_delivery_report_event(ev), _ => { - let buf = unsafe { + let evname = unsafe { let evname = rdsys::rd_kafka_event_name(ev.ptr()); - CStr::from_ptr(evname).to_bytes() + CStr::from_ptr(evname).to_string_lossy() }; - let evname = String::from_utf8(buf.to_vec()).unwrap(); warn!("Ignored event '{}' on base producer poll", evname); } } From 8d74526edd37fb838c445c6bb57cd00c055203df Mon Sep 17 00:00:00 2001 From: Misha Padalka Date: Tue, 13 Jun 2023 17:08:38 +0300 Subject: [PATCH 41/59] Fix panic on getting config value from NativeConfig Kafka can return string with multiple \0 chars (seen on Windows x64), and CStr::from_bytes_with_nul panics in that case. String::from_utf8_lossy() handles that ok --- src/config.rs | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/src/config.rs b/src/config.rs index da4885a78..296d9f867 100644 --- a/src/config.rs +++ b/src/config.rs @@ -150,10 +150,7 @@ impl NativeClientConfig { } // Convert the C string to a Rust string. - Ok(CStr::from_bytes_with_nul(&buf) - .unwrap() - .to_string_lossy() - .into()) + Ok(String::from_utf8_lossy(&buf).to_string()) } } From b97754b26e1c0a7f037e15bfbfc8cf20ea5779fb Mon Sep 17 00:00:00 2001 From: KowalczykBartek Date: Wed, 29 May 2024 19:26:40 +0200 Subject: [PATCH 42/59] Remove nul chars from string before parsing to numerical values --- src/config.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/config.rs b/src/config.rs index 296d9f867..58e062a5b 100644 --- a/src/config.rs +++ b/src/config.rs @@ -150,7 +150,7 @@ impl NativeClientConfig { } // Convert the C string to a Rust string. - Ok(String::from_utf8_lossy(&buf).to_string()) + Ok(String::from_utf8_lossy(&buf).trim_matches(char::from(0)).to_string()) } } From b3a9d3564684ba0f9a98a1637c58e6adcdcd26a4 Mon Sep 17 00:00:00 2001 From: Federico Giraud Date: Sun, 4 Aug 2024 16:41:07 +0100 Subject: [PATCH 43/59] Fix integration tests Fixed several issues with the current test suite: * Missing cmake dependency * Rust version being too old for some dependencies * Missing null pointer check in src/groups.rs * Test timeout being too strict * CI test jobs not working in parallel This also fixes a few compiler warnings (unused imports, unused code, etc.) and formatting issues. --- .github/workflows/ci.yml | 32 +++++++++++++++++--------------- Cargo.toml | 2 +- changelog.md | 4 ++++ docker-compose.yaml | 2 -- rdkafka-sys/Cargo.toml | 2 +- rdkafka-sys/src/lib.rs | 2 +- src/config.rs | 6 ++++-- src/groups.rs | 3 +++ src/lib.rs | 2 +- src/mocking.rs | 2 ++ src/util.rs | 33 ++++++--------------------------- test_suite.sh | 8 ++++++-- tests/test_admin.rs | 2 +- 13 files changed, 47 insertions(+), 53 deletions(-) diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index 96708961d..8d388f2e2 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -7,14 +7,14 @@ on: branches: [master] env: - rust_version: 1.61.0 + rust_version: 1.70.0 jobs: lint: runs-on: ubuntu-20.04 steps: - uses: actions/checkout@v3 - - uses: dtolnay/rust-toolchain@b44cb146d03e8d870c57ab64b80f04586349ca5d + - uses: dtolnay/rust-toolchain@stable with: toolchain: ${{ env.rust_version }} components: rustfmt, clippy @@ -38,7 +38,8 @@ jobs: runs-on: ${{ matrix.os }} steps: - uses: actions/checkout@v3 - - uses: dtolnay/rust-toolchain@b44cb146d03e8d870c57ab64b80f04586349ca5d + - uses: lukka/get-cmake@latest + - uses: dtolnay/rust-toolchain@stable with: toolchain: ${{ env.rust_version }} - run: cargo build --all-targets --verbose --features "${{ matrix.features }}" @@ -50,43 +51,44 @@ jobs: runs-on: ubuntu-20.04 steps: - uses: actions/checkout@v3 - - uses: dtolnay/rust-toolchain@b44cb146d03e8d870c57ab64b80f04586349ca5d + - uses: dtolnay/rust-toolchain@stable with: # The version of this toolchain doesn't matter much. It's only used to # generate the minimal-versions lockfile, not to actually run `cargo # check`. toolchain: nightly components: rustfmt, clippy - - uses: dtolnay/rust-toolchain@b44cb146d03e8d870c57ab64b80f04586349ca5d + - uses: dtolnay/rust-toolchain@stable with: toolchain: ${{ env.rust_version }} - run: rustup default ${{ env.rust_version }} - run: cargo +nightly -Z minimal-versions generate-lockfile - - run: cargo check + # Default features and features that require optional dependencies should be + # explicitly checked. + - run: cargo check --features libz,tokio,tracing test: strategy: fail-fast: false + # The test suite doesn't support concurrent runs. + max-parallel: 1 matrix: include: + - confluent-version: 7.7.0 + kafka-version: 3.7 - confluent-version: 7.5.1 kafka-version: 3.6 - confluent-version: 7.5.1 kafka-version: 3.5 - - confluent-version: 5.3.1 - kafka-version: 2.3 - - confluent-version: 5.0.3 - kafka-version: 2.0 - - confluent-version: 4.1.3 - kafka-version: 1.1 runs-on: ubuntu-20.04 steps: - uses: actions/checkout@v3 - - uses: dtolnay/rust-toolchain@b44cb146d03e8d870c57ab64b80f04586349ca5d + - uses: lukka/get-cmake@latest + - uses: dtolnay/rust-toolchain@stable with: toolchain: ${{ env.rust_version }} - - run: sudo apt-get update - - run: sudo apt-get install -qy valgrind + # - run: sudo apt-get update + # - run: sudo apt-get install -qy valgrind # Valgrind currently disabled in testing - run: ./test_suite.sh env: CONFLUENT_VERSION: ${{ matrix.confluent-version }} diff --git a/Cargo.toml b/Cargo.toml index e190bd18e..b9045ce58 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -10,7 +10,7 @@ keywords = ["kafka", "rdkafka"] categories = ["api-bindings"] edition = "2018" exclude = ["Cargo.lock"] -rust-version = "1.61" +rust-version = "1.70" [workspace] members = ["rdkafka-sys"] diff --git a/changelog.md b/changelog.md index 1806d5807..c87f5afc4 100644 --- a/changelog.md +++ b/changelog.md @@ -4,6 +4,10 @@ See also the [rdkafka-sys changelog](rdkafka-sys/changelog.md). ## Unreleased +* Update MSRV to 1.70 +* Remove testign for old Kafka versions (before 3.0). Add tests for 3.7. +* Fix test dependency on docker compose. + ## 0.36.2 (2024-01-16) * Update `BaseConsumer::poll` to return `None` when handling rebalance diff --git a/docker-compose.yaml b/docker-compose.yaml index a194b25f4..a20430ac6 100644 --- a/docker-compose.yaml +++ b/docker-compose.yaml @@ -1,5 +1,3 @@ -version: '3' - services: kafka: image: confluentinc/cp-kafka:${CONFLUENT_VERSION:-7.5.1} diff --git a/rdkafka-sys/Cargo.toml b/rdkafka-sys/Cargo.toml index 4870e15b3..7ea379ef8 100644 --- a/rdkafka-sys/Cargo.toml +++ b/rdkafka-sys/Cargo.toml @@ -10,7 +10,7 @@ description = "Native bindings to the librdkafka library" keywords = ["kafka", "rdkafka"] categories = ["external-ffi-bindings"] edition = "2018" -rust-version = "1.61" +rust-version = "1.70" [dependencies] num_enum = "0.5.0" diff --git a/rdkafka-sys/src/lib.rs b/rdkafka-sys/src/lib.rs index 8679ad407..565f00bce 100644 --- a/rdkafka-sys/src/lib.rs +++ b/rdkafka-sys/src/lib.rs @@ -100,7 +100,7 @@ extern crate sasl2_sys; #[cfg(feature = "libz-sys")] extern crate libz_sys; -#[cfg(any(feature = "curl-sys", feature = "curl-sys/static-curl"))] +#[cfg(any(feature = "curl-sys", feature = "curl-static"))] extern crate curl_sys; #[cfg(feature = "zstd-sys")] diff --git a/src/config.rs b/src/config.rs index 58e062a5b..fc116d2a7 100644 --- a/src/config.rs +++ b/src/config.rs @@ -23,7 +23,7 @@ //! [librdkafka-config]: https://github.com/edenhill/librdkafka/blob/master/CONFIGURATION.md use std::collections::HashMap; -use std::ffi::{CStr, CString}; +use std::ffi::CString; use std::iter::FromIterator; use std::os::raw::c_char; use std::ptr; @@ -150,7 +150,9 @@ impl NativeClientConfig { } // Convert the C string to a Rust string. - Ok(String::from_utf8_lossy(&buf).trim_matches(char::from(0)).to_string()) + Ok(String::from_utf8_lossy(&buf) + .trim_matches(char::from(0)) + .to_string()) } } diff --git a/src/groups.rs b/src/groups.rs index 2c805dc79..29bcbf297 100644 --- a/src/groups.rs +++ b/src/groups.rs @@ -84,6 +84,9 @@ impl GroupInfo { /// Returns the members of the group. pub fn members(&self) -> &[GroupMemberInfo] { + if self.0.members.is_null() { + return &[]; + } unsafe { slice::from_raw_parts( self.0.members as *const GroupMemberInfo, diff --git a/src/lib.rs b/src/lib.rs index 79a8d113f..46709c5a7 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -176,7 +176,7 @@ //! //! ### Minimum supported Rust version (MSRV) //! -//! The current minimum supported Rust version (MSRV) is 1.61.0. Note that +//! The current minimum supported Rust version (MSRV) is 1.70.0. Note that //! bumping the MSRV is not considered a breaking change. Any release of //! rust-rdkafka may bump the MSRV. //! diff --git a/src/mocking.rs b/src/mocking.rs index b07b05230..5117b532f 100644 --- a/src/mocking.rs +++ b/src/mocking.rs @@ -39,7 +39,9 @@ use crate::ClientContext; /// In this case, we **must neither** destroy the mock cluster in `MockCluster`'s `drop()`, /// **nor** outlive the `Client` from which the reference is obtained, hence the lifetime. enum MockClusterClient<'c, C: ClientContext> { + #[allow(dead_code)] Owned(Client), + #[allow(dead_code)] Borrowed(&'c Client), } diff --git a/src/util.rs b/src/util.rs index 543481d3f..dd4862825 100644 --- a/src/util.rs +++ b/src/util.rs @@ -251,24 +251,13 @@ impl fmt::Display for ErrBuf { } } -pub(crate) trait WrappedCPointer { - type Target; - - fn ptr(&self) -> *mut Self::Target; - - fn is_null(&self) -> bool { - self.ptr().is_null() - } +pub(crate) trait AsCArray { + fn as_c_array(&self) -> *mut *mut T; } -/// Converts a container into a C array. -pub(crate) trait AsCArray { - fn as_c_array(&self) -> *mut *mut T::Target; -} - -impl AsCArray for Vec { - fn as_c_array(&self) -> *mut *mut T::Target { - self.as_ptr() as *mut *mut T::Target +impl AsCArray for Vec> { + fn as_c_array(&self) -> *mut *mut T { + self.as_ptr() as *mut *mut T } } @@ -297,17 +286,6 @@ pub(crate) unsafe trait KafkaDrop { const DROP: unsafe extern "C" fn(*mut Self); } -impl WrappedCPointer for NativePtr -where - T: KafkaDrop, -{ - type Target = T; - - fn ptr(&self) -> *mut T { - self.ptr.as_ptr() - } -} - impl Deref for NativePtr where T: KafkaDrop, @@ -340,6 +318,7 @@ where } } +#[allow(dead_code)] pub(crate) struct OnDrop(pub F) where F: Fn(); diff --git a/test_suite.sh b/test_suite.sh index 900abd507..396638cb9 100755 --- a/test_suite.sh +++ b/test_suite.sh @@ -29,8 +29,12 @@ run_with_valgrind() { # Initialize. git submodule update --init -docker-compose up -d -cargo test +docker compose up -d --wait + +# Run integration tests + +RUST_LOG=1 RUST_BACKTRACE=1 cargo test + # Run unit tests. diff --git a/tests/test_admin.rs b/tests/test_admin.rs index 846a96c2a..dc7177708 100644 --- a/tests/test_admin.rs +++ b/tests/test_admin.rs @@ -119,7 +119,7 @@ fn verify_delete(topic: &str) { #[tokio::test] async fn test_topics() { let admin_client = create_admin_client(); - let opts = AdminOptions::new().operation_timeout(Some(Duration::from_secs(1))); + let opts = AdminOptions::new().operation_timeout(Some(Duration::from_secs(30))); // Verify that topics are created as specified, and that they can later // be deleted. From c2fb3188a6db132fac8271c96a31c63cc3a6a8f6 Mon Sep 17 00:00:00 2001 From: Federico Giraud Date: Sun, 4 Aug 2024 18:35:46 +0100 Subject: [PATCH 44/59] Fix clippy warnings Fix various clippy warnings in both source and tests --- .github/workflows/ci.yml | 1 + rdkafka-sys/build.rs | 2 +- src/consumer/mod.rs | 4 +-- src/message.rs | 8 +++--- src/producer/base_producer.rs | 2 ++ src/producer/future_producer.rs | 1 + src/statistics.rs | 4 +-- src/topic_partition_list.rs | 6 ++--- tests/test_admin.rs | 22 +++++++++------- tests/test_high_producers.rs | 4 +-- tests/test_low_consumers.rs | 10 +++---- tests/test_low_producers.rs | 22 +++++++++------- tests/test_transactions.rs | 4 +-- tests/utils.rs | 46 ++++++++++++++++----------------- 14 files changed, 72 insertions(+), 64 deletions(-) diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index 8d388f2e2..a18f057f9 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -20,6 +20,7 @@ jobs: components: rustfmt, clippy - run: cargo fmt -- --check - run: cargo clippy -- -Dwarnings + - run: cargo clippy --tests -- -Dwarnings - run: cargo test --doc check: diff --git a/rdkafka-sys/build.rs b/rdkafka-sys/build.rs index d615744f3..b7c3e42ca 100644 --- a/rdkafka-sys/build.rs +++ b/rdkafka-sys/build.rs @@ -77,7 +77,7 @@ fn main() { // Ensure that we are in the right directory let rdkafkasys_root = Path::new("rdkafka-sys"); if rdkafkasys_root.exists() { - assert!(env::set_current_dir(&rdkafkasys_root).is_ok()); + assert!(env::set_current_dir(rdkafkasys_root).is_ok()); } if !Path::new("librdkafka/LICENSE").exists() { eprintln!("Setting up submodules"); diff --git a/src/consumer/mod.rs b/src/consumer/mod.rs index fb3ff0460..8b42ffd2c 100644 --- a/src/consumer/mod.rs +++ b/src/consumer/mod.rs @@ -99,12 +99,12 @@ pub trait ConsumerContext: ClientContext { /// Pre-rebalance callback. This method will run before the rebalance and /// should terminate its execution quickly. #[allow(unused_variables)] - fn pre_rebalance<'a>(&self, rebalance: &Rebalance<'a>) {} + fn pre_rebalance(&self, rebalance: &Rebalance<'_>) {} /// Post-rebalance callback. This method will run after the rebalance and /// should terminate its execution quickly. #[allow(unused_variables)] - fn post_rebalance<'a>(&self, rebalance: &Rebalance<'a>) {} + fn post_rebalance(&self, rebalance: &Rebalance<'_>) {} // TODO: convert pointer to structure /// Post commit callback. This method will run after a group of offsets was diff --git a/src/message.rs b/src/message.rs index 76bac9c39..7a422608e 100644 --- a/src/message.rs +++ b/src/message.rs @@ -425,20 +425,20 @@ impl<'a> Message for BorrowedMessage<'a> { type Headers = BorrowedHeaders; fn key(&self) -> Option<&[u8]> { - unsafe { util::ptr_to_opt_slice((*self.ptr).key, (*self.ptr).key_len) } + unsafe { util::ptr_to_opt_slice(self.ptr.key, self.ptr.key_len) } } fn payload(&self) -> Option<&[u8]> { - unsafe { util::ptr_to_opt_slice((*self.ptr).payload, (*self.ptr).len) } + unsafe { util::ptr_to_opt_slice(self.ptr.payload, self.ptr.len) } } unsafe fn payload_mut(&mut self) -> Option<&mut [u8]> { - util::ptr_to_opt_mut_slice((*self.ptr).payload, (*self.ptr).len) + util::ptr_to_opt_mut_slice(self.ptr.payload, self.ptr.len) } fn topic(&self) -> &str { unsafe { - CStr::from_ptr(rdsys::rd_kafka_topic_name((*self.ptr).rkt)) + CStr::from_ptr(rdsys::rd_kafka_topic_name(self.ptr.rkt)) .to_str() .expect("Topic name is not valid UTF-8") } diff --git a/src/producer/base_producer.rs b/src/producer/base_producer.rs index 1cc6e05ce..7623869f2 100644 --- a/src/producer/base_producer.rs +++ b/src/producer/base_producer.rs @@ -425,6 +425,7 @@ where /// Note that this method will never block. // Simplifying the return type requires generic associated types, which are // unstable. + #[allow(clippy::result_large_err)] pub fn send<'a, K, P>( &self, mut record: BaseRecord<'a, K, P, C::DeliveryOpaque>, @@ -701,6 +702,7 @@ where /// See the documentation for [`BaseProducer::send`] for details. // Simplifying the return type requires generic associated types, which are // unstable. + #[allow(clippy::result_large_err)] pub fn send<'a, K, P>( &self, record: BaseRecord<'a, K, P, C::DeliveryOpaque>, diff --git a/src/producer/future_producer.rs b/src/producer/future_producer.rs index 0769a16a8..baae2cc15 100644 --- a/src/producer/future_producer.rs +++ b/src/producer/future_producer.rs @@ -346,6 +346,7 @@ where /// Like [`FutureProducer::send`], but if enqueuing fails, an error will be /// returned immediately, alongside the [`FutureRecord`] provided. + #[allow(clippy::result_large_err)] pub fn send_result<'a, K, P>( &self, record: FutureRecord<'a, K, P>, diff --git a/src/statistics.rs b/src/statistics.rs index 8c3fc4c45..6496aa09b 100644 --- a/src/statistics.rs +++ b/src/statistics.rs @@ -367,7 +367,7 @@ mod tests { assert_eq!(stats.brokers.len(), 1); - let broker = stats.brokers.values().into_iter().collect::>()[0]; + let broker = stats.brokers.values().collect::>()[0]; assert_eq!( broker.req, @@ -391,7 +391,7 @@ mod tests { } // Example from https://github.com/edenhill/librdkafka/wiki/Statistics - const EXAMPLE: &'static str = r#" + const EXAMPLE: &str = r#" { "name": "rdkafka#producer-1", "client_id": "rdkafka", diff --git a/src/topic_partition_list.rs b/src/topic_partition_list.rs index 1d8e77ce9..16063bfbc 100644 --- a/src/topic_partition_list.rs +++ b/src/topic_partition_list.rs @@ -317,7 +317,7 @@ impl TopicPartitionList { /// Sets all partitions in the list to the specified offset. pub fn set_all_offsets(&mut self, offset: Offset) -> Result<(), KafkaError> { - let slice = unsafe { slice::from_raw_parts_mut((*self.ptr).elems, self.count()) }; + 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); elem.set_offset(offset)?; @@ -327,7 +327,7 @@ impl TopicPartitionList { /// Returns all the elements of the list. pub fn elements(&self) -> Vec> { - let slice = unsafe { slice::from_raw_parts_mut((*self.ptr).elems, self.count()) }; + 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,7 +337,7 @@ 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 slice = unsafe { slice::from_raw_parts_mut((*self.ptr).elems, self.count()) }; + 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); diff --git a/tests/test_admin.rs b/tests/test_admin.rs index dc7177708..2e6034cb2 100644 --- a/tests/test_admin.rs +++ b/tests/test_admin.rs @@ -34,7 +34,7 @@ async fn create_consumer_group(consumer_group_name: &str) { let admin_client = create_admin_client(); let topic_name = &rand_test_topic(consumer_group_name); let consumer: BaseConsumer = create_config() - .set("group.id", consumer_group_name.clone()) + .set("group.id", consumer_group_name) .create() .expect("create consumer failed"); @@ -74,17 +74,19 @@ fn fetch_metadata(topic: &str) -> Metadata { create_config().create().expect("consumer creation failed"); let timeout = Some(Duration::from_secs(1)); - let mut backoff = ExponentialBackoff::default(); - backoff.max_elapsed_time = Some(Duration::from_secs(5)); + let mut backoff = ExponentialBackoff { + max_elapsed_time: Some(Duration::from_secs(5)), + ..Default::default() + }; (|| { let metadata = consumer .fetch_metadata(Some(topic), timeout) .map_err(|e| e.to_string())?; - if metadata.topics().len() == 0 { + if metadata.topics().is_empty() { Err("metadata fetch returned no topics".to_string())? } let topic = &metadata.topics()[0]; - if topic.partitions().len() == 0 { + if topic.partitions().is_empty() { Err("metadata fetch returned a topic with no partitions".to_string())? } Ok(metadata) @@ -98,8 +100,10 @@ fn verify_delete(topic: &str) { create_config().create().expect("consumer creation failed"); let timeout = Some(Duration::from_secs(1)); - let mut backoff = ExponentialBackoff::default(); - backoff.max_elapsed_time = Some(Duration::from_secs(5)); + let mut backoff = ExponentialBackoff { + max_elapsed_time: Some(Duration::from_secs(5)), + ..Default::default() + }; (|| { // Asking about the topic specifically will recreate it (under the // default Kafka configuration, at least) so we have to ask for the list @@ -107,7 +111,7 @@ fn verify_delete(topic: &str) { let metadata = consumer .fetch_metadata(None, timeout) .map_err(|e| e.to_string())?; - if let Some(_) = metadata.topics().iter().find(|t| t.name() == topic) { + if metadata.topics().iter().any(|t| t.name() == topic) { Err(format!("topic {} still exists", topic))? } Ok(()) @@ -416,7 +420,7 @@ async fn test_configs() { } } - let config = AlterConfig::new(broker).set("log.flush.interval.ms", &orig_val); + let config = AlterConfig::new(broker).set("log.flush.interval.ms", orig_val); let res = admin_client .alter_configs(&[config], &opts) .await diff --git a/tests/test_high_producers.rs b/tests/test_high_producers.rs index bddc1beae..72b73919e 100644 --- a/tests/test_high_producers.rs +++ b/tests/test_high_producers.rs @@ -64,13 +64,13 @@ async fn test_future_producer_send_full() { // Fill up the queue. producer - .send_result(FutureRecord::to(&topic_name).payload("A").key("B")) + .send_result(FutureRecord::to(topic_name).payload("A").key("B")) .unwrap(); let send_message = |timeout| async move { let start = Instant::now(); let res = producer - .send(FutureRecord::to(&topic_name).payload("A").key("B"), timeout) + .send(FutureRecord::to(topic_name).payload("A").key("B"), timeout) .await; match res { Ok(_) => panic!("send unexpectedly succeeded"), diff --git a/tests/test_low_consumers.rs b/tests/test_low_consumers.rs index e6642b688..3b4cb19e8 100644 --- a/tests/test_low_consumers.rs +++ b/tests/test_low_consumers.rs @@ -385,11 +385,11 @@ async fn test_produce_consume_message_queue_nonempty_callback() { let timeout = Duration::from_secs(15); loop { let w = wakeups.load(Ordering::SeqCst); - if w == target { - break; - } else if w > target { - panic!("wakeups {} exceeds target {}", w, target); - } + match w.cmp(&target) { + std::cmp::Ordering::Equal => break, + std::cmp::Ordering::Greater => panic!("wakeups {} exceeds target {}", w, target), + std::cmp::Ordering::Less => (), + }; thread::sleep(Duration::from_millis(100)); if start.elapsed() > timeout { panic!("timeout exceeded while waiting for wakeup"); diff --git a/tests/test_low_producers.rs b/tests/test_low_producers.rs index 493642617..2a7c9f7a6 100644 --- a/tests/test_low_producers.rs +++ b/tests/test_low_producers.rs @@ -97,7 +97,7 @@ impl ProducerContext for CollectingContex fn get_custom_partitioner(&self) -> Option<&Part> { match &self.partitioner { None => None, - Some(p) => Some(&p), + Some(p) => Some(p), } } } @@ -144,7 +144,7 @@ impl Partitioner for PanicPartitioner { fn default_config(config_overrides: HashMap<&str, &str>) -> ClientConfig { let mut config = ClientConfig::new(); config - .set("bootstrap.servers", &get_bootstrap_server()) + .set("bootstrap.servers", get_bootstrap_server()) .set("message.timeout.ms", "5000"); for (key, value) in config_overrides { @@ -210,11 +210,13 @@ fn test_base_producer_queue_full() { let errors = results .iter() .filter(|&e| { - if let &Err((KafkaError::MessageProduction(RDKafkaErrorCode::QueueFull), _)) = e { - true - } else { - false - } + matches!( + e, + &Err(( + KafkaError::MessageProduction(RDKafkaErrorCode::QueueFull), + _ + )) + ) }) .count(); @@ -496,7 +498,7 @@ fn test_register_custom_partitioner_linger_non_zero_key_null() { assert_eq!(delivery_results.len(), 1); - for &(_, ref error, _) in &(*delivery_results) { + for (_, error, _) in &(*delivery_results) { assert_eq!(*error, None); } } @@ -523,7 +525,7 @@ fn test_custom_partitioner_base_producer() { let delivery_results = context.results.lock().unwrap(); - for &(ref message, ref error, _) in &(*delivery_results) { + for (message, error, _) in &(*delivery_results) { assert_eq!(error, &None); assert_eq!(message.partition(), 2); } @@ -551,7 +553,7 @@ fn test_custom_partitioner_threaded_producer() { let delivery_results = context.results.lock().unwrap(); - for &(ref message, ref error, _) in &(*delivery_results) { + for (message, error, _) in &(*delivery_results) { assert_eq!(error, &None); assert_eq!(message.partition(), 2); } diff --git a/tests/test_transactions.rs b/tests/test_transactions.rs index 1fe84a98b..91e8c835d 100644 --- a/tests/test_transactions.rs +++ b/tests/test_transactions.rs @@ -26,10 +26,10 @@ fn create_consumer( fn create_producer() -> Result { let mut config = ClientConfig::new(); config - .set("bootstrap.servers", &get_bootstrap_server()) + .set("bootstrap.servers", get_bootstrap_server()) .set("message.timeout.ms", "5000") .set("enable.idempotence", "true") - .set("transactional.id", &rand_test_transactional_id()) + .set("transactional.id", rand_test_transactional_id()) .set("debug", "eos"); config.set_log_level(RDKafkaLogLevel::Debug); config.create() diff --git a/tests/utils.rs b/tests/utils.rs index 447213672..3a8e0137e 100644 --- a/tests/utils.rs +++ b/tests/utils.rs @@ -67,9 +67,7 @@ pub fn get_broker_version() -> KafkaVersion { panic!("KAFKA_VERSION env var contained non-unicode characters") } // If the environment variable is unset, assume we're running the latest version. - Err(VarError::NotPresent) => { - KafkaVersion(std::u32::MAX, std::u32::MAX, std::u32::MAX, std::u32::MAX) - } + Err(VarError::NotPresent) => KafkaVersion(u32::MAX, u32::MAX, u32::MAX, u32::MAX), } } @@ -164,27 +162,6 @@ pub fn key_fn(id: i32) -> String { format!("Key {}", id) } -#[cfg(test)] -mod tests { - use super::*; - - #[tokio::test] - async fn test_populate_topic() { - let topic_name = rand_test_topic("test_populate_topic"); - let message_map = populate_topic(&topic_name, 100, &value_fn, &key_fn, Some(0), None).await; - - let total_messages = message_map - .iter() - .filter(|&(&(partition, _), _)| partition == 0) - .count(); - assert_eq!(total_messages, 100); - - let mut ids = message_map.iter().map(|(_, id)| *id).collect::>(); - ids.sort(); - assert_eq!(ids, (0..100).collect::>()); - } -} - pub struct ConsumerTestContext { pub _n: i64, // Add data for memory access validation } @@ -228,3 +205,24 @@ pub fn consumer_config( config } + +#[cfg(test)] +mod tests { + use super::*; + + #[tokio::test] + async fn test_populate_topic() { + let topic_name = rand_test_topic("test_populate_topic"); + let message_map = populate_topic(&topic_name, 100, &value_fn, &key_fn, Some(0), None).await; + + let total_messages = message_map + .iter() + .filter(|&(&(partition, _), _)| partition == 0) + .count(); + assert_eq!(total_messages, 100); + + let mut ids = message_map.values().copied().collect::>(); + ids.sort(); + assert_eq!(ids, (0..100).collect::>()); + } +} From c613b37da3522106665f17ce1a79530415158bae Mon Sep 17 00:00:00 2001 From: Federico Giraud Date: Sun, 4 Aug 2024 19:08:07 +0100 Subject: [PATCH 45/59] Fix clone clippy warning --- src/producer/future_producer.rs | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/producer/future_producer.rs b/src/producer/future_producer.rs index baae2cc15..a0ee1a54c 100644 --- a/src/producer/future_producer.rs +++ b/src/producer/future_producer.rs @@ -443,6 +443,7 @@ mod tests { #[test] fn test_future_producer_clone() { let producer = ClientConfig::new().create::().unwrap(); + #[allow(clippy::redundant_clone)] let _producer_clone = producer.clone(); } @@ -453,6 +454,7 @@ mod tests { let producer = ClientConfig::new() .create_with_context::<_, FutureProducer>(test_context) .unwrap(); + #[allow(clippy::redundant_clone)] let _producer_clone = producer.clone(); } } From 05ba0a8ded7b9eaf5e0ab00e53b40960c064fd09 Mon Sep 17 00:00:00 2001 From: Maverick Liu Date: Wed, 10 Jul 2024 14:53:09 +0800 Subject: [PATCH 46/59] Impl ToBytes for [u8; N] with generic const --- src/message.rs | 18 +++--------------- 1 file changed, 3 insertions(+), 15 deletions(-) diff --git a/src/message.rs b/src/message.rs index 7a422608e..238261495 100644 --- a/src/message.rs +++ b/src/message.rs @@ -822,24 +822,12 @@ impl ToBytes for () { } } -// Implement to_bytes for arrays - https://github.com/rust-lang/rfcs/issues/1038 -macro_rules! array_impls { - ($($N:expr)+) => { - $( - impl ToBytes for [u8; $N] { - fn to_bytes(&self) -> &[u8] { self } - } - )+ +impl ToBytes for [u8; N] { + fn to_bytes(&self) -> &[u8] { + self } } -array_impls! { - 0 1 2 3 4 5 6 7 8 9 - 10 11 12 13 14 15 16 17 18 19 - 20 21 22 23 24 25 26 27 28 29 - 30 31 32 -} - #[cfg(test)] mod test { use super::*; From aaf72de4a06330efeb692814b7caa362524d72f5 Mon Sep 17 00:00:00 2001 From: Joshua Griffith Date: Tue, 6 Aug 2024 10:33:48 -0500 Subject: [PATCH 47/59] fix: check for non-zero count before calling slice::from_raw_parts_mut (#686) Fixes https://github.com/fede1024/rust-rdkafka/issues/681 Co-authored-by: Federico Giraud --- src/topic_partition_list.rs | 13 +++++++++++-- 1 file changed, 11 insertions(+), 2 deletions(-) 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 { From 75737f33bb0c5c6af760bb7163ee3181e044c6fd Mon Sep 17 00:00:00 2001 From: Andrei Nesterov Date: Sun, 2 Jun 2024 20:49:28 +0300 Subject: [PATCH 48/59] Enable the development of custom consumers It is currently impossible to develop a custom consumer based on `BaseConsumer` because its `queue` property, which is necessary to receive notifications about new incoming messages, is private. This defines `set_nonempty_callback` method on `BaseConsumer` similarly to how it has already been done for `PartitionQueue`. That will allow setting `rdkafka_sys::rd_kafka_queue_cb_event_enable` callback from within a custom consumer implementation. --- src/consumer/base_consumer.rs | 34 +++++++++++++ tests/test_low_consumers.rs | 95 +++++++++++++++++++++++++++++++++++ 2 files changed, 129 insertions(+) diff --git a/src/consumer/base_consumer.rs b/src/consumer/base_consumer.rs index 58d81078d..5f0c77953 100644 --- a/src/consumer/base_consumer.rs +++ b/src/consumer/base_consumer.rs @@ -38,6 +38,7 @@ where client: Client, queue: NativeQueue, group_id: Option, + nonempty_callback: Option>>, } impl FromClientConfig for BaseConsumer { @@ -98,6 +99,7 @@ where client, queue, group_id, + nonempty_callback: None, }) } @@ -357,6 +359,36 @@ where pub fn closed(&self) -> bool { unsafe { rdsys::rd_kafka_consumer_closed(self.client.native_ptr()) == 1 } } + + /// Sets a callback that will be invoked whenever the queue becomes + /// nonempty. + pub fn set_nonempty_callback(&mut self, f: F) + where + F: Fn() + Send + Sync + 'static, + { + // SAFETY: we keep `F` alive until the next call to + // `rd_kafka_queue_cb_event_enable`. That might be the next call to + // `set_nonempty_callback` or it might be when the queue is dropped. The + // double indirection is required because `&dyn Fn` is a fat pointer. + + unsafe extern "C" fn native_message_queue_nonempty_cb( + _: *mut RDKafka, + opaque_ptr: *mut c_void, + ) { + let f = opaque_ptr as *const *const (dyn Fn() + Send + Sync); + (**f)(); + } + + let f: Box> = Box::new(Box::new(f)); + unsafe { + rdsys::rd_kafka_queue_cb_event_enable( + self.queue.ptr(), + Some(native_message_queue_nonempty_cb), + &*f as *const _ as *mut c_void, + ) + } + self.nonempty_callback = Some(f); + } } impl Consumer for BaseConsumer @@ -719,6 +751,8 @@ where C: ConsumerContext, { fn drop(&mut self) { + unsafe { rdsys::rd_kafka_queue_cb_event_enable(self.queue.ptr(), None, ptr::null_mut()) } + trace!("Destroying consumer: {:?}", self.client.native_ptr()); if self.group_id.is_some() { if let Err(err) = self.close_queue() { diff --git a/tests/test_low_consumers.rs b/tests/test_low_consumers.rs index 3b4cb19e8..aaecffe96 100644 --- a/tests/test_low_consumers.rs +++ b/tests/test_low_consumers.rs @@ -447,6 +447,101 @@ async fn test_produce_consume_message_queue_nonempty_callback() { assert_eq!(wakeups.load(Ordering::SeqCst), 2); } +#[tokio::test] +async fn test_produce_consume_consumer_nonempty_callback() { + let _r = env_logger::try_init(); + + let topic_name = rand_test_topic("test_produce_consume_consumer_nonempty_callback"); + + create_topic(&topic_name, 1).await; + + // Turn off statistics to prevent interference with the wakeups counter. + let mut config_overrides = HashMap::new(); + config_overrides.insert("statistics.interval.ms", "0"); + + let mut consumer: BaseConsumer<_> = consumer_config(&rand_test_group(), Some(config_overrides)) + .create_with_context(ConsumerTestContext { _n: 64 }) + .expect("Consumer creation failed"); + + let mut tpl = TopicPartitionList::new(); + tpl.add_partition_offset(&topic_name, 0, Offset::Beginning) + .unwrap(); + consumer.assign(&tpl).unwrap(); + + let wakeups = Arc::new(AtomicUsize::new(0)); + consumer.set_nonempty_callback({ + let wakeups = wakeups.clone(); + move || { + wakeups.fetch_add(1, Ordering::SeqCst); + } + }); + + let wait_for_wakeups = |target| { + let start = Instant::now(); + let timeout = Duration::from_secs(15); + loop { + let w = wakeups.load(Ordering::SeqCst); + match w.cmp(&target) { + std::cmp::Ordering::Equal => break, + std::cmp::Ordering::Greater => panic!("wakeups {} exceeds target {}", w, target), + std::cmp::Ordering::Less => (), + }; + thread::sleep(Duration::from_millis(100)); + if start.elapsed() > timeout { + panic!("timeout exceeded while waiting for wakeup"); + } + } + }; + + // Initiate connection. + assert!(consumer.poll(Duration::from_secs(0)).is_none()); + + // Expect no wakeups for 1s. + thread::sleep(Duration::from_secs(1)); + assert_eq!(wakeups.load(Ordering::SeqCst), 0); + + // Verify there are no messages waiting. + assert!(consumer.poll(Duration::from_secs(0)).is_none()); + + // Populate the topic, and expect a wakeup notifying us of the new messages. + populate_topic(&topic_name, 2, &value_fn, &key_fn, None, None).await; + wait_for_wakeups(1); + + // Read one of the messages. + assert!(consumer.poll(Duration::from_secs(0)).is_some()); + + // Add more messages to the topic. Expect no additional wakeups, as the + // queue is not fully drained, for 1s. + populate_topic(&topic_name, 2, &value_fn, &key_fn, None, None).await; + thread::sleep(Duration::from_secs(1)); + assert_eq!(wakeups.load(Ordering::SeqCst), 1); + + // Drain the queue. + assert!(consumer.poll(None).is_some()); + assert!(consumer.poll(None).is_some()); + assert!(consumer.poll(None).is_some()); + + // Expect no additional wakeups for 1s. + thread::sleep(Duration::from_secs(1)); + assert_eq!(wakeups.load(Ordering::SeqCst), 1); + + // Add another message, and expect a wakeup. + populate_topic(&topic_name, 1, &value_fn, &key_fn, None, None).await; + wait_for_wakeups(2); + + // Expect no additional wakeups for 1s. + thread::sleep(Duration::from_secs(1)); + assert_eq!(wakeups.load(Ordering::SeqCst), 2); + + // Disable the queue and add another message. + consumer.set_nonempty_callback(|| ()); + populate_topic(&topic_name, 1, &value_fn, &key_fn, None, None).await; + + // Expect no additional wakeups for 1s. + thread::sleep(Duration::from_secs(1)); + assert_eq!(wakeups.load(Ordering::SeqCst), 2); +} + #[tokio::test] async fn test_invalid_consumer_position() { // Regression test for #360, in which calling `position` on a consumer which From 2b058641aae489da69d102ca8dd029b53ea029f4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Micha=C5=82=20=C5=81owicki?= Date: Wed, 21 Aug 2024 16:33:59 +0200 Subject: [PATCH 49/59] Avoid realloc when creating C-style strings --- src/message.rs | 2 +- src/producer/base_producer.rs | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/message.rs b/src/message.rs index 238261495..2048a1ced 100644 --- a/src/message.rs +++ b/src/message.rs @@ -534,7 +534,7 @@ impl OwnedHeaders { where V: ToBytes + ?Sized, { - let name_cstring = CString::new(header.key.to_owned()).unwrap(); + let name_cstring = CString::new(header.key).unwrap(); let (value_ptr, value_len) = match header.value { None => (ptr::null_mut(), 0), Some(value) => { diff --git a/src/producer/base_producer.rs b/src/producer/base_producer.rs index 7623869f2..f861b2d9e 100644 --- a/src/producer/base_producer.rs +++ b/src/producer/base_producer.rs @@ -442,7 +442,7 @@ where } let (payload_ptr, payload_len) = as_bytes(record.payload); let (key_ptr, key_len) = as_bytes(record.key); - let topic_cstring = CString::new(record.topic.to_owned()).unwrap(); + let topic_cstring = CString::new(record.topic).unwrap(); let opaque_ptr = record.delivery_opaque.into_ptr(); let produce_error = unsafe { rdsys::rd_kafka_producev( From 3209fcc869528b7d567eeabf2de95a3426d21f7e Mon Sep 17 00:00:00 2001 From: Nikhil Benesch Date: Fri, 30 Aug 2024 21:43:11 -0400 Subject: [PATCH 50/59] Add support for deleting records to admin client Fix #385. --- rdkafka-sys/src/types.rs | 3 ++ src/admin.rs | 85 ++++++++++++++++++++++++++++++ tests/test_admin.rs | 111 ++++++++++++++++++++++++++++++++++++++- 3 files changed, 198 insertions(+), 1 deletion(-) diff --git a/rdkafka-sys/src/types.rs b/rdkafka-sys/src/types.rs index 97b77b312..0005073ba 100644 --- a/rdkafka-sys/src/types.rs +++ b/rdkafka-sys/src/types.rs @@ -78,6 +78,9 @@ pub type RDKafkaDeleteGroup = bindings::rd_kafka_DeleteGroup_t; /// Native rdkafka new partitions object. pub type RDKafkaNewPartitions = bindings::rd_kafka_NewPartitions_t; +/// Native rdkafka delete records object. +pub type RDKafkaDeleteRecords = bindings::rd_kafka_DeleteRecords_t; + /// Native rdkafka config resource. pub type RDKafkaConfigResource = bindings::rd_kafka_ConfigResource_t; diff --git a/src/admin.rs b/src/admin.rs index 69dba537b..0418f0cac 100644 --- a/src/admin.rs +++ b/src/admin.rs @@ -26,6 +26,7 @@ use crate::config::{ClientConfig, FromClientConfig, FromClientConfigAndContext}; use crate::error::{IsError, KafkaError, KafkaResult}; use crate::log::{trace, warn}; use crate::util::{cstr_to_owned, AsCArray, ErrBuf, IntoOpaque, KafkaDrop, NativePtr, Timeout}; +use crate::TopicPartitionList; // // ********** ADMIN CLIENT ********** @@ -218,6 +219,53 @@ impl AdminClient { Ok(rx) } + /// Deletes records from a topic. + /// + /// The provided `offsets` is a topic partition list specifying which + /// records to delete from a list of topic partitions. For each entry in the + /// list, the messages at offsets before the specified offsets (exclusive) + /// in the specified partition will be deleted. Use offset [`Offset::End`] + /// to delete all records in the partition. + /// + /// Returns a topic partition list describing the result of the deletion. If + /// the operation succeeded for a partition, the offset for that partition + /// will be set to the post-deletion low-water mark for that partition. If + /// the operation failed for a partition, there will be an error for that + /// partition's entry in the list. + pub fn delete_records( + &self, + offsets: &TopicPartitionList, + opts: &AdminOptions, + ) -> impl Future> { + match self.delete_records_inner(offsets, opts) { + Ok(rx) => Either::Left(DeleteRecordsFuture { rx }), + Err(err) => Either::Right(future::err(err)), + } + } + + fn delete_records_inner( + &self, + offsets: &TopicPartitionList, + opts: &AdminOptions, + ) -> KafkaResult> { + let mut err_buf = ErrBuf::new(); + let delete_records = unsafe { + NativeDeleteRecords::from_ptr(rdsys::rd_kafka_DeleteRecords_new(offsets.ptr())) + } + .ok_or_else(|| KafkaError::AdminOpCreation(err_buf.to_string()))?; + let (native_opts, rx) = opts.to_native(self.client.native_ptr(), &mut err_buf)?; + unsafe { + rdsys::rd_kafka_DeleteRecords( + self.client.native_ptr(), + &mut delete_records.ptr(), + 1, + native_opts.ptr(), + self.queue.ptr(), + ); + } + Ok(rx) + } + /// Retrieves the configuration parameters for the specified resources. /// /// Note that while the API supports describing multiple configurations at @@ -950,6 +998,43 @@ impl Future for CreatePartitionsFuture { } } +// +// Delete records handling +// + +type NativeDeleteRecords = NativePtr; + +unsafe impl KafkaDrop for RDKafkaDeleteRecords { + const TYPE: &'static str = "delete records"; + const DROP: unsafe extern "C" fn(*mut Self) = rdsys::rd_kafka_DeleteRecords_destroy; +} + +struct DeleteRecordsFuture { + rx: oneshot::Receiver, +} + +impl Future for DeleteRecordsFuture { + type Output = KafkaResult; + + fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { + let event = ready!(self.rx.poll_unpin(cx)).map_err(|_| KafkaError::Canceled)?; + event.check_error()?; + let res = unsafe { rdsys::rd_kafka_event_DeleteRecords_result(event.ptr()) }; + if res.is_null() { + let typ = unsafe { rdsys::rd_kafka_event_type(event.ptr()) }; + return Poll::Ready(Err(KafkaError::AdminOpCreation(format!( + "delete records request received response of incorrect type ({})", + typ + )))); + } + let tpl = unsafe { + let tpl = rdsys::rd_kafka_DeleteRecords_result_offsets(res); + TopicPartitionList::from_ptr(rdsys::rd_kafka_topic_partition_list_copy(tpl)) + }; + Poll::Ready(Ok(tpl)) + } +} + // // Describe configs handling // diff --git a/tests/test_admin.rs b/tests/test_admin.rs index 2e6034cb2..4cf0e9a81 100644 --- a/tests/test_admin.rs +++ b/tests/test_admin.rs @@ -12,7 +12,8 @@ use rdkafka::client::DefaultClientContext; use rdkafka::consumer::{BaseConsumer, CommitMode, Consumer, DefaultConsumerContext}; use rdkafka::error::{KafkaError, RDKafkaErrorCode}; use rdkafka::metadata::Metadata; -use rdkafka::{ClientConfig, TopicPartitionList}; +use rdkafka::producer::{FutureProducer, FutureRecord, Producer}; +use rdkafka::{ClientConfig, Offset, TopicPartitionList}; use crate::utils::*; @@ -356,6 +357,114 @@ async fn test_topics() { } } +/// Test the admin client's delete records functionality. +#[tokio::test] +async fn test_delete_records() { + let producer = create_config().create::>().unwrap(); + let admin_client = create_admin_client(); + let timeout = Some(Duration::from_secs(1)); + let opts = AdminOptions::new().operation_timeout(timeout); + let topic = rand_test_topic("test_delete_records"); + let make_record = || FutureRecord::::to(&topic).payload("data"); + + // Create a topic with a single partition. + admin_client + .create_topics( + &[NewTopic::new(&topic, 1, TopicReplication::Fixed(1))], + &opts, + ) + .await + .expect("topic creation failed"); + + // Ensure that the topic begins with low and high water marks of 0. + let (lo, hi) = producer + .client() + .fetch_watermarks(&topic, 0, timeout) + .unwrap(); + assert_eq!(lo, 0); + assert_eq!(hi, 0); + + // Produce five messages to the topic. + for _ in 0..5 { + producer.send(make_record(), timeout).await.unwrap(); + } + + // Ensure that the high water mark has advanced to 5. + let (lo, hi) = producer + .client() + .fetch_watermarks(&topic, 0, timeout) + .unwrap(); + assert_eq!(lo, 0); + assert_eq!(hi, 5); + + // Delete the record at offset 0. + let mut tpl = TopicPartitionList::new(); + tpl.add_partition_offset(&topic, 0, Offset::Offset(1)) + .unwrap(); + let res_tpl = admin_client.delete_records(&tpl, &opts).await.unwrap(); + assert_eq!(res_tpl.count(), 1); + assert_eq!(res_tpl.elements()[0].topic(), topic); + assert_eq!(res_tpl.elements()[0].partition(), 0); + assert_eq!(res_tpl.elements()[0].offset(), Offset::Offset(1)); + assert_eq!(res_tpl.elements()[0].error(), Ok(())); + + // Ensure that the low water mark has advanced to 1. + let (lo, hi) = producer + .client() + .fetch_watermarks(&topic, 0, timeout) + .unwrap(); + assert_eq!(lo, 1); + assert_eq!(hi, 5); + + // Delete the record at offset 1 and also include an invalid partition in + // the request. The invalid partition should not cause the request to fail, + // but we should be able to see the per-partition error in the returned + // topic partition list. + let mut tpl = TopicPartitionList::new(); + tpl.add_partition_offset(&topic, 0, Offset::Offset(2)) + .unwrap(); + tpl.add_partition_offset(&topic, 1, Offset::Offset(1)) + .unwrap(); + let res_tpl = admin_client.delete_records(&tpl, &opts).await.unwrap(); + assert_eq!(res_tpl.count(), 2); + assert_eq!(res_tpl.elements()[0].topic(), topic); + assert_eq!(res_tpl.elements()[0].partition(), 0); + assert_eq!(res_tpl.elements()[0].offset(), Offset::Offset(2)); + assert_eq!(res_tpl.elements()[0].error(), Ok(())); + assert_eq!(res_tpl.elements()[1].topic(), topic); + assert_eq!(res_tpl.elements()[1].partition(), 1); + assert_eq!( + res_tpl.elements()[1].error(), + Err(KafkaError::OffsetFetch(RDKafkaErrorCode::UnknownPartition)) + ); + + // Ensure that the low water mark has advanced to 2. + let (lo, hi) = producer + .client() + .fetch_watermarks(&topic, 0, timeout) + .unwrap(); + assert_eq!(lo, 2); + assert_eq!(hi, 5); + + // Delete all records up to offset 5. + let mut tpl = TopicPartitionList::new(); + tpl.add_partition_offset(&topic, 0, Offset::End).unwrap(); + let res_tpl = admin_client.delete_records(&tpl, &opts).await.unwrap(); + assert_eq!(res_tpl.count(), 1); + assert_eq!(res_tpl.elements()[0].topic(), topic); + assert_eq!(res_tpl.elements()[0].partition(), 0); + assert_eq!(res_tpl.elements()[0].offset(), Offset::Offset(5)); + assert_eq!(res_tpl.elements()[0].error(), Ok(())); + + // Ensure that the low water mark has advanced to 5. + let (lo, hi) = producer + .client() + .fetch_watermarks(&topic, 0, timeout) + .unwrap(); + assert_eq!(lo, 5); + assert_eq!(hi, 5); +} + #[tokio::test] async fn test_configs() { let admin_client = create_admin_client(); From cd9157486fbbccc3fa2840dc9af0fb7b3501f927 Mon Sep 17 00:00:00 2001 From: Nikhil Benesch Date: Fri, 30 Aug 2024 21:47:20 -0400 Subject: [PATCH 51/59] Remove obsolete utilities The `OnDrop` struct is no longer used, so just remove it. It's easy to add back if we need it in the future. --- src/util.rs | 14 -------------- 1 file changed, 14 deletions(-) diff --git a/src/util.rs b/src/util.rs index dd4862825..fef12b87f 100644 --- a/src/util.rs +++ b/src/util.rs @@ -318,20 +318,6 @@ where } } -#[allow(dead_code)] -pub(crate) struct OnDrop(pub F) -where - F: Fn(); - -impl Drop for OnDrop -where - F: Fn(), -{ - fn drop(&mut self) { - (self.0)() - } -} - /// An abstraction over asynchronous runtimes. /// /// There are several asynchronous runtimes available for Rust. By default From 7e83013ab55297ee65f5aabab19b00e1e87eda71 Mon Sep 17 00:00:00 2001 From: Abdullah Sabaa Allil <36844223+Abdullahsab3@users.noreply.github.com> Date: Sat, 14 Sep 2024 12:01:04 +0200 Subject: [PATCH 52/59] Update README.md Update the MSRV of Rust --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 6070ef6a7..333fb5140 100644 --- a/README.md +++ b/README.md @@ -184,7 +184,7 @@ re-exported as rdkafka features. ### Minimum supported Rust version (MSRV) -The current minimum supported Rust version (MSRV) is 1.61.0. Note that +The current minimum supported Rust version (MSRV) is 1.70.0. Note that bumping the MSRV is not considered a breaking change. Any release of rust-rdkafka may bump the MSRV. From c01975feca777f773e72c3da59b51272cb280d42 Mon Sep 17 00:00:00 2001 From: Aleksandr Trukhin Date: Sat, 30 Mar 2024 09:18:49 +0400 Subject: [PATCH 53/59] Fix StreamConsumer wakeup races --- src/client.rs | 38 ++++++++++++--- src/consumer/base_consumer.rs | 85 +++++++++++++++++++-------------- src/consumer/stream_consumer.rs | 56 ++++++++++++++-------- src/producer/base_producer.rs | 4 +- 4 files changed, 117 insertions(+), 66 deletions(-) diff --git a/src/client.rs b/src/client.rs index 1b9f6bd1c..6ab430fb6 100644 --- a/src/client.rs +++ b/src/client.rs @@ -198,6 +198,21 @@ impl NativeClient { } } +pub(crate) enum EventPollResult { + None, + EventConsumed, + Event(T), +} + +impl Into> for EventPollResult { + fn into(self) -> Option { + match self { + EventPollResult::None | EventPollResult::EventConsumed => None, + EventPollResult::Event(evt) => Some(evt), + } + } +} + /// A low-level rdkafka client. /// /// This type is the basis of the consumers and producers in the [`consumer`] @@ -278,31 +293,42 @@ impl Client { &self.context } - pub(crate) fn poll_event(&self, queue: &NativeQueue, timeout: Timeout) -> Option { + pub(crate) fn poll_event( + &self, + queue: &NativeQueue, + timeout: Timeout, + ) -> EventPollResult { let event = unsafe { NativeEvent::from_ptr(queue.poll(timeout)) }; if let Some(ev) = event { let evtype = unsafe { rdsys::rd_kafka_event_type(ev.ptr()) }; match evtype { - rdsys::RD_KAFKA_EVENT_LOG => self.handle_log_event(ev.ptr()), - rdsys::RD_KAFKA_EVENT_STATS => self.handle_stats_event(ev.ptr()), + rdsys::RD_KAFKA_EVENT_LOG => { + self.handle_log_event(ev.ptr()); + return EventPollResult::EventConsumed; + } + rdsys::RD_KAFKA_EVENT_STATS => { + self.handle_stats_event(ev.ptr()); + return EventPollResult::EventConsumed; + } rdsys::RD_KAFKA_EVENT_ERROR => { // rdkafka reports consumer errors via RD_KAFKA_EVENT_ERROR but producer errors gets // embedded on the ack returned via RD_KAFKA_EVENT_DR. Hence we need to return this event // for the consumer case in order to return the error to the user. self.handle_error_event(ev.ptr()); - return Some(ev); + return EventPollResult::Event(ev); } rdsys::RD_KAFKA_EVENT_OAUTHBEARER_TOKEN_REFRESH => { if C::ENABLE_REFRESH_OAUTH_TOKEN { self.handle_oauth_refresh_event(ev.ptr()); } + return EventPollResult::EventConsumed; } _ => { - return Some(ev); + return EventPollResult::Event(ev); } } } - None + EventPollResult::None } fn handle_log_event(&self, event: *mut RDKafkaEvent) { diff --git a/src/consumer/base_consumer.rs b/src/consumer/base_consumer.rs index 5f0c77953..5a32150c3 100644 --- a/src/consumer/base_consumer.rs +++ b/src/consumer/base_consumer.rs @@ -11,7 +11,7 @@ use log::{error, warn}; use rdkafka_sys as rdsys; use rdkafka_sys::types::*; -use crate::client::{Client, NativeQueue}; +use crate::client::{Client, EventPollResult, NativeQueue}; use crate::config::{ ClientConfig, FromClientConfig, FromClientConfigAndContext, NativeClientConfig, }; @@ -117,59 +117,70 @@ where /// /// The returned message lives in the memory of the consumer and cannot outlive it. pub fn poll>(&self, timeout: T) -> Option>> { - self.poll_queue(self.get_queue(), timeout) + self.poll_queue(self.get_queue(), timeout).into() } pub(crate) fn poll_queue>( &self, queue: &NativeQueue, timeout: T, - ) -> Option>> { + ) -> EventPollResult>> { let now = Instant::now(); - let mut timeout = timeout.into(); + let initial_timeout = timeout.into(); + let mut timeout = initial_timeout; let min_poll_interval = self.context().main_queue_min_poll_interval(); loop { let op_timeout = std::cmp::min(timeout, min_poll_interval); let maybe_event = self.client().poll_event(queue, op_timeout); - if let Some(event) = maybe_event { - let evtype = unsafe { rdsys::rd_kafka_event_type(event.ptr()) }; - match evtype { - rdsys::RD_KAFKA_EVENT_FETCH => { - if let Some(result) = self.handle_fetch_event(event) { - return Some(result); + match maybe_event { + EventPollResult::Event(event) => { + let evtype = unsafe { rdsys::rd_kafka_event_type(event.ptr()) }; + match evtype { + rdsys::RD_KAFKA_EVENT_FETCH => { + if let Some(result) = self.handle_fetch_event(event) { + return EventPollResult::Event(result); + } } - } - rdsys::RD_KAFKA_EVENT_ERROR => { - if let Some(err) = self.handle_error_event(event) { - return Some(Err(err)); + rdsys::RD_KAFKA_EVENT_ERROR => { + if let Some(err) = self.handle_error_event(event) { + return EventPollResult::Event(Err(err)); + } } - } - rdsys::RD_KAFKA_EVENT_REBALANCE => { - self.handle_rebalance_event(event); - if timeout != Timeout::Never { - return None; + rdsys::RD_KAFKA_EVENT_REBALANCE => { + self.handle_rebalance_event(event); + if timeout != Timeout::Never { + return EventPollResult::EventConsumed; + } } - } - rdsys::RD_KAFKA_EVENT_OFFSET_COMMIT => { - self.handle_offset_commit_event(event); - if timeout != Timeout::Never { - return None; + rdsys::RD_KAFKA_EVENT_OFFSET_COMMIT => { + self.handle_offset_commit_event(event); + if timeout != Timeout::Never { + return EventPollResult::EventConsumed; + } + } + _ => { + let buf = unsafe { + let evname = rdsys::rd_kafka_event_name(event.ptr()); + CStr::from_ptr(evname).to_bytes() + }; + let evname = String::from_utf8(buf.to_vec()).unwrap(); + warn!("Ignored event '{}' on consumer poll", evname); } } - _ => { - let evname = unsafe { - let evname = rdsys::rd_kafka_event_name(event.ptr()); - CStr::from_ptr(evname).to_string_lossy() - }; - warn!("Ignored event '{evname}' on consumer poll"); + } + EventPollResult::None => { + timeout = initial_timeout.saturating_sub(now.elapsed()); + if timeout.is_zero() { + return EventPollResult::None; } } - } - - timeout = timeout.saturating_sub(now.elapsed()); - if timeout.is_zero() { - return None; - } + EventPollResult::EventConsumed => { + timeout = initial_timeout.saturating_sub(now.elapsed()); + if timeout.is_zero() { + return EventPollResult::EventConsumed; + } + } + }; } } @@ -833,7 +844,7 @@ where /// associated consumer regularly, even if no messages are expected, to /// serve events. pub fn poll>(&self, timeout: T) -> Option>> { - self.consumer.poll_queue(&self.queue, timeout) + self.consumer.poll_queue(&self.queue, timeout).into() } /// Sets a callback that will be invoked whenever the queue becomes diff --git a/src/consumer/stream_consumer.rs b/src/consumer/stream_consumer.rs index 5a7f60552..12aa8ee99 100644 --- a/src/consumer/stream_consumer.rs +++ b/src/consumer/stream_consumer.rs @@ -18,7 +18,7 @@ use slab::Slab; use rdkafka_sys as rdsys; use rdkafka_sys::types::*; -use crate::client::{Client, NativeQueue}; +use crate::client::{Client, EventPollResult, NativeQueue}; use crate::config::{ClientConfig, FromClientConfig, FromClientConfigAndContext}; use crate::consumer::base_consumer::{BaseConsumer, PartitionQueue}; use crate::consumer::{ @@ -122,11 +122,12 @@ impl<'a, C: ConsumerContext> MessageStream<'a, C> { } } - fn poll(&self) -> Option>> { + fn poll(&self) -> EventPollResult>> { if let Some(queue) = self.partition_queue { self.consumer.poll_queue(queue, Duration::ZERO) } else { - self.consumer.poll(Duration::ZERO) + self.consumer + .poll_queue(self.consumer.get_queue(), Duration::ZERO) } } } @@ -135,25 +136,38 @@ impl<'a, C: ConsumerContext> Stream for MessageStream<'a, C> { type Item = KafkaResult>; fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - // If there is a message ready, yield it immediately to avoid the - // taking the lock in `self.set_waker`. - if let Some(message) = self.poll() { - return Poll::Ready(Some(message)); - } - - // Otherwise, we need to wait for a message to become available. Store - // the waker so that we are woken up if the queue flips from non-empty - // to empty. We have to store the waker repatedly in case this future - // migrates between tasks. - self.wakers.set_waker(self.slot, cx.waker().clone()); - - // Check whether a new message became available after we installed the - // waker. This avoids a race where `poll` returns None to indicate that - // the queue is empty, but the queue becomes non-empty before we've - // installed the waker. match self.poll() { - None => Poll::Pending, - Some(message) => Poll::Ready(Some(message)), + EventPollResult::Event(message) => { + // If there is a message ready, yield it immediately to avoid the + // taking the lock in `self.set_waker`. + Poll::Ready(Some(message)) + } + EventPollResult::EventConsumed => { + // Event was consumed, yield to runtime + cx.waker().wake_by_ref(); + Poll::Pending + } + EventPollResult::None => { + // Otherwise, we need to wait for a message to become available. Store + // the waker so that we are woken up if the queue flips from non-empty + // to empty. We have to store the waker repatedly in case this future + // migrates between tasks. + self.wakers.set_waker(self.slot, cx.waker().clone()); + + // Check whether a new message became available after we installed the + // waker. This avoids a race where `poll` returns None to indicate that + // the queue is empty, but the queue becomes non-empty before we've + // installed the waker. + match self.poll() { + EventPollResult::Event(message) => Poll::Ready(Some(message)), + EventPollResult::EventConsumed => { + // Event was consumed, yield to runtime + cx.waker().wake_by_ref(); + Poll::Pending + } + EventPollResult::None => Poll::Pending, + } + } } } } diff --git a/src/producer/base_producer.rs b/src/producer/base_producer.rs index f861b2d9e..2905fc668 100644 --- a/src/producer/base_producer.rs +++ b/src/producer/base_producer.rs @@ -57,7 +57,7 @@ use rdkafka_sys as rdsys; use rdkafka_sys::rd_kafka_vtype_t::*; use rdkafka_sys::types::*; -use crate::client::{Client, NativeQueue}; +use crate::client::{Client, EventPollResult, NativeQueue}; use crate::config::{ClientConfig, FromClientConfig, FromClientConfigAndContext}; use crate::consumer::ConsumerGroupMetadata; use crate::error::{IsError, KafkaError, KafkaResult, RDKafkaError}; @@ -363,7 +363,7 @@ where /// the message delivery callbacks. pub fn poll>(&self, timeout: T) { let event = self.client().poll_event(&self.queue, timeout.into()); - if let Some(ev) = event { + if let EventPollResult::Event(ev) = event { let evtype = unsafe { rdsys::rd_kafka_event_type(ev.ptr()) }; match evtype { rdsys::RD_KAFKA_EVENT_DR => self.handle_delivery_report_event(ev), From e8f786d502a13eb1dac89b932ab1a7ecb009aa73 Mon Sep 17 00:00:00 2001 From: Aleksandr Trukhin Date: Thu, 29 Aug 2024 20:44:37 +0400 Subject: [PATCH 54/59] to_string_lossy --- src/consumer/base_consumer.rs | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/src/consumer/base_consumer.rs b/src/consumer/base_consumer.rs index 5a32150c3..ba9a878ce 100644 --- a/src/consumer/base_consumer.rs +++ b/src/consumer/base_consumer.rs @@ -159,12 +159,11 @@ where } } _ => { - let buf = unsafe { + let evname = unsafe { let evname = rdsys::rd_kafka_event_name(event.ptr()); - CStr::from_ptr(evname).to_bytes() + CStr::from_ptr(evname).to_string_lossy() }; - let evname = String::from_utf8(buf.to_vec()).unwrap(); - warn!("Ignored event '{}' on consumer poll", evname); + warn!("Ignored event '{evname}' on consumer poll"); } } } From c709cee7af07d2ad18e8d57f833ffeb29a0b5975 Mon Sep 17 00:00:00 2001 From: Aleksandr Trukhin Date: Thu, 29 Aug 2024 20:45:22 +0400 Subject: [PATCH 55/59] use from --- src/client.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/client.rs b/src/client.rs index 6ab430fb6..08aa5b9dd 100644 --- a/src/client.rs +++ b/src/client.rs @@ -204,9 +204,9 @@ pub(crate) enum EventPollResult { Event(T), } -impl Into> for EventPollResult { - fn into(self) -> Option { - match self { +impl From> for Option { + fn from(val: EventPollResult) -> Self { + match val { EventPollResult::None | EventPollResult::EventConsumed => None, EventPollResult::Event(evt) => Some(evt), } From bc2cf725bd4abcec6c5fc899b4198cb226e750cf Mon Sep 17 00:00:00 2001 From: David Blewett Date: Thu, 1 Aug 2024 10:00:29 -0400 Subject: [PATCH 56/59] Point submodule to our fork's datadog/main branch. --- .gitmodules | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/.gitmodules b/.gitmodules index 7d2f654d2..cad9858fa 100644 --- a/.gitmodules +++ b/.gitmodules @@ -1,3 +1,4 @@ [submodule "rdkafka-sys/librdkafka"] - path = rdkafka-sys/librdkafka - url = https://github.com/edenhill/librdkafka + path = rdkafka-sys/librdkafka + url = https://github.com/DataDog/librdkafka + branch = datadog/main From f86a51b5b99c5dc6b0dfade5516e8e8cd865cfeb Mon Sep 17 00:00:00 2001 From: David Blewett Date: Thu, 1 Aug 2024 10:07:05 -0400 Subject: [PATCH 57/59] Update librdkafka to 2.4.0+patches. --- Cargo.lock | 2 +- Cargo.toml | 2 +- rdkafka-sys/Cargo.toml | 2 +- rdkafka-sys/librdkafka | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 708bf88b1..f67048df4 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1099,7 +1099,7 @@ dependencies = [ [[package]] name = "rdkafka-sys" -version = "4.7.0+2.3.0" +version = "4.8.0+2.4.0" dependencies = [ "cmake", "curl-sys", diff --git a/Cargo.toml b/Cargo.toml index b9045ce58..0efd05f57 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -16,7 +16,7 @@ rust-version = "1.70" members = ["rdkafka-sys"] [dependencies] -rdkafka-sys = { path = "rdkafka-sys", version = "4.7.0", default-features = false } +rdkafka-sys = { path = "rdkafka-sys", version = "4.8.0", default-features = false } futures-channel = "0.3.0" futures-executor = { version = "0.3.0", optional = true } futures-util = { version = "0.3.0", default-features = false } diff --git a/rdkafka-sys/Cargo.toml b/rdkafka-sys/Cargo.toml index 7ea379ef8..9b03a10cd 100644 --- a/rdkafka-sys/Cargo.toml +++ b/rdkafka-sys/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "rdkafka-sys" -version = "4.7.0+2.3.0" +version = "4.8.0+2.4.0" authors = ["Federico Giraud "] build = "build.rs" links = "rdkafka" diff --git a/rdkafka-sys/librdkafka b/rdkafka-sys/librdkafka index 95a542c87..1bbb0464a 160000 --- a/rdkafka-sys/librdkafka +++ b/rdkafka-sys/librdkafka @@ -1 +1 @@ -Subproject commit 95a542c87c61d2c45b445f91c73dd5442eb04f3c +Subproject commit 1bbb0464a7f2f0e58f805d29d0d3c0f9469c4b87 From 78f33613c2507a193abf66b767375adb3fc7f475 Mon Sep 17 00:00:00 2001 From: David Blewett Date: Thu, 22 Aug 2024 17:50:24 -0400 Subject: [PATCH 58/59] Bump to librdkafka 2.5.0+patches --- Cargo.lock | 2 +- Cargo.toml | 2 +- rdkafka-sys/Cargo.toml | 2 +- rdkafka-sys/changelog.md | 8 +++++++ rdkafka-sys/librdkafka | 2 +- rdkafka-sys/src/bindings.rs | 48 +++++++++++++++++++++++++++++++++++-- rdkafka-sys/src/helpers.rs | 8 +++++++ rdkafka-sys/src/types.rs | 20 ++++++++++++++++ 8 files changed, 86 insertions(+), 6 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index f67048df4..ff10e243e 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1099,7 +1099,7 @@ dependencies = [ [[package]] name = "rdkafka-sys" -version = "4.8.0+2.4.0" +version = "4.9.0+2.5.0" dependencies = [ "cmake", "curl-sys", diff --git a/Cargo.toml b/Cargo.toml index 0efd05f57..ffd7cbf4e 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -16,7 +16,7 @@ rust-version = "1.70" members = ["rdkafka-sys"] [dependencies] -rdkafka-sys = { path = "rdkafka-sys", version = "4.8.0", default-features = false } +rdkafka-sys = { path = "rdkafka-sys", version = "4.9.0", default-features = false } futures-channel = "0.3.0" futures-executor = { version = "0.3.0", optional = true } futures-util = { version = "0.3.0", default-features = false } diff --git a/rdkafka-sys/Cargo.toml b/rdkafka-sys/Cargo.toml index 9b03a10cd..14fbbb9b2 100644 --- a/rdkafka-sys/Cargo.toml +++ b/rdkafka-sys/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "rdkafka-sys" -version = "4.8.0+2.4.0" +version = "4.9.0+2.5.0" authors = ["Federico Giraud "] build = "build.rs" links = "rdkafka" diff --git a/rdkafka-sys/changelog.md b/rdkafka-sys/changelog.md index 44451c738..7a77d759f 100644 --- a/rdkafka-sys/changelog.md +++ b/rdkafka-sys/changelog.md @@ -2,6 +2,14 @@ ## Unreleased +## v4.9.0+2.5.0 (2024-08-22) + +* Upgrade to librdkafka v2.5.0. + +## v4.8.0+2.4.0 (2024-08-01) + +* Upgrade to librdkafka v2.4.0. + ## v4.7.0+2.2.0 (2023-11-07) * Upgrade to librdkafka v2.3.0. diff --git a/rdkafka-sys/librdkafka b/rdkafka-sys/librdkafka index 1bbb0464a..10f988f49 160000 --- a/rdkafka-sys/librdkafka +++ b/rdkafka-sys/librdkafka @@ -1 +1 @@ -Subproject commit 1bbb0464a7f2f0e58f805d29d0d3c0f9469c4b87 +Subproject commit 10f988f493695ee2ff180d9927f1e881ab98a36d diff --git a/rdkafka-sys/src/bindings.rs b/rdkafka-sys/src/bindings.rs index 78ba16b0c..766ff5598 100644 --- a/rdkafka-sys/src/bindings.rs +++ b/rdkafka-sys/src/bindings.rs @@ -3,7 +3,7 @@ use libc::{c_char, c_int, c_void, sockaddr, FILE}; use num_enum::TryFromPrimitive; -pub const RD_KAFKA_VERSION: i32 = 33751295; +pub const RD_KAFKA_VERSION: i32 = 33882367; pub const RD_KAFKA_DEBUG_CONTEXTS : & [u8 ; 138] = b"all,generic,broker,topic,metadata,feature,queue,msg,protocol,cgrp,security,fetch,interceptor,plugin,consumer,admin,eos,mock,assignor,conf\0" ; pub const RD_KAFKA_DESTROY_F_NO_CONSUMER_CLOSE: i32 = 8; pub const RD_KAFKA_OFFSET_BEGINNING: i32 = -2; @@ -214,6 +214,7 @@ pub enum rd_kafka_resp_err_t { RD_KAFKA_RESP_ERR__NOOP = -141, RD_KAFKA_RESP_ERR__AUTO_OFFSET_RESET = -140, RD_KAFKA_RESP_ERR__LOG_TRUNCATION = -139, + RD_KAFKA_RESP_ERR__INVALID_DIFFERENT_RECORD = -138, RD_KAFKA_RESP_ERR__END = -100, RD_KAFKA_RESP_ERR_UNKNOWN = -1, RD_KAFKA_RESP_ERR_NO_ERROR = 0, @@ -314,7 +315,14 @@ pub enum rd_kafka_resp_err_t { RD_KAFKA_RESP_ERR_INVALID_UPDATE_VERSION = 95, RD_KAFKA_RESP_ERR_FEATURE_UPDATE_FAILED = 96, RD_KAFKA_RESP_ERR_PRINCIPAL_DESERIALIZATION_FAILURE = 97, - RD_KAFKA_RESP_ERR_END_ALL = 98, + RD_KAFKA_RESP_ERR_UNKNOWN_TOPIC_ID = 100, + RD_KAFKA_RESP_ERR_FENCED_MEMBER_EPOCH = 110, + RD_KAFKA_RESP_ERR_UNRELEASED_INSTANCE_ID = 111, + RD_KAFKA_RESP_ERR_UNSUPPORTED_ASSIGNOR = 112, + RD_KAFKA_RESP_ERR_STALE_MEMBER_EPOCH = 113, + RD_KAFKA_RESP_ERR_UNKNOWN_SUBSCRIPTION_ID = 117, + RD_KAFKA_RESP_ERR_TELEMETRY_TOO_LARGE = 118, + RD_KAFKA_RESP_ERR_END_ALL = 119, } #[repr(C)] #[derive(Debug, Copy, Clone)] @@ -604,6 +612,9 @@ extern "C" { extern "C" { pub fn rd_kafka_message_errstr(rkmessage: *const rd_kafka_message_t) -> *const c_char; } +extern "C" { + pub fn rd_kafka_message_produce_errstr(rkmessage: *const rd_kafka_message_t) -> *const c_char; +} extern "C" { pub fn rd_kafka_message_timestamp( rkmessage: *const rd_kafka_message_t, @@ -1500,6 +1511,11 @@ extern "C" { group_instance_id: *const c_char, ) -> *mut rd_kafka_consumer_group_metadata_t; } +extern "C" { + pub fn rd_kafka_consumer_group_metadata_member_id( + group_metadata: *const rd_kafka_consumer_group_metadata_t, + ) -> *const c_char; +} extern "C" { pub fn rd_kafka_consumer_group_metadata_destroy(arg1: *mut rd_kafka_consumer_group_metadata_t); } @@ -3604,6 +3620,15 @@ extern "C" { hi: i64, ) -> rd_kafka_resp_err_t; } +extern "C" { + pub fn rd_kafka_mock_partition_push_leader_response( + mcluster: *mut rd_kafka_mock_cluster_t, + topic: *const c_char, + partition: c_int, + leader_id: i32, + leader_epoch: i32, + ) -> rd_kafka_resp_err_t; +} extern "C" { pub fn rd_kafka_mock_broker_set_down( mcluster: *mut rd_kafka_mock_cluster_t, @@ -3661,6 +3686,12 @@ pub type rd_kafka_mock_request_t = rd_kafka_mock_request_s; extern "C" { pub fn rd_kafka_mock_request_destroy(mreq: *mut rd_kafka_mock_request_t); } +extern "C" { + pub fn rd_kafka_mock_request_destroy_array( + mreqs: *mut *mut rd_kafka_mock_request_t, + mreq_cnt: usize, + ); +} extern "C" { pub fn rd_kafka_mock_request_id(mreq: *mut rd_kafka_mock_request_t) -> i32; } @@ -3679,3 +3710,16 @@ extern "C" { extern "C" { pub fn rd_kafka_mock_clear_requests(mcluster: *mut rd_kafka_mock_cluster_t); } +extern "C" { + pub fn rd_kafka_mock_telemetry_set_requested_metrics( + mcluster: *mut rd_kafka_mock_cluster_t, + metrics: *mut *mut c_char, + metrics_cnt: usize, + ) -> rd_kafka_resp_err_t; +} +extern "C" { + pub fn rd_kafka_mock_telemetry_set_push_interval( + mcluster: *mut rd_kafka_mock_cluster_t, + push_interval_ms: i64, + ) -> rd_kafka_resp_err_t; +} diff --git a/rdkafka-sys/src/helpers.rs b/rdkafka-sys/src/helpers.rs index 8e5cc60b6..9608feb5d 100644 --- a/rdkafka-sys/src/helpers.rs +++ b/rdkafka-sys/src/helpers.rs @@ -174,5 +174,13 @@ pub fn rd_kafka_resp_err_t_to_rdkafka_error(err: RDKafkaRespErr) -> RDKafkaError RD_KAFKA_RESP_ERR_PRINCIPAL_DESERIALIZATION_FAILURE => PrincipalDeserializationFailure, RD_KAFKA_RESP_ERR_END_ALL => EndAll, RD_KAFKA_RESP_ERR__LOG_TRUNCATION => LogTruncation, + RD_KAFKA_RESP_ERR__INVALID_DIFFERENT_RECORD => InvalidDifferentRecord, + RD_KAFKA_RESP_ERR_UNKNOWN_TOPIC_ID => UnknownTopicId, + RD_KAFKA_RESP_ERR_FENCED_MEMBER_EPOCH => FencedMemberEpoch, + RD_KAFKA_RESP_ERR_UNRELEASED_INSTANCE_ID => UnreleasedInstanceId, + RD_KAFKA_RESP_ERR_UNSUPPORTED_ASSIGNOR => UnsupportedAssignor, + RD_KAFKA_RESP_ERR_STALE_MEMBER_EPOCH => StaleMemberEpoch, + RD_KAFKA_RESP_ERR_UNKNOWN_SUBSCRIPTION_ID => UnknownSubscriptionId, + RD_KAFKA_RESP_ERR_TELEMETRY_TOO_LARGE => TelemetryTooLarge, } } diff --git a/rdkafka-sys/src/types.rs b/rdkafka-sys/src/types.rs index 0005073ba..b06e152c4 100644 --- a/rdkafka-sys/src/types.rs +++ b/rdkafka-sys/src/types.rs @@ -249,6 +249,9 @@ pub enum RDKafkaErrorCode { AutoOffsetReset = -140, /// Partition log truncation detected LogTruncation = -139, + /// A different record in the batch was invalid and this message failed + /// persisting. + InvalidDifferentRecord = -138, #[doc(hidden)] End = -100, /// Unknown broker error. @@ -456,6 +459,23 @@ pub enum RDKafkaErrorCode { FeatureUpdateFailed = 96, /// Request principal deserialization failed during forwarding. PrincipalDeserializationFailure = 97, + /// Unknown topic ID. + UnknownTopicId = 100, + /// The member epoch is fenced by the group coordinator. + FencedMemberEpoch = 110, + /// The instance ID is still used by another member in the consumer group. + UnreleasedInstanceId = 111, + /// The assignor or its version range is not supported by the consumer + /// group. + UnsupportedAssignor = 112, + /// The member epoch is stale. + StaleMemberEpoch = 113, + /// Client sent a push telemetry request with an invalid or outdated + /// subscription ID. + UnknownSubscriptionId = 117, + /// Client sent a push telemetry request larger than the maximum size + /// the broker will accept. + TelemetryTooLarge = 118, #[doc(hidden)] EndAll, } From 0dcce2b35f8a4410fd458299b654a745be5bca40 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Micha=C5=82=20=C5=81owicki?= Date: Mon, 26 Aug 2024 10:23:36 +0000 Subject: [PATCH 59/59] optimise OwnedHeaders::insert --- src/message.rs | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/src/message.rs b/src/message.rs index 2048a1ced..dba9cf20f 100644 --- a/src/message.rs +++ b/src/message.rs @@ -1,9 +1,9 @@ //! Store and manipulate Kafka messages. -use std::ffi::{CStr, CString}; +use std::ffi::CStr; use std::fmt; use std::marker::PhantomData; -use std::os::raw::c_void; +use std::os::raw::{c_char, c_void}; use std::ptr; use std::str; use std::sync::Arc; @@ -534,7 +534,6 @@ impl OwnedHeaders { where V: ToBytes + ?Sized, { - let name_cstring = CString::new(header.key).unwrap(); let (value_ptr, value_len) = match header.value { None => (ptr::null_mut(), 0), Some(value) => { @@ -548,8 +547,8 @@ impl OwnedHeaders { let err = unsafe { rdsys::rd_kafka_header_add( self.ptr(), - name_cstring.as_ptr(), - name_cstring.as_bytes().len() as isize, + header.key.as_ptr() as *const c_char, + header.key.as_bytes().len() as isize, value_ptr, value_len, )