static void do_nonexist_commit (void) { rd_kafka_t *rk; char group_id[64]; rd_kafka_conf_t *conf; rd_kafka_topic_conf_t *tconf; rd_kafka_topic_partition_list_t *offsets; const char *unk_topic = test_mk_topic_name(__FUNCTION__, 1); rd_kafka_resp_err_t err; test_conf_init(&conf, &tconf, 20); test_str_id_generate(group_id, sizeof(group_id)); TEST_SAY(_C_MAG "[ do_nonexist_commit group.id %s ]\n", group_id); rk = test_create_consumer(group_id, NULL, conf, tconf, NULL); TEST_SAY("Try nonexist commit\n"); offsets = rd_kafka_topic_partition_list_new(2); rd_kafka_topic_partition_list_add(offsets, unk_topic, 0)->offset = 123; rd_kafka_topic_partition_list_add(offsets, unk_topic, 1)->offset = 456; err = rd_kafka_commit_queue(rk, offsets, NULL, nonexist_offset_commit_cb, NULL); TEST_SAY("nonexist commit returned %s\n", rd_kafka_err2str(err)); if (err != RD_KAFKA_RESP_ERR_NO_ERROR) TEST_FAIL("commit() should succeed, not: %s", rd_kafka_err2str(err)); rd_kafka_topic_partition_list_destroy(offsets); test_consumer_close(rk); rd_kafka_destroy(rk); }
static void assign_consume_many (char **topics, int topic_cnt, uint64_t testid){ rd_kafka_t *rk; rd_kafka_topic_partition_list_t *parts; int i; test_msgver_t mv; TEST_SAY(_C_MAG "%s\n" _C_CLR, __FUNCTION__); test_conf_init(NULL, NULL, 60); rk = test_create_consumer(__FUNCTION__, NULL, NULL, NULL); parts = rd_kafka_topic_partition_list_new(topic_cnt); for (i = 0 ; i < topic_cnt ; i++) rd_kafka_topic_partition_list_add(parts, topics[i], 0)-> offset = RD_KAFKA_OFFSET_TAIL(msgs_per_topic); test_consumer_assign("consume.assign", rk, parts); rd_kafka_topic_partition_list_destroy(parts); test_msgver_init(&mv, testid); test_consumer_poll("consume.assign", rk, testid, -1, 0, msgs_per_topic * topic_cnt, &mv); for (i = 0 ; i < topic_cnt ; i++) test_msgver_verify_part("assign", &mv, TEST_MSGVER_ALL_PART, topics[i], 0, i * msgs_per_topic, msgs_per_topic); test_msgver_clear(&mv); test_consumer_close(rk); rd_kafka_destroy(rk); }
int main_0028_long_topicnames (int argc, char **argv) { const int msgcnt = 1000; uint64_t testid; char topic[256]; rd_kafka_t *rk_c; memset(topic, 'a', sizeof(topic)-1); topic[sizeof(topic)-1] = '\0'; strncpy(topic, test_mk_topic_name(topic, 1), sizeof(topic)-1); TEST_SAY("Using topic name of %d bytes: %s\n", (int)strlen(topic), topic); /* Create topic */ test_create_topic(topic, 1, 1); /* First try a non-verifying consumer. The consumer has been known * to crash when the broker bug kicks in. */ rk_c = test_create_consumer(topic, NULL, NULL, NULL, NULL); test_consumer_subscribe(rk_c, topic); test_consumer_poll_no_msgs("consume.nomsgs", rk_c, 0, 5000); test_consumer_close(rk_c); /* Produce messages */ testid = test_produce_msgs_easy(topic, 0, RD_KAFKA_PARTITION_UA, msgcnt); /* Consume messages */ test_consume_msgs_easy(NULL, topic, testid, msgcnt); return 0; }
static void do_test_non_exist_and_partchange (void) { char *topic_a = rd_strdup(test_mk_topic_name("topic_a", 1)); rd_kafka_t *rk; rd_kafka_conf_t *conf; rd_kafka_queue_t *queue; /** * Test #1: * - Subscribe to non-existing topic. * - Verify empty assignment * - Create topic * - Verify new assignment containing topic */ TEST_SAY("#1 & #2 testing\n"); test_conf_init(&conf, NULL, 60); /* Decrease metadata interval to speed up topic change discovery. */ test_conf_set(conf, "metadata.max.age.ms", "5000"); rd_kafka_conf_set_events(conf, RD_KAFKA_EVENT_REBALANCE); rk = test_create_consumer(test_str_id_generate_tmp(), NULL, conf, NULL, NULL); queue = rd_kafka_queue_get_consumer(rk); TEST_SAY("#1: Subscribing to %s\n", topic_a); test_consumer_subscribe(rk, topic_a); /* Should not see a rebalance since no topics are matched. */ await_no_rebalance("#1: empty", rk, queue, 10000); TEST_SAY("#1: creating topic %s\n", topic_a); test_create_topic(topic_a, 2, 1); await_assignment("#1: proper", rk, queue, 1, topic_a, 2); /** * Test #2 (continue with #1 consumer) * - Increase the partition count * - Verify updated assignment */ test_kafka_topics("--alter --topic %s --partitions 4", topic_a); await_revoke("#2", rk, queue); await_assignment("#2: more partitions", rk, queue, 1, topic_a, 4); test_consumer_close(rk); rd_kafka_queue_destroy(queue); rd_kafka_destroy(rk); rd_free(topic_a); }
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); }
/** * Trigger an empty cgrp commit (issue #803) */ static void do_empty_commit (void) { rd_kafka_t *rk; char group_id[64]; rd_kafka_conf_t *conf; rd_kafka_topic_conf_t *tconf; rd_kafka_resp_err_t err, expect; test_conf_init(&conf, &tconf, 20); test_conf_set(conf, "enable.auto.commit", "false"); test_topic_conf_set(tconf, "auto.offset.reset", "earliest"); test_str_id_generate(group_id, sizeof(group_id)); TEST_SAY(_C_MAG "[ do_empty_commit group.id %s ]\n", group_id); rk = test_create_consumer(group_id, NULL, conf, tconf, NULL); test_consumer_subscribe(rk, topic); test_consumer_poll("consume", rk, testid, -1, -1, 100, NULL); TEST_SAY("First commit\n"); expect = RD_KAFKA_RESP_ERR_NO_ERROR; err = rd_kafka_commit_queue(rk, NULL, NULL, empty_offset_commit_cb, &expect); if (err != expect) TEST_FAIL("commit failed: %s", rd_kafka_err2str(err)); else TEST_SAY("First commit returned %s\n", rd_kafka_err2str(err)); TEST_SAY("Second commit, should be empty\n"); expect = RD_KAFKA_RESP_ERR__NO_OFFSET; err = rd_kafka_commit_queue(rk, NULL, NULL, empty_offset_commit_cb, &expect); if (err != RD_KAFKA_RESP_ERR__NO_OFFSET) TEST_FAIL("unexpected commit result, wanted NO_OFFSET, got: %s", rd_kafka_err2str(err)); else TEST_SAY("Second commit returned %s\n", rd_kafka_err2str(err)); test_consumer_close(rk); rd_kafka_destroy(rk); }
static void do_test_consumer (const char *topic) { rd_kafka_conf_t *conf; int i; rd_kafka_t *rk; TEST_SAY(_C_MAG "[ %s ]\n" _C_CLR, __FUNCTION__); test_conf_init(&conf, NULL, 0); rd_kafka_conf_interceptor_add_on_new(conf, "on_new_consumer", on_new_consumer, NULL); test_conf_set(conf, "auto.offset.reset", "earliest"); /* Create producer */ rk = test_create_consumer(topic, NULL, conf, NULL); test_consumer_subscribe(rk, topic); /* Consume messages (-1 for the one that failed producing) */ test_consumer_poll("interceptors.consume", rk, 0, -1, -1, msgcnt-1, NULL); /* Verify on_consume */ for (i = 0 ; i < msgcnt-1 ; i++) { struct msg_state *msg = &msgs[i]; msg_verify_ic_cnt(msg, "on_consume", msg->bits[_ON_CONSUME], consumer_ic_cnt); } /* Verify that the produce-failed message didnt have * interceptors called */ msg_verify_ic_cnt(&msgs[msgcnt-1], "on_consume", msgs[msgcnt-1].bits[_ON_CONSUME], 0); test_consumer_close(rk); verify_ic_cnt("on_commit", on_commit_bits, consumer_ic_cnt); rd_kafka_destroy(rk); }
int main_0036_partial_fetch (int argc, char **argv) { const char *topic = test_mk_topic_name(__FUNCTION__, 1); const int partition = 0; const int msgcnt = 100; const int msgsize = 1000; uint64_t testid; rd_kafka_conf_t *conf; rd_kafka_t *rk; rd_kafka_topic_t *rkt; TEST_SAY("Producing %d messages of size %d to %s [%d]\n", msgcnt, (int)msgsize, topic, partition); testid = test_id_generate(); rk = test_create_producer(); rkt = test_create_producer_topic(rk, topic, NULL); test_produce_msgs(rk, rkt, testid, partition, 0, msgcnt, NULL, msgsize); rd_kafka_topic_destroy(rkt); rd_kafka_destroy(rk); TEST_SAY("Creating consumer\n"); test_conf_init(&conf, NULL, 0); /* This should fetch 1.5 messages per fetch, thus resulting in * partial fetches, hopefully. */ test_conf_set(conf, "fetch.message.max.bytes", "1500"); rk = test_create_consumer(NULL, NULL, conf, NULL, NULL); rkt = rd_kafka_topic_new(rk, topic, NULL); test_consumer_start("CONSUME", rkt, partition, RD_KAFKA_OFFSET_BEGINNING); test_consume_msgs("CONSUME", rkt, testid, partition, TEST_NO_SEEK, 0, msgcnt, 1); test_consumer_stop("CONSUME", rkt, partition); rd_kafka_topic_destroy(rkt); rd_kafka_destroy(rk); return 0; }
static void subscribe_consume_many (char **topics, int topic_cnt, uint64_t testid) { rd_kafka_t *rk; int i; rd_kafka_topic_conf_t *tconf; rd_kafka_topic_partition_list_t *parts; rd_kafka_resp_err_t err; test_msgver_t mv; TEST_SAY(_C_MAG "%s\n" _C_CLR, __FUNCTION__); test_conf_init(NULL, &tconf, 60); test_topic_conf_set(tconf, "auto.offset.reset", "earliest"); rk = test_create_consumer(__FUNCTION__, NULL, NULL, tconf); parts = rd_kafka_topic_partition_list_new(topic_cnt); for (i = 0 ; i < topic_cnt ; i++) rd_kafka_topic_partition_list_add(parts, topics[i], RD_KAFKA_PARTITION_UA); TEST_SAY("Subscribing to %d topics\n", topic_cnt); err = rd_kafka_subscribe(rk, parts); if (err) TEST_FAIL("subscribe() failed: %s\n", rd_kafka_err2str(err)); rd_kafka_topic_partition_list_destroy(parts); test_msgver_init(&mv, testid); test_consumer_poll("consume.subscribe", rk, testid, -1, 0, msgs_per_topic * topic_cnt, &mv); for (i = 0 ; i < topic_cnt ; i++) test_msgver_verify_part("subscribe", &mv, TEST_MSGVER_ALL_PART, topics[i], 0, i * msgs_per_topic, msgs_per_topic); test_msgver_clear(&mv); test_consumer_close(rk); rd_kafka_destroy(rk); }
int main_0041_fetch_max_bytes (int argc, char **argv) { const char *topic = test_mk_topic_name(__FUNCTION__, 1); const int partition = 0; const int msgcnt = 2*1000; const int MAX_BYTES = 100000; uint64_t testid; rd_kafka_conf_t *conf; rd_kafka_t *rk; rd_kafka_topic_t *rkt; test_conf_init(NULL, NULL, 60); testid = test_id_generate(); rk = test_create_producer(); rkt = test_create_producer_topic(rk, topic, NULL); test_produce_msgs(rk, rkt, testid, partition, 0, msgcnt/2, NULL, MAX_BYTES/10); test_produce_msgs(rk, rkt, testid, partition, msgcnt/2, msgcnt/2, NULL, MAX_BYTES*5); rd_kafka_topic_destroy(rkt); rd_kafka_destroy(rk); TEST_SAY("Creating consumer\n"); test_conf_init(&conf, NULL, 0); test_conf_set(conf, "fetch.message.max.bytes", tsprintf("%d", MAX_BYTES)); rk = test_create_consumer(NULL, NULL, conf, NULL); rkt = rd_kafka_topic_new(rk, topic, NULL); test_consumer_start("CONSUME", rkt, partition, RD_KAFKA_OFFSET_BEGINNING); test_consume_msgs("CONSUME", rkt, testid, partition, TEST_NO_SEEK, 0, msgcnt, 1); test_consumer_stop("CONSUME", rkt, partition); rd_kafka_topic_destroy(rkt); rd_kafka_destroy(rk); return 0; }
/** * Create high-level consumer subscribing to \p topic from BEGINNING * and expects \d exp_msgcnt with matching \p testid * Destroys consumer when done. * * If \p group_id is NULL a new unique group is generated */ void test_consume_msgs_easy (const char *group_id, const char *topic, uint64_t testid, int exp_msgcnt) { rd_kafka_t *rk; rd_kafka_topic_conf_t *tconf; rd_kafka_resp_err_t err; rd_kafka_topic_partition_list_t *topics; char grpid0[64]; test_conf_init(NULL, &tconf, 0); if (!group_id) group_id = test_str_id_generate(grpid0, sizeof(grpid0)); test_topic_conf_set(tconf, "auto.offset.reset", "smallest"); rk = test_create_consumer(group_id, NULL, tconf, NULL); rd_kafka_poll_set_consumer(rk); topics = rd_kafka_topic_partition_list_new(1); rd_kafka_topic_partition_list_add(topics, topic, RD_KAFKA_PARTITION_UA); TEST_SAY("Subscribing to topic %s in group %s " "(expecting %d msgs with testid %"PRIu64")\n", topic, group_id, exp_msgcnt, testid); err = rd_kafka_subscribe(rk, topics); if (err) TEST_FAIL("Failed to subscribe to %s: %s\n", topic, rd_kafka_err2str(err)); rd_kafka_topic_partition_list_destroy(topics); /* Consume messages */ test_consumer_poll("consume.easy", rk, testid, -1, -1, exp_msgcnt); test_consumer_close(rk); rd_kafka_destroy(rk); }
static void do_consume (const char *topic, int msgcnt) { rd_kafka_t *rk; rd_kafka_topic_partition_list_t *parts; rk = test_create_consumer(topic, NULL, NULL, NULL); parts = rd_kafka_topic_partition_list_new(1); rd_kafka_topic_partition_list_add(parts, topic, 0)->offset = RD_KAFKA_OFFSET_BEGINNING; test_consumer_assign("assign", rk, parts); rd_kafka_topic_partition_list_destroy(parts); exp_msgid = 0; while (exp_msgid < msgcnt) { rd_kafka_message_t *rkm; rkm = rd_kafka_consumer_poll(rk, 1000); if (!rkm) continue; if (rkm->err) TEST_FAIL("consume error while expecting msgid %d/%d: " "%s", exp_msgid, msgcnt, rd_kafka_message_errstr(rkm)); handle_consumed_msg(rkm); rd_kafka_message_destroy(rkm); } test_consumer_close(rk); rd_kafka_destroy(rk); }
int main_0056_balanced_group_mt (int argc, char **argv) { const char *topic = test_mk_topic_name(__FUNCTION__, 1); rd_kafka_t *rk_p, *rk_c; rd_kafka_topic_t *rkt_p; int msg_cnt = 1000; int msg_base = 0; int partition_cnt = 2; int partition; uint64_t testid; rd_kafka_topic_conf_t *default_topic_conf; rd_kafka_topic_partition_list_t *sub, *topics; rd_kafka_resp_err_t err; test_timing_t t_assign, t_close, t_consume; int i; exp_msg_cnt = msg_cnt * partition_cnt; testid = test_id_generate(); /* Produce messages */ rk_p = test_create_producer(); rkt_p = test_create_producer_topic(rk_p, topic, NULL); for (partition = 0; partition < partition_cnt; partition++) { test_produce_msgs(rk_p, rkt_p, testid, partition, msg_base + (partition * msg_cnt), msg_cnt, NULL, 0); } rd_kafka_topic_destroy(rkt_p); rd_kafka_destroy(rk_p); if (mtx_init(&lock, mtx_plain) != thrd_success) TEST_FAIL("Cannot create mutex."); test_conf_init(NULL, &default_topic_conf, (test_session_timeout_ms * 3) / 1000); test_topic_conf_set(default_topic_conf, "auto.offset.reset", "smallest"); /* Fill in topic subscription set */ topics = rd_kafka_topic_partition_list_new(1); rd_kafka_topic_partition_list_add(topics, topic, RD_KAFKA_PARTITION_UA); /* Create consumers and start subscription */ rk_c = test_create_consumer( topic /*group_id*/, rebalance_cb, NULL, default_topic_conf); test_consumer_subscribe(rk_c, topic); rd_kafka_topic_partition_list_destroy(topics); /* Wait for both consumers to get an assignment */ TIMING_START(&t_assign, "WAIT.ASSIGN"); get_assignment(rk_c); TIMING_STOP(&t_assign); TIMING_START(&t_consume, "CONSUME.WAIT"); for (i = 0; i < MAX_THRD_CNT; ++i) { if (tids[i] != 0) thrd_join(tids[i], NULL); } TIMING_STOP(&t_consume); TEST_SAY("Closing remaining consumers\n"); /* Query subscription */ err = rd_kafka_subscription(rk_c, &sub); TEST_ASSERT(!err, "%s: subscription () failed: %s", rd_kafka_name(rk_c), rd_kafka_err2str(err)); TEST_SAY("%s: subscription (%d):\n", rd_kafka_name(rk_c), sub->cnt); for (i = 0; i < sub->cnt; ++i) TEST_SAY(" %s\n", sub->elems[i].topic); rd_kafka_topic_partition_list_destroy(sub); /* Run an explicit unsubscribe () (async) prior to close () * to trigger race condition issues on termination. */ TEST_SAY("Unsubscribing instance %s\n", rd_kafka_name(rk_c)); err = rd_kafka_unsubscribe(rk_c); TEST_ASSERT(!err, "%s: unsubscribe failed: %s", rd_kafka_name(rk_c), rd_kafka_err2str(err)); TEST_SAY("Closing %s\n", rd_kafka_name(rk_c)); TIMING_START(&t_close, "CONSUMER.CLOSE"); err = rd_kafka_consumer_close(rk_c); TIMING_STOP(&t_close); TEST_ASSERT(!err, "consumer_close failed: %s", rd_kafka_err2str(err)); rd_kafka_destroy(rk_c); rk_c = NULL; TEST_SAY("%d/%d messages consumed\n", consumed_msg_cnt, exp_msg_cnt); TEST_ASSERT(consumed_msg_cnt >= exp_msg_cnt, "Only %d/%d messages were consumed", consumed_msg_cnt, exp_msg_cnt); if (consumed_msg_cnt > exp_msg_cnt) TEST_SAY("At least %d/%d messages were consumed " "multiple times\n", consumed_msg_cnt - exp_msg_cnt, exp_msg_cnt); mtx_destroy(&lock); return 0; }
static void do_test_regex (void) { char *base_topic = rd_strdup(test_mk_topic_name("topic", 1)); char *topic_b = rd_strdup(tsprintf("%s_b", base_topic)); char *topic_c = rd_strdup(tsprintf("%s_c", base_topic)); char *topic_d = rd_strdup(tsprintf("%s_d", base_topic)); char *topic_e = rd_strdup(tsprintf("%s_e", base_topic)); rd_kafka_t *rk; rd_kafka_conf_t *conf; rd_kafka_queue_t *queue; /** * Regex test: * - Create topic b * - Subscribe to b & d & e * - Verify b assignment * - Create topic c * - Verify no rebalance * - Create topic d * - Verify b & d assignment */ TEST_SAY("Regex testing\n"); test_conf_init(&conf, NULL, 60); /* Decrease metadata interval to speed up topic change discovery. */ test_conf_set(conf, "metadata.max.age.ms", "5000"); rd_kafka_conf_set_events(conf, RD_KAFKA_EVENT_REBALANCE); rk = test_create_consumer(test_str_id_generate_tmp(), NULL, conf, NULL, NULL); queue = rd_kafka_queue_get_consumer(rk); TEST_SAY("Regex: creating topic %s (subscribed)\n", topic_b); test_create_topic(topic_b, 2, 1); rd_sleep(1); // FIXME: do check&wait loop instead TEST_SAY("Regex: Subscribing to %s & %s & %s\n", topic_b, topic_d, topic_e); test_consumer_subscribe(rk, tsprintf("^%s_[bde]$", base_topic)); await_assignment("Regex: just one topic exists", rk, queue, 1, topic_b, 2); TEST_SAY("Regex: creating topic %s (not subscribed)\n", topic_c); test_create_topic(topic_c, 4, 1); /* Should not see a rebalance since no topics are matched. */ await_no_rebalance("Regex: empty", rk, queue, 10000); TEST_SAY("Regex: creating topic %s (subscribed)\n", topic_d); test_create_topic(topic_d, 1, 1); await_revoke("Regex: rebalance after topic creation", rk, queue); await_assignment("Regex: two topics exist", rk, queue, 2, topic_b, 2, topic_d, 1); test_consumer_close(rk); rd_kafka_queue_destroy(queue); rd_kafka_destroy(rk); rd_free(base_topic); rd_free(topic_b); rd_free(topic_c); rd_free(topic_d); rd_free(topic_e); }
int main_0029_assign_offset (int argc, char **argv) { const char *topic = test_mk_topic_name(__FUNCTION__, 1); rd_kafka_t *rk; rd_kafka_topic_t *rkt; rd_kafka_topic_conf_t *tconf; rd_kafka_topic_partition_list_t *parts; uint64_t testid; int i; test_timing_t t_simple, t_hl; test_msgver_t mv; test_conf_init(NULL, &tconf, 20 + (test_session_timeout_ms * 3 / 1000)); test_topic_conf_set(tconf, "auto.offset.reset", "smallest"); /* Produce X messages to Y partitions so we get a * nice seekable 0..X offset one each partition. */ /* Produce messages */ testid = test_id_generate(); rk = test_create_producer(); rkt = test_create_producer_topic(rk, topic, NULL); parts = rd_kafka_topic_partition_list_new(partitions); for (i = 0 ; i < partitions ; i++) { test_produce_msgs(rk, rkt, testid, i, 0, msgcnt, NULL, 0); rd_kafka_topic_partition_list_add(parts, topic, i)->offset = msgcnt / 2; } rd_kafka_topic_destroy(rkt); rd_kafka_destroy(rk); /* Simple consumer */ TIMING_START(&t_simple, "SIMPLE.CONSUMER"); rk = test_create_consumer(topic, NULL, NULL, NULL); test_msgver_init(&mv, testid); test_consumer_assign("SIMPLE.ASSIGN", rk, parts); test_consumer_poll("SIMPLE.CONSUME", rk, testid, -1, 0, partitions * (msgcnt / 2), &mv); for (i = 0 ; i < partitions ; i++) test_msgver_verify_part("HL.MSGS", &mv, TEST_MSGVER_ALL_PART, topic, i, msgcnt/2, msgcnt/2); test_msgver_clear(&mv); test_consumer_close(rk); rd_kafka_destroy(rk); TIMING_STOP(&t_simple); rd_kafka_topic_partition_list_destroy(parts); /* High-level consumer. * Offsets are set in rebalance callback. */ TIMING_START(&t_hl, "HL.CONSUMER"); test_msgver_init(&mv, testid); rk = test_create_consumer(topic, rebalance_cb, NULL, NULL); test_consumer_subscribe(rk, topic); test_consumer_poll("HL.CONSUME", rk, testid, -1, 0, partitions * (msgcnt / 2), &mv); for (i = 0 ; i < partitions ; i++) test_msgver_verify_part("HL.MSGS", &mv, TEST_MSGVER_ALL_PART, topic, i, msgcnt/2, msgcnt/2); test_msgver_clear(&mv); test_consumer_close(rk); rd_kafka_destroy(rk); TIMING_STOP(&t_hl); rd_kafka_topic_conf_destroy(tconf); return 0; }
int main_0015_offsets_seek (int argc, char **argv) { const char *topic = test_mk_topic_name("0015", 1); rd_kafka_t *rk_p, *rk_c; rd_kafka_topic_t *rkt_p, *rkt_c; int msg_cnt = 1000; int msg_base = 0; int32_t partition = 0; int i; int64_t offset_last, offset_base; uint64_t testid; int dance_iterations = 10; int msgs_per_dance = 10; testid = test_id_generate(); /* Produce messages */ rk_p = test_create_producer(); rkt_p = test_create_producer_topic(rk_p, topic, NULL); test_produce_msgs(rk_p, rkt_p, testid, partition, msg_base, msg_cnt, NULL, 0); rd_kafka_topic_destroy(rkt_p); rd_kafka_destroy(rk_p); rk_c = test_create_consumer(NULL, NULL, NULL, NULL); rkt_c = test_create_consumer_topic(rk_c, topic); /* Start consumer tests */ test_consumer_start("verify.all", rkt_c, partition, RD_KAFKA_OFFSET_BEGINNING); /* Make sure all messages are available */ offset_last = test_consume_msgs("verify.all", rkt_c, testid, partition, TEST_NO_SEEK, msg_base, msg_cnt, 1/* parse format*/); /* Rewind offset back to its base. */ offset_base = offset_last - msg_cnt + 1; TEST_SAY("%s [%"PRId32"]: Do random seek&consume for msgs #%d+%d with " "offsets %"PRId64"..%"PRId64"\n", rd_kafka_topic_name(rkt_c), partition, msg_base, msg_cnt, offset_base, offset_last); /* Now go dancing over the entire range with offset seeks. */ for (i = 0 ; i < dance_iterations ; i++) { int64_t offset = jitter((int)offset_base, (int)offset_base+msg_cnt); test_consume_msgs("dance", rkt_c, testid, partition, offset, msg_base + (int)(offset - offset_base), RD_MIN(msgs_per_dance, (int)(offset_last - offset)), 1 /* parse format */); } test_consumer_stop("1", rkt_c, partition); rd_kafka_topic_destroy(rkt_c); rd_kafka_destroy(rk_c); return 0; }
static int nonexist_part (void) { const char *topic = test_mk_topic_name(__FUNCTION__, 1); rd_kafka_t *rk; rd_kafka_topic_partition_list_t *parts; rd_kafka_resp_err_t err; test_timing_t t_pos; const int msgcnt = 1000; uint64_t testid; int i; int it, iterations = 5; /* Produce messages */ testid = test_produce_msgs_easy(topic, 0, RD_KAFKA_PARTITION_UA, msgcnt); for (it = 0 ; it < iterations ; it++) { char group_id[32]; test_str_id_generate(group_id, sizeof(group_id)); TEST_SAY("Iteration %d/%d, using group.id %s\n", it, iterations, group_id); /* Consume messages */ test_consume_msgs_easy(group_id, topic, testid, -1, msgcnt, NULL); /* * Now start a new consumer and query stored offsets (positions) */ rk = test_create_consumer(group_id, NULL, NULL, NULL, NULL); /* Fill in partition set */ parts = rd_kafka_topic_partition_list_new(2); /* existing */ rd_kafka_topic_partition_list_add(parts, topic, 0); /* non-existing */ rd_kafka_topic_partition_list_add(parts, topic, 123); TIMING_START(&t_pos, "COMMITTED"); err = rd_kafka_committed(rk, parts, tmout_multip(5000)); TIMING_STOP(&t_pos); if (err) TEST_FAIL("Failed to acquire committed offsets: %s\n", rd_kafka_err2str(err)); for (i = 0 ; i < parts->cnt ; i++) { TEST_SAY("%s [%"PRId32"] returned offset %"PRId64 ": %s\n", parts->elems[i].topic, parts->elems[i].partition, parts->elems[i].offset, rd_kafka_err2str(parts->elems[i].err)); if (parts->elems[i].partition == 0 && parts->elems[i].offset <= 0) TEST_FAIL("Partition %"PRId32" should have a " "proper offset, not %"PRId64"\n", parts->elems[i].partition, parts->elems[i].offset); else if (parts->elems[i].partition == 123 && parts->elems[i].offset != RD_KAFKA_OFFSET_INVALID) TEST_FAIL("Partition %"PRId32 " should have failed\n", parts->elems[i].partition); } rd_kafka_topic_partition_list_destroy(parts); test_consumer_close(rk); /* Hangs if bug isn't fixed */ rd_kafka_destroy(rk); } return 0; }
int main_0040_io_event (int argc, char **argv) { rd_kafka_conf_t *conf; rd_kafka_topic_conf_t *tconf; rd_kafka_t *rk_p, *rk_c; const char *topic; rd_kafka_topic_t *rkt_p; rd_kafka_queue_t *queue; uint64_t testid; int msgcnt = 100; int recvd = 0; int fds[2]; int wait_multiplier = 1; struct pollfd pfd; int r; enum { _NOPE, _YEP, _REBALANCE } expecting_io = _REBALANCE; testid = test_id_generate(); topic = test_mk_topic_name(__FUNCTION__, 1); rk_p = test_create_producer(); rkt_p = test_create_producer_topic(rk_p, topic, NULL); test_auto_create_topic_rkt(rk_p, rkt_p); test_conf_init(&conf, &tconf, 0); rd_kafka_conf_set_events(conf, RD_KAFKA_EVENT_REBALANCE); test_conf_set(conf, "session.timeout.ms", "6000"); test_conf_set(conf, "enable.partition.eof", "false"); /* Speed up propagation of new topics */ test_conf_set(conf, "metadata.max.age.ms", "5000"); test_topic_conf_set(tconf, "auto.offset.reset", "earliest"); rk_c = test_create_consumer(topic, NULL, conf, tconf); queue = rd_kafka_queue_get_consumer(rk_c); test_consumer_subscribe(rk_c, topic); #ifndef _MSC_VER r = pipe(fds); #else r = _pipe(fds, 2, _O_BINARY); #endif if (r == -1) TEST_FAIL("pipe() failed: %s\n", strerror(errno)); rd_kafka_queue_io_event_enable(queue, fds[1], "1", 1); pfd.fd = fds[0]; pfd.events = POLLIN; pfd.revents = 0; /** * 1) Wait for rebalance event * 2) Wait 1 interval (1s) expecting no IO (nothing produced). * 3) Produce half the messages * 4) Expect IO * 5) Consume the available messages * 6) Wait 1 interval expecting no IO. * 7) Produce remaing half * 8) Expect IO * 9) Done. */ while (recvd < msgcnt) { int r; #ifndef _MSC_VER r = poll(&pfd, 1, 1000 * wait_multiplier); #else r = WSAPoll(&pfd, 1, 1000 * wait_multiplier); #endif if (r == -1) { TEST_FAIL("poll() failed: %s", strerror(errno)); } else if (r == 1) { rd_kafka_event_t *rkev; char b; int eventcnt = 0; if (pfd.events & POLLERR) TEST_FAIL("Poll error\n"); if (!(pfd.events & POLLIN)) { TEST_SAY("Stray event 0x%x\n", (int)pfd.events); continue; } TEST_SAY("POLLIN\n"); /* Read signaling token to purge socket queue and * eventually silence POLLIN */ #ifndef _MSC_VER r = read(pfd.fd, &b, 1); #else r = _read((int)pfd.fd, &b, 1); #endif if (r == -1) TEST_FAIL("read failed: %s\n", strerror(errno)); if (!expecting_io) TEST_WARN("Got unexpected IO after %d/%d msgs\n", recvd, msgcnt); while ((rkev = rd_kafka_queue_poll(queue, 0))) { eventcnt++; switch (rd_kafka_event_type(rkev)) { case RD_KAFKA_EVENT_REBALANCE: TEST_SAY("Got %s: %s\n", rd_kafka_event_name(rkev), rd_kafka_err2str(rd_kafka_event_error(rkev))); if (expecting_io != _REBALANCE) TEST_FAIL("Got Rebalance when expecting message\n"); if (rd_kafka_event_error(rkev) == RD_KAFKA_RESP_ERR__ASSIGN_PARTITIONS) { rd_kafka_assign(rk_c, rd_kafka_event_topic_partition_list(rkev)); expecting_io = _NOPE; } else rd_kafka_assign(rk_c, NULL); break; case RD_KAFKA_EVENT_FETCH: if (expecting_io != _YEP) TEST_FAIL("Did not expect more messages at %d/%d\n", recvd, msgcnt); recvd++; if (recvd == (msgcnt / 2) || recvd == msgcnt) expecting_io = _NOPE; break; case RD_KAFKA_EVENT_ERROR: TEST_FAIL("Error: %s\n", rd_kafka_event_error_string(rkev)); break; default: TEST_SAY("Ignoring event %s\n", rd_kafka_event_name(rkev)); } rd_kafka_event_destroy(rkev); } TEST_SAY("%d events, Consumed %d/%d messages\n", eventcnt, recvd, msgcnt); wait_multiplier = 1; } else { if (expecting_io == _REBALANCE) { continue; } else if (expecting_io == _YEP) { TEST_FAIL("Did not see expected IO after %d/%d msgs\n", recvd, msgcnt); } TEST_SAY("IO poll timeout (good)\n"); TEST_SAY("Got idle period, producing\n"); test_produce_msgs(rk_p, rkt_p, testid, 0, recvd, msgcnt/2, NULL, 10); expecting_io = _YEP; /* When running slowly (e.g., valgrind) it might take * some time before the first message is received * after producing. */ wait_multiplier = 3; } } TEST_SAY("Done\n"); rd_kafka_topic_destroy(rkt_p); rd_kafka_destroy(rk_p); rd_kafka_queue_destroy(queue); rd_kafka_consumer_close(rk_c); rd_kafka_destroy(rk_c); #ifndef _MSC_VER close(fds[0]); close(fds[1]); #else _close(fds[0]); _close(fds[1]); #endif return 0; }
int main_0093_holb_consumer (int argc, char **argv) { const char *topic = test_mk_topic_name("0093_holb_consumer", 1); int64_t testid; const int msgcnt = 100; struct _consumer c[_CONSUMER_CNT] = RD_ZERO_INIT; rd_kafka_conf_t *conf; testid = test_id_generate(); test_conf_init(&conf, NULL, 60); test_create_topic(topic, 1, 1); test_produce_msgs_easy(topic, testid, 0, msgcnt); test_conf_set(conf, "session.timeout.ms", "6000"); test_conf_set(conf, "max.poll.interval.ms", "20000"); test_conf_set(conf, "socket.timeout.ms", "3000"); test_conf_set(conf, "auto.offset.reset", "earliest"); /* Trigger other requests often */ test_conf_set(conf, "topic.metadata.refresh.interval.ms", "500"); rd_kafka_conf_set_rebalance_cb(conf, rebalance_cb); rd_kafka_conf_set_opaque(conf, &c[0]); c[0].rk = test_create_consumer(topic, NULL, rd_kafka_conf_dup(conf), NULL); rd_kafka_conf_set_opaque(conf, &c[1]); c[1].rk = test_create_consumer(topic, NULL, conf, NULL); test_consumer_subscribe(c[0].rk, topic); /* c0: assign */ c[0].max_rebalance_cnt = 1; /* c1: none, hasn't joined yet */ c[1].max_rebalance_cnt = 0; TEST_SAY("Waiting for c[0] assignment\n"); while (1) { rd_kafka_topic_partition_list_t *parts = NULL; do_consume(&c[0], 1/*1s*/); if (rd_kafka_assignment(c[0].rk, &parts) != RD_KAFKA_RESP_ERR_NO_ERROR || !parts || parts->cnt == 0) { if (parts) rd_kafka_topic_partition_list_destroy(parts); continue; } TEST_SAY("%s got assignment of %d partition(s)\n", rd_kafka_name(c[0].rk), parts->cnt); rd_kafka_topic_partition_list_destroy(parts); break; } TEST_SAY("c[0] got assignment, consuming..\n"); do_consume(&c[0], 5/*5s*/); TEST_SAY("Joining second consumer\n"); test_consumer_subscribe(c[1].rk, topic); /* Just poll second consumer for 10s, the rebalance will not * finish until the first consumer polls */ do_consume(&c[1], 10/*10s*/); /* c0: the next call to do_consume/poll will trigger * its rebalance callback, first revoke then assign. */ c[0].max_rebalance_cnt += 2; /* c1: first rebalance */ c[1].max_rebalance_cnt++; TEST_SAY("Expected rebalances: c[0]: %d/%d, c[1]: %d/%d\n", c[0].rebalance_cnt, c[0].max_rebalance_cnt, c[1].rebalance_cnt, c[1].max_rebalance_cnt); /* Let rebalances kick in, then consume messages. */ while (c[0].cnt + c[1].cnt < msgcnt) { do_consume(&c[0], 0); do_consume(&c[1], 0); } /* Allow the extra revoke rebalance on close() */ c[0].max_rebalance_cnt++; c[1].max_rebalance_cnt++; test_consumer_close(c[0].rk); test_consumer_close(c[1].rk); rd_kafka_destroy(c[0].rk); rd_kafka_destroy(c[1].rk); return 0; }
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; }
/* @remark This test will fail if auto topic creation is enabled on the broker * since the client will issue a topic-creating metadata request to find * a new leader when the topic is removed. * * To run with trivup, do: * ./interactive_broker_version.py .. -conf '{"auto_create_topics":"false"}' .. * TESTS=0045 ./run-test.sh -k ./merged */ static void do_test_topic_remove (void) { char *topic_f = rd_strdup(test_mk_topic_name("topic_f", 1)); char *topic_g = rd_strdup(test_mk_topic_name("topic_g", 1)); int parts_f = 5; int parts_g = 9; rd_kafka_t *rk; rd_kafka_conf_t *conf; rd_kafka_queue_t *queue; rd_kafka_topic_partition_list_t *topics; rd_kafka_resp_err_t err; /** * Topic removal test: * - Create topic f & g * - Subscribe to f & g * - Verify f & g assignment * - Remove topic f * - Verify g assignment * - Remove topic g * - Verify empty assignment */ TEST_SAY("Topic removal testing\n"); test_conf_init(&conf, NULL, 60); /* Decrease metadata interval to speed up topic change discovery. */ test_conf_set(conf, "metadata.max.age.ms", "5000"); rd_kafka_conf_set_events(conf, RD_KAFKA_EVENT_REBALANCE); rk = test_create_consumer(test_str_id_generate_tmp(), NULL, conf, NULL, NULL); queue = rd_kafka_queue_get_consumer(rk); TEST_SAY("Topic removal: creating topic %s (subscribed)\n", topic_f); test_create_topic(topic_f, parts_f, 1); TEST_SAY("Topic removal: creating topic %s (subscribed)\n", topic_g); test_create_topic(topic_g, parts_g, 1); rd_sleep(1); // FIXME: do check&wait loop instead TEST_SAY("Topic removal: Subscribing to %s & %s\n", topic_f, topic_g); topics = rd_kafka_topic_partition_list_new(2); rd_kafka_topic_partition_list_add(topics, topic_f, RD_KAFKA_PARTITION_UA); rd_kafka_topic_partition_list_add(topics, topic_g, RD_KAFKA_PARTITION_UA); err = rd_kafka_subscribe(rk, topics); TEST_ASSERT(err == RD_KAFKA_RESP_ERR_NO_ERROR, "%s", rd_kafka_err2str(err)); rd_kafka_topic_partition_list_destroy(topics); await_assignment("Topic removal: both topics exist", rk, queue, 2, topic_f, parts_f, topic_g, parts_g); TEST_SAY("Topic removal: removing %s\n", topic_f); test_kafka_topics("--delete --topic %s", topic_f); await_revoke("Topic removal: rebalance after topic removal", rk, queue); await_assignment("Topic removal: one topic exists", rk, queue, 1, topic_g, parts_g); TEST_SAY("Topic removal: removing %s\n", topic_g); test_kafka_topics("--delete --topic %s", topic_g); await_revoke("Topic removal: rebalance after 2nd topic removal", rk, queue); /* Should not see another rebalance since all topics now removed */ await_no_rebalance("Topic removal: empty", rk, queue, 10000); test_consumer_close(rk); rd_kafka_queue_destroy(queue); rd_kafka_destroy(rk); rd_free(topic_f); rd_free(topic_g); }
static int do_test_consume_batch (void) { #define topic_cnt 2 const char *topics[topic_cnt] = { test_mk_topic_name(__FUNCTION__, 1), test_mk_topic_name(__FUNCTION__, 1) }; const int partition_cnt = 2; rd_kafka_t *rk; rd_kafka_queue_t *rkq; rd_kafka_topic_t *rkts[topic_cnt]; rd_kafka_resp_err_t err; const int msgcnt = 10000; uint64_t testid; int i, p; int batch_cnt = 0; int remains; testid = test_id_generate(); /* Produce messages */ for (i = 0 ; i < topic_cnt ; i++) { for (p = 0 ; p < partition_cnt ; p++) test_produce_msgs_easy(topics[i], testid, p, msgcnt / topic_cnt / partition_cnt); } /* Create simple consumer */ rk = test_create_consumer(NULL, NULL, NULL, NULL); /* Create generic consume queue */ rkq = rd_kafka_queue_new(rk); for (i = 0 ; i < topic_cnt ; i++) { /* Create topic object */ rkts[i] = test_create_topic(rk, topics[i], "auto.offset.reset", "smallest", NULL); /* Start consuming each partition and redirect * messages to queue */ TEST_SAY("Start consuming topic %s partitions 0..%d\n", rd_kafka_topic_name(rkts[i]), partition_cnt); for (p = 0 ; p < partition_cnt ; p++) { err = rd_kafka_consume_start_queue( rkts[i], p, RD_KAFKA_OFFSET_BEGINNING, rkq); if (err) TEST_FAIL("Failed to start consuming: %s\n", rd_kafka_err2str(err)); } } remains = msgcnt; /* Consume messages from common queue using batch interface. */ TEST_SAY("Consume %d messages from queue\n", remains); while (remains > 0) { rd_kafka_message_t *rkmessage[1000]; ssize_t r; test_timing_t t_batch; TIMING_START(&t_batch, "CONSUME.BATCH"); r = rd_kafka_consume_batch_queue(rkq, 1000, rkmessage, 1000); TIMING_STOP(&t_batch); TEST_SAY("Batch consume iteration #%d: Consumed %"PRIdsz "/1000 messages\n", batch_cnt, r); if (r == -1) TEST_FAIL("Failed to consume messages: %s\n", rd_kafka_err2str(rd_kafka_errno2err(errno))); remains -= r; for (i = 0 ; i < r ; i++) rd_kafka_message_destroy(rkmessage[i]); batch_cnt++; } TEST_SAY("Stopping consumer\n"); for (i = 0 ; i < topic_cnt ; i++) { for (p = 0 ; p < partition_cnt ; p++) { err = rd_kafka_consume_stop(rkts[i], p); if (err) TEST_FAIL("Failed to stop consuming: %s\n", rd_kafka_err2str(err)); } rd_kafka_topic_destroy(rkts[i]); } rd_kafka_queue_destroy(rkq); rd_kafka_destroy(rk); return 0; }
int main_0049_consume_conn_close (int argc, char **argv) { rd_kafka_t *rk; const char *topic = test_mk_topic_name("0049_consume_conn_close", 1); uint64_t testid; int msgcnt = test_on_ci ? 1000 : 10000; test_msgver_t mv; rd_kafka_conf_t *conf; rd_kafka_topic_conf_t *tconf; rd_kafka_topic_partition_list_t *assignment; rd_kafka_resp_err_t err; if (!test_conf_match(NULL, "sasl.mechanisms", "GSSAPI")) { TEST_SKIP("KNOWN ISSUE: ApiVersionRequest+SaslHandshake " "will not play well with sudden disconnects\n"); return 0; } test_conf_init(&conf, &tconf, 60); /* Want an even number so it is divisable by two without surprises */ msgcnt = (msgcnt / (int)test_timeout_multiplier) & ~1; testid = test_id_generate(); test_produce_msgs_easy(topic, testid, RD_KAFKA_PARTITION_UA, msgcnt); test_socket_enable(conf); test_curr->connect_cb = connect_cb; test_curr->is_fatal_cb = is_fatal_cb; test_topic_conf_set(tconf, "auto.offset.reset", "smallest"); rk = test_create_consumer(topic, NULL, conf, tconf); test_consumer_subscribe(rk, topic); test_msgver_init(&mv, testid); test_consumer_poll("consume.up", rk, testid, -1, 0, msgcnt/2, &mv); err = rd_kafka_assignment(rk, &assignment); TEST_ASSERT(!err, "assignment() failed: %s", rd_kafka_err2str(err)); TEST_ASSERT(assignment->cnt > 0, "empty assignment"); TEST_SAY("Bringing down the network\n"); TEST_LOCK(); simulate_network_down = 1; TEST_UNLOCK(); test_socket_close_all(test_curr, 1/*reinit*/); TEST_SAY("Waiting for session timeout to expire (6s), and then some\n"); /* Commit an offset, which should fail, to trigger the offset commit * callback fallback (CONSUMER_ERR) */ assignment->elems[0].offset = 123456789; TEST_SAY("Committing offsets while down, should fail eventually\n"); err = rd_kafka_commit(rk, assignment, 1/*async*/); TEST_ASSERT(!err, "async commit failed: %s", rd_kafka_err2str(err)); rd_kafka_topic_partition_list_destroy(assignment); rd_sleep(10); TEST_SAY("Bringing network back up\n"); TEST_LOCK(); simulate_network_down = 0; TEST_UNLOCK(); TEST_SAY("Continuing to consume..\n"); test_consumer_poll("consume.up2", rk, testid, -1, msgcnt/2, msgcnt/2, &mv); test_msgver_verify("consume", &mv, TEST_MSGVER_ORDER|TEST_MSGVER_DUP, 0, msgcnt); test_msgver_clear(&mv); test_consumer_close(rk); rd_kafka_destroy(rk); return 0; }
int main_0089_max_poll_interval (int argc, char **argv) { const char *topic = test_mk_topic_name("0089_max_poll_interval", 1); uint64_t testid; const int msgcnt = 10; rd_kafka_t *c[2]; rd_kafka_conf_t *conf; int64_t ts_next[2] = { 0, 0 }; int64_t ts_exp_msg[2] = { 0, 0 }; int cmsgcnt = 0; int i; int bad = -1; testid = test_id_generate(); test_create_topic(topic, 1, 1); test_produce_msgs_easy(topic, testid, -1, msgcnt); test_conf_init(&conf, NULL, 60); test_conf_set(conf, "session.timeout.ms", "6000"); test_conf_set(conf, "max.poll.interval.ms", "10000" /*10s*/); test_conf_set(conf, "auto.offset.reset", "earliest"); c[0] = test_create_consumer(topic, NULL, rd_kafka_conf_dup(conf), NULL); c[1] = test_create_consumer(topic, NULL, conf, NULL); test_consumer_subscribe(c[0], topic); test_consumer_subscribe(c[1], topic); while (1) { for (i = 0 ; i < 2 ; i++) { int64_t now; rd_kafka_message_t *rkm; /* Consumer is "processing" */ if (ts_next[i] > test_clock()) continue; rkm = rd_kafka_consumer_poll(c[i], 100); if (!rkm) continue; if (rkm->err) { TEST_WARN("Consumer %d error: %s: " "ignoring\n", i, rd_kafka_message_errstr(rkm)); continue; } now = test_clock(); cmsgcnt++; TEST_SAY("Consumer %d received message (#%d) " "at offset %"PRId64"\n", i, cmsgcnt, rkm->offset); if (ts_exp_msg[i]) { /* This consumer is expecting a message * after a certain time, namely after the * rebalance following max.poll.. being * exceeded in the other consumer */ TEST_ASSERT(now > ts_exp_msg[i], "Consumer %d: did not expect " "message for at least %dms", i, (int)((ts_exp_msg[i] - now)/1000)); TEST_ASSERT(now < ts_exp_msg[i] + 10000*1000, "Consumer %d: expected message " "within 10s, not after %dms", i, (int)((now - ts_exp_msg[i])/1000)); TEST_SAY("Consumer %d: received message " "at offset %"PRId64 " after rebalance\n", i, rkm->offset); rd_kafka_message_destroy(rkm); goto done; } else if (cmsgcnt == 1) { /* Process this message for 20s */ ts_next[i] = now + (20000 * 1000); /* Exp message on other consumer after * max.poll.interval.ms */ ts_exp_msg[i^1] = now + (10000 * 1000); /* This is the bad consumer */ bad = i; TEST_SAY("Consumer %d processing message at " "offset %"PRId64"\n", i, rkm->offset); rd_kafka_message_destroy(rkm); } else { rd_kafka_message_destroy(rkm); TEST_FAIL("Consumer %d did not expect " "a message", i); } } } done: TEST_ASSERT(bad != -1, "Bad consumer not set"); /* Wait for error ERR__MAX_POLL_EXCEEDED on the bad consumer. */ while (1) { rd_kafka_message_t *rkm; rkm = rd_kafka_consumer_poll(c[bad], 1000); TEST_ASSERT(rkm, "Expected consumer result within 1s"); TEST_ASSERT(rkm->err, "Did not expect message on bad consumer"); TEST_SAY("Consumer error: %s: %s\n", rd_kafka_err2name(rkm->err), rd_kafka_message_errstr(rkm)); if (rkm->err == RD_KAFKA_RESP_ERR__MAX_POLL_EXCEEDED) { rd_kafka_message_destroy(rkm); break; } rd_kafka_message_destroy(rkm); } for (i = 0 ; i < 2 ; i++) rd_kafka_destroy_flags(c[i], RD_KAFKA_DESTROY_F_NO_CONSUMER_CLOSE); 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; }
int main_0038_performance (int argc, char **argv) { const char *topic = test_mk_topic_name(__FUNCTION__, 1); const int partition = 0; const int msgsize = 100; uint64_t testid; rd_kafka_conf_t *conf; rd_kafka_t *rk; rd_kafka_topic_t *rkt; test_timing_t t_create, t_produce, t_consume; int totsize = 1024*1024*128; int msgcnt; if (!strcmp(test_mode, "valgrind") || !strcmp(test_mode, "helgrind") || !strcmp(test_mode, "drd")) totsize = 1024*1024*8; /* 8 meg, valgrind is slow. */ msgcnt = totsize / msgsize; TEST_SAY("Producing %d messages of size %d to %s [%d]\n", msgcnt, (int)msgsize, topic, partition); testid = test_id_generate(); test_conf_init(&conf, NULL, 120); rd_kafka_conf_set_dr_cb(conf, test_dr_cb); test_conf_set(conf, "queue.buffering.max.messages", "10000000"); rk = test_create_handle(RD_KAFKA_PRODUCER, conf); rkt = test_create_producer_topic(rk, topic, "acks", "1", NULL); /* First produce one message to create the topic, etc, this might take * a while and we dont want this to affect the throughput timing. */ TIMING_START(&t_create, "CREATE TOPIC"); test_produce_msgs(rk, rkt, testid, partition, 0, 1, NULL, msgsize); TIMING_STOP(&t_create); TIMING_START(&t_produce, "PRODUCE"); test_produce_msgs(rk, rkt, testid, partition, 1, msgcnt-1, NULL, msgsize); TIMING_STOP(&t_produce); TEST_SAY("Destroying producer\n"); rd_kafka_topic_destroy(rkt); rd_kafka_destroy(rk); TEST_SAY("Creating consumer\n"); test_conf_init(&conf, NULL, 120); rk = test_create_consumer(NULL, NULL, conf, NULL, NULL); rkt = rd_kafka_topic_new(rk, topic, NULL); test_consumer_start("CONSUME", rkt, partition, RD_KAFKA_OFFSET_BEGINNING); TIMING_START(&t_consume, "CONSUME"); test_consume_msgs("CONSUME", rkt, testid, partition, TEST_NO_SEEK, 0, msgcnt, 1); TIMING_STOP(&t_consume); test_consumer_stop("CONSUME", rkt, partition); rd_kafka_topic_destroy(rkt); rd_kafka_destroy(rk); TEST_REPORT("{ \"producer\": " " { \"mb_per_sec\": %.2f, \"records_per_sec\": %.2f }," " \"consumer\": " "{ \"mb_per_sec\": %.2f, \"records_per_sec\": %.2f } " "}", (double) (totsize/((double)TIMING_DURATION(&t_produce)/1000000.0f)) / 1000000.0f, (float) (msgcnt/((double)TIMING_DURATION(&t_produce)/1000000.0f)), (double) (totsize/((double)TIMING_DURATION(&t_consume)/1000000.0f)) / 1000000.0f, (float) (msgcnt/((double)TIMING_DURATION(&t_consume)/1000000.0f))); return 0; }
int main_0017_compression(int argc, char **argv) { rd_kafka_t *rk_p, *rk_c; const int msg_cnt = 1000; int msg_base = 0; uint64_t testid; #define CODEC_CNT 4 const char *codecs[CODEC_CNT+1] = { "none", #if WITH_ZLIB "gzip", #endif #if WITH_SNAPPY "snappy", #endif #if WITH_LZ4 "lz4", #endif NULL }; const char *topics[CODEC_CNT]; const int32_t partition = 0; int i; testid = test_id_generate(); /* Produce messages */ rk_p = test_create_producer(); for (i = 0; codecs[i] != NULL ; i++) { rd_kafka_topic_t *rkt_p; topics[i] = test_mk_topic_name(codecs[i], 1); TEST_SAY("Produce %d messages with %s compression to " "topic %s\n", msg_cnt, codecs[i], topics[i]); rkt_p = test_create_producer_topic(rk_p, topics[i], "compression.codec", codecs[i], NULL); /* Produce small message that will not decrease with * compression (issue #781) */ test_produce_msgs(rk_p, rkt_p, testid, partition, msg_base + (partition*msg_cnt), 1, NULL, 5); /* Produce standard sized messages */ test_produce_msgs(rk_p, rkt_p, testid, partition, msg_base + (partition*msg_cnt) + 1, msg_cnt-1, NULL, 512); rd_kafka_topic_destroy(rkt_p); } rd_kafka_destroy(rk_p); /* Consume messages */ rk_c = test_create_consumer(NULL, NULL, NULL, NULL); for (i = 0; codecs[i] != NULL ; i++) { rd_kafka_topic_t *rkt_c = rd_kafka_topic_new(rk_c, topics[i], NULL); TEST_SAY("Consume %d messages from topic %s\n", msg_cnt, topics[i]); /* Start consuming */ test_consumer_start(codecs[i], rkt_c, partition, RD_KAFKA_OFFSET_BEGINNING); /* Consume messages */ test_consume_msgs(codecs[i], rkt_c, testid, partition, /* Use offset 0 here, which is wrong, should * be TEST_NO_SEEK, but it exposed a bug * where the Offset query was postponed * till after the seek, causing messages * to be replayed. */ 0, msg_base, msg_cnt, 1 /* parse format */); test_consumer_stop(codecs[i], rkt_c, partition); rd_kafka_topic_destroy(rkt_c); } rd_kafka_destroy(rk_c); return 0; }
int main_0018_cgrp_term (int argc, char **argv) { const char *topic = test_mk_topic_name(__FUNCTION__, 1); #define _CONS_CNT 2 rd_kafka_t *rk_p, *rk_c[_CONS_CNT]; rd_kafka_topic_t *rkt_p; int msg_cnt = 1000; int msg_base = 0; int partition_cnt = 2; int partition; uint64_t testid; rd_kafka_topic_conf_t *default_topic_conf; rd_kafka_topic_partition_list_t *topics; rd_kafka_resp_err_t err; test_timing_t t_assign, t_consume; char errstr[512]; int i; testid = test_id_generate(); /* Produce messages */ rk_p = test_create_producer(); rkt_p = test_create_producer_topic(rk_p, topic, NULL); for (partition = 0 ; partition < partition_cnt ; partition++) { test_produce_msgs(rk_p, rkt_p, testid, partition, msg_base+(partition*msg_cnt), msg_cnt, NULL, 0); } rd_kafka_topic_destroy(rkt_p); rd_kafka_destroy(rk_p); test_conf_init(NULL, &default_topic_conf, (test_session_timeout_ms * 3) / 1000); if (rd_kafka_topic_conf_set(default_topic_conf, "auto.offset.reset", "smallest", errstr, sizeof(errstr)) != RD_KAFKA_CONF_OK) TEST_FAIL("%s\n", errstr); /* 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++) { rk_c[i] = test_create_consumer(topic/*group_id*/, rebalance_cb, NULL, rd_kafka_topic_conf_dup( default_topic_conf), 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_conf_destroy(default_topic_conf); rd_kafka_topic_partition_list_destroy(topics); /* Wait for both consumers to get an assignment */ TIMING_START(&t_assign, "WAIT.ASSIGN"); while (assign_cnt < _CONS_CNT) consume_all(rk_c, _CONS_CNT, msg_cnt, test_session_timeout_ms + 3000); TIMING_STOP(&t_assign); /* Now close one of the consumers, this will cause a rebalance. */ TEST_SAY("Closing down 1/%d consumer(s): %s\n", _CONS_CNT, rd_kafka_name(rk_c[0])); err = rd_kafka_consumer_close(rk_c[0]); if (err) TEST_FAIL("consumer_close failed: %s\n", rd_kafka_err2str(err)); rd_kafka_destroy(rk_c[0]); rk_c[0] = NULL; /* Let remaining consumers run for a while to take over the now * lost partitions. */ if (assign_cnt != _CONS_CNT-1) TEST_FAIL("assign_cnt %d, should be %d\n", assign_cnt, _CONS_CNT-1); TIMING_START(&t_consume, "CONSUME.WAIT"); consume_all(rk_c, _CONS_CNT, msg_cnt, test_session_timeout_ms + 3000); TIMING_STOP(&t_consume); TEST_SAY("Closing remaining consumers\n"); for (i = 0 ; i < _CONS_CNT ; i++) { test_timing_t t_close; rd_kafka_topic_partition_list_t *sub; int j; if (!rk_c[i]) continue; /* Query subscription */ err = rd_kafka_subscription(rk_c[i], &sub); if (err) TEST_FAIL("%s: subscription() failed: %s\n", rd_kafka_name(rk_c[i]), rd_kafka_err2str(err)); TEST_SAY("%s: subscription (%d):\n", rd_kafka_name(rk_c[i]), sub->cnt); for (j = 0 ; j < sub->cnt ; j++) TEST_SAY(" %s\n", sub->elems[j].topic); rd_kafka_topic_partition_list_destroy(sub); /* Run an explicit unsubscribe() (async) prior to close() * to trigger race condition issues on termination. */ TEST_SAY("Unsubscribing instance %s\n", rd_kafka_name(rk_c[i])); err = rd_kafka_unsubscribe(rk_c[i]); if (err) TEST_FAIL("%s: unsubscribe failed: %s\n", rd_kafka_name(rk_c[i]), rd_kafka_err2str(err)); 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; } TEST_SAY("%d/%d messages consumed\n", consumed_msg_cnt, msg_cnt); if (consumed_msg_cnt < msg_cnt) TEST_FAIL("Only %d/%d messages were consumed\n", consumed_msg_cnt, msg_cnt); else if (consumed_msg_cnt > msg_cnt) TEST_SAY("At least %d/%d messages were consumed " "multiple times\n", consumed_msg_cnt - msg_cnt, msg_cnt); return 0; }
int main_0029_assign_offset (int argc, char **argv) { const char *topic = test_mk_topic_name(__FUNCTION__, 1); rd_kafka_t *rk; rd_kafka_topic_t *rkt; rd_kafka_topic_partition_list_t *parts; uint64_t testid; int i; test_timing_t t_simple, t_hl; test_msgver_t mv; test_conf_init(NULL, NULL, 20 + (test_session_timeout_ms * 3 / 1000)); /* Produce X messages to Y partitions so we get a * nice seekable 0..X offset one each partition. */ /* Produce messages */ testid = test_id_generate(); rk = test_create_producer(); rkt = test_create_producer_topic(rk, topic, NULL); parts = rd_kafka_topic_partition_list_new(partitions); for (i = 0 ; i < partitions ; i++) { test_produce_msgs(rk, rkt, testid, i, 0, msgcnt, NULL, 0); /* Set start offset */ rd_kafka_topic_partition_list_add(parts, topic, i)->offset = msgcnt / 2; } rd_kafka_topic_destroy(rkt); rd_kafka_destroy(rk); /* Simple consumer */ TIMING_START(&t_simple, "SIMPLE.CONSUMER"); rk = test_create_consumer(topic, NULL, NULL, NULL); test_msgver_init(&mv, testid); test_consumer_assign("SIMPLE.ASSIGN", rk, parts); test_consumer_poll("SIMPLE.CONSUME", rk, testid, -1, 0, partitions * (msgcnt / 2), &mv); for (i = 0 ; i < partitions ; i++) test_msgver_verify_part("HL.MSGS", &mv, TEST_MSGVER_ALL_PART, topic, i, msgcnt/2, msgcnt/2); test_msgver_clear(&mv); test_consumer_close(rk); rd_kafka_destroy(rk); TIMING_STOP(&t_simple); rd_kafka_topic_partition_list_destroy(parts); /* High-level consumer: method 1 * Offsets are set in rebalance callback. */ if (test_broker_version >= TEST_BRKVER(0,9,0,0)) { reb_method = REB_METHOD_1; TIMING_START(&t_hl, "HL.CONSUMER"); test_msgver_init(&mv, testid); rk = test_create_consumer(topic, rebalance_cb, NULL, NULL); test_consumer_subscribe(rk, topic); test_consumer_poll("HL.CONSUME", rk, testid, -1, 0, partitions * (msgcnt / 2), &mv); for (i = 0 ; i < partitions ; i++) test_msgver_verify_part("HL.MSGS", &mv, TEST_MSGVER_ALL_PART, topic, i, msgcnt/2, msgcnt/2); test_msgver_clear(&mv); test_consumer_close(rk); rd_kafka_destroy(rk); TIMING_STOP(&t_hl); /* High-level consumer: method 2: * first two partitions are with fixed absolute offset, rest are * auto offset (stored, which is now at end). * Offsets are set in rebalance callback. */ reb_method = REB_METHOD_2; TIMING_START(&t_hl, "HL.CONSUMER2"); test_msgver_init(&mv, testid); rk = test_create_consumer(topic, rebalance_cb, NULL, NULL); test_consumer_subscribe(rk, topic); test_consumer_poll("HL.CONSUME2", rk, testid, partitions, 0, 2 * (msgcnt / 2), &mv); for (i = 0 ; i < partitions ; i++) { if (i < 2) test_msgver_verify_part("HL.MSGS2.A", &mv, TEST_MSGVER_ALL_PART, topic, i, msgcnt/2, msgcnt/2); } test_msgver_clear(&mv); test_consumer_close(rk); rd_kafka_destroy(rk); TIMING_STOP(&t_hl); } return 0; }