summaryrefslogtreecommitdiffstats
path: root/src/fluent-bit/lib/librdkafka-2.1.0/tests/0019-list_groups.c
blob: 02729c3396465ba2f5bdaaf0176c883dfa2071a7 (plain)
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
/*
 * librdkafka - Apache Kafka C library
 *
 * Copyright (c) 2012-2015, 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.
 */

#include "test.h"

/* Typical include path would be <librdkafka/rdkafka.h>, but this program
 * is built from within the librdkafka source tree and thus differs. */
#include "rdkafka.h" /* for Kafka driver */


/**
 * List consumer groups
 *
 * Runs two consumers in two different groups and lists them.
 */



/**
 * Verify that all groups in 'groups' are seen, if so returns group_cnt,
 * else returns -1.
 */
static int verify_groups(const struct rd_kafka_group_list *grplist,
                         char **groups,
                         int group_cnt) {
        int i;
        int seen = 0;

        for (i = 0; i < grplist->group_cnt; i++) {
                const struct rd_kafka_group_info *gi = &grplist->groups[i];
                int j;

                for (j = 0; j < group_cnt; j++) {
                        if (strcmp(gi->group, groups[j]))
                                continue;

                        if (gi->err)
                                TEST_SAY(
                                    "Group %s has broker-reported "
                                    "error: %s\n",
                                    gi->group, rd_kafka_err2str(gi->err));

                        seen++;
                }
        }

        TEST_SAY("Found %d/%d desired groups in list of %d groups\n", seen,
                 group_cnt, grplist->group_cnt);

        if (seen != group_cnt)
                return -1;
        else
                return seen;
}


/**
 * List groups by:
 *   - List all groups, check that the groups in 'groups' are seen.
 *   - List each group in 'groups', one by one.
 *
 * Returns 'group_cnt' if all groups in 'groups' were seen by both
 * methods, else 0, or -1 on error.
 */
static int
list_groups(rd_kafka_t *rk, char **groups, int group_cnt, const char *desc) {
        rd_kafka_resp_err_t err = 0;
        const struct rd_kafka_group_list *grplist;
        int i, r;
        int fails    = 0;
        int seen     = 0;
        int seen_all = 0;
        int retries  = 5;

        TEST_SAY("List groups (expect %d): %s\n", group_cnt, desc);

        /* FIXME: Wait for broker to come up. This should really be abstracted
         *        by librdkafka. */
        do {
                if (err) {
                        TEST_SAY("Retrying group list in 1s because of: %s\n",
                                 rd_kafka_err2str(err));
                        rd_sleep(1);
                }
                err = rd_kafka_list_groups(rk, NULL, &grplist,
                                           tmout_multip(5000));
        } while ((err == RD_KAFKA_RESP_ERR__TRANSPORT ||
                  err == RD_KAFKA_RESP_ERR_GROUP_LOAD_IN_PROGRESS) &&
                 retries-- > 0);

        if (err) {
                TEST_SAY("Failed to list all groups: %s\n",
                         rd_kafka_err2str(err));
                return -1;
        }

        seen_all = verify_groups(grplist, groups, group_cnt);
        rd_kafka_group_list_destroy(grplist);

        for (i = 0; i < group_cnt; i++) {
                err = rd_kafka_list_groups(rk, groups[i], &grplist, 5000);
                if (err) {
                        TEST_SAY("Failed to list group %s: %s\n", groups[i],
                                 rd_kafka_err2str(err));
                        fails++;
                        continue;
                }

                r = verify_groups(grplist, &groups[i], 1);
                if (r == 1)
                        seen++;
                rd_kafka_group_list_destroy(grplist);
        }


        if (seen_all != seen)
                return 0;

        return seen;
}



