-
Notifications
You must be signed in to change notification settings - Fork 3.2k
/
rdkafka_broker.c
6186 lines (5173 loc) · 238 KB
/
rdkafka_broker.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.
*/
#if defined(__MINGW32__)
#include <ws2tcpip.h>
#endif
#ifndef _WIN32
#define _GNU_SOURCE
/*
* AIX defines this and the value needs to be set correctly. For Solaris,
* src/rd.h defines _POSIX_SOURCE to be 200809L, which corresponds to XPG7,
* which itself is not compatible with _XOPEN_SOURCE on that platform.
*/
#if !defined(_AIX) && !defined(__sun)
#define _XOPEN_SOURCE
#endif
#include <signal.h>
#endif
#include <stdio.h>
#include <stdarg.h>
#include <string.h>
#include <ctype.h>
#include "rd.h"
#include "rdaddr.h"
#include "rdkafka_int.h"
#include "rdkafka_msg.h"
#include "rdkafka_msgset.h"
#include "rdkafka_topic.h"
#include "rdkafka_partition.h"
#include "rdkafka_broker.h"
#include "rdkafka_offset.h"
#include "rdkafka_telemetry.h"
#include "rdkafka_transport.h"
#include "rdkafka_proto.h"
#include "rdkafka_buf.h"
#include "rdkafka_request.h"
#include "rdkafka_sasl.h"
#include "rdkafka_interceptor.h"
#include "rdkafka_idempotence.h"
#include "rdkafka_txnmgr.h"
#include "rdkafka_fetcher.h"
#include "rdtime.h"
#include "rdcrc32.h"
#include "rdrand.h"
#include "rdkafka_lz4.h"
#if WITH_SSL
#include <openssl/err.h>
#endif
#include "rdendian.h"
#include "rdunittest.h"
static const int rd_kafka_max_block_ms = 1000;
const char *rd_kafka_broker_state_names[] = {
"INIT", "DOWN", "TRY_CONNECT", "CONNECT", "SSL_HANDSHAKE",
"AUTH_LEGACY", "UP", "UPDATE", "APIVERSION_QUERY", "AUTH_HANDSHAKE",
"AUTH_REQ", "REAUTH"};
const char *rd_kafka_secproto_names[] = {
[RD_KAFKA_PROTO_PLAINTEXT] = "plaintext",
[RD_KAFKA_PROTO_SSL] = "ssl",
[RD_KAFKA_PROTO_SASL_PLAINTEXT] = "sasl_plaintext",
[RD_KAFKA_PROTO_SASL_SSL] = "sasl_ssl",
NULL};
/**
* @returns true for logical brokers (e.g., coordinators) without an address set
*
* @locks_required rkb_lock
*/
#define rd_kafka_broker_is_addrless(rkb) (*(rkb)->rkb_nodename == '\0')
/**
* @returns true if the broker needs a persistent connection
* @locaility broker thread
*/
static RD_INLINE rd_bool_t
rd_kafka_broker_needs_persistent_connection(rd_kafka_broker_t *rkb) {
return rkb->rkb_persistconn.internal ||
rd_atomic32_get(&rkb->rkb_persistconn.coord);
}
/**
* @returns > 0 if a connection to this broker is needed, else 0.
* @locality broker thread
* @locks none
*/
static RD_INLINE int rd_kafka_broker_needs_connection(rd_kafka_broker_t *rkb) {
return rkb->rkb_state == RD_KAFKA_BROKER_STATE_INIT &&
!rd_kafka_terminating(rkb->rkb_rk) &&
!rd_kafka_fatal_error_code(rkb->rkb_rk) &&
(!rkb->rkb_rk->rk_conf.sparse_connections ||
rd_kafka_broker_needs_persistent_connection(rkb));
}
static void rd_kafka_broker_handle_purge_queues(rd_kafka_broker_t *rkb,
rd_kafka_op_t *rko);
static void rd_kafka_broker_trigger_monitors(rd_kafka_broker_t *rkb);
#define rd_kafka_broker_terminating(rkb) \
(rd_refcnt_get(&(rkb)->rkb_refcnt) <= 1)
/**
* Construct broker nodename.
*/
static void rd_kafka_mk_nodename(char *dest,
size_t dsize,
const char *name,
uint16_t port) {
rd_snprintf(dest, dsize, "%s:%hu", name, port);
}
/**
* Construct descriptive broker name
*/
static void rd_kafka_mk_brokername(char *dest,
size_t dsize,
rd_kafka_secproto_t proto,
const char *nodename,
int32_t nodeid,
rd_kafka_confsource_t source) {
/* Prepend protocol name to brokername, unless it is a
* standard plaintext or logical broker in which case we
* omit the protocol part. */
if (proto != RD_KAFKA_PROTO_PLAINTEXT && source != RD_KAFKA_LOGICAL) {
int r = rd_snprintf(dest, dsize, "%s://",
rd_kafka_secproto_names[proto]);
if (r >= (int)dsize) /* Skip proto name if it wont fit.. */
r = 0;
dest += r;
dsize -= r;
}
if (nodeid == RD_KAFKA_NODEID_UA)
rd_snprintf(dest, dsize, "%s%s", nodename,
source == RD_KAFKA_LOGICAL
? ""
: (source == RD_KAFKA_INTERNAL ? "/internal"
: "/bootstrap"));
else
rd_snprintf(dest, dsize, "%s/%" PRId32, nodename, nodeid);
}
/**
* @brief Enable protocol feature(s) for the current broker.
*
* @locks broker_lock MUST be held
* @locality broker thread
*/
static void rd_kafka_broker_feature_enable(rd_kafka_broker_t *rkb,
int features) {
if (features & rkb->rkb_features)
return;
rkb->rkb_features |= features;
rd_rkb_dbg(rkb, BROKER | RD_KAFKA_DBG_PROTOCOL | RD_KAFKA_DBG_FEATURE,
"FEATURE", "Updated enabled protocol features +%s to %s",
rd_kafka_features2str(features),
rd_kafka_features2str(rkb->rkb_features));
}
/**
* @brief Disable protocol feature(s) for the current broker.
*
* @locks broker_lock MUST be held
* @locality broker thread
*/
static void rd_kafka_broker_feature_disable(rd_kafka_broker_t *rkb,
int features) {
if (!(features & rkb->rkb_features))
return;
rkb->rkb_features &= ~features;
rd_rkb_dbg(rkb, BROKER | RD_KAFKA_DBG_PROTOCOL | RD_KAFKA_DBG_FEATURE,
"FEATURE", "Updated enabled protocol features -%s to %s",
rd_kafka_features2str(features),
rd_kafka_features2str(rkb->rkb_features));
}
/**
* @brief Set protocol feature(s) for the current broker.
*
* @remark This replaces the previous feature set.
*
* @locality broker thread
* @locks rd_kafka_broker_lock()
*/
static void rd_kafka_broker_features_set(rd_kafka_broker_t *rkb, int features) {
if (rkb->rkb_features == features)
return;
rkb->rkb_features = features;
rd_rkb_dbg(rkb, BROKER, "FEATURE",
"Updated enabled protocol features to %s",
rd_kafka_features2str(rkb->rkb_features));
}
/**
* @brief Check and return supported ApiVersion for \p ApiKey.
*
* @returns the highest supported ApiVersion in the specified range (inclusive)
* or -1 if the ApiKey is not supported or no matching ApiVersion.
* The current feature set is also returned in \p featuresp
*
* @remark Same as rd_kafka_broker_ApiVersion_supported except for locking.
*
* @locks rd_kafka_broker_lock() if do_lock is rd_false
* @locks_acquired rd_kafka_broker_lock() if do_lock is rd_true
* @locality any
*/
int16_t rd_kafka_broker_ApiVersion_supported0(rd_kafka_broker_t *rkb,
int16_t ApiKey,
int16_t minver,
int16_t maxver,
int *featuresp,
rd_bool_t do_lock) {
struct rd_kafka_ApiVersion skel = {.ApiKey = ApiKey};
struct rd_kafka_ApiVersion ret = RD_ZERO_INIT, *retp;
if (do_lock)
rd_kafka_broker_lock(rkb);
if (featuresp)
*featuresp = rkb->rkb_features;
if (rkb->rkb_features & RD_KAFKA_FEATURE_UNITTEST) {
/* For unit tests let the broker support everything. */
if (do_lock)
rd_kafka_broker_unlock(rkb);
return maxver;
}
retp =
bsearch(&skel, rkb->rkb_ApiVersions, rkb->rkb_ApiVersions_cnt,
sizeof(*rkb->rkb_ApiVersions), rd_kafka_ApiVersion_key_cmp);
if (retp)
ret = *retp;
if (do_lock)
rd_kafka_broker_unlock(rkb);
if (!retp)
return -1;
if (ret.MaxVer < maxver) {
if (ret.MaxVer < minver)
return -1;
else
return ret.MaxVer;
} else if (ret.MinVer > maxver)
return -1;
else
return maxver;
}
/**
* @brief Check and return supported ApiVersion for \p ApiKey.
*
* @returns the highest supported ApiVersion in the specified range (inclusive)
* or -1 if the ApiKey is not supported or no matching ApiVersion.
* The current feature set is also returned in \p featuresp
* @locks none
* @locks_acquired rd_kafka_broker_lock()
* @locality any
*/
int16_t rd_kafka_broker_ApiVersion_supported(rd_kafka_broker_t *rkb,
int16_t ApiKey,
int16_t minver,
int16_t maxver,
int *featuresp) {
return rd_kafka_broker_ApiVersion_supported0(
rkb, ApiKey, minver, maxver, featuresp, rd_true /* do_lock */);
}
/**
* @brief Set broker state.
*
* \c rkb->rkb_state is the previous state, while
* \p state is the new state.
*
* @locks rd_kafka_broker_lock() MUST be held.
* @locality broker thread
*/
void rd_kafka_broker_set_state(rd_kafka_broker_t *rkb, int state) {
rd_bool_t trigger_monitors = rd_false;
if ((int)rkb->rkb_state == state)
return;
rd_kafka_dbg(rkb->rkb_rk, BROKER, "STATE",
"%s: Broker changed state %s -> %s", rkb->rkb_name,
rd_kafka_broker_state_names[rkb->rkb_state],
rd_kafka_broker_state_names[state]);
if (rkb->rkb_source == RD_KAFKA_INTERNAL) {
/* no-op */
} else if (state == RD_KAFKA_BROKER_STATE_DOWN &&
!rkb->rkb_down_reported) {
/* Propagate ALL_BROKERS_DOWN event if all brokers are
* now down, unless we're terminating.
* Only trigger for brokers that has an address set,
* e.g., not logical brokers that lost their address. */
if (rd_atomic32_add(&rkb->rkb_rk->rk_broker_down_cnt, 1) ==
rd_atomic32_get(&rkb->rkb_rk->rk_broker_cnt) -
rd_atomic32_get(
&rkb->rkb_rk->rk_broker_addrless_cnt) &&
!rd_kafka_broker_is_addrless(rkb) &&
!rd_kafka_terminating(rkb->rkb_rk))
rd_kafka_op_err(
rkb->rkb_rk, RD_KAFKA_RESP_ERR__ALL_BROKERS_DOWN,
"%i/%i brokers are down",
rd_atomic32_get(&rkb->rkb_rk->rk_broker_down_cnt),
rd_atomic32_get(&rkb->rkb_rk->rk_broker_cnt) -
rd_atomic32_get(
&rkb->rkb_rk->rk_broker_addrless_cnt));
rkb->rkb_down_reported = 1;
} else if (rd_kafka_broker_state_is_up(state) &&
rkb->rkb_down_reported) {
rd_atomic32_sub(&rkb->rkb_rk->rk_broker_down_cnt, 1);
rkb->rkb_down_reported = 0;
}
if (rkb->rkb_source != RD_KAFKA_INTERNAL) {
if (rd_kafka_broker_state_is_up(state) &&
!rd_kafka_broker_state_is_up(rkb->rkb_state)) {
/* Up -> Down */
rd_atomic32_add(&rkb->rkb_rk->rk_broker_up_cnt, 1);
trigger_monitors = rd_true;
if (RD_KAFKA_BROKER_IS_LOGICAL(rkb))
rd_atomic32_add(
&rkb->rkb_rk->rk_logical_broker_up_cnt, 1);
} else if (rd_kafka_broker_state_is_up(rkb->rkb_state) &&
!rd_kafka_broker_state_is_up(state)) {
/* ~Down(!Up) -> Up */
rd_atomic32_sub(&rkb->rkb_rk->rk_broker_up_cnt, 1);
trigger_monitors = rd_true;
if (RD_KAFKA_BROKER_IS_LOGICAL(rkb))
rd_atomic32_sub(
&rkb->rkb_rk->rk_logical_broker_up_cnt, 1);
}
/* If the connection or connection attempt failed and there
* are coord_reqs or cgrp awaiting this coordinator to come up
* then trigger the monitors so that rd_kafka_coord_req_fsm()
* is triggered, which in turn may trigger a new coordinator
* query. */
if (state == RD_KAFKA_BROKER_STATE_DOWN &&
rd_atomic32_get(&rkb->rkb_persistconn.coord) > 0)
trigger_monitors = rd_true;
}
rkb->rkb_state = state;
rkb->rkb_ts_state = rd_clock();
if (trigger_monitors)
rd_kafka_broker_trigger_monitors(rkb);
/* Call on_broker_state_change interceptors */
rd_kafka_interceptors_on_broker_state_change(
rkb->rkb_rk, rkb->rkb_nodeid,
rd_kafka_secproto_names[rkb->rkb_proto], rkb->rkb_origname,
rkb->rkb_port, rd_kafka_broker_state_names[rkb->rkb_state]);
rd_kafka_brokers_broadcast_state_change(rkb->rkb_rk);
}
/**
* @brief Set, log and propagate broker fail error.
*
* @param rkb Broker connection that failed.
* @param level Syslog level. LOG_DEBUG will not be logged unless debugging
* is enabled.
* @param err The type of error that occurred.
* @param fmt Format string.
* @param ap Format string arguments.
*
* @locks none
* @locality broker thread
*/
static void rd_kafka_broker_set_error(rd_kafka_broker_t *rkb,
int level,
rd_kafka_resp_err_t err,
const char *fmt,
va_list ap) {
char errstr[512];
char extra[128];
size_t of = 0, ofe;
rd_bool_t identical, suppress;
int state_duration_ms = (int)((rd_clock() - rkb->rkb_ts_state) / 1000);
/* If this is a logical broker we include its current nodename/address
* in the log message. */
rd_kafka_broker_lock(rkb);
if (rkb->rkb_source == RD_KAFKA_LOGICAL &&
!rd_kafka_broker_is_addrless(rkb)) {
of = (size_t)rd_snprintf(errstr, sizeof(errstr),
"%s: ", rkb->rkb_nodename);
if (of > sizeof(errstr))
of = 0; /* If nodename overflows the entire buffer we
* skip it completely since the error message
* itself is more important. */
}
rd_kafka_broker_unlock(rkb);
ofe = (size_t)rd_vsnprintf(errstr + of, sizeof(errstr) - of, fmt, ap);
if (ofe > sizeof(errstr) - of)
ofe = sizeof(errstr) - of;
of += ofe;
/* Provide more meaningful error messages in certain cases */
if (err == RD_KAFKA_RESP_ERR__TRANSPORT &&
!strcmp(errstr, "Disconnected")) {
if (rkb->rkb_state == RD_KAFKA_BROKER_STATE_APIVERSION_QUERY) {
/* A disconnect while requesting ApiVersion typically
* means we're connecting to a SSL-listener as
* PLAINTEXT, but may also be caused by connecting to
* a broker that does not support ApiVersion (<0.10). */
if (rkb->rkb_proto != RD_KAFKA_PROTO_SSL &&
rkb->rkb_proto != RD_KAFKA_PROTO_SASL_SSL)
rd_kafka_broker_set_error(
rkb, level, err,
"Disconnected while requesting "
"ApiVersion: "
"might be caused by incorrect "
"security.protocol configuration "
"(connecting to a SSL listener?) or "
"broker version is < 0.10 "
"(see api.version.request)",
ap /*ignored*/);
else
rd_kafka_broker_set_error(
rkb, level, err,
"Disconnected while requesting "
"ApiVersion: "
"might be caused by broker version "
"< 0.10 (see api.version.request)",
ap /*ignored*/);
return;
} else if (rkb->rkb_state == RD_KAFKA_BROKER_STATE_UP &&
state_duration_ms < 2000 /*2s*/ &&
rkb->rkb_rk->rk_conf.security_protocol !=
RD_KAFKA_PROTO_SASL_SSL &&
rkb->rkb_rk->rk_conf.security_protocol !=
RD_KAFKA_PROTO_SASL_PLAINTEXT) {
/* If disconnected shortly after transitioning to UP
* state it typically means the broker listener is
* configured for SASL authentication but the client
* is not. */
rd_kafka_broker_set_error(
rkb, level, err,
"Disconnected: verify that security.protocol "
"is correctly configured, broker might "
"require SASL authentication",
ap /*ignored*/);
return;
}
}
/* Check if error is identical to last error (prior to appending
* the variable suffix "after Xms in state Y"), if so we should
* suppress it. */
identical = err == rkb->rkb_last_err.err &&
!strcmp(rkb->rkb_last_err.errstr, errstr);
suppress = identical && rd_interval(&rkb->rkb_suppress.fail_error,
30 * 1000 * 1000 /*30s*/, 0) <= 0;
/* Copy last error prior to adding extras */
rkb->rkb_last_err.err = err;
rd_strlcpy(rkb->rkb_last_err.errstr, errstr,
sizeof(rkb->rkb_last_err.errstr));
/* Time since last state change to help debug connection issues */
ofe = rd_snprintf(extra, sizeof(extra), "after %dms in state %s",
state_duration_ms,
rd_kafka_broker_state_names[rkb->rkb_state]);
/* Number of suppressed identical logs */
if (identical && !suppress && rkb->rkb_last_err.cnt >= 1 &&
ofe + 30 < sizeof(extra)) {
size_t r =
(size_t)rd_snprintf(extra + ofe, sizeof(extra) - ofe,
", %d identical error(s) suppressed",
rkb->rkb_last_err.cnt);
if (r < sizeof(extra) - ofe)
ofe += r;
else
ofe = sizeof(extra);
}
/* Append the extra info if there is enough room */
if (ofe > 0 && of + ofe + 4 < sizeof(errstr))
rd_snprintf(errstr + of, sizeof(errstr) - of, " (%s)", extra);
/* Don't log interrupt-wakeups when terminating */
if (err == RD_KAFKA_RESP_ERR__INTR && rd_kafka_terminating(rkb->rkb_rk))
suppress = rd_true;
if (!suppress)
rkb->rkb_last_err.cnt = 1;
else
rkb->rkb_last_err.cnt++;
rd_rkb_dbg(rkb, BROKER, "FAIL", "%s (%s)%s%s", errstr,
rd_kafka_err2name(err),
identical ? ": identical to last error" : "",
suppress ? ": error log suppressed" : "");
if (level != LOG_DEBUG && (level <= LOG_CRIT || !suppress)) {
rd_kafka_log(rkb->rkb_rk, level, "FAIL", "%s: %s",
rkb->rkb_name, errstr);
/* Send ERR op to application for processing. */
rd_kafka_q_op_err(rkb->rkb_rk->rk_rep, err, "%s: %s",
rkb->rkb_name, errstr);
}
}
/**
* @brief Failure propagation to application.
*
* Will tear down connection to broker and trigger a reconnect.
*
* \p level is the log level, <=LOG_INFO will be logged while =LOG_DEBUG will
* be debug-logged.
*
* @locality broker thread
*/
void rd_kafka_broker_fail(rd_kafka_broker_t *rkb,
int level,
rd_kafka_resp_err_t err,
const char *fmt,
...) {
va_list ap;
rd_kafka_bufq_t tmpq_waitresp, tmpq;
int old_state;
rd_kafka_toppar_t *rktp;
rd_kafka_assert(rkb->rkb_rk, thrd_is_current(rkb->rkb_thread));
if (rkb->rkb_transport) {
rd_kafka_transport_close(rkb->rkb_transport);
rkb->rkb_transport = NULL;
if (rkb->rkb_state >= RD_KAFKA_BROKER_STATE_UP)
rd_atomic32_add(&rkb->rkb_c.disconnects, 1);
}
rkb->rkb_req_timeouts = 0;
if (rkb->rkb_recv_buf) {
rd_kafka_buf_destroy(rkb->rkb_recv_buf);
rkb->rkb_recv_buf = NULL;
}
rkb->rkb_reauth_in_progress = rd_false;
va_start(ap, fmt);
rd_kafka_broker_set_error(rkb, level, err, fmt, ap);
va_end(ap);
rd_kafka_broker_lock(rkb);
/* If we're currently asking for ApiVersion and the connection
* went down it probably means the broker does not support that request
* and tore down the connection. In this case we disable that feature
* flag. */
if (rkb->rkb_state == RD_KAFKA_BROKER_STATE_APIVERSION_QUERY)
rd_kafka_broker_feature_disable(rkb,
RD_KAFKA_FEATURE_APIVERSION);
/* Set broker state */
old_state = rkb->rkb_state;
rd_kafka_broker_set_state(rkb, RD_KAFKA_BROKER_STATE_DOWN);
/* Stop any pending reauth timer, since a teardown/reconnect will
* require a new timer. */
rd_kafka_timer_stop(&rkb->rkb_rk->rk_timers, &rkb->rkb_sasl_reauth_tmr,
1 /*lock*/);
/* Unlock broker since a requeue will try to lock it. */
rd_kafka_broker_unlock(rkb);
rd_atomic64_set(&rkb->rkb_c.ts_send, 0);
rd_atomic64_set(&rkb->rkb_c.ts_recv, 0);
/*
* Purge all buffers
* (put bufs on a temporary queue since bufs may be requeued,
* make sure outstanding requests are re-enqueued before
* bufs on outbufs queue.)
*/
rd_kafka_bufq_init(&tmpq_waitresp);
rd_kafka_bufq_init(&tmpq);
rd_kafka_bufq_concat(&tmpq_waitresp, &rkb->rkb_waitresps);
rd_kafka_bufq_concat(&tmpq, &rkb->rkb_outbufs);
rd_atomic32_init(&rkb->rkb_blocking_request_cnt, 0);
/* Purge the in-flight buffers (might get re-enqueued in case
* of retries). */
rd_kafka_bufq_purge(rkb, &tmpq_waitresp, err);
/* Purge the waiting-in-output-queue buffers,
* might also get re-enqueued. */
rd_kafka_bufq_purge(rkb, &tmpq,
/* If failure was caused by a timeout,
* adjust the error code for in-queue requests. */
err == RD_KAFKA_RESP_ERR__TIMED_OUT
? RD_KAFKA_RESP_ERR__TIMED_OUT_QUEUE
: err);
/* Update bufq for connection reset:
* - Purge connection-setup requests from outbufs since they will be
* reissued on the next connect.
* - Reset any partially sent buffer's offset.
*/
rd_kafka_bufq_connection_reset(rkb, &rkb->rkb_outbufs);
/* Extra debugging for tracking termination-hang issues:
* show what is keeping this broker from decommissioning. */
if (rd_kafka_terminating(rkb->rkb_rk) &&
!rd_kafka_broker_terminating(rkb)) {
rd_rkb_dbg(rkb, BROKER | RD_KAFKA_DBG_PROTOCOL, "BRKTERM",
"terminating: broker still has %d refcnt(s), "
"%" PRId32 " buffer(s), %d partition(s)",
rd_refcnt_get(&rkb->rkb_refcnt),
rd_kafka_bufq_cnt(&rkb->rkb_outbufs),
rkb->rkb_toppar_cnt);
rd_kafka_bufq_dump(rkb, "BRKOUTBUFS", &rkb->rkb_outbufs);
}
/* If this broker acts as the preferred (follower) replica for any
* partition, delegate the partition back to the leader. */
TAILQ_FOREACH(rktp, &rkb->rkb_toppars, rktp_rkblink) {
rd_kafka_toppar_lock(rktp);
if (unlikely(rktp->rktp_broker != rkb)) {
/* Currently migrating away from this
* broker, skip. */
rd_kafka_toppar_unlock(rktp);
continue;
}
rd_kafka_toppar_unlock(rktp);
if (rktp->rktp_leader_id != rktp->rktp_broker_id) {
rd_kafka_toppar_delegate_to_leader(rktp);
}
}
/* If the broker is the preferred telemetry broker, remove it. */
/* TODO(milind): check if this right. */
mtx_lock(&rkb->rkb_rk->rk_telemetry.lock);
if (rkb->rkb_rk->rk_telemetry.preferred_broker == rkb) {
rd_kafka_dbg(rkb->rkb_rk, TELEMETRY, "TELBRKLOST",
"Lost telemetry broker %s due to state change",
rkb->rkb_name);
rd_kafka_broker_destroy(
rkb->rkb_rk->rk_telemetry.preferred_broker);
rkb->rkb_rk->rk_telemetry.preferred_broker = NULL;
}
mtx_unlock(&rkb->rkb_rk->rk_telemetry.lock);
/* Query for topic leaders to quickly pick up on failover. */
if (err != RD_KAFKA_RESP_ERR__DESTROY &&
old_state >= RD_KAFKA_BROKER_STATE_UP)
rd_kafka_metadata_refresh_known_topics(
rkb->rkb_rk, NULL, rd_true /*force*/, "broker down");
}
/**
* @brief Handle broker connection close.
*
* @locality broker thread
*/
void rd_kafka_broker_conn_closed(rd_kafka_broker_t *rkb,
rd_kafka_resp_err_t err,
const char *errstr) {
int log_level = LOG_ERR;
if (!rkb->rkb_rk->rk_conf.log_connection_close) {
/* Silence all connection closes */
log_level = LOG_DEBUG;
} else {
/* Silence close logs for connections that are idle,
* it is most likely the broker's idle connection
* reaper kicking in.
*
* Indications there might be an error and not an
* idle disconnect:
* - If the connection age is low a disconnect
* typically indicates a failure, such as protocol mismatch.
* - If the connection hasn't been idle long enough.
* - There are outstanding requests, or requests enqueued.
*
* For non-idle connections, adjust log level:
* - requests in-flight: LOG_WARNING
* - else: LOG_INFO
*/
rd_ts_t now = rd_clock();
rd_ts_t minidle =
RD_MAX(60 * 1000 /*60s*/,
rkb->rkb_rk->rk_conf.socket_timeout_ms) *
1000;
int inflight = rd_kafka_bufq_cnt(&rkb->rkb_waitresps);
int inqueue = rd_kafka_bufq_cnt(&rkb->rkb_outbufs);
if (rkb->rkb_ts_state + minidle < now &&
rd_atomic64_get(&rkb->rkb_c.ts_send) + minidle < now &&
inflight + inqueue == 0)
log_level = LOG_DEBUG;
else if (inflight > 1)
log_level = LOG_WARNING;
else
log_level = LOG_INFO;
}
rd_kafka_broker_fail(rkb, log_level, err, "%s", errstr);
}
/**
* @brief Purge requests in \p rkbq matching request \p ApiKey
* and partition \p rktp.
*
* @warning ApiKey must be RD_KAFKAP_Produce
*
* @returns the number of purged buffers.
*
* @locality broker thread
*/
static int rd_kafka_broker_bufq_purge_by_toppar(rd_kafka_broker_t *rkb,
rd_kafka_bufq_t *rkbq,
int64_t ApiKey,
rd_kafka_toppar_t *rktp,
rd_kafka_resp_err_t err) {
rd_kafka_buf_t *rkbuf, *tmp;
int cnt = 0;
rd_assert(ApiKey == RD_KAFKAP_Produce);
TAILQ_FOREACH_SAFE(rkbuf, &rkbq->rkbq_bufs, rkbuf_link, tmp) {
if (rkbuf->rkbuf_reqhdr.ApiKey != ApiKey ||
rkbuf->rkbuf_u.Produce.batch.rktp != rktp ||
/* Skip partially sent buffers and let them transmit.
* The alternative would be to kill the connection here,
* which is more drastic and costly. */
rd_slice_offset(&rkbuf->rkbuf_reader) > 0)
continue;
rd_kafka_bufq_deq(rkbq, rkbuf);
rd_kafka_buf_callback(rkb->rkb_rk, rkb, err, NULL, rkbuf);
cnt++;
}
return cnt;
}
/**
* Scan bufq for buffer timeouts, trigger buffer callback on timeout.
*
* If \p partial_cntp is non-NULL any partially sent buffers will increase
* the provided counter by 1.
*
* @param ApiKey Only match requests with this ApiKey, or -1 for all.
* @param now If 0, all buffers will time out, else the current clock.
* @param description "N requests timed out <description>", e.g., "in flight".
* Only used if log_first_n > 0.
* @param log_first_n Log the first N request timeouts.
*
* @returns the number of timed out buffers.
*
* @locality broker thread
*/
static int rd_kafka_broker_bufq_timeout_scan(rd_kafka_broker_t *rkb,
int is_waitresp_q,
rd_kafka_bufq_t *rkbq,
int *partial_cntp,
int16_t ApiKey,
rd_kafka_resp_err_t err,
rd_ts_t now,
const char *description,
int log_first_n) {
rd_kafka_buf_t *rkbuf, *tmp;
int cnt = 0;
int idx = -1;
const rd_kafka_buf_t *holb;
restart:
holb = TAILQ_FIRST(&rkbq->rkbq_bufs);
TAILQ_FOREACH_SAFE(rkbuf, &rkbq->rkbq_bufs, rkbuf_link, tmp) {
rd_kafka_broker_state_t pre_state, post_state;
idx++;
if (likely(now && rkbuf->rkbuf_ts_timeout > now))
continue;
if (ApiKey != -1 && rkbuf->rkbuf_reqhdr.ApiKey != ApiKey)
continue;
if (partial_cntp && rd_slice_offset(&rkbuf->rkbuf_reader) > 0)
(*partial_cntp)++;
/* Convert rkbuf_ts_sent to elapsed time since request */
if (rkbuf->rkbuf_ts_sent)
rkbuf->rkbuf_ts_sent = now - rkbuf->rkbuf_ts_sent;
else
rkbuf->rkbuf_ts_sent = now - rkbuf->rkbuf_ts_enq;
rd_kafka_bufq_deq(rkbq, rkbuf);
if (now && cnt < log_first_n) {
char holbstr[256];
/* Head of line blocking:
* If this is not the first request in queue, but the
* initial first request did not time out,
* it typically means the first request is a
* long-running blocking one, holding up the
* sub-sequent requests.
* In this case log what is likely holding up the
* requests and what caused this request to time out. */
if (holb && holb == TAILQ_FIRST(&rkbq->rkbq_bufs)) {
rd_snprintf(
holbstr, sizeof(holbstr),
": possibly held back by "
"preceeding%s %sRequest with "
"timeout in %dms",
(holb->rkbuf_flags & RD_KAFKA_OP_F_BLOCKING)
? " blocking"
: "",
rd_kafka_ApiKey2str(
holb->rkbuf_reqhdr.ApiKey),
(int)((holb->rkbuf_ts_timeout - now) /
1000));
/* Only log the HOLB once */
holb = NULL;
} else {
*holbstr = '\0';
}
rd_rkb_log(
rkb, LOG_NOTICE, "REQTMOUT",
"Timed out %sRequest %s "
"(after %" PRId64 "ms, timeout #%d)%s",
rd_kafka_ApiKey2str(rkbuf->rkbuf_reqhdr.ApiKey),
description, rkbuf->rkbuf_ts_sent / 1000, cnt,
holbstr);
}
if (is_waitresp_q &&
rkbuf->rkbuf_flags & RD_KAFKA_OP_F_BLOCKING &&
rd_atomic32_sub(&rkb->rkb_blocking_request_cnt, 1) == 0)
rd_kafka_brokers_broadcast_state_change(rkb->rkb_rk);
pre_state = rd_kafka_broker_get_state(rkb);
rd_kafka_buf_callback(rkb->rkb_rk, rkb, err, NULL, rkbuf);
cnt++;
/* If the buf_callback() triggered a broker state change
* (typically through broker_fail()) we can't trust the
* queue we are scanning to not have been touched, so we
* either restart the scan or bail out (if broker is now down),
* depending on the new state. #2326 */
post_state = rd_kafka_broker_get_state(rkb);
if (pre_state != post_state) {
/* If the new state is DOWN it means broker_fail()
* was called which may have modified the queues,
* to keep things safe we stop scanning this queue. */
if (post_state == RD_KAFKA_BROKER_STATE_DOWN)
break;
/* Else start scanning the queue from the beginning. */
goto restart;
}
}
return cnt;
}
/**
* Scan the wait-response and outbuf queues for message timeouts.
*
* Locality: Broker thread
*/
static void rd_kafka_broker_timeout_scan(rd_kafka_broker_t *rkb, rd_ts_t now) {
int inflight_cnt, retry_cnt, outq_cnt;
int partial_cnt = 0;
rd_kafka_assert(rkb->rkb_rk, thrd_is_current(rkb->rkb_thread));
/* In-flight requests waiting for response */
inflight_cnt = rd_kafka_broker_bufq_timeout_scan(
rkb, 1, &rkb->rkb_waitresps, NULL, -1, RD_KAFKA_RESP_ERR__TIMED_OUT,
now, "in flight", 5);
/* Requests in retry queue */
retry_cnt = rd_kafka_broker_bufq_timeout_scan(
rkb, 0, &rkb->rkb_retrybufs, NULL, -1,
RD_KAFKA_RESP_ERR__TIMED_OUT_QUEUE, now, "in retry queue", 0);
/* Requests in local queue not sent yet.
* partial_cnt is included in outq_cnt and denotes a request
* that has been partially transmitted. */
outq_cnt = rd_kafka_broker_bufq_timeout_scan(
rkb, 0, &rkb->rkb_outbufs, &partial_cnt, -1,
RD_KAFKA_RESP_ERR__TIMED_OUT_QUEUE, now, "in output queue", 0);
if (inflight_cnt + retry_cnt + outq_cnt + partial_cnt > 0) {
rd_rkb_log(rkb, LOG_WARNING, "REQTMOUT",
"Timed out %i in-flight, %i retry-queued, "
"%i out-queue, %i partially-sent requests",
inflight_cnt, retry_cnt, outq_cnt, partial_cnt);
rkb->rkb_req_timeouts += inflight_cnt + outq_cnt;
rd_atomic64_add(&rkb->rkb_c.req_timeouts,
inflight_cnt + outq_cnt);
/* If this was a partially sent request that timed out, or the
* number of timed out requests have reached the
* socket.max.fails threshold, we need to take down the
* connection. */
if (partial_cnt > 0 ||
(rkb->rkb_rk->rk_conf.socket_max_fails &&
rkb->rkb_req_timeouts >=
rkb->rkb_rk->rk_conf.socket_max_fails &&
rkb->rkb_state >= RD_KAFKA_BROKER_STATE_UP)) {
char rttinfo[32];
/* Print average RTT (if avail) to help diagnose. */
rd_avg_calc(&rkb->rkb_avg_rtt, now);
rd_avg_calc(
&rkb->rkb_telemetry.rd_avg_current.rkb_avg_rtt,
now);
if (rkb->rkb_avg_rtt.ra_v.avg)
rd_snprintf(rttinfo, sizeof(rttinfo),
" (average rtt %.3fms)",
(float)(rkb->rkb_avg_rtt.ra_v.avg /
1000.0f));
else if (rkb->rkb_telemetry.rd_avg_current.rkb_avg_rtt
.ra_v.avg)
rd_snprintf(
rttinfo, sizeof(rttinfo),
" (average rtt %.3fms)",
(float)(rkb->rkb_telemetry.rd_avg_current
.rkb_avg_rtt.ra_v.avg /
1000.0f));
else