Blame tests/0051-assign_adds.c

Packit 2997f0
/*
Packit 2997f0
 * librdkafka - Apache Kafka C library
Packit 2997f0
 *
Packit 2997f0
 * Copyright (c) 2012-2015, Magnus Edenhill
Packit 2997f0
 * All rights reserved.
Packit 2997f0
 *
Packit 2997f0
 * Redistribution and use in source and binary forms, with or without
Packit 2997f0
 * modification, are permitted provided that the following conditions are met:
Packit 2997f0
 *
Packit 2997f0
 * 1. Redistributions of source code must retain the above copyright notice,
Packit 2997f0
 *    this list of conditions and the following disclaimer.
Packit 2997f0
 * 2. Redistributions in binary form must reproduce the above copyright notice,
Packit 2997f0
 *    this list of conditions and the following disclaimer in the documentation
Packit 2997f0
 *    and/or other materials provided with the distribution.
Packit 2997f0
 *
Packit 2997f0
 * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
Packit 2997f0
 * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
Packit 2997f0
 * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
Packit 2997f0
 * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE
Packit 2997f0
 * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
Packit 2997f0
 * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
Packit 2997f0
 * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
Packit 2997f0
 * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
Packit 2997f0
 * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
Packit 2997f0
 * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE
Packit 2997f0
 * POSSIBILITY OF SUCH DAMAGE.
Packit 2997f0
 */
Packit 2997f0
Packit 2997f0
#include "test.h"
Packit 2997f0
#include "rdkafka.h"
Packit 2997f0
Packit 2997f0
#include <stdarg.h>
Packit 2997f0
Packit 2997f0
/**
Packit 2997f0
 * Verify that quick assignment additions work.
Packit 2997f0
 *  * Create topics T1,T2,T3
Packit 2997f0
 *  * Create consumer
Packit 2997f0
 *  * Assign T1
Packit 2997f0
 *  * Assign T1,T2
Packit 2997f0
 *  * Assign T1,T2,T3
Packit 2997f0
 *  * Verify that all messages from all three topics are consumed
Packit 2997f0
 *  * Assign T1,T3
Packit 2997f0
 *  * Verify that there were no duplicate messages.
Packit 2997f0
 */
Packit 2997f0
Packit 2997f0
int main_0051_assign_adds (int argc, char **argv) {
Packit 2997f0
        rd_kafka_t *rk;
Packit 2997f0
        #define TOPIC_CNT 3
Packit 2997f0
        char *topic[TOPIC_CNT] = {
Packit 2997f0
                rd_strdup(test_mk_topic_name("0051_assign_adds_1", 1)),
Packit 2997f0
                rd_strdup(test_mk_topic_name("0051_assign_adds_2", 1)),
Packit 2997f0
                rd_strdup(test_mk_topic_name("0051_assign_adds_3", 1)),
Packit 2997f0
        };
Packit 2997f0
        uint64_t testid;
Packit 2997f0
        int msgcnt = 1000;
Packit 2997f0
        test_msgver_t mv;
Packit 2997f0
        rd_kafka_conf_t *conf;
Packit 2997f0
        rd_kafka_topic_conf_t *tconf;
Packit 2997f0
        int i;
Packit 2997f0
        rd_kafka_topic_partition_list_t *tlist;
Packit 2997f0
        rd_kafka_resp_err_t err;
Packit 2997f0
Packit 2997f0
        msgcnt = (msgcnt / TOPIC_CNT) * TOPIC_CNT;
Packit 2997f0
        testid = test_id_generate();
Packit 2997f0
Packit 2997f0
        rk = test_create_producer();
Packit 2997f0
        for (i = 0 ; i < TOPIC_CNT ; i++) {
Packit 2997f0
                rd_kafka_topic_t *rkt;
Packit 2997f0
Packit 2997f0
                rkt = test_create_producer_topic(rk, topic[i], NULL);
Packit 2997f0
Packit 2997f0
                test_produce_msgs(rk, rkt, testid, 0,
Packit 2997f0
                                  (msgcnt / TOPIC_CNT) * i,
Packit 2997f0
                                  (msgcnt / TOPIC_CNT), NULL, 100);
Packit 2997f0
Packit 2997f0
                rd_kafka_topic_destroy(rkt);
Packit 2997f0
        }
Packit 2997f0
Packit 2997f0
        rd_kafka_destroy(rk);
Packit 2997f0
Packit 2997f0
        test_conf_init(&conf, &tconf, 60);
Packit 2997f0
        test_topic_conf_set(tconf, "auto.offset.reset", "smallest");
Packit 2997f0
Packit 2997f0
        rk = test_create_consumer(topic[0], NULL, conf, tconf);
Packit 2997f0
Packit 2997f0
        tlist = rd_kafka_topic_partition_list_new(TOPIC_CNT);
Packit 2997f0
        for (i = 0 ; i < TOPIC_CNT ; i++) {
Packit 2997f0
                rd_kafka_topic_partition_list_add(tlist, topic[i], 0);
Packit 2997f0
                TEST_SAY("Assign %d topic(s):\n", tlist->cnt);
Packit 2997f0
                test_print_partition_list(tlist);
Packit 2997f0
Packit 2997f0
                err = rd_kafka_assign(rk, tlist);
Packit 2997f0
                TEST_ASSERT(!err, "assign() failed: %s",
Packit 2997f0
                            rd_kafka_err2str(err));
Packit 2997f0
        }
Packit 2997f0
Packit 2997f0
        test_msgver_init(&mv, testid);
Packit 2997f0
Packit 2997f0
        TEST_SAY("Expecting to consume all %d messages from %d topics\n",
Packit 2997f0
                 msgcnt, TOPIC_CNT);
Packit 2997f0
Packit 2997f0
        test_consumer_poll("consume", rk, testid, -1, 0, msgcnt, &mv;;
Packit 2997f0
Packit 2997f0
        /* Now remove T2 */
Packit 2997f0
        rd_kafka_topic_partition_list_del(tlist, topic[1], 0);
Packit 2997f0
        err = rd_kafka_assign(rk, tlist);
Packit 2997f0
        TEST_ASSERT(!err, "assign() failed: %s",
Packit 2997f0
                    rd_kafka_err2str(err));
Packit 2997f0
Packit 2997f0
        TEST_SAY("Should not see any messages for session.timeout.ms+some more\n");
Packit 2997f0
        test_consumer_poll_no_msgs("consume", rk, testid, (int)(6000*1.5));
Packit 2997f0
Packit 2997f0
        test_msgver_verify("consume", &mv, TEST_MSGVER_ORDER|TEST_MSGVER_DUP,
Packit 2997f0
                           0, msgcnt);
Packit 2997f0
Packit 2997f0
        test_msgver_clear(&mv;;
Packit 2997f0
Packit 2997f0
        rd_kafka_topic_partition_list_destroy(tlist);
Packit 2997f0
Packit 2997f0
        test_consumer_close(rk);
Packit 2997f0
        rd_kafka_destroy(rk);
Packit 2997f0
Packit 2997f0
        for (i = 0 ; i < TOPIC_CNT ; i++)
Packit 2997f0
                rd_free(topic[i]);
Packit 2997f0
Packit 2997f0
        return 0;
Packit 2997f0
}