Blame tests/0013-null-msgs.c

Packit 2997f0
/*
Packit 2997f0
 * librdkafka - Apache Kafka C library
Packit 2997f0
 *
Packit 2997f0
 * Copyright (c) 2012-2013, 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
/**
Packit 2997f0
 * Produce NULL payload messages, then consume them.
Packit 2997f0
 */
Packit 2997f0
Packit 2997f0
#include "test.h"
Packit 2997f0
Packit 2997f0
/* Typical include path would be <librdkafka/rdkafka.h>, but this program
Packit 2997f0
 * is built from within the librdkafka source tree and thus differs. */
Packit 2997f0
#include "rdkafka.h"  /* for Kafka driver */
Packit 2997f0
Packit 2997f0
Packit 2997f0
static int prod_msg_remains = 0;
Packit 2997f0
static int fails = 0;
Packit 2997f0
Packit 2997f0
/**
Packit 2997f0
 * Delivery reported callback.
Packit 2997f0
 * Called for each message once to signal its delivery status.
Packit 2997f0
 */
Packit 2997f0
static void dr_cb (rd_kafka_t *rk, void *payload, size_t len,
Packit 2997f0
		   rd_kafka_resp_err_t err, void *opaque, void *msg_opaque) {
Packit 2997f0
Packit 2997f0
	if (err != RD_KAFKA_RESP_ERR_NO_ERROR)
Packit 2997f0
		TEST_FAIL("Message delivery failed: %s\n",
Packit 2997f0
			  rd_kafka_err2str(err));
Packit 2997f0
Packit 2997f0
	if (prod_msg_remains == 0)
Packit 2997f0
		TEST_FAIL("Too many messages delivered (prod_msg_remains %i)",
Packit 2997f0
			  prod_msg_remains);
Packit 2997f0
Packit 2997f0
	prod_msg_remains--;
Packit 2997f0
}
Packit 2997f0
Packit 2997f0
Packit 2997f0
/**
Packit 2997f0
 * Produces 'msgcnt' messages split over 'partition_cnt' partitions.
Packit 2997f0
 */
