static void test_producer_no_connection (void) {
	rd_kafka_t *rk;
	rd_kafka_conf_t *conf;
	rd_kafka_topic_t *rkt;
	int i;
	const int partition_cnt = 2;
	int msgcnt = 0;
	test_timing_t t_destroy;

	test_conf_init(&conf, NULL, 20);

	test_conf_set(conf, "bootstrap.servers", NULL);

	rk = test_create_handle(RD_KAFKA_PRODUCER, conf);
	rkt = test_create_topic_object(rk, __FUNCTION__,
				       "message.timeout.ms", "5000", NULL);

	test_produce_msgs_nowait(rk, rkt, 0, RD_KAFKA_PARTITION_UA, 0, 100,
				 NULL, 100, 0, &msgcnt);
	for (i = 0 ; i < partition_cnt ; i++)
		test_produce_msgs_nowait(rk, rkt, 0, i,
					 0, 100, NULL, 100, 0, &msgcnt);

	rd_kafka_poll(rk, 1000);

	TEST_SAY("%d messages in queue\n", rd_kafka_outq_len(rk));

	rd_kafka_topic_destroy(rkt);

	TIMING_START(&t_destroy, "rd_kafka_destroy()");
	rd_kafka_destroy(rk);
	TIMING_STOP(&t_destroy);
}
static void legacy_consume_many (char **topics, int topic_cnt, uint64_t testid){
	rd_kafka_t *rk;
        test_timing_t t_rkt_create;
        int i;
	rd_kafka_topic_t **rkts;
	int msg_base = 0;

	TEST_SAY(_C_MAG "%s\n" _C_CLR, __FUNCTION__);

	test_conf_init(NULL, NULL, 60);

	rk = test_create_consumer(NULL, NULL, NULL, NULL);

	TEST_SAY("Creating %d topic objects\n", topic_cnt);
		 
	rkts = malloc(sizeof(*rkts) * topic_cnt);
	TIMING_START(&t_rkt_create, "Topic object create");
	for (i = 0 ; i < topic_cnt ; i++)
		rkts[i] = test_create_topic_object(rk, topics[i], NULL);
	TIMING_STOP(&t_rkt_create);

	TEST_SAY("Start consumer for %d topics\n", topic_cnt);
	for (i = 0 ; i < topic_cnt ; i++)
		test_consumer_start("legacy", rkts[i], 0,
				    RD_KAFKA_OFFSET_BEGINNING);
	
	TEST_SAY("Consuming from %d messages from each %d topics\n",
		 msgs_per_topic, topic_cnt);
	for (i = 0 ; i < topic_cnt ; i++) {
		test_consume_msgs("legacy", rkts[i], testid, 0, TEST_NO_SEEK,
				  msg_base, msgs_per_topic, 1);
		msg_base += msgs_per_topic;
	}

	TEST_SAY("Stopping consumers\n");
	for (i = 0 ; i < topic_cnt ; i++)
		test_consumer_stop("legacy", rkts[i], 0);


	TEST_SAY("Destroying %d topic objects\n", topic_cnt);
	for (i = 0 ; i < topic_cnt ; i++)
		rd_kafka_topic_destroy(rkts[i]);

	free(rkts);

	rd_kafka_destroy(rk);
}
static void test_producer_partition_cnt_change (void) {
	rd_kafka_t *rk;
	rd_kafka_conf_t *conf;
	rd_kafka_topic_t *rkt;
	const char *topic = test_mk_topic_name(__FUNCTION__, 1);
	const int partition_cnt = 4;
	int msgcnt = 100000;
	test_timing_t t_destroy;
	int produced = 0;

	test_kafka_topics("--create --topic %s --replication-factor 1 "
			  "--partitions %d",
			  topic, partition_cnt/2);

	test_conf_init(&conf, NULL, 20);

	rk = test_create_handle(RD_KAFKA_PRODUCER, conf);
	rkt = test_create_topic_object(rk, __FUNCTION__,
				       "message.timeout.ms",
                                       tsprintf("%d", tmout_multip(5000)),
                                       NULL);

	test_produce_msgs_nowait(rk, rkt, 0, RD_KAFKA_PARTITION_UA, 0, msgcnt/2,
				 NULL, 100, &produced);

	test_kafka_topics("--alter --topic %s --partitions %d",
			  topic, partition_cnt);

	test_produce_msgs_nowait(rk, rkt, 0, RD_KAFKA_PARTITION_UA,
				 msgcnt/2, msgcnt/2,
				 NULL, 100, &produced);

	test_wait_delivery(rk, &produced);

	rd_kafka_topic_destroy(rkt);

	TIMING_START(&t_destroy, "rd_kafka_destroy()");
	rd_kafka_destroy(rk);
	TIMING_STOP(&t_destroy);
}
static void produce_many (char **topics, int topic_cnt, uint64_t testid) {
	rd_kafka_t *rk;
        test_timing_t t_rkt_create;
        int i;
	rd_kafka_topic_t **rkts;

	TEST_SAY(_C_MAG "%s\n" _C_CLR, __FUNCTION__);

	rk = test_create_producer();
	
	TEST_SAY("Creating %d topic objects\n", topic_cnt);
		 
	rkts = malloc(sizeof(*rkts) * topic_cnt);
	TIMING_START(&t_rkt_create, "Topic object create");
	for (i = 0 ; i < topic_cnt ; i++) {
		rkts[i] = test_create_topic_object(rk, topics[i], NULL);
	}
	TIMING_STOP(&t_rkt_create);

	TEST_SAY("Producing %d messages to each %d topics\n",
		 msgs_per_topic, topic_cnt);
        /* Produce messages to each topic (so they are created) */
	for (i = 0 ; i < topic_cnt ; i++) {
		test_produce_msgs(rk, rkts[i], testid, 0,
				  i * msgs_per_topic, msgs_per_topic,
				  NULL, 100);
	}

	TEST_SAY("Destroying %d topic objects\n", topic_cnt);
	for (i = 0 ; i < topic_cnt ; i++) {
		rd_kafka_topic_destroy(rkts[i]);
	}
	free(rkts);

	test_flush(rk, 30000);

	rd_kafka_destroy(rk);
}
int main_0019_list_groups (int argc, char **argv) {
	const char *topic = test_mk_topic_name(__FUNCTION__, 1);
#define _CONS_CNT 2
        char *groups[_CONS_CNT];
	rd_kafka_t *rk, *rk_c[_CONS_CNT];
	rd_kafka_topic_partition_list_t *topics;
	rd_kafka_resp_err_t err;
        test_timing_t t_grps;
	int i;
        int groups_seen;
	rd_kafka_topic_t *rkt;

        /* Handle for group listings */
        rk = test_create_producer();

	/* Produce messages so that topic is auto created */
	rkt = test_create_topic_object(rk, topic, NULL);
	test_produce_msgs(rk, rkt, 0, 0, 0, 10, NULL, 64);
	rd_kafka_topic_destroy(rkt);

        /* Query groups before creation, should not list our groups. */
        groups_seen = list_groups(rk, NULL, 0, "should be none");
        if (groups_seen != 0)
                TEST_FAIL("Saw %d groups when there wasn't "
                          "supposed to be any\n", groups_seen);

	/* Fill in topic subscription set */
	topics = rd_kafka_topic_partition_list_new(1);
	rd_kafka_topic_partition_list_add(topics, topic, -1);

	/* Create consumers and start subscription */
	for (i = 0 ; i < _CONS_CNT ; i++) {
                groups[i] = malloc(32);
                test_str_id_generate(groups[i], 32);
		rk_c[i] = test_create_consumer(groups[i],
					       NULL, NULL, NULL);

		err = rd_kafka_poll_set_consumer(rk_c[i]);
		if (err)
			TEST_FAIL("poll_set_consumer: %s\n",
				  rd_kafka_err2str(err));

		err = rd_kafka_subscribe(rk_c[i], topics);
		if (err)
			TEST_FAIL("subscribe: %s\n", rd_kafka_err2str(err));
	}

        rd_kafka_topic_partition_list_destroy(topics);


        TIMING_START(&t_grps, "WAIT.GROUPS");
        /* Query groups again until both groups are seen. */
        while (1) {
                int groups_seen = list_groups(rk, (char **)groups, _CONS_CNT,
                                              "should see my groups");
                if (groups_seen == _CONS_CNT)
                        break;
                rd_sleep(1);
        }
        TIMING_STOP(&t_grps);


	TEST_SAY("Closing remaining consumers\n");
	for (i = 0 ; i < _CONS_CNT ; i++) {
		test_timing_t t_close;
		if (!rk_c[i])
			continue;

		TEST_SAY("Closing %s\n", rd_kafka_name(rk_c[i]));
		TIMING_START(&t_close, "CONSUMER.CLOSE");
		err = rd_kafka_consumer_close(rk_c[i]);
		TIMING_STOP(&t_close);
		if (err)
			TEST_FAIL("consumer_close failed: %s\n",
				  rd_kafka_err2str(err));

		rd_kafka_destroy(rk_c[i]);
		rk_c[i] = NULL;

                free(groups[i]);
	}

        rd_kafka_destroy(rk);

        return 0;
}
int main_0031_get_offsets (int argc, char **argv) {
	const char *topic = test_mk_topic_name(__FUNCTION__, 1);
        const int msgcnt = 100;
	rd_kafka_t *rk;
	rd_kafka_topic_t *rkt;
	int64_t qry_low = -1234, qry_high = -1235;
	int64_t get_low = -1234, get_high = -1235;
	rd_kafka_resp_err_t err;
	test_timing_t t_qry, t_get;
	uint64_t testid;

        /* Produce messages */
        testid = test_produce_msgs_easy(topic, 0, 0, msgcnt);

	/* Get offsets */
	rk = test_create_consumer(NULL, NULL, NULL, NULL
);

	TIMING_START(&t_qry, "query_watermark_offsets");
	err = rd_kafka_query_watermark_offsets(rk, topic, 0,
					       &qry_low, &qry_high, 10*1000);
	TIMING_STOP(&t_qry);
	if (err)
		TEST_FAIL("query_watermark_offsets failed: %s\n",
			  rd_kafka_err2str(err));

	if (qry_low != 0 && qry_high != msgcnt)
		TEST_FAIL("Expected low,high %d,%d, but got "
			  "%"PRId64",%"PRId64,
			  0, msgcnt, qry_low, qry_high);

	TEST_SAY("query_watermark_offsets: "
		 "offsets %"PRId64", %"PRId64"\n", qry_low, qry_high);

	/* Now start consuming to update the offset cache, then query it
	 * with the get_ API. */
	rkt = test_create_topic_object(rk, topic, NULL);

	test_consumer_start("get", rkt, 0, RD_KAFKA_OFFSET_BEGINNING);
	test_consume_msgs("get", rkt, testid, 0, TEST_NO_SEEK,
			  0, msgcnt, 0);
	/* After at least one message has been consumed the
	 * watermarks are cached. */

	TIMING_START(&t_get, "get_watermark_offsets");
	err = rd_kafka_get_watermark_offsets(rk, topic, 0,
					     &get_low, &get_high);
	TIMING_STOP(&t_get);
	if (err)
		TEST_FAIL("get_watermark_offsets failed: %s\n",
			  rd_kafka_err2str(err));

	TEST_SAY("get_watermark_offsets: "
		 "offsets %"PRId64", %"PRId64"\n", get_low, get_high);

	if (get_high != qry_high)
		TEST_FAIL("query/get discrepancies: "
			  "low: %"PRId64"/%"PRId64", high: %"PRId64"/%"PRId64,
			  qry_low, get_low, qry_high, get_high);
	if (get_low >= get_high)
		TEST_FAIL("get_watermark_offsets: "
			  "low %"PRId64" >= high %"PRId64,
			  get_low, get_high);

	/* FIXME: We currently dont bother checking the get_low offset
	 *        since it requires stats to be enabled. */

	test_consumer_stop("get", rkt, 0);

	rd_kafka_topic_destroy(rkt);
	rd_kafka_destroy(rk);

        return 0;
}