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
|
/*
* 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 */
/**
* Consume with batch + queue interface
*
*/
static void do_test_consume_batch(void) {
#define topic_cnt 2
char *topics[topic_cnt];
const int partition_cnt = 2;
rd_kafka_t *rk;
rd_kafka_queue_t *rkq;
rd_kafka_topic_t *rkts[topic_cnt];
rd_kafka_resp_err_t err;
const int msgcnt = test_quick ? 1000 : 10000;
uint64_t testid;
int i, p;
int batch_cnt = 0;
int remains;
SUB_TEST();
testid = test_id_generate();
/* Produce messages */
for (i = 0; i < topic_cnt; i++) {
topics[i] = rd_strdup(test_mk_topic_name(__FUNCTION__, 1));
for (p = 0; p < partition_cnt; p++)
test_produce_msgs_easy(topics[i], testid, p,
msgcnt / topic_cnt /
partition_cnt);
}
/* Create simple consumer */
rk = test_create_consumer(NULL, NULL, NULL, NULL);
/* Create generic consume queue */
rkq = rd_kafka_queue_new(rk);
for (i = 0; i < topic_cnt; i++) {
/* Create topic object */
rkts[i] = test_create_topic_object(
rk, topics[i], "auto.offset.reset", "smallest", NULL);
/* Start consuming each partition and redirect
* messages to queue */
TEST_SAY("Start consuming topic %s partitions 0..%d\n",
rd_kafka_topic_name(rkts[i]), partition_cnt);
for (p = 0; p < partition_cnt; p++) {
err = rd_kafka_consume_start_queue(
rkts[i], p, RD_KAFKA_OFFSET_BEGINNING, rkq);
if (err)
TEST_FAIL("Failed to start consuming: %s\n",
rd_kafka_err2str(err));
}
}
remains = msgcnt;
/* Consume messages from common queue using batch interface. */
TEST_SAY("Consume %d messages from queue\n", remains);
while (remains > 0) {
rd_kafka_message_t *rkmessage[1000];
ssize_t r;
test_timing_t t_batch;
TIMING_START(&t_batch, "CONSUME.BATCH");
r = rd_kafka_consume_batch_queue(rkq, 1000, rkmessage, 1000);
TIMING_STOP(&t_batch);
TEST_SAY("Batch consume iteration #%d: Consumed %" PRIdsz
"/1000 messages\n",
batch_cnt, r);
if (r == -1)
TEST_FAIL("Failed to consume messages: %s\n",
rd_kafka_err2str(rd_kafka_last_error()));
remains -= (int)r;
for (i = 0; i < r; i++)
rd_kafka_message_destroy(rkmessage[i]);
batch_cnt++;
}
TEST_SAY("Stopping consumer\n");
for (i = 0; i < topic_cnt; i++) {
for (p = 0; p < partition_cnt; p++) {
err = rd_kafka_consume_stop(rkts[i], p);
if (err)
TEST_FAIL("Failed to stop consuming: %s\n",
rd_kafka_err2str(err));
}
rd_kafka_topic_destroy(rkts[i]);
rd_free(topics[i]);
}
rd_kafka_queue_destroy(rkq);
rd_kafka_destroy(rk);
SUB_TEST_PASS();
}
#if WITH_SASL_OAUTHBEARER
/**
* @brief Verify that the oauthbearer_refresh_cb() is triggered
* when using consume_batch_queue() (as opposed to consumer_poll()).
*/
static rd_bool_t refresh_called = rd_false;
static void
refresh_cb(rd_kafka_t *rk, const char *oauthbearer_config, void *opaque) {
TEST_SAY("Refresh callback called\n");
TEST_ASSERT(!refresh_called);
refresh_called = rd_true;
rd_kafka_oauthbearer_set_token_failure(rk, "Refresh called");
}
static void do_test_consume_batch_oauthbearer_cb(void) {
rd_kafka_t *rk;
rd_kafka_conf_t *conf;
rd_kafka_queue_t *rkq;
rd_kafka_message_t *rkms[1];
ssize_t r;
SUB_TEST_QUICK();
refresh_called = rd_false;
conf = rd_kafka_conf_new();
test_conf_set(conf, "security.protocol", "sasl_plaintext");
test_conf_set(conf, "sasl.mechanism", "OAUTHBEARER");
rd_kafka_conf_set_oauthbearer_token_refresh_cb(conf, refresh_cb);
/* Create simple consumer */
rk = test_create_consumer(NULL, NULL, conf, NULL);
/* Create generic consume queue */
rkq = rd_kafka_queue_get_main(rk);
r = rd_kafka_consume_batch_queue(rkq, 1000, rkms, 1);
TEST_ASSERT(r == 0, "Expected return value 0, not %d", (int)r);
TEST_SAY("refresh_called = %d\n", refresh_called);
TEST_ASSERT(refresh_called,
"Expected refresh callback to have been called");
rd_kafka_queue_destroy(rkq);
rd_kafka_destroy(rk);
}
#endif
int main_0022_consume_batch(int argc, char **argv) {
do_test_consume_batch();
return 0;
}
int main_0022_consume_batch_local(int argc, char **argv) {
#if WITH_SASL_OAUTHBEARER
do_test_consume_batch_oauthbearer_cb();
#else
TEST_SKIP("No OAUTHBEARER support\n");
#endif
return 0;
}
|