forked from confluentinc/librdkafka
-
Notifications
You must be signed in to change notification settings - Fork 0
/
rdkafkacpp.h
3055 lines (2687 loc) · 99.7 KB
/
rdkafkacpp.h
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
/*
* librdkafka - Apache Kafka C/C++ library
*
* Copyright (c) 2014 Magnus Edenhill
* All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
* modification, are permitted provided that the following conditions are met:
*
* 1. Redistributions of source code must retain the above copyright notice,
* this list of conditions and the following disclaimer.
* 2. Redistributions in binary form must reproduce the above copyright notice,
* this list of conditions and the following disclaimer in the documentation
* and/or other materials provided with the distribution.
*
* THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
* AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
* IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
* ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE
* LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
* CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
* SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
* INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
* CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
* ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE
* POSSIBILITY OF SUCH DAMAGE.
*/
#ifndef _RDKAFKACPP_H_
#define _RDKAFKACPP_H_
/**
* @file rdkafkacpp.h
* @brief Apache Kafka C/C++ consumer and producer client library.
*
* rdkafkacpp.h contains the public C++ API for librdkafka.
* The API is documented in this file as comments prefixing the class,
* function, type, enum, define, etc.
* For more information, see the C interface in rdkafka.h and read the
* manual in INTRODUCTION.md.
* The C++ interface is STD C++ '03 compliant and adheres to the
* Google C++ Style Guide.
* @sa For the C interface see rdkafka.h
*
* @tableofcontents
*/
/**@cond NO_DOC*/
#include <string>
#include <list>
#include <vector>
#include <cstdlib>
#include <cstring>
#include <stdint.h>
#include <sys/types.h>
#ifdef _MSC_VER
#ifndef ssize_t
#ifndef _BASETSD_H_
#include <basetsd.h>
#endif
typedef SSIZE_T ssize_t;
#endif
#undef RD_EXPORT
#ifdef LIBRDKAFKA_STATICLIB
#define RD_EXPORT
#else
#ifdef LIBRDKAFKACPP_EXPORTS
#define RD_EXPORT __declspec(dllexport)
#else
#define RD_EXPORT __declspec(dllimport)
#endif
#endif
#else
#define RD_EXPORT
#endif
/**@endcond*/
extern "C" {
/* Forward declarations */
struct rd_kafka_s;
struct rd_kafka_topic_s;
struct rd_kafka_message_s;
struct rd_kafka_conf_s;
struct rd_kafka_topic_conf_s;
}
namespace RdKafka {
/**
* @name Miscellaneous APIs
* @{
*/
/**
* @brief librdkafka version
*
* Interpreted as hex \c MM.mm.rr.xx:
* - MM = Major
* - mm = minor
* - rr = revision
* - xx = pre-release id (0xff is the final release)
*
* E.g.: \c 0x000801ff = 0.8.1
*
* @remark This value should only be used during compile time,
* for runtime checks of version use RdKafka::version()
*/
#define RD_KAFKA_VERSION 0x010202ff
/**
* @brief Returns the librdkafka version as integer.
*
* @sa See RD_KAFKA_VERSION for how to parse the integer format.
*/
RD_EXPORT
int version ();
/**
* @brief Returns the librdkafka version as string.
*/
RD_EXPORT
std::string version_str();
/**
* @brief Returns a CSV list of the supported debug contexts
* for use with Conf::Set("debug", ..).
*/
RD_EXPORT
std::string get_debug_contexts();
/**
* @brief Wait for all rd_kafka_t objects to be destroyed.
*
* @returns 0 if all kafka objects are now destroyed, or -1 if the
* timeout was reached.
* Since RdKafka handle deletion is an asynch operation the
* \p wait_destroyed() function can be used for applications where
* a clean shutdown is required.
*/
RD_EXPORT
int wait_destroyed(int timeout_ms);
/**@}*/
/**
* @name Constants, errors, types
* @{
*
*
*/
/**
* @brief Error codes.
*
* The negative error codes delimited by two underscores
* (\c _ERR__..) denotes errors internal to librdkafka and are
* displayed as \c \"Local: \<error string..\>\", while the error codes
* delimited by a single underscore (\c ERR_..) denote broker
* errors and are displayed as \c \"Broker: \<error string..\>\".
*
* @sa Use RdKafka::err2str() to translate an error code a human readable string
*/
enum ErrorCode {
/* Internal errors to rdkafka: */
/** Begin internal error codes */
ERR__BEGIN = -200,
/** Received message is incorrect */
ERR__BAD_MSG = -199,
/** Bad/unknown compression */
ERR__BAD_COMPRESSION = -198,
/** Broker is going away */
ERR__DESTROY = -197,
/** Generic failure */
ERR__FAIL = -196,
/** Broker transport failure */
ERR__TRANSPORT = -195,
/** Critical system resource */
ERR__CRIT_SYS_RESOURCE = -194,
/** Failed to resolve broker */
ERR__RESOLVE = -193,
/** Produced message timed out*/
ERR__MSG_TIMED_OUT = -192,
/** Reached the end of the topic+partition queue on
* the broker. Not really an error.
* This event is disabled by default,
* see the `enable.partition.eof` configuration property. */
ERR__PARTITION_EOF = -191,
/** Permanent: Partition does not exist in cluster. */
ERR__UNKNOWN_PARTITION = -190,
/** File or filesystem error */
ERR__FS = -189,
/** Permanent: Topic does not exist in cluster. */
ERR__UNKNOWN_TOPIC = -188,
/** All broker connections are down. */
ERR__ALL_BROKERS_DOWN = -187,
/** Invalid argument, or invalid configuration */
ERR__INVALID_ARG = -186,
/** Operation timed out */
ERR__TIMED_OUT = -185,
/** Queue is full */
ERR__QUEUE_FULL = -184,
/** ISR count < required.acks */
ERR__ISR_INSUFF = -183,
/** Broker node update */
ERR__NODE_UPDATE = -182,
/** SSL error */
ERR__SSL = -181,
/** Waiting for coordinator to become available. */
ERR__WAIT_COORD = -180,
/** Unknown client group */
ERR__UNKNOWN_GROUP = -179,
/** Operation in progress */
ERR__IN_PROGRESS = -178,
/** Previous operation in progress, wait for it to finish. */
ERR__PREV_IN_PROGRESS = -177,
/** This operation would interfere with an existing subscription */
ERR__EXISTING_SUBSCRIPTION = -176,
/** Assigned partitions (rebalance_cb) */
ERR__ASSIGN_PARTITIONS = -175,
/** Revoked partitions (rebalance_cb) */
ERR__REVOKE_PARTITIONS = -174,
/** Conflicting use */
ERR__CONFLICT = -173,
/** Wrong state */
ERR__STATE = -172,
/** Unknown protocol */
ERR__UNKNOWN_PROTOCOL = -171,
/** Not implemented */
ERR__NOT_IMPLEMENTED = -170,
/** Authentication failure*/
ERR__AUTHENTICATION = -169,
/** No stored offset */
ERR__NO_OFFSET = -168,
/** Outdated */
ERR__OUTDATED = -167,
/** Timed out in queue */
ERR__TIMED_OUT_QUEUE = -166,
/** Feature not supported by broker */
ERR__UNSUPPORTED_FEATURE = -165,
/** Awaiting cache update */
ERR__WAIT_CACHE = -164,
/** Operation interrupted */
ERR__INTR = -163,
/** Key serialization error */
ERR__KEY_SERIALIZATION = -162,
/** Value serialization error */
ERR__VALUE_SERIALIZATION = -161,
/** Key deserialization error */
ERR__KEY_DESERIALIZATION = -160,
/** Value deserialization error */
ERR__VALUE_DESERIALIZATION = -159,
/** Partial response */
ERR__PARTIAL = -158,
/** Modification attempted on read-only object */
ERR__READ_ONLY = -157,
/** No such entry / item not found */
ERR__NOENT = -156,
/** Read underflow */
ERR__UNDERFLOW = -155,
/** Invalid type */
ERR__INVALID_TYPE = -154,
/** Retry operation */
ERR__RETRY = -153,
/** Purged in queue */
ERR__PURGE_QUEUE = -152,
/** Purged in flight */
ERR__PURGE_INFLIGHT = -151,
/** Fatal error: see RdKafka::Handle::fatal_error() */
ERR__FATAL = -150,
/** Inconsistent state */
ERR__INCONSISTENT = -149,
/** Gap-less ordering would not be guaranteed if proceeding */
ERR__GAPLESS_GUARANTEE = -148,
/** Maximum poll interval exceeded */
ERR__MAX_POLL_EXCEEDED = -147,
/** Unknown broker */
ERR__UNKNOWN_BROKER = -146,
/** End internal error codes */
ERR__END = -100,
/* Kafka broker errors: */
/** Unknown broker error */
ERR_UNKNOWN = -1,
/** Success */
ERR_NO_ERROR = 0,
/** Offset out of range */
ERR_OFFSET_OUT_OF_RANGE = 1,
/** Invalid message */
ERR_INVALID_MSG = 2,
/** Unknown topic or partition */
ERR_UNKNOWN_TOPIC_OR_PART = 3,
/** Invalid message size */
ERR_INVALID_MSG_SIZE = 4,
/** Leader not available */
ERR_LEADER_NOT_AVAILABLE = 5,
/** Not leader for partition */
ERR_NOT_LEADER_FOR_PARTITION = 6,
/** Request timed out */
ERR_REQUEST_TIMED_OUT = 7,
/** Broker not available */
ERR_BROKER_NOT_AVAILABLE = 8,
/** Replica not available */
ERR_REPLICA_NOT_AVAILABLE = 9,
/** Message size too large */
ERR_MSG_SIZE_TOO_LARGE = 10,
/** StaleControllerEpochCode */
ERR_STALE_CTRL_EPOCH = 11,
/** Offset metadata string too large */
ERR_OFFSET_METADATA_TOO_LARGE = 12,
/** Broker disconnected before response received */
ERR_NETWORK_EXCEPTION = 13,
/** Coordinator load in progress */
ERR_COORDINATOR_LOAD_IN_PROGRESS = 14,
/** Group coordinator load in progress */
#define ERR_GROUP_LOAD_IN_PROGRESS ERR_COORDINATOR_LOAD_IN_PROGRESS
/** Coordinator not available */
ERR_COORDINATOR_NOT_AVAILABLE = 15,
/** Group coordinator not available */
#define ERR_GROUP_COORDINATOR_NOT_AVAILABLE ERR_COORDINATOR_NOT_AVAILABLE
/** Not coordinator */
ERR_NOT_COORDINATOR = 16,
/** Not coordinator for group */
#define ERR_NOT_COORDINATOR_FOR_GROUP ERR_NOT_COORDINATOR
/** Invalid topic */
ERR_TOPIC_EXCEPTION = 17,
/** Message batch larger than configured server segment size */
ERR_RECORD_LIST_TOO_LARGE = 18,
/** Not enough in-sync replicas */
ERR_NOT_ENOUGH_REPLICAS = 19,
/** Message(s) written to insufficient number of in-sync replicas */
ERR_NOT_ENOUGH_REPLICAS_AFTER_APPEND = 20,
/** Invalid required acks value */
ERR_INVALID_REQUIRED_ACKS = 21,
/** Specified group generation id is not valid */
ERR_ILLEGAL_GENERATION = 22,
/** Inconsistent group protocol */
ERR_INCONSISTENT_GROUP_PROTOCOL = 23,
/** Invalid group.id */
ERR_INVALID_GROUP_ID = 24,
/** Unknown member */
ERR_UNKNOWN_MEMBER_ID = 25,
/** Invalid session timeout */
ERR_INVALID_SESSION_TIMEOUT = 26,
/** Group rebalance in progress */
ERR_REBALANCE_IN_PROGRESS = 27,
/** Commit offset data size is not valid */
ERR_INVALID_COMMIT_OFFSET_SIZE = 28,
/** Topic authorization failed */
ERR_TOPIC_AUTHORIZATION_FAILED = 29,
/** Group authorization failed */
ERR_GROUP_AUTHORIZATION_FAILED = 30,
/** Cluster authorization failed */
ERR_CLUSTER_AUTHORIZATION_FAILED = 31,
/** Invalid timestamp */
ERR_INVALID_TIMESTAMP = 32,
/** Unsupported SASL mechanism */
ERR_UNSUPPORTED_SASL_MECHANISM = 33,
/** Illegal SASL state */
ERR_ILLEGAL_SASL_STATE = 34,
/** Unuspported version */
ERR_UNSUPPORTED_VERSION = 35,
/** Topic already exists */
ERR_TOPIC_ALREADY_EXISTS = 36,
/** Invalid number of partitions */
ERR_INVALID_PARTITIONS = 37,
/** Invalid replication factor */
ERR_INVALID_REPLICATION_FACTOR = 38,
/** Invalid replica assignment */
ERR_INVALID_REPLICA_ASSIGNMENT = 39,
/** Invalid config */
ERR_INVALID_CONFIG = 40,
/** Not controller for cluster */
ERR_NOT_CONTROLLER = 41,
/** Invalid request */
ERR_INVALID_REQUEST = 42,
/** Message format on broker does not support request */
ERR_UNSUPPORTED_FOR_MESSAGE_FORMAT = 43,
/** Policy violation */
ERR_POLICY_VIOLATION = 44,
/** Broker received an out of order sequence number */
ERR_OUT_OF_ORDER_SEQUENCE_NUMBER = 45,
/** Broker received a duplicate sequence number */
ERR_DUPLICATE_SEQUENCE_NUMBER = 46,
/** Producer attempted an operation with an old epoch */
ERR_INVALID_PRODUCER_EPOCH = 47,
/** Producer attempted a transactional operation in an invalid state */
ERR_INVALID_TXN_STATE = 48,
/** Producer attempted to use a producer id which is not
* currently assigned to its transactional id */
ERR_INVALID_PRODUCER_ID_MAPPING = 49,
/** Transaction timeout is larger than the maximum
* value allowed by the broker's max.transaction.timeout.ms */
ERR_INVALID_TRANSACTION_TIMEOUT = 50,
/** Producer attempted to update a transaction while another
* concurrent operation on the same transaction was ongoing */
ERR_CONCURRENT_TRANSACTIONS = 51,
/** Indicates that the transaction coordinator sending a
* WriteTxnMarker is no longer the current coordinator for a
* given producer */
ERR_TRANSACTION_COORDINATOR_FENCED = 52,
/** Transactional Id authorization failed */
ERR_TRANSACTIONAL_ID_AUTHORIZATION_FAILED = 53,
/** Security features are disabled */
ERR_SECURITY_DISABLED = 54,
/** Operation not attempted */
ERR_OPERATION_NOT_ATTEMPTED = 55,
/** Disk error when trying to access log file on the disk */
ERR_KAFKA_STORAGE_ERROR = 56,
/** The user-specified log directory is not found in the broker config */
ERR_LOG_DIR_NOT_FOUND = 57,
/** SASL Authentication failed */
ERR_SASL_AUTHENTICATION_FAILED = 58,
/** Unknown Producer Id */
ERR_UNKNOWN_PRODUCER_ID = 59,
/** Partition reassignment is in progress */
ERR_REASSIGNMENT_IN_PROGRESS = 60,
/** Delegation Token feature is not enabled */
ERR_DELEGATION_TOKEN_AUTH_DISABLED = 61,
/** Delegation Token is not found on server */
ERR_DELEGATION_TOKEN_NOT_FOUND = 62,
/** Specified Principal is not valid Owner/Renewer */
ERR_DELEGATION_TOKEN_OWNER_MISMATCH = 63,
/** Delegation Token requests are not allowed on this connection */
ERR_DELEGATION_TOKEN_REQUEST_NOT_ALLOWED = 64,
/** Delegation Token authorization failed */
ERR_DELEGATION_TOKEN_AUTHORIZATION_FAILED = 65,
/** Delegation Token is expired */
ERR_DELEGATION_TOKEN_EXPIRED = 66,
/** Supplied principalType is not supported */
ERR_INVALID_PRINCIPAL_TYPE = 67,
/** The group is not empty */
ERR_NON_EMPTY_GROUP = 68,
/** The group id does not exist */
ERR_GROUP_ID_NOT_FOUND = 69,
/** The fetch session ID was not found */
ERR_FETCH_SESSION_ID_NOT_FOUND = 70,
/** The fetch session epoch is invalid */
ERR_INVALID_FETCH_SESSION_EPOCH = 71,
/** No matching listener */
ERR_LISTENER_NOT_FOUND = 72,
/** Topic deletion is disabled */
ERR_TOPIC_DELETION_DISABLED = 73,
/** Leader epoch is older than broker epoch */
ERR_FENCED_LEADER_EPOCH = 74,
/** Leader epoch is newer than broker epoch */
ERR_UNKNOWN_LEADER_EPOCH = 75,
/** Unsupported compression type */
ERR_UNSUPPORTED_COMPRESSION_TYPE = 76,
/** Broker epoch has changed */
ERR_STALE_BROKER_EPOCH = 77,
/** Leader high watermark is not caught up */
ERR_OFFSET_NOT_AVAILABLE = 78,
/** Group member needs a valid member ID */
ERR_MEMBER_ID_REQUIRED = 79,
/** Preferred leader was not available */
ERR_PREFERRED_LEADER_NOT_AVAILABLE = 80,
/** Consumer group has reached maximum size */
ERR_GROUP_MAX_SIZE_REACHED = 81,
};
/**
* @brief Returns a human readable representation of a kafka error.
*/
RD_EXPORT
std::string err2str(RdKafka::ErrorCode err);
/**
* @enum CertificateType
* @brief SSL certificate types
*/
enum CertificateType {
CERT_PUBLIC_KEY, /**< Client's public key */
CERT_PRIVATE_KEY, /**< Client's private key */
CERT_CA, /**< CA certificate */
CERT__CNT
};
/**
* @enum CertificateEncoding
* @brief SSL certificate encoding
*/
enum CertificateEncoding {
CERT_ENC_PKCS12, /**< PKCS#12 */
CERT_ENC_DER, /**< DER / binary X.509 ASN1 */
CERT_ENC_PEM, /**< PEM */
CERT_ENC__CNT
};
/**@} */
/**@cond NO_DOC*/
/* Forward declarations */
class Producer;
class Message;
class Headers;
class Queue;
class Event;
class Topic;
class TopicPartition;
class Metadata;
class KafkaConsumer;
/**@endcond*/
/**
* @name Callback classes
* @{
*
*
* librdkafka uses (optional) callbacks to propagate information and
* delegate decisions to the application logic.
*
* An application must call RdKafka::poll() at regular intervals to
* serve queued callbacks.
*/
/**
* @brief Delivery Report callback class
*
* The delivery report callback will be called once for each message
* accepted by RdKafka::Producer::produce() (et.al) with
* RdKafka::Message::err() set to indicate the result of the produce request.
*
* The callback is called when a message is succesfully produced or
* if librdkafka encountered a permanent failure, or the retry counter for
* temporary errors has been exhausted.
*
* An application must call RdKafka::poll() at regular intervals to
* serve queued delivery report callbacks.
*/
class RD_EXPORT DeliveryReportCb {
public:
/**
* @brief Delivery report callback.
*/
virtual void dr_cb (Message &message) = 0;
virtual ~DeliveryReportCb() { }
};
/**
* @brief SASL/OAUTHBEARER token refresh callback class
*
* The SASL/OAUTHBEARER token refresh callback is triggered via RdKafka::poll()
* whenever OAUTHBEARER is the SASL mechanism and a token needs to be retrieved,
* typically based on the configuration defined in \c sasl.oauthbearer.config.
*
* The \c oauthbearer_config argument is the value of the
* sasl.oauthbearer.config configuration property.
*
* The callback should invoke RdKafka::oauthbearer_set_token() or
* RdKafka::oauthbearer_set_token_failure() to indicate success or failure,
* respectively.
*
* The refresh operation is eventable and may be received when an event
* callback handler is set with an event type of
* \c RdKafka::Event::EVENT_OAUTHBEARER_TOKEN_REFRESH.
*
* Note that before any SASL/OAUTHBEARER broker connection can succeed the
* application must call RdKafka::oauthbearer_set_token() once -- either
* directly or, more typically, by invoking RdKafka::poll() -- in order to
* cause retrieval of an initial token to occur.
*
* An application must call RdKafka::poll() at regular intervals to
* serve queued SASL/OAUTHBEARER token refresh callbacks (when
* OAUTHBEARER is the SASL mechanism).
*/
class RD_EXPORT OAuthBearerTokenRefreshCb {
public:
/**
* @brief SASL/OAUTHBEARER token refresh callback class.
*/
virtual void oauthbearer_token_refresh_cb (const std::string &oauthbearer_config) = 0;
virtual ~OAuthBearerTokenRefreshCb() { }
};
/**
* @brief Partitioner callback class
*
* Generic partitioner callback class for implementing custom partitioners.
*
* @sa RdKafka::Conf::set() \c "partitioner_cb"
*/
class RD_EXPORT PartitionerCb {
public:
/**
* @brief Partitioner callback
*
* Return the partition to use for \p key in \p topic.
*
* The \p msg_opaque is the same \p msg_opaque provided in the
* RdKafka::Producer::produce() call.
*
* @remark \p key may be NULL or the empty.
*
* @returns Must return a value between 0 and \p partition_cnt (non-inclusive).
* May return RD_KAFKA_PARTITION_UA (-1) if partitioning failed.
*
* @sa The callback may use RdKafka::Topic::partition_available() to check
* if a partition has an active leader broker.
*/
virtual int32_t partitioner_cb (const Topic *topic,
const std::string *key,
int32_t partition_cnt,
void *msg_opaque) = 0;
virtual ~PartitionerCb() { }
};
/**
* @brief Variant partitioner with key pointer
*
*/
class PartitionerKeyPointerCb {
public:
/**
* @brief Variant partitioner callback that gets \p key as pointer and length
* instead of as a const std::string *.
*
* @remark \p key may be NULL or have \p key_len 0.
*
* @sa See RdKafka::PartitionerCb::partitioner_cb() for exact semantics
*/
virtual int32_t partitioner_cb (const Topic *topic,
const void *key,
size_t key_len,
int32_t partition_cnt,
void *msg_opaque) = 0;
virtual ~PartitionerKeyPointerCb() { }
};
/**
* @brief Event callback class
*
* Events are a generic interface for propagating errors, statistics, logs, etc
* from librdkafka to the application.
*
* @sa RdKafka::Event
*/
class RD_EXPORT EventCb {
public:
/**
* @brief Event callback
*
* @sa RdKafka::Event
*/
virtual void event_cb (Event &event) = 0;
virtual ~EventCb() { }
};
/**
* @brief Event object class as passed to the EventCb callback.
*/
class RD_EXPORT Event {
public:
/** @brief Event type */
enum Type {
EVENT_ERROR, /**< Event is an error condition */
EVENT_STATS, /**< Event is a statistics JSON document */
EVENT_LOG, /**< Event is a log message */
EVENT_THROTTLE /**< Event is a throttle level signaling from the broker */
};
/** @brief EVENT_LOG severities (conforms to syslog(3) severities) */
enum Severity {
EVENT_SEVERITY_EMERG = 0,
EVENT_SEVERITY_ALERT = 1,
EVENT_SEVERITY_CRITICAL = 2,
EVENT_SEVERITY_ERROR = 3,
EVENT_SEVERITY_WARNING = 4,
EVENT_SEVERITY_NOTICE = 5,
EVENT_SEVERITY_INFO = 6,
EVENT_SEVERITY_DEBUG = 7
};
virtual ~Event () { }
/*
* Event Accessor methods
*/
/**
* @returns The event type
* @remark Applies to all event types
*/
virtual Type type () const = 0;
/**
* @returns Event error, if any.
* @remark Applies to all event types except THROTTLE
*/
virtual ErrorCode err () const = 0;
/**
* @returns Log severity level.
* @remark Applies to LOG event type.
*/
virtual Severity severity () const = 0;
/**
* @returns Log facility string.
* @remark Applies to LOG event type.
*/
virtual std::string fac () const = 0;
/**
* @returns Log message string.
*
* \c EVENT_LOG: Log message string.
* \c EVENT_STATS: JSON object (as string).
*
* @remark Applies to LOG event type.
*/
virtual std::string str () const = 0;
/**
* @returns Throttle time in milliseconds.
* @remark Applies to THROTTLE event type.
*/
virtual int throttle_time () const = 0;
/**
* @returns Throttling broker's name.
* @remark Applies to THROTTLE event type.
*/
virtual std::string broker_name () const = 0;
/**
* @returns Throttling broker's id.
* @remark Applies to THROTTLE event type.
*/
virtual int broker_id () const = 0;
/**
* @returns true if this is a fatal error.
* @remark Applies to ERROR event type.
* @sa RdKafka::Handle::fatal_error()
*/
virtual bool fatal () const = 0;
};
/**
* @brief Consume callback class
*/
class RD_EXPORT ConsumeCb {
public:
/**
* @brief The consume callback is used with
* RdKafka::Consumer::consume_callback()
* methods and will be called for each consumed \p message.
*
* The callback interface is optional but provides increased performance.
*/
virtual void consume_cb (Message &message, void *opaque) = 0;
virtual ~ConsumeCb() { }
};
/**
* @brief \b KafkaConsumer: Rebalance callback class
*/
class RD_EXPORT RebalanceCb {
public:
/**
* @brief Group rebalance callback for use with RdKafka::KafkaConsumer
*
* Registering a \p rebalance_cb turns off librdkafka's automatic
* partition assignment/revocation and instead delegates that responsibility
* to the application's \p rebalance_cb.
*
* The rebalance callback is responsible for updating librdkafka's
* assignment set based on the two events: RdKafka::ERR__ASSIGN_PARTITIONS
* and RdKafka::ERR__REVOKE_PARTITIONS but should also be able to handle
* arbitrary rebalancing failures where \p err is neither of those.
* @remark In this latter case (arbitrary error), the application must
* call unassign() to synchronize state.
*
* Without a rebalance callback this is done automatically by librdkafka
* but registering a rebalance callback gives the application flexibility
* in performing other operations along with the assinging/revocation,
* such as fetching offsets from an alternate location (on assign)
* or manually committing offsets (on revoke).
*
* The following example show's the application's responsibilities:
* @code
* class MyRebalanceCb : public RdKafka::RebalanceCb {
* public:
* void rebalance_cb (RdKafka::KafkaConsumer *consumer,
* RdKafka::ErrorCode err,
* std::vector<RdKafka::TopicPartition*> &partitions) {
* if (err == RdKafka::ERR__ASSIGN_PARTITIONS) {
* // application may load offets from arbitrary external
* // storage here and update \p partitions
*
* consumer->assign(partitions);
*
* } else if (err == RdKafka::ERR__REVOKE_PARTITIONS) {
* // Application may commit offsets manually here
* // if auto.commit.enable=false
*
* consumer->unassign();
*
* } else {
* std::cerr << "Rebalancing error: <<
* RdKafka::err2str(err) << std::endl;
* consumer->unassign();
* }
* }
* }
* @endcode
*/
virtual void rebalance_cb (RdKafka::KafkaConsumer *consumer,
RdKafka::ErrorCode err,
std::vector<TopicPartition*>&partitions) = 0;
virtual ~RebalanceCb() { }
};
/**
* @brief Offset Commit callback class
*/
class RD_EXPORT OffsetCommitCb {
public:
/**
* @brief Set offset commit callback for use with consumer groups
*
* The results of automatic or manual offset commits will be scheduled
* for this callback and is served by RdKafka::KafkaConsumer::consume().
*
* If no partitions had valid offsets to commit this callback will be called
* with \p err == ERR__NO_OFFSET which is not to be considered an error.
*
* The \p offsets list contains per-partition information:
* - \c topic The topic committed
* - \c partition The partition committed
* - \c offset: Committed offset (attempted)
* - \c err: Commit error
*/
virtual void offset_commit_cb(RdKafka::ErrorCode err,
std::vector<TopicPartition*>&offsets) = 0;
virtual ~OffsetCommitCb() { }
};
/**
* @brief SSL broker certificate verification class.
*
* @remark Class instance must outlive the RdKafka client instance.
*/
class RD_EXPORT SslCertificateVerifyCb {
public:
/**
* @brief SSL broker certificate verification callback.
*
* The verification callback is triggered from internal librdkafka threads
* upon connecting to a broker. On each connection attempt the callback
* will be called for each certificate in the broker's certificate chain,
* starting at the root certification, as long as the application callback
* returns 1 (valid certificate).
*
* \p broker_name and \p broker_id correspond to the broker the connection
* is being made to.
* The \c x509_error argument indicates if OpenSSL's verification of
* the certificate succeed (0) or failed (an OpenSSL error code).
* The application may set the SSL context error code by returning 0
* from the verify callback and providing a non-zero SSL context error code
* in \p x509_error.
* If the verify callback sets \p x509_error to 0, returns 1, and the
* original \p x509_error was non-zero, the error on the SSL context will
* be cleared.
* \p x509_error is always a valid pointer to an int.
*
* \p depth is the depth of the current certificate in the chain, starting
* at the root certificate.
*
* The certificate itself is passed in binary DER format in \p buf of
* size \p size.
*
* The callback must 1 if verification succeeds, or 0 if verification fails
* and write a human-readable error message
* to \p errstr.
*
* @warning This callback will be called from internal librdkafka threads.
*
* @remark See <openssl/x509_vfy.h> in the OpenSSL source distribution
* for a list of \p x509_error codes.
*/
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;
virtual ~SslCertificateVerifyCb() {}
};
/**
* @brief \b Portability: SocketCb callback class
*
*/
class RD_EXPORT SocketCb {
public:
/**
* @brief Socket callback
*
* The socket callback is responsible for opening a socket
* according to the supplied \p domain, \p type and \p protocol.
* The socket shall be created with \c CLOEXEC set in a racefree fashion, if
* possible.
*
* It is typically not required to register an alternative socket
* implementation
*
* @returns The socket file descriptor or -1 on error (\c errno must be set)
*/
virtual int socket_cb (int domain, int type, int protocol) = 0;
virtual ~SocketCb() { }
};
/**
* @brief \b Portability: OpenCb callback class
*
*/
class RD_EXPORT OpenCb {
public:
/**
* @brief Open callback
* The open callback is responsible for opening the file specified by
* \p pathname, using \p flags and \p mode.
* The file shall be opened with \c CLOEXEC set in a racefree fashion, if
* possible.
*
* It is typically not required to register an alternative open implementation
*
* @remark Not currently available on native Win32
*/
virtual int open_cb (const std::string &path, int flags, int mode) = 0;
virtual ~OpenCb() { }
};
/**@}*/
/**
* @name Configuration interface
* @{
*
*/
/**
* @brief Configuration interface
*
* Holds either global or topic configuration that are passed to
* RdKafka::Consumer::create(), RdKafka::Producer::create(),
* RdKafka::KafkaConsumer::create(), etc.
*
* @sa CONFIGURATION.md for the full list of supported properties.
*/
class RD_EXPORT Conf {
public:
/**
* @brief Configuration object type