static void offset_commit_cb (rd_kafka_t *rk, rd_kafka_resp_err_t err, rd_kafka_topic_partition_list_t *offsets, void *opaque) { rd_kafka_topic_partition_t *rktpar; TEST_SAYL(3, "Offset committed: %s:\n", rd_kafka_err2str(err)); if (err == RD_KAFKA_RESP_ERR__NO_OFFSET) return; test_print_partition_list(offsets); if (err) TEST_FAIL("Offset commit failed: %s", rd_kafka_err2str(err)); if (offsets->cnt == 0) TEST_FAIL("Expected at least one partition in offset_commit_cb"); /* Find correct partition */ if (!(rktpar = rd_kafka_topic_partition_list_find(offsets, topic, partition))) return; if (rktpar->err) TEST_FAIL("Offset commit failed for partitioń : %s", rd_kafka_err2str(rktpar->err)); if (rktpar->offset > expected_offset) TEST_FAIL("Offset committed %"PRId64 " > expected offset %"PRId64, rktpar->offset, expected_offset); if (rktpar->offset <= committed_offset) TEST_FAIL("Old offset %"PRId64" (re)committed: " "should be above committed_offset %"PRId64, rktpar->offset, committed_offset); committed_offset = rktpar->offset; if (rktpar->offset < expected_offset) { TEST_SAYL(3, "Offset committed %"PRId64 " < expected offset %"PRId64"\n", rktpar->offset, expected_offset); return; } TEST_SAYL(3, "Expected offset committed: %"PRId64"\n", rktpar->offset); }
static void dr_msg_cb (rd_kafka_t *rk, const rd_kafka_message_t *rkmessage, void *opaque) { msg_dr_cnt++; TEST_SAYL(3, "Delivery for message %.*s: %s\n", (int)rkmessage->len, (const char *)rkmessage->payload, rd_kafka_err2name(rkmessage->err)); if (rkmessage->err) { TEST_FAIL_LATER("Expected message to succeed, got %s", rd_kafka_err2str(rkmessage->err)); msg_dr_fail_cnt++; } }
static rd_kafka_resp_err_t on_commit ( rd_kafka_t *rk, const rd_kafka_topic_partition_list_t *offsets, rd_kafka_resp_err_t err, void *ic_opaque) { int ic_id = (int)(intptr_t)ic_opaque; /* Since on_commit is triggered a bit randomly and not per * message we only try to make sure it gets fully set at least once. */ TEST_ASSERT(ic_opaque != NULL); /* Verify opaque (base | ic id) */ TEST_ASSERT((ic_id & base_mask) == on_commit_base); ic_id &= ~base_mask; TEST_ASSERT(ic_opaque != NULL); TEST_SAYL(3, "on_commit: interceptor #%d called: %s\n", ic_id, rd_kafka_err2str(err)); if (test_level >= 4) test_print_partition_list(offsets); /* Check for rollover where a previous on_commit stint was * succesful and it just now started over */ if (on_commit_bits > 0 && ic_id == 0) { /* Verify completeness of previous stint */ verify_ic_cnt("on_commit", on_commit_bits, consumer_ic_cnt); /* Reset */ on_commit_bits = 0; } verify_ic_cnt("on_commit", on_commit_bits, ic_id); /* Set this interceptor's bit */ on_commit_bits |= 1 << ic_id; return RD_KAFKA_RESP_ERR_NO_ERROR; }
static void verify_msg (const char *what, int base, int bitid, rd_kafka_message_t *rkmessage, void *ic_opaque) { const char *id_str = rkmessage->key; struct msg_state *msg; int id; int ic_id = (int)(intptr_t)ic_opaque; /* Verify opaque (base | ic id) */ TEST_ASSERT((ic_id & base_mask) == base); ic_id &= ~base_mask; /* Find message by id */ TEST_ASSERT(rkmessage->key && rkmessage->key_len > 0 && id_str[(int)rkmessage->key_len-1] == '\0' && strlen(id_str) > 0 && isdigit(*id_str)); id = atoi(id_str); TEST_ASSERT(id >= 0 && id < msgcnt, "%s: bad message id %s", what, id_str); msg = &msgs[id]; TEST_ASSERT(msg->id == id, "expected msg #%d has wrong id %d", id, msg->id); /* Verify message opaque */ if (!strcmp(what, "on_send") || !strncmp(what, "on_ack", 6)) TEST_ASSERT(rkmessage->_private == (void *)msg); TEST_SAYL(3, "%s: interceptor #%d called for message #%d (%d)\n", what, ic_id, id, msg->id); msg_verify_ic_cnt(msg, what, msg->bits[bitid], ic_id); /* Set this interceptor's bit */ msg->bits[bitid] |= 1 << ic_id; }
static void do_offset_test (const char *what, int auto_commit, int auto_store, int async) { test_timing_t t_all; char groupid[64]; rd_kafka_t *rk; rd_kafka_conf_t *conf; rd_kafka_topic_conf_t *tconf; int cnt = 0; const int extra_cnt = 5; rd_kafka_resp_err_t err; rd_kafka_topic_partition_list_t *parts; rd_kafka_topic_partition_t *rktpar; int64_t next_offset = -1; test_conf_init(&conf, &tconf, 20); test_conf_set(conf, "enable.auto.commit", auto_commit ? "true":"false"); test_conf_set(conf, "enable.auto.offset.store", auto_store ?"true":"false"); test_conf_set(conf, "auto.commit.interval.ms", "500"); rd_kafka_conf_set_offset_commit_cb(conf, offset_commit_cb); test_topic_conf_set(tconf, "auto.offset.reset", "smallest"); test_str_id_generate(groupid, sizeof(groupid)); test_conf_set(conf, "group.id", groupid); rd_kafka_conf_set_default_topic_conf(conf, tconf); TEST_SAY(_C_MAG "[ do_offset_test: %s with group.id %s ]\n", what, groupid); TIMING_START(&t_all, what); expected_offset = 0; committed_offset = -1; /* MO: * - Create consumer. * - Start consuming from beginning * - Perform store & commits according to settings * - Stop storing&committing when half of the messages are consumed, * - but consume 5 more to check against. * - Query position. * - Destroy consumer. * - Create new consumer with same group.id using stored offsets * - Should consume the expected message. */ /* Create kafka instance */ rk = test_create_handle(RD_KAFKA_CONSUMER, rd_kafka_conf_dup(conf)); rd_kafka_poll_set_consumer(rk); test_consumer_subscribe(rk, topic); while (cnt - extra_cnt < msgcnt / 2) { rd_kafka_message_t *rkm; rkm = rd_kafka_consumer_poll(rk, 10*1000); if (!rkm) continue; if (rkm->err == RD_KAFKA_RESP_ERR__TIMED_OUT) TEST_FAIL("%s: Timed out waiting for message %d", what,cnt); else if (rkm->err == RD_KAFKA_RESP_ERR__PARTITION_EOF) { rd_kafka_message_destroy(rkm); continue; } else if (rkm->err) TEST_FAIL("%s: Consumer error: %s", what, rd_kafka_message_errstr(rkm)); /* Offset of next message. */ next_offset = rkm->offset + 1; if (cnt < msgcnt / 2) { if (!auto_store) { err = rd_kafka_offset_store(rkm->rkt,rkm->partition, rkm->offset); if (err) TEST_FAIL("%s: offset_store failed: %s\n", what, rd_kafka_err2str(err)); } expected_offset = rkm->offset+1; if (!auto_commit) { test_timing_t t_commit; TIMING_START(&t_commit, async?"commit.async":"commit.sync"); err = rd_kafka_commit_message(rk, rkm, async); TIMING_STOP(&t_commit); if (err) TEST_FAIL("%s: commit failed: %s\n", what, rd_kafka_err2str(err)); } } else if (auto_store && auto_commit) expected_offset = rkm->offset+1; rd_kafka_message_destroy(rkm); cnt++; } TEST_SAY("%s: done consuming after %d messages, at offset %"PRId64"\n", what, cnt, expected_offset); if ((err = rd_kafka_assignment(rk, &parts))) TEST_FAIL("%s: failed to get assignment(): %s\n", what, rd_kafka_err2str(err)); /* Verify position */ if ((err = rd_kafka_position(rk, parts))) TEST_FAIL("%s: failed to get position(): %s\n", what, rd_kafka_err2str(err)); if (!(rktpar = rd_kafka_topic_partition_list_find(parts, topic, partition))) TEST_FAIL("%s: position(): topic lost\n", what); if (rktpar->offset != next_offset) TEST_FAIL("%s: Expected position() offset %"PRId64", got %"PRId64, what, next_offset, rktpar->offset); TEST_SAY("%s: Position is at %"PRId64", good!\n", what, rktpar->offset); /* Pause messages while waiting so we can serve callbacks * without having more messages received. */ if ((err = rd_kafka_pause_partitions(rk, parts))) TEST_FAIL("%s: failed to pause partitions: %s\n", what, rd_kafka_err2str(err)); rd_kafka_topic_partition_list_destroy(parts); /* Fire off any enqueued offset_commit_cb */ test_consumer_poll_no_msgs(what, rk, testid, 0); TEST_SAY("%s: committed_offset %"PRId64", expected_offset %"PRId64"\n", what, committed_offset, expected_offset); if (!auto_commit && !async) { /* Sync commits should be up to date at this point. */ if (committed_offset != expected_offset) TEST_FAIL("%s: Sync commit: committed offset %"PRId64 " should be same as expected offset " "%"PRId64, what, committed_offset, expected_offset); } else { /* Wait for offset commits to catch up */ while (committed_offset < expected_offset) { TEST_SAYL(3, "%s: Wait for committed offset %"PRId64 " to reach expected offset %"PRId64"\n", what, committed_offset, expected_offset); test_consumer_poll_no_msgs(what, rk, testid, 1000); } } TEST_SAY("%s: phase 1 complete, %d messages consumed, " "next expected offset is %"PRId64"\n", what, cnt, expected_offset); /* Issue #827: cause committed() to return prematurely by specifying * low timeout. The bug (use after free) will only * be catched by valgrind. */ do { parts = rd_kafka_topic_partition_list_new(1); rd_kafka_topic_partition_list_add(parts, topic, partition); err = rd_kafka_committed(rk, parts, 1); rd_kafka_topic_partition_list_destroy(parts); TEST_SAY("Issue #827: committed() returned %s\n", rd_kafka_err2str(err)); } while (err != RD_KAFKA_RESP_ERR__TIMED_OUT); /* Query position */ parts = rd_kafka_topic_partition_list_new(1); rd_kafka_topic_partition_list_add(parts, topic, partition); err = rd_kafka_committed(rk, parts, tmout_multip(5*1000)); if (err) TEST_FAIL("%s: committed() failed: %s", what, rd_kafka_err2str(err)); if (!(rktpar = rd_kafka_topic_partition_list_find(parts, topic, partition))) TEST_FAIL("%s: committed(): topic lost\n", what); if (rktpar->offset != expected_offset) TEST_FAIL("%s: Expected committed() offset %"PRId64", got %"PRId64, what, expected_offset, rktpar->offset); TEST_SAY("%s: Committed offset is at %"PRId64", good!\n", what, rktpar->offset); rd_kafka_topic_partition_list_destroy(parts); test_consumer_close(rk); rd_kafka_destroy(rk); /* Fire up a new consumer and continue from where we left off. */ TEST_SAY("%s: phase 2: starting new consumer to resume consumption\n",what); rk = test_create_handle(RD_KAFKA_CONSUMER, conf); rd_kafka_poll_set_consumer(rk); test_consumer_subscribe(rk, topic); while (cnt < msgcnt) { rd_kafka_message_t *rkm; rkm = rd_kafka_consumer_poll(rk, 10*1000); if (!rkm) continue; if (rkm->err == RD_KAFKA_RESP_ERR__TIMED_OUT) TEST_FAIL("%s: Timed out waiting for message %d", what,cnt); else if (rkm->err == RD_KAFKA_RESP_ERR__PARTITION_EOF) { rd_kafka_message_destroy(rkm); continue; } else if (rkm->err) TEST_FAIL("%s: Consumer error: %s", what, rd_kafka_message_errstr(rkm)); if (rkm->offset != expected_offset) TEST_FAIL("%s: Received message offset %"PRId64 ", expected %"PRId64" at msgcnt %d/%d\n", what, rkm->offset, expected_offset, cnt, msgcnt); rd_kafka_message_destroy(rkm); expected_offset++; cnt++; } TEST_SAY("%s: phase 2: complete\n", what); test_consumer_close(rk); rd_kafka_destroy(rk); TIMING_STOP(&t_all); }
/** * @brief Test AdminOptions */ static void do_test_options (rd_kafka_t *rk) { #define _all_apis { RD_KAFKA_ADMIN_OP_CREATETOPICS, \ RD_KAFKA_ADMIN_OP_DELETETOPICS, \ RD_KAFKA_ADMIN_OP_CREATEPARTITIONS, \ RD_KAFKA_ADMIN_OP_ALTERCONFIGS, \ RD_KAFKA_ADMIN_OP_DESCRIBECONFIGS, \ RD_KAFKA_ADMIN_OP_ANY /* Must be last */} struct { const char *setter; const rd_kafka_admin_op_t valid_apis[8]; } matrix[] = { { "request_timeout", _all_apis }, { "operation_timeout", { RD_KAFKA_ADMIN_OP_CREATETOPICS, RD_KAFKA_ADMIN_OP_DELETETOPICS, RD_KAFKA_ADMIN_OP_CREATEPARTITIONS } }, { "validate_only", { RD_KAFKA_ADMIN_OP_CREATETOPICS, RD_KAFKA_ADMIN_OP_CREATEPARTITIONS, RD_KAFKA_ADMIN_OP_ALTERCONFIGS } }, { "broker", _all_apis }, { "opaque", _all_apis }, { NULL }, }; int i; rd_kafka_AdminOptions_t *options; for (i = 0 ; matrix[i].setter ; i++) { static const rd_kafka_admin_op_t all_apis[] = _all_apis; const rd_kafka_admin_op_t *for_api; for (for_api = all_apis ; ; for_api++) { rd_kafka_resp_err_t err = RD_KAFKA_RESP_ERR_NO_ERROR; rd_kafka_resp_err_t exp_err = RD_KAFKA_RESP_ERR_NO_ERROR; char errstr[512]; int fi; options = rd_kafka_AdminOptions_new(rk, *for_api); TEST_ASSERT(options, "AdminOptions_new(%d) failed", *for_api); if (!strcmp(matrix[i].setter, "request_timeout")) err = rd_kafka_AdminOptions_set_request_timeout( options, 1234, errstr, sizeof(errstr)); else if (!strcmp(matrix[i].setter, "operation_timeout")) err = rd_kafka_AdminOptions_set_operation_timeout( options, 12345, errstr, sizeof(errstr)); else if (!strcmp(matrix[i].setter, "validate_only")) err = rd_kafka_AdminOptions_set_validate_only( options, 1, errstr, sizeof(errstr)); else if (!strcmp(matrix[i].setter, "broker")) err = rd_kafka_AdminOptions_set_broker( options, 5, errstr, sizeof(errstr)); else if (!strcmp(matrix[i].setter, "opaque")) { rd_kafka_AdminOptions_set_opaque( options, (void *)options); err = RD_KAFKA_RESP_ERR_NO_ERROR; } else TEST_FAIL("Invalid setter: %s", matrix[i].setter); TEST_SAYL(3, "AdminOptions_set_%s on " "RD_KAFKA_ADMIN_OP_%d options " "returned %s: %s\n", matrix[i].setter, *for_api, rd_kafka_err2name(err), err ? errstr : "success"); /* Scan matrix valid_apis to see if this * setter should be accepted or not. */ if (exp_err) { /* An expected error is already set */ } else if (*for_api != RD_KAFKA_ADMIN_OP_ANY) { exp_err = RD_KAFKA_RESP_ERR__INVALID_ARG; for (fi = 0 ; matrix[i].valid_apis[fi] ; fi++) { if (matrix[i].valid_apis[fi] == *for_api) exp_err = RD_KAFKA_RESP_ERR_NO_ERROR; } } else { exp_err = RD_KAFKA_RESP_ERR_NO_ERROR; } if (err != exp_err) TEST_FAIL_LATER("Expected AdminOptions_set_%s " "for RD_KAFKA_ADMIN_OP_%d " "options to return %s, " "not %s", matrix[i].setter, *for_api, rd_kafka_err2name(exp_err), rd_kafka_err2name(err)); rd_kafka_AdminOptions_destroy(options); if (*for_api == RD_KAFKA_ADMIN_OP_ANY) break; /* This was the last one */ } } /* Try an invalid for_api */ options = rd_kafka_AdminOptions_new(rk, (rd_kafka_admin_op_t)1234); TEST_ASSERT(!options, "Expectred AdminOptions_new() to fail " "with an invalid for_api, didn't."); TEST_LATER_CHECK(); }
static void expect_check (const char *what, const struct expect *expected, const rd_kafka_message_t *rkmessage) { const struct expect *exp; rd_kafka_resp_err_t err; size_t idx = 0; const char *name; const char *value; size_t size; rd_kafka_headers_t *hdrs; int msgid; if (rkmessage->len != sizeof(msgid)) TEST_FAIL("%s: expected message len %"PRIusz" == sizeof(int)", what, rkmessage->len); memcpy(&msgid, rkmessage->payload, rkmessage->len); if ((err = rd_kafka_message_headers(rkmessage, &hdrs))) { if (msgid == 0) { TEST_SAYL(3, "%s: Msg #%d: no headers, good\n", what, msgid); return; /* No headers expected for first message */ } TEST_FAIL("%s: Expected headers in message %d: %s", what, msgid, rd_kafka_err2str(err)); } else { TEST_ASSERT(msgid != 0, "%s: first message should have no headers", what); } test_headers_dump(what, 3, hdrs); for (idx = 0, exp = expected ; !rd_kafka_header_get_all(hdrs, idx, &name, (const void **)&value, &size) ; idx++, exp++) { TEST_SAYL(3, "%s: Msg #%d: " "Header #%"PRIusz": %s='%s' (expecting %s='%s')\n", what, msgid, idx, name, value ? value : "(NULL)", exp->name, exp->value ? exp->value : "(NULL)"); if (strcmp(name, exp->name)) TEST_FAIL("%s: Msg #%d: " "Expected header %s at idx #%"PRIusz ", not '%s' (%"PRIusz")", what, msgid, exp->name, idx, name, strlen(name)); if (!strcmp(name, "msgid")) { int vid; /* Special handling: compare msgid header value * to message body, should be identical */ if (size != rkmessage->len || size != sizeof(int)) TEST_FAIL("%s: " "Expected msgid/int-sized payload " "%"PRIusz", got %"PRIusz, what, size, rkmessage->len); /* Copy to avoid unaligned access (by cast) */ memcpy(&vid, value, size); if (vid != msgid) TEST_FAIL("%s: Header msgid %d != payload %d", what, vid, msgid); if (exp_msgid != vid) TEST_FAIL("%s: Expected msgid %d, not %d", what, exp_msgid, vid); continue; } if (!exp->value) { /* Expected NULL value */ TEST_ASSERT(!value, "%s: Expected NULL value for %s, got %s", what, exp->name, value); } else { TEST_ASSERT(value, "%s: " "Expected non-NULL value for %s, got NULL", what, exp->name); TEST_ASSERT(size == strlen(exp->value), "%s: Expected size %"PRIusz" for %s, " "not %"PRIusz, what, strlen(exp->value), exp->name, size); TEST_ASSERT(value[size] == '\0', "%s: " "Expected implicit null-terminator for %s", what, exp->name); TEST_ASSERT(!strcmp(exp->value, value), "%s: " "Expected value %s for %s, not %s", what, exp->value, exp->name, value); } } TEST_ASSERT(exp->name == NULL, "%s: Expected the expected, but stuck at %s which was " "unexpected", what, exp->name); }
/** * @brief Test delivery report events */ int main_0039_event_dr (int argc, char **argv) { int partition = 0; int r; rd_kafka_t *rk; rd_kafka_topic_t *rkt; rd_kafka_conf_t *conf; rd_kafka_topic_conf_t *topic_conf; char msg[128]; int msgcnt = test_on_ci ? 5000 : 50000; int i; test_timing_t t_produce, t_delivery; rd_kafka_queue_t *eventq; test_conf_init(&conf, &topic_conf, 10); /* Set delivery report callback */ rd_kafka_conf_set_dr_msg_cb(conf, test_dr_msg_cb); rd_kafka_conf_set_events(conf, RD_KAFKA_EVENT_DR); /* Create kafka instance */ rk = test_create_handle(RD_KAFKA_PRODUCER, conf); eventq = rd_kafka_queue_get_main(rk); rkt = rd_kafka_topic_new(rk, test_mk_topic_name("0005", 0), topic_conf); if (!rkt) TEST_FAIL("Failed to create topic: %s\n", rd_strerror(errno)); /* Produce messages */ TIMING_START(&t_produce, "PRODUCE"); for (i = 0 ; i < msgcnt ; i++) { int *msgidp = malloc(sizeof(*msgidp)); *msgidp = i; rd_snprintf(msg, sizeof(msg), "%s test message #%i", argv[0], i); r = rd_kafka_produce(rkt, partition, RD_KAFKA_MSG_F_COPY, msg, strlen(msg), NULL, 0, msgidp); if (r == -1) TEST_FAIL("Failed to produce message #%i: %s\n", i, rd_strerror(errno)); } TIMING_STOP(&t_produce); TEST_SAY("Produced %i messages, waiting for deliveries\n", msgcnt); /* Wait for messages to be delivered */ TIMING_START(&t_delivery, "DELIVERY"); while (rd_kafka_outq_len(rk) > 0) { rd_kafka_event_t *rkev; rkev = rd_kafka_queue_poll(eventq, 1000); switch (rd_kafka_event_type(rkev)) { case RD_KAFKA_EVENT_DR: TEST_SAYL(3, "%s event with %zd messages\n", rd_kafka_event_name(rkev), rd_kafka_event_message_count(rkev)); handle_drs(rkev); break; default: TEST_SAY("Unhandled event: %s\n", rd_kafka_event_name(rkev)); break; } rd_kafka_event_destroy(rkev); } TIMING_STOP(&t_delivery); if (fails) TEST_FAIL("%i failures, see previous errors", fails); if (msgid_next != msgcnt) TEST_FAIL("Still waiting for messages: next %i != end %i\n", msgid_next, msgcnt); rd_kafka_queue_destroy(eventq); /* Destroy topic */ rd_kafka_topic_destroy(rkt); /* Destroy rdkafka instance */ TEST_SAY("Destroying kafka instance %s\n", rd_kafka_name(rk)); rd_kafka_destroy(rk); return 0; }