forked from confluentinc/librdkafka
-
Notifications
You must be signed in to change notification settings - Fork 0
/
0034-offset_reset.c
137 lines (113 loc) · 4.39 KB
/
0034-offset_reset.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
/*
* librdkafka - Apache Kafka C library
*
* Copyright (c) 2016, 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 */
/**
* Issue #559: make sure auto.offset.reset works with invalid offsets.
*/
static void do_test_reset (const char *topic, int partition,
const char *reset, int64_t initial_offset,
int exp_eofcnt, int exp_msgcnt, int exp_errcnt) {
rd_kafka_t *rk;
rd_kafka_topic_t *rkt;
int eofcnt = 0, msgcnt = 0, errcnt = 0;
rd_kafka_conf_t *conf;
TEST_SAY("Test auto.offset.reset=%s, "
"expect %d msgs, %d EOFs, %d errors\n",
reset, exp_msgcnt, exp_eofcnt, exp_errcnt);
test_conf_init(&conf, NULL, 60);
test_conf_set(conf, "enable.partition.eof", "true");
rk = test_create_consumer(NULL, NULL, conf, NULL);
rkt = test_create_topic_object(rk, topic, "auto.offset.reset", reset,
NULL);
test_consumer_start(reset, rkt, partition, initial_offset);
while (1) {
rd_kafka_message_t *rkm;
rkm = rd_kafka_consume(rkt, partition, tmout_multip(1000*10));
if (!rkm)
TEST_FAIL("%s: no message for 10s: "
"%d/%d messages, %d/%d EOFs, %d/%d errors\n",
reset, msgcnt, exp_msgcnt,
eofcnt, exp_eofcnt,
errcnt, exp_errcnt);
if (rkm->err == RD_KAFKA_RESP_ERR__PARTITION_EOF) {
TEST_SAY("%s: received EOF at offset %"PRId64"\n",
reset, rkm->offset);
eofcnt++;
} else if (rkm->err) {
TEST_SAY("%s: consume error at offset %"PRId64": %s\n",
reset, rkm->offset,
rd_kafka_message_errstr(rkm));
errcnt++;
} else {
msgcnt++;
}
rd_kafka_message_destroy(rkm);
if (eofcnt == exp_eofcnt &&
errcnt == exp_errcnt &&
msgcnt == exp_msgcnt)
break;
else if (eofcnt > exp_eofcnt ||
errcnt > exp_errcnt ||
msgcnt > exp_msgcnt)
TEST_FAIL("%s: unexpected: "
"%d/%d messages, %d/%d EOFs, %d/%d errors\n",
reset,
msgcnt, exp_msgcnt,
eofcnt, exp_eofcnt,
errcnt, exp_errcnt);
}
TEST_SAY("%s: Done: "
"%d/%d messages, %d/%d EOFs, %d/%d errors\n",
reset,
msgcnt, exp_msgcnt,
eofcnt, exp_eofcnt,
errcnt, exp_errcnt);
test_consumer_stop(reset, rkt, partition);
rd_kafka_topic_destroy(rkt);
rd_kafka_destroy(rk);
}
int main_0034_offset_reset (int argc, char **argv) {
const char *topic = test_mk_topic_name(__FUNCTION__, 1);
const int partition = 0;
const int msgcnt = test_quick ? 20 : 100;
/* Produce messages */
test_produce_msgs_easy(topic, 0, partition, msgcnt);
/* auto.offset.reset=latest: Consume messages from invalid offset:
* Should return EOF. */
do_test_reset(topic, partition, "latest", msgcnt+5, 1, 0, 0);
/* auto.offset.reset=earliest: Consume messages from invalid offset:
* Should return messages from beginning. */
do_test_reset(topic, partition, "earliest", msgcnt+5, 1, msgcnt, 0);
/* auto.offset.reset=error: Consume messages from invalid offset:
* Should return error. */
do_test_reset(topic, partition, "error", msgcnt+5, 0, 0, 1);
return 0;
}