/** * Issue #530: * "Legacy Consumer. Delete hangs if done right after RdKafka::Consumer::create. * But If I put a start and stop in between, there is no issue." */ static int legacy_consumer_early_destroy (void) { rd_kafka_t *rk; rd_kafka_topic_t *rkt; int pass; const char *topic = test_mk_topic_name(__FUNCTION__, 0); for (pass = 0 ; pass < 2 ; pass++) { TEST_SAY("%s: pass #%d\n", __FUNCTION__, pass); rk = test_create_handle(RD_KAFKA_CONSUMER, NULL); if (pass == 1) { /* Second pass, create a topic too. */ rkt = rd_kafka_topic_new(rk, topic, NULL); TEST_ASSERT(rkt, "failed to create topic: %s", rd_kafka_err2str( rd_kafka_errno2err(errno))); rd_sleep(1); rd_kafka_topic_destroy(rkt); } rd_kafka_destroy(rk); } return 0; }
static void test_producer_no_connection (void) { rd_kafka_t *rk; rd_kafka_conf_t *conf; rd_kafka_topic_t *rkt; int i; const int partition_cnt = 2; int msgcnt = 0; test_timing_t t_destroy; test_conf_init(&conf, NULL, 20); test_conf_set(conf, "bootstrap.servers", NULL); rk = test_create_handle(RD_KAFKA_PRODUCER, conf); rkt = test_create_topic_object(rk, __FUNCTION__, "message.timeout.ms", "5000", NULL); test_produce_msgs_nowait(rk, rkt, 0, RD_KAFKA_PARTITION_UA, 0, 100, NULL, 100, 0, &msgcnt); for (i = 0 ; i < partition_cnt ; i++) test_produce_msgs_nowait(rk, rkt, 0, i, 0, 100, NULL, 100, 0, &msgcnt); rd_kafka_poll(rk, 1000); TEST_SAY("%d messages in queue\n", rd_kafka_outq_len(rk)); rd_kafka_topic_destroy(rkt); TIMING_START(&t_destroy, "rd_kafka_destroy()"); rd_kafka_destroy(rk); TIMING_STOP(&t_destroy); }
int main_0035_api_version (int argc, char **argv) { rd_kafka_t *rk; rd_kafka_conf_t *conf; const struct rd_kafka_metadata *metadata; rd_kafka_resp_err_t err; test_timing_t t_meta; test_conf_init(&conf, NULL, 30); test_conf_set(conf, "socket.timeout.ms", "12000"); rk = test_create_handle(RD_KAFKA_PRODUCER, conf); TEST_SAY("Querying for metadata\n"); TIMING_START(&t_meta, "metadata()"); err = rd_kafka_metadata(rk, 0, NULL, &metadata, 10*1000); TIMING_STOP(&t_meta); if (err) TEST_FAIL("metadata() failed: %s", rd_kafka_err2str(err)); if (TIMING_DURATION(&t_meta) / 1000 > 11*1000) TEST_FAIL("metadata() took too long: %.3fms", (float)TIMING_DURATION(&t_meta) / 1000.0f); rd_kafka_metadata_destroy(metadata); TEST_SAY("Metadata succeeded\n"); rd_kafka_destroy(rk); return 0; }
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); }
/** * @brief Test handling of implicit acks. * * @param batch_cnt Total number of batches, ProduceRequests, sent. * @param initial_fail_batch_cnt How many of the initial batches should * fail with an emulated network timeout. */ static void do_test_implicit_ack (const char *what, int batch_cnt, int initial_fail_batch_cnt) { rd_kafka_t *rk; const char *topic = test_mk_topic_name("0090_idempotence_impl_ack", 1); const int32_t partition = 0; uint64_t testid; int msgcnt = 10*batch_cnt; rd_kafka_conf_t *conf; rd_kafka_topic_t *rkt; test_msgver_t mv; TEST_SAY(_C_MAG "[ Test implicit ack: %s ]\n", what); rd_atomic32_init(&state.produce_cnt, 0); state.batch_cnt = batch_cnt; state.initial_fail_batch_cnt = initial_fail_batch_cnt; testid = test_id_generate(); test_conf_init(&conf, NULL, 60); rd_kafka_conf_set_dr_msg_cb(conf, test_dr_msg_cb); test_conf_set(conf, "enable.idempotence", "true"); test_conf_set(conf, "batch.num.messages", "10"); test_conf_set(conf, "linger.ms", "500"); test_conf_set(conf, "retry.backoff.ms", "2000"); /* The ProduceResponse handler will inject timed-out-in-flight * errors for the first N ProduceRequests, which will trigger retries * that in turn will result in OutOfSequence errors. */ test_conf_set(conf, "ut_handle_ProduceResponse", (char *)handle_ProduceResponse); test_create_topic(topic, 1, 1); rk = test_create_handle(RD_KAFKA_PRODUCER, conf); rkt = test_create_producer_topic(rk, topic, NULL); TEST_SAY("Producing %d messages\n", msgcnt); test_produce_msgs(rk, rkt, testid, -1, 0, msgcnt, NULL, 0); TEST_SAY("Flushing..\n"); rd_kafka_flush(rk, 10000); rd_kafka_topic_destroy(rkt); rd_kafka_destroy(rk); TEST_SAY("Verifying messages with consumer\n"); test_msgver_init(&mv, testid); test_consume_msgs_easy_mv(NULL, topic, partition, testid, 1, msgcnt, NULL, &mv); test_msgver_verify("verify", &mv, TEST_MSGVER_ALL, 0, msgcnt); test_msgver_clear(&mv); TEST_SAY(_C_GRN "[ Test implicit ack: %s : PASS ]\n", what); }
static void 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 do_test_apis (rd_kafka_type_t cltype) { rd_kafka_t *rk; rd_kafka_conf_t *conf; rd_kafka_queue_t *mainq; /* Get the available brokers, but use a separate rd_kafka_t instance * so we don't jinx the tests by having up-to-date metadata. */ avail_brokers = test_get_broker_ids(NULL, &avail_broker_cnt); TEST_SAY("%"PRIusz" brokers in cluster " "which will be used for replica sets\n", avail_broker_cnt); do_test_unclean_destroy(cltype, 0/*tempq*/); do_test_unclean_destroy(cltype, 1/*mainq*/); test_conf_init(&conf, NULL, 60); test_conf_set(conf, "socket.timeout.ms", "10000"); rk = test_create_handle(cltype, conf); mainq = rd_kafka_queue_get_main(rk); /* Create topics */ do_test_CreateTopics("temp queue, op timeout 0", rk, NULL, 0, 0); do_test_CreateTopics("temp queue, op timeout 15000", rk, NULL, 15000, 0); do_test_CreateTopics("temp queue, op timeout 300, " "validate only", rk, NULL, 300, rd_true); do_test_CreateTopics("temp queue, op timeout 9000, validate_only", rk, NULL, 9000, rd_true); do_test_CreateTopics("main queue, options", rk, mainq, -1, 0); /* Delete topics */ do_test_DeleteTopics("temp queue, op timeout 0", rk, NULL, 0); do_test_DeleteTopics("main queue, op timeout 15000", rk, mainq, 1500); /* Create Partitions */ do_test_CreatePartitions("temp queue, op timeout 6500", rk, NULL, 6500); do_test_CreatePartitions("main queue, op timeout 0", rk, mainq, 0); /* AlterConfigs */ do_test_AlterConfigs(rk, mainq); /* DescribeConfigs */ do_test_DescribeConfigs(rk, mainq); rd_kafka_queue_destroy(mainq); rd_kafka_destroy(rk); free(avail_brokers); }
int main_0001_multiobj (int argc, char **argv) { int partition = RD_KAFKA_PARTITION_UA; /* random */ int i; const int NUM_ITER = 10; const char *topic = NULL; TEST_SAY("Creating and destroying %i kafka instances\n", NUM_ITER); /* Create, use and destroy NUM_ITER kafka instances. */ for (i = 0 ; i < NUM_ITER ; i++) { rd_kafka_t *rk; rd_kafka_topic_t *rkt; rd_kafka_conf_t *conf; rd_kafka_topic_conf_t *topic_conf; char msg[128]; test_timing_t t_destroy; test_conf_init(&conf, &topic_conf, 30); if (!topic) topic = test_mk_topic_name("0001", 0); rk = test_create_handle(RD_KAFKA_PRODUCER, conf); rkt = rd_kafka_topic_new(rk, topic, topic_conf); if (!rkt) TEST_FAIL("Failed to create topic for " "rdkafka instance #%i: %s\n", i, rd_kafka_err2str(rd_kafka_errno2err(errno))); rd_snprintf(msg, sizeof(msg), "%s test message for iteration #%i", argv[0], i); /* Produce a message */ rd_kafka_produce(rkt, partition, RD_KAFKA_MSG_F_COPY, msg, strlen(msg), NULL, 0, NULL); /* Wait for it to be sent (and possibly acked) */ rd_kafka_flush(rk, -1); /* Destroy topic */ rd_kafka_topic_destroy(rkt); /* Destroy rdkafka instance */ TIMING_START(&t_destroy, "rd_kafka_destroy()"); rd_kafka_destroy(rk); TIMING_STOP(&t_destroy); } return 0; }
/** * @brief Local test: test event generation */ int main_0039_event (int argc, char **argv) { rd_kafka_t *rk; rd_kafka_conf_t *conf; rd_kafka_queue_t *eventq; int waitevent = 1; /* Set up a config with ERROR events enabled and * configure an invalid broker so that _TRANSPORT or ALL_BROKERS_DOWN * is promptly generated. */ conf = rd_kafka_conf_new(); rd_kafka_conf_set_events(conf, RD_KAFKA_EVENT_ERROR); rd_kafka_conf_set(conf, "bootstrap.servers", "0:65534", NULL, 0); /* Create kafka instance */ rk = test_create_handle(RD_KAFKA_PRODUCER, conf); eventq = rd_kafka_queue_get_main(rk); while (waitevent) { rd_kafka_event_t *rkev; rkev = rd_kafka_queue_poll(eventq, 1000); switch (rd_kafka_event_type(rkev)) { case RD_KAFKA_EVENT_ERROR: TEST_SAY("Got %s%s event: %s: %s\n", rd_kafka_event_error_is_fatal(rkev) ? "FATAL " : "", rd_kafka_event_name(rkev), rd_kafka_err2name(rd_kafka_event_error(rkev)), rd_kafka_event_error_string(rkev)); waitevent = 0; break; default: TEST_SAY("Unhandled event: %s\n", rd_kafka_event_name(rkev)); break; } rd_kafka_event_destroy(rkev); } rd_kafka_queue_destroy(eventq); /* Destroy rdkafka instance */ TEST_SAY("Destroying kafka instance %s\n", rd_kafka_name(rk)); rd_kafka_destroy(rk); return 0; }
/** * @brief Produce messages according to compress \p codec */ static void produce_msgs (const char *topic, int partition, uint64_t testid, int msgcnt, const char *broker_version, const char *codec) { rd_kafka_conf_t *conf; rd_kafka_t *rk; int i; char key[128], buf[100]; int msgcounter = msgcnt; test_conf_init(&conf, NULL, 0); rd_kafka_conf_set_dr_cb(conf, test_dr_cb); test_conf_set(conf, "compression.codec", codec); test_conf_set(conf, "broker.version.fallback", broker_version); if (strstr(broker_version, "0.10.")) test_conf_set(conf, "api.version.request", "true"); else test_conf_set(conf, "api.version.request", "false"); /* Make sure to trigger a bunch of MessageSets */ test_conf_set(conf, "batch.num.messages", tsprintf("%d", msgcnt/5)); rk = test_create_handle(RD_KAFKA_PRODUCER, conf); for (i = 0 ; i < msgcnt ; i++) { rd_kafka_resp_err_t err; test_prepare_msg(testid, partition, i, buf, sizeof(buf), key, sizeof(key)); err = rd_kafka_producev(rk, RD_KAFKA_V_TOPIC(topic), RD_KAFKA_V_VALUE(buf, sizeof(buf)), RD_KAFKA_V_KEY(key, sizeof(key)), RD_KAFKA_V_TIMESTAMP(my_timestamp.min), RD_KAFKA_V_MSGFLAGS(RD_KAFKA_MSG_F_COPY), RD_KAFKA_V_OPAQUE(&msgcounter), RD_KAFKA_V_END); if (err) TEST_FAIL("producev() failed at msg #%d/%d: %s", i, msgcnt, rd_kafka_err2str(err)); } TEST_SAY("Waiting for %d messages to be produced\n", msgcounter); while (msgcounter > 0) rd_kafka_poll(rk, 100); 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); }
/** * @brief Interceptors must not be copied automatically by conf_dup() * unless the interceptors have added on_conf_dup(). * This behaviour makes sure an interceptor's instance * is not duplicated without the interceptor's knowledge or * assistance. */ static void do_test_conf_copy (const char *topic) { rd_kafka_conf_t *conf, *conf2; int i; rd_kafka_t *rk; TEST_SAY(_C_MAG "[ %s ]\n" _C_CLR, __FUNCTION__); memset(&msgs[0], 0, sizeof(msgs)); test_conf_init(&conf, NULL, 0); rd_kafka_conf_interceptor_add_on_new(conf, "on_new_conf_copy", on_new_producer, NULL); /* Now copy the configuration to verify that interceptors are * NOT copied. */ conf2 = conf; conf = rd_kafka_conf_dup(conf2); rd_kafka_conf_destroy(conf2); /* Create producer */ rk = test_create_handle(RD_KAFKA_PRODUCER, conf); for (i = 0 ; i < msgcnt-1 ; i++) do_test_produce(rk, topic, RD_KAFKA_PARTITION_UA, i, 0, 0); /* Wait for messages to be delivered */ test_flush(rk, -1); /* Verify acks */ for (i = 0 ; i < msgcnt ; i++) { struct msg_state *msg = &msgs[i]; msg_verify_ic_cnt(msg, "on_ack", msg->bits[_ON_ACK], 0); } rd_kafka_destroy(rk); }
static void do_test_producer (const char *topic) { rd_kafka_conf_t *conf; int i; rd_kafka_t *rk; TEST_SAY(_C_MAG "[ %s ]\n" _C_CLR, __FUNCTION__); test_conf_init(&conf, NULL, 0); rd_kafka_conf_interceptor_add_on_new(conf, "on_new_prodcer", on_new_producer, NULL); /* Create producer */ rk = test_create_handle(RD_KAFKA_PRODUCER, conf); for (i = 0 ; i < msgcnt-1 ; i++) do_test_produce(rk, topic, RD_KAFKA_PARTITION_UA, i, 0, producer_ic_cnt); /* Wait for messages to be delivered */ test_flush(rk, -1); /* Now send a message that will fail in produce() * due to bad partition */ do_test_produce(rk, topic, 1234, i, 1, producer_ic_cnt); /* Verify acks */ for (i = 0 ; i < msgcnt ; i++) { struct msg_state *msg = &msgs[i]; msg_verify_ic_cnt(msg, "on_ack", msg->bits[_ON_ACK], producer_ic_cnt); } rd_kafka_destroy(rk); }
int main_0038_performance (int argc, char **argv) { const char *topic = test_mk_topic_name(__FUNCTION__, 1); const int partition = 0; const int msgsize = 100; uint64_t testid; rd_kafka_conf_t *conf; rd_kafka_t *rk; rd_kafka_topic_t *rkt; test_timing_t t_create, t_produce, t_consume; int totsize = 1024*1024*128; int msgcnt; if (!strcmp(test_mode, "valgrind") || !strcmp(test_mode, "helgrind") || !strcmp(test_mode, "drd")) totsize = 1024*1024*8; /* 8 meg, valgrind is slow. */ msgcnt = totsize / msgsize; TEST_SAY("Producing %d messages of size %d to %s [%d]\n", msgcnt, (int)msgsize, topic, partition); testid = test_id_generate(); test_conf_init(&conf, NULL, 120); rd_kafka_conf_set_dr_cb(conf, test_dr_cb); test_conf_set(conf, "queue.buffering.max.messages", "10000000"); rk = test_create_handle(RD_KAFKA_PRODUCER, conf); rkt = test_create_producer_topic(rk, topic, "acks", "1", NULL); /* First produce one message to create the topic, etc, this might take * a while and we dont want this to affect the throughput timing. */ TIMING_START(&t_create, "CREATE TOPIC"); test_produce_msgs(rk, rkt, testid, partition, 0, 1, NULL, msgsize); TIMING_STOP(&t_create); TIMING_START(&t_produce, "PRODUCE"); test_produce_msgs(rk, rkt, testid, partition, 1, msgcnt-1, NULL, msgsize); TIMING_STOP(&t_produce); TEST_SAY("Destroying producer\n"); rd_kafka_topic_destroy(rkt); rd_kafka_destroy(rk); TEST_SAY("Creating consumer\n"); test_conf_init(&conf, NULL, 120); rk = test_create_consumer(NULL, NULL, conf, NULL, NULL); rkt = rd_kafka_topic_new(rk, topic, NULL); test_consumer_start("CONSUME", rkt, partition, RD_KAFKA_OFFSET_BEGINNING); TIMING_START(&t_consume, "CONSUME"); test_consume_msgs("CONSUME", rkt, testid, partition, TEST_NO_SEEK, 0, msgcnt, 1); TIMING_STOP(&t_consume); test_consumer_stop("CONSUME", rkt, partition); rd_kafka_topic_destroy(rkt); rd_kafka_destroy(rk); TEST_REPORT("{ \"producer\": " " { \"mb_per_sec\": %.2f, \"records_per_sec\": %.2f }," " \"consumer\": " "{ \"mb_per_sec\": %.2f, \"records_per_sec\": %.2f } " "}", (double) (totsize/((double)TIMING_DURATION(&t_produce)/1000000.0f)) / 1000000.0f, (float) (msgcnt/((double)TIMING_DURATION(&t_produce)/1000000.0f)), (double) (totsize/((double)TIMING_DURATION(&t_consume)/1000000.0f)) / 1000000.0f, (float) (msgcnt/((double)TIMING_DURATION(&t_consume)/1000000.0f))); return 0; }
/** * Produces 'msgcnt' messages split over 'partition_cnt' partitions. */ static void produce_messages (uint64_t testid, const char *topic, int partition_cnt, int msgcnt) { int r; rd_kafka_t *rk; rd_kafka_topic_t *rkt; rd_kafka_conf_t *conf; rd_kafka_topic_conf_t *topic_conf; char errstr[512]; char msg[128]; int failcnt = 0; int i; rd_kafka_message_t *rkmessages; int32_t partition; int msgid = 0; test_conf_init(&conf, &topic_conf, 20); rd_kafka_conf_set_dr_cb(conf, dr_cb); /* Make sure all replicas are in-sync after producing * so that consume test wont fail. */ rd_kafka_topic_conf_set(topic_conf, "request.required.acks", "-1", errstr, sizeof(errstr)); /* Create kafka instance */ rk = test_create_handle(RD_KAFKA_PRODUCER, conf); rkt = rd_kafka_topic_new(rk, topic, topic_conf); if (!rkt) TEST_FAIL("Failed to create topic: %s\n", rd_strerror(errno)); /* Create messages. */ prod_msg_remains = msgcnt; rkmessages = calloc(sizeof(*rkmessages), msgcnt / partition_cnt); for (partition = 0 ; partition < partition_cnt ; partition++) { int batch_cnt = msgcnt / partition_cnt; for (i = 0 ; i < batch_cnt ; i++) { rd_snprintf(msg, sizeof(msg), "testid=%"PRIu64", partition=%i, msg=%i", testid, (int)partition, msgid); rkmessages[i].payload = rd_strdup(msg); rkmessages[i].len = strlen(msg); msgid++; } TEST_SAY("Start produce to partition %i: msgs #%d..%d\n", (int)partition, msgid-batch_cnt, msgid); /* Produce batch for this partition */ r = rd_kafka_produce_batch(rkt, partition, RD_KAFKA_MSG_F_FREE, rkmessages, batch_cnt); if (r == -1) TEST_FAIL("Failed to produce " "batch for partition %i: %s", (int)partition, rd_kafka_err2str(rd_kafka_errno2err(errno))); /* Scan through messages to check for errors. */ for (i = 0 ; i < batch_cnt ; i++) { if (rkmessages[i].err) { failcnt++; if (failcnt < 100) TEST_SAY("Message #%i failed: %s\n", i, rd_kafka_err2str(rkmessages[i]. err)); } } /* All messages should've been produced. */ if (r < batch_cnt) { TEST_SAY("Not all messages were accepted " "by produce_batch(): %i < %i\n", r, batch_cnt); if (batch_cnt - r != failcnt) TEST_SAY("Discrepency between failed " "messages (%i) " "and return value %i (%i - %i)\n", failcnt, batch_cnt - r, batch_cnt, r); TEST_FAIL("%i/%i messages failed\n", batch_cnt - r, batch_cnt); } TEST_SAY("Produced %i messages to partition %i, " "waiting for deliveries\n", r, partition); } free(rkmessages); /* Wait for messages to be delivered */ while (rd_kafka_outq_len(rk) > 0) rd_kafka_poll(rk, 100); if (fails) TEST_FAIL("%i failures, see previous errors", fails); if (prod_msg_remains != 0) TEST_FAIL("Still waiting for %i messages to be produced", prod_msg_remains); /* 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 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); }
/** * @brief Test delivery report events */ int main_0039_event_dr (int argc, char **argv) { int partition = 0; int r; rd_kafka_t *rk; rd_kafka_topic_t *rkt; rd_kafka_conf_t *conf; rd_kafka_topic_conf_t *topic_conf; char msg[128]; int msgcnt = test_on_ci ? 5000 : 50000; int i; test_timing_t t_produce, t_delivery; rd_kafka_queue_t *eventq; test_conf_init(&conf, &topic_conf, 10); /* Set delivery report callback */ rd_kafka_conf_set_dr_msg_cb(conf, test_dr_msg_cb); rd_kafka_conf_set_events(conf, RD_KAFKA_EVENT_DR); /* Create kafka instance */ rk = test_create_handle(RD_KAFKA_PRODUCER, conf); eventq = rd_kafka_queue_get_main(rk); rkt = rd_kafka_topic_new(rk, test_mk_topic_name("0005", 0), topic_conf); if (!rkt) TEST_FAIL("Failed to create topic: %s\n", rd_strerror(errno)); /* Produce messages */ TIMING_START(&t_produce, "PRODUCE"); for (i = 0 ; i < msgcnt ; i++) { int *msgidp = malloc(sizeof(*msgidp)); *msgidp = i; rd_snprintf(msg, sizeof(msg), "%s test message #%i", argv[0], i); r = rd_kafka_produce(rkt, partition, RD_KAFKA_MSG_F_COPY, msg, strlen(msg), NULL, 0, msgidp); if (r == -1) TEST_FAIL("Failed to produce message #%i: %s\n", i, rd_strerror(errno)); } TIMING_STOP(&t_produce); TEST_SAY("Produced %i messages, waiting for deliveries\n", msgcnt); /* Wait for messages to be delivered */ TIMING_START(&t_delivery, "DELIVERY"); while (rd_kafka_outq_len(rk) > 0) { rd_kafka_event_t *rkev; rkev = rd_kafka_queue_poll(eventq, 1000); switch (rd_kafka_event_type(rkev)) { case RD_KAFKA_EVENT_DR: TEST_SAYL(3, "%s event with %zd messages\n", rd_kafka_event_name(rkev), rd_kafka_event_message_count(rkev)); handle_drs(rkev); break; default: TEST_SAY("Unhandled event: %s\n", rd_kafka_event_name(rkev)); break; } rd_kafka_event_destroy(rkev); } TIMING_STOP(&t_delivery); if (fails) TEST_FAIL("%i failures, see previous errors", fails); if (msgid_next != msgcnt) TEST_FAIL("Still waiting for messages: next %i != end %i\n", msgid_next, msgcnt); rd_kafka_queue_destroy(eventq); /* Destroy topic */ rd_kafka_topic_destroy(rkt); /* Destroy rdkafka instance */ TEST_SAY("Destroying kafka instance %s\n", rd_kafka_name(rk)); rd_kafka_destroy(rk); return 0; }
int main_0088_produce_metadata_timeout (int argc, char **argv) { int64_t testid; rd_kafka_t *rk; rd_kafka_topic_t *rkt; const char *topic = test_mk_topic_name("0088_produce_metadata_timeout", 1); int msgcnt = 0; rd_kafka_conf_t *conf; testid = test_id_generate(); /* Create topic with single partition, for simplicity. */ test_create_topic(topic, 1, 1); test_conf_init(&conf, NULL, 15*60*2); // msgcnt * 2); rd_kafka_conf_set_dr_msg_cb(conf, dr_msg_cb); test_conf_set(conf, "metadata.max.age.ms", "10000"); test_conf_set(conf, "topic.metadata.refresh.interval.ms", "-1"); test_conf_set(conf, "linger.ms", "5000"); test_conf_set(conf, "batch.num.messages", "5"); 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 = rd_kafka_topic_new(rk, topic, NULL); /* Produce first set of messages and wait for delivery */ test_produce_msgs_nowait(rk, rkt, testid, RD_KAFKA_PARTITION_UA, msgcnt, 20, NULL, 0, &msgcnt); while (msg_dr_cnt < 5) rd_kafka_poll(rk, 1000); TEST_SAY(_C_YEL "Disconnecting sockets and " "refusing future connections\n"); rd_atomic32_set(&refuse_connect, 1); test_socket_close_all(test_curr, 1/*reinit*/); /* Wait for metadata timeout */ TEST_SAY("Waiting for metadata timeout\n"); rd_sleep(10+5); /* These messages will be put on the UA queue */ test_produce_msgs_nowait(rk, rkt, testid, RD_KAFKA_PARTITION_UA, msgcnt, 20, NULL, 0, &msgcnt); /* Restore the connection(s) when metadata has timed out. */ TEST_SAY(_C_YEL "Allowing connections\n"); rd_atomic32_set(&refuse_connect, 0); rd_sleep(3); test_produce_msgs_nowait(rk, rkt, testid, RD_KAFKA_PARTITION_UA, msgcnt, 20, NULL, 0, &msgcnt); test_flush(rk, 2*5*1000); /* linger.ms * 2 */ TEST_ASSERT(msg_dr_cnt == msgcnt, "expected %d, got %d", msgcnt, msg_dr_cnt); TEST_ASSERT(msg_dr_fail_cnt == 0, "expected %d dr failures, got %d", 0, msg_dr_fail_cnt); rd_kafka_topic_destroy(rkt); rd_kafka_destroy(rk); return 0; }
/** * @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); }
int main_0003_msgmaxsize (int argc, char **argv) { int partition = 0; int r; rd_kafka_t *rk; rd_kafka_topic_t *rkt; rd_kafka_conf_t *conf; rd_kafka_topic_conf_t *topic_conf; char errstr[512]; char *msg; static const int msgsize = 100000; int msgcnt = 10; int i; test_conf_init(&conf, &topic_conf, 10); /* Set a small maximum message size. */ if (rd_kafka_conf_set(conf, "message.max.bytes", "100000", errstr, sizeof(errstr)) != RD_KAFKA_CONF_OK) TEST_FAIL("%s\n", errstr); /* 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("0003", 0), topic_conf); if (!rkt) TEST_FAIL("Failed to create topic: %s\n", rd_strerror(errno)); msg = calloc(1, msgsize); /* Produce 'msgcnt' messages, size odd ones larger than max.bytes, * and even ones smaller than max.bytes. */ for (i = 0 ; i < msgcnt ; i++) { int *msgidp = malloc(sizeof(*msgidp)); size_t len; int toobig = i & 1; *msgidp = i; if (toobig) { /* Too big */ len = 200000; } else { /* Good size */ len = 5000; msgs_wait |= (1 << i); } rd_snprintf(msg, msgsize, "%s test message #%i", argv[0], i); r = rd_kafka_produce(rkt, partition, RD_KAFKA_MSG_F_COPY, msg, len, NULL, 0, msgidp); if (toobig) { if (r != -1) TEST_FAIL("Succeeded to produce too " "large message #%i\n", i); free(msgidp); } else if (r == -1) TEST_FAIL("Failed to produce message #%i: %s\n", i, rd_strerror(errno)); } /* Wait for messages to be delivered. */ while (rd_kafka_outq_len(rk) > 0) rd_kafka_poll(rk, 50); if (msgs_wait != 0) TEST_FAIL("Still waiting for messages: 0x%x\n", msgs_wait); free(msg); /* 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; }
/* Produce a batch of messages to a single partition. */ static void test_single_partition (void) { int partition = 0; int r; rd_kafka_t *rk; rd_kafka_topic_t *rkt; rd_kafka_conf_t *conf; rd_kafka_topic_conf_t *topic_conf; char msg[128]; int msgcnt = 100000; int failcnt = 0; int i; rd_kafka_message_t *rkmessages; int msgcounter = 0; msgid_next = 0; test_conf_init(&conf, &topic_conf, 20); /* Set delivery report callback */ rd_kafka_conf_set_dr_cb(conf, dr_single_partition_cb); rd_kafka_conf_set_opaque(conf, &msgcounter); /* Create kafka instance */ rk = test_create_handle(RD_KAFKA_PRODUCER, conf); TEST_SAY("test_single_partition: Created kafka instance %s\n", rd_kafka_name(rk)); rkt = rd_kafka_topic_new(rk, test_mk_topic_name("0011", 0), topic_conf); if (!rkt) TEST_FAIL("Failed to create topic: %s\n", rd_strerror(errno)); /* Create messages */ rkmessages = calloc(sizeof(*rkmessages), msgcnt); for (i = 0 ; i < msgcnt ; i++) { int *msgidp = malloc(sizeof(*msgidp)); *msgidp = i; rd_snprintf(msg, sizeof(msg), "%s:%s test message #%i", __FILE__, __FUNCTION__, i); rkmessages[i].payload = rd_strdup(msg); rkmessages[i].len = strlen(msg); rkmessages[i]._private = msgidp; } r = rd_kafka_produce_batch(rkt, partition, RD_KAFKA_MSG_F_FREE, rkmessages, msgcnt); /* Scan through messages to check for errors. */ for (i = 0 ; i < msgcnt ; i++) { if (rkmessages[i].err) { failcnt++; if (failcnt < 100) TEST_SAY("Message #%i failed: %s\n", i, rd_kafka_err2str(rkmessages[i].err)); } } /* All messages should've been produced. */ if (r < msgcnt) { TEST_SAY("Not all messages were accepted " "by produce_batch(): %i < %i\n", r, msgcnt); if (msgcnt - r != failcnt) TEST_SAY("Discrepency between failed messages (%i) " "and return value %i (%i - %i)\n", failcnt, msgcnt - r, msgcnt, r); TEST_FAIL("%i/%i messages failed\n", msgcnt - r, msgcnt); } free(rkmessages); TEST_SAY("Single partition: " "Produced %i messages, waiting for deliveries\n", r); msgcounter = msgcnt; /* Wait for messages to be delivered */ test_wait_delivery(rk, &msgcounter); if (fails) TEST_FAIL("%i failures, see previous errors", fails); if (msgid_next != msgcnt) TEST_FAIL("Still waiting for messages: next %i != end %i\n", msgid_next, msgcnt); /* 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; }
/** * Produces 'msgcnt' messages split over 'partition_cnt' partitions. */ static void produce_null_messages (uint64_t testid, const char *topic, int partition_cnt, int msgcnt) { int r; rd_kafka_t *rk; rd_kafka_topic_t *rkt; rd_kafka_conf_t *conf; rd_kafka_topic_conf_t *topic_conf; char errstr[512]; int i; int32_t partition; int msgid = 0; test_conf_init(&conf, &topic_conf, 20); rd_kafka_conf_set_dr_cb(conf, dr_cb); /* Make sure all replicas are in-sync after producing * so that consume test wont fail. */ rd_kafka_topic_conf_set(topic_conf, "request.required.acks", "-1", errstr, sizeof(errstr)); /* Create kafka instance */ rk = test_create_handle(RD_KAFKA_PRODUCER, 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())); /* Produce messages */ prod_msg_remains = msgcnt; for (partition = 0 ; partition < partition_cnt ; partition++) { int batch_cnt = msgcnt / partition_cnt; for (i = 0 ; i < batch_cnt ; i++) { char key[128]; rd_snprintf(key, sizeof(key), "testid=%"PRIu64", partition=%i, msg=%i", testid, (int)partition, msgid); r = rd_kafka_produce(rkt, partition, 0, NULL, 0, key, strlen(key), NULL); if (r == -1) TEST_FAIL("Failed to produce message %i " "to partition %i: %s", msgid, (int)partition, rd_kafka_err2str(rd_kafka_last_error())); msgid++; } } TEST_SAY("Produced %d messages to %d partition(s), " "waiting for deliveries\n", msgcnt, partition_cnt); /* Wait for messages to be delivered */ while (rd_kafka_outq_len(rk) > 0) rd_kafka_poll(rk, 100); if (fails) TEST_FAIL("%i failures, see previous errors", fails); if (prod_msg_remains != 0) TEST_FAIL("Still waiting for %i messages to be produced", prod_msg_remains); else TEST_SAY("All messages delivered\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); }
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 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_0004_conf (int argc, char **argv) { rd_kafka_t *rk; rd_kafka_topic_t *rkt; rd_kafka_conf_t *ignore_conf, *conf, *conf2; rd_kafka_topic_conf_t *ignore_topic_conf, *tconf, *tconf2; char errstr[512]; const char **arr_orig, **arr_dup; size_t cnt_orig, cnt_dup; int i; const char *topic; static const char *gconfs[] = { "message.max.bytes", "12345", /* int property */ "client.id", "my id", /* string property */ "debug", "topic,metadata", /* S2F property */ "topic.blacklist", "__.*", /* #778 */ "auto.offset.reset", "earliest", /* Global->Topic fallthru */ #if WITH_ZLIB "compression.codec", "gzip", /* S2I property */ #endif NULL }; static const char *tconfs[] = { "request.required.acks", "-1", /* int */ "auto.commit.enable", "false", /* bool */ "auto.offset.reset", "error", /* S2I */ "offset.store.path", "my/path", /* string */ NULL }; test_conf_init(&ignore_conf, &ignore_topic_conf, 10); rd_kafka_conf_destroy(ignore_conf); rd_kafka_topic_conf_destroy(ignore_topic_conf); topic = test_mk_topic_name("0004", 0); /* Set up a global config object */ conf = rd_kafka_conf_new(); rd_kafka_conf_set_dr_cb(conf, dr_cb); rd_kafka_conf_set_error_cb(conf, error_cb); for (i = 0 ; gconfs[i] ; i += 2) { if (rd_kafka_conf_set(conf, gconfs[i], gconfs[i+1], errstr, sizeof(errstr)) != RD_KAFKA_CONF_OK) TEST_FAIL("%s\n", errstr); } /* Set up a topic config object */ tconf = rd_kafka_topic_conf_new(); rd_kafka_topic_conf_set_partitioner_cb(tconf, partitioner); rd_kafka_topic_conf_set_opaque(tconf, (void *)0xbeef); for (i = 0 ; tconfs[i] ; i += 2) { if (rd_kafka_topic_conf_set(tconf, tconfs[i], tconfs[i+1], errstr, sizeof(errstr)) != RD_KAFKA_CONF_OK) TEST_FAIL("%s\n", errstr); } /* Verify global config */ arr_orig = rd_kafka_conf_dump(conf, &cnt_orig); conf_verify(__LINE__, arr_orig, cnt_orig, gconfs); /* Verify copied global config */ conf2 = rd_kafka_conf_dup(conf); arr_dup = rd_kafka_conf_dump(conf2, &cnt_dup); conf_verify(__LINE__, arr_dup, cnt_dup, gconfs); conf_cmp("global", arr_orig, cnt_orig, arr_dup, cnt_dup); rd_kafka_conf_dump_free(arr_orig, cnt_orig); rd_kafka_conf_dump_free(arr_dup, cnt_dup); /* Verify topic config */ arr_orig = rd_kafka_topic_conf_dump(tconf, &cnt_orig); conf_verify(__LINE__, arr_orig, cnt_orig, tconfs); /* Verify copied topic config */ tconf2 = rd_kafka_topic_conf_dup(tconf); arr_dup = rd_kafka_topic_conf_dump(tconf2, &cnt_dup); conf_verify(__LINE__, arr_dup, cnt_dup, tconfs); conf_cmp("topic", arr_orig, cnt_orig, arr_dup, cnt_dup); rd_kafka_conf_dump_free(arr_orig, cnt_orig); rd_kafka_conf_dump_free(arr_dup, cnt_dup); /* * Create kafka instances using original and copied confs */ /* original */ rk = test_create_handle(RD_KAFKA_PRODUCER, conf); rkt = rd_kafka_topic_new(rk, topic, tconf); if (!rkt) TEST_FAIL("Failed to create topic: %s\n", rd_strerror(errno)); rd_kafka_topic_destroy(rkt); rd_kafka_destroy(rk); /* copied */ rk = test_create_handle(RD_KAFKA_PRODUCER, conf2); rkt = rd_kafka_topic_new(rk, topic, tconf2); if (!rkt) TEST_FAIL("Failed to create topic: %s\n", rd_strerror(errno)); rd_kafka_topic_destroy(rkt); rd_kafka_destroy(rk); /* Incremental S2F property. * NOTE: The order of fields returned in get() is hardcoded here. */ { static const char *s2fs[] = { "generic,broker,queue,cgrp", "generic,broker,queue,cgrp", "-broker,+queue,topic", "generic,topic,queue,cgrp", "-all,security,-fetch,+metadata", "metadata,security", NULL }; TEST_SAY("Incremental S2F tests\n"); conf = rd_kafka_conf_new(); for (i = 0 ; s2fs[i] ; i += 2) { const char *val; TEST_SAY(" Set: %s\n", s2fs[i]); test_conf_set(conf, "debug", s2fs[i]); val = test_conf_get(conf, "debug"); TEST_SAY(" Now: %s\n", val); if (strcmp(val, s2fs[i+1])) TEST_FAIL_LATER("\n" "Expected: %s\n" " Got: %s", s2fs[i+1], val); } rd_kafka_conf_destroy(conf); } /* Canonical int values, aliases, s2i-verified strings */ { static const struct { const char *prop; const char *val; const char *exp; int is_global; } props[] = { { "request.required.acks", "0", "0" }, { "request.required.acks", "-1", "-1" }, { "request.required.acks", "1", "1" }, { "acks", "3", "3" }, /* alias test */ { "request.required.acks", "393", "393" }, { "request.required.acks", "bad", NULL }, { "request.required.acks", "all", "-1" }, { "request.required.acks", "all", "-1", 1/*fallthru*/ }, { "acks", "0", "0" }, /* alias test */ #if WITH_SASL { "sasl.mechanisms", "GSSAPI", "GSSAPI", 1 }, { "sasl.mechanisms", "PLAIN", "PLAIN", 1 }, { "sasl.mechanisms", "GSSAPI,PLAIN", NULL, 1 }, { "sasl.mechanisms", "", NULL, 1 }, #endif { NULL } }; TEST_SAY("Canonical tests\n"); tconf = rd_kafka_topic_conf_new(); conf = rd_kafka_conf_new(); for (i = 0 ; props[i].prop ; i++) { char dest[64]; size_t destsz; rd_kafka_conf_res_t res; TEST_SAY(" Set: %s=%s expect %s (%s)\n", props[i].prop, props[i].val, props[i].exp, props[i].is_global ? "global":"topic"); /* Set value */ if (props[i].is_global) res = rd_kafka_conf_set(conf, props[i].prop, props[i].val, errstr, sizeof(errstr)); else res = rd_kafka_topic_conf_set(tconf, props[i].prop, props[i].val, errstr, sizeof(errstr)); if ((res == RD_KAFKA_CONF_OK ? 1:0) != (props[i].exp ? 1:0)) TEST_FAIL("Expected %s, got %s", props[i].exp ? "success" : "failure", (res == RD_KAFKA_CONF_OK ? "OK" : (res == RD_KAFKA_CONF_INVALID ? "INVALID" : "UNKNOWN"))); if (!props[i].exp) continue; /* Get value and compare to expected result */ destsz = sizeof(dest); if (props[i].is_global) res = rd_kafka_conf_get(conf, props[i].prop, dest, &destsz); else res = rd_kafka_topic_conf_get(tconf, props[i].prop, dest, &destsz); TEST_ASSERT(res == RD_KAFKA_CONF_OK, ".._conf_get(%s) returned %d", props[i].prop, res); TEST_ASSERT(!strcmp(props[i].exp, dest), "Expected \"%s\", got \"%s\"", props[i].exp, dest); } rd_kafka_topic_conf_destroy(tconf); rd_kafka_conf_destroy(conf); } return 0; }
int main_0062_stats_event (int argc, char **argv) { rd_kafka_t *rk; rd_kafka_conf_t *conf; test_timing_t t_delivery; rd_kafka_queue_t *eventq; const int iterations = 5; int i; test_conf_init(NULL, NULL, 10); /* Set up a global config object */ conf = rd_kafka_conf_new(); rd_kafka_conf_set(conf,"statistics.interval.ms", "100", NULL, 0); rd_kafka_conf_set_events(conf, RD_KAFKA_EVENT_STATS); /* Create kafka instance */ rk = test_create_handle(RD_KAFKA_PRODUCER, conf); eventq = rd_kafka_queue_get_main(rk); /* Wait for stats event */ for (i = 0 ; i < iterations ; i++) { TIMING_START(&t_delivery, "STATS_EVENT"); stats_count = 0; while (stats_count == 0) { rd_kafka_event_t *rkev; rkev = rd_kafka_queue_poll(eventq, 100); switch (rd_kafka_event_type(rkev)) { case RD_KAFKA_EVENT_STATS: TEST_SAY("%s event\n", rd_kafka_event_name(rkev)); handle_stats(rkev); break; case RD_KAFKA_EVENT_NONE: break; default: TEST_SAY("Ignore event: %s\n", rd_kafka_event_name(rkev)); break; } rd_kafka_event_destroy(rkev); } TIMING_STOP(&t_delivery); if (!strcmp(test_mode, "bare")) { /* valgrind is too slow to make this meaningful. */ if (TIMING_DURATION(&t_delivery) < 1000 * 100 * 0.8 || TIMING_DURATION(&t_delivery) > 1000 * 100 * 1.2) TEST_FAIL("Stats duration %.3fms is >= 20%% " "outside statistics.interval.ms 100", (float)TIMING_DURATION(&t_delivery)/ 1000.0f); } } rd_kafka_queue_destroy(eventq); rd_kafka_destroy(rk); return 0; }