static void consume_messages (uint64_t testid, const char *topic, int32_t partition, int msg_base, int batch_cnt, int msgcnt) { rd_kafka_t *rk; rd_kafka_topic_t *rkt; rd_kafka_conf_t *conf; rd_kafka_topic_conf_t *topic_conf; int i; test_conf_init(&conf, &topic_conf, 20); /* Create kafka instance */ rk = test_create_handle(RD_KAFKA_CONSUMER, conf); rkt = rd_kafka_topic_new(rk, topic, topic_conf); if (!rkt) TEST_FAIL("Failed to create topic: %s\n", rd_kafka_err2str(rd_kafka_last_error())); TEST_SAY("Consuming %i messages from partition %i\n", batch_cnt, partition); /* Consume messages */ if (rd_kafka_consume_start(rkt, partition, RD_KAFKA_OFFSET_TAIL(batch_cnt)) == -1) TEST_FAIL("consume_start(%i, -%i) failed: %s", (int)partition, batch_cnt, rd_kafka_err2str(rd_kafka_last_error())); for (i = 0 ; i < batch_cnt ; i++) { rd_kafka_message_t *rkmessage; rkmessage = rd_kafka_consume(rkt, partition, tmout_multip(5000)); if (!rkmessage) TEST_FAIL("Failed to consume message %i/%i from " "partition %i: %s", i, batch_cnt, (int)partition, rd_kafka_err2str(rd_kafka_last_error())); if (rkmessage->err) TEST_FAIL("Consume message %i/%i from partition %i " "has error: %s", i, batch_cnt, (int)partition, rd_kafka_err2str(rkmessage->err)); verify_consumed_msg(testid, partition, msg_base+i, rkmessage); rd_kafka_message_destroy(rkmessage); } rd_kafka_consume_stop(rkt, partition); /* 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); }
static void do_produce (const char *topic, int msgcnt) { rd_kafka_t *rk; rd_kafka_conf_t *conf; int i; rd_kafka_resp_err_t err; test_conf_init(&conf, NULL, 0); test_conf_set(conf, "acks", "all"); rd_kafka_conf_set_dr_msg_cb(conf, dr_msg_cb); rd_kafka_conf_interceptor_add_on_new(conf, __FILE__, on_new, NULL); rk = test_create_handle(RD_KAFKA_PRODUCER, conf); /* First message is without headers (negative testing) */ i = 0; err = rd_kafka_producev( rk, RD_KAFKA_V_TOPIC(topic), RD_KAFKA_V_PARTITION(0), RD_KAFKA_V_VALUE(&i, sizeof(i)), RD_KAFKA_V_MSGFLAGS(RD_KAFKA_MSG_F_COPY), RD_KAFKA_V_END); TEST_ASSERT(!err, "producev() failed: %s", rd_kafka_err2str(err)); exp_msgid++; for (i = 1 ; i < msgcnt ; i++, exp_msgid++) { err = rd_kafka_producev( rk, RD_KAFKA_V_TOPIC(topic), RD_KAFKA_V_PARTITION(0), RD_KAFKA_V_VALUE(&i, sizeof(i)), RD_KAFKA_V_MSGFLAGS(RD_KAFKA_MSG_F_COPY), RD_KAFKA_V_HEADER("msgid", &i, sizeof(i)), RD_KAFKA_V_HEADER("static", "hey", -1), RD_KAFKA_V_HEADER("multi", "multi1", -1), RD_KAFKA_V_HEADER("multi", "multi2", 6), RD_KAFKA_V_HEADER("multi", "multi3", strlen("multi3")), RD_KAFKA_V_HEADER("null", NULL, 0), RD_KAFKA_V_HEADER("empty", "", 0), RD_KAFKA_V_END); TEST_ASSERT(!err, "producev() failed: %s", rd_kafka_err2str(err)); } /* Reset expected message id for dr */ exp_msgid = 0; /* Wait for timeouts and delivery reports */ rd_kafka_flush(rk, tmout_multip(5000)); rd_kafka_destroy(rk); }
static void test_producer_partition_cnt_change (void) { rd_kafka_t *rk; rd_kafka_conf_t *conf; rd_kafka_topic_t *rkt; const char *topic = test_mk_topic_name(__FUNCTION__, 1); const int partition_cnt = 4; int msgcnt = 100000; test_timing_t t_destroy; int produced = 0; test_kafka_topics("--create --topic %s --replication-factor 1 " "--partitions %d", topic, partition_cnt/2); test_conf_init(&conf, NULL, 20); rk = test_create_handle(RD_KAFKA_PRODUCER, conf); rkt = test_create_topic_object(rk, __FUNCTION__, "message.timeout.ms", tsprintf("%d", tmout_multip(5000)), NULL); test_produce_msgs_nowait(rk, rkt, 0, RD_KAFKA_PARTITION_UA, 0, msgcnt/2, NULL, 100, &produced); test_kafka_topics("--alter --topic %s --partitions %d", topic, partition_cnt); test_produce_msgs_nowait(rk, rkt, 0, RD_KAFKA_PARTITION_UA, msgcnt/2, msgcnt/2, NULL, 100, &produced); test_wait_delivery(rk, &produced); rd_kafka_topic_destroy(rkt); TIMING_START(&t_destroy, "rd_kafka_destroy()"); rd_kafka_destroy(rk); TIMING_STOP(&t_destroy); }
static void do_test_CreateTopics (const char *what, rd_kafka_t *rk, rd_kafka_queue_t *useq, int op_timeout, rd_bool_t validate_only) { rd_kafka_queue_t *q = useq ? useq : rd_kafka_queue_new(rk); #define MY_NEW_TOPICS_CNT 6 char *topics[MY_NEW_TOPICS_CNT]; rd_kafka_NewTopic_t *new_topics[MY_NEW_TOPICS_CNT]; rd_kafka_AdminOptions_t *options = NULL; rd_kafka_resp_err_t exp_topicerr[MY_NEW_TOPICS_CNT] = {0}; rd_kafka_resp_err_t exp_err = RD_KAFKA_RESP_ERR_NO_ERROR; /* Expected topics in metadata */ rd_kafka_metadata_topic_t exp_mdtopics[MY_NEW_TOPICS_CNT] = {{0}}; int exp_mdtopic_cnt = 0; /* Not expected topics in metadata */ rd_kafka_metadata_topic_t exp_not_mdtopics[MY_NEW_TOPICS_CNT] = {{0}}; int exp_not_mdtopic_cnt = 0; int i; char errstr[512]; const char *errstr2; rd_kafka_resp_err_t err; test_timing_t timing; rd_kafka_event_t *rkev; const rd_kafka_CreateTopics_result_t *res; const rd_kafka_topic_result_t **restopics; size_t restopic_cnt; int metadata_tmout ; int num_replicas = (int)avail_broker_cnt; int32_t *replicas; /* Set up replicas */ replicas = rd_alloca(sizeof(*replicas) * num_replicas); for (i = 0 ; i < num_replicas ; i++) replicas[i] = avail_brokers[i]; TEST_SAY(_C_MAG "[ %s CreateTopics with %s, " "op_timeout %d, validate_only %d ]\n", rd_kafka_name(rk), what, op_timeout, validate_only); /** * Construct NewTopic array with different properties for * different partitions. */ for (i = 0 ; i < MY_NEW_TOPICS_CNT ; i++) { char *topic = rd_strdup(test_mk_topic_name(__FUNCTION__, 1)); int num_parts = i * 7 + 1; int set_config = (i & 1); int add_invalid_config = (i == 1); int set_replicas = !(i % 3); rd_kafka_resp_err_t this_exp_err = RD_KAFKA_RESP_ERR_NO_ERROR; topics[i] = topic; new_topics[i] = rd_kafka_NewTopic_new(topic, num_parts, set_replicas ? -1 : num_replicas, NULL, 0); if (set_config) { /* * Add various configuration properties */ err = rd_kafka_NewTopic_set_config( new_topics[i], "compression.type", "lz4"); TEST_ASSERT(!err, "%s", rd_kafka_err2str(err)); err = rd_kafka_NewTopic_set_config( new_topics[i], "delete.retention.ms", "900"); TEST_ASSERT(!err, "%s", rd_kafka_err2str(err)); } if (add_invalid_config) { /* Add invalid config property */ err = rd_kafka_NewTopic_set_config( new_topics[i], "dummy.doesntexist", "broker is verifying this"); TEST_ASSERT(!err, "%s", rd_kafka_err2str(err)); this_exp_err = RD_KAFKA_RESP_ERR_INVALID_CONFIG; } TEST_SAY("Expected result for topic #%d: %s " "(set_config=%d, add_invalid_config=%d, " "set_replicas=%d)\n", i, rd_kafka_err2name(this_exp_err), set_config, add_invalid_config, set_replicas); if (set_replicas) { int32_t p; /* * Set valid replica assignments */ for (p = 0 ; p < num_parts ; p++) { err = rd_kafka_NewTopic_set_replica_assignment( new_topics[i], p, replicas, num_replicas, errstr, sizeof(errstr)); TEST_ASSERT(!err, "%s", errstr); } } if (this_exp_err || validate_only) { exp_topicerr[i] = this_exp_err; exp_not_mdtopics[exp_not_mdtopic_cnt++].topic = topic; } else { exp_mdtopics[exp_mdtopic_cnt].topic = topic; exp_mdtopics[exp_mdtopic_cnt].partition_cnt = num_parts; exp_mdtopic_cnt++; } } if (op_timeout != -1 || validate_only) { options = rd_kafka_AdminOptions_new( rk, RD_KAFKA_ADMIN_OP_CREATETOPICS); if (op_timeout != -1) { err = rd_kafka_AdminOptions_set_operation_timeout( options, op_timeout, errstr, sizeof(errstr)); TEST_ASSERT(!err, "%s", rd_kafka_err2str(err)); } if (validate_only) { err = rd_kafka_AdminOptions_set_validate_only( options, validate_only, errstr, sizeof(errstr)); TEST_ASSERT(!err, "%s", rd_kafka_err2str(err)); } } TIMING_START(&timing, "CreateTopics"); TEST_SAY("Call CreateTopics\n"); rd_kafka_CreateTopics(rk, new_topics, MY_NEW_TOPICS_CNT, options, q); TIMING_ASSERT_LATER(&timing, 0, 50); /* Poll result queue for CreateTopics result. * Print but otherwise ignore other event types * (typically generic Error events). */ TIMING_START(&timing, "CreateTopics.queue_poll"); do { rkev = rd_kafka_queue_poll(q, tmout_multip(20*1000)); TEST_SAY("CreateTopics: got %s in %.3fms\n", rd_kafka_event_name(rkev), TIMING_DURATION(&timing) / 1000.0f); if (rd_kafka_event_error(rkev)) TEST_SAY("%s: %s\n", rd_kafka_event_name(rkev), rd_kafka_event_error_string(rkev)); } while (rd_kafka_event_type(rkev) != RD_KAFKA_EVENT_CREATETOPICS_RESULT); /* Convert event to proper result */ res = rd_kafka_event_CreateTopics_result(rkev); TEST_ASSERT(res, "expected CreateTopics_result, not %s", rd_kafka_event_name(rkev)); /* Expecting error */ err = rd_kafka_event_error(rkev); errstr2 = rd_kafka_event_error_string(rkev); TEST_ASSERT(err == exp_err, "expected CreateTopics to return %s, not %s (%s)", rd_kafka_err2str(exp_err), rd_kafka_err2str(err), err ? errstr2 : "n/a"); TEST_SAY("CreateTopics: returned %s (%s)\n", rd_kafka_err2str(err), err ? errstr2 : "n/a"); /* Extract topics */ restopics = rd_kafka_CreateTopics_result_topics(res, &restopic_cnt); /* Scan topics for proper fields and expected failures. */ for (i = 0 ; i < (int)restopic_cnt ; i++) { const rd_kafka_topic_result_t *terr = restopics[i]; /* Verify that topic order matches our request. */ if (strcmp(rd_kafka_topic_result_name(terr), topics[i])) TEST_FAIL_LATER("Topic result order mismatch at #%d: " "expected %s, got %s", i, topics[i], rd_kafka_topic_result_name(terr)); TEST_SAY("CreateTopics result: #%d: %s: %s: %s\n", i, rd_kafka_topic_result_name(terr), rd_kafka_err2name(rd_kafka_topic_result_error(terr)), rd_kafka_topic_result_error_string(terr)); if (rd_kafka_topic_result_error(terr) != exp_topicerr[i]) TEST_FAIL_LATER( "Expected %s, not %d: %s", rd_kafka_err2name(exp_topicerr[i]), rd_kafka_topic_result_error(terr), rd_kafka_err2name(rd_kafka_topic_result_error( terr))); } /** * Verify that the expecteded topics are created and the non-expected * are not. Allow it some time to propagate. */ if (validate_only) { /* No topics should have been created, give it some time * before checking. */ rd_sleep(2); metadata_tmout = 5 * 1000; } else { if (op_timeout > 0) metadata_tmout = op_timeout + 1000; else metadata_tmout = 10 * 1000; } test_wait_metadata_update(rk, exp_mdtopics, exp_mdtopic_cnt, exp_not_mdtopics, exp_not_mdtopic_cnt, metadata_tmout); rd_kafka_event_destroy(rkev); for (i = 0 ; i < MY_NEW_TOPICS_CNT ; i++) { rd_kafka_NewTopic_destroy(new_topics[i]); rd_free(topics[i]); } if (options) rd_kafka_AdminOptions_destroy(options); if (!useq) rd_kafka_queue_destroy(q); #undef MY_NEW_TOPICS_CNT }
/** * @brief Test deletion of topics * * */ static void do_test_DeleteTopics (const char *what, rd_kafka_t *rk, rd_kafka_queue_t *useq, int op_timeout) { rd_kafka_queue_t *q = useq ? useq : rd_kafka_queue_new(rk); const int skip_topic_cnt = 2; #define MY_DEL_TOPICS_CNT 9 char *topics[MY_DEL_TOPICS_CNT]; rd_kafka_DeleteTopic_t *del_topics[MY_DEL_TOPICS_CNT]; rd_kafka_AdminOptions_t *options = NULL; rd_kafka_resp_err_t exp_topicerr[MY_DEL_TOPICS_CNT] = {0}; rd_kafka_resp_err_t exp_err = RD_KAFKA_RESP_ERR_NO_ERROR; /* Expected topics in metadata */ rd_kafka_metadata_topic_t exp_mdtopics[MY_DEL_TOPICS_CNT] = {{0}}; int exp_mdtopic_cnt = 0; /* Not expected topics in metadata */ rd_kafka_metadata_topic_t exp_not_mdtopics[MY_DEL_TOPICS_CNT] = {{0}}; int exp_not_mdtopic_cnt = 0; int i; char errstr[512]; const char *errstr2; rd_kafka_resp_err_t err; test_timing_t timing; rd_kafka_event_t *rkev; const rd_kafka_DeleteTopics_result_t *res; const rd_kafka_topic_result_t **restopics; size_t restopic_cnt; int metadata_tmout; TEST_SAY(_C_MAG "[ %s DeleteTopics with %s, op_timeout %d ]\n", rd_kafka_name(rk), what, op_timeout); /** * Construct DeleteTopic array */ for (i = 0 ; i < MY_DEL_TOPICS_CNT ; i++) { char *topic = rd_strdup(test_mk_topic_name(__FUNCTION__, 1)); int notexist_topic = i >= MY_DEL_TOPICS_CNT - skip_topic_cnt; topics[i] = topic; del_topics[i] = rd_kafka_DeleteTopic_new(topic); if (notexist_topic) exp_topicerr[i] = RD_KAFKA_RESP_ERR_UNKNOWN_TOPIC_OR_PART; else { exp_topicerr[i] = RD_KAFKA_RESP_ERR_NO_ERROR; exp_mdtopics[exp_mdtopic_cnt++].topic = topic; } exp_not_mdtopics[exp_not_mdtopic_cnt++].topic = topic; } if (op_timeout != -1) { options = rd_kafka_AdminOptions_new( rk, RD_KAFKA_ADMIN_OP_ANY); err = rd_kafka_AdminOptions_set_operation_timeout( options, op_timeout, errstr, sizeof(errstr)); TEST_ASSERT(!err, "%s", rd_kafka_err2str(err)); } /* Create the topics first, minus the skip count. */ test_CreateTopics_simple(rk, NULL, topics, MY_DEL_TOPICS_CNT-skip_topic_cnt, 2/*num_partitions*/, NULL); /* Verify that topics are reported by metadata */ test_wait_metadata_update(rk, exp_mdtopics, exp_mdtopic_cnt, NULL, 0, 15*1000); TIMING_START(&timing, "DeleteTopics"); TEST_SAY("Call DeleteTopics\n"); rd_kafka_DeleteTopics(rk, del_topics, MY_DEL_TOPICS_CNT, options, q); TIMING_ASSERT_LATER(&timing, 0, 50); /* Poll result queue for DeleteTopics result. * Print but otherwise ignore other event types * (typically generic Error events). */ TIMING_START(&timing, "DeleteTopics.queue_poll"); while (1) { rkev = rd_kafka_queue_poll(q, tmout_multip(20*1000)); TEST_SAY("DeleteTopics: got %s in %.3fms\n", rd_kafka_event_name(rkev), TIMING_DURATION(&timing) / 1000.0f); if (rd_kafka_event_error(rkev)) TEST_SAY("%s: %s\n", rd_kafka_event_name(rkev), rd_kafka_event_error_string(rkev)); if (rd_kafka_event_type(rkev) == RD_KAFKA_EVENT_DELETETOPICS_RESULT) break; rd_kafka_event_destroy(rkev); } /* Convert event to proper result */ res = rd_kafka_event_DeleteTopics_result(rkev); TEST_ASSERT(res, "expected DeleteTopics_result, not %s", rd_kafka_event_name(rkev)); /* Expecting error */ err = rd_kafka_event_error(rkev); errstr2 = rd_kafka_event_error_string(rkev); TEST_ASSERT(err == exp_err, "expected DeleteTopics to return %s, not %s (%s)", rd_kafka_err2str(exp_err), rd_kafka_err2str(err), err ? errstr2 : "n/a"); TEST_SAY("DeleteTopics: returned %s (%s)\n", rd_kafka_err2str(err), err ? errstr2 : "n/a"); /* Extract topics */ restopics = rd_kafka_DeleteTopics_result_topics(res, &restopic_cnt); /* Scan topics for proper fields and expected failures. */ for (i = 0 ; i < (int)restopic_cnt ; i++) { const rd_kafka_topic_result_t *terr = restopics[i]; /* Verify that topic order matches our request. */ if (strcmp(rd_kafka_topic_result_name(terr), topics[i])) TEST_FAIL_LATER("Topic result order mismatch at #%d: " "expected %s, got %s", i, topics[i], rd_kafka_topic_result_name(terr)); TEST_SAY("DeleteTopics result: #%d: %s: %s: %s\n", i, rd_kafka_topic_result_name(terr), rd_kafka_err2name(rd_kafka_topic_result_error(terr)), rd_kafka_topic_result_error_string(terr)); if (rd_kafka_topic_result_error(terr) != exp_topicerr[i]) TEST_FAIL_LATER( "Expected %s, not %d: %s", rd_kafka_err2name(exp_topicerr[i]), rd_kafka_topic_result_error(terr), rd_kafka_err2name(rd_kafka_topic_result_error( terr))); } /** * Verify that the expected topics are deleted and the non-expected * are not. Allow it some time to propagate. */ if (op_timeout > 0) metadata_tmout = op_timeout + 1000; else metadata_tmout = 10 * 1000; test_wait_metadata_update(rk, NULL, 0, exp_not_mdtopics, exp_not_mdtopic_cnt, metadata_tmout); rd_kafka_event_destroy(rkev); for (i = 0 ; i < MY_DEL_TOPICS_CNT ; i++) { rd_kafka_DeleteTopic_destroy(del_topics[i]); rd_free(topics[i]); } if (options) rd_kafka_AdminOptions_destroy(options); if (!useq) rd_kafka_queue_destroy(q); #undef MY_DEL_TOPICS_CNT }
static void consume_messages_with_queues (uint64_t testid, const char *topic, int partition_cnt, int msgcnt) { rd_kafka_t *rk; rd_kafka_topic_t *rkt; rd_kafka_conf_t *conf; rd_kafka_topic_conf_t *topic_conf; rd_kafka_queue_t *rkqu; int i; int32_t partition; int batch_cnt = msgcnt / partition_cnt; test_conf_init(&conf, &topic_conf, 20); /* Create kafka instance */ rk = test_create_handle(RD_KAFKA_CONSUMER, conf); /* Create queue */ rkqu = rd_kafka_queue_new(rk); rkt = rd_kafka_topic_new(rk, topic, topic_conf); if (!rkt) TEST_FAIL("Failed to create topic: %s\n", rd_strerror(errno)); TEST_SAY("Consuming %i messages from one queue serving %i partitions\n", msgcnt, partition_cnt); /* Start consuming each partition */ for (partition = 0 ; partition < partition_cnt ; partition++) { /* Consume messages */ TEST_SAY("Start consuming partition %i at offset -%i\n", partition, batch_cnt); if (rd_kafka_consume_start_queue(rkt, partition, RD_KAFKA_OFFSET_TAIL(batch_cnt), rkqu) == -1) TEST_FAIL("consume_start_queue(%i) failed: %s", (int)partition, rd_kafka_err2str(rd_kafka_errno2err(errno))); } /* Consume messages from queue */ for (i = 0 ; i < msgcnt ; ) { rd_kafka_message_t *rkmessage; rkmessage = rd_kafka_consume_queue(rkqu, tmout_multip(5000)); if (!rkmessage) TEST_FAIL("Failed to consume message %i/%i from " "queue: %s", i, msgcnt, rd_kafka_err2str(rd_kafka_errno2err(errno))); if (rkmessage->err) { if (rkmessage->err == RD_KAFKA_RESP_ERR__PARTITION_EOF){ TEST_SAY("Topic %s [%"PRId32"] reached " "EOF at offset %"PRId64"\n", rd_kafka_topic_name(rkmessage->rkt), rkmessage->partition, rkmessage->offset); rd_kafka_message_destroy(rkmessage); continue; } TEST_FAIL("Consume message %i/%i from queue " "has error (offset %"PRId64 ", partition %"PRId32"): %s", i, msgcnt, rkmessage->offset, rkmessage->partition, rd_kafka_err2str(rkmessage->err)); } verify_consumed_msg(testid, -1, -1, rkmessage); rd_kafka_message_destroy(rkmessage); i++; } /* Stop consuming each partition */ for (partition = 0 ; partition < partition_cnt ; partition++) rd_kafka_consume_stop(rkt, partition); /* Destroy queue */ rd_kafka_queue_destroy(rkqu); /* 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); }
static void do_test_null_empty (bool api_version_request) { std::string topic = Test::mk_topic_name("0070_null_empty", 1); const int partition = 0; Test::Say(tostr() << "Testing with api.version.request=" << api_version_request << " on topic " << topic << " partition " << partition << "\n"); RdKafka::Conf *conf; Test::conf_init(&conf, NULL, 0); Test::conf_set(conf, "api.version.request", api_version_request ? "true" : "false"); Test::conf_set(conf, "acks", "all"); std::string errstr; RdKafka::Producer *p = RdKafka::Producer::create(conf, errstr); if (!p) Test::Fail("Failed to create Producer: " + errstr); delete conf; const int msgcnt = 8; static const char *msgs[msgcnt*2] = { NULL, NULL, "key2", NULL, "key3", "val3", NULL, "val4", "", NULL, NULL, "", "", "" }; RdKafka::ErrorCode err; for (int i = 0 ; i < msgcnt * 2 ; i += 2) { Test::Say(3, tostr() << "Produce message #" << (i/2) << ": key=\"" << (msgs[i] ? msgs[i] : "Null") << "\", value=\"" << (msgs[i+1] ? msgs[i+1] : "Null") << "\"\n"); err = p->produce(topic, partition, RdKafka::Producer::RK_MSG_COPY, /* Value */ (void *)msgs[i+1], msgs[i+1] ? strlen(msgs[i+1]) : 0, /* Key */ (void *)msgs[i], msgs[i] ? strlen(msgs[i]) : 0, 0, NULL); if (err != RdKafka::ERR_NO_ERROR) Test::Fail("Produce failed: " + RdKafka::err2str(err)); } if (p->flush(tmout_multip(3*5000)) != 0) Test::Fail("Not all messages flushed"); Test::Say(tostr() << "Produced " << msgcnt << " messages to " << topic << "\n"); delete p; /* * Now consume messages from the beginning, making sure they match * what was produced. */ /* Create consumer */ Test::conf_init(&conf, NULL, 10); Test::conf_set(conf, "group.id", topic); Test::conf_set(conf, "api.version.request", api_version_request ? "true" : "false"); Test::conf_set(conf, "enable.auto.commit", "false"); RdKafka::KafkaConsumer *c = RdKafka::KafkaConsumer::create(conf, errstr); if (!c) Test::Fail("Failed to create KafkaConsumer: " + errstr); delete conf; /* Assign the partition */ std::vector<RdKafka::TopicPartition*> parts; parts.push_back(RdKafka::TopicPartition::create(topic, partition, RdKafka::Topic::OFFSET_BEGINNING)); err = c->assign(parts); if (err != RdKafka::ERR_NO_ERROR) Test::Fail("assign() failed: " + RdKafka::err2str(err)); RdKafka::TopicPartition::destroy(parts); /* Start consuming */ int failures = 0; for (int i = 0 ; i < msgcnt * 2 ; i += 2) { RdKafka::Message *msg = c->consume(tmout_multip(5000)); if (msg->err()) Test::Fail(tostr() << "consume() failed at message " << (i/2) << ": " << msg->errstr()); /* verify key */ failures += check_equal(msgs[i], msg->key() ? msg->key()->c_str() : NULL, msg->key_len(), tostr() << "message #" << (i/2) << " (offset " << msg->offset() << ") key"); /* verify key_pointer() API as too */ failures += check_equal(msgs[i], (const char *)msg->key_pointer(), msg->key_len(), tostr() << "message #" << (i/2) << " (offset " << msg->offset() << ") key"); /* verify value */ failures += check_equal(msgs[i+1], (const char *)msg->payload(), msg->len(), tostr() << "message #" << (i/2) << " (offset " << msg->offset() << ") value"); delete msg; } Test::Say(tostr() << "Done consuming, closing. " << failures << " test failures\n"); if (failures) Test::Fail(tostr() << "See " << failures << " previous test failure(s)"); c->close(); delete c; }
static void test_read_conf_file (const char *conf_path, rd_kafka_conf_t *conf, rd_kafka_topic_conf_t *topic_conf, int *timeoutp) { FILE *fp; char buf[512]; int line = 0; #ifndef _MSC_VER fp = fopen(conf_path, "r"); #else fp = NULL; errno = fopen_s(&fp, conf_path, "r"); #endif if (!fp) { if (errno == ENOENT) { TEST_SAY("Test config file %s not found\n", conf_path); return; } else TEST_FAIL("Failed to read %s: %s", conf_path, strerror(errno)); } while (fgets(buf, sizeof(buf)-1, fp)) { char *t; char *b = buf; rd_kafka_conf_res_t res = RD_KAFKA_CONF_UNKNOWN; char *name, *val; char errstr[512]; line++; if ((t = strchr(b, '\n'))) *t = '\0'; if (*b == '#' || !*b) continue; if (!(t = strchr(b, '='))) TEST_FAIL("%s:%i: expected name=value format\n", conf_path, line); name = b; *t = '\0'; val = t+1; if (!strcmp(name, "test.timeout.multiplier")) { TEST_LOCK(); test_timeout_multiplier = strtod(val, NULL); TEST_UNLOCK(); *timeoutp = tmout_multip((*timeoutp)*1000) / 1000; res = RD_KAFKA_CONF_OK; } else if (!strcmp(name, "test.topic.prefix")) { rd_snprintf(test_topic_prefix, sizeof(test_topic_prefix), "%s", val); res = RD_KAFKA_CONF_OK; } else if (!strcmp(name, "test.topic.random")) { if (!strcmp(val, "true") || !strcmp(val, "1")) test_topic_random = 1; else test_topic_random = 0; res = RD_KAFKA_CONF_OK; } else if (!strcmp(name, "test.concurrent.max")) { TEST_LOCK(); test_concurrent_max = strtod(val, NULL); TEST_UNLOCK(); res = RD_KAFKA_CONF_OK; } else if (!strncmp(name, "topic.", strlen("topic."))) { name += strlen("topic."); if (topic_conf) res = rd_kafka_topic_conf_set(topic_conf, name, val, errstr, sizeof(errstr)); else res = RD_KAFKA_CONF_OK; name -= strlen("topic."); } if (res == RD_KAFKA_CONF_UNKNOWN) { if (conf) res = rd_kafka_conf_set(conf, name, val, errstr, sizeof(errstr)); else res = RD_KAFKA_CONF_OK; } if (res != RD_KAFKA_CONF_OK) TEST_FAIL("%s:%i: %s\n", conf_path, line, errstr); } fclose(fp); }
static void do_test_empty_topic_consumer () { std::string errstr; std::string topic = Test::mk_topic_name("0067_empty_topic", 1); const int32_t partition = 0; RdKafka::Conf *conf; Test::conf_init(&conf, NULL, 0); /* Create simple consumer */ RdKafka::Consumer *consumer = RdKafka::Consumer::create(conf, errstr); if (!consumer) Test::Fail("Failed to create Consumer: " + errstr); RdKafka::Topic *rkt = RdKafka::Topic::create(consumer, topic, NULL, errstr); if (!rkt) Test::Fail("Simple Topic failed: " + errstr); /* Create the topic through a metadata request. */ Test::Say("Creating empty topic " + topic + "\n"); RdKafka::Metadata *md; RdKafka::ErrorCode err = consumer->metadata(false, rkt, &md, tmout_multip(10*1000)); if (err) Test::Fail("Failed to create topic " + topic + ": " + RdKafka::err2str(err)); delete md; /* Start consumer */ err = consumer->start(rkt, partition, RdKafka::Topic::OFFSET_BEGINNING); if (err) Test::Fail("Consume start() failed: " + RdKafka::err2str(err)); /* Consume using legacy consumer, should give an EOF and nothing else. */ Test::Say("Simple Consumer: consuming\n"); RdKafka::Message *msg = consumer->consume(rkt, partition, tmout_multip(10 * 1000)); if (msg->err() != RdKafka::ERR__PARTITION_EOF) Test::Fail("Simple consume() expected EOF, got " + RdKafka::err2str(msg->err())); delete msg; /* Nothing else should come now, just a consume() timeout */ msg = consumer->consume(rkt, partition, 1 * 1000); if (msg->err() != RdKafka::ERR__TIMED_OUT) Test::Fail("Simple consume() expected timeout, got " + RdKafka::err2str(msg->err())); delete msg; consumer->stop(rkt, partition); delete rkt; delete consumer; /* * Now do the same thing using the high-level KafkaConsumer. */ Test::conf_set(conf, "group.id", topic); RdKafka::KafkaConsumer *kconsumer = RdKafka::KafkaConsumer::create(conf, errstr); if (!kconsumer) Test::Fail("Failed to create KafkaConsumer: " + errstr); std::vector<RdKafka::TopicPartition*> part; part.push_back(RdKafka::TopicPartition::create(topic, partition)); err = kconsumer->assign(part); if (err) Test::Fail("assign() failed: " + RdKafka::err2str(err)); RdKafka::TopicPartition::destroy(part); Test::Say("KafkaConsumer: consuming\n"); msg = kconsumer->consume(tmout_multip(5 * 1000)); if (msg->err() != RdKafka::ERR__PARTITION_EOF) Test::Fail("KafkaConsumer consume() expected EOF, got " + RdKafka::err2str(msg->err())); delete msg; /* Nothing else should come now, just a consume() timeout */ msg = kconsumer->consume(1 * 1000); if (msg->err() != RdKafka::ERR__TIMED_OUT) Test::Fail("KafkaConsumer consume() expected timeout, got " + RdKafka::err2str(msg->err())); delete msg; kconsumer->close(); delete kconsumer; delete conf; }
static void do_test_fetch_max_bytes (void) { const int partcnt = 3; int msgcnt = 10 * partcnt; const int msgsize = 900*1024; /* Less than 1 Meg to account * for batch overhead */ std::string errstr; RdKafka::ErrorCode err; std::string topic = Test::mk_topic_name("0081-fetch_max_bytes", 1); /* Produce messages to partitions */ for (int32_t p = 0 ; p < (int32_t)partcnt ; p++) test_produce_msgs_easy_size(topic.c_str(), 0, p, msgcnt, msgsize); /* Create consumer */ RdKafka::Conf *conf; Test::conf_init(&conf, NULL, 10); Test::conf_set(conf, "group.id", topic); Test::conf_set(conf, "auto.offset.reset", "earliest"); /* We try to fetch 20 Megs per partition, but only allow 1 Meg as total * response size, this ends up serving the first batch from the * first partition. * receive.message.max.bytes is set low to trigger the original bug, * but this value is now adjusted upwards automatically by rd_kafka_new() * to hold both fetch.max.bytes and the protocol / batching overhead. * Prior to the introduction of fetch.max.bytes the fetcher code * would use receive.message.max.bytes to limit the total Fetch response, * but due to batching overhead it would result in situations where * the consumer asked for 1000000 bytes and got 1000096 bytes batch, which * was higher than the 1000000 limit. * See https://github.com/edenhill/librdkafka/issues/1616 * * With the added configuration strictness checks, a user-supplied * value is no longer over-written: * receive.message.max.bytes must be configured to be at least 512 bytes * larger than fetch.max.bytes. */ Test::conf_set(conf, "max.partition.fetch.bytes", "20000000"); /* ~20MB */ Test::conf_set(conf, "fetch.max.bytes", "1000000"); /* ~1MB */ Test::conf_set(conf, "receive.message.max.bytes", "1000512"); /* ~1MB+512 */ RdKafka::KafkaConsumer *c = RdKafka::KafkaConsumer::create(conf, errstr); if (!c) Test::Fail("Failed to create KafkaConsumer: " + errstr); delete conf; /* Subscribe */ std::vector<std::string> topics; topics.push_back(topic); if ((err = c->subscribe(topics))) Test::Fail("subscribe failed: " + RdKafka::err2str(err)); /* Start consuming */ Test::Say("Consuming topic " + topic + "\n"); int cnt = 0; while (cnt < msgcnt) { RdKafka::Message *msg = c->consume(tmout_multip(1000)); switch (msg->err()) { case RdKafka::ERR__TIMED_OUT: break; case RdKafka::ERR_NO_ERROR: cnt++; break; default: Test::Fail("Consume error: " + msg->errstr()); break; } delete msg; } Test::Say("Done\n"); c->close(); delete c; }
/** * @brief Test that Metadata requests are retried properly when * timing out due to high broker rtt. */ static void do_test_low_socket_timeout (const char *topic) { rd_kafka_t *rk; rd_kafka_conf_t *conf; rd_kafka_topic_t *rkt; rd_kafka_resp_err_t err; const struct rd_kafka_metadata *md; int res; mtx_init(&ctrl.lock, mtx_plain); cnd_init(&ctrl.cnd); TEST_SAY("Test Metadata request retries on timeout\n"); test_conf_init(&conf, NULL, 60); test_conf_set(conf, "socket.timeout.ms", "1000"); test_conf_set(conf, "socket.max.fails", "12345"); test_conf_set(conf, "retry.backoff.ms", "5000"); /* Avoid api version requests (with their own timeout) to get in * the way of our test */ test_conf_set(conf, "api.version.request", "false"); test_socket_enable(conf); test_curr->connect_cb = connect_cb; test_curr->is_fatal_cb = is_fatal_cb; rk = test_create_handle(RD_KAFKA_PRODUCER, conf); rkt = test_create_producer_topic(rk, topic, NULL); TEST_SAY("Waiting for sockem connect..\n"); mtx_lock(&ctrl.lock); while (!ctrl.skm) cnd_wait(&ctrl.cnd, &ctrl.lock); mtx_unlock(&ctrl.lock); TEST_SAY("Connected, fire off a undelayed metadata() to " "make sure connection is up\n"); err = rd_kafka_metadata(rk, 0, rkt, &md, tmout_multip(2000)); TEST_ASSERT(!err, "metadata(undelayed) failed: %s", rd_kafka_err2str(err)); rd_kafka_metadata_destroy(md); if (thrd_create(&ctrl.thrd, ctrl_thrd_main, NULL) != thrd_success) TEST_FAIL("Failed to create sockem ctrl thread"); set_delay(0, 3000); /* Takes effect immediately */ /* After two retries, remove the delay, the third retry * should kick in and work. */ set_delay(((1000 /*socket.timeout.ms*/ + 5000 /*retry.backoff.ms*/) * 2) - 2000, 0); TEST_SAY("Calling metadata() again which should succeed after " "3 internal retries\n"); /* Metadata should be returned after the third retry */ err = rd_kafka_metadata(rk, 0, rkt, &md, ((1000 /*socket.timeout.ms*/ + 5000 /*retry.backoff.ms*/) * 2) + 5000); TEST_SAY("metadata() returned %s\n", rd_kafka_err2str(err)); TEST_ASSERT(!err, "metadata(undelayed) failed: %s", rd_kafka_err2str(err)); rd_kafka_metadata_destroy(md); rd_kafka_topic_destroy(rkt); rd_kafka_destroy(rk); /* Join controller thread */ mtx_lock(&ctrl.lock); ctrl.term = 1; mtx_unlock(&ctrl.lock); thrd_join(ctrl.thrd, &res); cnd_destroy(&ctrl.cnd); mtx_destroy(&ctrl.lock); }
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_0002_unkpart (int argc, char **argv) { int partition = 99; /* non-existent */ 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 = 10; int i; int fails = 0; const struct rd_kafka_metadata *metadata; test_conf_init(&conf, &topic_conf, 10); /* Set delivery report callback */ rd_kafka_conf_set_dr_cb(conf, dr_cb); /* Create kafka instance */ rk = test_create_handle(RD_KAFKA_PRODUCER, conf); rkt = rd_kafka_topic_new(rk, test_mk_topic_name("0002", 0), topic_conf); if (!rkt) TEST_FAIL("Failed to create topic: %s\n", rd_strerror(errno)); /* Request metadata so that we know the cluster is up before producing * messages, otherwise erroneous partitions will not fail immediately.*/ if ((r = rd_kafka_metadata(rk, 0, rkt, &metadata, tmout_multip(15000))) != RD_KAFKA_RESP_ERR_NO_ERROR) TEST_FAIL("Failed to acquire metadata: %s\n", rd_kafka_err2str(r)); rd_kafka_metadata_destroy(metadata); /* Produce a message */ 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) { if (errno == ESRCH) TEST_SAY("Failed to produce message #%i: " "unknown partition: good!\n", i); else TEST_FAIL("Failed to produce message #%i: %s\n", i, rd_kafka_err2str( rd_kafka_errno2err(errno))); free(msgidp); } else { if (i > 5) { fails++; TEST_SAY("Message #%i produced: " "should've failed\n", i); } msgs_wait |= (1 << i); } /* After half the messages: sleep to allow the metadata * to be fetched from broker and update the actual partition * count: this will make subsequent produce() calls fail * immediately. */ if (i == 5) rd_sleep(2); } /* Wait for messages to time out */ rd_kafka_flush(rk, -1); if (msgs_wait != 0) TEST_FAIL("Still waiting for messages: 0x%x\n", msgs_wait); if (fails > 0) TEST_FAIL("See previous error(s)\n"); /* 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; }
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 void do_test_consumer_lag (void) { const int msgcnt = 10; std::string errstr; RdKafka::ErrorCode err; topic = Test::mk_topic_name("0061-consumer_lag", 1); test_produce_msgs_easy(topic.c_str(), 0, 0, msgcnt); /* * Create consumer */ /* Create consumer */ RdKafka::Conf *conf; Test::conf_init(&conf, NULL, 10); StatsCb stats; if (conf->set("event_cb", &stats, errstr) != RdKafka::Conf::CONF_OK) Test::Fail("set event_cb failed: " + errstr); Test::conf_set(conf, "group.id", topic); Test::conf_set(conf, "enable.auto.commit", "false"); Test::conf_set(conf, "enable.partition.eof", "false"); Test::conf_set(conf, "auto.offset.reset", "earliest"); Test::conf_set(conf, "statistics.interval.ms", "100"); RdKafka::KafkaConsumer *c = RdKafka::KafkaConsumer::create(conf, errstr); if (!c) Test::Fail("Failed to create KafkaConsumer: " + errstr); delete conf; /* Assign partitions */ /* Subscribe */ std::vector<RdKafka::TopicPartition*> parts; parts.push_back(RdKafka::TopicPartition::create(topic, 0)); if ((err = c->assign(parts))) Test::Fail("assign failed: " + RdKafka::err2str(err)); RdKafka::TopicPartition::destroy(parts); /* Start consuming */ Test::Say("Consuming topic " + topic + "\n"); int cnt = 0; while (cnt < msgcnt) { RdKafka::Message *msg = c->consume(tmout_multip(1000)); switch (msg->err()) { case RdKafka::ERR__TIMED_OUT: break; case RdKafka::ERR__PARTITION_EOF: Test::Fail(tostr() << "Consume error after " << cnt << "/" << msgcnt << " messages: " << msg->errstr()); break; case RdKafka::ERR_NO_ERROR: /* Proper message. Update calculated lag for later * checking in stats callback */ stats.calc_lag = msgcnt - (msg->offset()+1); cnt++; Test::Say(2, tostr() << "Received message #" << cnt << "/" << msgcnt << " at offset " << msg->offset() << " (calc lag " << stats.calc_lag << ")\n"); /* Slow down message "processing" to make sure we get * at least one stats callback per message. */ if (cnt < msgcnt) rd_sleep(1); break; default: Test::Fail("Consume error: " + msg->errstr()); break; } delete msg; } Test::Say(tostr() << "Done, lag was valid " << stats.lag_valid << " times\n"); if (stats.lag_valid == 0) Test::Fail("No valid consumer_lag in statistics seen"); c->close(); delete c; }