int main_0075_retry (int argc, char **argv) { const char *topic = test_mk_topic_name("0075_retry", 1); do_test_low_socket_timeout(topic); return 0; }
static void do_nonexist_commit (void) { rd_kafka_t *rk; char group_id[64]; rd_kafka_conf_t *conf; rd_kafka_topic_conf_t *tconf; rd_kafka_topic_partition_list_t *offsets; const char *unk_topic = test_mk_topic_name(__FUNCTION__, 1); rd_kafka_resp_err_t err; test_conf_init(&conf, &tconf, 20); test_str_id_generate(group_id, sizeof(group_id)); TEST_SAY(_C_MAG "[ do_nonexist_commit group.id %s ]\n", group_id); rk = test_create_consumer(group_id, NULL, conf, tconf, NULL); TEST_SAY("Try nonexist commit\n"); offsets = rd_kafka_topic_partition_list_new(2); rd_kafka_topic_partition_list_add(offsets, unk_topic, 0)->offset = 123; rd_kafka_topic_partition_list_add(offsets, unk_topic, 1)->offset = 456; err = rd_kafka_commit_queue(rk, offsets, NULL, nonexist_offset_commit_cb, NULL); TEST_SAY("nonexist commit returned %s\n", rd_kafka_err2str(err)); if (err != RD_KAFKA_RESP_ERR_NO_ERROR) TEST_FAIL("commit() should succeed, not: %s", rd_kafka_err2str(err)); rd_kafka_topic_partition_list_destroy(offsets); test_consumer_close(rk); rd_kafka_destroy(rk); }
int main_0028_long_topicnames (int argc, char **argv) { const int msgcnt = 1000; uint64_t testid; char topic[256]; rd_kafka_t *rk_c; memset(topic, 'a', sizeof(topic)-1); topic[sizeof(topic)-1] = '\0'; strncpy(topic, test_mk_topic_name(topic, 1), sizeof(topic)-1); TEST_SAY("Using topic name of %d bytes: %s\n", (int)strlen(topic), topic); /* Create topic */ test_create_topic(topic, 1, 1); /* First try a non-verifying consumer. The consumer has been known * to crash when the broker bug kicks in. */ rk_c = test_create_consumer(topic, NULL, NULL, NULL, NULL); test_consumer_subscribe(rk_c, topic); test_consumer_poll_no_msgs("consume.nomsgs", rk_c, 0, 5000); test_consumer_close(rk_c); /* Produce messages */ testid = test_produce_msgs_easy(topic, 0, RD_KAFKA_PARTITION_UA, msgcnt); /* Consume messages */ test_consume_msgs_easy(NULL, topic, testid, msgcnt); return 0; }
int main_0042_many_topics (int argc, char **argv) { char **topics; const int topic_cnt = 20; /* up this as needed, topic creation * takes time so unless hunting a bug * we keep this low to keep the * test suite run time down. */ uint64_t testid; int i; test_conf_init(NULL, NULL, 60); testid = test_id_generate(); /* Generate unique topic names */ topics = malloc(sizeof(*topics) * topic_cnt); for (i = 0 ; i < topic_cnt ; i++) topics[i] = rd_strdup(test_mk_topic_name(__FUNCTION__, 1)); produce_many(topics, topic_cnt, testid); legacy_consume_many(topics, topic_cnt, testid); if (test_broker_version >= TEST_BRKVER(0,9,0,0)) { subscribe_consume_many(topics, topic_cnt, testid); assign_consume_many(topics, topic_cnt, testid); } for (i = 0 ; i < topic_cnt ; i++) free(topics[i]); free(topics); return 0; }
/** * 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; }
int main_0021_rkt_destroy (int argc, char **argv) { const char *topic = test_mk_topic_name(__FUNCTION__, 0); rd_kafka_t *rk; rd_kafka_topic_t *rkt; const int msgcnt = 1000; uint64_t testid; int remains = 0; test_conf_init(NULL, NULL, 10); testid = test_id_generate(); rk = test_create_producer(); rkt = test_create_producer_topic(rk, topic, NULL); test_produce_msgs_nowait(rk, rkt, testid, RD_KAFKA_PARTITION_UA, 0, msgcnt, NULL, 0, &remains); TEST_ASSERT(msgcnt == remains, "Only %d/%d messages produced", remains, msgcnt); rd_kafka_topic_destroy(rkt); test_wait_delivery(rk, &remains); rd_kafka_destroy(rk); return 0; }
int main_0073_headers (int argc, char **argv) { const char *topic = test_mk_topic_name(__FUNCTION__ + 5, 1); const int msgcnt = 10; do_produce(topic, msgcnt); do_consume(topic, msgcnt); return 0; }
/** * @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_test_non_exist_and_partchange (void) { char *topic_a = rd_strdup(test_mk_topic_name("topic_a", 1)); rd_kafka_t *rk; rd_kafka_conf_t *conf; rd_kafka_queue_t *queue; /** * Test #1: * - Subscribe to non-existing topic. * - Verify empty assignment * - Create topic * - Verify new assignment containing topic */ TEST_SAY("#1 & #2 testing\n"); test_conf_init(&conf, NULL, 60); /* Decrease metadata interval to speed up topic change discovery. */ test_conf_set(conf, "metadata.max.age.ms", "5000"); rd_kafka_conf_set_events(conf, RD_KAFKA_EVENT_REBALANCE); rk = test_create_consumer(test_str_id_generate_tmp(), NULL, conf, NULL, NULL); queue = rd_kafka_queue_get_consumer(rk); TEST_SAY("#1: Subscribing to %s\n", topic_a); test_consumer_subscribe(rk, topic_a); /* Should not see a rebalance since no topics are matched. */ await_no_rebalance("#1: empty", rk, queue, 10000); TEST_SAY("#1: creating topic %s\n", topic_a); test_create_topic(topic_a, 2, 1); await_assignment("#1: proper", rk, queue, 1, topic_a, 2); /** * Test #2 (continue with #1 consumer) * - Increase the partition count * - Verify updated assignment */ test_kafka_topics("--alter --topic %s --partitions 4", topic_a); await_revoke("#2", rk, queue); await_assignment("#2: more partitions", rk, queue, 1, topic_a, 4); test_consumer_close(rk); rd_kafka_queue_destroy(queue); rd_kafka_destroy(rk); rd_free(topic_a); }
int main_0064_interceptors (int argc, char **argv) { const char *topic = test_mk_topic_name(__FUNCTION__, 1); do_test_producer(topic); do_test_consumer(topic); do_test_conf_copy(topic); return 0; }
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; }
int main_0041_fetch_max_bytes (int argc, char **argv) { const char *topic = test_mk_topic_name(__FUNCTION__, 1); const int partition = 0; const int msgcnt = 2*1000; const int MAX_BYTES = 100000; uint64_t testid; rd_kafka_conf_t *conf; rd_kafka_t *rk; rd_kafka_topic_t *rkt; test_conf_init(NULL, NULL, 60); testid = test_id_generate(); rk = test_create_producer(); rkt = test_create_producer_topic(rk, topic, NULL); test_produce_msgs(rk, rkt, testid, partition, 0, msgcnt/2, NULL, MAX_BYTES/10); test_produce_msgs(rk, rkt, testid, partition, msgcnt/2, msgcnt/2, NULL, MAX_BYTES*5); rd_kafka_topic_destroy(rkt); rd_kafka_destroy(rk); TEST_SAY("Creating consumer\n"); test_conf_init(&conf, NULL, 0); test_conf_set(conf, "fetch.message.max.bytes", tsprintf("%d", MAX_BYTES)); rk = test_create_consumer(NULL, NULL, conf, NULL); rkt = rd_kafka_topic_new(rk, topic, NULL); test_consumer_start("CONSUME", rkt, partition, RD_KAFKA_OFFSET_BEGINNING); test_consume_msgs("CONSUME", rkt, testid, partition, TEST_NO_SEEK, 0, msgcnt, 1); test_consumer_stop("CONSUME", rkt, partition); rd_kafka_topic_destroy(rkt); rd_kafka_destroy(rk); return 0; }
int main_0036_partial_fetch (int argc, char **argv) { const char *topic = test_mk_topic_name(__FUNCTION__, 1); const int partition = 0; const int msgcnt = 100; const int msgsize = 1000; uint64_t testid; rd_kafka_conf_t *conf; rd_kafka_t *rk; rd_kafka_topic_t *rkt; TEST_SAY("Producing %d messages of size %d to %s [%d]\n", msgcnt, (int)msgsize, topic, partition); testid = test_id_generate(); rk = test_create_producer(); rkt = test_create_producer_topic(rk, topic, NULL); test_produce_msgs(rk, rkt, testid, partition, 0, msgcnt, NULL, msgsize); rd_kafka_topic_destroy(rkt); rd_kafka_destroy(rk); TEST_SAY("Creating consumer\n"); test_conf_init(&conf, NULL, 0); /* This should fetch 1.5 messages per fetch, thus resulting in * partial fetches, hopefully. */ test_conf_set(conf, "fetch.message.max.bytes", "1500"); 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); test_consume_msgs("CONSUME", rkt, testid, partition, TEST_NO_SEEK, 0, msgcnt, 1); test_consumer_stop("CONSUME", rkt, partition); rd_kafka_topic_destroy(rkt); rd_kafka_destroy(rk); return 0; }
int main_0030_offset_commit (int argc, char **argv) { topic = test_mk_topic_name(__FUNCTION__, 1); testid = test_produce_msgs_easy(topic, 0, partition, msgcnt); do_offset_test("AUTO.COMMIT & AUTO.STORE", 1 /* enable.auto.commit */, 1 /* enable.auto.offset.store */, 0 /* not used. */); do_offset_test("AUTO.COMMIT & MANUAL.STORE", 1 /* enable.auto.commit */, 0 /* enable.auto.offset.store */, 0 /* not used */); do_offset_test("MANUAL.COMMIT.ASYNC & AUTO.STORE", 0 /* enable.auto.commit */, 1 /* enable.auto.offset.store */, 1 /* async */); do_offset_test("MANUAL.COMMIT.SYNC & AUTO.STORE", 0 /* enable.auto.commit */, 1 /* enable.auto.offset.store */, 0 /* async */); do_offset_test("MANUAL.COMMIT.ASYNC & MANUAL.STORE", 0 /* enable.auto.commit */, 0 /* enable.auto.offset.store */, 1 /* sync */); do_offset_test("MANUAL.COMMIT.SYNC & MANUAL.STORE", 0 /* enable.auto.commit */, 0 /* enable.auto.offset.store */, 0 /* sync */); do_empty_commit(); do_nonexist_commit(); return 0; }
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 Verify that an unclean rd_kafka_destroy() does not hang. */ static void do_test_unclean_destroy (rd_kafka_type_t cltype, int with_mainq) { rd_kafka_t *rk; char errstr[512]; rd_kafka_conf_t *conf; rd_kafka_queue_t *q; rd_kafka_NewTopic_t *topic; test_timing_t t_destroy; test_conf_init(&conf, NULL, 0); rk = rd_kafka_new(cltype, conf, errstr, sizeof(errstr)); TEST_ASSERT(rk, "kafka_new(%d): %s", cltype, errstr); TEST_SAY(_C_MAG "[ Test unclean destroy for %s using %s]\n", rd_kafka_name(rk), with_mainq ? "mainq" : "tempq"); if (with_mainq) q = rd_kafka_queue_get_main(rk); else q = rd_kafka_queue_new(rk); topic = rd_kafka_NewTopic_new(test_mk_topic_name(__FUNCTION__, 1), 3, 1, NULL, 0); rd_kafka_CreateTopics(rk, &topic, 1, NULL, q); rd_kafka_NewTopic_destroy(topic); rd_kafka_queue_destroy(q); TEST_SAY("Giving rd_kafka_destroy() 5s to finish, " "despite Admin API request being processed\n"); test_timeout_set(5); TIMING_START(&t_destroy, "rd_kafka_destroy()"); rd_kafka_destroy(rk); TIMING_STOP(&t_destroy); /* Restore timeout */ test_timeout_set(60);; }
/** * Produce to two partitions. * Consume with standard interface from both, one after the other. * Consume with queue interface from both, simultanously. */ static void test_produce_consume (void) { int msgcnt = 1000; int partition_cnt = 2; int i; uint64_t testid; int msg_base = 0; const char *topic; /* Generate a testid so we can differentiate messages * from other tests */ testid = test_id_generate(); /* Read test.conf to configure topic name */ test_conf_init(NULL, NULL, 20); topic = test_mk_topic_name("0012", 1); TEST_SAY("Topic %s, testid %"PRIu64"\n", topic, testid); /* Produce messages */ produce_messages(testid, topic, partition_cnt, msgcnt); /* Consume messages with standard interface */ verify_consumed_msg_reset(msgcnt); for (i = 0 ; i < partition_cnt ; i++) { consume_messages(testid, topic, i, msg_base, msgcnt / partition_cnt, msgcnt); msg_base += msgcnt / partition_cnt; } verify_consumed_msg_check(); /* Consume messages with queue interface */ verify_consumed_msg_reset(msgcnt); consume_messages_with_queues(testid, topic, partition_cnt, msgcnt); verify_consumed_msg_check(); return; }
/** * @brief Test creation of partitions * * */ static void do_test_CreatePartitions (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); #define MY_CRP_TOPICS_CNT 9 char *topics[MY_CRP_TOPICS_CNT]; rd_kafka_NewTopic_t *new_topics[MY_CRP_TOPICS_CNT]; rd_kafka_NewPartitions_t *crp_topics[MY_CRP_TOPICS_CNT]; rd_kafka_AdminOptions_t *options = NULL; /* Expected topics in metadata */ rd_kafka_metadata_topic_t exp_mdtopics[MY_CRP_TOPICS_CNT] = {{0}}; rd_kafka_metadata_partition_t exp_mdparts[2] = {{0}}; int exp_mdtopic_cnt = 0; int i; char errstr[512]; rd_kafka_resp_err_t err; test_timing_t timing; int metadata_tmout; int num_replicas = (int)avail_broker_cnt; TEST_SAY(_C_MAG "[ %s CreatePartitions with %s, op_timeout %d ]\n", rd_kafka_name(rk), what, op_timeout); /* Set up two expected partitions with different replication sets * so they can be matched by the metadata checker later. * Even partitions use exp_mdparts[0] while odd partitions * use exp_mdparts[1]. */ /* Set valid replica assignments (even, and odd (reverse) ) */ exp_mdparts[0].replicas = rd_alloca(sizeof(*exp_mdparts[0].replicas) * num_replicas); exp_mdparts[1].replicas = rd_alloca(sizeof(*exp_mdparts[1].replicas) * num_replicas); exp_mdparts[0].replica_cnt = num_replicas; exp_mdparts[1].replica_cnt = num_replicas; for (i = 0 ; i < num_replicas ; i++) { exp_mdparts[0].replicas[i] = avail_brokers[i]; exp_mdparts[1].replicas[i] = avail_brokers[num_replicas-i-1]; } /** * Construct CreatePartitions array */ for (i = 0 ; i < MY_CRP_TOPICS_CNT ; i++) { char *topic = rd_strdup(test_mk_topic_name(__FUNCTION__, 1)); int initial_part_cnt = 1 + (i * 2); int new_part_cnt = 1 + (i / 2); int final_part_cnt = initial_part_cnt + new_part_cnt; int set_replicas = !(i % 2); int pi; topics[i] = topic; /* Topic to create with initial partition count */ new_topics[i] = rd_kafka_NewTopic_new(topic, initial_part_cnt, set_replicas ? -1 : num_replicas, NULL, 0); /* .. and later add more partitions to */ crp_topics[i] = rd_kafka_NewPartitions_new(topic, final_part_cnt, errstr, sizeof(errstr)); if (set_replicas) { exp_mdtopics[exp_mdtopic_cnt].partitions = rd_alloca(final_part_cnt * sizeof(*exp_mdtopics[exp_mdtopic_cnt]. partitions)); for (pi = 0 ; pi < final_part_cnt ; pi++) { const rd_kafka_metadata_partition_t *exp_mdp = &exp_mdparts[pi & 1]; exp_mdtopics[exp_mdtopic_cnt]. partitions[pi] = *exp_mdp; /* copy */ exp_mdtopics[exp_mdtopic_cnt]. partitions[pi].id = pi; if (pi < initial_part_cnt) { /* Set replica assignment * for initial partitions */ err = rd_kafka_NewTopic_set_replica_assignment( new_topics[i], pi, exp_mdp->replicas, (size_t)exp_mdp->replica_cnt, errstr, sizeof(errstr)); TEST_ASSERT(!err, "NewTopic_set_replica_assignment: %s", errstr); } else { /* Set replica assignment for new * partitions */ err = rd_kafka_NewPartitions_set_replica_assignment( crp_topics[i], pi - initial_part_cnt, exp_mdp->replicas, (size_t)exp_mdp->replica_cnt, errstr, sizeof(errstr)); TEST_ASSERT(!err, "NewPartitions_set_replica_assignment: %s", errstr); } } } TEST_SAY(_C_YEL "Topic %s with %d initial partitions will grow " "by %d to %d total partitions with%s replicas set\n", topics[i], initial_part_cnt, new_part_cnt, final_part_cnt, set_replicas ? "" : "out"); exp_mdtopics[exp_mdtopic_cnt].topic = topic; exp_mdtopics[exp_mdtopic_cnt].partition_cnt = final_part_cnt; exp_mdtopic_cnt++; } 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 topics with initial partition count */ TIMING_START(&timing, "CreateTopics"); TEST_SAY("Creating topics with initial partition counts\n"); rd_kafka_CreateTopics(rk, new_topics, MY_CRP_TOPICS_CNT, options, q); TIMING_ASSERT_LATER(&timing, 0, 50); err = test_wait_topic_admin_result(q, RD_KAFKA_EVENT_CREATETOPICS_RESULT, NULL, 15000); TEST_ASSERT(!err, "CreateTopics failed: %s", rd_kafka_err2str(err)); rd_kafka_NewTopic_destroy_array(new_topics, MY_CRP_TOPICS_CNT); /* * Create new partitions */ TIMING_START(&timing, "CreatePartitions"); TEST_SAY("Creating partitions\n"); rd_kafka_CreatePartitions(rk, crp_topics, MY_CRP_TOPICS_CNT, options, q); TIMING_ASSERT_LATER(&timing, 0, 50); err = test_wait_topic_admin_result(q, RD_KAFKA_EVENT_CREATEPARTITIONS_RESULT, NULL, 15000); TEST_ASSERT(!err, "CreatePartitions failed: %s", rd_kafka_err2str(err)); rd_kafka_NewPartitions_destroy_array(crp_topics, MY_CRP_TOPICS_CNT); /** * 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, exp_mdtopics, exp_mdtopic_cnt, NULL, 0, metadata_tmout); for (i = 0 ; i < MY_CRP_TOPICS_CNT ; i++) rd_free(topics[i]); if (options) rd_kafka_AdminOptions_destroy(options); if (!useq) rd_kafka_queue_destroy(q); #undef MY_CRP_TOPICS_CNT }
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 }
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; }
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; }
int main (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 */ "compression.codec", "gzip", /* S2I property */ 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("generic", 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 = rd_kafka_new(RD_KAFKA_PRODUCER, conf, errstr, sizeof(errstr)); if (!rk) TEST_FAIL("Failed to create rdkafka instance: %s\n", errstr); rkt = rd_kafka_topic_new(rk, topic, tconf); if (!rkt) TEST_FAIL("Failed to create topic: %s\n", strerror(errno)); rd_kafka_topic_destroy(rkt); rd_kafka_destroy(rk); /* copied */ rk = rd_kafka_new(RD_KAFKA_PRODUCER, conf2, errstr, sizeof(errstr)); if (!rk) TEST_FAIL("Failed to create rdkafka instance: %s\n", errstr); rkt = rd_kafka_topic_new(rk, topic, tconf2); if (!rkt) TEST_FAIL("Failed to create topic: %s\n", strerror(errno)); rd_kafka_topic_destroy(rkt); rd_kafka_destroy(rk); /* Wait for everything to be cleaned up since broker destroys are * handled in its own thread. */ test_wait_exit(2); /* If we havent failed at this point then * there were no threads leaked */ return 0; }
/** * @brief Test AlterConfigs */ static void do_test_AlterConfigs (rd_kafka_t *rk, rd_kafka_queue_t *rkqu) { #define MY_CONFRES_CNT 3 char *topics[MY_CONFRES_CNT]; rd_kafka_ConfigResource_t *configs[MY_CONFRES_CNT]; rd_kafka_AdminOptions_t *options; rd_kafka_resp_err_t exp_err[MY_CONFRES_CNT]; rd_kafka_event_t *rkev; rd_kafka_resp_err_t err; const rd_kafka_AlterConfigs_result_t *res; const rd_kafka_ConfigResource_t **rconfigs; size_t rconfig_cnt; char errstr[128]; const char *errstr2; int ci = 0; int i; int fails = 0; /* * Only create one topic, the others will be non-existent. */ for (i = 0 ; i < MY_CONFRES_CNT ; i++) rd_strdupa(&topics[i], test_mk_topic_name(__FUNCTION__, 1)); test_CreateTopics_simple(rk, NULL, topics, 1, 1, NULL); /* * ConfigResource #0: valid topic config */ configs[ci] = rd_kafka_ConfigResource_new( RD_KAFKA_RESOURCE_TOPIC, topics[ci]); err = rd_kafka_ConfigResource_set_config(configs[ci], "compression.type", "gzip"); TEST_ASSERT(!err, "%s", rd_kafka_err2str(err)); err = rd_kafka_ConfigResource_set_config(configs[ci], "flush.ms", "12345678"); TEST_ASSERT(!err, "%s", rd_kafka_err2str(err)); exp_err[ci] = RD_KAFKA_RESP_ERR_NO_ERROR; ci++; if (test_broker_version >= TEST_BRKVER(1, 1, 0, 0)) { /* * ConfigResource #1: valid broker config */ configs[ci] = rd_kafka_ConfigResource_new( RD_KAFKA_RESOURCE_BROKER, tsprintf("%"PRId32, avail_brokers[0])); err = rd_kafka_ConfigResource_set_config( configs[ci], "sasl.kerberos.min.time.before.relogin", "58000"); TEST_ASSERT(!err, "%s", rd_kafka_err2str(err)); exp_err[ci] = RD_KAFKA_RESP_ERR_NO_ERROR; ci++; } else { TEST_WARN("Skipping RESOURCE_BROKER test on unsupported " "broker version\n"); } /* * ConfigResource #2: valid topic config, non-existent topic */ configs[ci] = rd_kafka_ConfigResource_new( RD_KAFKA_RESOURCE_TOPIC, topics[ci]); err = rd_kafka_ConfigResource_set_config(configs[ci], "compression.type", "lz4"); TEST_ASSERT(!err, "%s", rd_kafka_err2str(err)); err = rd_kafka_ConfigResource_set_config(configs[ci], "offset.metadata.max.bytes", "12345"); TEST_ASSERT(!err, "%s", rd_kafka_err2str(err)); exp_err[ci] = RD_KAFKA_RESP_ERR_UNKNOWN; ci++; /* * Timeout options */ options = rd_kafka_AdminOptions_new(rk, RD_KAFKA_ADMIN_OP_ALTERCONFIGS); err = rd_kafka_AdminOptions_set_request_timeout(options, 10000, errstr, sizeof(errstr)); TEST_ASSERT(!err, "%s", errstr); /* * Fire off request */ rd_kafka_AlterConfigs(rk, configs, ci, options, rkqu); rd_kafka_AdminOptions_destroy(options); /* * Wait for result */ rkev = test_wait_admin_result(rkqu, RD_KAFKA_EVENT_ALTERCONFIGS_RESULT, 10000+1000); /* * Extract result */ res = rd_kafka_event_AlterConfigs_result(rkev); TEST_ASSERT(res, "Expected AlterConfigs result, not %s", rd_kafka_event_name(rkev)); err = rd_kafka_event_error(rkev); errstr2 = rd_kafka_event_error_string(rkev); TEST_ASSERT(!err, "Expected success, not %s: %s", rd_kafka_err2name(err), errstr2); rconfigs = rd_kafka_AlterConfigs_result_resources(res, &rconfig_cnt); TEST_ASSERT((int)rconfig_cnt == ci, "Expected %d result resources, got %"PRIusz"\n", ci, rconfig_cnt); /* * Verify status per resource */ for (i = 0 ; i < (int)rconfig_cnt ; i++) { const rd_kafka_ConfigEntry_t **entries; size_t entry_cnt; err = rd_kafka_ConfigResource_error(rconfigs[i]); errstr2 = rd_kafka_ConfigResource_error_string(rconfigs[i]); entries = rd_kafka_ConfigResource_configs(rconfigs[i], &entry_cnt); TEST_SAY("ConfigResource #%d: type %s (%d), \"%s\": " "%"PRIusz" ConfigEntries, error %s (%s)\n", i, rd_kafka_ResourceType_name( rd_kafka_ConfigResource_type(rconfigs[i])), rd_kafka_ConfigResource_type(rconfigs[i]), rd_kafka_ConfigResource_name(rconfigs[i]), entry_cnt, rd_kafka_err2name(err), errstr2 ? errstr2 : ""); test_print_ConfigEntry_array(entries, entry_cnt, 1); if (rd_kafka_ConfigResource_type(rconfigs[i]) != rd_kafka_ConfigResource_type(configs[i]) || strcmp(rd_kafka_ConfigResource_name(rconfigs[i]), rd_kafka_ConfigResource_name(configs[i]))) { TEST_FAIL_LATER( "ConfigResource #%d: " "expected type %s name %s, " "got type %s name %s", i, rd_kafka_ResourceType_name(rd_kafka_ConfigResource_type(configs[i])), rd_kafka_ConfigResource_name(configs[i]), rd_kafka_ResourceType_name(rd_kafka_ConfigResource_type(rconfigs[i])), rd_kafka_ConfigResource_name(rconfigs[i])); fails++; continue; } if (err != exp_err[i]) { TEST_FAIL_LATER("ConfigResource #%d: " "expected %s (%d), got %s (%s)", i, rd_kafka_err2name(exp_err[i]), exp_err[i], rd_kafka_err2name(err), errstr2 ? errstr2 : ""); fails++; } } TEST_ASSERT(!fails, "See %d previous failure(s)", fails); rd_kafka_event_destroy(rkev); rd_kafka_ConfigResource_destroy_array(configs, ci); #undef MY_CONFRES_CNT }
/** * @brief Test DescribeConfigs */ static void do_test_DescribeConfigs (rd_kafka_t *rk, rd_kafka_queue_t *rkqu) { #define MY_CONFRES_CNT 3 char *topics[MY_CONFRES_CNT]; rd_kafka_ConfigResource_t *configs[MY_CONFRES_CNT]; rd_kafka_AdminOptions_t *options; rd_kafka_resp_err_t exp_err[MY_CONFRES_CNT]; rd_kafka_event_t *rkev; rd_kafka_resp_err_t err; const rd_kafka_DescribeConfigs_result_t *res; const rd_kafka_ConfigResource_t **rconfigs; size_t rconfig_cnt; char errstr[128]; const char *errstr2; int ci = 0; int i; int fails = 0; /* * Only create one topic, the others will be non-existent. */ rd_strdupa(&topics[0], test_mk_topic_name("DescribeConfigs_exist", 1)); for (i = 1 ; i < MY_CONFRES_CNT ; i++) rd_strdupa(&topics[i], test_mk_topic_name("DescribeConfigs_notexist", 1)); test_CreateTopics_simple(rk, NULL, topics, 1, 1, NULL); /* * ConfigResource #0: topic config, no config entries. */ configs[ci] = rd_kafka_ConfigResource_new( RD_KAFKA_RESOURCE_TOPIC, topics[ci]); exp_err[ci] = RD_KAFKA_RESP_ERR_NO_ERROR; ci++; /* * ConfigResource #1:broker config, no config entries */ configs[ci] = rd_kafka_ConfigResource_new( RD_KAFKA_RESOURCE_BROKER, tsprintf("%"PRId32, avail_brokers[0])); exp_err[ci] = RD_KAFKA_RESP_ERR_NO_ERROR; ci++; /* * ConfigResource #2: topic config, non-existent topic, no config entr. */ configs[ci] = rd_kafka_ConfigResource_new( RD_KAFKA_RESOURCE_TOPIC, topics[ci]); /* FIXME: This is a bug in the broker (<v2.0.0), it returns a full response * for unknown topics. * https://issues.apache.org/jira/browse/KAFKA-6778 */ if (test_broker_version < TEST_BRKVER(2,0,0,0)) exp_err[ci] = RD_KAFKA_RESP_ERR_NO_ERROR; else exp_err[ci] = RD_KAFKA_RESP_ERR_UNKNOWN_TOPIC_OR_PART; ci++; /* * Timeout options */ options = rd_kafka_AdminOptions_new(rk, RD_KAFKA_ADMIN_OP_ANY); err = rd_kafka_AdminOptions_set_request_timeout(options, 10000, errstr, sizeof(errstr)); TEST_ASSERT(!err, "%s", errstr); /* * Fire off request */ rd_kafka_DescribeConfigs(rk, configs, ci, options, rkqu); rd_kafka_AdminOptions_destroy(options); /* * Wait for result */ rkev = test_wait_admin_result(rkqu, RD_KAFKA_EVENT_DESCRIBECONFIGS_RESULT, 10000+1000); /* * Extract result */ res = rd_kafka_event_DescribeConfigs_result(rkev); TEST_ASSERT(res, "Expected DescribeConfigs result, not %s", rd_kafka_event_name(rkev)); err = rd_kafka_event_error(rkev); errstr2 = rd_kafka_event_error_string(rkev); TEST_ASSERT(!err, "Expected success, not %s: %s", rd_kafka_err2name(err), errstr2); rconfigs = rd_kafka_DescribeConfigs_result_resources(res, &rconfig_cnt); TEST_ASSERT((int)rconfig_cnt == ci, "Expected %d result resources, got %"PRIusz"\n", ci, rconfig_cnt); /* * Verify status per resource */ for (i = 0 ; i < (int)rconfig_cnt ; i++) { const rd_kafka_ConfigEntry_t **entries; size_t entry_cnt; err = rd_kafka_ConfigResource_error(rconfigs[i]); errstr2 = rd_kafka_ConfigResource_error_string(rconfigs[i]); entries = rd_kafka_ConfigResource_configs(rconfigs[i], &entry_cnt); TEST_SAY("ConfigResource #%d: type %s (%d), \"%s\": " "%"PRIusz" ConfigEntries, error %s (%s)\n", i, rd_kafka_ResourceType_name( rd_kafka_ConfigResource_type(rconfigs[i])), rd_kafka_ConfigResource_type(rconfigs[i]), rd_kafka_ConfigResource_name(rconfigs[i]), entry_cnt, rd_kafka_err2name(err), errstr2 ? errstr2 : ""); test_print_ConfigEntry_array(entries, entry_cnt, 1); if (rd_kafka_ConfigResource_type(rconfigs[i]) != rd_kafka_ConfigResource_type(configs[i]) || strcmp(rd_kafka_ConfigResource_name(rconfigs[i]), rd_kafka_ConfigResource_name(configs[i]))) { TEST_FAIL_LATER( "ConfigResource #%d: " "expected type %s name %s, " "got type %s name %s", i, rd_kafka_ResourceType_name(rd_kafka_ConfigResource_type(configs[i])), rd_kafka_ConfigResource_name(configs[i]), rd_kafka_ResourceType_name(rd_kafka_ConfigResource_type(rconfigs[i])), rd_kafka_ConfigResource_name(rconfigs[i])); fails++; continue; } if (err != exp_err[i]) { TEST_FAIL_LATER("ConfigResource #%d: " "expected %s (%d), got %s (%s)", i, rd_kafka_err2name(exp_err[i]), exp_err[i], rd_kafka_err2name(err), errstr2 ? errstr2 : ""); fails++; } } TEST_ASSERT(!fails, "See %d previous failure(s)", fails); rd_kafka_event_destroy(rkev); rd_kafka_ConfigResource_destroy_array(configs, ci); #undef MY_CONFRES_CNT }
int main_0008_reqacks (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[128]; int msgcnt = 100; int i; int reqacks; int idbase = 0; const char *topic = NULL; TEST_SAY("\033[33mNOTE! This test requires at " "least 3 brokers!\033[0m\n"); TEST_SAY("\033[33mNOTE! This test requires " "default.replication.factor=3 to be configured on " "all brokers!\033[0m\n"); /* Try different request.required.acks settings (issue #75) */ for (reqacks = -1 ; reqacks <= 1 ; reqacks++) { char tmp[10]; test_conf_init(&conf, &topic_conf, 10); if (!topic) topic = test_mk_topic_name("0008", 0); rd_snprintf(tmp, sizeof(tmp), "%i", reqacks); if (rd_kafka_topic_conf_set(topic_conf, "request.required.acks", tmp, errstr, sizeof(errstr)) != RD_KAFKA_CONF_OK) TEST_FAIL("%s", errstr); /* Set delivery report callback */ rd_kafka_conf_set_dr_cb(conf, dr_cb); /* Create kafka instance */ rk = rd_kafka_new(RD_KAFKA_PRODUCER, conf, errstr, sizeof(errstr)); if (!rk) TEST_FAIL("Failed to create rdkafka instance: %s\n", errstr); TEST_SAY("Created kafka instance %s with required acks %i\n", rd_kafka_name(rk), reqacks); rkt = rd_kafka_topic_new(rk, topic, topic_conf); if (!rkt) TEST_FAIL("Failed to create topic: %s\n", rd_strerror(errno)); /* Produce messages */ for (i = 0 ; i < msgcnt ; i++) { int *msgidp = malloc(sizeof(*msgidp)); *msgidp = idbase + i; rd_snprintf(msg, sizeof(msg), "%s test message #%i (acks=%i)", argv[0], *msgidp, reqacks); 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", *msgidp, rd_strerror(errno)); } TEST_SAY("Produced %i messages, waiting for deliveries\n", msgcnt); /* Wait for messages to time out */ while (rd_kafka_outq_len(rk) > 0) rd_kafka_poll(rk, 50); if (fails) TEST_FAIL("%i failures, see previous errors", fails); if (msgid_next != idbase + msgcnt) TEST_FAIL("Still waiting for messages: " "next %i != end %i\n", msgid_next, msgcnt); idbase += i; /* 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_0093_holb_consumer (int argc, char **argv) { const char *topic = test_mk_topic_name("0093_holb_consumer", 1); int64_t testid; const int msgcnt = 100; struct _consumer c[_CONSUMER_CNT] = RD_ZERO_INIT; rd_kafka_conf_t *conf; testid = test_id_generate(); test_conf_init(&conf, NULL, 60); test_create_topic(topic, 1, 1); test_produce_msgs_easy(topic, testid, 0, msgcnt); test_conf_set(conf, "session.timeout.ms", "6000"); test_conf_set(conf, "max.poll.interval.ms", "20000"); test_conf_set(conf, "socket.timeout.ms", "3000"); test_conf_set(conf, "auto.offset.reset", "earliest"); /* Trigger other requests often */ test_conf_set(conf, "topic.metadata.refresh.interval.ms", "500"); rd_kafka_conf_set_rebalance_cb(conf, rebalance_cb); rd_kafka_conf_set_opaque(conf, &c[0]); c[0].rk = test_create_consumer(topic, NULL, rd_kafka_conf_dup(conf), NULL); rd_kafka_conf_set_opaque(conf, &c[1]); c[1].rk = test_create_consumer(topic, NULL, conf, NULL); test_consumer_subscribe(c[0].rk, topic); /* c0: assign */ c[0].max_rebalance_cnt = 1; /* c1: none, hasn't joined yet */ c[1].max_rebalance_cnt = 0; TEST_SAY("Waiting for c[0] assignment\n"); while (1) { rd_kafka_topic_partition_list_t *parts = NULL; do_consume(&c[0], 1/*1s*/); if (rd_kafka_assignment(c[0].rk, &parts) != RD_KAFKA_RESP_ERR_NO_ERROR || !parts || parts->cnt == 0) { if (parts) rd_kafka_topic_partition_list_destroy(parts); continue; } TEST_SAY("%s got assignment of %d partition(s)\n", rd_kafka_name(c[0].rk), parts->cnt); rd_kafka_topic_partition_list_destroy(parts); break; } TEST_SAY("c[0] got assignment, consuming..\n"); do_consume(&c[0], 5/*5s*/); TEST_SAY("Joining second consumer\n"); test_consumer_subscribe(c[1].rk, topic); /* Just poll second consumer for 10s, the rebalance will not * finish until the first consumer polls */ do_consume(&c[1], 10/*10s*/); /* c0: the next call to do_consume/poll will trigger * its rebalance callback, first revoke then assign. */ c[0].max_rebalance_cnt += 2; /* c1: first rebalance */ c[1].max_rebalance_cnt++; TEST_SAY("Expected rebalances: c[0]: %d/%d, c[1]: %d/%d\n", c[0].rebalance_cnt, c[0].max_rebalance_cnt, c[1].rebalance_cnt, c[1].max_rebalance_cnt); /* Let rebalances kick in, then consume messages. */ while (c[0].cnt + c[1].cnt < msgcnt) { do_consume(&c[0], 0); do_consume(&c[1], 0); } /* Allow the extra revoke rebalance on close() */ c[0].max_rebalance_cnt++; c[1].max_rebalance_cnt++; test_consumer_close(c[0].rk); test_consumer_close(c[1].rk); rd_kafka_destroy(c[0].rk); rd_kafka_destroy(c[1].rk); return 0; }
/* @remark This test will fail if auto topic creation is enabled on the broker * since the client will issue a topic-creating metadata request to find * a new leader when the topic is removed. * * To run with trivup, do: * ./interactive_broker_version.py .. -conf '{"auto_create_topics":"false"}' .. * TESTS=0045 ./run-test.sh -k ./merged */ static void do_test_topic_remove (void) { char *topic_f = rd_strdup(test_mk_topic_name("topic_f", 1)); char *topic_g = rd_strdup(test_mk_topic_name("topic_g", 1)); int parts_f = 5; int parts_g = 9; rd_kafka_t *rk; rd_kafka_conf_t *conf; rd_kafka_queue_t *queue; rd_kafka_topic_partition_list_t *topics; rd_kafka_resp_err_t err; /** * Topic removal test: * - Create topic f & g * - Subscribe to f & g * - Verify f & g assignment * - Remove topic f * - Verify g assignment * - Remove topic g * - Verify empty assignment */ TEST_SAY("Topic removal testing\n"); test_conf_init(&conf, NULL, 60); /* Decrease metadata interval to speed up topic change discovery. */ test_conf_set(conf, "metadata.max.age.ms", "5000"); rd_kafka_conf_set_events(conf, RD_KAFKA_EVENT_REBALANCE); rk = test_create_consumer(test_str_id_generate_tmp(), NULL, conf, NULL, NULL); queue = rd_kafka_queue_get_consumer(rk); TEST_SAY("Topic removal: creating topic %s (subscribed)\n", topic_f); test_create_topic(topic_f, parts_f, 1); TEST_SAY("Topic removal: creating topic %s (subscribed)\n", topic_g); test_create_topic(topic_g, parts_g, 1); rd_sleep(1); // FIXME: do check&wait loop instead TEST_SAY("Topic removal: Subscribing to %s & %s\n", topic_f, topic_g); topics = rd_kafka_topic_partition_list_new(2); rd_kafka_topic_partition_list_add(topics, topic_f, RD_KAFKA_PARTITION_UA); rd_kafka_topic_partition_list_add(topics, topic_g, RD_KAFKA_PARTITION_UA); err = rd_kafka_subscribe(rk, topics); TEST_ASSERT(err == RD_KAFKA_RESP_ERR_NO_ERROR, "%s", rd_kafka_err2str(err)); rd_kafka_topic_partition_list_destroy(topics); await_assignment("Topic removal: both topics exist", rk, queue, 2, topic_f, parts_f, topic_g, parts_g); TEST_SAY("Topic removal: removing %s\n", topic_f); test_kafka_topics("--delete --topic %s", topic_f); await_revoke("Topic removal: rebalance after topic removal", rk, queue); await_assignment("Topic removal: one topic exists", rk, queue, 1, topic_g, parts_g); TEST_SAY("Topic removal: removing %s\n", topic_g); test_kafka_topics("--delete --topic %s", topic_g); await_revoke("Topic removal: rebalance after 2nd topic removal", rk, queue); /* Should not see another rebalance since all topics now removed */ await_no_rebalance("Topic removal: empty", rk, queue, 10000); test_consumer_close(rk); rd_kafka_queue_destroy(queue); rd_kafka_destroy(rk); rd_free(topic_f); rd_free(topic_g); }
int main_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_0089_max_poll_interval (int argc, char **argv) { const char *topic = test_mk_topic_name("0089_max_poll_interval", 1); uint64_t testid; const int msgcnt = 10; rd_kafka_t *c[2]; rd_kafka_conf_t *conf; int64_t ts_next[2] = { 0, 0 }; int64_t ts_exp_msg[2] = { 0, 0 }; int cmsgcnt = 0; int i; int bad = -1; testid = test_id_generate(); test_create_topic(topic, 1, 1); test_produce_msgs_easy(topic, testid, -1, msgcnt); test_conf_init(&conf, NULL, 60); test_conf_set(conf, "session.timeout.ms", "6000"); test_conf_set(conf, "max.poll.interval.ms", "10000" /*10s*/); test_conf_set(conf, "auto.offset.reset", "earliest"); c[0] = test_create_consumer(topic, NULL, rd_kafka_conf_dup(conf), NULL); c[1] = test_create_consumer(topic, NULL, conf, NULL); test_consumer_subscribe(c[0], topic); test_consumer_subscribe(c[1], topic); while (1) { for (i = 0 ; i < 2 ; i++) { int64_t now; rd_kafka_message_t *rkm; /* Consumer is "processing" */ if (ts_next[i] > test_clock()) continue; rkm = rd_kafka_consumer_poll(c[i], 100); if (!rkm) continue; if (rkm->err) { TEST_WARN("Consumer %d error: %s: " "ignoring\n", i, rd_kafka_message_errstr(rkm)); continue; } now = test_clock(); cmsgcnt++; TEST_SAY("Consumer %d received message (#%d) " "at offset %"PRId64"\n", i, cmsgcnt, rkm->offset); if (ts_exp_msg[i]) { /* This consumer is expecting a message * after a certain time, namely after the * rebalance following max.poll.. being * exceeded in the other consumer */ TEST_ASSERT(now > ts_exp_msg[i], "Consumer %d: did not expect " "message for at least %dms", i, (int)((ts_exp_msg[i] - now)/1000)); TEST_ASSERT(now < ts_exp_msg[i] + 10000*1000, "Consumer %d: expected message " "within 10s, not after %dms", i, (int)((now - ts_exp_msg[i])/1000)); TEST_SAY("Consumer %d: received message " "at offset %"PRId64 " after rebalance\n", i, rkm->offset); rd_kafka_message_destroy(rkm); goto done; } else if (cmsgcnt == 1) { /* Process this message for 20s */ ts_next[i] = now + (20000 * 1000); /* Exp message on other consumer after * max.poll.interval.ms */ ts_exp_msg[i^1] = now + (10000 * 1000); /* This is the bad consumer */ bad = i; TEST_SAY("Consumer %d processing message at " "offset %"PRId64"\n", i, rkm->offset); rd_kafka_message_destroy(rkm); } else { rd_kafka_message_destroy(rkm); TEST_FAIL("Consumer %d did not expect " "a message", i); } } } done: TEST_ASSERT(bad != -1, "Bad consumer not set"); /* Wait for error ERR__MAX_POLL_EXCEEDED on the bad consumer. */ while (1) { rd_kafka_message_t *rkm; rkm = rd_kafka_consumer_poll(c[bad], 1000); TEST_ASSERT(rkm, "Expected consumer result within 1s"); TEST_ASSERT(rkm->err, "Did not expect message on bad consumer"); TEST_SAY("Consumer error: %s: %s\n", rd_kafka_err2name(rkm->err), rd_kafka_message_errstr(rkm)); if (rkm->err == RD_KAFKA_RESP_ERR__MAX_POLL_EXCEEDED) { rd_kafka_message_destroy(rkm); break; } rd_kafka_message_destroy(rkm); } for (i = 0 ; i < 2 ; i++) rd_kafka_destroy_flags(c[i], RD_KAFKA_DESTROY_F_NO_CONSUMER_CLOSE); return 0; }