librdkafka
The Apache Kafka C/C++ client library
rdkafka.h
Go to the documentation of this file.
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 
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 
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 
161 #define RD_KAFKA_VERSION 0x010700ff
162 
171 RD_EXPORT
173 
179 RD_EXPORT
180 const char *rd_kafka_version_str (void);
181 
200 typedef enum rd_kafka_type_t {
204 
205 
216 
217 
218 
225 RD_EXPORT
226 const char *rd_kafka_get_debug_contexts(void);
227 
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 
268 typedef enum {
269  /* Internal errors to rdkafka: */
395 
398 
399  /* Kafka broker errors: */
433 #define RD_KAFKA_RESP_ERR_GROUP_LOAD_IN_PROGRESS \
434  RD_KAFKA_RESP_ERR_COORDINATOR_LOAD_IN_PROGRESS
438 #define RD_KAFKA_RESP_ERR_GROUP_COORDINATOR_NOT_AVAILABLE \
439  RD_KAFKA_RESP_ERR_COORDINATOR_NOT_AVAILABLE
443 #define RD_KAFKA_RESP_ERR_NOT_COORDINATOR_FOR_GROUP \
444  RD_KAFKA_RESP_ERR_NOT_COORDINATOR
616 
617  RD_KAFKA_RESP_ERR_END_ALL,
619 
620 
628  const char *name;
629  const char *desc;
630 };
631 
632 
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 
648 RD_EXPORT
650 
651 
652 
658 RD_EXPORT
660 
661 
687 RD_EXPORT
689 
690 
715 RD_EXPORT RD_DEPRECATED
717 
718 
731 RD_EXPORT RD_DEPRECATED
732 int rd_kafka_errno (void);
733 
734 
735 
736 
763 RD_EXPORT
765  char *errstr, size_t errstr_size);
766 
767 
785 RD_EXPORT rd_kafka_resp_err_t
787  const char *reason);
788 
789 
794 RD_EXPORT
795 rd_kafka_resp_err_t rd_kafka_error_code (const rd_kafka_error_t *error);
796 
805 RD_EXPORT
806 const char *rd_kafka_error_name (const rd_kafka_error_t *error);
807 
814 RD_EXPORT
815 const char *rd_kafka_error_string (const rd_kafka_error_t *error);
816 
817 
822 RD_EXPORT
823 int rd_kafka_error_is_fatal (const rd_kafka_error_t *error);
824 
825 
830 RD_EXPORT
831 int rd_kafka_error_is_retriable (const rd_kafka_error_t *error);
832 
833 
844 RD_EXPORT
845 int rd_kafka_error_txn_requires_abort (const rd_kafka_error_t *error);
846 
852 RD_EXPORT
853 void rd_kafka_error_destroy (rd_kafka_error_t *error);
854 
855 
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 
885 typedef struct rd_kafka_topic_partition_s {
886  char *topic;
887  int32_t partition;
888  int64_t offset;
889  void *metadata;
890  size_t metadata_size;
891  void *opaque;
893  void *_private;
896 
897 
902 RD_EXPORT
904 
905 
910 typedef struct rd_kafka_topic_partition_list_s {
911  int cnt;
912  int size;
915 
916 
931 RD_EXPORT
933 
934 
938 RD_EXPORT
939 void
941 
951 RD_EXPORT
954  const char *topic, int32_t partition);
955 
956 
965 RD_EXPORT
966 void
968  *rktparlist,
969  const char *topic,
970  int32_t start, int32_t stop);
971 
972 
973 
985 RD_EXPORT
986 int
988  const char *topic, int32_t partition);
989 
990 
998 RD_EXPORT
999 int
1001  rd_kafka_topic_partition_list_t *rktparlist,
1002  int idx);
1003 
1004 
1012 RD_EXPORT
1015 
1016 
1017 
1018 
1026 RD_EXPORT
1028  rd_kafka_topic_partition_list_t *rktparlist,
1029  const char *topic, int32_t partition, int64_t offset);
1030 
1031 
1032 
1038 RD_EXPORT
1041  const rd_kafka_topic_partition_list_t *rktparlist,
1042  const char *topic, int32_t partition);
1043 
1044 
1054 RD_EXPORT void
1056  int (*cmp) (const void *a, const void *b,
1057  void *cmp_opaque),
1058  void *cmp_opaque);
1059 
1060 
1078 typedef enum rd_kafka_vtype_t {
1096 
1097 
1104 typedef struct rd_kafka_vu_s {
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];
1125  } u;
1126 } rd_kafka_vu_t;
1127 
1136 #define RD_KAFKA_V_END RD_KAFKA_VTYPE_END
1137 
1143 #define RD_KAFKA_V_TOPIC(topic) \
1144  _LRK_TYPECHECK(RD_KAFKA_VTYPE_TOPIC, const char *, topic), \
1145  (const char *)topic
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
1159 #define RD_KAFKA_V_PARTITION(partition) \
1160  _LRK_TYPECHECK(RD_KAFKA_VTYPE_PARTITION, int32_t, partition), \
1161  (int32_t)partition
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
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
1185 #define RD_KAFKA_V_OPAQUE(msg_opaque) \
1186  _LRK_TYPECHECK(RD_KAFKA_VTYPE_OPAQUE, void *, msg_opaque), \
1187  (void *)msg_opaque
1194 #define RD_KAFKA_V_MSGFLAGS(msgflags) \
1195  _LRK_TYPECHECK(RD_KAFKA_VTYPE_MSGFLAGS, int, msgflags), \
1196  (int)msgflags
1203 #define RD_KAFKA_V_TIMESTAMP(timestamp) \
1204  _LRK_TYPECHECK(RD_KAFKA_VTYPE_TIMESTAMP, int64_t, timestamp), \
1205  (int64_t)timestamp
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 
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 
1265 RD_EXPORT rd_kafka_headers_t *rd_kafka_headers_new (size_t initial_count);
1266 
1271 RD_EXPORT void rd_kafka_headers_destroy (rd_kafka_headers_t *hdrs);
1272 
1276 RD_EXPORT rd_kafka_headers_t *
1277 rd_kafka_headers_copy (const rd_kafka_headers_t *src);
1278 
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 
1308 RD_EXPORT rd_kafka_resp_err_t
1309 rd_kafka_header_remove (rd_kafka_headers_t *hdrs, const char *name);
1310 
1311 
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 
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 
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 
1377 // FIXME: This doesn't show up in docs for some reason
1378 // "Compound rd_kafka_message_t is not documented."
1379 
1393 typedef struct rd_kafka_message_s {
1395  rd_kafka_topic_t *rkt;
1396  int32_t partition;
1397  void *payload;
1401  size_t len;
1404  void *key;
1406  size_t key_len;
1408  int64_t offset;
1416  void *_private;
1423 
1424 
1428 RD_EXPORT
1430 
1431 
1432 
1433 
1440 RD_EXPORT
1441 const char *rd_kafka_message_errstr (const rd_kafka_message_t *rkmessage);
1442 
1443 
1455 RD_EXPORT
1457  rd_kafka_timestamp_type_t *tstype);
1458 
1459 
1460 
1467 RD_EXPORT
1469 
1470 
1477 RD_EXPORT
1479 
1480 
1497 RD_EXPORT rd_kafka_resp_err_t
1499  rd_kafka_headers_t **hdrsp);
1500 
1512 RD_EXPORT rd_kafka_resp_err_t
1514  rd_kafka_headers_t **hdrsp);
1515 
1516 
1529 RD_EXPORT
1531  rd_kafka_headers_t *hdrs);
1532 
1533 
1539 RD_EXPORT size_t rd_kafka_header_cnt (const rd_kafka_headers_t *hdrs);
1540 
1541 
1547 typedef enum {
1552 
1557 
1563 
1564 
1571 RD_EXPORT rd_kafka_msg_status_t
1573 
1589 typedef enum {
1591  RD_KAFKA_CONF_INVALID = -1,
1594  RD_KAFKA_CONF_OK = 0
1596 
1597 
1631 RD_EXPORT
1632 rd_kafka_conf_t *rd_kafka_conf_new(void);
1633 
1634 
1638 RD_EXPORT
1639 void rd_kafka_conf_destroy(rd_kafka_conf_t *conf);
1640 
1641 
1648 RD_EXPORT
1649 rd_kafka_conf_t *rd_kafka_conf_dup(const rd_kafka_conf_t *conf);
1650 
1651 
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 
1671 RD_EXPORT
1672 const rd_kafka_conf_t *rd_kafka_conf (rd_kafka_t *rk);
1673 
1674 
1694 RD_EXPORT
1696  const char *name,
1697  const char *value,
1698  char *errstr, size_t errstr_size);
1699 
1700 
1706 RD_EXPORT
1707 void rd_kafka_conf_set_events(rd_kafka_conf_t *conf, int events);
1708 
1709 
1745 RD_EXPORT void
1747  void (*event_cb) (rd_kafka_t *rk,
1748  rd_kafka_event_t *rkev,
1749  void *opaque));
1750 
1751 
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 
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 
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 
1917 RD_EXPORT
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 
1944 RD_EXPORT
1946  rd_kafka_conf_t *conf,
1947  void (*offset_commit_cb) (rd_kafka_t *rk,
1948  rd_kafka_resp_err_t err,
1950  void *opaque));
1951 
1952 
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 
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 
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 
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 
2094 RD_EXPORT
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 
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 
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 
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
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 
2233 RD_EXPORT
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 
2253 typedef enum rd_kafka_cert_type_t {
2257  RD_KAFKA_CERT__CNT,
2259 
2267 typedef enum rd_kafka_cert_enc_t {
2271  RD_KAFKA_CERT_ENC__CNT,
2273 
2274 
2306 RD_EXPORT rd_kafka_conf_res_t
2307 rd_kafka_conf_set_ssl_cert (rd_kafka_conf_t *conf,
2308  rd_kafka_cert_type_t cert_type,
2309  rd_kafka_cert_enc_t cert_enc,
2310  const void *buffer, size_t size,
2311  char *errstr, size_t errstr_size);
2312 
2313 
2328 RD_EXPORT
2329 void rd_kafka_conf_set_engine_callback_data (rd_kafka_conf_t *conf,
2330  void *callback_data);
2331 
2332 
2338 RD_EXPORT
2339 void rd_kafka_conf_set_opaque(rd_kafka_conf_t *conf, void *opaque);
2340 
2345 RD_EXPORT
2346 void *rd_kafka_opaque(const rd_kafka_t *rk);
2347 
2348 
2349 
2363 RD_EXPORT
2364 void rd_kafka_conf_set_default_topic_conf (rd_kafka_conf_t *conf,
2365  rd_kafka_topic_conf_t *tconf);
2366 
2379 RD_EXPORT rd_kafka_topic_conf_t *
2381 
2382 
2402 RD_EXPORT
2403 rd_kafka_conf_res_t rd_kafka_conf_get (const rd_kafka_conf_t *conf,
2404  const char *name,
2405  char *dest, size_t *dest_size);
2406 
2407 
2413 RD_EXPORT
2414 rd_kafka_conf_res_t rd_kafka_topic_conf_get (const rd_kafka_topic_conf_t *conf,
2415  const char *name,
2416  char *dest, size_t *dest_size);
2417 
2418 
2427 RD_EXPORT
2428 const char **rd_kafka_conf_dump(rd_kafka_conf_t *conf, size_t *cntp);
2429 
2430 
2439 RD_EXPORT
2440 const char **rd_kafka_topic_conf_dump(rd_kafka_topic_conf_t *conf,
2441  size_t *cntp);
2442 
2447 RD_EXPORT
2448 void rd_kafka_conf_dump_free(const char **arr, size_t cnt);
2449 
2457 RD_EXPORT
2459 
2477 RD_EXPORT
2478 rd_kafka_topic_conf_t *rd_kafka_topic_conf_new(void);
2479 
2480 
2484 RD_EXPORT
2485 rd_kafka_topic_conf_t *rd_kafka_topic_conf_dup(const rd_kafka_topic_conf_t
2486  *conf);
2487 
2492 RD_EXPORT
2493 rd_kafka_topic_conf_t *rd_kafka_default_topic_conf_dup (rd_kafka_t *rk);
2494 
2495 
2499 RD_EXPORT
2500 void rd_kafka_topic_conf_destroy(rd_kafka_topic_conf_t *topic_conf);
2501 
2502 
2511 RD_EXPORT
2512 rd_kafka_conf_res_t rd_kafka_topic_conf_set(rd_kafka_topic_conf_t *conf,
2513  const char *name,
2514  const char *value,
2515  char *errstr, size_t errstr_size);
2516 
2523 RD_EXPORT
2524 void rd_kafka_topic_conf_set_opaque(rd_kafka_topic_conf_t *conf,
2525  void *rkt_opaque);
2526 
2527 
2547 RD_EXPORT
2548 void
2549 rd_kafka_topic_conf_set_partitioner_cb (rd_kafka_topic_conf_t *topic_conf,
2550  int32_t (*partitioner) (
2551  const rd_kafka_topic_t *rkt,
2552  const void *keydata,
2553  size_t keylen,
2554  int32_t partition_cnt,
2555  void *rkt_opaque,
2556  void *msg_opaque));
2557 
2558 
2584 RD_EXPORT void
2585 rd_kafka_topic_conf_set_msg_order_cmp (rd_kafka_topic_conf_t *topic_conf,
2586  int (*msg_order_cmp) (
2587  const rd_kafka_message_t *a,
2588  const rd_kafka_message_t *b));
2589 
2590 
2598 RD_EXPORT
2599 int rd_kafka_topic_partition_available(const rd_kafka_topic_t *rkt,
2600  int32_t partition);
2601 
2602 
2603 /*******************************************************************
2604  * *
2605  * Partitioners provided by rdkafka *
2606  * *
2607  *******************************************************************/
2608 
2622 RD_EXPORT
2623 int32_t rd_kafka_msg_partitioner_random(const rd_kafka_topic_t *rkt,
2624  const void *key, size_t keylen,
2625  int32_t partition_cnt,
2626  void *rkt_opaque, void *msg_opaque);
2627 
2641 RD_EXPORT
2642 int32_t rd_kafka_msg_partitioner_consistent (const rd_kafka_topic_t *rkt,
2643  const void *key, size_t keylen,
2644  int32_t partition_cnt,
2645  void *rkt_opaque, void *msg_opaque);
2646 
2662 RD_EXPORT
2663 int32_t rd_kafka_msg_partitioner_consistent_random (const rd_kafka_topic_t *rkt,
2664  const void *key, size_t keylen,
2665  int32_t partition_cnt,
2666  void *rkt_opaque, void *msg_opaque);
2667 
2668 
2682 RD_EXPORT
2683 int32_t rd_kafka_msg_partitioner_murmur2 (const rd_kafka_topic_t *rkt,
2684  const void *key, size_t keylen,
2685  int32_t partition_cnt,
2686  void *rkt_opaque,
2687  void *msg_opaque);
2688 
2703 RD_EXPORT
2704 int32_t rd_kafka_msg_partitioner_murmur2_random (const rd_kafka_topic_t *rkt,
2705  const void *key, size_t keylen,
2706  int32_t partition_cnt,
2707  void *rkt_opaque,
2708  void *msg_opaque);
2709 
2710 
2724 RD_EXPORT
2725 int32_t rd_kafka_msg_partitioner_fnv1a (const rd_kafka_topic_t *rkt,
2726  const void *key, size_t keylen,
2727  int32_t partition_cnt,
2728  void *rkt_opaque,
2729  void *msg_opaque);
2730 
2731 
2746 RD_EXPORT
2747 int32_t rd_kafka_msg_partitioner_fnv1a_random (const rd_kafka_topic_t *rkt,
2748  const void *key, size_t keylen,
2749  int32_t partition_cnt,
2750  void *rkt_opaque,
2751  void *msg_opaque);
2752 
2753 
2794 RD_EXPORT
2795 rd_kafka_t *rd_kafka_new(rd_kafka_type_t type, rd_kafka_conf_t *conf,
2796  char *errstr, size_t errstr_size);
2797 
2798 
2813 RD_EXPORT
2814 void rd_kafka_destroy(rd_kafka_t *rk);
2815 
2816 
2821 RD_EXPORT
2822 void rd_kafka_destroy_flags (rd_kafka_t *rk, int flags);
2823 
2838 #define RD_KAFKA_DESTROY_F_NO_CONSUMER_CLOSE 0x8
2839 
2840 
2841 
2845 RD_EXPORT
2846 const char *rd_kafka_name(const rd_kafka_t *rk);
2847 
2848 
2852 RD_EXPORT
2853 rd_kafka_type_t rd_kafka_type(const rd_kafka_t *rk);
2854 
2855 
2866 RD_EXPORT
2867 char *rd_kafka_memberid (const rd_kafka_t *rk);
2868 
2869 
2870 
2889 RD_EXPORT
2890 char *rd_kafka_clusterid (rd_kafka_t *rk, int timeout_ms);
2891 
2892 
2908 RD_EXPORT
2909 int32_t rd_kafka_controllerid (rd_kafka_t *rk, int timeout_ms);
2910 
2911 
2933 RD_EXPORT
2934 rd_kafka_topic_t *rd_kafka_topic_new(rd_kafka_t *rk, const char *topic,
2935  rd_kafka_topic_conf_t *conf);
2936 
2937 
2938 
2947 RD_EXPORT
2948 void rd_kafka_topic_destroy(rd_kafka_topic_t *rkt);
2949 
2950 
2954 RD_EXPORT
2955 const char *rd_kafka_topic_name(const rd_kafka_topic_t *rkt);
2956 
2957 
2962 RD_EXPORT
2963 void *rd_kafka_topic_opaque (const rd_kafka_topic_t *rkt);
2964 
2965 
2972 #define RD_KAFKA_PARTITION_UA ((int32_t)-1)
2973 
2974 
3001 RD_EXPORT
3002 int rd_kafka_poll(rd_kafka_t *rk, int timeout_ms);
3003 
3004 
3015 RD_EXPORT
3016 void rd_kafka_yield (rd_kafka_t *rk);
3017 
3018 
3019 
3020 
3028 RD_EXPORT rd_kafka_resp_err_t
3030  rd_kafka_topic_partition_list_t *partitions);
3031 
3032 
3033 
3041 RD_EXPORT rd_kafka_resp_err_t
3043  rd_kafka_topic_partition_list_t *partitions);
3044 
3045 
3046 
3047 
3056 RD_EXPORT rd_kafka_resp_err_t
3058  const char *topic, int32_t partition,
3059  int64_t *low, int64_t *high, int timeout_ms);
3060 
3061 
3078 RD_EXPORT rd_kafka_resp_err_t
3080  const char *topic, int32_t partition,
3081  int64_t *low, int64_t *high);
3082 
3083 
3084 
3109 RD_EXPORT rd_kafka_resp_err_t
3112  int timeout_ms);
3113 
3114 
3115 
3129 RD_EXPORT
3130 void *rd_kafka_mem_calloc (rd_kafka_t *rk, size_t num, size_t size);
3131 
3132 
3133 
3147 RD_EXPORT
3148 void *rd_kafka_mem_malloc (rd_kafka_t *rk, size_t size);
3149 
3150 
3151 
3168 RD_EXPORT
3169 void rd_kafka_mem_free (rd_kafka_t *rk, void *ptr);
3170 
3171 
3195 RD_EXPORT
3196 rd_kafka_queue_t *rd_kafka_queue_new(rd_kafka_t *rk);
3197 
3201 RD_EXPORT
3202 void rd_kafka_queue_destroy(rd_kafka_queue_t *rkqu);
3203 
3204 
3211 RD_EXPORT
3212 rd_kafka_queue_t *rd_kafka_queue_get_main (rd_kafka_t *rk);
3213 
3214 
3224 RD_EXPORT
3225 rd_kafka_queue_t *rd_kafka_queue_get_consumer (rd_kafka_t *rk);
3226 
3237 RD_EXPORT
3238 rd_kafka_queue_t *rd_kafka_queue_get_partition (rd_kafka_t *rk,
3239  const char *topic,
3240  int32_t partition);
3241 
3264 RD_EXPORT
3265 rd_kafka_queue_t *rd_kafka_queue_get_background (rd_kafka_t *rk);
3266 
3267 
3278 RD_EXPORT
3279 void rd_kafka_queue_forward (rd_kafka_queue_t *src, rd_kafka_queue_t *dst);
3280 
3298 RD_EXPORT
3300  rd_kafka_queue_t *rkqu);
3301 
3302 
3306 RD_EXPORT
3307 size_t rd_kafka_queue_length (rd_kafka_queue_t *rkqu);
3308 
3309 
3327 RD_EXPORT
3328 void rd_kafka_queue_io_event_enable (rd_kafka_queue_t *rkqu, int fd,
3329  const void *payload, size_t size);
3330 
3347 RD_EXPORT
3348 void rd_kafka_queue_cb_event_enable (rd_kafka_queue_t *rkqu,
3349  void (*event_cb) (rd_kafka_t *rk,
3350  void *qev_opaque),
3351  void *qev_opaque);
3352 
3353 
3361 RD_EXPORT
3362 void rd_kafka_queue_yield (rd_kafka_queue_t *rkqu);
3363 
3364 
3375 #define RD_KAFKA_OFFSET_BEGINNING -2
3377 #define RD_KAFKA_OFFSET_END -1
3379 #define RD_KAFKA_OFFSET_STORED -1000
3381 #define RD_KAFKA_OFFSET_INVALID -1001
3385 #define RD_KAFKA_OFFSET_TAIL_BASE -2000 /* internal: do not use */
3393 #define RD_KAFKA_OFFSET_TAIL(CNT) (RD_KAFKA_OFFSET_TAIL_BASE - (CNT))
3394 
3428 RD_EXPORT
3429 int rd_kafka_consume_start(rd_kafka_topic_t *rkt, int32_t partition,
3430  int64_t offset);
3431 
3446 RD_EXPORT
3447 int rd_kafka_consume_start_queue(rd_kafka_topic_t *rkt, int32_t partition,
3448  int64_t offset, rd_kafka_queue_t *rkqu);
3449 
3463 RD_EXPORT
3464 int rd_kafka_consume_stop(rd_kafka_topic_t *rkt, int32_t partition);
3465 
3466 
3467 
3491 RD_EXPORT
3492 rd_kafka_resp_err_t rd_kafka_seek (rd_kafka_topic_t *rkt,
3493  int32_t partition,
3494  int64_t offset,
3495  int timeout_ms);
3496 
3497 
3498 
3528 RD_EXPORT rd_kafka_error_t *
3530  rd_kafka_topic_partition_list_t *partitions,
3531  int timeout_ms);
3532 
3533 
3558 RD_EXPORT
3559 rd_kafka_message_t *rd_kafka_consume(rd_kafka_topic_t *rkt, int32_t partition,
3560  int timeout_ms);
3561 
3562 
3563 
3589 RD_EXPORT
3590 ssize_t rd_kafka_consume_batch(rd_kafka_topic_t *rkt, int32_t partition,
3591  int timeout_ms,
3592  rd_kafka_message_t **rkmessages,
3593  size_t rkmessages_size);
3594 
3595 
3596 
3630 RD_EXPORT
3631 int rd_kafka_consume_callback(rd_kafka_topic_t *rkt, int32_t partition,
3632  int timeout_ms,
3633  void (*consume_cb) (rd_kafka_message_t
3634  *rkmessage,
3635  void *commit_opaque),
3636  void *commit_opaque);
3637 
3638 
3655 RD_EXPORT
3657  int timeout_ms);
3658 
3664 RD_EXPORT
3665 ssize_t rd_kafka_consume_batch_queue(rd_kafka_queue_t *rkqu,
3666  int timeout_ms,
3667  rd_kafka_message_t **rkmessages,
3668  size_t rkmessages_size);
3669 
3679 RD_EXPORT
3680 int rd_kafka_consume_callback_queue (rd_kafka_queue_t *rkqu,
3681  int timeout_ms,
3682  void (*consume_cb) (rd_kafka_message_t
3683  *rkmessage,
3684  void *commit_opaque),
3685  void *commit_opaque);
3686 
3687 
3714 RD_EXPORT
3716  int32_t partition, int64_t offset);
3717 
3718 
3739 RD_EXPORT rd_kafka_resp_err_t
3740 rd_kafka_offsets_store (rd_kafka_t *rk,
3795 RD_EXPORT rd_kafka_resp_err_t
3796 rd_kafka_subscribe (rd_kafka_t *rk,
3797  const rd_kafka_topic_partition_list_t *topics);
3798 
3799 
3803 RD_EXPORT
3805 
3806 
3816 RD_EXPORT rd_kafka_resp_err_t
3817 rd_kafka_subscription (rd_kafka_t *rk,
3819 
3820 
3821 
3851 RD_EXPORT
3852 rd_kafka_message_t *rd_kafka_consumer_poll (rd_kafka_t *rk, int timeout_ms);
3853 
3871 RD_EXPORT
3873 
3874 
3892 RD_EXPORT rd_kafka_error_t *
3895  *partitions);
3896 
3897 
3915 RD_EXPORT rd_kafka_error_t *
3918  *partitions);
3919 
3920 
3934 RD_EXPORT
3935 const char *rd_kafka_rebalance_protocol (rd_kafka_t *rk);
3936 
3937 
3958 RD_EXPORT rd_kafka_resp_err_t
3959 rd_kafka_assign (rd_kafka_t *rk,
3960  const rd_kafka_topic_partition_list_t *partitions);
3961 
3978 RD_EXPORT rd_kafka_resp_err_t
3979 rd_kafka_assignment (rd_kafka_t *rk,
3980  rd_kafka_topic_partition_list_t **partitions);
3981 
3982 
3999 RD_EXPORT int
4000 rd_kafka_assignment_lost (rd_kafka_t *rk);
4001 
4002 
4023 RD_EXPORT rd_kafka_resp_err_t
4024 rd_kafka_commit (rd_kafka_t *rk, const rd_kafka_topic_partition_list_t *offsets,
4025  int async);
4026 
4027 
4034 RD_EXPORT rd_kafka_resp_err_t
4035 rd_kafka_commit_message (rd_kafka_t *rk, const rd_kafka_message_t *rkmessage,
4036  int async);
4037 
4038 
4061 RD_EXPORT rd_kafka_resp_err_t
4062 rd_kafka_commit_queue (rd_kafka_t *rk,
4063  const rd_kafka_topic_partition_list_t *offsets,
4064  rd_kafka_queue_t *rkqu,
4065  void (*cb) (rd_kafka_t *rk,
4066  rd_kafka_resp_err_t err,
4068  void *commit_opaque),
4069  void *commit_opaque);
4070 
4071 
4089 RD_EXPORT rd_kafka_resp_err_t
4090 rd_kafka_committed (rd_kafka_t *rk,
4091  rd_kafka_topic_partition_list_t *partitions,
4092  int timeout_ms);
4093 
4094 
4095 
4112 RD_EXPORT rd_kafka_resp_err_t
4113 rd_kafka_position (rd_kafka_t *rk,
4114  rd_kafka_topic_partition_list_t *partitions);
4115 
4116 
4117 
4118 
4130 RD_EXPORT rd_kafka_consumer_group_metadata_t *
4132 
4133 
4143 RD_EXPORT rd_kafka_consumer_group_metadata_t *
4145 
4146 
4159 RD_EXPORT rd_kafka_consumer_group_metadata_t *
4161  int32_t generation_id,
4162  const char *member_id,
4163  const char
4164  *group_instance_id);
4165 
4166 
4171 RD_EXPORT void
4172 rd_kafka_consumer_group_metadata_destroy (rd_kafka_consumer_group_metadata_t *);
4173 
4174 
4195 RD_EXPORT rd_kafka_error_t *
4197  const rd_kafka_consumer_group_metadata_t *cgmd,
4198  void **bufferp, size_t *sizep);
4199 
4220 RD_EXPORT rd_kafka_error_t *
4222  rd_kafka_consumer_group_metadata_t **cgmdp,
4223  const void *buffer, size_t size);
4224 
4240 #define RD_KAFKA_MSG_F_FREE 0x1
4241 #define RD_KAFKA_MSG_F_COPY 0x2
4242 #define RD_KAFKA_MSG_F_BLOCK 0x4
4253 #define RD_KAFKA_MSG_F_PARTITION 0x8
4360 RD_EXPORT
4361 int rd_kafka_produce(rd_kafka_topic_t *rkt, int32_t partition,
4362  int msgflags,
4363  void *payload, size_t len,
4364  const void *key, size_t keylen,
4365  void *msg_opaque);
4366 
4367 
4381 RD_EXPORT
4383 
4384 
4396 RD_EXPORT
4397 rd_kafka_error_t *rd_kafka_produceva (rd_kafka_t *rk,
4398  const rd_kafka_vu_t *vus,
4399  size_t cnt);
4400 
4401 
4431 RD_EXPORT
4432 int rd_kafka_produce_batch(rd_kafka_topic_t *rkt, int32_t partition,
4433  int msgflags,
4434  rd_kafka_message_t *rkmessages, int message_cnt);
4435 
4436 
4437 
4438 
4460 RD_EXPORT
4461 rd_kafka_resp_err_t rd_kafka_flush (rd_kafka_t *rk, int timeout_ms);
4462 
4463 
4464 
4495 RD_EXPORT
4496 rd_kafka_resp_err_t rd_kafka_purge (rd_kafka_t *rk, int purge_flags);
4497 
4498 
4506 #define RD_KAFKA_PURGE_F_QUEUE 0x1
4507 
4515 #define RD_KAFKA_PURGE_F_INFLIGHT 0x2
4516 
4517 
4521 #define RD_KAFKA_PURGE_F_NON_BLOCKING 0x4
4522 
4523 
4538 typedef struct rd_kafka_metadata_broker {
4539  int32_t id;
4540  char *host;
4541  int port;
4543 
4547 typedef struct rd_kafka_metadata_partition {
4548  int32_t id;
4550  int32_t leader;
4552  int32_t *replicas;
4553  int isr_cnt;
4554  int32_t *isrs;
4556 
4560 typedef struct rd_kafka_metadata_topic {
4561  char *topic;
4563  struct rd_kafka_metadata_partition *partitions;
4566 
4567 
4571 typedef struct rd_kafka_metadata {
4573  struct rd_kafka_metadata_broker *brokers;
4576  struct rd_kafka_metadata_topic *topics;
4578  int32_t orig_broker_id;
4581 
4582 
4603 RD_EXPORT
4605 rd_kafka_metadata (rd_kafka_t *rk, int all_topics,
4606  rd_kafka_topic_t *only_rkt,
4607  const struct rd_kafka_metadata **metadatap,
4608  int timeout_ms);
4609 
4613 RD_EXPORT
4614 void rd_kafka_metadata_destroy(const struct rd_kafka_metadata *metadata);
4615 
4616 
4637  char *member_id;
4638  char *client_id;
4639  char *client_host;
4646 };
4647 
4652  struct rd_kafka_metadata_broker broker;
4653  char *group;
4655  char *state;
4657  char *protocol;
4660 };
4661 
4670 };
4671 
4672 
4700 RD_EXPORT
4702 rd_kafka_list_groups (rd_kafka_t *rk, const char *group,
4703  const struct rd_kafka_group_list **grplistp,
4704  int timeout_ms);
4705 
4709 RD_EXPORT
4711 
4712 
4756 RD_EXPORT
4757 int rd_kafka_brokers_add(rd_kafka_t *rk, const char *brokerlist);
4758 
4759 
4760 
4761 
4774 RD_EXPORT RD_DEPRECATED
4775 void rd_kafka_set_logger(rd_kafka_t *rk,
4776  void (*func) (const rd_kafka_t *rk, int level,
4777  const char *fac, const char *buf));
4778 
4779 
4789 RD_EXPORT
4790 void rd_kafka_set_log_level(rd_kafka_t *rk, int level);
4791 
4792 
4796 RD_EXPORT
4797 void rd_kafka_log_print(const rd_kafka_t *rk, int level,
4798  const char *fac, const char *buf);
4799 
4800 
4806 RD_EXPORT
4807 void rd_kafka_log_syslog(const rd_kafka_t *rk, int level,
4808  const char *fac, const char *buf);
4809 
4810 
4833 RD_EXPORT
4834 int rd_kafka_outq_len(rd_kafka_t *rk);
4835 
4836 
4837 
4844 RD_EXPORT
4845 void rd_kafka_dump(FILE *fp, rd_kafka_t *rk);
4846 
4847 
4848 
4854 RD_EXPORT
4856 
4857 
4870 
4871 
4880 RD_EXPORT
4881 int rd_kafka_wait_destroyed(int timeout_ms);
4882 
4883 
4889 RD_EXPORT
4891 
4892 
4910 RD_EXPORT
4912 
4913 
4930 #define RD_KAFKA_EVENT_NONE 0x0
4931 #define RD_KAFKA_EVENT_DR 0x1
4932 #define RD_KAFKA_EVENT_FETCH 0x2
4933 #define RD_KAFKA_EVENT_LOG 0x4
4934 #define RD_KAFKA_EVENT_ERROR 0x8
4935 #define RD_KAFKA_EVENT_REBALANCE 0x10
4936 #define RD_KAFKA_EVENT_OFFSET_COMMIT 0x20
4937 #define RD_KAFKA_EVENT_STATS 0x40
4938 #define RD_KAFKA_EVENT_CREATETOPICS_RESULT 100
4939 #define RD_KAFKA_EVENT_DELETETOPICS_RESULT 101
4940 #define RD_KAFKA_EVENT_CREATEPARTITIONS_RESULT 102
4941 #define RD_KAFKA_EVENT_ALTERCONFIGS_RESULT 103
4942 #define RD_KAFKA_EVENT_DESCRIBECONFIGS_RESULT 104
4943 #define RD_KAFKA_EVENT_DELETERECORDS_RESULT 105
4944 #define RD_KAFKA_EVENT_DELETEGROUPS_RESULT 106
4946 #define RD_KAFKA_EVENT_DELETECONSUMERGROUPOFFSETS_RESULT 107
4947 #define RD_KAFKA_EVENT_OAUTHBEARER_TOKEN_REFRESH 0x100
4958 RD_EXPORT
4959 rd_kafka_event_type_t rd_kafka_event_type (const rd_kafka_event_t *rkev);
4960 
4967 RD_EXPORT
4968 const char *rd_kafka_event_name (const rd_kafka_event_t *rkev);
4969 
4970 
4980 RD_EXPORT
4981 void rd_kafka_event_destroy (rd_kafka_event_t *rkev);
4982 
4983 
4999 RD_EXPORT
5000 const rd_kafka_message_t *rd_kafka_event_message_next (rd_kafka_event_t *rkev);
5001 
5002 
5016 RD_EXPORT
5017 size_t rd_kafka_event_message_array (rd_kafka_event_t *rkev,
5018  const rd_kafka_message_t **rkmessages,
5019  size_t size);
5020 
5021 
5029 RD_EXPORT
5030 size_t rd_kafka_event_message_count (rd_kafka_event_t *rkev);
5031 
5032 
5044 RD_EXPORT
5045 const char *rd_kafka_event_config_string (rd_kafka_event_t *rkev);
5046 
5047 
5056 RD_EXPORT
5058 
5059 
5068 RD_EXPORT
5069 const char *rd_kafka_event_error_string (rd_kafka_event_t *rkev);
5070 
5071 
5080 RD_EXPORT
5081 int rd_kafka_event_error_is_fatal (rd_kafka_event_t *rkev);
5082 
5083 
5099 RD_EXPORT
5100 void *rd_kafka_event_opaque (rd_kafka_event_t *rkev);
5101 
5102 
5111 RD_EXPORT
5112 int rd_kafka_event_log (rd_kafka_event_t *rkev,
5113  const char **fac, const char **str, int *level);
5114 
5115 
5127 RD_EXPORT
5128 int rd_kafka_event_debug_contexts (rd_kafka_event_t *rkev,
5129  char *dst, size_t dstsize);
5130 
5131 
5143 RD_EXPORT
5144 const char *rd_kafka_event_stats (rd_kafka_event_t *rkev);
5145 
5146 
5157 rd_kafka_event_topic_partition_list (rd_kafka_event_t *rkev);
5158 
5159 
5169 RD_EXPORT rd_kafka_topic_partition_t *
5170 rd_kafka_event_topic_partition (rd_kafka_event_t *rkev);
5171 
5172 
5174 typedef rd_kafka_event_t rd_kafka_CreateTopics_result_t;
5176 typedef rd_kafka_event_t rd_kafka_DeleteTopics_result_t;
5178 typedef rd_kafka_event_t rd_kafka_CreatePartitions_result_t;
5180 typedef rd_kafka_event_t rd_kafka_AlterConfigs_result_t;
5182 typedef rd_kafka_event_t rd_kafka_DescribeConfigs_result_t;
5184 typedef rd_kafka_event_t rd_kafka_DeleteRecords_result_t;
5186 typedef rd_kafka_event_t rd_kafka_DeleteGroups_result_t;
5189 
5199 RD_EXPORT const rd_kafka_CreateTopics_result_t *
5200 rd_kafka_event_CreateTopics_result (rd_kafka_event_t *rkev);
5201 
5211 RD_EXPORT const rd_kafka_DeleteTopics_result_t *
5212 rd_kafka_event_DeleteTopics_result (rd_kafka_event_t *rkev);
5213 
5223 RD_EXPORT const rd_kafka_CreatePartitions_result_t *
5225 
5235 RD_EXPORT const rd_kafka_AlterConfigs_result_t *
5236 rd_kafka_event_AlterConfigs_result (rd_kafka_event_t *rkev);
5237 
5247 RD_EXPORT const rd_kafka_DescribeConfigs_result_t *
5248 rd_kafka_event_DescribeConfigs_result (rd_kafka_event_t *rkev);
5249 
5257 RD_EXPORT const rd_kafka_DeleteRecords_result_t *
5258 rd_kafka_event_DeleteRecords_result (rd_kafka_event_t *rkev);
5259 
5269 RD_EXPORT const rd_kafka_DeleteGroups_result_t *
5270 rd_kafka_event_DeleteGroups_result (rd_kafka_event_t *rkev);
5271 
5283 
5293 RD_EXPORT
5294 rd_kafka_event_t *rd_kafka_queue_poll (rd_kafka_queue_t *rkqu, int timeout_ms);
5295 
5309 RD_EXPORT
5310 int rd_kafka_queue_poll_callback (rd_kafka_queue_t *rkqu, int timeout_ms);
5311 
5312 
5356 (rd_kafka_plugin_f_conf_init_t) (rd_kafka_conf_t *conf,
5357  void **plug_opaquep,
5358  char *errstr, size_t errstr_size);
5359 
5440 (rd_kafka_interceptor_f_on_conf_set_t) (rd_kafka_conf_t *conf,
5441  const char *name, const char *val,
5442  char *errstr, size_t errstr_size,
5443  void *ic_opaque);
5444 
5445 
5463 (rd_kafka_interceptor_f_on_conf_dup_t) (rd_kafka_conf_t *new_conf,
5464  const rd_kafka_conf_t *old_conf,
5465  size_t filter_cnt,
5466  const char **filter,
5467  void *ic_opaque);
5468 
5469 
5477 (rd_kafka_interceptor_f_on_conf_destroy_t) (void *ic_opaque);
5478 
5479 
5498 (rd_kafka_interceptor_f_on_new_t) (rd_kafka_t *rk, const rd_kafka_conf_t *conf,
5499  void *ic_opaque,
5500  char *errstr, size_t errstr_size);
5501 
5502 
5511 (rd_kafka_interceptor_f_on_destroy_t) (rd_kafka_t *rk, void *ic_opaque);
5512 
5513 
5514 
5515 
5537 (rd_kafka_interceptor_f_on_send_t) (rd_kafka_t *rk,
5538  rd_kafka_message_t *rkmessage,
5539  void *ic_opaque);
5540 
5565  rd_kafka_message_t *rkmessage,
5566  void *ic_opaque);
5567 
5568 
5586 (rd_kafka_interceptor_f_on_consume_t) (rd_kafka_t *rk,
5587  rd_kafka_message_t *rkmessage,
5588  void *ic_opaque);
5589 
5612  rd_kafka_t *rk,
5613  const rd_kafka_topic_partition_list_t *offsets,
5614  rd_kafka_resp_err_t err, void *ic_opaque);
5615 
5616 
5640  rd_kafka_t *rk,
5641  int sockfd,
5642  const char *brokername,
5643  int32_t brokerid,
5644  int16_t ApiKey,
5645  int16_t ApiVersion,
5646  int32_t CorrId,
5647  size_t size,
5648  void *ic_opaque);
5649 
5650 
5678  rd_kafka_t *rk,
5679  int sockfd,
5680  const char *brokername,
5681  int32_t brokerid,
5682  int16_t ApiKey,
5683  int16_t ApiVersion,
5684  int32_t CorrId,
5685  size_t size,
5686  int64_t rtt,
5688  void *ic_opaque);
5689 
5690 
5709  rd_kafka_t *rk,
5710  rd_kafka_thread_type_t thread_type,
5711  const char *thread_name,
5712  void *ic_opaque);
5713 
5714 
5736  rd_kafka_t *rk,
5737  rd_kafka_thread_type_t thread_type,
5738  const char *thread_name,
5739  void *ic_opaque);
5740 
5741 
5742 
5755 RD_EXPORT rd_kafka_resp_err_t
5757  rd_kafka_conf_t *conf, const char *ic_name,
5759  void *ic_opaque);
5760 
5761 
5774 RD_EXPORT rd_kafka_resp_err_t
5776  rd_kafka_conf_t *conf, const char *ic_name,
5778  void *ic_opaque);
5779 
5793 RD_EXPORT rd_kafka_resp_err_t
5795  rd_kafka_conf_t *conf, const char *ic_name,
5797  void *ic_opaque);
5798 
5799 
5821 RD_EXPORT rd_kafka_resp_err_t
5823  rd_kafka_conf_t *conf, const char *ic_name,
5825  void *ic_opaque);
5826 
5827 
5828 
5841 RD_EXPORT rd_kafka_resp_err_t
5843  rd_kafka_t *rk, const char *ic_name,
5845  void *ic_opaque);
5846 
5847 
5860 RD_EXPORT rd_kafka_resp_err_t
5862  rd_kafka_t *rk, const char *ic_name,
5864  void *ic_opaque);
5865 
5878 RD_EXPORT rd_kafka_resp_err_t
5880  rd_kafka_t *rk, const char *ic_name,
5881  rd_kafka_interceptor_f_on_acknowledgement_t *on_acknowledgement,
5882  void *ic_opaque);
5883 
5884 
5897 RD_EXPORT rd_kafka_resp_err_t
5899  rd_kafka_t *rk, const char *ic_name,
5901  void *ic_opaque);
5902 
5903 
5916 RD_EXPORT rd_kafka_resp_err_t
5918  rd_kafka_t *rk, const char *ic_name,
5920  void *ic_opaque);
5921 
5922 
5935 RD_EXPORT rd_kafka_resp_err_t
5937  rd_kafka_t *rk, const char *ic_name,
5939  void *ic_opaque);
5940 
5941 
5954 RD_EXPORT rd_kafka_resp_err_t
5956  rd_kafka_t *rk, const char *ic_name,
5957  rd_kafka_interceptor_f_on_response_received_t *on_response_received,
5958  void *ic_opaque);
5959 
5960 
5973 RD_EXPORT rd_kafka_resp_err_t
5975  rd_kafka_t *rk, const char *ic_name,
5977  void *ic_opaque);
5978 
5979 
5992 RD_EXPORT rd_kafka_resp_err_t
5994  rd_kafka_t *rk, const char *ic_name,
5996  void *ic_opaque);
5997 
5998 
5999 
6020 RD_EXPORT rd_kafka_resp_err_t
6021 rd_kafka_topic_result_error (const rd_kafka_topic_result_t *topicres);
6022 
6029 RD_EXPORT const char *
6030 rd_kafka_topic_result_error_string (const rd_kafka_topic_result_t *topicres);
6031 
6037 RD_EXPORT const char *
6038 rd_kafka_topic_result_name (const rd_kafka_topic_result_t *topicres);
6039 
6049 RD_EXPORT const rd_kafka_error_t *
6050 rd_kafka_group_result_error (const rd_kafka_group_result_t *groupres);
6051 
6057 RD_EXPORT const char *
6058 rd_kafka_group_result_name (const rd_kafka_group_result_t *groupres);
6059 
6065 RD_EXPORT const rd_kafka_topic_partition_list_t *
6066 rd_kafka_group_result_partitions (const rd_kafka_group_result_t *groupres);
6067 
6068 
6115 typedef enum rd_kafka_admin_op_t {
6128 
6141 typedef struct rd_kafka_AdminOptions_s rd_kafka_AdminOptions_t;
6142 
6162 RD_EXPORT rd_kafka_AdminOptions_t *
6164 
6165 
6170 
6171 
6190 RD_EXPORT rd_kafka_resp_err_t
6192  int timeout_ms,
6193  char *errstr, size_t errstr_size);
6194 
6195 
6222 RD_EXPORT rd_kafka_resp_err_t
6224  int timeout_ms,
6225  char *errstr, size_t errstr_size);
6226 
6227 
6246 RD_EXPORT rd_kafka_resp_err_t
6248  int true_or_false,
6249  char *errstr, size_t errstr_size);
6250 
6251 
6277 RD_EXPORT rd_kafka_resp_err_t
6279  int32_t broker_id,
6280  char *errstr, size_t errstr_size);
6281 
6282 
6283 
6288 RD_EXPORT void
6290  void *ev_opaque);
6291 
6292 
6293 
6294 
6295 
6296 
6297 /*
6298  * CreateTopics - create topics in cluster.
6299  *
6300  */
6301 
6302 
6304 typedef struct rd_kafka_NewTopic_s rd_kafka_NewTopic_t;
6305 
6327 RD_EXPORT rd_kafka_NewTopic_t *
6328 rd_kafka_NewTopic_new (const char *topic, int num_partitions,
6329  int replication_factor,
6330  char *errstr, size_t errstr_size);
6331 
6336 RD_EXPORT void
6338 
6339 
6345 RD_EXPORT void
6347  size_t new_topic_cnt);
6348 
6349 
6371 RD_EXPORT rd_kafka_resp_err_t
6373  int32_t partition,
6374  int32_t *broker_ids,
6375  size_t broker_id_cnt,
6376  char *errstr, size_t errstr_size);
6377 
6390 RD_EXPORT rd_kafka_resp_err_t
6392  const char *name, const char *value);
6393 
6394 
6413 RD_EXPORT void
6414 rd_kafka_CreateTopics (rd_kafka_t *rk,
6415  rd_kafka_NewTopic_t **new_topics,
6416  size_t new_topic_cnt,
6417  const rd_kafka_AdminOptions_t *options,
6418  rd_kafka_queue_t *rkqu);
6419 
6420 
6421 /*
6422  * CreateTopics result type and methods
6423  */
6424 
6433 RD_EXPORT const rd_kafka_topic_result_t **
6435  const rd_kafka_CreateTopics_result_t *result,
6436  size_t *cntp);
6437 
6438 
6439 
6440 
6441 
6442 /*
6443  * DeleteTopics - delete topics from cluster
6444  *
6445  */
6446 
6448 typedef struct rd_kafka_DeleteTopic_s rd_kafka_DeleteTopic_t;
6449 
6459 RD_EXPORT rd_kafka_DeleteTopic_t *
6460 rd_kafka_DeleteTopic_new (const char *topic);
6461 
6466 RD_EXPORT void
6468 
6474 RD_EXPORT void
6476  size_t del_topic_cnt);
6477 
6491 RD_EXPORT
6492 void rd_kafka_DeleteTopics (rd_kafka_t *rk,
6493  rd_kafka_DeleteTopic_t **del_topics,
6494  size_t del_topic_cnt,
6495  const rd_kafka_AdminOptions_t *options,
6496  rd_kafka_queue_t *rkqu);
6497 
6498 
6499 
6500 /*
6501  * DeleteTopics result type and methods
6502  */
6503 
6512 RD_EXPORT const rd_kafka_topic_result_t **
6514  const rd_kafka_DeleteTopics_result_t *result,
6515  size_t *cntp);
6516 
6517 
6518 
6519 
6520 
6521 
6522 /*
6523  * CreatePartitions - add partitions to topic.
6524  *
6525  */
6526 
6528 typedef struct rd_kafka_NewPartitions_s rd_kafka_NewPartitions_t;
6529 
6546 RD_EXPORT rd_kafka_NewPartitions_t *
6547 rd_kafka_NewPartitions_new (const char *topic, size_t new_total_cnt,
6548  char *errstr, size_t errstr_size);
6549 
6554 RD_EXPORT void
6556 
6562 RD_EXPORT void
6564  size_t new_parts_cnt);
6565 
6588 RD_EXPORT rd_kafka_resp_err_t
6590  int32_t new_partition_idx,
6591  int32_t *broker_ids,
6592  size_t broker_id_cnt,
6593  char *errstr,
6594  size_t errstr_size);
6595 
6596 
6615 RD_EXPORT void
6617  rd_kafka_NewPartitions_t **new_parts,
6618  size_t new_parts_cnt,
6619  const rd_kafka_AdminOptions_t *options,
6620  rd_kafka_queue_t *rkqu);
6621 
6622 
6623 
6624 /*
6625  * CreatePartitions result type and methods
6626  */
6627 
6636 RD_EXPORT const rd_kafka_topic_result_t **
6638  const rd_kafka_CreatePartitions_result_t *result,
6639  size_t *cntp);
6640 
6641 
6642 
6643 
6644 
6645 /*
6646  * Cluster, broker, topic configuration entries, sources, etc.
6647  *
6648  */
6649 
6676 
6680 
6681 
6685 RD_EXPORT const char *
6687 
6688 
6690 typedef struct rd_kafka_ConfigEntry_s rd_kafka_ConfigEntry_t;
6691 
6695 RD_EXPORT const char *
6697 
6702 RD_EXPORT const char *
6704 
6708 RD_EXPORT rd_kafka_ConfigSource_t
6710 
6715 RD_EXPORT int
6717 
6723 RD_EXPORT int
6725 
6733 RD_EXPORT int
6735 
6739 RD_EXPORT int
6741 
6742 
6753 RD_EXPORT const rd_kafka_ConfigEntry_t **
6755  size_t *cntp);
6756 
6757 
6758 
6759 
6769 
6773 RD_EXPORT const char *
6775 
6777 typedef struct rd_kafka_ConfigResource_s rd_kafka_ConfigResource_t;
6778 
6779 
6788 RD_EXPORT rd_kafka_ConfigResource_t *
6790  const char *resname);
6791 
6796 RD_EXPORT void
6798 
6799 
6805 RD_EXPORT void
6807  size_t config_cnt);
6808 
6809 
6823 RD_EXPORT rd_kafka_resp_err_t
6825  const char *name, const char *value);
6826 
6827 
6836 RD_EXPORT const rd_kafka_ConfigEntry_t **
6838  size_t *cntp);
6839 
6840 
6841 
6845 RD_EXPORT rd_kafka_ResourceType_t
6847 
6851 RD_EXPORT const char *
6853 
6857 RD_EXPORT rd_kafka_resp_err_t
6859 
6864 RD_EXPORT const char *
6866 
6867 
6868 /*
6869  * AlterConfigs - alter cluster configuration.
6870  *
6871  */
6872 
6873 
6894 RD_EXPORT
6895 void rd_kafka_AlterConfigs (rd_kafka_t *rk,
6896  rd_kafka_ConfigResource_t **configs,
6897  size_t config_cnt,
6898  const rd_kafka_AdminOptions_t *options,
6899  rd_kafka_queue_t *rkqu);
6900 
6901 
6902 /*
6903  * AlterConfigs result type and methods
6904  */
6905 
6920 RD_EXPORT const rd_kafka_ConfigResource_t **
6922  const rd_kafka_AlterConfigs_result_t *result,
6923  size_t *cntp);
6924 
6925 
6926 
6927 
6928 
6929 
6930 /*
6931  * DescribeConfigs - retrieve cluster configuration.
6932  *
6933  */
6934 
6935 
6960 RD_EXPORT
6961 void rd_kafka_DescribeConfigs (rd_kafka_t *rk,
6962  rd_kafka_ConfigResource_t **configs,
6963  size_t config_cnt,
6964  const rd_kafka_AdminOptions_t *options,
6965  rd_kafka_queue_t *rkqu);
6966 
6967 
6968 
6969 
6970 /*
6971  * DescribeConfigs result type and methods
6972  */
6973 
6982 RD_EXPORT const rd_kafka_ConfigResource_t **
6984  const rd_kafka_DescribeConfigs_result_t *result,
6985  size_t *cntp);
6986 
6987 
6988 /*
6989  * DeleteRecords - delete records (messages) from partitions
6990  *
6991  *
6992  */
6993 
6995 typedef struct rd_kafka_DeleteRecords_s rd_kafka_DeleteRecords_t;
6996 
7013 RD_EXPORT rd_kafka_DeleteRecords_t *
7015  before_offsets);
7016 
7021 RD_EXPORT void
7023 
7029 RD_EXPORT void
7031  size_t del_record_cnt);
7032 
7054 RD_EXPORT void
7055 rd_kafka_DeleteRecords (rd_kafka_t *rk,
7056  rd_kafka_DeleteRecords_t **del_records,
7057  size_t del_record_cnt,
7058  const rd_kafka_AdminOptions_t *options,
7059  rd_kafka_queue_t *rkqu);
7060 
7061 
7062 /*
7063  * DeleteRecords result type and methods
7064  */
7065 
7075 RD_EXPORT const rd_kafka_topic_partition_list_t *
7077  const rd_kafka_DeleteRecords_result_t *result);
7078 
7079 /*
7080  * DeleteGroups - delete groups from cluster
7081  *
7082  *
7083  */
7084 
7086 typedef struct rd_kafka_DeleteGroup_s rd_kafka_DeleteGroup_t;
7087 
7097 RD_EXPORT rd_kafka_DeleteGroup_t *
7098 rd_kafka_DeleteGroup_new (const char *group);
7099 
7104 RD_EXPORT void
7106 
7112 RD_EXPORT void
7114  size_t del_group_cnt);
7115 
7129 RD_EXPORT
7130 void rd_kafka_DeleteGroups (rd_kafka_t *rk,
7131  rd_kafka_DeleteGroup_t **del_groups,
7132  size_t del_group_cnt,
7133  const rd_kafka_AdminOptions_t *options,
7134  rd_kafka_queue_t *rkqu);
7135 
7136 
7137 
7138 /*
7139  * DeleteGroups result type and methods
7140  */
7141 
7150 RD_EXPORT const rd_kafka_group_result_t **
7152  const rd_kafka_DeleteGroups_result_t *result,
7153  size_t *cntp);
7154 
7155 
7156 /*
7157  * DeleteConsumerGroupOffsets - delete groups from cluster
7158  *
7159  *
7160  */
7161 
7163 typedef struct rd_kafka_DeleteConsumerGroupOffsets_s
7165 
7181  *partitions);
7182 
7187 RD_EXPORT void
7189  rd_kafka_DeleteConsumerGroupOffsets_t *del_grpoffsets);
7190 
7196 RD_EXPORT void
7198  rd_kafka_DeleteConsumerGroupOffsets_t **del_grpoffsets,
7199  size_t del_grpoffset_cnt);
7200 
7219 RD_EXPORT
7221  rd_kafka_t *rk,
7222  rd_kafka_DeleteConsumerGroupOffsets_t **del_grpoffsets,
7223  size_t del_grpoffsets_cnt,
7224  const rd_kafka_AdminOptions_t *options,
7225  rd_kafka_queue_t *rkqu);
7226 
7227 
7228 
7229 /*
7230  * DeleteConsumerGroupOffsets result type and methods
7231  */
7232 
7241 RD_EXPORT const rd_kafka_group_result_t **
7244  size_t *cntp);
7245 
7246 
7300 RD_EXPORT
7303  const char *token_value,
7304  int64_t md_lifetime_ms,
7305  const char *md_principal_name,
7306  const char **extensions, size_t extension_size,
7307  char *errstr, size_t errstr_size);
7308 
7329 RD_EXPORT
7331 rd_kafka_oauthbearer_set_token_failure (rd_kafka_t *rk, const char *errstr);
7332 
7513 RD_EXPORT
7514 rd_kafka_error_t *
7515 rd_kafka_init_transactions (rd_kafka_t *rk, int timeout_ms);
7516 
7517 
7518 
7563 RD_EXPORT
7564 rd_kafka_error_t *rd_kafka_begin_transaction (rd_kafka_t *rk);
7565 
7566 
7631 RD_EXPORT
7632 rd_kafka_error_t *
7634  rd_kafka_t *rk,
7635  const rd_kafka_topic_partition_list_t *offsets,
7636  const rd_kafka_consumer_group_metadata_t *cgmetadata,
7637  int timeout_ms);
7638 
7639 
7703 RD_EXPORT
7704 rd_kafka_error_t *
7705 rd_kafka_commit_transaction (rd_kafka_t *rk, int timeout_ms);
7706 
7707 
7763 RD_EXPORT
7764 rd_kafka_error_t *
7765 rd_kafka_abort_transaction (rd_kafka_t *rk, int timeout_ms);
7766 
7767 
7770 /* @cond NO_DOC */
7771 #ifdef __cplusplus
7772 }
7773 #endif
7774 #endif /* _RDKAFKA_H_ */
7775 /* @endcond NO_DOC */
RD_EXPORT rd_kafka_resp_err_t rd_kafka_conf_interceptor_add_on_new(rd_kafka_conf_t *conf, const char *ic_name, rd_kafka_interceptor_f_on_new_t *on_new, void *ic_opaque)
Append an on_new() interceptor.
RD_EXPORT int32_t rd_kafka_msg_partitioner_murmur2(const rd_kafka_topic_t *rkt, const void *key, size_t keylen, int32_t partition_cnt, void *rkt_opaque, void *msg_opaque)
Murmur2 partitioner (Java compatible).
RD_EXPORT const rd_kafka_group_result_t ** rd_kafka_DeleteConsumerGroupOffsets_result_groups(const rd_kafka_DeleteConsumerGroupOffsets_result_t *result, size_t *cntp)
Get an array of results from a DeleteConsumerGroupOffsets result.
RD_EXPORT void * rd_kafka_topic_opaque(const rd_kafka_topic_t *rkt)
Get the rkt_opaque pointer that was set in the topic configuration with rd_kafka_topic_conf_set_opaqu...
rd_kafka_resp_err_t
Error codes.
Definition: rdkafka.h:268
@ RD_KAFKA_RESP_ERR_DELEGATION_TOKEN_EXPIRED
Definition: rdkafka.h:549
@ RD_KAFKA_RESP_ERR_POLICY_VIOLATION
Definition: rdkafka.h:500
@ RD_KAFKA_RESP_ERR_TRANSACTION_COORDINATOR_FENCED
Definition: rdkafka.h:521
@ RD_KAFKA_RESP_ERR__AUTO_OFFSET_RESET
Definition: rdkafka.h:394
@ RD_KAFKA_RESP_ERR_RESOURCE_NOT_FOUND
Definition: rdkafka.h:602
@ RD_KAFKA_RESP_ERR__PURGE_QUEUE
Definition: rdkafka.h:370
@ RD_KAFKA_RESP_ERR_UNSUPPORTED_FOR_MESSAGE_FORMAT
Definition: rdkafka.h:498
@ RD_KAFKA_RESP_ERR_OPERATION_NOT_ATTEMPTED
Definition: rdkafka.h:527
@ RD_KAFKA_RESP_ERR_INVALID_TRANSACTION_TIMEOUT
Definition: rdkafka.h:514
@ RD_KAFKA_RESP_ERR_INVALID_PRINCIPAL_TYPE
Definition: rdkafka.h:551
@ RD_KAFKA_RESP_ERR_PREFERRED_LEADER_NOT_AVAILABLE
Definition: rdkafka.h:577
@ RD_KAFKA_RESP_ERR__UNKNOWN_PARTITION
Definition: rdkafka.h:294
@ RD_KAFKA_RESP_ERR__FS
Definition: rdkafka.h:296
@ RD_KAFKA_RESP_ERR__TIMED_OUT
Definition: rdkafka.h:304
@ RD_KAFKA_RESP_ERR_REQUEST_TIMED_OUT
Definition: rdkafka.h:417
@ RD_KAFKA_RESP_ERR_LEADER_NOT_AVAILABLE
Definition: rdkafka.h:413
@ RD_KAFKA_RESP_ERR_INVALID_REQUIRED_ACKS
Definition: rdkafka.h:454
@ RD_KAFKA_RESP_ERR__FATAL
Definition: rdkafka.h:374
@ RD_KAFKA_RESP_ERR__UNDERFLOW
Definition: rdkafka.h:364
@ RD_KAFKA_RESP_ERR__BEGIN
Definition: rdkafka.h:271
@ RD_KAFKA_RESP_ERR__NOENT
Definition: rdkafka.h:362
@ RD_KAFKA_RESP_ERR_UNKNOWN_TOPIC_OR_PART
Definition: rdkafka.h:409
@ RD_KAFKA_RESP_ERR_RECORD_LIST_TOO_LARGE
Definition: rdkafka.h:448
@ RD_KAFKA_RESP_ERR_INVALID_PRODUCER_EPOCH
Definition: rdkafka.h:506
@ RD_KAFKA_RESP_ERR_NETWORK_EXCEPTION
Definition: rdkafka.h:429
@ RD_KAFKA_RESP_ERR_KAFKA_STORAGE_ERROR
Definition: rdkafka.h:529
@ RD_KAFKA_RESP_ERR_GROUP_ID_NOT_FOUND
Definition: rdkafka.h:555
@ RD_KAFKA_RESP_ERR_TOPIC_ALREADY_EXISTS
Definition: rdkafka.h:484
@ RD_KAFKA_RESP_ERR_COORDINATOR_NOT_AVAILABLE
Definition: rdkafka.h:436
@ RD_KAFKA_RESP_ERR_MSG_SIZE_TOO_LARGE
Definition: rdkafka.h:423
@ RD_KAFKA_RESP_ERR_DELEGATION_TOKEN_AUTH_DISABLED
Definition: rdkafka.h:539
@ RD_KAFKA_RESP_ERR_DUPLICATE_RESOURCE
Definition: rdkafka.h:604
@ RD_KAFKA_RESP_ERR_INVALID_CONFIG
Definition: rdkafka.h:492
@ RD_KAFKA_RESP_ERR_FEATURE_UPDATE_FAILED
Definition: rdkafka.h:613
@ RD_KAFKA_RESP_ERR__MAX_POLL_EXCEEDED
Definition: rdkafka.h:380
@ RD_KAFKA_RESP_ERR_INVALID_UPDATE_VERSION
Definition: rdkafka.h:611
@ RD_KAFKA_RESP_ERR__UNKNOWN_GROUP
Definition: rdkafka.h:316
@ RD_KAFKA_RESP_ERR__ISR_INSUFF
Definition: rdkafka.h:308
@ RD_KAFKA_RESP_ERR_SASL_AUTHENTICATION_FAILED
Definition: rdkafka.h:533
@ RD_KAFKA_RESP_ERR_INVALID_PRODUCER_ID_MAPPING
Definition: rdkafka.h:511
@ RD_KAFKA_RESP_ERR__IN_PROGRESS
Definition: rdkafka.h:318
@ RD_KAFKA_RESP_ERR__READ_ONLY
Definition: rdkafka.h:360
@ RD_KAFKA_RESP_ERR_NOT_ENOUGH_REPLICAS_AFTER_APPEND
Definition: rdkafka.h:452
@ RD_KAFKA_RESP_ERR__UNKNOWN_BROKER
Definition: rdkafka.h:382
@ RD_KAFKA_RESP_ERR_INCONSISTENT_GROUP_PROTOCOL
Definition: rdkafka.h:458
@ RD_KAFKA_RESP_ERR_NON_EMPTY_GROUP
Definition: rdkafka.h:553
@ RD_KAFKA_RESP_ERR_SECURITY_DISABLED
Definition: rdkafka.h:525
@ RD_KAFKA_RESP_ERR_TOPIC_DELETION_DISABLED
Definition: rdkafka.h:563
@ RD_KAFKA_RESP_ERR_DELEGATION_TOKEN_OWNER_MISMATCH
Definition: rdkafka.h:543
@ RD_KAFKA_RESP_ERR__INVALID_ARG
Definition: rdkafka.h:302
@ RD_KAFKA_RESP_ERR_UNSUPPORTED_VERSION
Definition: rdkafka.h:482
@ RD_KAFKA_RESP_ERR__OUTDATED
Definition: rdkafka.h:340
@ RD_KAFKA_RESP_ERR_NOT_LEADER_FOR_PARTITION
Definition: rdkafka.h:415
@ RD_KAFKA_RESP_ERR__FAIL
Definition: rdkafka.h:279
@ RD_KAFKA_RESP_ERR_INVALID_MSG_SIZE
Definition: rdkafka.h:411
@ RD_KAFKA_RESP_ERR_STALE_BROKER_EPOCH
Definition: rdkafka.h:571
@ RD_KAFKA_RESP_ERR_COORDINATOR_LOAD_IN_PROGRESS
Definition: rdkafka.h:431
@ RD_KAFKA_RESP_ERR__ASSIGN_PARTITIONS
Definition: rdkafka.h:324
@ RD_KAFKA_RESP_ERR__VALUE_SERIALIZATION
Definition: rdkafka.h:352
@ RD_KAFKA_RESP_ERR_FETCH_SESSION_ID_NOT_FOUND
Definition: rdkafka.h:557
@ RD_KAFKA_RESP_ERR__REVOKE_PARTITIONS
Definition: rdkafka.h:326
@ RD_KAFKA_RESP_ERR__APPLICATION
Definition: rdkafka.h:388
@ RD_KAFKA_RESP_ERR_REPLICA_NOT_AVAILABLE
Definition: rdkafka.h:421
@ RD_KAFKA_RESP_ERR__EXISTING_SUBSCRIPTION
Definition: rdkafka.h:322
@ RD_KAFKA_RESP_ERR_NOT_CONTROLLER
Definition: rdkafka.h:494
@ RD_KAFKA_RESP_ERR_DELEGATION_TOKEN_NOT_FOUND
Definition: rdkafka.h:541
@ RD_KAFKA_RESP_ERR_LISTENER_NOT_FOUND
Definition: rdkafka.h:561
@ RD_KAFKA_RESP_ERR_INVALID_GROUP_ID
Definition: rdkafka.h:460
@ RD_KAFKA_RESP_ERR_UNSTABLE_OFFSET_COMMIT
Definition: rdkafka.h:595
@ RD_KAFKA_RESP_ERR__GAPLESS_GUARANTEE
Definition: rdkafka.h:378
@ RD_KAFKA_RESP_ERR__INVALID_TYPE
Definition: rdkafka.h:366
@ RD_KAFKA_RESP_ERR_OFFSET_NOT_AVAILABLE
Definition: rdkafka.h:573
@ RD_KAFKA_RESP_ERR__PURGE_INFLIGHT
Definition: rdkafka.h:372
@ RD_KAFKA_RESP_ERR__STATE
Definition: rdkafka.h:330
@ RD_KAFKA_RESP_ERR_UNKNOWN_PRODUCER_ID
Definition: rdkafka.h:535
@ RD_KAFKA_RESP_ERR_UNKNOWN_LEADER_EPOCH
Definition: rdkafka.h:567
@ RD_KAFKA_RESP_ERR_UNACCEPTABLE_CREDENTIAL
Definition: rdkafka.h:606
@ RD_KAFKA_RESP_ERR_BROKER_NOT_AVAILABLE
Definition: rdkafka.h:419
@ RD_KAFKA_RESP_ERR_INCONSISTENT_VOTER_SET
Definition: rdkafka.h:609
@ RD_KAFKA_RESP_ERR_ELIGIBLE_LEADERS_NOT_AVAILABLE
Definition: rdkafka.h:584
@ RD_KAFKA_RESP_ERR__RETRY
Definition: rdkafka.h:368
@ RD_KAFKA_RESP_ERR_INVALID_REPLICATION_FACTOR
Definition: rdkafka.h:488
@ RD_KAFKA_RESP_ERR_NOT_COORDINATOR
Definition: rdkafka.h:441
@ RD_KAFKA_RESP_ERR__PREV_IN_PROGRESS
Definition: rdkafka.h:320
@ RD_KAFKA_RESP_ERR_ILLEGAL_GENERATION
Definition: rdkafka.h:456
@ RD_KAFKA_RESP_ERR__BAD_MSG
Definition: rdkafka.h:273
@ RD_KAFKA_RESP_ERR__BAD_COMPRESSION
Definition: rdkafka.h:275
@ RD_KAFKA_RESP_ERR__QUEUE_FULL
Definition: rdkafka.h:306
@ RD_KAFKA_RESP_ERR__WAIT_CACHE
Definition: rdkafka.h:346
@ RD_KAFKA_RESP_ERR__RESOLVE
Definition: rdkafka.h:285
@ RD_KAFKA_RESP_ERR_PRODUCER_FENCED
Definition: rdkafka.h:600
@ RD_KAFKA_RESP_ERR__AUTHENTICATION
Definition: rdkafka.h:336
@ RD_KAFKA_RESP_ERR__NOT_IMPLEMENTED
Definition: rdkafka.h:334
@ RD_KAFKA_RESP_ERR_INVALID_RECORD
Definition: rdkafka.h:593
@ RD_KAFKA_RESP_ERR__UNKNOWN_PROTOCOL
Definition: rdkafka.h:332
@ RD_KAFKA_RESP_ERR__PARTIAL
Definition: rdkafka.h:358
@ RD_KAFKA_RESP_ERR_STALE_CTRL_EPOCH
Definition: rdkafka.h:425
@ RD_KAFKA_RESP_ERR_CLUSTER_AUTHORIZATION_FAILED
Definition: rdkafka.h:474
@ RD_KAFKA_RESP_ERR_DELEGATION_TOKEN_AUTHORIZATION_FAILED
Definition: rdkafka.h:547
@ RD_KAFKA_RESP_ERR_NOT_ENOUGH_REPLICAS
Definition: rdkafka.h:450
@ RD_KAFKA_RESP_ERR_GROUP_MAX_SIZE_REACHED
Definition: rdkafka.h:579
@ RD_KAFKA_RESP_ERR_INVALID_TXN_STATE
Definition: rdkafka.h:508
@ RD_KAFKA_RESP_ERR__DESTROY
Definition: rdkafka.h:277
@ RD_KAFKA_RESP_ERR__INCONSISTENT
Definition: rdkafka.h:376
@ RD_KAFKA_RESP_ERR__KEY_SERIALIZATION
Definition: rdkafka.h:350
@ RD_KAFKA_RESP_ERR_UNKNOWN_MEMBER_ID
Definition: rdkafka.h:462
@ RD_KAFKA_RESP_ERR_MEMBER_ID_REQUIRED
Definition: rdkafka.h:575
@ RD_KAFKA_RESP_ERR_CONCURRENT_TRANSACTIONS
Definition: rdkafka.h:517
@ RD_KAFKA_RESP_ERR__TRANSPORT
Definition: rdkafka.h:281
@ RD_KAFKA_RESP_ERR_INVALID_TIMESTAMP
Definition: rdkafka.h:476
@ RD_KAFKA_RESP_ERR_INVALID_REQUEST
Definition: rdkafka.h:496
@ RD_KAFKA_RESP_ERR_OFFSET_OUT_OF_RANGE
Definition: rdkafka.h:405
@ RD_KAFKA_RESP_ERR_FENCED_INSTANCE_ID
Definition: rdkafka.h:582
@ RD_KAFKA_RESP_ERR_DELEGATION_TOKEN_REQUEST_NOT_ALLOWED
Definition: rdkafka.h:545
@ RD_KAFKA_RESP_ERR_UNSUPPORTED_SASL_MECHANISM
Definition: rdkafka.h:478
@ RD_KAFKA_RESP_ERR__CONFLICT
Definition: rdkafka.h:328
@ RD_KAFKA_RESP_ERR_REASSIGNMENT_IN_PROGRESS
Definition: rdkafka.h:537
@ RD_KAFKA_RESP_ERR_INVALID_COMMIT_OFFSET_SIZE
Definition: rdkafka.h:468
@ RD_KAFKA_RESP_ERR__NOOP
Definition: rdkafka.h:392
@ RD_KAFKA_RESP_ERR_THROTTLING_QUOTA_EXCEEDED
Definition: rdkafka.h:597
@ RD_KAFKA_RESP_ERR__TIMED_OUT_QUEUE
Definition: rdkafka.h:342
@ RD_KAFKA_RESP_ERR_DUPLICATE_SEQUENCE_NUMBER
Definition: rdkafka.h:504
@ RD_KAFKA_RESP_ERR_ELECTION_NOT_NEEDED
Definition: rdkafka.h:586
@ RD_KAFKA_RESP_ERR_TOPIC_AUTHORIZATION_FAILED
Definition: rdkafka.h:470
@ RD_KAFKA_RESP_ERR_TRANSACTIONAL_ID_AUTHORIZATION_FAILED
Definition: rdkafka.h:523
@ RD_KAFKA_RESP_ERR_GROUP_SUBSCRIBED_TO_TOPIC
Definition: rdkafka.h:591
@ RD_KAFKA_RESP_ERR__ASSIGNMENT_LOST
Definition: rdkafka.h:390
@ RD_KAFKA_RESP_ERR__MSG_TIMED_OUT
Definition: rdkafka.h:287
@ RD_KAFKA_RESP_ERR_INVALID_SESSION_TIMEOUT
Definition: rdkafka.h:464
@ RD_KAFKA_RESP_ERR__UNKNOWN_TOPIC
Definition: rdkafka.h:298
@ RD_KAFKA_RESP_ERR_PRINCIPAL_DESERIALIZATION_FAILURE
Definition: rdkafka.h:615
@ RD_KAFKA_RESP_ERR_OFFSET_METADATA_TOO_LARGE
Definition: rdkafka.h:427
@ RD_KAFKA_RESP_ERR__NODE_UPDATE
Definition: rdkafka.h:310
@ RD_KAFKA_RESP_ERR_NO_REASSIGNMENT_IN_PROGRESS
Definition: rdkafka.h:588
@ RD_KAFKA_RESP_ERR__CRIT_SYS_RESOURCE
Definition: rdkafka.h:283
@ RD_KAFKA_RESP_ERR_INVALID_FETCH_SESSION_EPOCH
Definition: rdkafka.h:559
@ RD_KAFKA_RESP_ERR__VALUE_DESERIALIZATION
Definition: rdkafka.h:356
@ RD_KAFKA_RESP_ERR__NO_OFFSET
Definition: rdkafka.h:338
@ RD_KAFKA_RESP_ERR_INVALID_MSG
Definition: rdkafka.h:407
@ RD_KAFKA_RESP_ERR_LOG_DIR_NOT_FOUND
Definition: rdkafka.h:531
@ RD_KAFKA_RESP_ERR__ALL_BROKERS_DOWN
Definition: rdkafka.h:300
@ RD_KAFKA_RESP_ERR_INVALID_REPLICA_ASSIGNMENT
Definition: rdkafka.h:490
@ RD_KAFKA_RESP_ERR__FENCED
Definition: rdkafka.h:386
@ RD_KAFKA_RESP_ERR_INVALID_PARTITIONS
Definition: rdkafka.h:486
@ RD_KAFKA_RESP_ERR__NOT_CONFIGURED
Definition: rdkafka.h:384
@ RD_KAFKA_RESP_ERR__UNSUPPORTED_FEATURE
Definition: rdkafka.h:344
@ RD_KAFKA_RESP_ERR_NO_ERROR
Definition: rdkafka.h:403
@ RD_KAFKA_RESP_ERR__END
Definition: rdkafka.h:397
@ RD_KAFKA_RESP_ERR__INTR
Definition: rdkafka.h:348
@ RD_KAFKA_RESP_ERR_ILLEGAL_SASL_STATE
Definition: rdkafka.h:480
@ RD_KAFKA_RESP_ERR_GROUP_AUTHORIZATION_FAILED
Definition: rdkafka.h:472
@ RD_KAFKA_RESP_ERR_UNSUPPORTED_COMPRESSION_TYPE
Definition: rdkafka.h:569
@ RD_KAFKA_RESP_ERR_OUT_OF_ORDER_SEQUENCE_NUMBER
Definition: rdkafka.h:502
@ RD_KAFKA_RESP_ERR__KEY_DESERIALIZATION
Definition: rdkafka.h:354
@ RD_KAFKA_RESP_ERR__SSL
Definition: rdkafka.h:312
@ RD_KAFKA_RESP_ERR_REBALANCE_IN_PROGRESS
Definition: rdkafka.h:466
@ RD_KAFKA_RESP_ERR_UNKNOWN
Definition: rdkafka.h:401
@ RD_KAFKA_RESP_ERR_FENCED_LEADER_EPOCH
Definition: rdkafka.h:565
@ RD_KAFKA_RESP_ERR__WAIT_COORD
Definition: rdkafka.h:314
@ RD_KAFKA_RESP_ERR__PARTITION_EOF
Definition: rdkafka.h:292
@ RD_KAFKA_RESP_ERR_TOPIC_EXCEPTION
Definition: rdkafka.h:446
RD_EXPORT void rd_kafka_conf_set_throttle_cb(rd_kafka_conf_t *conf, void(*throttle_cb)(rd_kafka_t *rk, const char *broker_name, int32_t broker_id, int throttle_time_ms, void *opaque))
Set throttle callback.
RD_EXPORT void rd_kafka_NewTopic_destroy(rd_kafka_NewTopic_t *new_topic)
Destroy and free a NewTopic object previously created with rd_kafka_NewTopic_new()
RD_EXPORT void rd_kafka_conf_set_socket_cb(rd_kafka_conf_t *conf, int(*socket_cb)(int domain, int type, int protocol, void *opaque))
Set socket callback.
RD_EXPORT void rd_kafka_get_err_descs(const struct rd_kafka_err_desc **errdescs, size_t *cntp)
Returns the full list of error codes.
RD_EXPORT rd_kafka_resp_err_t rd_kafka_offsets_store(rd_kafka_t *rk, rd_kafka_topic_partition_list_t *offsets)
Store offsets for next auto-commit for one or more partitions.
RD_EXPORT void rd_kafka_DeleteRecords_destroy_array(rd_kafka_DeleteRecords_t **del_records, size_t del_record_cnt)
Helper function to destroy all DeleteRecords objects in the del_groups array (of del_group_cnt elemen...
RD_EXPORT rd_kafka_resp_err_t rd_kafka_assign(rd_kafka_t *rk, const rd_kafka_topic_partition_list_t *partitions)
Atomic assignment of partitions to consume.
struct rd_kafka_NewTopic_s rd_kafka_NewTopic_t
Definition: rdkafka.h:6304
struct rd_kafka_NewPartitions_s rd_kafka_NewPartitions_t
Definition: rdkafka.h:6528
RD_EXPORT void rd_kafka_conf_set_log_cb(rd_kafka_conf_t *conf, void(*log_cb)(const rd_kafka_t *rk, int level, const char *fac, const char *buf))
Set logger callback.
RD_EXPORT void rd_kafka_conf_set_dr_cb(rd_kafka_conf_t *conf, void(*dr_cb)(rd_kafka_t *rk, void *payload, size_t len, rd_kafka_resp_err_t err, void *opaque, void *msg_opaque))
RD_EXPORT const char * rd_kafka_ConfigResource_error_string(const rd_kafka_ConfigResource_t *config)
RD_EXPORT void rd_kafka_conf_set_engine_callback_data(rd_kafka_conf_t *conf, void *callback_data)
Set callback_data for OpenSSL engine.
RD_EXPORT void rd_kafka_conf_set_opaque(rd_kafka_conf_t *conf, void *opaque)
Sets the application's opaque pointer that will be passed to callbacks.
struct rd_kafka_DeleteRecords_s rd_kafka_DeleteRecords_t
Definition: rdkafka.h:6995
RD_EXPORT int rd_kafka_thread_cnt(void)
Retrieve the current number of threads in use by librdkafka.
RD_EXPORT const rd_kafka_CreateTopics_result_t * rd_kafka_event_CreateTopics_result(rd_kafka_event_t *rkev)
Get CreateTopics result.
RD_EXPORT void rd_kafka_metadata_destroy(const struct rd_kafka_metadata *metadata)
Release metadata memory.
rd_kafka_resp_err_t() rd_kafka_interceptor_f_on_request_sent_t(rd_kafka_t *rk, int sockfd, const char *brokername, int32_t brokerid, int16_t ApiKey, int16_t ApiVersion, int32_t CorrId, size_t size, void *ic_opaque)
on_request_sent() is called when a request has been fully written to a broker TCP connections socket.
Definition: rdkafka.h:5639
RD_EXPORT void rd_kafka_topic_partition_list_destroy(rd_kafka_topic_partition_list_t *rkparlist)
Free all resources used by the list and the list itself.
RD_EXPORT const char * rd_kafka_version_str(void)
Returns the librdkafka version as string.
RD_EXPORT rd_kafka_resp_err_t rd_kafka_interceptor_add_on_commit(rd_kafka_t *rk, const char *ic_name, rd_kafka_interceptor_f_on_commit_t *on_commit, void *ic_opaque)
Append an on_commit() interceptor.
rd_kafka_event_t rd_kafka_DeleteRecords_result_t
Definition: rdkafka.h:5184
RD_EXPORT const rd_kafka_DescribeConfigs_result_t * rd_kafka_event_DescribeConfigs_result(rd_kafka_event_t *rkev)
Get DescribeConfigs result.
RD_EXPORT rd_kafka_resp_err_t rd_kafka_interceptor_add_on_consume(rd_kafka_t *rk, const char *ic_name, rd_kafka_interceptor_f_on_consume_t *on_consume, void *ic_opaque)
Append an on_consume() interceptor.
RD_EXPORT rd_kafka_resp_err_t rd_kafka_subscribe(rd_kafka_t *rk, const rd_kafka_topic_partition_list_t *topics)
Subscribe to topic set using balanced consumer groups.
RD_EXPORT rd_kafka_resp_err_t rd_kafka_topic_partition_list_set_offset(rd_kafka_topic_partition_list_t *rktparlist, const char *topic, int32_t partition, int64_t offset)
Set offset to offset for topic and partition.
RD_EXPORT rd_kafka_event_type_t rd_kafka_event_type(const rd_kafka_event_t *rkev)
RD_EXPORT rd_kafka_resp_err_t rd_kafka_assignment(rd_kafka_t *rk, rd_kafka_topic_partition_list_t **partitions)
Returns the current partition assignment as set by rd_kafka_assign() or rd_kafka_incremental_assign()...
RD_EXPORT void rd_kafka_conf_set_rebalance_cb(rd_kafka_conf_t *conf, void(*rebalance_cb)(rd_kafka_t *rk, rd_kafka_resp_err_t err, rd_kafka_topic_partition_list_t *partitions, void *opaque))
Consumer: Set rebalance callback for use with coordinated consumer group balancing.
RD_EXPORT void rd_kafka_yield(rd_kafka_t *rk)
Cancels the current callback dispatcher (rd_kafka_poll(), rd_kafka_consume_callback(),...
RD_EXPORT rd_kafka_resp_err_t rd_kafka_interceptor_add_on_destroy(rd_kafka_t *rk, const char *ic_name, rd_kafka_interceptor_f_on_destroy_t *on_destroy, void *ic_opaque)
Append an on_destroy() interceptor.
RD_EXPORT rd_kafka_topic_partition_t * rd_kafka_topic_partition_list_find(const rd_kafka_topic_partition_list_t *rktparlist, const char *topic, int32_t partition)
Find element by topic and partition.
RD_EXPORT rd_kafka_topic_partition_t * rd_kafka_topic_partition_list_add(rd_kafka_topic_partition_list_t *rktparlist, const char *topic, int32_t partition)
Add topic+partition to list.
RD_EXPORT void rd_kafka_conf_set_default_topic_conf(rd_kafka_conf_t *conf, rd_kafka_topic_conf_t *tconf)
Sets the default topic configuration to use for automatically subscribed topics (e....
RD_EXPORT int rd_kafka_queue_poll_callback(rd_kafka_queue_t *rkqu, int timeout_ms)
Poll a queue for events served through callbacks for max timeout_ms.
RD_EXPORT rd_kafka_topic_conf_t * rd_kafka_topic_conf_dup(const rd_kafka_topic_conf_t *conf)
Creates a copy/duplicate of topic configuration object conf.
rd_kafka_resp_err_t() rd_kafka_interceptor_f_on_conf_dup_t(rd_kafka_conf_t *new_conf, const rd_kafka_conf_t *old_conf, size_t filter_cnt, const char **filter, void *ic_opaque)
on_conf_dup() is called from rd_kafka_conf_dup() in the order the interceptors were added and is used...
Definition: rdkafka.h:5463
RD_EXPORT rd_kafka_consumer_group_metadata_t * rd_kafka_consumer_group_metadata(rd_kafka_t *rk)
RD_EXPORT rd_kafka_resp_err_t rd_kafka_oauthbearer_set_token(rd_kafka_t *rk, const char *token_value, int64_t md_lifetime_ms, const char *md_principal_name, const char **extensions, size_t extension_size, char *errstr, size_t errstr_size)
Set SASL/OAUTHBEARER token and metadata.
RD_EXPORT size_t rd_kafka_event_message_array(rd_kafka_event_t *rkev, const rd_kafka_message_t **rkmessages, size_t size)
Extacts size message(s) from the event into the pre-allocated array rkmessages.
RD_EXPORT int rd_kafka_ConfigEntry_is_synonym(const rd_kafka_ConfigEntry_t *entry)
RD_EXPORT rd_kafka_topic_conf_t * rd_kafka_topic_conf_new(void)
Create topic configuration object.
RD_EXPORT void rd_kafka_conf_set_offset_commit_cb(rd_kafka_conf_t *conf, void(*offset_commit_cb)(rd_kafka_t *rk, rd_kafka_resp_err_t err, rd_kafka_topic_partition_list_t *offsets, void *opaque))
Consumer: Set offset commit callback for use with consumer groups.
RD_EXPORT void rd_kafka_CreatePartitions(rd_kafka_t *rk, rd_kafka_NewPartitions_t **new_parts, size_t new_parts_cnt, const rd_kafka_AdminOptions_t *options, rd_kafka_queue_t *rkqu)
Create additional partitions for the given topics, as specified by the new_parts array of size new_pa...
rd_kafka_resp_err_t() rd_kafka_interceptor_f_on_new_t(rd_kafka_t *rk, const rd_kafka_conf_t *conf, void *ic_opaque, char *errstr, size_t errstr_size)
on_new() is called from rd_kafka_new() prior toreturning the newly created client instance to the app...
Definition: rdkafka.h:5498
RD_EXPORT rd_kafka_DeleteTopic_t * rd_kafka_DeleteTopic_new(const char *topic)
Create a new DeleteTopic object. This object is later passed to rd_kafka_DeleteTopics().
RD_EXPORT const char * rd_kafka_topic_result_error_string(const rd_kafka_topic_result_t *topicres)
RD_EXPORT const rd_kafka_ConfigResource_t ** rd_kafka_DescribeConfigs_result_resources(const rd_kafka_DescribeConfigs_result_t *result, size_t *cntp)
Get an array of resource results from a DescribeConfigs result.
RD_EXPORT int rd_kafka_event_error_is_fatal(rd_kafka_event_t *rkev)
RD_EXPORT const rd_kafka_group_result_t ** rd_kafka_DeleteGroups_result_groups(const rd_kafka_DeleteGroups_result_t *result, size_t *cntp)
Get an array of group results from a DeleteGroups result.
rd_kafka_event_t rd_kafka_CreatePartitions_result_t
Definition: rdkafka.h:5178
RD_EXPORT void rd_kafka_topic_conf_set_opaque(rd_kafka_topic_conf_t *conf, void *rkt_opaque)
Sets the application's opaque pointer that will be passed to all topic callbacks as the rkt_opaque ar...
RD_EXPORT char * rd_kafka_clusterid(rd_kafka_t *rk, int timeout_ms)
Returns the ClusterId as reported in broker metadata.
RD_EXPORT rd_kafka_error_t * rd_kafka_produceva(rd_kafka_t *rk, const rd_kafka_vu_t *vus, size_t cnt)
Produce and send a single message to broker.
RD_EXPORT void rd_kafka_dump(FILE *fp, rd_kafka_t *rk)
Dumps rdkafka's internal state for handle rk to stream fp.
RD_EXPORT rd_kafka_error_t * rd_kafka_incremental_unassign(rd_kafka_t *rk, const rd_kafka_topic_partition_list_t *partitions)
Incrementally remove partitions from the current assignment.
RD_EXPORT rd_kafka_error_t * rd_kafka_send_offsets_to_transaction(rd_kafka_t *rk, const rd_kafka_topic_partition_list_t *offsets, const rd_kafka_consumer_group_metadata_t *cgmetadata, int timeout_ms)
Sends a list of topic partition offsets to the consumer group coordinator for cgmetadata,...
RD_EXPORT rd_kafka_resp_err_t rd_kafka_pause_partitions(rd_kafka_t *rk, rd_kafka_topic_partition_list_t *partitions)
Pause producing or consumption for the provided list of partitions.
RD_EXPORT rd_kafka_resp_err_t rd_kafka_NewTopic_set_config(rd_kafka_NewTopic_t *new_topic, const char *name, const char *value)
Set (broker-side) topic configuration name/value pair.
RD_EXPORT rd_kafka_queue_t * rd_kafka_queue_new(rd_kafka_t *rk)
Create a new message queue.
RD_EXPORT rd_kafka_queue_t * rd_kafka_queue_get_background(rd_kafka_t *rk)
RD_EXPORT rd_kafka_conf_t * rd_kafka_conf_dup_filter(const rd_kafka_conf_t *conf, size_t filter_cnt, const char **filter)
Same as rd_kafka_conf_dup() but with an array of property name prefixes to filter out (ignore) when c...
RD_EXPORT const rd_kafka_topic_result_t ** rd_kafka_CreatePartitions_result_topics(const rd_kafka_CreatePartitions_result_t *result, size_t *cntp)
Get an array of topic results from a CreatePartitions result.
RD_EXPORT rd_kafka_error_t * rd_kafka_abort_transaction(rd_kafka_t *rk, int timeout_ms)
Aborts the ongoing transaction.
RD_EXPORT rd_kafka_resp_err_t rd_kafka_test_fatal_error(rd_kafka_t *rk, rd_kafka_resp_err_t err, const char *reason)
Trigger a fatal error for testing purposes.
RD_EXPORT rd_kafka_event_t * rd_kafka_queue_poll(rd_kafka_queue_t *rkqu, int timeout_ms)
Poll a queue for an event for max timeout_ms.
RD_EXPORT void rd_kafka_error_destroy(rd_kafka_error_t *error)
Free and destroy an error object.
RD_EXPORT rd_kafka_consumer_group_metadata_t * rd_kafka_consumer_group_metadata_new(const char *group_id)
Create a new consumer group metadata object. This is typically only used for writing tests.
rd_kafka_event_t rd_kafka_DeleteTopics_result_t
Definition: rdkafka.h:5176
RD_EXPORT void rd_kafka_message_destroy(rd_kafka_message_t *rkmessage)
Frees resources for rkmessage and hands ownership back to rdkafka.
RD_EXPORT const rd_kafka_DeleteConsumerGroupOffsets_result_t * rd_kafka_event_DeleteConsumerGroupOffsets_result(rd_kafka_event_t *rkev)
Get DeleteConsumerGroupOffsets result.
RD_EXPORT rd_kafka_resp_err_t rd_kafka_conf_interceptor_add_on_conf_dup(rd_kafka_conf_t *conf, const char *ic_name, rd_kafka_interceptor_f_on_conf_dup_t *on_conf_dup, void *ic_opaque)
Append an on_conf_dup() interceptor.
RD_EXPORT rd_kafka_resp_err_t rd_kafka_consumer_close(rd_kafka_t *rk)
Close down the KafkaConsumer.
RD_EXPORT rd_kafka_resp_err_t rd_kafka_committed(rd_kafka_t *rk, rd_kafka_topic_partition_list_t *partitions, int timeout_ms)
Retrieve committed offsets for topics+partitions.
rd_kafka_resp_err_t() rd_kafka_interceptor_f_on_send_t(rd_kafka_t *rk, rd_kafka_message_t *rkmessage, void *ic_opaque)
on_send() is called from rd_kafka_produce*() (et.al) prior to the partitioner being called.
Definition: rdkafka.h:5537
RD_EXPORT rd_kafka_resp_err_t rd_kafka_header_get_all(const rd_kafka_headers_t *hdrs, size_t idx, const char **namep, const void **valuep, size_t *sizep)
Iterator for all headers.
RD_EXPORT const rd_kafka_message_t * rd_kafka_event_message_next(rd_kafka_event_t *rkev)
RD_EXPORT const char * rd_kafka_error_name(const rd_kafka_error_t *error)
RD_EXPORT rd_kafka_error_t * rd_kafka_incremental_assign(rd_kafka_t *rk, const rd_kafka_topic_partition_list_t *partitions)
Incrementally add partitions to the current assignment.
RD_EXPORT rd_kafka_error_t * rd_kafka_begin_transaction(rd_kafka_t *rk)
Begin a new transaction.
RD_EXPORT rd_kafka_resp_err_t rd_kafka_unsubscribe(rd_kafka_t *rk)
Unsubscribe from the current subscription set.
RD_EXPORT void rd_kafka_log_print(const rd_kafka_t *rk, int level, const char *fac, const char *buf)
Builtin (default) log sink: print to stderr.
RD_EXPORT ssize_t rd_kafka_consume_batch_queue(rd_kafka_queue_t *rkqu, int timeout_ms, rd_kafka_message_t **rkmessages, size_t rkmessages_size)
Consume batch of messages from queue.
RD_EXPORT const char * rd_kafka_topic_name(const rd_kafka_topic_t *rkt)
Returns the topic name.
RD_EXPORT const char * rd_kafka_topic_result_name(const rd_kafka_topic_result_t *topicres)
RD_EXPORT const char ** rd_kafka_topic_conf_dump(rd_kafka_topic_conf_t *conf, size_t *cntp)
Dump the topic configuration properties and values of conf to an array with "key",...
RD_EXPORT void rd_kafka_queue_cb_event_enable(rd_kafka_queue_t *rkqu, void(*event_cb)(rd_kafka_t *rk, void *qev_opaque), void *qev_opaque)
Enable callback event triggering for queue.
RD_EXPORT int64_t rd_kafka_message_timestamp(const rd_kafka_message_t *rkmessage, rd_kafka_timestamp_type_t *tstype)
Returns the message timestamp for a consumed message.
RD_EXPORT rd_kafka_resp_err_t rd_kafka_interceptor_add_on_request_sent(rd_kafka_t *rk, const char *ic_name, rd_kafka_interceptor_f_on_request_sent_t *on_request_sent, void *ic_opaque)
Append an on_request_sent() interceptor.
RD_EXPORT rd_kafka_resp_err_t rd_kafka_fatal_error(rd_kafka_t *rk, char *errstr, size_t errstr_size)
Returns the first fatal error set on this client instance, or RD_KAFKA_RESP_ERR_NO_ERROR if no fatal ...
RD_EXPORT rd_kafka_resp_err_t rd_kafka_query_watermark_offsets(rd_kafka_t *rk, const char *topic, int32_t partition, int64_t *low, int64_t *high, int timeout_ms)
Query broker for low (oldest/beginning) and high (newest/end) offsets for partition.
rd_kafka_event_t rd_kafka_CreateTopics_result_t
Definition: rdkafka.h:5174
RD_EXPORT int rd_kafka_consume_callback_queue(rd_kafka_queue_t *rkqu, int timeout_ms, void(*consume_cb)(rd_kafka_message_t *rkmessage, void *commit_opaque), void *commit_opaque)
Consume multiple messages from queue with callback.
RD_EXPORT void rd_kafka_conf_set_open_cb(rd_kafka_conf_t *conf, int(*open_cb)(const char *pathname, int flags, mode_t mode, void *opaque))
Set open callback.
RD_EXPORT const rd_kafka_topic_result_t ** rd_kafka_CreateTopics_result_topics(const rd_kafka_CreateTopics_result_t *result, size_t *cntp)
Get an array of topic results from a CreateTopics result.
RD_EXPORT void rd_kafka_NewTopic_destroy_array(rd_kafka_NewTopic_t **new_topics, size_t new_topic_cnt)
Helper function to destroy all NewTopic objects in the new_topics array (of new_topic_cnt elements)....
RD_EXPORT const rd_kafka_DeleteTopics_result_t * rd_kafka_event_DeleteTopics_result(rd_kafka_event_t *rkev)
Get DeleteTopics result.
RD_EXPORT void rd_kafka_mem_free(rd_kafka_t *rk, void *ptr)
Free pointer returned by librdkafka.
RD_EXPORT ssize_t rd_kafka_consume_batch(rd_kafka_topic_t *rkt, int32_t partition, int timeout_ms, rd_kafka_message_t **rkmessages, size_t rkmessages_size)
Consume up to rkmessages_size from topic rkt and partition putting a pointer to each message in the a...
RD_EXPORT int rd_kafka_event_log(rd_kafka_event_t *rkev, const char **fac, const char **str, int *level)
Extract log message from the event.
RD_EXPORT int rd_kafka_assignment_lost(rd_kafka_t *rk)
Check whether the consumer considers the current assignment to have been lost involuntarily....
RD_EXPORT void rd_kafka_conf_set_connect_cb(rd_kafka_conf_t *conf, int(*connect_cb)(int sockfd, const struct sockaddr *addr, int addrlen, const char *id, void *opaque))
Set connect callback.
RD_EXPORT rd_kafka_conf_res_t rd_kafka_conf_set_ssl_cert(rd_kafka_conf_t *conf, rd_kafka_cert_type_t cert_type, rd_kafka_cert_enc_t cert_enc, const void *buffer, size_t size, char *errstr, size_t errstr_size)
Set certificate/key cert_type from the cert_enc encoded memory at buffer of size bytes.
RD_EXPORT rd_kafka_conf_res_t rd_kafka_topic_conf_get(const rd_kafka_topic_conf_t *conf, const char *name, char *dest, size_t *dest_size)
Retrieve topic configuration value for property name.
RD_EXPORT int rd_kafka_consume_callback(rd_kafka_topic_t *rkt, int32_t partition, int timeout_ms, void(*consume_cb)(rd_kafka_message_t *rkmessage, void *commit_opaque), void *commit_opaque)
Consumes messages from topic rkt and partition, calling the provided callback for each consumed messs...
RD_EXPORT const char * rd_kafka_rebalance_protocol(rd_kafka_t *rk)
The rebalance protocol currently in use. This will be "NONE" if the consumer has not (yet) joined a g...
RD_EXPORT int32_t rd_kafka_msg_partitioner_random(const rd_kafka_topic_t *rkt, const void *key, size_t keylen, int32_t partition_cnt, void *rkt_opaque, void *msg_opaque)
Random partitioner.
RD_EXPORT const char * rd_kafka_message_errstr(const rd_kafka_message_t *rkmessage)
Returns the error string for an errored rd_kafka_message_t or NULL if there was no error.
RD_EXPORT void rd_kafka_conf_set_stats_cb(rd_kafka_conf_t *conf, int(*stats_cb)(rd_kafka_t *rk, char *json, size_t json_len, void *opaque))
Set statistics callback in provided conf object.
RD_EXPORT rd_kafka_resp_err_t rd_kafka_purge(rd_kafka_t *rk, int purge_flags)
Purge messages currently handled by the producer instance.
RD_EXPORT rd_kafka_resp_err_t rd_kafka_oauthbearer_set_token_failure(rd_kafka_t *rk, const char *errstr)
SASL/OAUTHBEARER token refresh failure indicator.
RD_EXPORT const rd_kafka_topic_partition_list_t * rd_kafka_group_result_partitions(const rd_kafka_group_result_t *groupres)
RD_EXPORT int rd_kafka_event_debug_contexts(rd_kafka_event_t *rkev, char *dst, size_t dstsize)
Extract log debug context from event.
RD_EXPORT rd_kafka_resp_err_t rd_kafka_set_log_queue(rd_kafka_t *rk, rd_kafka_queue_t *rkqu)
Forward librdkafka logs (and debug) to the specified queue for serving with one of the ....
RD_EXPORT void rd_kafka_conf_set_background_event_cb(rd_kafka_conf_t *conf, void(*event_cb)(rd_kafka_t *rk, rd_kafka_event_t *rkev, void *opaque))
Generic event callback to be used with the event API to trigger callbacks for rd_kafka_event_t object...
RD_EXPORT rd_kafka_resp_err_t rd_kafka_header_get(const rd_kafka_headers_t *hdrs, size_t idx, const char *name, const void **valuep, size_t *sizep)
Iterator for headers matching name.
RD_EXPORT void rd_kafka_queue_forward(rd_kafka_queue_t *src, rd_kafka_queue_t *dst)
Forward/re-route queue src to dst. If dst is NULL the forwarding is removed.
RD_EXPORT const rd_kafka_ConfigEntry_t ** rd_kafka_ConfigResource_configs(const rd_kafka_ConfigResource_t *config, size_t *cntp)
Get an array of config entries from a ConfigResource object.
RD_EXPORT void rd_kafka_DeleteConsumerGroupOffsets_destroy_array(rd_kafka_DeleteConsumerGroupOffsets_t **del_grpoffsets, size_t del_grpoffset_cnt)
Helper function to destroy all DeleteConsumerGroupOffsets objects in the del_grpoffsets array (of del...
rd_kafka_cert_type_t
SSL certificate type.
Definition: rdkafka.h:2253
@ RD_KAFKA_CERT_PUBLIC_KEY
Definition: rdkafka.h:2254
@ RD_KAFKA_CERT_PRIVATE_KEY
Definition: rdkafka.h:2255
@ RD_KAFKA_CERT_CA
Definition: rdkafka.h:2256
RD_EXPORT rd_kafka_headers_t * rd_kafka_headers_copy(const rd_kafka_headers_t *src)
Make a copy of headers list src.
RD_EXPORT const char * rd_kafka_event_name(const rd_kafka_event_t *rkev)
RD_EXPORT void * rd_kafka_mem_malloc(rd_kafka_t *rk, size_t size)
Allocate memory using the same allocator librdkafka uses.
RD_EXPORT void rd_kafka_DeleteConsumerGroupOffsets_destroy(rd_kafka_DeleteConsumerGroupOffsets_t *del_grpoffsets)
Destroy and free a DeleteConsumerGroupOffsets object previously created with rd_kafka_DeleteConsumerG...
RD_EXPORT size_t rd_kafka_event_message_count(rd_kafka_event_t *rkev)
RD_EXPORT rd_kafka_topic_conf_t * rd_kafka_conf_get_default_topic_conf(rd_kafka_conf_t *conf)
Gets the default topic configuration as previously set with rd_kafka_conf_set_default_topic_conf() or...
RD_EXPORT void rd_kafka_topic_partition_list_add_range(rd_kafka_topic_partition_list_t *rktparlist, const char *topic, int32_t start, int32_t stop)
Add range of partitions from start to stop inclusive.
RD_EXPORT rd_kafka_error_t * rd_kafka_commit_transaction(rd_kafka_t *rk, int timeout_ms)
Commit the current transaction (as started with rd_kafka_begin_transaction()).
RD_EXPORT rd_kafka_t * rd_kafka_new(rd_kafka_type_t type, rd_kafka_conf_t *conf, char *errstr, size_t errstr_size)
Creates a new Kafka handle and starts its operation according to the specified type (RD_KAFKA_CONSUME...
RD_EXPORT rd_kafka_message_t * rd_kafka_consumer_poll(rd_kafka_t *rk, int timeout_ms)
Poll the consumer for messages or events.
RD_EXPORT void rd_kafka_conf_set_consume_cb(rd_kafka_conf_t *conf, void(*consume_cb)(rd_kafka_message_t *rkmessage, void *opaque))
Consumer: Set consume callback for use with rd_kafka_consumer_poll()
RD_EXPORT rd_kafka_resp_err_t rd_kafka_seek(rd_kafka_topic_t *rkt, int32_t partition, int64_t offset, int timeout_ms)
Seek consumer for topic+partition to offset which is either an absolute or logical offset.
RD_EXPORT rd_kafka_topic_partition_list_t * rd_kafka_topic_partition_list_copy(const rd_kafka_topic_partition_list_t *src)
Make a copy of an existing list.
RD_EXPORT rd_kafka_consumer_group_metadata_t * rd_kafka_consumer_group_metadata_new_with_genid(const char *group_id, int32_t generation_id, const char *member_id, const char *group_instance_id)
Create a new consumer group metadata object. This is typically only used for writing tests.
RD_EXPORT void rd_kafka_DeleteRecords_destroy(rd_kafka_DeleteRecords_t *del_records)
Destroy and free a DeleteRecords object previously created with rd_kafka_DeleteRecords_new()
RD_EXPORT rd_kafka_AdminOptions_t * rd_kafka_AdminOptions_new(rd_kafka_t *rk, rd_kafka_admin_op_t for_api)
Create a new AdminOptions object.
RD_EXPORT void rd_kafka_CreateTopics(rd_kafka_t *rk, rd_kafka_NewTopic_t **new_topics, size_t new_topic_cnt, const rd_kafka_AdminOptions_t *options, rd_kafka_queue_t *rkqu)
Create topics in cluster as specified by the new_topics array of size new_topic_cnt elements.
RD_EXPORT rd_kafka_resp_err_t rd_kafka_ConfigResource_error(const rd_kafka_ConfigResource_t *config)
RD_EXPORT rd_kafka_resp_err_t rd_kafka_NewPartitions_set_replica_assignment(rd_kafka_NewPartitions_t *new_parts, int32_t new_partition_idx, int32_t *broker_ids, size_t broker_id_cnt, char *errstr, size_t errstr_size)
Set the replica (broker id) assignment for new_partition_idx to the replica set in broker_ids (of bro...
RD_EXPORT rd_kafka_resp_err_t rd_kafka_error_code(const rd_kafka_error_t *error)
rd_kafka_conf_res_t() rd_kafka_interceptor_f_on_conf_set_t(rd_kafka_conf_t *conf, const char *name, const char *val, char *errstr, size_t errstr_size, void *ic_opaque)
on_conf_set() is called from rd_kafka_*_conf_set() in the order the interceptors were added.
Definition: rdkafka.h:5440
RD_EXPORT rd_kafka_DeleteGroup_t * rd_kafka_DeleteGroup_new(const char *group)
Create a new DeleteGroup object. This object is later passed to rd_kafka_DeleteGroups().
RD_EXPORT rd_kafka_resp_err_t rd_kafka_header_get_last(const rd_kafka_headers_t *hdrs, const char *name, const void **valuep, size_t *sizep)
Find last header in list hdrs matching name.
RD_EXPORT rd_kafka_resp_err_t rd_kafka_list_groups(rd_kafka_t *rk, const char *group, const struct rd_kafka_group_list **grplistp, int timeout_ms)
List and describe client groups in cluster.
rd_kafka_resp_err_t() rd_kafka_plugin_f_conf_init_t(rd_kafka_conf_t *conf, void **plug_opaquep, char *errstr, size_t errstr_size)
Plugin's configuration initializer method called each time the library is referenced from configurati...
Definition: rdkafka.h:5356
RD_EXPORT rd_kafka_resp_err_t rd_kafka_position(rd_kafka_t *rk, rd_kafka_topic_partition_list_t *partitions)
Retrieve current positions (offsets) for topics+partitions.
RD_EXPORT const char ** rd_kafka_conf_dump(rd_kafka_conf_t *conf, size_t *cntp)
Dump the configuration properties and values of conf to an array with "key", "value" pairs.
RD_EXPORT rd_kafka_resp_err_t rd_kafka_AdminOptions_set_operation_timeout(rd_kafka_AdminOptions_t *options, int timeout_ms, char *errstr, size_t errstr_size)
Sets the broker's operation timeout, such as the timeout for CreateTopics to complete the creation of...
RD_EXPORT const rd_kafka_conf_t * rd_kafka_conf(rd_kafka_t *rk)
RD_EXPORT void rd_kafka_DeleteGroup_destroy(rd_kafka_DeleteGroup_t *del_group)
Destroy and free a DeleteGroup object previously created with rd_kafka_DeleteGroup_new()
RD_EXPORT void * rd_kafka_mem_calloc(rd_kafka_t *rk, size_t num, size_t size)
Allocate and zero memory using the same allocator librdkafka uses.
RD_EXPORT const char * rd_kafka_ConfigEntry_value(const rd_kafka_ConfigEntry_t *entry)
RD_EXPORT void rd_kafka_log_syslog(const rd_kafka_t *rk, int level, const char *fac, const char *buf)
Builtin log sink: print to syslog.
struct rd_kafka_DeleteTopic_s rd_kafka_DeleteTopic_t
Definition: rdkafka.h:6448
RD_EXPORT void rd_kafka_topic_conf_set_msg_order_cmp(rd_kafka_topic_conf_t *topic_conf, int(*msg_order_cmp)(const rd_kafka_message_t *a, const rd_kafka_message_t *b))
Producer: Set message queueing order comparator callback.
RD_EXPORT rd_kafka_error_t * rd_kafka_seek_partitions(rd_kafka_t *rk, rd_kafka_topic_partition_list_t *partitions, int timeout_ms)
Seek consumer for partitions in partitions to the per-partition offset in the .offset field of partit...
RD_EXPORT rd_kafka_resp_err_t rd_kafka_offset_store(rd_kafka_topic_t *rkt, int32_t partition, int64_t offset)
Store offset offset + 1 for topic rkt partition partition.
RD_EXPORT void rd_kafka_DeleteGroup_destroy_array(rd_kafka_DeleteGroup_t **del_groups, size_t del_group_cnt)
Helper function to destroy all DeleteGroup objects in the del_groups array (of del_group_cnt elements...
RD_EXPORT const rd_kafka_ConfigResource_t ** rd_kafka_AlterConfigs_result_resources(const rd_kafka_AlterConfigs_result_t *result, size_t *cntp)
Get an array of resource results from a AlterConfigs result.
RD_EXPORT int rd_kafka_produce_batch(rd_kafka_topic_t *rkt, int32_t partition, int msgflags, rd_kafka_message_t *rkmessages, int message_cnt)
Produce multiple messages.
RD_EXPORT int rd_kafka_ConfigEntry_is_sensitive(const rd_kafka_ConfigEntry_t *entry)
RD_EXPORT const char * rd_kafka_event_config_string(rd_kafka_event_t *rkev)
RD_EXPORT rd_kafka_queue_t * rd_kafka_queue_get_main(rd_kafka_t *rk)
RD_EXPORT const char * rd_kafka_group_result_name(const rd_kafka_group_result_t *groupres)
rd_kafka_admin_op_t
Admin operation enum name for use with rd_kafka_AdminOptions_new()
Definition: rdkafka.h:6115
@ RD_KAFKA_ADMIN_OP_DELETERECORDS
Definition: rdkafka.h:6122
@ RD_KAFKA_ADMIN_OP_CREATETOPICS
Definition: rdkafka.h:6117
@ RD_KAFKA_ADMIN_OP_DELETEGROUPS
Definition: rdkafka.h:6123
@ RD_KAFKA_ADMIN_OP_ANY
Definition: rdkafka.h:6116
@ RD_KAFKA_ADMIN_OP_DELETECONSUMERGROUPOFFSETS
Definition: rdkafka.h:6125
@ RD_KAFKA_ADMIN_OP_DELETETOPICS
Definition: rdkafka.h:6118
@ RD_KAFKA_ADMIN_OP__CNT
Definition: rdkafka.h:6126
@ RD_KAFKA_ADMIN_OP_ALTERCONFIGS
Definition: rdkafka.h:6120
@ RD_KAFKA_ADMIN_OP_CREATEPARTITIONS
Definition: rdkafka.h:6119
@ RD_KAFKA_ADMIN_OP_DESCRIBECONFIGS
Definition: rdkafka.h:6121
RD_EXPORT rd_kafka_resp_err_t rd_kafka_interceptor_add_on_response_received(rd_kafka_t *rk, const char *ic_name, rd_kafka_interceptor_f_on_response_received_t *on_response_received, void *ic_opaque)
Append an on_response_received() interceptor.
RD_EXPORT void rd_kafka_NewPartitions_destroy_array(rd_kafka_NewPartitions_t **new_parts, size_t new_parts_cnt)
Helper function to destroy all NewPartitions objects in the new_parts array (of new_parts_cnt element...
RD_EXPORT int rd_kafka_version(void)
Returns the librdkafka version as integer.
RD_EXPORT int rd_kafka_topic_partition_list_del_by_idx(rd_kafka_topic_partition_list_t *rktparlist, int idx)
Delete partition from list by elems[] index.
RD_EXPORT rd_kafka_resp_err_t rd_kafka_metadata(rd_kafka_t *rk, int all_topics, rd_kafka_topic_t *only_rkt, const struct rd_kafka_metadata **metadatap, int timeout_ms)
Request Metadata from broker.
RD_EXPORT char * rd_kafka_memberid(const rd_kafka_t *rk)
Returns this client's broker-assigned group member id.
RD_EXPORT void * rd_kafka_event_opaque(rd_kafka_event_t *rkev)
RD_EXPORT rd_kafka_resp_err_t rd_kafka_interceptor_add_on_thread_exit(rd_kafka_t *rk, const char *ic_name, rd_kafka_interceptor_f_on_thread_exit_t *on_thread_exit, void *ic_opaque)
Append an on_thread_exit() interceptor.
rd_kafka_resp_err_t() rd_kafka_interceptor_f_on_commit_t(rd_kafka_t *rk, const rd_kafka_topic_partition_list_t *offsets, rd_kafka_resp_err_t err, void *ic_opaque)
on_commit() is called on completed or failed offset commit. It is called from internal librdkafka thr...
Definition: rdkafka.h:5611
RD_EXPORT rd_kafka_resp_err_t rd_kafka_header_remove(rd_kafka_headers_t *hdrs, const char *name)
Remove all headers for the given key (if any).
RD_EXPORT const rd_kafka_ConfigEntry_t ** rd_kafka_ConfigEntry_synonyms(const rd_kafka_ConfigEntry_t *entry, size_t *cntp)
RD_EXPORT RD_DEPRECATED int rd_kafka_errno(void)
Returns the thread-local system errno.
RD_EXPORT rd_kafka_topic_conf_t * rd_kafka_default_topic_conf_dup(rd_kafka_t *rk)
Creates a copy/duplicate of rk 's default topic configuration object.
RD_EXPORT rd_kafka_DeleteConsumerGroupOffsets_t * rd_kafka_DeleteConsumerGroupOffsets_new(const char *group, const rd_kafka_topic_partition_list_t *partitions)
Create a new DeleteConsumerGroupOffsets object. This object is later passed to rd_kafka_DeleteConsume...
RD_EXPORT rd_kafka_conf_t * rd_kafka_conf_dup(const rd_kafka_conf_t *conf)
Creates a copy/duplicate of configuration object conf.
RD_EXPORT void rd_kafka_DeleteTopic_destroy(rd_kafka_DeleteTopic_t *del_topic)
Destroy and free a DeleteTopic object previously created with rd_kafka_DeleteTopic_new()
RD_EXPORT const char * rd_kafka_err2name(rd_kafka_resp_err_t err)
Returns the error code name (enum name).
rd_kafka_resp_err_t() rd_kafka_interceptor_f_on_response_received_t(rd_kafka_t *rk, int sockfd, const char *brokername, int32_t brokerid, int16_t ApiKey, int16_t ApiVersion, int32_t CorrId, size_t size, int64_t rtt, rd_kafka_resp_err_t err, void *ic_opaque)
on_response_received() is called when a protocol response has been fully received from a broker TCP c...
Definition: rdkafka.h:5677
RD_EXPORT int rd_kafka_consume_start_queue(rd_kafka_topic_t *rkt, int32_t partition, int64_t offset, rd_kafka_queue_t *rkqu)
Same as rd_kafka_consume_start() but re-routes incoming messages to the provided queue rkqu (which mu...
int rd_kafka_event_type_t
Event types.
Definition: rdkafka.h:4929
RD_EXPORT size_t rd_kafka_queue_length(rd_kafka_queue_t *rkqu)
RD_EXPORT void rd_kafka_conf_dump_free(const char **arr, size_t cnt)
Frees a configuration dump returned from rd_kafka_conf_dump() or `rd_kafka_topic_conf_dump().
RD_EXPORT rd_kafka_message_t * rd_kafka_consume_queue(rd_kafka_queue_t *rkqu, int timeout_ms)
Consume from queue.
struct rd_kafka_ConfigEntry_s rd_kafka_ConfigEntry_t
Definition: rdkafka.h:6690
RD_EXPORT rd_kafka_resp_err_t rd_kafka_conf_interceptor_add_on_conf_destroy(rd_kafka_conf_t *conf, const char *ic_name, rd_kafka_interceptor_f_on_conf_destroy_t *on_conf_destroy, void *ic_opaque)
Append an on_conf_destroy() interceptor.
RD_EXPORT void rd_kafka_topic_destroy(rd_kafka_topic_t *rkt)
Loose application's topic handle refcount as previously created with rd_kafka_topic_new().
RD_EXPORT void rd_kafka_queue_destroy(rd_kafka_queue_t *rkqu)
struct rd_kafka_AdminOptions_s rd_kafka_AdminOptions_t
AdminOptions provides a generic mechanism for setting optional parameters for the Admin API requests.
Definition: rdkafka.h:6141
RD_EXPORT int rd_kafka_topic_partition_list_del(rd_kafka_topic_partition_list_t *rktparlist, const char *topic, int32_t partition)
Delete partition from list.
RD_EXPORT rd_kafka_resp_err_t rd_kafka_topic_result_error(const rd_kafka_topic_result_t *topicres)
Topic result provides per-topic operation result information.
RD_EXPORT int rd_kafka_error_is_fatal(const rd_kafka_error_t *error)
RD_EXPORT rd_kafka_resp_err_t rd_kafka_conf_interceptor_add_on_conf_set(rd_kafka_conf_t *conf, const char *ic_name, rd_kafka_interceptor_f_on_conf_set_t *on_conf_set, void *ic_opaque)
Append an on_conf_set() interceptor.
RD_EXPORT const rd_kafka_error_t * rd_kafka_group_result_error(const rd_kafka_group_result_t *groupres)
Group result provides per-group operation result information.
RD_EXPORT const rd_kafka_AlterConfigs_result_t * rd_kafka_event_AlterConfigs_result(rd_kafka_event_t *rkev)
Get AlterConfigs result.
RD_EXPORT void rd_kafka_conf_set_oauthbearer_token_refresh_cb(rd_kafka_conf_t *conf, void(*oauthbearer_token_refresh_cb)(rd_kafka_t *rk, const char *oauthbearer_config, void *opaque))
Set SASL/OAUTHBEARER token refresh callback in provided conf object.
RD_EXPORT rd_kafka_resp_err_t rd_kafka_AdminOptions_set_request_timeout(rd_kafka_AdminOptions_t *options, int timeout_ms, char *errstr, size_t errstr_size)
Sets the overall request timeout, including broker lookup, request transmission, operation time on br...
RD_EXPORT void rd_kafka_message_set_headers(rd_kafka_message_t *rkmessage, rd_kafka_headers_t *hdrs)
Replace the message's current headers with a new list.
RD_EXPORT rd_kafka_resp_err_t rd_kafka_commit_message(rd_kafka_t *rk, const rd_kafka_message_t *rkmessage, int async)
Commit message's offset on broker for the message's partition. The committed offset is the message's ...
rd_kafka_vtype_t
Var-arg tag types.
Definition: rdkafka.h:1078
@ RD_KAFKA_VTYPE_PARTITION
Definition: rdkafka.h:1082
@ RD_KAFKA_VTYPE_HEADERS
Definition: rdkafka.h:1094
@ RD_KAFKA_VTYPE_TOPIC
Definition: rdkafka.h:1080
@ RD_KAFKA_VTYPE_MSGFLAGS
Definition: rdkafka.h:1090
@ RD_KAFKA_VTYPE_RKT
Definition: rdkafka.h:1081
@ RD_KAFKA_VTYPE_HEADER
Definition: rdkafka.h:1092
@ RD_KAFKA_VTYPE_KEY
Definition: rdkafka.h:1084
@ RD_KAFKA_VTYPE_END
Definition: rdkafka.h:1079
@ RD_KAFKA_VTYPE_OPAQUE
Definition: rdkafka.h:1085
@ RD_KAFKA_VTYPE_VALUE
Definition: rdkafka.h:1083
@ RD_KAFKA_VTYPE_TIMESTAMP
Definition: rdkafka.h:1091
struct rd_kafka_ConfigResource_s rd_kafka_ConfigResource_t
Definition: rdkafka.h:6777
RD_EXPORT const char * rd_kafka_name(const rd_kafka_t *rk)
Returns Kafka handle name.
struct rd_kafka_DeleteGroup_s rd_kafka_DeleteGroup_t
Definition: rdkafka.h:7086
RD_EXPORT rd_kafka_resp_err_t rd_kafka_poll_set_consumer(rd_kafka_t *rk)
Redirect the main (rd_kafka_poll()) queue to the KafkaConsumer's queue (rd_kafka_consumer_poll()).
RD_EXPORT void rd_kafka_AdminOptions_set_opaque(rd_kafka_AdminOptions_t *options, void *ev_opaque)
Set application opaque value that can be extracted from the result event using rd_kafka_event_opaque(...
RD_EXPORT RD_DEPRECATED void rd_kafka_set_logger(rd_kafka_t *rk, void(*func)(const rd_kafka_t *rk, int level, const char *fac, const char *buf))
Set logger function.
RD_EXPORT rd_kafka_NewTopic_t * rd_kafka_NewTopic_new(const char *topic, int num_partitions, int replication_factor, char *errstr, size_t errstr_size)
Create a new NewTopic object. This object is later passed to rd_kafka_CreateTopics().
RD_EXPORT rd_kafka_headers_t * rd_kafka_headers_new(size_t initial_count)
Create a new headers list.
RD_EXPORT rd_kafka_resp_err_t rd_kafka_event_error(rd_kafka_event_t *rkev)
RD_EXPORT void rd_kafka_headers_destroy(rd_kafka_headers_t *hdrs)
Destroy the headers list. The object and any returned value pointers are not usable after this call.
RD_EXPORT int32_t rd_kafka_message_broker_id(const rd_kafka_message_t *rkmessage)
Returns the broker id of the broker the message was produced to or fetched from.
RD_EXPORT void rd_kafka_ConfigResource_destroy_array(rd_kafka_ConfigResource_t **config, size_t config_cnt)
Helper function to destroy all ConfigResource objects in the configs array (of config_cnt elements)....
RD_EXPORT rd_kafka_resp_err_t rd_kafka_interceptor_add_on_thread_start(rd_kafka_t *rk, const char *ic_name, rd_kafka_interceptor_f_on_thread_start_t *on_thread_start, void *ic_opaque)
Append an on_thread_start() interceptor.
RD_EXPORT rd_kafka_message_t * rd_kafka_consume(rd_kafka_topic_t *rkt, int32_t partition, int timeout_ms)
Consume a single message from topic rkt and partition.
RD_EXPORT void rd_kafka_conf_properties_show(FILE *fp)
Prints a table to fp of all supported configuration properties, their default values as well as a des...
rd_kafka_resp_err_t() rd_kafka_interceptor_f_on_acknowledgement_t(rd_kafka_t *rk, rd_kafka_message_t *rkmessage, void *ic_opaque)
on_acknowledgement() is called to inform interceptors that a message was succesfully delivered or per...
Definition: rdkafka.h:5564
struct rd_kafka_DeleteConsumerGroupOffsets_s rd_kafka_DeleteConsumerGroupOffsets_t
Definition: rdkafka.h:7163
RD_EXPORT rd_kafka_conf_t * rd_kafka_conf_new(void)
Create configuration object.
RD_EXPORT int rd_kafka_ConfigEntry_is_read_only(const rd_kafka_ConfigEntry_t *entry)
rd_kafka_resp_err_t() rd_kafka_interceptor_f_on_thread_start_t(rd_kafka_t *rk, rd_kafka_thread_type_t thread_type, const char *thread_name, void *ic_opaque)
on_thread_start() is called from a newly created librdkafka-managed thread.
Definition: rdkafka.h:5708
RD_EXPORT int rd_kafka_wait_destroyed(int timeout_ms)
Wait for all rd_kafka_t objects to be destroyed.
rd_kafka_ConfigSource_t
Apache Kafka config sources.
Definition: rdkafka.h:6659
@ RD_KAFKA_CONFIG_SOURCE_UNKNOWN_CONFIG
Definition: rdkafka.h:6662
@ RD_KAFKA_CONFIG_SOURCE__CNT
Definition: rdkafka.h:6678
@ RD_KAFKA_CONFIG_SOURCE_DYNAMIC_BROKER_CONFIG
Definition: rdkafka.h:6666
@ RD_KAFKA_CONFIG_SOURCE_DYNAMIC_TOPIC_CONFIG
Definition: rdkafka.h:6664
@ RD_KAFKA_CONFIG_SOURCE_DEFAULT_CONFIG
Definition: rdkafka.h:6675
@ RD_KAFKA_CONFIG_SOURCE_STATIC_BROKER_CONFIG
Definition: rdkafka.h:6672
@ RD_KAFKA_CONFIG_SOURCE_DYNAMIC_DEFAULT_BROKER_CONFIG
Definition: rdkafka.h:6669
RD_EXPORT int32_t rd_kafka_msg_partitioner_consistent(const rd_kafka_topic_t *rkt, const void *key, size_t keylen, int32_t partition_cnt, void *rkt_opaque, void *msg_opaque)
Consistent partitioner.
RD_EXPORT rd_kafka_resp_err_t rd_kafka_message_headers(const rd_kafka_message_t *rkmessage, rd_kafka_headers_t **hdrsp)
Get the message header list.
RD_EXPORT void rd_kafka_DeleteRecords(rd_kafka_t *rk, rd_kafka_DeleteRecords_t **del_records, size_t del_record_cnt, const rd_kafka_AdminOptions_t *options, rd_kafka_queue_t *rkqu)
Delete records (messages) in topic partitions older than the offsets provided.
rd_kafka_resp_err_t() rd_kafka_interceptor_f_on_destroy_t(rd_kafka_t *rk, void *ic_opaque)
on_destroy() is called from rd_kafka_destroy() or (rd_kafka_new() if rd_kafka_new() fails during init...
Definition: rdkafka.h:5511
RD_EXPORT rd_kafka_resp_err_t rd_kafka_flush(rd_kafka_t *rk, int timeout_ms)
Wait until all outstanding produce requests, et.al, are completed. This should typically be done prio...
RD_EXPORT int rd_kafka_error_txn_requires_abort(const rd_kafka_error_t *error)
RD_EXPORT rd_kafka_topic_t * rd_kafka_topic_new(rd_kafka_t *rk, const char *topic, rd_kafka_topic_conf_t *conf)
Creates a new topic handle for topic named topic.
RD_EXPORT void rd_kafka_group_list_destroy(const struct rd_kafka_group_list *grplist)
Release list memory.
rd_kafka_thread_type_t
librdkafka internal thread type.
Definition: rdkafka.h:4865
@ RD_KAFKA_THREAD_MAIN
Definition: rdkafka.h:4866
@ RD_KAFKA_THREAD_BACKGROUND
Definition: rdkafka.h:4867
@ RD_KAFKA_THREAD_BROKER
Definition: rdkafka.h:4868
RD_EXPORT void rd_kafka_destroy_flags(rd_kafka_t *rk, int flags)
Destroy Kafka handle according to specified destroy flags.
RD_EXPORT const rd_kafka_CreatePartitions_result_t * rd_kafka_event_CreatePartitions_result(rd_kafka_event_t *rkev)
Get CreatePartitions result.
RD_EXPORT void rd_kafka_conf_set_closesocket_cb(rd_kafka_conf_t *conf, int(*closesocket_cb)(int sockfd, void *opaque))
Set close socket callback.
RD_EXPORT int32_t rd_kafka_msg_partitioner_murmur2_random(const rd_kafka_topic_t *rkt, const void *key, size_t keylen, int32_t partition_cnt, void *rkt_opaque, void *msg_opaque)
Consistent-Random Murmur2 partitioner (Java compatible).
RD_EXPORT rd_kafka_resp_err_t rd_kafka_subscription(rd_kafka_t *rk, rd_kafka_topic_partition_list_t **topics)
Returns the current topic subscription.
RD_EXPORT rd_kafka_type_t rd_kafka_type(const rd_kafka_t *rk)
Returns Kafka handle type.
rd_kafka_cert_enc_t
SSL certificate encoding.
Definition: rdkafka.h:2267
@ RD_KAFKA_CERT_ENC_PEM
Definition: rdkafka.h:2270
@ RD_KAFKA_CERT_ENC_PKCS12
Definition: rdkafka.h:2268
@ RD_KAFKA_CERT_ENC_DER
Definition: rdkafka.h:2269
RD_EXPORT const char * rd_kafka_err2str(rd_kafka_resp_err_t err)
Returns a human readable representation of a kafka error.
RD_EXPORT int rd_kafka_brokers_add(rd_kafka_t *rk, const char *brokerlist)
Adds one or more brokers to the kafka handle's list of initial bootstrap brokers.
RD_EXPORT int rd_kafka_unittest(void)
Run librdkafka's built-in unit-tests.
RD_EXPORT rd_kafka_resp_err_t rd_kafka_commit(rd_kafka_t *rk, const rd_kafka_topic_partition_list_t *offsets, int async)
Commit offsets on broker for the provided list of partitions.
RD_EXPORT int64_t rd_kafka_message_latency(const rd_kafka_message_t *rkmessage)
Returns the latency for a produced message measured from the produce() call.
RD_EXPORT rd_kafka_resp_err_t rd_kafka_interceptor_add_on_acknowledgement(rd_kafka_t *rk, const char *ic_name, rd_kafka_interceptor_f_on_acknowledgement_t *on_acknowledgement, void *ic_opaque)
Append an on_acknowledgement() interceptor.
rd_kafka_event_t rd_kafka_DeleteGroups_result_t
Definition: rdkafka.h:5186
RD_EXPORT rd_kafka_conf_res_t rd_kafka_conf_set(rd_kafka_conf_t *conf, const char *name, const char *value, char *errstr, size_t errstr_size)
Sets a configuration property.
rd_kafka_event_t rd_kafka_DescribeConfigs_result_t
Definition: rdkafka.h:5182
RD_EXPORT rd_kafka_topic_partition_list_t * rd_kafka_event_topic_partition_list(rd_kafka_event_t *rkev)
RD_EXPORT int32_t rd_kafka_controllerid(rd_kafka_t *rk, int timeout_ms)
Returns the current ControllerId as reported in broker metadata.
RD_EXPORT int rd_kafka_ConfigEntry_is_default(const rd_kafka_ConfigEntry_t *entry)
RD_EXPORT rd_kafka_NewPartitions_t * rd_kafka_NewPartitions_new(const char *topic, size_t new_total_cnt, char *errstr, size_t errstr_size)
Create a new NewPartitions. This object is later passed to rd_kafka_CreatePartitions() to increase th...
RD_EXPORT void rd_kafka_conf_set_events(rd_kafka_conf_t *conf, int events)
Enable event sourcing. events is a bitmask of RD_KAFKA_EVENT_* of events to enable for consumption by...
RD_EXPORT rd_kafka_conf_res_t rd_kafka_conf_get(const rd_kafka_conf_t *conf, const char *name, char *dest, size_t *dest_size)
Retrieve configuration value for property name.
RD_EXPORT RD_DEPRECATED rd_kafka_resp_err_t rd_kafka_errno2err(int errnox)
Converts the system errno value errnox to a rd_kafka_resp_err_t error code upon failure from the foll...
RD_EXPORT int32_t rd_kafka_msg_partitioner_fnv1a(const rd_kafka_topic_t *rkt, const void *key, size_t keylen, int32_t partition_cnt, void *rkt_opaque, void *msg_opaque)
FNV-1a partitioner.
RD_EXPORT rd_kafka_error_t * rd_kafka_init_transactions(rd_kafka_t *rk, int timeout_ms)
Initialize transactions for the producer instance.
RD_EXPORT rd_kafka_topic_partition_t * rd_kafka_event_topic_partition(rd_kafka_event_t *rkev)
rd_kafka_event_t rd_kafka_DeleteConsumerGroupOffsets_result_t
Definition: rdkafka.h:5188
RD_EXPORT void rd_kafka_topic_conf_set_partitioner_cb(rd_kafka_topic_conf_t *topic_conf, int32_t(*partitioner)(const rd_kafka_topic_t *rkt, const void *keydata, size_t keylen, int32_t partition_cnt, void *rkt_opaque, void *msg_opaque))
Producer: Set partitioner callback in provided topic conf object.
RD_EXPORT const char * rd_kafka_ResourceType_name(rd_kafka_ResourceType_t restype)
RD_EXPORT void rd_kafka_destroy(rd_kafka_t *rk)
Destroy Kafka handle.
RD_EXPORT void rd_kafka_conf_set_dr_msg_cb(rd_kafka_conf_t *conf, void(*dr_msg_cb)(rd_kafka_t *rk, const rd_kafka_message_t *rkmessage, void *opaque))
Producer: Set delivery report callback in provided conf object.
RD_EXPORT rd_kafka_resp_err_t rd_kafka_producev(rd_kafka_t *rk,...)
Produce and send a single message to broker.
RD_EXPORT void rd_kafka_topic_conf_destroy(rd_kafka_topic_conf_t *topic_conf)
Destroys a topic conf object.
RD_EXPORT void rd_kafka_topic_partition_destroy(rd_kafka_topic_partition_t *rktpar)
Destroy a rd_kafka_topic_partition_t.
RD_EXPORT int32_t rd_kafka_msg_partitioner_fnv1a_random(const rd_kafka_topic_t *rkt, const void *key, size_t keylen, int32_t partition_cnt, void *rkt_opaque, void *msg_opaque)
Consistent-Random FNV-1a partitioner.
RD_EXPORT void rd_kafka_DescribeConfigs(rd_kafka_t *rk, rd_kafka_ConfigResource_t **configs, size_t config_cnt, const rd_kafka_AdminOptions_t *options, rd_kafka_queue_t *rkqu)
Get configuration for the specified resources in configs.
rd_kafka_type_t
rd_kafka_t handle type.
Definition: rdkafka.h:200
@ RD_KAFKA_CONSUMER
Definition: rdkafka.h:202
@ RD_KAFKA_PRODUCER
Definition: rdkafka.h:201
RD_EXPORT const rd_kafka_topic_partition_list_t * rd_kafka_DeleteRecords_result_offsets(const rd_kafka_DeleteRecords_result_t *result)
Get a list of topic and partition results from a DeleteRecords result. The returned objects will cont...
RD_EXPORT rd_kafka_conf_res_t rd_kafka_topic_conf_set(rd_kafka_topic_conf_t *conf, const char *name, const char *value, char *errstr, size_t errstr_size)
Sets a single rd_kafka_topic_conf_t value by property name.
RD_EXPORT rd_kafka_queue_t * rd_kafka_queue_get_consumer(rd_kafka_t *rk)
RD_EXPORT void rd_kafka_set_log_level(rd_kafka_t *rk, int level)
Specifies the maximum logging level emitted by internal kafka logging and debugging.
RD_EXPORT void rd_kafka_queue_io_event_enable(rd_kafka_queue_t *rkqu, int fd, const void *payload, size_t size)
Enable IO event triggering for queue.
RD_EXPORT int rd_kafka_error_is_retriable(const rd_kafka_error_t *error)
rd_kafka_ResourceType_t
Definition: rdkafka.h:6761
@ RD_KAFKA_RESOURCE_BROKER
Definition: rdkafka.h:6766
@ RD_KAFKA_RESOURCE__CNT
Definition: rdkafka.h:6767
@ RD_KAFKA_RESOURCE_TOPIC
Definition: rdkafka.h:6764
@ RD_KAFKA_RESOURCE_GROUP
Definition: rdkafka.h:6765
@ RD_KAFKA_RESOURCE_UNKNOWN
Definition: rdkafka.h:6762
@ RD_KAFKA_RESOURCE_ANY
Definition: rdkafka.h:6763
RD_EXPORT void rd_kafka_conf_set_error_cb(rd_kafka_conf_t *conf, void(*error_cb)(rd_kafka_t *rk, int err, const char *reason, void *opaque))
Set error callback in provided conf object.
RD_EXPORT int rd_kafka_consume_stop(rd_kafka_topic_t *rkt, int32_t partition)
Stop consuming messages for topic rkt and partition, purging all messages currently in the local queu...
RD_EXPORT rd_kafka_resp_err_t rd_kafka_resume_partitions(rd_kafka_t *rk, rd_kafka_topic_partition_list_t *partitions)
Resume producing consumption for the provided list of partitions.
RD_EXPORT void rd_kafka_ConfigResource_destroy(rd_kafka_ConfigResource_t *config)
Destroy and free a ConfigResource object previously created with rd_kafka_ConfigResource_new()
RD_EXPORT void rd_kafka_DeleteTopic_destroy_array(rd_kafka_DeleteTopic_t **del_topics, size_t del_topic_cnt)
Helper function to destroy all DeleteTopic objects in the del_topics array (of del_topic_cnt elements...
RD_EXPORT rd_kafka_resp_err_t rd_kafka_get_watermark_offsets(rd_kafka_t *rk, const char *topic, int32_t partition, int64_t *low, int64_t *high)
Get last known low (oldest/beginning) and high (newest/end) offsets for partition.
RD_EXPORT int rd_kafka_topic_partition_available(const rd_kafka_topic_t *rkt, int32_t partition)
Check if partition is available (has a leader broker).
RD_EXPORT rd_kafka_error_t * rd_kafka_consumer_group_metadata_read(rd_kafka_consumer_group_metadata_t **cgmdp, const void *buffer, size_t size)
Reads serialized consumer group metadata and returns a consumer group metadata object....
RD_EXPORT const rd_kafka_DeleteGroups_result_t * rd_kafka_event_DeleteGroups_result(rd_kafka_event_t *rkev)
Get DeleteGroups result.
rd_kafka_msg_status_t
Message persistence status can be used by the application to find out if a produced message was persi...
Definition: rdkafka.h:1547
@ RD_KAFKA_MSG_STATUS_POSSIBLY_PERSISTED
Definition: rdkafka.h:1556
@ RD_KAFKA_MSG_STATUS_PERSISTED
Definition: rdkafka.h:1561
@ RD_KAFKA_MSG_STATUS_NOT_PERSISTED
Definition: rdkafka.h:1551
RD_EXPORT int rd_kafka_outq_len(rd_kafka_t *rk)
Returns the current out queue length.
RD_EXPORT int rd_kafka_poll(rd_kafka_t *rk, int timeout_ms)
Polls the provided kafka handle for events.
RD_EXPORT rd_kafka_queue_t * rd_kafka_queue_get_partition(rd_kafka_t *rk, const char *topic, int32_t partition)
RD_EXPORT const char * rd_kafka_ConfigEntry_name(const rd_kafka_ConfigEntry_t *entry)
rd_kafka_event_t rd_kafka_AlterConfigs_result_t
Definition: rdkafka.h:5180
RD_EXPORT const char * rd_kafka_event_error_string(rd_kafka_event_t *rkev)
rd_kafka_conf_res_t
Configuration result type.
Definition: rdkafka.h:1589
@ RD_KAFKA_CONF_OK
Definition: rdkafka.h:1594
@ RD_KAFKA_CONF_INVALID
Definition: rdkafka.h:1591
@ RD_KAFKA_CONF_UNKNOWN
Definition: rdkafka.h:1590
RD_EXPORT rd_kafka_resp_err_t rd_kafka_header_add(rd_kafka_headers_t *hdrs, const char *name, ssize_t name_size, const void *value, ssize_t value_size)
Add header with name name and value val (copied) of size size (not including null-terminator).
RD_EXPORT const char * rd_kafka_ConfigSource_name(rd_kafka_ConfigSource_t confsource)
RD_EXPORT const rd_kafka_DeleteRecords_result_t * rd_kafka_event_DeleteRecords_result(rd_kafka_event_t *rkev)
RD_EXPORT rd_kafka_DeleteRecords_t * rd_kafka_DeleteRecords_new(const rd_kafka_topic_partition_list_t *before_offsets)
Create a new DeleteRecords object. This object is later passed to rd_kafka_DeleteRecords().
RD_EXPORT void rd_kafka_NewPartitions_destroy(rd_kafka_NewPartitions_t *new_parts)
Destroy and free a NewPartitions object previously created with rd_kafka_NewPartitions_new()
RD_EXPORT rd_kafka_resp_err_t rd_kafka_NewTopic_set_replica_assignment(rd_kafka_NewTopic_t *new_topic, int32_t partition, int32_t *broker_ids, size_t broker_id_cnt, char *errstr, size_t errstr_size)
Set the replica (broker) assignment for partition to the replica set in broker_ids (of broker_id_cnt ...
RD_EXPORT void rd_kafka_AlterConfigs(rd_kafka_t *rk, rd_kafka_ConfigResource_t **configs, size_t config_cnt, const rd_kafka_AdminOptions_t *options, rd_kafka_queue_t *rkqu)
Update the configuration for the specified resources. Updates are not transactional so they may succe...
RD_EXPORT const char * rd_kafka_get_debug_contexts(void)
Retrieve supported debug contexts for use with the "debug" configuration property....
RD_EXPORT rd_kafka_ConfigResource_t * rd_kafka_ConfigResource_new(rd_kafka_ResourceType_t restype, const char *resname)
Create new ConfigResource object.
RD_EXPORT void rd_kafka_AdminOptions_destroy(rd_kafka_AdminOptions_t *options)
Destroy a AdminOptions object.
RD_EXPORT rd_kafka_conf_res_t rd_kafka_conf_set_ssl_cert_verify_cb(rd_kafka_conf_t *conf, int(*ssl_cert_verify_cb)(rd_kafka_t *rk, const char *broker_name, int32_t broker_id, int *x509_error, int depth, const char *buf, size_t size, char *errstr, size_t errstr_size, void *opaque))
Sets the verification callback of the broker certificate.
RD_EXPORT int rd_kafka_consume_start(rd_kafka_topic_t *rkt, int32_t partition, int64_t offset)
Start consuming messages for topic rkt and partition at offset offset which may either be an absolute...
RD_EXPORT int rd_kafka_produce(rd_kafka_topic_t *rkt, int32_t partition, int msgflags, void *payload, size_t len, const void *key, size_t keylen, void *msg_opaque)
Produce and send a single message to broker.
RD_EXPORT rd_kafka_resp_err_t rd_kafka_offsets_for_times(rd_kafka_t *rk, rd_kafka_topic_partition_list_t *offsets, int timeout_ms)
Look up the offsets for the given partitions by timestamp.
rd_kafka_resp_err_t() rd_kafka_interceptor_f_on_consume_t(rd_kafka_t *rk, rd_kafka_message_t *rkmessage, void *ic_opaque)
on_consume() is called just prior to passing the message to the application in rd_kafka_consumer_poll...
Definition: rdkafka.h:5586
RD_EXPORT rd_kafka_resp_err_t rd_kafka_AdminOptions_set_broker(rd_kafka_AdminOptions_t *options, int32_t broker_id, char *errstr, size_t errstr_size)
Override what broker the Admin request will be sent to.
RD_EXPORT void rd_kafka_DeleteTopics(rd_kafka_t *rk, rd_kafka_DeleteTopic_t **del_topics, size_t del_topic_cnt, const rd_kafka_AdminOptions_t *options, rd_kafka_queue_t *rkqu)
Delete topics from cluster as specified by the topics array of size topic_cnt elements.
RD_EXPORT rd_kafka_resp_err_t rd_kafka_last_error(void)
Returns the last error code generated by a legacy API call in the current thread.
RD_EXPORT const char * rd_kafka_event_stats(rd_kafka_event_t *rkev)
Extract stats from the event.
rd_kafka_resp_err_t() rd_kafka_interceptor_f_on_thread_exit_t(rd_kafka_t *rk, rd_kafka_thread_type_t thread_type, const char *thread_name, void *ic_opaque)
on_thread_exit() is called just prior to a librdkafka-managed thread exiting from the exiting thread ...
Definition: rdkafka.h:5735
RD_EXPORT rd_kafka_resp_err_t rd_kafka_AdminOptions_set_validate_only(rd_kafka_AdminOptions_t *options, int true_or_false, char *errstr, size_t errstr_size)
Tell broker to only validate the request, without performing the requested operation (create topics,...
RD_EXPORT const char * rd_kafka_error_string(const rd_kafka_error_t *error)
RD_EXPORT int32_t rd_kafka_msg_partitioner_consistent_random(const rd_kafka_topic_t *rkt, const void *key, size_t keylen, int32_t partition_cnt, void *rkt_opaque, void *msg_opaque)
Consistent-Random partitioner.
RD_EXPORT rd_kafka_error_t * rd_kafka_consumer_group_metadata_write(const rd_kafka_consumer_group_metadata_t *cgmd, void **bufferp, size_t *sizep)
Serialize the consumer group metadata to a binary format. This is mainly for client binding use and n...
RD_EXPORT const rd_kafka_topic_result_t ** rd_kafka_DeleteTopics_result_topics(const rd_kafka_DeleteTopics_result_t *result, size_t *cntp)
Get an array of topic results from a DeleteTopics result.
RD_EXPORT void rd_kafka_consumer_group_metadata_destroy(rd_kafka_consumer_group_metadata_t *)
Frees the consumer group metadata object as returned by rd_kafka_consumer_group_metadata().
RD_EXPORT void * rd_kafka_opaque(const rd_kafka_t *rk)
Retrieves the opaque pointer previously set with rd_kafka_conf_set_opaque()
RD_EXPORT void rd_kafka_event_destroy(rd_kafka_event_t *rkev)
Destroy an event.
RD_EXPORT rd_kafka_msg_status_t rd_kafka_message_status(const rd_kafka_message_t *rkmessage)
Returns the message's persistence status in the topic log.
RD_EXPORT void rd_kafka_conf_destroy(rd_kafka_conf_t *conf)
Destroys a conf object.
RD_EXPORT void rd_kafka_queue_yield(rd_kafka_queue_t *rkqu)
Cancels the current rd_kafka_queue_poll() on rkqu.
RD_EXPORT void rd_kafka_topic_partition_list_sort(rd_kafka_topic_partition_list_t *rktparlist, int(*cmp)(const void *a, const void *b, void *cmp_opaque), void *cmp_opaque)
Sort list using comparator cmp.
RD_EXPORT rd_kafka_resp_err_t rd_kafka_interceptor_add_on_send(rd_kafka_t *rk, const char *ic_name, rd_kafka_interceptor_f_on_send_t *on_send, void *ic_opaque)
Append an on_send() interceptor.
RD_EXPORT const char * rd_kafka_ConfigResource_name(const rd_kafka_ConfigResource_t *config)
RD_EXPORT rd_kafka_resp_err_t rd_kafka_commit_queue(rd_kafka_t *rk, const rd_kafka_topic_partition_list_t *offsets, rd_kafka_queue_t *rkqu, void(*cb)(rd_kafka_t *rk, rd_kafka_resp_err_t err, rd_kafka_topic_partition_list_t *offsets, void *commit_opaque), void *commit_opaque)
Commit offsets on broker for the provided list of partitions.
rd_kafka_timestamp_type_t
Definition: rdkafka.h:211
@ RD_KAFKA_TIMESTAMP_NOT_AVAILABLE
Definition: rdkafka.h:212
@ RD_KAFKA_TIMESTAMP_CREATE_TIME
Definition: rdkafka.h:213
@ RD_KAFKA_TIMESTAMP_LOG_APPEND_TIME
Definition: rdkafka.h:214
RD_EXPORT rd_kafka_resp_err_t rd_kafka_message_detach_headers(rd_kafka_message_t *rkmessage, rd_kafka_headers_t **hdrsp)
Get the message header list and detach the list from the message making the application the owner of ...
RD_EXPORT size_t rd_kafka_header_cnt(const rd_kafka_headers_t *hdrs)
Returns the number of header key/value pairs.
RD_EXPORT void rd_kafka_DeleteGroups(rd_kafka_t *rk, rd_kafka_DeleteGroup_t **del_groups, size_t del_group_cnt, const rd_kafka_AdminOptions_t *options, rd_kafka_queue_t *rkqu)
Delete groups from cluster as specified by the del_groups array of size del_group_cnt elements.
RD_EXPORT void rd_kafka_DeleteConsumerGroupOffsets(rd_kafka_t *rk, rd_kafka_DeleteConsumerGroupOffsets_t **del_grpoffsets, size_t del_grpoffsets_cnt, const rd_kafka_AdminOptions_t *options, rd_kafka_queue_t *rkqu)
Delete committed offsets for a set of partitions in a conusmer group. This will succeed at the partit...
RD_EXPORT rd_kafka_ResourceType_t rd_kafka_ConfigResource_type(const rd_kafka_ConfigResource_t *config)
RD_EXPORT rd_kafka_topic_partition_list_t * rd_kafka_topic_partition_list_new(int size)
Create a new list/vector Topic+Partition container.
RD_EXPORT rd_kafka_resp_err_t rd_kafka_ConfigResource_set_config(rd_kafka_ConfigResource_t *config, const char *name, const char *value)
Set configuration name value pair.
RD_EXPORT rd_kafka_error_t * rd_kafka_error_new(rd_kafka_resp_err_t code, const char *fmt,...) RD_FORMAT(printf
Create a new error object with error code and optional human readable error string in fmt.
rd_kafka_resp_err_t() rd_kafka_interceptor_f_on_conf_destroy_t(void *ic_opaque)
on_conf_destroy() is called from rd_kafka_*_conf_destroy() in the order the interceptors were added.
Definition: rdkafka.h:5477
RD_EXPORT rd_kafka_ConfigSource_t rd_kafka_ConfigEntry_source(const rd_kafka_ConfigEntry_t *entry)
Error code value, name and description. Typically for use with language bindings to automatically exp...
Definition: rdkafka.h:626
const char * name
Definition: rdkafka.h:628
rd_kafka_resp_err_t code
Definition: rdkafka.h:627
const char * desc
Definition: rdkafka.h:629
Group information.
Definition: rdkafka.h:4651
struct rd_kafka_metadata_broker broker
Definition: rdkafka.h:4652
int member_cnt
Definition: rdkafka.h:4659
char * state
Definition: rdkafka.h:4655
char * group
Definition: rdkafka.h:4653
struct rd_kafka_group_member_info * members
Definition: rdkafka.h:4658
char * protocol
Definition: rdkafka.h:4657
char * protocol_type
Definition: rdkafka.h:4656
rd_kafka_resp_err_t err
Definition: rdkafka.h:4654
List of groups.
Definition: rdkafka.h:4667
int group_cnt
Definition: rdkafka.h:4669
struct rd_kafka_group_info * groups
Definition: rdkafka.h:4668
Group member information.
Definition: rdkafka.h:4636
char * member_id
Definition: rdkafka.h:4637
int member_assignment_size
Definition: rdkafka.h:4645
int member_metadata_size
Definition: rdkafka.h:4642
void * member_metadata
Definition: rdkafka.h:4640
char * client_host
Definition: rdkafka.h:4639
void * member_assignment
Definition: rdkafka.h:4643
char * client_id
Definition: rdkafka.h:4638
A Kafka message as returned by the rd_kafka_consume*() family of functions as well as provided to the...
Definition: rdkafka.h:1393
size_t key_len
Definition: rdkafka.h:1406
size_t len
Definition: rdkafka.h:1401
void * _private
Definition: rdkafka.h:1416
void * key
Definition: rdkafka.h:1404
int64_t offset
Definition: rdkafka.h:1408
void * payload
Definition: rdkafka.h:1397
int32_t partition
Definition: rdkafka.h:1396
rd_kafka_topic_t * rkt
Definition: rdkafka.h:1395
rd_kafka_resp_err_t err
Definition: rdkafka.h:1394
Broker information.
Definition: rdkafka.h:4538
int32_t id
Definition: rdkafka.h:4539
int port
Definition: rdkafka.h:4541
char * host
Definition: rdkafka.h:4540
Partition information.
Definition: rdkafka.h:4547
int32_t leader
Definition: rdkafka.h:4550
int32_t * isrs
Definition: rdkafka.h:4554
int replica_cnt
Definition: rdkafka.h:4551
rd_kafka_resp_err_t err
Definition: rdkafka.h:4549
int isr_cnt
Definition: rdkafka.h:4553
int32_t id
Definition: rdkafka.h:4548
int32_t * replicas
Definition: rdkafka.h:4552
Metadata container.
Definition: rdkafka.h:4571
int32_t orig_broker_id
Definition: rdkafka.h:4578
char * orig_broker_name
Definition: rdkafka.h:4579
int broker_cnt
Definition: rdkafka.h:4572
struct rd_kafka_metadata_topic * topics
Definition: rdkafka.h:4576
int topic_cnt
Definition: rdkafka.h:4575
struct rd_kafka_metadata_broker * brokers
Definition: rdkafka.h:4573
Topic information.
Definition: rdkafka.h:4560
char * topic
Definition: rdkafka.h:4561
struct rd_kafka_metadata_partition * partitions
Definition: rdkafka.h:4563
rd_kafka_resp_err_t err
Definition: rdkafka.h:4564
int partition_cnt
Definition: rdkafka.h:4562
A growable list of Topic+Partitions.
Definition: rdkafka.h:910
int cnt
Definition: rdkafka.h:911
int size
Definition: rdkafka.h:912
rd_kafka_topic_partition_t * elems
Definition: rdkafka.h:913
Topic+Partition place holder.
Definition: rdkafka.h:885
int64_t offset
Definition: rdkafka.h:888
size_t metadata_size
Definition: rdkafka.h:890
void * opaque
Definition: rdkafka.h:891
char * topic
Definition: rdkafka.h:886
void * metadata
Definition: rdkafka.h:889
int32_t partition
Definition: rdkafka.h:887
rd_kafka_resp_err_t err
Definition: rdkafka.h:892
void * _private
Definition: rdkafka.h:893
VTYPE + argument container for use with rd_kafka_produce_va()
Definition: rdkafka.h:1104
rd_kafka_vtype_t vtype
Definition: rdkafka.h:1105