librdkafka
The Apache Kafka C/C++ client library
rdkafkacpp.h
Go to the documentation of this file.
1 /*
2  * librdkafka - Apache Kafka C/C++ library
3  *
4  * Copyright (c) 2014 Magnus Edenhill
5  * All rights reserved.
6  *
7  * Redistribution and use in source and binary forms, with or without
8  * modification, are permitted provided that the following conditions are met:
9  *
10  * 1. Redistributions of source code must retain the above copyright notice,
11  * this list of conditions and the following disclaimer.
12  * 2. Redistributions in binary form must reproduce the above copyright notice,
13  * this list of conditions and the following disclaimer in the documentation
14  * and/or other materials provided with the distribution.
15  *
16  * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
17  * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
18  * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
19  * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE
20  * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
21  * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
22  * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
23  * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
24  * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
25  * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE
26  * POSSIBILITY OF SUCH DAMAGE.
27  */
28 
29 #ifndef _RDKAFKACPP_H_
30 #define _RDKAFKACPP_H_
31 
50 #include <string>
51 #include <list>
52 #include <vector>
53 #include <cstdlib>
54 #include <cstring>
55 #include <stdint.h>
56 #include <sys/types.h>
57 
58 #ifdef _WIN32
59 #ifndef ssize_t
60 #ifndef _BASETSD_H_
61 #include <basetsd.h>
62 #endif
63 #ifndef _SSIZE_T_DEFINED
64 #define _SSIZE_T_DEFINED
65 typedef SSIZE_T ssize_t;
66 #endif
67 #endif
68 #undef RD_EXPORT
69 #ifdef LIBRDKAFKA_STATICLIB
70 #define RD_EXPORT
71 #else
72 #ifdef LIBRDKAFKACPP_EXPORTS
73 #define RD_EXPORT __declspec(dllexport)
74 #else
75 #define RD_EXPORT __declspec(dllimport)
76 #endif
77 #endif
78 #else
79 #define RD_EXPORT
80 #endif
81 
84 extern "C" {
85 /* Forward declarations */
86 struct rd_kafka_s;
87 struct rd_kafka_topic_s;
88 struct rd_kafka_message_s;
89 struct rd_kafka_conf_s;
90 struct rd_kafka_topic_conf_s;
91 }
92 
93 namespace RdKafka {
94 
114 #define RD_KAFKA_VERSION 0x010900ff
115 
121 RD_EXPORT
122 int version();
123 
127 RD_EXPORT
128 std::string version_str();
129 
134 RD_EXPORT
135 std::string get_debug_contexts();
136 
146 RD_EXPORT
147 int wait_destroyed(int timeout_ms);
148 
159 RD_EXPORT
160 void *mem_malloc(size_t size);
161 
175 RD_EXPORT
176 void mem_free(void *ptr);
177 
200 enum ErrorCode {
201  /* Internal errors to rdkafka: */
203  ERR__BEGIN = -200,
205  ERR__BAD_MSG = -199,
209  ERR__DESTROY = -197,
211  ERR__FAIL = -196,
217  ERR__RESOLVE = -193,
228  ERR__FS = -189,
244  ERR__SSL = -181,
262  ERR__STATE = -172,
280  ERR__INTR = -163,
290  ERR__PARTIAL = -158,
294  ERR__NOENT = -156,
300  ERR__RETRY = -153,
306  ERR__FATAL = -150,
318  ERR__FENCED = -144,
324  ERR__NOOP = -141,
327 
329  ERR__END = -100,
330 
331  /* Kafka broker errors: */
365 #define ERR_GROUP_LOAD_IN_PROGRESS ERR_COORDINATOR_LOAD_IN_PROGRESS
369 #define ERR_GROUP_COORDINATOR_NOT_AVAILABLE ERR_COORDINATOR_NOT_AVAILABLE
373 #define ERR_NOT_COORDINATOR_FOR_GROUP ERR_NOT_COORDINATOR
545 };
546 
547 
551 RD_EXPORT
552 std::string err2str(RdKafka::ErrorCode err);
553 
554 
555 
564  CERT__CNT
565 };
566 
575  CERT_ENC__CNT
576 };
577 
583 /* Forward declarations */
584 class Handle;
585 class Producer;
586 class Message;
587 class Headers;
588 class Queue;
589 class Event;
590 class Topic;
591 class TopicPartition;
592 class Metadata;
593 class KafkaConsumer;
613 class RD_EXPORT Error {
614  public:
618  static Error *create(ErrorCode code, const std::string *errstr);
619 
620  virtual ~Error() {
621  }
622 
623  /*
624  * Error accessor methods
625  */
626 
630  virtual ErrorCode code() const = 0;
631 
635  virtual std::string name() const = 0;
636 
640  virtual std::string str() const = 0;
641 
646  virtual bool is_fatal() const = 0;
647 
651  virtual bool is_retriable() const = 0;
652 
664  virtual bool txn_requires_abort() const = 0;
665 };
666 
698 class RD_EXPORT DeliveryReportCb {
699  public:
703  virtual void dr_cb(Message &message) = 0;
704 
705  virtual ~DeliveryReportCb() {
706  }
707 };
708 
709 
737 class RD_EXPORT OAuthBearerTokenRefreshCb {
738  public:
747  RdKafka::Handle *handle,
748  const std::string &oauthbearer_config) = 0;
749 
750  virtual ~OAuthBearerTokenRefreshCb() {
751  }
752 };
753 
754 
762 class RD_EXPORT PartitionerCb {
763  public:
781  virtual int32_t partitioner_cb(const Topic *topic,
782  const std::string *key,
783  int32_t partition_cnt,
784  void *msg_opaque) = 0;
785 
786  virtual ~PartitionerCb() {
787  }
788 };
789 
795  public:
804  virtual int32_t partitioner_cb(const Topic *topic,
805  const void *key,
806  size_t key_len,
807  int32_t partition_cnt,
808  void *msg_opaque) = 0;
809 
810  virtual ~PartitionerKeyPointerCb() {
811  }
812 };
813 
814 
815 
824 class RD_EXPORT EventCb {
825  public:
831  virtual void event_cb(Event &event) = 0;
832 
833  virtual ~EventCb() {
834  }
835 };
836 
837 
841 class RD_EXPORT Event {
842  public:
844  enum Type {
848  EVENT_THROTTLE
849  };
850 
852  enum Severity {
853  EVENT_SEVERITY_EMERG = 0,
854  EVENT_SEVERITY_ALERT = 1,
855  EVENT_SEVERITY_CRITICAL = 2,
856  EVENT_SEVERITY_ERROR = 3,
857  EVENT_SEVERITY_WARNING = 4,
858  EVENT_SEVERITY_NOTICE = 5,
859  EVENT_SEVERITY_INFO = 6,
860  EVENT_SEVERITY_DEBUG = 7
861  };
862 
863  virtual ~Event() {
864  }
865 
866  /*
867  * Event Accessor methods
868  */
869 
874  virtual Type type() const = 0;
875 
880  virtual ErrorCode err() const = 0;
881 
886  virtual Severity severity() const = 0;
887 
892  virtual std::string fac() const = 0;
893 
902  virtual std::string str() const = 0;
903 
908  virtual int throttle_time() const = 0;
909 
914  virtual std::string broker_name() const = 0;
915 
920  virtual int broker_id() const = 0;
921 
922 
928  virtual bool fatal() const = 0;
929 };
930 
931 
932 
936 class RD_EXPORT ConsumeCb {
937  public:
945  virtual void consume_cb(Message &message, void *opaque) = 0;
946 
947  virtual ~ConsumeCb() {
948  }
949 };
950 
951 
955 class RD_EXPORT RebalanceCb {
956  public:
1025  virtual void rebalance_cb(RdKafka::KafkaConsumer *consumer,
1026  RdKafka::ErrorCode err,
1027  std::vector<TopicPartition *> &partitions) = 0;
1028 
1029  virtual ~RebalanceCb() {
1030  }
1031 };
1032 
1033 
1037 class RD_EXPORT OffsetCommitCb {
1038  public:
1055  std::vector<TopicPartition *> &offsets) = 0;
1056 
1057  virtual ~OffsetCommitCb() {
1058  }
1059 };
1060 
1061 
1062 
1068 class RD_EXPORT SslCertificateVerifyCb {
1069  public:
1106  virtual bool ssl_cert_verify_cb(const std::string &broker_name,
1107  int32_t broker_id,
1108  int *x509_error,
1109  int depth,
1110  const char *buf,
1111  size_t size,
1112  std::string &errstr) = 0;
1113 
1114  virtual ~SslCertificateVerifyCb() {
1115  }
1116 };
1117 
1118 
1123 class RD_EXPORT SocketCb {
1124  public:
1138  virtual int socket_cb(int domain, int type, int protocol) = 0;
1139 
1140  virtual ~SocketCb() {
1141  }
1142 };
1143 
1144 
1149 class RD_EXPORT OpenCb {
1150  public:
1162  virtual int open_cb(const std::string &path, int flags, int mode) = 0;
1163 
1164  virtual ~OpenCb() {
1165  }
1166 };
1167 
1168 
1188 class RD_EXPORT Conf {
1189  public:
1193  enum ConfType {
1195  CONF_TOPIC
1196  };
1197 
1201  enum ConfResult {
1202  CONF_UNKNOWN = -2,
1203  CONF_INVALID = -1,
1204  CONF_OK = 0
1205  };
1206 
1207 
1211  static Conf *create(ConfType type);
1212 
1213  virtual ~Conf() {
1214  }
1215 
1229  virtual Conf::ConfResult set(const std::string &name,
1230  const std::string &value,
1231  std::string &errstr) = 0;
1232 
1234  virtual Conf::ConfResult set(const std::string &name,
1235  DeliveryReportCb *dr_cb,
1236  std::string &errstr) = 0;
1237 
1240  const std::string &name,
1241  OAuthBearerTokenRefreshCb *oauthbearer_token_refresh_cb,
1242  std::string &errstr) = 0;
1243 
1245  virtual Conf::ConfResult set(const std::string &name,
1246  EventCb *event_cb,
1247  std::string &errstr) = 0;
1248 
1256  virtual Conf::ConfResult set(const std::string &name,
1257  const Conf *topic_conf,
1258  std::string &errstr) = 0;
1259 
1261  virtual Conf::ConfResult set(const std::string &name,
1262  PartitionerCb *partitioner_cb,
1263  std::string &errstr) = 0;
1264 
1266  virtual Conf::ConfResult set(const std::string &name,
1267  PartitionerKeyPointerCb *partitioner_kp_cb,
1268  std::string &errstr) = 0;
1269 
1271  virtual Conf::ConfResult set(const std::string &name,
1272  SocketCb *socket_cb,
1273  std::string &errstr) = 0;
1274 
1276  virtual Conf::ConfResult set(const std::string &name,
1277  OpenCb *open_cb,
1278  std::string &errstr) = 0;
1279 
1281  virtual Conf::ConfResult set(const std::string &name,
1282  RebalanceCb *rebalance_cb,
1283  std::string &errstr) = 0;
1284 
1286  virtual Conf::ConfResult set(const std::string &name,
1287  OffsetCommitCb *offset_commit_cb,
1288  std::string &errstr) = 0;
1289 
1294  virtual Conf::ConfResult set(const std::string &name,
1295  SslCertificateVerifyCb *ssl_cert_verify_cb,
1296  std::string &errstr) = 0;
1297 
1332  const void *buffer,
1333  size_t size,
1334  std::string &errstr) = 0;
1335 
1348  virtual Conf::ConfResult get(const std::string &name,
1349  std::string &value) const = 0;
1350 
1354  virtual Conf::ConfResult get(DeliveryReportCb *&dr_cb) const = 0;
1355 
1360  OAuthBearerTokenRefreshCb *&oauthbearer_token_refresh_cb) const = 0;
1361 
1365  virtual Conf::ConfResult get(EventCb *&event_cb) const = 0;
1366 
1370  virtual Conf::ConfResult get(PartitionerCb *&partitioner_cb) const = 0;
1371 
1376  PartitionerKeyPointerCb *&partitioner_kp_cb) const = 0;
1377 
1381  virtual Conf::ConfResult get(SocketCb *&socket_cb) const = 0;
1382 
1386  virtual Conf::ConfResult get(OpenCb *&open_cb) const = 0;
1387 
1391  virtual Conf::ConfResult get(RebalanceCb *&rebalance_cb) const = 0;
1392 
1396  virtual Conf::ConfResult get(OffsetCommitCb *&offset_commit_cb) const = 0;
1397 
1400  SslCertificateVerifyCb *&ssl_cert_verify_cb) const = 0;
1401 
1404  virtual std::list<std::string> *dump() = 0;
1405 
1407  virtual Conf::ConfResult set(const std::string &name,
1408  ConsumeCb *consume_cb,
1409  std::string &errstr) = 0;
1410 
1427  virtual struct rd_kafka_conf_s *c_ptr_global() = 0;
1428 
1446  virtual struct rd_kafka_topic_conf_s *c_ptr_topic() = 0;
1447 
1461  std::string &errstr) = 0;
1462 
1463 
1487  std::string &errstr) = 0;
1488 };
1489 
1502 class RD_EXPORT Handle {
1503  public:
1504  virtual ~Handle() {
1505  }
1506 
1508  virtual const std::string name() const = 0;
1509 
1518  virtual const std::string memberid() const = 0;
1519 
1520 
1545  virtual int poll(int timeout_ms) = 0;
1546 
1553  virtual int outq_len() = 0;
1554 
1570  virtual ErrorCode metadata(bool all_topics,
1571  const Topic *only_rkt,
1572  Metadata **metadatap,
1573  int timeout_ms) = 0;
1574 
1575 
1585  virtual ErrorCode pause(std::vector<TopicPartition *> &partitions) = 0;
1586 
1587 
1597  virtual ErrorCode resume(std::vector<TopicPartition *> &partitions) = 0;
1598 
1599 
1608  virtual ErrorCode query_watermark_offsets(const std::string &topic,
1609  int32_t partition,
1610  int64_t *low,
1611  int64_t *high,
1612  int timeout_ms) = 0;
1613 
1631  virtual ErrorCode get_watermark_offsets(const std::string &topic,
1632  int32_t partition,
1633  int64_t *low,
1634  int64_t *high) = 0;
1635 
1636 
1658  virtual ErrorCode offsetsForTimes(std::vector<TopicPartition *> &offsets,
1659  int timeout_ms) = 0;
1660 
1661 
1670  virtual Queue *get_partition_queue(const TopicPartition *partition) = 0;
1671 
1688  virtual ErrorCode set_log_queue(Queue *queue) = 0;
1689 
1701  virtual void yield() = 0;
1702 
1717  virtual const std::string clusterid(int timeout_ms) = 0;
1718 
1735  virtual struct rd_kafka_s *c_ptr() = 0;
1736 
1752  virtual int32_t controllerid(int timeout_ms) = 0;
1753 
1754 
1776  virtual ErrorCode fatal_error(std::string &errstr) const = 0;
1777 
1818  const std::string &token_value,
1819  int64_t md_lifetime_ms,
1820  const std::string &md_principal_name,
1821  const std::list<std::string> &extensions,
1822  std::string &errstr) = 0;
1823 
1842  const std::string &errstr) = 0;
1843 
1852 
1853 
1859  virtual Queue *get_sasl_queue() = 0;
1860 
1864  virtual Queue *get_background_queue() = 0;
1865 
1866 
1867 
1878  virtual void *mem_malloc(size_t size) = 0;
1879 
1893  virtual void mem_free(void *ptr) = 0;
1894 };
1895 
1896 
1915 class RD_EXPORT TopicPartition {
1916  public:
1922  static TopicPartition *create(const std::string &topic, int partition);
1923 
1930  static TopicPartition *create(const std::string &topic,
1931  int partition,
1932  int64_t offset);
1933 
1934  virtual ~TopicPartition() = 0;
1935 
1940  static void destroy(std::vector<TopicPartition *> &partitions);
1941 
1943  virtual const std::string &topic() const = 0;
1944 
1946  virtual int partition() const = 0;
1947 
1949  virtual int64_t offset() const = 0;
1950 
1952  virtual void set_offset(int64_t offset) = 0;
1953 
1955  virtual ErrorCode err() const = 0;
1956 };
1957 
1958 
1959 
1964 class RD_EXPORT Topic {
1965  public:
1972  static const int32_t PARTITION_UA;
1973 
1975  static const int64_t OFFSET_BEGINNING;
1976  static const int64_t OFFSET_END;
1977  static const int64_t OFFSET_STORED;
1978  static const int64_t OFFSET_INVALID;
1990  static Topic *create(Handle *base,
1991  const std::string &topic_str,
1992  const Conf *conf,
1993  std::string &errstr);
1994 
1995  virtual ~Topic() = 0;
1996 
1997 
1999  virtual const std::string name() const = 0;
2000 
2006  virtual bool partition_available(int32_t partition) const = 0;
2007 
2019  virtual ErrorCode offset_store(int32_t partition, int64_t offset) = 0;
2020 
2037  virtual struct rd_kafka_topic_s *c_ptr() = 0;
2038 };
2039 
2040 
2062 class RD_EXPORT MessageTimestamp {
2063  public:
2068  MSG_TIMESTAMP_LOG_APPEND_TIME
2069  };
2070 
2072  int64_t timestamp;
2073 };
2074 
2075 
2085 class RD_EXPORT Headers {
2086  public:
2087  virtual ~Headers() = 0;
2088 
2097  class Header {
2098  public:
2109  Header(const std::string &key, const void *value, size_t value_size) :
2110  key_(key), err_(ERR_NO_ERROR), value_size_(value_size) {
2111  value_ = copy_value(value, value_size);
2112  }
2113 
2127  Header(const std::string &key,
2128  const void *value,
2129  size_t value_size,
2130  const RdKafka::ErrorCode err) :
2131  key_(key), err_(err), value_(NULL), value_size_(value_size) {
2132  if (err == ERR_NO_ERROR)
2133  value_ = copy_value(value, value_size);
2134  }
2135 
2141  Header(const Header &other) :
2142  key_(other.key_), err_(other.err_), value_size_(other.value_size_) {
2143  value_ = copy_value(other.value_, value_size_);
2144  }
2145 
2151  Header &operator=(const Header &other) {
2152  if (&other == this) {
2153  return *this;
2154  }
2155 
2156  key_ = other.key_;
2157  err_ = other.err_;
2158  value_size_ = other.value_size_;
2159 
2160  if (value_ != NULL)
2161  mem_free(value_);
2162 
2163  value_ = copy_value(other.value_, value_size_);
2164 
2165  return *this;
2166  }
2167 
2168  ~Header() {
2169  if (value_ != NULL)
2170  mem_free(value_);
2171  }
2172 
2174  std::string key() const {
2175  return key_;
2176  }
2177 
2179  const void *value() const {
2180  return value_;
2181  }
2182 
2185  const char *value_string() const {
2186  return static_cast<const char *>(value_);
2187  }
2188 
2190  size_t value_size() const {
2191  return value_size_;
2192  }
2193 
2196  return err_;
2197  }
2198 
2199  private:
2200  char *copy_value(const void *value, size_t value_size) {
2201  if (!value)
2202  return NULL;
2203 
2204  char *dest = (char *)mem_malloc(value_size + 1);
2205  memcpy(dest, (const char *)value, value_size);
2206  dest[value_size] = '\0';
2207 
2208  return dest;
2209  }
2210 
2211  std::string key_;
2212  RdKafka::ErrorCode err_;
2213  char *value_;
2214  size_t value_size_;
2215  void *operator new(size_t); /* Prevent dynamic allocation */
2216  };
2217 
2223  static Headers *create();
2224 
2233  static Headers *create(const std::vector<Header> &headers);
2234 
2244  virtual ErrorCode add(const std::string &key,
2245  const void *value,
2246  size_t value_size) = 0;
2247 
2258  virtual ErrorCode add(const std::string &key, const std::string &value) = 0;
2259 
2269  virtual ErrorCode add(const Header &header) = 0;
2270 
2278  virtual ErrorCode remove(const std::string &key) = 0;
2279 
2289  virtual std::vector<Header> get(const std::string &key) const = 0;
2290 
2301  virtual Header get_last(const std::string &key) const = 0;
2302 
2308  virtual std::vector<Header> get_all() const = 0;
2309 
2313  virtual size_t size() const = 0;
2314 };
2315 
2316 
2328 class RD_EXPORT Message {
2329  public:
2332  enum Status {
2336  MSG_STATUS_NOT_PERSISTED = 0,
2337 
2341  MSG_STATUS_POSSIBLY_PERSISTED = 1,
2342 
2346  MSG_STATUS_PERSISTED = 2,
2347  };
2348 
2356  virtual std::string errstr() const = 0;
2357 
2359  virtual ErrorCode err() const = 0;
2360 
2365  virtual Topic *topic() const = 0;
2366 
2368  virtual std::string topic_name() const = 0;
2369 
2371  virtual int32_t partition() const = 0;
2372 
2374  virtual void *payload() const = 0;
2375 
2377  virtual size_t len() const = 0;
2378 
2380  virtual const std::string *key() const = 0;
2381 
2383  virtual const void *key_pointer() const = 0;
2384 
2386  virtual size_t key_len() const = 0;
2387 
2389  virtual int64_t offset() const = 0;
2390 
2392  virtual MessageTimestamp timestamp() const = 0;
2393 
2395  virtual void *msg_opaque() const = 0;
2396 
2397  virtual ~Message() = 0;
2398 
2401  virtual int64_t latency() const = 0;
2402 
2419  virtual struct rd_kafka_message_s *c_ptr() = 0;
2420 
2424  virtual Status status() const = 0;
2425 
2430  virtual RdKafka::Headers *headers() = 0;
2431 
2439 
2442  virtual int32_t broker_id() const = 0;
2443 };
2444 
2468 class RD_EXPORT Queue {
2469  public:
2473  static Queue *create(Handle *handle);
2474 
2485  virtual ErrorCode forward(Queue *dst) = 0;
2486 
2487 
2499  virtual Message *consume(int timeout_ms) = 0;
2500 
2508  virtual int poll(int timeout_ms) = 0;
2509 
2510  virtual ~Queue() = 0;
2511 
2527  virtual void io_event_enable(int fd, const void *payload, size_t size) = 0;
2528 };
2529 
2543 class RD_EXPORT ConsumerGroupMetadata {
2544  public:
2545  virtual ~ConsumerGroupMetadata() = 0;
2546 };
2547 
2565 class RD_EXPORT KafkaConsumer : public virtual Handle {
2566  public:
2578  static KafkaConsumer *create(const Conf *conf, std::string &errstr);
2579 
2580  virtual ~KafkaConsumer() = 0;
2581 
2582 
2586  std::vector<RdKafka::TopicPartition *> &partitions) = 0;
2587 
2590  virtual ErrorCode subscription(std::vector<std::string> &topics) = 0;
2591 
2626  virtual ErrorCode subscribe(const std::vector<std::string> &topics) = 0;
2627 
2629  virtual ErrorCode unsubscribe() = 0;
2630 
2637  virtual ErrorCode assign(const std::vector<TopicPartition *> &partitions) = 0;
2638 
2642  virtual ErrorCode unassign() = 0;
2643 
2668  virtual Message *consume(int timeout_ms) = 0;
2669 
2683  virtual ErrorCode commitSync() = 0;
2684 
2690  virtual ErrorCode commitAsync() = 0;
2691 
2701  virtual ErrorCode commitSync(Message *message) = 0;
2702 
2712  virtual ErrorCode commitAsync(Message *message) = 0;
2713 
2723  virtual ErrorCode commitSync(std::vector<TopicPartition *> &offsets) = 0;
2724 
2735  const std::vector<TopicPartition *> &offsets) = 0;
2736 
2747  virtual ErrorCode commitSync(OffsetCommitCb *offset_commit_cb) = 0;
2748 
2759  virtual ErrorCode commitSync(std::vector<TopicPartition *> &offsets,
2760  OffsetCommitCb *offset_commit_cb) = 0;
2761 
2762 
2763 
2772  virtual ErrorCode committed(std::vector<TopicPartition *> &partitions,
2773  int timeout_ms) = 0;
2774 
2783  virtual ErrorCode position(std::vector<TopicPartition *> &partitions) = 0;
2784 
2785 
2808  virtual ErrorCode close() = 0;
2809 
2810 
2828  virtual ErrorCode seek(const TopicPartition &partition, int timeout_ms) = 0;
2829 
2830 
2848  virtual ErrorCode offsets_store(std::vector<TopicPartition *> &offsets) = 0;
2849 
2850 
2862 
2863 
2878  virtual bool assignment_lost() = 0;
2879 
2895  virtual std::string rebalance_protocol() = 0;
2896 
2897 
2914  const std::vector<TopicPartition *> &partitions) = 0;
2915 
2916 
2933  const std::vector<TopicPartition *> &partitions) = 0;
2934 
2952  virtual Error *close(Queue *queue) = 0;
2953 
2954 
2959  virtual bool closed() = 0;
2960 };
2961 
2962 
2977 class RD_EXPORT Consumer : public virtual Handle {
2978  public:
2989  static Consumer *create(const Conf *conf, std::string &errstr);
2990 
2991  virtual ~Consumer() = 0;
2992 
2993 
3013  virtual ErrorCode start(Topic *topic, int32_t partition, int64_t offset) = 0;
3014 
3021  virtual ErrorCode start(Topic *topic,
3022  int32_t partition,
3023  int64_t offset,
3024  Queue *queue) = 0;
3025 
3035  virtual ErrorCode stop(Topic *topic, int32_t partition) = 0;
3036 
3051  virtual ErrorCode seek(Topic *topic,
3052  int32_t partition,
3053  int64_t offset,
3054  int timeout_ms) = 0;
3055 
3073  virtual Message *consume(Topic *topic, int32_t partition, int timeout_ms) = 0;
3074 
3096  virtual Message *consume(Queue *queue, int timeout_ms) = 0;
3097 
3117  virtual int consume_callback(Topic *topic,
3118  int32_t partition,
3119  int timeout_ms,
3120  ConsumeCb *consume_cb,
3121  void *opaque) = 0;
3122 
3129  virtual int consume_callback(Queue *queue,
3130  int timeout_ms,
3131  RdKafka::ConsumeCb *consume_cb,
3132  void *opaque) = 0;
3133 
3143  static int64_t OffsetTail(int64_t offset);
3144 };
3145 
3159 class RD_EXPORT Producer : public virtual Handle {
3160  public:
3171  static Producer *create(const Conf *conf, std::string &errstr);
3172 
3173 
3174  virtual ~Producer() = 0;
3175 
3181  enum {
3182  RK_MSG_FREE = 0x1,
3185  RK_MSG_COPY = 0x2,
3190  RK_MSG_BLOCK = 0x4
3207  /* For backwards compatibility: */
3208 #ifndef MSG_COPY /* defined in sys/msg.h */
3209  ,
3212  MSG_FREE = RK_MSG_FREE,
3213  MSG_COPY = RK_MSG_COPY
3214 #endif
3216  };
3217 
3274  virtual ErrorCode produce(Topic *topic,
3275  int32_t partition,
3276  int msgflags,
3277  void *payload,
3278  size_t len,
3279  const std::string *key,
3280  void *msg_opaque) = 0;
3281 
3286  virtual ErrorCode produce(Topic *topic,
3287  int32_t partition,
3288  int msgflags,
3289  void *payload,
3290  size_t len,
3291  const void *key,
3292  size_t key_len,
3293  void *msg_opaque) = 0;
3294 
3301  virtual ErrorCode produce(const std::string topic_name,
3302  int32_t partition,
3303  int msgflags,
3304  void *payload,
3305  size_t len,
3306  const void *key,
3307  size_t key_len,
3308  int64_t timestamp,
3309  void *msg_opaque) = 0;
3310 
3318  virtual ErrorCode produce(const std::string topic_name,
3319  int32_t partition,
3320  int msgflags,
3321  void *payload,
3322  size_t len,
3323  const void *key,
3324  size_t key_len,
3325  int64_t timestamp,
3326  RdKafka::Headers *headers,
3327  void *msg_opaque) = 0;
3328 
3329 
3334  virtual ErrorCode produce(Topic *topic,
3335  int32_t partition,
3336  const std::vector<char> *payload,
3337  const std::vector<char> *key,
3338  void *msg_opaque) = 0;
3339 
3340 
3356  virtual ErrorCode flush(int timeout_ms) = 0;
3357 
3358 
3386  virtual ErrorCode purge(int purge_flags) = 0;
3387 
3391  enum {
3392  PURGE_QUEUE = 0x1,
3394  PURGE_INFLIGHT = 0x2,
3401  PURGE_NON_BLOCKING = 0x4 /* Don't wait for background queue
3402  * purging to finish. */
3403  };
3404 
3431  virtual Error *init_transactions(int timeout_ms) = 0;
3432 
3433 
3446  virtual Error *begin_transaction() = 0;
3447 
3495  const std::vector<TopicPartition *> &offsets,
3496  const ConsumerGroupMetadata *group_metadata,
3497  int timeout_ms) = 0;
3498 
3527  virtual Error *commit_transaction(int timeout_ms) = 0;
3528 
3559  virtual Error *abort_transaction(int timeout_ms) = 0;
3560 
3562 };
3563 
3578  public:
3580  virtual int32_t id() const = 0;
3581 
3583  virtual const std::string host() const = 0;
3584 
3586  virtual int port() const = 0;
3587 
3588  virtual ~BrokerMetadata() = 0;
3589 };
3590 
3591 
3592 
3597  public:
3599  typedef std::vector<int32_t> ReplicasVector;
3601  typedef std::vector<int32_t> ISRSVector;
3602 
3604  typedef ReplicasVector::const_iterator ReplicasIterator;
3606  typedef ISRSVector::const_iterator ISRSIterator;
3607 
3608 
3610  virtual int32_t id() const = 0;
3611 
3613  virtual ErrorCode err() const = 0;
3614 
3616  virtual int32_t leader() const = 0;
3617 
3619  virtual const std::vector<int32_t> *replicas() const = 0;
3620 
3624  virtual const std::vector<int32_t> *isrs() const = 0;
3625 
3626  virtual ~PartitionMetadata() = 0;
3627 };
3628 
3629 
3630 
3635  public:
3637  typedef std::vector<const PartitionMetadata *> PartitionMetadataVector;
3639  typedef PartitionMetadataVector::const_iterator PartitionMetadataIterator;
3640 
3642  virtual const std::string topic() const = 0;
3643 
3645  virtual const PartitionMetadataVector *partitions() const = 0;
3646 
3648  virtual ErrorCode err() const = 0;
3649 
3650  virtual ~TopicMetadata() = 0;
3651 };
3652 
3653 
3657 class Metadata {
3658  public:
3660  typedef std::vector<const BrokerMetadata *> BrokerMetadataVector;
3662  typedef std::vector<const TopicMetadata *> TopicMetadataVector;
3663 
3665  typedef BrokerMetadataVector::const_iterator BrokerMetadataIterator;
3667  typedef TopicMetadataVector::const_iterator TopicMetadataIterator;
3668 
3669 
3675  virtual const BrokerMetadataVector *brokers() const = 0;
3676 
3682  virtual const TopicMetadataVector *topics() const = 0;
3683 
3685  virtual int32_t orig_broker_id() const = 0;
3686 
3688  virtual const std::string orig_broker_name() const = 0;
3689 
3690  virtual ~Metadata() = 0;
3691 };
3692 
3695 } // namespace RdKafka
3696 
3697 
3698 #endif /* _RDKAFKACPP_H_ */
Metadata: Broker information.
Definition: rdkafkacpp.h:3577
virtual const std::string host() const =0
virtual int port() const =0
virtual int32_t id() const =0
Configuration interface.
Definition: rdkafkacpp.h:1188
virtual Conf::ConfResult get(OpenCb *&open_cb) const =0
Query single configuration value.
ConfType
Configuration object type.
Definition: rdkafkacpp.h:1193
@ CONF_GLOBAL
Definition: rdkafkacpp.h:1194
virtual Conf::ConfResult set(const std::string &name, EventCb *event_cb, std::string &errstr)=0
Use with name = "event_cb".
virtual struct rd_kafka_topic_conf_s * c_ptr_topic()=0
Returns the underlying librdkafka C rd_kafka_topic_conf_t handle.
virtual Conf::ConfResult set(const std::string &name, OffsetCommitCb *offset_commit_cb, std::string &errstr)=0
Use with name = "offset_commit_cb".
ConfResult
RdKafka::Conf::Set() result code.
Definition: rdkafkacpp.h:1201
virtual Conf::ConfResult set(const std::string &name, PartitionerKeyPointerCb *partitioner_kp_cb, std::string &errstr)=0
Use with name = "partitioner_key_pointer_cb".
virtual Conf::ConfResult get(EventCb *&event_cb) const =0
Query single configuration value.
virtual Conf::ConfResult set_engine_callback_data(void *value, std::string &errstr)=0
Set callback_data for ssl engine.
static Conf * create(ConfType type)
Create configuration object.
virtual Conf::ConfResult set(const std::string &name, DeliveryReportCb *dr_cb, std::string &errstr)=0
Use with name = "dr_cb".
virtual Conf::ConfResult get(SocketCb *&socket_cb) const =0
Query single configuration value.
virtual Conf::ConfResult set(const std::string &name, const Conf *topic_conf, std::string &errstr)=0
Use with name = "default_topic_conf".
virtual Conf::ConfResult get(OAuthBearerTokenRefreshCb *&oauthbearer_token_refresh_cb) const =0
Query single configuration value.
virtual Conf::ConfResult set(const std::string &name, RebalanceCb *rebalance_cb, std::string &errstr)=0
Use with name = "rebalance_cb".
virtual Conf::ConfResult set(const std::string &name, ConsumeCb *consume_cb, std::string &errstr)=0
Use with name = "consume_cb".
virtual Conf::ConfResult get(RebalanceCb *&rebalance_cb) const =0
Query single configuration value.
virtual Conf::ConfResult get(PartitionerCb *&partitioner_cb) const =0
Query single configuration value.
virtual Conf::ConfResult get(OffsetCommitCb *&offset_commit_cb) const =0
Query single configuration value.
virtual Conf::ConfResult get(PartitionerKeyPointerCb *&partitioner_kp_cb) const =0
Query single configuration value.
virtual std::list< std::string > * dump()=0
Dump configuration names and values to list containing name,value tuples.
virtual Conf::ConfResult set_ssl_cert(RdKafka::CertificateType cert_type, RdKafka::CertificateEncoding cert_enc, const void *buffer, size_t size, std::string &errstr)=0
Set certificate/key cert_type from the cert_enc encoded memory at buffer of size bytes.
virtual Conf::ConfResult set(const std::string &name, SocketCb *socket_cb, std::string &errstr)=0
Use with name = "socket_cb".
virtual Conf::ConfResult set(const std::string &name, OpenCb *open_cb, std::string &errstr)=0
Use with name = "open_cb".
virtual Conf::ConfResult enable_sasl_queue(bool enable, std::string &errstr)=0
Enable/disable creation of a queue specific to SASL events and callbacks.
virtual Conf::ConfResult set(const std::string &name, SslCertificateVerifyCb *ssl_cert_verify_cb, std::string &errstr)=0
Use with name = "ssl_cert_verify_cb".
virtual Conf::ConfResult set(const std::string &name, OAuthBearerTokenRefreshCb *oauthbearer_token_refresh_cb, std::string &errstr)=0
Use with name = "oauthbearer_token_refresh_cb".
virtual Conf::ConfResult set(const std::string &name, const std::string &value, std::string &errstr)=0
Set configuration property name to value value.
virtual Conf::ConfResult get(SslCertificateVerifyCb *&ssl_cert_verify_cb) const =0
Use with name = "ssl_cert_verify_cb".
virtual struct rd_kafka_conf_s * c_ptr_global()=0
Returns the underlying librdkafka C rd_kafka_conf_t handle.
virtual Conf::ConfResult set(const std::string &name, PartitionerCb *partitioner_cb, std::string &errstr)=0
Use with name = "partitioner_cb".
virtual Conf::ConfResult get(DeliveryReportCb *&dr_cb) const =0
Query single configuration value.
virtual Conf::ConfResult get(const std::string &name, std::string &value) const =0
Query single configuration value.
Consume callback class.
Definition: rdkafkacpp.h:936
virtual void consume_cb(Message &message, void *opaque)=0
The consume callback is used with RdKafka::Consumer::consume_callback() methods and will be called fo...
ConsumerGroupMetadata holds a consumer instance's group metadata state.
Definition: rdkafkacpp.h:2543
Simple Consumer (legacy)
Definition: rdkafkacpp.h:2977
virtual ErrorCode start(Topic *topic, int32_t partition, int64_t offset, Queue *queue)=0
Start consuming messages for topic and partition on queue queue.
virtual Message * consume(Topic *topic, int32_t partition, int timeout_ms)=0
Consume a single message from topic and partition.
virtual ErrorCode stop(Topic *topic, int32_t partition)=0
Stop consuming messages for topic and partition, purging all messages currently in the local queue.
virtual int consume_callback(Topic *topic, int32_t partition, int timeout_ms, ConsumeCb *consume_cb, void *opaque)=0
Consumes messages from topic and partition, calling the provided callback for each consumed messsage.
virtual Message * consume(Queue *queue, int timeout_ms)=0
Consume a single message from the specified queue.
static int64_t OffsetTail(int64_t offset)
Converts an offset into the logical offset from the tail of a topic.
virtual int consume_callback(Queue *queue, int timeout_ms, RdKafka::ConsumeCb *consume_cb, void *opaque)=0
Consumes messages from queue, calling the provided callback for each consumed messsage.
static Consumer * create(const Conf *conf, std::string &errstr)
Creates a new Kafka consumer handle.
virtual ErrorCode seek(Topic *topic, int32_t partition, int64_t offset, int timeout_ms)=0
Seek consumer for topic+partition to offset which is either an absolute or logical offset.
virtual ErrorCode start(Topic *topic, int32_t partition, int64_t offset)=0
Start consuming messages for topic and partition at offset offset which may either be a proper offset...
Delivery Report callback class.
Definition: rdkafkacpp.h:698
virtual void dr_cb(Message &message)=0
Delivery report callback.
The Error class is used as a return value from APIs to propagate an error. The error consists of an e...
Definition: rdkafkacpp.h:613
virtual ErrorCode code() const =0
virtual std::string name() const =0
virtual std::string str() const =0
virtual bool is_fatal() const =0
static Error * create(ErrorCode code, const std::string *errstr)
Create error object.
virtual bool is_retriable() const =0
virtual bool txn_requires_abort() const =0
Event callback class.
Definition: rdkafkacpp.h:824
virtual void event_cb(Event &event)=0
Event callback.
Event object class as passed to the EventCb callback.
Definition: rdkafkacpp.h:841
virtual int throttle_time() const =0
virtual std::string broker_name() const =0
virtual bool fatal() const =0
virtual std::string fac() const =0
virtual ErrorCode err() const =0
virtual Type type() const =0
virtual std::string str() const =0
Severity
EVENT_LOG severities (conforms to syslog(3) severities)
Definition: rdkafkacpp.h:852
Type
Event type.
Definition: rdkafkacpp.h:844
@ EVENT_ERROR
Definition: rdkafkacpp.h:845
@ EVENT_STATS
Definition: rdkafkacpp.h:846
@ EVENT_LOG
Definition: rdkafkacpp.h:847
virtual Severity severity() const =0
virtual int broker_id() const =0
Base handle, super class for specific clients.
Definition: rdkafkacpp.h:1502
virtual Queue * get_partition_queue(const TopicPartition *partition)=0
Retrieve queue for a given partition.
virtual ErrorCode offsetsForTimes(std::vector< TopicPartition * > &offsets, int timeout_ms)=0
Look up the offsets for the given partitions by timestamp.
virtual ErrorCode oauthbearer_set_token(const std::string &token_value, int64_t md_lifetime_ms, const std::string &md_principal_name, const std::list< std::string > &extensions, std::string &errstr)=0
Set SASL/OAUTHBEARER token and metadata.
virtual ErrorCode resume(std::vector< TopicPartition * > &partitions)=0
Resume producing or consumption for the provided list of partitions.
virtual Error * sasl_background_callbacks_enable()=0
Enable SASL OAUTHBEARER refresh callbacks on the librdkafka background thread.
virtual void mem_free(void *ptr)=0
Free pointer returned by librdkafka.
virtual const std::string clusterid(int timeout_ms)=0
Returns the ClusterId as reported in broker metadata.
virtual int outq_len()=0
Returns the current out queue length.
virtual void * mem_malloc(size_t size)=0
Allocate memory using the same allocator librdkafka uses.
virtual void yield()=0
Cancels the current callback dispatcher (Handle::poll(), KafkaConsumer::consume(),...
virtual ErrorCode pause(std::vector< TopicPartition * > &partitions)=0
Pause producing or consumption for the provided list of partitions.
virtual int32_t controllerid(int timeout_ms)=0
Returns the current ControllerId (controller broker id) as reported in broker metadata.
virtual struct rd_kafka_s * c_ptr()=0
Returns the underlying librdkafka C rd_kafka_t handle.
virtual Queue * get_sasl_queue()=0
virtual int poll(int timeout_ms)=0
Polls the provided kafka handle for events.
virtual ErrorCode oauthbearer_set_token_failure(const std::string &errstr)=0
SASL/OAUTHBEARER token refresh failure indicator.
virtual ErrorCode query_watermark_offsets(const std::string &topic, int32_t partition, int64_t *low, int64_t *high, int timeout_ms)=0
Query broker for low (oldest/beginning) and high (newest/end) offsets for partition.
virtual ErrorCode get_watermark_offsets(const std::string &topic, int32_t partition, int64_t *low, int64_t *high)=0
Get last known low (oldest/beginning) and high (newest/end) offsets for partition.
virtual const std::string name() const =0
virtual ErrorCode fatal_error(std::string &errstr) const =0
Returns the first fatal error set on this client instance, or ERR_NO_ERROR if no fatal error has occu...
virtual ErrorCode metadata(bool all_topics, const Topic *only_rkt, Metadata **metadatap, int timeout_ms)=0
Request Metadata from broker.
virtual ErrorCode set_log_queue(Queue *queue)=0
Forward librdkafka logs (and debug) to the specified queue for serving with one of the ....
virtual const std::string memberid() const =0
Returns the client's broker-assigned group member id.
virtual Queue * get_background_queue()=0
Header object.
Definition: rdkafkacpp.h:2097
size_t value_size() const
Definition: rdkafkacpp.h:2190
Header(const std::string &key, const void *value, size_t value_size)
Header object to encapsulate a single Header.
Definition: rdkafkacpp.h:2109
std::string key() const
Definition: rdkafkacpp.h:2174
const char * value_string() const
Definition: rdkafkacpp.h:2185
Header(const std::string &key, const void *value, size_t value_size, const RdKafka::ErrorCode err)
Header object to encapsulate a single Header.
Definition: rdkafkacpp.h:2127
const void * value() const
Definition: rdkafkacpp.h:2179
Header(const Header &other)
Copy constructor.
Definition: rdkafkacpp.h:2141
RdKafka::ErrorCode err() const
Definition: rdkafkacpp.h:2195
Header & operator=(const Header &other)
Assignment operator.
Definition: rdkafkacpp.h:2151
Headers object.
Definition: rdkafkacpp.h:2085
virtual std::vector< Header > get(const std::string &key) const =0
Gets all of the Headers of a given key.
virtual std::vector< Header > get_all() const =0
Returns all Headers.
virtual ErrorCode remove(const std::string &key)=0
Removes all the Headers of a given key.
static Headers * create()
Create a new instance of the Headers object.
virtual ErrorCode add(const std::string &key, const void *value, size_t value_size)=0
Adds a Header to the end of the list.
virtual ErrorCode add(const Header &header)=0
Adds a Header to the end of the list.
virtual ErrorCode add(const std::string &key, const std::string &value)=0
Adds a Header to the end of the list.
virtual Header get_last(const std::string &key) const =0
Gets the last occurrence of a Header of a given key.
static Headers * create(const std::vector< Header > &headers)
Create a new instance of the Headers object from a std::vector.
virtual size_t size() const =0
High-level KafkaConsumer (for brokers 0.9 and later)
Definition: rdkafkacpp.h:2565
virtual ErrorCode commitSync()=0
Commit offsets for the current assignment.
static KafkaConsumer * create(const Conf *conf, std::string &errstr)
Creates a KafkaConsumer.
virtual ErrorCode unassign()=0
Stop consumption and remove the current assignment.
virtual ErrorCode commitAsync(Message *message)=0
Commit offset for a single topic+partition based on message.
virtual Error * incremental_assign(const std::vector< TopicPartition * > &partitions)=0
Incrementally add partitions to the current assignment.
virtual ErrorCode subscription(std::vector< std::string > &topics)=0
Returns the current subscription as set by RdKafka::KafkaConsumer::subscribe()
virtual ErrorCode position(std::vector< TopicPartition * > &partitions)=0
Retrieve current positions (offsets) for topics+partitions.
virtual std::string rebalance_protocol()=0
The rebalance protocol currently in use. This will be "NONE" if the consumer has not (yet) joined a g...
virtual ErrorCode close()=0
Close and shut down the consumer.
virtual ErrorCode commitAsync(const std::vector< TopicPartition * > &offsets)=0
Commit offset for the provided list of partitions.
virtual ErrorCode commitSync(OffsetCommitCb *offset_commit_cb)=0
Commit offsets for the current assignment.
virtual bool assignment_lost()=0
Check whether the consumer considers the current assignment to have been lost involuntarily....
virtual ErrorCode commitAsync()=0
Asynchronous version of RdKafka::KafkaConsumer::CommitSync()
virtual ErrorCode subscribe(const std::vector< std::string > &topics)=0
Update the subscription set to topics.
virtual ConsumerGroupMetadata * groupMetadata()=0
virtual Error * close(Queue *queue)=0
Close and shut down the consumer.
virtual bool closed()=0
virtual Message * consume(int timeout_ms)=0
Consume message or get error event, triggers callbacks.
virtual ErrorCode commitSync(std::vector< TopicPartition * > &offsets)=0
Commit offsets for the provided list of partitions.
virtual ErrorCode unsubscribe()=0
Unsubscribe from the current subscription set.
virtual ErrorCode commitSync(std::vector< TopicPartition * > &offsets, OffsetCommitCb *offset_commit_cb)=0
Commit offsets for the provided list of partitions.
virtual ErrorCode committed(std::vector< TopicPartition * > &partitions, int timeout_ms)=0
Retrieve committed offsets for topics+partitions.
virtual ErrorCode seek(const TopicPartition &partition, int timeout_ms)=0
Seek consumer for topic+partition to offset which is either an absolute or logical offset.
virtual ErrorCode offsets_store(std::vector< TopicPartition * > &offsets)=0
Store offset offset for topic partition partition. The offset will be committed (written) to the offs...
virtual ErrorCode commitSync(Message *message)=0
Commit offset for a single topic+partition based on message.
virtual Error * incremental_unassign(const std::vector< TopicPartition * > &partitions)=0
Incrementally remove partitions from the current assignment.
virtual ErrorCode assign(const std::vector< TopicPartition * > &partitions)=0
Update the assignment set to partitions.
virtual ErrorCode assignment(std::vector< RdKafka::TopicPartition * > &partitions)=0
Returns the current partition assignment as set by RdKafka::KafkaConsumer::assign()
Message timestamp object.
Definition: rdkafkacpp.h:2062
int64_t timestamp
Definition: rdkafkacpp.h:2072
MessageTimestampType
Definition: rdkafkacpp.h:2065
@ MSG_TIMESTAMP_CREATE_TIME
Definition: rdkafkacpp.h:2067
@ MSG_TIMESTAMP_NOT_AVAILABLE
Definition: rdkafkacpp.h:2066
MessageTimestampType type
Definition: rdkafkacpp.h:2071
Message object.
Definition: rdkafkacpp.h:2328
virtual std::string topic_name() const =0
virtual Status status() const =0
Returns the message's persistence status in the topic log.
virtual const std::string * key() const =0
virtual std::string errstr() const =0
Accessor functions*.
virtual int32_t broker_id() const =0
virtual const void * key_pointer() const =0
virtual RdKafka::Headers * headers()=0
virtual int64_t offset() const =0
virtual size_t len() const =0
virtual struct rd_kafka_message_s * c_ptr()=0
Returns the underlying librdkafka C rd_kafka_message_t handle.
virtual Topic * topic() const =0
virtual MessageTimestamp timestamp() const =0
virtual int64_t latency() const =0
virtual ErrorCode err() const =0
Status
Message persistence status can be used by the application to find out if a produced message was persi...
Definition: rdkafkacpp.h:2332
virtual size_t key_len() const =0
virtual RdKafka::Headers * headers(RdKafka::ErrorCode *err)=0
virtual void * msg_opaque() const =0
virtual void * payload() const =0
virtual int32_t partition() const =0
Metadata container.
Definition: rdkafkacpp.h:3657
virtual const TopicMetadataVector * topics() const =0
Topic list.
virtual int32_t orig_broker_id() const =0
Broker (id) originating this metadata.
virtual const BrokerMetadataVector * brokers() const =0
Broker list.
std::vector< const TopicMetadata * > TopicMetadataVector
Topics.
Definition: rdkafkacpp.h:3662
virtual const std::string orig_broker_name() const =0
Broker (name) originating this metadata.
std::vector< const BrokerMetadata * > BrokerMetadataVector
Brokers.
Definition: rdkafkacpp.h:3660
BrokerMetadataVector::const_iterator BrokerMetadataIterator
Brokers iterator.
Definition: rdkafkacpp.h:3665
TopicMetadataVector::const_iterator TopicMetadataIterator
Topics iterator.
Definition: rdkafkacpp.h:3667
SASL/OAUTHBEARER token refresh callback class.
Definition: rdkafkacpp.h:737
virtual void oauthbearer_token_refresh_cb(RdKafka::Handle *handle, const std::string &oauthbearer_config)=0
SASL/OAUTHBEARER token refresh callback class.
Offset Commit callback class.
Definition: rdkafkacpp.h:1037
virtual void offset_commit_cb(RdKafka::ErrorCode err, std::vector< TopicPartition * > &offsets)=0
Set offset commit callback for use with consumer groups.
Portability: OpenCb callback class
Definition: rdkafkacpp.h:1149
virtual int open_cb(const std::string &path, int flags, int mode)=0
Open callback The open callback is responsible for opening the file specified by pathname,...
Metadata: Partition information.
Definition: rdkafkacpp.h:3596
std::vector< int32_t > ISRSVector
ISRs (In-Sync-Replicas)
Definition: rdkafkacpp.h:3601
ISRSVector::const_iterator ISRSIterator
ISRs iterator.
Definition: rdkafkacpp.h:3606
virtual int32_t id() const =0
ReplicasVector::const_iterator ReplicasIterator
Replicas iterator.
Definition: rdkafkacpp.h:3604
virtual ErrorCode err() const =0
virtual const std::vector< int32_t > * isrs() const =0
virtual const std::vector< int32_t > * replicas() const =0
virtual int32_t leader() const =0
std::vector< int32_t > ReplicasVector
Replicas.
Definition: rdkafkacpp.h:3599
Partitioner callback class.
Definition: rdkafkacpp.h:762
virtual int32_t partitioner_cb(const Topic *topic, const std::string *key, int32_t partition_cnt, void *msg_opaque)=0
Partitioner callback.
Variant partitioner with key pointer.
Definition: rdkafkacpp.h:794
virtual int32_t partitioner_cb(const Topic *topic, const void *key, size_t key_len, int32_t partition_cnt, void *msg_opaque)=0
Variant partitioner callback that gets key as pointer and length instead of as a const std::string *.
Producer.
Definition: rdkafkacpp.h:3159
virtual Error * abort_transaction(int timeout_ms)=0
Aborts the ongoing transaction.
virtual Error * init_transactions(int timeout_ms)=0
Initialize transactions for the producer instance.
virtual ErrorCode produce(const std::string topic_name, int32_t partition, int msgflags, void *payload, size_t len, const void *key, size_t key_len, int64_t timestamp, void *msg_opaque)=0
produce() variant that takes topic as a string (no need for creating a Topic object),...
virtual Error * commit_transaction(int timeout_ms)=0
Commit the current transaction as started with begin_transaction().
virtual ErrorCode produce(Topic *topic, int32_t partition, int msgflags, void *payload, size_t len, const void *key, size_t key_len, void *msg_opaque)=0
Variant produce() that passes the key as a pointer and length instead of as a const std::string *.
virtual Error * send_offsets_to_transaction(const std::vector< TopicPartition * > &offsets, const ConsumerGroupMetadata *group_metadata, int timeout_ms)=0
Sends a list of topic partition offsets to the consumer group coordinator for group_metadata,...
virtual ErrorCode purge(int purge_flags)=0
Purge messages currently handled by the producer instance.
virtual ErrorCode produce(const std::string topic_name, int32_t partition, int msgflags, void *payload, size_t len, const void *key, size_t key_len, int64_t timestamp, RdKafka::Headers *headers, void *msg_opaque)=0
produce() variant that that allows for Header support on produce Otherwise identical to produce() abo...
virtual ErrorCode produce(Topic *topic, int32_t partition, int msgflags, void *payload, size_t len, const std::string *key, void *msg_opaque)=0
Produce and send a single message to broker.
virtual ErrorCode produce(Topic *topic, int32_t partition, const std::vector< char > *payload, const std::vector< char > *key, void *msg_opaque)=0
Variant produce() that accepts vectors for key and payload. The vector data will be copied.
virtual Error * begin_transaction()=0
init_transactions() must have been called successfully (once) before this function is called.
static Producer * create(const Conf *conf, std::string &errstr)
Creates a new Kafka producer handle.
virtual ErrorCode flush(int timeout_ms)=0
Wait until all outstanding produce requests, et.al, are completed. This should typically be done prio...
Queue interface.
Definition: rdkafkacpp.h:2468
static Queue * create(Handle *handle)
Create Queue object.
virtual ErrorCode forward(Queue *dst)=0
Forward/re-route queue to dst. If dst is NULL, the forwarding is removed.
virtual int poll(int timeout_ms)=0
Poll queue, serving any enqueued callbacks.
virtual void io_event_enable(int fd, const void *payload, size_t size)=0
Enable IO event triggering for queue.
virtual Message * consume(int timeout_ms)=0
Consume message or get error event from the queue.
KafkaConsumer: Rebalance callback class
Definition: rdkafkacpp.h:955
virtual void rebalance_cb(RdKafka::KafkaConsumer *consumer, RdKafka::ErrorCode err, std::vector< TopicPartition * > &partitions)=0
Group rebalance callback for use with RdKafka::KafkaConsumer.
Portability: SocketCb callback class
Definition: rdkafkacpp.h:1123
virtual int socket_cb(int domain, int type, int protocol)=0
Socket callback.
SSL broker certificate verification class.
Definition: rdkafkacpp.h:1068
virtual bool ssl_cert_verify_cb(const std::string &broker_name, int32_t broker_id, int *x509_error, int depth, const char *buf, size_t size, std::string &errstr)=0
SSL broker certificate verification callback.
Metadata: Topic information.
Definition: rdkafkacpp.h:3634
virtual ErrorCode err() const =0
virtual const PartitionMetadataVector * partitions() const =0
std::vector< const PartitionMetadata * > PartitionMetadataVector
Partitions.
Definition: rdkafkacpp.h:3637
virtual const std::string topic() const =0
PartitionMetadataVector::const_iterator PartitionMetadataIterator
Partitions iterator.
Definition: rdkafkacpp.h:3639
Topic+Partition.
Definition: rdkafkacpp.h:1915
static void destroy(std::vector< TopicPartition * > &partitions)
Destroy/delete the TopicPartitions in partitions and clear the vector.
virtual const std::string & topic() const =0
static TopicPartition * create(const std::string &topic, int partition, int64_t offset)
Create topic+partition object for topic and partition with offset offset.
virtual void set_offset(int64_t offset)=0
Set offset.
virtual int partition() const =0
static TopicPartition * create(const std::string &topic, int partition)
Create topic+partition object for topic and partition.
virtual int64_t offset() const =0
virtual ErrorCode err() const =0
Topic handle.
Definition: rdkafkacpp.h:1964
static const int64_t OFFSET_INVALID
Definition: rdkafkacpp.h:1978
virtual ErrorCode offset_store(int32_t partition, int64_t offset)=0
Store offset offset + 1 for topic partition partition. The offset will be committed (written) to the ...
static const int64_t OFFSET_BEGINNING
Special offsets.
Definition: rdkafkacpp.h:1975
static Topic * create(Handle *base, const std::string &topic_str, const Conf *conf, std::string &errstr)
Creates a new topic handle for topic named topic_str.
virtual bool partition_available(int32_t partition) const =0
static const int64_t OFFSET_STORED
Definition: rdkafkacpp.h:1977
virtual struct rd_kafka_topic_s * c_ptr()=0
Returns the underlying librdkafka C rd_kafka_topic_t handle.
static const int32_t PARTITION_UA
Unassigned partition.
Definition: rdkafkacpp.h:1972
static const int64_t OFFSET_END
Definition: rdkafkacpp.h:1976
virtual const std::string name() const =0
RD_EXPORT std::string get_debug_contexts()
Returns a CSV list of the supported debug contexts for use with Conf::Set("debug",...
RD_EXPORT std::string version_str()
Returns the librdkafka version as string.
CertificateEncoding
SSL certificate encoding.
Definition: rdkafkacpp.h:571
@ CERT_ENC_PKCS12
Definition: rdkafkacpp.h:572
@ CERT_ENC_DER
Definition: rdkafkacpp.h:573
@ CERT_ENC_PEM
Definition: rdkafkacpp.h:574
RD_EXPORT int wait_destroyed(int timeout_ms)
Wait for all rd_kafka_t objects to be destroyed.
RD_EXPORT std::string err2str(RdKafka::ErrorCode err)
Returns a human readable representation of a kafka error.
ErrorCode
Error codes.
Definition: rdkafkacpp.h:200
@ ERR__APPLICATION
Definition: rdkafkacpp.h:320
@ ERR_PRINCIPAL_DESERIALIZATION_FAILURE
Definition: rdkafkacpp.h:544
@ ERR_NOT_LEADER_FOR_PARTITION
Definition: rdkafkacpp.h:347
@ ERR_NON_EMPTY_GROUP
Definition: rdkafkacpp.h:482
@ ERR__DESTROY
Definition: rdkafkacpp.h:209
@ ERR_INVALID_REQUEST
Definition: rdkafkacpp.h:425
@ ERR_GROUP_ID_NOT_FOUND
Definition: rdkafkacpp.h:484
@ ERR_INVALID_SESSION_TIMEOUT
Definition: rdkafkacpp.h:393
@ ERR__UNDERFLOW
Definition: rdkafkacpp.h:296
@ ERR__ASSIGN_PARTITIONS
Definition: rdkafkacpp.h:256
@ ERR__UNKNOWN_BROKER
Definition: rdkafkacpp.h:314
@ ERR_CLUSTER_AUTHORIZATION_FAILED
Definition: rdkafkacpp.h:403
@ ERR_UNKNOWN
Definition: rdkafkacpp.h:333
@ ERR_ELIGIBLE_LEADERS_NOT_AVAILABLE
Definition: rdkafkacpp.h:513
@ ERR__REVOKE_PARTITIONS
Definition: rdkafkacpp.h:258
@ ERR__FATAL
Definition: rdkafkacpp.h:306
@ ERR_SASL_AUTHENTICATION_FAILED
Definition: rdkafkacpp.h:462
@ ERR_FEATURE_UPDATE_FAILED
Definition: rdkafkacpp.h:542
@ ERR__CRIT_SYS_RESOURCE
Definition: rdkafkacpp.h:215
@ ERR__ASSIGNMENT_LOST
Definition: rdkafkacpp.h:322
@ ERR_STALE_BROKER_EPOCH
Definition: rdkafkacpp.h:500
@ ERR_INVALID_TXN_STATE
Definition: rdkafkacpp.h:437
@ ERR__UNKNOWN_PARTITION
Definition: rdkafkacpp.h:226
@ ERR__WAIT_CACHE
Definition: rdkafkacpp.h:278
@ ERR_UNSTABLE_OFFSET_COMMIT
Definition: rdkafkacpp.h:524
@ ERR__NODE_UPDATE
Definition: rdkafkacpp.h:242
@ ERR_NO_ERROR
Definition: rdkafkacpp.h:335
@ ERR_INCONSISTENT_GROUP_PROTOCOL
Definition: rdkafkacpp.h:387
@ ERR__UNKNOWN_GROUP
Definition: rdkafkacpp.h:248
@ ERR_INVALID_PARTITIONS
Definition: rdkafkacpp.h:415
@ ERR_DELEGATION_TOKEN_NOT_FOUND
Definition: rdkafkacpp.h:470
@ ERR__NOOP
Definition: rdkafkacpp.h:324
@ ERR_NO_REASSIGNMENT_IN_PROGRESS
Definition: rdkafkacpp.h:517
@ ERR__NOT_IMPLEMENTED
Definition: rdkafkacpp.h:266
@ ERR__MSG_TIMED_OUT
Definition: rdkafkacpp.h:219
@ ERR_INVALID_REPLICATION_FACTOR
Definition: rdkafkacpp.h:417
@ ERR__READ_ONLY
Definition: rdkafkacpp.h:292
@ ERR_UNKNOWN_MEMBER_ID
Definition: rdkafkacpp.h:391
@ ERR_OUT_OF_ORDER_SEQUENCE_NUMBER
Definition: rdkafkacpp.h:431
@ ERR_INVALID_GROUP_ID
Definition: rdkafkacpp.h:389
@ ERR__RETRY
Definition: rdkafkacpp.h:300
@ ERR__TIMED_OUT_QUEUE
Definition: rdkafkacpp.h:274
@ ERR__NOT_CONFIGURED
Definition: rdkafkacpp.h:316
@ ERR_DUPLICATE_RESOURCE
Definition: rdkafkacpp.h:533
@ ERR_LEADER_NOT_AVAILABLE
Definition: rdkafkacpp.h:345
@ ERR_INVALID_FETCH_SESSION_EPOCH
Definition: rdkafkacpp.h:488
@ ERR_KAFKA_STORAGE_ERROR
Definition: rdkafkacpp.h:458
@ ERR_ILLEGAL_GENERATION
Definition: rdkafkacpp.h:385
@ ERR_TOPIC_ALREADY_EXISTS
Definition: rdkafkacpp.h:413
@ ERR__AUTHENTICATION
Definition: rdkafkacpp.h:268
@ ERR_TOPIC_EXCEPTION
Definition: rdkafkacpp.h:375
@ ERR__RESOLVE
Definition: rdkafkacpp.h:217
@ ERR_INVALID_REQUIRED_ACKS
Definition: rdkafkacpp.h:383
@ ERR_FETCH_SESSION_ID_NOT_FOUND
Definition: rdkafkacpp.h:486
@ ERR__SSL
Definition: rdkafkacpp.h:244
@ ERR_COORDINATOR_NOT_AVAILABLE
Definition: rdkafkacpp.h:367
@ ERR__CONFLICT
Definition: rdkafkacpp.h:260
@ ERR_GROUP_SUBSCRIBED_TO_TOPIC
Definition: rdkafkacpp.h:520
@ ERR__STATE
Definition: rdkafkacpp.h:262
@ ERR__KEY_SERIALIZATION
Definition: rdkafkacpp.h:282
@ ERR_INVALID_MSG_SIZE
Definition: rdkafkacpp.h:343
@ ERR_DELEGATION_TOKEN_OWNER_MISMATCH
Definition: rdkafkacpp.h:472
@ ERR_REASSIGNMENT_IN_PROGRESS
Definition: rdkafkacpp.h:466
@ ERR__OUTDATED
Definition: rdkafkacpp.h:272
@ ERR_OFFSET_NOT_AVAILABLE
Definition: rdkafkacpp.h:502
@ ERR_THROTTLING_QUOTA_EXCEEDED
Definition: rdkafkacpp.h:526
@ ERR_REQUEST_TIMED_OUT
Definition: rdkafkacpp.h:349
@ ERR_INVALID_TRANSACTION_TIMEOUT
Definition: rdkafkacpp.h:443
@ ERR_CONCURRENT_TRANSACTIONS
Definition: rdkafkacpp.h:446
@ ERR_STALE_CTRL_EPOCH
Definition: rdkafkacpp.h:357
@ ERR_UNSUPPORTED_VERSION
Definition: rdkafkacpp.h:411
@ ERR__WAIT_COORD
Definition: rdkafkacpp.h:246
@ ERR_ELECTION_NOT_NEEDED
Definition: rdkafkacpp.h:515
@ ERR_INVALID_COMMIT_OFFSET_SIZE
Definition: rdkafkacpp.h:397
@ ERR_DELEGATION_TOKEN_EXPIRED
Definition: rdkafkacpp.h:478
@ ERR_TOPIC_AUTHORIZATION_FAILED
Definition: rdkafkacpp.h:399
@ ERR_MSG_SIZE_TOO_LARGE
Definition: rdkafkacpp.h:355
@ ERR_NETWORK_EXCEPTION
Definition: rdkafkacpp.h:361
@ ERR_INVALID_PRINCIPAL_TYPE
Definition: rdkafkacpp.h:480
@ ERR__MAX_POLL_EXCEEDED
Definition: rdkafkacpp.h:312
@ ERR_FENCED_LEADER_EPOCH
Definition: rdkafkacpp.h:494
@ ERR__BEGIN
Definition: rdkafkacpp.h:203
@ ERR_DELEGATION_TOKEN_AUTHORIZATION_FAILED
Definition: rdkafkacpp.h:476
@ ERR_UNACCEPTABLE_CREDENTIAL
Definition: rdkafkacpp.h:535
@ ERR_RESOURCE_NOT_FOUND
Definition: rdkafkacpp.h:531
@ ERR_GROUP_AUTHORIZATION_FAILED
Definition: rdkafkacpp.h:401
@ ERR_LOG_DIR_NOT_FOUND
Definition: rdkafkacpp.h:460
@ ERR__AUTO_OFFSET_RESET
Definition: rdkafkacpp.h:326
@ ERR__NO_OFFSET
Definition: rdkafkacpp.h:270
@ ERR__PURGE_INFLIGHT
Definition: rdkafkacpp.h:304
@ ERR_INVALID_RECORD
Definition: rdkafkacpp.h:522
@ ERR_FENCED_INSTANCE_ID
Definition: rdkafkacpp.h:511
@ ERR_ILLEGAL_SASL_STATE
Definition: rdkafkacpp.h:409
@ ERR__FAIL
Definition: rdkafkacpp.h:211
@ ERR_MEMBER_ID_REQUIRED
Definition: rdkafkacpp.h:504
@ ERR__EXISTING_SUBSCRIPTION
Definition: rdkafkacpp.h:254
@ ERR__PREV_IN_PROGRESS
Definition: rdkafkacpp.h:252
@ ERR_PRODUCER_FENCED
Definition: rdkafkacpp.h:529
@ ERR__TIMED_OUT
Definition: rdkafkacpp.h:236
@ ERR_PREFERRED_LEADER_NOT_AVAILABLE
Definition: rdkafkacpp.h:506
@ ERR__PARTITION_EOF
Definition: rdkafkacpp.h:224
@ ERR_NOT_ENOUGH_REPLICAS_AFTER_APPEND
Definition: rdkafkacpp.h:381
@ ERR__QUEUE_FULL
Definition: rdkafkacpp.h:238
@ ERR_INVALID_MSG
Definition: rdkafkacpp.h:339
@ ERR_TOPIC_DELETION_DISABLED
Definition: rdkafkacpp.h:492
@ ERR__VALUE_DESERIALIZATION
Definition: rdkafkacpp.h:288
@ ERR_DELEGATION_TOKEN_REQUEST_NOT_ALLOWED
Definition: rdkafkacpp.h:474
@ ERR__GAPLESS_GUARANTEE
Definition: rdkafkacpp.h:310
@ ERR_NOT_COORDINATOR
Definition: rdkafkacpp.h:371
@ ERR__UNSUPPORTED_FEATURE
Definition: rdkafkacpp.h:276
@ ERR_INVALID_PRODUCER_EPOCH
Definition: rdkafkacpp.h:435
@ ERR_DUPLICATE_SEQUENCE_NUMBER
Definition: rdkafkacpp.h:433
@ ERR_OFFSET_METADATA_TOO_LARGE
Definition: rdkafkacpp.h:359
@ ERR_NOT_ENOUGH_REPLICAS
Definition: rdkafkacpp.h:379
@ ERR_UNSUPPORTED_SASL_MECHANISM
Definition: rdkafkacpp.h:407
@ ERR__INCONSISTENT
Definition: rdkafkacpp.h:308
@ ERR_UNKNOWN_PRODUCER_ID
Definition: rdkafkacpp.h:464
@ ERR_TRANSACTIONAL_ID_AUTHORIZATION_FAILED
Definition: rdkafkacpp.h:452
@ ERR_UNSUPPORTED_COMPRESSION_TYPE
Definition: rdkafkacpp.h:498
@ ERR_UNKNOWN_LEADER_EPOCH
Definition: rdkafkacpp.h:496
@ ERR_LISTENER_NOT_FOUND
Definition: rdkafkacpp.h:490
@ ERR__PURGE_QUEUE
Definition: rdkafkacpp.h:302
@ ERR_SECURITY_DISABLED
Definition: rdkafkacpp.h:454
@ ERR__ISR_INSUFF
Definition: rdkafkacpp.h:240
@ ERR__FENCED
Definition: rdkafkacpp.h:318
@ ERR_INCONSISTENT_VOTER_SET
Definition: rdkafkacpp.h:538
@ ERR__BAD_COMPRESSION
Definition: rdkafkacpp.h:207
@ ERR__ALL_BROKERS_DOWN
Definition: rdkafkacpp.h:232
@ ERR__VALUE_SERIALIZATION
Definition: rdkafkacpp.h:284
@ ERR_OPERATION_NOT_ATTEMPTED
Definition: rdkafkacpp.h:456
@ ERR__END
Definition: rdkafkacpp.h:329
@ ERR__IN_PROGRESS
Definition: rdkafkacpp.h:250
@ ERR_INVALID_CONFIG
Definition: rdkafkacpp.h:421
@ ERR_REPLICA_NOT_AVAILABLE
Definition: rdkafkacpp.h:353
@ ERR_INVALID_REPLICA_ASSIGNMENT
Definition: rdkafkacpp.h:419
@ ERR_BROKER_NOT_AVAILABLE
Definition: rdkafkacpp.h:351
@ ERR__UNKNOWN_PROTOCOL
Definition: rdkafkacpp.h:264
@ ERR__BAD_MSG
Definition: rdkafkacpp.h:205
@ ERR__NOENT
Definition: rdkafkacpp.h:294
@ ERR_REBALANCE_IN_PROGRESS
Definition: rdkafkacpp.h:395
@ ERR_COORDINATOR_LOAD_IN_PROGRESS
Definition: rdkafkacpp.h:363
@ ERR_RECORD_LIST_TOO_LARGE
Definition: rdkafkacpp.h:377
@ ERR_INVALID_PRODUCER_ID_MAPPING
Definition: rdkafkacpp.h:440
@ ERR__PARTIAL
Definition: rdkafkacpp.h:290
@ ERR_TRANSACTION_COORDINATOR_FENCED
Definition: rdkafkacpp.h:450
@ ERR_UNKNOWN_TOPIC_OR_PART
Definition: rdkafkacpp.h:341
@ ERR__UNKNOWN_TOPIC
Definition: rdkafkacpp.h:230
@ ERR__INVALID_ARG
Definition: rdkafkacpp.h:234
@ ERR_UNSUPPORTED_FOR_MESSAGE_FORMAT
Definition: rdkafkacpp.h:427
@ ERR_INVALID_TIMESTAMP
Definition: rdkafkacpp.h:405
@ ERR__INVALID_TYPE
Definition: rdkafkacpp.h:298
@ ERR_OFFSET_OUT_OF_RANGE
Definition: rdkafkacpp.h:337
@ ERR_INVALID_UPDATE_VERSION
Definition: rdkafkacpp.h:540
@ ERR_DELEGATION_TOKEN_AUTH_DISABLED
Definition: rdkafkacpp.h:468
@ ERR_POLICY_VIOLATION
Definition: rdkafkacpp.h:429
@ ERR__KEY_DESERIALIZATION
Definition: rdkafkacpp.h:286
@ ERR__FS
Definition: rdkafkacpp.h:228
@ ERR_GROUP_MAX_SIZE_REACHED
Definition: rdkafkacpp.h:508
@ ERR_NOT_CONTROLLER
Definition: rdkafkacpp.h:423
@ ERR__INTR
Definition: rdkafkacpp.h:280
@ ERR__TRANSPORT
Definition: rdkafkacpp.h:213
RD_EXPORT int version()
Returns the librdkafka version as integer.
RD_EXPORT void * mem_malloc(size_t size)
Allocate memory using the same allocator librdkafka uses.
CertificateType
SSL certificate types.
Definition: rdkafkacpp.h:560
@ CERT_PRIVATE_KEY
Definition: rdkafkacpp.h:562
@ CERT_PUBLIC_KEY
Definition: rdkafkacpp.h:561
@ CERT_CA
Definition: rdkafkacpp.h:563
RD_EXPORT void mem_free(void *ptr)
Free pointer returned by librdkafka.