static void do_test_list_groups(void) {
        const char *topic = test_mk_topic_name(__FUNCTION__, 1);
#define _CONS_CNT 2
        char *groups[_CONS_CNT];
        rd_kafka_t *rk, *rk_c[_CONS_CNT];
        rd_kafka_topic_partition_list_t *topics;
        rd_kafka_resp_err_t err;
        test_timing_t t_grps;
        int i;
        int groups_seen;
        rd_kafka_topic_t *rkt;
        const struct rd_kafka_group_list *grplist;

        SUB_TEST();

        /* Handle for group listings */
        rk = test_create_producer();

        /* Produce messages so that topic is auto created */
        rkt = test_create_topic_object(rk, topic, NULL);
        test_produce_msgs(rk, rkt, 0, 0, 0, 10, NULL, 64);
        rd_kafka_topic_destroy(rkt);

        /* Query groups before creation, should not list our groups. */
        groups_seen = list_groups(rk, NULL, 0, "should be none");
        if (groups_seen != 0)
                TEST_FAIL(
                    "Saw %d groups when there wasn't "
                    "supposed to be any\n",
                    groups_seen);

        /* Fill in topic subscription set */
        topics = rd_kafka_topic_partition_list_new(1);
        rd_kafka_topic_partition_list_add(topics, topic, -1);

        /* Create consumers and start subscription */
        for (i = 0; i < _CONS_CNT; i++) {
                groups[i] = malloc(32);
                test_str_id_generate(groups[i], 32);
                rk_c[i] = test_create_consumer(groups[i], NULL, NULL, NULL);

                err = rd_kafka_poll_set_consumer(rk_c[i]);
                if (err)
                        TEST_FAIL("poll_set_consumer: %s\n",
                                  rd_kafka_err2str(err));

                err = rd_kafka_subscribe(rk_c[i], topics);
                if (err)
                        TEST_FAIL("subscribe: %s\n", rd_kafka_err2str(err));
        }

        rd_kafka_topic_partition_list_destroy(topics);


        TIMING_START(&t_grps, "WAIT.GROUPS");
        /* Query groups again until both groups are seen. */
        while (1) {
                groups_seen = list_groups(rk, (char **)groups, _CONS_CNT,
                                          "should see my groups");
                if (groups_seen == _CONS_CNT)
                        break;
                rd_sleep(1);
        }
        TIMING_STOP(&t_grps);

        /* Try a list_groups with a low enough timeout to fail. */
        grplist = NULL;
        TIMING_START(&t_grps, "WAIT.GROUPS.TIMEOUT0");
        err = rd_kafka_list_groups(rk, NULL, &grplist, 0);
        TIMING_STOP(&t_grps);
        TEST_SAY("list_groups(timeout=0) returned %d groups and status: %s\n",
                 grplist ? grplist->group_cnt : -1, rd_kafka_err2str(err));
        TEST_ASSERT(err == RD_KAFKA_RESP_ERR__TIMED_OUT,
                    "expected list_groups(timeout=0) to fail "
                    "with timeout, got %s",
                    rd_kafka_err2str(err));


        TEST_SAY("Closing remaining consumers\n");
        for (i = 0; i < _CONS_CNT; i++) {
                test_timing_t t_close;
                if (!rk_c[i])
                        continue;

                TEST_SAY("Closing %s\n", rd_kafka_name(rk_c[i]));
                TIMING_START(&t_close, "CONSUMER.CLOSE");
                err = rd_kafka_consumer_close(rk_c[i]);
                TIMING_STOP(&t_close);
                if (err)
                        TEST_FAIL("consumer_close failed: %s\n",
                                  rd_kafka_err2str(err));

                rd_kafka_destroy(rk_c[i]);
                rk_c[i] = NULL;

                free(groups[i]);
        }

        rd_kafka_destroy(rk);

        SUB_TEST_PASS();
}



/**
 * @brief #3705: Verify that list_groups() doesn't hang if unable to
 *        connect to the cluster.
 */
static void do_test_list_groups_hang(void) {
        rd_kafka_conf_t *conf;
        rd_kafka_t *rk;
        const struct rd_kafka_group_list *grplist;
        rd_kafka_resp_err_t err;
        test_timing_t timing;

        SUB_TEST();
        test_conf_init(&conf, NULL, 20);

        /* An unavailable broker */
        test_conf_set(conf, "bootstrap.servers", "127.0.0.1:65531");

        rk = test_create_handle(RD_KAFKA_CONSUMER, conf);

        TIMING_START(&timing, "list_groups");
        err = rd_kafka_list_groups(rk, NULL, &grplist, 5 * 1000);
        TEST_ASSERT(err == RD_KAFKA_RESP_ERR__TIMED_OUT,
                    "Expected ERR__TIMED_OUT, not %s", rd_kafka_err2name(err));
        TIMING_ASSERT(&timing, 5 * 1000, 7 * 1000);

        rd_kafka_destroy(rk);

        SUB_TEST_PASS();
}


int main_0019_list_groups(int argc, char **argv) {
        do_test_list_groups();
        do_test_list_groups_hang();
        return 0;
}