Packit 2997f0
static void produce_null_messages (uint64_t testid, const char *topic,
Packit 2997f0
                                   int partition_cnt, int msgcnt) {
Packit 2997f0
	int r;
Packit 2997f0
	rd_kafka_t *rk;
Packit 2997f0
	rd_kafka_topic_t *rkt;
Packit 2997f0
	rd_kafka_conf_t *conf;
Packit 2997f0
	rd_kafka_topic_conf_t *topic_conf;
Packit 2997f0
	char errstr[512];
Packit 2997f0
	int i;
Packit 2997f0
	int32_t partition;
Packit 2997f0
	int msgid = 0;
Packit 2997f0
Packit 2997f0
	test_conf_init(&conf, &topic_conf, 20);
Packit 2997f0
Packit 2997f0
	rd_kafka_conf_set_dr_cb(conf, dr_cb);
Packit 2997f0
Packit 2997f0
        /* Make sure all replicas are in-sync after producing
Packit 2997f0
         * so that consume test wont fail. */
Packit 2997f0
        rd_kafka_topic_conf_set(topic_conf, "request.required.acks", "-1",
Packit 2997f0
                                errstr, sizeof(errstr));
Packit 2997f0
Packit 2997f0
	/* Create kafka instance */
Packit 2997f0
	rk = test_create_handle(RD_KAFKA_PRODUCER, conf);
Packit 2997f0
Packit 2997f0
	rkt = rd_kafka_topic_new(rk, topic, topic_conf);
Packit 2997f0
	if (!rkt)
Packit 2997f0
		TEST_FAIL("Failed to create topic: %s\n",
Packit 2997f0
			  rd_kafka_err2str(rd_kafka_last_error()));
Packit 2997f0
Packit 2997f0
        /* Produce messages */
Packit 2997f0
	prod_msg_remains = msgcnt;
Packit 2997f0
	for (partition = 0 ; partition < partition_cnt ; partition++) {
Packit 2997f0
		int batch_cnt = msgcnt / partition_cnt;
Packit 2997f0
Packit 2997f0
		for (i = 0 ; i < batch_cnt ; i++) {
Packit 2997f0
                        char key[128];
Packit 2997f0
			rd_snprintf(key, sizeof(key),
Packit 2997f0
				 "testid=%"PRIu64", partition=%i, msg=%i",
Packit 2997f0
				 testid, (int)partition, msgid);
Packit 2997f0
                        r = rd_kafka_produce(rkt, partition, 0,
Packit 2997f0
                                             NULL, 0,
Packit 2997f0
                                             key, strlen(key),
Packit 2997f0
                                             NULL);
Packit 2997f0
                        if (r == -1)
Packit 2997f0
                                TEST_FAIL("Failed to produce message %i "
Packit 2997f0
                                          "to partition %i: %s",
Packit 2997f0
                                          msgid, (int)partition,
Packit 2997f0
                                          rd_kafka_err2str(rd_kafka_last_error()));
Packit 2997f0
			msgid++;
Packit 2997f0
		}
Packit 2997f0
        }
Packit 2997f0
Packit 2997f0
Packit 2997f0
        TEST_SAY("Produced %d messages to %d partition(s), "
Packit 2997f0
                 "waiting for deliveries\n", msgcnt, partition_cnt);
Packit 2997f0
	/* Wait for messages to be delivered */
Packit 2997f0
	while (rd_kafka_outq_len(rk) > 0)
Packit 2997f0
		rd_kafka_poll(rk, 100);
Packit 2997f0
Packit 2997f0
	if (fails)
Packit 2997f0
		TEST_FAIL("%i failures, see previous errors", fails);
Packit 2997f0
Packit 2997f0
	if (prod_msg_remains != 0)
Packit 2997f0
		TEST_FAIL("Still waiting for %i messages to be produced",
Packit 2997f0
			  prod_msg_remains);
Packit 2997f0
        else
Packit 2997f0
                TEST_SAY("All messages delivered\n");
Packit 2997f0
Packit 2997f0
	/* Destroy topic */
Packit 2997f0
	rd_kafka_topic_destroy(rkt);
Packit 2997f0
Packit 2997f0
	/* Destroy rdkafka instance */
Packit 2997f0
	TEST_SAY("Destroying kafka instance %s\n", rd_kafka_name(rk));
Packit 2997f0
	rd_kafka_destroy(rk);
Packit 2997f0
}
Packit 2997f0
Packit 2997f0
Packit 2997f0
Packit 2997f0
static int *cons_msgs;
Packit 2997f0
static int  cons_msgs_size;
Packit 2997f0
static int  cons_msgs_cnt;
Packit 2997f0
Packit 2997f0
static void verify_consumed_msg_reset (int msgcnt) {
Packit 2997f0
	if (cons_msgs) {
Packit 2997f0
		free(cons_msgs);
Packit 2997f0
		cons_msgs = NULL;
Packit 2997f0
	}
Packit 2997f0
Packit 2997f0
	if (msgcnt) {
Packit 2997f0
		int i;
Packit 2997f0
Packit 2997f0
		cons_msgs = malloc(sizeof(*cons_msgs) * msgcnt);
Packit 2997f0
		for (i = 0 ; i < msgcnt ; i++)
Packit 2997f0
			cons_msgs[i] = -1;
Packit 2997f0
	}
Packit 2997f0
Packit 2997f0
	cons_msgs_size = msgcnt;
Packit 2997f0
	cons_msgs_cnt = 0;
Packit 2997f0
}
Packit 2997f0
Packit 2997f0
Packit 2997f0
static int int_cmp (const void *_a, const void *_b) {
Packit 2997f0
	int a = *(int *)_a;
Packit 2997f0
	int b = *(int *)_b;
Packit 2997f0
	return a - b;
Packit 2997f0
}
Packit 2997f0
Packit 2997f0
static void verify_consumed_msg_check0 (const char *func, int line) {
Packit 2997f0
	int i;
Packit 2997f0
	int fails = 0;
Packit 2997f0
Packit 2997f0
	if (cons_msgs_cnt < cons_msgs_size) {
Packit 2997f0
		TEST_SAY("Missing %i messages in consumer\n",
Packit 2997f0
			 cons_msgs_size - cons_msgs_cnt);
Packit 2997f0
		fails++;
Packit 2997f0
	}
Packit 2997f0
Packit 2997f0
	qsort(cons_msgs, cons_msgs_size, sizeof(*cons_msgs), int_cmp);
Packit 2997f0
Packit 2997f0
	for (i = 0 ; i < cons_msgs_size ; i++) {
Packit 2997f0
		if (cons_msgs[i] != i) {
Packit 2997f0
			TEST_SAY("Consumed message #%i is wrong, "
Packit 2997f0
				 "expected #%i\n",
Packit 2997f0
				 cons_msgs[i], i);
Packit 2997f0
			fails++;
Packit 2997f0
		}
Packit 2997f0
	}
Packit 2997f0
Packit 2997f0
	if (fails)
Packit 2997f0
		TEST_FAIL("See above error(s)");
Packit 2997f0
Packit 2997f0
	verify_consumed_msg_reset(0);
Packit 2997f0
}
Packit 2997f0
Packit 2997f0
Packit 2997f0
#define verify_consumed_msg_check() \
Packit 2997f0
	verify_consumed_msg_check0(__FUNCTION__,__LINE__)
