1 /*
2  * librdkafka - Apache Kafka C library
3  *
4  * Copyright (c) 2012-2015, Magnus Edenhill
5  * All rights reserved.
6  *
7  * Redistribution and use in source and binary forms, with or without
8  * modification, are permitted provided that the following conditions are met:
9  *
10  * 1. Redistributions of source code must retain the above copyright notice,
11  *    this list of conditions and the following disclaimer.
12  * 2. Redistributions in binary form must reproduce the above copyright notice,
13  *    this list of conditions and the following disclaimer in the documentation
14  *    and/or other materials provided with the distribution.
15  *
16  * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
17  * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
18  * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
19  * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE
20  * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
21  * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
22  * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
23  * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
24  * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
25  * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE
26  * POSSIBILITY OF SUCH DAMAGE.
27  */
28 
29 #include "test.h"
30 #include "rdkafka.h"
31 
32 #include <stdarg.h>
33 
34 /**
35  * Verify that quick assignment additions work.
36  *  * Create topics T1,T2,T3
37  *  * Create consumer
38  *  * Assign T1
39  *  * Assign T1,T2
40  *  * Assign T1,T2,T3
41  *  * Verify that all messages from all three topics are consumed
42  *  * Assign T1,T3
43  *  * Verify that there were no duplicate messages.
44  */
45 
main_0051_assign_adds(int argc,char ** argv)46 int main_0051_assign_adds (int argc, char **argv) {
47         rd_kafka_t *rk;
48         #define TOPIC_CNT 3
49         char *topic[TOPIC_CNT] = {
50                 rd_strdup(test_mk_topic_name("0051_assign_adds_1", 1)),
51                 rd_strdup(test_mk_topic_name("0051_assign_adds_2", 1)),
52                 rd_strdup(test_mk_topic_name("0051_assign_adds_3", 1)),
53         };
54         uint64_t testid;
55         int msgcnt = test_quick ? 100 : 1000;
56         test_msgver_t mv;
57         rd_kafka_conf_t *conf;
58         rd_kafka_topic_conf_t *tconf;
59         int i;
60         rd_kafka_topic_partition_list_t *tlist;
61         rd_kafka_resp_err_t err;
62 
63         msgcnt = (msgcnt / TOPIC_CNT) * TOPIC_CNT;
64         testid = test_id_generate();
65 
66         rk = test_create_producer();
67         for (i = 0 ; i < TOPIC_CNT ; i++) {
68                 rd_kafka_topic_t *rkt;
69 
70                 rkt = test_create_producer_topic(rk, topic[i], NULL);
71 
72                 test_produce_msgs(rk, rkt, testid, 0,
73                                   (msgcnt / TOPIC_CNT) * i,
74                                   (msgcnt / TOPIC_CNT), NULL, 100);
75 
76                 rd_kafka_topic_destroy(rkt);
77         }
78 
79         rd_kafka_destroy(rk);
80 
81         test_conf_init(&conf, &tconf, 60);
82         test_topic_conf_set(tconf, "auto.offset.reset", "smallest");
83 
84         rk = test_create_consumer(topic[0], NULL, conf, tconf);
85 
86         tlist = rd_kafka_topic_partition_list_new(TOPIC_CNT);
87         for (i = 0 ; i < TOPIC_CNT ; i++) {
88                 rd_kafka_topic_partition_list_add(tlist, topic[i], 0);
89                 TEST_SAY("Assign %d topic(s):\n", tlist->cnt);
90                 test_print_partition_list(tlist);
91 
92                 err = rd_kafka_assign(rk, tlist);
93                 TEST_ASSERT(!err, "assign() failed: %s",
94                             rd_kafka_err2str(err));
95         }
96 
97         test_msgver_init(&mv, testid);
98 
99         TEST_SAY("Expecting to consume all %d messages from %d topics\n",
100                  msgcnt, TOPIC_CNT);
101 
102         test_consumer_poll("consume", rk, testid, -1, 0, msgcnt, &mv);
103 
104         /* Now remove T2 */
105         rd_kafka_topic_partition_list_del(tlist, topic[1], 0);
106         err = rd_kafka_assign(rk, tlist);
107         TEST_ASSERT(!err, "assign() failed: %s",
108                     rd_kafka_err2str(err));
109 
110         TEST_SAY("Should not see any messages for session.timeout.ms+some more\n");
111         test_consumer_poll_no_msgs("consume", rk, testid, (int)(6000*1.5));
112 
113         test_msgver_verify("consume", &mv, TEST_MSGVER_ORDER|TEST_MSGVER_DUP,
114                            0, msgcnt);
115 
116         test_msgver_clear(&mv);
117 
118         rd_kafka_topic_partition_list_destroy(tlist);
119 
120         test_consumer_close(rk);
121         rd_kafka_destroy(rk);
122 
123         for (i = 0 ; i < TOPIC_CNT ; i++)
124                 rd_free(topic[i]);
125 
126         return 0;
127 }
128