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
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
//! Kafka producers.
//!
//! ## The C librdkafka producer
//!
//! Rust-rdkafka relies on the C librdkafka producer to communicate with Kafka,
//! so in order to understand how the Rust producers work it is important to
//! understand the basics of the C one as well.
//!
//! ### Async
//!
//! The librdkafka producer is completely asynchronous: it maintains a memory
//! buffer where messages waiting to be sent or currently in flight are stored.
//! Once a message is delivered or an error occurred and the maximum number of
//! retries has been reached, the producer will enqueue a delivery event with
//! the appropriate delivery result into an internal event queue.
//!
//! The librdkafka user is responsible for calling the `poll` function at
//! regular intervals to process those events; the thread calling `poll` will be
//! the one executing the user-specified delivery callback for every delivery
//! event. If `poll` is not called, or not frequently enough, the producer will
//! return a [`RDKafkaErrorCode::QueueFull`] error and it won't be able to send
//! any other message until more delivery events are processed via `poll`. The
//! `QueueFull` error can also be returned if Kafka is not able to receive the
//! messages quickly enough.
//!
//! ### Error reporting
//!
//! The C library will try deal with all the transient errors such as broker
//! disconnection, timeouts etc. These errors, called global errors, are
//! automatically logged in rust-rdkafka, but they normally don't require any
//! handling as they are automatically handled internally. To see the logs, make
//! sure you initialize the logger.
//!
//! As mentioned earlier, errors specific to message production will be reported
//! in the delivery callback.
//!
//! ### Buffering
//!
//! Buffering is done automatically by librdkafka. When `send` is called, the
//! message is enqueued internally and once enough messages have been enqueued,
//! or when enough time has passed, they will be sent to Kafka as a single
//! batch. You can control the behavior of the buffer by configuring the the
//! `queue.buffering.max.*` parameters listed below.
//!
//! ## `rust-rdkafka` producers
//!
//! `rust-rdkafka` (rdkafka for brevity) provides two sets of producers: low
//! level and high level.
//!
//! ### Low-level producers
//!
//! The lowest level producer provided by rdkafka is called [`BaseProducer`].
//! The goal of the `BaseProducer` is to be as close as possible to the C one
//! while maintaining a safe Rust interface. In particular, the `BaseProducer`
//! needs to be polled at regular intervals to execute any delivery callback
//! that might be waiting and to make sure the queue doesn't fill up.
//!
//! Another low lever producer is the [`ThreadedProducer`], which is a
//! `BaseProducer` with a dedicated thread for polling.
//!
//! The delivery callback can be defined using a `ProducerContext`. See the
//! [`base_producer`] module for more information.
//!
//! ### High-level producer
//!
//! At the moment the only high level producer implemented is the
//! [`FutureProducer`]. The `FutureProducer` doesn't rely on user-defined
//! callbacks to notify the delivery or failure of a message; instead, this
//! information will be returned in a Future. The `FutureProducer` also uses an
//! internal thread that is used for polling, which makes calling poll
//! explicitly not necessary. The returned future will contain information about
//! the delivered message in case of success, or a copy of the original message
//! in case of failure. Additional computation can be chained to the returned
//! future, and it will executed by the future executor once the value is
//! available (for more information, check the documentation of the futures
//! crate).
//!
//! ## Transactions
//!
//! All rust-rdkafka producers support transactions. Transactional producers
//! work together with transaction-aware consumers configured with the default
//! `isolation.level` of `read_committed`.
//!
//! To configure a producer for transactions set `transactional.id` to an
//! identifier unique to the application when creating the producer. After
//! creating the producer, you must initialize it with
//! [`Producer::init_transactions`].
//!
//! To start a new transaction use [`Producer::begin_transaction`]. There can be
//! **only one ongoing transaction** at a time per producer. All records sent
//! after starting a transaction and before committing or aborting it will
//! automatically be associated with that transaction.
//!
//! Once you have initialized transactions on a producer, you are not permitted
//! to produce messages outside of a transaction.
//!
//! Consumer offsets can be sent as part of the ongoing transaction using
//! `send_offsets_to_transaction` and will be committed atomically with the
//! other records sent in the transaction.
//!
//! The current transaction can be committed with
//! [`Producer::commit_transaction`] or aborted using
//! [`Producer::abort_transaction`]. Afterwards, a new transaction can begin.
//!
//! ### Errors
//!
//! Errors returned by transaction methods may:
//!
//! * be retriable ([`RDKafkaError::is_retriable`]), in which case the operation
//! that encountered the error may be retried.
//! * require abort ([`RDKafkaError::txn_requires_abort`], in which case the
//! current transaction must be aborted and a new transaction begun.
//! * be fatal ([`RDKafkaError::is_fatal`]), in which case the producer must be
//! stopped and the application terminated.
//!
//! For more details about transactions, see the [Transactional Producer]
//! section of the librdkafka introduction.
//!
//! ## Configuration
//!
//! ### Producer configuration
//!
//! For the configuration parameters common to both producers and consumers,
//! refer to the documentation in the `config` module. Here are listed the most
//! commonly used producer configuration. Click
//! [here](https://github.com/edenhill/librdkafka/blob/master/CONFIGURATION.md)
//! for the full list.
//!
//! - `queue.buffering.max.messages`: Maximum number of messages allowed on the
//! producer queue. Default: 100000.
//! - `queue.buffering.max.kbytes`: Maximum total message size sum allowed on
//! the producer queue. This property has higher priority than
//! queue.buffering.max.messages. Default: 1048576.
//! - `queue.buffering.max.ms`: Delay in milliseconds to wait for messages in
//! the producer queue to accumulate before sending a request to the brokers.
//! A higher value allows larger and more effective (less overhead, improved
//! compression) batches of messages to accumulate at the expense of increased
//! message delivery latency. Default: 0.
//! - `message.send.max.retries`: How many times to retry sending a failing
//! batch. Note: retrying may cause reordering. Default: 2.
//! - `compression.codec`: Compression codec to use for compressing message
//! sets. Default: none.
//! - `request.required.acks`: This field indicates how many acknowledgements
//! the leader broker must receive from ISR brokers before responding to the
//! request: 0=Broker does not send any response/ack to client, 1=Only the
//! leader broker will need to ack the message, -1 or all=broker will block
//! until message is committed by all in sync replicas (ISRs) or broker's
//! in.sync.replicas setting before sending response. Default: 1.
//! - `request.timeout.ms`: The ack timeout of the producer request in
//! milliseconds. This value is only enforced by the broker and relies on
//! request.required.acks being != 0. Default: 5000.
//! - `message.timeout.ms`: Local message timeout. This value is only enforced
//! locally and limits the time a produced message waits for successful
//! delivery. A time of 0 is infinite. Default: 300000.
//!
//! [`RDKafkaErrorCode::QueueFull`]: crate::error::RDKafkaErrorCode::QueueFull
//! [`RDKafkaError::is_retriable`]: crate::error::RDKafkaError::is_retriable
//! [`RDKafkaError::txn_requires_abort`]: crate::error::RDKafkaError::txn_requires_abort
//! [`RDKafkaError::is_fatal`]: crate::error::RDKafkaError::is_fatal
//! [Transactional Producer]: https://github.com/edenhill/librdkafka/blob/master/INTRODUCTION.md#transactional-producer
use Arc;
use crate;
use crateConsumerGroupMetadata;
use crateKafkaResult;
use crateTopicPartitionList;
use crate;
pub use ;
pub use ;
//
// ********** PRODUCER CONTEXT **********
//
/// Producer-specific context.
///
/// This user-defined object can be used to provide custom callbacks for
/// producer events. Refer to the list of methods to check which callbacks can
/// be specified. It can also specify custom partitioner to register and to be
/// used for deciding to which partition write message into.
///
/// In particular, it can be used to specify the `delivery` callback that will
/// be called when the acknowledgement for a delivered message is received.
///
/// See also the [`ClientContext`] trait.
/// Unassigned partition.
/// See RD_KAFKA_PARTITION_UA from librdkafka.
pub const PARTITION_UA: i32 = -1;
/// Trait allowing to customize the partitioning of messages.
/// Placeholder used when no custom partitioner is needed.
/// An inert producer context that can be used when customizations are not
/// required.
;
/// Common trait for all producers.
/// Settings to provide to [`Producer::purge`] to parametrize the purge behavior
///
/// `PurgeConfig::default()` corresponds to a setting where nothing is purged.
///
/// # Example
/// To purge both queued messages and in-flight messages:
/// ```
/// # use rdkafka::producer::PurgeConfig;
/// let settings = PurgeConfig::default().queue().inflight();
/// ```
negative_and_debug_impls!