Packit 2997f0
Packit 2997f0
Packit 2997f0
Packit 2997f0
static void verify_consumed_msg0 (const char *func, int line,
Packit 2997f0
				  uint64_t testid, int32_t partition,
Packit 2997f0
				  int msgnum,
Packit 2997f0
				  rd_kafka_message_t *rkmessage) {
Packit 2997f0
	uint64_t in_testid;
Packit 2997f0
	int in_part;
Packit 2997f0
	int in_msgnum;
Packit 2997f0
	char buf[128];
Packit 2997f0
Packit 2997f0
        if (rkmessage->len != 0)
Packit 2997f0
                TEST_FAIL("Incoming message not NULL: %i bytes",
Packit 2997f0
                          (int)rkmessage->len);
Packit 2997f0
Packit 2997f0
	if (rkmessage->key_len +1 >= sizeof(buf))
Packit 2997f0
		TEST_FAIL("Incoming message key too large (%i): "
Packit 2997f0
			  "not sourced by this test",
Packit 2997f0
			  (int)rkmessage->key_len);
Packit 2997f0
Packit 2997f0
	rd_snprintf(buf, sizeof(buf), "%.*s",
Packit 2997f0
		 (int)rkmessage->key_len, (char *)rkmessage->key);
Packit 2997f0
Packit 2997f0
	if (sscanf(buf, "testid=%"SCNu64", partition=%i, msg=%i",
Packit 2997f0
		   &in_testid, &in_part, &in_msgnum) != 3)
Packit 2997f0
		TEST_FAIL("Incorrect key format: %s", buf);
Packit 2997f0
Packit 2997f0
	if (testid != in_testid ||
Packit 2997f0
	    (partition != -1 && partition != in_part) ||
Packit 2997f0
	    (msgnum != -1 && msgnum != in_msgnum) ||
Packit 2997f0
	    (in_msgnum < 0 || in_msgnum > cons_msgs_size))
Packit 2997f0
		goto fail_match;
Packit 2997f0
Packit 2997f0
	if (test_level > 2) {
Packit 2997f0
		TEST_SAY("%s:%i: Our testid %"PRIu64", part %i (%i), "
Packit 2997f0
			 "msg %i/%i did "
Packit 2997f0
			 ", key's: \"%s\"\n",
Packit 2997f0
			 func, line,
Packit 2997f0
			 testid, (int)partition, (int)rkmessage->partition,
Packit 2997f0
			 msgnum, cons_msgs_size, buf);
Packit 2997f0
	}
Packit 2997f0
Packit 2997f0
	if (cons_msgs_cnt == cons_msgs_size) {
Packit 2997f0
		TEST_SAY("Too many messages in cons_msgs (%i) while reading "
Packit 2997f0
			 "message key \"%s\"\n",
Packit 2997f0
			 cons_msgs_cnt, buf);
Packit 2997f0
		verify_consumed_msg_check();
Packit 2997f0
		TEST_FAIL("See above error(s)");
Packit 2997f0
	}
Packit 2997f0
Packit 2997f0
	cons_msgs[cons_msgs_cnt++] = in_msgnum;
Packit 2997f0
Packit 2997f0
	return;
Packit 2997f0
Packit 2997f0
 fail_match:
Packit 2997f0
	TEST_FAIL("%s:%i: Our testid %"PRIu64", part %i, msg %i/%i did "
Packit 2997f0
		  "not match message's key: \"%s\"\n",
Packit 2997f0
		  func, line,
Packit 2997f0
		  testid, (int)partition, msgnum, cons_msgs_size, buf);
Packit 2997f0
}
Packit 2997f0
Packit 2997f0
#define verify_consumed_msg(testid,part,msgnum,rkmessage) \
Packit 2997f0
	verify_consumed_msg0(__FUNCTION__,__LINE__,testid,part,msgnum,rkmessage)
