forked from confluentinc/librdkafka
-
Notifications
You must be signed in to change notification settings - Fork 0
/
Copy pathrdkafka_mock.c
2066 lines (1608 loc) · 71.5 KB
/
rdkafka_mock.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) 2019 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.
*/
/**
* Mocks
*
*/
#include "rdkafka_int.h"
#include "rdbuf.h"
#include "rdrand.h"
#include "rdkafka_interceptor.h"
#include "rdkafka_mock_int.h"
#include "rdkafka_transport_int.h"
#include <stdarg.h>
static rd_kafka_mock_broker_t *
rd_kafka_mock_broker_find (const rd_kafka_mock_cluster_t *mcluster,
int32_t broker_id) {
const rd_kafka_mock_broker_t *mrkb;
TAILQ_FOREACH(mrkb, &mcluster->brokers, link)
if (mrkb->id == broker_id)
return (rd_kafka_mock_broker_t *)mrkb;
return NULL;
}
/**
* @brief Unlink and free message set.
*/
static void rd_kafka_mock_msgset_destroy (rd_kafka_mock_partition_t *mpart,
rd_kafka_mock_msgset_t *mset) {
const rd_kafka_mock_msgset_t *next = TAILQ_NEXT(mset, link);
/* Removing last messageset */
if (!next)
mpart->start_offset = mpart->end_offset;
else if (mset == TAILQ_FIRST(&mpart->msgsets))
/* Removing first messageset */
mpart->start_offset = next->first_offset;
if (mpart->update_follower_start_offset)
mpart->follower_start_offset = mpart->start_offset;
rd_assert(mpart->cnt > 0);
mpart->cnt--;
mpart->size -= RD_KAFKAP_BYTES_LEN(&mset->bytes);
TAILQ_REMOVE(&mpart->msgsets, mset, link);
rd_free(mset);
}
/**
* @brief Create a new msgset object with a copy of \p bytes
* and appends it to the partition log.
*/
static rd_kafka_mock_msgset_t *
rd_kafka_mock_msgset_new (rd_kafka_mock_partition_t *mpart,
const rd_kafkap_bytes_t *bytes, size_t msgcnt) {
rd_kafka_mock_msgset_t *mset;
size_t totsize = sizeof(*mset) + RD_KAFKAP_BYTES_LEN(bytes);
int64_t BaseOffset;
int64_t orig_start_offset = mpart->start_offset;
rd_assert(!RD_KAFKAP_BYTES_IS_NULL(bytes));
mset = rd_malloc(totsize);
rd_assert(mset != NULL);
mset->first_offset = mpart->end_offset;
mset->last_offset = mset->first_offset + msgcnt - 1;
mpart->end_offset = mset->last_offset + 1;
if (mpart->update_follower_end_offset)
mpart->follower_end_offset = mpart->end_offset;
mpart->cnt++;
mset->bytes.len = bytes->len;
mset->bytes.data = (void *)(mset+1);
memcpy((void *)mset->bytes.data, bytes->data, mset->bytes.len);
mpart->size += mset->bytes.len;
/* Update the base Offset in the MessageSet with the
* actual absolute log offset. */
BaseOffset = htobe64(mset->first_offset);
memcpy((void *)mset->bytes.data, &BaseOffset, sizeof(BaseOffset));
/* Remove old msgsets until within limits */
while (mpart->cnt > 1 &&
(mpart->cnt > mpart->max_cnt ||
mpart->size > mpart->max_size))
rd_kafka_mock_msgset_destroy(mpart,
TAILQ_FIRST(&mpart->msgsets));
TAILQ_INSERT_TAIL(&mpart->msgsets, mset, link);
rd_kafka_dbg(mpart->topic->cluster->rk, MOCK, "MOCK",
"Broker %"PRId32": Log append %s [%"PRId32"] "
"%"PRIusz" messages, %"PRId32" bytes at offset %"PRId64
" (log now %"PRId64"..%"PRId64", "
"original start %"PRId64")",
mpart->leader->id, mpart->topic->name, mpart->id,
msgcnt, RD_KAFKAP_BYTES_LEN(&mset->bytes),
mset->first_offset,
mpart->start_offset, mpart->end_offset,
orig_start_offset);
return mset;
}
/**
* @brief Find message set containing \p offset
*/
const rd_kafka_mock_msgset_t *
rd_kafka_mock_msgset_find (const rd_kafka_mock_partition_t *mpart,
int64_t offset, rd_bool_t on_follower) {
const rd_kafka_mock_msgset_t *mset;
if (!on_follower &&
(offset < mpart->start_offset ||
offset > mpart->end_offset))
return NULL;
if (on_follower &&
(offset < mpart->follower_start_offset ||
offset > mpart->follower_end_offset))
return NULL;
/* FIXME: Maintain an index */
TAILQ_FOREACH(mset, &mpart->msgsets, link) {
if (mset->first_offset <= offset &&
offset <= mset->last_offset)
return mset;
}
return NULL;
}
/**
* @brief Append the MessageSets in \p bytes to the \p mpart partition log.
*
* @param BaseOffset will contain the first assigned offset of the message set.
*/
rd_kafka_resp_err_t
rd_kafka_mock_partition_log_append (rd_kafka_mock_partition_t *mpart,
const rd_kafkap_bytes_t *bytes,
int64_t *BaseOffset) {
const int log_decode_errors = LOG_ERR;
rd_kafka_buf_t *rkbuf;
rd_kafka_resp_err_t err = RD_KAFKA_RESP_ERR_NO_ERROR;
int8_t MagicByte;
int32_t RecordCount;
rd_kafka_mock_msgset_t *mset;
/* Partially parse the MessageSet in \p bytes to get
* the message count. */
rkbuf = rd_kafka_buf_new_shadow(bytes->data,
RD_KAFKAP_BYTES_LEN(bytes), NULL);
rd_kafka_buf_peek_i8(rkbuf, 8+4+4, &MagicByte);
if (MagicByte != 2) {
/* We only support MsgVersion 2 for now */
err = RD_KAFKA_RESP_ERR_UNSUPPORTED_VERSION;
goto err;
}
rd_kafka_buf_peek_i32(rkbuf, RD_KAFKAP_MSGSET_V2_OF_RecordCount,
&RecordCount);
if (RecordCount < 1 ||
(size_t)RecordCount >
RD_KAFKAP_BYTES_LEN(bytes) / RD_KAFKAP_MESSAGE_V2_OVERHEAD) {
err = RD_KAFKA_RESP_ERR_INVALID_MSG_SIZE;
goto err;
}
rd_kafka_buf_destroy(rkbuf);
mset = rd_kafka_mock_msgset_new(mpart, bytes, (size_t)RecordCount);
*BaseOffset = mset->first_offset;
return RD_KAFKA_RESP_ERR_NO_ERROR;
err_parse:
err = rkbuf->rkbuf_err;
err:
rd_kafka_buf_destroy(rkbuf);
return err;
}
/**
* @brief Set the partition leader
*/
static void
rd_kafka_mock_partition_set_leader0 (rd_kafka_mock_partition_t *mpart,
rd_kafka_mock_broker_t *mrkb) {
mpart->leader = mrkb;
}
/**
* @brief Automatically assign replicas for partition
*/
static void
rd_kafka_mock_partition_assign_replicas (rd_kafka_mock_partition_t *mpart) {
rd_kafka_mock_cluster_t *mcluster = mpart->topic->cluster;
int replica_cnt = RD_MIN(mcluster->defaults.replication_factor,
mcluster->broker_cnt);
rd_kafka_mock_broker_t *mrkb;
int i = 0;
if (mpart->replicas)
rd_free(mpart->replicas);
mpart->replicas = rd_calloc(replica_cnt, sizeof(*mpart->replicas));
mpart->replica_cnt = replica_cnt;
/* FIXME: randomize this using perhaps reservoir sampling */
TAILQ_FOREACH(mrkb, &mcluster->brokers, link) {
if (i == mpart->replica_cnt)
break;
mpart->replicas[i++] = mrkb;
}
/* Select a random leader */
rd_kafka_mock_partition_set_leader0(
mpart, mpart->replicas[rd_jitter(0, replica_cnt-1)]);
}
/**
* @brief Unlink and destroy committed offset
*/
static void
rd_kafka_mock_committed_offset_destroy (rd_kafka_mock_partition_t *mpart,
rd_kafka_mock_committed_offset_t *coff){
rd_kafkap_str_destroy(coff->metadata);
TAILQ_REMOVE(&mpart->committed_offsets, coff, link);
rd_free(coff);
}
/**
* @brief Find previously committed offset for group.
*/
rd_kafka_mock_committed_offset_t *
rd_kafka_mock_committed_offset_find (const rd_kafka_mock_partition_t *mpart,
const rd_kafkap_str_t *group) {
const rd_kafka_mock_committed_offset_t *coff;
TAILQ_FOREACH(coff, &mpart->committed_offsets, link) {
if (rd_kafkap_str_cmp_str(group, coff->group))
return (rd_kafka_mock_committed_offset_t *)coff;
}
return NULL;
}
/**
* @brief Commit offset for group
*/
rd_kafka_mock_committed_offset_t *
rd_kafka_mock_commit_offset (rd_kafka_mock_partition_t *mpart,
const rd_kafkap_str_t *group, int64_t offset,
const rd_kafkap_str_t *metadata) {
rd_kafka_mock_committed_offset_t *coff;
if (!(coff = rd_kafka_mock_committed_offset_find(mpart, group))) {
size_t slen = (size_t)RD_KAFKAP_STR_LEN(group);
coff = rd_malloc(sizeof(*coff) + slen + 1);
coff->group = (char *)(coff + 1);
memcpy(coff->group, group->str, slen);
coff->group[slen] = '\0';
coff->metadata = NULL;
TAILQ_INSERT_HEAD(&mpart->committed_offsets, coff, link);
}
if (coff->metadata)
rd_kafkap_str_destroy(coff->metadata);
coff->metadata = rd_kafkap_str_copy(metadata);
coff->offset = offset;
rd_kafka_dbg(mpart->topic->cluster->rk, MOCK, "MOCK",
"Topic %s [%"PRId32"] committing offset %"PRId64
" for group %.*s",
mpart->topic->name, mpart->id, offset,
RD_KAFKAP_STR_PR(group));
return coff;
}
/**
* @brief Destroy resources for partition, but the \p mpart itself is not freed.
*/
static void rd_kafka_mock_partition_destroy (rd_kafka_mock_partition_t *mpart) {
rd_kafka_mock_msgset_t *mset, *tmp;
rd_kafka_mock_committed_offset_t *coff, *tmpcoff;
TAILQ_FOREACH_SAFE(mset, &mpart->msgsets, link, tmp)
rd_kafka_mock_msgset_destroy(mpart, mset);
TAILQ_FOREACH_SAFE(coff, &mpart->committed_offsets, link, tmpcoff)
rd_kafka_mock_committed_offset_destroy(mpart, coff);
rd_free(mpart->replicas);
}
static void rd_kafka_mock_partition_init (rd_kafka_mock_topic_t *mtopic,
rd_kafka_mock_partition_t *mpart,
int id, int replication_factor) {
mpart->topic = mtopic;
mpart->id = id;
mpart->follower_id = -1;
TAILQ_INIT(&mpart->msgsets);
mpart->max_size = 1024*1024*5;
mpart->max_cnt = 100000;
mpart->update_follower_start_offset = rd_true;
mpart->update_follower_end_offset = rd_true;
TAILQ_INIT(&mpart->committed_offsets);
rd_kafka_mock_partition_assign_replicas(mpart);
}
rd_kafka_mock_partition_t *
rd_kafka_mock_partition_find (const rd_kafka_mock_topic_t *mtopic,
int32_t partition) {
if (partition < 0 || partition >= mtopic->partition_cnt)
return NULL;
return (rd_kafka_mock_partition_t *)&mtopic->partitions[partition];
}
static void rd_kafka_mock_topic_destroy (rd_kafka_mock_topic_t *mtopic) {
int i;
for (i = 0 ; i < mtopic->partition_cnt ; i++)
rd_kafka_mock_partition_destroy(&mtopic->partitions[i]);
TAILQ_REMOVE(&mtopic->cluster->topics, mtopic, link);
mtopic->cluster->topic_cnt--;
rd_free(mtopic->partitions);
rd_free(mtopic->name);
rd_free(mtopic);
}
static rd_kafka_mock_topic_t *
rd_kafka_mock_topic_new (rd_kafka_mock_cluster_t *mcluster, const char *topic,
int partition_cnt, int replication_factor) {
rd_kafka_mock_topic_t *mtopic;
int i;
mtopic = rd_calloc(1, sizeof(*mtopic));
mtopic->name = rd_strdup(topic);
mtopic->cluster = mcluster;
mtopic->partition_cnt = partition_cnt;
mtopic->partitions = rd_calloc(partition_cnt,
sizeof(*mtopic->partitions));
for (i = 0 ; i < partition_cnt ; i++)
rd_kafka_mock_partition_init(mtopic, &mtopic->partitions[i],
i, replication_factor);
TAILQ_INSERT_TAIL(&mcluster->topics, mtopic, link);
mcluster->topic_cnt++;
rd_kafka_dbg(mcluster->rk, MOCK, "MOCK",
"Created topic \"%s\" with %d partition(s) and "
"replication-factor %d",
mtopic->name, mtopic->partition_cnt, replication_factor);
return mtopic;
}
rd_kafka_mock_topic_t *
rd_kafka_mock_topic_find (const rd_kafka_mock_cluster_t *mcluster,
const char *name) {
const rd_kafka_mock_topic_t *mtopic;
TAILQ_FOREACH(mtopic, &mcluster->topics, link) {
if (!strcmp(mtopic->name, name))
return (rd_kafka_mock_topic_t *)mtopic;
}
return NULL;
}
rd_kafka_mock_topic_t *
rd_kafka_mock_topic_find_by_kstr (const rd_kafka_mock_cluster_t *mcluster,
const rd_kafkap_str_t *kname) {
const rd_kafka_mock_topic_t *mtopic;
TAILQ_FOREACH(mtopic, &mcluster->topics, link) {
if (!strncmp(mtopic->name, kname->str,
RD_KAFKAP_STR_LEN(kname)) &&
mtopic->name[RD_KAFKAP_STR_LEN(kname)] == '\0')
return (rd_kafka_mock_topic_t *)mtopic;
}
return NULL;
}
/**
* @brief Create a topic using default settings.
* The topic must not already exist.
*
* @param errp will be set to an error code that is consistent with
* new topics on real clusters.
*/
rd_kafka_mock_topic_t *
rd_kafka_mock_topic_auto_create (rd_kafka_mock_cluster_t *mcluster,
const char *topic, int partition_cnt,
rd_kafka_resp_err_t *errp) {
rd_assert(!rd_kafka_mock_topic_find(mcluster, topic));
*errp = 0; // FIXME? RD_KAFKA_RESP_ERR_LEADER_NOT_AVAILABLE;
return rd_kafka_mock_topic_new(mcluster, topic,
partition_cnt == -1 ?
mcluster->defaults.partition_cnt :
partition_cnt,
mcluster->defaults.replication_factor);
}
/**
* @brief Find or create topic.
*
* @param partition_cnt If not -1 and the topic does not exist, the automatic
* topic creation will create this number of topics.
* Otherwise use the default.
*/
rd_kafka_mock_topic_t *
rd_kafka_mock_topic_get (rd_kafka_mock_cluster_t *mcluster, const char *topic,
int partition_cnt) {
rd_kafka_mock_topic_t *mtopic;
rd_kafka_resp_err_t err;
if ((mtopic = rd_kafka_mock_topic_find(mcluster, topic)))
return mtopic;
return rd_kafka_mock_topic_auto_create(mcluster, topic,
partition_cnt, &err);
}
/**
* @brief Find or create a partition.
*
* @returns NULL if topic already exists and partition is out of range.
*/
static rd_kafka_mock_partition_t *
rd_kafka_mock_partition_get (rd_kafka_mock_cluster_t *mcluster,
const char *topic, int32_t partition) {
rd_kafka_mock_topic_t *mtopic;
rd_kafka_resp_err_t err;
if (!(mtopic = rd_kafka_mock_topic_find(mcluster, topic)))
mtopic = rd_kafka_mock_topic_auto_create(mcluster, topic,
partition+1, &err);
if (partition >= mtopic->partition_cnt)
return NULL;
return &mtopic->partitions[partition];
}
/**
* @brief Set IO events for fd
*/
static void
rd_kafka_mock_cluster_io_set_events (rd_kafka_mock_cluster_t *mcluster,
rd_socket_t fd, int events) {
int i;
for (i = 0 ; i < mcluster->fd_cnt ; i++) {
if (mcluster->fds[i].fd == fd) {
mcluster->fds[i].events |= events;
return;
}
}
rd_assert(!*"mock_cluster_io_set_events: fd not found");
}
/**
* @brief Set or clear single IO events for fd
*/
static void
rd_kafka_mock_cluster_io_set_event (rd_kafka_mock_cluster_t *mcluster,
rd_socket_t fd, rd_bool_t set, int event) {
int i;
for (i = 0 ; i < mcluster->fd_cnt ; i++) {
if (mcluster->fds[i].fd == fd) {
if (set)
mcluster->fds[i].events |= event;
else
mcluster->fds[i].events &= ~event;
return;
}
}
rd_assert(!*"mock_cluster_io_set_event: fd not found");
}
/**
* @brief Clear IO events for fd
*/
static void
rd_kafka_mock_cluster_io_clear_events (rd_kafka_mock_cluster_t *mcluster,
rd_socket_t fd, int events) {
int i;
for (i = 0 ; i < mcluster->fd_cnt ; i++) {
if (mcluster->fds[i].fd == fd) {
mcluster->fds[i].events &= ~events;
return;
}
}
rd_assert(!*"mock_cluster_io_set_events: fd not found");
}
static void rd_kafka_mock_cluster_io_del (rd_kafka_mock_cluster_t *mcluster,
rd_socket_t fd) {
int i;
for (i = 0 ; i < mcluster->fd_cnt ; i++) {
if (mcluster->fds[i].fd == fd) {
if (i + 1 < mcluster->fd_cnt) {
memmove(&mcluster->fds[i],
&mcluster->fds[i+1],
sizeof(*mcluster->fds) *
(mcluster->fd_cnt - i));
memmove(&mcluster->handlers[i],
&mcluster->handlers[i+1],
sizeof(*mcluster->handlers) *
(mcluster->fd_cnt - i));
}
mcluster->fd_cnt--;
return;
}
}
rd_assert(!*"mock_cluster_io_del: fd not found");
}
/**
* @brief Add \p fd to IO poll with initial desired events (POLLIN, et.al).
*/
static void rd_kafka_mock_cluster_io_add (rd_kafka_mock_cluster_t *mcluster,
rd_socket_t fd, int events,
rd_kafka_mock_io_handler_t handler,
void *opaque) {
if (mcluster->fd_cnt + 1 >= mcluster->fd_size) {
mcluster->fd_size += 8;
mcluster->fds = rd_realloc(mcluster->fds,
sizeof(*mcluster->fds) *
mcluster->fd_size);
mcluster->handlers = rd_realloc(mcluster->handlers,
sizeof(*mcluster->handlers) *
mcluster->fd_size);
}
memset(&mcluster->fds[mcluster->fd_cnt], 0,
sizeof(mcluster->fds[mcluster->fd_cnt]));
mcluster->fds[mcluster->fd_cnt].fd = fd;
mcluster->fds[mcluster->fd_cnt].events = events;
mcluster->fds[mcluster->fd_cnt].revents = 0;
mcluster->handlers[mcluster->fd_cnt].cb = handler;
mcluster->handlers[mcluster->fd_cnt].opaque = opaque;
mcluster->fd_cnt++;
}
static void rd_kafka_mock_connection_close (rd_kafka_mock_connection_t *mconn,
const char *reason) {
rd_kafka_buf_t *rkbuf;
rd_kafka_dbg(mconn->broker->cluster->rk, MOCK, "MOCK",
"Broker %"PRId32": Connection from %s closed: %s",
mconn->broker->id,
rd_sockaddr2str(&mconn->peer, RD_SOCKADDR2STR_F_PORT),
reason);
rd_kafka_mock_cgrps_connection_closed(mconn->broker->cluster, mconn);
rd_kafka_timer_stop(&mconn->broker->cluster->timers,
&mconn->write_tmr, rd_true);
while ((rkbuf = TAILQ_FIRST(&mconn->outbufs.rkbq_bufs))) {
rd_kafka_bufq_deq(&mconn->outbufs, rkbuf);
rd_kafka_buf_destroy(rkbuf);
}
if (mconn->rxbuf)
rd_kafka_buf_destroy(mconn->rxbuf);
rd_kafka_mock_cluster_io_del(mconn->broker->cluster,
mconn->transport->rktrans_s);
TAILQ_REMOVE(&mconn->broker->connections, mconn, link);
rd_kafka_transport_close(mconn->transport);
rd_free(mconn);
}
void rd_kafka_mock_connection_send_response (rd_kafka_mock_connection_t *mconn,
rd_kafka_buf_t *resp) {
resp->rkbuf_reshdr.Size =
(int32_t)(rd_buf_write_pos(&resp->rkbuf_buf) - 4);
rd_kafka_buf_update_i32(resp, 0, resp->rkbuf_reshdr.Size);
rd_kafka_dbg(mconn->broker->cluster->rk, MOCK, "MOCK",
"Broker %"PRId32": Sending %sResponseV%hd to %s",
mconn->broker->id,
rd_kafka_ApiKey2str(resp->rkbuf_reqhdr.ApiKey),
resp->rkbuf_reqhdr.ApiVersion,
rd_sockaddr2str(&mconn->peer, RD_SOCKADDR2STR_F_PORT));
/* Set up a buffer reader for sending the buffer. */
rd_slice_init_full(&resp->rkbuf_reader, &resp->rkbuf_buf);
rd_kafka_bufq_enq(&mconn->outbufs, resp);
rd_kafka_mock_cluster_io_set_events(mconn->broker->cluster,
mconn->transport->rktrans_s,
POLLOUT);
}
/**
* @returns 1 if a complete request is available in which case \p slicep
* is set to a new slice containing the data,
* 0 if a complete request is not yet available,
* -1 on error.
*/
static int
rd_kafka_mock_connection_read_request (rd_kafka_mock_connection_t *mconn,
rd_kafka_buf_t **rkbufp) {
rd_kafka_mock_cluster_t *mcluster = mconn->broker->cluster;
rd_kafka_t *rk = mcluster->rk;
const rd_bool_t log_decode_errors = rd_true;
rd_kafka_buf_t *rkbuf;
char errstr[128];
ssize_t r;
if (!(rkbuf = mconn->rxbuf)) {
/* Initial read for a protocol request.
* Allocate enough room for the protocol header
* (where the total size is located). */
rkbuf = mconn->rxbuf = rd_kafka_buf_new(2,
RD_KAFKAP_REQHDR_SIZE);
/* Protocol parsing code needs the rkb for logging */
rkbuf->rkbuf_rkb = mconn->broker->cluster->dummy_rkb;
rd_kafka_broker_keep(rkbuf->rkbuf_rkb);
/* Make room for request header */
rd_buf_write_ensure(&rkbuf->rkbuf_buf,
RD_KAFKAP_REQHDR_SIZE,
RD_KAFKAP_REQHDR_SIZE);
}
/* Read as much data as possible from the socket into the
* connection receive buffer. */
r = rd_kafka_transport_recv(mconn->transport, &rkbuf->rkbuf_buf,
errstr, sizeof(errstr));
if (r == -1) {
rd_kafka_dbg(rk, MOCK, "MOCK",
"Broker %"PRId32": Connection %s: "
"receive failed: %s",
mconn->broker->id,
rd_sockaddr2str(&mconn->peer,
RD_SOCKADDR2STR_F_PORT),
errstr);
return -1;
} else if (r == 0) {
return 0; /* Need more data */
}
if (rd_buf_write_pos(&rkbuf->rkbuf_buf) ==
RD_KAFKAP_REQHDR_SIZE) {
/* Received the full header, now check full request
* size and allocate the buffer accordingly. */
/* Initialize reader */
rd_slice_init(&rkbuf->rkbuf_reader,
&rkbuf->rkbuf_buf, 0,
RD_KAFKAP_REQHDR_SIZE);
rd_kafka_buf_read_i32(rkbuf,
&rkbuf->rkbuf_reqhdr.Size);
rd_kafka_buf_read_i16(rkbuf,
&rkbuf->rkbuf_reqhdr.ApiKey);
rd_kafka_buf_read_i16(rkbuf,
&rkbuf->rkbuf_reqhdr.ApiVersion);
if (rkbuf->rkbuf_reqhdr.ApiKey < 0 ||
rkbuf->rkbuf_reqhdr.ApiKey >= RD_KAFKAP__NUM) {
rd_kafka_buf_parse_fail(rkbuf,
"Invalid ApiKey %hd from %s",
rkbuf->rkbuf_reqhdr.ApiKey,
rd_sockaddr2str(
&mconn->peer,
RD_SOCKADDR2STR_F_PORT));
RD_NOTREACHED();
}
/* Check if request version has flexible fields (KIP-482) */
if (mcluster->api_handlers[rkbuf->rkbuf_reqhdr.ApiKey].
FlexVersion != -1 &&
rkbuf->rkbuf_reqhdr.ApiVersion >=
mcluster->api_handlers[rkbuf->rkbuf_reqhdr.ApiKey].
FlexVersion)
rkbuf->rkbuf_flags |= RD_KAFKA_OP_F_FLEXVER;
rd_kafka_buf_read_i32(rkbuf,
&rkbuf->rkbuf_reqhdr.CorrId);
rkbuf->rkbuf_totlen = rkbuf->rkbuf_reqhdr.Size + 4;
if (rkbuf->rkbuf_totlen < RD_KAFKAP_REQHDR_SIZE + 2 ||
rkbuf->rkbuf_totlen >
(size_t)rk->rk_conf.recv_max_msg_size) {
rd_kafka_buf_parse_fail(
rkbuf,
"Invalid request size %"PRId32
" from %s",
rkbuf->rkbuf_reqhdr.Size,
rd_sockaddr2str(
&mconn->peer,
RD_SOCKADDR2STR_F_PORT));
RD_NOTREACHED();
}
/* Now adjust totlen to skip the header */
rkbuf->rkbuf_totlen -= RD_KAFKAP_REQHDR_SIZE;
if (!rkbuf->rkbuf_totlen) {
/* Empty request (valid) */
*rkbufp = rkbuf;
mconn->rxbuf = NULL;
return 1;
}
/* Allocate space for the request payload */
rd_buf_write_ensure(&rkbuf->rkbuf_buf,
rkbuf->rkbuf_totlen,
rkbuf->rkbuf_totlen);
} else if (rd_buf_write_pos(&rkbuf->rkbuf_buf) -
RD_KAFKAP_REQHDR_SIZE == rkbuf->rkbuf_totlen) {
/* The full request is now read into the buffer. */
rd_kafkap_str_t clientid;
/* Set up response reader slice starting past the
* request header */
rd_slice_init(&rkbuf->rkbuf_reader, &rkbuf->rkbuf_buf,
RD_KAFKAP_REQHDR_SIZE,
rd_buf_len(&rkbuf->rkbuf_buf) -
RD_KAFKAP_REQHDR_SIZE);
/* For convenience, shave off the ClientId */
rd_kafka_buf_read_compact_str(rkbuf, &clientid);
/* Return the buffer to the caller */
*rkbufp = rkbuf;
mconn->rxbuf = NULL;
return 1;
}
return 0;
err_parse:
return -1;
}
rd_kafka_buf_t *rd_kafka_mock_buf_new_response (const rd_kafka_buf_t *request) {
rd_kafka_buf_t *rkbuf = rd_kafka_buf_new(1, 100);
/* Copy request header so the ApiVersion remains known */
rkbuf->rkbuf_reqhdr = request->rkbuf_reqhdr;
/* Size, updated later */
rd_kafka_buf_write_i32(rkbuf, 0);
/* CorrId */
rd_kafka_buf_write_i32(rkbuf, request->rkbuf_reqhdr.CorrId);
return rkbuf;
}
/**
* @brief Parse protocol request.
*
* @returns 0 on success, -1 on parse error.
*/
static int
rd_kafka_mock_connection_parse_request (rd_kafka_mock_connection_t *mconn,
rd_kafka_buf_t *rkbuf) {
rd_kafka_mock_cluster_t *mcluster = mconn->broker->cluster;
rd_kafka_t *rk = mcluster->rk;
if (rkbuf->rkbuf_reqhdr.ApiKey < 0 ||
rkbuf->rkbuf_reqhdr.ApiKey >= RD_KAFKAP__NUM ||
!mcluster->api_handlers[rkbuf->rkbuf_reqhdr.ApiKey].cb) {
rd_kafka_log(rk, LOG_ERR, "MOCK",
"Broker %"PRId32": unsupported %sRequestV%hd "
"from %s",
mconn->broker->id,
rd_kafka_ApiKey2str(rkbuf->rkbuf_reqhdr.ApiKey),
rkbuf->rkbuf_reqhdr.ApiVersion,
rd_sockaddr2str(&mconn->peer,
RD_SOCKADDR2STR_F_PORT));
return -1;
}
/* ApiVersionRequest handles future versions, for everything else
* make sure the ApiVersion is supported. */
if (rkbuf->rkbuf_reqhdr.ApiKey != RD_KAFKAP_ApiVersion &&
!rd_kafka_mock_cluster_ApiVersion_check(
mcluster,
rkbuf->rkbuf_reqhdr.ApiKey,
rkbuf->rkbuf_reqhdr.ApiVersion)) {
rd_kafka_log(rk, LOG_ERR, "MOCK",
"Broker %"PRId32": unsupported %sRequest "
"version %hd from %s",
mconn->broker->id,
rd_kafka_ApiKey2str(rkbuf->rkbuf_reqhdr.ApiKey),
rkbuf->rkbuf_reqhdr.ApiVersion,
rd_sockaddr2str(&mconn->peer,
RD_SOCKADDR2STR_F_PORT));
return -1;
}
rd_kafka_dbg(rk, MOCK, "MOCK",
"Broker %"PRId32": Received %sRequestV%hd from %s",
mconn->broker->id,
rd_kafka_ApiKey2str(rkbuf->rkbuf_reqhdr.ApiKey),
rkbuf->rkbuf_reqhdr.ApiVersion,
rd_sockaddr2str(&mconn->peer, RD_SOCKADDR2STR_F_PORT));
return mcluster->api_handlers[rkbuf->rkbuf_reqhdr.ApiKey].cb(mconn,
rkbuf);
}
/**
* @brief Timer callback to set the POLLOUT flag for a connection after
* the delay has expired.
*/
static void rd_kafka_mock_connection_write_out_tmr_cb (rd_kafka_timers_t *rkts,
void *arg) {
rd_kafka_mock_connection_t *mconn = arg;
rd_kafka_mock_cluster_io_set_events(mconn->broker->cluster,
mconn->transport->rktrans_s,
POLLOUT);
}
/**
* @brief Send as many bytes as possible from the output buffer.
*
* @returns 1 if all buffers were sent, 0 if more buffers need to be sent, or
* -1 on error.
*/
static ssize_t
rd_kafka_mock_connection_write_out (rd_kafka_mock_connection_t *mconn) {
rd_kafka_buf_t *rkbuf;
rd_ts_t now = rd_clock();
while ((rkbuf = TAILQ_FIRST(&mconn->outbufs.rkbq_bufs))) {
ssize_t r;
char errstr[128];
if (rkbuf->rkbuf_ts_retry && rkbuf->rkbuf_ts_retry > now) {
/* Response is being delayed */
rd_kafka_timer_start_oneshot(
&mconn->broker->cluster->timers,
&mconn->write_tmr,
rd_false,
rkbuf->rkbuf_ts_retry-now,
rd_kafka_mock_connection_write_out_tmr_cb,
mconn);
break;
}
if ((r = rd_kafka_transport_send(mconn->transport,
&rkbuf->rkbuf_reader,
errstr,
sizeof(errstr))) == -1)
return -1;
if (rd_slice_remains(&rkbuf->rkbuf_reader) > 0)
return 0; /* Partial send, continue next time */
/* Entire buffer sent, unlink and free */
rd_kafka_bufq_deq(&mconn->outbufs, rkbuf);
rd_kafka_buf_destroy(rkbuf);
}
rd_kafka_mock_cluster_io_clear_events(mconn->broker->cluster,
mconn->transport->rktrans_s,
POLLOUT);
return 1;
}
/**
* @brief Per-Connection IO handler
*/
static void rd_kafka_mock_connection_io (rd_kafka_mock_cluster_t *mcluster,
rd_socket_t fd,
int events, void *opaque) {
rd_kafka_mock_connection_t *mconn = opaque;
if (events & POLLIN) {
rd_kafka_buf_t *rkbuf;
int r;
while (1) {
/* Read full request */
r = rd_kafka_mock_connection_read_request(mconn,
&rkbuf);
if (r == 0)
break; /* Need more data */