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); }
static rd_kafka_resp_err_t rd_kafka_offset_broker_commit (rd_kafka_toppar_t *rktp) { rd_kafka_topic_partition_list_t *offsets; rd_kafka_topic_partition_t *rktpar; rd_kafka_assert(rktp->rktp_rkt->rkt_rk, rktp->rktp_cgrp != NULL); rd_kafka_assert(rktp->rktp_rkt->rkt_rk, rktp->rktp_flags & RD_KAFKA_TOPPAR_F_OFFSET_STORE); rktp->rktp_committing_offset = rktp->rktp_stored_offset; offsets = rd_kafka_topic_partition_list_new(1); rktpar = rd_kafka_topic_partition_list_add( offsets, rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition); rktpar->offset = rktp->rktp_committing_offset; rd_kafka_dbg(rktp->rktp_rkt->rkt_rk, TOPIC, "OFFSETCMT", "%.*s [%"PRId32"]: committing offset %"PRId64, RD_KAFKAP_STR_PR(rktp->rktp_rkt->rkt_topic), rktp->rktp_partition, rktp->rktp_committing_offset); rd_kafka_commit0(rktp->rktp_rkt->rkt_rk, offsets, rktp, RD_KAFKA_REPLYQ(rktp->rktp_ops, 0), rd_kafka_offset_broker_commit_cb, NULL); rd_kafka_topic_partition_list_destroy(offsets); return RD_KAFKA_RESP_ERR__IN_PROGRESS; }
rd_kafka_resp_err_t rd_kafka_assignment (rd_kafka_t *rk, rd_kafka_topic_partition_list_t **partitions) { rd_kafka_op_t *rko; rd_kafka_resp_err_t err; rd_kafka_cgrp_t *rkcg; if (!(rkcg = rd_kafka_cgrp_get(rk))) return RD_KAFKA_RESP_ERR__UNKNOWN_GROUP; rko = rd_kafka_op_req2(rkcg->rkcg_ops, RD_KAFKA_OP_GET_ASSIGNMENT); if (!rko) return RD_KAFKA_RESP_ERR__TIMED_OUT; err = rko->rko_err; *partitions = rko->rko_u.assign.partitions; rko->rko_u.assign.partitions = NULL; rd_kafka_op_destroy(rko); if (!*partitions && !err) { /* Create an empty list for convenience of the caller */ *partitions = rd_kafka_topic_partition_list_new(0); } return err; }
rd_kafka_resp_err_t rd_kafka_subscription (rd_kafka_t *rk, rd_kafka_topic_partition_list_t **topics){ rd_kafka_op_t *rko; rd_kafka_resp_err_t err; rd_kafka_cgrp_t *rkcg; if (!(rkcg = rd_kafka_cgrp_get(rk))) return RD_KAFKA_RESP_ERR__UNKNOWN_GROUP; rko = rd_kafka_op_req2(rkcg->rkcg_ops, RD_KAFKA_OP_GET_SUBSCRIPTION); if (!rko) return RD_KAFKA_RESP_ERR__TIMED_OUT; err = rko->rko_err; *topics = rko->rko_u.subscribe.topics; rko->rko_u.subscribe.topics = NULL; rd_kafka_op_destroy(rko); if (!*topics && !err) { /* Create an empty list for convenience of the caller */ *topics = rd_kafka_topic_partition_list_new(0); } return err; }
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); }
RdKafka::ErrorCode RdKafka::KafkaConsumerImpl::subscribe (const std::vector<std::string> &topics) { rd_kafka_topic_partition_list_t *c_topics; rd_kafka_resp_err_t err; c_topics = rd_kafka_topic_partition_list_new((int)topics.size()); for (unsigned int i = 0 ; i < topics.size() ; i++) rd_kafka_topic_partition_list_add(c_topics, topics[i].c_str(), RD_KAFKA_PARTITION_UA); err = rd_kafka_subscribe(rk_, c_topics); rd_kafka_topic_partition_list_destroy(c_topics); return static_cast<RdKafka::ErrorCode>(err); }
rd_kafka_topic_partition_list_t * partitions_to_c_parts (const std::vector<RdKafka::TopicPartition*> &partitions) { rd_kafka_topic_partition_list_t *c_parts; c_parts = rd_kafka_topic_partition_list_new((int)partitions.size()); for (unsigned int i = 0 ; i < partitions.size() ; i++) { const RdKafka::TopicPartitionImpl *tpi = dynamic_cast<const RdKafka::TopicPartitionImpl*>(partitions[i]); rd_kafka_topic_partition_t *rktpar = rd_kafka_topic_partition_list_add(c_parts, tpi->topic_.c_str(), tpi->partition_); rktpar->offset = tpi->offset_; } return c_parts; }
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); }
/** * 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); }
rd_kafka_resp_err_t rd_kafka_commit_message (rd_kafka_t *rk, const rd_kafka_message_t *rkmessage, int async) { rd_kafka_topic_partition_list_t *offsets; rd_kafka_topic_partition_t *rktpar; rd_kafka_resp_err_t err; if (rkmessage->err) return RD_KAFKA_RESP_ERR__INVALID_ARG; offsets = rd_kafka_topic_partition_list_new(1); rktpar = rd_kafka_topic_partition_list_add( offsets, rd_kafka_topic_name(rkmessage->rkt), rkmessage->partition); rktpar->offset = rkmessage->offset+1; err = rd_kafka_commit(rk, offsets, async); rd_kafka_topic_partition_list_destroy(offsets); return err; }
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); }
static PyObject *Consumer_store_offsets (Handle *self, PyObject *args, PyObject *kwargs) { #if RD_KAFKA_VERSION < 0x000b0000 PyErr_Format(PyExc_NotImplementedError, "Consumer store_offsets require " "confluent-kafka-python built for librdkafka " "version >=v0.11.0 (librdkafka runtime 0x%x, " "buildtime 0x%x)", rd_kafka_version(), RD_KAFKA_VERSION); return NULL; #else rd_kafka_resp_err_t err; PyObject *msg = NULL, *offsets = NULL; rd_kafka_topic_partition_list_t *c_offsets; static char *kws[] = { "message", "offsets", NULL }; if (!self->rk) { PyErr_SetString(PyExc_RuntimeError, "Consumer closed"); return NULL; } if (!PyArg_ParseTupleAndKeywords(args, kwargs, "|OO", kws, &msg, &offsets)) return NULL; if (msg && offsets) { PyErr_SetString(PyExc_ValueError, "message and offsets are mutually exclusive"); return NULL; } if (!msg && !offsets) { PyErr_SetString(PyExc_ValueError, "expected either message or offsets"); return NULL; } if (offsets) { if (!(c_offsets = py_to_c_parts(offsets))) return NULL; } else { Message *m; PyObject *uo8; if (PyObject_Type((PyObject *)msg) != (PyObject *)&MessageType) { PyErr_Format(PyExc_TypeError, "expected %s", MessageType.tp_name); return NULL; } m = (Message *)msg; c_offsets = rd_kafka_topic_partition_list_new(1); rd_kafka_topic_partition_list_add( c_offsets, cfl_PyUnistr_AsUTF8(m->topic, &uo8), m->partition)->offset = m->offset + 1; Py_XDECREF(uo8); } err = rd_kafka_offsets_store(self->rk, c_offsets); rd_kafka_topic_partition_list_destroy(c_offsets); if (err) { cfl_PyErr_Format(err, "StoreOffsets failed: %s", rd_kafka_err2str(err)); return NULL; } Py_RETURN_NONE; #endif }
/* @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); }
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_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; }
int main_0006_symbols (int argc, char **argv) { if (argc < 0 /* always false */) { rd_kafka_version(); rd_kafka_version_str(); rd_kafka_get_debug_contexts(); rd_kafka_get_err_descs(NULL, NULL); rd_kafka_err2str(RD_KAFKA_RESP_ERR_NO_ERROR); rd_kafka_err2name(RD_KAFKA_RESP_ERR_NO_ERROR); rd_kafka_errno2err(EINVAL); rd_kafka_errno(); rd_kafka_last_error(); rd_kafka_conf_new(); rd_kafka_conf_destroy(NULL); rd_kafka_conf_dup(NULL); rd_kafka_conf_set(NULL, NULL, NULL, NULL, 0); rd_kafka_conf_set_dr_cb(NULL, NULL); rd_kafka_conf_set_dr_msg_cb(NULL, NULL); rd_kafka_conf_set_error_cb(NULL, NULL); rd_kafka_conf_set_stats_cb(NULL, NULL); rd_kafka_conf_set_log_cb(NULL, NULL); rd_kafka_conf_set_socket_cb(NULL, NULL); rd_kafka_conf_set_rebalance_cb(NULL, NULL); rd_kafka_conf_set_offset_commit_cb(NULL, NULL); rd_kafka_conf_set_throttle_cb(NULL, NULL); rd_kafka_conf_set_default_topic_conf(NULL, NULL); rd_kafka_conf_get(NULL, NULL, NULL, NULL); #ifndef _MSC_VER rd_kafka_conf_set_open_cb(NULL, NULL); #endif rd_kafka_conf_set_opaque(NULL, NULL); rd_kafka_opaque(NULL); rd_kafka_conf_dump(NULL, NULL); rd_kafka_topic_conf_dump(NULL, NULL); rd_kafka_conf_dump_free(NULL, 0); rd_kafka_conf_properties_show(NULL); rd_kafka_topic_conf_new(); rd_kafka_topic_conf_dup(NULL); rd_kafka_topic_conf_destroy(NULL); rd_kafka_topic_conf_set(NULL, NULL, NULL, NULL, 0); rd_kafka_topic_conf_set_opaque(NULL, NULL); rd_kafka_topic_conf_get(NULL, NULL, NULL, NULL); rd_kafka_topic_conf_set_partitioner_cb(NULL, NULL); rd_kafka_topic_partition_available(NULL, 0); rd_kafka_topic_opaque(NULL); rd_kafka_msg_partitioner_random(NULL, NULL, 0, 0, NULL, NULL); rd_kafka_msg_partitioner_consistent(NULL, NULL, 0, 0, NULL, NULL); rd_kafka_msg_partitioner_consistent_random(NULL, NULL, 0, 0, NULL, NULL); rd_kafka_new(0, NULL, NULL, 0); rd_kafka_destroy(NULL); rd_kafka_name(NULL); rd_kafka_memberid(NULL); rd_kafka_topic_new(NULL, NULL, NULL); rd_kafka_topic_destroy(NULL); rd_kafka_topic_name(NULL); rd_kafka_message_destroy(NULL); rd_kafka_message_errstr(NULL); rd_kafka_message_timestamp(NULL, NULL); rd_kafka_consume_start(NULL, 0, 0); rd_kafka_consume_stop(NULL, 0); rd_kafka_consume(NULL, 0, 0); rd_kafka_consume_batch(NULL, 0, 0, NULL, 0); rd_kafka_consume_callback(NULL, 0, 0, NULL, NULL); rd_kafka_offset_store(NULL, 0, 0); rd_kafka_produce(NULL, 0, 0, NULL, 0, NULL, 0, NULL); rd_kafka_produce_batch(NULL, 0, 0, NULL, 0); rd_kafka_poll(NULL, 0); rd_kafka_brokers_add(NULL, NULL); /* DEPRECATED: rd_kafka_set_logger(NULL, NULL); */ rd_kafka_set_log_level(NULL, 0); rd_kafka_log_print(NULL, 0, NULL, NULL); #ifndef _MSC_VER rd_kafka_log_syslog(NULL, 0, NULL, NULL); #endif rd_kafka_outq_len(NULL); rd_kafka_dump(NULL, NULL); rd_kafka_thread_cnt(); rd_kafka_wait_destroyed(0); rd_kafka_metadata(NULL, 0, NULL, NULL, 0); rd_kafka_metadata_destroy(NULL); rd_kafka_queue_destroy(NULL); rd_kafka_consume_start_queue(NULL, 0, 0, NULL); rd_kafka_consume_queue(NULL, 0); rd_kafka_consume_batch_queue(NULL, 0, NULL, 0); rd_kafka_consume_callback_queue(NULL, 0, NULL, NULL); rd_kafka_seek(NULL, 0, 0, 0); rd_kafka_yield(NULL); rd_kafka_mem_free(NULL, NULL); rd_kafka_list_groups(NULL, NULL, NULL, 0); rd_kafka_group_list_destroy(NULL); /* KafkaConsumer API */ rd_kafka_subscribe(NULL, NULL); rd_kafka_unsubscribe(NULL); rd_kafka_subscription(NULL, NULL); rd_kafka_consumer_poll(NULL, 0); rd_kafka_consumer_close(NULL); rd_kafka_assign(NULL, NULL); rd_kafka_assignment(NULL, NULL); rd_kafka_commit(NULL, NULL, 0); rd_kafka_commit_message(NULL, NULL, 0); rd_kafka_committed(NULL, NULL, 0); rd_kafka_position(NULL, NULL); /* TopicPartition */ rd_kafka_topic_partition_list_new(0); rd_kafka_topic_partition_list_destroy(NULL); rd_kafka_topic_partition_list_add(NULL, NULL, 0); rd_kafka_topic_partition_list_add_range(NULL, NULL, 0, 0); rd_kafka_topic_partition_list_del(NULL, NULL, 0); rd_kafka_topic_partition_list_del_by_idx(NULL, 0); rd_kafka_topic_partition_list_copy(NULL); rd_kafka_topic_partition_list_set_offset(NULL, NULL, 0, 0); rd_kafka_topic_partition_list_find(NULL, NULL, 0); rd_kafka_query_watermark_offsets(NULL, NULL, 0, NULL, NULL, 0); rd_kafka_get_watermark_offsets(NULL, NULL, 0, NULL, NULL); } 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; }
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); }
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 (int argc, char **argv) { char mode = 'C'; char *brokers = "localhost:9092"; int opt; rd_kafka_conf_t *conf; rd_kafka_topic_conf_t *topic_conf; char errstr[512]; const char *debug = NULL; int do_conf_dump = 0; char tmp[16]; rd_kafka_resp_err_t err; char *group = NULL; rd_kafka_topic_partition_list_t *topics; int i; quiet = !isatty(STDIN_FILENO); /* Kafka configuration */ conf = rd_kafka_conf_new(); /* Set logger */ rd_kafka_conf_set_log_cb(conf, logger); /* Quick termination */ snprintf(tmp, sizeof(tmp), "%i", SIGIO); rd_kafka_conf_set(conf, "internal.termination.signal", tmp, NULL, 0); /* Topic configuration */ topic_conf = rd_kafka_topic_conf_new(); while ((opt = getopt(argc, argv, "g:b:qd:eX:As:DO")) != -1) { switch (opt) { case 'b': brokers = optarg; break; case 'g': group = optarg; break; case 'e': exit_eof = 1; break; case 'd': debug = optarg; break; case 'q': quiet = 1; break; case 'A': output = OUTPUT_RAW; break; case 'X': { char *name, *val; rd_kafka_conf_res_t res; if (!strcmp(optarg, "list") || !strcmp(optarg, "help")) { rd_kafka_conf_properties_show(stdout); exit(0); } if (!strcmp(optarg, "dump")) { do_conf_dump = 1; continue; } name = optarg; if (!(val = strchr(name, '='))) { fprintf(stderr, "%% Expected " "-X property=value, not %s\n", name); exit(1); } *val = '\0'; val++; res = RD_KAFKA_CONF_UNKNOWN; /* Try "topic." prefixed properties on topic * conf first, and then fall through to global if * it didnt match a topic configuration property. */ if (!strncmp(name, "topic.", strlen("topic."))) res = rd_kafka_topic_conf_set(topic_conf, name+ strlen("topic."), val, errstr, sizeof(errstr)); if (res == RD_KAFKA_CONF_UNKNOWN) res = rd_kafka_conf_set(conf, name, val, errstr, sizeof(errstr)); if (res != RD_KAFKA_CONF_OK) { fprintf(stderr, "%% %s\n", errstr); exit(1); } } break; case 'D': case 'O': mode = opt; break; default: goto usage; } } if (do_conf_dump) { const char **arr; size_t cnt; int pass; for (pass = 0 ; pass < 2 ; pass++) { if (pass == 0) { arr = rd_kafka_conf_dump(conf, &cnt); printf("# Global config\n"); } else { printf("# Topic config\n"); arr = rd_kafka_topic_conf_dump(topic_conf, &cnt); } for (i = 0 ; i < (int)cnt ; i += 2) printf("%s = %s\n", arr[i], arr[i+1]); printf("\n"); rd_kafka_conf_dump_free(arr, cnt); } exit(0); } if (strchr("OC", mode) && optind == argc) { usage: fprintf(stderr, "Usage: %s [options] <topic[:part]> <topic[:part]>..\n" "\n" "librdkafka version %s (0x%08x)\n" "\n" " Options:\n" " -g <group> Consumer group (%s)\n" " -b <brokers> Broker address (%s)\n" " -e Exit consumer when last message\n" " in partition has been received.\n" " -D Describe group.\n" " -O Get commmitted offset(s)\n" " -d [facs..] Enable debugging contexts:\n" " %s\n" " -q Be quiet\n" " -A Raw payload output (consumer)\n" " -X <prop=name> Set arbitrary librdkafka " "configuration property\n" " Properties prefixed with \"topic.\" " "will be set on topic object.\n" " Use '-X list' to see the full list\n" " of supported properties.\n" "\n" "\n", argv[0], rd_kafka_version_str(), rd_kafka_version(), group, brokers, RD_KAFKA_DEBUG_CONTEXTS); exit(1); } signal(SIGINT, stop); signal(SIGUSR1, sig_usr1); if (debug && rd_kafka_conf_set(conf, "debug", debug, errstr, sizeof(errstr)) != RD_KAFKA_CONF_OK) { fprintf(stderr, "%% Debug configuration failed: %s: %s\n", errstr, debug); exit(1); } /* * Client/Consumer group */ if (strchr("CO", mode)) { /* Consumer groups require a group id */ if (!group) group = "rdkafka_consumer_example"; if (rd_kafka_conf_set(conf, "group.id", group, errstr, sizeof(errstr)) != RD_KAFKA_CONF_OK) { fprintf(stderr, "%% %s\n", errstr); exit(1); } /* Consumer groups always use broker based offset storage */ if (rd_kafka_topic_conf_set(topic_conf, "offset.store.method", "broker", errstr, sizeof(errstr)) != RD_KAFKA_CONF_OK) { fprintf(stderr, "%% %s\n", errstr); exit(1); } /* Set default topic config for pattern-matched topics. */ rd_kafka_conf_set_default_topic_conf(conf, topic_conf); /* Callback called on partition assignment changes */ rd_kafka_conf_set_rebalance_cb(conf, rebalance_cb); } /* Create Kafka handle */ if (!(rk = rd_kafka_new(RD_KAFKA_CONSUMER, conf, errstr, sizeof(errstr)))) { fprintf(stderr, "%% Failed to create new consumer: %s\n", errstr); exit(1); } rd_kafka_set_log_level(rk, LOG_DEBUG); /* Add brokers */ if (rd_kafka_brokers_add(rk, brokers) == 0) { fprintf(stderr, "%% No valid brokers specified\n"); exit(1); } if (mode == 'D') { int r; /* Describe groups */ r = describe_groups(rk, group); rd_kafka_destroy(rk); exit(r == -1 ? 1 : 0); } /* Redirect rd_kafka_poll() to consumer_poll() */ rd_kafka_poll_set_consumer(rk); topics = rd_kafka_topic_partition_list_new(argc - optind); for (i = optind ; i < argc ; i++) { /* Parse "topic[:part] */ char *topic = argv[i]; char *t; int32_t partition = -1; if ((t = strstr(topic, ":"))) { *t = '\0'; partition = atoi(t+1); } rd_kafka_topic_partition_list_add(topics, topic, partition); } if (mode == 'O') { /* Offset query */ err = rd_kafka_position(rk, topics, 5000); if (err) { fprintf(stderr, "%% Failed to fetch offsets: %s\n", rd_kafka_err2str(err)); exit(1); } for (i = 0 ; i < topics->cnt ; i++) { rd_kafka_topic_partition_t *p = &topics->elems[i]; printf("Topic \"%s\" partition %"PRId32, p->topic, p->partition); if (p->err) printf(" error %s", rd_kafka_err2str(p->err)); else { printf(" offset %"PRId64"", p->offset); if (p->metadata_size) printf(" (%d bytes of metadata)", (int)p->metadata_size); } printf("\n"); } goto done; } if ((err = rd_kafka_subscribe(rk, topics))) { fprintf(stderr, "%% Failed to start consuming topics: %s\n", rd_kafka_err2str(err)); exit(1); } while (run) { rd_kafka_message_t *rkmessage; rkmessage = rd_kafka_consumer_poll(rk, 1000); if (rkmessage) { msg_consume(rkmessage, NULL); rd_kafka_message_destroy(rkmessage); } } done: err = rd_kafka_consumer_close(rk); if (err) fprintf(stderr, "%% Failed to close consumer: %s\n", rd_kafka_err2str(err)); else fprintf(stderr, "%% Consumer closed\n"); rd_kafka_topic_partition_list_destroy(topics); /* Destroy handle */ rd_kafka_destroy(rk); /* Let background threads clean up and terminate cleanly. */ run = 5; while (run-- > 0 && rd_kafka_wait_destroyed(1000) == -1) printf("Waiting for librdkafka to decommission\n"); if (run <= 0) rd_kafka_dump(stdout, 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; }
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; }
static PyObject *Consumer_commit (Handle *self, PyObject *args, PyObject *kwargs) { rd_kafka_resp_err_t err; PyObject *msg = NULL, *offsets = NULL, *async_o = NULL; rd_kafka_topic_partition_list_t *c_offsets; int async = 1; static char *kws[] = { "message", "offsets", "async", "asynchronous", NULL }; rd_kafka_queue_t *rkqu = NULL; struct commit_return commit_return; PyThreadState *thread_state; if (!self->rk) { PyErr_SetString(PyExc_RuntimeError, "Consumer closed"); return NULL; } if (!PyArg_ParseTupleAndKeywords(args, kwargs, "|OOOO", kws, &msg, &offsets, &async_o, &async_o)) return NULL; if (msg && offsets) { PyErr_SetString(PyExc_ValueError, "message and offsets are mutually exclusive"); return NULL; } if (async_o) async = PyObject_IsTrue(async_o); if (offsets) { if (!(c_offsets = py_to_c_parts(offsets))) return NULL; } else if (msg) { Message *m; PyObject *uo8; if (PyObject_Type((PyObject *)msg) != (PyObject *)&MessageType) { PyErr_Format(PyExc_TypeError, "expected %s", MessageType.tp_name); return NULL; } m = (Message *)msg; c_offsets = rd_kafka_topic_partition_list_new(1); rd_kafka_topic_partition_list_add( c_offsets, cfl_PyUnistr_AsUTF8(m->topic, &uo8), m->partition)->offset =m->offset + 1; Py_XDECREF(uo8); } else { c_offsets = NULL; } if (async) { /* Async mode: Use consumer queue for offset commit * served by consumer_poll() */ rkqu = self->u.Consumer.rkqu; } else { /* Sync mode: Let commit_queue() trigger the callback. */ memset(&commit_return, 0, sizeof(commit_return)); /* Unlock GIL while we are blocking. */ thread_state = PyEval_SaveThread(); } err = rd_kafka_commit_queue(self->rk, c_offsets, rkqu, async ? Consumer_offset_commit_cb : Consumer_offset_commit_return_cb, async ? (void *)self : (void *)&commit_return); if (c_offsets) rd_kafka_topic_partition_list_destroy(c_offsets); if (!async) { /* Re-lock GIL */ PyEval_RestoreThread(thread_state); /* Honour inner error (richer) from offset_commit_return_cb */ if (commit_return.err) err = commit_return.err; } if (err) { /* Outer error from commit_queue() */ if (!async && commit_return.c_parts) rd_kafka_topic_partition_list_destroy(commit_return.c_parts); cfl_PyErr_Format(err, "Commit failed: %s", rd_kafka_err2str(err)); return NULL; } if (async) { /* async commit returns None when commit is in progress */ Py_RETURN_NONE; } else { PyObject *plist; /* sync commit returns the topic,partition,offset,err list */ assert(commit_return.c_parts); plist = c_parts_to_py(commit_return.c_parts); rd_kafka_topic_partition_list_destroy(commit_return.c_parts); return plist; } }
static PyObject *Consumer_subscribe (Handle *self, PyObject *args, PyObject *kwargs) { rd_kafka_topic_partition_list_t *topics; static char *kws[] = { "topics", "on_assign", "on_revoke", NULL }; PyObject *tlist, *on_assign = NULL, *on_revoke = NULL; Py_ssize_t pos = 0; rd_kafka_resp_err_t err; if (!self->rk) { PyErr_SetString(PyExc_RuntimeError, "Consumer closed"); return NULL; } if (!PyArg_ParseTupleAndKeywords(args, kwargs, "O|OO", kws, &tlist, &on_assign, &on_revoke)) return NULL; if (!PyList_Check(tlist)) { PyErr_Format(PyExc_TypeError, "expected list of topic unicode strings"); return NULL; } if (on_assign && !PyCallable_Check(on_assign)) { PyErr_Format(PyExc_TypeError, "on_assign expects a callable"); return NULL; } if (on_revoke && !PyCallable_Check(on_revoke)) { PyErr_Format(PyExc_TypeError, "on_revoke expects a callable"); return NULL; } topics = rd_kafka_topic_partition_list_new((int)PyList_Size(tlist)); for (pos = 0 ; pos < PyList_Size(tlist) ; pos++) { PyObject *o = PyList_GetItem(tlist, pos); PyObject *uo, *uo8; if (!(uo = cfl_PyObject_Unistr(o))) { PyErr_Format(PyExc_TypeError, "expected list of unicode strings"); rd_kafka_topic_partition_list_destroy(topics); return NULL; } rd_kafka_topic_partition_list_add(topics, cfl_PyUnistr_AsUTF8(uo, &uo8), RD_KAFKA_PARTITION_UA); Py_XDECREF(uo8); Py_DECREF(uo); } err = rd_kafka_subscribe(self->rk, topics); rd_kafka_topic_partition_list_destroy(topics); if (err) { cfl_PyErr_Format(err, "Failed to set subscription: %s", rd_kafka_err2str(err)); return NULL; } /* * Update rebalance callbacks */ if (self->u.Consumer.on_assign) { Py_DECREF(self->u.Consumer.on_assign); self->u.Consumer.on_assign = NULL; } if (on_assign) { self->u.Consumer.on_assign = on_assign; Py_INCREF(self->u.Consumer.on_assign); } if (self->u.Consumer.on_revoke) { Py_DECREF(self->u.Consumer.on_revoke); self->u.Consumer.on_revoke = NULL; } if (on_revoke) { self->u.Consumer.on_revoke = on_revoke; Py_INCREF(self->u.Consumer.on_revoke); } Py_RETURN_NONE; }