@@ -23,6 +23,25 @@ use crate::client::Client;
23
23
use crate :: config:: ClientConfig ;
24
24
use crate :: error:: { IsError , KafkaError , KafkaResult } ;
25
25
use crate :: producer:: DefaultProducerContext ;
26
+ use crate :: ClientContext ;
27
+
28
+ /// Used internally by `MockCluster` to distinguish whether the mock cluster is owned or referenced.
29
+ ///
30
+ /// The mock cluster can be created in two ways:
31
+ ///
32
+ /// - With `rd_kafka_mock_cluster_new()`. In this case the caller of the c-tor is responsible
33
+ /// for destroying the returned mock cluster instance.
34
+ ///
35
+ /// - By setting `test.mock.num.brokers` in a configuration of a producer/consumer client.
36
+ /// In this case, the client creates the mock cluster internally and destroys it in its d-tor,
37
+ /// and we only hold a reference to the mock cluster obtained with `rd_kafka_handle_mock_cluster()` (cf. `Client::mock_cluser()`).
38
+ ///
39
+ /// In this case, we **must neither** destroy the mock clsuter in `MockCluster`'s `drop()`,
40
+ /// **nor** outlive the `Client` from which the reference is obtained, hence the lifetime.
41
+ enum MockClusterClient < ' c , C : ClientContext > {
42
+ Owned ( Client < C > ) ,
43
+ Ref ( & ' c Client < C > ) ,
44
+ }
26
45
27
46
/// Mock Kafka cluster with a configurable number of brokers that support a reasonable subset of
28
47
/// Kafka protocol operations, error injection, etc.
@@ -38,10 +57,18 @@ use crate::producer::DefaultProducerContext;
38
57
/// - High-level balanced consumer groups with offset commits
39
58
/// - Topic Metadata and auto creation
40
59
///
60
+ /// The mock cluster can be either created with [`MockCluster::new()`]
61
+ /// or by configuring the `test.mock.num.brokers` property when creating a producer/consumer.
62
+ /// This will override that producer/consumer's bootstrap servers setting and internally
63
+ /// create a mock cluster. You can then obtain this mock cluster using [`Client::mock_cluster()`].
64
+ ///
41
65
/// Warning THIS IS AN EXPERIMENTAL API, SUBJECT TO CHANGE OR REMOVAL.
42
- pub struct MockCluster {
66
+ ///
67
+ /// [`MockCluster::new()`]: MockCluster::new()
68
+ /// [`Client::mock_cluster()`]: crate::client::Client::mock_cluster()
69
+ pub struct MockCluster < ' c , C : ClientContext > {
43
70
mock_cluster : * mut RDKafkaMockCluster ,
44
- _client : Client < DefaultProducerContext > ,
71
+ client : MockClusterClient < ' c , C > ,
45
72
}
46
73
47
74
/// Utility macro to simplify returns for operations done on the mock API
@@ -62,7 +89,7 @@ pub enum MockCoordinator {
62
89
Group ( String ) ,
63
90
}
64
91
65
- impl MockCluster {
92
+ impl MockCluster < ' static , DefaultProducerContext > {
66
93
/// Create new mock cluster with a given number of brokers
67
94
pub fn new ( broker_count : i32 ) -> KafkaResult < Self > {
68
95
let config = ClientConfig :: new ( ) ;
@@ -84,9 +111,21 @@ impl MockCluster {
84
111
85
112
Ok ( MockCluster {
86
113
mock_cluster,
87
- _client : client,
114
+ client : MockClusterClient :: Owned ( client) ,
88
115
} )
89
116
}
117
+ }
118
+
119
+ impl < ' c , C > MockCluster < ' c , C >
120
+ where
121
+ C : ClientContext ,
122
+ {
123
+ pub ( crate ) fn new_ref ( mock_cluster : * mut RDKafkaMockCluster , client : & ' c Client < C > ) -> Self {
124
+ Self {
125
+ mock_cluster,
126
+ client : MockClusterClient :: Ref ( client) ,
127
+ }
128
+ }
90
129
91
130
/// Obtain the bootstrap address for the mock cluster
92
131
pub fn bootstrap_servers ( & self ) -> String {
@@ -275,21 +314,26 @@ impl MockCluster {
275
314
}
276
315
}
277
316
278
- impl Drop for MockCluster {
317
+ impl < ' c , C > Drop for MockCluster < ' c , C >
318
+ where
319
+ C : ClientContext ,
320
+ {
279
321
fn drop ( & mut self ) {
280
- unsafe {
281
- rdsys:: rd_kafka_mock_cluster_destroy ( self . mock_cluster ) ;
322
+ if let MockClusterClient :: Owned ( ..) = self . client {
323
+ unsafe {
324
+ rdsys:: rd_kafka_mock_cluster_destroy ( self . mock_cluster ) ;
325
+ }
282
326
}
283
327
}
284
328
}
285
329
286
330
#[ cfg( test) ]
287
331
mod tests {
288
- use tokio;
289
- use crate :: Message ;
332
+ use crate :: consumer:: { Consumer , StreamConsumer } ;
290
333
use crate :: message:: ToBytes ;
291
334
use crate :: producer:: { FutureProducer , FutureRecord } ;
292
- use crate :: consumer:: { Consumer , StreamConsumer } ;
335
+ use crate :: Message ;
336
+ use tokio;
293
337
294
338
use super :: * ;
295
339
0 commit comments