Blame tests/0068-produce_timeout.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
Packit 2997f0
#if WITH_SOCKEM
Packit 2997f0
#include "rdkafka.h"
Packit 2997f0
Packit 2997f0
#include <stdarg.h>
Packit 2997f0
Packit 2997f0
/**
Packit 2997f0
 * Force produce requests to timeout to test error handling.
Packit 2997f0
 */
Packit 2997f0
Packit 2997f0
/**
Packit 2997f0
 * @brief Sockem connect, called from **internal librdkafka thread** through
Packit 2997f0
 *        librdkafka's connect_cb
Packit 2997f0
 */
Packit 2997f0
static int connect_cb (struct test *test, sockem_t *skm, const char *id) {
Packit 2997f0
Packit 2997f0
        /* Let delay be high to trigger the local timeout */
Packit 2997f0
        sockem_set(skm, "delay", 2000, NULL);
Packit 2997f0
        return 0;
Packit 2997f0
}
Packit 2997f0
Packit 2997f0
static int is_fatal_cb (rd_kafka_t *rk, rd_kafka_resp_err_t err,
Packit 2997f0
                        const char *reason) {
Packit 2997f0
        /* Ignore connectivity errors since we'll be bringing down
Packit 2997f0
         * .. connectivity.
Packit 2997f0
         * SASL auther will think a connection-down even in the auth
Packit 2997f0
         * state means the broker doesn't support SASL PLAIN. */
Packit 2997f0
        TEST_SAY("is_fatal?: %s: %s\n", rd_kafka_err2str(err), reason);
Packit 2997f0
        if (err == RD_KAFKA_RESP_ERR__TRANSPORT ||
Packit 2997f0
            err == RD_KAFKA_RESP_ERR__ALL_BROKERS_DOWN ||
Packit 2997f0
            err == RD_KAFKA_RESP_ERR__AUTHENTICATION ||
Packit 2997f0
            err == RD_KAFKA_RESP_ERR__TIMED_OUT)
Packit 2997f0
                return 0;
Packit 2997f0
        return 1;
Packit 2997f0
}
Packit 2997f0
Packit 2997f0
static int msg_dr_cnt = 0;
Packit 2997f0
static int msg_dr_fail_cnt = 0;
Packit 2997f0
Packit 2997f0
static void dr_msg_cb (rd_kafka_t *rk, const rd_kafka_message_t *rkmessage,
Packit 2997f0
                       void *opaque) {
Packit 2997f0
        msg_dr_cnt++;
Packit 2997f0
        if (rkmessage->err != RD_KAFKA_RESP_ERR__MSG_TIMED_OUT)
Packit 2997f0
                TEST_FAIL_LATER("Expected message to fail with MSG_TIMED_OUT, "
Packit 2997f0
                                "got: %s",
Packit 2997f0
                                rd_kafka_err2str(rkmessage->err));
Packit 2997f0
        else {
Packit 2997f0
                msg_dr_fail_cnt++;
Packit 2997f0
        }
Packit 2997f0
}
Packit 2997f0
Packit 2997f0
Packit 2997f0
Packit 2997f0
int main_0068_produce_timeout (int argc, char **argv) {
Packit 2997f0
        rd_kafka_t *rk;
Packit 2997f0
        const char *topic = test_mk_topic_name("0068_produce_timeout", 1);
Packit 2997f0
        uint64_t testid;
Packit 2997f0
        const int msgcnt = 10;
Packit 2997f0
        rd_kafka_conf_t *conf;
Packit 2997f0
        rd_kafka_topic_t *rkt;
Packit 2997f0
        int msgcounter = 0;
Packit 2997f0
Packit 2997f0
        testid = test_id_generate();
Packit 2997f0
Packit 2997f0
        test_conf_init(&conf, NULL, 60);
Packit 2997f0
        rd_kafka_conf_set_dr_msg_cb(conf, dr_msg_cb);
Packit 2997f0
Packit 2997f0
        test_socket_enable(conf);
Packit 2997f0
        test_curr->connect_cb = connect_cb;
Packit 2997f0
        test_curr->is_fatal_cb = is_fatal_cb;
Packit 2997f0
Packit 2997f0
        rk = test_create_handle(RD_KAFKA_PRODUCER, conf);
Packit 2997f0
        rkt = test_create_producer_topic(rk, topic,
Packit 2997f0
                                         "message.timeout.ms", "100", NULL);
Packit 2997f0
Packit 2997f0
        TEST_SAY("Auto-creating topic %s\n", topic);
Packit 2997f0
        test_auto_create_topic_rkt(rk, rkt);
Packit 2997f0
Packit 2997f0
        TEST_SAY("Producing %d messages that should timeout\n", msgcnt);
Packit 2997f0
        test_produce_msgs_nowait(rk, rkt, testid, 0, 0, msgcnt,
Packit 2997f0
                                 NULL, 0, &msgcounter);
Packit 2997f0
Packit 2997f0
Packit 2997f0
        TEST_SAY("Flushing..\n");
Packit 2997f0
        rd_kafka_flush(rk, 10000);
Packit 2997f0
Packit 2997f0
        TEST_SAY("%d/%d delivery reports, where of %d with proper error\n",
Packit 2997f0
                 msg_dr_cnt, msgcnt, msg_dr_fail_cnt);
Packit 2997f0
Packit 2997f0
        TEST_ASSERT(msg_dr_cnt == msgcnt,
Packit 2997f0
                    "expected %d, got %d", msgcnt, msg_dr_cnt);
Packit 2997f0
        TEST_ASSERT(msg_dr_fail_cnt == msgcnt,
Packit 2997f0
                    "expected %d, got %d", msgcnt, msg_dr_fail_cnt);
Packit 2997f0
Packit 2997f0
        rd_kafka_topic_destroy(rkt);
Packit 2997f0
        rd_kafka_destroy(rk);
Packit 2997f0
Packit 2997f0
        return 0;
Packit 2997f0
}
Packit 2997f0
Packit 2997f0
Packit 2997f0
#endif