-
Notifications
You must be signed in to change notification settings - Fork 3.2k
/
rdkafka.c
5116 lines (4249 loc) · 189 KB
/
rdkafka.c
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
* 2023, Confluent Inc.
* 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.
*/
#define _GNU_SOURCE
#include <errno.h>
#include <string.h>
#include <stdarg.h>
#include <signal.h>
#include <stdlib.h>
#include <sys/stat.h>
#if !_WIN32
#include <sys/types.h>
#include <dirent.h>
#endif
#include "rdkafka_int.h"
#include "rdkafka_msg.h"
#include "rdkafka_broker.h"
#include "rdkafka_topic.h"
#include "rdkafka_partition.h"
#include "rdkafka_offset.h"
#include "rdkafka_transport.h"
#include "rdkafka_cgrp.h"
#include "rdkafka_assignor.h"
#include "rdkafka_request.h"
#include "rdkafka_event.h"
#include "rdkafka_error.h"
#include "rdkafka_sasl.h"
#include "rdkafka_interceptor.h"
#include "rdkafka_idempotence.h"
#include "rdkafka_sasl_oauthbearer.h"
#if WITH_OAUTHBEARER_OIDC
#include "rdkafka_sasl_oauthbearer_oidc.h"
#endif
#if WITH_SSL
#include "rdkafka_ssl.h"
#endif
#include "rdtime.h"
#include "crc32c.h"
#include "rdunittest.h"
#ifdef _WIN32
#include <sys/types.h>
#include <sys/timeb.h>
#endif
#define CJSON_HIDE_SYMBOLS
#include "cJSON.h"
#if WITH_CURL
#include "rdhttp.h"
#endif
static once_flag rd_kafka_global_init_once = ONCE_FLAG_INIT;
static once_flag rd_kafka_global_srand_once = ONCE_FLAG_INIT;
/**
* @brief Global counter+lock for all active librdkafka instances
*/
mtx_t rd_kafka_global_lock;
int rd_kafka_global_cnt;
/**
* Last API error code, per thread.
* Shared among all rd_kafka_t instances.
*/
rd_kafka_resp_err_t RD_TLS rd_kafka_last_error_code;
/**
* Current number of threads created by rdkafka.
* This is used in regression tests.
*/
rd_atomic32_t rd_kafka_thread_cnt_curr;
int rd_kafka_thread_cnt(void) {
return rd_atomic32_get(&rd_kafka_thread_cnt_curr);
}
/**
* Current thread's log name (TLS)
*/
char RD_TLS rd_kafka_thread_name[64] = "app";
void rd_kafka_set_thread_name(const char *fmt, ...) {
va_list ap;
va_start(ap, fmt);
rd_vsnprintf(rd_kafka_thread_name, sizeof(rd_kafka_thread_name), fmt,
ap);
va_end(ap);
}
/**
* @brief Current thread's system name (TLS)
*
* Note the name must be 15 characters or less, because it is passed to
* pthread_setname_np on Linux which imposes this limit.
*/
static char RD_TLS rd_kafka_thread_sysname[16] = "app";
void rd_kafka_set_thread_sysname(const char *fmt, ...) {
va_list ap;
va_start(ap, fmt);
rd_vsnprintf(rd_kafka_thread_sysname, sizeof(rd_kafka_thread_sysname),
fmt, ap);
va_end(ap);
thrd_setname(rd_kafka_thread_sysname);
}
static void rd_kafka_global_init0(void) {
cJSON_Hooks json_hooks = {.malloc_fn = rd_malloc, .free_fn = rd_free};
mtx_init(&rd_kafka_global_lock, mtx_plain);
#if ENABLE_DEVEL
rd_atomic32_init(&rd_kafka_op_cnt, 0);
#endif
rd_crc32c_global_init();
#if WITH_SSL
/* The configuration interface might need to use
* OpenSSL to parse keys, prior to any rd_kafka_t
* object has been created. */
rd_kafka_ssl_init();
#endif
cJSON_InitHooks(&json_hooks);
#if WITH_CURL
rd_http_global_init();
#endif
}
/**
* @brief Initialize once per process
*/
void rd_kafka_global_init(void) {
call_once(&rd_kafka_global_init_once, rd_kafka_global_init0);
}
/**
* @brief Seed the PRNG with current_time.milliseconds
*/
static void rd_kafka_global_srand(void) {
struct timeval tv;
rd_gettimeofday(&tv, NULL);
srand((unsigned int)(tv.tv_usec / 1000));
}
/**
* @returns the current number of active librdkafka instances
*/
static int rd_kafka_global_cnt_get(void) {
int r;
mtx_lock(&rd_kafka_global_lock);
r = rd_kafka_global_cnt;
mtx_unlock(&rd_kafka_global_lock);
return r;
}
/**
* @brief Increase counter for active librdkafka instances.
* If this is the first instance the global constructors will be called, if any.
*/
static void rd_kafka_global_cnt_incr(void) {
mtx_lock(&rd_kafka_global_lock);
rd_kafka_global_cnt++;
if (rd_kafka_global_cnt == 1) {
rd_kafka_transport_init();
#if WITH_SSL
rd_kafka_ssl_init();
#endif
rd_kafka_sasl_global_init();
}
mtx_unlock(&rd_kafka_global_lock);
}
/**
* @brief Decrease counter for active librdkafka instances.
* If this counter reaches 0 the global destructors will be called, if any.
*/
static void rd_kafka_global_cnt_decr(void) {
mtx_lock(&rd_kafka_global_lock);
rd_kafka_assert(NULL, rd_kafka_global_cnt > 0);
rd_kafka_global_cnt--;
if (rd_kafka_global_cnt == 0) {
rd_kafka_sasl_global_term();
#if WITH_SSL
rd_kafka_ssl_term();
#endif
}
mtx_unlock(&rd_kafka_global_lock);
}
/**
* 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.
*/
int rd_kafka_wait_destroyed(int timeout_ms) {
rd_ts_t timeout = rd_clock() + (timeout_ms * 1000);
while (rd_kafka_thread_cnt() > 0 || rd_kafka_global_cnt_get() > 0) {
if (rd_clock() >= timeout) {
rd_kafka_set_last_error(RD_KAFKA_RESP_ERR__TIMED_OUT,
ETIMEDOUT);
return -1;
}
rd_usleep(25000, NULL); /* 25ms */
}
return 0;
}
static void rd_kafka_log_buf(const rd_kafka_conf_t *conf,
const rd_kafka_t *rk,
int level,
int ctx,
const char *fac,
const char *buf) {
if (level > conf->log_level)
return;
else if (rk && conf->log_queue) {
rd_kafka_op_t *rko;
if (!rk->rk_logq)
return; /* Terminating */
rko = rd_kafka_op_new(RD_KAFKA_OP_LOG);
rd_kafka_op_set_prio(rko, RD_KAFKA_PRIO_MEDIUM);
rko->rko_u.log.level = level;
rd_strlcpy(rko->rko_u.log.fac, fac, sizeof(rko->rko_u.log.fac));
rko->rko_u.log.str = rd_strdup(buf);
rko->rko_u.log.ctx = ctx;
rd_kafka_q_enq(rk->rk_logq, rko);
} else if (conf->log_cb) {
conf->log_cb(rk, level, fac, buf);
}
}
/**
* @brief Logger
*
* @remark conf must be set, but rk may be NULL
*/
void rd_kafka_log0(const rd_kafka_conf_t *conf,
const rd_kafka_t *rk,
const char *extra,
int level,
int ctx,
const char *fac,
const char *fmt,
...) {
char buf[2048];
va_list ap;
unsigned int elen = 0;
unsigned int of = 0;
if (level > conf->log_level)
return;
if (conf->log_thread_name) {
elen = rd_snprintf(buf, sizeof(buf),
"[thrd:%s]: ", rd_kafka_thread_name);
if (unlikely(elen >= sizeof(buf)))
elen = sizeof(buf);
of = elen;
}
if (extra) {
elen = rd_snprintf(buf + of, sizeof(buf) - of, "%s: ", extra);
if (unlikely(elen >= sizeof(buf) - of))
elen = sizeof(buf) - of;
of += elen;
}
va_start(ap, fmt);
rd_vsnprintf(buf + of, sizeof(buf) - of, fmt, ap);
va_end(ap);
rd_kafka_log_buf(conf, rk, level, ctx, fac, buf);
}
rd_kafka_resp_err_t
rd_kafka_oauthbearer_set_token(rd_kafka_t *rk,
const char *token_value,
int64_t md_lifetime_ms,
const char *md_principal_name,
const char **extensions,
size_t extension_size,
char *errstr,
size_t errstr_size) {
#if WITH_SASL_OAUTHBEARER
return rd_kafka_oauthbearer_set_token0(
rk, token_value, md_lifetime_ms, md_principal_name, extensions,
extension_size, errstr, errstr_size);
#else
rd_snprintf(errstr, errstr_size,
"librdkafka not built with SASL OAUTHBEARER support");
return RD_KAFKA_RESP_ERR__NOT_IMPLEMENTED;
#endif
}
rd_kafka_resp_err_t rd_kafka_oauthbearer_set_token_failure(rd_kafka_t *rk,
const char *errstr) {
#if WITH_SASL_OAUTHBEARER
return rd_kafka_oauthbearer_set_token_failure0(rk, errstr);
#else
return RD_KAFKA_RESP_ERR__NOT_IMPLEMENTED;
#endif
}
void rd_kafka_log_print(const rd_kafka_t *rk,
int level,
const char *fac,
const char *buf) {
int secs, msecs;
struct timeval tv;
rd_gettimeofday(&tv, NULL);
secs = (int)tv.tv_sec;
msecs = (int)(tv.tv_usec / 1000);
fprintf(stderr, "%%%i|%u.%03u|%s|%s| %s\n", level, secs, msecs, fac,
rk ? rk->rk_name : "", buf);
}
void rd_kafka_log_syslog(const rd_kafka_t *rk,
int level,
const char *fac,
const char *buf) {
#if WITH_SYSLOG
static int initialized = 0;
if (!initialized)
openlog("rdkafka", LOG_PID | LOG_CONS, LOG_USER);
syslog(level, "%s: %s: %s", fac, rk ? rk->rk_name : "", buf);
#else
rd_assert(!*"syslog support not enabled in this build");
#endif
}
void rd_kafka_set_logger(rd_kafka_t *rk,
void (*func)(const rd_kafka_t *rk,
int level,
const char *fac,
const char *buf)) {
#if !WITH_SYSLOG
if (func == rd_kafka_log_syslog)
rd_assert(!*"syslog support not enabled in this build");
#endif
rk->rk_conf.log_cb = func;
}
void rd_kafka_set_log_level(rd_kafka_t *rk, int level) {
rk->rk_conf.log_level = level;
}
static const char *rd_kafka_type2str(rd_kafka_type_t type) {
static const char *types[] = {
[RD_KAFKA_PRODUCER] = "producer",
[RD_KAFKA_CONSUMER] = "consumer",
};
return types[type];
}
#define _ERR_DESC(ENUM, DESC) \
[ENUM - RD_KAFKA_RESP_ERR__BEGIN] = {ENUM, &(#ENUM)[18] /*pfx*/, DESC}
static const struct rd_kafka_err_desc rd_kafka_err_descs[] = {
_ERR_DESC(RD_KAFKA_RESP_ERR__BEGIN, NULL),
_ERR_DESC(RD_KAFKA_RESP_ERR__BAD_MSG, "Local: Bad message format"),
_ERR_DESC(RD_KAFKA_RESP_ERR__BAD_COMPRESSION,
"Local: Invalid compressed data"),
_ERR_DESC(RD_KAFKA_RESP_ERR__DESTROY, "Local: Broker handle destroyed"),
_ERR_DESC(
RD_KAFKA_RESP_ERR__FAIL,
"Local: Communication failure with broker"), // FIXME: too specific
_ERR_DESC(RD_KAFKA_RESP_ERR__TRANSPORT, "Local: Broker transport failure"),
_ERR_DESC(RD_KAFKA_RESP_ERR__CRIT_SYS_RESOURCE,
"Local: Critical system resource failure"),
_ERR_DESC(RD_KAFKA_RESP_ERR__RESOLVE, "Local: Host resolution failure"),
_ERR_DESC(RD_KAFKA_RESP_ERR__MSG_TIMED_OUT, "Local: Message timed out"),
_ERR_DESC(RD_KAFKA_RESP_ERR__PARTITION_EOF, "Broker: No more messages"),
_ERR_DESC(RD_KAFKA_RESP_ERR__UNKNOWN_PARTITION, "Local: Unknown partition"),
_ERR_DESC(RD_KAFKA_RESP_ERR__FS, "Local: File or filesystem error"),
_ERR_DESC(RD_KAFKA_RESP_ERR__UNKNOWN_TOPIC, "Local: Unknown topic"),
_ERR_DESC(RD_KAFKA_RESP_ERR__ALL_BROKERS_DOWN,
"Local: All broker connections are down"),
_ERR_DESC(RD_KAFKA_RESP_ERR__INVALID_ARG,
"Local: Invalid argument or configuration"),
_ERR_DESC(RD_KAFKA_RESP_ERR__TIMED_OUT, "Local: Timed out"),
_ERR_DESC(RD_KAFKA_RESP_ERR__QUEUE_FULL, "Local: Queue full"),
_ERR_DESC(RD_KAFKA_RESP_ERR__ISR_INSUFF, "Local: ISR count insufficient"),
_ERR_DESC(RD_KAFKA_RESP_ERR__NODE_UPDATE, "Local: Broker node update"),
_ERR_DESC(RD_KAFKA_RESP_ERR__SSL, "Local: SSL error"),
_ERR_DESC(RD_KAFKA_RESP_ERR__WAIT_COORD, "Local: Waiting for coordinator"),
_ERR_DESC(RD_KAFKA_RESP_ERR__UNKNOWN_GROUP, "Local: Unknown group"),
_ERR_DESC(RD_KAFKA_RESP_ERR__IN_PROGRESS, "Local: Operation in progress"),
_ERR_DESC(RD_KAFKA_RESP_ERR__PREV_IN_PROGRESS,
"Local: Previous operation in progress"),
_ERR_DESC(RD_KAFKA_RESP_ERR__EXISTING_SUBSCRIPTION,
"Local: Existing subscription"),
_ERR_DESC(RD_KAFKA_RESP_ERR__ASSIGN_PARTITIONS, "Local: Assign partitions"),
_ERR_DESC(RD_KAFKA_RESP_ERR__REVOKE_PARTITIONS, "Local: Revoke partitions"),
_ERR_DESC(RD_KAFKA_RESP_ERR__CONFLICT, "Local: Conflicting use"),
_ERR_DESC(RD_KAFKA_RESP_ERR__STATE, "Local: Erroneous state"),
_ERR_DESC(RD_KAFKA_RESP_ERR__UNKNOWN_PROTOCOL, "Local: Unknown protocol"),
_ERR_DESC(RD_KAFKA_RESP_ERR__NOT_IMPLEMENTED, "Local: Not implemented"),
_ERR_DESC(RD_KAFKA_RESP_ERR__AUTHENTICATION,
"Local: Authentication failure"),
_ERR_DESC(RD_KAFKA_RESP_ERR__NO_OFFSET, "Local: No offset stored"),
_ERR_DESC(RD_KAFKA_RESP_ERR__OUTDATED, "Local: Outdated"),
_ERR_DESC(RD_KAFKA_RESP_ERR__TIMED_OUT_QUEUE, "Local: Timed out in queue"),
_ERR_DESC(RD_KAFKA_RESP_ERR__UNSUPPORTED_FEATURE,
"Local: Required feature not supported by broker"),
_ERR_DESC(RD_KAFKA_RESP_ERR__WAIT_CACHE, "Local: Awaiting cache update"),
_ERR_DESC(RD_KAFKA_RESP_ERR__INTR, "Local: Operation interrupted"),
_ERR_DESC(RD_KAFKA_RESP_ERR__KEY_SERIALIZATION,
"Local: Key serialization error"),
_ERR_DESC(RD_KAFKA_RESP_ERR__VALUE_SERIALIZATION,
"Local: Value serialization error"),
_ERR_DESC(RD_KAFKA_RESP_ERR__KEY_DESERIALIZATION,
"Local: Key deserialization error"),
_ERR_DESC(RD_KAFKA_RESP_ERR__VALUE_DESERIALIZATION,
"Local: Value deserialization error"),
_ERR_DESC(RD_KAFKA_RESP_ERR__PARTIAL, "Local: Partial response"),
_ERR_DESC(RD_KAFKA_RESP_ERR__READ_ONLY, "Local: Read-only object"),
_ERR_DESC(RD_KAFKA_RESP_ERR__NOENT, "Local: No such entry"),
_ERR_DESC(RD_KAFKA_RESP_ERR__UNDERFLOW, "Local: Read underflow"),
_ERR_DESC(RD_KAFKA_RESP_ERR__INVALID_TYPE, "Local: Invalid type"),
_ERR_DESC(RD_KAFKA_RESP_ERR__RETRY, "Local: Retry operation"),
_ERR_DESC(RD_KAFKA_RESP_ERR__PURGE_QUEUE, "Local: Purged in queue"),
_ERR_DESC(RD_KAFKA_RESP_ERR__PURGE_INFLIGHT, "Local: Purged in flight"),
_ERR_DESC(RD_KAFKA_RESP_ERR__FATAL, "Local: Fatal error"),
_ERR_DESC(RD_KAFKA_RESP_ERR__INCONSISTENT, "Local: Inconsistent state"),
_ERR_DESC(RD_KAFKA_RESP_ERR__GAPLESS_GUARANTEE,
"Local: Gap-less ordering would not be guaranteed "
"if proceeding"),
_ERR_DESC(RD_KAFKA_RESP_ERR__MAX_POLL_EXCEEDED,
"Local: Maximum application poll interval "
"(max.poll.interval.ms) exceeded"),
_ERR_DESC(RD_KAFKA_RESP_ERR__UNKNOWN_BROKER, "Local: Unknown broker"),
_ERR_DESC(RD_KAFKA_RESP_ERR__NOT_CONFIGURED,
"Local: Functionality not configured"),
_ERR_DESC(RD_KAFKA_RESP_ERR__FENCED,
"Local: This instance has been fenced by a newer instance"),
_ERR_DESC(RD_KAFKA_RESP_ERR__APPLICATION,
"Local: Application generated error"),
_ERR_DESC(RD_KAFKA_RESP_ERR__ASSIGNMENT_LOST,
"Local: Group partition assignment lost"),
_ERR_DESC(RD_KAFKA_RESP_ERR__NOOP, "Local: No operation performed"),
_ERR_DESC(RD_KAFKA_RESP_ERR__AUTO_OFFSET_RESET,
"Local: No offset to automatically reset to"),
_ERR_DESC(RD_KAFKA_RESP_ERR__LOG_TRUNCATION,
"Local: Partition log truncation detected"),
_ERR_DESC(RD_KAFKA_RESP_ERR_UNKNOWN, "Unknown broker error"),
_ERR_DESC(RD_KAFKA_RESP_ERR_NO_ERROR, "Success"),
_ERR_DESC(RD_KAFKA_RESP_ERR_OFFSET_OUT_OF_RANGE,
"Broker: Offset out of range"),
_ERR_DESC(RD_KAFKA_RESP_ERR_INVALID_MSG, "Broker: Invalid message"),
_ERR_DESC(RD_KAFKA_RESP_ERR_UNKNOWN_TOPIC_OR_PART,
"Broker: Unknown topic or partition"),
_ERR_DESC(RD_KAFKA_RESP_ERR_INVALID_MSG_SIZE,
"Broker: Invalid message size"),
_ERR_DESC(RD_KAFKA_RESP_ERR_LEADER_NOT_AVAILABLE,
"Broker: Leader not available"),
_ERR_DESC(RD_KAFKA_RESP_ERR_NOT_LEADER_FOR_PARTITION,
"Broker: Not leader for partition"),
_ERR_DESC(RD_KAFKA_RESP_ERR_REQUEST_TIMED_OUT, "Broker: Request timed out"),
_ERR_DESC(RD_KAFKA_RESP_ERR_BROKER_NOT_AVAILABLE,
"Broker: Broker not available"),
_ERR_DESC(RD_KAFKA_RESP_ERR_REPLICA_NOT_AVAILABLE,
"Broker: Replica not available"),
_ERR_DESC(RD_KAFKA_RESP_ERR_MSG_SIZE_TOO_LARGE,
"Broker: Message size too large"),
_ERR_DESC(RD_KAFKA_RESP_ERR_STALE_CTRL_EPOCH,
"Broker: StaleControllerEpochCode"),
_ERR_DESC(RD_KAFKA_RESP_ERR_OFFSET_METADATA_TOO_LARGE,
"Broker: Offset metadata string too large"),
_ERR_DESC(RD_KAFKA_RESP_ERR_NETWORK_EXCEPTION,
"Broker: Broker disconnected before response received"),
_ERR_DESC(RD_KAFKA_RESP_ERR_COORDINATOR_LOAD_IN_PROGRESS,
"Broker: Coordinator load in progress"),
_ERR_DESC(RD_KAFKA_RESP_ERR_COORDINATOR_NOT_AVAILABLE,
"Broker: Coordinator not available"),
_ERR_DESC(RD_KAFKA_RESP_ERR_NOT_COORDINATOR, "Broker: Not coordinator"),
_ERR_DESC(RD_KAFKA_RESP_ERR_TOPIC_EXCEPTION, "Broker: Invalid topic"),
_ERR_DESC(RD_KAFKA_RESP_ERR_RECORD_LIST_TOO_LARGE,
"Broker: Message batch larger than configured server "
"segment size"),
_ERR_DESC(RD_KAFKA_RESP_ERR_NOT_ENOUGH_REPLICAS,
"Broker: Not enough in-sync replicas"),
_ERR_DESC(RD_KAFKA_RESP_ERR_NOT_ENOUGH_REPLICAS_AFTER_APPEND,
"Broker: Message(s) written to insufficient number of "
"in-sync replicas"),
_ERR_DESC(RD_KAFKA_RESP_ERR_INVALID_REQUIRED_ACKS,
"Broker: Invalid required acks value"),
_ERR_DESC(RD_KAFKA_RESP_ERR_ILLEGAL_GENERATION,
"Broker: Specified group generation id is not valid"),
_ERR_DESC(RD_KAFKA_RESP_ERR_INCONSISTENT_GROUP_PROTOCOL,
"Broker: Inconsistent group protocol"),
_ERR_DESC(RD_KAFKA_RESP_ERR_INVALID_GROUP_ID, "Broker: Invalid group.id"),
_ERR_DESC(RD_KAFKA_RESP_ERR_UNKNOWN_MEMBER_ID, "Broker: Unknown member"),
_ERR_DESC(RD_KAFKA_RESP_ERR_INVALID_SESSION_TIMEOUT,
"Broker: Invalid session timeout"),
_ERR_DESC(RD_KAFKA_RESP_ERR_REBALANCE_IN_PROGRESS,
"Broker: Group rebalance in progress"),
_ERR_DESC(RD_KAFKA_RESP_ERR_INVALID_COMMIT_OFFSET_SIZE,
"Broker: Commit offset data size is not valid"),
_ERR_DESC(RD_KAFKA_RESP_ERR_TOPIC_AUTHORIZATION_FAILED,
"Broker: Topic authorization failed"),
_ERR_DESC(RD_KAFKA_RESP_ERR_GROUP_AUTHORIZATION_FAILED,
"Broker: Group authorization failed"),
_ERR_DESC(RD_KAFKA_RESP_ERR_CLUSTER_AUTHORIZATION_FAILED,
"Broker: Cluster authorization failed"),
_ERR_DESC(RD_KAFKA_RESP_ERR_INVALID_TIMESTAMP, "Broker: Invalid timestamp"),
_ERR_DESC(RD_KAFKA_RESP_ERR_UNSUPPORTED_SASL_MECHANISM,
"Broker: Unsupported SASL mechanism"),
_ERR_DESC(RD_KAFKA_RESP_ERR_ILLEGAL_SASL_STATE,
"Broker: Request not valid in current SASL state"),
_ERR_DESC(RD_KAFKA_RESP_ERR_UNSUPPORTED_VERSION,
"Broker: API version not supported"),
_ERR_DESC(RD_KAFKA_RESP_ERR_TOPIC_ALREADY_EXISTS,
"Broker: Topic already exists"),
_ERR_DESC(RD_KAFKA_RESP_ERR_INVALID_PARTITIONS,
"Broker: Invalid number of partitions"),
_ERR_DESC(RD_KAFKA_RESP_ERR_INVALID_REPLICATION_FACTOR,
"Broker: Invalid replication factor"),
_ERR_DESC(RD_KAFKA_RESP_ERR_INVALID_REPLICA_ASSIGNMENT,
"Broker: Invalid replica assignment"),
_ERR_DESC(RD_KAFKA_RESP_ERR_INVALID_CONFIG,
"Broker: Configuration is invalid"),
_ERR_DESC(RD_KAFKA_RESP_ERR_NOT_CONTROLLER,
"Broker: Not controller for cluster"),
_ERR_DESC(RD_KAFKA_RESP_ERR_INVALID_REQUEST, "Broker: Invalid request"),
_ERR_DESC(RD_KAFKA_RESP_ERR_UNSUPPORTED_FOR_MESSAGE_FORMAT,
"Broker: Message format on broker does not support request"),
_ERR_DESC(RD_KAFKA_RESP_ERR_POLICY_VIOLATION, "Broker: Policy violation"),
_ERR_DESC(RD_KAFKA_RESP_ERR_OUT_OF_ORDER_SEQUENCE_NUMBER,
"Broker: Broker received an out of order sequence number"),
_ERR_DESC(RD_KAFKA_RESP_ERR_DUPLICATE_SEQUENCE_NUMBER,
"Broker: Broker received a duplicate sequence number"),
_ERR_DESC(RD_KAFKA_RESP_ERR_INVALID_PRODUCER_EPOCH,
"Broker: Producer attempted an operation with an old epoch"),
_ERR_DESC(RD_KAFKA_RESP_ERR_INVALID_TXN_STATE,
"Broker: Producer attempted a transactional operation in "
"an invalid state"),
_ERR_DESC(RD_KAFKA_RESP_ERR_INVALID_PRODUCER_ID_MAPPING,
"Broker: Producer attempted to use a producer id which is "
"not currently assigned to its transactional id"),
_ERR_DESC(RD_KAFKA_RESP_ERR_INVALID_TRANSACTION_TIMEOUT,
"Broker: Transaction timeout is larger than the maximum "
"value allowed by the broker's max.transaction.timeout.ms"),
_ERR_DESC(RD_KAFKA_RESP_ERR_CONCURRENT_TRANSACTIONS,
"Broker: Producer attempted to update a transaction while "
"another concurrent operation on the same transaction was "
"ongoing"),
_ERR_DESC(RD_KAFKA_RESP_ERR_TRANSACTION_COORDINATOR_FENCED,
"Broker: Indicates that the transaction coordinator sending "
"a WriteTxnMarker is no longer the current coordinator for "
"a given producer"),
_ERR_DESC(RD_KAFKA_RESP_ERR_TRANSACTIONAL_ID_AUTHORIZATION_FAILED,
"Broker: Transactional Id authorization failed"),
_ERR_DESC(RD_KAFKA_RESP_ERR_SECURITY_DISABLED,
"Broker: Security features are disabled"),
_ERR_DESC(RD_KAFKA_RESP_ERR_OPERATION_NOT_ATTEMPTED,
"Broker: Operation not attempted"),
_ERR_DESC(RD_KAFKA_RESP_ERR_KAFKA_STORAGE_ERROR,
"Broker: Disk error when trying to access log file on disk"),
_ERR_DESC(RD_KAFKA_RESP_ERR_LOG_DIR_NOT_FOUND,
"Broker: The user-specified log directory is not found "
"in the broker config"),
_ERR_DESC(RD_KAFKA_RESP_ERR_SASL_AUTHENTICATION_FAILED,
"Broker: SASL Authentication failed"),
_ERR_DESC(RD_KAFKA_RESP_ERR_UNKNOWN_PRODUCER_ID,
"Broker: Unknown Producer Id"),
_ERR_DESC(RD_KAFKA_RESP_ERR_REASSIGNMENT_IN_PROGRESS,
"Broker: Partition reassignment is in progress"),
_ERR_DESC(RD_KAFKA_RESP_ERR_DELEGATION_TOKEN_AUTH_DISABLED,
"Broker: Delegation Token feature is not enabled"),
_ERR_DESC(RD_KAFKA_RESP_ERR_DELEGATION_TOKEN_NOT_FOUND,
"Broker: Delegation Token is not found on server"),
_ERR_DESC(RD_KAFKA_RESP_ERR_DELEGATION_TOKEN_OWNER_MISMATCH,
"Broker: Specified Principal is not valid Owner/Renewer"),
_ERR_DESC(RD_KAFKA_RESP_ERR_DELEGATION_TOKEN_REQUEST_NOT_ALLOWED,
"Broker: Delegation Token requests are not allowed on "
"this connection"),
_ERR_DESC(RD_KAFKA_RESP_ERR_DELEGATION_TOKEN_AUTHORIZATION_FAILED,
"Broker: Delegation Token authorization failed"),
_ERR_DESC(RD_KAFKA_RESP_ERR_DELEGATION_TOKEN_EXPIRED,
"Broker: Delegation Token is expired"),
_ERR_DESC(RD_KAFKA_RESP_ERR_INVALID_PRINCIPAL_TYPE,
"Broker: Supplied principalType is not supported"),
_ERR_DESC(RD_KAFKA_RESP_ERR_NON_EMPTY_GROUP,
"Broker: The group is not empty"),
_ERR_DESC(RD_KAFKA_RESP_ERR_GROUP_ID_NOT_FOUND,
"Broker: The group id does not exist"),
_ERR_DESC(RD_KAFKA_RESP_ERR_FETCH_SESSION_ID_NOT_FOUND,
"Broker: The fetch session ID was not found"),
_ERR_DESC(RD_KAFKA_RESP_ERR_INVALID_FETCH_SESSION_EPOCH,
"Broker: The fetch session epoch is invalid"),
_ERR_DESC(RD_KAFKA_RESP_ERR_LISTENER_NOT_FOUND,
"Broker: No matching listener"),
_ERR_DESC(RD_KAFKA_RESP_ERR_TOPIC_DELETION_DISABLED,
"Broker: Topic deletion is disabled"),
_ERR_DESC(RD_KAFKA_RESP_ERR_FENCED_LEADER_EPOCH,
"Broker: Leader epoch is older than broker epoch"),
_ERR_DESC(RD_KAFKA_RESP_ERR_UNKNOWN_LEADER_EPOCH,
"Broker: Leader epoch is newer than broker epoch"),
_ERR_DESC(RD_KAFKA_RESP_ERR_UNSUPPORTED_COMPRESSION_TYPE,
"Broker: Unsupported compression type"),
_ERR_DESC(RD_KAFKA_RESP_ERR_STALE_BROKER_EPOCH,
"Broker: Broker epoch has changed"),
_ERR_DESC(RD_KAFKA_RESP_ERR_OFFSET_NOT_AVAILABLE,
"Broker: Leader high watermark is not caught up"),
_ERR_DESC(RD_KAFKA_RESP_ERR_MEMBER_ID_REQUIRED,
"Broker: Group member needs a valid member ID"),
_ERR_DESC(RD_KAFKA_RESP_ERR_PREFERRED_LEADER_NOT_AVAILABLE,
"Broker: Preferred leader was not available"),
_ERR_DESC(RD_KAFKA_RESP_ERR_GROUP_MAX_SIZE_REACHED,
"Broker: Consumer group has reached maximum size"),
_ERR_DESC(RD_KAFKA_RESP_ERR_FENCED_INSTANCE_ID,
"Broker: Static consumer fenced by other consumer with same "
"group.instance.id"),
_ERR_DESC(RD_KAFKA_RESP_ERR_ELIGIBLE_LEADERS_NOT_AVAILABLE,
"Broker: Eligible partition leaders are not available"),
_ERR_DESC(RD_KAFKA_RESP_ERR_ELECTION_NOT_NEEDED,
"Broker: Leader election not needed for topic partition"),
_ERR_DESC(RD_KAFKA_RESP_ERR_NO_REASSIGNMENT_IN_PROGRESS,
"Broker: No partition reassignment is in progress"),
_ERR_DESC(RD_KAFKA_RESP_ERR_GROUP_SUBSCRIBED_TO_TOPIC,
"Broker: Deleting offsets of a topic while the consumer "
"group is subscribed to it"),
_ERR_DESC(RD_KAFKA_RESP_ERR_INVALID_RECORD,
"Broker: Broker failed to validate record"),
_ERR_DESC(RD_KAFKA_RESP_ERR_UNSTABLE_OFFSET_COMMIT,
"Broker: There are unstable offsets that need to be cleared"),
_ERR_DESC(RD_KAFKA_RESP_ERR_THROTTLING_QUOTA_EXCEEDED,
"Broker: Throttling quota has been exceeded"),
_ERR_DESC(RD_KAFKA_RESP_ERR_PRODUCER_FENCED,
"Broker: There is a newer producer with the same "
"transactionalId which fences the current one"),
_ERR_DESC(RD_KAFKA_RESP_ERR_RESOURCE_NOT_FOUND,
"Broker: Request illegally referred to resource that "
"does not exist"),
_ERR_DESC(RD_KAFKA_RESP_ERR_DUPLICATE_RESOURCE,
"Broker: Request illegally referred to the same resource "
"twice"),
_ERR_DESC(RD_KAFKA_RESP_ERR_UNACCEPTABLE_CREDENTIAL,
"Broker: Requested credential would not meet criteria for "
"acceptability"),
_ERR_DESC(RD_KAFKA_RESP_ERR_INCONSISTENT_VOTER_SET,
"Broker: Indicates that the either the sender or recipient "
"of a voter-only request is not one of the expected voters"),
_ERR_DESC(RD_KAFKA_RESP_ERR_INVALID_UPDATE_VERSION,
"Broker: Invalid update version"),
_ERR_DESC(RD_KAFKA_RESP_ERR_FEATURE_UPDATE_FAILED,
"Broker: Unable to update finalized features due to "
"server error"),
_ERR_DESC(RD_KAFKA_RESP_ERR_PRINCIPAL_DESERIALIZATION_FAILURE,
"Broker: Request principal deserialization failed during "
"forwarding"),
_ERR_DESC(RD_KAFKA_RESP_ERR__END, NULL)};
void rd_kafka_get_err_descs(const struct rd_kafka_err_desc **errdescs,
size_t *cntp) {
*errdescs = rd_kafka_err_descs;
*cntp = RD_ARRAYSIZE(rd_kafka_err_descs);
}
const char *rd_kafka_err2str(rd_kafka_resp_err_t err) {
static RD_TLS char ret[32];
int idx = err - RD_KAFKA_RESP_ERR__BEGIN;
if (unlikely(err <= RD_KAFKA_RESP_ERR__BEGIN ||
err >= RD_KAFKA_RESP_ERR_END_ALL ||
!rd_kafka_err_descs[idx].desc)) {
rd_snprintf(ret, sizeof(ret), "Err-%i?", err);
return ret;
}
return rd_kafka_err_descs[idx].desc;
}
const char *rd_kafka_err2name(rd_kafka_resp_err_t err) {
static RD_TLS char ret[32];
int idx = err - RD_KAFKA_RESP_ERR__BEGIN;
if (unlikely(err <= RD_KAFKA_RESP_ERR__BEGIN ||
err >= RD_KAFKA_RESP_ERR_END_ALL ||
!rd_kafka_err_descs[idx].desc)) {
rd_snprintf(ret, sizeof(ret), "ERR_%i?", err);
return ret;
}
return rd_kafka_err_descs[idx].name;
}
rd_kafka_resp_err_t rd_kafka_last_error(void) {
return rd_kafka_last_error_code;
}
rd_kafka_resp_err_t rd_kafka_errno2err(int errnox) {
switch (errnox) {
case EINVAL:
return RD_KAFKA_RESP_ERR__INVALID_ARG;
case EBUSY:
return RD_KAFKA_RESP_ERR__CONFLICT;
case ENOENT:
return RD_KAFKA_RESP_ERR__UNKNOWN_TOPIC;
case ESRCH:
return RD_KAFKA_RESP_ERR__UNKNOWN_PARTITION;
case ETIMEDOUT:
return RD_KAFKA_RESP_ERR__TIMED_OUT;
case EMSGSIZE:
return RD_KAFKA_RESP_ERR_MSG_SIZE_TOO_LARGE;
case ENOBUFS:
return RD_KAFKA_RESP_ERR__QUEUE_FULL;
case ECANCELED:
return RD_KAFKA_RESP_ERR__FATAL;
default:
return RD_KAFKA_RESP_ERR__FAIL;
}
}
rd_kafka_resp_err_t
rd_kafka_fatal_error(rd_kafka_t *rk, char *errstr, size_t errstr_size) {
rd_kafka_resp_err_t err;
if (unlikely((err = rd_atomic32_get(&rk->rk_fatal.err)))) {
rd_kafka_rdlock(rk);
rd_snprintf(errstr, errstr_size, "%s", rk->rk_fatal.errstr);
rd_kafka_rdunlock(rk);
}
return err;
}
/**
* @brief Set's the fatal error for this instance.
*
* @param do_lock RD_DO_LOCK: rd_kafka_wrlock() will be acquired and released,
* RD_DONT_LOCK: caller must hold rd_kafka_wrlock().
*
* @returns 1 if the error was set, or 0 if a previous fatal error
* has already been set on this instance.
*
* @locality any
* @locks none
*/
int rd_kafka_set_fatal_error0(rd_kafka_t *rk,
rd_dolock_t do_lock,
rd_kafka_resp_err_t err,
const char *fmt,
...) {
va_list ap;
char buf[512];
if (do_lock)
rd_kafka_wrlock(rk);
rk->rk_fatal.cnt++;
if (rd_atomic32_get(&rk->rk_fatal.err)) {
if (do_lock)
rd_kafka_wrunlock(rk);
rd_kafka_dbg(rk, GENERIC, "FATAL",
"Suppressing subsequent fatal error: %s",
rd_kafka_err2name(err));
return 0;
}
rd_atomic32_set(&rk->rk_fatal.err, err);
va_start(ap, fmt);
rd_vsnprintf(buf, sizeof(buf), fmt, ap);
va_end(ap);
rk->rk_fatal.errstr = rd_strdup(buf);
if (do_lock)
rd_kafka_wrunlock(rk);
/* If there is an error callback or event handler we
* also log the fatal error as it happens.
* If there is no error callback the error event
* will be automatically logged, and this check here
* prevents us from duplicate logs. */
if (rk->rk_conf.enabled_events & RD_KAFKA_EVENT_ERROR)
rd_kafka_log(rk, LOG_EMERG, "FATAL", "Fatal error: %s: %s",
rd_kafka_err2str(err), rk->rk_fatal.errstr);
else
rd_kafka_dbg(rk, ALL, "FATAL", "Fatal error: %s: %s",
rd_kafka_err2str(err), rk->rk_fatal.errstr);
/* Indicate to the application that a fatal error was raised,
* the app should use rd_kafka_fatal_error() to extract the
* fatal error code itself.
* For the high-level consumer we propagate the error as a
* consumer error so it is returned from consumer_poll(),
* while for all other client types (the producer) we propagate to
* the standard error handler (typically error_cb). */
if (rk->rk_type == RD_KAFKA_CONSUMER && rk->rk_cgrp)
rd_kafka_consumer_err(
rk->rk_cgrp->rkcg_q, RD_KAFKA_NODEID_UA,
RD_KAFKA_RESP_ERR__FATAL, 0, NULL, NULL,
RD_KAFKA_OFFSET_INVALID, "Fatal error: %s: %s",
rd_kafka_err2str(err), rk->rk_fatal.errstr);
else
rd_kafka_op_err(rk, RD_KAFKA_RESP_ERR__FATAL,
"Fatal error: %s: %s", rd_kafka_err2str(err),
rk->rk_fatal.errstr);
/* Tell rdkafka main thread to purge producer queues, but not
* in-flight since we'll want proper delivery status for transmitted
* requests.
* Need NON_BLOCKING to avoid dead-lock if user is
* calling purge() at the same time, which could be
* waiting for this broker thread to handle its
* OP_PURGE request. */
if (rk->rk_type == RD_KAFKA_PRODUCER) {
rd_kafka_op_t *rko = rd_kafka_op_new(RD_KAFKA_OP_PURGE);
rko->rko_u.purge.flags =
RD_KAFKA_PURGE_F_QUEUE | RD_KAFKA_PURGE_F_NON_BLOCKING;
rd_kafka_q_enq(rk->rk_ops, rko);
}
return 1;
}
/**
* @returns a copy of the current fatal error, if any, else NULL.
*
* @locks_acquired rd_kafka_rdlock(rk)
*/
rd_kafka_error_t *rd_kafka_get_fatal_error(rd_kafka_t *rk) {
rd_kafka_error_t *error;
rd_kafka_resp_err_t err;
if (!(err = rd_atomic32_get(&rk->rk_fatal.err)))
return NULL; /* No fatal error raised */
rd_kafka_rdlock(rk);
error = rd_kafka_error_new_fatal(err, "%s", rk->rk_fatal.errstr);
rd_kafka_rdunlock(rk);
return error;
}
rd_kafka_resp_err_t rd_kafka_test_fatal_error(rd_kafka_t *rk,
rd_kafka_resp_err_t err,
const char *reason) {
if (!rd_kafka_set_fatal_error(rk, err, "test_fatal_error: %s", reason))
return RD_KAFKA_RESP_ERR__PREV_IN_PROGRESS;
else
return RD_KAFKA_RESP_ERR_NO_ERROR;
}
/**
* @brief Final destructor for rd_kafka_t, must only be called with refcnt 0.
*
* @locality application thread
*/
void rd_kafka_destroy_final(rd_kafka_t *rk) {
rd_kafka_assert(rk, rd_kafka_terminating(rk));
/* Synchronize state */
rd_kafka_wrlock(rk);
rd_kafka_wrunlock(rk);
/* Terminate SASL provider */
if (rk->rk_conf.sasl.provider)
rd_kafka_sasl_term(rk);
rd_kafka_timers_destroy(&rk->rk_timers);
rd_kafka_dbg(rk, GENERIC, "TERMINATE", "Destroying op queues");
/* Destroy cgrp */
if (rk->rk_cgrp) {
rd_kafka_dbg(rk, GENERIC, "TERMINATE", "Destroying cgrp");
/* Reset queue forwarding (rep -> cgrp) */
rd_kafka_q_fwd_set(rk->rk_rep, NULL);
rd_kafka_cgrp_destroy_final(rk->rk_cgrp);
}
rd_kafka_assignors_term(rk);
if (rk->rk_type == RD_KAFKA_CONSUMER) {
rd_kafka_assignment_destroy(rk);
if (rk->rk_consumer.q)
rd_kafka_q_destroy(rk->rk_consumer.q);
}
/* Purge op-queues */
rd_kafka_q_destroy_owner(rk->rk_rep);
rd_kafka_q_destroy_owner(rk->rk_ops);
#if WITH_SSL
if (rk->rk_conf.ssl.ctx) {
rd_kafka_dbg(rk, GENERIC, "TERMINATE", "Destroying SSL CTX");
rd_kafka_ssl_ctx_term(rk);
}
rd_list_destroy(&rk->rk_conf.ssl.loaded_providers);
#endif
/* It is not safe to log after this point. */
rd_kafka_dbg(rk, GENERIC, "TERMINATE",
"Termination done: freeing resources");
if (rk->rk_logq) {
rd_kafka_q_destroy_owner(rk->rk_logq);
rk->rk_logq = NULL;
}
if (rk->rk_type == RD_KAFKA_PRODUCER) {
cnd_destroy(&rk->rk_curr_msgs.cnd);
mtx_destroy(&rk->rk_curr_msgs.lock);
}
if (rk->rk_fatal.errstr) {
rd_free(rk->rk_fatal.errstr);
rk->rk_fatal.errstr = NULL;
}
cnd_destroy(&rk->rk_broker_state_change_cnd);
mtx_destroy(&rk->rk_broker_state_change_lock);
mtx_destroy(&rk->rk_suppress.sparse_connect_lock);
cnd_destroy(&rk->rk_init_cnd);
mtx_destroy(&rk->rk_init_lock);
if (rk->rk_full_metadata)
rd_kafka_metadata_destroy(&rk->rk_full_metadata->metadata);
rd_kafkap_str_destroy(rk->rk_client_id);
rd_kafkap_str_destroy(rk->rk_group_id);
rd_kafkap_str_destroy(rk->rk_eos.transactional_id);
rd_kafka_anyconf_destroy(_RK_GLOBAL, &rk->rk_conf);
rd_list_destroy(&rk->rk_broker_by_id);
mtx_destroy(&rk->rk_conf.sasl.lock);