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 #pragma once
30 
49 #include <string>
50 #include <list>
51 #include <vector>
52 #include <stdint.h>
53 
54 
55 #ifdef _MSC_VER
56 #undef RD_EXPORT
57 #ifdef LIBRDKAFKACPP_EXPORTS
58 #define RD_EXPORT __declspec(dllexport)
59 #else
60 #define RD_EXPORT __declspec(dllimport)
61 #endif
62 #else
63 #define RD_EXPORT
64 #endif
65 
68 namespace RdKafka {
69 
70 
90 #define RD_KAFKA_VERSION 0x000902ff
91 
97 RD_EXPORT
98 int version ();
99 
103 RD_EXPORT
104 std::string version_str();
105 
110 RD_EXPORT
111 std::string get_debug_contexts();
112 
122 RD_EXPORT
123 int wait_destroyed(int timeout_ms);
124 
125 
148 enum ErrorCode {
149  /* Internal errors to rdkafka: */
151  ERR__BEGIN = -200,
153  ERR__BAD_MSG = -199,
157  ERR__DESTROY = -197,
159  ERR__FAIL = -196,
165  ERR__RESOLVE = -193,
174  ERR__FS = -189,
190  ERR__SSL = -181,
208  ERR__STATE = -172,
221 
223  ERR__END = -100,
224 
225  /* Kafka broker errors: */
292 };
293 
294 
298 RD_EXPORT
299 std::string err2str(RdKafka::ErrorCode err);
300 
301 
307 /* Forward declarations */
308 class Producer;
309 class Message;
310 class Event;
311 class Topic;
312 class TopicPartition;
313 class Metadata;
314 class KafkaConsumer;
346 class RD_EXPORT DeliveryReportCb {
347  public:
351  virtual void dr_cb (Message &message) = 0;
352 
353  virtual ~DeliveryReportCb() { }
354 };
355 
356 
364 class RD_EXPORT PartitionerCb {
365  public:
382  virtual int32_t partitioner_cb (const Topic *topic,
383  const std::string *key,
384  int32_t partition_cnt,
385  void *msg_opaque) = 0;
386 
387  virtual ~PartitionerCb() { }
388 };
389 
395  public:
404  virtual int32_t partitioner_cb (const Topic *topic,
405  const void *key,
406  size_t key_len,
407  int32_t partition_cnt,
408  void *msg_opaque) = 0;
409 
410  virtual ~PartitionerKeyPointerCb() { }
411 };
412 
413 
414 
423 class RD_EXPORT EventCb {
424  public:
430  virtual void event_cb (Event &event) = 0;
431 
432  virtual ~EventCb() { }
433 };
434 
435 
439 class RD_EXPORT Event {
440  public:
442  enum Type {
446  EVENT_THROTTLE
447  };
448 
450  enum Severity {
451  EVENT_SEVERITY_EMERG = 0,
452  EVENT_SEVERITY_ALERT = 1,
453  EVENT_SEVERITY_CRITICAL = 2,
454  EVENT_SEVERITY_ERROR = 3,
455  EVENT_SEVERITY_WARNING = 4,
456  EVENT_SEVERITY_NOTICE = 5,
457  EVENT_SEVERITY_INFO = 6,
458  EVENT_SEVERITY_DEBUG = 7
459  };
460 
461  virtual ~Event () { }
462 
463  /*
464  * Event Accessor methods
465  */
466 
471  virtual Type type () const = 0;
472 
477  virtual ErrorCode err () const = 0;
478 
483  virtual Severity severity () const = 0;
484 
489  virtual std::string fac () const = 0;
490 
499  virtual std::string str () const = 0;
500 
505  virtual int throttle_time () const = 0;
506 
511  virtual std::string broker_name () const = 0;
512 
517  virtual int broker_id () const = 0;
518 };
519 
520 
521 
525 class RD_EXPORT ConsumeCb {
526  public:
534  virtual void consume_cb (Message &message, void *opaque) = 0;
535 
536  virtual ~ConsumeCb() { }
537 };
538 
539 
543 class RD_EXPORT RebalanceCb {
544 public:
594  virtual void rebalance_cb (RdKafka::KafkaConsumer *consumer,
595  RdKafka::ErrorCode err,
596  std::vector<TopicPartition*>&partitions) = 0;
597 
598  virtual ~RebalanceCb() { }
599 };
600 
601 
605 class RD_EXPORT OffsetCommitCb {
606 public:
623  virtual void offset_commit_cb(RdKafka::ErrorCode err,
624  std::vector<TopicPartition*>&offsets) = 0;
625 
626  virtual ~OffsetCommitCb() { }
627 };
628 
629 
630 
635 class RD_EXPORT SocketCb {
636  public:
650  virtual int socket_cb (int domain, int type, int protocol) = 0;
651 
652  virtual ~SocketCb() { }
653 };
654 
655 
660 class RD_EXPORT OpenCb {
661  public:
673  virtual int open_cb (const std::string &path, int flags, int mode) = 0;
674 
675  virtual ~OpenCb() { }
676 };
677 
678 
699 class RD_EXPORT Conf {
700  public:
704  enum ConfType {
706  CONF_TOPIC
707  };
708 
712  enum ConfResult {
713  CONF_UNKNOWN = -2,
714  CONF_INVALID = -1,
715  CONF_OK = 0
716  };
717 
718 
722  static Conf *create (ConfType type);
723 
724  virtual ~Conf () { }
725 
731  virtual Conf::ConfResult set (const std::string &name,
732  const std::string &value,
733  std::string &errstr) = 0;
734 
736  virtual Conf::ConfResult set (const std::string &name,
737  DeliveryReportCb *dr_cb,
738  std::string &errstr) = 0;
739 
741  virtual Conf::ConfResult set (const std::string &name,
742  EventCb *event_cb,
743  std::string &errstr) = 0;
744 
752  virtual Conf::ConfResult set (const std::string &name,
753  const Conf *topic_conf,
754  std::string &errstr) = 0;
755 
757  virtual Conf::ConfResult set (const std::string &name,
758  PartitionerCb *partitioner_cb,
759  std::string &errstr) = 0;
760 
762  virtual Conf::ConfResult set (const std::string &name,
763  PartitionerKeyPointerCb *partitioner_kp_cb,
764  std::string &errstr) = 0;
765 
767  virtual Conf::ConfResult set (const std::string &name, SocketCb *socket_cb,
768  std::string &errstr) = 0;
769 
771  virtual Conf::ConfResult set (const std::string &name, OpenCb *open_cb,
772  std::string &errstr) = 0;
773 
775  virtual Conf::ConfResult set (const std::string &name,
776  RebalanceCb *rebalance_cb,
777  std::string &errstr) = 0;
778 
780  virtual Conf::ConfResult set (const std::string &name,
781  OffsetCommitCb *offset_commit_cb,
782  std::string &errstr) = 0;
783 
787  virtual Conf::ConfResult get(const std::string &name,
788  std::string &value) const = 0;
789 
792  virtual std::list<std::string> *dump () = 0;
793 };
794 
807 class RD_EXPORT Handle {
808  public:
809  virtual ~Handle() { }
810 
812  virtual const std::string name () const = 0;
813 
822  virtual const std::string memberid () const = 0;
823 
824 
847  virtual int poll (int timeout_ms) = 0;
848 
855  virtual int outq_len () = 0;
856 
872  virtual ErrorCode metadata (bool all_topics, const Topic *only_rkt,
873  Metadata **metadatap, int timeout_ms) = 0;
874 
875 
885  virtual ErrorCode pause (std::vector<TopicPartition*> &partitions) = 0;
886 
887 
897  virtual ErrorCode resume (std::vector<TopicPartition*> &partitions) = 0;
898 
899 
908  virtual ErrorCode query_watermark_offsets (const std::string &topic,
909  int32_t partition,
910  int64_t *low, int64_t *high,
911  int timeout_ms) = 0;
912 
930  virtual ErrorCode get_watermark_offsets (const std::string &topic,
931  int32_t partition,
932  int64_t *low, int64_t *high) = 0;
933 };
934 
935 
954 class RD_EXPORT TopicPartition {
955 public:
961  static TopicPartition *create (const std::string &topic, int partition);
962 
963  virtual ~TopicPartition() = 0;
964 
966  virtual const std::string &topic () const = 0;
967 
969  virtual int partition () = 0;
970 
972  virtual int64_t offset () = 0;
973 
975  virtual void set_offset (int64_t offset) = 0;
976 
978  virtual ErrorCode err () = 0;
979 };
980 
981 
982 
987 class RD_EXPORT Topic {
988  public:
995  static const int32_t PARTITION_UA = -1;
996 
998  static const int64_t OFFSET_BEGINNING = -2;
999  static const int64_t OFFSET_END = -1;
1000  static const int64_t OFFSET_STORED = -1000;
1001  static const int64_t OFFSET_INVALID = -1001;
1013  static Topic *create (Handle *base, const std::string &topic_str,
1014  Conf *conf, std::string &errstr);
1015 
1016  virtual ~Topic () = 0;
1017 
1018 
1020  virtual const std::string name () const = 0;
1021 
1027  virtual bool partition_available (int32_t partition) const = 0;
1028 
1040  virtual ErrorCode offset_store (int32_t partition, int64_t offset) = 0;
1041 };
1042 
1043 
1065 class RD_EXPORT MessageTimestamp {
1066 public:
1070  MSG_TIMESTAMP_LOG_APPEND_TIME
1071  };
1072 
1074  int64_t timestamp;
1075 };
1076 
1077 
1078 
1090 class RD_EXPORT Message {
1091  public:
1099  virtual std::string errstr() const = 0;
1100 
1102  virtual ErrorCode err () const = 0;
1103 
1108  virtual Topic *topic () const = 0;
1109 
1111  virtual std::string topic_name () const = 0;
1112 
1114  virtual int32_t partition () const = 0;
1115 
1117  virtual void *payload () const = 0 ;
1118 
1120  virtual size_t len () const = 0;
1121 
1123  virtual const std::string *key () const = 0;
1124 
1126  virtual const void *key_pointer () const = 0 ;
1127 
1129  virtual size_t key_len () const = 0;
1130 
1132  virtual int64_t offset () const = 0;
1133 
1135  virtual MessageTimestamp timestamp () const = 0;
1136 
1138  virtual void *msg_opaque () const = 0;
1139 
1140  virtual ~Message () = 0;
1141 };
1142 
1166 class Queue {
1167  public:
1171  static Queue *create (Handle *handle);
1172 
1173  virtual ~Queue () { }
1174 };
1175 
1194 class RD_EXPORT KafkaConsumer : public virtual Handle {
1195 public:
1207  static KafkaConsumer *create (Conf *conf, std::string &errstr);
1208 
1209  virtual ~KafkaConsumer () = 0;
1210 
1211 
1214  virtual ErrorCode assignment (std::vector<RdKafka::TopicPartition*> &partitions) = 0;
1215 
1218  virtual ErrorCode subscription (std::vector<std::string> &topics) = 0;
1219 
1242  virtual ErrorCode subscribe (const std::vector<std::string> &topics) = 0;
1243 
1245  virtual ErrorCode unsubscribe () = 0;
1246 
1253  virtual ErrorCode assign (const std::vector<TopicPartition*> &partitions) = 0;
1254 
1258  virtual ErrorCode unassign () = 0;
1259 
1284  virtual Message *consume (int timeout_ms) = 0;
1285 
1299  virtual ErrorCode commitSync () = 0;
1300 
1306  virtual ErrorCode commitAsync () = 0;
1307 
1315  virtual ErrorCode commitSync (Message *message) = 0;
1316 
1324  virtual ErrorCode commitAsync (Message *message) = 0;
1325 
1331  virtual ErrorCode commitSync (std::vector<TopicPartition*> &offsets) = 0;
1332 
1338  virtual ErrorCode commitAsync (const std::vector<TopicPartition*> &offsets) = 0;
1339 
1340 
1349  virtual ErrorCode committed (std::vector<TopicPartition*> &partitions,
1350  int timeout_ms) = 0;
1351 
1360  virtual ErrorCode position (std::vector<TopicPartition*> &partitions) = 0;
1361 
1362 
1385  virtual ErrorCode close () = 0;
1386 };
1387 
1388 
1403 class RD_EXPORT Consumer : public virtual Handle {
1404  public:
1415  static Consumer *create (Conf *conf, std::string &errstr);
1416 
1417  virtual ~Consumer () = 0;
1418 
1419 
1439  virtual ErrorCode start (Topic *topic, int32_t partition, int64_t offset) = 0;
1440 
1447  virtual ErrorCode start (Topic *topic, int32_t partition, int64_t offset,
1448  Queue *queue) = 0;
1449 
1459  virtual ErrorCode stop (Topic *topic, int32_t partition) = 0;
1460 
1475  virtual ErrorCode seek (Topic *topic, int32_t partition, int64_t offset,
1476  int timeout_ms) = 0;
1477 
1495  virtual Message *consume (Topic *topic, int32_t partition,
1496  int timeout_ms) = 0;
1497 
1519  virtual Message *consume (Queue *queue, int timeout_ms) = 0;
1520 
1540  virtual int consume_callback (Topic *topic, int32_t partition,
1541  int timeout_ms,
1542  ConsumeCb *consume_cb,
1543  void *opaque) = 0;
1544 
1551  virtual int consume_callback (Queue *queue, int timeout_ms,
1552  RdKafka::ConsumeCb *consume_cb,
1553  void *opaque) = 0;
1554 
1564  static int64_t OffsetTail(int64_t offset);
1565 };
1566 
1580 class RD_EXPORT Producer : public virtual Handle {
1581  public:
1592  static Producer *create (Conf *conf, std::string &errstr);
1593 
1594 
1595  virtual ~Producer () = 0;
1596 
1602  static const int RK_MSG_FREE = 0x1;
1604  static const int RK_MSG_COPY = 0x2;
1608  static const int RK_MSG_BLOCK = 0x4;
1625  /* For backwards compatibility: */
1626 #ifndef MSG_COPY /* defined in sys/msg.h */
1627  static const int MSG_FREE = RK_MSG_FREE;
1628  static const int MSG_COPY = RK_MSG_COPY;
1629 #endif
1630 
1686  virtual ErrorCode produce (Topic *topic, int32_t partition,
1687  int msgflags,
1688  void *payload, size_t len,
1689  const std::string *key,
1690  void *msg_opaque) = 0;
1691 
1696  virtual ErrorCode produce (Topic *topic, int32_t partition,
1697  int msgflags,
1698  void *payload, size_t len,
1699  const void *key, size_t key_len,
1700  void *msg_opaque) = 0;
1701 
1702 
1707  virtual ErrorCode produce (Topic *topic, int32_t partition,
1708  const std::vector<char> *payload,
1709  const std::vector<char> *key,
1710  void *msg_opaque) = 0;
1711 
1712 
1724  virtual ErrorCode flush (int timeout_ms) = 0;
1725 };
1726 
1741  public:
1743  virtual int32_t id() const = 0;
1744 
1746  virtual const std::string host() const = 0;
1747 
1749  virtual int port() const = 0;
1750 
1751  virtual ~BrokerMetadata() = 0;
1752 };
1753 
1754 
1755 
1760  public:
1762  typedef std::vector<int32_t> ReplicasVector;
1764  typedef std::vector<int32_t> ISRSVector;
1765 
1767  typedef ReplicasVector::const_iterator ReplicasIterator;
1769  typedef ISRSVector::const_iterator ISRSIterator;
1770 
1771 
1773  virtual int32_t id() const = 0;
1774 
1776  virtual ErrorCode err() const = 0;
1777 
1779  virtual int32_t leader() const = 0;
1780 
1782  virtual const std::vector<int32_t> *replicas() const = 0;
1783 
1787  virtual const std::vector<int32_t> *isrs() const = 0;
1788 
1789  virtual ~PartitionMetadata() = 0;
1790 };
1791 
1792 
1793 
1798  public:
1800  typedef std::vector<const PartitionMetadata*> PartitionMetadataVector;
1802  typedef PartitionMetadataVector::const_iterator PartitionMetadataIterator;
1803 
1805  virtual const std::string topic() const = 0;
1806 
1808  virtual const PartitionMetadataVector *partitions() const = 0;
1809 
1811  virtual ErrorCode err() const = 0;
1812 
1813  virtual ~TopicMetadata() = 0;
1814 };
1815 
1816 
1820 class Metadata {
1821  public:
1823  typedef std::vector<const BrokerMetadata*> BrokerMetadataVector;
1825  typedef std::vector<const TopicMetadata*> TopicMetadataVector;
1826 
1828  typedef BrokerMetadataVector::const_iterator BrokerMetadataIterator;
1830  typedef TopicMetadataVector::const_iterator TopicMetadataIterator;
1831 
1832 
1834  virtual const BrokerMetadataVector *brokers() const = 0;
1835 
1837  virtual const TopicMetadataVector *topics() const = 0;
1838 
1840  virtual int32_t orig_broker_id() const = 0;
1841 
1843  virtual const std::string orig_broker_name() const = 0;
1844 
1845  virtual ~Metadata() = 0;
1846 };
1847 
1850 }
1851 
Definition: rdkafkacpp.h:174
Definition: rdkafkacpp.h:275
ISRSVector::const_iterator ISRSIterator
ISRs iterator.
Definition: rdkafkacpp.h:1769
Definition: rdkafkacpp.h:167
Definition: rdkafkacpp.h:277
Definition: rdkafkacpp.h:220
High-level KafkaConsumer (for brokers 0.9 and later)
Definition: rdkafkacpp.h:1194
Definition: rdkafkacpp.h:190
ConfType
Configuration object type.
Definition: rdkafkacpp.h:704
Partitioner callback class.
Definition: rdkafkacpp.h:364
Definition: rdkafkacpp.h:255
Definition: rdkafkacpp.h:241
Definition: rdkafkacpp.h:281
Type
Event type.
Definition: rdkafkacpp.h:442
Definition: rdkafkacpp.h:161
Definition: rdkafkacpp.h:218
Definition: rdkafkacpp.h:267
Definition: rdkafkacpp.h:165
Definition: rdkafkacpp.h:202
Definition: rdkafkacpp.h:279
Definition: rdkafkacpp.h:163
std::vector< const PartitionMetadata * > PartitionMetadataVector
Partitions.
Definition: rdkafkacpp.h:1800
Definition: rdkafkacpp.h:206
Message object.
Definition: rdkafkacpp.h:1090
Definition: rdkafkacpp.h:184
Definition: rdkafkacpp.h:159
Severity
EVENT_LOG severities (conforms to syslog(3) severities)
Definition: rdkafkacpp.h:450
Event callback class.
Definition: rdkafkacpp.h:423
Definition: rdkafkacpp.h:265
ReplicasVector::const_iterator ReplicasIterator
Replicas iterator.
Definition: rdkafkacpp.h:1767
Definition: rdkafkacpp.h:291
Definition: rdkafkacpp.h:188
Definition: rdkafkacpp.h:212
Definition: rdkafkacpp.h:196
KafkaConsunmer: Rebalance callback class
Definition: rdkafkacpp.h:543
std::vector< int32_t > ReplicasVector
Replicas.
Definition: rdkafkacpp.h:1762
Definition: rdkafkacpp.h:153
Definition: rdkafkacpp.h:178
Definition: rdkafkacpp.h:180
Definition: rdkafkacpp.h:198
Definition: rdkafkacpp.h:186
int64_t timestamp
Definition: rdkafkacpp.h:1074
Definition: rdkafkacpp.h:285
Variant partitioner with key pointer.
Definition: rdkafkacpp.h:394
Definition: rdkafkacpp.h:251
BrokerMetadataVector::const_iterator BrokerMetadataIterator
Brokers iterator.
Definition: rdkafkacpp.h:1828
Definition: rdkafkacpp.h:257
Definition: rdkafkacpp.h:155
Definition: rdkafkacpp.h:249
Definition: rdkafkacpp.h:445
Definition: rdkafkacpp.h:172
Definition: rdkafkacpp.h:204
Definition: rdkafkacpp.h:245
Topic handle.
Definition: rdkafkacpp.h:987
Definition: rdkafkacpp.h:157
Definition: rdkafkacpp.h:237
Metadata: Partition information.
Definition: rdkafkacpp.h:1759
Definition: rdkafkacpp.h:287
Producer.
Definition: rdkafkacpp.h:1580
Definition: rdkafkacpp.h:151
Definition: rdkafkacpp.h:261
Metadata: Topic information.
Definition: rdkafkacpp.h:1797
Definition: rdkafkacpp.h:705
Definition: rdkafkacpp.h:443
Definition: rdkafkacpp.h:200
Definition: rdkafkacpp.h:208
std::vector< const BrokerMetadata * > BrokerMetadataVector
Brokers.
Definition: rdkafkacpp.h:1823
ConfResult
RdKafka::Conf::Set() result code.
Definition: rdkafkacpp.h:712
Definition: rdkafkacpp.h:68
Queue interface.
Definition: rdkafkacpp.h:1166
MessageTimestampType type
Definition: rdkafkacpp.h:1073
Message timestamp object.
Definition: rdkafkacpp.h:1065
std::vector< int32_t > ISRSVector
ISRs (In-Sync-Replicas)
Definition: rdkafkacpp.h:1764
Definition: rdkafkacpp.h:289
Definition: rdkafkacpp.h:231
Definition: rdkafkacpp.h:223
Definition: rdkafkacpp.h:176
Definition: rdkafkacpp.h:235
Definition: rdkafkacpp.h:194
Definition: rdkafkacpp.h:182
PartitionMetadataVector::const_iterator PartitionMetadataIterator
Partitions iterator.
Definition: rdkafkacpp.h:1802
Portability: OpenCb callback class
Definition: rdkafkacpp.h:660
Definition: rdkafkacpp.h:192
Definition: rdkafkacpp.h:233
Definition: rdkafkacpp.h:227
Definition: rdkafkacpp.h:253
Metadata: Broker information.
Definition: rdkafkacpp.h:1740
Definition: rdkafkacpp.h:444
Definition: rdkafkacpp.h:210
Definition: rdkafkacpp.h:214
Definition: rdkafkacpp.h:283
Configuration interface.
Definition: rdkafkacpp.h:699
Definition: rdkafkacpp.h:239
Definition: rdkafkacpp.h:247
Definition: rdkafkacpp.h:170
Offset Commit callback class.
Definition: rdkafkacpp.h:605
Base handle, super class for specific clients.
Definition: rdkafkacpp.h:807
Definition: rdkafkacpp.h:216
RD_EXPORT std::string err2str(RdKafka::ErrorCode err)
Returns a human readable representation of a kafka error.
std::vector< const TopicMetadata * > TopicMetadataVector
Topics.
Definition: rdkafkacpp.h:1825
Definition: rdkafkacpp.h:273
Topic+Partition.
Definition: rdkafkacpp.h:954
Definition: rdkafkacpp.h:263
Consume callback class.
Definition: rdkafkacpp.h:525
Simple Consumer (legacy)
Definition: rdkafkacpp.h:1403
MessageTimestampType
Definition: rdkafkacpp.h:1067
TopicMetadataVector::const_iterator TopicMetadataIterator
Topics iterator.
Definition: rdkafkacpp.h:1830
Definition: rdkafkacpp.h:229
Event object class as passed to the EventCb callback.
Definition: rdkafkacpp.h:439
Metadata container.
Definition: rdkafkacpp.h:1820
ErrorCode
Error codes.
Definition: rdkafkacpp.h:148
Definition: rdkafkacpp.h:271
Definition: rdkafkacpp.h:243
Portability: SocketCb callback class
Definition: rdkafkacpp.h:635
Delivery Report callback class.
Definition: rdkafkacpp.h:346