diff --git a/examples/.gitignore b/examples/.gitignore index 4190608c42..a19747b5cd 100644 --- a/examples/.gitignore +++ b/examples/.gitignore @@ -15,5 +15,7 @@ rdkafka_performance transactions list_consumer_groups describe_consumer_groups +describe_topics +describe_cluster list_consumer_group_offsets alter_consumer_group_offsets diff --git a/examples/Makefile b/examples/Makefile index 15fba3c2af..b8d39643d2 100644 --- a/examples/Makefile +++ b/examples/Makefile @@ -6,6 +6,8 @@ EXAMPLES ?= rdkafka_example rdkafka_performance rdkafka_example_cpp \ openssl_engine_example_cpp \ list_consumer_groups \ describe_consumer_groups \ + describe_topics \ + describe_cluster \ list_consumer_group_offsets \ alter_consumer_group_offsets \ misc @@ -72,6 +74,14 @@ describe_consumer_groups: ../src/librdkafka.a describe_consumer_groups.c $(CC) $(CPPFLAGS) $(CFLAGS) $@.c -o $@ $(LDFLAGS) \ ../src/librdkafka.a $(LIBS) +describe_topics: ../src/librdkafka.a describe_topics.c + $(CC) $(CPPFLAGS) $(CFLAGS) $@.c -o $@ $(LDFLAGS) \ + ../src/librdkafka.a $(LIBS) + +describe_cluster: ../src/librdkafka.a describe_cluster.c + $(CC) $(CPPFLAGS) $(CFLAGS) $@.c -o $@ $(LDFLAGS) \ + ../src/librdkafka.a $(LIBS) + list_consumer_group_offsets: ../src/librdkafka.a list_consumer_group_offsets.c $(CC) $(CPPFLAGS) $(CFLAGS) $@.c -o $@ $(LDFLAGS) \ ../src/librdkafka.a $(LIBS) diff --git a/examples/README.md b/examples/README.md index 3caee3b861..ce05ab5be7 100644 --- a/examples/README.md +++ b/examples/README.md @@ -34,5 +34,7 @@ For more complex uses, see: * [delete_records.c](delete_records.c) - Delete records. * [list_consumer_groups.c](list_consumer_groups.c) - List consumer groups. * [describe_consumer_groups.c](describe_consumer_groups.c) - Describe consumer groups. + * [describe_topics.c](describe_topics.c) - Describe topics. + * [describe_cluster.c](describe_cluster.c) - Describe cluster. * [list_consumer_group_offsets.c](list_consumer_group_offsets.c) - List offsets of a consumer group. * [alter_consumer_group_offsets.c](alter_consumer_group_offsets.c) - Alter offsets of a consumer group. diff --git a/examples/describe_cluster.c b/examples/describe_cluster.c new file mode 100644 index 0000000000..5e66230a86 --- /dev/null +++ b/examples/describe_cluster.c @@ -0,0 +1,317 @@ +/* + * librdkafka - Apache Kafka C library + * + * Copyright (c) 2022, Magnus Edenhill + * All rights reserved. + * + * Redistribution and use in source and binary forms, with or without + * modification, are permitted provided that the following conditions are met: + * + * 1. Redistributions of source code must retain the above copyright notice, + * this list of conditions and the following disclaimer. + * 2. Redistributions in binary form must reproduce the above copyright notice, + * this list of conditions and the following disclaimer in the documentation + * and/or other materials provided with the distribution. + * + * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" + * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE + * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE + * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE + * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR + * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF + * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS + * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN + * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) + * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE + * POSSIBILITY OF SUCH DAMAGE. + */ + +/** + * DescribeCluster usage example. + */ + +#include +#include +#include +#include +#include + +#ifdef _WIN32 +#include "../win32/wingetopt.h" +#else +#include +#endif + + +/* Typical include path would be , but this program + * is builtin from within the librdkafka source tree and thus differs. */ +#include "rdkafka.h" + + +const char *argv0; +static rd_kafka_queue_t *queue; /** Admin result queue. + * This is a global so we can + * yield in stop() */ +static volatile sig_atomic_t run = 1; + +/** + * @brief Signal termination of program + */ +static void stop(int sig) { + if (!run) { + fprintf(stderr, "%% Forced termination\n"); + exit(2); + } + run = 0; + rd_kafka_queue_yield(queue); +} + + +static void usage(const char *reason, ...) { + + fprintf( + stderr, + "Describe cluster usage examples\n" + "\n" + "Usage: %s ...\n" + "\n" + "Options:\n" + " -b Bootstrap server list to connect to.\n" + " -X Set librdkafka configuration property.\n" + " See CONFIGURATION.md for full list.\n" + " -d Enable librdkafka debugging (%s).\n" + "\n", + argv0, rd_kafka_get_debug_contexts()); + + if (reason) { + va_list ap; + char reasonbuf[512]; + + va_start(ap, reason); + vsnprintf(reasonbuf, sizeof(reasonbuf), reason, ap); + va_end(ap); + + fprintf(stderr, "ERROR: %s\n", reasonbuf); + } + + exit(reason ? 1 : 0); +} + + +#define fatal(...) \ + do { \ + fprintf(stderr, "ERROR: "); \ + fprintf(stderr, __VA_ARGS__); \ + fprintf(stderr, "\n"); \ + exit(2); \ + } while (0) + + +/** + * @brief Set config property. Exit on failure. + */ +static void conf_set(rd_kafka_conf_t *conf, const char *name, const char *val) { + char errstr[512]; + + if (rd_kafka_conf_set(conf, name, val, errstr, sizeof(errstr)) != + RD_KAFKA_CONF_OK) + fatal("Failed to set %s=%s: %s", name, val, errstr); +} + +/** + * @brief Parse an integer or fail. + */ +int64_t parse_int(const char *what, const char *str) { + char *end; + unsigned long n = strtoull(str, &end, 0); + + if (end != str + strlen(str)) { + fprintf(stderr, "%% Invalid input for %s: %s: not an integer\n", + what, str); + exit(1); + } + + return (int64_t)n; +} + +/** + * @brief Print cluster information. + */ +static int +print_cluster_info(const rd_kafka_DescribeCluster_result_t *clusterdesc) { + size_t i; + int j, acl_operation; + const rd_kafka_ClusterDescription_t *desc; + int controller_id, node_cnt, cluster_authorized_operations_cnt; + const char *cluster_id; + + desc = rd_kafka_DescribeCluster_result_description(clusterdesc); + + controller_id = rd_kafka_ClusterDescription_controller_id(desc); + node_cnt = rd_kafka_ClusterDescription_node_cnt(desc); + cluster_authorized_operations_cnt = + rd_kafka_ClusterDescription_cluster_acl_operations_cnt(desc); + cluster_id = rd_kafka_ClusterDescription_cluster_id(desc); + + printf( + "Cluster id: %s\t Controller id: %d\t ACL operations count " + "allowed: %d\n", + cluster_id, controller_id, cluster_authorized_operations_cnt); + for (j = 0; j < cluster_authorized_operations_cnt; j++) { + acl_operation = + rd_kafka_ClusterDescription_authorized_operation_idx(desc, + j); + printf("\t%s operation is allowed\n", + rd_kafka_AclOperation_name(acl_operation)); + } + + for (j = 0; j < node_cnt; j++) { + const rd_kafka_Node_t *node = NULL; + node = rd_kafka_ClusterDescription_node_idx(desc, j); + printf("Node [id: %" PRId32 + ", host: %s" + ", port: %" PRIu16 "]\n", + rd_kafka_Node_id(node), rd_kafka_Node_host(node), + rd_kafka_Node_port(node)); + } + return 0; +} + +/** + * @brief Call rd_kafka_DescribeCluster() + */ +static void cmd_describe_cluster(rd_kafka_conf_t *conf, int argc, char **argv) { + rd_kafka_t *rk; + char errstr[512]; + rd_kafka_AdminOptions_t *options; + rd_kafka_event_t *event = NULL; + rd_kafka_error_t *error; + int retval = 0; + + int include_cluster_authorized_operations = + parse_int("include_cluster_authorized_operations", argv[0]); + if (include_cluster_authorized_operations < 0 || + include_cluster_authorized_operations > 1) + usage("include_cluster_authorized_operations not a 0-1 int"); + + /* + * Create consumer instance + * NOTE: rd_kafka_new() takes ownership of the conf object + * and the application must not reference it again after + * this call. + */ + rk = rd_kafka_new(RD_KAFKA_CONSUMER, conf, errstr, sizeof(errstr)); + if (!rk) + fatal("Failed to create new consumer: %s", errstr); + + /* + * Describe cluster + */ + queue = rd_kafka_queue_new(rk); + + /* Signal handler for clean shutdown */ + signal(SIGINT, stop); + + options = + rd_kafka_AdminOptions_new(rk, RD_KAFKA_ADMIN_OP_DESCRIBECLUSTER); + + if (rd_kafka_AdminOptions_set_request_timeout( + options, 10 * 1000 /* 10s */, errstr, sizeof(errstr))) { + fprintf(stderr, "%% Failed to set timeout: %s\n", errstr); + goto exit; + } + if ((error = + rd_kafka_AdminOptions_set_include_cluster_authorized_operations( + options, include_cluster_authorized_operations))) { + fprintf(stderr, + "%% Failed to set require cluster authorized " + "operations: %s\n", + rd_kafka_error_string(error)); + rd_kafka_error_destroy(error); + exit(1); + } + + rd_kafka_DescribeCluster(rk, options, queue); + + /* Wait for results */ + event = rd_kafka_queue_poll(queue, -1 /* indefinitely but limited by + * the request timeout set + * above (10s) */); + + if (!event) { + /* User hit Ctrl-C, + * see yield call in stop() signal handler */ + fprintf(stderr, "%% Cancelled by user\n"); + + } else if (rd_kafka_event_error(event)) { + rd_kafka_resp_err_t err = rd_kafka_event_error(event); + /* DescribeCluster request failed */ + fprintf(stderr, "%% DescribeCluster failed[%" PRId32 "]: %s\n", + err, rd_kafka_event_error_string(event)); + goto exit; + + } else { + /* DescribeCluster request succeeded */ + const rd_kafka_DescribeCluster_result_t *result; + + result = rd_kafka_event_DescribeCluster_result(event); + printf("DescribeCluster results:\n"); + retval = print_cluster_info(result); + } + + +exit: + if (event) + rd_kafka_event_destroy(event); + rd_kafka_AdminOptions_destroy(options); + rd_kafka_queue_destroy(queue); + /* Destroy the client instance */ + rd_kafka_destroy(rk); + + exit(retval); +} + +int main(int argc, char **argv) { + rd_kafka_conf_t *conf; /**< Client configuration object */ + int opt; + argv0 = argv[0]; + + /* + * Create Kafka client configuration place-holder + */ + conf = rd_kafka_conf_new(); + + /* + * Parse common options + */ + while ((opt = getopt(argc, argv, "b:X:d:")) != -1) { + switch (opt) { + case 'b': + conf_set(conf, "bootstrap.servers", optarg); + break; + + case 'X': { + char *name = optarg, *val; + + if (!(val = strchr(name, '='))) + fatal("-X expects a name=value argument"); + + *val = '\0'; + val++; + + conf_set(conf, name, val); + break; + } + + case 'd': + conf_set(conf, "debug", optarg); + break; + + default: + usage("Unknown option %c", (char)opt); + } + } + + cmd_describe_cluster(conf, argc - optind, &argv[optind]); + return 0; +} \ No newline at end of file diff --git a/examples/describe_consumer_groups.c b/examples/describe_consumer_groups.c index 45b6b8d0b0..b392f3fa42 100644 --- a/examples/describe_consumer_groups.c +++ b/examples/describe_consumer_groups.c @@ -73,7 +73,8 @@ static void usage(const char *reason, ...) { fprintf(stderr, "Describe groups usage examples\n" "\n" - "Usage: %s ...\n" + "Usage: %s " + " ...\n" "\n" "Options:\n" " -b Bootstrap server list to connect to.\n" @@ -167,7 +168,7 @@ print_groups_info(const rd_kafka_DescribeConsumerGroups_result_t *grpdesc, } for (i = 0; i < result_groups_cnt; i++) { - int j, member_cnt; + int j, member_cnt, authorized_operation_count, acl_operation; const rd_kafka_error_t *error; const rd_kafka_ConsumerGroupDescription_t *group = result_groups[i]; @@ -179,6 +180,9 @@ print_groups_info(const rd_kafka_DescribeConsumerGroups_result_t *grpdesc, rd_kafka_ConsumerGroupDescription_partition_assignor(group); rd_kafka_consumer_group_state_t state = rd_kafka_ConsumerGroupDescription_state(group); + authorized_operation_count = + rd_kafka_ConsumerGroupDescription_authorized_operations_count( + group); member_cnt = rd_kafka_ConsumerGroupDescription_member_count(group); error = rd_kafka_ConsumerGroupDescription_error(group); @@ -197,10 +201,17 @@ print_groups_info(const rd_kafka_DescribeConsumerGroups_result_t *grpdesc, } printf( "Group \"%s\", partition assignor \"%s\", " - "state %s%s, with %" PRId32 " member(s)", + " state %s%s, with %" PRId32 " member(s)\n", group_id, partition_assignor, rd_kafka_consumer_group_state_name(state), coordinator_desc, member_cnt); + for (j = 0; j < authorized_operation_count; j++) { + acl_operation = + rd_kafka_ConsumerGroupDescription_authorized_operation( + group, j); + printf("%s operation is allowed\n", + rd_kafka_AclOperation_name(acl_operation)); + } if (error) printf(" error[%" PRId32 "]: %s", rd_kafka_error_code(error), @@ -233,10 +244,25 @@ print_groups_info(const rd_kafka_DescribeConsumerGroups_result_t *grpdesc, 0, " "); } } + printf("\n"); } return 0; } +/** + * @brief Parse an integer or fail. + */ +int64_t parse_int(const char *what, const char *str) { + char *end; + unsigned long n = strtoull(str, &end, 0); + + if (end != str + strlen(str)) { + fprintf(stderr, "%% Invalid input for %s: %s: not an integer\n", + what, str); + exit(1); + } + return (int64_t)n; +} /** * @brief Call rd_kafka_DescribeConsumerGroups() with a list of * groups. @@ -248,12 +274,19 @@ cmd_describe_consumer_groups(rd_kafka_conf_t *conf, int argc, char **argv) { char errstr[512]; rd_kafka_AdminOptions_t *options; rd_kafka_event_t *event = NULL; - int retval = 0; - int groups_cnt = 0; + rd_kafka_error_t *error; + int retval = 0; + int groups_cnt = 0; + + int include_authorized_operations = + parse_int("include_authorized_operations", argv[0]); + if (include_authorized_operations < 0 || + include_authorized_operations > 1) + usage("Require stable not a 0-1 int"); if (argc >= 1) { - groups = (const char **)&argv[0]; - groups_cnt = argc; + groups = (const char **)&argv[1]; + groups_cnt = argc - 1; } /* @@ -282,6 +315,14 @@ cmd_describe_consumer_groups(rd_kafka_conf_t *conf, int argc, char **argv) { fprintf(stderr, "%% Failed to set timeout: %s\n", errstr); goto exit; } + if ((error = rd_kafka_AdminOptions_set_include_authorized_operations( + options, include_authorized_operations))) { + fprintf(stderr, + "%% Failed to set require authorized operations: %s\n", + rd_kafka_error_string(error)); + rd_kafka_error_destroy(error); + exit(1); + } rd_kafka_DescribeConsumerGroups(rk, groups, groups_cnt, options, queue); @@ -335,7 +376,6 @@ int main(int argc, char **argv) { */ conf = rd_kafka_conf_new(); - /* * Parse common options */ @@ -368,6 +408,5 @@ int main(int argc, char **argv) { } cmd_describe_consumer_groups(conf, argc - optind, &argv[optind]); - return 0; } diff --git a/examples/describe_topics.c b/examples/describe_topics.c new file mode 100644 index 0000000000..0287dd31bd --- /dev/null +++ b/examples/describe_topics.c @@ -0,0 +1,391 @@ +/* + * librdkafka - Apache Kafka C library + * + * Copyright (c) 2022, Magnus Edenhill + * All rights reserved. + * + * Redistribution and use in source and binary forms, with or without + * modification, are permitted provided that the following conditions are met: + * + * 1. Redistributions of source code must retain the above copyright notice, + * this list of conditions and the following disclaimer. + * 2. Redistributions in binary form must reproduce the above copyright notice, + * this list of conditions and the following disclaimer in the documentation + * and/or other materials provided with the distribution. + * + * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" + * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE + * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE + * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE + * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR + * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF + * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS + * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN + * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) + * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE + * POSSIBILITY OF SUCH DAMAGE. + */ + +/** + * DescribeTopics usage example. + */ + +#include +#include +#include +#include +#include + +#ifdef _WIN32 +#include "../win32/wingetopt.h" +#else +#include +#endif + + +/* Typical include path would be , but this program + * is builtin from within the librdkafka source tree and thus differs. */ +#include "rdkafka.h" + + +const char *argv0; +static rd_kafka_queue_t *queue; /** Admin result queue. + * This is a global so we can + * yield in stop() */ +static volatile sig_atomic_t run = 1; + +/** + * @brief Signal termination of program + */ +static void stop(int sig) { + if (!run) { + fprintf(stderr, "%% Forced termination\n"); + exit(2); + } + run = 0; + rd_kafka_queue_yield(queue); +} + + +static void usage(const char *reason, ...) { + + fprintf(stderr, + "Describe topics usage examples\n" + "\n" + "Usage: %s \n" + " ...\n" + "\n" + "Options:\n" + " -b Bootstrap server list to connect to.\n" + " -X Set librdkafka configuration property.\n" + " See CONFIGURATION.md for full list.\n" + " -d Enable librdkafka debugging (%s).\n" + "\n", + argv0, rd_kafka_get_debug_contexts()); + + if (reason) { + va_list ap; + char reasonbuf[512]; + + va_start(ap, reason); + vsnprintf(reasonbuf, sizeof(reasonbuf), reason, ap); + va_end(ap); + + fprintf(stderr, "ERROR: %s\n", reasonbuf); + } + + exit(reason ? 1 : 0); +} + + +#define fatal(...) \ + do { \ + fprintf(stderr, "ERROR: "); \ + fprintf(stderr, __VA_ARGS__); \ + fprintf(stderr, "\n"); \ + exit(2); \ + } while (0) + + +/** + * @brief Set config property. Exit on failure. + */ +static void conf_set(rd_kafka_conf_t *conf, const char *name, const char *val) { + char errstr[512]; + + if (rd_kafka_conf_set(conf, name, val, errstr, sizeof(errstr)) != + RD_KAFKA_CONF_OK) + fatal("Failed to set %s=%s: %s", name, val, errstr); +} + +/** + * @brief Parse an integer or fail. + */ +int64_t parse_int(const char *what, const char *str) { + char *end; + unsigned long n = strtoull(str, &end, 0); + + if (end != str + strlen(str)) { + fprintf(stderr, "%% Invalid input for %s: %s: not an integer\n", + what, str); + exit(1); + } + + return (int64_t)n; +} +/** + * @brief Print topics information. + */ +static int print_topics_info(const rd_kafka_DescribeTopics_result_t *topicdesc, + int topic_cnt) { + size_t i, j; + const rd_kafka_TopicDescription_t **result_topics; + size_t result_topics_cnt; + result_topics = rd_kafka_DescribeTopics_result_topics( + topicdesc, &result_topics_cnt); + + if (result_topics_cnt == 0) { + if (topic_cnt > 0) { + fprintf(stderr, "No matching topics found\n"); + return 1; + } else { + fprintf(stderr, "No topics in cluster\n"); + } + } + + for (i = 0; i < result_topics_cnt; i++) { + int j, acl_operation; + const rd_kafka_error_t *error; + const rd_kafka_TopicDescription_t *topic = result_topics[i]; + const char *topic_name = + rd_kafka_TopicDescription_topic_name(topic); + int topic_authorized_operations_cnt = + rd_kafka_TopicDescription_topic_authorized_operations_cnt( + topic); + int partition_cnt = + rd_kafka_TopicDescription_topic_partition_cnt(topic); + error = rd_kafka_TopicDescription_error(topic); + + if (rd_kafka_error_code(error)) { + printf("Topic: %s has error[%" PRId32 "]: %s\n", + topic_name, rd_kafka_error_code(error), + rd_kafka_error_string(error)); + continue; + } + + printf( + "Topic: %s succeeded, has %d topic authorized operations " + "allowed, they are:\n", + topic_name, topic_authorized_operations_cnt); + for (j = 0; j < topic_authorized_operations_cnt; j++) { + acl_operation = + rd_kafka_TopicDescription_authorized_operation_idx( + topic, j); + printf("\t%s operation is allowed\n", + rd_kafka_AclOperation_name(acl_operation)); + } + + printf("partiton count is: %d\n", partition_cnt); + for (j = 0; j < partition_cnt; j++) { + const rd_kafka_error_t *partition_error; + int leader, id, isr_cnt, replica_cnt, k; + partition_error = + rd_kafka_TopicDescription_partition_error(topic, j); + + if (rd_kafka_error_code(partition_error)) { + printf( + "\tPartition at index %d has error[%" PRId32 + "]: %s\n", + j, rd_kafka_error_code(partition_error), + rd_kafka_error_string(partition_error)); + continue; + } + printf("\tPartition at index %d succeeded\n", j); + id = rd_kafka_TopicDescription_partiton_id(topic, j); + leader = + rd_kafka_TopicDescription_partiton_leader(topic, j); + isr_cnt = rd_kafka_TopicDescription_partiton_isr_cnt( + topic, j); + replica_cnt = + rd_kafka_TopicDescription_partiton_replica_cnt( + topic, j); + printf("\tPartition has id: %d with leader: %d\n", id, + leader); + if (isr_cnt) { + printf( + "\tThe in-sync replica count is: %d, they " + "are: ", + isr_cnt); + for (k = 0; k < isr_cnt; k++) + printf( + "%d ", + rd_kafka_TopicDescription_partiton_isrs_idx( + topic, j, k)); + printf("\n"); + } else + printf("\tThe in-sync replica count is 0\n"); + + if (replica_cnt) { + printf("\tThe replica count is: %d, they are: ", + replica_cnt); + for (k = 0; k < replica_cnt; k++) + printf( + "%d ", + rd_kafka_TopicDescription_partiton_replica_idx( + topic, j, k)); + printf("\n"); + } else + printf("\tThe replica count is 0\n"); + printf("\n"); + } + printf("\n"); + } + return 0; +} +/** + * @brief Call rd_kafka_DescribeTopics() with a list of + * topics. + */ +static void cmd_describe_topics(rd_kafka_conf_t *conf, int argc, char **argv) { + rd_kafka_t *rk; + int i; + const char **topics = NULL; + char errstr[512]; + rd_kafka_AdminOptions_t *options; + rd_kafka_event_t *event = NULL; + rd_kafka_error_t *error; + int retval = 0; + int topics_cnt = 0; + + int include_topic_authorized_operations = + parse_int("include_topic_authorized_operations", argv[0]); + if (include_topic_authorized_operations < 0 || + include_topic_authorized_operations > 1) + usage("include_topic_authorized_operations not a 0-1 int"); + + if (argc >= 1) { + topics = (const char **)&argv[1]; + topics_cnt = argc - 1; + } + /* + * Create consumer instance + * NOTE: rd_kafka_new() takes ownership of the conf object + * and the application must not reference it again after + * this call. + */ + rk = rd_kafka_new(RD_KAFKA_CONSUMER, conf, errstr, sizeof(errstr)); + if (!rk) + fatal("Failed to create new consumer: %s", errstr); + + /* + * Describe topics + */ + queue = rd_kafka_queue_new(rk); + + /* Signal handler for clean shutdown */ + signal(SIGINT, stop); + + options = + rd_kafka_AdminOptions_new(rk, RD_KAFKA_ADMIN_OP_DESCRIBETOPICS); + + if (rd_kafka_AdminOptions_set_request_timeout( + options, 10 * 1000 /* 10s */, errstr, sizeof(errstr))) { + fprintf(stderr, "%% Failed to set timeout: %s\n", errstr); + goto exit; + } + if ((error = + rd_kafka_AdminOptions_set_include_topic_authorized_operations( + options, include_topic_authorized_operations))) { + fprintf(stderr, + "%% Failed to set require topic authorized operations: " + "%s\n", + rd_kafka_error_string(error)); + rd_kafka_error_destroy(error); + exit(1); + } + + rd_kafka_DescribeTopics(rk, topics, topics_cnt, options, queue); + + /* Wait for results */ + event = rd_kafka_queue_poll(queue, -1 /* indefinitely but limited by + * the request timeout set + * above (10s) */); + + if (!event) { + /* User hit Ctrl-C, + * see yield call in stop() signal handler */ + fprintf(stderr, "%% Cancelled by user\n"); + + } else if (rd_kafka_event_error(event)) { + rd_kafka_resp_err_t err = rd_kafka_event_error(event); + /* DescribeTopics request failed */ + fprintf(stderr, "%% DescribeTopics failed[%" PRId32 "]: %s\n", + err, rd_kafka_event_error_string(event)); + goto exit; + + } else { + /* DescribeTopics request succeeded, but individual + * groups may have errors. */ + const rd_kafka_DescribeTopics_result_t *result; + + result = rd_kafka_event_DescribeTopics_result(event); + printf("DescribeTopics results:\n"); + retval = print_topics_info(result, topics_cnt); + } + + +exit: + if (event) + rd_kafka_event_destroy(event); + rd_kafka_AdminOptions_destroy(options); + rd_kafka_queue_destroy(queue); + /* Destroy the client instance */ + rd_kafka_destroy(rk); + + exit(retval); +} + +int main(int argc, char **argv) { + rd_kafka_conf_t *conf; /**< Client configuration object */ + int opt; + argv0 = argv[0]; + + /* + * Create Kafka client configuration place-holder + */ + conf = rd_kafka_conf_new(); + + /* + * Parse common options + */ + while ((opt = getopt(argc, argv, "b:X:d:")) != -1) { + switch (opt) { + case 'b': + conf_set(conf, "bootstrap.servers", optarg); + break; + + case 'X': { + char *name = optarg, *val; + + if (!(val = strchr(name, '='))) + fatal("-X expects a name=value argument"); + + *val = '\0'; + val++; + + conf_set(conf, name, val); + break; + } + + case 'd': + conf_set(conf, "debug", optarg); + break; + + default: + usage("Unknown option %c", (char)opt); + } + } + + cmd_describe_topics(conf, argc - optind, &argv[optind]); + return 0; +} diff --git a/src/rdkafka.c b/src/rdkafka.c index 2a5e040b68..ebccb5afb5 100644 --- a/src/rdkafka.c +++ b/src/rdkafka.c @@ -4794,7 +4794,7 @@ static void rd_kafka_ListGroups_resp_cb(rd_kafka_t *rk, state->wait_cnt++; error = rd_kafka_DescribeGroupsRequest( - rkb, 0, grps, i, RD_KAFKA_REPLYQ(state->q, 0), + rkb, 0, grps, i, rd_false, RD_KAFKA_REPLYQ(state->q, 0), rd_kafka_DescribeGroups_resp_cb, state); if (error) { rd_kafka_DescribeGroups_resp_cb( diff --git a/src/rdkafka.h b/src/rdkafka.h index 103e6294cf..d26924c415 100644 --- a/src/rdkafka.h +++ b/src/rdkafka.h @@ -5366,6 +5366,10 @@ typedef int rd_kafka_event_type_t; #define RD_KAFKA_EVENT_LISTCONSUMERGROUPOFFSETS_RESULT 0x8000 /** AlterConsumerGroupOffsets_result_t */ #define RD_KAFKA_EVENT_ALTERCONSUMERGROUPOFFSETS_RESULT 0x10000 +/** DescribeTopics_result_t */ +#define RD_KAFKA_EVENT_DESCRIBETOPICS_RESULT 0x20000 +/** DescribeCluster_result_t */ +#define RD_KAFKA_EVENT_DESCRIBECLUSTER_RESULT 0x40000 /** @@ -5521,6 +5525,8 @@ int rd_kafka_event_error_is_fatal(rd_kafka_event_t *rkev); * - RD_KAFKA_EVENT_DESCRIBECONSUMERGROUPS_RESULT * - RD_KAFKA_EVENT_LISTCONSUMERGROUPOFFSETS_RESULT * - RD_KAFKA_EVENT_ALTERCONSUMERGROUPOFFSETS_RESULT + * - RD_KAFKA_EVENT_DESCRIBETOPICS_RESULT + * - RD_KAFKA_EVENT_DESCRIBECLUSTER_RESULT */ RD_EXPORT void *rd_kafka_event_opaque(rd_kafka_event_t *rkev); @@ -5632,6 +5638,10 @@ typedef rd_kafka_event_t rd_kafka_DeleteConsumerGroupOffsets_result_t; typedef rd_kafka_event_t rd_kafka_AlterConsumerGroupOffsets_result_t; /*! ListConsumerGroupOffsets result type */ typedef rd_kafka_event_t rd_kafka_ListConsumerGroupOffsets_result_t; +/*! DescribeTopics result type */ +typedef rd_kafka_event_t rd_kafka_DescribeTopics_result_t; +/*! DescribeCluster result type */ +typedef rd_kafka_event_t rd_kafka_DescribeCluster_result_t; /** * @brief Get CreateTopics result. @@ -5733,6 +5743,35 @@ rd_kafka_event_ListConsumerGroups_result(rd_kafka_event_t *rkev); RD_EXPORT const rd_kafka_DescribeConsumerGroups_result_t * rd_kafka_event_DescribeConsumerGroups_result(rd_kafka_event_t *rkev); +/** + * @brief Get DescribeTopics result. + * + * @returns the result of a DescribeTopics request, or NULL if event is + * of different type. + * + * @remark The lifetime of the returned memory is the same + * as the lifetime of the \p rkev object. + * + * Event types: + * RD_KAFKA_EVENT_DESCRIBETOPICS_RESULT + */ +RD_EXPORT const rd_kafka_DescribeTopics_result_t * +rd_kafka_event_DescribeTopics_result(rd_kafka_event_t *rkev); + +/** + * @brief Get DescribeCluster result. + * + * @returns the result of a DescribeCluster request, or NULL if event is + * of different type. + * + * @remark The lifetime of the returned memory is the same + * as the lifetime of the \p rkev object. + * + * Event types: + * RD_KAFKA_EVENT_DESCRIBECLUSTER_RESULT + */ +RD_EXPORT const rd_kafka_DescribeCluster_result_t * +rd_kafka_event_DescribeCluster_result(rd_kafka_event_t *rkev); /** * @brief Get DeleteGroups result. * @@ -6720,6 +6759,10 @@ typedef enum rd_kafka_admin_op_t { RD_KAFKA_ADMIN_OP_LISTCONSUMERGROUPOFFSETS, /** AlterConsumerGroupOffsets */ RD_KAFKA_ADMIN_OP_ALTERCONSUMERGROUPOFFSETS, + /**< DescribeTopics */ + RD_KAFKA_ADMIN_OP_DESCRIBETOPICS, + /**< DescribeCluster */ + RD_KAFKA_ADMIN_OP_DESCRIBECLUSTER, RD_KAFKA_ADMIN_OP__CNT /**< Number of ops defined */ } rd_kafka_admin_op_t; @@ -6897,6 +6940,23 @@ rd_kafka_error_t *rd_kafka_AdminOptions_set_require_stable_offsets( rd_kafka_AdminOptions_t *options, int true_or_false); +/** + * @brief Whether broker should return authorized operations + * (DescribeConsumerGroups). + * + * @param options Admin options. + * @param true_or_false Defaults to false. + * + * @return NULL on success, a new error instance that must be + * released with rd_kafka_error_destroy() in case of error. + * + * @remark This option is valid for DescribeConsumerGroups. + */ +RD_EXPORT +rd_kafka_error_t *rd_kafka_AdminOptions_set_include_authorized_operations( + rd_kafka_AdminOptions_t *options, + int true_or_false); + /** * @brief Set consumer groups states to query for. * @@ -7730,6 +7790,418 @@ rd_kafka_DeleteRecords_result_offsets( /**@}*/ +/** + * @name Admin API - DescribeTopic + * @{ + */ +/** + * @brief Describe a topic and the authorized acl operations + * + * @param rk Client instance. + * @param topics Topic names + * @param topics_cnt Count of topics sent in topic names. + * @param options Optional admin options, or NULL for defaults. + * @param rkqu Queue to emit result on. + * + * @remark The result event type emitted on the supplied queue is of type + * \c RD_KAFKA_EVENT_DESCRIBETOPICS_RESULT + */ + +/** + * @brief DescribeTopics result type. + * + */ +typedef struct rd_kafka_TopicDescription_s rd_kafka_TopicDescription_t; + +/** + * @brief Gets the partiton id for partition at index position for the \p + * topicdesc topic. + * + * @param topicdesc The topic description. + * @param idx Index for the partitions. + * + * @return The partiton id. + * + * @remark The lifetime of the returned memory is the same + * as the lifetime of the \p topicdesc object. + */ +RD_EXPORT +const int rd_kafka_TopicDescription_partiton_id( + const rd_kafka_TopicDescription_t *topicdesc, + int idx); + +/** + * @brief Gets the partiton leader for partition at index position for the \p + * topicdesc topic. + * + * @param topicdesc The topic description. + * @param idx Index for the partitions. + * + * @return The partiton leader. + * + * @remark The lifetime of the returned memory is the same + * as the lifetime of the \p topicdesc object. + */ +RD_EXPORT +const int rd_kafka_TopicDescription_partiton_leader( + const rd_kafka_TopicDescription_t *topicdesc, + int idx); + +/** + * @brief Gets the partiton in-sync replica count for partition at index + * position for the \p topicdesc topic. + * + * @param topicdesc The topic description. + * @param idx Index for the partitions. + * + * @return The partiton replica count. + * + * @remark The lifetime of the returned memory is the same + * as the lifetime of the \p topicdesc object. + */ +RD_EXPORT +const int rd_kafka_TopicDescription_partiton_isr_cnt( + const rd_kafka_TopicDescription_t *topicdesc, + int idx); + +/** + * @brief Gets the partiton replica count for partition at index position for + * the \p topicdesc topic. + * + * @param topicdesc The topic description. + * @param idx Index for the partitions. + * + * @return The partiton replica count. + * + * @remark The lifetime of the returned memory is the same + * as the lifetime of the \p topicdesc object. + */ +RD_EXPORT +const int rd_kafka_TopicDescription_partiton_replica_cnt( + const rd_kafka_TopicDescription_t *topicdesc, + int idx); + +/** + * @brief Gets the partiton in-sync replica at isr index + * for partition at partition index position for the \p topicdesc topic. + * + * @param topicdesc The topic description. + * @param partition_idx Index for the partitions. + * @param isr_idx Index for the in-sync replica. + * + * @return The partiton in-sync replica. + * + * @remark The lifetime of the returned memory is the same + * as the lifetime of the \p topicdesc object. + */ +RD_EXPORT +const int rd_kafka_TopicDescription_partiton_isrs_idx( + const rd_kafka_TopicDescription_t *topicdesc, + int partition_idx, + int isr_idx); + +/** + * @brief Gets the partiton replica at replica index + * for partition at partition index position for the \p topicdesc topic. + * + * @param topicdesc The topic description. + * @param partition_idx Index for the partitions. + * @param replica_idx Index for the in-sync replica. + * + * @return The partiton replica. + * + * @remark The lifetime of the returned memory is the same + * as the lifetime of the \p topicdesc object. + */ +RD_EXPORT +const int rd_kafka_TopicDescription_partiton_replica_idx( + const rd_kafka_TopicDescription_t *topicdesc, + int partition_idx, + int replica_idx); + +/** + * @brief Gets the topic partiton count for the \p topicdesc topic. + * + * @param topicdesc The topic description. + * + * @return The topic partiton count. + * + * @remark The lifetime of the returned memory is the same + * as the lifetime of the \p topicdesc object. + */ +RD_EXPORT +const int rd_kafka_TopicDescription_topic_partition_cnt( + const rd_kafka_TopicDescription_t *topicdesc); + +/** + * @brief Gets the partiton error for partition at index position for the \p + * topicdesc topic. + * + * @param topicdesc The topic description. + * @param idx Index for the partitions. + * + * @return The partiton error. + * + * @remark The lifetime of the returned memory is the same + * as the lifetime of the \p topicdesc object. + */ +RD_EXPORT +const rd_kafka_error_t *rd_kafka_TopicDescription_partition_error( + const rd_kafka_TopicDescription_t *topicdesc, + int idx); + +/** + * @brief Gets operation at idx index of topic authorized operations for the \p + * topicdesc topic. + * + * @param topicdesc The topic description. + * @param idx The index for which element is needed. + * + * @return Authorized operation at given index. + */ +RD_EXPORT +const int rd_kafka_TopicDescription_authorized_operation_idx( + const rd_kafka_TopicDescription_t *topicdesc, + size_t idx); + +/** + * @brief Gets the topic authorized acl operations count for the \p topicdesc + * topic. + * + * @param topicdesc The topic description. + * + * @return The topic authorized operations count. + * + * @remark The lifetime of the returned memory is the same + * as the lifetime of the \p topicdesc object. + */ +RD_EXPORT +const int rd_kafka_TopicDescription_topic_authorized_operations_cnt( + const rd_kafka_TopicDescription_t *topicdesc); + +/** + * @brief Gets the topic name for the \p topicdesc topic. + * + * @param topicdesc The topic description. + * + * @return The topic name. + * + * @remark The lifetime of the returned memory is the same + * as the lifetime of the \p topicdesc object. + */ +RD_EXPORT +const char *rd_kafka_TopicDescription_topic_name( + const rd_kafka_TopicDescription_t *topicdesc); + +/** + * @brief Gets the error for the \p topicdesc topic. + * + * @param topicdesc The topic description. + * + * @return The topic description error. + * + * @remark The lifetime of the returned memory is the same + * as the lifetime of the \p topicdesc object. + */ +RD_EXPORT +const rd_kafka_error_t * +rd_kafka_TopicDescription_error(const rd_kafka_TopicDescription_t *topicdesc); +/** + * @brief Get an array of topic results from a DescribeTopics result. + * + * The returned topics life-time is the same as the \p result object. + * + * @param result Result to get topics results from. + * @param cntp is updated to the number of elements in the array. + * + * @remark The lifetime of the returned memory is the same + * as the lifetime of the \p result object. + */ +RD_EXPORT +const rd_kafka_TopicDescription_t **rd_kafka_DescribeTopics_result_topics( + const rd_kafka_DescribeTopics_result_t *result, + size_t *cntp); +/** + * @brief Describe topics as specified by the \p topics + * array of size \p topics_cnt elements. + * + * @param rk Client instance. + * @param topics Array of topics to describe. + * @param topics_cnt Number of elements in \p topics array. + * @param options Optional admin options, or NULL for defaults. + * @param rkqu Queue to emit result on. + * + * @remark The result event type emitted on the supplied queue is of type + * \c RD_KAFKA_EVENT_DESCRIBETOPICS_RESULT + */ +RD_EXPORT +void rd_kafka_DescribeTopics(rd_kafka_t *rk, + const char **topics, + size_t topics_cnt, + const rd_kafka_AdminOptions_t *options, + rd_kafka_queue_t *rkqu); + +/** + * @brief Whether broker should return topic authorized operations + * (DescribeTopic). + * + * @param options Admin options. + * @param true_or_false Defaults to false. + * + * @return NULL on success, a new error instance that must be + * released with rd_kafka_error_destroy() in case of error. + * + * @remark This option is valid for DescribeTopic. + */ +RD_EXPORT +rd_kafka_error_t *rd_kafka_AdminOptions_set_include_topic_authorized_operations( + rd_kafka_AdminOptions_t *options, + int true_or_false); + +/**@}*/ + +/** + * @name Admin API - DescribeCluster + * @{ + */ + +typedef struct rd_kafka_ClusterDescription_s rd_kafka_ClusterDescription_t; + +/** + * @brief Gets the node for the \p clusterdesc cluster at idx position. + * + * @param clusterdesc The cluster description. + * + * @return The node at idx position. + * + * @remark The lifetime of the returned memory is the same + * as the lifetime of the \p clusterdesc object. + */ +RD_EXPORT +const rd_kafka_Node_t *rd_kafka_ClusterDescription_node_idx( + const rd_kafka_ClusterDescription_t *clusterdesc, + int idx); + +/** + * @brief Gets operation at idx index of cluster authorized operations for the + * \p clusterdesc cluster. + * + * @param clusterdesc The cluster description. + * @param idx The index for which element is needed. + * + * @return Authorized operation at given index. + */ +RD_EXPORT +const int rd_kafka_ClusterDescription_authorized_operation_idx( + const rd_kafka_ClusterDescription_t *clusterdesc, + size_t idx); + +/** + * @brief Gets the cluster authorized acl operations for the \p clusterdesc + * cluster. + * + * @param clusterdesc The cluster description. + * + * @return The cluster authorized operations. + * + * @remark The lifetime of the returned memory is the same + * as the lifetime of the \p clusterdesc object. + */ +RD_EXPORT +const int rd_kafka_ClusterDescription_cluster_acl_operations_cnt( + const rd_kafka_ClusterDescription_t *clusterdesc); + +/** + * @brief Gets the cluster current controller id for the \p clusterdesc cluster. + * + * @param clusterdesc The cluster description. + * + * @return The cluster current controller id. + * + * @remark The lifetime of the returned memory is the same + * as the lifetime of the \p clusterdesc object. + */ +RD_EXPORT +const int rd_kafka_ClusterDescription_controller_id( + const rd_kafka_ClusterDescription_t *clusterdesc); + +/** + * @brief Gets the cluster current cluster id for the \p clusterdesc cluster. + * + * @param clusterdesc The cluster description. + * + * @return The cluster current cluster id (char*). + * + * @remark The lifetime of the returned memory is the same + * as the lifetime of the \p clusterdesc object. + */ +RD_EXPORT +const char *rd_kafka_ClusterDescription_cluster_id( + const rd_kafka_ClusterDescription_t *clusterdesc); + +/** + * @brief Gets the node count for the \p clusterdesc cluster. + * + * @param clusterdesc The cluster description. + * + * @return The node count. + * + * @remark The lifetime of the returned memory is the same + * as the lifetime of the \p clusterdesc object. + */ +RD_EXPORT +const int rd_kafka_ClusterDescription_node_cnt( + const rd_kafka_ClusterDescription_t *clusterdesc); + +/** + * @brief Get the DescribeCluster result. + * + * The returned description life-time is the same as the \p result object. + * + * @param result Result to get group results from. + * + * @remark The lifetime of the returned memory is the same + * as the lifetime of the \p result object. + */ +RD_EXPORT +const rd_kafka_ClusterDescription_t * +rd_kafka_DescribeCluster_result_description( + const rd_kafka_DescribeCluster_result_t *result); + +/** + * @brief Describes the cluster. + * + * @param rk Client instance. + * @param options Optional admin options, or NULL for defaults. + * @param rkqu Queue to emit result on. + * + * @remark The result event type emitted on the supplied queue is of type + * \c RD_KAFKA_EVENT_DESCRIBECLUSTER_RESULT + */ +RD_EXPORT +void rd_kafka_DescribeCluster(rd_kafka_t *rk, + const rd_kafka_AdminOptions_t *options, + rd_kafka_queue_t *rkqu); + +/** + * @brief Whether broker should return cluster authorized operations + * (DescribeCluster). + * + * @param options Admin options. + * @param true_or_false Defaults to false. + * + * @return NULL on success, a new error instance that must be + * released with rd_kafka_error_destroy() in case of error. + * + * @remark This option is valid for DescribeCluster. + */ +RD_EXPORT +rd_kafka_error_t * +rd_kafka_AdminOptions_set_include_cluster_authorized_operations( + rd_kafka_AdminOptions_t *options, + int true_or_false); +/**@}*/ + /** * @name Admin API - ListConsumerGroups * @{ @@ -7952,6 +8424,31 @@ RD_EXPORT const char *rd_kafka_ConsumerGroupDescription_partition_assignor( const rd_kafka_ConsumerGroupDescription_t *grpdesc); +/** + * @brief Gets count of authorized operations for the \p grpdesc group. + * + * @param grpdesc The group description. + * + * @return count of Authorized operations allowed, 0 if authorized operations + * list is NULL or empty. + */ +RD_EXPORT +size_t rd_kafka_ConsumerGroupDescription_authorized_operations_count( + const rd_kafka_ConsumerGroupDescription_t *grpdesc); + +/** + * @brief Gets operation at idx index of authorized operations for the \p + * grpdesc group. + * + * @param grpdesc The group description. + * @param idx The index for which element is needed. + * + * @return Authorized operation at given index. + */ +RD_EXPORT +int rd_kafka_ConsumerGroupDescription_authorized_operation( + const rd_kafka_ConsumerGroupDescription_t *grpdesc, + size_t idx); /** * @brief Gets state for the \p grpdesc group. diff --git a/src/rdkafka_admin.c b/src/rdkafka_admin.c index 2226899477..ed5ea1d30b 100644 --- a/src/rdkafka_admin.c +++ b/src/rdkafka_admin.c @@ -230,11 +230,12 @@ static const char *rd_kafka_admin_state_desc[] = { * @enum Admin request target broker. Must be negative values since the field * used is broker_id. */ -enum { RD_KAFKA_ADMIN_TARGET_CONTROLLER = -1, /**< Cluster controller */ - RD_KAFKA_ADMIN_TARGET_COORDINATOR = -2, /**< (Group) Coordinator */ - RD_KAFKA_ADMIN_TARGET_FANOUT = -3, /**< This rko is a fanout and - * and has no target broker */ - RD_KAFKA_ADMIN_TARGET_ALL = -4, /**< All available brokers */ +enum { + RD_KAFKA_ADMIN_TARGET_CONTROLLER = -1, /**< Cluster controller */ + RD_KAFKA_ADMIN_TARGET_COORDINATOR = -2, /**< (Group) Coordinator */ + RD_KAFKA_ADMIN_TARGET_FANOUT = -3, /**< This rko is a fanout and + * and has no target broker */ + RD_KAFKA_ADMIN_TARGET_ALL = -4, /**< All available brokers */ }; /** @@ -1556,6 +1557,36 @@ rd_kafka_error_t *rd_kafka_AdminOptions_set_require_stable_offsets( &true_or_false, errstr, sizeof(errstr)); return !err ? NULL : rd_kafka_error_new(err, "%s", errstr); } +rd_kafka_error_t *rd_kafka_AdminOptions_set_include_authorized_operations( + rd_kafka_AdminOptions_t *options, + int true_or_false) { + char errstr[512]; + rd_kafka_resp_err_t err = rd_kafka_confval_set_type( + &options->include_authorized_operations, RD_KAFKA_CONFVAL_INT, + &true_or_false, errstr, sizeof(errstr)); + return !err ? NULL : rd_kafka_error_new(err, "%s", errstr); +} + +rd_kafka_error_t *rd_kafka_AdminOptions_set_include_topic_authorized_operations( + rd_kafka_AdminOptions_t *options, + int true_or_false) { + char errstr[512]; + rd_kafka_resp_err_t err = rd_kafka_confval_set_type( + &options->include_topic_authorized_operations, RD_KAFKA_CONFVAL_INT, + &true_or_false, errstr, sizeof(errstr)); + return !err ? NULL : rd_kafka_error_new(err, "%s", errstr); +} + +rd_kafka_error_t * +rd_kafka_AdminOptions_set_include_cluster_authorized_operations( + rd_kafka_AdminOptions_t *options, + int true_or_false) { + char errstr[512]; + rd_kafka_resp_err_t err = rd_kafka_confval_set_type( + &options->include_cluster_authorized_operations, + RD_KAFKA_CONFVAL_INT, &true_or_false, errstr, sizeof(errstr)); + return !err ? NULL : rd_kafka_error_new(err, "%s", errstr); +} rd_kafka_error_t *rd_kafka_AdminOptions_set_match_consumer_group_states( rd_kafka_AdminOptions_t *options, @@ -1656,6 +1687,35 @@ static void rd_kafka_AdminOptions_init(rd_kafka_t *rk, else rd_kafka_confval_disable(&options->require_stable_offsets, "require_stable_offsets"); + if (options->for_api == RD_KAFKA_ADMIN_OP_ANY || + options->for_api == RD_KAFKA_ADMIN_OP_DESCRIBETOPICS) + rd_kafka_confval_init_int( + &options->include_topic_authorized_operations, + "include_topic_authorized_operations", 0, 1, 0); + else + rd_kafka_confval_disable( + &options->include_topic_authorized_operations, + "include_topic_authorized_operations"); + + if (options->for_api == RD_KAFKA_ADMIN_OP_ANY || + options->for_api == RD_KAFKA_ADMIN_OP_DESCRIBECLUSTER) + rd_kafka_confval_init_int( + &options->include_cluster_authorized_operations, + "include_cluster_authorized_operations", 0, 1, 0); + else + rd_kafka_confval_disable( + &options->include_cluster_authorized_operations, + "include_cluster_authorized_operations"); + + if (options->for_api == RD_KAFKA_ADMIN_OP_ANY || + options->for_api == RD_KAFKA_ADMIN_OP_DESCRIBECONSUMERGROUPS) + rd_kafka_confval_init_int( + &options->include_authorized_operations, + "include_authorized_operations", 0, 1, 0); + else + rd_kafka_confval_disable( + &options->include_authorized_operations, + "include_authorized_operations"); if (options->for_api == RD_KAFKA_ADMIN_OP_ANY || options->for_api == RD_KAFKA_ADMIN_OP_LISTCONSUMERGROUPS) @@ -5981,6 +6041,32 @@ const rd_kafka_error_t **rd_kafka_ListConsumerGroups_result_errors( * */ +/** + * @brief Parse authorized_operations returned in + * - DescribeConsumerGroups + * - DescribeTopics + * - DescribeCluster + * @returns list of acl operations + */ +rd_list_t *rd_kafka_AuthorizedOperations_parse(int32_t authorized_operations) { + int i, bit; + rd_list_t *authorized_operations_list = NULL; + rd_kafka_AclOperation_t *entry; + /* in case of authorized_operations not requested, return NULL*/ + if (authorized_operations < 0) + return NULL; + authorized_operations_list = rd_list_new(0, rd_free); + for (i = 0; i < RD_KAFKA_ACL_OPERATION__CNT; i++) { + bit = (authorized_operations >> i) & 1; + if (bit) { + entry = rd_malloc(sizeof(rd_kafka_AclOperation_t)); + *entry = i; + rd_list_add(authorized_operations_list, entry); + } + } + return authorized_operations_list; +} + /** * @brief Create a new MemberDescription object. This object is used for * creating a ConsumerGroupDescription. @@ -6099,6 +6185,7 @@ const rd_kafka_topic_partition_list_t *rd_kafka_MemberAssignment_partitions( * @param members List of members (rd_kafka_MemberDescription_t) of this * group. * @param partition_assignor (optional) Chosen assignor. + * @param authorized_operations authorized operations. * @param state Group state. * @param coordinator (optional) Group coordinator. * @param error (optional) Error received for this group. @@ -6110,10 +6197,12 @@ rd_kafka_ConsumerGroupDescription_new(const char *group_id, rd_bool_t is_simple_consumer_group, const rd_list_t *members, const char *partition_assignor, + rd_list_t *authorized_operations, rd_kafka_consumer_group_state_t state, const rd_kafka_Node_t *coordinator, rd_kafka_error_t *error) { rd_kafka_ConsumerGroupDescription_t *grpdesc; + int i; grpdesc = rd_calloc(1, sizeof(*grpdesc)); grpdesc->group_id = rd_strdup(group_id); grpdesc->is_simple_consumer_group = is_simple_consumer_group; @@ -6128,6 +6217,18 @@ rd_kafka_ConsumerGroupDescription_new(const char *group_id, grpdesc->partition_assignor = !partition_assignor ? (char *)partition_assignor : rd_strdup(partition_assignor); + if (authorized_operations == NULL) + grpdesc->authorized_operations = authorized_operations; + else { + grpdesc->authorized_operations = + rd_list_new(rd_list_cnt(authorized_operations), rd_free); + for (i = 0; i < rd_list_cnt(authorized_operations); i++) { + int *entry = rd_list_elem(authorized_operations, i); + int *oper = rd_malloc(sizeof(int)); + *oper = *entry; + rd_list_add(grpdesc->authorized_operations, oper); + } + } grpdesc->state = state; if (coordinator != NULL) grpdesc->coordinator = rd_kafka_Node_copy(coordinator); @@ -6149,7 +6250,7 @@ static rd_kafka_ConsumerGroupDescription_t * rd_kafka_ConsumerGroupDescription_new_error(const char *group_id, rd_kafka_error_t *error) { return rd_kafka_ConsumerGroupDescription_new( - group_id, rd_false, NULL, NULL, + group_id, rd_false, NULL, NULL, NULL, RD_KAFKA_CONSUMER_GROUP_STATE_UNKNOWN, NULL, error); } @@ -6164,7 +6265,8 @@ rd_kafka_ConsumerGroupDescription_copy( const rd_kafka_ConsumerGroupDescription_t *grpdesc) { return rd_kafka_ConsumerGroupDescription_new( grpdesc->group_id, grpdesc->is_simple_consumer_group, - &grpdesc->members, grpdesc->partition_assignor, grpdesc->state, + &grpdesc->members, grpdesc->partition_assignor, + grpdesc->authorized_operations, grpdesc->state, grpdesc->coordinator, grpdesc->error); } @@ -6188,6 +6290,8 @@ static void rd_kafka_ConsumerGroupDescription_destroy( rd_kafka_error_destroy(grpdesc->error); if (grpdesc->coordinator) rd_kafka_Node_destroy(grpdesc->coordinator); + if (likely(grpdesc->authorized_operations != NULL)) + rd_list_destroy(grpdesc->authorized_operations); rd_free(grpdesc); } @@ -6217,6 +6321,23 @@ const char *rd_kafka_ConsumerGroupDescription_partition_assignor( return grpdesc->partition_assignor; } +size_t rd_kafka_ConsumerGroupDescription_authorized_operations_count( + const rd_kafka_ConsumerGroupDescription_t *grpdesc) { + if (grpdesc->authorized_operations) + return grpdesc->authorized_operations->rl_cnt; + return 0; +} + +int rd_kafka_ConsumerGroupDescription_authorized_operation( + const rd_kafka_ConsumerGroupDescription_t *grpdesc, + size_t idx) { + if (grpdesc->authorized_operations) { + rd_kafka_AclOperation_t *entry = + rd_list_elem(grpdesc->authorized_operations, idx); + return *entry; + } + return 0; +} rd_kafka_consumer_group_state_t rd_kafka_ConsumerGroupDescription_state( const rd_kafka_ConsumerGroupDescription_t *grpdesc) { @@ -6314,7 +6435,7 @@ static rd_kafka_resp_err_t rd_kafka_admin_DescribeConsumerGroupsRequest( rd_kafka_replyq_t replyq, rd_kafka_resp_cb_t *resp_cb, void *opaque) { - int i; + int i, include_authorized_operations; char *group; rd_kafka_resp_err_t err; int groups_cnt = rd_list_cnt(groups); @@ -6324,7 +6445,10 @@ static rd_kafka_resp_err_t rd_kafka_admin_DescribeConsumerGroupsRequest( RD_LIST_FOREACH(group, groups, i) { groups_arr[i] = rd_list_elem(groups, i); } + include_authorized_operations = + rd_kafka_confval_get_int(&options->include_authorized_operations); error = rd_kafka_DescribeGroupsRequest(rkb, -1, groups_arr, groups_cnt, + include_authorized_operations, replyq, resp_cb, opaque); rd_free(groups_arr); @@ -6380,6 +6504,8 @@ rd_kafka_DescribeConsumerGroupsResponse_parse(rd_kafka_op_t *rko_req, node = rd_kafka_Node_new(nodeid, host, port, NULL); while (cnt-- > 0) { int16_t error_code; + int32_t authorized_operations = 0; + rd_list_t *authorized_operations_list = NULL; rd_kafkap_str_t GroupId, GroupState, ProtocolType, ProtocolData; rd_bool_t is_simple_consumer_group, is_consumer_protocol_type; int32_t member_cnt; @@ -6489,16 +6615,27 @@ rd_kafka_DescribeConsumerGroupsResponse_parse(rd_kafka_op_t *rko_req, } if (api_version >= 3) { - /* TODO: implement KIP-430 */ - int32_t authorized_operations; rd_kafka_buf_read_i32(reply, &authorized_operations); + /* assert that the last 3 bits are never set*/ + rd_assert(!((authorized_operations >> 0) & 1)); + rd_assert(!((authorized_operations >> 1) & 1)); + rd_assert(!((authorized_operations >> 2) & 1)); + /* authorized_operations is -2147483648 + * in case of not requested, list has no elements in + * that case*/ + authorized_operations_list = + rd_kafka_AuthorizedOperations_parse( + authorized_operations); } if (error == NULL) { grpdesc = rd_kafka_ConsumerGroupDescription_new( group_id, is_simple_consumer_group, &members, proto, + authorized_operations_list, rd_kafka_consumer_group_state_code(group_state), node, error); + if (authorized_operations_list) + rd_list_destroy(authorized_operations_list); } else { grpdesc = rd_kafka_ConsumerGroupDescription_new_error( group_id, error); @@ -6666,3 +6803,754 @@ rd_kafka_DescribeConsumerGroups_result_groups( } /**@}*/ + +/** + * @name Describe Topic + * @{ + * + * + * + * + */ + +const int rd_kafka_TopicDescription_authorized_operation_idx( + const rd_kafka_TopicDescription_t *topicdesc, + size_t idx) { + rd_kafka_AclOperation_t *entry = + rd_list_elem(topicdesc->topic_authorized_operations, idx); + return *entry; +} + +const int rd_kafka_TopicDescription_topic_authorized_operations_cnt( + const rd_kafka_TopicDescription_t *topicdesc) { + if (topicdesc->topic_authorized_operations) + return rd_list_cnt(topicdesc->topic_authorized_operations); + return 0; +} + +const int rd_kafka_TopicDescription_partiton_id( + const rd_kafka_TopicDescription_t *topicdesc, + int idx) { + return topicdesc->partitions[idx].id; +} + +const int rd_kafka_TopicDescription_partiton_leader( + const rd_kafka_TopicDescription_t *topicdesc, + int idx) { + return topicdesc->partitions[idx].leader; +} + +const int rd_kafka_TopicDescription_partiton_isr_cnt( + const rd_kafka_TopicDescription_t *topicdesc, + int idx) { + return topicdesc->partitions[idx].isr_cnt; +} + +const int rd_kafka_TopicDescription_partiton_replica_cnt( + const rd_kafka_TopicDescription_t *topicdesc, + int idx) { + return topicdesc->partitions[idx].replica_cnt; +} + +const int rd_kafka_TopicDescription_partiton_isrs_idx( + const rd_kafka_TopicDescription_t *topicdesc, + int partition_idx, + int isr_idx) { + return topicdesc->partitions[partition_idx].isrs[isr_idx]; +} + +const int rd_kafka_TopicDescription_partiton_replica_idx( + const rd_kafka_TopicDescription_t *topicdesc, + int partition_idx, + int replica_idx) { + return topicdesc->partitions[partition_idx].replicas[replica_idx]; +} + +const rd_kafka_error_t *rd_kafka_TopicDescription_partition_error( + const rd_kafka_TopicDescription_t *topicdesc, + int idx) { + return rd_kafka_error_new(topicdesc->partitions[idx].err, NULL); +} + +const int rd_kafka_TopicDescription_topic_partition_cnt( + const rd_kafka_TopicDescription_t *topicdesc) { + return topicdesc->partition_cnt; +} + +const char *rd_kafka_TopicDescription_topic_name( + const rd_kafka_TopicDescription_t *topicdesc) { + return topicdesc->topic; +} + +const rd_kafka_error_t * +rd_kafka_TopicDescription_error(const rd_kafka_TopicDescription_t *topicdesc) { + return rd_kafka_error_new(topicdesc->err, NULL); +} +const rd_kafka_TopicDescription_t **rd_kafka_DescribeTopics_result_topics( + const rd_kafka_DescribeTopics_result_t *result, + size_t *cntp) { + const rd_kafka_op_t *rko = (const rd_kafka_op_t *)result; + rd_kafka_op_type_t reqtype = + rko->rko_u.admin_result.reqtype & ~RD_KAFKA_OP_FLAGMASK; + rd_assert(reqtype == RD_KAFKA_OP_DESCRIBETOPICS); + + *cntp = rd_list_cnt(&rko->rko_u.admin_result.results); + return (const rd_kafka_TopicDescription_t **) + rko->rko_u.admin_result.results.rl_elems; +} +static void +rd_kafka_TopicDescription_destroy(rd_kafka_TopicDescription_t *topicdesc) { + int i; + if (likely(topicdesc->topic != NULL)) + rd_free(topicdesc->topic); + // if (likely(topicdesc->err != NULL)) + // rd_kafka_error_destroy(topicdesc->err); + for (i = 0; i < topicdesc->partition_cnt; i++) { + if (likely(topicdesc->partitions[i].isrs != NULL)) + rd_free(topicdesc->partitions[i].isrs); + if (likely(topicdesc->partitions[i].replicas != NULL)) + rd_free(topicdesc->partitions[i].replicas); + } + if (likely(topicdesc->partitions != NULL)) + rd_free(topicdesc->partitions); + if (likely(topicdesc->topic_authorized_operations != NULL)) + rd_list_destroy(topicdesc->topic_authorized_operations); + rd_free(topicdesc); +} +static void rd_kafka_TopicDescription_free(void *ptr) { + rd_kafka_TopicDescription_destroy(ptr); +} + +/** + * @brief Topics arguments comparator for DescribeTopics args + */ +static int rd_kafka_DescribeTopics_cmp(const void *a, const void *b) { + return strcmp(a, b); +} +/** + * @brief Create a new ConsumerGroupDescription object. + * + * @param topic topic name + * @param partitions Array of partition metadata (rd_kafka_metadata_partition). + * @param partition_cnt Number of partitons in partition metadata. + * @param topic_authorized_operations acl operations allowed for topic. + * @param err Topic error reported by broker. + * @return A new allocated TopicDescription object. + * Use rd_kafka_TopicDescription_destroy() to free when done. + */ +static rd_kafka_TopicDescription_t * +rd_kafka_TopicDescription_new(const char *topic, + struct rd_kafka_metadata_partition *partitions, + int partition_cnt, + rd_list_t *topic_authorized_operations, + rd_kafka_resp_err_t err) { + rd_kafka_TopicDescription_t *topicdesc; + int i, j; + topicdesc = rd_calloc(1, sizeof(*topicdesc)); + topicdesc->topic = rd_strdup(topic); + topicdesc->partition_cnt = partition_cnt; + topicdesc->err = err; + if (topic_authorized_operations) { + topicdesc->topic_authorized_operations = rd_list_new( + rd_list_cnt(topic_authorized_operations), rd_free); + for (i = 0; i < rd_list_cnt(topic_authorized_operations); i++) { + int *entry = + rd_list_elem(topic_authorized_operations, i); + int *oper = rd_malloc(sizeof(int)); + *oper = *entry; + rd_list_add(topicdesc->topic_authorized_operations, + oper); + } + } else + topicdesc->topic_authorized_operations = NULL; + + if (partitions == NULL) { + topicdesc->partitions = NULL; + } else { + topicdesc->partitions = + rd_malloc(sizeof(*partitions) * partition_cnt); + for (i = 0; i < partition_cnt; i++) { + topicdesc->partitions[i].err = partitions[i].err; + topicdesc->partitions[i].id = partitions[i].id; + topicdesc->partitions[i].isr_cnt = + partitions[i].isr_cnt; + topicdesc->partitions[i].isrs = rd_malloc( + sizeof(int32_t) * topicdesc->partitions[i].isr_cnt); + for (j = 0; j < topicdesc->partitions[i].isr_cnt; j++) { + topicdesc->partitions[i].isrs[j] = + partitions[i].isrs[j]; + } + topicdesc->partitions[i].leader = partitions[i].leader; + topicdesc->partitions[i].replica_cnt = + partitions[i].replica_cnt; + topicdesc->partitions[i].replicas = + rd_malloc(sizeof(int32_t) * + topicdesc->partitions[i].replica_cnt); + for (j = 0; j < topicdesc->partitions[i].replica_cnt; + j++) { + topicdesc->partitions[i].replicas[j] = + partitions[i].replicas[j]; + } + } + } + return topicdesc; +} +/** + * @brief Copy \p desc TopicDescription. + * + * @param desc The topic description to copy. + * @return A new allocated copy of the passed TopicDescription. + */ +static rd_kafka_TopicDescription_t * +rd_kafka_TopicDescription_copy(const rd_kafka_TopicDescription_t *topicdesc) { + return rd_kafka_TopicDescription_new( + topicdesc->topic, topicdesc->partitions, topicdesc->partition_cnt, + topicdesc->topic_authorized_operations, topicdesc->err); +} +/** + * @brief Same as rd_kafka_TopicDescription_copy() but suitable for + * rd_list_copy(). The \p opaque is ignored. + */ +static void *rd_kafka_TopicDescription_copy_opaque(const void *topicdesc, + void *opaque) { + return rd_kafka_TopicDescription_copy(topicdesc); +} +/** + * @brief New instance of TopicDescription from an error. + * + * @param group_id The topic. + * @param error The error. + * @return A new allocated TopicDescription with the passed error. + */ +static rd_kafka_TopicDescription_t * +rd_kafka_TopicDescription_new_error(const char *topic, + rd_kafka_resp_err_t err) { + return rd_kafka_TopicDescription_new(topic, NULL, 0, NULL, err); +} +/** @brief Merge the DescribeTopics response from a single broker + * into the user response list. + */ +static void +rd_kafka_DescribeTopics_response_merge(rd_kafka_op_t *rko_fanout, + const rd_kafka_op_t *rko_partial) { + rd_kafka_TopicDescription_t *topicres = NULL; + rd_kafka_TopicDescription_t *newtopicres; + const char *topic = rko_partial->rko_u.admin_result.opaque; + int orig_pos; + + rd_assert(rko_partial->rko_evtype == + RD_KAFKA_EVENT_DESCRIBETOPICS_RESULT); + + if (!rko_partial->rko_err) { + /* Proper results. + * We only send one topic per request, make sure it matches */ + topicres = + rd_list_elem(&rko_partial->rko_u.admin_result.results, 0); + rd_assert(topicres); + rd_assert(!strcmp(topicres->topic, topic)); + newtopicres = rd_kafka_TopicDescription_copy(topicres); + } else { + /* Op errored, e.g. timeout */ + rd_kafka_error_t *error = + rd_kafka_error_new(rko_partial->rko_err, NULL); + newtopicres = + rd_kafka_TopicDescription_new_error(topic, error->code); + rd_kafka_error_destroy(error); + } + + /* As a convenience to the application we insert group result + * in the same order as they were requested. */ + orig_pos = rd_list_index(&rko_fanout->rko_u.admin_request.args, topic, + rd_kafka_DescribeTopics_cmp); + rd_assert(orig_pos != -1); + + /* Make sure result is not already set */ + rd_assert(rd_list_elem(&rko_fanout->rko_u.admin_request.fanout.results, + orig_pos) == NULL); + + rd_list_set(&rko_fanout->rko_u.admin_request.fanout.results, orig_pos, + newtopicres); +} +/** + * @brief Construct and send DescribeTopicsRequest to \p rkb + * with the topics (char *) in \p groups, using + * \p options. + * + * The response (unparsed) will be enqueued on \p replyq + * for handling by \p resp_cb (with \p opaque passed). + * + * @returns RD_KAFKA_RESP_ERR_NO_ERROR if the request was enqueued for + * transmission, otherwise an error code and errstr will be + * updated with a human readable error string. + */ +static rd_kafka_resp_err_t +rd_kafka_admin_DescribeTopicsRequest(rd_kafka_broker_t *rkb, + const rd_list_t *topics /*(char*)*/, + rd_kafka_AdminOptions_t *options, + char *errstr, + size_t errstr_size, + rd_kafka_replyq_t replyq, + rd_kafka_resp_cb_t *resp_cb, + void *opaque) { + rd_kafka_resp_err_t err; + int include_topic_authorized_operations; + + include_topic_authorized_operations = rd_kafka_confval_get_int( + &options->include_topic_authorized_operations); + /* resp_cb = rd_kafka_admin_handle_response; */ + + // error = Call metadata request + err = rd_kafka_MetadataRequest(rkb, topics, "describe topics", rd_false, + rd_false, + include_topic_authorized_operations, + rd_false, rd_false, NULL, resp_cb, 0, opaque); + + if (err) { + rd_snprintf(errstr, errstr_size, "%s", rd_kafka_err2str(err)); + return err; + } + + return RD_KAFKA_RESP_ERR_NO_ERROR; +} + +/** + * @brief Parse DescribeTopicsResponse and create ADMIN_RESULT op. + */ +static rd_kafka_resp_err_t +rd_kafka_DescribeTopicsResponse_parse(rd_kafka_op_t *rko_req, + rd_kafka_op_t **rko_resultp, + rd_kafka_buf_t *reply, + char *errstr, + size_t errstr_size) { + rd_kafka_metadata_internal_t *mdi = NULL; + struct rd_kafka_metadata *md = NULL; + rd_kafka_resp_err_t err; + rd_list_t topics = rko_req->rko_u.admin_request.args; + rd_kafka_broker_t *rkb = reply->rkbuf_rkb; + int i, cnt; + rd_kafka_op_t *rko_result = NULL; + // rd_kafka_assert(NULL, err == RD_KAFKA_RESP_ERR__DESTROY || + // thrd_is_current(rk->rk_thread)); + // rd_kafka_assert(NULL, err == RD_KAFKA_RESP_ERR__DESTROY); + + err = rd_kafka_parse_Metadata(rkb, NULL, reply, &mdi, &topics); + if (err) + goto err; + rko_result = rd_kafka_admin_result_new(rko_req); + md = &mdi->metadata; + rd_list_init(&rko_result->rko_u.admin_result.results, md->topic_cnt, + rd_kafka_TopicDescription_free); + cnt = md->topic_cnt; + i = 0; + while (cnt--) { + rd_kafka_TopicDescription_t *topicdesc = NULL; + /* topics in md should be in the same order as in + * mdi->topics[i]*/ + rd_assert(strcmp(md->topics[i].topic, + mdi->topics[i].topic_name) == + 0); + if (md->topics[i].err == RD_KAFKA_RESP_ERR_NO_ERROR) { + rd_list_t *authorized_operations; + authorized_operations = + rd_kafka_AuthorizedOperations_parse( + mdi->topics[i].topic_authorized_operations); + topicdesc = rd_kafka_TopicDescription_new( + md->topics[i].topic, md->topics[i].partitions, + md->topics[i].partition_cnt, authorized_operations, + md->topics[i].err); + if (authorized_operations) + rd_list_destroy(authorized_operations); + } else + topicdesc = rd_kafka_TopicDescription_new_error( + md->topics[i].topic, md->topics[i].err); + rd_list_add(&rko_result->rko_u.admin_result.results, topicdesc); + } + *rko_resultp = rko_result; + return RD_KAFKA_RESP_ERR_NO_ERROR; + +err: + if (rko_result) + rd_kafka_op_destroy(rko_result); + rd_snprintf(errstr, errstr_size, + "DescribeTopics response protocol parse failure: %s", + rd_kafka_err2str(reply->rkbuf_err)); + return reply->rkbuf_err; +} +void rd_kafka_DescribeTopics(rd_kafka_t *rk, + const char **topics, + size_t topics_cnt, + const rd_kafka_AdminOptions_t *options, + rd_kafka_queue_t *rkqu) { + rd_kafka_op_t *rko_fanout; + rd_list_t dup_list; + size_t i; + + static const struct rd_kafka_admin_fanout_worker_cbs fanout_cbs = { + rd_kafka_DescribeTopics_response_merge, + rd_kafka_TopicDescription_copy_opaque}; + + rd_assert(rkqu); + + rko_fanout = + rd_kafka_admin_fanout_op_new(rk, RD_KAFKA_OP_DESCRIBETOPICS, + RD_KAFKA_EVENT_DESCRIBETOPICS_RESULT, + &fanout_cbs, options, rkqu->rkqu_q); + + if (topics_cnt == 0) { + rd_kafka_admin_result_fail(rko_fanout, + RD_KAFKA_RESP_ERR__INVALID_ARG, + "No topics to describe"); + rd_kafka_admin_common_worker_destroy(rk, rko_fanout, + rd_true /*destroy*/); + return; + } + + /* Copy topic list and store it on the request op. + * Maintain original ordering. */ + rd_list_init(&rko_fanout->rko_u.admin_request.args, (int)topics_cnt, + rd_free); + for (i = 0; i < topics_cnt; i++) + rd_list_add(&rko_fanout->rko_u.admin_request.args, + rd_strdup(topics[i])); + + /* Check for duplicates. + * Make a temporary copy of the topic list and sort it to check for + * duplicates, we don't want the original list sorted since we want + * to maintain ordering. */ + rd_list_init(&dup_list, + rd_list_cnt(&rko_fanout->rko_u.admin_request.args), NULL); + rd_list_copy_to(&dup_list, &rko_fanout->rko_u.admin_request.args, NULL, + NULL); + rd_list_sort(&dup_list, rd_kafka_DescribeTopics_cmp); + if (rd_list_find_duplicate(&dup_list, rd_kafka_DescribeTopics_cmp)) { + rd_list_destroy(&dup_list); + rd_kafka_admin_result_fail(rko_fanout, + RD_KAFKA_RESP_ERR__INVALID_ARG, + "Duplicate topics not allowed"); + rd_kafka_admin_common_worker_destroy(rk, rko_fanout, + rd_true /*destroy*/); + return; + } + + rd_list_destroy(&dup_list); + + /* Prepare results list where fanned out op's results will be + * accumulated. */ + rd_list_init(&rko_fanout->rko_u.admin_request.fanout.results, + (int)topics_cnt, rd_kafka_TopicDescription_free); + rko_fanout->rko_u.admin_request.fanout.outstanding = (int)topics_cnt; + + /* Create individual request ops for each topic. + * FIXME: A future optimization is to coalesce all topic for a single + * coordinator into one op. */ + for (i = 0; i < topics_cnt; i++) { + static const struct rd_kafka_admin_worker_cbs cbs = { + rd_kafka_admin_DescribeTopicsRequest, + rd_kafka_DescribeTopicsResponse_parse, + }; + char *topic = + rd_list_elem(&rko_fanout->rko_u.admin_request.args, (int)i); + rd_kafka_op_t *rko = rd_kafka_admin_request_op_new( + rk, RD_KAFKA_OP_DESCRIBETOPICS, + RD_KAFKA_EVENT_DESCRIBETOPICS_RESULT, &cbs, options, + rk->rk_ops); + + rko->rko_u.admin_request.fanout_parent = rko_fanout; + // rko->rko_u.admin_request.broker_id = + // RD_KAFKA_ADMIN_TARGET_COORDINATOR; + rko->rko_u.admin_request.coordtype = RD_KAFKA_COORD_GROUP; + rko->rko_u.admin_request.coordkey = rd_strdup(topic); + + /* Set the topic name as the opaque so the fanout worker use it + * to fill in errors. + * References rko_fanout's memory, which will always outlive + * the fanned out op. */ + rd_kafka_AdminOptions_set_opaque( + &rko->rko_u.admin_request.options, topic); + + rd_list_init(&rko->rko_u.admin_request.args, 1, rd_free); + rd_list_add(&rko->rko_u.admin_request.args, + rd_strdup(topics[i])); + + rd_kafka_q_enq(rk->rk_ops, rko); + } +} + +/**@}*/ + +/** + * @name Describe cluster + * @{ + * + * + * + * + */ + +const rd_kafka_Node_t *rd_kafka_ClusterDescription_node_idx( + const rd_kafka_ClusterDescription_t *clusterdesc, + int idx) { + return &(clusterdesc->Nodes[idx]); +} + +const int rd_kafka_ClusterDescription_authorized_operation_idx( + const rd_kafka_ClusterDescription_t *clusterdesc, + size_t idx) { + rd_kafka_AclOperation_t *entry = + rd_list_elem(clusterdesc->cluster_authorized_operations, idx); + return *entry; +} + +const char *rd_kafka_ClusterDescription_cluster_id( + const rd_kafka_ClusterDescription_t *clusterdesc) { + return clusterdesc->cluster_id; +} +const int rd_kafka_ClusterDescription_controller_id( + const rd_kafka_ClusterDescription_t *clusterdesc) { + return clusterdesc->controller_id; +} +const int rd_kafka_ClusterDescription_cluster_acl_operations_cnt( + const rd_kafka_ClusterDescription_t *clusterdesc) { + if (clusterdesc->cluster_authorized_operations) + return rd_list_cnt(clusterdesc->cluster_authorized_operations); + return 0; +} +const int rd_kafka_ClusterDescription_node_cnt( + const rd_kafka_ClusterDescription_t *clusterdesc) { + return clusterdesc->node_cnt; +} + +const rd_kafka_ClusterDescription_t * +rd_kafka_DescribeCluster_result_description( + const rd_kafka_DescribeTopics_result_t *result) { + int cluster_result_cnt; + const rd_kafka_ClusterDescription_t *clusterdesc; + const rd_kafka_op_t *rko = (const rd_kafka_op_t *)result; + rd_kafka_op_type_t reqtype = + rko->rko_u.admin_result.reqtype & ~RD_KAFKA_OP_FLAGMASK; + rd_assert(reqtype == RD_KAFKA_OP_DESCRIBECLUSTER); + + cluster_result_cnt = rd_list_cnt(&rko->rko_u.admin_result.results); + rd_assert(cluster_result_cnt == 1); + clusterdesc = rd_list_elem(&rko->rko_u.admin_result.results, 0); + + return clusterdesc; +} + +/** + * @brief Copy \p desc ClusterDescription. + * + * @param desc The cluster description to copy. + * @return A new allocated copy of the passed ClusterDescription. + */ +static rd_kafka_ClusterDescription_t * +rd_kafka_ClusterDescription_copy(const rd_kafka_ClusterDescription_t *desc) { + rd_kafka_ClusterDescription_t *clusterdesc; + int i; + clusterdesc = rd_calloc(1, sizeof(*clusterdesc)); + + clusterdesc->cluster_id = rd_strdup(desc->cluster_id); + clusterdesc->controller_id = desc->controller_id; + + if (desc->cluster_authorized_operations) { + clusterdesc->cluster_authorized_operations = rd_list_new( + rd_list_cnt(desc->cluster_authorized_operations), rd_free); + for (i = 0; + i < rd_list_cnt(desc->cluster_authorized_operations); + i++) { + int *entry = rd_list_elem( + desc->cluster_authorized_operations, i); + int *oper = malloc(sizeof(int)); + *oper = *entry; + rd_list_add(clusterdesc->cluster_authorized_operations, + oper); + } + } else + clusterdesc->cluster_authorized_operations = NULL; + + clusterdesc->node_cnt = desc->node_cnt; + clusterdesc->Nodes = malloc(sizeof(rd_kafka_Node_t) * desc->node_cnt); + for (i = 0; i < desc->node_cnt; i++) { + clusterdesc->Nodes[i].host = rd_strdup(desc->Nodes[i].host); + clusterdesc->Nodes[i].port = desc->Nodes[i].port; + clusterdesc->Nodes[i].id = desc->Nodes[i].id; + } + return clusterdesc; +} + +/** + * @brief Create a new ClusterDescription object. + * + * @param cluster_id current cluster_id + * @param controller_id current controller_id. + * @param md metadata struct returned by parse_metadata(). + * + * @returns new ClusterDescription object. + */ +static rd_kafka_ClusterDescription_t * +rd_kafka_ClusterDescription_new(const char *cluster_id, + int controller_id, + rd_list_t *cluster_authorized_operations, + struct rd_kafka_metadata *md) { + rd_kafka_ClusterDescription_t *clusterdesc; + int i; + clusterdesc = rd_calloc(1, sizeof(*clusterdesc)); + + clusterdesc->cluster_id = rd_strdup(cluster_id); + clusterdesc->controller_id = controller_id; + + if (cluster_authorized_operations) { + clusterdesc->cluster_authorized_operations = rd_list_new( + rd_list_cnt(cluster_authorized_operations), rd_free); + for (i = 0; i < rd_list_cnt(cluster_authorized_operations); + i++) { + int *entry = + rd_list_elem(cluster_authorized_operations, i); + int *oper = malloc(sizeof(int)); + *oper = *entry; + rd_list_add(clusterdesc->cluster_authorized_operations, + oper); + } + } else + clusterdesc->cluster_authorized_operations = NULL; + + clusterdesc->node_cnt = md->broker_cnt; + clusterdesc->Nodes = + rd_malloc(sizeof(rd_kafka_Node_t) * md->broker_cnt); + for (i = 0; i < md->broker_cnt; i++) { + clusterdesc->Nodes[i].host = rd_strdup(md->brokers[i].host); + clusterdesc->Nodes[i].port = md->brokers[i].port; + clusterdesc->Nodes[i].id = md->brokers[i].id; + } + return clusterdesc; +} + +static void rd_kafka_ClusterDescription_destroy( + rd_kafka_ClusterDescription_t *clusterdesc) { + int i; + RD_IF_FREE(clusterdesc->cluster_id, rd_free); + if (clusterdesc->cluster_authorized_operations) + rd_list_destroy(clusterdesc->cluster_authorized_operations); + if (clusterdesc->Nodes) { + for (i = 0; i < clusterdesc->node_cnt; i++) { + rd_kafka_Node_t *node = &(clusterdesc->Nodes[i]); + RD_IF_FREE(node->host, rd_free); + RD_IF_FREE(node->rack_id, rd_free); + } + RD_IF_FREE(clusterdesc->Nodes, rd_free); + } + rd_free(clusterdesc); +} +static void rd_kafka_ClusterDescription_free(void *ptr) { + rd_kafka_ClusterDescription_destroy(ptr); +} +/** + * @brief Send DescribeClusterRequest. Admin worker compatible callback. + */ +static rd_kafka_resp_err_t +rd_kafka_admin_DescribeClusterRequest(rd_kafka_broker_t *rkb, + const rd_list_t *topics /*(char*)*/, + rd_kafka_AdminOptions_t *options, + char *errstr, + size_t errstr_size, + rd_kafka_replyq_t replyq, + rd_kafka_resp_cb_t *resp_cb, + void *opaque) { + rd_kafka_resp_err_t err; + int include_cluster_authorized_operations; + + include_cluster_authorized_operations = rd_kafka_confval_get_int( + &options->include_cluster_authorized_operations); + /* resp_cb = rd_kafka_admin_handle_response; */ + + // err = Call metadata request with NULL topics + err = rd_kafka_MetadataRequest(rkb, NULL, "describe cluster", + rd_false /*no auto create*/, + include_cluster_authorized_operations, + rd_false /*!include topic authorized operations */, + rd_false /*cgrp update*/, + rd_false /* force_rack */, + NULL, resp_cb, 1, opaque); + + if (err) { + rd_snprintf(errstr, errstr_size, "%s", rd_kafka_err2str(err)); + return err; + } + + return RD_KAFKA_RESP_ERR_NO_ERROR; +} +/** + * @brief Parse DescribeCluster and create ADMIN_RESULT op. + */ +static rd_kafka_resp_err_t +rd_kafka_DescribeClusterResponse_parse(rd_kafka_op_t *rko_req, + rd_kafka_op_t **rko_resultp, + rd_kafka_buf_t *reply, + char *errstr, + size_t errstr_size) { + rd_kafka_metadata_internal_t *mdi = NULL; + rd_kafka_resp_err_t err; + rd_kafka_ClusterDescription_t *clusterdesc = NULL; + rd_list_t topics = rko_req->rko_u.admin_request.args; + rd_kafka_broker_t *rkb = reply->rkbuf_rkb; + int32_t cluster_authorized_operations; + char *cluster_id = NULL; + int controller_id; + rd_kafka_op_t *rko_result = NULL; + // rd_kafka_assert(NULL, err == RD_KAFKA_RESP_ERR__DESTROY || + // thrd_is_current(rk->rk_thread)); + // rd_kafka_assert(NULL, err == RD_KAFKA_RESP_ERR__DESTROY); + + err = rd_kafka_parse_Metadata(rkb, NULL, reply, &mdi, &topics); + cluster_id = mdi->cluster_id; + controller_id = mdi->controller_id; + cluster_authorized_operations = mdi->cluster_authorized_operations; + if (err) + goto err; + rko_result = rd_kafka_admin_result_new(rko_req); + rd_list_init(&rko_result->rko_u.admin_result.results, 1, + rd_kafka_ClusterDescription_free); + + rd_list_t *authorized_operations; + authorized_operations = + rd_kafka_AuthorizedOperations_parse(cluster_authorized_operations); + + clusterdesc = rd_kafka_ClusterDescription_new( + cluster_id, controller_id, authorized_operations, &mdi->metadata); + if (authorized_operations) + rd_list_destroy(authorized_operations); + rd_free(cluster_id); + + rd_list_add(&rko_result->rko_u.admin_result.results, clusterdesc); + *rko_resultp = rko_result; + return RD_KAFKA_RESP_ERR_NO_ERROR; + +err: + if (rko_result) + rd_kafka_op_destroy(rko_result); + rd_snprintf(errstr, errstr_size, + "DescribeTopics response protocol parse failure: %s", + rd_kafka_err2str(reply->rkbuf_err)); + return reply->rkbuf_err; +} +void rd_kafka_DescribeCluster(rd_kafka_t *rk, + const rd_kafka_AdminOptions_t *options, + rd_kafka_queue_t *rkqu) { + rd_kafka_op_t *rko; + static const struct rd_kafka_admin_worker_cbs cbs = { + rd_kafka_admin_DescribeClusterRequest, + rd_kafka_DescribeClusterResponse_parse}; + // static const struct rd_kafka_admin_fanout_worker_cbs fanout_cbs = { + // rd_kafka_ClusterDescription_response_merge, + // rd_kafka_ClusterDescription_copy_opaque, + // }; + rko = rd_kafka_admin_request_op_new( + rk, RD_KAFKA_OP_DESCRIBECLUSTER, + RD_KAFKA_EVENT_DESCRIBECLUSTER_RESULT, &cbs, options, rkqu->rkqu_q); + // rko = rd_kafka_admin_request_op_target_all_new( + // rk, RD_KAFKA_OP_DESCRIBECLUSTER, + // RD_KAFKA_EVENT_DESCRIBECLUSTER_RESULT, &cbs, &fanout_cbs, + // rd_kafka_ClusterDescriptionResult_free, options, rkqu->rkqu_q); + rd_kafka_q_enq(rk->rk_ops, rko); +} +/**@}*/ \ No newline at end of file diff --git a/src/rdkafka_admin.h b/src/rdkafka_admin.h index 62fe9e87a3..0175f3b3e6 100644 --- a/src/rdkafka_admin.h +++ b/src/rdkafka_admin.h @@ -91,6 +91,28 @@ struct rd_kafka_AdminOptions_s { * Valid for: * ListConsumerGroupOffsets */ + rd_kafka_confval_t + include_authorized_operations; /**< BOOL: Whether broker should + * return authorized operations. + * Valid for: + * DescribeConsumerGroups + */ + + rd_kafka_confval_t + include_topic_authorized_operations; /**< BOOL: Whether broker + * should return topic + * authorized operations. Valid + * for: DescribeTopic + * MetadataRequest + */ + rd_kafka_confval_t + include_cluster_authorized_operations; /**< BOOL: Whether broker + * should return cluster + * authorized operations. + * Valid for: + * DescribeCluster + * MetadataRequest + */ rd_kafka_confval_t match_consumer_group_states; /**< PTR: list of consumer group states @@ -473,10 +495,48 @@ struct rd_kafka_ConsumerGroupDescription_s { rd_kafka_consumer_group_state_t state; /** Consumer group coordinator. */ rd_kafka_Node_t *coordinator; + /** Authorized operations. */ + rd_list_t *authorized_operations; /** Group specific error. */ rd_kafka_error_t *error; }; /**@}*/ +/** + * @name DescribeTopics + * @{ + */ +/** + * @struct DescribeTopics result + */ +struct rd_kafka_TopicDescription_s { + char *topic; /**< Topic name */ + int partition_cnt; /**< Number of partitions in \p partitions*/ + struct rd_kafka_metadata_partition *partitions; /**< Partitions */ + rd_kafka_resp_err_t err; /**< Topic error reported by broker */ + rd_list_t *topic_authorized_operations; /**< ACL operations allowed for + topics */ +}; + +/**@}*/ + +/** + * @name DescribeCluster + * @{ + */ +/** + * @struct DescribeCluster result + */ +struct rd_kafka_ClusterDescription_s { + char *cluster_id; /**< current cluster id in \p cluster*/ + int controller_id; /**< current controller id in \p cluster*/ + int node_cnt; /**< Number of brokers in \p cluster*/ + rd_kafka_Node_t *Nodes; /**< Nodes */ + rd_list_t *cluster_authorized_operations; /**< ACL operations allowed + for cluster */ +}; + +/**@}*/ + #endif /* _RDKAFKA_ADMIN_H_ */ diff --git a/src/rdkafka_cgrp.c b/src/rdkafka_cgrp.c index 48ef02514c..0a38f76980 100644 --- a/src/rdkafka_cgrp.c +++ b/src/rdkafka_cgrp.c @@ -2042,6 +2042,8 @@ static void rd_kafka_cgrp_handle_JoinGroup(rd_kafka_t *rk, rd_kafka_MetadataRequest( rkb, &topics, "partition assignor", rd_false /*!allow_auto_create*/, + rd_false /*!include cluster authorized operations */, + rd_false /*!include topic authorized operations */, /* cgrp_update=false: * Since the subscription list may not be identical * across all members of the group and thus the @@ -2054,7 +2056,7 @@ static void rd_kafka_cgrp_handle_JoinGroup(rd_kafka_t *rk, /* force_racks is true if any memeber has a client rack set, since we will require partition to rack mapping in that case for rack-aware assignors. */ - any_member_rack, rko); + any_member_rack, rko, NULL, 0, NULL); rd_list_destroy(&topics); } else { @@ -2240,9 +2242,11 @@ static int rd_kafka_cgrp_metadata_refresh(rd_kafka_cgrp_t *rkcg, rd_kafka_cgrp_handle_Metadata_op); rd_kafka_op_set_replyq(rko, rkcg->rkcg_ops, 0); - err = rd_kafka_metadata_request(rkcg->rkcg_rk, NULL, &topics, - rd_false /*!allow auto create */, - rd_true /*cgrp_update*/, reason, rko); + err = rd_kafka_metadata_request( + rkcg->rkcg_rk, NULL, &topics, rd_false /*!allow auto create */, + rd_false /*!include cluster authorized operations */, + rd_false /*!include topic authorized operations */, + rd_true /*cgrp_update*/, reason, rko); if (err) { rd_kafka_dbg(rk, CGRP | RD_KAFKA_DBG_METADATA, "CGRPMETADATA", "%s: need to refresh metadata (%dms old) " diff --git a/src/rdkafka_event.c b/src/rdkafka_event.c index ffd1a17805..7e2aa2e4e0 100644 --- a/src/rdkafka_event.c +++ b/src/rdkafka_event.c @@ -68,6 +68,10 @@ const char *rd_kafka_event_name(const rd_kafka_event_t *rkev) { return "ListConsumerGroupsResult"; case RD_KAFKA_EVENT_DESCRIBECONSUMERGROUPS_RESULT: return "DescribeConsumerGroupsResult"; + case RD_KAFKA_EVENT_DESCRIBETOPICS_RESULT: + return "DescribeTopicsResult"; + case RD_KAFKA_EVENT_DESCRIBECLUSTER_RESULT: + return "DescribeClusterResult"; case RD_KAFKA_EVENT_DELETEGROUPS_RESULT: return "DeleteGroupsResult"; case RD_KAFKA_EVENT_DELETECONSUMERGROUPOFFSETS_RESULT: @@ -364,6 +368,22 @@ rd_kafka_event_DescribeConsumerGroups_result(rd_kafka_event_t *rkev) { return (const rd_kafka_DescribeConsumerGroups_result_t *)rkev; } +const rd_kafka_DescribeTopics_result_t * +rd_kafka_event_DescribeTopics_result(rd_kafka_event_t *rkev) { + if (!rkev || rkev->rko_evtype != RD_KAFKA_EVENT_DESCRIBETOPICS_RESULT) + return NULL; + else + return (const rd_kafka_DescribeTopics_result_t *)rkev; +} + +const rd_kafka_DescribeCluster_result_t * +rd_kafka_event_DescribeCluster_result(rd_kafka_event_t *rkev) { + if (!rkev || rkev->rko_evtype != RD_KAFKA_EVENT_DESCRIBECLUSTER_RESULT) + return NULL; + else + return (const rd_kafka_DescribeCluster_result_t *)rkev; +} + const rd_kafka_DeleteGroups_result_t * rd_kafka_event_DeleteGroups_result(rd_kafka_event_t *rkev) { if (!rkev || rkev->rko_evtype != RD_KAFKA_EVENT_DELETEGROUPS_RESULT) diff --git a/src/rdkafka_event.h b/src/rdkafka_event.h index 3f9c22e34b..5abf9ea2b3 100644 --- a/src/rdkafka_event.h +++ b/src/rdkafka_event.h @@ -102,6 +102,8 @@ static RD_UNUSED RD_INLINE int rd_kafka_event_setup(rd_kafka_t *rk, case RD_KAFKA_EVENT_DELETERECORDS_RESULT: case RD_KAFKA_EVENT_LISTCONSUMERGROUPS_RESULT: case RD_KAFKA_EVENT_DESCRIBECONSUMERGROUPS_RESULT: + case RD_KAFKA_EVENT_DESCRIBETOPICS_RESULT: + case RD_KAFKA_EVENT_DESCRIBECLUSTER_RESULT: case RD_KAFKA_EVENT_DELETEGROUPS_RESULT: case RD_KAFKA_EVENT_DELETECONSUMERGROUPOFFSETS_RESULT: case RD_KAFKA_EVENT_CREATEACLS_RESULT: diff --git a/src/rdkafka_metadata.c b/src/rdkafka_metadata.c index b12e8b796c..84ed53ac20 100644 --- a/src/rdkafka_metadata.c +++ b/src/rdkafka_metadata.c @@ -103,13 +103,15 @@ rd_kafka_metadata(rd_kafka_t *rk, * of outstanding metadata requests. */ rd_kafka_MetadataRequest( rkb, &topics, "application requested", allow_auto_create_topics, + rd_false /*!include cluster authorized operations */, + rd_false /*!include topic authorized operations */, /* cgrp_update: * Only update consumer group state * on response if this lists all * topics in the cluster, since a * partial request may make it seem * like some subscribed topics are missing. */ - all_topics ? rd_true : rd_false, rd_false /* force_racks */, rko); + all_topics ? rd_true : rd_false, rd_false /* force_racks */, rko, NULL, 0, NULL); rd_list_destroy(&topics); rd_kafka_broker_destroy(rkb); @@ -438,6 +440,7 @@ rd_kafka_populate_metadata_topic_racks(rd_tmpabuf_t *tbuf, * @brief Handle a Metadata response message. * * @param topics are the requested topics (may be NULL) + * @param request_topics Use when rd_kafka_buf_t* request is NULL * * The metadata will be marshalled into 'rd_kafka_metadata_internal_t *'. * @@ -451,25 +454,32 @@ rd_kafka_resp_err_t rd_kafka_parse_Metadata(rd_kafka_broker_t *rkb, rd_kafka_buf_t *request, rd_kafka_buf_t *rkbuf, - rd_kafka_metadata_internal_t **mdip) { + rd_kafka_metadata_internal_t **mdip, + rd_list_t *request_topics) { rd_kafka_t *rk = rkb->rkb_rk; int i, j, k; rd_tmpabuf_t tbuf; rd_kafka_metadata_internal_t *mdi = NULL; rd_kafka_metadata_t *md = NULL; size_t rkb_namelen; - const int log_decode_errors = LOG_ERR; - rd_list_t *missing_topics = NULL; - const rd_list_t *requested_topics = request->rkbuf_u.Metadata.topics; - rd_bool_t all_topics = request->rkbuf_u.Metadata.all_topics; + const int log_decode_errors = LOG_ERR; + rd_list_t *missing_topics = NULL; + const rd_list_t *requested_topics = + request ? request->rkbuf_u.Metadata.topics : request_topics; + rd_bool_t all_topics = + request ? request->rkbuf_u.Metadata.all_topics : rd_false; rd_bool_t cgrp_update = - request->rkbuf_u.Metadata.cgrp_update && rk->rk_cgrp; + request ? (request->rkbuf_u.Metadata.cgrp_update && rk->rk_cgrp) + : rd_false; rd_bool_t has_reliable_leader_epochs = rd_kafka_has_reliable_leader_epochs(rkb); - const char *reason = request->rkbuf_u.Metadata.reason - ? request->rkbuf_u.Metadata.reason - : "(no reason)"; - int ApiVersion = request->rkbuf_reqhdr.ApiVersion; + const char *reason = request ? (request->rkbuf_u.Metadata.reason + ? request->rkbuf_u.Metadata.reason + : "(no reason)") + : "(admin request)"; + /* changed from request->rkbuf_reqhdr.ApiVersion as request buffer may + * be NULL*/ + int ApiVersion = rkbuf->rkbuf_reqhdr.ApiVersion; rd_kafkap_str_t cluster_id = RD_ZERO_INIT; int32_t controller_id = -1; rd_kafka_resp_err_t err = RD_KAFKA_RESP_ERR_NO_ERROR; @@ -483,7 +493,7 @@ rd_kafka_parse_Metadata(rd_kafka_broker_t *rkb, /* If force_racks is true, the outptr mdip has to contain the partition * to rack map. */ rd_bool_t force_rack_computation = - request->rkbuf_u.Metadata.force_racks; + request ? request->rkbuf_u.Metadata.force_racks : rd_false; rd_bool_t compute_racks = has_client_rack || force_rack_computation; /* Ignore metadata updates when terminating */ @@ -559,11 +569,15 @@ rd_kafka_parse_Metadata(rd_kafka_broker_t *rkb, rd_kafka_buf_skip_tags(rkbuf); } - if (ApiVersion >= 2) + if (ApiVersion >= 2) { rd_kafka_buf_read_str(rkbuf, &cluster_id); + mdi->cluster_id = RD_KAFKAP_STR_DUP(&cluster_id); + } + if (ApiVersion >= 1) { rd_kafka_buf_read_i32(rkbuf, &controller_id); + mdi->controller_id = controller_id; rd_rkb_dbg(rkb, METADATA, "METADATA", "ClusterId: %.*s, ControllerId: %" PRId32, RD_KAFKAP_STR_PR(&cluster_id), controller_id); @@ -581,7 +595,7 @@ rd_kafka_parse_Metadata(rd_kafka_broker_t *rkb, rd_tmpabuf_alloc(&tbuf, md->topic_cnt * sizeof(*md->topics)))) rd_kafka_buf_parse_fail( rkbuf, "%d topics: tmpabuf memory shortage", md->topic_cnt); - + if (!(mdi->topics = rd_tmpabuf_alloc(&tbuf, md->topic_cnt * sizeof(*mdi->topics)))) rd_kafka_buf_parse_fail( @@ -719,6 +733,9 @@ rd_kafka_parse_Metadata(rd_kafka_broker_t *rkb, /* TopicAuthorizedOperations */ rd_kafka_buf_read_i32(rkbuf, &TopicAuthorizedOperations); + mdi->topics[i].topic_name = md->topics[i].topic; + mdi->topics[i].topic_authorized_operations = + TopicAuthorizedOperations; } rd_kafka_buf_skip_tags(rkbuf); @@ -781,6 +798,8 @@ rd_kafka_parse_Metadata(rd_kafka_broker_t *rkb, int32_t ClusterAuthorizedOperations; /* ClusterAuthorizedOperations */ rd_kafka_buf_read_i32(rkbuf, &ClusterAuthorizedOperations); + mdi->cluster_authorized_operations = + ClusterAuthorizedOperations; } rd_kafka_buf_skip_tags(rkbuf); @@ -1245,7 +1264,10 @@ rd_kafka_metadata_refresh_topics(rd_kafka_t *rk, rd_list_cnt(&q_topics), rd_list_cnt(topics), reason); rd_kafka_MetadataRequest(rkb, &q_topics, reason, allow_auto_create, - cgrp_update, rd_false /* force_racks */, NULL); + rd_false /*!include cluster authorized operations */, + rd_false /*!include topic authorized operations */, + cgrp_update, rd_false /* force_racks */, NULL, + NULL, 0, NULL); rd_list_destroy(&q_topics); @@ -1385,10 +1407,12 @@ rd_kafka_metadata_refresh_consumer_topics(rd_kafka_t *rk, rd_kafka_resp_err_t rd_kafka_metadata_refresh_brokers(rd_kafka_t *rk, rd_kafka_broker_t *rkb, const char *reason) { - return rd_kafka_metadata_request(rk, rkb, NULL /*brokers only*/, - rd_false /*!allow auto create topics*/, - rd_false /*no cgrp update */, reason, - NULL); + return rd_kafka_metadata_request( + rk, rkb, NULL /*brokers only*/, + rd_false /*!allow auto create topics*/, + rd_false /*!include cluster authorized operations */, + rd_false /*!include topic authorized operations */, + rd_false /*no cgrp update */, reason, NULL); } @@ -1422,7 +1446,10 @@ rd_kafka_resp_err_t rd_kafka_metadata_refresh_all(rd_kafka_t *rk, rd_list_init(&topics, 0, NULL); /* empty list = all topics */ rd_kafka_MetadataRequest( rkb, &topics, reason, rd_false /*no auto create*/, - rd_true /*cgrp update*/, rd_false /* force_rack */, NULL); + rd_false /*!include cluster authorized operations */, + rd_false /*!include topic authorized operations */, + rd_true /*cgrp update*/, rd_false /* force_rack */, + NULL, NULL, 0, NULL); rd_list_destroy(&topics); if (destroy_rkb) @@ -1447,6 +1474,8 @@ rd_kafka_metadata_request(rd_kafka_t *rk, rd_kafka_broker_t *rkb, const rd_list_t *topics, rd_bool_t allow_auto_create_topics, + rd_bool_t include_cluster_authorized_operations, + rd_bool_t include_topic_authorized_operations, rd_bool_t cgrp_update, const char *reason, rd_kafka_op_t *rko) { @@ -1460,7 +1489,9 @@ rd_kafka_metadata_request(rd_kafka_t *rk, } rd_kafka_MetadataRequest(rkb, topics, reason, allow_auto_create_topics, - cgrp_update, rd_false /* force racks */, rko); + include_cluster_authorized_operations, + include_topic_authorized_operations, + cgrp_update, rd_false /* force racks */, rko, NULL, 0, NULL); if (destroy_rkb) rd_kafka_broker_destroy(rkb); diff --git a/src/rdkafka_metadata.h b/src/rdkafka_metadata.h index 2598401363..f7f97a7195 100644 --- a/src/rdkafka_metadata.h +++ b/src/rdkafka_metadata.h @@ -53,6 +53,10 @@ typedef struct rd_kafka_metadata_topic_internal_s { * same count as metadata.topics[i].partition_cnt. * Sorted by Partition Id. */ rd_kafka_metadata_partition_internal_t *partitions; + /** Topic Name. */ + char* topic_name; + int32_t topic_authorized_operations; /**< ACL operations allowed + for topic */ } rd_kafka_metadata_topic_internal_t; @@ -80,6 +84,10 @@ typedef struct rd_kafka_metadata_internal_s { rd_kafka_metadata_broker_internal_t *brokers; /* Internal metadata topics. Same count as metadata.topic_cnt. */ rd_kafka_metadata_topic_internal_t *topics; + char *cluster_id; /**< current cluster id in \p cluster*/ + int controller_id; /**< current controller id in \p cluster*/ + int32_t cluster_authorized_operations; /**< ACL operations allowed + for cluster */ } rd_kafka_metadata_internal_t; /** @@ -93,7 +101,8 @@ rd_bool_t rd_kafka_has_reliable_leader_epochs(rd_kafka_broker_t *rkb); rd_kafka_resp_err_t rd_kafka_parse_Metadata(rd_kafka_broker_t *rkb, rd_kafka_buf_t *request, rd_kafka_buf_t *rkbuf, - rd_kafka_metadata_internal_t **mdp); + rd_kafka_metadata_internal_t **mdp, + rd_list_t *request_topics); rd_kafka_metadata_internal_t * rd_kafka_metadata_copy(const rd_kafka_metadata_internal_t *mdi, size_t size); @@ -148,6 +157,8 @@ rd_kafka_metadata_request(rd_kafka_t *rk, rd_kafka_broker_t *rkb, const rd_list_t *topics, rd_bool_t allow_auto_create_topics, + rd_bool_t include_cluster_authorized_operations, + rd_bool_t include_topic_authorized_operations, rd_bool_t cgrp_update, const char *reason, rd_kafka_op_t *rko); diff --git a/src/rdkafka_op.c b/src/rdkafka_op.c index 0a4635f9ab..db9df2b4b9 100644 --- a/src/rdkafka_op.c +++ b/src/rdkafka_op.c @@ -84,7 +84,9 @@ const char *rd_kafka_op2str(rd_kafka_op_type_t type) { [RD_KAFKA_OP_LISTCONSUMERGROUPS] = "REPLY:LISTCONSUMERGROUPS", [RD_KAFKA_OP_DESCRIBECONSUMERGROUPS] = "REPLY:DESCRIBECONSUMERGROUPS", - [RD_KAFKA_OP_DELETEGROUPS] = "REPLY:DELETEGROUPS", + [RD_KAFKA_OP_DESCRIBETOPICS] = "REPLY:DESCRIBETOPICS", + [RD_KAFKA_OP_DESCRIBECLUSTER] = "REPLY:DESCRIBECLUSTER", + [RD_KAFKA_OP_DELETEGROUPS] = "REPLY:DELETEGROUPS", [RD_KAFKA_OP_DELETECONSUMERGROUPOFFSETS] = "REPLY:DELETECONSUMERGROUPOFFSETS", [RD_KAFKA_OP_CREATEACLS] = "REPLY:CREATEACLS", @@ -234,7 +236,9 @@ rd_kafka_op_t *rd_kafka_op_new0(const char *source, rd_kafka_op_type_t type) { [RD_KAFKA_OP_LISTCONSUMERGROUPS] = sizeof(rko->rko_u.admin_request), [RD_KAFKA_OP_DESCRIBECONSUMERGROUPS] = sizeof(rko->rko_u.admin_request), - [RD_KAFKA_OP_DELETEGROUPS] = sizeof(rko->rko_u.admin_request), + [RD_KAFKA_OP_DESCRIBETOPICS] = sizeof(rko->rko_u.admin_request), + [RD_KAFKA_OP_DESCRIBECLUSTER] = sizeof(rko->rko_u.admin_request), + [RD_KAFKA_OP_DELETEGROUPS] = sizeof(rko->rko_u.admin_request), [RD_KAFKA_OP_DELETECONSUMERGROUPOFFSETS] = sizeof(rko->rko_u.admin_request), [RD_KAFKA_OP_CREATEACLS] = sizeof(rko->rko_u.admin_request), @@ -401,6 +405,8 @@ void rd_kafka_op_destroy(rd_kafka_op_t *rko) { case RD_KAFKA_OP_DESCRIBEACLS: case RD_KAFKA_OP_DELETEACLS: case RD_KAFKA_OP_ALTERCONSUMERGROUPOFFSETS: + case RD_KAFKA_OP_DESCRIBETOPICS: + case RD_KAFKA_OP_DESCRIBECLUSTER: case RD_KAFKA_OP_LISTCONSUMERGROUPOFFSETS: rd_kafka_replyq_destroy(&rko->rko_u.admin_request.replyq); rd_list_destroy(&rko->rko_u.admin_request.args); diff --git a/src/rdkafka_op.h b/src/rdkafka_op.h index 45f8d55bbb..e57f99098d 100644 --- a/src/rdkafka_op.h +++ b/src/rdkafka_op.h @@ -137,6 +137,12 @@ typedef enum { RD_KAFKA_OP_DESCRIBECONSUMERGROUPS, /**< Admin: * DescribeConsumerGroups * u.admin_request */ + RD_KAFKA_OP_DESCRIBETOPICS, /**< Admin: + * DescribeTopics + * u.admin_request */ + RD_KAFKA_OP_DESCRIBECLUSTER, /**< Admin: + * DescribeCluster + * u.admin_request */ RD_KAFKA_OP_DELETEGROUPS, /**< Admin: DeleteGroups: u.admin_request*/ RD_KAFKA_OP_DELETECONSUMERGROUPOFFSETS, /**< Admin: * DeleteConsumerGroupOffsets @@ -441,13 +447,14 @@ struct rd_kafka_op_s { struct rd_kafka_admin_worker_cbs *cbs; /** Worker state */ - enum { RD_KAFKA_ADMIN_STATE_INIT, - RD_KAFKA_ADMIN_STATE_WAIT_BROKER, - RD_KAFKA_ADMIN_STATE_WAIT_CONTROLLER, - RD_KAFKA_ADMIN_STATE_WAIT_FANOUTS, - RD_KAFKA_ADMIN_STATE_CONSTRUCT_REQUEST, - RD_KAFKA_ADMIN_STATE_WAIT_RESPONSE, - RD_KAFKA_ADMIN_STATE_WAIT_BROKER_LIST, + enum { + RD_KAFKA_ADMIN_STATE_INIT, + RD_KAFKA_ADMIN_STATE_WAIT_BROKER, + RD_KAFKA_ADMIN_STATE_WAIT_CONTROLLER, + RD_KAFKA_ADMIN_STATE_WAIT_FANOUTS, + RD_KAFKA_ADMIN_STATE_CONSTRUCT_REQUEST, + RD_KAFKA_ADMIN_STATE_WAIT_RESPONSE, + RD_KAFKA_ADMIN_STATE_WAIT_BROKER_LIST, } state; int32_t broker_id; /**< Requested broker id to @@ -537,16 +544,17 @@ struct rd_kafka_op_s { /**< Mock cluster command */ struct { - enum { RD_KAFKA_MOCK_CMD_TOPIC_SET_ERROR, - RD_KAFKA_MOCK_CMD_TOPIC_CREATE, - RD_KAFKA_MOCK_CMD_PART_SET_LEADER, - RD_KAFKA_MOCK_CMD_PART_SET_FOLLOWER, - RD_KAFKA_MOCK_CMD_PART_SET_FOLLOWER_WMARKS, - RD_KAFKA_MOCK_CMD_BROKER_SET_UPDOWN, - RD_KAFKA_MOCK_CMD_BROKER_SET_RTT, - RD_KAFKA_MOCK_CMD_BROKER_SET_RACK, - RD_KAFKA_MOCK_CMD_COORD_SET, - RD_KAFKA_MOCK_CMD_APIVERSION_SET, + enum { + RD_KAFKA_MOCK_CMD_TOPIC_SET_ERROR, + RD_KAFKA_MOCK_CMD_TOPIC_CREATE, + RD_KAFKA_MOCK_CMD_PART_SET_LEADER, + RD_KAFKA_MOCK_CMD_PART_SET_FOLLOWER, + RD_KAFKA_MOCK_CMD_PART_SET_FOLLOWER_WMARKS, + RD_KAFKA_MOCK_CMD_BROKER_SET_UPDOWN, + RD_KAFKA_MOCK_CMD_BROKER_SET_RTT, + RD_KAFKA_MOCK_CMD_BROKER_SET_RACK, + RD_KAFKA_MOCK_CMD_COORD_SET, + RD_KAFKA_MOCK_CMD_APIVERSION_SET, } cmd; rd_kafka_resp_err_t err; /**< Error for: diff --git a/src/rdkafka_partition.h b/src/rdkafka_partition.h index 355cbeb7ac..16e9a86496 100644 --- a/src/rdkafka_partition.h +++ b/src/rdkafka_partition.h @@ -285,13 +285,14 @@ struct rd_kafka_toppar_s { /* rd_kafka_toppar_t */ int32_t rktp_fetch_version; /* Op version of curr fetch. (broker thread) */ - enum { RD_KAFKA_TOPPAR_FETCH_NONE = 0, - RD_KAFKA_TOPPAR_FETCH_STOPPING, - RD_KAFKA_TOPPAR_FETCH_STOPPED, - RD_KAFKA_TOPPAR_FETCH_OFFSET_QUERY, - RD_KAFKA_TOPPAR_FETCH_OFFSET_WAIT, - RD_KAFKA_TOPPAR_FETCH_VALIDATE_EPOCH_WAIT, - RD_KAFKA_TOPPAR_FETCH_ACTIVE, + enum { + RD_KAFKA_TOPPAR_FETCH_NONE = 0, + RD_KAFKA_TOPPAR_FETCH_STOPPING, + RD_KAFKA_TOPPAR_FETCH_STOPPED, + RD_KAFKA_TOPPAR_FETCH_OFFSET_QUERY, + RD_KAFKA_TOPPAR_FETCH_OFFSET_WAIT, + RD_KAFKA_TOPPAR_FETCH_VALIDATE_EPOCH_WAIT, + RD_KAFKA_TOPPAR_FETCH_ACTIVE, } rktp_fetch_state; /* Broker thread's state */ #define RD_KAFKA_TOPPAR_FETCH_IS_STARTED(fetch_state) \ diff --git a/src/rdkafka_request.c b/src/rdkafka_request.c index e9f1506945..457c15ce3f 100644 --- a/src/rdkafka_request.c +++ b/src/rdkafka_request.c @@ -2016,6 +2016,8 @@ rd_kafka_error_t *rd_kafka_ListGroupsRequest(rd_kafka_broker_t *rkb, * with the groups (const char *) in \p groups. * Uses \p max_ApiVersion as maximum API version, * pass -1 to use the maximum available version. + * Uses \p include_authorized_operations to get + * group ACL authorized operations, 1 to request. * * The response (unparsed) will be enqueued on \p replyq * for handling by \p resp_cb (with \p opaque passed). @@ -2023,13 +2025,15 @@ rd_kafka_error_t *rd_kafka_ListGroupsRequest(rd_kafka_broker_t *rkb, * @return NULL on success, a new error instance that must be * released with rd_kafka_error_destroy() in case of error. */ -rd_kafka_error_t *rd_kafka_DescribeGroupsRequest(rd_kafka_broker_t *rkb, - int16_t max_ApiVersion, - char **groups, - size_t group_cnt, - rd_kafka_replyq_t replyq, - rd_kafka_resp_cb_t *resp_cb, - void *opaque) { +rd_kafka_error_t * +rd_kafka_DescribeGroupsRequest(rd_kafka_broker_t *rkb, + int16_t max_ApiVersion, + char **groups, + size_t group_cnt, + rd_bool_t include_authorized_operations, + rd_kafka_replyq_t replyq, + rd_kafka_resp_cb_t *resp_cb, + void *opaque) { rd_kafka_buf_t *rkbuf; int16_t ApiVersion = 0; size_t of_GroupsArrayCnt; @@ -2066,8 +2070,7 @@ rd_kafka_error_t *rd_kafka_DescribeGroupsRequest(rd_kafka_broker_t *rkb, /* write IncludeAuthorizedOperations */ if (ApiVersion >= 3) { - /* TODO: implement KIP-430 */ - rd_kafka_buf_write_bool(rkbuf, rd_false); + rd_kafka_buf_write_bool(rkbuf, include_authorized_operations); } rd_kafka_buf_ApiVersion_set(rkbuf, ApiVersion, 0); @@ -2115,7 +2118,7 @@ static void rd_kafka_handle_Metadata(rd_kafka_t *rk, rd_list_cnt(topics), request->rkbuf_u.Metadata.reason); - err = rd_kafka_parse_Metadata(rkb, request, rkbuf, &mdi); + err = rd_kafka_parse_Metadata(rkb, request, rkbuf, &mdi, NULL); if (err) goto err; @@ -2187,36 +2190,49 @@ static void rd_kafka_handle_Metadata(rd_kafka_t *rk, * @param allow_auto_create_topics - allow broker-side auto topic creation. * This is best-effort, depending on broker * config and version. + * @param include_cluster_authorized_operations - request for cluster + * authorized operations. + * @param include_topic_authorized_operations - request for topic authorized + * operations. * @param cgrp_update - Update cgrp in parse_Metadata (see comment there). * @param force_racks - Force partition to rack mapping computation in * parse_Metadata (see comment there). * @param rko - (optional) rko with replyq for handling response. * Specifying an rko forces a metadata request even if * there is already a matching one in-transit. - * + * @param resp_cb - callback to be used for handling response. If NULL, then + * rd_kafka_handle_Metadata() is used. + * @param force - 1: force a full request. + * 0: check if there are multiple outstanding full requests. * If full metadata for all topics is requested (or all brokers, which * results in all-topics on older brokers) and there is already a full request * in transit then this function will return RD_KAFKA_RESP_ERR__PREV_IN_PROGRESS * otherwise RD_KAFKA_RESP_ERR_NO_ERROR. If \p rko is non-NULL the request * is sent regardless. */ -rd_kafka_resp_err_t rd_kafka_MetadataRequest(rd_kafka_broker_t *rkb, - const rd_list_t *topics, - const char *reason, - rd_bool_t allow_auto_create_topics, - rd_bool_t cgrp_update, - rd_bool_t force_racks, - rd_kafka_op_t *rko) { +rd_kafka_resp_err_t +rd_kafka_MetadataRequest(rd_kafka_broker_t *rkb, + const rd_list_t *topics, + const char *reason, + rd_bool_t allow_auto_create_topics, + rd_bool_t include_cluster_authorized_operations, + rd_bool_t include_topic_authorized_operations, + rd_bool_t cgrp_update, + rd_bool_t force_racks, + rd_kafka_op_t *rko, + rd_kafka_resp_cb_t *resp_cb, + int force, + void *opaque) { rd_kafka_buf_t *rkbuf; int16_t ApiVersion = 0; + int i; size_t of_TopicArrayCnt; int features; int topic_cnt = topics ? rd_list_cnt(topics) : 0; int *full_incr = NULL; ApiVersion = rd_kafka_broker_ApiVersion_supported( - rkb, RD_KAFKAP_Metadata, 0, 9, &features); - + rkb, RD_KAFKAP_Metadata, 0, 8, &features); rkbuf = rd_kafka_buf_new_flexver_request(rkb, RD_KAFKAP_Metadata, 1, 4 + (50 * topic_cnt) + 1, ApiVersion >= 9); @@ -2282,7 +2298,8 @@ rd_kafka_resp_err_t rd_kafka_MetadataRequest(rd_kafka_broker_t *rkb, * through regardless. */ mtx_lock(&rkb->rkb_rk->rk_metadata_cache.rkmc_full_lock); - if (*full_incr > 0 && (!rko || !rko->rko_u.metadata.force)) { + if (!force && + (*full_incr > 0 && (!rko || !rko->rko_u.metadata.force))) { mtx_unlock( &rkb->rkb_rk->rk_metadata_cache.rkmc_full_lock); rd_rkb_dbg(rkb, METADATA, "METADATA", @@ -2339,13 +2356,16 @@ rd_kafka_resp_err_t rd_kafka_MetadataRequest(rd_kafka_broker_t *rkb, if (ApiVersion >= 8 && ApiVersion < 10) { /* TODO: implement KIP-430 */ /* IncludeClusterAuthorizedOperations */ - rd_kafka_buf_write_bool(rkbuf, rd_false); + + rd_kafka_buf_write_bool(rkbuf, + include_cluster_authorized_operations); } if (ApiVersion >= 8) { /* TODO: implement KIP-430 */ /* IncludeTopicAuthorizedOperations */ - rd_kafka_buf_write_bool(rkbuf, rd_false); + rd_kafka_buf_write_bool(rkbuf, + include_topic_authorized_operations); } rd_kafka_buf_ApiVersion_set(rkbuf, ApiVersion, 0); @@ -2354,12 +2374,13 @@ rd_kafka_resp_err_t rd_kafka_MetadataRequest(rd_kafka_broker_t *rkb, * and should go before most other requests (Produce, Fetch, etc). */ rkbuf->rkbuf_prio = RD_KAFKA_PRIO_HIGH; - rd_kafka_broker_buf_enq_replyq(rkb, rkbuf, - /* Handle response thru rk_ops, - * but forward parsed result to - * rko's replyq when done. */ - RD_KAFKA_REPLYQ(rkb->rkb_rk->rk_ops, 0), - rd_kafka_handle_Metadata, rko); + rd_kafka_broker_buf_enq_replyq( + rkb, rkbuf, + /* Handle response thru rk_ops, + * but forward parsed result to + * rko's replyq when done. */ + RD_KAFKA_REPLYQ(rkb->rkb_rk->rk_ops, 0), + resp_cb ? resp_cb : rd_kafka_handle_Metadata, rko ? rko : opaque); return RD_KAFKA_RESP_ERR_NO_ERROR; } diff --git a/src/rdkafka_request.h b/src/rdkafka_request.h index de5b8510a6..b8743ffadf 100644 --- a/src/rdkafka_request.h +++ b/src/rdkafka_request.h @@ -235,13 +235,15 @@ rd_kafka_error_t *rd_kafka_ListGroupsRequest(rd_kafka_broker_t *rkb, rd_kafka_resp_cb_t *resp_cb, void *opaque); -rd_kafka_error_t *rd_kafka_DescribeGroupsRequest(rd_kafka_broker_t *rkb, - int16_t max_ApiVersion, - char **groups, - size_t group_cnt, - rd_kafka_replyq_t replyq, - rd_kafka_resp_cb_t *resp_cb, - void *opaque); +rd_kafka_error_t * +rd_kafka_DescribeGroupsRequest(rd_kafka_broker_t *rkb, + int16_t max_ApiVersion, + char **groups, + size_t group_cnt, + rd_bool_t include_authorized_operations, + rd_kafka_replyq_t replyq, + rd_kafka_resp_cb_t *resp_cb, + void *opaque); void rd_kafka_HeartbeatRequest(rd_kafka_broker_t *rkb, @@ -253,13 +255,19 @@ void rd_kafka_HeartbeatRequest(rd_kafka_broker_t *rkb, rd_kafka_resp_cb_t *resp_cb, void *opaque); -rd_kafka_resp_err_t rd_kafka_MetadataRequest(rd_kafka_broker_t *rkb, - const rd_list_t *topics, - const char *reason, - rd_bool_t allow_auto_create_topics, - rd_bool_t cgrp_update, - rd_bool_t force_racks, - rd_kafka_op_t *rko); +rd_kafka_resp_err_t +rd_kafka_MetadataRequest(rd_kafka_broker_t *rkb, + const rd_list_t *topics, + const char *reason, + rd_bool_t allow_auto_create_topics, + rd_bool_t include_cluster_authorized_operations, + rd_bool_t include_topic_authorized_operations, + rd_bool_t cgrp_update, + rd_bool_t force_racks, + rd_kafka_op_t *rko, + rd_kafka_resp_cb_t *resp_cb, + int force, + void *opaque); rd_kafka_resp_err_t rd_kafka_handle_ApiVersion(rd_kafka_t *rk, diff --git a/src/rdkafka_topic.h b/src/rdkafka_topic.h index f5d4c00c95..8b7ff894ad 100644 --- a/src/rdkafka_topic.h +++ b/src/rdkafka_topic.h @@ -147,11 +147,12 @@ struct rd_kafka_topic_s { rd_refcnt_t rkt_app_refcnt; /**< Number of active rkt's new()ed * by application. */ - enum { RD_KAFKA_TOPIC_S_UNKNOWN, /* No cluster information yet */ - RD_KAFKA_TOPIC_S_EXISTS, /* Topic exists in cluster */ - RD_KAFKA_TOPIC_S_NOTEXISTS, /* Topic is not known in cluster */ - RD_KAFKA_TOPIC_S_ERROR, /* Topic exists but is in an errored - * state, such as auth failure. */ + enum { + RD_KAFKA_TOPIC_S_UNKNOWN, /* No cluster information yet */ + RD_KAFKA_TOPIC_S_EXISTS, /* Topic exists in cluster */ + RD_KAFKA_TOPIC_S_NOTEXISTS, /* Topic is not known in cluster */ + RD_KAFKA_TOPIC_S_ERROR, /* Topic exists but is in an errored + * state, such as auth failure. */ } rkt_state; int rkt_flags; diff --git a/tests/0080-admin_ut.c b/tests/0080-admin_ut.c index 9d049e5b14..5b6eb1936a 100644 --- a/tests/0080-admin_ut.c +++ b/tests/0080-admin_ut.c @@ -633,9 +633,10 @@ static void do_test_DescribeConsumerGroups(const char *what, char errstr[512]; const char *errstr2; rd_kafka_resp_err_t err; + rd_kafka_error_t *error; test_timing_t timing; rd_kafka_event_t *rkev; - const rd_kafka_DeleteGroups_result_t *res; + const rd_kafka_DescribeConsumerGroups_result_t *res; const rd_kafka_ConsumerGroupDescription_t **resgroups; size_t resgroup_cnt; void *my_opaque = NULL, *opaque; @@ -657,6 +658,17 @@ static void do_test_DescribeConsumerGroups(const char *what, err = rd_kafka_AdminOptions_set_request_timeout( options, exp_timeout, errstr, sizeof(errstr)); TEST_ASSERT(!err, "%s", rd_kafka_err2str(err)); + if ((error = + rd_kafka_AdminOptions_set_include_authorized_operations( + options, 0))) { + fprintf(stderr, + "%% Failed to set require authorized " + "operations: %s\n", + rd_kafka_error_string(error)); + rd_kafka_error_destroy(error); + TEST_FAIL( + "Failed to set include authorized operations\n"); + } if (useq) { my_opaque = (void *)456; @@ -724,6 +736,11 @@ static void do_test_DescribeConsumerGroups(const char *what, group_names[i], rd_kafka_error_string( rd_kafka_ConsumerGroupDescription_error(resgroups[i]))); + TEST_ASSERT( + rd_kafka_ConsumerGroupDescription_authorized_operations_count( + resgroups[i]) == 0, + "Got authorized operations" + "when not requested"); } rd_kafka_event_destroy(rkev); @@ -743,6 +760,250 @@ static void do_test_DescribeConsumerGroups(const char *what, SUB_TEST_PASS(); } +/** + * @brief DescribeTopics tests + * + * + * + */ +static void do_test_DescribeTopics(const char *what, + rd_kafka_t *rk, + rd_kafka_queue_t *useq, + int with_options, + rd_bool_t destroy) { + rd_kafka_queue_t *q; +#define TEST_DESCRIBE_TOPICS_CNT 4 + const char *topic_names[TEST_DESCRIBE_TOPICS_CNT]; + rd_kafka_AdminOptions_t *options = NULL; + int exp_timeout = MY_SOCKET_TIMEOUT_MS; + int i; + char errstr[512]; + const char *errstr2; + rd_kafka_resp_err_t err; + rd_kafka_error_t *error; + test_timing_t timing; + rd_kafka_event_t *rkev; + const rd_kafka_DescribeTopics_result_t *res; + const rd_kafka_TopicDescription_t **restopics; + size_t restopic_cnt; + void *my_opaque = NULL, *opaque; + + SUB_TEST_QUICK("%s DescribeTopics with %s, timeout %dms", + rd_kafka_name(rk), what, exp_timeout); + + q = useq ? useq : rd_kafka_queue_new(rk); + + for (i = 0; i < TEST_DESCRIBE_TOPICS_CNT; i++) { + topic_names[i] = rd_strdup(test_mk_topic_name(__FUNCTION__, 1)); + } + + if (with_options) { + options = rd_kafka_AdminOptions_new( + rk, RD_KAFKA_ADMIN_OP_DESCRIBETOPICS); + + exp_timeout = MY_SOCKET_TIMEOUT_MS * 2; + err = rd_kafka_AdminOptions_set_request_timeout( + options, exp_timeout, errstr, sizeof(errstr)); + TEST_ASSERT(!err, "%s", rd_kafka_err2str(err)); + if ((error = + rd_kafka_AdminOptions_set_include_topic_authorized_operations( + options, 0))) { + fprintf(stderr, + "%% Failed to set topic authorized operations: " + "%s\n", + rd_kafka_error_string(error)); + rd_kafka_error_destroy(error); + TEST_FAIL( + "Failed to set topic authorized operations\n"); + } + + if (useq) { + my_opaque = (void *)456; + rd_kafka_AdminOptions_set_opaque(options, my_opaque); + } + } + + TIMING_START(&timing, "DescribeTopics"); + TEST_SAY("Call DescribeTopics, timeout is %dms\n", exp_timeout); + rd_kafka_DescribeTopics(rk, topic_names, TEST_DESCRIBE_TOPICS_CNT, + options, q); + TIMING_ASSERT_LATER(&timing, 0, 50); + + if (destroy) + goto destroy; + + /* Poll result queue */ + TIMING_START(&timing, "DescribeTopics.queue_poll"); + rkev = rd_kafka_queue_poll(q, exp_timeout + 1000); + TIMING_ASSERT_LATER(&timing, exp_timeout - 100, exp_timeout + 100); + TEST_ASSERT(rkev != NULL, "expected result in %dms", exp_timeout); + TEST_SAY("DescribeTopics: got %s in %.3fs\n", rd_kafka_event_name(rkev), + TIMING_DURATION(&timing) / 1000.0f); + + /* Convert event to proper result */ + res = rd_kafka_event_DescribeTopics_result(rkev); + TEST_ASSERT(res, "expected DescribeTopics_result, not %s", + rd_kafka_event_name(rkev)); + + opaque = rd_kafka_event_opaque(rkev); + TEST_ASSERT(opaque == my_opaque, "expected opaque to be %p, not %p", + my_opaque, opaque); + + /* Expecting no error (errors will be per-topic) */ + err = rd_kafka_event_error(rkev); + errstr2 = rd_kafka_event_error_string(rkev); + TEST_ASSERT(err == RD_KAFKA_RESP_ERR_NO_ERROR, + "expected DescribeTopics to return error %s, not %s (%s)", + rd_kafka_err2str(RD_KAFKA_RESP_ERR_NO_ERROR), + rd_kafka_err2str(err), err ? errstr2 : "n/a"); + + /* Extract topics, should return TEST_DESCRIBE_TOPICS_CNT topics. */ + restopics = rd_kafka_DescribeTopics_result_topics(res, &restopic_cnt); + TEST_ASSERT(restopics && restopic_cnt == TEST_DESCRIBE_TOPICS_CNT, + "expected %d result_topics, got %p cnt %" PRIusz, + TEST_DESCRIBE_TOPICS_CNT, restopics, restopic_cnt); + + /* The returned topics should be in the original order, and + * should all have timed out. */ + for (i = 0; i < TEST_DESCRIBE_TOPICS_CNT; i++) { + TEST_ASSERT( + !strcmp(topic_names[i], + rd_kafka_TopicDescription_topic_name(restopics[i])), + "expected topic '%s' at position %d, not '%s'", + topic_names[i], i, + rd_kafka_TopicDescription_topic_name(restopics[i])); + TEST_ASSERT(rd_kafka_error_code(rd_kafka_TopicDescription_error( + restopics[i])) == RD_KAFKA_RESP_ERR__TIMED_OUT, + "expected topic '%s' to have timed out, got %s", + topic_names[i], + rd_kafka_error_string( + rd_kafka_TopicDescription_error(restopics[i]))); + TEST_ASSERT( + rd_kafka_TopicDescription_topic_authorized_operations_cnt( + restopics[i]) == 0, + "Got topic authorized operations" + "when not requested"); + } + + rd_kafka_event_destroy(rkev); + +destroy: + for (i = 0; i < TEST_DESCRIBE_TOPICS_CNT; i++) { + rd_free((char *)topic_names[i]); + } + + if (options) + rd_kafka_AdminOptions_destroy(options); + + if (!useq) + rd_kafka_queue_destroy(q); +#undef TEST_DESCRIBE_TOPICS_CNT + + SUB_TEST_PASS(); +} + +/** + * @brief DescribeCluster tests + * + * + * + */ +static void do_test_DescribeCluster(const char *what, + rd_kafka_t *rk, + rd_kafka_queue_t *useq, + int with_options, + rd_bool_t destroy) { + rd_kafka_queue_t *q; + rd_kafka_AdminOptions_t *options = NULL; + int exp_timeout = MY_SOCKET_TIMEOUT_MS; + int i; + char errstr[512]; + const char *errstr2; + rd_kafka_resp_err_t err; + rd_kafka_error_t *error; + test_timing_t timing; + rd_kafka_event_t *rkev; + const rd_kafka_DescribeCluster_result_t *res; + const rd_kafka_ClusterDescription_t *clusterdesc; + void *my_opaque = NULL, *opaque; + + SUB_TEST_QUICK("%s DescribeCluster with %s, timeout %dms", + rd_kafka_name(rk), what, exp_timeout); + + q = useq ? useq : rd_kafka_queue_new(rk); + + if (with_options) { + options = rd_kafka_AdminOptions_new( + rk, RD_KAFKA_ADMIN_OP_DESCRIBECLUSTER); + + exp_timeout = MY_SOCKET_TIMEOUT_MS * 2; + err = rd_kafka_AdminOptions_set_request_timeout( + options, exp_timeout, errstr, sizeof(errstr)); + TEST_ASSERT(!err, "%s", rd_kafka_err2str(err)); + if ((error = + rd_kafka_AdminOptions_set_include_cluster_authorized_operations( + options, 0))) { + fprintf(stderr, + "%% Failed to set cluster authorized " + "operations: %s\n", + rd_kafka_error_string(error)); + rd_kafka_error_destroy(error); + TEST_FAIL( + "Failed to set cluster authorized operations\n"); + } + + if (useq) { + my_opaque = (void *)456; + rd_kafka_AdminOptions_set_opaque(options, my_opaque); + } + } + + TIMING_START(&timing, "DescribeCluster"); + TEST_SAY("Call DescribeCluster, timeout is %dms\n", exp_timeout); + rd_kafka_DescribeCluster(rk, options, q); + TIMING_ASSERT_LATER(&timing, 0, 50); + + if (destroy) + goto destroy; + + /* Poll result queue */ + TIMING_START(&timing, "DescribeCluster.queue_poll"); + rkev = rd_kafka_queue_poll(q, exp_timeout + 1000); + TIMING_ASSERT_LATER(&timing, exp_timeout - 100, exp_timeout + 100); + TEST_ASSERT(rkev != NULL, "expected result in %dms", exp_timeout); + TEST_SAY("DescribeCluster: got %s in %.3fs\n", + rd_kafka_event_name(rkev), TIMING_DURATION(&timing) / 1000.0f); + + /* Convert event to proper result */ + res = rd_kafka_event_DescribeCluster_result(rkev); + TEST_ASSERT(res, "expected DescribeCluster_result, not %s", + rd_kafka_event_name(rkev)); + + opaque = rd_kafka_event_opaque(rkev); + TEST_ASSERT(opaque == my_opaque, "expected opaque to be %p, not %p", + my_opaque, opaque); + + /* Expecting error (Fail while waiting for controller)*/ + err = rd_kafka_event_error(rkev); + errstr2 = rd_kafka_event_error_string(rkev); + TEST_ASSERT(err == RD_KAFKA_RESP_ERR__TIMED_OUT, + "expected DescribeCluster to return error %s, not %s (%s)", + rd_kafka_err2str(RD_KAFKA_RESP_ERR__TIMED_OUT), + rd_kafka_err2str(err), err ? errstr2 : "n/a"); + + rd_kafka_event_destroy(rkev); + +destroy: + + if (options) + rd_kafka_AdminOptions_destroy(options); + + if (!useq) + rd_kafka_queue_destroy(q); + + SUB_TEST_PASS(); +} + static void do_test_DeleteRecords(const char *what, rd_kafka_t *rk, rd_kafka_queue_t *useq, @@ -2444,6 +2705,15 @@ static void do_test_apis(rd_kafka_type_t cltype) { do_test_DescribeConsumerGroups("main queue, options", rk, mainq, 1, rd_false); + do_test_DescribeTopics("temp queue, no options", rk, NULL, 0, rd_false); + do_test_DescribeTopics("temp queue, options", rk, NULL, 1, rd_false); + do_test_DescribeTopics("main queue, options", rk, mainq, 1, rd_false); + + do_test_DescribeCluster("temp queue, no options", rk, NULL, 0, + rd_false); + do_test_DescribeCluster("temp queue, options", rk, NULL, 1, rd_false); + do_test_DescribeCluster("main queue, options", rk, mainq, 1, rd_false); + do_test_DeleteGroups("temp queue, no options", rk, NULL, 0, rd_false); do_test_DeleteGroups("temp queue, options", rk, NULL, 1, rd_false); do_test_DeleteGroups("main queue, options", rk, mainq, 1, rd_false); diff --git a/tests/0081-admin.c b/tests/0081-admin.c index 7da2dff156..96ab22be54 100644 --- a/tests/0081-admin.c +++ b/tests/0081-admin.c @@ -2739,6 +2739,10 @@ static void do_test_DescribeConsumerGroups(const char *what, rd_kafka_ConsumerGroupDescription_error(act)); rd_kafka_consumer_group_state_t state = rd_kafka_ConsumerGroupDescription_state(act); + TEST_ASSERT( + rd_kafka_ConsumerGroupDescription_authorized_operations_count( + act) == 0, + "Authorized operations returned when not requested\n"); TEST_ASSERT( strcmp(exp->group_id, rd_kafka_ConsumerGroupDescription_group_id(act)) == @@ -2748,84 +2752,891 @@ static void do_test_DescribeConsumerGroups(const char *what, i, exp->group_id, rd_kafka_ConsumerGroupDescription_group_id(act)); if (i < known_groups) { - int member_count; - const rd_kafka_MemberDescription_t *member; - const rd_kafka_MemberAssignment_t *assignment; - const char *client_id; - const char *group_instance_id; - const rd_kafka_topic_partition_list_t *partitions; + int member_count; + const rd_kafka_MemberDescription_t *member; + const rd_kafka_MemberAssignment_t *assignment; + const char *client_id; + const char *group_instance_id; + const rd_kafka_topic_partition_list_t *partitions; + + TEST_ASSERT(state == + RD_KAFKA_CONSUMER_GROUP_STATE_STABLE, + "Expected Stable state, got %s.", + rd_kafka_consumer_group_state_name(state)); + + TEST_ASSERT( + !rd_kafka_ConsumerGroupDescription_is_simple_consumer_group( + act), + "Expected a normal consumer group, got a simple " + "one."); + + member_count = + rd_kafka_ConsumerGroupDescription_member_count(act); + TEST_ASSERT(member_count == 1, + "Expected one member, got %d.", + member_count); + + member = + rd_kafka_ConsumerGroupDescription_member(act, 0); + + client_id = + rd_kafka_MemberDescription_client_id(member); + TEST_ASSERT(!strcmp(client_id, client_ids[i]), + "Expected client id \"%s\"," + " got \"%s\".", + client_ids[i], client_id); + + if (has_group_instance_id) { + group_instance_id = + rd_kafka_MemberDescription_group_instance_id( + member); + TEST_ASSERT(!strcmp(group_instance_id, + group_instance_ids[i]), + "Expected group instance id \"%s\"," + " got \"%s\".", + group_instance_ids[i], + group_instance_id); + } + + assignment = + rd_kafka_MemberDescription_assignment(member); + TEST_ASSERT(assignment != NULL, + "Expected non-NULL member assignment"); + + partitions = + rd_kafka_MemberAssignment_partitions(assignment); + TEST_ASSERT(partitions != NULL, + "Expected non-NULL member partitions"); + + TEST_SAY( + "Member client.id=\"%s\", " + "group.instance.id=\"%s\", " + "consumer_id=\"%s\", " + "host=\"%s\", assignment:\n", + rd_kafka_MemberDescription_client_id(member), + rd_kafka_MemberDescription_group_instance_id( + member), + rd_kafka_MemberDescription_consumer_id(member), + rd_kafka_MemberDescription_host(member)); + /* This is just to make sure the returned memory + * is valid. */ + test_print_partition_list(partitions); + } else { + TEST_ASSERT(state == RD_KAFKA_CONSUMER_GROUP_STATE_DEAD, + "Expected Dead state, got %s.", + rd_kafka_consumer_group_state_name(state)); + } + TEST_ASSERT(exp_err == act_err, + "expected err=%d for group %s, got %d (%s)", + exp_err, exp->group_id, act_err, + rd_kafka_err2str(act_err)); + } + + rd_kafka_event_destroy(rkev); + + for (i = 0; i < known_groups; i++) { + test_consumer_close(rks[i]); + rd_kafka_destroy(rks[i]); + } + + /* Wait session timeout + 1s. Because using static group membership */ + rd_sleep(6); + + test_DeleteGroups_simple(rk, NULL, (char **)describe_groups, + known_groups, NULL); + + for (i = 0; i < TEST_DESCRIBE_CONSUMER_GROUPS_CNT; i++) { + rd_free(expected[i].group_id); + } + + test_DeleteTopics_simple(rk, q, &topic, 1, NULL); + + rd_free(topic); + + if (options) + rd_kafka_AdminOptions_destroy(options); + + if (!useq) + rd_kafka_queue_destroy(q); + + TEST_LATER_CHECK(); +#undef TEST_DESCRIBE_CONSUMER_GROUPS_CNT + + SUB_TEST_PASS(); +} + +/** + * @brief Test DescribeTopics + */ +static void do_test_DescribeTopics(const char *what, + rd_kafka_t *rk, + rd_kafka_queue_t *rkqu, + int request_timeout) { +#define MY_TOPICS_CNT 3 + rd_kafka_queue_t *q; + char *topics[MY_TOPICS_CNT]; + rd_kafka_AdminOptions_t *options; + rd_kafka_event_t *rkev; + rd_kafka_error_t *error; + rd_kafka_resp_err_t err; + test_timing_t timing; + const rd_kafka_DescribeTopics_result_t *res; + const rd_kafka_TopicDescription_t **result_topics; + size_t result_topics_cnt; + char errstr[128]; + const char *errstr2; + const char *user_test1 = "User:broker"; + rd_kafka_AclBinding_t *acl_bindings[1]; + rd_kafka_AdminOptions_t *admin_options; + const rd_kafka_CreateAcls_result_t *acl_res; + const rd_kafka_acl_result_t **acl_res_acls; + rd_kafka_event_t *rkev_acl_create; + size_t resacl_cnt; + int i; + int initial_acl_cnt, final_acl_cnt; + + SUB_TEST_QUICK("%s DescribeTopics with %s, request_timeout %d", + rd_kafka_name(rk), what, request_timeout); + + q = rkqu ? rkqu : rd_kafka_queue_new(rk); + /* + * Only create one topic, the others will be non-existent. + */ + for (i = 0; i < MY_TOPICS_CNT; i++) { + rd_strdupa(&topics[i], test_mk_topic_name(__FUNCTION__, 1)); + } + test_CreateTopics_simple(rk, NULL, topics, 1, 1, NULL); + + test_wait_topic_exists(rk, topics[0], 10000); + + /* + * Timeout options + */ + options = + rd_kafka_AdminOptions_new(rk, RD_KAFKA_ADMIN_OP_DESCRIBETOPICS); + + err = rd_kafka_AdminOptions_set_request_timeout( + options, request_timeout, errstr, sizeof(errstr)); + TEST_ASSERT(!err, "%s", errstr); + if ((error = + rd_kafka_AdminOptions_set_include_topic_authorized_operations( + options, 1))) { + fprintf(stderr, + "%% Failed to set require authorized operations: %s\n", + rd_kafka_error_string(error)); + rd_kafka_error_destroy(error); + TEST_FAIL("Failed to set include authorized operations\n"); + } + + TIMING_START(&timing, "DescribeTopics"); + TEST_SAY("Call DescribeTopics\n"); + rd_kafka_DescribeTopics(rk, topics, MY_TOPICS_CNT, options, q); + TIMING_ASSERT_LATER(&timing, 0, 50); + + TIMING_START(&timing, "DescribeTopics.queue_poll"); + + /* Poll result queue for DescribeTopics result. + * Print but otherwise ignore other event types + * (typically generic Error events). */ + while (1) { + rkev = rd_kafka_queue_poll(q, tmout_multip(20 * 1000)); + TEST_SAY("DescribeTopics: got %s in %.3fms\n", + rd_kafka_event_name(rkev), + TIMING_DURATION(&timing) / 1000.0f); + if (rkev == NULL) + continue; + 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_DESCRIBETOPICS_RESULT) { + break; + } + rd_kafka_event_destroy(rkev); + } + rd_kafka_AdminOptions_destroy(options); + + /* + * Extract result + */ + res = rd_kafka_event_DescribeTopics_result(rkev); + TEST_ASSERT(res, "Expected DescribeTopics 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); + + result_topics = + rd_kafka_DescribeTopics_result_topics(res, &result_topics_cnt); + + /* + * Check if results have been received for all topics + */ + TEST_ASSERT(result_topics_cnt == MY_TOPICS_CNT, + "Number of topics in result didnt match"); + + /* + * Check if topics[0] succeeded. + */ + TEST_ASSERT(rd_kafka_error_code(rd_kafka_TopicDescription_error( + result_topics[0])) == RD_KAFKA_RESP_ERR_NO_ERROR, + "Expected expected unknown Topic or partition, not %s\n", + rd_kafka_error_string( + rd_kafka_TopicDescription_error(result_topics[0]))); + + /* + * Check whether the topics which are non-existent have + * RD_KAFKA_RESP_ERR_UNKNOWN_TOPIC_OR_PART error. + */ + for (i = 1; i < MY_TOPICS_CNT; i++) + TEST_ASSERT( + rd_kafka_error_code( + rd_kafka_TopicDescription_error(result_topics[i])) == + RD_KAFKA_RESP_ERR_UNKNOWN_TOPIC_OR_PART, + "Expected expected unknown Topic or partition, not %s\n", + rd_kafka_error_string( + rd_kafka_TopicDescription_error(result_topics[i]))); + + initial_acl_cnt = + rd_kafka_TopicDescription_topic_authorized_operations_cnt( + result_topics[0]); + TEST_ASSERT(initial_acl_cnt > 0, "Expected 8 acl operations allowed"); + rd_kafka_event_destroy(rkev); + + acl_bindings[0] = rd_kafka_AclBinding_new( + RD_KAFKA_RESOURCE_TOPIC, topics[0], + RD_KAFKA_RESOURCE_PATTERN_LITERAL, user_test1, "*", + RD_KAFKA_ACL_OPERATION_READ, RD_KAFKA_ACL_PERMISSION_TYPE_ALLOW, + NULL, 0); + admin_options = + rd_kafka_AdminOptions_new(rk, RD_KAFKA_ADMIN_OP_CREATEACLS); + err = rd_kafka_AdminOptions_set_request_timeout(admin_options, 10000, + errstr, sizeof(errstr)); + TEST_ASSERT(!err, "%s", errstr); + rd_kafka_CreateAcls(rk, acl_bindings, 1, admin_options, q); + rkev_acl_create = test_wait_admin_result( + q, RD_KAFKA_EVENT_CREATEACLS_RESULT, 10000 + 1000); + acl_res = rd_kafka_event_CreateAcls_result(rkev_acl_create); + acl_res_acls = rd_kafka_CreateAcls_result_acls(acl_res, &resacl_cnt); + for (i = 0; i < resacl_cnt; i++) { + const rd_kafka_acl_result_t *acl_res_acl = *(acl_res_acls + i); + const rd_kafka_error_t *error_acl = + rd_kafka_acl_result_error(acl_res_acl); + + TEST_ASSERT(!error_acl, + "Expected RD_KAFKA_RESP_ERR_NO_ERROR, not %s", + rd_kafka_error_string(error)); + } + rd_kafka_event_destroy(rkev_acl_create); + rd_kafka_AdminOptions_destroy(admin_options); + rd_kafka_AclBinding_destroy(acl_bindings[0]); + /* + * Timeout options + */ + options = + rd_kafka_AdminOptions_new(rk, RD_KAFKA_ADMIN_OP_DESCRIBETOPICS); + + err = rd_kafka_AdminOptions_set_request_timeout( + options, request_timeout, errstr, sizeof(errstr)); + TEST_ASSERT(!err, "%s", errstr); + if ((error = + rd_kafka_AdminOptions_set_include_topic_authorized_operations( + options, 1))) { + fprintf(stderr, + "%% Failed to set require authorized operations: %s\n", + rd_kafka_error_string(error)); + rd_kafka_error_destroy(error); + TEST_FAIL("Failed to set include authorized operations\n"); + } + + TIMING_START(&timing, "DescribeTopics"); + TEST_SAY("Call DescribeTopics\n"); + rd_kafka_DescribeTopics(rk, topics, MY_TOPICS_CNT, options, q); + TIMING_ASSERT_LATER(&timing, 0, 50); + + TIMING_START(&timing, "DescribeTopics.queue_poll"); + + /* Poll result queue for DescribeTopics result. + * Print but otherwise ignore other event types + * (typically generic Error events). */ + while (1) { + rkev = rd_kafka_queue_poll(q, tmout_multip(20 * 1000)); + TEST_SAY("DescribeTopics: got %s in %.3fms\n", + rd_kafka_event_name(rkev), + TIMING_DURATION(&timing) / 1000.0f); + if (rkev == NULL) + continue; + 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_DESCRIBETOPICS_RESULT) { + break; + } + rd_kafka_event_destroy(rkev); + } + rd_kafka_AdminOptions_destroy(options); + + /* + * Extract result + */ + res = rd_kafka_event_DescribeTopics_result(rkev); + TEST_ASSERT(res, "Expected DescribeTopics 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); + + result_topics = + rd_kafka_DescribeTopics_result_topics(res, &result_topics_cnt); + + /* + * Check if results have been received for all topics + */ + TEST_ASSERT(result_topics_cnt == MY_TOPICS_CNT, + "Number of topics in result didnt match"); + + /* + * Check if topics[0] succeeded. + */ + TEST_ASSERT(rd_kafka_error_code(rd_kafka_TopicDescription_error( + result_topics[0])) == RD_KAFKA_RESP_ERR_NO_ERROR, + "Expected expected unknown Topic or partition, not %s\n", + rd_kafka_error_string( + rd_kafka_TopicDescription_error(result_topics[0]))); + final_acl_cnt = + rd_kafka_TopicDescription_topic_authorized_operations_cnt( + result_topics[0]); + + /* + * Initally count should be 8. After createAcls call with + * only RD_KAFKA_ACL_OPERATION_READ allowed, it should reduce + * to 2 (read and describe). 8>2 should hold. + */ + TEST_ASSERT(initial_acl_cnt > final_acl_cnt, + "Expected the acl operations allowed to have reduced after" + " call to CreateAcls"); + + /* + * Allow RD_KAFKA_ACL_OPERATION_DELETE to allow deletion + * of the created topic as currently it only has read and + * describe. + */ + acl_bindings[0] = rd_kafka_AclBinding_new( + RD_KAFKA_RESOURCE_TOPIC, topics[0], + RD_KAFKA_RESOURCE_PATTERN_LITERAL, user_test1, "*", + RD_KAFKA_ACL_OPERATION_DELETE, RD_KAFKA_ACL_PERMISSION_TYPE_ALLOW, + NULL, 0); + admin_options = + rd_kafka_AdminOptions_new(rk, RD_KAFKA_ADMIN_OP_CREATEACLS); + err = rd_kafka_AdminOptions_set_request_timeout(admin_options, 10000, + errstr, sizeof(errstr)); + TEST_ASSERT(!err, "%s", errstr); + rd_kafka_CreateAcls(rk, acl_bindings, 1, admin_options, q); + rkev_acl_create = test_wait_admin_result( + q, RD_KAFKA_EVENT_CREATEACLS_RESULT, 10000 + 1000); + acl_res = rd_kafka_event_CreateAcls_result(rkev_acl_create); + acl_res_acls = rd_kafka_CreateAcls_result_acls(acl_res, &resacl_cnt); + for (i = 0; i < resacl_cnt; i++) { + const rd_kafka_acl_result_t *acl_res_acl = *(acl_res_acls + i); + const rd_kafka_error_t *error_acl = + rd_kafka_acl_result_error(acl_res_acl); + + TEST_ASSERT(!error_acl, + "Expected RD_KAFKA_RESP_ERR_NO_ERROR, not %s", + rd_kafka_error_string(error)); + } + rd_kafka_event_destroy(rkev_acl_create); + rd_kafka_AdminOptions_destroy(admin_options); + rd_kafka_event_destroy(rkev); + rd_kafka_AclBinding_destroy(acl_bindings[0]); + test_DeleteTopics_simple(rk, q, topics, 1, NULL); + + TEST_LATER_CHECK(); +#undef MY_TOPICS_CNT + + SUB_TEST_PASS(); +} + +/** + * @brief Test DescribeCluster + */ +static void do_test_DescribeCluster(const char *what, + rd_kafka_t *rk, + rd_kafka_queue_t *rkqu, + int request_timeout) { + rd_kafka_queue_t *q; + rd_kafka_AdminOptions_t *options; + rd_kafka_event_t *rkev; + rd_kafka_error_t *error; + rd_kafka_resp_err_t err; + test_timing_t timing; + const rd_kafka_DescribeCluster_result_t *res; + const rd_kafka_ClusterDescription_t *result_cluster; + char errstr[128]; + const char *errstr2; + const char *user_test1 = "User:broker"; + rd_kafka_AclBinding_t *acl_bindings[1]; + rd_kafka_AdminOptions_t *admin_options; + const rd_kafka_CreateAcls_result_t *acl_res; + const rd_kafka_acl_result_t **acl_res_acls; + rd_kafka_event_t *rkev_acl_create; + size_t resacl_cnt; + rd_kafka_AdminOptions_t *admin_options_delete; + rd_kafka_AclBindingFilter_t *acl_bindings_delete; + rd_kafka_event_t *rkev_acl_delete; + const rd_kafka_DeleteAcls_result_t *acl_delete_result; + const rd_kafka_DeleteAcls_result_response_t * + *DeleteAcls_result_responses; + const rd_kafka_DeleteAcls_result_response_t *DeleteAcls_result_response; + size_t DeleteAcls_result_responses_cntp; + int i; + int initial_acl_cnt, final_acl_cnt; + + SUB_TEST_QUICK("%s DescribeCluster with %s, request_timeout %d", + rd_kafka_name(rk), what, request_timeout); + + q = rkqu ? rkqu : rd_kafka_queue_new(rk); + + /* + * Timeout options + */ + options = + rd_kafka_AdminOptions_new(rk, RD_KAFKA_ADMIN_OP_DESCRIBECLUSTER); + + err = rd_kafka_AdminOptions_set_request_timeout( + options, request_timeout, errstr, sizeof(errstr)); + TEST_ASSERT(!err, "%s", errstr); + if ((error = + rd_kafka_AdminOptions_set_include_cluster_authorized_operations( + options, 1))) { + fprintf(stderr, + "%% Failed to set require authorized operations: %s\n", + rd_kafka_error_string(error)); + rd_kafka_error_destroy(error); + TEST_FAIL("Failed to set include authorized operations\n"); + } + + TIMING_START(&timing, "DescribeCluster"); + TEST_SAY("Call DescribeCluster\n"); + rd_kafka_DescribeCluster(rk, options, q); + TIMING_ASSERT_LATER(&timing, 0, 50); + + TIMING_START(&timing, "DescribeCluster.queue_poll"); + + /* Poll result queue for DescribeCluster result. + * Print but otherwise ignore other event types + * (typically generic Error events). */ + while (1) { + rkev = rd_kafka_queue_poll(q, tmout_multip(20 * 1000)); + TEST_SAY("DescribeCluster: got %s in %.3fms\n", + rd_kafka_event_name(rkev), + TIMING_DURATION(&timing) / 1000.0f); + if (rkev == NULL) + continue; + 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_DESCRIBECLUSTER_RESULT) { + break; + } + rd_kafka_event_destroy(rkev); + } + rd_kafka_AdminOptions_destroy(options); + + /* + * Extract result + */ + res = rd_kafka_event_DescribeCluster_result(rkev); + TEST_ASSERT(res, "Expected DescribeCluster 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); + + result_cluster = rd_kafka_DescribeCluster_result_description(res); + + initial_acl_cnt = + rd_kafka_ClusterDescription_cluster_acl_operations_cnt( + result_cluster); + + TEST_ASSERT(initial_acl_cnt > 0, "Expected 7 acl operations allowed"); + TEST_SAY("initial count is: %d\n", initial_acl_cnt); + rd_kafka_event_destroy(rkev); + + acl_bindings[0] = rd_kafka_AclBinding_new( + RD_KAFKA_RESOURCE_BROKER, "kafka-cluster", + RD_KAFKA_RESOURCE_PATTERN_LITERAL, user_test1, "*", + RD_KAFKA_ACL_OPERATION_ALTER, RD_KAFKA_ACL_PERMISSION_TYPE_ALLOW, + NULL, 0); + admin_options = + rd_kafka_AdminOptions_new(rk, RD_KAFKA_ADMIN_OP_CREATEACLS); + err = rd_kafka_AdminOptions_set_request_timeout(admin_options, 10000, + errstr, sizeof(errstr)); + TEST_ASSERT(!err, "%s", errstr); + rd_kafka_CreateAcls(rk, acl_bindings, 1, admin_options, q); + rkev_acl_create = test_wait_admin_result( + q, RD_KAFKA_EVENT_CREATEACLS_RESULT, 10000 + 1000); + acl_res = rd_kafka_event_CreateAcls_result(rkev_acl_create); + acl_res_acls = rd_kafka_CreateAcls_result_acls(acl_res, &resacl_cnt); + for (i = 0; i < resacl_cnt; i++) { + const rd_kafka_acl_result_t *acl_res_acl = *(acl_res_acls + i); + const rd_kafka_error_t *error_acl = + rd_kafka_acl_result_error(acl_res_acl); + + TEST_ASSERT(!error_acl, + "Expected RD_KAFKA_RESP_ERR_NO_ERROR, not %s", + rd_kafka_error_string(error)); + } + rd_kafka_event_destroy(rkev_acl_create); + rd_kafka_AdminOptions_destroy(admin_options); + rd_kafka_AclBinding_destroy(acl_bindings[0]); + /* + * Timeout options + */ + options = + rd_kafka_AdminOptions_new(rk, RD_KAFKA_ADMIN_OP_DESCRIBECLUSTER); + + err = rd_kafka_AdminOptions_set_request_timeout( + options, request_timeout, errstr, sizeof(errstr)); + TEST_ASSERT(!err, "%s", errstr); + if ((error = + rd_kafka_AdminOptions_set_include_cluster_authorized_operations( + options, 1))) { + fprintf(stderr, + "%% Failed to set require authorized operations: %s\n", + rd_kafka_error_string(error)); + rd_kafka_error_destroy(error); + TEST_FAIL("Failed to set include authorized operations\n"); + } + + TIMING_START(&timing, "DescribeCluster"); + TEST_SAY("Call DescribeCluster\n"); + rd_kafka_DescribeCluster(rk, options, q); + TIMING_ASSERT_LATER(&timing, 0, 50); + + TIMING_START(&timing, "DescribeCluster.queue_poll"); + + /* Poll result queue for DescribeCluster result. + * Print but otherwise ignore other event types + * (typically generic Error events). */ + while (1) { + rkev = rd_kafka_queue_poll(q, tmout_multip(20 * 1000)); + TEST_SAY("DescribeCluster: got %s in %.3fms\n", + rd_kafka_event_name(rkev), + TIMING_DURATION(&timing) / 1000.0f); + if (rkev == NULL) + continue; + 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_DESCRIBECLUSTER_RESULT) { + break; + } + rd_kafka_event_destroy(rkev); + } + rd_kafka_AdminOptions_destroy(options); + + /* + * Extract result + */ + res = rd_kafka_event_DescribeCluster_result(rkev); + TEST_ASSERT(res, "Expected DescribeCluster 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); + + result_cluster = rd_kafka_DescribeCluster_result_description(res); + + final_acl_cnt = rd_kafka_ClusterDescription_cluster_acl_operations_cnt( + result_cluster); + + /* + * Initally count should be 7. After createAcls call with + * only RD_KAFKA_ACL_OPERATION_ALTER allowed, it should reduce + * to 2 (DESCRIBE is implicitly derived, and ALTER ). 7>2 should hold. + */ + TEST_SAY("final count is: %d\n", final_acl_cnt); + TEST_ASSERT(initial_acl_cnt > final_acl_cnt, + "Expected the acl operations allowed to have reduced after" + " call to CreateAcls"); + + rd_kafka_event_destroy(rkev); + + /* + * Remove the previously created Acl so that it doesn't affect other + * tests + */ + admin_options_delete = + rd_kafka_AdminOptions_new(rk, RD_KAFKA_ADMIN_OP_DELETEACLS); + rd_kafka_AdminOptions_set_request_timeout(admin_options_delete, 10000, + errstr, sizeof(errstr)); + + acl_bindings_delete = rd_kafka_AclBindingFilter_new( + RD_KAFKA_RESOURCE_BROKER, "kafka-cluster", + RD_KAFKA_RESOURCE_PATTERN_MATCH, user_test1, "*", + RD_KAFKA_ACL_OPERATION_ALTER, RD_KAFKA_ACL_PERMISSION_TYPE_ALLOW, + NULL, 0); + + TIMING_START(&timing, "DeleteAcls"); + rd_kafka_DeleteAcls(rk, &acl_bindings_delete, 1, admin_options_delete, + q); + TIMING_ASSERT_LATER(&timing, 0, 50); + + /* + * Wait for result + */ + rkev_acl_delete = test_wait_admin_result( + q, RD_KAFKA_EVENT_DELETEACLS_RESULT, 10000 + 1000); + + acl_delete_result = rd_kafka_event_DeleteAcls_result(rkev_acl_delete); + + TEST_ASSERT(acl_delete_result, "acl_delete_result should not be NULL"); + + DeleteAcls_result_responses_cntp = 0; + DeleteAcls_result_responses = rd_kafka_DeleteAcls_result_responses( + acl_delete_result, &DeleteAcls_result_responses_cntp); + + TEST_ASSERT(DeleteAcls_result_responses_cntp == 1, + "DeleteAcls_result_responses_cntp should be 1, not %zu\n", + DeleteAcls_result_responses_cntp); + + DeleteAcls_result_response = DeleteAcls_result_responses[0]; + + TEST_CALL_ERROR__(rd_kafka_DeleteAcls_result_response_error( + DeleteAcls_result_response)); + rd_kafka_event_destroy(rkev_acl_delete); + rd_kafka_AclBinding_destroy(acl_bindings_delete); + rd_kafka_AdminOptions_destroy(admin_options_delete); + TEST_LATER_CHECK(); + + if (!rkqu) + rd_kafka_queue_destroy(q); + SUB_TEST_PASS(); +} + +/** + * @brief Test describe groups, creating consumers for a set of groups, + * describing and deleting them at the end. + */ +static void do_test_DescribeConsumerGroups_with_authorized_ops( + const char *what, + rd_kafka_t *rk, + rd_kafka_queue_t *useq, + int request_timeout, + rd_bool_t include_authorized_operations) { + rd_kafka_queue_t *q; + rd_kafka_AdminOptions_t *options = NULL; + rd_kafka_event_t *rkev = NULL; + rd_kafka_resp_err_t err; + rd_kafka_error_t *error; + rd_kafka_event_t *rkev_acl_create; + rd_kafka_AclOperation_t acl_operation; + rd_kafka_ResourcePatternType_t pattern_type_first_topic = + RD_KAFKA_RESOURCE_PATTERN_LITERAL; + char errstr[512]; + const char *errstr2; + const char *user_test1 = "User:broker"; +#define TEST_DESCRIBE_CONSUMER_GROUPS_CNT 4 + int known_groups = TEST_DESCRIBE_CONSUMER_GROUPS_CNT - 1; + int i, j; + const int partitions_cnt = 1; + const int msgs_cnt = 100; + char *topic; + size_t resacl_cnt; + rd_kafka_AclBinding_t *acl_bindings[TEST_DESCRIBE_CONSUMER_GROUPS_CNT]; + rd_kafka_metadata_topic_t exp_mdtopic = {0}; + int64_t testid = test_id_generate(); + test_timing_t timing; + rd_kafka_resp_err_t exp_err = RD_KAFKA_RESP_ERR_NO_ERROR; + const rd_kafka_ConsumerGroupDescription_t **results = NULL; + expected_DescribeConsumerGroups_result_t + expected[TEST_DESCRIBE_CONSUMER_GROUPS_CNT] = RD_ZERO_INIT; + const char *describe_groups[TEST_DESCRIBE_CONSUMER_GROUPS_CNT]; + char group_instance_ids[TEST_DESCRIBE_CONSUMER_GROUPS_CNT][512]; + char client_ids[TEST_DESCRIBE_CONSUMER_GROUPS_CNT][512]; + rd_kafka_t *rks[TEST_DESCRIBE_CONSUMER_GROUPS_CNT]; + const rd_kafka_DescribeConsumerGroups_result_t *res; + rd_bool_t has_group_instance_id = + test_broker_version >= TEST_BRKVER(2, 4, 0, 0); + rd_kafka_AdminOptions_t *admin_options; + const rd_kafka_CreateAcls_result_t *acl_res; + const rd_kafka_acl_result_t **acl_res_acls; + + SUB_TEST_QUICK("%s DescribeConsumerGroups with %s, request_timeout %d", + rd_kafka_name(rk), what, request_timeout); + + q = useq ? useq : rd_kafka_queue_new(rk); + + if (request_timeout != -1) { + options = rd_kafka_AdminOptions_new( + rk, RD_KAFKA_ADMIN_OP_DESCRIBECONSUMERGROUPS); + + err = rd_kafka_AdminOptions_set_request_timeout( + options, request_timeout, errstr, sizeof(errstr)); + TEST_ASSERT(!err, "%s", rd_kafka_err2str(err)); + if ((error = + rd_kafka_AdminOptions_set_include_authorized_operations( + options, 1))) { + fprintf(stderr, + "%% Failed to set require authorized " + "operations: %s\n", + rd_kafka_error_string(error)); + rd_kafka_error_destroy(error); + TEST_FAIL( + "Failed to set include authorized operations\n"); + } + } + + topic = rd_strdup(test_mk_topic_name(__FUNCTION__, 1)); + exp_mdtopic.topic = topic; + + /* Create the topics first. */ + test_CreateTopics_simple(rk, NULL, &topic, 1, partitions_cnt, NULL); + + /* Verify that topics are reported by metadata */ + test_wait_metadata_update(rk, &exp_mdtopic, 1, NULL, 0, 15 * 1000); + + /* Produce 100 msgs */ + test_produce_msgs_easy(topic, testid, 0, msgs_cnt); + + for (i = 0; i < TEST_DESCRIBE_CONSUMER_GROUPS_CNT; i++) { + rd_kafka_conf_t *conf; + char *group_id = rd_strdup(test_mk_topic_name(__FUNCTION__, 1)); + acl_bindings[i] = rd_kafka_AclBinding_new( + RD_KAFKA_RESOURCE_GROUP, group_id, + RD_KAFKA_RESOURCE_PATTERN_LITERAL, user_test1, "*", + RD_KAFKA_ACL_OPERATION_READ, + RD_KAFKA_ACL_PERMISSION_TYPE_ALLOW, NULL, 0); + if (i < known_groups) { + snprintf(group_instance_ids[i], + sizeof(group_instance_ids[i]), + "group_instance_id_%" PRId32, i); + snprintf(client_ids[i], sizeof(client_ids[i]), + "client_id_%" PRId32, i); - TEST_ASSERT(state == - RD_KAFKA_CONSUMER_GROUP_STATE_STABLE, - "Expected Stable state, got %s.", - rd_kafka_consumer_group_state_name(state)); + test_conf_init(&conf, NULL, 0); + test_conf_set(conf, "client.id", client_ids[i]); + test_conf_set(conf, "group.instance.id", + group_instance_ids[i]); + test_conf_set(conf, "session.timeout.ms", "5000"); + test_conf_set(conf, "auto.offset.reset", "earliest"); + rks[i] = + test_create_consumer(group_id, NULL, conf, NULL); + test_consumer_subscribe(rks[i], topic); + /* Consume messages */ + test_consumer_poll("consumer", rks[i], testid, -1, -1, + msgs_cnt, NULL); + } + expected[i].group_id = group_id; + expected[i].err = RD_KAFKA_RESP_ERR_NO_ERROR; + describe_groups[i] = group_id; + } + // acl_bindings[0] = rd_kafka_AclBinding_new( + // RD_KAFKA_RESOURCE_TOPIC, topic, pattern_type_first_topic, + // user_test1, "*", RD_KAFKA_ACL_OPERATION_WRITE, + // RD_KAFKA_ACL_PERMISSION_TYPE_ALLOW, NULL, 0); + // acl_bindings[1] = rd_kafka_AclBinding_new( + // RD_KAFKA_RESOURCE_TOPIC, topic, pattern_type_first_topic, + // user_test1, "*", RD_KAFKA_ACL_OPERATION_READ, + // RD_KAFKA_ACL_PERMISSION_TYPE_ALLOW, NULL, 0); + admin_options = + rd_kafka_AdminOptions_new(rk, RD_KAFKA_ADMIN_OP_CREATEACLS); + err = rd_kafka_AdminOptions_set_request_timeout(admin_options, 10000, + errstr, sizeof(errstr)); + TEST_ASSERT(!err, "%s", errstr); + rd_kafka_CreateAcls(rk, acl_bindings, TEST_DESCRIBE_CONSUMER_GROUPS_CNT, + admin_options, q); + rkev_acl_create = test_wait_admin_result( + q, RD_KAFKA_EVENT_CREATEACLS_RESULT, 10000 + 1000); + acl_res = rd_kafka_event_CreateAcls_result(rkev_acl_create); + acl_res_acls = rd_kafka_CreateAcls_result_acls(acl_res, &resacl_cnt); - TEST_ASSERT( - !rd_kafka_ConsumerGroupDescription_is_simple_consumer_group( - act), - "Expected a normal consumer group, got a simple " - "one."); + TIMING_START(&timing, "DescribeConsumerGroups"); + TEST_SAY("Call DescribeConsumerGroups with authorized operations\n"); + rd_kafka_DescribeConsumerGroups( + rk, describe_groups, TEST_DESCRIBE_CONSUMER_GROUPS_CNT, options, q); + TIMING_ASSERT_LATER(&timing, 0, 50); - member_count = - rd_kafka_ConsumerGroupDescription_member_count(act); - TEST_ASSERT(member_count == 1, - "Expected one member, got %d.", - member_count); + TIMING_START(&timing, "DescribeConsumerGroups.queue_poll"); - member = - rd_kafka_ConsumerGroupDescription_member(act, 0); + /* Poll result queue for DescribeConsumerGroups result. + * Print but otherwise ignore other event types + * (typically generic Error events). */ + while (1) { + rkev = rd_kafka_queue_poll(q, tmout_multip(20 * 1000)); + TEST_SAY("DescribeConsumerGroups: got %s in %.3fms\n", + rd_kafka_event_name(rkev), + TIMING_DURATION(&timing) / 1000.0f); + if (rkev == NULL) + continue; + if (rd_kafka_event_error(rkev)) + TEST_SAY("%s: %s\n", rd_kafka_event_name(rkev), + rd_kafka_event_error_string(rkev)); - client_id = - rd_kafka_MemberDescription_client_id(member); - TEST_ASSERT(!strcmp(client_id, client_ids[i]), - "Expected client id \"%s\"," - " got \"%s\".", - client_ids[i], client_id); + if (rd_kafka_event_type(rkev) == + RD_KAFKA_EVENT_DESCRIBECONSUMERGROUPS_RESULT) { + break; + } - if (has_group_instance_id) { - group_instance_id = - rd_kafka_MemberDescription_group_instance_id( - member); - TEST_ASSERT(!strcmp(group_instance_id, - group_instance_ids[i]), - "Expected group instance id \"%s\"," - " got \"%s\".", - group_instance_ids[i], - group_instance_id); - } + rd_kafka_event_destroy(rkev); + } + /* Convert event to proper result */ + res = rd_kafka_event_DescribeConsumerGroups_result(rkev); + TEST_ASSERT(res, "expected DescribeConsumerGroups_result, got %s", + rd_kafka_event_name(rkev)); - assignment = - rd_kafka_MemberDescription_assignment(member); - TEST_ASSERT(assignment != NULL, - "Expected non-NULL member assignment"); + /* Expecting error */ + err = rd_kafka_event_error(rkev); + errstr2 = rd_kafka_event_error_string(rkev); + TEST_ASSERT(err == exp_err, + "expected DescribeConsumerGroups to return %s, got %s (%s)", + rd_kafka_err2str(exp_err), rd_kafka_err2str(err), + err ? errstr2 : "n/a"); - partitions = - rd_kafka_MemberAssignment_partitions(assignment); - TEST_ASSERT(partitions != NULL, - "Expected non-NULL member partitions"); + TEST_SAY("DescribeConsumerGroups: returned %s (%s)\n", + rd_kafka_err2str(err), err ? errstr2 : "n/a"); - TEST_SAY( - "Member client.id=\"%s\", " - "group.instance.id=\"%s\", " - "consumer_id=\"%s\", " - "host=\"%s\", assignment:\n", - rd_kafka_MemberDescription_client_id(member), - rd_kafka_MemberDescription_group_instance_id( - member), - rd_kafka_MemberDescription_consumer_id(member), - rd_kafka_MemberDescription_host(member)); - /* This is just to make sure the returned memory - * is valid. */ - test_print_partition_list(partitions); - } else { - TEST_ASSERT(state == RD_KAFKA_CONSUMER_GROUP_STATE_DEAD, - "Expected Dead state, got %s.", - rd_kafka_consumer_group_state_name(state)); + size_t cnt = 0; + results = rd_kafka_DescribeConsumerGroups_result_groups(res, &cnt); + + TEST_ASSERT( + TEST_DESCRIBE_CONSUMER_GROUPS_CNT == cnt, + "expected DescribeConsumerGroups_result_groups to return %d items, " + "got %" PRIusz, + TEST_DESCRIBE_CONSUMER_GROUPS_CNT, cnt); + for (i = 0; i < TEST_DESCRIBE_CONSUMER_GROUPS_CNT; i++) { + expected_DescribeConsumerGroups_result_t *exp = &expected[i]; + rd_kafka_resp_err_t exp_err = exp->err; + const rd_kafka_ConsumerGroupDescription_t *act = results[i]; + rd_kafka_resp_err_t act_err = rd_kafka_error_code( + rd_kafka_ConsumerGroupDescription_error(act)); + rd_kafka_consumer_group_state_t state = + rd_kafka_ConsumerGroupDescription_state(act); + if (include_authorized_operations) { + TEST_ASSERT( + rd_kafka_ConsumerGroupDescription_authorized_operations_count( + act) != 0, + "Authorized operations not returned when " + "requested\n"); + TEST_ASSERT( + rd_kafka_ConsumerGroupDescription_authorized_operations_count( + act) < 3, + "Expected only READ and DESCRIBE operations after " + "createAcl(), got DELETE" + "as well\n"); } - TEST_ASSERT(exp_err == act_err, - "expected err=%d for group %s, got %d (%s)", - exp_err, exp->group_id, act_err, - rd_kafka_err2str(act_err)); } rd_kafka_event_destroy(rkev); @@ -2837,7 +3648,28 @@ static void do_test_DescribeConsumerGroups(const char *what, /* Wait session timeout + 1s. Because using static group membership */ rd_sleep(6); - + for (i = 0; i < TEST_DESCRIBE_CONSUMER_GROUPS_CNT; i++) { + acl_bindings[i] = rd_kafka_AclBinding_new( + RD_KAFKA_RESOURCE_GROUP, expected[i].group_id, + RD_KAFKA_RESOURCE_PATTERN_LITERAL, user_test1, "*", + RD_KAFKA_ACL_OPERATION_DELETE, + RD_KAFKA_ACL_PERMISSION_TYPE_ALLOW, NULL, 0); + } + admin_options = + rd_kafka_AdminOptions_new(rk, RD_KAFKA_ADMIN_OP_CREATEACLS); + err = rd_kafka_AdminOptions_set_request_timeout(admin_options, 10000, + errstr, sizeof(errstr)); + TEST_ASSERT(!err, "%s", errstr); + rd_kafka_CreateAcls(rk, acl_bindings, TEST_DESCRIBE_CONSUMER_GROUPS_CNT, + admin_options, q); + rkev_acl_create = test_wait_admin_result( + q, RD_KAFKA_EVENT_CREATEACLS_RESULT, 10000 + 1000); + acl_res = rd_kafka_event_CreateAcls_result(rkev_acl_create); + acl_res_acls = rd_kafka_CreateAcls_result_acls(acl_res, &resacl_cnt); + rd_kafka_AdminOptions_destroy(admin_options); + rd_kafka_event_destroy(rkev_acl_create); + rd_kafka_AclBinding_destroy_array(acl_bindings, + TEST_DESCRIBE_CONSUMER_GROUPS_CNT); test_DeleteGroups_simple(rk, NULL, (char **)describe_groups, known_groups, NULL); @@ -2845,6 +3677,8 @@ static void do_test_DescribeConsumerGroups(const char *what, rd_free(expected[i].group_id); } + test_DeleteTopics_simple(rk, q, &topic, 1, NULL); + rd_free(topic); if (options) @@ -2858,7 +3692,6 @@ static void do_test_DescribeConsumerGroups(const char *what, SUB_TEST_PASS(); } - /** * @brief Test deletion of committed offsets. * @@ -3728,6 +4561,19 @@ static void do_test_apis(rd_kafka_type_t cltype) { do_test_DescribeConsumerGroups("temp queue", rk, NULL, -1); do_test_DescribeConsumerGroups("main queue", rk, mainq, 1500); + /* Describe topics */ + do_test_DescribeTopics("temp queue", rk, NULL, 15000); + do_test_DescribeTopics("main queue", rk, mainq, 15000); + + /* Describe cluster */ + do_test_DescribeCluster("temp queue", rk, NULL, 1500); + do_test_DescribeCluster("main queue", rk, mainq, 1500); + + do_test_DescribeConsumerGroups_with_authorized_ops("temp queue", rk, + NULL, -1, rd_false); + do_test_DescribeConsumerGroups_with_authorized_ops( + "main queue", rk, mainq, 1500, rd_true); + /* Delete groups */ do_test_DeleteGroups("temp queue", rk, NULL, -1); do_test_DeleteGroups("main queue", rk, mainq, 1500);