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); }
/** * @brief Test handling of implicit acks. * * @param batch_cnt Total number of batches, ProduceRequests, sent. * @param initial_fail_batch_cnt How many of the initial batches should * fail with an emulated network timeout. */ static void do_test_implicit_ack (const char *what, int batch_cnt, int initial_fail_batch_cnt) { rd_kafka_t *rk; const char *topic = test_mk_topic_name("0090_idempotence_impl_ack", 1); const int32_t partition = 0; uint64_t testid; int msgcnt = 10*batch_cnt; rd_kafka_conf_t *conf; rd_kafka_topic_t *rkt; test_msgver_t mv; TEST_SAY(_C_MAG "[ Test implicit ack: %s ]\n", what); rd_atomic32_init(&state.produce_cnt, 0); state.batch_cnt = batch_cnt; state.initial_fail_batch_cnt = initial_fail_batch_cnt; testid = test_id_generate(); test_conf_init(&conf, NULL, 60); rd_kafka_conf_set_dr_msg_cb(conf, test_dr_msg_cb); test_conf_set(conf, "enable.idempotence", "true"); test_conf_set(conf, "batch.num.messages", "10"); test_conf_set(conf, "linger.ms", "500"); test_conf_set(conf, "retry.backoff.ms", "2000"); /* The ProduceResponse handler will inject timed-out-in-flight * errors for the first N ProduceRequests, which will trigger retries * that in turn will result in OutOfSequence errors. */ test_conf_set(conf, "ut_handle_ProduceResponse", (char *)handle_ProduceResponse); test_create_topic(topic, 1, 1); rk = test_create_handle(RD_KAFKA_PRODUCER, conf); rkt = test_create_producer_topic(rk, topic, NULL); TEST_SAY("Producing %d messages\n", msgcnt); test_produce_msgs(rk, rkt, testid, -1, 0, msgcnt, NULL, 0); TEST_SAY("Flushing..\n"); rd_kafka_flush(rk, 10000); rd_kafka_topic_destroy(rkt); rd_kafka_destroy(rk); TEST_SAY("Verifying messages with consumer\n"); test_msgver_init(&mv, testid); test_consume_msgs_easy_mv(NULL, topic, partition, testid, 1, msgcnt, NULL, &mv); test_msgver_verify("verify", &mv, TEST_MSGVER_ALL, 0, msgcnt); test_msgver_clear(&mv); TEST_SAY(_C_GRN "[ Test implicit ack: %s : PASS ]\n", what); }
static void consume_msgs_verify_timestamps (const char *topic, int partition, uint64_t testid, int msgcnt, const struct timestamp_range *exp_timestamp) { test_msgver_t mv; test_msgver_init(&mv, testid); test_consume_msgs_easy_mv(topic, topic, testid, -1, msgcnt, NULL, &mv); test_msgver_verify0(__FUNCTION__, __LINE__, topic, &mv, TEST_MSGVER_RANGE| TEST_MSGVER_BY_MSGID|TEST_MSGVER_BY_TIMESTAMP, (struct test_mv_vs){ .msg_base = 0, .exp_cnt = msgcnt, .timestamp_min = exp_timestamp->min, .timestamp_max = exp_timestamp->max });
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_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_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_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; }