Packit 2997f0
Packit 2997f0
Packit 2997f0
static void consume_messages (uint64_t testid, const char *topic,
Packit 2997f0
			      int32_t partition, int msg_base, int batch_cnt,
Packit 2997f0
			      int msgcnt) {
Packit 2997f0
	rd_kafka_t *rk;
Packit 2997f0
	rd_kafka_topic_t *rkt;
Packit 2997f0
	rd_kafka_conf_t *conf;
Packit 2997f0
	rd_kafka_topic_conf_t *topic_conf;
Packit 2997f0
	int i;
Packit 2997f0
Packit 2997f0
	test_conf_init(&conf, &topic_conf, 20);
Packit 2997f0
Packit 2997f0
	/* Create kafka instance */
Packit 2997f0
	rk = test_create_handle(RD_KAFKA_CONSUMER, conf);
Packit 2997f0
Packit 2997f0
	rkt = rd_kafka_topic_new(rk, topic, topic_conf);
Packit 2997f0
	if (!rkt)
Packit 2997f0
		TEST_FAIL("Failed to create topic: %s\n",
Packit 2997f0
                          rd_kafka_err2str(rd_kafka_last_error()));
Packit 2997f0
Packit 2997f0
	TEST_SAY("Consuming %i messages from partition %i\n",
Packit 2997f0
		 batch_cnt, partition);
Packit 2997f0
Packit 2997f0
	/* Consume messages */
Packit 2997f0
	if (rd_kafka_consume_start(rkt, partition,
Packit 2997f0
			     RD_KAFKA_OFFSET_TAIL(batch_cnt)) == -1)
Packit 2997f0
		TEST_FAIL("consume_start(%i, -%i) failed: %s",
Packit 2997f0
			  (int)partition, batch_cnt,
Packit 2997f0
			  rd_kafka_err2str(rd_kafka_last_error()));
Packit 2997f0
Packit 2997f0
	for (i = 0 ; i < batch_cnt ; i++) {
Packit 2997f0
		rd_kafka_message_t *rkmessage;
Packit 2997f0
Packit 2997f0
		rkmessage = rd_kafka_consume(rkt, partition, tmout_multip(5000));
Packit 2997f0
		if (!rkmessage)
Packit 2997f0
			TEST_FAIL("Failed to consume message %i/%i from "
Packit 2997f0
				  "partition %i: %s",
Packit 2997f0
				  i, batch_cnt, (int)partition,
Packit 2997f0
				  rd_kafka_err2str(rd_kafka_last_error()));
Packit 2997f0
		if (rkmessage->err)
Packit 2997f0
			TEST_FAIL("Consume message %i/%i from partition %i "
Packit 2997f0
				  "has error: %s",
Packit 2997f0
				  i, batch_cnt, (int)partition,
Packit 2997f0
				  rd_kafka_err2str(rkmessage->err));
Packit 2997f0
Packit 2997f0
		verify_consumed_msg(testid, partition, msg_base+i, rkmessage);
Packit 2997f0
Packit 2997f0
		rd_kafka_message_destroy(rkmessage);
Packit 2997f0
	}
Packit 2997f0
Packit 2997f0
	rd_kafka_consume_stop(rkt, partition);
Packit 2997f0
Packit 2997f0
	/* Destroy topic */
Packit 2997f0
	rd_kafka_topic_destroy(rkt);
Packit 2997f0
Packit 2997f0
	/* Destroy rdkafka instance */
Packit 2997f0
	TEST_SAY("Destroying kafka instance %s\n", rd_kafka_name(rk));
Packit 2997f0
	rd_kafka_destroy(rk);
Packit 2997f0
}
Packit 2997f0
Packit 2997f0
Packit 2997f0
static void consume_messages_with_queues (uint64_t testid, const char *topic,
Packit 2997f0
					  int partition_cnt, int msgcnt) {
Packit 2997f0
	rd_kafka_t *rk;
Packit 2997f0
	rd_kafka_topic_t *rkt;
Packit 2997f0
	rd_kafka_conf_t *conf;
Packit 2997f0
	rd_kafka_topic_conf_t *topic_conf;
Packit 2997f0
	rd_kafka_queue_t *rkqu;
Packit 2997f0
	int i;
Packit 2997f0
	int32_t partition;
Packit 2997f0
	int batch_cnt = msgcnt / partition_cnt;
Packit 2997f0
Packit 2997f0
	test_conf_init(&conf, &topic_conf, 20);
Packit 2997f0
Packit 2997f0
	/* Create kafka instance */
Packit 2997f0
	rk = test_create_handle(RD_KAFKA_CONSUMER, conf);
Packit 2997f0
Packit 2997f0
	/* Create queue */
Packit 2997f0
	rkqu = rd_kafka_queue_new(rk);
Packit 2997f0
Packit 2997f0
Packit 2997f0
	rkt = rd_kafka_topic_new(rk, topic, topic_conf);
Packit 2997f0
        if (!rkt)
Packit 2997f0
                TEST_FAIL("Failed to create topic: %s\n",
Packit 2997f0
                          rd_kafka_err2str(rd_kafka_last_error()));
Packit 2997f0
Packit 2997f0
	TEST_SAY("Consuming %i messages from one queue serving %i partitions\n",
Packit 2997f0
		 msgcnt, partition_cnt);
Packit 2997f0
Packit 2997f0
	/* Start consuming each partition */
Packit 2997f0
	for (partition = 0 ; partition < partition_cnt ; partition++) {
Packit 2997f0
		/* Consume messages */
Packit 2997f0
		TEST_SAY("Start consuming partition %i at tail offset -%i\n",
Packit 2997f0
			 partition, batch_cnt);
Packit 2997f0
		if (rd_kafka_consume_start_queue(rkt, partition,
Packit 2997f0
						 RD_KAFKA_OFFSET_TAIL(batch_cnt),
Packit 2997f0
						 rkqu) == -1)
Packit 2997f0
			TEST_FAIL("consume_start_queue(%i) failed: %s",
Packit 2997f0
				  (int)partition,
Packit 2997f0
				  rd_kafka_err2str(rd_kafka_last_error()));
Packit 2997f0
	}
Packit 2997f0
Packit 2997f0
Packit 2997f0
	/* Consume messages from queue */
Packit 2997f0
	for (i = 0 ; i < msgcnt ; i++) {
Packit 2997f0
		rd_kafka_message_t *rkmessage;
Packit 2997f0
Packit 2997f0
		rkmessage = rd_kafka_consume_queue(rkqu, tmout_multip(5000));
Packit 2997f0
		if (!rkmessage)
Packit 2997f0
			TEST_FAIL("Failed to consume message %i/%i from "
Packit 2997f0
				  "queue: %s",
Packit 2997f0
				  i, msgcnt,
Packit 2997f0
				  rd_kafka_err2str(rd_kafka_last_error()));
Packit 2997f0
		if (rkmessage->err)
Packit 2997f0
			TEST_FAIL("Consume message %i/%i from queue "
Packit 2997f0
				  "has error (partition %"PRId32"): %s",
Packit 2997f0
				  i, msgcnt,
Packit 2997f0
				  rkmessage->partition,
Packit 2997f0
				  rd_kafka_err2str(rkmessage->err));
Packit 2997f0
Packit 2997f0
		verify_consumed_msg(testid, -1, -1, rkmessage);
Packit 2997f0
Packit 2997f0
		rd_kafka_message_destroy(rkmessage);
Packit 2997f0
	}
Packit 2997f0
Packit 2997f0
	/* Stop consuming each partition */
Packit 2997f0
	for (partition = 0 ; partition < partition_cnt ; partition++)
Packit 2997f0
		rd_kafka_consume_stop(rkt, partition);
Packit 2997f0
Packit 2997f0
	/* Destroy queue */
Packit 2997f0
	rd_kafka_queue_destroy(rkqu);
Packit 2997f0
Packit 2997f0
	/* Destroy topic */
Packit 2997f0
	rd_kafka_topic_destroy(rkt);
Packit 2997f0
Packit 2997f0
	/* Destroy rdkafka instance */
Packit 2997f0
	TEST_SAY("Destroying kafka instance %s\n", rd_kafka_name(rk));
Packit 2997f0
	rd_kafka_destroy(rk);
Packit 2997f0
}
Packit 2997f0
Packit 2997f0
Packit 2997f0
static void test_produce_consume (void) {
Packit 2997f0
	int msgcnt = 1000;
Packit 2997f0
        int partition_cnt = 1;
Packit 2997f0
	int i;
Packit 2997f0
	uint64_t testid;
Packit 2997f0
	int msg_base = 0;
Packit 2997f0
        const char *topic;
Packit 2997f0
Packit 2997f0
	/* Generate a testid so we can differentiate messages
Packit 2997f0
	 * from other tests */
Packit 2997f0
	testid = test_id_generate();
Packit 2997f0
Packit 2997f0
        /* Read test.conf to configure topic name */
Packit 2997f0
        test_conf_init(NULL, NULL, 20);
Packit 2997f0
        topic = test_mk_topic_name("0013", 0);
Packit 2997f0
Packit 2997f0
	TEST_SAY("Topic %s, testid %"PRIu64"\n", topic, testid);
Packit 2997f0
Packit 2997f0
	/* Produce messages */
Packit 2997f0
	produce_null_messages(testid, topic, partition_cnt, msgcnt);
Packit 2997f0
Packit 2997f0
Packit 2997f0
	/* Consume messages with standard interface */
Packit 2997f0
	verify_consumed_msg_reset(msgcnt);
Packit 2997f0
	for (i = 0 ; i < partition_cnt ; i++) {
Packit 2997f0
		consume_messages(testid, topic, i,
Packit 2997f0
				 msg_base, msgcnt / partition_cnt, msgcnt);
Packit 2997f0
		msg_base += msgcnt / partition_cnt;
Packit 2997f0
	}
Packit 2997f0
	verify_consumed_msg_check();
Packit 2997f0
Packit 2997f0
	/* Consume messages with queue interface */
Packit 2997f0
	verify_consumed_msg_reset(msgcnt);
Packit 2997f0
	consume_messages_with_queues(testid, topic, partition_cnt, msgcnt);
Packit 2997f0
	verify_consumed_msg_check();
Packit 2997f0
Packit 2997f0
	return;
Packit 2997f0
}
Packit 2997f0
Packit 2997f0
Packit 2997f0
Packit 2997f0
Packit 2997f0
int main_0013_null_msgs (int argc, char **argv) {
Packit 2997f0
	test_produce_consume();
Packit 2997f0
	return 0;
Packit 2997f0
}