1 /*
2  * librdkafka - Apache Kafka C library
3  *
4  * Copyright (c) 2012-2020 Magnus Edenhill
5  * All rights reserved.
6  *
7  * Redistribution and use in source and binary forms, with or without
8  * modification, are permitted provided that the following conditions are met:
9  *
10  * 1. Redistributions of source code must retain the above copyright notice,
11  *    this list of conditions and the following disclaimer.
12  * 2. Redistributions in binary form must reproduce the above copyright notice,
13  *    this list of conditions and the following disclaimer in the documentation
14  *    and/or other materials provided with the distribution.
15  *
16  * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
17  * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
18  * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
19  * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE
20  * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
21  * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
22  * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
23  * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
24  * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
25  * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE
26  * POSSIBILITY OF SUCH DAMAGE.
27  */
28 
29 /**
30  * @file rdkafka.h
31  * @brief Apache Kafka C/C++ consumer and producer client library.
32  *
33  * rdkafka.h contains the public API for librdkafka.
34  * The API is documented in this file as comments prefixing the function, type,
35  * enum, define, etc.
36  *
37  * @sa For the C++ interface see rdkafkacpp.h
38  *
39  * @tableofcontents
40  */
41 
42 
43 /* @cond NO_DOC */
44 #ifndef _RDKAFKA_H_
45 #define _RDKAFKA_H_
46 
47 #include <stdio.h>
48 #include <inttypes.h>
49 #include <sys/types.h>
50 
51 #ifdef __cplusplus
52 extern "C" {
53 #if 0
54 } /* Restore indent */
55 #endif
56 #endif
57 
58 #ifdef _WIN32
59 #include <basetsd.h>
60 #ifndef WIN32_MEAN_AND_LEAN
61 #define WIN32_MEAN_AND_LEAN
62 #endif
63 #include <winsock2.h>  /* for sockaddr, .. */
64 #ifndef _SSIZE_T_DEFINED
65 #define _SSIZE_T_DEFINED
66 typedef SSIZE_T ssize_t;
67 #endif
68 #define RD_UNUSED
69 #define RD_INLINE __inline
70 #define RD_DEPRECATED __declspec(deprecated)
71 #define RD_FORMAT(...)
72 #undef RD_EXPORT
73 #ifdef LIBRDKAFKA_STATICLIB
74 #define RD_EXPORT
75 #else
76 #ifdef LIBRDKAFKA_EXPORTS
77 #define RD_EXPORT __declspec(dllexport)
78 #else
79 #define RD_EXPORT __declspec(dllimport)
80 #endif
81 #ifndef LIBRDKAFKA_TYPECHECKS
82 #define LIBRDKAFKA_TYPECHECKS 0
83 #endif
84 #endif
85 
86 #else
87 #include <sys/socket.h> /* for sockaddr, .. */
88 
89 #define RD_UNUSED __attribute__((unused))
90 #define RD_INLINE inline
91 #define RD_EXPORT
92 #define RD_DEPRECATED __attribute__((deprecated))
93 
94 #if defined(__clang__) || defined(__GNUC__) || defined(__GNUG__)
95 #define RD_FORMAT(...) __attribute__((format (__VA_ARGS__)))
96 #else
97 #define RD_FORMAT(...)
98 #endif
99 
100 #ifndef LIBRDKAFKA_TYPECHECKS
101 #define LIBRDKAFKA_TYPECHECKS 1
102 #endif
103 #endif
104 
105 
106 /**
107  * @brief Type-checking macros
108  * Compile-time checking that \p ARG is of type \p TYPE.
109  * @returns \p RET
110  */
111 #if LIBRDKAFKA_TYPECHECKS
112 #define _LRK_TYPECHECK(RET,TYPE,ARG)                    \
113         ({ if (0) { TYPE __t RD_UNUSED = (ARG); } RET; })
114 
115 #define _LRK_TYPECHECK2(RET,TYPE,ARG,TYPE2,ARG2)        \
116         ({                                              \
117                 if (0) {                                \
118                         TYPE __t RD_UNUSED = (ARG);     \
119                         TYPE2 __t2 RD_UNUSED = (ARG2);  \
120                 }                                       \
121                 RET; })
122 
123 #define _LRK_TYPECHECK3(RET,TYPE,ARG,TYPE2,ARG2,TYPE3,ARG3) \
124         ({                                              \
125                 if (0) {                                \
126                         TYPE __t RD_UNUSED = (ARG);     \
127                         TYPE2 __t2 RD_UNUSED = (ARG2);  \
128                         TYPE3 __t3 RD_UNUSED = (ARG3);  \
129                 }                                       \
130                 RET; })
131 #else
132 #define _LRK_TYPECHECK(RET,TYPE,ARG)  (RET)
133 #define _LRK_TYPECHECK2(RET,TYPE,ARG,TYPE2,ARG2) (RET)
134 #define _LRK_TYPECHECK3(RET,TYPE,ARG,TYPE2,ARG2,TYPE3,ARG3) (RET)
135 #endif
136 
137 /* @endcond */
138 
139 
140 /**
141  * @name librdkafka version
142  * @{
143  *
144  *
145  */
146 
147 /**
148  * @brief librdkafka version
149  *
150  * Interpreted as hex \c MM.mm.rr.xx:
151  *  - MM = Major
152  *  - mm = minor
153  *  - rr = revision
154  *  - xx = pre-release id (0xff is the final release)
155  *
156  * E.g.: \c 0x000801ff = 0.8.1
157  *
158  * @remark This value should only be used during compile time,
159  *         for runtime checks of version use rd_kafka_version()
160  */
161 #define RD_KAFKA_VERSION  0x010802ff
162 
163 /**
164  * @brief Returns the librdkafka version as integer.
165  *
166  * @returns Version integer.
167  *
168  * @sa See RD_KAFKA_VERSION for how to parse the integer format.
169  * @sa Use rd_kafka_version_str() to retreive the version as a string.
170  */
171 RD_EXPORT
172 int rd_kafka_version(void);
173 
174 /**
175  * @brief Returns the librdkafka version as string.
176  *
177  * @returns Version string
178  */
179 RD_EXPORT
180 const char *rd_kafka_version_str (void);
181 
182 /**@}*/
183 
184 
185 /**
186  * @name Constants, errors, types
187  * @{
188  *
189  *
190  */
191 
192 
193 /**
194  * @enum rd_kafka_type_t
195  *
196  * @brief rd_kafka_t handle type.
197  *
198  * @sa rd_kafka_new()
199  */
200 typedef enum rd_kafka_type_t {
201 	RD_KAFKA_PRODUCER, /**< Producer client */
202 	RD_KAFKA_CONSUMER  /**< Consumer client */
203 } rd_kafka_type_t;
204 
205 
206 /*!
207  * Timestamp types
208  *
209  * @sa rd_kafka_message_timestamp()
210  */
211 typedef enum rd_kafka_timestamp_type_t {
212 	RD_KAFKA_TIMESTAMP_NOT_AVAILABLE,   /**< Timestamp not available */
213 	RD_KAFKA_TIMESTAMP_CREATE_TIME,     /**< Message creation time */
214 	RD_KAFKA_TIMESTAMP_LOG_APPEND_TIME  /**< Log append time */
215 } rd_kafka_timestamp_type_t;
216 
217 
218 
219 /**
220  * @brief Retrieve supported debug contexts for use with the \c \"debug\"
221  *        configuration property. (runtime)
222  *
223  * @returns Comma-separated list of available debugging contexts.
224  */
225 RD_EXPORT
226 const char *rd_kafka_get_debug_contexts(void);
227 
228 /**
229  * @brief Supported debug contexts. (compile time)
230  *
231  * @deprecated This compile time value may be outdated at runtime due to
232  *             linking another version of the library.
233  *             Use rd_kafka_get_debug_contexts() instead.
234  */
235 #define RD_KAFKA_DEBUG_CONTEXTS \
236         "all,generic,broker,topic,metadata,feature,queue,msg,protocol,cgrp,security,fetch,interceptor,plugin,consumer,admin,eos,mock,assignor,conf"
237 
238 
239 /* @cond NO_DOC */
240 /* Private types to provide ABI compatibility */
241 typedef struct rd_kafka_s rd_kafka_t;
242 typedef struct rd_kafka_topic_s rd_kafka_topic_t;
243 typedef struct rd_kafka_conf_s rd_kafka_conf_t;
244 typedef struct rd_kafka_topic_conf_s rd_kafka_topic_conf_t;
245 typedef struct rd_kafka_queue_s rd_kafka_queue_t;
246 typedef struct rd_kafka_op_s rd_kafka_event_t;
247 typedef struct rd_kafka_topic_result_s rd_kafka_topic_result_t;
248 typedef struct rd_kafka_consumer_group_metadata_s
249 rd_kafka_consumer_group_metadata_t;
250 typedef struct rd_kafka_error_s rd_kafka_error_t;
251 typedef struct rd_kafka_headers_s rd_kafka_headers_t;
252 typedef struct rd_kafka_group_result_s rd_kafka_group_result_t;
253 /* @endcond */
254 
255 
256 /**
257  * @enum rd_kafka_resp_err_t
258  * @brief Error codes.
259  *
260  * The negative error codes delimited by two underscores
261  * (\c RD_KAFKA_RESP_ERR__..) denotes errors internal to librdkafka and are
262  * displayed as \c \"Local: \<error string..\>\", while the error codes
263  * delimited by a single underscore (\c RD_KAFKA_RESP_ERR_..) denote broker
264  * errors and are displayed as \c \"Broker: \<error string..\>\".
265  *
266  * @sa Use rd_kafka_err2str() to translate an error code a human readable string
267  */
268 typedef enum {
269 	/* Internal errors to rdkafka: */
270 	/** Begin internal error codes */
271 	RD_KAFKA_RESP_ERR__BEGIN = -200,
272 	/** Received message is incorrect */
273 	RD_KAFKA_RESP_ERR__BAD_MSG = -199,
274 	/** Bad/unknown compression */
275 	RD_KAFKA_RESP_ERR__BAD_COMPRESSION = -198,
276 	/** Broker is going away */
277 	RD_KAFKA_RESP_ERR__DESTROY = -197,
278 	/** Generic failure */
279 	RD_KAFKA_RESP_ERR__FAIL = -196,
280 	/** Broker transport failure */
281 	RD_KAFKA_RESP_ERR__TRANSPORT = -195,
282 	/** Critical system resource */
283 	RD_KAFKA_RESP_ERR__CRIT_SYS_RESOURCE = -194,
284 	/** Failed to resolve broker */
285 	RD_KAFKA_RESP_ERR__RESOLVE = -193,
286 	/** Produced message timed out*/
287 	RD_KAFKA_RESP_ERR__MSG_TIMED_OUT = -192,
288 	/** Reached the end of the topic+partition queue on
289 	 * the broker. Not really an error.
290 	 * This event is disabled by default,
291 	 * see the `enable.partition.eof` configuration property. */
292 	RD_KAFKA_RESP_ERR__PARTITION_EOF = -191,
293 	/** Permanent: Partition does not exist in cluster. */
294 	RD_KAFKA_RESP_ERR__UNKNOWN_PARTITION = -190,
295 	/** File or filesystem error */
296 	RD_KAFKA_RESP_ERR__FS = -189,
297 	 /** Permanent: Topic does not exist in cluster. */
298 	RD_KAFKA_RESP_ERR__UNKNOWN_TOPIC = -188,
299 	/** All broker connections are down. */
300 	RD_KAFKA_RESP_ERR__ALL_BROKERS_DOWN = -187,
301 	/** Invalid argument, or invalid configuration */
302 	RD_KAFKA_RESP_ERR__INVALID_ARG = -186,
303 	/** Operation timed out */
304 	RD_KAFKA_RESP_ERR__TIMED_OUT = -185,
305 	/** Queue is full */
306 	RD_KAFKA_RESP_ERR__QUEUE_FULL = -184,
307 	/** ISR count < required.acks */
308         RD_KAFKA_RESP_ERR__ISR_INSUFF = -183,
309 	/** Broker node update */
310         RD_KAFKA_RESP_ERR__NODE_UPDATE = -182,
311 	/** SSL error */
312 	RD_KAFKA_RESP_ERR__SSL = -181,
313 	/** Waiting for coordinator to become available. */
314         RD_KAFKA_RESP_ERR__WAIT_COORD = -180,
315 	/** Unknown client group */
316         RD_KAFKA_RESP_ERR__UNKNOWN_GROUP = -179,
317 	/** Operation in progress */
318         RD_KAFKA_RESP_ERR__IN_PROGRESS = -178,
319 	 /** Previous operation in progress, wait for it to finish. */
320         RD_KAFKA_RESP_ERR__PREV_IN_PROGRESS = -177,
321 	 /** This operation would interfere with an existing subscription */
322         RD_KAFKA_RESP_ERR__EXISTING_SUBSCRIPTION = -176,
323 	/** Assigned partitions (rebalance_cb) */
324         RD_KAFKA_RESP_ERR__ASSIGN_PARTITIONS = -175,
325 	/** Revoked partitions (rebalance_cb) */
326         RD_KAFKA_RESP_ERR__REVOKE_PARTITIONS = -174,
327 	/** Conflicting use */
328         RD_KAFKA_RESP_ERR__CONFLICT = -173,
329 	/** Wrong state */
330         RD_KAFKA_RESP_ERR__STATE = -172,
331 	/** Unknown protocol */
332         RD_KAFKA_RESP_ERR__UNKNOWN_PROTOCOL = -171,
333 	/** Not implemented */
334         RD_KAFKA_RESP_ERR__NOT_IMPLEMENTED = -170,
335 	/** Authentication failure*/
336 	RD_KAFKA_RESP_ERR__AUTHENTICATION = -169,
337 	/** No stored offset */
338 	RD_KAFKA_RESP_ERR__NO_OFFSET = -168,
339 	/** Outdated */
340 	RD_KAFKA_RESP_ERR__OUTDATED = -167,
341 	/** Timed out in queue */
342 	RD_KAFKA_RESP_ERR__TIMED_OUT_QUEUE = -166,
343         /** Feature not supported by broker */
344         RD_KAFKA_RESP_ERR__UNSUPPORTED_FEATURE = -165,
345         /** Awaiting cache update */
346         RD_KAFKA_RESP_ERR__WAIT_CACHE = -164,
347         /** Operation interrupted (e.g., due to yield)) */
348         RD_KAFKA_RESP_ERR__INTR = -163,
349         /** Key serialization error */
350         RD_KAFKA_RESP_ERR__KEY_SERIALIZATION = -162,
351         /** Value serialization error */
352         RD_KAFKA_RESP_ERR__VALUE_SERIALIZATION = -161,
353         /** Key deserialization error */
354         RD_KAFKA_RESP_ERR__KEY_DESERIALIZATION = -160,
355         /** Value deserialization error */
356         RD_KAFKA_RESP_ERR__VALUE_DESERIALIZATION = -159,
357         /** Partial response */
358         RD_KAFKA_RESP_ERR__PARTIAL = -158,
359         /** Modification attempted on read-only object */
360         RD_KAFKA_RESP_ERR__READ_ONLY = -157,
361         /** No such entry / item not found */
362         RD_KAFKA_RESP_ERR__NOENT = -156,
363         /** Read underflow */
364         RD_KAFKA_RESP_ERR__UNDERFLOW = -155,
365         /** Invalid type */
366         RD_KAFKA_RESP_ERR__INVALID_TYPE = -154,
367         /** Retry operation */
368         RD_KAFKA_RESP_ERR__RETRY = -153,
369         /** Purged in queue */
370         RD_KAFKA_RESP_ERR__PURGE_QUEUE = -152,
371         /** Purged in flight */
372         RD_KAFKA_RESP_ERR__PURGE_INFLIGHT = -151,
373         /** Fatal error: see rd_kafka_fatal_error() */
374         RD_KAFKA_RESP_ERR__FATAL = -150,
375         /** Inconsistent state */
376         RD_KAFKA_RESP_ERR__INCONSISTENT = -149,
377         /** Gap-less ordering would not be guaranteed if proceeding */
378         RD_KAFKA_RESP_ERR__GAPLESS_GUARANTEE = -148,
379         /** Maximum poll interval exceeded */
380         RD_KAFKA_RESP_ERR__MAX_POLL_EXCEEDED = -147,
381         /** Unknown broker */
382         RD_KAFKA_RESP_ERR__UNKNOWN_BROKER = -146,
383         /** Functionality not configured */
384         RD_KAFKA_RESP_ERR__NOT_CONFIGURED = -145,
385         /** Instance has been fenced */
386         RD_KAFKA_RESP_ERR__FENCED = -144,
387         /** Application generated error */
388         RD_KAFKA_RESP_ERR__APPLICATION = -143,
389         /** Assignment lost */
390         RD_KAFKA_RESP_ERR__ASSIGNMENT_LOST = -142,
391         /** No operation performed */
392         RD_KAFKA_RESP_ERR__NOOP = -141,
393         /** No offset to automatically reset to */
394         RD_KAFKA_RESP_ERR__AUTO_OFFSET_RESET = -140,
395 
396 	/** End internal error codes */
397 	RD_KAFKA_RESP_ERR__END = -100,
398 
399 	/* Kafka broker errors: */
400 	/** Unknown broker error */
401 	RD_KAFKA_RESP_ERR_UNKNOWN = -1,
402 	/** Success */
403 	RD_KAFKA_RESP_ERR_NO_ERROR = 0,
404 	/** Offset out of range */
405 	RD_KAFKA_RESP_ERR_OFFSET_OUT_OF_RANGE = 1,
406 	/** Invalid message */
407 	RD_KAFKA_RESP_ERR_INVALID_MSG = 2,
408 	/** Unknown topic or partition */
409 	RD_KAFKA_RESP_ERR_UNKNOWN_TOPIC_OR_PART = 3,
410 	/** Invalid message size */
411 	RD_KAFKA_RESP_ERR_INVALID_MSG_SIZE = 4,
412 	/** Leader not available */
413 	RD_KAFKA_RESP_ERR_LEADER_NOT_AVAILABLE = 5,
414 	/** Not leader for partition */
415 	RD_KAFKA_RESP_ERR_NOT_LEADER_FOR_PARTITION = 6,
416 	/** Request timed out */
417 	RD_KAFKA_RESP_ERR_REQUEST_TIMED_OUT = 7,
418 	/** Broker not available */
419 	RD_KAFKA_RESP_ERR_BROKER_NOT_AVAILABLE = 8,
420 	/** Replica not available */
421 	RD_KAFKA_RESP_ERR_REPLICA_NOT_AVAILABLE = 9,
422 	/** Message size too large */
423 	RD_KAFKA_RESP_ERR_MSG_SIZE_TOO_LARGE = 10,
424 	/** StaleControllerEpochCode */
425 	RD_KAFKA_RESP_ERR_STALE_CTRL_EPOCH = 11,
426 	/** Offset metadata string too large */
427 	RD_KAFKA_RESP_ERR_OFFSET_METADATA_TOO_LARGE = 12,
428 	/** Broker disconnected before response received */
429 	RD_KAFKA_RESP_ERR_NETWORK_EXCEPTION = 13,
430         /** Coordinator load in progress */
431         RD_KAFKA_RESP_ERR_COORDINATOR_LOAD_IN_PROGRESS = 14,
432         /** Group coordinator load in progress */
433 #define RD_KAFKA_RESP_ERR_GROUP_LOAD_IN_PROGRESS        \
434         RD_KAFKA_RESP_ERR_COORDINATOR_LOAD_IN_PROGRESS
435         /** Coordinator not available */
436         RD_KAFKA_RESP_ERR_COORDINATOR_NOT_AVAILABLE = 15,
437         /** Group coordinator not available */
438 #define RD_KAFKA_RESP_ERR_GROUP_COORDINATOR_NOT_AVAILABLE       \
439         RD_KAFKA_RESP_ERR_COORDINATOR_NOT_AVAILABLE
440         /** Not coordinator */
441         RD_KAFKA_RESP_ERR_NOT_COORDINATOR = 16,
442         /** Not coordinator for group */
443 #define RD_KAFKA_RESP_ERR_NOT_COORDINATOR_FOR_GROUP     \
444         RD_KAFKA_RESP_ERR_NOT_COORDINATOR
445 	/** Invalid topic */
446         RD_KAFKA_RESP_ERR_TOPIC_EXCEPTION = 17,
447 	/** Message batch larger than configured server segment size */
448         RD_KAFKA_RESP_ERR_RECORD_LIST_TOO_LARGE = 18,
449 	/** Not enough in-sync replicas */
450         RD_KAFKA_RESP_ERR_NOT_ENOUGH_REPLICAS = 19,
451 	/** Message(s) written to insufficient number of in-sync replicas */
452         RD_KAFKA_RESP_ERR_NOT_ENOUGH_REPLICAS_AFTER_APPEND = 20,
453 	/** Invalid required acks value */
454         RD_KAFKA_RESP_ERR_INVALID_REQUIRED_ACKS = 21,
455 	/** Specified group generation id is not valid */
456         RD_KAFKA_RESP_ERR_ILLEGAL_GENERATION = 22,
457 	/** Inconsistent group protocol */
458         RD_KAFKA_RESP_ERR_INCONSISTENT_GROUP_PROTOCOL = 23,
459 	/** Invalid group.id */
460 	RD_KAFKA_RESP_ERR_INVALID_GROUP_ID = 24,
461 	/** Unknown member */
462         RD_KAFKA_RESP_ERR_UNKNOWN_MEMBER_ID = 25,
463 	/** Invalid session timeout */
464         RD_KAFKA_RESP_ERR_INVALID_SESSION_TIMEOUT = 26,
465 	/** Group rebalance in progress */
466 	RD_KAFKA_RESP_ERR_REBALANCE_IN_PROGRESS = 27,
467 	/** Commit offset data size is not valid */
468         RD_KAFKA_RESP_ERR_INVALID_COMMIT_OFFSET_SIZE = 28,
469 	/** Topic authorization failed */
470         RD_KAFKA_RESP_ERR_TOPIC_AUTHORIZATION_FAILED = 29,
471 	/** Group authorization failed */
472 	RD_KAFKA_RESP_ERR_GROUP_AUTHORIZATION_FAILED = 30,
473 	/** Cluster authorization failed */
474 	RD_KAFKA_RESP_ERR_CLUSTER_AUTHORIZATION_FAILED = 31,
475 	/** Invalid timestamp */
476 	RD_KAFKA_RESP_ERR_INVALID_TIMESTAMP = 32,
477 	/** Unsupported SASL mechanism */
478 	RD_KAFKA_RESP_ERR_UNSUPPORTED_SASL_MECHANISM = 33,
479 	/** Illegal SASL state */
480 	RD_KAFKA_RESP_ERR_ILLEGAL_SASL_STATE = 34,
481 	/** Unuspported version */
482 	RD_KAFKA_RESP_ERR_UNSUPPORTED_VERSION = 35,
483 	/** Topic already exists */
484 	RD_KAFKA_RESP_ERR_TOPIC_ALREADY_EXISTS = 36,
485 	/** Invalid number of partitions */
486 	RD_KAFKA_RESP_ERR_INVALID_PARTITIONS = 37,
487 	/** Invalid replication factor */
488 	RD_KAFKA_RESP_ERR_INVALID_REPLICATION_FACTOR = 38,
489 	/** Invalid replica assignment */
490 	RD_KAFKA_RESP_ERR_INVALID_REPLICA_ASSIGNMENT = 39,
491 	/** Invalid config */
492 	RD_KAFKA_RESP_ERR_INVALID_CONFIG = 40,
493 	/** Not controller for cluster */
494 	RD_KAFKA_RESP_ERR_NOT_CONTROLLER = 41,
495 	/** Invalid request */
496 	RD_KAFKA_RESP_ERR_INVALID_REQUEST = 42,
497 	/** Message format on broker does not support request */
498 	RD_KAFKA_RESP_ERR_UNSUPPORTED_FOR_MESSAGE_FORMAT = 43,
499         /** Policy violation */
500         RD_KAFKA_RESP_ERR_POLICY_VIOLATION = 44,
501         /** Broker received an out of order sequence number */
502         RD_KAFKA_RESP_ERR_OUT_OF_ORDER_SEQUENCE_NUMBER = 45,
503         /** Broker received a duplicate sequence number */
504         RD_KAFKA_RESP_ERR_DUPLICATE_SEQUENCE_NUMBER = 46,
505         /** Producer attempted an operation with an old epoch */
506         RD_KAFKA_RESP_ERR_INVALID_PRODUCER_EPOCH = 47,
507         /** Producer attempted a transactional operation in an invalid state */
508         RD_KAFKA_RESP_ERR_INVALID_TXN_STATE = 48,
509         /** Producer attempted to use a producer id which is not
510          *  currently assigned to its transactional id */
511         RD_KAFKA_RESP_ERR_INVALID_PRODUCER_ID_MAPPING = 49,
512         /** Transaction timeout is larger than the maximum
513          *  value allowed by the broker's max.transaction.timeout.ms */
514         RD_KAFKA_RESP_ERR_INVALID_TRANSACTION_TIMEOUT = 50,
515         /** Producer attempted to update a transaction while another
516          *  concurrent operation on the same transaction was ongoing */
517         RD_KAFKA_RESP_ERR_CONCURRENT_TRANSACTIONS = 51,
518         /** Indicates that the transaction coordinator sending a
519          *  WriteTxnMarker is no longer the current coordinator for a
520          *  given producer */
521         RD_KAFKA_RESP_ERR_TRANSACTION_COORDINATOR_FENCED = 52,
522         /** Transactional Id authorization failed */
523         RD_KAFKA_RESP_ERR_TRANSACTIONAL_ID_AUTHORIZATION_FAILED = 53,
524         /** Security features are disabled */
525         RD_KAFKA_RESP_ERR_SECURITY_DISABLED = 54,
526         /** Operation not attempted */
527         RD_KAFKA_RESP_ERR_OPERATION_NOT_ATTEMPTED = 55,
528         /** Disk error when trying to access log file on the disk */
529         RD_KAFKA_RESP_ERR_KAFKA_STORAGE_ERROR = 56,
530         /** The user-specified log directory is not found in the broker config */
531         RD_KAFKA_RESP_ERR_LOG_DIR_NOT_FOUND = 57,
532         /** SASL Authentication failed */
533         RD_KAFKA_RESP_ERR_SASL_AUTHENTICATION_FAILED = 58,
534         /** Unknown Producer Id */
535         RD_KAFKA_RESP_ERR_UNKNOWN_PRODUCER_ID = 59,
536         /** Partition reassignment is in progress */
537         RD_KAFKA_RESP_ERR_REASSIGNMENT_IN_PROGRESS = 60,
538         /** Delegation Token feature is not enabled */
539         RD_KAFKA_RESP_ERR_DELEGATION_TOKEN_AUTH_DISABLED = 61,
540         /** Delegation Token is not found on server */
541         RD_KAFKA_RESP_ERR_DELEGATION_TOKEN_NOT_FOUND = 62,
542         /** Specified Principal is not valid Owner/Renewer */
543         RD_KAFKA_RESP_ERR_DELEGATION_TOKEN_OWNER_MISMATCH = 63,
544         /** Delegation Token requests are not allowed on this connection */
545         RD_KAFKA_RESP_ERR_DELEGATION_TOKEN_REQUEST_NOT_ALLOWED = 64,
546         /** Delegation Token authorization failed */
547         RD_KAFKA_RESP_ERR_DELEGATION_TOKEN_AUTHORIZATION_FAILED = 65,
548         /** Delegation Token is expired */
549         RD_KAFKA_RESP_ERR_DELEGATION_TOKEN_EXPIRED = 66,
550         /** Supplied principalType is not supported */
551         RD_KAFKA_RESP_ERR_INVALID_PRINCIPAL_TYPE = 67,
552         /** The group is not empty */
553         RD_KAFKA_RESP_ERR_NON_EMPTY_GROUP = 68,
554         /** The group id does not exist */
555         RD_KAFKA_RESP_ERR_GROUP_ID_NOT_FOUND = 69,
556         /** The fetch session ID was not found */
557         RD_KAFKA_RESP_ERR_FETCH_SESSION_ID_NOT_FOUND = 70,
558         /** The fetch session epoch is invalid */
559         RD_KAFKA_RESP_ERR_INVALID_FETCH_SESSION_EPOCH = 71,
560         /** No matching listener */
561         RD_KAFKA_RESP_ERR_LISTENER_NOT_FOUND = 72,
562         /** Topic deletion is disabled */
563         RD_KAFKA_RESP_ERR_TOPIC_DELETION_DISABLED = 73,
564         /** Leader epoch is older than broker epoch */
565         RD_KAFKA_RESP_ERR_FENCED_LEADER_EPOCH = 74,
566         /** Leader epoch is newer than broker epoch */
567         RD_KAFKA_RESP_ERR_UNKNOWN_LEADER_EPOCH = 75,
568         /** Unsupported compression type */
569         RD_KAFKA_RESP_ERR_UNSUPPORTED_COMPRESSION_TYPE = 76,
570         /** Broker epoch has changed */
571         RD_KAFKA_RESP_ERR_STALE_BROKER_EPOCH = 77,
572         /** Leader high watermark is not caught up */
573         RD_KAFKA_RESP_ERR_OFFSET_NOT_AVAILABLE = 78,
574         /** Group member needs a valid member ID */
575         RD_KAFKA_RESP_ERR_MEMBER_ID_REQUIRED = 79,
576         /** Preferred leader was not available */
577         RD_KAFKA_RESP_ERR_PREFERRED_LEADER_NOT_AVAILABLE = 80,
578         /** Consumer group has reached maximum size */
579         RD_KAFKA_RESP_ERR_GROUP_MAX_SIZE_REACHED = 81,
580         /** Static consumer fenced by other consumer with same
581          *  group.instance.id. */
582         RD_KAFKA_RESP_ERR_FENCED_INSTANCE_ID = 82,
583         /** Eligible partition leaders are not available */
584         RD_KAFKA_RESP_ERR_ELIGIBLE_LEADERS_NOT_AVAILABLE = 83,
585         /** Leader election not needed for topic partition */
586         RD_KAFKA_RESP_ERR_ELECTION_NOT_NEEDED = 84,
587         /** No partition reassignment is in progress */
588         RD_KAFKA_RESP_ERR_NO_REASSIGNMENT_IN_PROGRESS = 85,
589         /** Deleting offsets of a topic while the consumer group is
590          *  subscribed to it */
591         RD_KAFKA_RESP_ERR_GROUP_SUBSCRIBED_TO_TOPIC = 86,
592         /** Broker failed to validate record */
593         RD_KAFKA_RESP_ERR_INVALID_RECORD = 87,
594         /** There are unstable offsets that need to be cleared */
595         RD_KAFKA_RESP_ERR_UNSTABLE_OFFSET_COMMIT = 88,
596         /** Throttling quota has been exceeded */
597         RD_KAFKA_RESP_ERR_THROTTLING_QUOTA_EXCEEDED = 89,
598         /** There is a newer producer with the same transactionalId
599          *  which fences the current one */
600         RD_KAFKA_RESP_ERR_PRODUCER_FENCED = 90,
601         /** Request illegally referred to resource that does not exist */
602         RD_KAFKA_RESP_ERR_RESOURCE_NOT_FOUND = 91,
603         /** Request illegally referred to the same resource twice */
604         RD_KAFKA_RESP_ERR_DUPLICATE_RESOURCE = 92,
605         /** Requested credential would not meet criteria for acceptability */
606         RD_KAFKA_RESP_ERR_UNACCEPTABLE_CREDENTIAL = 93,
607         /** Indicates that the either the sender or recipient of a
608          *  voter-only request is not one of the expected voters */
609         RD_KAFKA_RESP_ERR_INCONSISTENT_VOTER_SET = 94,
610         /** Invalid update version */
611         RD_KAFKA_RESP_ERR_INVALID_UPDATE_VERSION = 95,
612         /** Unable to update finalized features due to server error */
613         RD_KAFKA_RESP_ERR_FEATURE_UPDATE_FAILED = 96,
614         /** Request principal deserialization failed during forwarding */
615         RD_KAFKA_RESP_ERR_PRINCIPAL_DESERIALIZATION_FAILURE = 97,
616 
617         RD_KAFKA_RESP_ERR_END_ALL,
618 } rd_kafka_resp_err_t;
619 
620 
621 /**
622  * @brief Error code value, name and description.
623  *        Typically for use with language bindings to automatically expose
624  *        the full set of librdkafka error codes.
625  */
626 struct rd_kafka_err_desc {
627 	rd_kafka_resp_err_t code;/**< Error code */
628 	const char *name;      /**< Error name, same as code enum sans prefix */
629 	const char *desc;      /**< Human readable error description. */
630 };
631 
632 
633 /**
634  * @brief Returns the full list of error codes.
635  */
636 RD_EXPORT
637 void rd_kafka_get_err_descs (const struct rd_kafka_err_desc **errdescs,
638 			     size_t *cntp);
639 
640 
641 
642 
643 /**
644  * @brief Returns a human readable representation of a kafka error.
645  *
646  * @param err Error code to translate
647  */
648 RD_EXPORT
649 const char *rd_kafka_err2str (rd_kafka_resp_err_t err);
650 
651 
652 
653 /**
654  * @brief Returns the error code name (enum name).
655  *
656  * @param err Error code to translate
657  */
658 RD_EXPORT
659 const char *rd_kafka_err2name (rd_kafka_resp_err_t err);
660 
661 
662 /**
663  * @brief Returns the last error code generated by a legacy API call
664  *        in the current thread.
665  *
666  * The legacy APIs are the ones using errno to propagate error value, namely:
667  *  - rd_kafka_topic_new()
668  *  - rd_kafka_consume_start()
669  *  - rd_kafka_consume_stop()
670  *  - rd_kafka_consume()
671  *  - rd_kafka_consume_batch()
672  *  - rd_kafka_consume_callback()
673  *  - rd_kafka_consume_queue()
674  *  - rd_kafka_produce()
675  *
676  * The main use for this function is to avoid converting system \p errno
677  * values to rd_kafka_resp_err_t codes for legacy APIs.
678  *
679  * @remark The last error is stored per-thread, if multiple rd_kafka_t handles
680  *         are used in the same application thread the developer needs to
681  *         make sure rd_kafka_last_error() is called immediately after
682  *         a failed API call.
683  *
684  * @remark errno propagation from librdkafka is not safe on Windows
685  *         and should not be used, use rd_kafka_last_error() instead.
686  */
687 RD_EXPORT
688 rd_kafka_resp_err_t rd_kafka_last_error (void);
689 
690 
691 /**
692  * @brief Converts the system errno value \p errnox to a rd_kafka_resp_err_t
693  *        error code upon failure from the following functions:
694  *  - rd_kafka_topic_new()
695  *  - rd_kafka_consume_start()
696  *  - rd_kafka_consume_stop()
697  *  - rd_kafka_consume()
698  *  - rd_kafka_consume_batch()
699  *  - rd_kafka_consume_callback()
700  *  - rd_kafka_consume_queue()
701  *  - rd_kafka_produce()
702  *
703  * @param errnox  System errno value to convert
704  *
705  * @returns Appropriate error code for \p errnox
706  *
707  * @remark A better alternative is to call rd_kafka_last_error() immediately
708  *         after any of the above functions return -1 or NULL.
709  *
710  * @deprecated Use rd_kafka_last_error() to retrieve the last error code
711  *             set by the legacy librdkafka APIs.
712  *
713  * @sa rd_kafka_last_error()
714  */
715 RD_EXPORT RD_DEPRECATED
716 rd_kafka_resp_err_t rd_kafka_errno2err(int errnox);
717 
718 
719 /**
720  * @brief Returns the thread-local system errno
721  *
722  * On most platforms this is the same as \p errno but in case of different
723  * runtimes between library and application (e.g., Windows static DLLs)
724  * this provides a means for exposing the errno librdkafka uses.
725  *
726  * @remark The value is local to the current calling thread.
727  *
728  * @deprecated Use rd_kafka_last_error() to retrieve the last error code
729  *             set by the legacy librdkafka APIs.
730  */
731 RD_EXPORT RD_DEPRECATED
732 int rd_kafka_errno (void);
733 
734 
735 
736 
737 /**
738  * @brief Returns the first fatal error set on this client instance,
739  *        or RD_KAFKA_RESP_ERR_NO_ERROR if no fatal error has occurred.
740  *
741  * This function is to be used with the Idempotent Producer and \c error_cb
742  * to detect fatal errors.
743  *
744  * Generally all errors raised by \c error_cb are to be considered
745  * informational and temporary, the client will try to recover from all
746  * errors in a graceful fashion (by retrying, etc).
747  *
748  * However, some errors should logically be considered fatal to retain
749  * consistency; in particular a set of errors that may occur when using the
750  * Idempotent Producer and the in-order or exactly-once producer guarantees
751  * can't be satisfied.
752  *
753  * @param rk Client instance.
754  * @param errstr A human readable error string (nul-terminated) is written to
755  *               this location that must be of at least \p errstr_size bytes.
756  *               The \p errstr is only written to if there is a fatal error.
757  * @param errstr_size Writable size in \p errstr.
758  *
759  *
760  * @returns RD_KAFKA_RESP_ERR_NO_ERROR if no fatal error has been raised, else
761  *          any other error code.
762  */
763 RD_EXPORT
764 rd_kafka_resp_err_t rd_kafka_fatal_error (rd_kafka_t *rk,
765                                           char *errstr, size_t errstr_size);
766 
767 
768 /**
769  * @brief Trigger a fatal error for testing purposes.
770  *
771  * Since there is no practical way to trigger real fatal errors in the
772  * idempotent producer, this method allows an application to trigger
773  * fabricated fatal errors in tests to check its error handling code.
774  *
775  * @param rk Client instance.
776  * @param err The underlying error code.
777  * @param reason A human readable error reason.
778  *               Will be prefixed with "test_fatal_error: " to differentiate
779  *               from real fatal errors.
780  *
781  * @returns RD_KAFKA_RESP_ERR_NO_ERROR if a fatal error was triggered, or
782  *          RD_KAFKA_RESP_ERR__PREV_IN_PROGRESS if a previous fatal error
783  *          has already been triggered.
784  */
785 RD_EXPORT rd_kafka_resp_err_t
786 rd_kafka_test_fatal_error (rd_kafka_t *rk, rd_kafka_resp_err_t err,
787                            const char *reason);
788 
789 
790 /**
791  * @returns the error code for \p error or RD_KAFKA_RESP_ERR_NO_ERROR if
792  *          \p error is NULL.
793  */
794 RD_EXPORT
795 rd_kafka_resp_err_t rd_kafka_error_code (const rd_kafka_error_t *error);
796 
797 /**
798  * @returns the error code name for \p error, e.g, "ERR_UNKNOWN_MEMBER_ID",
799  *          or an empty string if \p error is NULL.
800  *
801  * @remark The lifetime of the returned pointer is the same as the error object.
802  *
803  * @sa rd_kafka_err2name()
804  */
805 RD_EXPORT
806 const char *rd_kafka_error_name (const rd_kafka_error_t *error);
807 
808 /**
809  * @returns a human readable error string for \p error,
810  *          or an empty string if \p error is NULL.
811  *
812  * @remark The lifetime of the returned pointer is the same as the error object.
813  */
814 RD_EXPORT
815 const char *rd_kafka_error_string (const rd_kafka_error_t *error);
816 
817 
818 /**
819  * @returns 1 if the error is a fatal error, indicating that the client
820  *          instance is no longer usable, else 0 (also if \p error is NULL).
821  */
822 RD_EXPORT
823 int rd_kafka_error_is_fatal (const rd_kafka_error_t *error);
824 
825 
826 /**
827  * @returns 1 if the operation may be retried,
828  *          else 0 (also if \p error is NULL).
829  */
830 RD_EXPORT
831 int rd_kafka_error_is_retriable (const rd_kafka_error_t *error);
832 
833 
834 /**
835  * @returns 1 if the error is an abortable transaction error in which case
836  *          the application must call rd_kafka_abort_transaction() and
837  *          start a new transaction with rd_kafka_begin_transaction() if it
838  *          wishes to proceed with transactions.
839  *          Else returns 0 (also if \p error is NULL).
840  *
841  * @remark The return value of this method is only valid for errors returned
842  *         by the transactional API.
843  */
844 RD_EXPORT
845 int rd_kafka_error_txn_requires_abort (const rd_kafka_error_t *error);
846 
847 /**
848  * @brief Free and destroy an error object.
849  *
850  * @remark As a conveniance it is permitted to pass a NULL \p error.
851  */
852 RD_EXPORT
853 void rd_kafka_error_destroy (rd_kafka_error_t *error);
854 
855 
856 /**
857  * @brief Create a new error object with error \p code and optional
858  *        human readable error string in \p fmt.
859  *
860  * This method is mainly to be used for mocking errors in application test code.
861  *
862  * The returned object must be destroyed with rd_kafka_error_destroy().
863  */
864 RD_EXPORT
865 rd_kafka_error_t *rd_kafka_error_new (rd_kafka_resp_err_t code,
866                                       const char *fmt, ...)
867         RD_FORMAT(printf, 2, 3);
868 
869 
870 /**
871  * @brief Topic+Partition place holder
872  *
873  * Generic place holder for a Topic+Partition and its related information
874  * used for multiple purposes:
875  *   - consumer offset (see rd_kafka_commit(), et.al.)
876  *   - group rebalancing callback (rd_kafka_conf_set_rebalance_cb())
877  *   - offset commit result callback (rd_kafka_conf_set_offset_commit_cb())
878  */
879 
880 /**
881  * @brief Generic place holder for a specific Topic+Partition.
882  *
883  * @sa rd_kafka_topic_partition_list_new()
884  */
885 typedef struct rd_kafka_topic_partition_s {
886         char        *topic;             /**< Topic name */
887         int32_t      partition;         /**< Partition */
888 	int64_t      offset;            /**< Offset */
889         void        *metadata;          /**< Metadata */
890         size_t       metadata_size;     /**< Metadata size */
891         void        *opaque;            /**< Opaque value for application use */
892         rd_kafka_resp_err_t err;        /**< Error code, depending on use. */
893         void       *_private;           /**< INTERNAL USE ONLY,
894                                          *   INITIALIZE TO ZERO, DO NOT TOUCH */
895 } rd_kafka_topic_partition_t;
896 
897 
898 /**
899  * @brief Destroy a rd_kafka_topic_partition_t.
900  * @remark This must not be called for elements in a topic partition list.
901  */
902 RD_EXPORT
903 void rd_kafka_topic_partition_destroy (rd_kafka_topic_partition_t *rktpar);
904 
905 
906 /**
907  * @brief A growable list of Topic+Partitions.
908  *
909  */
910 typedef struct rd_kafka_topic_partition_list_s {
911         int cnt;               /**< Current number of elements */
912         int size;              /**< Current allocated size */
913         rd_kafka_topic_partition_t *elems; /**< Element array[] */
914 } rd_kafka_topic_partition_list_t;
915 
916 
917 /**
918  * @brief Create a new list/vector Topic+Partition container.
919  *
920  * @param size  Initial allocated size used when the expected number of
921  *              elements is known or can be estimated.
922  *              Avoids reallocation and possibly relocation of the
923  *              elems array.
924  *
925  * @returns A newly allocated Topic+Partition list.
926  *
927  * @remark Use rd_kafka_topic_partition_list_destroy() to free all resources
928  *         in use by a list and the list itself.
929  * @sa     rd_kafka_topic_partition_list_add()
930  */
931 RD_EXPORT
932 rd_kafka_topic_partition_list_t *rd_kafka_topic_partition_list_new (int size);
933 
934 
935 /**
936  * @brief Free all resources used by the list and the list itself.
937  */
938 RD_EXPORT
939 void
940 rd_kafka_topic_partition_list_destroy (rd_kafka_topic_partition_list_t *rkparlist);
941 
942 /**
943  * @brief Add topic+partition to list
944  *
945  * @param rktparlist List to extend
946  * @param topic      Topic name (copied)
947  * @param partition  Partition id
948  *
949  * @returns The object which can be used to fill in additionals fields.
950  */
951 RD_EXPORT
952 rd_kafka_topic_partition_t *
953 rd_kafka_topic_partition_list_add (rd_kafka_topic_partition_list_t *rktparlist,
954                                    const char *topic, int32_t partition);
955 
956 
957 /**
958  * @brief Add range of partitions from \p start to \p stop inclusive.
959  *
960  * @param rktparlist List to extend
961  * @param topic      Topic name (copied)
962  * @param start      Start partition of range
963  * @param stop       Last partition of range (inclusive)
964  */
965 RD_EXPORT
966 void
967 rd_kafka_topic_partition_list_add_range (rd_kafka_topic_partition_list_t
968                                          *rktparlist,
969                                          const char *topic,
970                                          int32_t start, int32_t stop);
971 
972 
973 
974 /**
975  * @brief Delete partition from list.
976  *
977  * @param rktparlist List to modify
978  * @param topic      Topic name to match
979  * @param partition  Partition to match
980  *
981  * @returns 1 if partition was found (and removed), else 0.
982  *
983  * @remark Any held indices to elems[] are unusable after this call returns 1.
984  */
985 RD_EXPORT
986 int
987 rd_kafka_topic_partition_list_del (rd_kafka_topic_partition_list_t *rktparlist,
988 				   const char *topic, int32_t partition);
989 
990 
991 /**
992  * @brief Delete partition from list by elems[] index.
993  *
994  * @returns 1 if partition was found (and removed), else 0.
995  *
996  * @sa rd_kafka_topic_partition_list_del()
997  */
998 RD_EXPORT
999 int
1000 rd_kafka_topic_partition_list_del_by_idx (
1001 	rd_kafka_topic_partition_list_t *rktparlist,
1002 	int idx);
1003 
1004 
1005 /**
1006  * @brief Make a copy of an existing list.
1007  *
1008  * @param src   The existing list to copy.
1009  *
1010  * @returns A new list fully populated to be identical to \p src
1011  */
1012 RD_EXPORT
1013 rd_kafka_topic_partition_list_t *
1014 rd_kafka_topic_partition_list_copy (const rd_kafka_topic_partition_list_t *src);
1015 
1016 
1017 
1018 
1019 /**
1020  * @brief Set offset to \p offset for \p topic and \p partition
1021  *
1022  * @returns RD_KAFKA_RESP_ERR_NO_ERROR on success or
1023  *          RD_KAFKA_RESP_ERR__UNKNOWN_PARTITION if \p partition was not found
1024  *          in the list.
1025  */
1026 RD_EXPORT
1027 rd_kafka_resp_err_t rd_kafka_topic_partition_list_set_offset (
1028 	rd_kafka_topic_partition_list_t *rktparlist,
1029 	const char *topic, int32_t partition, int64_t offset);
1030 
1031 
1032 
1033 /**
1034  * @brief Find element by \p topic and \p partition.
1035  *
1036  * @returns a pointer to the first matching element, or NULL if not found.
1037  */
1038 RD_EXPORT
1039 rd_kafka_topic_partition_t *
1040 rd_kafka_topic_partition_list_find (
1041         const rd_kafka_topic_partition_list_t *rktparlist,
1042         const char *topic, int32_t partition);
1043 
1044 
1045 /**
1046  * @brief Sort list using comparator \p cmp.
1047  *
1048  * If \p cmp is NULL the default comparator will be used that
1049  * sorts by ascending topic name and partition.
1050  *
1051  * \p cmp_opaque is provided as the \p cmp_opaque argument to \p cmp.
1052  *
1053  */
1054 RD_EXPORT void
1055 rd_kafka_topic_partition_list_sort (rd_kafka_topic_partition_list_t *rktparlist,
1056                                     int (*cmp) (const void *a, const void *b,
1057                                                 void *cmp_opaque),
1058                                     void *cmp_opaque);
1059 
1060 
1061 /**@}*/
1062 
1063 
1064 
1065 /**
1066  * @name Var-arg tag types
1067  * @{
1068  *
1069  */
1070 
1071 /**
1072  * @enum rd_kafka_vtype_t
1073  *
1074  * @brief Var-arg tag types
1075  *
1076  * @sa rd_kafka_producev()
1077  */
1078 typedef enum rd_kafka_vtype_t {
1079         RD_KAFKA_VTYPE_END,       /**< va-arg sentinel */
1080         RD_KAFKA_VTYPE_TOPIC,     /**< (const char *) Topic name */
1081         RD_KAFKA_VTYPE_RKT,       /**< (rd_kafka_topic_t *) Topic handle */
1082         RD_KAFKA_VTYPE_PARTITION, /**< (int32_t) Partition */
1083         RD_KAFKA_VTYPE_VALUE,     /**< (void *, size_t) Message value (payload)*/
1084         RD_KAFKA_VTYPE_KEY,       /**< (void *, size_t) Message key */
1085         RD_KAFKA_VTYPE_OPAQUE,    /**< (void *) Per-message application opaque
1086                                    *            value. This is the same as
1087                                    *            the _private field in
1088                                    *            rd_kafka_message_t, also known
1089                                    *            as the msg_opaque. */
1090         RD_KAFKA_VTYPE_MSGFLAGS,  /**< (int) RD_KAFKA_MSG_F_.. flags */
1091         RD_KAFKA_VTYPE_TIMESTAMP, /**< (int64_t) Milliseconds since epoch UTC */
1092         RD_KAFKA_VTYPE_HEADER,    /**< (const char *, const void *, ssize_t)
1093                                    *   Message Header */
1094         RD_KAFKA_VTYPE_HEADERS,   /**< (rd_kafka_headers_t *) Headers list */
1095 } rd_kafka_vtype_t;
1096 
1097 
1098 /**
1099  * @brief VTYPE + argument container for use with rd_kafka_produce_va()
1100  *
1101  * See RD_KAFKA_V_..() macros below for which union field corresponds
1102  * to which RD_KAFKA_VTYPE_...
1103  */
1104 typedef struct rd_kafka_vu_s {
1105         rd_kafka_vtype_t vtype;           /**< RD_KAFKA_VTYPE_.. */
1106         /** Value union, see RD_KAFKA_V_.. macros for which field to use. */
1107         union {
1108                 const char *cstr;
1109                 rd_kafka_topic_t *rkt;
1110                 int i;
1111                 int32_t i32;
1112                 int64_t i64;
1113                 struct {
1114                         void *ptr;
1115                         size_t size;
1116                 } mem;
1117                 struct {
1118                         const char *name;
1119                         const void *val;
1120                         ssize_t size;
1121                 } header;
1122                 rd_kafka_headers_t *headers;
1123                 void *ptr;
1124                 char _pad[64];  /**< Padding size for future-proofness */
1125         } u;
1126 } rd_kafka_vu_t;
1127 
1128 /**
1129  * @brief Convenience macros for rd_kafka_vtype_t that takes the
1130  *        correct arguments for each vtype.
1131  */
1132 
1133 /*!
1134  * va-arg end sentinel used to terminate the variable argument list
1135  */
1136 #define RD_KAFKA_V_END RD_KAFKA_VTYPE_END
1137 
1138 /*!
1139  * Topic name (const char *)
1140  *
1141  * rd_kafka_vu_t field: u.cstr
1142  */
1143 #define RD_KAFKA_V_TOPIC(topic)                                         \
1144         _LRK_TYPECHECK(RD_KAFKA_VTYPE_TOPIC, const char *, topic),      \
1145         (const char *)topic
1146 /*!
1147  * Topic object (rd_kafka_topic_t *)
1148  *
1149  * rd_kafka_vu_t field: u.rkt
1150  */
1151 #define RD_KAFKA_V_RKT(rkt)                                             \
1152         _LRK_TYPECHECK(RD_KAFKA_VTYPE_RKT, rd_kafka_topic_t *, rkt),    \
1153         (rd_kafka_topic_t *)rkt
1154 /*!
1155  * Partition (int32_t)
1156  *
1157  * rd_kafka_vu_t field: u.i32
1158  */
1159 #define RD_KAFKA_V_PARTITION(partition)                                 \
1160         _LRK_TYPECHECK(RD_KAFKA_VTYPE_PARTITION, int32_t, partition),   \
1161         (int32_t)partition
1162 /*!
1163  * Message value/payload pointer and length (void *, size_t)
1164  *
1165  * rd_kafka_vu_t fields: u.mem.ptr, u.mem.size
1166  */
1167 #define RD_KAFKA_V_VALUE(VALUE,LEN)                                     \
1168         _LRK_TYPECHECK2(RD_KAFKA_VTYPE_VALUE, void *, VALUE, size_t, LEN), \
1169         (void *)VALUE, (size_t)LEN
1170 /*!
1171  * Message key pointer and length (const void *, size_t)
1172  *
1173  * rd_kafka_vu_t field: u.mem.ptr, rd_kafka_vu.t.u.mem.size
1174  */
1175 #define RD_KAFKA_V_KEY(KEY,LEN)                                         \
1176         _LRK_TYPECHECK2(RD_KAFKA_VTYPE_KEY, const void *, KEY, size_t, LEN), \
1177         (void *)KEY, (size_t)LEN
1178 /*!
1179  * Message opaque pointer (void *)
1180  * Same as \c msg_opaque, \c produce(.., msg_opaque),
1181  * and \c rkmessage->_private .
1182  *
1183  * rd_kafka_vu_t field: u.ptr
1184  */
1185 #define RD_KAFKA_V_OPAQUE(msg_opaque)                                   \
1186         _LRK_TYPECHECK(RD_KAFKA_VTYPE_OPAQUE, void *, msg_opaque),      \
1187         (void *)msg_opaque
1188 /*!
1189  * Message flags (int)
1190  * @sa RD_KAFKA_MSG_F_COPY, et.al.
1191  *
1192  * rd_kafka_vu_t field: u.i
1193  */
1194 #define RD_KAFKA_V_MSGFLAGS(msgflags)                                 \
1195         _LRK_TYPECHECK(RD_KAFKA_VTYPE_MSGFLAGS, int, msgflags),       \
1196         (int)msgflags
1197 /*!
1198  * Timestamp in milliseconds since epoch UTC (int64_t).
1199  * A value of 0 will use the current wall-clock time.
1200  *
1201  * rd_kafka_vu_t field: u.i64
1202  */
1203 #define RD_KAFKA_V_TIMESTAMP(timestamp)                                 \
1204         _LRK_TYPECHECK(RD_KAFKA_VTYPE_TIMESTAMP, int64_t, timestamp),   \
1205         (int64_t)timestamp
1206 /*!
1207  * Add Message Header (const char *NAME, const void *VALUE, ssize_t LEN).
1208  * @sa rd_kafka_header_add()
1209  * @remark RD_KAFKA_V_HEADER() and RD_KAFKA_V_HEADERS() MUST NOT be mixed
1210  *         in the same call to producev().
1211  *
1212  * rd_kafka_vu_t fields: u.header.name, u.header.val, u.header.size
1213  */
1214 #define RD_KAFKA_V_HEADER(NAME,VALUE,LEN)                               \
1215         _LRK_TYPECHECK3(RD_KAFKA_VTYPE_HEADER, const char *, NAME,      \
1216                         const void *, VALUE, ssize_t, LEN),             \
1217                 (const char *)NAME, (const void *)VALUE, (ssize_t)LEN
1218 
1219 /*!
1220  * Message Headers list (rd_kafka_headers_t *).
1221  * The message object will assume ownership of the headers (unless producev()
1222  * fails).
1223  * Any existing headers will be replaced.
1224  * @sa rd_kafka_message_set_headers()
1225  * @remark RD_KAFKA_V_HEADER() and RD_KAFKA_V_HEADERS() MUST NOT be mixed
1226  *         in the same call to producev().
1227  *
1228  * rd_kafka_vu_t fields: u.headers
1229  */
1230 #define RD_KAFKA_V_HEADERS(HDRS)                                        \
1231         _LRK_TYPECHECK(RD_KAFKA_VTYPE_HEADERS, rd_kafka_headers_t *, HDRS), \
1232                 (rd_kafka_headers_t *)HDRS
1233 
1234 
1235 /**@}*/
1236 
1237 
1238 /**
1239  * @name Message headers
1240  * @{
1241  *
1242  * @brief Message headers consist of a list of (string key, binary value) pairs.
1243  *        Duplicate keys are supported and the order in which keys were
1244  *        added are retained.
1245  *
1246  *        Header values are considered binary and may have three types of
1247  *        value:
1248  *          - proper value with size > 0 and a valid pointer
1249  *          - empty value with size = 0 and any non-NULL pointer
1250  *          - null value with size = 0 and a NULL pointer
1251  *
1252  *        Headers require Apache Kafka broker version v0.11.0.0 or later.
1253  *
1254  *        Header operations are O(n).
1255  */
1256 
1257 
1258 /**
1259  * @brief Create a new headers list.
1260  *
1261  * @param initial_count Preallocate space for this number of headers.
1262  *                      Any number of headers may be added, updated and
1263  *                      removed regardless of the initial count.
1264  */
1265 RD_EXPORT rd_kafka_headers_t *rd_kafka_headers_new (size_t initial_count);
1266 
1267 /**
1268  * @brief Destroy the headers list. The object and any returned value pointers
1269  *        are not usable after this call.
1270  */
1271 RD_EXPORT void rd_kafka_headers_destroy (rd_kafka_headers_t *hdrs);
1272 
1273 /**
1274  * @brief Make a copy of headers list \p src.
1275  */
1276 RD_EXPORT rd_kafka_headers_t *
1277 rd_kafka_headers_copy (const rd_kafka_headers_t *src);
1278 
1279 /**
1280  * @brief Add header with name \p name and value \p val (copied) of size
1281  *        \p size (not including null-terminator).
1282  *
1283  * @param hdrs       Headers list.
1284  * @param name       Header name.
1285  * @param name_size  Header name size (not including the null-terminator).
1286  *                   If -1 the \p name length is automatically acquired using
1287  *                   strlen().
1288  * @param value      Pointer to header value, or NULL (set size to 0 or -1).
1289  * @param value_size Size of header value. If -1 the \p value is assumed to be a
1290  *                   null-terminated string and the length is automatically
1291  *                   acquired using strlen().
1292  *
1293  * @returns RD_KAFKA_RESP_ERR__READ_ONLY if the headers are read-only,
1294  *          else RD_KAFKA_RESP_ERR_NO_ERROR.
1295  */
1296 RD_EXPORT rd_kafka_resp_err_t
1297 rd_kafka_header_add (rd_kafka_headers_t *hdrs,
1298                      const char *name, ssize_t name_size,
1299                      const void *value, ssize_t value_size);
1300 
1301 /**
1302  * @brief Remove all headers for the given key (if any).
1303  *
1304  * @returns RD_KAFKA_RESP_ERR__READ_ONLY if the headers are read-only,
1305  *          RD_KAFKA_RESP_ERR__NOENT if no matching headers were found,
1306  *          else RD_KAFKA_RESP_ERR_NO_ERROR if headers were removed.
1307  */
1308 RD_EXPORT rd_kafka_resp_err_t
1309 rd_kafka_header_remove (rd_kafka_headers_t *hdrs, const char *name);
1310 
1311 
1312 /**
1313  * @brief Find last header in list \p hdrs matching \p name.
1314  *
1315  * @param hdrs   Headers list.
1316  * @param name   Header to find (last match).
1317  * @param valuep (out) Set to a (null-terminated) const pointer to the value
1318  *               (may be NULL).
1319  * @param sizep  (out) Set to the value's size (not including null-terminator).
1320  *
1321  * @returns RD_KAFKA_RESP_ERR_NO_ERROR if an entry was found, else
1322  *          RD_KAFKA_RESP_ERR__NOENT.
1323  *
1324  * @remark The returned pointer in \p valuep includes a trailing null-terminator
1325  *         that is not accounted for in \p sizep.
1326  * @remark The returned pointer is only valid as long as the headers list and
1327  *         the header item is valid.
1328  */
1329 RD_EXPORT rd_kafka_resp_err_t
1330 rd_kafka_header_get_last (const rd_kafka_headers_t *hdrs,
1331                           const char *name, const void **valuep, size_t *sizep);
1332 
1333 /**
1334  * @brief Iterator for headers matching \p name.
1335  *
1336  *        Same semantics as rd_kafka_header_get_last()
1337  *
1338  * @param hdrs   Headers to iterate.
1339  * @param idx    Iterator index, start at 0 and increment by one for each call
1340  *               as long as RD_KAFKA_RESP_ERR_NO_ERROR is returned.
1341  * @param name   Header name to match.
1342  * @param valuep (out) Set to a (null-terminated) const pointer to the value
1343  *               (may be NULL).
1344  * @param sizep  (out) Set to the value's size (not including null-terminator).
1345  */
1346 RD_EXPORT rd_kafka_resp_err_t
1347 rd_kafka_header_get (const rd_kafka_headers_t *hdrs, size_t idx,
1348                      const char *name, const void **valuep, size_t *sizep);
1349 
1350 
1351 /**
1352  * @brief Iterator for all headers.
1353  *
1354  *        Same semantics as rd_kafka_header_get()
1355  *
1356  * @sa rd_kafka_header_get()
1357  */
1358 RD_EXPORT rd_kafka_resp_err_t
1359 rd_kafka_header_get_all (const rd_kafka_headers_t *hdrs, size_t idx,
1360                          const char **namep,
1361                          const void **valuep, size_t *sizep);
1362 
1363 
1364 
1365 /**@}*/
1366 
1367 
1368 
1369 /**
1370  * @name Kafka messages
1371  * @{
1372  *
1373  */
1374 
1375 
1376 
1377 // FIXME: This doesn't show up in docs for some reason
1378 // "Compound rd_kafka_message_t is not documented."
1379 
1380 /**
1381  * @brief A Kafka message as returned by the \c rd_kafka_consume*() family
1382  *        of functions as well as provided to the Producer \c dr_msg_cb().
1383  *
1384  * For the consumer this object has two purposes:
1385  *  - provide the application with a consumed message. (\c err == 0)
1386  *  - report per-topic+partition consumer errors (\c err != 0)
1387  *
1388  * The application must check \c err to decide what action to take.
1389  *
1390  * When the application is finished with a message it must call
1391  * rd_kafka_message_destroy() unless otherwise noted.
1392  */
1393 typedef struct rd_kafka_message_s {
1394 	rd_kafka_resp_err_t err;   /**< Non-zero for error signaling. */
1395 	rd_kafka_topic_t *rkt;     /**< Topic */
1396 	int32_t partition;         /**< Partition */
1397 	void   *payload;           /**< Producer: original message payload.
1398 				    * Consumer: Depends on the value of \c err :
1399 				    * - \c err==0: Message payload.
1400 				    * - \c err!=0: Error string */
1401 	size_t  len;               /**< Depends on the value of \c err :
1402 				    * - \c err==0: Message payload length
1403 				    * - \c err!=0: Error string length */
1404 	void   *key;               /**< Depends on the value of \c err :
1405 				    * - \c err==0: Optional message key */
1406 	size_t  key_len;           /**< Depends on the value of \c err :
1407 				    * - \c err==0: Optional message key length*/
1408 	int64_t offset;            /**< Consumer:
1409                                     * - Message offset (or offset for error
1410 				    *   if \c err!=0 if applicable).
1411                                     *   Producer, dr_msg_cb:
1412                                     *   Message offset assigned by broker.
1413                                     *   May be RD_KAFKA_OFFSET_INVALID
1414                                     *   for retried messages when
1415                                     *   idempotence is enabled. */
1416         void  *_private;           /**< Consumer:
1417                                     *  - rdkafka private pointer: DO NOT MODIFY
1418                                     *  Producer:
1419                                     *  - dr_msg_cb:
1420                                     *    msg_opaque from produce() call or
1421                                     *    RD_KAFKA_V_OPAQUE from producev(). */
1422 } rd_kafka_message_t;
1423 
1424 
1425 /**
1426  * @brief Frees resources for \p rkmessage and hands ownership back to rdkafka.
1427  */
1428 RD_EXPORT
1429 void rd_kafka_message_destroy(rd_kafka_message_t *rkmessage);
1430 
1431 
1432 
1433 
1434 /**
1435  * @brief Returns the error string for an errored rd_kafka_message_t or NULL if
1436  *        there was no error.
1437  *
1438  * @remark This function MUST NOT be used with the producer.
1439  */
1440 RD_EXPORT
1441 const char *rd_kafka_message_errstr (const rd_kafka_message_t *rkmessage);
1442 
1443 
1444 /**
1445  * @brief Returns the message timestamp for a consumed message.
1446  *
1447  * The timestamp is the number of milliseconds since the epoch (UTC).
1448  *
1449  * \p tstype (if not NULL) is updated to indicate the type of timestamp.
1450  *
1451  * @returns message timestamp, or -1 if not available.
1452  *
1453  * @remark Message timestamps require broker version 0.10.0 or later.
1454  */
1455 RD_EXPORT
1456 int64_t rd_kafka_message_timestamp (const rd_kafka_message_t *rkmessage,
1457 				    rd_kafka_timestamp_type_t *tstype);
1458 
1459 
1460 
1461 /**
1462  * @brief Returns the latency for a produced message measured from
1463  *        the produce() call.
1464  *
1465  * @returns the latency in microseconds, or -1 if not available.
1466  */
1467 RD_EXPORT
1468 int64_t rd_kafka_message_latency (const rd_kafka_message_t *rkmessage);
1469 
1470 
1471 /**
1472  * @brief Returns the broker id of the broker the message was produced to
1473  *        or fetched from.
1474  *
1475  * @returns a broker id if known, else -1.
1476  */
1477 RD_EXPORT
1478 int32_t rd_kafka_message_broker_id (const rd_kafka_message_t *rkmessage);
1479 
1480 
1481 /**
1482  * @brief Get the message header list.
1483  *
1484  * The returned pointer in \p *hdrsp is associated with the \p rkmessage and
1485  * must not be used after destruction of the message object or the header
1486  * list is replaced with rd_kafka_message_set_headers().
1487  *
1488  * @returns RD_KAFKA_RESP_ERR_NO_ERROR if headers were returned,
1489  *          RD_KAFKA_RESP_ERR__NOENT if the message has no headers,
1490  *          or another error code if the headers could not be parsed.
1491  *
1492  * @remark Headers require broker version 0.11.0.0 or later.
1493  *
1494  * @remark As an optimization the raw protocol headers are parsed on
1495  *         the first call to this function.
1496  */
1497 RD_EXPORT rd_kafka_resp_err_t
1498 rd_kafka_message_headers (const rd_kafka_message_t *rkmessage,
1499                           rd_kafka_headers_t **hdrsp);
1500 
1501 /**
1502  * @brief Get the message header list and detach the list from the message
1503  *        making the application the owner of the headers.
1504  *        The application must eventually destroy the headers using
1505  *        rd_kafka_headers_destroy().
1506  *        The message's headers will be set to NULL.
1507  *
1508  *        Otherwise same semantics as rd_kafka_message_headers()
1509  *
1510  * @sa rd_kafka_message_headers
1511  */
1512 RD_EXPORT rd_kafka_resp_err_t
1513 rd_kafka_message_detach_headers (rd_kafka_message_t *rkmessage,
1514                                  rd_kafka_headers_t **hdrsp);
1515 
1516 
1517 /**
1518  * @brief Replace the message's current headers with a new list.
1519  *
1520  * @param rkmessage The message to set headers.
1521  * @param hdrs New header list. The message object assumes ownership of
1522  *             the list, the list will be destroyed automatically with
1523  *             the message object.
1524  *             The new headers list may be updated until the message object
1525  *             is passed or returned to librdkafka.
1526  *
1527  * @remark The existing headers object, if any, will be destroyed.
1528  */
1529 RD_EXPORT
1530 void rd_kafka_message_set_headers (rd_kafka_message_t *rkmessage,
1531                                    rd_kafka_headers_t *hdrs);
1532 
1533 
1534 /**
1535  * @brief Returns the number of header key/value pairs
1536  *
1537  * @param hdrs   Headers to count
1538  */
1539 RD_EXPORT size_t rd_kafka_header_cnt (const rd_kafka_headers_t *hdrs);
1540 
1541 
1542 /**
1543  * @enum rd_kafka_msg_status_t
1544  * @brief Message persistence status can be used by the application to
1545  *        find out if a produced message was persisted in the topic log.
1546  */
1547 typedef enum {
1548         /** Message was never transmitted to the broker, or failed with
1549          *  an error indicating it was not written to the log.
1550          *  Application retry risks ordering, but not duplication. */
1551         RD_KAFKA_MSG_STATUS_NOT_PERSISTED = 0,
1552 
1553         /** Message was transmitted to broker, but no acknowledgement was
1554          *  received.
1555          *  Application retry risks ordering and duplication. */
1556         RD_KAFKA_MSG_STATUS_POSSIBLY_PERSISTED = 1,
1557 
1558         /** Message was written to the log and acknowledged by the broker.
1559          *  No reason for application to retry.
1560          *  Note: this value should only be trusted with \c acks=all. */
1561         RD_KAFKA_MSG_STATUS_PERSISTED =  2
1562 } rd_kafka_msg_status_t;
1563 
1564 
1565 /**
1566  * @brief Returns the message's persistence status in the topic log.
1567  *
1568  * @remark The message status is not available in on_acknowledgement
1569  *         interceptors.
1570  */
1571 RD_EXPORT rd_kafka_msg_status_t
1572 rd_kafka_message_status (const rd_kafka_message_t *rkmessage);
1573 
1574 /**@}*/
1575 
1576 
1577 /**
1578  * @name Configuration interface
1579  * @{
1580  *
1581  * @brief Main/global configuration property interface
1582  *
1583  */
1584 
1585 /**
1586  * @enum rd_kafka_conf_res_t
1587  * @brief Configuration result type
1588  */
1589 typedef enum {
1590 	RD_KAFKA_CONF_UNKNOWN = -2, /**< Unknown configuration name. */
1591 	RD_KAFKA_CONF_INVALID = -1, /**< Invalid configuration value or
1592                                      *   property or value not supported in
1593                                      *   this build. */
1594 	RD_KAFKA_CONF_OK = 0        /**< Configuration okay */
1595 } rd_kafka_conf_res_t;
1596 
1597 
1598 /**
1599  * @brief Create configuration object.
1600  *
1601  * When providing your own configuration to the \c rd_kafka_*_new_*() calls
1602  * the rd_kafka_conf_t objects needs to be created with this function
1603  * which will set up the defaults.
1604  * I.e.:
1605  * @code
1606  *   rd_kafka_conf_t *myconf;
1607  *   rd_kafka_conf_res_t res;
1608  *
1609  *   myconf = rd_kafka_conf_new();
1610  *   res = rd_kafka_conf_set(myconf, "socket.timeout.ms", "600",
1611  *                           errstr, sizeof(errstr));
1612  *   if (res != RD_KAFKA_CONF_OK)
1613  *      die("%s\n", errstr);
1614  *
1615  *   rk = rd_kafka_new(..., myconf);
1616  * @endcode
1617  *
1618  * Please see CONFIGURATION.md for the default settings or use
1619  * rd_kafka_conf_properties_show() to provide the information at runtime.
1620  *
1621  * The properties are identical to the Apache Kafka configuration properties
1622  * whenever possible.
1623  *
1624  * @remark A successful call to rd_kafka_new() will assume ownership of
1625  * the conf object and rd_kafka_conf_destroy() must not be called.
1626  *
1627  * @returns A new rd_kafka_conf_t object with defaults set.
1628  *
1629  * @sa rd_kafka_new(), rd_kafka_conf_set(), rd_kafka_conf_destroy()
1630  */
1631 RD_EXPORT
1632 rd_kafka_conf_t *rd_kafka_conf_new(void);
1633 
1634 
1635 /**
1636  * @brief Destroys a conf object.
1637  */
1638 RD_EXPORT
1639 void rd_kafka_conf_destroy(rd_kafka_conf_t *conf);
1640 
1641 
1642 /**
1643  * @brief Creates a copy/duplicate of configuration object \p conf
1644  *
1645  * @remark Interceptors are NOT copied to the new configuration object.
1646  * @sa rd_kafka_interceptor_f_on_conf_dup
1647  */
1648 RD_EXPORT
1649 rd_kafka_conf_t *rd_kafka_conf_dup(const rd_kafka_conf_t *conf);
1650 
1651 
1652 /**
1653  * @brief Same as rd_kafka_conf_dup() but with an array of property name
1654  *        prefixes to filter out (ignore) when copying.
1655  */
1656 RD_EXPORT
1657 rd_kafka_conf_t *rd_kafka_conf_dup_filter (const rd_kafka_conf_t *conf,
1658                                            size_t filter_cnt,
1659                                            const char **filter);
1660 
1661 
1662 
1663 /**
1664  * @returns the configuration object used by an rd_kafka_t instance.
1665  *          For use with rd_kafka_conf_get(), et.al., to extract configuration
1666  *          properties from a running client.
1667  *
1668  * @remark the returned object is read-only and its lifetime is the same
1669  *         as the rd_kafka_t object.
1670  */
1671 RD_EXPORT
1672 const rd_kafka_conf_t *rd_kafka_conf (rd_kafka_t *rk);
1673 
1674 
1675 /**
1676  * @brief Sets a configuration property.
1677  *
1678  * \p conf must have been previously created with rd_kafka_conf_new().
1679  *
1680  * Fallthrough:
1681  * Topic-level configuration properties may be set using this interface
1682  * in which case they are applied on the \c default_topic_conf.
1683  * If no \c default_topic_conf has been set one will be created.
1684  * Any sub-sequent rd_kafka_conf_set_default_topic_conf() calls will
1685  * replace the current default topic configuration.
1686  *
1687  * @returns \c rd_kafka_conf_res_t to indicate success or failure.
1688  * In case of failure \p errstr is updated to contain a human readable
1689  * error string.
1690  *
1691  * @remark Setting properties or values that were disabled at build time due to
1692  *         missing dependencies will return RD_KAFKA_CONF_INVALID.
1693  */
1694 RD_EXPORT
1695 rd_kafka_conf_res_t rd_kafka_conf_set(rd_kafka_conf_t *conf,
1696 				       const char *name,
1697 				       const char *value,
1698 				       char *errstr, size_t errstr_size);
1699 
1700 
1701 /**
1702  * @brief Enable event sourcing.
1703  * \p events is a bitmask of \c RD_KAFKA_EVENT_* of events to enable
1704  * for consumption by `rd_kafka_queue_poll()`.
1705  */
1706 RD_EXPORT
1707 void rd_kafka_conf_set_events(rd_kafka_conf_t *conf, int events);
1708 
1709 
1710 /**
1711  * @brief Generic event callback to be used with the event API to trigger
1712  *        callbacks for \c rd_kafka_event_t objects from a background
1713  *        thread serving the background queue.
1714  *
1715  * How to use:
1716  *  1. First set the event callback on the configuration object with this
1717  *     function, followed by creating an rd_kafka_t instance
1718  *     with rd_kafka_new().
1719  *  2. Get the instance's background queue with rd_kafka_queue_get_background()
1720  *     and pass it as the reply/response queue to an API that takes an
1721  *     event queue, such as rd_kafka_CreateTopics().
1722  *  3. As the response event is ready and enqueued on the background queue the
1723  *     event callback will be triggered from the background thread.
1724  *  4. Prior to destroying the client instance, loose your reference to the
1725  *     background queue by calling rd_kafka_queue_destroy().
1726  *
1727  * The application must destroy the \c rkev passed to \p event cb using
1728  * rd_kafka_event_destroy().
1729  *
1730  * The \p event_cb \c opaque argument is the opaque set with
1731  * rd_kafka_conf_set_opaque().
1732  *
1733  * @remark This callback is a specialized alternative to the poll-based
1734  *         event API described in the Event interface section.
1735  *
1736  * @remark The \p event_cb will be called spontaneously from a background
1737  *         thread completely managed by librdkafka.
1738  *         Take care to perform proper locking of application objects.
1739  *
1740  * @warning The application MUST NOT call rd_kafka_destroy() from the
1741  *          event callback.
1742  *
1743  * @sa rd_kafka_queue_get_background
1744  */
1745 RD_EXPORT void
1746 rd_kafka_conf_set_background_event_cb (rd_kafka_conf_t *conf,
1747                                        void (*event_cb) (rd_kafka_t *rk,
1748                                                          rd_kafka_event_t *rkev,
1749                                                          void *opaque));
1750 
1751 
1752 /**
1753  * @deprecated See rd_kafka_conf_set_dr_msg_cb()
1754  */
1755 RD_EXPORT
1756 void rd_kafka_conf_set_dr_cb(rd_kafka_conf_t *conf,
1757 			      void (*dr_cb) (rd_kafka_t *rk,
1758 					     void *payload, size_t len,
1759 					     rd_kafka_resp_err_t err,
1760 					     void *opaque, void *msg_opaque));
1761 
1762 /**
1763  * @brief \b Producer: Set delivery report callback in provided \p conf object.
1764  *
1765  * The delivery report callback will be called once for each message
1766  * accepted by rd_kafka_produce() (et.al) with \p err set to indicate
1767  * the result of the produce request.
1768  *
1769  * The callback is called when a message is succesfully produced or
1770  * if librdkafka encountered a permanent failure.
1771  * Delivery errors occur when the retry count is exceeded, when the
1772  * message.timeout.ms timeout is exceeded or there is a permanent error
1773  * like RD_KAFKA_RESP_ERR_UNKNOWN_TOPIC_OR_PART.
1774  *
1775  * An application must call rd_kafka_poll() at regular intervals to
1776  * serve queued delivery report callbacks.
1777  *
1778  * The broker-assigned offset can be retrieved with \c rkmessage->offset
1779  * and the timestamp can be retrieved using rd_kafka_message_timestamp().
1780  *
1781  * The \p dr_msg_cb \c opaque argument is the opaque set with
1782  * rd_kafka_conf_set_opaque().
1783  * The per-message msg_opaque value is available in
1784  * \c rd_kafka_message_t._private.
1785  *
1786  * @remark The Idempotent Producer may return invalid timestamp
1787  *         (RD_KAFKA_TIMESTAMP_NOT_AVAILABLE), and
1788  *         and offset (RD_KAFKA_OFFSET_INVALID) for retried messages
1789  *         that were previously successfully delivered but not properly
1790  *         acknowledged.
1791  */
1792 RD_EXPORT
1793 void rd_kafka_conf_set_dr_msg_cb(rd_kafka_conf_t *conf,
1794                                   void (*dr_msg_cb) (rd_kafka_t *rk,
1795                                                      const rd_kafka_message_t *
1796                                                      rkmessage,
1797                                                      void *opaque));
1798 
1799 
1800 /**
1801  * @brief \b Consumer: Set consume callback for use with
1802  *        rd_kafka_consumer_poll()
1803  *
1804  * The \p consume_cb \p opaque argument is the opaque set with
1805  * rd_kafka_conf_set_opaque().
1806  */
1807 RD_EXPORT
1808 void rd_kafka_conf_set_consume_cb (rd_kafka_conf_t *conf,
1809                                    void (*consume_cb) (rd_kafka_message_t *
1810                                                        rkmessage,
1811                                                        void *opaque));
1812 
1813 /**
1814  * @brief \b Consumer: Set rebalance callback for use with
1815  *                     coordinated consumer group balancing.
1816  *
1817  * The \p err field is set to either RD_KAFKA_RESP_ERR__ASSIGN_PARTITIONS
1818  * or RD_KAFKA_RESP_ERR__REVOKE_PARTITIONS and 'partitions'
1819  * contains the full partition set that was either assigned or revoked.
1820  *
1821  * Registering a \p rebalance_cb turns off librdkafka's automatic
1822  * partition assignment/revocation and instead delegates that responsibility
1823  * to the application's \p rebalance_cb.
1824  *
1825  * The rebalance callback is responsible for updating librdkafka's
1826  * assignment set based on the two events: RD_KAFKA_RESP_ERR__ASSIGN_PARTITIONS
1827  * and RD_KAFKA_RESP_ERR__REVOKE_PARTITIONS but should also be able to handle
1828  * arbitrary rebalancing failures where \p err is neither of those.
1829  * @remark In this latter case (arbitrary error), the application must
1830  *         call rd_kafka_assign(rk, NULL) to synchronize state.
1831  *
1832  * For eager/non-cooperative `partition.assignment.strategy` assignors,
1833  * such as `range` and `roundrobin`, the application must use
1834  * rd_kafka_assign() to set or clear the entire assignment.
1835  * For the cooperative assignors, such as `cooperative-sticky`, the application
1836  * must use rd_kafka_incremental_assign() for
1837  * RD_KAFKA_RESP_ERR__ASSIGN_PARTITIONS and rd_kafka_incremental_unassign()
1838  * for RD_KAFKA_RESP_ERR__REVOKE_PARTITIONS.
1839  *
1840  * Without a rebalance callback this is done automatically by librdkafka
1841  * but registering a rebalance callback gives the application flexibility
1842  * in performing other operations along with the assigning/revocation,
1843  * such as fetching offsets from an alternate location (on assign)
1844  * or manually committing offsets (on revoke).
1845  *
1846  * rebalance_cb is always triggered exactly once when a rebalance completes
1847  * with a new assignment, even if that assignment is empty. If an
1848  * eager/non-cooperative assignor is configured, there will eventually be
1849  * exactly one corresponding call to rebalance_cb to revoke these partitions
1850  * (even if empty), whether this is due to a group rebalance or lost
1851  * partitions. In the cooperative case, rebalance_cb will never be called if
1852  * the set of partitions being revoked is empty (whether or not lost).
1853  *
1854  * The callback's \p opaque argument is the opaque set with
1855  * rd_kafka_conf_set_opaque().
1856  *
1857  * @remark The \p partitions list is destroyed by librdkafka on return
1858  *         return from the rebalance_cb and must not be freed or
1859  *         saved by the application.
1860  *
1861  * @remark Be careful when modifying the \p partitions list.
1862  *         Changing this list should only be done to change the initial
1863  *         offsets for each partition.
1864  *         But a function like `rd_kafka_position()` might have unexpected
1865  *         effects for instance when a consumer gets assigned a partition
1866  *         it used to consume at an earlier rebalance. In this case, the
1867  *         list of partitions will be updated with the old offset for that
1868  *         partition. In this case, it is generally better to pass a copy
1869  *         of the list (see `rd_kafka_topic_partition_list_copy()`).
1870  *         The result of `rd_kafka_position()` is typically outdated in
1871  *         RD_KAFKA_RESP_ERR__ASSIGN_PARTITIONS.
1872  *
1873  * @sa rd_kafka_assign()
1874  * @sa rd_kafka_incremental_assign()
1875  * @sa rd_kafka_incremental_unassign()
1876  * @sa rd_kafka_assignment_lost()
1877  * @sa rd_kafka_rebalance_protocol()
1878  *
1879  * The following example shows the application's responsibilities:
1880  * @code
1881  *    static void rebalance_cb (rd_kafka_t *rk, rd_kafka_resp_err_t err,
1882  *                              rd_kafka_topic_partition_list_t *partitions,
1883  *                              void *opaque) {
1884  *
1885  *        switch (err)
1886  *        {
1887  *          case RD_KAFKA_RESP_ERR__ASSIGN_PARTITIONS:
1888  *             // application may load offets from arbitrary external
1889  *             // storage here and update \p partitions
1890  *             if (!strcmp(rd_kafka_rebalance_protocol(rk), "COOPERATIVE"))
1891  *                     rd_kafka_incremental_assign(rk, partitions);
1892  *             else // EAGER
1893  *                     rd_kafka_assign(rk, partitions);
1894  *             break;
1895  *
1896  *          case RD_KAFKA_RESP_ERR__REVOKE_PARTITIONS:
1897  *             if (manual_commits) // Optional explicit manual commit
1898  *                 rd_kafka_commit(rk, partitions, 0); // sync commit
1899  *
1900  *             if (!strcmp(rd_kafka_rebalance_protocol(rk), "COOPERATIVE"))
1901  *                     rd_kafka_incremental_unassign(rk, partitions);
1902  *             else // EAGER
1903  *                     rd_kafka_assign(rk, NULL);
1904  *             break;
1905  *
1906  *          default:
1907  *             handle_unlikely_error(err);
1908  *             rd_kafka_assign(rk, NULL); // sync state
1909  *             break;
1910  *         }
1911  *    }
1912  * @endcode
1913  *
1914  * @remark The above example lacks error handling for assign calls, see
1915  *         the examples/ directory.
1916  */
1917 RD_EXPORT
1918 void rd_kafka_conf_set_rebalance_cb (
1919         rd_kafka_conf_t *conf,
1920         void (*rebalance_cb) (rd_kafka_t *rk,
1921                               rd_kafka_resp_err_t err,
1922                               rd_kafka_topic_partition_list_t *partitions,
1923                               void *opaque));
1924 
1925 
1926 
1927 /**
1928  * @brief \b Consumer: Set offset commit callback for use with consumer groups.
1929  *
1930  * The results of automatic or manual offset commits will be scheduled
1931  * for this callback and is served by rd_kafka_consumer_poll().
1932  *
1933  * If no partitions had valid offsets to commit this callback will be called
1934  * with \p err == RD_KAFKA_RESP_ERR__NO_OFFSET which is not to be considered
1935  * an error.
1936  *
1937  * The \p offsets list contains per-partition information:
1938  *   - \c offset: committed offset (attempted)
1939  *   - \c err:    commit error
1940  *
1941  * The callback's \p opaque argument is the opaque set with
1942  * rd_kafka_conf_set_opaque().
1943  */
1944 RD_EXPORT
1945 void rd_kafka_conf_set_offset_commit_cb (
1946         rd_kafka_conf_t *conf,
1947         void (*offset_commit_cb) (rd_kafka_t *rk,
1948                                   rd_kafka_resp_err_t err,
1949                                   rd_kafka_topic_partition_list_t *offsets,
1950                                   void *opaque));
1951 
1952 
1953 /**
1954  * @brief Set error callback in provided conf object.
1955  *
1956  * The error callback is used by librdkafka to signal warnings and errors
1957  * back to the application.
1958  *
1959  * These errors should generally be considered informational and non-permanent,
1960  * the client will try to recover automatically from all type of errors.
1961  * Given that the client and cluster configuration is correct the
1962  * application should treat these as temporary errors.
1963  *
1964  * \p error_cb will be triggered with \c err set to RD_KAFKA_RESP_ERR__FATAL
1965  * if a fatal error has been raised; in this case use rd_kafka_fatal_error() to
1966  * retrieve the fatal error code and error string, and then begin terminating
1967  * the client instance.
1968  *
1969  * If no \p error_cb is registered, or RD_KAFKA_EVENT_ERROR has not been set
1970  * with rd_kafka_conf_set_events, then the errors will be logged instead.
1971  *
1972  * The callback's \p opaque argument is the opaque set with
1973  * rd_kafka_conf_set_opaque().
1974  */
1975 RD_EXPORT
1976 void rd_kafka_conf_set_error_cb(rd_kafka_conf_t *conf,
1977 				 void  (*error_cb) (rd_kafka_t *rk, int err,
1978 						    const char *reason,
1979 						    void *opaque));
1980 
1981 /**
1982  * @brief Set throttle callback.
1983  *
1984  * The throttle callback is used to forward broker throttle times to the
1985  * application for Produce and Fetch (consume) requests.
1986  *
1987  * Callbacks are triggered whenever a non-zero throttle time is returned by
1988  * the broker, or when the throttle time drops back to zero.
1989  *
1990  * An application must call rd_kafka_poll() or rd_kafka_consumer_poll() at
1991  * regular intervals to serve queued callbacks.
1992  *
1993  * The callback's \p opaque argument is the opaque set with
1994  * rd_kafka_conf_set_opaque().
1995  *
1996  * @remark Requires broker version 0.9.0 or later.
1997  */
1998 RD_EXPORT
1999 void rd_kafka_conf_set_throttle_cb (rd_kafka_conf_t *conf,
2000 				    void (*throttle_cb) (
2001 					    rd_kafka_t *rk,
2002 					    const char *broker_name,
2003 					    int32_t broker_id,
2004 					    int throttle_time_ms,
2005 					    void *opaque));
2006 
2007 
2008 /**
2009  * @brief Set logger callback.
2010  *
2011  * The default is to print to stderr, but a syslog logger is also available,
2012  * see rd_kafka_log_print and rd_kafka_log_syslog for the builtin alternatives.
2013  * Alternatively the application may provide its own logger callback.
2014  * Or pass \p func as NULL to disable logging.
2015  *
2016  * This is the configuration alternative to the deprecated rd_kafka_set_logger()
2017  *
2018  * @remark The log_cb will be called spontaneously from librdkafka's internal
2019  *         threads unless logs have been forwarded to a poll queue through
2020  *         \c rd_kafka_set_log_queue().
2021  *         An application MUST NOT call any librdkafka APIs or do any prolonged
2022  *         work in a non-forwarded \c log_cb.
2023  */
2024 RD_EXPORT
2025 void rd_kafka_conf_set_log_cb(rd_kafka_conf_t *conf,
2026 			  void (*log_cb) (const rd_kafka_t *rk, int level,
2027                                           const char *fac, const char *buf));
2028 
2029 
2030 /**
2031  * @brief Set statistics callback in provided conf object.
2032  *
2033  * The statistics callback is triggered from rd_kafka_poll() every
2034  * \c statistics.interval.ms (needs to be configured separately).
2035  * Function arguments:
2036  *   - \p rk - Kafka handle
2037  *   - \p json - String containing the statistics data in JSON format
2038  *   - \p json_len - Length of \p json string.
2039  *   - \p opaque - Application-provided opaque as set by
2040  *                 rd_kafka_conf_set_opaque().
2041  *
2042  * For more information on the format of \p json, see
2043  * https://github.com/edenhill/librdkafka/wiki/Statistics
2044  *
2045  * If the application wishes to hold on to the \p json pointer and free
2046  * it at a later time it must return 1 from the \p stats_cb.
2047  * If the application returns 0 from the \p stats_cb then librdkafka
2048  * will immediately free the \p json pointer.
2049  *
2050  * See STATISTICS.md for a full definition of the JSON object.
2051  */
2052 RD_EXPORT
2053 void rd_kafka_conf_set_stats_cb(rd_kafka_conf_t *conf,
2054 				 int (*stats_cb) (rd_kafka_t *rk,
2055 						  char *json,
2056 						  size_t json_len,
2057 						  void *opaque));
2058 
2059 /**
2060  * @brief Set SASL/OAUTHBEARER token refresh callback in provided conf object.
2061  *
2062  * @param conf the configuration to mutate.
2063  * @param oauthbearer_token_refresh_cb the callback to set; callback function
2064  *  arguments:<br>
2065  *   \p rk - Kafka handle<br>
2066  *   \p oauthbearer_config - Value of configuration property
2067  *                           sasl.oauthbearer.config.
2068  *   \p opaque - Application-provided opaque set via
2069  *               rd_kafka_conf_set_opaque()
2070  *
2071  * The SASL/OAUTHBEARER token refresh callback is triggered via rd_kafka_poll()
2072  * whenever OAUTHBEARER is the SASL mechanism and a token needs to be retrieved,
2073  * typically based on the configuration defined in \c sasl.oauthbearer.config.
2074  *
2075  * The callback should invoke rd_kafka_oauthbearer_set_token()
2076  * or rd_kafka_oauthbearer_set_token_failure() to indicate success
2077  * or failure, respectively.
2078  *
2079  * The refresh operation is eventable and may be received via
2080  * rd_kafka_queue_poll() with an event type of
2081  * \c RD_KAFKA_EVENT_OAUTHBEARER_TOKEN_REFRESH.
2082  *
2083  * Note that before any SASL/OAUTHBEARER broker connection can succeed the
2084  * application must call rd_kafka_oauthbearer_set_token() once -- either
2085  * directly or, more typically, by invoking either rd_kafka_poll() or
2086  * rd_kafka_queue_poll() -- in order to cause retrieval of an initial token to
2087  * occur.
2088  *
2089  * An unsecured JWT refresh handler is provided by librdkafka for development
2090  * and testing purposes, it is enabled by setting
2091  * the \c enable.sasl.oauthbearer.unsecure.jwt property to true and is
2092  * mutually exclusive to using a refresh callback.
2093  */
2094 RD_EXPORT
2095 void rd_kafka_conf_set_oauthbearer_token_refresh_cb (
2096         rd_kafka_conf_t *conf,
2097         void (*oauthbearer_token_refresh_cb) (rd_kafka_t *rk,
2098                                               const char *oauthbearer_config,
2099                                               void *opaque));
2100 
2101 /**
2102  * @brief Set socket callback.
2103  *
2104  * The socket callback is responsible for opening a socket
2105  * according to the supplied \p domain, \p type and \p protocol.
2106  * The socket shall be created with \c CLOEXEC set in a racefree fashion, if
2107  * possible.
2108  *
2109  * The callback's \p opaque argument is the opaque set with
2110  * rd_kafka_conf_set_opaque().
2111  *
2112  * Default:
2113  *  - on linux: racefree CLOEXEC
2114  *  - others  : non-racefree CLOEXEC
2115  *
2116  * @remark The callback will be called from an internal librdkafka thread.
2117  */
2118 RD_EXPORT
2119 void rd_kafka_conf_set_socket_cb(rd_kafka_conf_t *conf,
2120                                   int (*socket_cb) (int domain, int type,
2121                                                     int protocol,
2122                                                     void *opaque));
2123 
2124 
2125 
2126 /**
2127  * @brief Set connect callback.
2128  *
2129  * The connect callback is responsible for connecting socket \p sockfd
2130  * to peer address \p addr.
2131  * The \p id field contains the broker identifier.
2132  *
2133  * \p connect_cb shall return 0 on success (socket connected) or an error
2134  * number (errno) on error.
2135  *
2136  * The callback's \p opaque argument is the opaque set with
2137  * rd_kafka_conf_set_opaque().
2138  *
2139  * @remark The callback will be called from an internal librdkafka thread.
2140  */
2141 RD_EXPORT void
2142 rd_kafka_conf_set_connect_cb (rd_kafka_conf_t *conf,
2143                               int (*connect_cb) (int sockfd,
2144                                                  const struct sockaddr *addr,
2145                                                  int addrlen,
2146                                                  const char *id,
2147                                                  void *opaque));
2148 
2149 /**
2150  * @brief Set close socket callback.
2151  *
2152  * Close a socket (optionally opened with socket_cb()).
2153  *
2154  * The callback's \p opaque argument is the opaque set with
2155  * rd_kafka_conf_set_opaque().
2156  *
2157  * @remark The callback will be called from an internal librdkafka thread.
2158  */
2159 RD_EXPORT void
2160 rd_kafka_conf_set_closesocket_cb (rd_kafka_conf_t *conf,
2161                                   int (*closesocket_cb) (int sockfd,
2162                                                          void *opaque));
2163 
2164 
2165 
2166 #ifndef _WIN32
2167 /**
2168  * @brief Set open callback.
2169  *
2170  * The open callback is responsible for opening the file specified by
2171  * pathname, flags and mode.
2172  * The file shall be opened with \c CLOEXEC set in a racefree fashion, if
2173  * possible.
2174  *
2175  * Default:
2176  *  - on linux: racefree CLOEXEC
2177  *  - others  : non-racefree CLOEXEC
2178  *
2179  * The callback's \p opaque argument is the opaque set with
2180  * rd_kafka_conf_set_opaque().
2181  *
2182  * @remark The callback will be called from an internal librdkafka thread.
2183  */
2184 RD_EXPORT
2185 void rd_kafka_conf_set_open_cb (rd_kafka_conf_t *conf,
2186                                 int (*open_cb) (const char *pathname,
2187                                                 int flags, mode_t mode,
2188                                                 void *opaque));
2189 #endif
2190 
2191 
2192 /**
2193  * @brief Sets the verification callback of the broker certificate
2194  *
2195  * The verification callback is triggered from internal librdkafka threads
2196  * upon connecting to a broker. On each connection attempt the callback
2197  * will be called for each certificate in the broker's certificate chain,
2198  * starting at the root certification, as long as the application callback
2199  * returns 1 (valid certificate).
2200  * \c broker_name and \c broker_id correspond to the broker the connection
2201  * is being made to.
2202  * The \c x509_error argument indicates if OpenSSL's verification of
2203  * the certificate succeed (0) or failed (an OpenSSL error code).
2204  * The application may set the SSL context error code by returning 0
2205  * from the verify callback and providing a non-zero SSL context error code
2206  * in \c x509_error.
2207  * If the verify callback sets \c x509_error to 0, returns 1, and the
2208  * original \c x509_error was non-zero, the error on the SSL context will
2209  * be cleared.
2210  * \c x509_error is always a valid pointer to an int.
2211  *
2212  * \c depth is the depth of the current certificate in the chain, starting
2213  * at the root certificate.
2214  *
2215  * The certificate itself is passed in binary DER format in \c buf of
2216  * size \c size.
2217  *
2218  * The callback must return 1 if verification succeeds, or
2219  * 0 if verification fails and then write a human-readable error message
2220  * to \c errstr (limited to \c errstr_size bytes, including nul-term).
2221  *
2222  * The callback's \p opaque argument is the opaque set with
2223  * rd_kafka_conf_set_opaque().
2224  *
2225  * @returns RD_KAFKA_CONF_OK if SSL is supported in this build, else
2226  *          RD_KAFKA_CONF_INVALID.
2227  *
2228  * @warning This callback will be called from internal librdkafka threads.
2229  *
2230  * @remark See <openssl/x509_vfy.h> in the OpenSSL source distribution
2231  *         for a list of \p x509_error codes.
2232  */
2233 RD_EXPORT
2234 rd_kafka_conf_res_t rd_kafka_conf_set_ssl_cert_verify_cb (
2235         rd_kafka_conf_t *conf,
2236         int (*ssl_cert_verify_cb) (rd_kafka_t *rk,
2237                                    const char *broker_name,
2238                                    int32_t broker_id,
2239                                    int *x509_error,
2240                                    int depth,
2241                                    const char *buf, size_t size,
2242                                    char *errstr, size_t errstr_size,
2243                                    void *opaque));
2244 
2245 
2246 /**
2247  * @enum rd_kafka_cert_type_t
2248  *
2249  * @brief SSL certificate type
2250  *
2251  * @sa rd_kafka_conf_set_ssl_cert
2252  */
2253 typedef enum rd_kafka_cert_type_t {
2254         RD_KAFKA_CERT_PUBLIC_KEY,  /**< Client's public key */
2255         RD_KAFKA_CERT_PRIVATE_KEY, /**< Client's private key */
2256         RD_KAFKA_CERT_CA,          /**< CA certificate */
2257         RD_KAFKA_CERT__CNT,
2258 } rd_kafka_cert_type_t;
2259 
2260 /**
2261  * @enum rd_kafka_cert_enc_t
2262  *
2263  * @brief SSL certificate encoding
2264  *
2265  * @sa rd_kafka_conf_set_ssl_cert
2266  */
2267 typedef enum rd_kafka_cert_enc_t {
2268         RD_KAFKA_CERT_ENC_PKCS12,  /**< PKCS#12 */
2269         RD_KAFKA_CERT_ENC_DER,     /**< DER / binary X.509 ASN1 */
2270         RD_KAFKA_CERT_ENC_PEM,     /**< PEM */
2271         RD_KAFKA_CERT_ENC__CNT,
2272 } rd_kafka_cert_enc_t;
2273 
2274 
2275 /**
2276  * @brief Set certificate/key \p cert_type from the \p cert_enc encoded
2277  *        memory at \p buffer of \p size bytes.
2278  *
2279  * @param conf Configuration object.
2280  * @param cert_type Certificate or key type to configure.
2281  * @param cert_enc  Buffer \p encoding type.
2282  * @param buffer Memory pointer to encoded certificate or key.
2283  *               The memory is not referenced after this function returns.
2284  * @param size Size of memory at \p buffer.
2285  * @param errstr Memory were a human-readable error string will be written
2286  *               on failure.
2287  * @param errstr_size Size of \p errstr, including space for nul-terminator.
2288  *
2289  * @returns RD_KAFKA_CONF_OK on success or RD_KAFKA_CONF_INVALID if the
2290  *          memory in \p buffer is of incorrect encoding, or if librdkafka
2291  *          was not built with SSL support.
2292  *
2293  * @remark Calling this method multiple times with the same \p cert_type
2294  *         will replace the previous value.
2295  *
2296  * @remark Calling this method with \p buffer set to NULL will clear the
2297  *         configuration for \p cert_type.
2298  *
2299  * @remark The private key may require a password, which must be specified
2300  *         with the `ssl.key.password` configuration property prior to
2301  *         calling this function.
2302  *
2303  * @remark Private and public keys in PEM format may also be set with the
2304  *         `ssl.key.pem` and `ssl.certificate.pem` configuration properties.
2305  *
2306  * @remark CA certificate in PEM format may also be set with the
2307  *         `ssl.ca.pem` configuration property.
2308  */
2309 RD_EXPORT rd_kafka_conf_res_t
2310 rd_kafka_conf_set_ssl_cert (rd_kafka_conf_t *conf,
2311                             rd_kafka_cert_type_t cert_type,
2312                             rd_kafka_cert_enc_t cert_enc,
2313                             const void *buffer, size_t size,
2314                             char *errstr, size_t errstr_size);
2315 
2316 
2317 /**
2318  * @brief Set callback_data for OpenSSL engine.
2319  *
2320  * @param conf Configuration object.
2321  * @param callback_data passed to engine callbacks,
2322  *                      e.g. \c ENGINE_load_ssl_client_cert.
2323  *
2324  * @remark The \c ssl.engine.location configuration must be set for this
2325  *         to have affect.
2326  *
2327  * @remark The memory pointed to by \p value must remain valid for the
2328  *         lifetime of the configuration object and any Kafka clients that
2329  *         use it.
2330  */
2331 RD_EXPORT
2332 void rd_kafka_conf_set_engine_callback_data (rd_kafka_conf_t *conf,
2333                                              void *callback_data);
2334 
2335 
2336 /**
2337  * @brief Sets the application's opaque pointer that will be passed to callbacks
2338  *
2339  * @sa rd_kafka_opaque()
2340  */
2341 RD_EXPORT
2342 void rd_kafka_conf_set_opaque(rd_kafka_conf_t *conf, void *opaque);
2343 
2344 /**
2345  * @brief Retrieves the opaque pointer previously set
2346  *        with rd_kafka_conf_set_opaque()
2347  */
2348 RD_EXPORT
2349 void *rd_kafka_opaque(const rd_kafka_t *rk);
2350 
2351 
2352 
2353 /**
2354  * @brief Sets the default topic configuration to use for automatically
2355  *        subscribed topics (e.g., through pattern-matched topics).
2356  *        The topic config object is not usable after this call.
2357  *
2358  * @warning Any topic configuration settings that have been set on the
2359  *          global rd_kafka_conf_t object will be overwritten by this call
2360  *          since the implicitly created default topic config object is
2361  *          replaced by the user-supplied one.
2362  *
2363  * @deprecated Set default topic level configuration on the
2364  *             global rd_kafka_conf_t object instead.
2365  */
2366 RD_EXPORT
2367 void rd_kafka_conf_set_default_topic_conf (rd_kafka_conf_t *conf,
2368                                            rd_kafka_topic_conf_t *tconf);
2369 
2370 /**
2371  * @brief Gets the default topic configuration as previously set with
2372  *        rd_kafka_conf_set_default_topic_conf() or that was implicitly created
2373  *        by configuring a topic-level property on the global \p conf object.
2374  *
2375  * @returns the \p conf's default topic configuration (if any), or NULL.
2376  *
2377  * @warning The returned topic configuration object is owned by the \p conf
2378  *          object. It may be modified but not destroyed and its lifetime is
2379  *          the same as the \p conf object or the next call to
2380  *          rd_kafka_conf_set_default_topic_conf().
2381  */
2382 RD_EXPORT rd_kafka_topic_conf_t *
2383 rd_kafka_conf_get_default_topic_conf (rd_kafka_conf_t *conf);
2384 
2385 
2386 /**
2387  * @brief Retrieve configuration value for property \p name.
2388  *
2389  * If \p dest is non-NULL the value will be written to \p dest with at
2390  * most \p dest_size.
2391  *
2392  * \p *dest_size is updated to the full length of the value, thus if
2393  * \p *dest_size initially is smaller than the full length the application
2394  * may reallocate \p dest to fit the returned \p *dest_size and try again.
2395  *
2396  * If \p dest is NULL only the full length of the value is returned.
2397  *
2398  * Fallthrough:
2399  * Topic-level configuration properties from the \c default_topic_conf
2400  * may be retrieved using this interface.
2401  *
2402  * @returns \p RD_KAFKA_CONF_OK if the property name matched, else
2403  * \p RD_KAFKA_CONF_UNKNOWN.
2404  */
2405 RD_EXPORT
2406 rd_kafka_conf_res_t rd_kafka_conf_get (const rd_kafka_conf_t *conf,
2407                                        const char *name,
2408                                        char *dest, size_t *dest_size);
2409 
2410 
2411 /**
2412  * @brief Retrieve topic configuration value for property \p name.
2413  *
2414  * @sa rd_kafka_conf_get()
2415  */
2416 RD_EXPORT
2417 rd_kafka_conf_res_t rd_kafka_topic_conf_get (const rd_kafka_topic_conf_t *conf,
2418                                              const char *name,
2419                                              char *dest, size_t *dest_size);
2420 
2421 
2422 /**
2423  * @brief Dump the configuration properties and values of \p conf to an array
2424  *        with \"key\", \"value\" pairs.
2425  *
2426  * The number of entries in the array is returned in \p *cntp.
2427  *
2428  * The dump must be freed with `rd_kafka_conf_dump_free()`.
2429  */
2430 RD_EXPORT
2431 const char **rd_kafka_conf_dump(rd_kafka_conf_t *conf, size_t *cntp);
2432 
2433 
2434 /**
2435  * @brief Dump the topic configuration properties and values of \p conf
2436  *        to an array with \"key\", \"value\" pairs.
2437  *
2438  * The number of entries in the array is returned in \p *cntp.
2439  *
2440  * The dump must be freed with `rd_kafka_conf_dump_free()`.
2441  */
2442 RD_EXPORT
2443 const char **rd_kafka_topic_conf_dump(rd_kafka_topic_conf_t *conf,
2444 				       size_t *cntp);
2445 
2446 /**
2447  * @brief Frees a configuration dump returned from `rd_kafka_conf_dump()` or
2448  *        `rd_kafka_topic_conf_dump().
2449  */
2450 RD_EXPORT
2451 void rd_kafka_conf_dump_free(const char **arr, size_t cnt);
2452 
2453 /**
2454  * @brief Prints a table to \p fp of all supported configuration properties,
2455  *        their default values as well as a description.
2456  *
2457  * @remark All properties and properties and values are shown, even those
2458  *         that have been disabled at build time due to missing dependencies.
2459  */
2460 RD_EXPORT
2461 void rd_kafka_conf_properties_show(FILE *fp);
2462 
2463 /**@}*/
2464 
2465 
2466 /**
2467  * @name Topic configuration
2468  * @{
2469  *
2470  * @brief Topic configuration property interface
2471  *
2472  */
2473 
2474 
2475 /**
2476  * @brief Create topic configuration object
2477  *
2478  * @sa Same semantics as for rd_kafka_conf_new().
2479  */
2480 RD_EXPORT
2481 rd_kafka_topic_conf_t *rd_kafka_topic_conf_new(void);
2482 
2483 
2484 /**
2485  * @brief Creates a copy/duplicate of topic configuration object \p conf.
2486  */
2487 RD_EXPORT
2488 rd_kafka_topic_conf_t *rd_kafka_topic_conf_dup(const rd_kafka_topic_conf_t
2489 						*conf);
2490 
2491 /**
2492  * @brief Creates a copy/duplicate of \p rk 's default topic configuration
2493  *        object.
2494  */
2495 RD_EXPORT
2496 rd_kafka_topic_conf_t *rd_kafka_default_topic_conf_dup (rd_kafka_t *rk);
2497 
2498 
2499 /**
2500  * @brief Destroys a topic conf object.
2501  */
2502 RD_EXPORT
2503 void rd_kafka_topic_conf_destroy(rd_kafka_topic_conf_t *topic_conf);
2504 
2505 
2506 /**
2507  * @brief Sets a single rd_kafka_topic_conf_t value by property name.
2508  *
2509  * \p topic_conf should have been previously set up
2510  * with `rd_kafka_topic_conf_new()`.
2511  *
2512  * @returns rd_kafka_conf_res_t to indicate success or failure.
2513  */
2514 RD_EXPORT
2515 rd_kafka_conf_res_t rd_kafka_topic_conf_set(rd_kafka_topic_conf_t *conf,
2516 					     const char *name,
2517 					     const char *value,
2518 					     char *errstr, size_t errstr_size);
2519 
2520 /**
2521  * @brief Sets the application's opaque pointer that will be passed to all topic
2522  * callbacks as the \c rkt_opaque argument.
2523  *
2524  * @sa rd_kafka_topic_opaque()
2525  */
2526 RD_EXPORT
2527 void rd_kafka_topic_conf_set_opaque(rd_kafka_topic_conf_t *conf,
2528                                     void *rkt_opaque);
2529 
2530 
2531 /**
2532  * @brief \b Producer: Set partitioner callback in provided topic conf object.
2533  *
2534  * The partitioner may be called in any thread at any time,
2535  * it may be called multiple times for the same message/key.
2536  *
2537  * The callback's \p rkt_opaque argument is the opaque set by
2538  * rd_kafka_topic_conf_set_opaque().
2539  * The callback's \p msg_opaque argument is the per-message opaque
2540  * passed to produce().
2541  *
2542  * Partitioner function constraints:
2543  *   - MUST NOT call any rd_kafka_*() functions except:
2544  *       rd_kafka_topic_partition_available()
2545  *   - MUST NOT block or execute for prolonged periods of time.
2546  *   - MUST return a value between 0 and partition_cnt-1, or the
2547  *     special \c RD_KAFKA_PARTITION_UA value if partitioning
2548  *     could not be performed.
2549  */
2550 RD_EXPORT
2551 void
2552 rd_kafka_topic_conf_set_partitioner_cb (rd_kafka_topic_conf_t *topic_conf,
2553 					int32_t (*partitioner) (
2554 						const rd_kafka_topic_t *rkt,
2555 						const void *keydata,
2556 						size_t keylen,
2557 						int32_t partition_cnt,
2558 						void *rkt_opaque,
2559 						void *msg_opaque));
2560 
2561 
2562 /**
2563  * @brief \b Producer: Set message queueing order comparator callback.
2564  *
2565  * The callback may be called in any thread at any time,
2566  * it may be called multiple times for the same message.
2567  *
2568  * Ordering comparator function constraints:
2569  *   - MUST be stable sort (same input gives same output).
2570  *   - MUST NOT call any rd_kafka_*() functions.
2571  *   - MUST NOT block or execute for prolonged periods of time.
2572  *
2573  * The comparator shall compare the two messages and return:
2574  *  - < 0 if message \p a should be inserted before message \p b.
2575  *  - >=0 if message \p a should be inserted after message \p b.
2576  *
2577  * @remark Insert sorting will be used to enqueue the message in the
2578  *         correct queue position, this comes at a cost of O(n).
2579  *
2580  * @remark If `queuing.strategy=fifo` new messages are enqueued to the
2581  *         tail of the queue regardless of msg_order_cmp, but retried messages
2582  *         are still affected by msg_order_cmp.
2583  *
2584  * @warning THIS IS AN EXPERIMENTAL API, SUBJECT TO CHANGE OR REMOVAL,
2585  *          DO NOT USE IN PRODUCTION.
2586  */
2587 RD_EXPORT void
2588 rd_kafka_topic_conf_set_msg_order_cmp (rd_kafka_topic_conf_t *topic_conf,
2589                                        int (*msg_order_cmp) (
2590                                                const rd_kafka_message_t *a,
2591                                                const rd_kafka_message_t *b));
2592 
2593 
2594 /**
2595  * @brief Check if partition is available (has a leader broker).
2596  *
2597  * @returns 1 if the partition is available, else 0.
2598  *
2599  * @warning This function must only be called from inside a partitioner function
2600  */
2601 RD_EXPORT
2602 int rd_kafka_topic_partition_available(const rd_kafka_topic_t *rkt,
2603 					int32_t partition);
2604 
2605 
2606 /*******************************************************************
2607  *								   *
2608  * Partitioners provided by rdkafka                                *
2609  *								   *
2610  *******************************************************************/
2611 
2612 /**
2613  * @brief Random partitioner.
2614  *
2615  * Will try not to return unavailable partitions.
2616  *
2617  * The \p rkt_opaque argument is the opaque set by
2618  * rd_kafka_topic_conf_set_opaque().
2619  * The \p msg_opaque argument is the per-message opaque
2620  * passed to produce().
2621  *
2622  * @returns a random partition between 0 and \p partition_cnt - 1.
2623  *
2624  */
2625 RD_EXPORT
2626 int32_t rd_kafka_msg_partitioner_random(const rd_kafka_topic_t *rkt,
2627 					 const void *key, size_t keylen,
2628 					 int32_t partition_cnt,
2629 					 void *rkt_opaque, void *msg_opaque);
2630 
2631 /**
2632  * @brief Consistent partitioner.
2633  *
2634  * Uses consistent hashing to map identical keys onto identical partitions.
2635  *
2636  * The \p rkt_opaque argument is the opaque set by
2637  * rd_kafka_topic_conf_set_opaque().
2638  * The \p msg_opaque argument is the per-message opaque
2639  * passed to produce().
2640  *
2641  * @returns a \"random\" partition between 0 and \p partition_cnt - 1 based on
2642  *          the CRC value of the key
2643  */
2644 RD_EXPORT
2645 int32_t rd_kafka_msg_partitioner_consistent (const rd_kafka_topic_t *rkt,
2646 					 const void *key, size_t keylen,
2647 					 int32_t partition_cnt,
2648 					 void *rkt_opaque, void *msg_opaque);
2649 
2650 /**
2651  * @brief Consistent-Random partitioner.
2652  *
2653  * This is the default partitioner.
2654  * Uses consistent hashing to map identical keys onto identical partitions, and
2655  * messages without keys will be assigned via the random partitioner.
2656  *
2657  * The \p rkt_opaque argument is the opaque set by
2658  * rd_kafka_topic_conf_set_opaque().
2659  * The \p msg_opaque argument is the per-message opaque
2660  * passed to produce().
2661  *
2662  * @returns a \"random\" partition between 0 and \p partition_cnt - 1 based on
2663  *          the CRC value of the key (if provided)
2664  */
2665 RD_EXPORT
2666 int32_t rd_kafka_msg_partitioner_consistent_random (const rd_kafka_topic_t *rkt,
2667            const void *key, size_t keylen,
2668            int32_t partition_cnt,
2669            void *rkt_opaque, void *msg_opaque);
2670 
2671 
2672 /**
2673  * @brief Murmur2 partitioner (Java compatible).
2674  *
2675  * Uses consistent hashing to map identical keys onto identical partitions
2676  * using Java-compatible Murmur2 hashing.
2677  *
2678  * The \p rkt_opaque argument is the opaque set by
2679  * rd_kafka_topic_conf_set_opaque().
2680  * The \p msg_opaque argument is the per-message opaque
2681  * passed to produce().
2682  *
2683  * @returns a partition between 0 and \p partition_cnt - 1.
2684  */
2685 RD_EXPORT
2686 int32_t rd_kafka_msg_partitioner_murmur2 (const rd_kafka_topic_t *rkt,
2687                                           const void *key, size_t keylen,
2688                                           int32_t partition_cnt,
2689                                           void *rkt_opaque,
2690                                           void *msg_opaque);
2691 
2692 /**
2693  * @brief Consistent-Random Murmur2 partitioner (Java compatible).
2694  *
2695  * Uses consistent hashing to map identical keys onto identical partitions
2696  * using Java-compatible Murmur2 hashing.
2697  * Messages without keys will be assigned via the random partitioner.
2698  *
2699  * The \p rkt_opaque argument is the opaque set by
2700  * rd_kafka_topic_conf_set_opaque().
2701  * The \p msg_opaque argument is the per-message opaque
2702  * passed to produce().
2703  *
2704  * @returns a partition between 0 and \p partition_cnt - 1.
2705  */
2706 RD_EXPORT
2707 int32_t rd_kafka_msg_partitioner_murmur2_random (const rd_kafka_topic_t *rkt,
2708                                                  const void *key, size_t keylen,
2709                                                  int32_t partition_cnt,
2710                                                  void *rkt_opaque,
2711                                                  void *msg_opaque);
2712 
2713 
2714 /**
2715  * @brief FNV-1a partitioner.
2716  *
2717  * Uses consistent hashing to map identical keys onto identical partitions
2718  * using FNV-1a hashing.
2719  *
2720  * The \p rkt_opaque argument is the opaque set by
2721  * rd_kafka_topic_conf_set_opaque().
2722  * The \p msg_opaque argument is the per-message opaque
2723  * passed to produce().
2724  *
2725  * @returns a partition between 0 and \p partition_cnt - 1.
2726  */
2727 RD_EXPORT
2728 int32_t rd_kafka_msg_partitioner_fnv1a (const rd_kafka_topic_t *rkt,
2729                                         const void *key, size_t keylen,
2730                                         int32_t partition_cnt,
2731                                         void *rkt_opaque,
2732                                         void *msg_opaque);
2733 
2734 
2735 /**
2736  * @brief Consistent-Random FNV-1a partitioner.
2737  *
2738  * Uses consistent hashing to map identical keys onto identical partitions
2739  * using FNV-1a hashing.
2740  * Messages without keys will be assigned via the random partitioner.
2741  *
2742  * The \p rkt_opaque argument is the opaque set by
2743  * rd_kafka_topic_conf_set_opaque().
2744  * The \p msg_opaque argument is the per-message opaque
2745  * passed to produce().
2746  *
2747  * @returns a partition between 0 and \p partition_cnt - 1.
2748  */
2749 RD_EXPORT
2750 int32_t rd_kafka_msg_partitioner_fnv1a_random (const rd_kafka_topic_t *rkt,
2751                                                const void *key, size_t keylen,
2752                                                int32_t partition_cnt,
2753                                                void *rkt_opaque,
2754                                                void *msg_opaque);
2755 
2756 
2757 /**@}*/
2758 
2759 
2760 
2761 /**
2762  * @name Main Kafka and Topic object handles
2763  * @{
2764  *
2765  *
2766  */
2767 
2768 
2769 
2770 
2771 /**
2772  * @brief Creates a new Kafka handle and starts its operation according to the
2773  *        specified \p type (\p RD_KAFKA_CONSUMER or \p RD_KAFKA_PRODUCER).
2774  *
2775  * \p conf is an optional struct created with `rd_kafka_conf_new()` that will
2776  * be used instead of the default configuration.
2777  * The \p conf object is freed by this function on success and must not be used
2778  * or destroyed by the application sub-sequently.
2779  * See `rd_kafka_conf_set()` et.al for more information.
2780  *
2781  * \p errstr must be a pointer to memory of at least size \p errstr_size where
2782  * `rd_kafka_new()` may write a human readable error message in case the
2783  * creation of a new handle fails. In which case the function returns NULL.
2784  *
2785  * @remark \b RD_KAFKA_CONSUMER: When a new \p RD_KAFKA_CONSUMER
2786  *           rd_kafka_t handle is created it may either operate in the
2787  *           legacy simple consumer mode using the rd_kafka_consume_start()
2788  *           interface, or the High-level KafkaConsumer API.
2789  * @remark An application must only use one of these groups of APIs on a given
2790  *         rd_kafka_t RD_KAFKA_CONSUMER handle.
2791 
2792  *
2793  * @returns The Kafka handle on success or NULL on error (see \p errstr)
2794  *
2795  * @sa To destroy the Kafka handle, use rd_kafka_destroy().
2796  */
2797 RD_EXPORT
2798 rd_kafka_t *rd_kafka_new(rd_kafka_type_t type, rd_kafka_conf_t *conf,
2799 			  char *errstr, size_t errstr_size);
2800 
2801 
2802 /**
2803  * @brief Destroy Kafka handle.
2804  *
2805  * @remark This is a blocking operation.
2806  * @remark rd_kafka_consumer_close() will be called from this function
2807  *         if the instance type is RD_KAFKA_CONSUMER, a \c group.id was
2808  *         configured, and the rd_kafka_consumer_close() was not
2809  *         explicitly called by the application. This in turn may
2810  *         trigger consumer callbacks, such as rebalance_cb.
2811  *         Use rd_kafka_destroy_flags() with
2812  *         RD_KAFKA_DESTROY_F_NO_CONSUMER_CLOSE to avoid this behaviour.
2813  *
2814  * @sa rd_kafka_destroy_flags()
2815  */
2816 RD_EXPORT
2817 void        rd_kafka_destroy(rd_kafka_t *rk);
2818 
2819 
2820 /**
2821  * @brief Destroy Kafka handle according to specified destroy flags
2822  *
2823  */
2824 RD_EXPORT
2825 void rd_kafka_destroy_flags (rd_kafka_t *rk, int flags);
2826 
2827 /**
2828  * @brief Flags for rd_kafka_destroy_flags()
2829  */
2830 
2831 /*!
2832  * Don't call consumer_close() to leave group and commit final offsets.
2833  *
2834  * This also disables consumer callbacks to be called from rd_kafka_destroy*(),
2835  * such as rebalance_cb.
2836  *
2837  * The consumer group handler is still closed internally, but from an
2838  * application perspective none of the functionality from consumer_close()
2839  * is performed.
2840  */
2841 #define RD_KAFKA_DESTROY_F_NO_CONSUMER_CLOSE 0x8
2842 
2843 
2844 
2845 /**
2846  * @brief Returns Kafka handle name.
2847  */
2848 RD_EXPORT
2849 const char *rd_kafka_name(const rd_kafka_t *rk);
2850 
2851 
2852 /**
2853  * @brief Returns Kafka handle type.
2854  */
2855 RD_EXPORT
2856 rd_kafka_type_t rd_kafka_type(const rd_kafka_t *rk);
2857 
2858 
2859 /**
2860  * @brief Returns this client's broker-assigned group member id.
2861  *
2862  * @remark This currently requires the high-level KafkaConsumer
2863  *
2864  * @returns An allocated string containing the current broker-assigned group
2865  *          member id, or NULL if not available.
2866  *          The application must free the string with \p free() or
2867  *          rd_kafka_mem_free()
2868  */
2869 RD_EXPORT
2870 char *rd_kafka_memberid (const rd_kafka_t *rk);
2871 
2872 
2873 
2874 /**
2875  * @brief Returns the ClusterId as reported in broker metadata.
2876  *
2877  * @param rk         Client instance.
2878  * @param timeout_ms If there is no cached value from metadata retrieval
2879  *                   then this specifies the maximum amount of time
2880  *                   (in milliseconds) the call will block waiting
2881  *                   for metadata to be retrieved.
2882  *                   Use 0 for non-blocking calls.
2883 
2884  * @remark Requires broker version >=0.10.0 and api.version.request=true.
2885  *
2886  * @remark The application must free the returned pointer
2887  *         using rd_kafka_mem_free().
2888  *
2889  * @returns a newly allocated string containing the ClusterId, or NULL
2890  *          if no ClusterId could be retrieved in the allotted timespan.
2891  */
2892 RD_EXPORT
2893 char *rd_kafka_clusterid (rd_kafka_t *rk, int timeout_ms);
2894 
2895 
2896 /**
2897  * @brief Returns the current ControllerId as reported in broker metadata.
2898  *
2899  * @param rk         Client instance.
2900  * @param timeout_ms If there is no cached value from metadata retrieval
2901  *                   then this specifies the maximum amount of time
2902  *                   (in milliseconds) the call will block waiting
2903  *                   for metadata to be retrieved.
2904  *                   Use 0 for non-blocking calls.
2905 
2906  * @remark Requires broker version >=0.10.0 and api.version.request=true.
2907  *
2908  * @returns the controller broker id (>= 0), or -1 if no ControllerId could be
2909  *          retrieved in the allotted timespan.
2910  */
2911 RD_EXPORT
2912 int32_t rd_kafka_controllerid (rd_kafka_t *rk, int timeout_ms);
2913 
2914 
2915 /**
2916  * @brief Creates a new topic handle for topic named \p topic.
2917  *
2918  * \p conf is an optional configuration for the topic created with
2919  * `rd_kafka_topic_conf_new()` that will be used instead of the default
2920  * topic configuration.
2921  * The \p conf object is freed by this function and must not be used or
2922  * destroyed by the application sub-sequently.
2923  * See `rd_kafka_topic_conf_set()` et.al for more information.
2924  *
2925  * Topic handles are refcounted internally and calling rd_kafka_topic_new()
2926  * again with the same topic name will return the previous topic handle
2927  * without updating the original handle's configuration.
2928  * Applications must eventually call rd_kafka_topic_destroy() for each
2929  * succesfull call to rd_kafka_topic_new() to clear up resources.
2930  *
2931  * @returns the new topic handle or NULL on error (use rd_kafka_errno2err()
2932  *          to convert system \p errno to an rd_kafka_resp_err_t error code.
2933  *
2934  * @sa rd_kafka_topic_destroy()
2935  */
2936 RD_EXPORT
2937 rd_kafka_topic_t *rd_kafka_topic_new(rd_kafka_t *rk, const char *topic,
2938 				      rd_kafka_topic_conf_t *conf);
2939 
2940 
2941 
2942 /**
2943  * @brief Loose application's topic handle refcount as previously created
2944  *        with `rd_kafka_topic_new()`.
2945  *
2946  * @remark Since topic objects are refcounted (both internally and for the app)
2947  *         the topic object might not actually be destroyed by this call,
2948  *         but the application must consider the object destroyed.
2949  */
2950 RD_EXPORT
2951 void rd_kafka_topic_destroy(rd_kafka_topic_t *rkt);
2952 
2953 
2954 /**
2955  * @brief Returns the topic name.
2956  */
2957 RD_EXPORT
2958 const char *rd_kafka_topic_name(const rd_kafka_topic_t *rkt);
2959 
2960 
2961 /**
2962  * @brief Get the \p rkt_opaque pointer that was set in the topic configuration
2963  *        with rd_kafka_topic_conf_set_opaque().
2964  */
2965 RD_EXPORT
2966 void *rd_kafka_topic_opaque (const rd_kafka_topic_t *rkt);
2967 
2968 
2969 /**
2970  * @brief Unassigned partition.
2971  *
2972  * The unassigned partition is used by the producer API for messages
2973  * that should be partitioned using the configured or default partitioner.
2974  */
2975 #define RD_KAFKA_PARTITION_UA  ((int32_t)-1)
2976 
2977 
2978 /**
2979  * @brief Polls the provided kafka handle for events.
2980  *
2981  * Events will cause application provided callbacks to be called.
2982  *
2983  * The \p timeout_ms argument specifies the maximum amount of time
2984  * (in milliseconds) that the call will block waiting for events.
2985  * For non-blocking calls, provide 0 as \p timeout_ms.
2986  * To wait indefinately for an event, provide -1.
2987  *
2988  * @remark  An application should make sure to call poll() at regular
2989  *          intervals to serve any queued callbacks waiting to be called.
2990  * @remark  If your producer doesn't have any callback set (in particular
2991  *          via rd_kafka_conf_set_dr_msg_cb or rd_kafka_conf_set_error_cb)
2992  *          you might chose not to call poll(), though this is not
2993  *          recommended.
2994  *
2995  * Events:
2996  *   - delivery report callbacks  (if dr_cb/dr_msg_cb is configured) [producer]
2997  *   - error callbacks (rd_kafka_conf_set_error_cb()) [all]
2998  *   - stats callbacks (rd_kafka_conf_set_stats_cb()) [all]
2999  *   - throttle callbacks (rd_kafka_conf_set_throttle_cb()) [all]
3000  *   - OAUTHBEARER token refresh callbacks (rd_kafka_conf_set_oauthbearer_token_refresh_cb()) [all]
3001  *
3002  * @returns the number of events served.
3003  */
3004 RD_EXPORT
3005 int rd_kafka_poll(rd_kafka_t *rk, int timeout_ms);
3006 
3007 
3008 /**
3009  * @brief Cancels the current callback dispatcher (rd_kafka_poll(),
3010  *        rd_kafka_consume_callback(), etc).
3011  *
3012  * A callback may use this to force an immediate return to the calling
3013  * code (caller of e.g. rd_kafka_poll()) without processing any further
3014  * events.
3015  *
3016  * @remark This function MUST ONLY be called from within a librdkafka callback.
3017  */
3018 RD_EXPORT
3019 void rd_kafka_yield (rd_kafka_t *rk);
3020 
3021 
3022 
3023 
3024 /**
3025  * @brief Pause producing or consumption for the provided list of partitions.
3026  *
3027  * Success or error is returned per-partition \p err in the \p partitions list.
3028  *
3029  * @returns RD_KAFKA_RESP_ERR_NO_ERROR
3030  */
3031 RD_EXPORT rd_kafka_resp_err_t
3032 rd_kafka_pause_partitions (rd_kafka_t *rk,
3033 			   rd_kafka_topic_partition_list_t *partitions);
3034 
3035 
3036 
3037 /**
3038  * @brief Resume producing consumption for the provided list of partitions.
3039  *
3040  * Success or error is returned per-partition \p err in the \p partitions list.
3041  *
3042  * @returns RD_KAFKA_RESP_ERR_NO_ERROR
3043  */
3044 RD_EXPORT rd_kafka_resp_err_t
3045 rd_kafka_resume_partitions (rd_kafka_t *rk,
3046 			    rd_kafka_topic_partition_list_t *partitions);
3047 
3048 
3049 
3050 
3051 /**
3052  * @brief Query broker for low (oldest/beginning) and high (newest/end) offsets
3053  *        for partition.
3054  *
3055  * Offsets are returned in \p *low and \p *high respectively.
3056  *
3057  * @returns RD_KAFKA_RESP_ERR_NO_ERROR on success or an error code on failure.
3058  */
3059 RD_EXPORT rd_kafka_resp_err_t
3060 rd_kafka_query_watermark_offsets (rd_kafka_t *rk,
3061 		      const char *topic, int32_t partition,
3062 		      int64_t *low, int64_t *high, int timeout_ms);
3063 
3064 
3065 /**
3066  * @brief Get last known low (oldest/beginning) and high (newest/end) offsets
3067  *        for partition.
3068  *
3069  * The low offset is updated periodically (if statistics.interval.ms is set)
3070  * while the high offset is updated on each fetched message set from the broker.
3071  *
3072  * If there is no cached offset (either low or high, or both) then
3073  * RD_KAFKA_OFFSET_INVALID will be returned for the respective offset.
3074  *
3075  * Offsets are returned in \p *low and \p *high respectively.
3076  *
3077  * @returns RD_KAFKA_RESP_ERR_NO_ERROR on success or an error code on failure.
3078  *
3079  * @remark Shall only be used with an active consumer instance.
3080  */
3081 RD_EXPORT rd_kafka_resp_err_t
3082 rd_kafka_get_watermark_offsets (rd_kafka_t *rk,
3083 				const char *topic, int32_t partition,
3084 				int64_t *low, int64_t *high);
3085 
3086 
3087 
3088 /**
3089  * @brief Look up the offsets for the given partitions by timestamp.
3090  *
3091  * The returned offset for each partition is the earliest offset whose
3092  * timestamp is greater than or equal to the given timestamp in the
3093  * corresponding partition.
3094  *
3095  * The timestamps to query are represented as \c offset in \p offsets
3096  * on input, and \c offset will contain the offset on output.
3097  *
3098  * The function will block for at most \p timeout_ms milliseconds.
3099  *
3100  * @remark Duplicate Topic+Partitions are not supported.
3101  * @remark Per-partition errors may be returned in \c rd_kafka_topic_partition_t.err
3102  *
3103  * @returns RD_KAFKA_RESP_ERR_NO_ERROR if offsets were be queried (do note
3104  *          that per-partition errors might be set),
3105  *          RD_KAFKA_RESP_ERR__TIMED_OUT if not all offsets could be fetched
3106  *          within \p timeout_ms,
3107  *          RD_KAFKA_RESP_ERR__INVALID_ARG if the \p offsets list is empty,
3108  *          RD_KAFKA_RESP_ERR__UNKNOWN_PARTITION if all partitions are unknown,
3109  *          RD_KAFKA_RESP_ERR_LEADER_NOT_AVAILABLE if unable to query leaders
3110  *          for the given partitions.
3111  */
3112 RD_EXPORT rd_kafka_resp_err_t
3113 rd_kafka_offsets_for_times (rd_kafka_t *rk,
3114                             rd_kafka_topic_partition_list_t *offsets,
3115                             int timeout_ms);
3116 
3117 
3118 
3119 /**
3120  * @brief Allocate and zero memory using the same allocator librdkafka uses.
3121  *
3122  * This is typically an abstraction for the calloc(3) call and makes sure
3123  * the application can use the same memory allocator as librdkafka for
3124  * allocating pointers that are used by librdkafka.
3125  *
3126  * \p rk can be set to return memory allocated by a specific \c rk instance
3127  * otherwise pass NULL for \p rk.
3128  *
3129  * @remark Memory allocated by rd_kafka_mem_calloc() must be freed using
3130  *         rd_kafka_mem_free()
3131  */
3132 RD_EXPORT
3133 void *rd_kafka_mem_calloc (rd_kafka_t *rk, size_t num, size_t size);
3134 
3135 
3136 
3137 /**
3138  * @brief Allocate memory using the same allocator librdkafka uses.
3139  *
3140  * This is typically an abstraction for the malloc(3) call and makes sure
3141  * the application can use the same memory allocator as librdkafka for
3142  * allocating pointers that are used by librdkafka.
3143  *
3144  * \p rk can be set to return memory allocated by a specific \c rk instance
3145  * otherwise pass NULL for \p rk.
3146  *
3147  * @remark Memory allocated by rd_kafka_mem_malloc() must be freed using
3148  *         rd_kafka_mem_free()
3149  */
3150 RD_EXPORT
3151 void *rd_kafka_mem_malloc (rd_kafka_t *rk, size_t size);
3152 
3153 
3154 
3155 /**
3156  * @brief Free pointer returned by librdkafka
3157  *
3158  * This is typically an abstraction for the free(3) call and makes sure
3159  * the application can use the same memory allocator as librdkafka for
3160  * freeing pointers returned by librdkafka.
3161  *
3162  * In standard setups it is usually not necessary to use this interface
3163  * rather than the free(3) functione.
3164  *
3165  * \p rk must be set for memory returned by APIs that take an \c rk argument,
3166  * for other APIs pass NULL for \p rk.
3167  *
3168  * @remark rd_kafka_mem_free() must only be used for pointers returned by APIs
3169  *         that explicitly mention using this function for freeing.
3170  */
3171 RD_EXPORT
3172 void rd_kafka_mem_free (rd_kafka_t *rk, void *ptr);
3173 
3174 
3175 /**@}*/
3176 
3177 
3178 
3179 
3180 
3181 /**
3182  * @name Queue API
3183  * @{
3184  *
3185  * Message queues allows the application to re-route consumed messages
3186  * from multiple topic+partitions into one single queue point.
3187  * This queue point containing messages from a number of topic+partitions
3188  * may then be served by a single rd_kafka_consume*_queue() call,
3189  * rather than one call per topic+partition combination.
3190  */
3191 
3192 
3193 /**
3194  * @brief Create a new message queue.
3195  *
3196  * See rd_kafka_consume_start_queue(), rd_kafka_consume_queue(), et.al.
3197  */
3198 RD_EXPORT
3199 rd_kafka_queue_t *rd_kafka_queue_new(rd_kafka_t *rk);
3200 
3201 /**
3202  * Destroy a queue, purging all of its enqueued messages.
3203  */
3204 RD_EXPORT
3205 void rd_kafka_queue_destroy(rd_kafka_queue_t *rkqu);
3206 
3207 
3208 /**
3209  * @returns a reference to the main librdkafka event queue.
3210  * This is the queue served by rd_kafka_poll().
3211  *
3212  * Use rd_kafka_queue_destroy() to loose the reference.
3213  */
3214 RD_EXPORT
3215 rd_kafka_queue_t *rd_kafka_queue_get_main (rd_kafka_t *rk);
3216 
3217 
3218 /**
3219  * @returns a reference to the librdkafka consumer queue.
3220  * This is the queue served by rd_kafka_consumer_poll().
3221  *
3222  * Use rd_kafka_queue_destroy() to loose the reference.
3223  *
3224  * @remark rd_kafka_queue_destroy() MUST be called on this queue
3225  *         prior to calling rd_kafka_consumer_close().
3226  */
3227 RD_EXPORT
3228 rd_kafka_queue_t *rd_kafka_queue_get_consumer (rd_kafka_t *rk);
3229 
3230 /**
3231  * @returns a reference to the partition's queue, or NULL if
3232  *          partition is invalid.
3233  *
3234  * Use rd_kafka_queue_destroy() to loose the reference.
3235  *
3236  * @remark rd_kafka_queue_destroy() MUST be called on this queue
3237  *
3238  * @remark This function only works on consumers.
3239  */
3240 RD_EXPORT
3241 rd_kafka_queue_t *rd_kafka_queue_get_partition (rd_kafka_t *rk,
3242                                                 const char *topic,
3243                                                 int32_t partition);
3244 
3245 /**
3246  * @returns a reference to the background thread queue, or NULL if the
3247  *          background queue is not enabled.
3248  *
3249  * To enable the background thread queue set a generic event handler callback
3250  * with rd_kafka_conf_set_background_event_cb() on the client instance
3251  * configuration object (rd_kafka_conf_t).
3252  *
3253  * The background queue is polled and served by librdkafka and MUST NOT be
3254  * polled, forwarded, or otherwise managed by the application, it may only
3255  * be used as the destination queue passed to queue-enabled APIs, such as
3256  * the Admin API.
3257  *
3258  * The background thread queue provides the application with an automatically
3259  * polled queue that triggers the event callback in a background thread,
3260  * this background thread is completely managed by librdkafka.
3261  *
3262  * Use rd_kafka_queue_destroy() to loose the reference.
3263  *
3264  * @warning The background queue MUST NOT be read from (polled, consumed, etc),
3265  *          or forwarded from.
3266  */
3267 RD_EXPORT
3268 rd_kafka_queue_t *rd_kafka_queue_get_background (rd_kafka_t *rk);
3269 
3270 
3271 /**
3272  * @brief Forward/re-route queue \p src to \p dst.
3273  * If \p dst is \c NULL the forwarding is removed.
3274  *
3275  * The internal refcounts for both queues are increased.
3276  *
3277  * @remark Regardless of whether \p dst is NULL or not, after calling this
3278  *         function, \p src will not forward it's fetch queue to the consumer
3279  *         queue.
3280  */
3281 RD_EXPORT
3282 void rd_kafka_queue_forward (rd_kafka_queue_t *src, rd_kafka_queue_t *dst);
3283 
3284 /**
3285  * @brief Forward librdkafka logs (and debug) to the specified queue
3286  *        for serving with one of the ..poll() calls.
3287  *
3288  *        This allows an application to serve log callbacks (\c log_cb)
3289  *        in its thread of choice.
3290  *
3291  * @param rk   Client instance.
3292  * @param rkqu Queue to forward logs to. If the value is NULL the logs
3293  *        are forwarded to the main queue.
3294  *
3295  * @remark The configuration property \c log.queue MUST also be set to true.
3296  *
3297  * @remark librdkafka maintains its own reference to the provided queue.
3298  *
3299  * @returns RD_KAFKA_RESP_ERR_NO_ERROR on success or an error code on error.
3300  */
3301 RD_EXPORT
3302 rd_kafka_resp_err_t rd_kafka_set_log_queue (rd_kafka_t *rk,
3303                                             rd_kafka_queue_t *rkqu);
3304 
3305 
3306 /**
3307  * @returns the current number of elements in queue.
3308  */
3309 RD_EXPORT
3310 size_t rd_kafka_queue_length (rd_kafka_queue_t *rkqu);
3311 
3312 
3313 /**
3314  * @brief Enable IO event triggering for queue.
3315  *
3316  * To ease integration with IO based polling loops this API
3317  * allows an application to create a separate file-descriptor
3318  * that librdkafka will write \p payload (of size \p size) to
3319  * whenever a new element is enqueued on a previously empty queue.
3320  *
3321  * To remove event triggering call with \p fd = -1.
3322  *
3323  * librdkafka will maintain a copy of the \p payload.
3324  *
3325  * @remark IO and callback event triggering are mutually exclusive.
3326  * @remark When using forwarded queues the IO event must only be enabled
3327  *         on the final forwarded-to (destination) queue.
3328  * @remark The file-descriptor/socket must be set to non-blocking.
3329  */
3330 RD_EXPORT
3331 void rd_kafka_queue_io_event_enable (rd_kafka_queue_t *rkqu, int fd,
3332                                      const void *payload, size_t size);
3333 
3334 /**
3335  * @brief Enable callback event triggering for queue.
3336  *
3337  * The callback will be called from an internal librdkafka thread
3338  * when a new element is enqueued on a previously empty queue.
3339  *
3340  * To remove event triggering call with \p event_cb = NULL.
3341  *
3342  * The \p qev_opaque is passed to the callback's \p qev_opaque argument.
3343  *
3344  * @remark IO and callback event triggering are mutually exclusive.
3345  * @remark Since the callback may be triggered from internal librdkafka
3346  *         threads, the application must not perform any pro-longed work in
3347  *         the callback, or call any librdkafka APIs (for the same rd_kafka_t
3348  *         handle).
3349  */
3350 RD_EXPORT
3351 void rd_kafka_queue_cb_event_enable (rd_kafka_queue_t *rkqu,
3352                                      void (*event_cb) (rd_kafka_t *rk,
3353                                                        void *qev_opaque),
3354                                      void *qev_opaque);
3355 
3356 
3357 /**
3358  * @brief Cancels the current rd_kafka_queue_poll() on \p rkqu.
3359  *
3360  * An application may use this from another thread to force
3361  * an immediate return to the calling code (caller of rd_kafka_queue_poll()).
3362  * Must not be used from signal handlers since that may cause deadlocks.
3363  */
3364 RD_EXPORT
3365 void rd_kafka_queue_yield (rd_kafka_queue_t *rkqu);
3366 
3367 
3368 /**@}*/
3369 
3370 /**
3371  *
3372  * @name Simple Consumer API (legacy)
3373  * @{
3374  *
3375  */
3376 
3377 
3378 #define RD_KAFKA_OFFSET_BEGINNING -2  /**< Start consuming from beginning of
3379 				       *   kafka partition queue: oldest msg */
3380 #define RD_KAFKA_OFFSET_END       -1  /**< Start consuming from end of kafka
3381 				       *   partition queue: next msg */
3382 #define RD_KAFKA_OFFSET_STORED -1000  /**< Start consuming from offset retrieved
3383 				       *   from offset store */
3384 #define RD_KAFKA_OFFSET_INVALID -1001 /**< Invalid offset */
3385 
3386 
3387 /** @cond NO_DOC */
3388 #define RD_KAFKA_OFFSET_TAIL_BASE -2000 /* internal: do not use */
3389 /** @endcond */
3390 
3391 /**
3392  * @brief Start consuming \p CNT messages from topic's current end offset.
3393  *
3394  * That is, if current end offset is 12345 and \p CNT is 200, it will start
3395  * consuming from offset \c 12345-200 = \c 12145. */
3396 #define RD_KAFKA_OFFSET_TAIL(CNT)  (RD_KAFKA_OFFSET_TAIL_BASE - (CNT))
3397 
3398 /**
3399  * @brief Start consuming messages for topic \p rkt and \p partition
3400  * at offset \p offset which may either be an absolute \c (0..N)
3401  * or one of the logical offsets:
3402  *  - RD_KAFKA_OFFSET_BEGINNING
3403  *  - RD_KAFKA_OFFSET_END
3404  *  - RD_KAFKA_OFFSET_STORED
3405  *  - RD_KAFKA_OFFSET_TAIL
3406  *
3407  * rdkafka will attempt to keep \c queued.min.messages (config property)
3408  * messages in the local queue by repeatedly fetching batches of messages
3409  * from the broker until the threshold is reached.
3410  *
3411  * The application shall use one of the `rd_kafka_consume*()` functions
3412  * to consume messages from the local queue, each kafka message being
3413  * represented as a `rd_kafka_message_t *` object.
3414  *
3415  * `rd_kafka_consume_start()` must not be called multiple times for the same
3416  * topic and partition without stopping consumption first with
3417  * `rd_kafka_consume_stop()`.
3418  *
3419  * @returns 0 on success or -1 on error in which case errno is set accordingly:
3420  *  - EBUSY    - Conflicts with an existing or previous subscription
3421  *               (RD_KAFKA_RESP_ERR__CONFLICT)
3422  *  - EINVAL   - Invalid offset, or incomplete configuration (lacking group.id)
3423  *               (RD_KAFKA_RESP_ERR__INVALID_ARG)
3424  *  - ESRCH    - requested \p partition is invalid.
3425  *               (RD_KAFKA_RESP_ERR__UNKNOWN_PARTITION)
3426  *  - ENOENT   - topic is unknown in the Kafka cluster.
3427  *               (RD_KAFKA_RESP_ERR__UNKNOWN_TOPIC)
3428  *
3429  * Use `rd_kafka_errno2err()` to convert sytem \c errno to `rd_kafka_resp_err_t`
3430  */
3431 RD_EXPORT
3432 int rd_kafka_consume_start(rd_kafka_topic_t *rkt, int32_t partition,
3433 			    int64_t offset);
3434 
3435 /**
3436  * @brief Same as rd_kafka_consume_start() but re-routes incoming messages to
3437  * the provided queue \p rkqu (which must have been previously allocated
3438  * with `rd_kafka_queue_new()`.
3439  *
3440  * The application must use one of the `rd_kafka_consume_*_queue()` functions
3441  * to receive fetched messages.
3442  *
3443  * `rd_kafka_consume_start_queue()` must not be called multiple times for the
3444  * same topic and partition without stopping consumption first with
3445  * `rd_kafka_consume_stop()`.
3446  * `rd_kafka_consume_start()` and `rd_kafka_consume_start_queue()` must not
3447  * be combined for the same topic and partition.
3448  */
3449 RD_EXPORT
3450 int rd_kafka_consume_start_queue(rd_kafka_topic_t *rkt, int32_t partition,
3451 				  int64_t offset, rd_kafka_queue_t *rkqu);
3452 
3453 /**
3454  * @brief Stop consuming messages for topic \p rkt and \p partition, purging
3455  * all messages currently in the local queue.
3456  *
3457  * NOTE: To enforce synchronisation this call will block until the internal
3458  *       fetcher has terminated and offsets are committed to configured
3459  *       storage method.
3460  *
3461  * The application needs to be stop all consumers before calling
3462  * `rd_kafka_destroy()` on the main object handle.
3463  *
3464  * @returns 0 on success or -1 on error (see `errno`).
3465  */
3466 RD_EXPORT
3467 int rd_kafka_consume_stop(rd_kafka_topic_t *rkt, int32_t partition);
3468 
3469 
3470 
3471 /**
3472  * @brief Seek consumer for topic+partition to \p offset which is either an
3473  *        absolute or logical offset.
3474  *
3475  * If \p timeout_ms is not 0 the call will wait this long for the
3476  * seek to be performed. If the timeout is reached the internal state
3477  * will be unknown and this function returns `RD_KAFKA_RESP_ERR__TIMED_OUT`.
3478  * If \p timeout_ms is 0 it will initiate the seek but return
3479  * immediately without any error reporting (e.g., async).
3480  *
3481  * This call will purge all pre-fetched messages for the given partition, which
3482  * may be up to \c queued.max.message.kbytes in size. Repeated use of seek
3483  * may thus lead to increased network usage as messages are re-fetched from
3484  * the broker.
3485  *
3486  * @remark Seek must only be performed for already assigned/consumed partitions,
3487  *         use rd_kafka_assign() (et.al) to set the initial starting offset
3488  *         for a new assignmenmt.
3489  *
3490  * @returns `RD_KAFKA_RESP_ERR__NO_ERROR` on success else an error code.
3491  *
3492  * @deprecated Use rd_kafka_seek_partitions().
3493  */
3494 RD_EXPORT
3495 rd_kafka_resp_err_t rd_kafka_seek (rd_kafka_topic_t *rkt,
3496                                    int32_t partition,
3497                                    int64_t offset,
3498                                    int timeout_ms);
3499 
3500 
3501 
3502 /**
3503  * @brief Seek consumer for partitions in \p partitions to the per-partition
3504  *        offset in the \c .offset field of \p partitions.
3505  *
3506  * The offset may be either absolute (>= 0) or a logical offset.
3507  *
3508  * If \p timeout_ms is not 0 the call will wait this long for the
3509  * seeks to be performed. If the timeout is reached the internal state
3510  * will be unknown for the remaining partitions to seek and this function
3511  * will return an error with the error code set to
3512  * `RD_KAFKA_RESP_ERR__TIMED_OUT`.
3513  *
3514  * If \p timeout_ms is 0 it will initiate the seek but return
3515  * immediately without any error reporting (e.g., async).
3516  *
3517  * This call will purge all pre-fetched messages for the given partition, which
3518  * may be up to \c queued.max.message.kbytes in size. Repeated use of seek
3519  * may thus lead to increased network usage as messages are re-fetched from
3520  * the broker.
3521  *
3522  * Individual partition errors are reported in the per-partition \c .err field
3523  * of \p partitions.
3524  *
3525  * @remark Seek must only be performed for already assigned/consumed partitions,
3526  *         use rd_kafka_assign() (et.al) to set the initial starting offset
3527  *         for a new assignmenmt.
3528  *
3529  * @returns NULL on success or an error object on failure.
3530  */
3531 RD_EXPORT rd_kafka_error_t *
3532 rd_kafka_seek_partitions (rd_kafka_t *rk,
3533                           rd_kafka_topic_partition_list_t *partitions,
3534                           int timeout_ms);
3535 
3536 
3537 /**
3538  * @brief Consume a single message from topic \p rkt and \p partition
3539  *
3540  * \p timeout_ms is maximum amount of time to wait for a message to be received.
3541  * Consumer must have been previously started with `rd_kafka_consume_start()`.
3542  *
3543  * @returns a message object on success or \c NULL on error.
3544  * The message object must be destroyed with `rd_kafka_message_destroy()`
3545  * when the application is done with it.
3546  *
3547  * Errors (when returning NULL):
3548  *  - ETIMEDOUT - \p timeout_ms was reached with no new messages fetched.
3549  *  - ENOENT    - \p rkt + \p partition is unknown.
3550  *                 (no prior `rd_kafka_consume_start()` call)
3551  *
3552  * NOTE: The returned message's \c ..->err must be checked for errors.
3553  * NOTE: \c ..->err \c == \c RD_KAFKA_RESP_ERR__PARTITION_EOF signals that the
3554  *       end of the partition has been reached, which should typically not be
3555  *       considered an error. The application should handle this case
3556  *       (e.g., ignore).
3557  *
3558  * @remark on_consume() interceptors may be called from this function prior to
3559  *         passing message to application.
3560  */
3561 RD_EXPORT
3562 rd_kafka_message_t *rd_kafka_consume(rd_kafka_topic_t *rkt, int32_t partition,
3563 				      int timeout_ms);
3564 
3565 
3566 
3567 /**
3568  * @brief Consume up to \p rkmessages_size from topic \p rkt and \p partition
3569  *        putting a pointer to each message in the application provided
3570  *        array \p rkmessages (of size \p rkmessages_size entries).
3571  *
3572  * `rd_kafka_consume_batch()` provides higher throughput performance
3573  * than `rd_kafka_consume()`.
3574  *
3575  * \p timeout_ms is the maximum amount of time to wait for all of
3576  * \p rkmessages_size messages to be put into \p rkmessages.
3577  * If no messages were available within the timeout period this function
3578  * returns 0 and \p rkmessages remains untouched.
3579  * This differs somewhat from `rd_kafka_consume()`.
3580  *
3581  * The message objects must be destroyed with `rd_kafka_message_destroy()`
3582  * when the application is done with it.
3583  *
3584  * @returns the number of rkmessages added in \p rkmessages,
3585  * or -1 on error (same error codes as for `rd_kafka_consume()`.
3586  *
3587  * @sa rd_kafka_consume()
3588  *
3589  * @remark on_consume() interceptors may be called from this function prior to
3590  *         passing message to application.
3591  */
3592 RD_EXPORT
3593 ssize_t rd_kafka_consume_batch(rd_kafka_topic_t *rkt, int32_t partition,
3594 				int timeout_ms,
3595 				rd_kafka_message_t **rkmessages,
3596 				size_t rkmessages_size);
3597 
3598 
3599 
3600 /**
3601  * @brief Consumes messages from topic \p rkt and \p partition, calling
3602  * the provided callback for each consumed messsage.
3603  *
3604  * `rd_kafka_consume_callback()` provides higher throughput performance
3605  * than both `rd_kafka_consume()` and `rd_kafka_consume_batch()`.
3606  *
3607  * \p timeout_ms is the maximum amount of time to wait for one or more messages
3608  * to arrive.
3609  *
3610  * The provided \p consume_cb function is called for each message,
3611  * the application \b MUST \b NOT call `rd_kafka_message_destroy()` on the
3612  * provided \p rkmessage.
3613  *
3614  * The \p commit_opaque argument is passed to the \p consume_cb
3615  * as \p commit_opaque.
3616  *
3617  * @returns the number of messages processed or -1 on error.
3618  *
3619  * @sa rd_kafka_consume()
3620  *
3621  * @remark on_consume() interceptors may be called from this function prior to
3622  *         passing message to application.
3623  *
3624  * @remark This function will return early if a transaction control message is
3625  *         received, these messages are not exposed to the application but
3626  *         still enqueued on the consumer queue to make sure their
3627  *         offsets are stored.
3628  *
3629  * @deprecated This API is deprecated and subject for future removal.
3630  *             There is no new callback-based consume interface, use the
3631  *             poll/queue based alternatives.
3632  */
3633 RD_EXPORT
3634 int rd_kafka_consume_callback(rd_kafka_topic_t *rkt, int32_t partition,
3635                               int timeout_ms,
3636                               void (*consume_cb) (rd_kafka_message_t
3637                                                   *rkmessage,
3638                                                   void *commit_opaque),
3639                               void *commit_opaque);
3640 
3641 
3642 /**
3643  * @name Simple Consumer API (legacy): Queue consumers
3644  * @{
3645  *
3646  * The following `..._queue()` functions are analogue to the functions above
3647  * but reads messages from the provided queue \p rkqu instead.
3648  * \p rkqu must have been previously created with `rd_kafka_queue_new()`
3649  * and the topic consumer must have been started with
3650  * `rd_kafka_consume_start_queue()` utilising the the same queue.
3651  */
3652 
3653 /**
3654  * @brief Consume from queue
3655  *
3656  * @sa rd_kafka_consume()
3657  */
3658 RD_EXPORT
3659 rd_kafka_message_t *rd_kafka_consume_queue(rd_kafka_queue_t *rkqu,
3660                                            int timeout_ms);
3661 
3662 /**
3663  * @brief Consume batch of messages from queue
3664  *
3665  * @sa rd_kafka_consume_batch()
3666  */
3667 RD_EXPORT
3668 ssize_t rd_kafka_consume_batch_queue(rd_kafka_queue_t *rkqu,
3669 				      int timeout_ms,
3670 				      rd_kafka_message_t **rkmessages,
3671 				      size_t rkmessages_size);
3672 
3673 /**
3674  * @brief Consume multiple messages from queue with callback
3675  *
3676  * @sa rd_kafka_consume_callback()
3677  *
3678  * @deprecated This API is deprecated and subject for future removal.
3679  *             There is no new callback-based consume interface, use the
3680  *             poll/queue based alternatives.
3681  */
3682 RD_EXPORT
3683 int rd_kafka_consume_callback_queue (rd_kafka_queue_t *rkqu,
3684                                      int timeout_ms,
3685                                      void (*consume_cb) (rd_kafka_message_t
3686                                                          *rkmessage,
3687                                                          void *commit_opaque),
3688                                      void *commit_opaque);
3689 
3690 
3691 /**@}*/
3692 
3693 
3694 
3695 
3696 /**
3697  * @name Simple Consumer API (legacy): Topic+partition offset store.
3698  * @{
3699  *
3700  * If \c auto.commit.enable is true the offset is stored automatically prior to
3701  * returning of the message(s) in each of the rd_kafka_consume*() functions
3702  * above.
3703  */
3704 
3705 
3706 /**
3707  * @brief Store offset \p offset + 1 for topic \p rkt partition \p partition.
3708  *
3709  * The \c offset + 1 will be committed (written) to broker (or file) according
3710  * to \c `auto.commit.interval.ms` or manual offset-less commit()
3711  *
3712  * @remark \c `enable.auto.offset.store` must be set to "false" when using
3713  *         this API.
3714  *
3715  * @returns RD_KAFKA_RESP_ERR_NO_ERROR on success or an error code on error.
3716  */
3717 RD_EXPORT
3718 rd_kafka_resp_err_t rd_kafka_offset_store (rd_kafka_topic_t *rkt,
3719 					   int32_t partition, int64_t offset);
3720 
3721 
3722 /**
3723  * @brief Store offsets for next auto-commit for one or more partitions.
3724  *
3725  * The offset will be committed (written) to the offset store according
3726  * to \c `auto.commit.interval.ms` or manual offset-less commit().
3727  *
3728  * Per-partition success/error status propagated through each partition's
3729  * \c .err field.
3730  *
3731  * @remark The \c .offset field is stored as is, it will NOT be + 1.
3732  *
3733  * @remark \c `enable.auto.offset.store` must be set to "false" when using
3734  *         this API.
3735  *
3736  * @returns RD_KAFKA_RESP_ERR_NO_ERROR on success, or
3737  *          RD_KAFKA_RESP_ERR__UNKNOWN_PARTITION if none of the
3738  *          offsets could be stored, or
3739  *          RD_KAFKA_RESP_ERR__INVALID_ARG if \c enable.auto.offset.store
3740  *          is true.
3741  */
3742 RD_EXPORT rd_kafka_resp_err_t
3743 rd_kafka_offsets_store (rd_kafka_t *rk,
3744                         rd_kafka_topic_partition_list_t *offsets);
3745 /**@}*/
3746 
3747 
3748 
3749 
3750 /**
3751  * @name KafkaConsumer (C)
3752  * @{
3753  * @brief High-level KafkaConsumer C API
3754  *
3755  *
3756  *
3757  */
3758 
3759 /**
3760  * @brief Subscribe to topic set using balanced consumer groups.
3761  *
3762  * Wildcard (regex) topics are supported:
3763  * any topic name in the \p topics list that is prefixed with \c \"^\" will
3764  * be regex-matched to the full list of topics in the cluster and matching
3765  * topics will be added to the subscription list.
3766  *
3767  * The full topic list is retrieved every \c topic.metadata.refresh.interval.ms
3768  * to pick up new or delete topics that match the subscription.
3769  * If there is any change to the matched topics the consumer will
3770  * immediately rejoin the group with the updated set of subscribed topics.
3771  *
3772  * Regex and full topic names can be mixed in \p topics.
3773  *
3774  * @remark Only the \c .topic field is used in the supplied \p topics list,
3775  *         all other fields are ignored.
3776  *
3777  * @remark subscribe() is an asynchronous method which returns immediately:
3778  *         background threads will (re)join the group, wait for group rebalance,
3779  *         issue any registered rebalance_cb, assign() the assigned partitions,
3780  *         and then start fetching messages. This cycle may take up to
3781  *         \c session.timeout.ms * 2 or more to complete.
3782  *
3783  * @remark A consumer error will be raised for each unavailable topic in the
3784  *         \p topics. The error will be RD_KAFKA_RESP_ERR_UNKNOWN_TOPIC_OR_PART
3785  *         for non-existent topics, and
3786  *         RD_KAFKA_RESP_ERR_TOPIC_AUTHORIZATION_FAILED for unauthorized topics.
3787  *         The consumer error will be raised through rd_kafka_consumer_poll()
3788  *         (et.al.) with the \c rd_kafka_message_t.err field set to one of the
3789  *         error codes mentioned above.
3790  *         The subscribe function itself is asynchronous and will not return
3791  *         an error on unavailable topics.
3792  *
3793  * @returns RD_KAFKA_RESP_ERR_NO_ERROR on success or
3794  *          RD_KAFKA_RESP_ERR__INVALID_ARG if list is empty, contains invalid
3795  *          topics or regexes or duplicate entries,
3796  *          RD_KAFKA_RESP_ERR__FATAL if the consumer has raised a fatal error.
3797  */
3798 RD_EXPORT rd_kafka_resp_err_t
3799 rd_kafka_subscribe (rd_kafka_t *rk,
3800                     const rd_kafka_topic_partition_list_t *topics);
3801 
3802 
3803 /**
3804  * @brief Unsubscribe from the current subscription set.
3805  */
3806 RD_EXPORT
3807 rd_kafka_resp_err_t rd_kafka_unsubscribe (rd_kafka_t *rk);
3808 
3809 
3810 /**
3811  * @brief Returns the current topic subscription
3812  *
3813  * @returns An error code on failure, otherwise \p topic is updated
3814  *          to point to a newly allocated topic list (possibly empty).
3815  *
3816  * @remark The application is responsible for calling
3817  *         rd_kafka_topic_partition_list_destroy on the returned list.
3818  */
3819 RD_EXPORT rd_kafka_resp_err_t
3820 rd_kafka_subscription (rd_kafka_t *rk,
3821                        rd_kafka_topic_partition_list_t **topics);
3822 
3823 
3824 
3825 /**
3826  * @brief Poll the consumer for messages or events.
3827  *
3828  * Will block for at most \p timeout_ms milliseconds.
3829  *
3830  * @remark  An application should make sure to call consumer_poll() at regular
3831  *          intervals, even if no messages are expected, to serve any
3832  *          queued callbacks waiting to be called. This is especially
3833  *          important when a rebalance_cb has been registered as it needs
3834  *          to be called and handled properly to synchronize internal
3835  *          consumer state.
3836  *
3837  * @returns A message object which is a proper message if \p ->err is
3838  *          RD_KAFKA_RESP_ERR_NO_ERROR, or an event or error for any other
3839  *          value.
3840  *
3841  * @remark on_consume() interceptors may be called from this function prior to
3842  *         passing message to application.
3843  *
3844  * @remark When subscribing to topics the application must call poll at
3845  *         least every \c max.poll.interval.ms to remain a member of the
3846  *         consumer group.
3847  *
3848  * Noteworthy errors returned in \c ->err:
3849  * - RD_KAFKA_RESP_ERR__MAX_POLL_EXCEEDED - application failed to call
3850  *   poll within `max.poll.interval.ms`.
3851  *
3852  * @sa rd_kafka_message_t
3853  */
3854 RD_EXPORT
3855 rd_kafka_message_t *rd_kafka_consumer_poll (rd_kafka_t *rk, int timeout_ms);
3856 
3857 /**
3858  * @brief Close down the KafkaConsumer.
3859  *
3860  * @remark This call will block until the consumer has revoked its assignment,
3861  *         calling the \c rebalance_cb if it is configured, committed offsets
3862  *         to broker, and left the consumer group.
3863  *         The maximum blocking time is roughly limited to session.timeout.ms.
3864  *
3865  * @returns An error code indicating if the consumer close was succesful
3866  *          or not.
3867  *          RD_KAFKA_RESP_ERR__FATAL is returned if the consumer has raised
3868  *          a fatal error.
3869  *
3870  * @remark The application still needs to call rd_kafka_destroy() after
3871  *         this call finishes to clean up the underlying handle resources.
3872  *
3873  */
3874 RD_EXPORT
3875 rd_kafka_resp_err_t rd_kafka_consumer_close (rd_kafka_t *rk);
3876 
3877 
3878 /**
3879  * @brief Incrementally add \p partitions to the current assignment.
3880  *
3881  * If a COOPERATIVE assignor (i.e. incremental rebalancing) is being used,
3882  * this method should be used in a rebalance callback to adjust the current
3883  * assignment appropriately in the case where the rebalance type is
3884  * RD_KAFKA_RESP_ERR__ASSIGN_PARTITIONS. The application must pass the
3885  * partition list passed to the callback (or a copy of it), even if the
3886  * list is empty. \p partitions must not be NULL. This method may also be
3887  * used outside the context of a rebalance callback.
3888  *
3889  * @returns NULL on success, or an error object if the operation was
3890  *          unsuccessful.
3891  *
3892  * @remark The returned error object (if not NULL) must be destroyed with
3893  *         rd_kafka_error_destroy().
3894  */
3895 RD_EXPORT rd_kafka_error_t *
3896 rd_kafka_incremental_assign (rd_kafka_t *rk,
3897                              const rd_kafka_topic_partition_list_t
3898                              *partitions);
3899 
3900 
3901 /**
3902  * @brief Incrementally remove \p partitions from the current assignment.
3903  *
3904  * If a COOPERATIVE assignor (i.e. incremental rebalancing) is being used,
3905  * this method should be used in a rebalance callback to adjust the current
3906  * assignment appropriately in the case where the rebalance type is
3907  * RD_KAFKA_RESP_ERR__REVOKE_PARTITIONS. The application must pass the
3908  * partition list passed to the callback (or a copy of it), even if the
3909  * list is empty. \p partitions must not be NULL. This method may also be
3910  * used outside the context of a rebalance callback.
3911  *
3912  * @returns NULL on success, or an error object if the operation was
3913  *          unsuccessful.
3914  *
3915  * @remark The returned error object (if not NULL) must be destroyed with
3916  *         rd_kafka_error_destroy().
3917  */
3918 RD_EXPORT rd_kafka_error_t *
3919 rd_kafka_incremental_unassign (rd_kafka_t *rk,
3920                                const rd_kafka_topic_partition_list_t
3921                                *partitions);
3922 
3923 
3924 /**
3925  * @brief The rebalance protocol currently in use. This will be
3926  *        "NONE" if the consumer has not (yet) joined a group, else it will
3927  *        match the rebalance protocol ("EAGER", "COOPERATIVE") of the
3928  *        configured and selected assignor(s). All configured
3929  *        assignors must have the same protocol type, meaning
3930  *        online migration of a consumer group from using one
3931  *        protocol to another (in particular upgading from EAGER
3932  *        to COOPERATIVE) without a restart is not currently
3933  *        supported.
3934  *
3935  * @returns NULL on error, or one of "NONE", "EAGER", "COOPERATIVE" on success.
3936  */
3937 RD_EXPORT
3938 const char *rd_kafka_rebalance_protocol (rd_kafka_t *rk);
3939 
3940 
3941 /**
3942  * @brief Atomic assignment of partitions to consume.
3943  *
3944  * The new \p partitions will replace the existing assignment.
3945  *
3946  * A zero-length \p partitions will treat the partitions as a valid,
3947  * albeit empty assignment, and maintain internal state, while a \c NULL
3948  * value for \p partitions will reset and clear the internal state.
3949  *
3950  * When used from a rebalance callback, the application should pass the
3951  * partition list passed to the callback (or a copy of it) even if the list
3952  * is empty (i.e. should not pass NULL in this case) so as to maintain
3953  * internal join state. This is not strictly required - the application
3954  * may adjust the assignment provided by the group. However, this is rarely
3955  * useful in practice.
3956  *
3957  * @returns An error code indicating if the new assignment was applied or not.
3958  *          RD_KAFKA_RESP_ERR__FATAL is returned if the consumer has raised
3959  *          a fatal error.
3960  */
3961 RD_EXPORT rd_kafka_resp_err_t
3962 rd_kafka_assign (rd_kafka_t *rk,
3963                  const rd_kafka_topic_partition_list_t *partitions);
3964 
3965 /**
3966  * @brief Returns the current partition assignment as set by rd_kafka_assign()
3967  *        or rd_kafka_incremental_assign().
3968  *
3969  * @returns An error code on failure, otherwise \p partitions is updated
3970  *          to point to a newly allocated partition list (possibly empty).
3971  *
3972  * @remark The application is responsible for calling
3973  *         rd_kafka_topic_partition_list_destroy on the returned list.
3974  *
3975  * @remark This assignment represents the partitions assigned through the
3976  *         assign functions and not the partitions assigned to this consumer
3977  *         instance by the consumer group leader.
3978  *         They are usually the same following a rebalance but not necessarily
3979  *         since an application is free to assign any partitions.
3980  */
3981 RD_EXPORT rd_kafka_resp_err_t
3982 rd_kafka_assignment (rd_kafka_t *rk,
3983                      rd_kafka_topic_partition_list_t **partitions);
3984 
3985 
3986 /**
3987  * @brief Check whether the consumer considers the current assignment to
3988  *        have been lost involuntarily. This method is only applicable for
3989  *        use with a high level subscribing consumer. Assignments are revoked
3990  *        immediately when determined to have been lost, so this method
3991  *        is only useful when reacting to a RD_KAFKA_EVENT_REBALANCE event
3992  *        or from within a rebalance_cb. Partitions that have been lost may
3993  *        already be owned by other members in the group and therefore
3994  *        commiting offsets, for example, may fail.
3995  *
3996  * @remark Calling rd_kafka_assign(), rd_kafka_incremental_assign() or
3997  *         rd_kafka_incremental_unassign() resets this flag.
3998  *
3999  * @returns Returns 1 if the current partition assignment is considered
4000  *          lost, 0 otherwise.
4001  */
4002 RD_EXPORT int
4003 rd_kafka_assignment_lost (rd_kafka_t *rk);
4004 
4005 
4006 /**
4007  * @brief Commit offsets on broker for the provided list of partitions.
4008  *
4009  * \p offsets should contain \c topic, \c partition, \c offset and possibly
4010  * \c metadata. The \c offset should be the offset where consumption will
4011  * resume, i.e., the last processed offset + 1.
4012  * If \p offsets is NULL the current partition assignment will be used instead.
4013  *
4014  * If \p async is false this operation will block until the broker offset commit
4015  * is done, returning the resulting success or error code.
4016  *
4017  * If a rd_kafka_conf_set_offset_commit_cb() offset commit callback has been
4018  * configured the callback will be enqueued for a future call to
4019  * rd_kafka_poll(), rd_kafka_consumer_poll() or similar.
4020  *
4021  * @returns An error code indiciating if the commit was successful,
4022  *          or successfully scheduled if asynchronous, or failed.
4023  *          RD_KAFKA_RESP_ERR__FATAL is returned if the consumer has raised
4024  *          a fatal error.
4025  */
4026 RD_EXPORT rd_kafka_resp_err_t
4027 rd_kafka_commit (rd_kafka_t *rk, const rd_kafka_topic_partition_list_t *offsets,
4028                  int async);
4029 
4030 
4031 /**
4032  * @brief Commit message's offset on broker for the message's partition.
4033  *        The committed offset is the message's offset + 1.
4034  *
4035  * @sa rd_kafka_commit
4036  */
4037 RD_EXPORT rd_kafka_resp_err_t
4038 rd_kafka_commit_message (rd_kafka_t *rk, const rd_kafka_message_t *rkmessage,
4039                          int async);
4040 
4041 
4042 /**
4043  * @brief Commit offsets on broker for the provided list of partitions.
4044  *
4045  * See rd_kafka_commit for \p offsets semantics.
4046  *
4047  * The result of the offset commit will be posted on the provided \p rkqu queue.
4048  *
4049  * If the application uses one of the poll APIs (rd_kafka_poll(),
4050  * rd_kafka_consumer_poll(), rd_kafka_queue_poll(), ..) to serve the queue
4051  * the \p cb callback is required.
4052  *
4053  * The \p commit_opaque argument is passed to the callback as \p commit_opaque,
4054  * or if using the event API the callback is ignored and the offset commit
4055  * result will be returned as an RD_KAFKA_EVENT_COMMIT event and the
4056  * \p commit_opaque value will be available with rd_kafka_event_opaque().
4057  *
4058  * If \p rkqu is NULL a temporary queue will be created and the callback will
4059  * be served by this call.
4060  *
4061  * @sa rd_kafka_commit()
4062  * @sa rd_kafka_conf_set_offset_commit_cb()
4063  */
4064 RD_EXPORT rd_kafka_resp_err_t
4065 rd_kafka_commit_queue (rd_kafka_t *rk,
4066 		       const rd_kafka_topic_partition_list_t *offsets,
4067 		       rd_kafka_queue_t *rkqu,
4068 		       void (*cb) (rd_kafka_t *rk,
4069 				   rd_kafka_resp_err_t err,
4070 				   rd_kafka_topic_partition_list_t *offsets,
4071                                    void *commit_opaque),
4072 		       void *commit_opaque);
4073 
4074 
4075 /**
4076  * @brief Retrieve committed offsets for topics+partitions.
4077  *
4078  * The \p offset field of each requested partition will either be set to
4079  * stored offset or to RD_KAFKA_OFFSET_INVALID in case there was no stored
4080  * offset for that partition.
4081  *
4082  * Committed offsets will be returned according to the `isolation.level`
4083  * configuration property, if set to `read_committed` (default) then only
4084  * stable offsets for fully committed transactions will be returned, while
4085  * `read_uncommitted` may return offsets for not yet committed transactions.
4086  *
4087  * @returns RD_KAFKA_RESP_ERR_NO_ERROR on success in which case the
4088  *          \p offset or \p err field of each \p partitions' element is filled
4089  *          in with the stored offset, or a partition specific error.
4090  *          Else returns an error code.
4091  */
4092 RD_EXPORT rd_kafka_resp_err_t
4093 rd_kafka_committed (rd_kafka_t *rk,
4094 		    rd_kafka_topic_partition_list_t *partitions,
4095 		    int timeout_ms);
4096 
4097 
4098 
4099 /**
4100  * @brief Retrieve current positions (offsets) for topics+partitions.
4101  *
4102  * The \p offset field of each requested partition will be set to the offset
4103  * of the last consumed message + 1, or RD_KAFKA_OFFSET_INVALID in case there was
4104  * no previous message.
4105  *
4106  * @remark  In this context the last consumed message is the offset consumed
4107  *          by the current librdkafka instance and, in case of rebalancing, not
4108  *          necessarily the last message fetched from the partition.
4109  *
4110  * @returns RD_KAFKA_RESP_ERR_NO_ERROR on success in which case the
4111  *          \p offset or \p err field of each \p partitions' element is filled
4112  *          in with the stored offset, or a partition specific error.
4113  *          Else returns an error code.
4114  */
4115 RD_EXPORT rd_kafka_resp_err_t
4116 rd_kafka_position (rd_kafka_t *rk,
4117 		   rd_kafka_topic_partition_list_t *partitions);
4118 
4119 
4120 
4121 
4122 /**
4123  * @returns the current consumer group metadata associated with this consumer,
4124  *          or NULL if \p rk is not a consumer configured with a \c group.id.
4125  *          This metadata object should be passed to the transactional
4126  *          producer's rd_kafka_send_offsets_to_transaction() API.
4127  *
4128  * @remark The returned pointer must be freed by the application using
4129  *         rd_kafka_consumer_group_metadata_destroy().
4130  *
4131  * @sa rd_kafka_send_offsets_to_transaction()
4132  */
4133 RD_EXPORT rd_kafka_consumer_group_metadata_t *
4134 rd_kafka_consumer_group_metadata (rd_kafka_t *rk);
4135 
4136 
4137 /**
4138  * @brief Create a new consumer group metadata object.
4139  *        This is typically only used for writing tests.
4140  *
4141  * @param group_id The group id.
4142  *
4143  * @remark The returned pointer must be freed by the application using
4144  *         rd_kafka_consumer_group_metadata_destroy().
4145  */
4146 RD_EXPORT rd_kafka_consumer_group_metadata_t *
4147 rd_kafka_consumer_group_metadata_new (const char *group_id);
4148 
4149 
4150 /**
4151  * @brief Create a new consumer group metadata object.
4152  *        This is typically only used for writing tests.
4153  *
4154  * @param group_id The group id.
4155  * @param generation_id The group generation id.
4156  * @param member_id The group member id.
4157  * @param group_instance_id The group instance id (may be NULL).
4158  *
4159  * @remark The returned pointer must be freed by the application using
4160  *         rd_kafka_consumer_group_metadata_destroy().
4161  */
4162 RD_EXPORT rd_kafka_consumer_group_metadata_t *
4163 rd_kafka_consumer_group_metadata_new_with_genid (const char *group_id,
4164                                                  int32_t generation_id,
4165                                                  const char *member_id,
4166                                                  const char
4167                                                  *group_instance_id);
4168 
4169 
4170 /**
4171  * @brief Frees the consumer group metadata object as returned by
4172  *        rd_kafka_consumer_group_metadata().
4173  */
4174 RD_EXPORT void
4175 rd_kafka_consumer_group_metadata_destroy (rd_kafka_consumer_group_metadata_t *);
4176 
4177 
4178 /**
4179  * @brief Serialize the consumer group metadata to a binary format.
4180  *        This is mainly for client binding use and not for application use.
4181  *
4182  * @remark The serialized metadata format is private and is not compatible
4183  *         across different versions or even builds of librdkafka.
4184  *         It should only be used in the same process runtime and must only
4185  *         be passed to rd_kafka_consumer_group_metadata_read().
4186  *
4187  * @param cgmd Metadata to be serialized.
4188  * @param bufferp On success this pointer will be updated to point to na
4189  *                allocated buffer containing the serialized metadata.
4190  *                The buffer must be freed with rd_kafka_mem_free().
4191  * @param sizep The pointed to size will be updated with the size of
4192  *              the serialized buffer.
4193  *
4194  * @returns NULL on success or an error object on failure.
4195  *
4196  * @sa rd_kafka_consumer_group_metadata_read()
4197  */
4198 RD_EXPORT rd_kafka_error_t *
4199 rd_kafka_consumer_group_metadata_write (
4200         const rd_kafka_consumer_group_metadata_t *cgmd,
4201         void **bufferp, size_t *sizep);
4202 
4203 /**
4204  * @brief Reads serialized consumer group metadata and returns a
4205  *        consumer group metadata object.
4206  *        This is mainly for client binding use and not for application use.
4207  *
4208  * @remark The serialized metadata format is private and is not compatible
4209  *         across different versions or even builds of librdkafka.
4210  *         It should only be used in the same process runtime and must only
4211  *         be passed to rd_kafka_consumer_group_metadata_read().
4212  *
4213  * @param cgmdp On success this pointer will be updated to point to a new
4214  *              consumer group metadata object which must be freed with
4215  *              rd_kafka_consumer_group_metadata_destroy().
4216  * @param buffer Pointer to the serialized data.
4217  * @param size Size of the serialized data.
4218  *
4219  * @returns NULL on success or an error object on failure.
4220  *
4221  * @sa rd_kafka_consumer_group_metadata_write()
4222  */
4223 RD_EXPORT rd_kafka_error_t *
4224 rd_kafka_consumer_group_metadata_read (
4225         rd_kafka_consumer_group_metadata_t **cgmdp,
4226         const void *buffer, size_t size);
4227 
4228 /**@}*/
4229 
4230 
4231 
4232 /**
4233  * @name Producer API
4234  * @{
4235  *
4236  *
4237  */
4238 
4239 
4240 /**
4241  * @brief Producer message flags
4242  */
4243 #define RD_KAFKA_MSG_F_FREE  0x1 /**< Delegate freeing of payload to rdkafka. */
4244 #define RD_KAFKA_MSG_F_COPY  0x2 /**< rdkafka will make a copy of the payload. */
4245 #define RD_KAFKA_MSG_F_BLOCK 0x4 /**< Block produce*() on message queue full.
4246 				  *   WARNING: If a delivery report callback
4247 				  *            is used the application MUST
4248 				  *            call rd_kafka_poll() (or equiv.)
4249 				  *            to make sure delivered messages
4250 				  *            are drained from the internal
4251 				  *            delivery report queue.
4252 				  *            Failure to do so will result
4253 				  *            in indefinately blocking on
4254 				  *            the produce() call when the
4255 				  *            message queue is full. */
4256 #define RD_KAFKA_MSG_F_PARTITION 0x8 /**< produce_batch() will honor
4257                                       * per-message partition. */
4258 
4259 
4260 
4261 /**
4262  * @brief Produce and send a single message to broker.
4263  *
4264  * \p rkt is the target topic which must have been previously created with
4265  * `rd_kafka_topic_new()`.
4266  *
4267  * `rd_kafka_produce()` is an asynch non-blocking API.
4268  * See `rd_kafka_conf_set_dr_msg_cb` on how to setup a callback to be called
4269  * once the delivery status (success or failure) is known. The delivery report
4270  * is trigged by the application calling `rd_kafka_poll()` (at regular
4271  * intervals) or `rd_kafka_flush()` (at termination).
4272  *
4273  * Since producing is asynchronous, you should call `rd_kafka_flush()` before
4274  * you destroy the producer. Otherwise, any outstanding messages will be
4275  * silently discarded.
4276  *
4277  * When temporary errors occur, librdkafka automatically retries to produce the
4278  * messages. Retries are triggered after retry.backoff.ms and when the
4279  * leader broker for the given partition is available. Otherwise, librdkafka
4280  * falls back to polling the topic metadata to monitor when a new leader is
4281  * elected (see the topic.metadata.refresh.fast.interval.ms and
4282  * topic.metadata.refresh.interval.ms configurations) and then performs a
4283  * retry. A delivery error will occur if the message could not be produced
4284  * within message.timeout.ms.
4285  *
4286  * See the "Message reliability" chapter in INTRODUCTION.md for more
4287  * information.
4288  *
4289  * \p partition is the target partition, either:
4290  *   - RD_KAFKA_PARTITION_UA (unassigned) for
4291  *     automatic partitioning using the topic's partitioner function, or
4292  *   - a fixed partition (0..N)
4293  *
4294  * \p msgflags is zero or more of the following flags OR:ed together:
4295  *    RD_KAFKA_MSG_F_BLOCK - block \p produce*() call if
4296  *                           \p queue.buffering.max.messages or
4297  *                           \p queue.buffering.max.kbytes are exceeded.
4298  *                           Messages are considered in-queue from the point they
4299  *                           are accepted by produce() until their corresponding
4300  *                           delivery report callback/event returns.
4301  *                           It is thus a requirement to call
4302  *                           rd_kafka_poll() (or equiv.) from a separate
4303  *                           thread when F_BLOCK is used.
4304  *                           See WARNING on \c RD_KAFKA_MSG_F_BLOCK above.
4305  *
4306  *    RD_KAFKA_MSG_F_FREE - rdkafka will free(3) \p payload when it is done
4307  *                          with it.
4308  *    RD_KAFKA_MSG_F_COPY - the \p payload data will be copied and the
4309  *                          \p payload pointer will not be used by rdkafka
4310  *                          after the call returns.
4311  *    RD_KAFKA_MSG_F_PARTITION - produce_batch() will honour per-message
4312  *                               partition, either set manually or by the
4313  *                               configured partitioner.
4314  *
4315  *    .._F_FREE and .._F_COPY are mutually exclusive. If neither of these are
4316  *    set, the caller must ensure that the memory backing \p payload remains
4317  *    valid and is not modified or reused until the delivery callback is
4318  *    invoked. Other buffers passed to `rd_kafka_produce()` don't have this
4319  *    restriction on reuse, i.e. the memory backing the key or the topic name
4320  *    may be reused as soon as `rd_kafka_produce()` returns.
4321  *
4322  *    If the function returns -1 and RD_KAFKA_MSG_F_FREE was specified, then
4323  *    the memory associated with the payload is still the caller's
4324  *    responsibility.
4325  *
4326  * \p payload is the message payload of size \p len bytes.
4327  *
4328  * \p key is an optional message key of size \p keylen bytes, if non-NULL it
4329  * will be passed to the topic partitioner as well as be sent with the
4330  * message to the broker and passed on to the consumer.
4331  *
4332  * \p msg_opaque is an optional application-provided per-message opaque
4333  * pointer that will provided in the message's delivery report callback
4334  * (\c dr_msg_cb or \c dr_cb) and the \c rd_kafka_message_t \c _private field.
4335  *
4336  * @remark on_send() and on_acknowledgement() interceptors may be called
4337  *         from this function. on_acknowledgement() will only be called if the
4338  *         message fails partitioning.
4339  *
4340  * @remark If the producer is transactional (\c transactional.id is configured)
4341  *         producing is only allowed during an on-going transaction, namely
4342  *         after rd_kafka_begin_transaction() has been called.
4343  *
4344  * @returns 0 on success or -1 on error in which case errno is set accordingly:
4345  *  - ENOBUFS  - maximum number of outstanding messages has been reached:
4346  *               "queue.buffering.max.messages"
4347  *               (RD_KAFKA_RESP_ERR__QUEUE_FULL)
4348  *  - EMSGSIZE - message is larger than configured max size:
4349  *               "messages.max.bytes".
4350  *               (RD_KAFKA_RESP_ERR_MSG_SIZE_TOO_LARGE)
4351  *  - ESRCH    - requested \p partition is unknown in the Kafka cluster.
4352  *               (RD_KAFKA_RESP_ERR__UNKNOWN_PARTITION)
4353  *  - ENOENT   - topic is unknown in the Kafka cluster.
4354  *               (RD_KAFKA_RESP_ERR__UNKNOWN_TOPIC)
4355  *  - ECANCELED - fatal error has been raised on producer, see
4356  *                rd_kafka_fatal_error(),
4357  *               (RD_KAFKA_RESP_ERR__FATAL).
4358  *  - ENOEXEC  - transactional state forbids producing
4359  *               (RD_KAFKA_RESP_ERR__STATE)
4360  *
4361  * @sa Use rd_kafka_errno2err() to convert `errno` to rdkafka error code.
4362  */
4363 RD_EXPORT
4364 int rd_kafka_produce(rd_kafka_topic_t *rkt, int32_t partition,
4365 		      int msgflags,
4366 		      void *payload, size_t len,
4367 		      const void *key, size_t keylen,
4368 		      void *msg_opaque);
4369 
4370 
4371 /**
4372  * @brief Produce and send a single message to broker.
4373  *
4374  * The message is defined by a va-arg list using \c rd_kafka_vtype_t
4375  * tag tuples which must be terminated with a single \c RD_KAFKA_V_END.
4376  *
4377  * @returns \c RD_KAFKA_RESP_ERR_NO_ERROR on success, else an error code as
4378  *          described in rd_kafka_produce().
4379  *          \c RD_KAFKA_RESP_ERR__CONFLICT is returned if _V_HEADER and
4380  *          _V_HEADERS are mixed.
4381  *
4382  * @sa rd_kafka_produce, rd_kafka_produceva, RD_KAFKA_V_END
4383  */
4384 RD_EXPORT
4385 rd_kafka_resp_err_t rd_kafka_producev (rd_kafka_t *rk, ...);
4386 
4387 
4388 /**
4389  * @brief Produce and send a single message to broker.
4390  *
4391  * The message is defined by an array of \c rd_kafka_vu_t of
4392  * count \p cnt.
4393  *
4394  * @returns an error object on failure or NULL on success.
4395  *          See rd_kafka_producev() for specific error codes.
4396  *
4397  * @sa rd_kafka_produce, rd_kafka_producev, RD_KAFKA_V_END
4398  */
4399 RD_EXPORT
4400 rd_kafka_error_t *rd_kafka_produceva (rd_kafka_t *rk,
4401                                       const rd_kafka_vu_t *vus,
4402                                       size_t cnt);
4403 
4404 
4405 /**
4406  * @brief Produce multiple messages.
4407  *
4408  * If partition is RD_KAFKA_PARTITION_UA the configured partitioner will
4409  * be run for each message (slower), otherwise the messages will be enqueued
4410  * to the specified partition directly (faster).
4411  *
4412  * The messages are provided in the array \p rkmessages of count \p message_cnt
4413  * elements.
4414  * The \p partition and \p msgflags are used for all provided messages.
4415  *
4416  * Honoured \p rkmessages[] fields are:
4417  *  - payload,len    Message payload and length
4418  *  - key,key_len    Optional message key
4419  *  - _private       Message opaque pointer (msg_opaque)
4420  *  - err            Will be set according to success or failure, see
4421  *                   rd_kafka_produce() for possible error codes.
4422  *                   Application only needs to check for errors if
4423  *                   return value != \p message_cnt.
4424  *
4425  * @remark If \c RD_KAFKA_MSG_F_PARTITION is set in \p msgflags, the
4426  *         \c .partition field of the \p rkmessages is used instead of
4427  *         \p partition.
4428  *
4429  * @returns the number of messages succesfully enqueued for producing.
4430  *
4431  * @remark This interface does NOT support setting message headers on
4432  *         the provided \p rkmessages.
4433  */
4434 RD_EXPORT
4435 int rd_kafka_produce_batch(rd_kafka_topic_t *rkt, int32_t partition,
4436                             int msgflags,
4437                             rd_kafka_message_t *rkmessages, int message_cnt);
4438 
4439 
4440 
4441 
4442 /**
4443  * @brief Wait until all outstanding produce requests, et.al, are completed.
4444  *        This should typically be done prior to destroying a producer instance
4445  *        to make sure all queued and in-flight produce requests are completed
4446  *        before terminating.
4447  *
4448  * @remark This function will call rd_kafka_poll() and thus trigger callbacks.
4449  *
4450  * @remark The \c linger.ms time will be ignored for the duration of the call,
4451  *         queued messages will be sent to the broker as soon as possible.
4452  *
4453  * @remark If RD_KAFKA_EVENT_DR has been enabled
4454  *         (through rd_kafka_conf_set_events()) this function will not call
4455  *         rd_kafka_poll() but instead wait for the librdkafka-handled
4456  *         message count to reach zero. This requires the application to
4457  *         serve the event queue in a separate thread.
4458  *         In this mode only messages are counted, not other types of
4459  *         queued events.
4460  *
4461  * @returns RD_KAFKA_RESP_ERR__TIMED_OUT if \p timeout_ms was reached before all
4462  *          outstanding requests were completed, else RD_KAFKA_RESP_ERR_NO_ERROR
4463  *
4464  * @sa rd_kafka_outq_len()
4465  */
4466 RD_EXPORT
4467 rd_kafka_resp_err_t rd_kafka_flush (rd_kafka_t *rk, int timeout_ms);
4468 
4469 
4470 
4471 /**
4472  * @brief Purge messages currently handled by the producer instance.
4473  *
4474  * @param rk          Client instance.
4475  * @param purge_flags Tells which messages to purge and how.
4476  *
4477  * The application will need to call rd_kafka_poll() or rd_kafka_flush()
4478  * afterwards to serve the delivery report callbacks of the purged messages.
4479  *
4480  * Messages purged from internal queues fail with the delivery report
4481  * error code set to RD_KAFKA_RESP_ERR__PURGE_QUEUE, while purged messages that
4482  * are in-flight to or from the broker will fail with the error code set to
4483  * RD_KAFKA_RESP_ERR__PURGE_INFLIGHT.
4484  *
4485  * @warning Purging messages that are in-flight to or from the broker
4486  *          will ignore any sub-sequent acknowledgement for these messages
4487  *          received from the broker, effectively making it impossible
4488  *          for the application to know if the messages were successfully
4489  *          produced or not. This may result in duplicate messages if the
4490  *          application retries these messages at a later time.
4491  *
4492  * @remark This call may block for a short time while background thread
4493  *         queues are purged.
4494  *
4495  * @returns RD_KAFKA_RESP_ERR_NO_ERROR on success,
4496  *          RD_KAFKA_RESP_ERR__INVALID_ARG if the \p purge flags are invalid
4497  *          or unknown,
4498  *          RD_KAFKA_RESP_ERR__NOT_IMPLEMENTED if called on a non-producer
4499  *          client instance.
4500  */
4501 RD_EXPORT
4502 rd_kafka_resp_err_t rd_kafka_purge (rd_kafka_t *rk, int purge_flags);
4503 
4504 
4505 /**
4506  * @brief Flags for rd_kafka_purge()
4507  */
4508 
4509 /*!
4510  * Purge messages in internal queues.
4511  */
4512 #define RD_KAFKA_PURGE_F_QUEUE 0x1
4513 
4514 /*!
4515  * Purge messages in-flight to or from the broker.
4516  * Purging these messages will void any future acknowledgements from the
4517  * broker, making it impossible for the application to know if these
4518  * messages were successfully delivered or not.
4519  * Retrying these messages may lead to duplicates.
4520  */
4521 #define RD_KAFKA_PURGE_F_INFLIGHT 0x2
4522 
4523 
4524 /*!
4525  * Don't wait for background thread queue purging to finish.
4526  */
4527 #define RD_KAFKA_PURGE_F_NON_BLOCKING 0x4
4528 
4529 
4530 /**@}*/
4531 
4532 
4533 /**
4534 * @name Metadata API
4535 * @{
4536 *
4537 *
4538 */
4539 
4540 
4541 /**
4542  * @brief Broker information
4543  */
4544 typedef struct rd_kafka_metadata_broker {
4545         int32_t     id;             /**< Broker Id */
4546         char       *host;           /**< Broker hostname */
4547         int         port;           /**< Broker listening port */
4548 } rd_kafka_metadata_broker_t;
4549 
4550 /**
4551  * @brief Partition information
4552  */
4553 typedef struct rd_kafka_metadata_partition {
4554         int32_t     id;             /**< Partition Id */
4555         rd_kafka_resp_err_t err;    /**< Partition error reported by broker */
4556         int32_t     leader;         /**< Leader broker */
4557         int         replica_cnt;    /**< Number of brokers in \p replicas */
4558         int32_t    *replicas;       /**< Replica brokers */
4559         int         isr_cnt;        /**< Number of ISR brokers in \p isrs */
4560         int32_t    *isrs;           /**< In-Sync-Replica brokers */
4561 } rd_kafka_metadata_partition_t;
4562 
4563 /**
4564  * @brief Topic information
4565  */
4566 typedef struct rd_kafka_metadata_topic {
4567         char       *topic;          /**< Topic name */
4568         int         partition_cnt;  /**< Number of partitions in \p partitions*/
4569         struct rd_kafka_metadata_partition *partitions; /**< Partitions */
4570         rd_kafka_resp_err_t err;    /**< Topic error reported by broker */
4571 } rd_kafka_metadata_topic_t;
4572 
4573 
4574 /**
4575  * @brief Metadata container
4576  */
4577 typedef struct rd_kafka_metadata {
4578         int         broker_cnt;     /**< Number of brokers in \p brokers */
4579         struct rd_kafka_metadata_broker *brokers;  /**< Brokers */
4580 
4581         int         topic_cnt;      /**< Number of topics in \p topics */
4582         struct rd_kafka_metadata_topic *topics;    /**< Topics */
4583 
4584         int32_t     orig_broker_id;   /**< Broker originating this metadata */
4585         char       *orig_broker_name; /**< Name of originating broker */
4586 } rd_kafka_metadata_t;
4587 
4588 
4589 /**
4590  * @brief Request Metadata from broker.
4591  *
4592  * Parameters:
4593  *  - \p all_topics  if non-zero: request info about all topics in cluster,
4594  *                   if zero: only request info about locally known topics.
4595  *  - \p only_rkt    only request info about this topic
4596  *  - \p metadatap   pointer to hold metadata result.
4597  *                   The \p *metadatap pointer must be released
4598  *                   with rd_kafka_metadata_destroy().
4599  *  - \p timeout_ms  maximum response time before failing.
4600  *
4601  * @remark Consumer: If \p all_topics is non-zero the Metadata response
4602  *         information may trigger a re-join if any subscribed topics
4603  *         have changed partition count or existence state.
4604  *
4605  * @returns RD_KAFKA_RESP_ERR_NO_ERROR on success (in which case *metadatap)
4606  *          will be set, else RD_KAFKA_RESP_ERR__TIMED_OUT on timeout or
4607  *          other error code on error.
4608  */
4609 RD_EXPORT
4610 rd_kafka_resp_err_t
4611 rd_kafka_metadata (rd_kafka_t *rk, int all_topics,
4612                    rd_kafka_topic_t *only_rkt,
4613                    const struct rd_kafka_metadata **metadatap,
4614                    int timeout_ms);
4615 
4616 /**
4617  * @brief Release metadata memory.
4618  */
4619 RD_EXPORT
4620 void rd_kafka_metadata_destroy(const struct rd_kafka_metadata *metadata);
4621 
4622 
4623 /**@}*/
4624 
4625 
4626 
4627 /**
4628 * @name Client group information
4629 * @{
4630 *
4631 *
4632 */
4633 
4634 
4635 /**
4636  * @brief Group member information
4637  *
4638  * For more information on \p member_metadata format, see
4639  * https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-GroupMembershipAPI
4640  *
4641  */
4642 struct rd_kafka_group_member_info {
4643         char *member_id;            /**< Member id (generated by broker) */
4644         char *client_id;            /**< Client's \p client.id */
4645         char *client_host;          /**< Client's hostname */
4646         void *member_metadata;      /**< Member metadata (binary),
4647                                      *   format depends on \p protocol_type. */
4648         int   member_metadata_size; /**< Member metadata size in bytes */
4649         void *member_assignment;    /**< Member assignment (binary),
4650                                      *    format depends on \p protocol_type. */
4651         int   member_assignment_size; /**< Member assignment size in bytes */
4652 };
4653 
4654 /**
4655  * @brief Group information
4656  */
4657 struct rd_kafka_group_info {
4658         struct rd_kafka_metadata_broker broker; /**< Originating broker info */
4659         char *group;                            /**< Group name */
4660         rd_kafka_resp_err_t err;                /**< Broker-originated error */
4661         char *state;                            /**< Group state */
4662         char *protocol_type;                    /**< Group protocol type */
4663         char *protocol;                         /**< Group protocol */
4664         struct rd_kafka_group_member_info *members; /**< Group members */
4665         int   member_cnt;                       /**< Group member count */
4666 };
4667 
4668 /**
4669  * @brief List of groups
4670  *
4671  * @sa rd_kafka_group_list_destroy() to release list memory.
4672  */
4673 struct rd_kafka_group_list {
4674         struct rd_kafka_group_info *groups;   /**< Groups */
4675         int    group_cnt;                     /**< Group count */
4676 };
4677 
4678 
4679 /**
4680  * @brief List and describe client groups in cluster.
4681  *
4682  * \p group is an optional group name to describe, otherwise (\p NULL) all
4683  * groups are returned.
4684  *
4685  * \p timeout_ms is the (approximate) maximum time to wait for response
4686  * from brokers and must be a positive value.
4687  *
4688  * @returns \c RD_KAFKA_RESP_ERR__NO_ERROR on success and \p grplistp is
4689  *           updated to point to a newly allocated list of groups.
4690  *           \c RD_KAFKA_RESP_ERR__PARTIAL if not all brokers responded
4691  *           in time but at least one group is returned in  \p grplistlp.
4692  *           \c RD_KAFKA_RESP_ERR__TIMED_OUT if no groups were returned in the
4693  *           given timeframe but not all brokers have yet responded, or
4694  *           if the list of brokers in the cluster could not be obtained within
4695  *           the given timeframe.
4696  *           \c RD_KAFKA_RESP_ERR__TRANSPORT if no brokers were found.
4697  *           Other error codes may also be returned from the request layer.
4698  *
4699  *           The \p grplistp remains untouched if any error code is returned,
4700  *           with the exception of RD_KAFKA_RESP_ERR__PARTIAL which behaves
4701  *           as RD_KAFKA_RESP_ERR__NO_ERROR (success) but with an incomplete
4702  *           group list.
4703  *
4704  * @sa Use rd_kafka_group_list_destroy() to release list memory.
4705  */
4706 RD_EXPORT
4707 rd_kafka_resp_err_t
4708 rd_kafka_list_groups (rd_kafka_t *rk, const char *group,
4709                       const struct rd_kafka_group_list **grplistp,
4710                       int timeout_ms);
4711 
4712 /**
4713  * @brief Release list memory
4714  */
4715 RD_EXPORT
4716 void rd_kafka_group_list_destroy (const struct rd_kafka_group_list *grplist);
4717 
4718 
4719 /**@}*/
4720 
4721 
4722 
4723 /**
4724  * @name Miscellaneous APIs
4725  * @{
4726  *
4727  */
4728 
4729 
4730 /**
4731  * @brief Adds one or more brokers to the kafka handle's list of initial
4732  *        bootstrap brokers.
4733  *
4734  * Additional brokers will be discovered automatically as soon as rdkafka
4735  * connects to a broker by querying the broker metadata.
4736  *
4737  * If a broker name resolves to multiple addresses (and possibly
4738  * address families) all will be used for connection attempts in
4739  * round-robin fashion.
4740  *
4741  * \p brokerlist is a ,-separated list of brokers in the format:
4742  *   \c \<broker1\>,\<broker2\>,..
4743  * Where each broker is in either the host or URL based format:
4744  *   \c \<host\>[:\<port\>]
4745  *   \c \<proto\>://\<host\>[:port]
4746  * \c \<proto\> is either \c PLAINTEXT, \c SSL, \c SASL, \c SASL_PLAINTEXT
4747  * The two formats can be mixed but ultimately the value of the
4748  * `security.protocol` config property decides what brokers are allowed.
4749  *
4750  * Example:
4751  *    brokerlist = "broker1:10000,broker2"
4752  *    brokerlist = "SSL://broker3:9000,ssl://broker2"
4753  *
4754  * @returns the number of brokers successfully added.
4755  *
4756  * @remark Brokers may also be defined with the \c metadata.broker.list or
4757  *         \c bootstrap.servers configuration property (preferred method).
4758  *
4759  * @deprecated Set bootstrap servers with the \c bootstrap.servers
4760  *             configuration property.
4761  */
4762 RD_EXPORT
4763 int rd_kafka_brokers_add(rd_kafka_t *rk, const char *brokerlist);
4764 
4765 
4766 
4767 
4768 /**
4769  * @brief Set logger function.
4770  *
4771  * The default is to print to stderr, but a syslog logger is also available,
4772  * see rd_kafka_log_(print|syslog) for the builtin alternatives.
4773  * Alternatively the application may provide its own logger callback.
4774  * Or pass 'func' as NULL to disable logging.
4775  *
4776  * @deprecated Use rd_kafka_conf_set_log_cb()
4777  *
4778  * @remark \p rk may be passed as NULL in the callback.
4779  */
4780 RD_EXPORT RD_DEPRECATED
4781 void rd_kafka_set_logger(rd_kafka_t *rk,
4782 			  void (*func) (const rd_kafka_t *rk, int level,
4783 					const char *fac, const char *buf));
4784 
4785 
4786 /**
4787  * @brief Specifies the maximum logging level emitted by
4788  *        internal kafka logging and debugging.
4789  *
4790  * @deprecated Set the \c "log_level" configuration property instead.
4791  *
4792  * @remark If the \p \"debug\" configuration property is set the log level is
4793  *         automatically adjusted to \c LOG_DEBUG (7).
4794  */
4795 RD_EXPORT
4796 void rd_kafka_set_log_level(rd_kafka_t *rk, int level);
4797 
4798 
4799 /**
4800  * @brief Builtin (default) log sink: print to stderr
4801  */
4802 RD_EXPORT
4803 void rd_kafka_log_print(const rd_kafka_t *rk, int level,
4804 			 const char *fac, const char *buf);
4805 
4806 
4807 /**
4808  * @brief Builtin log sink: print to syslog.
4809  * @remark This logger is only available if librdkafka was built
4810  *         with syslog support.
4811  */
4812 RD_EXPORT
4813 void rd_kafka_log_syslog(const rd_kafka_t *rk, int level,
4814 			  const char *fac, const char *buf);
4815 
4816 
4817 /**
4818  * @brief Returns the current out queue length.
4819  *
4820  * The out queue length is the sum of:
4821  *  - number of messages waiting to be sent to, or acknowledged by,
4822  *    the broker.
4823  *  - number of delivery reports (e.g., dr_msg_cb) waiting to be served
4824  *    by rd_kafka_poll() or rd_kafka_flush().
4825  *  - number of callbacks (e.g., error_cb, stats_cb, etc) waiting to be
4826  *    served by rd_kafka_poll(), rd_kafka_consumer_poll() or rd_kafka_flush().
4827  *  - number of events waiting to be served by background_event_cb() in
4828  *    the background queue (see rd_kafka_conf_set_background_event_cb).
4829  *
4830  * An application should wait for the return value of this function to reach
4831  * zero before terminating to make sure outstanding messages,
4832  * requests (such as offset commits), callbacks and events are fully processed.
4833  * See rd_kafka_flush().
4834  *
4835  * @returns number of messages and events waiting in queues.
4836  *
4837  * @sa rd_kafka_flush()
4838  */
4839 RD_EXPORT
4840 int         rd_kafka_outq_len(rd_kafka_t *rk);
4841 
4842 
4843 
4844 /**
4845  * @brief Dumps rdkafka's internal state for handle \p rk to stream \p fp
4846  *
4847  * This is only useful for debugging rdkafka, showing state and statistics
4848  * for brokers, topics, partitions, etc.
4849  */
4850 RD_EXPORT
4851 void rd_kafka_dump(FILE *fp, rd_kafka_t *rk);
4852 
4853 
4854 
4855 /**
4856  * @brief Retrieve the current number of threads in use by librdkafka.
4857  *
4858  * Used by regression tests.
4859  */
4860 RD_EXPORT
4861 int rd_kafka_thread_cnt(void);
4862 
4863 
4864 /**
4865  * @enum rd_kafka_thread_type_t
4866  *
4867  * @brief librdkafka internal thread type.
4868  *
4869  * @sa rd_kafka_interceptor_add_on_thread_start()
4870  */
4871 typedef enum rd_kafka_thread_type_t {
4872         RD_KAFKA_THREAD_MAIN,       /**< librdkafka's internal main thread */
4873         RD_KAFKA_THREAD_BACKGROUND, /**< Background thread (if enabled) */
4874         RD_KAFKA_THREAD_BROKER      /**< Per-broker thread */
4875 } rd_kafka_thread_type_t;
4876 
4877 
4878 /**
4879  * @brief Wait for all rd_kafka_t objects to be destroyed.
4880  *
4881  * Returns 0 if all kafka objects are now destroyed, or -1 if the
4882  * timeout was reached.
4883  *
4884  * @remark This function is deprecated.
4885  */
4886 RD_EXPORT
4887 int rd_kafka_wait_destroyed(int timeout_ms);
4888 
4889 
4890 /**
4891  * @brief Run librdkafka's built-in unit-tests.
4892  *
4893  * @returns the number of failures, or 0 if all tests passed.
4894  */
4895 RD_EXPORT
4896 int rd_kafka_unittest (void);
4897 
4898 
4899 /**@}*/
4900 
4901 
4902 
4903 
4904 /**
4905  * @name Experimental APIs
4906  * @{
4907  */
4908 
4909 /**
4910  * @brief Redirect the main (rd_kafka_poll()) queue to the KafkaConsumer's
4911  *        queue (rd_kafka_consumer_poll()).
4912  *
4913  * @warning It is not permitted to call rd_kafka_poll() after directing the
4914  *          main queue with rd_kafka_poll_set_consumer().
4915  */
4916 RD_EXPORT
4917 rd_kafka_resp_err_t rd_kafka_poll_set_consumer (rd_kafka_t *rk);
4918 
4919 
4920 /**@}*/
4921 
4922 /**
4923  * @name Event interface
4924  *
4925  * @brief The event API provides an alternative pollable non-callback interface
4926  *        to librdkafka's message and event queues.
4927  *
4928  * @{
4929  */
4930 
4931 
4932 /**
4933  * @brief Event types
4934  */
4935 typedef int rd_kafka_event_type_t;
4936 #define RD_KAFKA_EVENT_NONE          0x0  /**< Unset value */
4937 #define RD_KAFKA_EVENT_DR            0x1  /**< Producer Delivery report batch */
4938 #define RD_KAFKA_EVENT_FETCH         0x2  /**< Fetched message (consumer) */
4939 #define RD_KAFKA_EVENT_LOG           0x4  /**< Log message */
4940 #define RD_KAFKA_EVENT_ERROR         0x8  /**< Error */
4941 #define RD_KAFKA_EVENT_REBALANCE     0x10 /**< Group rebalance (consumer) */
4942 #define RD_KAFKA_EVENT_OFFSET_COMMIT 0x20 /**< Offset commit result */
4943 #define RD_KAFKA_EVENT_STATS         0x40 /**< Stats */
4944 #define RD_KAFKA_EVENT_CREATETOPICS_RESULT 100 /**< CreateTopics_result_t */
4945 #define RD_KAFKA_EVENT_DELETETOPICS_RESULT 101 /**< DeleteTopics_result_t */
4946 #define RD_KAFKA_EVENT_CREATEPARTITIONS_RESULT 102 /**< CreatePartitions_result_t */
4947 #define RD_KAFKA_EVENT_ALTERCONFIGS_RESULT 103 /**< AlterConfigs_result_t */
4948 #define RD_KAFKA_EVENT_DESCRIBECONFIGS_RESULT 104 /**< DescribeConfigs_result_t */
4949 #define RD_KAFKA_EVENT_DELETERECORDS_RESULT 105 /**< DeleteRecords_result_t */
4950 #define RD_KAFKA_EVENT_DELETEGROUPS_RESULT 106 /**< DeleteGroups_result_t */
4951 /** DeleteConsumerGroupOffsets_result_t */
4952 #define RD_KAFKA_EVENT_DELETECONSUMERGROUPOFFSETS_RESULT 107
4953 #define RD_KAFKA_EVENT_OAUTHBEARER_TOKEN_REFRESH 0x100 /**< SASL/OAUTHBEARER
4954                                                              token needs to be
4955                                                              refreshed */
4956 
4957 
4958 /**
4959  * @returns the event type for the given event.
4960  *
4961  * @remark As a convenience it is okay to pass \p rkev as NULL in which case
4962  *         RD_KAFKA_EVENT_NONE is returned.
4963  */
4964 RD_EXPORT
4965 rd_kafka_event_type_t rd_kafka_event_type (const rd_kafka_event_t *rkev);
4966 
4967 /**
4968  * @returns the event type's name for the given event.
4969  *
4970  * @remark As a convenience it is okay to pass \p rkev as NULL in which case
4971  *         the name for RD_KAFKA_EVENT_NONE is returned.
4972  */
4973 RD_EXPORT
4974 const char *rd_kafka_event_name (const rd_kafka_event_t *rkev);
4975 
4976 
4977 /**
4978  * @brief Destroy an event.
4979  *
4980  * @remark Any references to this event, such as extracted messages,
4981  *         will not be usable after this call.
4982  *
4983  * @remark As a convenience it is okay to pass \p rkev as NULL in which case
4984  *         no action is performed.
4985  */
4986 RD_EXPORT
4987 void rd_kafka_event_destroy (rd_kafka_event_t *rkev);
4988 
4989 
4990 /**
4991  * @returns the next message from an event.
4992  *
4993  * Call repeatedly until it returns NULL.
4994  *
4995  * Event types:
4996  *  - RD_KAFKA_EVENT_FETCH  (1 message)
4997  *  - RD_KAFKA_EVENT_DR     (>=1 message(s))
4998  *
4999  * @remark The returned message(s) MUST NOT be
5000  *         freed with rd_kafka_message_destroy().
5001  *
5002  * @remark on_consume() interceptor may be called
5003  *         from this function prior to passing message to application.
5004  */
5005 RD_EXPORT
5006 const rd_kafka_message_t *rd_kafka_event_message_next (rd_kafka_event_t *rkev);
5007 
5008 
5009 /**
5010  * @brief Extacts \p size message(s) from the event into the
5011  *        pre-allocated array \p rkmessages.
5012  *
5013  * Event types:
5014  *  - RD_KAFKA_EVENT_FETCH  (1 message)
5015  *  - RD_KAFKA_EVENT_DR     (>=1 message(s))
5016  *
5017  * @returns the number of messages extracted.
5018  *
5019  * @remark on_consume() interceptor may be called
5020  *         from this function prior to passing message to application.
5021  */
5022 RD_EXPORT
5023 size_t rd_kafka_event_message_array (rd_kafka_event_t *rkev,
5024 				     const rd_kafka_message_t **rkmessages,
5025 				     size_t size);
5026 
5027 
5028 /**
5029  * @returns the number of remaining messages in the event.
5030  *
5031  * Event types:
5032  *  - RD_KAFKA_EVENT_FETCH  (1 message)
5033  *  - RD_KAFKA_EVENT_DR     (>=1 message(s))
5034  */
5035 RD_EXPORT
5036 size_t rd_kafka_event_message_count (rd_kafka_event_t *rkev);
5037 
5038 
5039 /**
5040  * @returns the associated configuration string for the event, or NULL
5041  *          if the configuration property is not set or if
5042  *          not applicable for the given event type.
5043  *
5044  * The returned memory is read-only and its lifetime is the same as the
5045  * event object.
5046  *
5047  * Event types:
5048  *  - RD_KAFKA_EVENT_OAUTHBEARER_TOKEN_REFRESH: value of sasl.oauthbearer.config
5049  */
5050 RD_EXPORT
5051 const char *rd_kafka_event_config_string (rd_kafka_event_t *rkev);
5052 
5053 
5054 /**
5055  * @returns the error code for the event.
5056  *
5057  * Use rd_kafka_event_error_is_fatal() to detect if this is a fatal error.
5058  *
5059  * Event types:
5060  *  - all
5061  */
5062 RD_EXPORT
5063 rd_kafka_resp_err_t rd_kafka_event_error (rd_kafka_event_t *rkev);
5064 
5065 
5066 /**
5067  * @returns the error string (if any).
5068  *          An application should check that rd_kafka_event_error() returns
5069  *          non-zero before calling this function.
5070  *
5071  * Event types:
5072  *  - all
5073  */
5074 RD_EXPORT
5075 const char *rd_kafka_event_error_string (rd_kafka_event_t *rkev);
5076 
5077 
5078 /**
5079  * @returns 1 if the error is a fatal error, else 0.
5080  *
5081  * Event types:
5082  *  - RD_KAFKA_EVENT_ERROR
5083  *
5084  * @sa rd_kafka_fatal_error()
5085  */
5086 RD_EXPORT
5087 int rd_kafka_event_error_is_fatal (rd_kafka_event_t *rkev);
5088 
5089 
5090 /**
5091  * @returns the event opaque (if any) as passed to rd_kafka_commit() (et.al) or
5092  *          rd_kafka_AdminOptions_set_opaque(), depending on event type.
5093  *
5094  * Event types:
5095  *  - RD_KAFKA_EVENT_OFFSET_COMMIT
5096  *  - RD_KAFKA_EVENT_CREATETOPICS_RESULT
5097  *  - RD_KAFKA_EVENT_DELETETOPICS_RESULT
5098  *  - RD_KAFKA_EVENT_CREATEPARTITIONS_RESULT
5099  *  - RD_KAFKA_EVENT_ALTERCONFIGS_RESULT
5100  *  - RD_KAFKA_EVENT_DESCRIBECONFIGS_RESULT
5101  *  - RD_KAFKA_EVENT_DELETEGROUPS_RESULT
5102  *  - RD_KAFKA_EVENT_DELETECONSUMERGROUPOFFSETS_RESULT
5103  *  - RD_KAFKA_EVENT_DELETERECORDS_RESULT
5104  */
5105 RD_EXPORT
5106 void *rd_kafka_event_opaque (rd_kafka_event_t *rkev);
5107 
5108 
5109 /**
5110  * @brief Extract log message from the event.
5111  *
5112  * Event types:
5113  *  - RD_KAFKA_EVENT_LOG
5114  *
5115  * @returns 0 on success or -1 if unsupported event type.
5116  */
5117 RD_EXPORT
5118 int rd_kafka_event_log (rd_kafka_event_t *rkev,
5119 			const char **fac, const char **str, int *level);
5120 
5121 
5122 /**
5123  * @brief Extract log debug context from event.
5124  *
5125  * Event types:
5126  *  - RD_KAFKA_EVENT_LOG
5127  *
5128  *  @param rkev the event to extract data from.
5129  *  @param dst destination string for comma separated list.
5130  *  @param dstsize size of provided dst buffer.
5131  *  @returns 0 on success or -1 if unsupported event type.
5132  */
5133 RD_EXPORT
5134 int rd_kafka_event_debug_contexts (rd_kafka_event_t *rkev,
5135             char *dst, size_t dstsize);
5136 
5137 
5138 /**
5139  * @brief Extract stats from the event.
5140  *
5141  * Event types:
5142  *  - RD_KAFKA_EVENT_STATS
5143  *
5144  * @returns stats json string.
5145  *
5146  * @remark the returned string will be freed automatically along with the event object
5147  *
5148  */
5149 RD_EXPORT
5150 const char *rd_kafka_event_stats (rd_kafka_event_t *rkev);
5151 
5152 
5153 /**
5154  * @returns the topic partition list from the event.
5155  *
5156  * @remark The list MUST NOT be freed with rd_kafka_topic_partition_list_destroy()
5157  *
5158  * Event types:
5159  *  - RD_KAFKA_EVENT_REBALANCE
5160  *  - RD_KAFKA_EVENT_OFFSET_COMMIT
5161  */
5162 RD_EXPORT rd_kafka_topic_partition_list_t *
5163 rd_kafka_event_topic_partition_list (rd_kafka_event_t *rkev);
5164 
5165 
5166 /**
5167  * @returns a newly allocated topic_partition container, if applicable for the event type,
5168  *          else NULL.
5169  *
5170  * @remark The returned pointer MUST be freed with rd_kafka_topic_partition_destroy().
5171  *
5172  * Event types:
5173  *   RD_KAFKA_EVENT_ERROR  (for partition level errors)
5174  */
5175 RD_EXPORT rd_kafka_topic_partition_t *
5176 rd_kafka_event_topic_partition (rd_kafka_event_t *rkev);
5177 
5178 
5179 /*! CreateTopics result type */
5180 typedef rd_kafka_event_t rd_kafka_CreateTopics_result_t;
5181 /*! DeleteTopics result type */
5182 typedef rd_kafka_event_t rd_kafka_DeleteTopics_result_t;
5183 /*! CreatePartitions result type */
5184 typedef rd_kafka_event_t rd_kafka_CreatePartitions_result_t;
5185 /*! AlterConfigs result type */
5186 typedef rd_kafka_event_t rd_kafka_AlterConfigs_result_t;
5187 /*! CreateTopics result type */
5188 typedef rd_kafka_event_t rd_kafka_DescribeConfigs_result_t;
5189 /*! DeleteRecords result type */
5190 typedef rd_kafka_event_t rd_kafka_DeleteRecords_result_t;
5191 /*! DeleteGroups result type */
5192 typedef rd_kafka_event_t rd_kafka_DeleteGroups_result_t;
5193 /*! DeleteConsumerGroupOffsets result type */
5194 typedef rd_kafka_event_t rd_kafka_DeleteConsumerGroupOffsets_result_t;
5195 
5196 /**
5197  * @brief Get CreateTopics result.
5198  *
5199  * @returns the result of a CreateTopics request, or NULL if event is of
5200  *          different type.
5201  *
5202  * Event types:
5203  *   RD_KAFKA_EVENT_CREATETOPICS_RESULT
5204  */
5205 RD_EXPORT const rd_kafka_CreateTopics_result_t *
5206 rd_kafka_event_CreateTopics_result (rd_kafka_event_t *rkev);
5207 
5208 /**
5209  * @brief Get DeleteTopics result.
5210  *
5211  * @returns the result of a DeleteTopics request, or NULL if event is of
5212  *          different type.
5213  *
5214  * Event types:
5215  *   RD_KAFKA_EVENT_DELETETOPICS_RESULT
5216  */
5217 RD_EXPORT const rd_kafka_DeleteTopics_result_t *
5218 rd_kafka_event_DeleteTopics_result (rd_kafka_event_t *rkev);
5219 
5220 /**
5221  * @brief Get CreatePartitions result.
5222  *
5223  * @returns the result of a CreatePartitions request, or NULL if event is of
5224  *          different type.
5225  *
5226  * Event types:
5227  *   RD_KAFKA_EVENT_CREATEPARTITIONS_RESULT
5228  */
5229 RD_EXPORT const rd_kafka_CreatePartitions_result_t *
5230 rd_kafka_event_CreatePartitions_result (rd_kafka_event_t *rkev);
5231 
5232 /**
5233  * @brief Get AlterConfigs result.
5234  *
5235  * @returns the result of a AlterConfigs request, or NULL if event is of
5236  *          different type.
5237  *
5238  * Event types:
5239  *   RD_KAFKA_EVENT_ALTERCONFIGS_RESULT
5240  */
5241 RD_EXPORT const rd_kafka_AlterConfigs_result_t *
5242 rd_kafka_event_AlterConfigs_result (rd_kafka_event_t *rkev);
5243 
5244 /**
5245  * @brief Get DescribeConfigs result.
5246  *
5247  * @returns the result of a DescribeConfigs request, or NULL if event is of
5248  *          different type.
5249  *
5250  * Event types:
5251  *   RD_KAFKA_EVENT_DESCRIBECONFIGS_RESULT
5252  */
5253 RD_EXPORT const rd_kafka_DescribeConfigs_result_t *
5254 rd_kafka_event_DescribeConfigs_result (rd_kafka_event_t *rkev);
5255 
5256 /**
5257  * @returns the result of a DeleteRecords request, or NULL if event is of
5258  *          different type.
5259  *
5260  * Event types:
5261  *   RD_KAFKA_EVENT_DELETERECORDS_RESULT
5262  */
5263 RD_EXPORT const rd_kafka_DeleteRecords_result_t *
5264 rd_kafka_event_DeleteRecords_result (rd_kafka_event_t *rkev);
5265 
5266 /**
5267  * @brief Get DeleteGroups result.
5268  *
5269  * @returns the result of a DeleteGroups request, or NULL if event is of
5270  *          different type.
5271  *
5272  * Event types:
5273  *   RD_KAFKA_EVENT_DELETEGROUPS_RESULT
5274  */
5275 RD_EXPORT const rd_kafka_DeleteGroups_result_t *
5276 rd_kafka_event_DeleteGroups_result (rd_kafka_event_t *rkev);
5277 
5278 /**
5279  * @brief Get DeleteConsumerGroupOffsets result.
5280  *
5281  * @returns the result of a DeleteConsumerGroupOffsets request, or NULL if
5282  *          event is of different type.
5283  *
5284  * Event types:
5285  *   RD_KAFKA_EVENT_DELETECONSUMERGROUPOFFSETS_RESULT
5286  */
5287 RD_EXPORT const rd_kafka_DeleteConsumerGroupOffsets_result_t *
5288 rd_kafka_event_DeleteConsumerGroupOffsets_result (rd_kafka_event_t *rkev);
5289 
5290 /**
5291  * @brief Poll a queue for an event for max \p timeout_ms.
5292  *
5293  * @returns an event, or NULL.
5294  *
5295  * @remark Use rd_kafka_event_destroy() to free the event.
5296  *
5297  * @sa rd_kafka_conf_set_background_event_cb()
5298  */
5299 RD_EXPORT
5300 rd_kafka_event_t *rd_kafka_queue_poll (rd_kafka_queue_t *rkqu, int timeout_ms);
5301 
5302 /**
5303 * @brief Poll a queue for events served through callbacks for max \p timeout_ms.
5304 *
5305 * @returns the number of events served.
5306 *
5307 * @remark This API must only be used for queues with callbacks registered
5308 *         for all expected event types. E.g., not a message queue.
5309 *
5310 * @remark Also see rd_kafka_conf_set_background_event_cb() for triggering
5311 *         event callbacks from a librdkafka-managed background thread.
5312 *
5313 * @sa rd_kafka_conf_set_background_event_cb()
5314 */
5315 RD_EXPORT
5316 int rd_kafka_queue_poll_callback (rd_kafka_queue_t *rkqu, int timeout_ms);
5317 
5318 
5319 /**@}*/
5320 
5321 
5322 /**
5323  * @name Plugin interface
5324  *
5325  * @brief A plugin interface that allows external runtime-loaded libraries
5326  *        to integrate with a client instance without modifications to
5327  *        the application code.
5328  *
5329  *        Plugins are loaded when referenced through the `plugin.library.paths`
5330  *        configuration property and operates on the \c rd_kafka_conf_t
5331  *        object prior \c rd_kafka_t instance creation.
5332  *
5333  * @warning Plugins require the application to link librdkafka dynamically
5334  *          and not statically. Failure to do so will lead to missing symbols
5335  *          or finding symbols in another librdkafka library than the
5336  *          application was linked with.
5337  */
5338 
5339 
5340 /**
5341  * @brief Plugin's configuration initializer method called each time the
5342  *        library is referenced from configuration (even if previously loaded by
5343  *        another client instance).
5344  *
5345  * @remark This method MUST be implemented by plugins and have the symbol name
5346  *         \c conf_init
5347  *
5348  * @param conf Configuration set up to this point.
5349  * @param plug_opaquep Plugin can set this pointer to a per-configuration
5350  *                     opaque pointer.
5351  * @param errstr String buffer of size \p errstr_size where plugin must write
5352  *               a human readable error string in the case the initializer
5353  *               fails (returns non-zero).
5354  * @param errstr_size Maximum space (including \0) in \p errstr.
5355  *
5356  * @remark A plugin may add an on_conf_destroy() interceptor to clean up
5357  *         plugin-specific resources created in the plugin's conf_init() method.
5358  *
5359  * @returns RD_KAFKA_RESP_ERR_NO_ERROR on success or an error code on error.
5360  */
5361 typedef rd_kafka_resp_err_t
5362 (rd_kafka_plugin_f_conf_init_t) (rd_kafka_conf_t *conf,
5363                                  void **plug_opaquep,
5364                                  char *errstr, size_t errstr_size);
5365 
5366 /**@}*/
5367 
5368 
5369 
5370 /**
5371  * @name Interceptors
5372  *
5373  * @{
5374  *
5375  * @brief A callback interface that allows message interception for both
5376  *        producer and consumer data pipelines.
5377  *
5378  * Except for the on_new(), on_conf_set(), on_conf_dup() and on_conf_destroy()
5379  * interceptors, interceptors are added to the
5380  * newly created rd_kafka_t client instance. These interceptors MUST only
5381  * be added from on_new() and MUST NOT be added after rd_kafka_new() returns.
5382  *
5383  * The on_new(), on_conf_set(), on_conf_dup() and on_conf_destroy() interceptors
5384  * are added to the configuration object which is later passed to
5385  * rd_kafka_new() where on_new() is called to allow addition of
5386  * other interceptors.
5387  *
5388  * Each interceptor reference consists of a display name (ic_name),
5389  * a callback function, and an application-specified opaque value that is
5390  * passed as-is to the callback.
5391  * The ic_name must be unique for the interceptor implementation and is used
5392  * to reject duplicate interceptor methods.
5393  *
5394  * Any number of interceptors can be added and they are called in the order
5395  * they were added, unless otherwise noted.
5396  * The list of registered interceptor methods are referred to as
5397  * interceptor chains.
5398  *
5399  * @remark Contrary to the Java client the librdkafka interceptor interface
5400  *         does not support message key and value modification.
5401  *         Message mutability is discouraged in the Java client and the
5402  *         combination of serializers and headers cover most use-cases.
5403  *
5404  * @remark Interceptors are NOT copied to the new configuration on
5405  *         rd_kafka_conf_dup() since it would be hard for interceptors to
5406  *         track usage of the interceptor's opaque value.
5407  *         An interceptor should rely on the plugin, which will be copied
5408  *         in rd_kafka_conf_conf_dup(), to set up the initial interceptors.
5409  *         An interceptor should implement the on_conf_dup() method
5410  *         to manually set up its internal configuration on the newly created
5411  *         configuration object that is being copied-to based on the
5412  *         interceptor-specific configuration properties.
5413  *         conf_dup() should thus be treated the same as conf_init().
5414  *
5415  * @remark Interceptors are keyed by the interceptor type (on_..()), the
5416  *         interceptor name (ic_name) and the interceptor method function.
5417  *         Duplicates are not allowed and the .._add_on_..() method will
5418  *         return RD_KAFKA_RESP_ERR__CONFLICT if attempting to add a duplicate
5419  *         method.
5420  *         The only exception is on_conf_destroy() which may be added multiple
5421  *         times by the same interceptor to allow proper cleanup of
5422  *         interceptor configuration state.
5423  */
5424 
5425 
5426 /**
5427  * @brief on_conf_set() is called from rd_kafka_*_conf_set() in the order
5428  *        the interceptors were added.
5429  *
5430  * @param ic_opaque The interceptor's opaque pointer specified in ..add..().
5431  * @param name The configuration property to set.
5432  * @param val The configuration value to set, or NULL for reverting to default
5433  *            in which case the previous value should be freed.
5434  * @param errstr A human readable error string in case the interceptor fails.
5435  * @param errstr_size Maximum space (including \0) in \p errstr.
5436  *
5437  * @returns RD_KAFKA_CONF_OK if the property was known and successfully
5438  *          handled by the interceptor, RD_KAFKA_CONF_INVALID if the
5439  *          property was handled by the interceptor but the value was invalid,
5440  *          or RD_KAFKA_CONF_UNKNOWN if the interceptor did not handle
5441  *          this property, in which case the property is passed on on the
5442  *          interceptor in the chain, finally ending up at the built-in
5443  *          configuration handler.
5444  */
5445 typedef rd_kafka_conf_res_t
5446 (rd_kafka_interceptor_f_on_conf_set_t) (rd_kafka_conf_t *conf,
5447                                         const char *name, const char *val,
5448                                         char *errstr, size_t errstr_size,
5449                                         void *ic_opaque);
5450 
5451 
5452 /**
5453  * @brief on_conf_dup() is called from rd_kafka_conf_dup() in the
5454  *        order the interceptors were added and is used to let
5455  *        an interceptor re-register its conf interecptors with a new
5456  *        opaque value.
5457  *        The on_conf_dup() method is called prior to the configuration from
5458  *        \p old_conf being copied to \p new_conf.
5459  *
5460  * @param ic_opaque The interceptor's opaque pointer specified in ..add..().
5461  *
5462  * @returns RD_KAFKA_RESP_ERR_NO_ERROR on success or an error code
5463  *          on failure (which is logged but otherwise ignored).
5464  *
5465  * @remark No on_conf_* interceptors are copied to the new configuration
5466  *         object on rd_kafka_conf_dup().
5467  */
5468 typedef rd_kafka_resp_err_t
5469 (rd_kafka_interceptor_f_on_conf_dup_t) (rd_kafka_conf_t *new_conf,
5470                                         const rd_kafka_conf_t *old_conf,
5471                                         size_t filter_cnt,
5472                                         const char **filter,
5473                                         void *ic_opaque);
5474 
5475 
5476 /**
5477  * @brief on_conf_destroy() is called from rd_kafka_*_conf_destroy() in the
5478  *        order the interceptors were added.
5479  *
5480  * @param ic_opaque The interceptor's opaque pointer specified in ..add..().
5481  */
5482 typedef rd_kafka_resp_err_t
5483 (rd_kafka_interceptor_f_on_conf_destroy_t) (void *ic_opaque);
5484 
5485 
5486 /**
5487  * @brief on_new() is called from rd_kafka_new() prior toreturning
5488  *        the newly created client instance to the application.
5489  *
5490  * @param rk The client instance.
5491  * @param conf The client instance's final configuration.
5492  * @param ic_opaque The interceptor's opaque pointer specified in ..add..().
5493  * @param errstr A human readable error string in case the interceptor fails.
5494  * @param errstr_size Maximum space (including \0) in \p errstr.
5495  *
5496  * @returns an error code on failure, the error is logged but otherwise ignored.
5497  *
5498  * @warning The \p rk client instance will not be fully set up when this
5499  *          interceptor is called and the interceptor MUST NOT call any
5500  *          other rk-specific APIs than rd_kafka_interceptor_add..().
5501  *
5502  */
5503 typedef rd_kafka_resp_err_t
5504 (rd_kafka_interceptor_f_on_new_t) (rd_kafka_t *rk, const rd_kafka_conf_t *conf,
5505                                    void *ic_opaque,
5506                                    char *errstr, size_t errstr_size);
5507 
5508 
5509 /**
5510  * @brief on_destroy() is called from rd_kafka_destroy() or (rd_kafka_new()
5511  *        if rd_kafka_new() fails during initialization).
5512  *
5513  * @param rk The client instance.
5514  * @param ic_opaque The interceptor's opaque pointer specified in ..add..().
5515  */
5516 typedef rd_kafka_resp_err_t
5517 (rd_kafka_interceptor_f_on_destroy_t) (rd_kafka_t *rk, void *ic_opaque);
5518 
5519 
5520 
5521 
5522 /**
5523  * @brief on_send() is called from rd_kafka_produce*() (et.al) prior to
5524  *        the partitioner being called.
5525  *
5526  * @param rk The client instance.
5527  * @param rkmessage The message being produced. Immutable.
5528  * @param ic_opaque The interceptor's opaque pointer specified in ..add..().
5529  *
5530  * @remark This interceptor is only used by producer instances.
5531  *
5532  * @remark The \p rkmessage object is NOT mutable and MUST NOT be modified
5533  *         by the interceptor.
5534  *
5535  * @remark If the partitioner fails or an unknown partition was specified,
5536  *         the on_acknowledgement() interceptor chain will be called from
5537  *         within the rd_kafka_produce*() call to maintain send-acknowledgement
5538  *         symmetry.
5539  *
5540  * @returns an error code on failure, the error is logged but otherwise ignored.
5541  */
5542 typedef rd_kafka_resp_err_t
5543 (rd_kafka_interceptor_f_on_send_t) (rd_kafka_t *rk,
5544                                     rd_kafka_message_t *rkmessage,
5545                                     void *ic_opaque);
5546 
5547 /**
5548  * @brief on_acknowledgement() is called to inform interceptors that a message
5549  *        was succesfully delivered or permanently failed delivery.
5550  *        The interceptor chain is called from internal librdkafka background
5551  *        threads, or rd_kafka_produce*() if the partitioner failed.
5552  *
5553  * @param rk The client instance.
5554  * @param rkmessage The message being produced. Immutable.
5555  * @param ic_opaque The interceptor's opaque pointer specified in ..add..().
5556  *
5557  * @remark This interceptor is only used by producer instances.
5558  *
5559  * @remark The \p rkmessage object is NOT mutable and MUST NOT be modified
5560  *         by the interceptor.
5561  *
5562  * @warning The on_acknowledgement() method may be called from internal
5563  *         librdkafka threads. An on_acknowledgement() interceptor MUST NOT
5564  *         call any librdkafka API's associated with the \p rk, or perform
5565  *         any blocking or prolonged work.
5566  *
5567  * @returns an error code on failure, the error is logged but otherwise ignored.
5568  */
5569 typedef rd_kafka_resp_err_t
5570 (rd_kafka_interceptor_f_on_acknowledgement_t) (rd_kafka_t *rk,
5571                                                rd_kafka_message_t *rkmessage,
5572                                                void *ic_opaque);
5573 
5574 
5575 /**
5576  * @brief on_consume() is called just prior to passing the message to the
5577  *        application in rd_kafka_consumer_poll(), rd_kafka_consume*(),
5578  *        the event interface, etc.
5579  *
5580  * @param rk The client instance.
5581  * @param rkmessage The message being consumed. Immutable.
5582  * @param ic_opaque The interceptor's opaque pointer specified in ..add..().
5583  *
5584  * @remark This interceptor is only used by consumer instances.
5585  *
5586  * @remark The \p rkmessage object is NOT mutable and MUST NOT be modified
5587  *         by the interceptor.
5588  *
5589  * @returns an error code on failure, the error is logged but otherwise ignored.
5590  */
5591 typedef rd_kafka_resp_err_t
5592 (rd_kafka_interceptor_f_on_consume_t) (rd_kafka_t *rk,
5593                                        rd_kafka_message_t *rkmessage,
5594                                        void *ic_opaque);
5595 
5596 /**
5597  * @brief on_commit() is called on completed or failed offset commit.
5598  *        It is called from internal librdkafka threads.
5599  *
5600  * @param rk The client instance.
5601  * @param offsets List of topic+partition+offset+error that were committed.
5602  *                The error message of each partition should be checked for
5603  *                error.
5604  * @param ic_opaque The interceptor's opaque pointer specified in ..add..().
5605  *
5606  * @remark This interceptor is only used by consumer instances.
5607  *
5608  * @warning The on_commit() interceptor is called from internal
5609  *          librdkafka threads. An on_commit() interceptor MUST NOT
5610  *          call any librdkafka API's associated with the \p rk, or perform
5611  *          any blocking or prolonged work.
5612  *
5613  *
5614  * @returns an error code on failure, the error is logged but otherwise ignored.
5615  */
5616 typedef rd_kafka_resp_err_t
5617 (rd_kafka_interceptor_f_on_commit_t) (
5618         rd_kafka_t *rk,
5619         const rd_kafka_topic_partition_list_t *offsets,
5620         rd_kafka_resp_err_t err, void *ic_opaque);
5621 
5622 
5623 /**
5624  * @brief on_request_sent() is called when a request has been fully written
5625  *        to a broker TCP connections socket.
5626  *
5627  * @param rk The client instance.
5628  * @param sockfd Socket file descriptor.
5629  * @param brokername Broker request is being sent to.
5630  * @param brokerid Broker request is being sent to.
5631  * @param ApiKey Kafka protocol request type.
5632  * @param ApiVersion Kafka protocol request type version.
5633  * @param Corrid Kafka protocol request correlation id.
5634  * @param size Size of request.
5635  * @param ic_opaque The interceptor's opaque pointer specified in ..add..().
5636  *
5637  * @warning The on_request_sent() interceptor is called from internal
5638  *          librdkafka broker threads. An on_request_sent() interceptor MUST NOT
5639  *          call any librdkafka API's associated with the \p rk, or perform
5640  *          any blocking or prolonged work.
5641  *
5642  * @returns an error code on failure, the error is logged but otherwise ignored.
5643  */
5644 typedef rd_kafka_resp_err_t
5645 (rd_kafka_interceptor_f_on_request_sent_t) (
5646         rd_kafka_t *rk,
5647         int sockfd,
5648         const char *brokername,
5649         int32_t brokerid,
5650         int16_t ApiKey,
5651         int16_t ApiVersion,
5652         int32_t CorrId,
5653         size_t  size,
5654         void *ic_opaque);
5655 
5656 
5657 /**
5658  * @brief on_response_received() is called when a protocol response has been
5659  *        fully received from a broker TCP connection socket but before the
5660  *        response payload is parsed.
5661  *
5662  * @param rk The client instance.
5663  * @param sockfd Socket file descriptor (always -1).
5664  * @param brokername Broker response was received from, possibly empty string
5665  *                   on error.
5666  * @param brokerid Broker response was received from.
5667  * @param ApiKey Kafka protocol request type or -1 on error.
5668  * @param ApiVersion Kafka protocol request type version or -1 on error.
5669  * @param Corrid Kafka protocol request correlation id, possibly -1 on error.
5670  * @param size Size of response, possibly 0 on error.
5671  * @param rtt Request round-trip-time in microseconds, possibly -1 on error.
5672  * @param err Receive error.
5673  * @param ic_opaque The interceptor's opaque pointer specified in ..add..().
5674  *
5675  * @warning The on_response_received() interceptor is called from internal
5676  *          librdkafka broker threads. An on_response_received() interceptor
5677  *          MUST NOT call any librdkafka API's associated with the \p rk, or
5678  *          perform any blocking or prolonged work.
5679  *
5680  * @returns an error code on failure, the error is logged but otherwise ignored.
5681  */
5682 typedef rd_kafka_resp_err_t
5683 (rd_kafka_interceptor_f_on_response_received_t) (
5684         rd_kafka_t *rk,
5685         int sockfd,
5686         const char *brokername,
5687         int32_t brokerid,
5688         int16_t ApiKey,
5689         int16_t ApiVersion,
5690         int32_t CorrId,
5691         size_t  size,
5692         int64_t rtt,
5693         rd_kafka_resp_err_t err,
5694         void *ic_opaque);
5695 
5696 
5697 /**
5698  * @brief on_thread_start() is called from a newly created librdkafka-managed
5699  *        thread.
5700 
5701  * @param rk The client instance.
5702  * @param thread_type Thread type.
5703  * @param thread_name Human-readable thread name, may not be unique.
5704  * @param ic_opaque The interceptor's opaque pointer specified in ..add..().
5705  *
5706  * @warning The on_thread_start() interceptor is called from internal
5707  *          librdkafka threads. An on_thread_start() interceptor MUST NOT
5708  *          call any librdkafka API's associated with the \p rk, or perform
5709  *          any blocking or prolonged work.
5710  *
5711  * @returns an error code on failure, the error is logged but otherwise ignored.
5712  */
5713 typedef rd_kafka_resp_err_t
5714 (rd_kafka_interceptor_f_on_thread_start_t) (
5715         rd_kafka_t *rk,
5716         rd_kafka_thread_type_t thread_type,
5717         const char *thread_name,
5718         void *ic_opaque);
5719 
5720 
5721 /**
5722  * @brief on_thread_exit() is called just prior to a librdkafka-managed
5723  *        thread exiting from the exiting thread itself.
5724  *
5725  * @param rk The client instance.
5726  * @param thread_type Thread type.n
5727  * @param thread_name Human-readable thread name, may not be unique.
5728  * @param ic_opaque The interceptor's opaque pointer specified in ..add..().
5729  *
5730  * @remark Depending on the thread type, librdkafka may execute additional
5731  *         code on the thread after on_thread_exit() returns.
5732  *
5733  * @warning The on_thread_exit() interceptor is called from internal
5734  *          librdkafka threads. An on_thread_exit() interceptor MUST NOT
5735  *          call any librdkafka API's associated with the \p rk, or perform
5736  *          any blocking or prolonged work.
5737  *
5738  * @returns an error code on failure, the error is logged but otherwise ignored.
5739  */
5740 typedef rd_kafka_resp_err_t
5741 (rd_kafka_interceptor_f_on_thread_exit_t) (
5742         rd_kafka_t *rk,
5743         rd_kafka_thread_type_t thread_type,
5744         const char *thread_name,
5745         void *ic_opaque);
5746 
5747 
5748 
5749 /**
5750  * @brief Append an on_conf_set() interceptor.
5751  *
5752  * @param conf Configuration object.
5753  * @param ic_name Interceptor name, used in logging.
5754  * @param on_conf_set Function pointer.
5755  * @param ic_opaque Opaque value that will be passed to the function.
5756  *
5757  * @returns RD_KAFKA_RESP_ERR_NO_ERROR on success or RD_KAFKA_RESP_ERR__CONFLICT
5758  *          if an existing intercepted with the same \p ic_name and function
5759  *          has already been added to \p conf.
5760  */
5761 RD_EXPORT rd_kafka_resp_err_t
5762 rd_kafka_conf_interceptor_add_on_conf_set (
5763         rd_kafka_conf_t *conf, const char *ic_name,
5764         rd_kafka_interceptor_f_on_conf_set_t *on_conf_set,
5765         void *ic_opaque);
5766 
5767 
5768 /**
5769  * @brief Append an on_conf_dup() interceptor.
5770  *
5771  * @param conf Configuration object.
5772  * @param ic_name Interceptor name, used in logging.
5773  * @param on_conf_dup Function pointer.
5774  * @param ic_opaque Opaque value that will be passed to the function.
5775  *
5776  * @returns RD_KAFKA_RESP_ERR_NO_ERROR on success or RD_KAFKA_RESP_ERR__CONFLICT
5777  *          if an existing intercepted with the same \p ic_name and function
5778  *          has already been added to \p conf.
5779  */
5780 RD_EXPORT rd_kafka_resp_err_t
5781 rd_kafka_conf_interceptor_add_on_conf_dup (
5782         rd_kafka_conf_t *conf, const char *ic_name,
5783         rd_kafka_interceptor_f_on_conf_dup_t *on_conf_dup,
5784         void *ic_opaque);
5785 
5786 /**
5787  * @brief Append an on_conf_destroy() interceptor.
5788  *
5789  * @param conf Configuration object.
5790  * @param ic_name Interceptor name, used in logging.
5791  * @param on_conf_destroy Function pointer.
5792  * @param ic_opaque Opaque value that will be passed to the function.
5793  *
5794  * @returns RD_KAFKA_RESP_ERR_NO_ERROR
5795  *
5796  * @remark Multiple on_conf_destroy() interceptors are allowed to be added
5797  *         to the same configuration object.
5798  */
5799 RD_EXPORT rd_kafka_resp_err_t
5800 rd_kafka_conf_interceptor_add_on_conf_destroy (
5801         rd_kafka_conf_t *conf, const char *ic_name,
5802         rd_kafka_interceptor_f_on_conf_destroy_t *on_conf_destroy,
5803         void *ic_opaque);
5804 
5805 
5806 /**
5807  * @brief Append an on_new() interceptor.
5808  *
5809  * @param conf Configuration object.
5810  * @param ic_name Interceptor name, used in logging.
5811  * @param on_new Function pointer.
5812  * @param ic_opaque Opaque value that will be passed to the function.
5813   *
5814  * @remark Since the on_new() interceptor is added to the configuration object
5815  *         it may be copied by rd_kafka_conf_dup().
5816  *         An interceptor implementation must thus be able to handle
5817  *         the same interceptor,ic_opaque tuple to be used by multiple
5818  *         client instances.
5819  *
5820  * @remark An interceptor plugin should check the return value to make sure it
5821  *         has not already been added.
5822  *
5823  * @returns RD_KAFKA_RESP_ERR_NO_ERROR on success or RD_KAFKA_RESP_ERR__CONFLICT
5824  *          if an existing intercepted with the same \p ic_name and function
5825  *          has already been added to \p conf.
5826  */
5827 RD_EXPORT rd_kafka_resp_err_t
5828 rd_kafka_conf_interceptor_add_on_new (
5829         rd_kafka_conf_t *conf, const char *ic_name,
5830         rd_kafka_interceptor_f_on_new_t *on_new,
5831         void *ic_opaque);
5832 
5833 
5834 
5835 /**
5836  * @brief Append an on_destroy() interceptor.
5837  *
5838  * @param rk Client instance.
5839  * @param ic_name Interceptor name, used in logging.
5840  * @param on_destroy Function pointer.
5841  * @param ic_opaque Opaque value that will be passed to the function.
5842  *
5843  * @returns RD_KAFKA_RESP_ERR_NO_ERROR on success or RD_KAFKA_RESP_ERR__CONFLICT
5844  *          if an existing intercepted with the same \p ic_name and function
5845  *          has already been added to \p conf.
5846  */
5847 RD_EXPORT rd_kafka_resp_err_t
5848 rd_kafka_interceptor_add_on_destroy (
5849         rd_kafka_t *rk, const char *ic_name,
5850         rd_kafka_interceptor_f_on_destroy_t *on_destroy,
5851         void *ic_opaque);
5852 
5853 
5854 /**
5855  * @brief Append an on_send() interceptor.
5856  *
5857  * @param rk Client instance.
5858  * @param ic_name Interceptor name, used in logging.
5859  * @param on_send Function pointer.
5860  * @param ic_opaque Opaque value that will be passed to the function.
5861  *
5862  * @returns RD_KAFKA_RESP_ERR_NO_ERROR on success or RD_KAFKA_RESP_ERR__CONFLICT
5863  *          if an existing intercepted with the same \p ic_name and function
5864  *          has already been added to \p conf.
5865  */
5866 RD_EXPORT rd_kafka_resp_err_t
5867 rd_kafka_interceptor_add_on_send (
5868         rd_kafka_t *rk, const char *ic_name,
5869         rd_kafka_interceptor_f_on_send_t *on_send,
5870         void *ic_opaque);
5871 
5872 /**
5873  * @brief Append an on_acknowledgement() interceptor.
5874  *
5875  * @param rk Client instance.
5876  * @param ic_name Interceptor name, used in logging.
5877  * @param on_acknowledgement Function pointer.
5878  * @param ic_opaque Opaque value that will be passed to the function.
5879  *
5880  * @returns RD_KAFKA_RESP_ERR_NO_ERROR on success or RD_KAFKA_RESP_ERR__CONFLICT
5881  *          if an existing intercepted with the same \p ic_name and function
5882  *          has already been added to \p conf.
5883  */
5884 RD_EXPORT rd_kafka_resp_err_t
5885 rd_kafka_interceptor_add_on_acknowledgement (
5886         rd_kafka_t *rk, const char *ic_name,
5887         rd_kafka_interceptor_f_on_acknowledgement_t *on_acknowledgement,
5888         void *ic_opaque);
5889 
5890 
5891 /**
5892  * @brief Append an on_consume() interceptor.
5893  *
5894  * @param rk Client instance.
5895  * @param ic_name Interceptor name, used in logging.
5896  * @param on_consume Function pointer.
5897  * @param ic_opaque Opaque value that will be passed to the function.
5898  *
5899  * @returns RD_KAFKA_RESP_ERR_NO_ERROR on success or RD_KAFKA_RESP_ERR__CONFLICT
5900  *          if an existing intercepted with the same \p ic_name and function
5901  *          has already been added to \p conf.
5902  */
5903 RD_EXPORT rd_kafka_resp_err_t
5904 rd_kafka_interceptor_add_on_consume (
5905         rd_kafka_t *rk, const char *ic_name,
5906         rd_kafka_interceptor_f_on_consume_t *on_consume,
5907         void *ic_opaque);
5908 
5909 
5910 /**
5911  * @brief Append an on_commit() interceptor.
5912  *
5913  * @param rk Client instance.
5914  * @param ic_name Interceptor name, used in logging.
5915  * @param on_commit() Function pointer.
5916  * @param ic_opaque Opaque value that will be passed to the function.
5917  *
5918  * @returns RD_KAFKA_RESP_ERR_NO_ERROR on success or RD_KAFKA_RESP_ERR__CONFLICT
5919  *          if an existing intercepted with the same \p ic_name and function
5920  *          has already been added to \p conf.
5921  */
5922 RD_EXPORT rd_kafka_resp_err_t
5923 rd_kafka_interceptor_add_on_commit (
5924         rd_kafka_t *rk, const char *ic_name,
5925         rd_kafka_interceptor_f_on_commit_t *on_commit,
5926         void *ic_opaque);
5927 
5928 
5929 /**
5930  * @brief Append an on_request_sent() interceptor.
5931  *
5932  * @param rk Client instance.
5933  * @param ic_name Interceptor name, used in logging.
5934  * @param on_request_sent() Function pointer.
5935  * @param ic_opaque Opaque value that will be passed to the function.
5936  *
5937  * @returns RD_KAFKA_RESP_ERR_NO_ERROR on success or RD_KAFKA_RESP_ERR__CONFLICT
5938  *          if an existing intercepted with the same \p ic_name and function
5939  *          has already been added to \p conf.
5940  */
5941 RD_EXPORT rd_kafka_resp_err_t
5942 rd_kafka_interceptor_add_on_request_sent (
5943         rd_kafka_t *rk, const char *ic_name,
5944         rd_kafka_interceptor_f_on_request_sent_t *on_request_sent,
5945         void *ic_opaque);
5946 
5947 
5948 /**
5949  * @brief Append an on_response_received() interceptor.
5950  *
5951  * @param rk Client instance.
5952  * @param ic_name Interceptor name, used in logging.
5953  * @param on_response_received() Function pointer.
5954  * @param ic_opaque Opaque value that will be passed to the function.
5955  *
5956  * @returns RD_KAFKA_RESP_ERR_NO_ERROR on success or RD_KAFKA_RESP_ERR__CONFLICT
5957  *          if an existing intercepted with the same \p ic_name and function
5958  *          has already been added to \p conf.
5959  */
5960 RD_EXPORT rd_kafka_resp_err_t
5961 rd_kafka_interceptor_add_on_response_received (
5962         rd_kafka_t *rk, const char *ic_name,
5963         rd_kafka_interceptor_f_on_response_received_t *on_response_received,
5964         void *ic_opaque);
5965 
5966 
5967 /**
5968  * @brief Append an on_thread_start() interceptor.
5969  *
5970  * @param rk Client instance.
5971  * @param ic_name Interceptor name, used in logging.
5972  * @param on_thread_start() Function pointer.
5973  * @param ic_opaque Opaque value that will be passed to the function.
5974  *
5975  * @returns RD_KAFKA_RESP_ERR_NO_ERROR on success or RD_KAFKA_RESP_ERR__CONFLICT
5976  *          if an existing intercepted with the same \p ic_name and function
5977  *          has already been added to \p conf.
5978  */
5979 RD_EXPORT rd_kafka_resp_err_t
5980 rd_kafka_interceptor_add_on_thread_start (
5981         rd_kafka_t *rk, const char *ic_name,
5982         rd_kafka_interceptor_f_on_thread_start_t *on_thread_start,
5983         void *ic_opaque);
5984 
5985 
5986 /**
5987  * @brief Append an on_thread_exit() interceptor.
5988  *
5989  * @param rk Client instance.
5990  * @param ic_name Interceptor name, used in logging.
5991  * @param on_thread_exit() Function pointer.
5992  * @param ic_opaque Opaque value that will be passed to the function.
5993  *
5994  * @returns RD_KAFKA_RESP_ERR_NO_ERROR on success or RD_KAFKA_RESP_ERR__CONFLICT
5995  *          if an existing intercepted with the same \p ic_name and function
5996  *          has already been added to \p conf.
5997  */
5998 RD_EXPORT rd_kafka_resp_err_t
5999 rd_kafka_interceptor_add_on_thread_exit (
6000         rd_kafka_t *rk, const char *ic_name,
6001         rd_kafka_interceptor_f_on_thread_exit_t *on_thread_exit,
6002         void *ic_opaque);
6003 
6004 
6005 
6006 /**@}*/
6007 
6008 
6009 
6010 /**
6011  * @name Auxiliary types
6012  *
6013  * @{
6014  */
6015 
6016 
6017 
6018 /**
6019  * @brief Topic result provides per-topic operation result information.
6020  *
6021  */
6022 
6023 /**
6024  * @returns the error code for the given topic result.
6025  */
6026 RD_EXPORT rd_kafka_resp_err_t
6027 rd_kafka_topic_result_error (const rd_kafka_topic_result_t *topicres);
6028 
6029 /**
6030  * @returns the human readable error string for the given topic result,
6031  *          or NULL if there was no error.
6032  *
6033  * @remark lifetime of the returned string is the same as the \p topicres.
6034  */
6035 RD_EXPORT const char *
6036 rd_kafka_topic_result_error_string (const rd_kafka_topic_result_t *topicres);
6037 
6038 /**
6039  * @returns the name of the topic for the given topic result.
6040  * @remark lifetime of the returned string is the same as the \p topicres.
6041  *
6042  */
6043 RD_EXPORT const char *
6044 rd_kafka_topic_result_name (const rd_kafka_topic_result_t *topicres);
6045 
6046 /**
6047  * @brief Group result provides per-group operation result information.
6048  *
6049  */
6050 
6051 /**
6052  * @returns the error for the given group result, or NULL on success.
6053  * @remark lifetime of the returned error is the same as the \p groupres.
6054  */
6055 RD_EXPORT const rd_kafka_error_t *
6056 rd_kafka_group_result_error (const rd_kafka_group_result_t *groupres);
6057 
6058 /**
6059  * @returns the name of the group for the given group result.
6060  * @remark lifetime of the returned string is the same as the \p groupres.
6061  *
6062  */
6063 RD_EXPORT const char *
6064 rd_kafka_group_result_name (const rd_kafka_group_result_t *groupres);
6065 
6066 /**
6067  * @returns the partitions/offsets for the given group result, if applicable
6068  *          to the request type, else NULL.
6069  * @remark lifetime of the returned list is the same as the \p groupres.
6070  */
6071 RD_EXPORT const rd_kafka_topic_partition_list_t *
6072 rd_kafka_group_result_partitions (const rd_kafka_group_result_t *groupres);
6073 
6074 
6075 /**@}*/
6076 
6077 
6078 /**
6079  * @name Admin API
6080  * @{
6081  *
6082  * @brief The Admin API enables applications to perform administrative
6083  *        Apache Kafka tasks, such as creating and deleting topics,
6084  *        altering and reading broker configuration, etc.
6085  *
6086  * The Admin API is asynchronous and makes use of librdkafka's standard
6087  * \c rd_kafka_queue_t queues to propagate the result of an admin operation
6088  * back to the application.
6089  * The supplied queue may be any queue, such as a temporary single-call queue,
6090  * a shared queue used for multiple requests, or even the main queue or
6091  * consumer queues.
6092  *
6093  * Use \c rd_kafka_queue_poll() to collect the result of an admin operation
6094  * from the queue of your choice, then extract the admin API-specific result
6095  * type by using the corresponding \c rd_kafka_event_CreateTopics_result,
6096  * \c rd_kafka_event_DescribeConfigs_result, etc, methods.
6097  * Use the getter methods on the \c .._result_t type to extract response
6098  * information and finally destroy the result and event by calling
6099  * \c rd_kafka_event_destroy().
6100  *
6101  * Use rd_kafka_event_error() and rd_kafka_event_error_string() to acquire
6102  * the request-level error/success for an Admin API request.
6103  * Even if the returned value is \c RD_KAFKA_RESP_ERR_NO_ERROR there
6104  * may be individual objects (topics, resources, etc) that have failed.
6105  * Extract per-object error information with the corresponding
6106  * \c rd_kafka_..._result_topics|resources|..() to check per-object errors.
6107  *
6108  * Locally triggered errors:
6109  *  - \c RD_KAFKA_RESP_ERR__TIMED_OUT - (Controller) broker connection did not
6110  *    become available in the time allowed by AdminOption_set_request_timeout.
6111   */
6112 
6113 
6114 /**
6115  * @enum rd_kafka_admin_op_t
6116  *
6117  * @brief Admin operation enum name for use with rd_kafka_AdminOptions_new()
6118  *
6119  * @sa rd_kafka_AdminOptions_new()
6120  */
6121 typedef enum rd_kafka_admin_op_t {
6122         RD_KAFKA_ADMIN_OP_ANY = 0,          /**< Default value */
6123         RD_KAFKA_ADMIN_OP_CREATETOPICS,     /**< CreateTopics */
6124         RD_KAFKA_ADMIN_OP_DELETETOPICS,     /**< DeleteTopics */
6125         RD_KAFKA_ADMIN_OP_CREATEPARTITIONS, /**< CreatePartitions */
6126         RD_KAFKA_ADMIN_OP_ALTERCONFIGS,     /**< AlterConfigs */
6127         RD_KAFKA_ADMIN_OP_DESCRIBECONFIGS,  /**< DescribeConfigs */
6128         RD_KAFKA_ADMIN_OP_DELETERECORDS,    /**< DeleteRecords */
6129         RD_KAFKA_ADMIN_OP_DELETEGROUPS,     /**< DeleteGroups */
6130         /** DeleteConsumerGroupOffsets */
6131         RD_KAFKA_ADMIN_OP_DELETECONSUMERGROUPOFFSETS,
6132         RD_KAFKA_ADMIN_OP__CNT              /**< Number of ops defined */
6133 } rd_kafka_admin_op_t;
6134 
6135 /**
6136  * @brief AdminOptions provides a generic mechanism for setting optional
6137  *        parameters for the Admin API requests.
6138  *
6139  * @remark Since AdminOptions is decoupled from the actual request type
6140  *         there is no enforcement to prevent setting unrelated properties,
6141  *         e.g. setting validate_only on a DescribeConfigs request is allowed
6142  *         but is silently ignored by DescribeConfigs.
6143  *         Future versions may introduce such enforcement.
6144  */
6145 
6146 
6147 typedef struct rd_kafka_AdminOptions_s rd_kafka_AdminOptions_t;
6148 
6149 /**
6150  * @brief Create a new AdminOptions object.
6151  *
6152  *        The options object is not modified by the Admin API request APIs,
6153  *        (e.g. CreateTopics) and may be reused for multiple calls.
6154  *
6155  * @param rk Client instance.
6156  * @param for_api Specifies what Admin API this AdminOptions object will be used
6157  *                for, which will enforce what AdminOptions_set_..() calls may
6158  *                be used based on the API, causing unsupported set..() calls
6159  *                to fail.
6160  *                Specifying RD_KAFKA_ADMIN_OP_ANY disables the enforcement
6161  *                allowing any option to be set, even if the option
6162  *                is not used in a future call to an Admin API method.
6163  *
6164  * @returns a new AdminOptions object (which must be freed with
6165  *          rd_kafka_AdminOptions_destroy()), or NULL if \p for_api was set to
6166  *          an unknown API op type.
6167  */
6168 RD_EXPORT rd_kafka_AdminOptions_t *
6169 rd_kafka_AdminOptions_new (rd_kafka_t *rk, rd_kafka_admin_op_t for_api);
6170 
6171 
6172 /**
6173  * @brief Destroy a AdminOptions object.
6174  */
6175 RD_EXPORT void rd_kafka_AdminOptions_destroy (rd_kafka_AdminOptions_t *options);
6176 
6177 
6178 /**
6179  * @brief Sets the overall request timeout, including broker lookup,
6180  *        request transmission, operation time on broker, and response.
6181  *
6182  * @param options Admin options.
6183  * @param timeout_ms Timeout in milliseconds, use -1 for indefinite timeout.
6184  *                   Defaults to `socket.timeout.ms`.
6185  * @param errstr A human readable error string (nul-terminated) is written to
6186  *               this location that must be of at least \p errstr_size bytes.
6187  *               The \p errstr is only written in case of error.
6188  * @param errstr_size Writable size in \p errstr.
6189  *
6190  * @returns RD_KAFKA_RESP_ERR_NO_ERROR on success, or
6191  *          RD_KAFKA_RESP_ERR__INVALID_ARG if timeout was out of range in which
6192  *          case an error string will be written \p errstr.
6193  *
6194  * @remark This option is valid for all Admin API requests.
6195  */
6196 RD_EXPORT rd_kafka_resp_err_t
6197 rd_kafka_AdminOptions_set_request_timeout (rd_kafka_AdminOptions_t *options,
6198                                            int timeout_ms,
6199                                            char *errstr, size_t errstr_size);
6200 
6201 
6202 /**
6203  * @brief Sets the broker's operation timeout, such as the timeout for
6204  *        CreateTopics to complete the creation of topics on the controller
6205  *        before returning a result to the application.
6206  *
6207  * CreateTopics: values <= 0 will return immediately after triggering topic
6208  * creation, while > 0 will wait this long for topic creation to propagate
6209  * in cluster. Default: 60 seconds.
6210  *
6211  * DeleteTopics: same semantics as CreateTopics.
6212  * CreatePartitions: same semantics as CreateTopics.
6213  *
6214  * @param options Admin options.
6215  * @param timeout_ms Timeout in milliseconds.
6216  * @param errstr A human readable error string (nul-terminated) is written to
6217  *               this location that must be of at least \p errstr_size bytes.
6218  *               The \p errstr is only written in case of error.
6219  * @param errstr_size Writable size in \p errstr.
6220  *
6221  * @returns RD_KAFKA_RESP_ERR_NO_ERROR on success, or
6222  *          RD_KAFKA_RESP_ERR__INVALID_ARG if timeout was out of range in which
6223  *          case an error string will be written \p errstr.
6224  *
6225  * @remark This option is valid for CreateTopics, DeleteTopics,
6226  *         CreatePartitions, and DeleteRecords.
6227  */
6228 RD_EXPORT rd_kafka_resp_err_t
6229 rd_kafka_AdminOptions_set_operation_timeout (rd_kafka_AdminOptions_t *options,
6230                                              int timeout_ms,
6231                                              char *errstr, size_t errstr_size);
6232 
6233 
6234 /**
6235  * @brief Tell broker to only validate the request, without performing
6236  *        the requested operation (create topics, etc).
6237  *
6238  * @param options Admin options.
6239  * @param true_or_false Defaults to false.
6240  * @param errstr A human readable error string (nul-terminated) is written to
6241  *               this location that must be of at least \p errstr_size bytes.
6242  *               The \p errstr is only written in case of error.
6243  * @param errstr_size Writable size in \p errstr.
6244  *
6245  * @returns RD_KAFKA_RESP_ERR_NO_ERROR on success or an
6246  *          error code on failure in which case an error string will
6247  *          be written \p errstr.
6248  *
6249  * @remark This option is valid for CreateTopics,
6250  *         CreatePartitions, AlterConfigs.
6251  */
6252 RD_EXPORT rd_kafka_resp_err_t
6253 rd_kafka_AdminOptions_set_validate_only (rd_kafka_AdminOptions_t *options,
6254                                         int true_or_false,
6255                                         char *errstr, size_t errstr_size);
6256 
6257 
6258 /**
6259  * @brief Override what broker the Admin request will be sent to.
6260  *
6261  * By default, Admin requests are sent to the controller broker, with
6262  * the following exceptions:
6263  *   - AlterConfigs with a BROKER resource are sent to the broker id set
6264  *     as the resource name.
6265  *   - DescribeConfigs with a BROKER resource are sent to the broker id set
6266  *     as the resource name.
6267  *
6268  * @param options Admin Options.
6269  * @param broker_id The broker to send the request to.
6270  * @param errstr A human readable error string (nul-terminated) is written to
6271  *               this location that must be of at least \p errstr_size bytes.
6272  *               The \p errstr is only written in case of error.
6273  * @param errstr_size Writable size in \p errstr.
6274  *
6275  * @returns RD_KAFKA_RESP_ERR_NO_ERROR on success or an
6276  *          error code on failure in which case an error string will
6277  *          be written \p errstr.
6278  *
6279  * @remark This API should typically not be used, but serves as a workaround
6280  *         if new resource types are to the broker that the client
6281  *         does not know where to send.
6282  */
6283 RD_EXPORT rd_kafka_resp_err_t
6284 rd_kafka_AdminOptions_set_broker (rd_kafka_AdminOptions_t *options,
6285                                   int32_t broker_id,
6286                                   char *errstr, size_t errstr_size);
6287 
6288 
6289 
6290 /**
6291  * @brief Set application opaque value that can be extracted from the
6292  *        result event using rd_kafka_event_opaque()
6293  */
6294 RD_EXPORT void
6295 rd_kafka_AdminOptions_set_opaque (rd_kafka_AdminOptions_t *options,
6296                                   void *ev_opaque);
6297 
6298 
6299 
6300 
6301 
6302 
6303 /*
6304  * CreateTopics - create topics in cluster.
6305  *
6306  */
6307 
6308 
6309 /*! Defines a new topic to be created. */
6310 typedef struct rd_kafka_NewTopic_s rd_kafka_NewTopic_t;
6311 
6312 /**
6313  * @brief Create a new NewTopic object. This object is later passed to
6314  *        rd_kafka_CreateTopics().
6315  *
6316  * @param topic Topic name to create.
6317  * @param num_partitions Number of partitions in topic, or -1 to use the
6318  *                       broker's default partition count (>= 2.4.0).
6319  * @param replication_factor Default replication factor for the topic's
6320  *                           partitions, or -1 to use the broker's default
6321  *                           replication factor (>= 2.4.0) or if
6322  *                           set_replica_assignment() will be used.
6323  * @param errstr A human readable error string (nul-terminated) is written to
6324  *               this location that must be of at least \p errstr_size bytes.
6325  *               The \p errstr is only written in case of error.
6326  * @param errstr_size Writable size in \p errstr.
6327  *
6328  *
6329  * @returns a new allocated NewTopic object, or NULL if the input parameters
6330  *          are invalid.
6331  *          Use rd_kafka_NewTopic_destroy() to free object when done.
6332  */
6333 RD_EXPORT rd_kafka_NewTopic_t *
6334 rd_kafka_NewTopic_new (const char *topic, int num_partitions,
6335                        int replication_factor,
6336                        char *errstr, size_t errstr_size);
6337 
6338 /**
6339  * @brief Destroy and free a NewTopic object previously created with
6340  *        rd_kafka_NewTopic_new()
6341  */
6342 RD_EXPORT void
6343 rd_kafka_NewTopic_destroy (rd_kafka_NewTopic_t *new_topic);
6344 
6345 
6346 /**
6347  * @brief Helper function to destroy all NewTopic objects in the \p new_topics
6348  *        array (of \p new_topic_cnt elements).
6349  *        The array itself is not freed.
6350  */
6351 RD_EXPORT void
6352 rd_kafka_NewTopic_destroy_array (rd_kafka_NewTopic_t **new_topics,
6353                                  size_t new_topic_cnt);
6354 
6355 
6356 /**
6357  * @brief Set the replica (broker) assignment for \p partition to the
6358  *        replica set in \p broker_ids (of \p broker_id_cnt elements).
6359  *
6360  * @remark When this method is used, rd_kafka_NewTopic_new() must have
6361  *         been called with a \c replication_factor of -1.
6362  *
6363  * @remark An application must either set the replica assignment for
6364  *         all new partitions, or none.
6365  *
6366  * @remark If called, this function must be called consecutively for each
6367  *         partition, starting at 0.
6368  *
6369  * @remark Use rd_kafka_metadata() to retrieve the list of brokers
6370  *         in the cluster.
6371  *
6372  * @returns RD_KAFKA_RESP_ERR_NO_ERROR on success, or an error code
6373  *          if the arguments were invalid.
6374  *
6375  * @sa rd_kafka_AdminOptions_set_validate_only()
6376  */
6377 RD_EXPORT rd_kafka_resp_err_t
6378 rd_kafka_NewTopic_set_replica_assignment (rd_kafka_NewTopic_t *new_topic,
6379                                           int32_t partition,
6380                                           int32_t *broker_ids,
6381                                           size_t broker_id_cnt,
6382                                           char *errstr, size_t errstr_size);
6383 
6384 /**
6385  * @brief Set (broker-side) topic configuration name/value pair.
6386  *
6387  * @remark The name and value are not validated by the client, the validation
6388  *         takes place on the broker.
6389  *
6390  * @returns RD_KAFKA_RESP_ERR_NO_ERROR on success, or an error code
6391  *          if the arguments were invalid.
6392  *
6393  * @sa rd_kafka_AdminOptions_set_validate_only()
6394  * @sa http://kafka.apache.org/documentation.html#topicconfigs
6395  */
6396 RD_EXPORT rd_kafka_resp_err_t
6397 rd_kafka_NewTopic_set_config (rd_kafka_NewTopic_t *new_topic,
6398                               const char *name, const char *value);
6399 
6400 
6401 /**
6402  * @brief Create topics in cluster as specified by the \p new_topics
6403  *        array of size \p new_topic_cnt elements.
6404  *
6405  * @param rk Client instance.
6406  * @param new_topics Array of new topics to create.
6407  * @param new_topic_cnt Number of elements in \p new_topics array.
6408  * @param options Optional admin options, or NULL for defaults.
6409  * @param rkqu Queue to emit result on.
6410  *
6411  * Supported admin options:
6412  *  - rd_kafka_AdminOptions_set_validate_only() - default false
6413  *  - rd_kafka_AdminOptions_set_operation_timeout() - default 60 seconds
6414  *  - rd_kafka_AdminOptions_set_request_timeout() - default socket.timeout.ms
6415  *
6416  * @remark The result event type emitted on the supplied queue is of type
6417  *         \c RD_KAFKA_EVENT_CREATETOPICS_RESULT
6418  */
6419 RD_EXPORT void
6420 rd_kafka_CreateTopics (rd_kafka_t *rk,
6421                        rd_kafka_NewTopic_t **new_topics,
6422                        size_t new_topic_cnt,
6423                        const rd_kafka_AdminOptions_t *options,
6424                        rd_kafka_queue_t *rkqu);
6425 
6426 
6427 /*
6428  * CreateTopics result type and methods
6429  */
6430 
6431 /**
6432  * @brief Get an array of topic results from a CreateTopics result.
6433  *
6434  * The returned \p topics life-time is the same as the \p result object.
6435  *
6436  * @param result Result to get topics from.
6437  * @param cntp Updated to the number of elements in the array.
6438  */
6439 RD_EXPORT const rd_kafka_topic_result_t **
6440 rd_kafka_CreateTopics_result_topics (
6441         const rd_kafka_CreateTopics_result_t *result,
6442         size_t *cntp);
6443 
6444 
6445 
6446 
6447 
6448 /*
6449  * DeleteTopics - delete topics from cluster
6450  *
6451  */
6452 
6453 /*! Represents a topic to be deleted. */
6454 typedef struct rd_kafka_DeleteTopic_s rd_kafka_DeleteTopic_t;
6455 
6456 /**
6457  * @brief Create a new DeleteTopic object. This object is later passed to
6458  *        rd_kafka_DeleteTopics().
6459  *
6460  * @param topic Topic name to delete.
6461  *
6462  * @returns a new allocated DeleteTopic object.
6463  *          Use rd_kafka_DeleteTopic_destroy() to free object when done.
6464  */
6465 RD_EXPORT rd_kafka_DeleteTopic_t *
6466 rd_kafka_DeleteTopic_new (const char *topic);
6467 
6468 /**
6469  * @brief Destroy and free a DeleteTopic object previously created with
6470  *        rd_kafka_DeleteTopic_new()
6471  */
6472 RD_EXPORT void
6473 rd_kafka_DeleteTopic_destroy (rd_kafka_DeleteTopic_t *del_topic);
6474 
6475 /**
6476  * @brief Helper function to destroy all DeleteTopic objects in
6477  *        the \p del_topics array (of \p del_topic_cnt elements).
6478  *        The array itself is not freed.
6479  */
6480 RD_EXPORT void
6481 rd_kafka_DeleteTopic_destroy_array (rd_kafka_DeleteTopic_t **del_topics,
6482                                     size_t del_topic_cnt);
6483 
6484 /**
6485  * @brief Delete topics from cluster as specified by the \p topics
6486  *        array of size \p topic_cnt elements.
6487  *
6488  * @param rk Client instance.
6489  * @param del_topics Array of topics to delete.
6490  * @param del_topic_cnt Number of elements in \p topics array.
6491  * @param options Optional admin options, or NULL for defaults.
6492  * @param rkqu Queue to emit result on.
6493  *
6494  * @remark The result event type emitted on the supplied queue is of type
6495  *         \c RD_KAFKA_EVENT_DELETETOPICS_RESULT
6496  */
6497 RD_EXPORT
6498 void rd_kafka_DeleteTopics (rd_kafka_t *rk,
6499                             rd_kafka_DeleteTopic_t **del_topics,
6500                             size_t del_topic_cnt,
6501                             const rd_kafka_AdminOptions_t *options,
6502                             rd_kafka_queue_t *rkqu);
6503 
6504 
6505 
6506 /*
6507  * DeleteTopics result type and methods
6508  */
6509 
6510 /**
6511  * @brief Get an array of topic results from a DeleteTopics result.
6512  *
6513  * The returned \p topics life-time is the same as the \p result object.
6514  *
6515  * @param result Result to get topic results from.
6516  * @param cntp is updated to the number of elements in the array.
6517  */
6518 RD_EXPORT const rd_kafka_topic_result_t **
6519 rd_kafka_DeleteTopics_result_topics (
6520         const rd_kafka_DeleteTopics_result_t *result,
6521         size_t *cntp);
6522 
6523 
6524 
6525 
6526 
6527 
6528 /*
6529  * CreatePartitions - add partitions to topic.
6530  *
6531  */
6532 
6533 /*! Defines a new partition to be created. */
6534 typedef struct rd_kafka_NewPartitions_s rd_kafka_NewPartitions_t;
6535 
6536 /**
6537  * @brief Create a new NewPartitions. This object is later passed to
6538  *        rd_kafka_CreatePartitions() to increase the number of partitions
6539  *        to \p new_total_cnt for an existing topic.
6540  *
6541  * @param topic Topic name to create more partitions for.
6542  * @param new_total_cnt Increase the topic's partition count to this value.
6543  * @param errstr A human readable error string (nul-terminated) is written to
6544  *               this location that must be of at least \p errstr_size bytes.
6545  *               The \p errstr is only written in case of error.
6546  * @param errstr_size Writable size in \p errstr.
6547  *
6548  * @returns a new allocated NewPartitions object, or NULL if the
6549  *          input parameters are invalid.
6550  *          Use rd_kafka_NewPartitions_destroy() to free object when done.
6551  */
6552 RD_EXPORT rd_kafka_NewPartitions_t *
6553 rd_kafka_NewPartitions_new (const char *topic, size_t new_total_cnt,
6554                             char *errstr, size_t errstr_size);
6555 
6556 /**
6557  * @brief Destroy and free a NewPartitions object previously created with
6558  *        rd_kafka_NewPartitions_new()
6559  */
6560 RD_EXPORT void
6561 rd_kafka_NewPartitions_destroy (rd_kafka_NewPartitions_t *new_parts);
6562 
6563 /**
6564  * @brief Helper function to destroy all NewPartitions objects in the
6565  *        \p new_parts array (of \p new_parts_cnt elements).
6566  *        The array itself is not freed.
6567  */
6568 RD_EXPORT void
6569 rd_kafka_NewPartitions_destroy_array (rd_kafka_NewPartitions_t **new_parts,
6570                                       size_t new_parts_cnt);
6571 
6572 /**
6573  * @brief Set the replica (broker id) assignment for \p new_partition_idx to the
6574  *        replica set in \p broker_ids (of \p broker_id_cnt elements).
6575  *
6576  * @remark An application must either set the replica assignment for
6577  *         all new partitions, or none.
6578  *
6579  * @remark If called, this function must be called consecutively for each
6580  *         new partition being created,
6581  *         where \p new_partition_idx 0 is the first new partition,
6582  *         1 is the second, and so on.
6583  *
6584  * @remark \p broker_id_cnt should match the topic's replication factor.
6585  *
6586  * @remark Use rd_kafka_metadata() to retrieve the list of brokers
6587  *         in the cluster.
6588  *
6589  * @returns RD_KAFKA_RESP_ERR_NO_ERROR on success, or an error code
6590  *          if the arguments were invalid.
6591  *
6592  * @sa rd_kafka_AdminOptions_set_validate_only()
6593  */
6594 RD_EXPORT rd_kafka_resp_err_t
6595 rd_kafka_NewPartitions_set_replica_assignment (rd_kafka_NewPartitions_t *new_parts,
6596                                                int32_t new_partition_idx,
6597                                                int32_t *broker_ids,
6598                                                size_t broker_id_cnt,
6599                                                char *errstr,
6600                                                size_t errstr_size);
6601 
6602 
6603 /**
6604  * @brief Create additional partitions for the given topics, as specified
6605  *        by the \p new_parts array of size \p new_parts_cnt elements.
6606  *
6607  * @param rk Client instance.
6608  * @param new_parts Array of topics for which new partitions are to be created.
6609  * @param new_parts_cnt Number of elements in \p new_parts array.
6610  * @param options Optional admin options, or NULL for defaults.
6611  * @param rkqu Queue to emit result on.
6612  *
6613  * Supported admin options:
6614  *  - rd_kafka_AdminOptions_set_validate_only() - default false
6615  *  - rd_kafka_AdminOptions_set_operation_timeout() - default 60 seconds
6616  *  - rd_kafka_AdminOptions_set_request_timeout() - default socket.timeout.ms
6617  *
6618  * @remark The result event type emitted on the supplied queue is of type
6619  *         \c RD_KAFKA_EVENT_CREATEPARTITIONS_RESULT
6620  */
6621 RD_EXPORT void
6622 rd_kafka_CreatePartitions (rd_kafka_t *rk,
6623                            rd_kafka_NewPartitions_t **new_parts,
6624                            size_t new_parts_cnt,
6625                            const rd_kafka_AdminOptions_t *options,
6626                            rd_kafka_queue_t *rkqu);
6627 
6628 
6629 
6630 /*
6631  * CreatePartitions result type and methods
6632  */
6633 
6634 /**
6635  * @brief Get an array of topic results from a CreatePartitions result.
6636  *
6637  * The returned \p topics life-time is the same as the \p result object.
6638  *
6639  * @param result Result o get topic results from.
6640  * @param cntp is updated to the number of elements in the array.
6641  */
6642 RD_EXPORT const rd_kafka_topic_result_t **
6643 rd_kafka_CreatePartitions_result_topics (
6644         const rd_kafka_CreatePartitions_result_t *result,
6645         size_t *cntp);
6646 
6647 
6648 
6649 
6650 
6651 /*
6652  * Cluster, broker, topic configuration entries, sources, etc.
6653  *
6654  */
6655 
6656 /**
6657  * @enum rd_kafka_ConfigSource_t
6658  *
6659  * @brief Apache Kafka config sources.
6660  *
6661  * @remark These entities relate to the cluster, not the local client.
6662  *
6663  * @sa rd_kafka_conf_set(), et.al. for local client configuration.
6664  */
6665 typedef enum rd_kafka_ConfigSource_t {
6666         /** Source unknown, e.g., in the ConfigEntry used for alter requests
6667          *  where source is not set */
6668         RD_KAFKA_CONFIG_SOURCE_UNKNOWN_CONFIG = 0,
6669         /** Dynamic topic config that is configured for a specific topic */
6670         RD_KAFKA_CONFIG_SOURCE_DYNAMIC_TOPIC_CONFIG = 1,
6671         /** Dynamic broker config that is configured for a specific broker */
6672         RD_KAFKA_CONFIG_SOURCE_DYNAMIC_BROKER_CONFIG = 2,
6673         /** Dynamic broker config that is configured as default for all
6674          *  brokers in the cluster */
6675         RD_KAFKA_CONFIG_SOURCE_DYNAMIC_DEFAULT_BROKER_CONFIG = 3,
6676         /** Static broker config provided as broker properties at startup
6677          *  (e.g. from server.properties file) */
6678         RD_KAFKA_CONFIG_SOURCE_STATIC_BROKER_CONFIG = 4,
6679         /** Built-in default configuration for configs that have a
6680          *  default value */
6681         RD_KAFKA_CONFIG_SOURCE_DEFAULT_CONFIG = 5,
6682 
6683         /** Number of source types defined */
6684         RD_KAFKA_CONFIG_SOURCE__CNT,
6685 } rd_kafka_ConfigSource_t;
6686 
6687 
6688 /**
6689  * @returns a string representation of the \p confsource.
6690  */
6691 RD_EXPORT const char *
6692 rd_kafka_ConfigSource_name (rd_kafka_ConfigSource_t confsource);
6693 
6694 
6695 /*! Apache Kafka configuration entry. */
6696 typedef struct rd_kafka_ConfigEntry_s rd_kafka_ConfigEntry_t;
6697 
6698 /**
6699  * @returns the configuration property name
6700  */
6701 RD_EXPORT const char *
6702 rd_kafka_ConfigEntry_name (const rd_kafka_ConfigEntry_t *entry);
6703 
6704 /**
6705  * @returns the configuration value, may be NULL for sensitive or unset
6706  *          properties.
6707  */
6708 RD_EXPORT const char *
6709 rd_kafka_ConfigEntry_value (const rd_kafka_ConfigEntry_t *entry);
6710 
6711 /**
6712  * @returns the config source.
6713  */
6714 RD_EXPORT rd_kafka_ConfigSource_t
6715 rd_kafka_ConfigEntry_source (const rd_kafka_ConfigEntry_t *entry);
6716 
6717 /**
6718  * @returns 1 if the config property is read-only on the broker, else 0.
6719  * @remark Shall only be used on a DescribeConfigs result, otherwise returns -1.
6720  */
6721 RD_EXPORT int
6722 rd_kafka_ConfigEntry_is_read_only (const rd_kafka_ConfigEntry_t *entry);
6723 
6724 /**
6725  * @returns 1 if the config property is set to its default value on the broker,
6726  *          else 0.
6727  * @remark Shall only be used on a DescribeConfigs result, otherwise returns -1.
6728  */
6729 RD_EXPORT int
6730 rd_kafka_ConfigEntry_is_default (const rd_kafka_ConfigEntry_t *entry);
6731 
6732 /**
6733  * @returns 1 if the config property contains sensitive information (such as
6734  *          security configuration), else 0.
6735  * @remark An application should take care not to include the value of
6736  *         sensitive configuration entries in its output.
6737  * @remark Shall only be used on a DescribeConfigs result, otherwise returns -1.
6738  */
6739 RD_EXPORT int
6740 rd_kafka_ConfigEntry_is_sensitive (const rd_kafka_ConfigEntry_t *entry);
6741 
6742 /**
6743  * @returns 1 if this entry is a synonym, else 0.
6744  */
6745 RD_EXPORT int
6746 rd_kafka_ConfigEntry_is_synonym (const rd_kafka_ConfigEntry_t *entry);
6747 
6748 
6749 /**
6750  * @returns the synonym config entry array.
6751  *
6752  * @param entry Entry to get synonyms for.
6753  * @param cntp is updated to the number of elements in the array.
6754  *
6755  * @remark The lifetime of the returned entry is the same as \p conf .
6756  * @remark Shall only be used on a DescribeConfigs result,
6757  *         otherwise returns NULL.
6758  */
6759 RD_EXPORT const rd_kafka_ConfigEntry_t **
6760 rd_kafka_ConfigEntry_synonyms (const rd_kafka_ConfigEntry_t *entry,
6761                                size_t *cntp);
6762 
6763 
6764 
6765 
6766 /*! Apache Kafka resource types */
6767 typedef enum rd_kafka_ResourceType_t {
6768         RD_KAFKA_RESOURCE_UNKNOWN = 0, /**< Unknown */
6769         RD_KAFKA_RESOURCE_ANY = 1,     /**< Any (used for lookups) */
6770         RD_KAFKA_RESOURCE_TOPIC = 2,   /**< Topic */
6771         RD_KAFKA_RESOURCE_GROUP = 3,   /**< Group */
6772         RD_KAFKA_RESOURCE_BROKER = 4,  /**< Broker */
6773         RD_KAFKA_RESOURCE__CNT,        /**< Number of resource types defined */
6774 } rd_kafka_ResourceType_t;
6775 
6776 /**
6777  * @returns a string representation of the \p restype
6778  */
6779 RD_EXPORT const char *
6780 rd_kafka_ResourceType_name (rd_kafka_ResourceType_t restype);
6781 
6782 /*! Apache Kafka configuration resource. */
6783 typedef struct rd_kafka_ConfigResource_s rd_kafka_ConfigResource_t;
6784 
6785 
6786 /**
6787  * @brief Create new ConfigResource object.
6788  *
6789  * @param restype The resource type (e.g., RD_KAFKA_RESOURCE_TOPIC)
6790  * @param resname The resource name (e.g., the topic name)
6791  *
6792  * @returns a newly allocated object
6793  */
6794 RD_EXPORT rd_kafka_ConfigResource_t *
6795 rd_kafka_ConfigResource_new (rd_kafka_ResourceType_t restype,
6796                              const char *resname);
6797 
6798 /**
6799  * @brief Destroy and free a ConfigResource object previously created with
6800  *        rd_kafka_ConfigResource_new()
6801  */
6802 RD_EXPORT void
6803 rd_kafka_ConfigResource_destroy (rd_kafka_ConfigResource_t *config);
6804 
6805 
6806 /**
6807  * @brief Helper function to destroy all ConfigResource objects in
6808  *        the \p configs array (of \p config_cnt elements).
6809  *        The array itself is not freed.
6810  */
6811 RD_EXPORT void
6812 rd_kafka_ConfigResource_destroy_array (rd_kafka_ConfigResource_t **config,
6813                                        size_t config_cnt);
6814 
6815 
6816 /**
6817  * @brief Set configuration name value pair.
6818  *
6819  * @param config ConfigResource to set config property on.
6820  * @param name Configuration name, depends on resource type.
6821  * @param value Configuration value, depends on resource type and \p name.
6822  *              Set to \c NULL to revert configuration value to default.
6823  *
6824  * This will overwrite the current value.
6825  *
6826  * @returns RD_KAFKA_RESP_ERR_NO_ERROR if config was added to resource,
6827  *          or RD_KAFKA_RESP_ERR__INVALID_ARG on invalid input.
6828  */
6829 RD_EXPORT rd_kafka_resp_err_t
6830 rd_kafka_ConfigResource_set_config (rd_kafka_ConfigResource_t *config,
6831                                     const char *name, const char *value);
6832 
6833 
6834 /**
6835  * @brief Get an array of config entries from a ConfigResource object.
6836  *
6837  * The returned object life-times are the same as the \p config object.
6838  *
6839  * @param config ConfigResource to get configs from.
6840  * @param cntp is updated to the number of elements in the array.
6841  */
6842 RD_EXPORT const rd_kafka_ConfigEntry_t **
6843 rd_kafka_ConfigResource_configs (const rd_kafka_ConfigResource_t *config,
6844                                  size_t *cntp);
6845 
6846 
6847 
6848 /**
6849  * @returns the ResourceType for \p config
6850  */
6851 RD_EXPORT rd_kafka_ResourceType_t
6852 rd_kafka_ConfigResource_type (const rd_kafka_ConfigResource_t *config);
6853 
6854 /**
6855  * @returns the name for \p config
6856  */
6857 RD_EXPORT const char *
6858 rd_kafka_ConfigResource_name (const rd_kafka_ConfigResource_t *config);
6859 
6860 /**
6861  * @returns the error for this resource from an AlterConfigs request
6862  */
6863 RD_EXPORT rd_kafka_resp_err_t
6864 rd_kafka_ConfigResource_error (const rd_kafka_ConfigResource_t *config);
6865 
6866 /**
6867  * @returns the error string for this resource from an AlterConfigs
6868  *          request, or NULL if no error.
6869  */
6870 RD_EXPORT const char *
6871 rd_kafka_ConfigResource_error_string (const rd_kafka_ConfigResource_t *config);
6872 
6873 
6874 /*
6875  * AlterConfigs - alter cluster configuration.
6876  *
6877  */
6878 
6879 
6880 /**
6881  * @brief Update the configuration for the specified resources.
6882  *        Updates are not transactional so they may succeed for a subset
6883  *        of the provided resources while the others fail.
6884  *        The configuration for a particular resource is updated atomically,
6885  *        replacing values using the provided ConfigEntrys and reverting
6886  *        unspecified ConfigEntrys to their default values.
6887  *
6888  * @remark Requires broker version >=0.11.0.0
6889  *
6890  * @warning AlterConfigs will replace all existing configuration for
6891  *          the provided resources with the new configuration given,
6892  *          reverting all other configuration to their default values.
6893  *
6894  * @remark Multiple resources and resource types may be set, but at most one
6895  *         resource of type \c RD_KAFKA_RESOURCE_BROKER is allowed per call
6896  *         since these resource requests must be sent to the broker specified
6897  *         in the resource.
6898  *
6899  */
6900 RD_EXPORT
6901 void rd_kafka_AlterConfigs (rd_kafka_t *rk,
6902                             rd_kafka_ConfigResource_t **configs,
6903                             size_t config_cnt,
6904                             const rd_kafka_AdminOptions_t *options,
6905                             rd_kafka_queue_t *rkqu);
6906 
6907 
6908 /*
6909  * AlterConfigs result type and methods
6910  */
6911 
6912 /**
6913  * @brief Get an array of resource results from a AlterConfigs result.
6914  *
6915  * Use \c rd_kafka_ConfigResource_error() and
6916  * \c rd_kafka_ConfigResource_error_string() to extract per-resource error
6917  * results on the returned array elements.
6918  *
6919  * The returned object life-times are the same as the \p result object.
6920  *
6921  * @param result Result object to get resource results from.
6922  * @param cntp is updated to the number of elements in the array.
6923  *
6924  * @returns an array of ConfigResource elements, or NULL if not available.
6925  */
6926 RD_EXPORT const rd_kafka_ConfigResource_t **
6927 rd_kafka_AlterConfigs_result_resources (
6928         const rd_kafka_AlterConfigs_result_t *result,
6929         size_t *cntp);
6930 
6931 
6932 
6933 
6934 
6935 
6936 /*
6937  * DescribeConfigs - retrieve cluster configuration.
6938  *
6939  */
6940 
6941 
6942 /**
6943  * @brief Get configuration for the specified resources in \p configs.
6944  *
6945  * The returned configuration includes default values and the
6946  * rd_kafka_ConfigEntry_is_default() or rd_kafka_ConfigEntry_source()
6947  * methods may be used to distinguish them from user supplied values.
6948  *
6949  * The value of config entries where rd_kafka_ConfigEntry_is_sensitive()
6950  * is true will always be NULL to avoid disclosing sensitive
6951  * information, such as security settings.
6952  *
6953  * Configuration entries where rd_kafka_ConfigEntry_is_read_only()
6954  * is true can't be updated (with rd_kafka_AlterConfigs()).
6955  *
6956  * Synonym configuration entries are returned if the broker supports
6957  * it (broker version >= 1.1.0). See rd_kafka_ConfigEntry_synonyms().
6958  *
6959  * @remark Requires broker version >=0.11.0.0
6960  *
6961  * @remark Multiple resources and resource types may be requested, but at most
6962  *         one resource of type \c RD_KAFKA_RESOURCE_BROKER is allowed per call
6963  *         since these resource requests must be sent to the broker specified
6964  *         in the resource.
6965  */
6966 RD_EXPORT
6967 void rd_kafka_DescribeConfigs (rd_kafka_t *rk,
6968                                rd_kafka_ConfigResource_t **configs,
6969                                size_t config_cnt,
6970                                const rd_kafka_AdminOptions_t *options,
6971                                rd_kafka_queue_t *rkqu);
6972 
6973 
6974 
6975 
6976 /*
6977  * DescribeConfigs result type and methods
6978  */
6979 
6980 /**
6981  * @brief Get an array of resource results from a DescribeConfigs result.
6982  *
6983  * The returned \p resources life-time is the same as the \p result object.
6984  *
6985  * @param result Result object to get resource results from.
6986  * @param cntp is updated to the number of elements in the array.
6987  */
6988 RD_EXPORT const rd_kafka_ConfigResource_t **
6989 rd_kafka_DescribeConfigs_result_resources (
6990         const rd_kafka_DescribeConfigs_result_t *result,
6991         size_t *cntp);
6992 
6993 
6994 /*
6995  * DeleteRecords - delete records (messages) from partitions
6996  *
6997  *
6998  */
6999 
7000 /**! Represents records to be deleted */
7001 typedef struct rd_kafka_DeleteRecords_s rd_kafka_DeleteRecords_t;
7002 
7003 /**
7004  * @brief Create a new DeleteRecords object. This object is later passed to
7005  *        rd_kafka_DeleteRecords().
7006  *
7007  * \p before_offsets must contain \c topic, \c partition, and
7008  * \c offset is the offset before which the messages will
7009  * be deleted (exclusive).
7010  * Set \c offset to RD_KAFKA_OFFSET_END (high-watermark) in order to
7011  * delete all data in the partition.
7012  *
7013  * @param before_offsets For each partition delete all messages up to but not
7014  *                       including the specified offset.
7015  *
7016  * @returns a new allocated DeleteRecords object.
7017  *          Use rd_kafka_DeleteRecords_destroy() to free object when done.
7018  */
7019 RD_EXPORT rd_kafka_DeleteRecords_t *
7020 rd_kafka_DeleteRecords_new (const rd_kafka_topic_partition_list_t *
7021                             before_offsets);
7022 
7023 /**
7024  * @brief Destroy and free a DeleteRecords object previously created with
7025  *        rd_kafka_DeleteRecords_new()
7026  */
7027 RD_EXPORT void
7028 rd_kafka_DeleteRecords_destroy (rd_kafka_DeleteRecords_t *del_records);
7029 
7030 /**
7031  * @brief Helper function to destroy all DeleteRecords objects in
7032  *        the \p del_groups array (of \p del_group_cnt elements).
7033  *        The array itself is not freed.
7034  */
7035 RD_EXPORT void
7036 rd_kafka_DeleteRecords_destroy_array (rd_kafka_DeleteRecords_t **del_records,
7037                                       size_t del_record_cnt);
7038 
7039 /**
7040  * @brief Delete records (messages) in topic partitions older than the
7041  *        offsets provided.
7042  *
7043  * @param rk Client instance.
7044  * @param del_records The offsets to delete (up to).
7045  *                    Currently only one DeleteRecords_t (but containing
7046  *                    multiple offsets) is supported.
7047  * @param del_record_cnt The number of elements in del_records, must be 1.
7048  * @param options Optional admin options, or NULL for defaults.
7049  * @param rkqu Queue to emit result on.
7050  *
7051  * Supported admin options:
7052  *  - rd_kafka_AdminOptions_set_operation_timeout() - default 60 seconds.
7053  *    Controls how long the brokers will wait for records to be deleted.
7054  *  - rd_kafka_AdminOptions_set_request_timeout() - default socket.timeout.ms.
7055  *    Controls how long \c rdkafka will wait for the request to complete.
7056  *
7057  * @remark The result event type emitted on the supplied queue is of type
7058  *         \c RD_KAFKA_EVENT_DELETERECORDS_RESULT
7059  */
7060 RD_EXPORT void
7061 rd_kafka_DeleteRecords (rd_kafka_t *rk,
7062                         rd_kafka_DeleteRecords_t **del_records,
7063                         size_t del_record_cnt,
7064                         const rd_kafka_AdminOptions_t *options,
7065                         rd_kafka_queue_t *rkqu);
7066 
7067 
7068 /*
7069  * DeleteRecords result type and methods
7070  */
7071 
7072 /**
7073  * @brief Get a list of topic and partition results from a DeleteRecords result.
7074  *        The returned objects will contain \c topic, \c partition, \c offset
7075  *        and \c err. \c offset will be set to the post-deletion low-watermark
7076  *        (smallest available offset of all live replicas). \c err will be set
7077  *        per-partition if deletion failed.
7078  *
7079  * The returned object's life-time is the same as the \p result object.
7080  */
7081 RD_EXPORT const rd_kafka_topic_partition_list_t *
7082 rd_kafka_DeleteRecords_result_offsets (
7083     const rd_kafka_DeleteRecords_result_t *result);
7084 
7085 /*
7086  * DeleteGroups - delete groups from cluster
7087  *
7088  *
7089  */
7090 
7091 /*! Represents a group to be deleted. */
7092 typedef struct rd_kafka_DeleteGroup_s rd_kafka_DeleteGroup_t;
7093 
7094 /**
7095  * @brief Create a new DeleteGroup object. This object is later passed to
7096  *        rd_kafka_DeleteGroups().
7097  *
7098  * @param group Name of group to delete.
7099  *
7100  * @returns a new allocated DeleteGroup object.
7101  *          Use rd_kafka_DeleteGroup_destroy() to free object when done.
7102  */
7103 RD_EXPORT rd_kafka_DeleteGroup_t *
7104 rd_kafka_DeleteGroup_new (const char *group);
7105 
7106 /**
7107  * @brief Destroy and free a DeleteGroup object previously created with
7108  *        rd_kafka_DeleteGroup_new()
7109  */
7110 RD_EXPORT void
7111 rd_kafka_DeleteGroup_destroy (rd_kafka_DeleteGroup_t *del_group);
7112 
7113 /**
7114  * @brief Helper function to destroy all DeleteGroup objects in
7115  *        the \p del_groups array (of \p del_group_cnt elements).
7116  *        The array itself is not freed.
7117  */
7118 RD_EXPORT void
7119 rd_kafka_DeleteGroup_destroy_array (rd_kafka_DeleteGroup_t **del_groups,
7120                                     size_t del_group_cnt);
7121 
7122 /**
7123  * @brief Delete groups from cluster as specified by the \p del_groups
7124  *        array of size \p del_group_cnt elements.
7125  *
7126  * @param rk Client instance.
7127  * @param del_groups Array of groups to delete.
7128  * @param del_group_cnt Number of elements in \p del_groups array.
7129  * @param options Optional admin options, or NULL for defaults.
7130  * @param rkqu Queue to emit result on.
7131  *
7132  * @remark The result event type emitted on the supplied queue is of type
7133  *         \c RD_KAFKA_EVENT_DELETEGROUPS_RESULT
7134  */
7135 RD_EXPORT
7136 void rd_kafka_DeleteGroups (rd_kafka_t *rk,
7137                             rd_kafka_DeleteGroup_t **del_groups,
7138                             size_t del_group_cnt,
7139                             const rd_kafka_AdminOptions_t *options,
7140                             rd_kafka_queue_t *rkqu);
7141 
7142 
7143 
7144 /*
7145  * DeleteGroups result type and methods
7146  */
7147 
7148 /**
7149  * @brief Get an array of group results from a DeleteGroups result.
7150  *
7151  * The returned groups life-time is the same as the \p result object.
7152  *
7153  * @param result Result to get group results from.
7154  * @param cntp is updated to the number of elements in the array.
7155  */
7156 RD_EXPORT const rd_kafka_group_result_t **
7157 rd_kafka_DeleteGroups_result_groups (
7158         const rd_kafka_DeleteGroups_result_t *result,
7159         size_t *cntp);
7160 
7161 
7162 /*
7163  * DeleteConsumerGroupOffsets - delete groups from cluster
7164  *
7165  *
7166  */
7167 
7168 /*! Represents consumer group committed offsets to be deleted. */
7169 typedef struct rd_kafka_DeleteConsumerGroupOffsets_s
7170 rd_kafka_DeleteConsumerGroupOffsets_t;
7171 
7172 /**
7173  * @brief Create a new DeleteConsumerGroupOffsets object.
7174  *        This object is later passed to rd_kafka_DeleteConsumerGroupOffsets().
7175  *
7176  * @param group Consumer group id.
7177  * @param partitions Partitions to delete committed offsets for.
7178  *                   Only the topic and partition fields are used.
7179  *
7180  * @returns a new allocated DeleteConsumerGroupOffsets object.
7181  *          Use rd_kafka_DeleteConsumerGroupOffsets_destroy() to free
7182  *          object when done.
7183  */
7184 RD_EXPORT rd_kafka_DeleteConsumerGroupOffsets_t *
7185 rd_kafka_DeleteConsumerGroupOffsets_new (const char *group,
7186                                          const rd_kafka_topic_partition_list_t
7187                                          *partitions);
7188 
7189 /**
7190  * @brief Destroy and free a DeleteConsumerGroupOffsets object previously
7191  *        created with rd_kafka_DeleteConsumerGroupOffsets_new()
7192  */
7193 RD_EXPORT void
7194 rd_kafka_DeleteConsumerGroupOffsets_destroy (
7195         rd_kafka_DeleteConsumerGroupOffsets_t *del_grpoffsets);
7196 
7197 /**
7198  * @brief Helper function to destroy all DeleteConsumerGroupOffsets objects in
7199  *        the \p del_grpoffsets array (of \p del_grpoffsets_cnt elements).
7200  *        The array itself is not freed.
7201  */
7202 RD_EXPORT void
7203 rd_kafka_DeleteConsumerGroupOffsets_destroy_array (
7204         rd_kafka_DeleteConsumerGroupOffsets_t **del_grpoffsets,
7205         size_t del_grpoffset_cnt);
7206 
7207 /**
7208  * @brief Delete committed offsets for a set of partitions in a conusmer
7209  *        group. This will succeed at the partition level only if the group
7210  *        is not actively subscribed to the corresponding topic.
7211  *
7212  * @param rk Client instance.
7213  * @param del_grpoffsets Array of group committed offsets to delete.
7214  *                       MUST only be one single element.
7215  * @param del_grpoffsets_cnt Number of elements in \p del_grpoffsets array.
7216  *                           MUST always be 1.
7217  * @param options Optional admin options, or NULL for defaults.
7218  * @param rkqu Queue to emit result on.
7219  *
7220  * @remark The result event type emitted on the supplied queue is of type
7221  *         \c RD_KAFKA_EVENT_DELETECONSUMERGROUPOFFSETS_RESULT
7222  *
7223  * @remark The current implementation only supports one group per invocation.
7224  */
7225 RD_EXPORT
7226 void rd_kafka_DeleteConsumerGroupOffsets (
7227         rd_kafka_t *rk,
7228         rd_kafka_DeleteConsumerGroupOffsets_t **del_grpoffsets,
7229         size_t del_grpoffsets_cnt,
7230         const rd_kafka_AdminOptions_t *options,
7231         rd_kafka_queue_t *rkqu);
7232 
7233 
7234 
7235 /*
7236  * DeleteConsumerGroupOffsets result type and methods
7237  */
7238 
7239 /**
7240  * @brief Get an array of results from a DeleteConsumerGroupOffsets result.
7241  *
7242  * The returned groups life-time is the same as the \p result object.
7243  *
7244  * @param result Result to get group results from.
7245  * @param cntp is updated to the number of elements in the array.
7246  */
7247 RD_EXPORT const rd_kafka_group_result_t **
7248 rd_kafka_DeleteConsumerGroupOffsets_result_groups (
7249         const rd_kafka_DeleteConsumerGroupOffsets_result_t *result,
7250         size_t *cntp);
7251 
7252 
7253 /**@}*/
7254 
7255 
7256 /**
7257  * @name Security APIs
7258  * @{
7259  *
7260  */
7261 
7262 /**
7263  * @brief Set SASL/OAUTHBEARER token and metadata
7264  *
7265  * @param rk Client instance.
7266  * @param token_value the mandatory token value to set, often (but not
7267  *  necessarily) a JWS compact serialization as per
7268  *  https://tools.ietf.org/html/rfc7515#section-3.1.
7269  * @param md_lifetime_ms when the token expires, in terms of the number of
7270  *  milliseconds since the epoch.
7271  * @param md_principal_name the mandatory Kafka principal name associated
7272  *  with the token.
7273  * @param extensions optional SASL extensions key-value array with
7274  *  \p extensions_size elements (number of keys * 2), where [i] is the key and
7275  *  [i+1] is the key's value, to be communicated to the broker
7276  *  as additional key-value pairs during the initial client response as per
7277  *  https://tools.ietf.org/html/rfc7628#section-3.1. The key-value pairs are
7278  *  copied.
7279  * @param extension_size the number of SASL extension keys plus values,
7280  *  which must be a non-negative multiple of 2.
7281  * @param errstr A human readable error string (nul-terminated) is written to
7282  *               this location that must be of at least \p errstr_size bytes.
7283  *               The \p errstr is only written in case of error.
7284  * @param errstr_size Writable size in \p errstr.
7285  *
7286  * The SASL/OAUTHBEARER token refresh callback or event handler should invoke
7287  * this method upon success. The extension keys must not include the reserved
7288  * key "`auth`", and all extension keys and values must conform to the required
7289  * format as per https://tools.ietf.org/html/rfc7628#section-3.1:
7290  *
7291  *     key            = 1*(ALPHA)
7292  *     value          = *(VCHAR / SP / HTAB / CR / LF )
7293  *
7294  * @returns \c RD_KAFKA_RESP_ERR_NO_ERROR on success, otherwise \p errstr set
7295  *              and:<br>
7296  *          \c RD_KAFKA_RESP_ERR__INVALID_ARG if any of the arguments are
7297  *              invalid;<br>
7298  *          \c RD_KAFKA_RESP_ERR__NOT_IMPLEMENTED if SASL/OAUTHBEARER is not
7299  *              supported by this build;<br>
7300  *          \c RD_KAFKA_RESP_ERR__STATE if SASL/OAUTHBEARER is supported but is
7301  *              not configured as the client's authentication mechanism.<br>
7302  *
7303  * @sa rd_kafka_oauthbearer_set_token_failure
7304  * @sa rd_kafka_conf_set_oauthbearer_token_refresh_cb
7305  */
7306 RD_EXPORT
7307 rd_kafka_resp_err_t
7308 rd_kafka_oauthbearer_set_token (rd_kafka_t *rk,
7309                                 const char *token_value,
7310                                 int64_t md_lifetime_ms,
7311                                 const char *md_principal_name,
7312                                 const char **extensions, size_t extension_size,
7313                                 char *errstr, size_t errstr_size);
7314 
7315 /**
7316  * @brief SASL/OAUTHBEARER token refresh failure indicator.
7317  *
7318  * @param rk Client instance.
7319  * @param errstr mandatory human readable error reason for failing to acquire
7320  *  a token.
7321  *
7322  * The SASL/OAUTHBEARER token refresh callback or event handler should invoke
7323  * this method upon failure.
7324  *
7325  * @returns \c RD_KAFKA_RESP_ERR_NO_ERROR on success, otherwise:<br>
7326  *          \c RD_KAFKA_RESP_ERR__NOT_IMPLEMENTED if SASL/OAUTHBEARER is not
7327  *              supported by this build;<br>
7328  *          \c RD_KAFKA_RESP_ERR__STATE if SASL/OAUTHBEARER is supported but is
7329  *              not configured as the client's authentication mechanism,<br>
7330  *          \c RD_KAFKA_RESP_ERR__INVALID_ARG if no error string is supplied.
7331  *
7332  * @sa rd_kafka_oauthbearer_set_token
7333  * @sa rd_kafka_conf_set_oauthbearer_token_refresh_cb
7334  */
7335 RD_EXPORT
7336 rd_kafka_resp_err_t
7337 rd_kafka_oauthbearer_set_token_failure (rd_kafka_t *rk, const char *errstr);
7338 
7339 /**@}*/
7340 
7341 
7342 /**
7343  * @name Transactional producer API
7344  *
7345  * The transactional producer operates on top of the idempotent producer,
7346  * and provides full exactly-once semantics (EOS) for Apache Kafka when used
7347  * with the transaction aware consumer (\c isolation.level=read_committed).
7348  *
7349  * A producer instance is configured for transactions by setting the
7350  * \c transactional.id to an identifier unique for the application. This
7351  * id will be used to fence stale transactions from previous instances of
7352  * the application, typically following an outage or crash.
7353  *
7354  * After creating the transactional producer instance using rd_kafka_new()
7355  * the transactional state must be initialized by calling
7356  * rd_kafka_init_transactions(). This is a blocking call that will
7357  * acquire a runtime producer id from the transaction coordinator broker
7358  * as well as abort any stale transactions and fence any still running producer
7359  * instances with the same \c transactional.id.
7360  *
7361  * Once transactions are initialized the application may begin a new
7362  * transaction by calling rd_kafka_begin_transaction().
7363  * A producer instance may only have one single on-going transaction.
7364  *
7365  * Any messages produced after the transaction has been started will
7366  * belong to the ongoing transaction and will be committed or aborted
7367  * atomically.
7368  * It is not permitted to produce messages outside a transaction
7369  * boundary, e.g., before rd_kafka_begin_transaction() or after
7370  * rd_kafka_commit_transaction(), rd_kafka_abort_transaction(), or after
7371  * the current transaction has failed.
7372  *
7373  * If consumed messages are used as input to the transaction, the consumer
7374  * instance must be configured with \c enable.auto.commit set to \c false.
7375  * To commit the consumed offsets along with the transaction pass the
7376  * list of consumed partitions and the last offset processed + 1 to
7377  * rd_kafka_send_offsets_to_transaction() prior to committing the transaction.
7378  * This allows an aborted transaction to be restarted using the previously
7379  * committed offsets.
7380  *
7381  * To commit the produced messages, and any consumed offsets, to the
7382  * current transaction, call rd_kafka_commit_transaction().
7383  * This call will block until the transaction has been fully committed or
7384  * failed (typically due to fencing by a newer producer instance).
7385  *
7386  * Alternatively, if processing fails, or an abortable transaction error is
7387  * raised, the transaction needs to be aborted by calling
7388  * rd_kafka_abort_transaction() which marks any produced messages and
7389  * offset commits as aborted.
7390  *
7391  * After the current transaction has been committed or aborted a new
7392  * transaction may be started by calling rd_kafka_begin_transaction() again.
7393  *
7394  * @par Retriable errors
7395  * Some error cases allow the attempted operation to be retried, this is
7396  * indicated by the error object having the retriable flag set which can
7397  * be detected by calling rd_kafka_error_is_retriable().
7398  * When this flag is set the application may retry the operation immediately
7399  * or preferably after a shorter grace period (to avoid busy-looping).
7400  * Retriable errors include timeouts, broker transport failures, etc.
7401  *
7402  * @par Abortable errors
7403  * An ongoing transaction may fail permanently due to various errors,
7404  * such as transaction coordinator becoming unavailable, write failures to the
7405  * Apache Kafka log, under-replicated partitions, etc.
7406  * At this point the producer application must abort the current transaction
7407  * using rd_kafka_abort_transaction() and optionally start a new transaction
7408  * by calling rd_kafka_begin_transaction().
7409  * Whether an error is abortable or not is detected by calling
7410  * rd_kafka_error_txn_requires_abort() on the returned error object.
7411  *
7412  * @par Fatal errors
7413  * While the underlying idempotent producer will typically only raise
7414  * fatal errors for unrecoverable cluster errors where the idempotency
7415  * guarantees can't be maintained, most of these are treated as abortable by
7416  * the transactional producer since transactions may be aborted and retried
7417  * in their entirety;
7418  * The transactional producer on the other hand introduces a set of additional
7419  * fatal errors which the application needs to handle by shutting down the
7420  * producer and terminate. There is no way for a producer instance to recover
7421  * from fatal errors.
7422  * Whether an error is fatal or not is detected by calling
7423  * rd_kafka_error_is_fatal() on the returned error object or by checking
7424  * the global rd_kafka_fatal_error() code.
7425  * Fatal errors are raised by triggering the \c error_cb (see the
7426  * Fatal error chapter in INTRODUCTION.md for more information), and any
7427  * sub-sequent transactional API calls will return RD_KAFKA_RESP_ERR__FATAL
7428  * or have the fatal flag set (see rd_kafka_error_is_fatal()).
7429  * The originating fatal error code can be retrieved by calling
7430  * rd_kafka_fatal_error().
7431  *
7432  * @par Handling of other errors
7433  * For errors that have neither retriable, abortable or the fatal flag set
7434  * it is not always obvious how to handle them. While some of these errors
7435  * may be indicative of bugs in the application code, such as when
7436  * an invalid parameter is passed to a method, other errors might originate
7437  * from the broker and be passed thru as-is to the application.
7438  * The general recommendation is to treat these errors, that have
7439  * neither the retriable or abortable flags set, as fatal.
7440  *
7441  * @par Error handling example
7442  * @code
7443  *     retry:
7444  *        rd_kafka_error_t *error;
7445  *
7446  *        error = rd_kafka_commit_transaction(producer, 10*1000);
7447  *        if (!error)
7448  *            return success;
7449  *        else if (rd_kafka_error_txn_requires_abort(error)) {
7450  *            do_abort_transaction_and_reset_inputs();
7451  *        } else if (rd_kafka_error_is_retriable(error)) {
7452  *            rd_kafka_error_destroy(error);
7453  *            goto retry;
7454  *        } else { // treat all other errors as fatal errors
7455  *            fatal_error(rd_kafka_error_string(error));
7456  *        }
7457  *        rd_kafka_error_destroy(error);
7458  * @endcode
7459  *
7460  *
7461  * @{
7462  */
7463 
7464 
7465 /**
7466  * @brief Initialize transactions for the producer instance.
7467  *
7468  * This function ensures any transactions initiated by previous instances
7469  * of the producer with the same \c transactional.id are completed.
7470  * If the previous instance failed with a transaction in progress the
7471  * previous transaction will be aborted.
7472  * This function needs to be called before any other transactional or
7473  * produce functions are called when the \c transactional.id is configured.
7474  *
7475  * If the last transaction had begun completion (following transaction commit)
7476  * but not yet finished, this function will await the previous transaction's
7477  * completion.
7478  *
7479  * When any previous transactions have been fenced this function
7480  * will acquire the internal producer id and epoch, used in all future
7481  * transactional messages issued by this producer instance.
7482  *
7483  * @param rk Producer instance.
7484  * @param timeout_ms The maximum time to block. On timeout the operation
7485  *                   may continue in the background, depending on state,
7486  *                   and it is okay to call init_transactions() again.
7487  *
7488  * @remark This function may block up to \p timeout_ms milliseconds.
7489  *
7490  * @returns NULL on success or an error object on failure.
7491  *          Check whether the returned error object permits retrying
7492  *          by calling rd_kafka_error_is_retriable(), or whether a fatal
7493  *          error has been raised by calling rd_kafka_error_is_fatal().
7494  *          Error codes:
7495  *          RD_KAFKA_RESP_ERR__TIMED_OUT if the transaction coordinator
7496  *          could be not be contacted within \p timeout_ms (retriable),
7497  *          RD_KAFKA_RESP_ERR_COORDINATOR_NOT_AVAILABLE if the transaction
7498  *          coordinator is not available (retriable),
7499  *          RD_KAFKA_RESP_ERR_CONCURRENT_TRANSACTIONS if a previous transaction
7500  *          would not complete within \p timeout_ms (retriable),
7501  *          RD_KAFKA_RESP_ERR__STATE if transactions have already been started
7502  *          or upon fatal error,
7503  *          RD_KAFKA_RESP_ERR__UNSUPPORTED_FEATURE if the broker(s) do not
7504  *          support transactions (<Apache Kafka 0.11), this also raises a
7505  *          fatal error,
7506  *          RD_KAFKA_RESP_ERR_INVALID_TRANSACTION_TIMEOUT if the configured
7507  *          \c transaction.timeout.ms is outside the broker-configured range,
7508  *          this also raises a fatal error,
7509  *          RD_KAFKA_RESP_ERR__NOT_CONFIGURED if transactions have not been
7510  *          configured for the producer instance,
7511  *          RD_KAFKA_RESP_ERR__INVALID_ARG if \p rk is not a producer instance,
7512  *          or \p timeout_ms is out of range.
7513  *          Other error codes not listed here may be returned, depending on
7514  *          broker version.
7515  *
7516  * @remark The returned error object (if not NULL) must be destroyed with
7517  *         rd_kafka_error_destroy().
7518  */
7519 RD_EXPORT
7520 rd_kafka_error_t *
7521 rd_kafka_init_transactions (rd_kafka_t *rk, int timeout_ms);
7522 
7523 
7524 
7525 /**
7526  * @brief Begin a new transaction.
7527  *
7528  * rd_kafka_init_transactions() must have been called successfully (once)
7529  * before this function is called.
7530  *
7531  * Upon successful return from this function the application has to perform at
7532  * least one of the following operations within \c transaction.timeout.ms to
7533  * avoid timing out the transaction on the broker:
7534  *   * rd_kafka_produce() (et.al)
7535  *   * rd_kafka_send_offsets_to_transaction()
7536  *   * rd_kafka_commit_transaction()
7537  *   * rd_kafka_abort_transaction()
7538  *
7539  * Any messages produced, offsets sent (rd_kafka_send_offsets_to_transaction()),
7540  * etc, after the successful return of this function will be part of
7541  * the transaction and committed or aborted atomatically.
7542  *
7543  * Finish the transaction by calling rd_kafka_commit_transaction() or
7544  * abort the transaction by calling rd_kafka_abort_transaction().
7545  *
7546  * @param rk Producer instance.
7547  *
7548  * @returns NULL on success or an error object on failure.
7549  *          Check whether a fatal error has been raised by
7550  *          calling rd_kafka_error_is_fatal().
7551  *          Error codes:
7552  *          RD_KAFKA_RESP_ERR__STATE if a transaction is already in progress
7553  *          or upon fatal error,
7554  *          RD_KAFKA_RESP_ERR__NOT_CONFIGURED if transactions have not been
7555  *          configured for the producer instance,
7556  *          RD_KAFKA_RESP_ERR__INVALID_ARG if \p rk is not a producer instance.
7557  *          Other error codes not listed here may be returned, depending on
7558  *          broker version.
7559  *
7560  * @remark With the transactional producer, rd_kafka_produce(),
7561  *         rd_kafka_producev(), et.al, are only allowed during an on-going
7562  *         transaction, as started with this function.
7563  *         Any produce call outside an on-going transaction, or for a failed
7564  *         transaction, will fail.
7565  *
7566  * @remark The returned error object (if not NULL) must be destroyed with
7567  *         rd_kafka_error_destroy().
7568  */
7569 RD_EXPORT
7570 rd_kafka_error_t *rd_kafka_begin_transaction (rd_kafka_t *rk);
7571 
7572 
7573 /**
7574  * @brief Sends a list of topic partition offsets to the consumer group
7575  *        coordinator for \p cgmetadata, and marks the offsets as part
7576  *        part of the current transaction.
7577  *        These offsets will be considered committed only if the transaction is
7578  *        committed successfully.
7579  *
7580  *        The offsets should be the next message your application will consume,
7581  *        i.e., the last processed message's offset + 1 for each partition.
7582  *        Either track the offsets manually during processing or use
7583  *        rd_kafka_position() (on the consumer) to get the current offsets for
7584  *        the partitions assigned to the consumer.
7585  *
7586  *        Use this method at the end of a consume-transform-produce loop prior
7587  *        to committing the transaction with rd_kafka_commit_transaction().
7588  *
7589  * @param rk Producer instance.
7590  * @param offsets List of offsets to commit to the consumer group upon
7591  *                successful commit of the transaction. Offsets should be
7592  *                the next message to consume, e.g., last processed message + 1.
7593  * @param cgmetadata The current consumer group metadata as returned by
7594  *                   rd_kafka_consumer_group_metadata() on the consumer
7595  *                   instance the provided offsets were consumed from.
7596  * @param timeout_ms Maximum time allowed to register the offsets on the broker.
7597  *
7598  * @remark This function must be called on the transactional producer instance,
7599  *         not the consumer.
7600  *
7601  * @remark The consumer must disable auto commits
7602  *         (set \c enable.auto.commit to false on the consumer).
7603  *
7604  * @remark Logical and invalid offsets (such as RD_KAFKA_OFFSET_INVALID) in
7605  *         \p offsets will be ignored, if there are no valid offsets in
7606  *         \p offsets the function will return RD_KAFKA_RESP_ERR_NO_ERROR
7607  *         and no action will be taken.
7608  *
7609  * @returns NULL on success or an error object on failure.
7610  *          Check whether the returned error object permits retrying
7611  *          by calling rd_kafka_error_is_retriable(), or whether an abortable
7612  *          or fatal error has been raised by calling
7613  *          rd_kafka_error_txn_requires_abort() or rd_kafka_error_is_fatal()
7614  *          respectively.
7615  *          Error codes:
7616  *          RD_KAFKA_RESP_ERR__STATE if not currently in a transaction,
7617  *          RD_KAFKA_RESP_ERR_INVALID_PRODUCER_EPOCH if the current producer
7618  *          transaction has been fenced by a newer producer instance,
7619  *          RD_KAFKA_RESP_ERR_TRANSACTIONAL_ID_AUTHORIZATION_FAILED if the
7620  *          producer is no longer authorized to perform transactional
7621  *          operations,
7622  *          RD_KAFKA_RESP_ERR_GROUP_AUTHORIZATION_FAILED if the producer is
7623  *          not authorized to write the consumer offsets to the group
7624  *          coordinator,
7625  *          RD_KAFKA_RESP_ERR__NOT_CONFIGURED if transactions have not been
7626  *          configured for the producer instance,
7627  *          RD_KAFKA_RESP_ERR__INVALID_ARG if \p rk is not a producer instance,
7628  *          or if the \p consumer_group_id or \p offsets are empty,
7629  *          RD_KAFKA_RESP_ERR__PREV_IN_PROGRESS if a previous
7630  *          rd_kafka_send_offsets_to_transaction() call is still in progress.
7631  *          Other error codes not listed here may be returned, depending on
7632  *          broker version.
7633  *
7634  * @remark The returned error object (if not NULL) must be destroyed with
7635  *         rd_kafka_error_destroy().
7636  */
7637 RD_EXPORT
7638 rd_kafka_error_t *
7639 rd_kafka_send_offsets_to_transaction (
7640         rd_kafka_t *rk,
7641         const rd_kafka_topic_partition_list_t *offsets,
7642         const rd_kafka_consumer_group_metadata_t *cgmetadata,
7643         int timeout_ms);
7644 
7645 
7646 /**
7647  * @brief Commit the current transaction (as started with
7648  *        rd_kafka_begin_transaction()).
7649  *
7650  *        Any outstanding messages will be flushed (delivered) before actually
7651  *        committing the transaction.
7652  *
7653  *        If any of the outstanding messages fail permanently the current
7654  *        transaction will enter the abortable error state and this
7655  *        function will return an abortable error, in this case the application
7656  *        must call rd_kafka_abort_transaction() before attempting a new
7657  *        transaction with rd_kafka_begin_transaction().
7658  *
7659  * @param rk Producer instance.
7660  * @param timeout_ms The maximum time to block. On timeout the operation
7661  *                   may continue in the background, depending on state,
7662  *                   and it is okay to call this function again.
7663  *                   Pass -1 to use the remaining transaction timeout,
7664  *                   this is the recommended use.
7665  *
7666  * @remark It is strongly recommended to always pass -1 (remaining transaction
7667  *         time) as the \p timeout_ms. Using other values risk internal
7668  *         state desynchronization in case any of the underlying protocol
7669  *         requests fail.
7670  *
7671  * @remark This function will block until all outstanding messages are
7672  *         delivered and the transaction commit request has been successfully
7673  *         handled by the transaction coordinator, or until \p timeout_ms
7674  *         expires, which ever comes first. On timeout the application may
7675  *         call the function again.
7676  *
7677  * @remark Will automatically call rd_kafka_flush() to ensure all queued
7678  *         messages are delivered before attempting to commit the
7679  *         transaction.
7680  *         If the application has enabled RD_KAFKA_EVENT_DR it must
7681  *         serve the event queue in a separate thread since rd_kafka_flush()
7682  *         will not serve delivery reports in this mode.
7683  *
7684  * @returns NULL on success or an error object on failure.
7685  *          Check whether the returned error object permits retrying
7686  *          by calling rd_kafka_error_is_retriable(), or whether an abortable
7687  *          or fatal error has been raised by calling
7688  *          rd_kafka_error_txn_requires_abort() or rd_kafka_error_is_fatal()
7689  *          respectively.
7690  *          Error codes:
7691  *          RD_KAFKA_RESP_ERR__STATE if not currently in a transaction,
7692  *          RD_KAFKA_RESP_ERR__TIMED_OUT if the transaction could not be
7693  *          complete commmitted within \p timeout_ms, this is a retriable
7694  *          error as the commit continues in the background,
7695  *          RD_KAFKA_RESP_ERR_INVALID_PRODUCER_EPOCH if the current producer
7696  *          transaction has been fenced by a newer producer instance,
7697  *          RD_KAFKA_RESP_ERR_TRANSACTIONAL_ID_AUTHORIZATION_FAILED if the
7698  *          producer is no longer authorized to perform transactional
7699  *          operations,
7700  *          RD_KAFKA_RESP_ERR__NOT_CONFIGURED if transactions have not been
7701  *          configured for the producer instance,
7702  *          RD_KAFKA_RESP_ERR__INVALID_ARG if \p rk is not a producer instance,
7703  *          Other error codes not listed here may be returned, depending on
7704  *          broker version.
7705  *
7706  * @remark The returned error object (if not NULL) must be destroyed with
7707  *         rd_kafka_error_destroy().
7708  */
7709 RD_EXPORT
7710 rd_kafka_error_t *
7711 rd_kafka_commit_transaction (rd_kafka_t *rk, int timeout_ms);
7712 
7713 
7714 /**
7715  * @brief Aborts the ongoing transaction.
7716  *
7717  *        This function should also be used to recover from non-fatal abortable
7718  *        transaction errors.
7719  *
7720  *        Any outstanding messages will be purged and fail with
7721  *        RD_KAFKA_RESP_ERR__PURGE_INFLIGHT or RD_KAFKA_RESP_ERR__PURGE_QUEUE.
7722  *        See rd_kafka_purge() for details.
7723  *
7724  * @param rk Producer instance.
7725  * @param timeout_ms The maximum time to block. On timeout the operation
7726  *                   may continue in the background, depending on state,
7727  *                   and it is okay to call this function again.
7728  *                   Pass -1 to use the remaining transaction timeout,
7729  *                   this is the recommended use.
7730  *
7731  * @remark It is strongly recommended to always pass -1 (remaining transaction
7732  *         time) as the \p timeout_ms. Using other values risk internal
7733  *         state desynchronization in case any of the underlying protocol
7734  *         requests fail.
7735  *
7736  * @remark This function will block until all outstanding messages are purged
7737  *         and the transaction abort request has been successfully
7738  *         handled by the transaction coordinator, or until \p timeout_ms
7739  *         expires, which ever comes first. On timeout the application may
7740  *         call the function again.
7741  *         If the application has enabled RD_KAFKA_EVENT_DR it must
7742  *         serve the event queue in a separate thread since rd_kafka_flush()
7743  *         will not serve delivery reports in this mode.
7744 
7745  *
7746  * @returns NULL on success or an error object on failure.
7747  *          Check whether the returned error object permits retrying
7748  *          by calling rd_kafka_error_is_retriable(), or whether a fatal error
7749  *          has been raised by calling rd_kafka_error_is_fatal().
7750  *          Error codes:
7751  *          RD_KAFKA_RESP_ERR__STATE if not currently in a transaction,
7752  *          RD_KAFKA_RESP_ERR__TIMED_OUT if the transaction could not be
7753  *          complete commmitted within \p timeout_ms, this is a retriable
7754  *          error as the commit continues in the background,
7755  *          RD_KAFKA_RESP_ERR_INVALID_PRODUCER_EPOCH if the current producer
7756  *          transaction has been fenced by a newer producer instance,
7757  *          RD_KAFKA_RESP_ERR_TRANSACTIONAL_ID_AUTHORIZATION_FAILED if the
7758  *          producer is no longer authorized to perform transactional
7759  *          operations,
7760  *          RD_KAFKA_RESP_ERR__NOT_CONFIGURED if transactions have not been
7761  *          configured for the producer instance,
7762  *          RD_KAFKA_RESP_ERR__INVALID_ARG if \p rk is not a producer instance,
7763  *          Other error codes not listed here may be returned, depending on
7764  *          broker version.
7765  *
7766  * @remark The returned error object (if not NULL) must be destroyed with
7767  *         rd_kafka_error_destroy().
7768  */
7769 RD_EXPORT
7770 rd_kafka_error_t *
7771 rd_kafka_abort_transaction (rd_kafka_t *rk, int timeout_ms);
7772 
7773 
7774 /**@}*/
7775 
7776 /* @cond NO_DOC */
7777 #ifdef __cplusplus
7778 }
7779 #endif
7780 #endif /* _RDKAFKA_H_ */
7781 /* @endcond NO_DOC */
7782