forked from confluentinc/librdkafka
-
Notifications
You must be signed in to change notification settings - Fork 0
/
rdkafka.h
9295 lines (8330 loc) · 328 KB
/
rdkafka.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 library
*
* Copyright (c) 2012-2022 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.
*/
/**
* @file rdkafka.h
* @brief Apache Kafka C/C++ consumer and producer client library.
*
* rdkafka.h contains the public API for librdkafka.
* The API is documented in this file as comments prefixing the function, type,
* enum, define, etc.
*
* @sa For the C++ interface see rdkafkacpp.h
*
* @tableofcontents
*/
/* @cond NO_DOC */
#ifndef _RDKAFKA_H_
#define _RDKAFKA_H_
#include <stdio.h>
#include <inttypes.h>
#include <sys/types.h>
#ifdef __cplusplus
extern "C" {
#if 0
} /* Restore indent */
#endif
#endif
#ifdef _WIN32
#include <basetsd.h>
#ifndef WIN32_MEAN_AND_LEAN
#define WIN32_MEAN_AND_LEAN
#endif
#include <winsock2.h> /* for sockaddr, .. */
#ifndef _SSIZE_T_DEFINED
#define _SSIZE_T_DEFINED
typedef SSIZE_T ssize_t;
#endif
#define RD_UNUSED
#define RD_INLINE __inline
#define RD_DEPRECATED __declspec(deprecated)
#define RD_FORMAT(...)
#undef RD_EXPORT
#ifdef LIBRDKAFKA_STATICLIB
#define RD_EXPORT
#else
#ifdef LIBRDKAFKA_EXPORTS
#define RD_EXPORT __declspec(dllexport)
#else
#define RD_EXPORT __declspec(dllimport)
#endif
#ifndef LIBRDKAFKA_TYPECHECKS
#define LIBRDKAFKA_TYPECHECKS 0
#endif
#endif
#else
#include <sys/socket.h> /* for sockaddr, .. */
#define RD_UNUSED __attribute__((unused))
#define RD_INLINE inline
#define RD_EXPORT
#define RD_DEPRECATED __attribute__((deprecated))
#if defined(__clang__) || defined(__GNUC__) || defined(__GNUG__)
#define RD_FORMAT(...) __attribute__((format(__VA_ARGS__)))
#else
#define RD_FORMAT(...)
#endif
#ifndef LIBRDKAFKA_TYPECHECKS
#define LIBRDKAFKA_TYPECHECKS 1
#endif
#endif
/**
* @brief Type-checking macros
* Compile-time checking that \p ARG is of type \p TYPE.
* @returns \p RET
*/
#if LIBRDKAFKA_TYPECHECKS
#define _LRK_TYPECHECK(RET, TYPE, ARG) \
({ \
if (0) { \
TYPE __t RD_UNUSED = (ARG); \
} \
RET; \
})
#define _LRK_TYPECHECK2(RET, TYPE, ARG, TYPE2, ARG2) \
({ \
if (0) { \
TYPE __t RD_UNUSED = (ARG); \
TYPE2 __t2 RD_UNUSED = (ARG2); \
} \
RET; \
})
#define _LRK_TYPECHECK3(RET, TYPE, ARG, TYPE2, ARG2, TYPE3, ARG3) \
({ \
if (0) { \
TYPE __t RD_UNUSED = (ARG); \
TYPE2 __t2 RD_UNUSED = (ARG2); \
TYPE3 __t3 RD_UNUSED = (ARG3); \
} \
RET; \
})
#else
#define _LRK_TYPECHECK(RET, TYPE, ARG) (RET)
#define _LRK_TYPECHECK2(RET, TYPE, ARG, TYPE2, ARG2) (RET)
#define _LRK_TYPECHECK3(RET, TYPE, ARG, TYPE2, ARG2, TYPE3, ARG3) (RET)
#endif
/* @endcond */
/**
* @name librdkafka version
* @{
*
*
*/
/**
* @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 rd_kafka_version()
*/
#define RD_KAFKA_VERSION 0x020002ff
/**
* @brief Returns the librdkafka version as integer.
*
* @returns Version integer.
*
* @sa See RD_KAFKA_VERSION for how to parse the integer format.
* @sa Use rd_kafka_version_str() to retreive the version as a string.
*/
RD_EXPORT
int rd_kafka_version(void);
/**
* @brief Returns the librdkafka version as string.
*
* @returns Version string
*/
RD_EXPORT
const char *rd_kafka_version_str(void);
/**@}*/
/**
* @name Constants, errors, types
* @{
*
*
*/
/**
* @enum rd_kafka_type_t
*
* @brief rd_kafka_t handle type.
*
* @sa rd_kafka_new()
*/
typedef enum rd_kafka_type_t {
RD_KAFKA_PRODUCER, /**< Producer client */
RD_KAFKA_CONSUMER /**< Consumer client */
} rd_kafka_type_t;
/*!
* Timestamp types
*
* @sa rd_kafka_message_timestamp()
*/
typedef enum rd_kafka_timestamp_type_t {
RD_KAFKA_TIMESTAMP_NOT_AVAILABLE, /**< Timestamp not available */
RD_KAFKA_TIMESTAMP_CREATE_TIME, /**< Message creation time */
RD_KAFKA_TIMESTAMP_LOG_APPEND_TIME /**< Log append time */
} rd_kafka_timestamp_type_t;
/**
* @brief Retrieve supported debug contexts for use with the \c \"debug\"
* configuration property. (runtime)
*
* @returns Comma-separated list of available debugging contexts.
*/
RD_EXPORT
const char *rd_kafka_get_debug_contexts(void);
/**
* @brief Supported debug contexts. (compile time)
*
* @deprecated This compile time value may be outdated at runtime due to
* linking another version of the library.
* Use rd_kafka_get_debug_contexts() instead.
*/
#define RD_KAFKA_DEBUG_CONTEXTS \
"all,generic,broker,topic,metadata,feature,queue,msg,protocol,cgrp," \
"security,fetch,interceptor,plugin,consumer,admin,eos,mock,assignor," \
"conf"
/* @cond NO_DOC */
/* Private types to provide ABI compatibility */
typedef struct rd_kafka_s rd_kafka_t;
typedef struct rd_kafka_topic_s rd_kafka_topic_t;
typedef struct rd_kafka_conf_s rd_kafka_conf_t;
typedef struct rd_kafka_topic_conf_s rd_kafka_topic_conf_t;
typedef struct rd_kafka_queue_s rd_kafka_queue_t;
typedef struct rd_kafka_op_s rd_kafka_event_t;
typedef struct rd_kafka_topic_result_s rd_kafka_topic_result_t;
typedef struct rd_kafka_consumer_group_metadata_s
rd_kafka_consumer_group_metadata_t;
typedef struct rd_kafka_error_s rd_kafka_error_t;
typedef struct rd_kafka_headers_s rd_kafka_headers_t;
typedef struct rd_kafka_group_result_s rd_kafka_group_result_t;
typedef struct rd_kafka_acl_result_s rd_kafka_acl_result_t;
/* @endcond */
/**
* @enum rd_kafka_resp_err_t
* @brief Error codes.
*
* The negative error codes delimited by two underscores
* (\c RD_KAFKA_RESP_ERR__..) denotes errors internal to librdkafka and are
* displayed as \c \"Local: \<error string..\>\", while the error codes
* delimited by a single underscore (\c RD_KAFKA_RESP_ERR_..) denote broker
* errors and are displayed as \c \"Broker: \<error string..\>\".
*
* @sa Use rd_kafka_err2str() to translate an error code a human readable string
*/
typedef enum {
/* Internal errors to rdkafka: */
/** Begin internal error codes */
RD_KAFKA_RESP_ERR__BEGIN = -200,
/** Received message is incorrect */
RD_KAFKA_RESP_ERR__BAD_MSG = -199,
/** Bad/unknown compression */
RD_KAFKA_RESP_ERR__BAD_COMPRESSION = -198,
/** Broker is going away */
RD_KAFKA_RESP_ERR__DESTROY = -197,
/** Generic failure */
RD_KAFKA_RESP_ERR__FAIL = -196,
/** Broker transport failure */
RD_KAFKA_RESP_ERR__TRANSPORT = -195,
/** Critical system resource */
RD_KAFKA_RESP_ERR__CRIT_SYS_RESOURCE = -194,
/** Failed to resolve broker */
RD_KAFKA_RESP_ERR__RESOLVE = -193,
/** Produced message timed out*/
RD_KAFKA_RESP_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. */
RD_KAFKA_RESP_ERR__PARTITION_EOF = -191,
/** Permanent: Partition does not exist in cluster. */
RD_KAFKA_RESP_ERR__UNKNOWN_PARTITION = -190,
/** File or filesystem error */
RD_KAFKA_RESP_ERR__FS = -189,
/** Permanent: Topic does not exist in cluster. */
RD_KAFKA_RESP_ERR__UNKNOWN_TOPIC = -188,
/** All broker connections are down. */
RD_KAFKA_RESP_ERR__ALL_BROKERS_DOWN = -187,
/** Invalid argument, or invalid configuration */
RD_KAFKA_RESP_ERR__INVALID_ARG = -186,
/** Operation timed out */
RD_KAFKA_RESP_ERR__TIMED_OUT = -185,
/** Queue is full */
RD_KAFKA_RESP_ERR__QUEUE_FULL = -184,
/** ISR count < required.acks */
RD_KAFKA_RESP_ERR__ISR_INSUFF = -183,
/** Broker node update */
RD_KAFKA_RESP_ERR__NODE_UPDATE = -182,
/** SSL error */
RD_KAFKA_RESP_ERR__SSL = -181,
/** Waiting for coordinator to become available. */
RD_KAFKA_RESP_ERR__WAIT_COORD = -180,
/** Unknown client group */
RD_KAFKA_RESP_ERR__UNKNOWN_GROUP = -179,
/** Operation in progress */
RD_KAFKA_RESP_ERR__IN_PROGRESS = -178,
/** Previous operation in progress, wait for it to finish. */
RD_KAFKA_RESP_ERR__PREV_IN_PROGRESS = -177,
/** This operation would interfere with an existing subscription */
RD_KAFKA_RESP_ERR__EXISTING_SUBSCRIPTION = -176,
/** Assigned partitions (rebalance_cb) */
RD_KAFKA_RESP_ERR__ASSIGN_PARTITIONS = -175,
/** Revoked partitions (rebalance_cb) */
RD_KAFKA_RESP_ERR__REVOKE_PARTITIONS = -174,
/** Conflicting use */
RD_KAFKA_RESP_ERR__CONFLICT = -173,
/** Wrong state */
RD_KAFKA_RESP_ERR__STATE = -172,
/** Unknown protocol */
RD_KAFKA_RESP_ERR__UNKNOWN_PROTOCOL = -171,
/** Not implemented */
RD_KAFKA_RESP_ERR__NOT_IMPLEMENTED = -170,
/** Authentication failure*/
RD_KAFKA_RESP_ERR__AUTHENTICATION = -169,
/** No stored offset */
RD_KAFKA_RESP_ERR__NO_OFFSET = -168,
/** Outdated */
RD_KAFKA_RESP_ERR__OUTDATED = -167,
/** Timed out in queue */
RD_KAFKA_RESP_ERR__TIMED_OUT_QUEUE = -166,
/** Feature not supported by broker */
RD_KAFKA_RESP_ERR__UNSUPPORTED_FEATURE = -165,
/** Awaiting cache update */
RD_KAFKA_RESP_ERR__WAIT_CACHE = -164,
/** Operation interrupted (e.g., due to yield)) */
RD_KAFKA_RESP_ERR__INTR = -163,
/** Key serialization error */
RD_KAFKA_RESP_ERR__KEY_SERIALIZATION = -162,
/** Value serialization error */
RD_KAFKA_RESP_ERR__VALUE_SERIALIZATION = -161,
/** Key deserialization error */
RD_KAFKA_RESP_ERR__KEY_DESERIALIZATION = -160,
/** Value deserialization error */
RD_KAFKA_RESP_ERR__VALUE_DESERIALIZATION = -159,
/** Partial response */
RD_KAFKA_RESP_ERR__PARTIAL = -158,
/** Modification attempted on read-only object */
RD_KAFKA_RESP_ERR__READ_ONLY = -157,
/** No such entry / item not found */
RD_KAFKA_RESP_ERR__NOENT = -156,
/** Read underflow */
RD_KAFKA_RESP_ERR__UNDERFLOW = -155,
/** Invalid type */
RD_KAFKA_RESP_ERR__INVALID_TYPE = -154,
/** Retry operation */
RD_KAFKA_RESP_ERR__RETRY = -153,
/** Purged in queue */
RD_KAFKA_RESP_ERR__PURGE_QUEUE = -152,
/** Purged in flight */
RD_KAFKA_RESP_ERR__PURGE_INFLIGHT = -151,
/** Fatal error: see rd_kafka_fatal_error() */
RD_KAFKA_RESP_ERR__FATAL = -150,
/** Inconsistent state */
RD_KAFKA_RESP_ERR__INCONSISTENT = -149,
/** Gap-less ordering would not be guaranteed if proceeding */
RD_KAFKA_RESP_ERR__GAPLESS_GUARANTEE = -148,
/** Maximum poll interval exceeded */
RD_KAFKA_RESP_ERR__MAX_POLL_EXCEEDED = -147,
/** Unknown broker */
RD_KAFKA_RESP_ERR__UNKNOWN_BROKER = -146,
/** Functionality not configured */
RD_KAFKA_RESP_ERR__NOT_CONFIGURED = -145,
/** Instance has been fenced */
RD_KAFKA_RESP_ERR__FENCED = -144,
/** Application generated error */
RD_KAFKA_RESP_ERR__APPLICATION = -143,
/** Assignment lost */
RD_KAFKA_RESP_ERR__ASSIGNMENT_LOST = -142,
/** No operation performed */
RD_KAFKA_RESP_ERR__NOOP = -141,
/** No offset to automatically reset to */
RD_KAFKA_RESP_ERR__AUTO_OFFSET_RESET = -140,
/** End internal error codes */
RD_KAFKA_RESP_ERR__END = -100,
/* Kafka broker errors: */
/** Unknown broker error */
RD_KAFKA_RESP_ERR_UNKNOWN = -1,
/** Success */
RD_KAFKA_RESP_ERR_NO_ERROR = 0,
/** Offset out of range */
RD_KAFKA_RESP_ERR_OFFSET_OUT_OF_RANGE = 1,
/** Invalid message */
RD_KAFKA_RESP_ERR_INVALID_MSG = 2,
/** Unknown topic or partition */
RD_KAFKA_RESP_ERR_UNKNOWN_TOPIC_OR_PART = 3,
/** Invalid message size */
RD_KAFKA_RESP_ERR_INVALID_MSG_SIZE = 4,
/** Leader not available */
RD_KAFKA_RESP_ERR_LEADER_NOT_AVAILABLE = 5,
/** Not leader for partition */
RD_KAFKA_RESP_ERR_NOT_LEADER_FOR_PARTITION = 6,
/** Request timed out */
RD_KAFKA_RESP_ERR_REQUEST_TIMED_OUT = 7,
/** Broker not available */
RD_KAFKA_RESP_ERR_BROKER_NOT_AVAILABLE = 8,
/** Replica not available */
RD_KAFKA_RESP_ERR_REPLICA_NOT_AVAILABLE = 9,
/** Message size too large */
RD_KAFKA_RESP_ERR_MSG_SIZE_TOO_LARGE = 10,
/** StaleControllerEpochCode */
RD_KAFKA_RESP_ERR_STALE_CTRL_EPOCH = 11,
/** Offset metadata string too large */
RD_KAFKA_RESP_ERR_OFFSET_METADATA_TOO_LARGE = 12,
/** Broker disconnected before response received */
RD_KAFKA_RESP_ERR_NETWORK_EXCEPTION = 13,
/** Coordinator load in progress */
RD_KAFKA_RESP_ERR_COORDINATOR_LOAD_IN_PROGRESS = 14,
/** Group coordinator load in progress */
#define RD_KAFKA_RESP_ERR_GROUP_LOAD_IN_PROGRESS \
RD_KAFKA_RESP_ERR_COORDINATOR_LOAD_IN_PROGRESS
/** Coordinator not available */
RD_KAFKA_RESP_ERR_COORDINATOR_NOT_AVAILABLE = 15,
/** Group coordinator not available */
#define RD_KAFKA_RESP_ERR_GROUP_COORDINATOR_NOT_AVAILABLE \
RD_KAFKA_RESP_ERR_COORDINATOR_NOT_AVAILABLE
/** Not coordinator */
RD_KAFKA_RESP_ERR_NOT_COORDINATOR = 16,
/** Not coordinator for group */
#define RD_KAFKA_RESP_ERR_NOT_COORDINATOR_FOR_GROUP \
RD_KAFKA_RESP_ERR_NOT_COORDINATOR
/** Invalid topic */
RD_KAFKA_RESP_ERR_TOPIC_EXCEPTION = 17,
/** Message batch larger than configured server segment size */
RD_KAFKA_RESP_ERR_RECORD_LIST_TOO_LARGE = 18,
/** Not enough in-sync replicas */
RD_KAFKA_RESP_ERR_NOT_ENOUGH_REPLICAS = 19,
/** Message(s) written to insufficient number of in-sync replicas */
RD_KAFKA_RESP_ERR_NOT_ENOUGH_REPLICAS_AFTER_APPEND = 20,
/** Invalid required acks value */
RD_KAFKA_RESP_ERR_INVALID_REQUIRED_ACKS = 21,
/** Specified group generation id is not valid */
RD_KAFKA_RESP_ERR_ILLEGAL_GENERATION = 22,
/** Inconsistent group protocol */
RD_KAFKA_RESP_ERR_INCONSISTENT_GROUP_PROTOCOL = 23,
/** Invalid group.id */
RD_KAFKA_RESP_ERR_INVALID_GROUP_ID = 24,
/** Unknown member */
RD_KAFKA_RESP_ERR_UNKNOWN_MEMBER_ID = 25,
/** Invalid session timeout */
RD_KAFKA_RESP_ERR_INVALID_SESSION_TIMEOUT = 26,
/** Group rebalance in progress */
RD_KAFKA_RESP_ERR_REBALANCE_IN_PROGRESS = 27,
/** Commit offset data size is not valid */
RD_KAFKA_RESP_ERR_INVALID_COMMIT_OFFSET_SIZE = 28,
/** Topic authorization failed */
RD_KAFKA_RESP_ERR_TOPIC_AUTHORIZATION_FAILED = 29,
/** Group authorization failed */
RD_KAFKA_RESP_ERR_GROUP_AUTHORIZATION_FAILED = 30,
/** Cluster authorization failed */
RD_KAFKA_RESP_ERR_CLUSTER_AUTHORIZATION_FAILED = 31,
/** Invalid timestamp */
RD_KAFKA_RESP_ERR_INVALID_TIMESTAMP = 32,
/** Unsupported SASL mechanism */
RD_KAFKA_RESP_ERR_UNSUPPORTED_SASL_MECHANISM = 33,
/** Illegal SASL state */
RD_KAFKA_RESP_ERR_ILLEGAL_SASL_STATE = 34,
/** Unuspported version */
RD_KAFKA_RESP_ERR_UNSUPPORTED_VERSION = 35,
/** Topic already exists */
RD_KAFKA_RESP_ERR_TOPIC_ALREADY_EXISTS = 36,
/** Invalid number of partitions */
RD_KAFKA_RESP_ERR_INVALID_PARTITIONS = 37,
/** Invalid replication factor */
RD_KAFKA_RESP_ERR_INVALID_REPLICATION_FACTOR = 38,
/** Invalid replica assignment */
RD_KAFKA_RESP_ERR_INVALID_REPLICA_ASSIGNMENT = 39,
/** Invalid config */
RD_KAFKA_RESP_ERR_INVALID_CONFIG = 40,
/** Not controller for cluster */
RD_KAFKA_RESP_ERR_NOT_CONTROLLER = 41,
/** Invalid request */
RD_KAFKA_RESP_ERR_INVALID_REQUEST = 42,
/** Message format on broker does not support request */
RD_KAFKA_RESP_ERR_UNSUPPORTED_FOR_MESSAGE_FORMAT = 43,
/** Policy violation */
RD_KAFKA_RESP_ERR_POLICY_VIOLATION = 44,
/** Broker received an out of order sequence number */
RD_KAFKA_RESP_ERR_OUT_OF_ORDER_SEQUENCE_NUMBER = 45,
/** Broker received a duplicate sequence number */
RD_KAFKA_RESP_ERR_DUPLICATE_SEQUENCE_NUMBER = 46,
/** Producer attempted an operation with an old epoch */
RD_KAFKA_RESP_ERR_INVALID_PRODUCER_EPOCH = 47,
/** Producer attempted a transactional operation in an invalid state */
RD_KAFKA_RESP_ERR_INVALID_TXN_STATE = 48,
/** Producer attempted to use a producer id which is not
* currently assigned to its transactional id */
RD_KAFKA_RESP_ERR_INVALID_PRODUCER_ID_MAPPING = 49,
/** Transaction timeout is larger than the maximum
* value allowed by the broker's max.transaction.timeout.ms */
RD_KAFKA_RESP_ERR_INVALID_TRANSACTION_TIMEOUT = 50,
/** Producer attempted to update a transaction while another
* concurrent operation on the same transaction was ongoing */
RD_KAFKA_RESP_ERR_CONCURRENT_TRANSACTIONS = 51,
/** Indicates that the transaction coordinator sending a
* WriteTxnMarker is no longer the current coordinator for a
* given producer */
RD_KAFKA_RESP_ERR_TRANSACTION_COORDINATOR_FENCED = 52,
/** Transactional Id authorization failed */
RD_KAFKA_RESP_ERR_TRANSACTIONAL_ID_AUTHORIZATION_FAILED = 53,
/** Security features are disabled */
RD_KAFKA_RESP_ERR_SECURITY_DISABLED = 54,
/** Operation not attempted */
RD_KAFKA_RESP_ERR_OPERATION_NOT_ATTEMPTED = 55,
/** Disk error when trying to access log file on the disk */
RD_KAFKA_RESP_ERR_KAFKA_STORAGE_ERROR = 56,
/** The user-specified log directory is not found in the broker config
*/
RD_KAFKA_RESP_ERR_LOG_DIR_NOT_FOUND = 57,
/** SASL Authentication failed */
RD_KAFKA_RESP_ERR_SASL_AUTHENTICATION_FAILED = 58,
/** Unknown Producer Id */
RD_KAFKA_RESP_ERR_UNKNOWN_PRODUCER_ID = 59,
/** Partition reassignment is in progress */
RD_KAFKA_RESP_ERR_REASSIGNMENT_IN_PROGRESS = 60,
/** Delegation Token feature is not enabled */
RD_KAFKA_RESP_ERR_DELEGATION_TOKEN_AUTH_DISABLED = 61,
/** Delegation Token is not found on server */
RD_KAFKA_RESP_ERR_DELEGATION_TOKEN_NOT_FOUND = 62,
/** Specified Principal is not valid Owner/Renewer */
RD_KAFKA_RESP_ERR_DELEGATION_TOKEN_OWNER_MISMATCH = 63,
/** Delegation Token requests are not allowed on this connection */
RD_KAFKA_RESP_ERR_DELEGATION_TOKEN_REQUEST_NOT_ALLOWED = 64,
/** Delegation Token authorization failed */
RD_KAFKA_RESP_ERR_DELEGATION_TOKEN_AUTHORIZATION_FAILED = 65,
/** Delegation Token is expired */
RD_KAFKA_RESP_ERR_DELEGATION_TOKEN_EXPIRED = 66,
/** Supplied principalType is not supported */
RD_KAFKA_RESP_ERR_INVALID_PRINCIPAL_TYPE = 67,
/** The group is not empty */
RD_KAFKA_RESP_ERR_NON_EMPTY_GROUP = 68,
/** The group id does not exist */
RD_KAFKA_RESP_ERR_GROUP_ID_NOT_FOUND = 69,
/** The fetch session ID was not found */
RD_KAFKA_RESP_ERR_FETCH_SESSION_ID_NOT_FOUND = 70,
/** The fetch session epoch is invalid */
RD_KAFKA_RESP_ERR_INVALID_FETCH_SESSION_EPOCH = 71,
/** No matching listener */
RD_KAFKA_RESP_ERR_LISTENER_NOT_FOUND = 72,
/** Topic deletion is disabled */
RD_KAFKA_RESP_ERR_TOPIC_DELETION_DISABLED = 73,
/** Leader epoch is older than broker epoch */
RD_KAFKA_RESP_ERR_FENCED_LEADER_EPOCH = 74,
/** Leader epoch is newer than broker epoch */
RD_KAFKA_RESP_ERR_UNKNOWN_LEADER_EPOCH = 75,
/** Unsupported compression type */
RD_KAFKA_RESP_ERR_UNSUPPORTED_COMPRESSION_TYPE = 76,
/** Broker epoch has changed */
RD_KAFKA_RESP_ERR_STALE_BROKER_EPOCH = 77,
/** Leader high watermark is not caught up */
RD_KAFKA_RESP_ERR_OFFSET_NOT_AVAILABLE = 78,
/** Group member needs a valid member ID */
RD_KAFKA_RESP_ERR_MEMBER_ID_REQUIRED = 79,
/** Preferred leader was not available */
RD_KAFKA_RESP_ERR_PREFERRED_LEADER_NOT_AVAILABLE = 80,
/** Consumer group has reached maximum size */
RD_KAFKA_RESP_ERR_GROUP_MAX_SIZE_REACHED = 81,
/** Static consumer fenced by other consumer with same
* group.instance.id. */
RD_KAFKA_RESP_ERR_FENCED_INSTANCE_ID = 82,
/** Eligible partition leaders are not available */
RD_KAFKA_RESP_ERR_ELIGIBLE_LEADERS_NOT_AVAILABLE = 83,
/** Leader election not needed for topic partition */
RD_KAFKA_RESP_ERR_ELECTION_NOT_NEEDED = 84,
/** No partition reassignment is in progress */
RD_KAFKA_RESP_ERR_NO_REASSIGNMENT_IN_PROGRESS = 85,
/** Deleting offsets of a topic while the consumer group is
* subscribed to it */
RD_KAFKA_RESP_ERR_GROUP_SUBSCRIBED_TO_TOPIC = 86,
/** Broker failed to validate record */
RD_KAFKA_RESP_ERR_INVALID_RECORD = 87,
/** There are unstable offsets that need to be cleared */
RD_KAFKA_RESP_ERR_UNSTABLE_OFFSET_COMMIT = 88,
/** Throttling quota has been exceeded */
RD_KAFKA_RESP_ERR_THROTTLING_QUOTA_EXCEEDED = 89,
/** There is a newer producer with the same transactionalId
* which fences the current one */
RD_KAFKA_RESP_ERR_PRODUCER_FENCED = 90,
/** Request illegally referred to resource that does not exist */
RD_KAFKA_RESP_ERR_RESOURCE_NOT_FOUND = 91,
/** Request illegally referred to the same resource twice */
RD_KAFKA_RESP_ERR_DUPLICATE_RESOURCE = 92,
/** Requested credential would not meet criteria for acceptability */
RD_KAFKA_RESP_ERR_UNACCEPTABLE_CREDENTIAL = 93,
/** Indicates that the either the sender or recipient of a
* voter-only request is not one of the expected voters */
RD_KAFKA_RESP_ERR_INCONSISTENT_VOTER_SET = 94,
/** Invalid update version */
RD_KAFKA_RESP_ERR_INVALID_UPDATE_VERSION = 95,
/** Unable to update finalized features due to server error */
RD_KAFKA_RESP_ERR_FEATURE_UPDATE_FAILED = 96,
/** Request principal deserialization failed during forwarding */
RD_KAFKA_RESP_ERR_PRINCIPAL_DESERIALIZATION_FAILURE = 97,
RD_KAFKA_RESP_ERR_END_ALL,
} rd_kafka_resp_err_t;
/**
* @brief Error code value, name and description.
* Typically for use with language bindings to automatically expose
* the full set of librdkafka error codes.
*/
struct rd_kafka_err_desc {
rd_kafka_resp_err_t code; /**< Error code */
const char *name; /**< Error name, same as code enum sans prefix */
const char *desc; /**< Human readable error description. */
};
/**
* @brief Returns the full list of error codes.
*/
RD_EXPORT
void rd_kafka_get_err_descs(const struct rd_kafka_err_desc **errdescs,
size_t *cntp);
/**
* @brief Returns a human readable representation of a kafka error.
*
* @param err Error code to translate
*/
RD_EXPORT
const char *rd_kafka_err2str(rd_kafka_resp_err_t err);
/**
* @brief Returns the error code name (enum name).
*
* @param err Error code to translate
*/
RD_EXPORT
const char *rd_kafka_err2name(rd_kafka_resp_err_t err);
/**
* @brief Returns the last error code generated by a legacy API call
* in the current thread.
*
* The legacy APIs are the ones using errno to propagate error value, namely:
* - rd_kafka_topic_new()
* - rd_kafka_consume_start()
* - rd_kafka_consume_stop()
* - rd_kafka_consume()
* - rd_kafka_consume_batch()
* - rd_kafka_consume_callback()
* - rd_kafka_consume_queue()
* - rd_kafka_produce()
*
* The main use for this function is to avoid converting system \p errno
* values to rd_kafka_resp_err_t codes for legacy APIs.
*
* @remark The last error is stored per-thread, if multiple rd_kafka_t handles
* are used in the same application thread the developer needs to
* make sure rd_kafka_last_error() is called immediately after
* a failed API call.
*
* @remark errno propagation from librdkafka is not safe on Windows
* and should not be used, use rd_kafka_last_error() instead.
*/
RD_EXPORT
rd_kafka_resp_err_t rd_kafka_last_error(void);
/**
* @brief Converts the system errno value \p errnox to a rd_kafka_resp_err_t
* error code upon failure from the following functions:
* - rd_kafka_topic_new()
* - rd_kafka_consume_start()
* - rd_kafka_consume_stop()
* - rd_kafka_consume()
* - rd_kafka_consume_batch()
* - rd_kafka_consume_callback()
* - rd_kafka_consume_queue()
* - rd_kafka_produce()
*
* @param errnox System errno value to convert
*
* @returns Appropriate error code for \p errnox
*
* @remark A better alternative is to call rd_kafka_last_error() immediately
* after any of the above functions return -1 or NULL.
*
* @deprecated Use rd_kafka_last_error() to retrieve the last error code
* set by the legacy librdkafka APIs.
*
* @sa rd_kafka_last_error()
*/
RD_EXPORT RD_DEPRECATED rd_kafka_resp_err_t rd_kafka_errno2err(int errnox);
/**
* @brief Returns the thread-local system errno
*
* On most platforms this is the same as \p errno but in case of different
* runtimes between library and application (e.g., Windows static DLLs)
* this provides a means for exposing the errno librdkafka uses.
*
* @remark The value is local to the current calling thread.
*
* @deprecated Use rd_kafka_last_error() to retrieve the last error code
* set by the legacy librdkafka APIs.
*/
RD_EXPORT RD_DEPRECATED int rd_kafka_errno(void);
/**
* @brief Returns the first fatal error set on this client instance,
* or RD_KAFKA_RESP_ERR_NO_ERROR if no fatal error has occurred.
*
* This function is to be used with the Idempotent Producer and \c error_cb
* to detect fatal errors.
*
* Generally all errors raised by \c error_cb are to be considered
* informational and temporary, the client will try to recover from all
* errors in a graceful fashion (by retrying, etc).
*
* However, some errors should logically be considered fatal to retain
* consistency; in particular a set of errors that may occur when using the
* Idempotent Producer and the in-order or exactly-once producer guarantees
* can't be satisfied.
*
* @param rk Client instance.
* @param errstr A human readable error string (nul-terminated) is written to
* this location that must be of at least \p errstr_size bytes.
* The \p errstr is only written to if there is a fatal error.
* @param errstr_size Writable size in \p errstr.
*
*
* @returns RD_KAFKA_RESP_ERR_NO_ERROR if no fatal error has been raised, else
* any other error code.
*/
RD_EXPORT
rd_kafka_resp_err_t
rd_kafka_fatal_error(rd_kafka_t *rk, char *errstr, size_t errstr_size);
/**
* @brief Trigger a fatal error for testing purposes.
*
* Since there is no practical way to trigger real fatal errors in the
* idempotent producer, this method allows an application to trigger
* fabricated fatal errors in tests to check its error handling code.
*
* @param rk Client instance.
* @param err The underlying error code.
* @param reason A human readable error reason.
* Will be prefixed with "test_fatal_error: " to differentiate
* from real fatal errors.
*
* @returns RD_KAFKA_RESP_ERR_NO_ERROR if a fatal error was triggered, or
* RD_KAFKA_RESP_ERR__PREV_IN_PROGRESS if a previous fatal error
* has already been triggered.
*/
RD_EXPORT rd_kafka_resp_err_t rd_kafka_test_fatal_error(rd_kafka_t *rk,
rd_kafka_resp_err_t err,
const char *reason);
/**
* @brief Sets the idempotent producer PID and epoch for testing purposes.
*
* @param rk Client instance.
* @param id The producer ID.
* @param epoch The producer ID epoch.
*
* @returns RD_KAFKA_RESP_ERR_NO_ERROR on success.
* RD_KAFKA_RESP_ERR__NOT_CONFIGURED if idempotence have not been
* configured for the producer instance,
* RD_KAFKA_RESP_ERR__INVALID_ARG if the \p id is not PID.
*/
RD_EXPORT
rd_kafka_resp_err_t
rd_kafka_test_idemp_set_pid(rd_kafka_t *rk, int64_t id, int16_t epoch);
/**
* @brief Gets the current assigned PID
*
* @param rk Client instance.
* @param id The returned value of producer ID.
* @param epoch The returned value of producer ID epoch.
*
* @returns RD_KAFKA_RESP_ERR_NO_ERROR on success.
* RD_KAFKA_RESP_ERR__NOT_CONFIGURED if idempotence have not been
* configured for the producer instance,
* RD_KAFKA_RESP_ERR__INVALID_ARG if the \p id is not PID.
*/
RD_EXPORT
rd_kafka_resp_err_t
rd_kafka_test_idemp_get_pid(rd_kafka_t *rk, int64_t *id, int16_t *epoch);
/**
* @returns the error code for \p error or RD_KAFKA_RESP_ERR_NO_ERROR if
* \p error is NULL.
*/
RD_EXPORT
rd_kafka_resp_err_t rd_kafka_error_code(const rd_kafka_error_t *error);
/**
* @returns the error code name for \p error, e.g, "ERR_UNKNOWN_MEMBER_ID",
* or an empty string if \p error is NULL.
*
* @remark The lifetime of the returned pointer is the same as the error object.
*
* @sa rd_kafka_err2name()
*/
RD_EXPORT
const char *rd_kafka_error_name(const rd_kafka_error_t *error);
/**
* @returns a human readable error string for \p error,
* or an empty string if \p error is NULL.
*
* @remark The lifetime of the returned pointer is the same as the error object.
*/
RD_EXPORT
const char *rd_kafka_error_string(const rd_kafka_error_t *error);
/**
* @returns 1 if the error is a fatal error, indicating that the client
* instance is no longer usable, else 0 (also if \p error is NULL).
*/
RD_EXPORT
int rd_kafka_error_is_fatal(const rd_kafka_error_t *error);
/**
* @returns 1 if the operation may be retried,
* else 0 (also if \p error is NULL).
*/
RD_EXPORT
int rd_kafka_error_is_retriable(const rd_kafka_error_t *error);
/**
* @returns 1 if the error is an abortable transaction error in which case
* the application must call rd_kafka_abort_transaction() and
* start a new transaction with rd_kafka_begin_transaction() if it
* wishes to proceed with transactions.
* Else returns 0 (also if \p error is NULL).
*
* @remark The return value of this method is only valid for errors returned
* by the transactional API.
*/
RD_EXPORT
int rd_kafka_error_txn_requires_abort(const rd_kafka_error_t *error);
/**
* @brief Free and destroy an error object.
*
* @remark As a conveniance it is permitted to pass a NULL \p error.
*/
RD_EXPORT
void rd_kafka_error_destroy(rd_kafka_error_t *error);
/**
* @brief Create a new error object with error \p code and optional
* human readable error string in \p fmt.
*
* This method is mainly to be used for mocking errors in application test code.
*
* The returned object must be destroyed with rd_kafka_error_destroy().
*/
RD_EXPORT
rd_kafka_error_t *rd_kafka_error_new(rd_kafka_resp_err_t code,
const char *fmt,
...) RD_FORMAT(printf, 2, 3);
/**
* @brief Topic+Partition place holder
*
* Generic place holder for a Topic+Partition and its related information
* used for multiple purposes:
* - consumer offset (see rd_kafka_commit(), et.al.)
* - group rebalancing callback (rd_kafka_conf_set_rebalance_cb())
* - offset commit result callback (rd_kafka_conf_set_offset_commit_cb())
*/
/**
* @brief Generic place holder for a specific Topic+Partition.
*
* @sa rd_kafka_topic_partition_list_new()
*/
typedef struct rd_kafka_topic_partition_s {
char *topic; /**< Topic name */
int32_t partition; /**< Partition */
int64_t offset; /**< Offset */
void *metadata; /**< Metadata */
size_t metadata_size; /**< Metadata size */
void *opaque; /**< Opaque value for application use */
rd_kafka_resp_err_t err; /**< Error code, depending on use. */
void *_private; /**< INTERNAL USE ONLY,
* INITIALIZE TO ZERO, DO NOT TOUCH */
} rd_kafka_topic_partition_t;
/**
* @brief Destroy a rd_kafka_topic_partition_t.
* @remark This must not be called for elements in a topic partition list.
*/
RD_EXPORT
void rd_kafka_topic_partition_destroy(rd_kafka_topic_partition_t *rktpar);
/**
* @brief A growable list of Topic+Partitions.
*
*/
typedef struct rd_kafka_topic_partition_list_s {
int cnt; /**< Current number of elements */
int size; /**< Current allocated size */
rd_kafka_topic_partition_t *elems; /**< Element array[] */
} rd_kafka_topic_partition_list_t;
/**
* @brief Create a new list/vector Topic+Partition container.
*
* @param size Initial allocated size used when the expected number of
* elements is known or can be estimated.
* Avoids reallocation and possibly relocation of the
* elems array.
*
* @returns A newly allocated Topic+Partition list.
*
* @remark Use rd_kafka_topic_partition_list_destroy() to free all resources
* in use by a list and the list itself.
* @sa rd_kafka_topic_partition_list_add()
*/
RD_EXPORT
rd_kafka_topic_partition_list_t *rd_kafka_topic_partition_list_new(int size);
/**
* @brief Free all resources used by the list and the list itself.
*/
RD_EXPORT
void rd_kafka_topic_partition_list_destroy(
rd_kafka_topic_partition_list_t *rkparlist);
/**
* @brief Add topic+partition to list
*
* @param rktparlist List to extend
* @param topic Topic name (copied)
* @param partition Partition id
*
* @returns The object which can be used to fill in additionals fields.
*/
RD_EXPORT
rd_kafka_topic_partition_t *
rd_kafka_topic_partition_list_add(rd_kafka_topic_partition_list_t *rktparlist,
const char *topic,
int32_t partition);
/**
* @brief Add range of partitions from \p start to \p stop inclusive.