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
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
use std::ffi::CStr;
use std::slice;
use rdkafka_sys as rdsys;
use rdkafka_sys::types::*;
use crate::error::IsError;
use crate::util::{KafkaDrop, NativePtr};
pub struct MetadataBroker(RDKafkaMetadataBroker);
impl MetadataBroker {
pub fn id(&self) -> i32 {
self.0.id
}
pub fn host(&self) -> &str {
unsafe {
CStr::from_ptr(self.0.host)
.to_str()
.expect("Broker host is not a valid UTF-8 string")
}
}
pub fn port(&self) -> i32 {
self.0.port
}
}
pub struct MetadataPartition(RDKafkaMetadataPartition);
impl MetadataPartition {
pub fn id(&self) -> i32 {
self.0.id
}
pub fn leader(&self) -> i32 {
self.0.leader
}
pub fn error(&self) -> Option<RDKafkaRespErr> {
if self.0.err.is_error() {
Some(self.0.err)
} else {
None
}
}
pub fn replicas(&self) -> &[i32] {
unsafe { slice::from_raw_parts(self.0.replicas, self.0.replica_cnt as usize) }
}
pub fn isr(&self) -> &[i32] {
unsafe { slice::from_raw_parts(self.0.isrs, self.0.isr_cnt as usize) }
}
}
pub struct MetadataTopic(RDKafkaMetadataTopic);
impl MetadataTopic {
pub fn name(&self) -> &str {
unsafe {
CStr::from_ptr(self.0.topic)
.to_str()
.expect("Topic name is not a valid UTF-8 string")
}
}
pub fn partitions(&self) -> &[MetadataPartition] {
unsafe {
slice::from_raw_parts(
self.0.partitions as *const MetadataPartition,
self.0.partition_cnt as usize,
)
}
}
pub fn error(&self) -> Option<RDKafkaRespErr> {
if self.0.err.is_error() {
Some(self.0.err)
} else {
None
}
}
}
pub struct Metadata(NativePtr<RDKafkaMetadata>);
unsafe impl KafkaDrop for RDKafkaMetadata {
const TYPE: &'static str = "metadata";
const DROP: unsafe extern "C" fn(*mut Self) = drop_metadata;
}
unsafe extern "C" fn drop_metadata(ptr: *mut RDKafkaMetadata) {
rdsys::rd_kafka_metadata_destroy(ptr as *const _)
}
impl Metadata {
pub(crate) unsafe fn from_ptr(ptr: *const RDKafkaMetadata) -> Metadata {
Metadata(NativePtr::from_ptr(ptr as *mut _).unwrap())
}
pub fn orig_broker_id(&self) -> i32 {
self.0.orig_broker_id
}
pub fn orig_broker_name(&self) -> &str {
unsafe {
CStr::from_ptr(self.0.orig_broker_name)
.to_str()
.expect("Broker name is not a valid UTF-8 string")
}
}
pub fn brokers(&self) -> &[MetadataBroker] {
unsafe {
slice::from_raw_parts(
self.0.brokers as *const MetadataBroker,
self.0.broker_cnt as usize,
)
}
}
pub fn topics(&self) -> &[MetadataTopic] {
unsafe {
slice::from_raw_parts(
self.0.topics as *const MetadataTopic,
self.0.topic_cnt as usize,
)
}
}
}
unsafe impl Send for Metadata {}
unsafe impl Sync for Metadata {}