/** * List groups by: * - List all groups, check that the groups in 'groups' are seen. * - List each group in 'groups', one by one. * * Returns 'group_cnt' if all groups in 'groups' were seen by both * methods, else 0, or -1 on error. */ static int list_groups (rd_kafka_t *rk, char **groups, int group_cnt, const char *desc) { rd_kafka_resp_err_t err = 0; const struct rd_kafka_group_list *grplist; int i, r; int fails = 0; int seen = 0; int seen_all = 0; int retries = 5; TEST_SAY("List groups (expect %d): %s\n", group_cnt, desc); /* FIXME: Wait for broker to come up. This should really be abstracted * by librdkafka. */ do { if (err) { TEST_SAY("Retrying group list in 1s because of: %s\n", rd_kafka_err2str(err)); rd_sleep(1); } err = rd_kafka_list_groups(rk, NULL, &grplist, 5000); } while ((err == RD_KAFKA_RESP_ERR__TRANSPORT || err == RD_KAFKA_RESP_ERR_GROUP_LOAD_IN_PROGRESS) && retries-- > 0); if (err) { TEST_SAY("Failed to list all groups: %s\n", rd_kafka_err2str(err)); return -1; } seen_all = verify_groups(grplist, groups, group_cnt); rd_kafka_group_list_destroy(grplist); for (i = 0 ; i < group_cnt ; i++) { err = rd_kafka_list_groups(rk, groups[i], &grplist, 5000); if (err) { TEST_SAY("Failed to list group %s: %s\n", groups[i], rd_kafka_err2str(err)); fails++; continue; } r = verify_groups(grplist, &groups[i], 1); if (r == 1) seen++; rd_kafka_group_list_destroy(grplist); } if (seen_all != seen) return 0; return seen; }
static int describe_groups (rd_kafka_t *rk, const char *group) { rd_kafka_resp_err_t err; const struct rd_kafka_group_list *grplist; int i; err = rd_kafka_list_groups(rk, group, &grplist, 10000); if (err) { fprintf(stderr, "%% Failed to acquire group list: %s\n", rd_kafka_err2str(err)); return -1; } for (i = 0 ; i < grplist->group_cnt ; i++) { const struct rd_kafka_group_info *gi = &grplist->groups[i]; int j; printf("Group \"%s\" in state %s on broker %d (%s:%hu)\n", gi->group, gi->state, gi->broker.id, gi->broker.host, gi->broker.port); if (gi->err) printf(" Error: %s\n", rd_kafka_err2str(gi->err)); printf(" Protocol type \"%s\", protocol \"%s\", " "with %d member(s):\n", gi->protocol_type, gi->protocol, gi->member_cnt); for (j = 0 ; j < gi->member_cnt ; j++) { const struct rd_kafka_group_member_info *mi; mi = &gi->members[j]; printf(" \"%s\", client id \"%s\" on host %s\n", mi->member_id, mi->client_id, mi->client_host); printf(" metadata: %d bytes\n", mi->member_metadata_size); printf(" assignment: %d bytes\n", mi->member_assignment_size); } printf("\n"); } if (group && !grplist->group_cnt) fprintf(stderr, "%% No matching group (%s)\n", group); rd_kafka_group_list_destroy(grplist); return 0; }
int main_0006_symbols (int argc, char **argv) { if (argc < 0 /* always false */) { rd_kafka_version(); rd_kafka_version_str(); rd_kafka_get_debug_contexts(); rd_kafka_get_err_descs(NULL, NULL); rd_kafka_err2str(RD_KAFKA_RESP_ERR_NO_ERROR); rd_kafka_err2name(RD_KAFKA_RESP_ERR_NO_ERROR); rd_kafka_errno2err(EINVAL); rd_kafka_errno(); rd_kafka_last_error(); rd_kafka_conf_new(); rd_kafka_conf_destroy(NULL); rd_kafka_conf_dup(NULL); rd_kafka_conf_set(NULL, NULL, NULL, NULL, 0); rd_kafka_conf_set_dr_cb(NULL, NULL); rd_kafka_conf_set_dr_msg_cb(NULL, NULL); rd_kafka_conf_set_error_cb(NULL, NULL); rd_kafka_conf_set_stats_cb(NULL, NULL); rd_kafka_conf_set_log_cb(NULL, NULL); rd_kafka_conf_set_socket_cb(NULL, NULL); rd_kafka_conf_set_rebalance_cb(NULL, NULL); rd_kafka_conf_set_offset_commit_cb(NULL, NULL); rd_kafka_conf_set_throttle_cb(NULL, NULL); rd_kafka_conf_set_default_topic_conf(NULL, NULL); rd_kafka_conf_get(NULL, NULL, NULL, NULL); #ifndef _MSC_VER rd_kafka_conf_set_open_cb(NULL, NULL); #endif rd_kafka_conf_set_opaque(NULL, NULL); rd_kafka_opaque(NULL); rd_kafka_conf_dump(NULL, NULL); rd_kafka_topic_conf_dump(NULL, NULL); rd_kafka_conf_dump_free(NULL, 0); rd_kafka_conf_properties_show(NULL); rd_kafka_topic_conf_new(); rd_kafka_topic_conf_dup(NULL); rd_kafka_topic_conf_destroy(NULL); rd_kafka_topic_conf_set(NULL, NULL, NULL, NULL, 0); rd_kafka_topic_conf_set_opaque(NULL, NULL); rd_kafka_topic_conf_get(NULL, NULL, NULL, NULL); rd_kafka_topic_conf_set_partitioner_cb(NULL, NULL); rd_kafka_topic_partition_available(NULL, 0); rd_kafka_topic_opaque(NULL); rd_kafka_msg_partitioner_random(NULL, NULL, 0, 0, NULL, NULL); rd_kafka_msg_partitioner_consistent(NULL, NULL, 0, 0, NULL, NULL); rd_kafka_msg_partitioner_consistent_random(NULL, NULL, 0, 0, NULL, NULL); rd_kafka_new(0, NULL, NULL, 0); rd_kafka_destroy(NULL); rd_kafka_name(NULL); rd_kafka_memberid(NULL); rd_kafka_topic_new(NULL, NULL, NULL); rd_kafka_topic_destroy(NULL); rd_kafka_topic_name(NULL); rd_kafka_message_destroy(NULL); rd_kafka_message_errstr(NULL); rd_kafka_message_timestamp(NULL, NULL); rd_kafka_consume_start(NULL, 0, 0); rd_kafka_consume_stop(NULL, 0); rd_kafka_consume(NULL, 0, 0); rd_kafka_consume_batch(NULL, 0, 0, NULL, 0); rd_kafka_consume_callback(NULL, 0, 0, NULL, NULL); rd_kafka_offset_store(NULL, 0, 0); rd_kafka_produce(NULL, 0, 0, NULL, 0, NULL, 0, NULL); rd_kafka_produce_batch(NULL, 0, 0, NULL, 0); rd_kafka_poll(NULL, 0); rd_kafka_brokers_add(NULL, NULL); /* DEPRECATED: rd_kafka_set_logger(NULL, NULL); */ rd_kafka_set_log_level(NULL, 0); rd_kafka_log_print(NULL, 0, NULL, NULL); #ifndef _MSC_VER rd_kafka_log_syslog(NULL, 0, NULL, NULL); #endif rd_kafka_outq_len(NULL); rd_kafka_dump(NULL, NULL); rd_kafka_thread_cnt(); rd_kafka_wait_destroyed(0); rd_kafka_metadata(NULL, 0, NULL, NULL, 0); rd_kafka_metadata_destroy(NULL); rd_kafka_queue_destroy(NULL); rd_kafka_consume_start_queue(NULL, 0, 0, NULL); rd_kafka_consume_queue(NULL, 0); rd_kafka_consume_batch_queue(NULL, 0, NULL, 0); rd_kafka_consume_callback_queue(NULL, 0, NULL, NULL); rd_kafka_seek(NULL, 0, 0, 0); rd_kafka_yield(NULL); rd_kafka_mem_free(NULL, NULL); rd_kafka_list_groups(NULL, NULL, NULL, 0); rd_kafka_group_list_destroy(NULL); /* KafkaConsumer API */ rd_kafka_subscribe(NULL, NULL); rd_kafka_unsubscribe(NULL); rd_kafka_subscription(NULL, NULL); rd_kafka_consumer_poll(NULL, 0); rd_kafka_consumer_close(NULL); rd_kafka_assign(NULL, NULL); rd_kafka_assignment(NULL, NULL); rd_kafka_commit(NULL, NULL, 0); rd_kafka_commit_message(NULL, NULL, 0); rd_kafka_committed(NULL, NULL, 0); rd_kafka_position(NULL, NULL); /* TopicPartition */ rd_kafka_topic_partition_list_new(0); rd_kafka_topic_partition_list_destroy(NULL); rd_kafka_topic_partition_list_add(NULL, NULL, 0); rd_kafka_topic_partition_list_add_range(NULL, NULL, 0, 0); rd_kafka_topic_partition_list_del(NULL, NULL, 0); rd_kafka_topic_partition_list_del_by_idx(NULL, 0); rd_kafka_topic_partition_list_copy(NULL); rd_kafka_topic_partition_list_set_offset(NULL, NULL, 0, 0); rd_kafka_topic_partition_list_find(NULL, NULL, 0); rd_kafka_query_watermark_offsets(NULL, NULL, 0, NULL, NULL, 0); rd_kafka_get_watermark_offsets(NULL, NULL, 0, NULL, NULL); } return 0; }