From dfbacdccc430db8cce5d4535cc90025f588a87ee Mon Sep 17 00:00:00 2001 From: Milind L Date: Wed, 27 Sep 2023 11:42:30 +0530 Subject: [PATCH] [admin] KIP-430: Add authorized operations to describe API commit bee8d6a80b81a8f31c823750e3e504537cd954ac Author: Milind L Date: Wed Sep 27 11:40:35 2023 +0530 Squashed commit of the following: commit 73cf87c20f76a95d598547a70699d16bd8a07089 Author: Emanuele Sabellico Date: Tue Sep 26 14:48:57 2023 +0200 Fix segfault if assignor state is NULL, (#4381) e.g. a destroy happens before the first assignment. Only affects the cooperative-sticky assignor commit 1cb80090dfc75f5a36eae3f4f8844b14885c045e Author: Emanuele Sabellico Date: Thu Sep 21 10:04:07 2023 +0200 Add destroy call when leader epoch is stale (#4429) and partition is in state active commit 49f180a36c247100dc246379ace84833b91f4038 Author: Pranav Rathi <4427674+pranavrth@users.noreply.github.com> Date: Tue Aug 15 23:47:28 2023 +0530 Added new environment variable to tests - TESTS_SKIP_BEFORE (#4317) commit ff1aaf579512e79129b869c266b6add0a3d3a613 Author: Pranav Rathi <4427674+pranavrth@users.noreply.github.com> Date: Mon Aug 14 18:31:48 2023 +0530 Fix ListConsumerGroupOffsets not fetching offsets for all the topics in a group with Apache Kafka version below 2.4.0. (#4346) commit f6f0a62aa270661143cf31284f4adab62ef76257 Author: Milind L Date: Mon Sep 25 17:00:05 2023 +0530 Fix case where broker RPC might not contain authorized operations commit 63e0748f81bccd001df5d757872514cf16641649 Author: Milind L Date: Mon Sep 25 16:14:47 2023 +0530 Fix possible initialization issue commit 7205354438883dd3bebd644adc97c34e684cb8f4 Author: Milind L Date: Mon Sep 25 16:02:37 2023 +0530 Differentiate between 0 and unrequested authorized ops commit adc0c4f93f5d3d25dacde195fc31322066811137 Author: Milind L Date: Mon Sep 25 09:35:36 2023 +0530 Avoid malloc(0) commit 4257dc97a0c10dcba397229efce7b5f3b78775c5 Author: Milind L Date: Fri Sep 22 10:35:33 2023 +0530 Fix CI issues commit 0e05deafc420d118bc3c669be82d4d0535251edc Author: Milind L Date: Fri Sep 22 07:45:53 2023 +0530 Change TopicCollection create method name commit 74d98dc10fa26d1c9ede0397d131043b2dc9e894 Author: Milind L Date: Thu Sep 21 15:02:51 2023 +0530 Fix warning and add rack to cluster nodes commit fd14d242d90e86a8d65885e632b52d8458c7b933 Author: Milind L Date: Fri Sep 15 12:38:56 2023 +0530 Retain topic ordering for DescribeTopics commit d90bb98304ea492409ccc9f956a55809f0891e32 Author: Milind L Date: Thu Sep 14 12:41:10 2023 +0530 Change rack_id to rack in Node public API commit 0233c0f914ee006f19a3b0ce38d4e07df61d844c Author: Milind L Date: Wed Sep 6 15:50:37 2023 +0530 Fix typo in result type commit e6d4d63b3f5b9533910c505052056c76d58af89d Author: Milind L Date: Mon Sep 4 17:26:44 2023 +0530 Add brokers_sorted to internal metadata commit 2c7f5e194e3a0337fab6b5a35e0aa364ea990e43 Author: Milind L Date: Mon Sep 4 16:53:20 2023 +0530 Address comments: AuthorizedOperations_parse uses array commit 9a6ef10f2708fceb187adfdd24ae26743127945b Author: Milind L Date: Mon Sep 4 15:47:41 2023 +0530 Address comments: cosmetic and copyright changes commit beccb0c6b37f913604a0fa273c55a06c6b6f2892 Author: Milind L Date: Mon Sep 4 15:10:54 2023 +0530 Address comments: Cosmetic changes commit 6239372e7a196b1c7d0834b11427690fcfde6c3f Author: Milind L Date: Mon Sep 4 14:59:20 2023 +0530 Address comment and RD_IF_FREE fixes commit bdd8a4bcfe05c1e5dd735d6cbee2582f343068ea Author: Milind L Date: Mon Sep 4 14:55:26 2023 +0530 Fix style and refactor MetadataRequest into op/cb commit 53cd9834fa81ce306057a9748717e39521ef4260 Author: Milind L Date: Thu Aug 31 17:47:54 2023 +0530 Address DescribeCluster/DescribeConsumerGroup changes, fix memleak commit 59ab5bb99d2946a66b1494b2bcfbe60b8ec2615e Author: Milind L Date: Thu Aug 31 13:57:51 2023 +0530 Address comments for DescribeTopics API commit 85f7f91a6c4d9f989748f8fcae5ff290f92f0dd4 Author: Milind L Date: Fri Aug 11 15:08:51 2023 +0530 Get rid of stray conflict marker commit bd7f5a7c73723f7e05a1ae88e910dc1c559c541d Author: Milind L Date: Wed Aug 9 19:27:39 2023 +0530 Style fixes commit ef3f480bbd56dd121ee255f68fed93950a1eb9b6 Author: Milind L Date: Wed Aug 9 19:26:48 2023 +0530 Fix examples commit 0ed725019fa3d52bc354521c872a1f87ff81610f Merge: 61840ab2 07262c48 Author: Milind L Date: Wed Aug 9 14:08:09 2023 +0530 Merge branch 'master' into dev_kip430_cp_review_changes commit 61840ab28689e1aff7bd58544a3113092442ad15 Author: Milind L Date: Wed Aug 9 13:53:33 2023 +0530 Address review comments: Part 2 commit 45780f8b4a2b2561d04f7021f6e85cdfb1f74d1e Author: Milind L Date: Wed Aug 9 10:13:23 2023 +0530 Address review comments: Part 1 commit 0bc97b37c64daca4576474c0d204aef22fdb7cea Author: Milind L Date: Wed Aug 2 11:31:37 2023 +0530 Break into functions, update Supported KIPS commit 6f8aa0df2806d26d83f49bbf5ea6e010ca437ca0 Author: Milind L Date: Wed Aug 2 10:40:36 2023 +0530 Fix 0080 after addressing comments commit 8d8a47324d805b84839ed4a77fcb9472af580c69 Author: Milind L Date: Tue Aug 1 19:59:39 2023 +0530 Address review comments: Refactor MetadataRequest into two parts commit 28615cdb8c48bc9e600b9552e1122e2d9db02820 Author: Milind L Date: Tue Aug 1 18:10:46 2023 +0530 Address review comments - Part 2 commit e01ad4126bc79251965c82222f4e45c37715ad08 Author: Milind L Date: Tue Aug 1 15:09:46 2023 +0530 Address review comments - Part 1 commit 3215a904ab4a9a2cb102ff8a141af8012abb27c0 Author: Milind L Date: Tue Aug 1 09:50:59 2023 +0530 Fixed changelog commit 4a4f50320e8247eaedb64249c55a9ebb9fbe2034 Author: Milind L Date: Mon Jul 31 20:41:57 2023 +0530 Remove fanout from DescribeTopics commit 9787d6c0284e8949cb1b340bca24b91b392b0b8a Merge: 8c2b4d4b c07a3351 Author: Milind L Date: Mon Jul 31 14:47:43 2023 +0530 Merge branch 'master' into dev_kip430_cp_review_changes commit 8c2b4d4bd3938a3318c7589139013c2b4ae9dfae Author: Milind L Date: Mon Jun 26 11:44:24 2023 +0530 Fix cluster_id allocation not on tmpabuf commit 4a4d9dfab701965e744a3d0bc9d306b82401bb09 Author: Milind L Date: Mon Jun 26 11:29:26 2023 +0530 Revert to macro to fix msvc compilation commit 7fff6ebfd23ffb1628281e93693969bfba8f0298 Merge: 3a9a3406 aa50e52a Author: Milind L Date: Mon Jun 26 11:18:39 2023 +0530 Remove warnings and add CHANGELOG commit 3a9a3406b39251d43c20977fe21402a3df41ae56 Author: Milind L Date: Sat Jun 24 15:36:22 2023 +0530 Change API for common stuff commit 6ddbaba378a76ab92120f52d3c568a5ce6b3a956 Merge: f4973071 e52aa3bb Author: Milind L Date: Fri Jun 23 12:24:15 2023 +0530 Merge branch 'master' into dev_kip430_cp commit f4973071a4fd6c13d55b12471b7cdb2cc41fbf90 Author: Milind L Date: Fri Jun 23 12:22:39 2023 +0530 Fix tests and change API to be similar to existing ones commit 3fec50a823b5877e82bafe968814e1e22170ac4b Author: Milind L Date: Thu Jun 15 09:52:33 2023 +0530 Fix style and docs issues commit 99a368d14fce247d1806ac6a83886eb2f7c0c042 Merge: 7c157463 865cf605 Author: Ruchir Jain <122954065+jainruchir@users.noreply.github.com> Date: Wed Jun 14 18:52:29 2023 +0530 Merge branch 'master' into dev_kip430_cp commit 7c157463da420830230f874bf2c7d676121902ea Author: jainruchir Date: Wed Jun 14 18:49:32 2023 +0530 bug fix: request NULL, force_racks in MetadataReq commit 88707a012420d8064c0915b9e9b56db179747b28 Author: jainruchir Date: Wed Jun 14 18:11:47 2023 +0530 bug fixes and include auth ops false default commit 060d69456d28533b2e29304ab79a54d592d97ac4 Merge: 0e16f981 966b63dc Author: jainruchir Date: Wed Jun 14 15:54:10 2023 +0530 Merge 'master' of librdkafka into dev_kip430_cp commit 0e16f981138df2b95e5d092650f89cdd54b72805 Author: jainruchir Date: Tue Apr 11 10:04:40 2023 +0530 style-fix commit b9fea3cd0d22426aa668d3c28d0940f6608558c0 Author: jainruchir Date: Tue Apr 11 09:33:57 2023 +0530 ClusterDescription destroy having double free bug removed commit 7d512bf26ff1ab14911c576e21d1d063e0044ee3 Merge: 830d9b76 2c6dd0a5 Author: Ruchir Jain <122954065+jainruchir@users.noreply.github.com> Date: Tue Apr 11 09:32:39 2023 +0530 Merge branch 'master' into dev_kip430_cp commit 830d9b7635720e7d1f00ef4d28d0a0b4673f63b8 Author: jainruchir Date: Thu Mar 30 14:36:25 2023 +0530 bug fix for adminoptions commit 0bfbf67d9080b999b449a9440f70021efa1d6b4c Author: jainruchir Date: Fri Mar 17 16:05:22 2023 +0530 dev_KIP430 branch merged commit c28152aa5c8be39e2c4edcc8fb96051391ab0521 Author: jainruchir Date: Fri Mar 17 15:05:36 2023 +0530 changes to describeCG tests commit d2c902b0460a7fda78d55f0d0f425f341182b168 Author: jainruchir Date: Tue Feb 21 15:55:50 2023 +0530 added test for include auth ops in 0081 initial commit commit 3a79ed615d389e3950cb970e68bb5bccbc1e52a3 Author: jainruchir Date: Fri Feb 17 14:25:37 2023 +0530 KIP-430 initial changes without checking tests commit d0e21562b734633ee620fe7914ebcb6db989ad94 Author: jainruchir Date: Fri Mar 17 10:01:08 2023 +0530 bug fix when include authorized ops is 0 desc Topics/Cluster commit a542f4dcc8a635def2635164a718ba495672f78b Author: jainruchir Date: Wed Mar 15 10:25:40 2023 +0530 integration tests for describecluster commit 8df6f625d7cfbb288c5a59e637347cdaab520cf4 Author: jainruchir Date: Tue Mar 14 14:04:44 2023 +0530 describe topics and cluster commit 069b8e8f8223f1fded0c46e445b4b10ace2af856 Author: jainruchir Date: Fri Feb 24 14:32:49 2023 +0530 include authorized topic/cluster ops initial draft --- CHANGELOG.md | 9 +- INTRODUCTION.md | 2 +- examples/.gitignore | 2 + examples/CMakeLists.txt | 6 + examples/Makefile | 10 + examples/README.md | 2 + examples/describe_cluster.c | 322 +++++++++++ examples/describe_consumer_groups.c | 233 +++++--- examples/describe_topics.c | 425 ++++++++++++++ src/rdkafka.c | 4 +- src/rdkafka.h | 439 +++++++++++++- src/rdkafka_admin.c | 849 +++++++++++++++++++++++++++- src/rdkafka_admin.h | 84 +++ src/rdkafka_aux.c | 67 ++- src/rdkafka_aux.h | 10 +- src/rdkafka_event.c | 21 + src/rdkafka_event.h | 2 + src/rdkafka_metadata.c | 163 ++++-- src/rdkafka_metadata.h | 31 +- src/rdkafka_op.c | 10 +- src/rdkafka_op.h | 9 +- src/rdkafka_request.c | 249 ++++++-- src/rdkafka_request.h | 30 +- tests/0080-admin_ut.c | 242 +++++++- tests/0081-admin.c | 738 ++++++++++++++++++++++++ tests/test.c | 91 ++- tests/test.h | 7 + 27 files changed, 3802 insertions(+), 255 deletions(-) create mode 100644 examples/describe_cluster.c create mode 100644 examples/describe_topics.c diff --git a/CHANGELOG.md b/CHANGELOG.md index d02d3dd944..93f2302c6f 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,9 +1,14 @@ -# librdkafka v2.2.1 +# librdkafka v2.3.0 -librdkafka v2.2.1 is a maintenance release: +librdkafka v2.3.0 is a feature release: * Added Topic id to the metadata response which is part of the [KIP-516](https://cwiki.apache.org/confluence/display/KAFKA/KIP-516%3A+Topic+Identifiers) * Fixed ListConsumerGroupOffsets not fetching offsets for all the topics in a group with Apache Kafka version below 2.4.0. + * Add support for AdminAPI `DescribeCluster()` and `DescribeTopics()` + (#4240, @jainruchir). + * [KIP-430](https://cwiki.apache.org/confluence/display/KAFKA/KIP-430+-+Return+Authorized+Operations+in+Describe+Responses): + Return authorized operations in Describe Responses. + (#4240, @jainruchir). * Add missing destroy that leads to leaking partition structure memory when there are partition leader changes and a stale leader epoch is received (#4429). * Fix a segmentation fault when closing a consumer using the diff --git a/INTRODUCTION.md b/INTRODUCTION.md index c360719d26..49bd2950d5 100644 --- a/INTRODUCTION.md +++ b/INTRODUCTION.md @@ -1926,7 +1926,7 @@ The [Apache Kafka Implementation Proposals (KIPs)](https://cwiki.apache.org/conf | KIP-412 - AdminAPI: adjust log levels | 2.4.0 | Not supported | | KIP-421 - Variables in client config files | 2.3.0 | Not applicable (librdkafka, et.al, does not provide a config file interface, and shouldn't) | | KIP-429 - Consumer: incremental rebalance protocol | 2.4.0 | Supported | -| KIP-430 - AdminAPI: return authorized operations in Describe.. responses | 2.3.0 | Not supported | +| KIP-430 - AdminAPI: return authorized operations in Describe.. responses | 2.3.0 | Supported | | KIP-436 - Start time in stats | 2.3.0 | Supported | | KIP-447 - Producer scalability for EOS | 2.5.0 | Supported | | KIP-455 - AdminAPI: Replica assignment | 2.4.0 (WIP) | Not supported | diff --git a/examples/.gitignore b/examples/.gitignore index 4df12d6233..f56e06bf2e 100644 --- a/examples/.gitignore +++ b/examples/.gitignore @@ -15,6 +15,8 @@ rdkafka_performance transactions list_consumer_groups describe_consumer_groups +describe_topics +describe_cluster list_consumer_group_offsets alter_consumer_group_offsets incremental_alter_configs diff --git a/examples/CMakeLists.txt b/examples/CMakeLists.txt index b3f974424f..9b1478ea2d 100644 --- a/examples/CMakeLists.txt +++ b/examples/CMakeLists.txt @@ -53,6 +53,12 @@ target_link_libraries(incremental_alter_configs PUBLIC rdkafka) add_executable(user_scram user_scram.c ${win32_sources}) target_link_libraries(user_scram PUBLIC rdkafka) +add_executable(describe_topics describe_topics.c ${win32_sources}) +target_link_libraries(describe_topics PUBLIC rdkafka) + +add_executable(describe_cluster describe_cluster.c ${win32_sources}) +target_link_libraries(describe_cluster PUBLIC rdkafka) + # The targets below has Unix include dirs and do not compile on Windows. if(NOT WIN32) add_executable(rdkafka_example rdkafka_example.c) diff --git a/examples/Makefile b/examples/Makefile index add586de8c..f97e33eacd 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 \ incremental_alter_configs \ @@ -74,6 +76,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 32e93e6056..236291c549 100644 --- a/examples/README.md +++ b/examples/README.md @@ -34,6 +34,8 @@ 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. * [incremental_alter_configs.c](incremental_alter_configs.c) - Incrementally alter resource configurations. diff --git a/examples/describe_cluster.c b/examples/describe_cluster.c new file mode 100644 index 0000000000..c37da17f9f --- /dev/null +++ b/examples/describe_cluster.c @@ -0,0 +1,322 @@ +/* + * librdkafka - Apache Kafka C library + * + * Copyright (c) 2023, Confluent Inc. + * 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 = NULL; /** 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; + + if (queue) + rd_kafka_queue_yield(queue); +} + + +static void usage(const char *reason, ...) { + + fprintf(stderr, + "Describe cluster usage examples\n" + "\n" + "Usage: %s " + "\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; + long n = strtol(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 j; + size_t node_cnt; + size_t authorized_operations_cnt; + const char *cluster_id = + rd_kafka_DescribeCluster_result_cluster_id(clusterdesc); + const rd_kafka_Node_t **nodes = + rd_kafka_DescribeCluster_result_nodes(clusterdesc, &node_cnt); + const rd_kafka_AclOperation_t *authorized_operations = + rd_kafka_DescribeCluster_result_authorized_operations( + clusterdesc, &authorized_operations_cnt); + const rd_kafka_Node_t *controller = + rd_kafka_DescribeCluster_result_controller(clusterdesc); + + printf( + "Cluster id: %s\t Controller id: %d\t authorized operations count " + "allowed: %d\n", + cluster_id, controller ? rd_kafka_Node_id(controller) : -1, + (int)authorized_operations_cnt); + + for (j = 0; j < authorized_operations_cnt; j++) { + printf("\t%s operation is allowed\n", + rd_kafka_AclOperation_name(authorized_operations[j])); + } + + for (j = 0; j < node_cnt; j++) { + const rd_kafka_Node_t *node = nodes[j]; + printf("Node [id: %" PRId32 + ", host: %s" + ", port: %" PRIu16 ", rack: %s]\n", + rd_kafka_Node_id(node), rd_kafka_Node_host(node), + rd_kafka_Node_port(node), rd_kafka_Node_rack(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 = NULL; + char errstr[512]; + rd_kafka_AdminOptions_t *options = NULL; + rd_kafka_event_t *event = NULL; + rd_kafka_error_t *error; + int retval = 0; + const int min_argc = 1; + + if (argc < min_argc) + usage("Wrong number of arguments."); + + 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 producer 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_PRODUCER, conf, errstr, sizeof(errstr)); + if (!rk) + fatal("Failed to create new producer: %s", errstr); + + 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); + retval = 1; + goto exit; + } + if ((error = rd_kafka_AdminOptions_set_include_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); + retval = 1; + goto exit; + } + + /* Call DescribeCluster. */ + 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)); + retval = 1; + } 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: + /* Cleanup. */ + if (event) + rd_kafka_event_destroy(event); + if (options) + rd_kafka_AdminOptions_destroy(options); + if (queue) + rd_kafka_queue_destroy(queue); + if (rk) + 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..daacc1d021 100644 --- a/examples/describe_consumer_groups.c +++ b/examples/describe_consumer_groups.c @@ -2,6 +2,7 @@ * librdkafka - Apache Kafka C library * * Copyright (c) 2022, Magnus Edenhill + * 2023, Confluent Inc. * All rights reserved. * * Redistribution and use in source and binary forms, with or without @@ -50,9 +51,9 @@ const char *argv0; -static rd_kafka_queue_t *queue; /** Admin result queue. - * This is a global so we can - * yield in stop() */ +static rd_kafka_queue_t *queue = NULL; /** Admin result queue. + * This is a global so we can + * yield in stop() */ static volatile sig_atomic_t run = 1; /** @@ -64,7 +65,9 @@ static void stop(int sig) { exit(2); } run = 0; - rd_kafka_queue_yield(queue); + + if (queue) + rd_kafka_queue_yield(queue); } @@ -73,7 +76,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" @@ -145,9 +149,93 @@ print_partition_list(FILE *fp, fprintf(fp, "\n"); } + +/** + * @brief Print group member information. + */ +static void +print_group_member_info(const rd_kafka_MemberDescription_t *member) { + printf( + " Member \"%s\" with client-id %s," + " group instance id: %s, host %s\n", + rd_kafka_MemberDescription_consumer_id(member), + rd_kafka_MemberDescription_client_id(member), + rd_kafka_MemberDescription_group_instance_id(member), + rd_kafka_MemberDescription_host(member)); + const rd_kafka_MemberAssignment_t *assignment = + rd_kafka_MemberDescription_assignment(member); + const rd_kafka_topic_partition_list_t *topic_partitions = + rd_kafka_MemberAssignment_partitions(assignment); + if (!topic_partitions) { + printf(" No assignment\n"); + } else if (topic_partitions->cnt == 0) { + printf(" Empty assignment\n"); + } else { + printf(" Assignment:\n"); + print_partition_list(stdout, topic_partitions, 0, " "); + } +} + + /** * @brief Print group information. */ +static void print_group_info(const rd_kafka_ConsumerGroupDescription_t *group) { + int member_cnt; + size_t j; + size_t authorized_operations_cnt; + const rd_kafka_AclOperation_t *authorized_operations; + const rd_kafka_error_t *error; + char coordinator_desc[512]; + const rd_kafka_Node_t *coordinator = NULL; + const char *group_id = + rd_kafka_ConsumerGroupDescription_group_id(group); + const char *partition_assignor = + rd_kafka_ConsumerGroupDescription_partition_assignor(group); + rd_kafka_consumer_group_state_t state = + rd_kafka_ConsumerGroupDescription_state(group); + authorized_operations = + rd_kafka_ConsumerGroupDescription_authorized_operations( + group, &authorized_operations_cnt); + member_cnt = rd_kafka_ConsumerGroupDescription_member_count(group); + error = rd_kafka_ConsumerGroupDescription_error(group); + coordinator = rd_kafka_ConsumerGroupDescription_coordinator(group); + *coordinator_desc = '\0'; + + if (coordinator != NULL) { + snprintf(coordinator_desc, sizeof(coordinator_desc), + ", coordinator [id: %" PRId32 + ", host: %s" + ", port: %" PRIu16 "]", + rd_kafka_Node_id(coordinator), + rd_kafka_Node_host(coordinator), + rd_kafka_Node_port(coordinator)); + } + printf( + "Group \"%s\", partition assignor \"%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_operations_cnt; j++) { + printf("%s operation is allowed\n", + rd_kafka_AclOperation_name(authorized_operations[j])); + } + if (error) + printf(" error[%" PRId32 "]: %s", rd_kafka_error_code(error), + rd_kafka_error_string(error)); + printf("\n"); + for (j = 0; j < (size_t)member_cnt; j++) { + const rd_kafka_MemberDescription_t *member = + rd_kafka_ConsumerGroupDescription_member(group, j); + print_group_member_info(member); + } +} + + +/** + * @brief Print groups information. + */ static int print_groups_info(const rd_kafka_DescribeConsumerGroups_result_t *grpdesc, int groups_cnt) { @@ -167,94 +255,56 @@ print_groups_info(const rd_kafka_DescribeConsumerGroups_result_t *grpdesc, } for (i = 0; i < result_groups_cnt; i++) { - int j, member_cnt; - const rd_kafka_error_t *error; - const rd_kafka_ConsumerGroupDescription_t *group = - result_groups[i]; - char coordinator_desc[512]; - const rd_kafka_Node_t *coordinator = NULL; - const char *group_id = - rd_kafka_ConsumerGroupDescription_group_id(group); - const char *partition_assignor = - rd_kafka_ConsumerGroupDescription_partition_assignor(group); - rd_kafka_consumer_group_state_t state = - rd_kafka_ConsumerGroupDescription_state(group); - member_cnt = - rd_kafka_ConsumerGroupDescription_member_count(group); - error = rd_kafka_ConsumerGroupDescription_error(group); - coordinator = - rd_kafka_ConsumerGroupDescription_coordinator(group); - *coordinator_desc = '\0'; - - if (coordinator != NULL) { - snprintf(coordinator_desc, sizeof(coordinator_desc), - ", coordinator [id: %" PRId32 - ", host: %s" - ", port: %" PRIu16 "]", - rd_kafka_Node_id(coordinator), - rd_kafka_Node_host(coordinator), - rd_kafka_Node_port(coordinator)); - } - printf( - "Group \"%s\", partition assignor \"%s\", " - "state %s%s, with %" PRId32 " member(s)", - group_id, partition_assignor, - rd_kafka_consumer_group_state_name(state), coordinator_desc, - member_cnt); - if (error) - printf(" error[%" PRId32 "]: %s", - rd_kafka_error_code(error), - rd_kafka_error_string(error)); + print_group_info(result_groups[i]); printf("\n"); - for (j = 0; j < member_cnt; j++) { - const rd_kafka_MemberDescription_t *member = - rd_kafka_ConsumerGroupDescription_member(group, j); - printf( - " Member \"%s\" with client-id %s," - " group instance id: %s, host %s\n", - rd_kafka_MemberDescription_consumer_id(member), - rd_kafka_MemberDescription_client_id(member), - rd_kafka_MemberDescription_group_instance_id( - member), - rd_kafka_MemberDescription_host(member)); - const rd_kafka_MemberAssignment_t *assignment = - rd_kafka_MemberDescription_assignment(member); - const rd_kafka_topic_partition_list_t - *topic_partitions = - rd_kafka_MemberAssignment_partitions( - assignment); - if (!topic_partitions) { - printf(" No assignment\n"); - } else if (topic_partitions->cnt == 0) { - printf(" Empty assignment\n"); - } else { - printf(" Assignment:\n"); - print_partition_list(stdout, topic_partitions, - 0, " "); - } - } } return 0; } +/** + * @brief Parse an integer or fail. + */ +int64_t parse_int(const char *what, const char *str) { + char *end; + long n = strtol(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. */ static void cmd_describe_consumer_groups(rd_kafka_conf_t *conf, int argc, char **argv) { - rd_kafka_t *rk; + rd_kafka_t *rk = NULL; const char **groups = NULL; char errstr[512]; - rd_kafka_AdminOptions_t *options; - rd_kafka_event_t *event = NULL; - int retval = 0; - int groups_cnt = 0; - - if (argc >= 1) { - groups = (const char **)&argv[0]; - groups_cnt = argc; - } + rd_kafka_AdminOptions_t *options = NULL; + rd_kafka_event_t *event = NULL; + rd_kafka_error_t *error; + int retval = 0; + int groups_cnt = 0; + const int min_argc = 2; + int include_authorized_operations; + + if (argc < min_argc) + usage("Wrong number of arguments"); + + include_authorized_operations = + parse_int("include_authorized_operations", argv[0]); + if (include_authorized_operations < 0 || + include_authorized_operations > 1) + usage("include_authorized_operations not a 0-1 int"); + + groups = (const char **)&argv[1]; + groups_cnt = argc - 1; /* * Create consumer instance @@ -280,6 +330,16 @@ cmd_describe_consumer_groups(rd_kafka_conf_t *conf, int argc, char **argv) { if (rd_kafka_AdminOptions_set_request_timeout( options, 10 * 1000 /* 10s */, errstr, sizeof(errstr))) { fprintf(stderr, "%% Failed to set timeout: %s\n", errstr); + retval = 1; + 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); + retval = 1; goto exit; } @@ -301,7 +361,7 @@ cmd_describe_consumer_groups(rd_kafka_conf_t *conf, int argc, char **argv) { fprintf(stderr, "%% DescribeConsumerGroups failed[%" PRId32 "]: %s\n", err, rd_kafka_event_error_string(event)); - goto exit; + retval = 1; } else { /* DescribeConsumerGroups request succeeded, but individual @@ -315,12 +375,15 @@ cmd_describe_consumer_groups(rd_kafka_conf_t *conf, int argc, char **argv) { exit: + /* Cleanup. */ 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); + if (options) + rd_kafka_AdminOptions_destroy(options); + if (queue) + rd_kafka_queue_destroy(queue); + if (rk) + rd_kafka_destroy(rk); exit(retval); } diff --git a/examples/describe_topics.c b/examples/describe_topics.c new file mode 100644 index 0000000000..7008693d82 --- /dev/null +++ b/examples/describe_topics.c @@ -0,0 +1,425 @@ +/* + * librdkafka - Apache Kafka C library + * + * Copyright (c) 2023, Confluent Inc. + * 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 = NULL; /** 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; + if (queue) + rd_kafka_queue_yield(queue); +} + + +static void usage(const char *reason, ...) { + + fprintf(stderr, + "Describe topics 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; + long n = strtol(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 node information. + */ +static void print_node_info(const rd_kafka_Node_t *node) { + if (!node) { + printf("\t\t(null)\n"); + return; + } + + printf("\t\tNode [id: %" PRId32 + ", host: %s" + ", port: %" PRIu16 ", rack %s]\n", + rd_kafka_Node_id(node), rd_kafka_Node_host(node), + rd_kafka_Node_port(node), rd_kafka_Node_rack(node)); +} + +/** + * @brief Print partition information. + */ +static void +print_partition_info(const rd_kafka_TopicPartitionInfo_t *partition) { + size_t k; + int id; + const rd_kafka_Node_t **isr; + size_t isr_cnt; + const rd_kafka_Node_t **replicas; + size_t replica_cnt; + + id = rd_kafka_TopicPartitionInfo_partition(partition); + printf("\tPartition id: %d\n", id); + + printf("\tPartition leader: \n"); + print_node_info(rd_kafka_TopicPartitionInfo_leader(partition)); + + isr = rd_kafka_TopicPartitionInfo_isr(partition, &isr_cnt); + if (isr_cnt) { + printf( + "\tThe in-sync replica count is: %d, they " + "are: \n", + (int)isr_cnt); + for (k = 0; k < isr_cnt; k++) + print_node_info(isr[k]); + } else + printf("\tThe in-sync replica count is 0\n"); + + replicas = rd_kafka_TopicPartitionInfo_isr(partition, &replica_cnt); + if (replica_cnt) { + printf( + "\tThe replica count is: %d, they " + "are: \n", + (int)replica_cnt); + for (k = 0; k < replica_cnt; k++) + print_node_info(replicas[k]); + } else + printf("\tThe replica count is 0\n"); +} + +/** + * @brief Print topic information. + */ +static void print_topic_info(const rd_kafka_TopicDescription_t *topic) { + size_t j; + const rd_kafka_error_t *error; + const char *topic_name = rd_kafka_TopicDescription_name(topic); + error = rd_kafka_TopicDescription_error(topic); + const rd_kafka_AclOperation_t *authorized_operations; + size_t authorized_operations_cnt; + const rd_kafka_TopicPartitionInfo_t **partitions; + size_t partition_cnt; + + 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)); + return; + } + + authorized_operations = rd_kafka_TopicDescription_authorized_operations( + topic, &authorized_operations_cnt); + + printf( + "Topic: %s succeeded, has %d topic authorized operations " + "allowed, they are:\n", + topic_name, (int)authorized_operations_cnt); + + for (j = 0; j < authorized_operations_cnt; j++) + printf("\t%s operation is allowed\n", + rd_kafka_AclOperation_name(authorized_operations[j])); + + + partitions = + rd_kafka_TopicDescription_partitions(topic, &partition_cnt); + + printf("partition count is: %d\n", (int)partition_cnt); + for (j = 0; j < partition_cnt; j++) { + print_partition_info(partitions[j]); + printf("\n"); + } +} + + +/** + * @brief Print topics information. + */ +static int print_topics_info(const rd_kafka_DescribeTopics_result_t *topicdesc, + int topic_cnt) { + size_t i; + 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++) { + print_topic_info(result_topics[i]); + 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 = NULL; + const char **topic_names = NULL; + rd_kafka_TopicCollection_t *topics = NULL; + char errstr[512]; + rd_kafka_AdminOptions_t *options = NULL; + rd_kafka_event_t *event = NULL; + rd_kafka_error_t *error; + int retval = 0; + int topics_cnt = 0; + const int min_argc = 2; + int include_topic_authorized_operations; + + if (argc < min_argc) + usage("Wrong number of arguments"); + + 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"); + + topic_names = (const char **)&argv[1]; + topics_cnt = argc - 1; + topics = + rd_kafka_TopicCollection_of_topic_names(topic_names, topics_cnt); + + /* + * Create producer 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_PRODUCER, conf, errstr, sizeof(errstr)); + if (!rk) + fatal("Failed to create new producer: %s", errstr); + + 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_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); + retval = 1; + goto exit; + } + + /* Call DescribeTopics */ + rd_kafka_DescribeTopics(rk, topics, 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)); + retval = 1; + 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: + /* Cleanup. */ + if (topics) + rd_kafka_TopicCollection_destroy(topics); + if (event) + rd_kafka_event_destroy(event); + if (options) + rd_kafka_AdminOptions_destroy(options); + if (queue) + rd_kafka_queue_destroy(queue); + if (rk) + 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 a353f7b46f..f460334cd6 100644 --- a/src/rdkafka.c +++ b/src/rdkafka.c @@ -4796,7 +4796,9 @@ 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 /* don't include authorized operations */, + 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 b24a9917f9..2c8bb93bab 100644 --- a/src/rdkafka.h +++ b/src/rdkafka.h @@ -4973,6 +4973,16 @@ const char *rd_kafka_Node_host(const rd_kafka_Node_t *node); RD_EXPORT uint16_t rd_kafka_Node_port(const rd_kafka_Node_t *node); +/** + * @brief Get the rack of \p node. + * + * @param node The Node instance + * + * @return The node rack id. May be NULL. + */ +RD_EXPORT +const char *rd_kafka_Node_rack(const rd_kafka_Node_t *node); + /**@}*/ @@ -5370,6 +5380,10 @@ typedef int rd_kafka_event_type_t; #define RD_KAFKA_EVENT_DESCRIBEUSERSCRAMCREDENTIALS_RESULT 0x40000 /** AlterUserScramCredentials_result_t */ #define RD_KAFKA_EVENT_ALTERUSERSCRAMCREDENTIALS_RESULT 0x80000 +/** DescribeTopics_result_t */ +#define RD_KAFKA_EVENT_DESCRIBETOPICS_RESULT 0x100000 +/** DescribeCluster_result_t */ +#define RD_KAFKA_EVENT_DESCRIBECLUSTER_RESULT 0x200000 /** * @returns the event type for the given event. @@ -5525,6 +5539,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); @@ -5638,6 +5654,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; /*! DescribeUserScramCredentials result type */ typedef rd_kafka_event_t rd_kafka_DescribeUserScramCredentials_result_t; /*! AlterUserScramCredentials result type */ @@ -5755,6 +5775,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. * @@ -6778,7 +6827,9 @@ typedef enum rd_kafka_admin_op_t { RD_KAFKA_ADMIN_OP_DESCRIBEUSERSCRAMCREDENTIALS, /** AlterUserScramCredentials */ RD_KAFKA_ADMIN_OP_ALTERUSERSCRAMCREDENTIALS, - RD_KAFKA_ADMIN_OP__CNT /**< Number of ops defined */ + RD_KAFKA_ADMIN_OP_DESCRIBETOPICS, /**< DescribeTopics */ + RD_KAFKA_ADMIN_OP_DESCRIBECLUSTER, /**< DescribeCluster */ + RD_KAFKA_ADMIN_OP__CNT /**< Number of ops defined */ } rd_kafka_admin_op_t; /** @@ -6957,6 +7008,25 @@ 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 for the given + * resource in the DescribeConsumerGroups, DescribeTopics, or + * DescribeCluster calls. + * + * @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, DescribeTopics, + * DescribeCluster. + */ +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. * @@ -6983,8 +7053,38 @@ RD_EXPORT void rd_kafka_AdminOptions_set_opaque(rd_kafka_AdminOptions_t *options, void *ev_opaque); + + +/** + * @enum rd_kafka_AclOperation_t + * @brief Apache Kafka ACL operation types. Common type for multiple Admin API + * functions. + */ +typedef enum rd_kafka_AclOperation_t { + RD_KAFKA_ACL_OPERATION_UNKNOWN = 0, /**< Unknown */ + RD_KAFKA_ACL_OPERATION_ANY = + 1, /**< In a filter, matches any AclOperation */ + RD_KAFKA_ACL_OPERATION_ALL = 2, /**< ALL operation */ + RD_KAFKA_ACL_OPERATION_READ = 3, /**< READ operation */ + RD_KAFKA_ACL_OPERATION_WRITE = 4, /**< WRITE operation */ + RD_KAFKA_ACL_OPERATION_CREATE = 5, /**< CREATE operation */ + RD_KAFKA_ACL_OPERATION_DELETE = 6, /**< DELETE operation */ + RD_KAFKA_ACL_OPERATION_ALTER = 7, /**< ALTER operation */ + RD_KAFKA_ACL_OPERATION_DESCRIBE = 8, /**< DESCRIBE operation */ + RD_KAFKA_ACL_OPERATION_CLUSTER_ACTION = + 9, /**< CLUSTER_ACTION operation */ + RD_KAFKA_ACL_OPERATION_DESCRIBE_CONFIGS = + 10, /**< DESCRIBE_CONFIGS operation */ + RD_KAFKA_ACL_OPERATION_ALTER_CONFIGS = + 11, /**< ALTER_CONFIGS operation */ + RD_KAFKA_ACL_OPERATION_IDEMPOTENT_WRITE = + 12, /**< IDEMPOTENT_WRITE operation */ + RD_KAFKA_ACL_OPERATION__CNT +} rd_kafka_AclOperation_t; + /**@}*/ + /** * @name Admin API - Topics * @brief Topic related operations. @@ -7889,6 +7989,298 @@ rd_kafka_DeleteRecords_result_offsets( /**@}*/ +/** + * @name Admin API - DescribeTopics + * @{ + */ + +/** + * @brief Represents a collection of topics, to be passed to DescribeTopics. + * + */ +typedef struct rd_kafka_TopicCollection_s rd_kafka_TopicCollection_t; + +/** + * @brief TopicPartition represents a partition in the DescribeTopics result. + * + */ +typedef struct rd_kafka_TopicPartitionInfo_s rd_kafka_TopicPartitionInfo_t; + +/** + * @brief DescribeTopics result type. + * + */ +typedef struct rd_kafka_TopicDescription_s rd_kafka_TopicDescription_t; + +/** + * @brief Creates a new TopicCollection for passing to rd_kafka_DescribeTopics. + * + * @param topics A list of topics. + * @param topics_cnt Count of topics. + * + * @return a newly allocated TopicCollection object. Must be freed using + * rd_kafka_TopicCollection_destroy when done. + */ +RD_EXPORT +rd_kafka_TopicCollection_t * +rd_kafka_TopicCollection_of_topic_names(const char **topics, size_t topics_cnt); + +/** + * @brief Destroy and free a TopicCollection object created with + * rd_kafka_TopicCollection_new_* methods. + */ +RD_EXPORT void +rd_kafka_TopicCollection_destroy(rd_kafka_TopicCollection_t *topics); + +/** + * @brief Describe topics as specified by the \p topics + * array of size \p topics_cnt elements. + * + * @param rk Client instance. + * @param topics Collection of topics to describe. + * @param options Optional admin options, or NULL for defaults. + * Valid options: + * - include_authorized_operations + * @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 rd_kafka_TopicCollection_t *topics, + const rd_kafka_AdminOptions_t *options, + rd_kafka_queue_t *rkqu); + +/** + * @brief Get an array of topic results from a DescribeTopics result. + * + * @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 Gets an array of partitions for the \p topicdesc topic. + * + * @param topicdesc The topic description. + * @param cntp is updated to the number of partitions in the array. + * + * @return An array of TopicPartitionInfos. + * @remark The lifetime of the returned memory is the same + * as the lifetime of the \p topicdesc object. + */ +RD_EXPORT +const rd_kafka_TopicPartitionInfo_t **rd_kafka_TopicDescription_partitions( + const rd_kafka_TopicDescription_t *topicdesc, + size_t *cntp); + + +/** + * @brief Gets the partition id for \p partition. + * + * @param partition The partition info. + * + * @return The partition id. + */ +RD_EXPORT +const int rd_kafka_TopicPartitionInfo_partition( + const rd_kafka_TopicPartitionInfo_t *partition); + + +/** + * @brief Gets the partition leader for \p partition. + * + * @param partition The partition info. + * + * @return The partition leader. + * + * @remark The lifetime of the returned memory is the same + * as the lifetime of the \p partition object. + */ +RD_EXPORT +const rd_kafka_Node_t *rd_kafka_TopicPartitionInfo_leader( + const rd_kafka_TopicPartitionInfo_t *partition); + +/** + * @brief Gets the partition in-sync replicas for \p partition. + * + * @param partition The partition info. + * @param cntp is updated with in-sync replicas count. + * + * @return The in-sync replica nodes. + * + * @remark The lifetime of the returned memory is the same + * as the lifetime of the \p partition object. + */ +RD_EXPORT +const rd_kafka_Node_t ** +rd_kafka_TopicPartitionInfo_isr(const rd_kafka_TopicPartitionInfo_t *partition, + size_t *cntp); + +/** + * @brief Gets the partition replicas for \p partition. + * + * @param partition The partition info. + * @param cntp is updated with partition replicas count. + * + * @return The partition replicas nodes. + * + * @remark The lifetime of the returned memory is the same + * as the lifetime of the \p partition object. + */ +RD_EXPORT +const rd_kafka_Node_t **rd_kafka_TopicPartitionInfo_replicas( + const rd_kafka_TopicPartitionInfo_t *partition, + size_t *cntp); + +/** + * @brief Gets the topic authorized ACL operations for the \p topicdesc topic. + * + * @param topicdesc The topic description. + * @param cntp is updated with authorized ACL operations count. + * + * @return The topic authorized operations. Is NULL if operations were not + * requested. + * + * @remark The lifetime of the returned memory is the same + * as the lifetime of the \p topicdesc object. + */ +RD_EXPORT +const rd_kafka_AclOperation_t *rd_kafka_TopicDescription_authorized_operations( + const rd_kafka_TopicDescription_t *topicdesc, + size_t *cntp); + +/** + * @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_name(const rd_kafka_TopicDescription_t *topicdesc); + +/** + * @brief Gets if the \p topicdesc topic is internal. + * + * @param topicdesc The topic description. + * + * @return 1 if the topic is internal to Kafka, 0 otherwise. + */ +RD_EXPORT +int rd_kafka_TopicDescription_is_internal( + 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); + + +/**@}*/ + +/** + * @name Admin API - DescribeCluster + * @{ + */ + +/** + * @brief Describes the cluster. + * + * @param rk Client instance. + * @param options Optional admin options, or NULL for defaults. + * Valid options: + * - include_authorized_operations + * @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 Gets the broker nodes for the \p result cluster. + * + * @param result The result of DescribeCluster. + * @param cntp is updated with the count of broker nodes. + * + * @return An array of broker nodes. + * @remark The lifetime of the returned memory is the same + * as the lifetime of the \p result object. + */ +RD_EXPORT +const rd_kafka_Node_t **rd_kafka_DescribeCluster_result_nodes( + const rd_kafka_DescribeCluster_result_t *result, + size_t *cntp); + +/** + * @brief Gets the authorized ACL operations for the \p result cluster. + * + * @param result The result of DescribeCluster. + * @param cntp is updated with authorized ACL operations count. + * + * @return The cluster authorized operations. Is NULL if operations were not + * requested. + * @remark The lifetime of the returned memory is the same + * as the lifetime of the \p result object. + */ +RD_EXPORT +const rd_kafka_AclOperation_t * +rd_kafka_DescribeCluster_result_authorized_operations( + const rd_kafka_DescribeCluster_result_t *result, + size_t *cntp); + +/** + * @brief Gets the current controller for the \p result cluster. + * + * @param result The result of DescribeCluster. + * + * @return The cluster current controller. + */ +RD_EXPORT +const rd_kafka_Node_t *rd_kafka_DescribeCluster_result_controller( + const rd_kafka_DescribeCluster_result_t *result); + +/** + * @brief Gets the cluster id for the \p result cluster. + * + * @param result The result of DescribeCluster. + * + * @return The cluster id. + * @remark The lifetime of the returned memory is the same + * as the lifetime of the \p result object. + */ +RD_EXPORT +const char *rd_kafka_DescribeCluster_result_cluster_id( + const rd_kafka_DescribeCluster_result_t *result); + +/**@}*/ + + /** * @name Admin API - ListConsumerGroups * @{ @@ -8027,6 +8419,8 @@ typedef struct rd_kafka_MemberAssignment_s rd_kafka_MemberAssignment_t; * @param groups Array of groups to describe. * @param groups_cnt Number of elements in \p groups array. * @param options Optional admin options, or NULL for defaults. + * Valid options: + * - include_authorized_operations * @param rkqu Queue to emit result on. * * @remark The result event type emitted on the supplied queue is of type @@ -8111,6 +8505,23 @@ RD_EXPORT const char *rd_kafka_ConsumerGroupDescription_partition_assignor( const rd_kafka_ConsumerGroupDescription_t *grpdesc); +/** + * @brief Gets the authorized ACL operations for the \p grpdesc group. + * + * @param grpdesc The group description. + * @param cntp is updated with authorized ACL operations count. + * + * @return The group authorized operations. Is NULL if operations were not + * requested. + * + * @remark The lifetime of the returned memory is the same + * as the lifetime of the \p grpdesc object. + */ +RD_EXPORT +const rd_kafka_AclOperation_t * +rd_kafka_ConsumerGroupDescription_authorized_operations( + const rd_kafka_ConsumerGroupDescription_t *grpdesc, + size_t *cntp); /** * @brief Gets state for the \p grpdesc group. @@ -8878,32 +9289,6 @@ RD_EXPORT const rd_kafka_error_t * rd_kafka_acl_result_error(const rd_kafka_acl_result_t *aclres); -/** - * @enum rd_kafka_AclOperation_t - * @brief Apache Kafka ACL operation types. - */ -typedef enum rd_kafka_AclOperation_t { - RD_KAFKA_ACL_OPERATION_UNKNOWN = 0, /**< Unknown */ - RD_KAFKA_ACL_OPERATION_ANY = - 1, /**< In a filter, matches any AclOperation */ - RD_KAFKA_ACL_OPERATION_ALL = 2, /**< ALL operation */ - RD_KAFKA_ACL_OPERATION_READ = 3, /**< READ operation */ - RD_KAFKA_ACL_OPERATION_WRITE = 4, /**< WRITE operation */ - RD_KAFKA_ACL_OPERATION_CREATE = 5, /**< CREATE operation */ - RD_KAFKA_ACL_OPERATION_DELETE = 6, /**< DELETE operation */ - RD_KAFKA_ACL_OPERATION_ALTER = 7, /**< ALTER operation */ - RD_KAFKA_ACL_OPERATION_DESCRIBE = 8, /**< DESCRIBE operation */ - RD_KAFKA_ACL_OPERATION_CLUSTER_ACTION = - 9, /**< CLUSTER_ACTION operation */ - RD_KAFKA_ACL_OPERATION_DESCRIBE_CONFIGS = - 10, /**< DESCRIBE_CONFIGS operation */ - RD_KAFKA_ACL_OPERATION_ALTER_CONFIGS = - 11, /**< ALTER_CONFIGS operation */ - RD_KAFKA_ACL_OPERATION_IDEMPOTENT_WRITE = - 12, /**< IDEMPOTENT_WRITE operation */ - RD_KAFKA_ACL_OPERATION__CNT -} rd_kafka_AclOperation_t; - /** * @returns a string representation of the \p acl_operation */ diff --git a/src/rdkafka_admin.c b/src/rdkafka_admin.c index 8628dd14c3..3107332a7f 100644 --- a/src/rdkafka_admin.c +++ b/src/rdkafka_admin.c @@ -1482,6 +1482,34 @@ static rd_kafka_op_t *rd_kafka_admin_request_op_target_all_new( return rko; } + +/** + * @brief Construct MetadataRequest for use with AdminAPI (does not send). + * Common for DescribeTopics and DescribeCluster. + * + * @sa rd_kafka_MetadataRequest_resp_cb. + */ +static rd_kafka_resp_err_t +rd_kafka_admin_MetadataRequest(rd_kafka_broker_t *rkb, + const rd_list_t *topics, + const char *reason, + rd_bool_t include_cluster_authorized_operations, + rd_bool_t include_topic_authorized_operations, + rd_bool_t force_racks, + rd_kafka_resp_cb_t *resp_cb, + rd_kafka_replyq_t replyq, + void *opaque) { + return rd_kafka_MetadataRequest_resp_cb( + rkb, topics, reason, + rd_false /* No admin operation requires topic creation. */, + include_cluster_authorized_operations, + include_topic_authorized_operations, + rd_false /* No admin operation should update cgrp. */, force_racks, + resp_cb, replyq, + rd_true /* Admin operation metadata requests are always forced. */, + opaque); +} + /**@}*/ @@ -1545,6 +1573,16 @@ rd_kafka_error_t *rd_kafka_AdminOptions_set_require_stable_offsets( 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_match_consumer_group_states( rd_kafka_AdminOptions_t *options, const rd_kafka_consumer_group_state_t *consumer_group_states, @@ -1639,6 +1677,18 @@ static void rd_kafka_AdminOptions_init(rd_kafka_t *rk, 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_DESCRIBECONSUMERGROUPS || + options->for_api == RD_KAFKA_ADMIN_OP_DESCRIBECLUSTER || + options->for_api == RD_KAFKA_ADMIN_OP_DESCRIBETOPICS) + 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) rd_kafka_confval_init_ptr(&options->match_consumer_group_states, @@ -7049,6 +7099,89 @@ const rd_kafka_error_t **rd_kafka_ListConsumerGroups_result_errors( * */ +/** + * @brief Parse authorized_operations returned in + * - DescribeConsumerGroups + * - DescribeTopics + * - DescribeCluster + * + * @param authorized_operations returned by RPC, containing operations encoded + * per-bit. + * @param cntp is set to the count of the operations, or -1 if the operations + * were not requested. + * @returns rd_kafka_AclOperation_t *. May be NULL. + */ +static rd_kafka_AclOperation_t * +rd_kafka_AuthorizedOperations_parse(int32_t authorized_operations, int *cntp) { + rd_kafka_AclOperation_t i; + int j = 0; + int count = 0; + rd_kafka_AclOperation_t *operations = NULL; + + /* In case of authorized_operations not requested, return NULL. */ + if (authorized_operations < 0) { + *cntp = -1; + return NULL; + } + + /* Count number of bits set. ALL, ANY and UNKNOWN bits are skipped as + * they are always unset as per KIP-430. */ + for (i = RD_KAFKA_ACL_OPERATION_READ; i < RD_KAFKA_ACL_OPERATION__CNT; + i++) + count += ((authorized_operations >> i) & 1); + *cntp = count; + + /* In case no operations exist, allocate 1 byte so that the returned + * pointer is non-NULL. A NULL pointer implies that authorized + * operations were not requested. */ + if (count == 0) + return rd_malloc(1); + + operations = rd_malloc(sizeof(rd_kafka_AclOperation_t) * count); + j = 0; + for (i = RD_KAFKA_ACL_OPERATION_READ; i < RD_KAFKA_ACL_OPERATION__CNT; + i++) { + if ((authorized_operations >> i) & 1) { + operations[j] = i; + j++; + } + } + + return operations; +} + +/** + * @brief Copy a list of rd_kafka_AclOperation_t. + * + * @param src Array of rd_kafka_AclOperation_t to copy from. May be NULL if + * authorized operations were not requested. + * @param authorized_operations_cnt Count of \p src. May be -1 if authorized + * operations were not requested. + * @returns Copy of \p src. May be NULL. + */ +static rd_kafka_AclOperation_t * +rd_kafka_AuthorizedOperations_copy(const rd_kafka_AclOperation_t *src, + int authorized_operations_cnt) { + size_t copy_bytes = 0; + rd_kafka_AclOperation_t *dst = NULL; + + if (authorized_operations_cnt == -1 || src == NULL) + return NULL; + + /* Allocate and copy 1 byte so that the returned pointer + * is non-NULL. A NULL pointer implies that authorized operations were + * not requested. */ + if (authorized_operations_cnt == 0) + copy_bytes = 1; + else + copy_bytes = + sizeof(rd_kafka_AclOperation_t) * authorized_operations_cnt; + + dst = rd_malloc(copy_bytes); + memcpy(dst, src, copy_bytes); + return dst; +} + /** * @brief Create a new MemberDescription object. This object is used for * creating a ConsumerGroupDescription. @@ -7167,6 +7300,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 (optional) authorized operations. * @param state Group state. * @param coordinator (optional) Group coordinator. * @param error (optional) Error received for this group. @@ -7174,13 +7308,16 @@ const rd_kafka_topic_partition_list_t *rd_kafka_MemberAssignment_partitions( * Use rd_kafka_ConsumerGroupDescription_destroy() to free when done. */ static rd_kafka_ConsumerGroupDescription_t * -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_kafka_consumer_group_state_t state, - const rd_kafka_Node_t *coordinator, - rd_kafka_error_t *error) { +rd_kafka_ConsumerGroupDescription_new( + const char *group_id, + rd_bool_t is_simple_consumer_group, + const rd_list_t *members, + const char *partition_assignor, + const rd_kafka_AclOperation_t *authorized_operations, + int authorized_operations_cnt, + rd_kafka_consumer_group_state_t state, + const rd_kafka_Node_t *coordinator, + rd_kafka_error_t *error) { rd_kafka_ConsumerGroupDescription_t *grpdesc; grpdesc = rd_calloc(1, sizeof(*grpdesc)); grpdesc->group_id = rd_strdup(group_id); @@ -7196,6 +7333,11 @@ rd_kafka_ConsumerGroupDescription_new(const char *group_id, grpdesc->partition_assignor = !partition_assignor ? (char *)partition_assignor : rd_strdup(partition_assignor); + + grpdesc->authorized_operations_cnt = authorized_operations_cnt; + grpdesc->authorized_operations = rd_kafka_AuthorizedOperations_copy( + authorized_operations, authorized_operations_cnt); + grpdesc->state = state; if (coordinator != NULL) grpdesc->coordinator = rd_kafka_Node_copy(coordinator); @@ -7210,14 +7352,15 @@ rd_kafka_ConsumerGroupDescription_new(const char *group_id, * @brief New instance of ConsumerGroupDescription from an error. * * @param group_id The group id. - * @param error The error. + * @param error Error received for this group. * @return A new allocated ConsumerGroupDescription with the passed error. + * Use rd_kafka_ConsumerGroupDescription_destroy() to free when done. */ 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, 0, RD_KAFKA_CONSUMER_GROUP_STATE_UNKNOWN, NULL, error); } @@ -7232,8 +7375,9 @@ 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->coordinator, grpdesc->error); + &grpdesc->members, grpdesc->partition_assignor, + grpdesc->authorized_operations, grpdesc->authorized_operations_cnt, + grpdesc->state, grpdesc->coordinator, grpdesc->error); } /** @@ -7256,6 +7400,8 @@ static void rd_kafka_ConsumerGroupDescription_destroy( rd_kafka_error_destroy(grpdesc->error); if (grpdesc->coordinator) rd_kafka_Node_destroy(grpdesc->coordinator); + if (grpdesc->authorized_operations_cnt) + rd_free(grpdesc->authorized_operations); rd_free(grpdesc); } @@ -7285,6 +7431,13 @@ const char *rd_kafka_ConsumerGroupDescription_partition_assignor( return grpdesc->partition_assignor; } +const rd_kafka_AclOperation_t * +rd_kafka_ConsumerGroupDescription_authorized_operations( + const rd_kafka_ConsumerGroupDescription_t *grpdesc, + size_t *cntp) { + *cntp = RD_MAX(grpdesc->authorized_operations_cnt, 0); + return grpdesc->authorized_operations; +} rd_kafka_consumer_group_state_t rd_kafka_ConsumerGroupDescription_state( const rd_kafka_ConsumerGroupDescription_t *grpdesc) { @@ -7382,7 +7535,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); @@ -7392,7 +7545,12 @@ 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); @@ -7427,6 +7585,8 @@ rd_kafka_DescribeConsumerGroupsResponse_parse(rd_kafka_op_t *rko_req, rd_kafka_error_t *error = NULL; char *group_id = NULL, *group_state = NULL, *proto_type = NULL, *proto = NULL, *host = NULL; + rd_kafka_AclOperation_t *operations = NULL; + int operation_cnt = -1; api_version = rd_kafka_buf_ApiVersion(reply); if (api_version >= 1) { @@ -7448,6 +7608,7 @@ 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 = -1; rd_kafkap_str_t GroupId, GroupState, ProtocolType, ProtocolData; rd_bool_t is_simple_consumer_group, is_consumer_protocol_type; int32_t member_cnt; @@ -7557,33 +7718,40 @@ 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); + /* Authorized_operations is INT_MIN + * in case of not being requested, and the list is NULL + * that case. */ + operations = rd_kafka_AuthorizedOperations_parse( + authorized_operations, &operation_cnt); } if (error == NULL) { grpdesc = rd_kafka_ConsumerGroupDescription_new( group_id, is_simple_consumer_group, &members, proto, + operations, operation_cnt, rd_kafka_consumer_group_state_code(group_state), node, error); - } else { + } else grpdesc = rd_kafka_ConsumerGroupDescription_new_error( group_id, error); - } + rd_list_add(&rko_result->rko_u.admin_result.results, grpdesc); - if (error) - rd_kafka_error_destroy(error); + rd_list_destroy(&members); rd_free(group_id); rd_free(group_state); rd_free(proto_type); rd_free(proto); + RD_IF_FREE(error, rd_kafka_error_destroy); + RD_IF_FREE(operations, rd_free); + error = NULL; group_id = NULL; group_state = NULL; proto_type = NULL; proto = NULL; + operations = NULL; } if (host) @@ -7610,6 +7778,7 @@ rd_kafka_DescribeConsumerGroupsResponse_parse(rd_kafka_op_t *rko_req, rd_kafka_Node_destroy(node); if (rko_result) rd_kafka_op_destroy(rko_result); + RD_IF_FREE(operations, rd_free); rd_snprintf( errstr, errstr_size, @@ -7734,3 +7903,647 @@ rd_kafka_DescribeConsumerGroups_result_groups( } /**@}*/ + +/** + * @name Describe Topic + * @{ + * + * + * + * + */ + +rd_kafka_TopicCollection_t * +rd_kafka_TopicCollection_of_topic_names(const char **topics, + size_t topics_cnt) { + size_t i; + rd_kafka_TopicCollection_t *ret = + rd_calloc(1, sizeof(rd_kafka_TopicCollection_t)); + + ret->topics_cnt = topics_cnt; + if (!ret->topics_cnt) + return ret; + + ret->topics = rd_calloc(topics_cnt, sizeof(char *)); + for (i = 0; i < topics_cnt; i++) + ret->topics[i] = rd_strdup(topics[i]); + + return ret; +} + +void rd_kafka_TopicCollection_destroy(rd_kafka_TopicCollection_t *topics) { + size_t i; + + for (i = 0; i < topics->topics_cnt; i++) + rd_free(topics->topics[i]); + + RD_IF_FREE(topics->topics, rd_free); + rd_free(topics); +} + +/** + * @brief Create a new TopicPartitionInfo object. + * + * @return A newly allocated TopicPartitionInfo. Use + * rd_kafka_TopicPartitionInfo_destroy() to free when done. + */ +static rd_kafka_TopicPartitionInfo_t *rd_kafka_TopicPartitionInfo_new( + const struct rd_kafka_metadata_partition *partition, + const struct rd_kafka_metadata_broker *brokers_sorted, + const rd_kafka_metadata_broker_internal_t *brokers_internal, + int broker_cnt) { + size_t i; + rd_kafka_TopicPartitionInfo_t *pinfo = + rd_calloc(1, sizeof(rd_kafka_TopicPartitionInfo_t)); + + pinfo->partition = partition->id; + pinfo->isr_cnt = partition->isr_cnt; + pinfo->replica_cnt = partition->replica_cnt; + + if (partition->leader >= 0) { + pinfo->leader = rd_kafka_Node_new_from_brokers( + partition->leader, brokers_sorted, brokers_internal, + broker_cnt); + } + + if (pinfo->isr_cnt > 0) { + pinfo->isr = + rd_calloc(pinfo->isr_cnt, sizeof(rd_kafka_Node_t *)); + for (i = 0; i < pinfo->isr_cnt; i++) + pinfo->isr[i] = rd_kafka_Node_new_from_brokers( + partition->isrs[i], brokers_sorted, + brokers_internal, broker_cnt); + } + + if (pinfo->replica_cnt > 0) { + pinfo->replicas = + rd_calloc(pinfo->replica_cnt, sizeof(rd_kafka_Node_t *)); + for (i = 0; i < pinfo->replica_cnt; i++) + pinfo->replicas[i] = rd_kafka_Node_new_from_brokers( + partition->replicas[i], brokers_sorted, + brokers_internal, broker_cnt); + } + + return pinfo; +} + +/** + * @brief Destroy and deallocate a TopicPartitionInfo. + */ +static void +rd_kafka_TopicPartitionInfo_destroy(rd_kafka_TopicPartitionInfo_t *pinfo) { + size_t i; + RD_IF_FREE(pinfo->leader, rd_kafka_Node_destroy); + + for (i = 0; i < pinfo->isr_cnt; i++) + rd_kafka_Node_destroy(pinfo->isr[i]); + RD_IF_FREE(pinfo->isr, rd_free); + + for (i = 0; i < pinfo->replica_cnt; i++) + rd_kafka_Node_destroy(pinfo->replicas[i]); + RD_IF_FREE(pinfo->replicas, rd_free); + + rd_free(pinfo); +} + +/** + * @brief Create a new TopicDescription object. + * + * @param topic topic name + * @param partitions Array of partition metadata (rd_kafka_metadata_partition). + * @param partition_cnt Number of partitions in partition metadata. + * @param authorized_operations acl operations allowed for topic. + * @param error Topic error reported by the broker. + * @return A newly allocated TopicDescription object. + * @remark Use rd_kafka_TopicDescription_destroy() to free when done. + */ +static rd_kafka_TopicDescription_t *rd_kafka_TopicDescription_new( + const char *topic, + const struct rd_kafka_metadata_partition *partitions, + int partition_cnt, + const struct rd_kafka_metadata_broker *brokers_sorted, + const rd_kafka_metadata_broker_internal_t *brokers_internal, + int broker_cnt, + const rd_kafka_AclOperation_t *authorized_operations, + int authorized_operations_cnt, + rd_bool_t is_internal, + rd_kafka_error_t *error) { + rd_kafka_TopicDescription_t *topicdesc; + int i; + topicdesc = rd_calloc(1, sizeof(*topicdesc)); + topicdesc->topic = rd_strdup(topic); + topicdesc->partition_cnt = partition_cnt; + topicdesc->is_internal = is_internal; + if (error) + topicdesc->error = rd_kafka_error_copy(error); + + topicdesc->authorized_operations_cnt = authorized_operations_cnt; + topicdesc->authorized_operations = rd_kafka_AuthorizedOperations_copy( + authorized_operations, authorized_operations_cnt); + + if (partitions) { + topicdesc->partitions = + rd_calloc(partition_cnt, sizeof(*partitions)); + for (i = 0; i < partition_cnt; i++) + topicdesc->partitions[i] = + rd_kafka_TopicPartitionInfo_new( + &partitions[i], brokers_sorted, + brokers_internal, broker_cnt); + } + return topicdesc; +} + +/** + * @brief Create a new TopicDescription object from an error. + * + * @param topic topic name + * @param error Topic error reported by the broker. + * @return A newly allocated TopicDescription with the passed error. + * @remark Use rd_kafka_TopicDescription_destroy() to free when done. + */ +static rd_kafka_TopicDescription_t * +rd_kafka_TopicDescription_new_error(const char *topic, + rd_kafka_error_t *error) { + return rd_kafka_TopicDescription_new(topic, NULL, 0, NULL, NULL, 0, + NULL, 0, rd_false, error); +} + +static void +rd_kafka_TopicDescription_destroy(rd_kafka_TopicDescription_t *topicdesc) { + int i; + + RD_IF_FREE(topicdesc->topic, rd_free); + RD_IF_FREE(topicdesc->error, rd_kafka_error_destroy); + RD_IF_FREE(topicdesc->authorized_operations, rd_free); + + for (i = 0; i < topicdesc->partition_cnt; i++) + rd_kafka_TopicPartitionInfo_destroy(topicdesc->partitions[i]); + rd_free(topicdesc->partitions); + + rd_free(topicdesc); +} + +static void rd_kafka_TopicDescription_free(void *ptr) { + rd_kafka_TopicDescription_destroy(ptr); +} + +const int rd_kafka_TopicPartitionInfo_partition( + const rd_kafka_TopicPartitionInfo_t *partition) { + return partition->partition; +} + +const rd_kafka_Node_t *rd_kafka_TopicPartitionInfo_leader( + const rd_kafka_TopicPartitionInfo_t *partition) { + return partition->leader; +} + + +const rd_kafka_Node_t ** +rd_kafka_TopicPartitionInfo_isr(const rd_kafka_TopicPartitionInfo_t *partition, + size_t *cntp) { + *cntp = partition->isr_cnt; + return (const rd_kafka_Node_t **)partition->isr; +} + +const rd_kafka_Node_t **rd_kafka_TopicPartitionInfo_replicas( + const rd_kafka_TopicPartitionInfo_t *partition, + size_t *cntp) { + *cntp = partition->replica_cnt; + return (const rd_kafka_Node_t **)partition->replicas; +} + +const rd_kafka_TopicPartitionInfo_t **rd_kafka_TopicDescription_partitions( + const rd_kafka_TopicDescription_t *topicdesc, + size_t *cntp) { + *cntp = topicdesc->partition_cnt; + return (const rd_kafka_TopicPartitionInfo_t **)topicdesc->partitions; +} + +const rd_kafka_AclOperation_t *rd_kafka_TopicDescription_authorized_operations( + const rd_kafka_TopicDescription_t *topicdesc, + size_t *cntp) { + *cntp = RD_MAX(topicdesc->authorized_operations_cnt, 0); + return topicdesc->authorized_operations; +} + + +const char * +rd_kafka_TopicDescription_name(const rd_kafka_TopicDescription_t *topicdesc) { + return topicdesc->topic; +} + +int rd_kafka_TopicDescription_is_internal( + const rd_kafka_TopicDescription_t *topicdesc) { + return topicdesc->is_internal; +} + +const rd_kafka_error_t * +rd_kafka_TopicDescription_error(const rd_kafka_TopicDescription_t *topicdesc) { + return topicdesc->error; +} + +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; +} + +/** + * @brief Topics arguments comparator for DescribeTopics args + */ +static int rd_kafka_DescribeTopics_cmp(const void *a, const void *b) { + return strcmp(a, b); +} + +/** + * @brief Construct and send DescribeTopicsRequest to \p rkb + * with the topics (char *) in \p topics, 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 = + rd_kafka_confval_get_int(&options->include_authorized_operations); + + err = rd_kafka_admin_MetadataRequest( + rkb, topics, "describe topics", + rd_false /* don't include_topic_authorized_operations */, + include_topic_authorized_operations, + rd_false /* don't force_racks */, resp_cb, replyq, 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; + const int log_decode_errors = LOG_ERR; + rd_kafka_op_t *rko_result = NULL; + + err = rd_kafka_parse_Metadata_admin(rkb, reply, &topics, &mdi); + if (err) + goto err_parse; + + 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); + + for (i = 0; i < md->topic_cnt; i++) { + rd_kafka_TopicDescription_t *topicdesc = NULL; + int orig_pos; + + if (md->topics[i].err == RD_KAFKA_RESP_ERR_NO_ERROR) { + rd_kafka_AclOperation_t *authorized_operations; + int authorized_operation_cnt; + authorized_operations = + rd_kafka_AuthorizedOperations_parse( + mdi->topics[i].topic_authorized_operations, + &authorized_operation_cnt); + topicdesc = rd_kafka_TopicDescription_new( + md->topics[i].topic, md->topics[i].partitions, + md->topics[i].partition_cnt, mdi->brokers_sorted, + mdi->brokers, md->broker_cnt, authorized_operations, + authorized_operation_cnt, + mdi->topics[i].is_internal, NULL); + RD_IF_FREE(authorized_operations, rd_free); + } else { + rd_kafka_error_t *error = rd_kafka_error_new( + md->topics[i].err, "%s", + rd_kafka_err2str(md->topics[i].err)); + topicdesc = rd_kafka_TopicDescription_new_error( + md->topics[i].topic, error); + rd_kafka_error_destroy(error); + } + orig_pos = rd_list_index(&rko_result->rko_u.admin_result.args, + topicdesc->topic, + rd_kafka_DescribeTopics_cmp); + if (orig_pos == -1) { + rd_kafka_TopicDescription_destroy(topicdesc); + rd_kafka_buf_parse_fail( + reply, + "Broker returned topic %s that was not " + "included in the original request", + topicdesc->topic); + } + + if (rd_list_elem(&rko_result->rko_u.admin_result.results, + orig_pos) != NULL) { + rd_kafka_TopicDescription_destroy(topicdesc); + rd_kafka_buf_parse_fail( + reply, "Broker returned topic %s multiple times", + topicdesc->topic); + } + + rd_list_set(&rko_result->rko_u.admin_result.results, orig_pos, + topicdesc); + } + rd_free(mdi); + + *rko_resultp = rko_result; + return RD_KAFKA_RESP_ERR_NO_ERROR; + +err_parse: + RD_IF_FREE(rko_result, rd_kafka_op_destroy); + 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 rd_kafka_TopicCollection_t *topics, + const rd_kafka_AdminOptions_t *options, + rd_kafka_queue_t *rkqu) { + rd_kafka_op_t *rko; + rd_list_t dup_list; + size_t i; + + static const struct rd_kafka_admin_worker_cbs cbs = { + rd_kafka_admin_DescribeTopicsRequest, + rd_kafka_DescribeTopicsResponse_parse, + }; + + rd_assert(rkqu); + + rko = rd_kafka_admin_request_op_new( + rk, RD_KAFKA_OP_DESCRIBETOPICS, + RD_KAFKA_EVENT_DESCRIBETOPICS_RESULT, &cbs, options, rkqu->rkqu_q); + + if (topics->topics_cnt == 0) { + rd_kafka_admin_result_fail(rko, RD_KAFKA_RESP_ERR__INVALID_ARG, + "No topics to describe"); + rd_kafka_admin_common_worker_destroy(rk, rko, + rd_true /*destroy*/); + return; + } + + rd_list_init(&rko->rko_u.admin_request.args, (int)topics->topics_cnt, + rd_free); + for (i = 0; i < topics->topics_cnt; i++) + rd_list_add(&rko->rko_u.admin_request.args, + rd_strdup(topics->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->rko_u.admin_request.args), + NULL); + rd_list_copy_to(&dup_list, &rko->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, RD_KAFKA_RESP_ERR__INVALID_ARG, + "Duplicate topics not allowed"); + rd_kafka_admin_common_worker_destroy(rk, rko, + rd_true /*destroy*/); + return; + } + + rd_list_destroy(&dup_list); + rd_kafka_q_enq(rk->rk_ops, rko); +} + +/**@}*/ + +/** + * @name Describe cluster + * @{ + * + * + * + * + */ + +static const rd_kafka_ClusterDescription_t * +rd_kafka_DescribeCluster_result_description( + const rd_kafka_DescribeCluster_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; +} + + +const rd_kafka_Node_t **rd_kafka_DescribeCluster_result_nodes( + const rd_kafka_DescribeCluster_result_t *result, + size_t *cntp) { + const rd_kafka_ClusterDescription_t *clusterdesc = + rd_kafka_DescribeCluster_result_description(result); + *cntp = clusterdesc->node_cnt; + return (const rd_kafka_Node_t **)clusterdesc->nodes; +} + +const rd_kafka_AclOperation_t * +rd_kafka_DescribeCluster_result_authorized_operations( + const rd_kafka_DescribeCluster_result_t *result, + size_t *cntp) { + const rd_kafka_ClusterDescription_t *clusterdesc = + rd_kafka_DescribeCluster_result_description(result); + *cntp = RD_MAX(clusterdesc->authorized_operations_cnt, 0); + return clusterdesc->authorized_operations; +} + +const char *rd_kafka_DescribeCluster_result_cluster_id( + const rd_kafka_DescribeCluster_result_t *result) { + return rd_kafka_DescribeCluster_result_description(result)->cluster_id; +} + +const rd_kafka_Node_t *rd_kafka_DescribeCluster_result_controller( + const rd_kafka_DescribeCluster_result_t *result) { + return rd_kafka_DescribeCluster_result_description(result)->controller; +} + +/** + * @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 newly allocated ClusterDescription object. + * @remark Use rd_kafka_ClusterDescription_destroy() to free when done. + */ +static rd_kafka_ClusterDescription_t * +rd_kafka_ClusterDescription_new(const rd_kafka_metadata_internal_t *mdi) { + const rd_kafka_metadata_t *md = &mdi->metadata; + rd_kafka_ClusterDescription_t *clusterdesc = + rd_calloc(1, sizeof(*clusterdesc)); + int i; + + clusterdesc->cluster_id = rd_strdup(mdi->cluster_id); + + if (mdi->controller_id >= 0) + clusterdesc->controller = rd_kafka_Node_new_from_brokers( + mdi->controller_id, mdi->brokers_sorted, mdi->brokers, + md->broker_cnt); + + clusterdesc->authorized_operations = + rd_kafka_AuthorizedOperations_parse( + mdi->cluster_authorized_operations, + &clusterdesc->authorized_operations_cnt); + + clusterdesc->node_cnt = md->broker_cnt; + clusterdesc->nodes = + rd_calloc(clusterdesc->node_cnt, sizeof(rd_kafka_Node_t *)); + + for (i = 0; i < md->broker_cnt; i++) + clusterdesc->nodes[i] = rd_kafka_Node_new_from_brokers( + md->brokers[i].id, mdi->brokers_sorted, mdi->brokers, + md->broker_cnt); + + return clusterdesc; +} + +static void rd_kafka_ClusterDescription_destroy( + rd_kafka_ClusterDescription_t *clusterdesc) { + RD_IF_FREE(clusterdesc->cluster_id, rd_free); + RD_IF_FREE(clusterdesc->controller, rd_kafka_Node_free); + RD_IF_FREE(clusterdesc->authorized_operations, rd_free); + + if (clusterdesc->node_cnt) { + size_t i; + for (i = 0; i < clusterdesc->node_cnt; i++) + rd_kafka_Node_free(clusterdesc->nodes[i]); + rd_free(clusterdesc->nodes); + } + 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 *ignored /* We don't use any arguments set here. */, + 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 = + rd_kafka_confval_get_int(&options->include_authorized_operations); + + err = rd_kafka_admin_MetadataRequest( + rkb, NULL /* topics */, "describe cluster", + include_cluster_authorized_operations, + rd_false /* don't include_topic_authorized_operations */, + rd_false /* don't force racks */, resp_cb, replyq, 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; + rd_kafka_op_t *rko_result = NULL; + + err = rd_kafka_parse_Metadata_admin(rkb, reply, &topics, &mdi); + 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); + + clusterdesc = rd_kafka_ClusterDescription_new(mdi); + + rd_free(mdi); + + rd_list_add(&rko_result->rko_u.admin_result.results, clusterdesc); + *rko_resultp = rko_result; + return RD_KAFKA_RESP_ERR_NO_ERROR; + +err: + RD_IF_FREE(rko_result, rd_kafka_op_destroy); + rd_snprintf(errstr, errstr_size, + "DescribeCluster 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}; + + rko = rd_kafka_admin_request_op_new( + rk, RD_KAFKA_OP_DESCRIBECLUSTER, + RD_KAFKA_EVENT_DESCRIBECLUSTER_RESULT, &cbs, 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 05fbf8db97..4eb015fad0 100644 --- a/src/rdkafka_admin.h +++ b/src/rdkafka_admin.h @@ -2,6 +2,7 @@ * librdkafka - Apache Kafka C library * * Copyright (c) 2018-2022, Magnus Edenhill + * 2023, Confluent Inc. * All rights reserved. * * Redistribution and use in source and binary forms, with or without @@ -93,6 +94,14 @@ 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 + * DescribeCluster + * DescribeTopics + */ rd_kafka_confval_t match_consumer_group_states; /**< PTR: list of consumer group states @@ -473,10 +482,85 @@ struct rd_kafka_ConsumerGroupDescription_s { rd_kafka_consumer_group_state_t state; /** Consumer group coordinator. */ rd_kafka_Node_t *coordinator; + /** Count of operations allowed for topic. -1 indicates operations not + * requested.*/ + int authorized_operations_cnt; + /** Operations allowed for topic. May be NULL if operations were not + * requested */ + rd_kafka_AclOperation_t *authorized_operations; /** Group specific error. */ rd_kafka_error_t *error; }; /**@}*/ +/** + * @name DescribeTopics + * @{ + */ + +/** + * @brief TopicCollection contains a list of topics. + * + */ +struct rd_kafka_TopicCollection_s { + char **topics; /**< List of topic names. */ + size_t topics_cnt; /**< Count of topic names. */ +}; + +/** + * @brief TopicPartition result type in DescribeTopics result. + * + */ +struct rd_kafka_TopicPartitionInfo_s { + int partition; /**< Partition id. */ + rd_kafka_Node_t *leader; /**< Leader of the partition. */ + size_t isr_cnt; /**< Count of insync replicas. */ + rd_kafka_Node_t **isr; /**< List of in sync replica nodes. */ + size_t replica_cnt; /**< Count of partition replicas. */ + rd_kafka_Node_t **replicas; /**< List of replica nodes. */ +}; + +/** + * @struct DescribeTopics result + */ +struct rd_kafka_TopicDescription_s { + char *topic; /**< Topic name */ + int partition_cnt; /**< Number of partitions in \p partitions*/ + rd_bool_t is_internal; /**< Is the topic is internal to Kafka? */ + rd_kafka_TopicPartitionInfo_t **partitions; /**< Partitions */ + rd_kafka_error_t *error; /**< Topic error reported by broker */ + int authorized_operations_cnt; /**< Count of operations allowed for + * topic. -1 indicates operations not + * requested. */ + rd_kafka_AclOperation_t + *authorized_operations; /**< Operations allowed for topic. May be + * NULL if operations were not requested */ +}; + +/**@}*/ + +/** + * @name DescribeCluster + * @{ + */ +/** + * @struct DescribeCluster result - internal type. + */ +typedef struct rd_kafka_ClusterDescription_s { + char *cluster_id; /**< Cluster id */ + rd_kafka_Node_t *controller; /**< Current controller. */ + size_t node_cnt; /**< Count of brokers in the cluster. */ + rd_kafka_Node_t **nodes; /**< Brokers in the cluster. */ + int authorized_operations_cnt; /**< Count of operations allowed for + * cluster. -1 indicates operations not + * requested. */ + rd_kafka_AclOperation_t + *authorized_operations; /**< Operations allowed for cluster. May be + * NULL if operations were not requested */ + +} rd_kafka_ClusterDescription_t; + +/**@}*/ + #endif /* _RDKAFKA_ADMIN_H_ */ diff --git a/src/rdkafka_aux.c b/src/rdkafka_aux.c index 8f4f2b9f05..d327b6c8b0 100644 --- a/src/rdkafka_aux.c +++ b/src/rdkafka_aux.c @@ -238,16 +238,57 @@ void rd_kafka_acl_result_free(void *ptr) { rd_kafka_Node_t *rd_kafka_Node_new(int32_t id, const char *host, uint16_t port, - const char *rack_id) { + const char *rack) { rd_kafka_Node_t *ret = rd_calloc(1, sizeof(*ret)); ret->id = id; ret->port = port; ret->host = rd_strdup(host); - if (rack_id != NULL) - ret->rack_id = rd_strdup(rack_id); + if (rack != NULL) + ret->rack = rd_strdup(rack); return ret; } +/** + * @brief Create a new Node object given a node id, and use broker information + * to populate other fields. + * + * @return A new allocated Node object. + * Use rd_kafka_Node_destroy() to free when done. + * @remark The \p brokers_sorted and \p brokers_internal arrays are asumed to be + * sorted by id. + */ +rd_kafka_Node_t *rd_kafka_Node_new_from_brokers( + int32_t id, + const struct rd_kafka_metadata_broker *brokers_sorted, + const rd_kafka_metadata_broker_internal_t *brokers_internal, + int broker_cnt) { + rd_kafka_Node_t *node = rd_calloc(1, sizeof(*node)); + struct rd_kafka_metadata_broker key_sorted = {.id = id}; + rd_kafka_metadata_broker_internal_t key_internal = {.id = id}; + + struct rd_kafka_metadata_broker *broker = + bsearch(&key_sorted, brokers_sorted, broker_cnt, + sizeof(struct rd_kafka_metadata_broker), + rd_kafka_metadata_broker_cmp); + + rd_kafka_metadata_broker_internal_t *broker_internal = + bsearch(&key_internal, brokers_internal, broker_cnt, + sizeof(rd_kafka_metadata_broker_internal_t), + rd_kafka_metadata_broker_internal_cmp); + + node->id = id; + + if (!broker) + return node; + + node->host = rd_strdup(broker->host); + node->port = broker->port; + if (broker_internal && broker_internal->rack_id) + node->rack = rd_strdup(broker_internal->rack_id); + + return node; +} + /** * @brief Copy \p src Node object * @@ -256,16 +297,26 @@ rd_kafka_Node_t *rd_kafka_Node_new(int32_t id, * Use rd_kafka_Node_destroy() to free when done. */ rd_kafka_Node_t *rd_kafka_Node_copy(const rd_kafka_Node_t *src) { - return rd_kafka_Node_new(src->id, src->host, src->port, src->rack_id); + return rd_kafka_Node_new(src->id, src->host, src->port, src->rack); } void rd_kafka_Node_destroy(rd_kafka_Node_t *node) { rd_free(node->host); - if (node->rack_id) - rd_free(node->rack_id); + if (node->rack) + rd_free(node->rack); rd_free(node); } +/** + * @brief Same as rd_kafka_Node_destroy, but for use as callback which accepts + * (void *) arguments. + * + * @param node + */ +void rd_kafka_Node_free(void *node) { + rd_kafka_Node_destroy((rd_kafka_Node_t *)node); +} + int rd_kafka_Node_id(const rd_kafka_Node_t *node) { return node->id; } @@ -277,3 +328,7 @@ const char *rd_kafka_Node_host(const rd_kafka_Node_t *node) { uint16_t rd_kafka_Node_port(const rd_kafka_Node_t *node) { return node->port; } + +const char *rd_kafka_Node_rack(const rd_kafka_Node_t *node) { + return node->rack; +} diff --git a/src/rdkafka_aux.h b/src/rdkafka_aux.h index cc9db3bbda..fec88cb2ad 100644 --- a/src/rdkafka_aux.h +++ b/src/rdkafka_aux.h @@ -108,7 +108,7 @@ typedef struct rd_kafka_Node_s { int id; /*< Node id */ char *host; /*< Node host */ uint16_t port; /*< Node port */ - char *rack_id; /*< (optional) Node rack id */ + char *rack; /*< (optional) Node rack id */ } rd_kafka_Node_t; rd_kafka_Node_t *rd_kafka_Node_new(int32_t id, @@ -116,8 +116,16 @@ rd_kafka_Node_t *rd_kafka_Node_new(int32_t id, uint16_t port, const char *rack_id); +rd_kafka_Node_t *rd_kafka_Node_new_from_brokers( + int32_t id, + const struct rd_kafka_metadata_broker *brokers_sorted, + const rd_kafka_metadata_broker_internal_t *brokers_internal, + int broker_cnt); + rd_kafka_Node_t *rd_kafka_Node_copy(const rd_kafka_Node_t *src); void rd_kafka_Node_destroy(rd_kafka_Node_t *node); +void rd_kafka_Node_free(void *node); + #endif /* _RDKAFKA_AUX_H_ */ diff --git a/src/rdkafka_event.c b/src/rdkafka_event.c index 28e602b23b..8fd93280ab 100644 --- a/src/rdkafka_event.c +++ b/src/rdkafka_event.c @@ -2,6 +2,7 @@ * librdkafka - Apache Kafka C library * * Copyright (c) 2016-2022, Magnus Edenhill + * 2023, Confluent Inc. * All rights reserved. * * Redistribution and use in source and binary forms, with or without @@ -70,6 +71,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: @@ -379,6 +384,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 aa8e4c6270..4b6f29a203 100644 --- a/src/rdkafka_event.h +++ b/src/rdkafka_event.h @@ -103,6 +103,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 f96edf6583..4fe8ced8d0 100644 --- a/src/rdkafka_metadata.c +++ b/src/rdkafka_metadata.c @@ -48,6 +48,17 @@ int rd_kafka_metadata_broker_internal_cmp(const void *_a, const void *_b) { return RD_CMP(a->id, b->id); } + +/** + * @brief Id comparator for struct rd_kafka_metadata_broker* + */ +int rd_kafka_metadata_broker_cmp(const void *_a, const void *_b) { + const struct rd_kafka_metadata_broker *a = _a; + const struct rd_kafka_metadata_broker *b = _b; + return RD_CMP(a->id, b->id); +} + + /** * @brief Id comparator for rd_kafka_metadata_partition_internal_t */ @@ -58,6 +69,19 @@ static int rd_kafka_metadata_partition_internal_cmp(const void *_a, return RD_CMP(a->id, b->id); } +/** + * @brief Helper function to clear a rd_kafka_metadata_partition. + * + * @note Does not deallocate the rd_kafka_metadata_partition itself. + * @note Should not be used if there is an metadata struct allocated with + * tmpabuf in which rd_kafka_metadata_partition is contained. + */ +void rd_kafka_metadata_partition_clear( + struct rd_kafka_metadata_partition *rkmp) { + RD_IF_FREE(rkmp->isrs, rd_free); + RD_IF_FREE(rkmp->replicas, rd_free); +} + rd_kafka_resp_err_t rd_kafka_metadata(rd_kafka_t *rk, @@ -434,25 +458,14 @@ 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) - * - * The metadata will be marshalled into 'rd_kafka_metadata_internal_t *'. - * - * The marshalled metadata is returned in \p *mdip, (NULL on error). - * - * @returns an error code on parse failure, else NO_ERRRO. - * - * @locality rdkafka main thread - */ -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) { +/* Internal implementation for parsing Metadata. */ +static rd_kafka_resp_err_t +rd_kafka_parse_Metadata0(rd_kafka_broker_t *rkb, + rd_kafka_buf_t *request, + rd_kafka_buf_t *rkbuf, + rd_kafka_metadata_internal_t **mdip, + rd_list_t *request_topics, + const char *reason) { rd_kafka_t *rk = rkb->rkb_rk; int i, j, k; rd_tmpabuf_t tbuf; @@ -461,16 +474,12 @@ rd_kafka_parse_Metadata(rd_kafka_broker_t *rkb, 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; - rd_bool_t cgrp_update = - request->rkbuf_u.Metadata.cgrp_update && rk->rk_cgrp; + const rd_list_t *requested_topics = request_topics; + rd_bool_t all_topics = rd_false; + rd_bool_t cgrp_update = 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; + 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; @@ -481,11 +490,19 @@ rd_kafka_parse_Metadata(rd_kafka_broker_t *rkb, * to contain the partition to rack map. */ rd_bool_t has_client_rack = rk->rk_conf.client_rack && RD_KAFKAP_STR_LEN(rk->rk_conf.client_rack); - /* 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; - rd_bool_t compute_racks = has_client_rack || force_rack_computation; + rd_bool_t compute_racks = has_client_rack; + + if (request) { + requested_topics = request->rkbuf_u.Metadata.topics; + all_topics = request->rkbuf_u.Metadata.all_topics; + cgrp_update = + request->rkbuf_u.Metadata.cgrp_update && rk->rk_cgrp; + compute_racks |= request->rkbuf_u.Metadata.force_racks; + } + + /* If there's reason is NULL, set it to a human-readable string. */ + if (!reason) + reason = "(no reason)"; /* Ignore metadata updates when terminating */ if (rd_kafka_terminating(rkb->rkb_rk)) { @@ -543,6 +560,12 @@ rd_kafka_parse_Metadata(rd_kafka_broker_t *rkb, rkbuf, "%d internal brokers: tmpabuf memory shortage", md->broker_cnt); + if (!(mdi->brokers_sorted = rd_tmpabuf_alloc( + &tbuf, md->broker_cnt * sizeof(*mdi->brokers_sorted)))) + rd_kafka_buf_parse_fail( + rkbuf, "%d sorted brokers: tmpabuf memory shortage", + md->broker_cnt); + for (i = 0; i < md->broker_cnt; i++) { rd_kafka_buf_read_i32a(rkbuf, md->brokers[i].id); rd_kafka_buf_read_str_tmpabuf(rkbuf, &tbuf, @@ -560,11 +583,18 @@ rd_kafka_parse_Metadata(rd_kafka_broker_t *rkb, rd_kafka_buf_skip_tags(rkbuf); } - if (ApiVersion >= 2) + mdi->cluster_id = NULL; + if (ApiVersion >= 2) { rd_kafka_buf_read_str(rkbuf, &cluster_id); + if (cluster_id.str) + mdi->cluster_id = + rd_tmpabuf_write_str(&tbuf, cluster_id.str); + } + mdi->controller_id = -1; 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); @@ -572,6 +602,10 @@ rd_kafka_parse_Metadata(rd_kafka_broker_t *rkb, qsort(mdi->brokers, md->broker_cnt, sizeof(mdi->brokers[i]), rd_kafka_metadata_broker_internal_cmp); + memcpy(mdi->brokers_sorted, md->brokers, + sizeof(*mdi->brokers_sorted) * md->broker_cnt); + qsort(mdi->brokers_sorted, md->broker_cnt, sizeof(*mdi->brokers_sorted), + rd_kafka_metadata_broker_cmp); /* Read TopicMetadata */ rd_kafka_buf_read_arraycnt(rkbuf, &md->topic_cnt, RD_KAFKAP_TOPICS_MAX); @@ -598,10 +632,9 @@ rd_kafka_parse_Metadata(rd_kafka_broker_t *rkb, rd_kafka_buf_read_uuid(rkbuf, &mdi->topics[i].topic_id); } - if (ApiVersion >= 1) { - int8_t is_internal; - rd_kafka_buf_read_i8(rkbuf, &is_internal); - } + if (ApiVersion >= 1) + rd_kafka_buf_read_bool(rkbuf, + &mdi->topics[i].is_internal); /* PartitionMetadata */ rd_kafka_buf_read_arraycnt(rkbuf, &md->topics[i].partition_cnt, @@ -720,20 +753,26 @@ rd_kafka_parse_Metadata(rd_kafka_broker_t *rkb, rd_kafka_buf_skip_tags(rkbuf); } + mdi->topics[i].topic_authorized_operations = -1; if (ApiVersion >= 8) { int32_t TopicAuthorizedOperations; /* TopicAuthorizedOperations */ rd_kafka_buf_read_i32(rkbuf, &TopicAuthorizedOperations); + mdi->topics[i].topic_authorized_operations = + TopicAuthorizedOperations; } rd_kafka_buf_skip_tags(rkbuf); } + mdi->cluster_authorized_operations = -1; if (ApiVersion >= 8 && ApiVersion <= 10) { int32_t ClusterAuthorizedOperations; /* ClusterAuthorizedOperations */ rd_kafka_buf_read_i32(rkbuf, &ClusterAuthorizedOperations); + mdi->cluster_authorized_operations = + ClusterAuthorizedOperations; } rd_kafka_buf_skip_tags(rkbuf); @@ -977,6 +1016,56 @@ rd_kafka_parse_Metadata(rd_kafka_broker_t *rkb, } +/** + * @brief Handle a Metadata response message. + * + * @param request Initial Metadata request, containing the topic information. + * Must not be NULL. + * We require the topic information while parsing to make sure + * that there are no missing topics. + * @param mdip A pointer to (rd_kafka_metadata_internal_t *) into which the + * metadata will be marshalled (set to NULL on error.) + * + * @returns an error code on parse failure, else NO_ERROR. + * + * @locality rdkafka main thread + */ +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) { + const char *reason = request->rkbuf_u.Metadata.reason; + return rd_kafka_parse_Metadata0(rkb, request, rkbuf, mdip, NULL, + reason); +} + +/** + * @brief Handle a Metadata response message for admin requests. + * + * @param request_topics List containing topics in Metadata request. Must not + * be NULL. It is more convenient in the Admin flow to + * preserve the topic names rather than the initial + * Metadata request. + * We require the topic information while parsing to make + * sure that there are no missing topics. + * @param mdip A pointer to (rd_kafka_metadata_internal_t *) into which the + * metadata will be marshalled (set to NULL on error.) + * + * @returns an error code on parse failure, else NO_ERROR. + * + * @locality rdkafka main thread + */ +rd_kafka_resp_err_t +rd_kafka_parse_Metadata_admin(rd_kafka_broker_t *rkb, + rd_kafka_buf_t *rkbuf, + rd_list_t *request_topics, + rd_kafka_metadata_internal_t **mdip) { + return rd_kafka_parse_Metadata0(rkb, NULL, rkbuf, mdip, request_topics, + "(admin request)"); +} + + /** * @brief Add all topics in current cached full metadata * that matches the topics in \p match diff --git a/src/rdkafka_metadata.h b/src/rdkafka_metadata.h index 8a8f16dbfa..2b81e0ddec 100644 --- a/src/rdkafka_metadata.h +++ b/src/rdkafka_metadata.h @@ -55,6 +55,10 @@ typedef struct rd_kafka_metadata_topic_internal_s { * Sorted by Partition Id. */ rd_kafka_metadata_partition_internal_t *partitions; rd_kafka_uuid_t topic_id; + int32_t topic_authorized_operations; /**< ACL operations allowed + * for topic, -1 if not + * supported by broker */ + rd_bool_t is_internal; /**< Is topic internal to Kafka? */ } rd_kafka_metadata_topic_internal_t; @@ -77,11 +81,19 @@ typedef struct rd_kafka_metadata_internal_s { be kept the first field so the pointer can be cast to *rd_kafka_metadata_internal_t when needed */ + /* Identical to metadata->brokers, but sorted by broker id. */ + struct rd_kafka_metadata_broker *brokers_sorted; /* Internal metadata brokers. Same count as metadata.broker_cnt. * Sorted by broker id. */ 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; /**< Cluster id (optionally populated)*/ + int controller_id; /**< current controller id for cluster, -1 if not + * supported by broker. */ + int32_t cluster_authorized_operations; /**< ACL operations allowed + * for cluster, -1 if not + * supported by broker */ } rd_kafka_metadata_internal_t; /** @@ -92,10 +104,17 @@ typedef struct rd_kafka_metadata_internal_s { 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_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_resp_err_t +rd_kafka_parse_Metadata_admin(rd_kafka_broker_t *rkb, + rd_kafka_buf_t *rkbuf, + rd_list_t *request_topics, + rd_kafka_metadata_internal_t **mdip); rd_kafka_metadata_internal_t * rd_kafka_metadata_copy(const rd_kafka_metadata_internal_t *mdi, size_t size); @@ -160,6 +179,10 @@ int rd_kafka_metadata_partition_id_cmp(const void *_a, const void *_b); int rd_kafka_metadata_broker_internal_cmp(const void *_a, const void *_b); +int rd_kafka_metadata_broker_cmp(const void *_a, const void *_b); + +void rd_kafka_metadata_partition_clear( + struct rd_kafka_metadata_partition *rkmp); #define rd_kafka_metadata_broker_internal_find(mdi, broker_id, broker) \ do { \ diff --git a/src/rdkafka_op.c b/src/rdkafka_op.c index 6ecb6cd14c..b18ed427bf 100644 --- a/src/rdkafka_op.c +++ b/src/rdkafka_op.c @@ -87,7 +87,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", @@ -243,7 +245,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), @@ -415,6 +419,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: case RD_KAFKA_OP_ALTERUSERSCRAMCREDENTIALS: case RD_KAFKA_OP_DESCRIBEUSERSCRAMCREDENTIALS: diff --git a/src/rdkafka_op.h b/src/rdkafka_op.h index 6018a2659d..ea88ee5a28 100644 --- a/src/rdkafka_op.h +++ b/src/rdkafka_op.h @@ -141,7 +141,14 @@ typedef enum { RD_KAFKA_OP_DESCRIBECONSUMERGROUPS, /**< Admin: * DescribeConsumerGroups * u.admin_request */ - RD_KAFKA_OP_DELETEGROUPS, /**< Admin: DeleteGroups: u.admin_request*/ + RD_KAFKA_OP_DESCRIBECLUSTER, /**< Admin: + * DescribeCluster + * u.admin_request */ + + RD_KAFKA_OP_DESCRIBETOPICS, /**< Admin: + * DescribeTopics + * u.admin_request */ + RD_KAFKA_OP_DELETEGROUPS, /**< Admin: DeleteGroups: u.admin_request*/ RD_KAFKA_OP_DELETECONSUMERGROUPOFFSETS, /**< Admin: * DeleteConsumerGroupOffsets * u.admin_request */ diff --git a/src/rdkafka_request.c b/src/rdkafka_request.c index 005833d204..2cbf596e74 100644 --- a/src/rdkafka_request.c +++ b/src/rdkafka_request.c @@ -2017,6 +2017,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. * * The response (unparsed) will be enqueued on \p replyq * for handling by \p resp_cb (with \p opaque passed). @@ -2024,13 +2026,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; @@ -2067,8 +2071,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); @@ -2175,45 +2178,44 @@ static void rd_kafka_handle_Metadata(rd_kafka_t *rk, /** - * @brief Construct MetadataRequest (does not send) - * - * \p topics is a list of topic names (char *) to request. + * @brief Internal implementation of MetadataRequest (does not send). * - * !topics - only request brokers (if supported by broker, else - * all topics) - * topics.cnt==0 - all topics in cluster are requested - * topics.cnt >0 - only specified topics are requested + * @param force - rd_true: force a full request (including all topics and + * brokers) even if there is such a request already + * in flight. + * - rd_false: check if there are multiple outstanding full + * requests, and don't send one if there is already + * one present. (See note below.) * - * @param reason - metadata request reason - * @param allow_auto_create_topics - allow broker-side auto topic creation. - * This is best-effort, depending on broker - * config and version. - * @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. - * - * 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. + * 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 or if \p force is true, 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) { +static rd_kafka_resp_err_t +rd_kafka_MetadataRequest0(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, + rd_kafka_replyq_t replyq, + rd_bool_t force, + void *opaque) { rd_kafka_buf_t *rkbuf; int16_t ApiVersion = 0; size_t of_TopicArrayCnt; int features; - int topic_cnt = topics ? rd_list_cnt(topics) : 0; - int *full_incr = NULL; + int topic_cnt = topics ? rd_list_cnt(topics) : 0; + int *full_incr = NULL; + void *handler_arg = NULL; + rd_kafka_resp_cb_t *handler_cb = rd_kafka_handle_Metadata; + rd_kafka_replyq_t use_replyq = replyq; ApiVersion = rd_kafka_broker_ApiVersion_supported( rkb, RD_KAFKAP_Metadata, 0, 12, &features); @@ -2279,11 +2281,12 @@ rd_kafka_resp_err_t rd_kafka_MetadataRequest(rd_kafka_broker_t *rkb, if (full_incr) { /* Avoid multiple outstanding full requests * (since they are redundant and side-effect-less). - * Forced requests (app using metadata() API) are passed - * through regardless. */ + * Forced requests (app using metadata() API or Admin API) are + * passed 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", @@ -2345,15 +2348,15 @@ 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); @@ -2362,17 +2365,155 @@ 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); + /* The default handler is rd_kafka_handle_Metadata, but it can be + * overriden to use a custom handler. */ + if (resp_cb) + handler_cb = resp_cb; + + /* If a custom handler is provided, we also allow the caller to set a + * custom argument which is passed as the opaque argument to the + * handler. However, if we're using the default handler, it expects + * either rko or NULL as its opaque argument (it forwards the response + * to rko's replyq if it's non-NULL). */ + if (resp_cb && opaque) + handler_arg = opaque; + else + handler_arg = rko; + + /* If a custom replyq is provided (and is valid), the response is + * handled through on that replyq. By default, response is handled on + * rk_ops, and the default handler (rd_kafka_handle_Metadata) forwards + * the parsed result to rko's replyq when done. */ + if (!use_replyq.q) + use_replyq = RD_KAFKA_REPLYQ(rkb->rkb_rk->rk_ops, 0); + + rd_kafka_broker_buf_enq_replyq( + rkb, rkbuf, use_replyq, + /* The default response handler is rd_kafka_handle_Metadata, but we + allow alternate handlers to be configured. */ + handler_cb, handler_arg); return RD_KAFKA_RESP_ERR_NO_ERROR; } +/** + * @brief Construct a MetadataRequest which uses an optional rko, and the + * default handler callback. + * @sa rd_kafka_MetadataRequest. + */ +static rd_kafka_resp_err_t +rd_kafka_MetadataRequest_op(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) { + return rd_kafka_MetadataRequest0( + rkb, topics, reason, allow_auto_create_topics, + include_cluster_authorized_operations, + include_topic_authorized_operations, cgrp_update, force_racks, rko, + /* We use the default rd_kafka_handle_Metadata rather than a custom + resp_cb */ + NULL, + /* Use default replyq which works with the default handler + rd_kafka_handle_Metadata. */ + RD_KAFKA_NO_REPLYQ, + /* If the request needs to be forced, rko_u.metadata.force will be + set. We don't provide an explicit parameter force. */ + rd_false, NULL); +} + +/** + * @brief Construct MetadataRequest (does not send) + * + * \p topics is a list of topic names (char *) to request. + * + * !topics - only request brokers (if supported by broker, else + * all topics) + * topics.cnt==0 - all topics in cluster are requested + * topics.cnt >0 - only specified topics are requested + * + * @param reason - metadata request reason + * @param allow_auto_create_topics - allow broker-side auto topic creation. + * This is best-effort, depending on broker + * config and version. + * @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. + * + * 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) { + return rd_kafka_MetadataRequest_op( + rkb, topics, reason, allow_auto_create_topics, + /* cluster and topic authorized operations are used by admin + operations only. */ + rd_false, rd_false, cgrp_update, force_racks, rko); +} + + +/** + * @brief Construct MetadataRequest for use with AdminAPI (does not send). + * + * \p topics is a list of topic names (char *) to request. + * + * !topics - only request brokers (if supported by broker, else + * all topics) + * topics.cnt==0 - all topics in cluster are requested + * topics.cnt >0 - only specified topics are requested + * + * @param reason - metadata request reason + * @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 resp_cb - callback to be used for handling response. + * @param replyq - replyq on which response is handled. + * @param opaque - (optional) parameter to be passed to resp_cb. + */ +rd_kafka_resp_err_t rd_kafka_MetadataRequest_resp_cb( + 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_resp_cb_t *resp_cb, + rd_kafka_replyq_t replyq, + rd_bool_t force, + void *opaque) { + return rd_kafka_MetadataRequest0( + rkb, topics, reason, allow_auto_create_topics, + include_cluster_authorized_operations, + include_topic_authorized_operations, cgrp_update, force_racks, + NULL /* No op - using custom resp_cb. */, resp_cb, replyq, + rd_true /* Admin operation metadata requests are always forced. */, + opaque); +} + + /** * @brief Parses and handles ApiVersion reply. diff --git a/src/rdkafka_request.h b/src/rdkafka_request.h index 097b2fcb36..a921c26684 100644 --- a/src/rdkafka_request.h +++ b/src/rdkafka_request.h @@ -236,13 +236,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, @@ -262,6 +264,20 @@ rd_kafka_resp_err_t rd_kafka_MetadataRequest(rd_kafka_broker_t *rkb, rd_bool_t force_racks, rd_kafka_op_t *rko); +rd_kafka_resp_err_t rd_kafka_MetadataRequest_resp_cb( + 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_resp_cb_t *resp_cb, + rd_kafka_replyq_t replyq, + rd_bool_t force, + void *opaque); + rd_kafka_resp_err_t rd_kafka_handle_ApiVersion(rd_kafka_t *rk, rd_kafka_broker_t *rkb, diff --git a/tests/0080-admin_ut.c b/tests/0080-admin_ut.c index 66693d3fdd..3a3b980f0a 100644 --- a/tests/0080-admin_ut.c +++ b/tests/0080-admin_ut.c @@ -634,9 +634,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; @@ -658,6 +659,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; @@ -711,6 +723,7 @@ static void do_test_DescribeConsumerGroups(const char *what, /* The returned groups should be in the original order, and * should all have timed out. */ for (i = 0; i < TEST_DESCRIBE_CONSUMER_GROUPS_CNT; i++) { + size_t authorized_operation_cnt; TEST_ASSERT( !strcmp(group_names[i], rd_kafka_ConsumerGroupDescription_group_id( @@ -725,6 +738,12 @@ static void do_test_DescribeConsumerGroups(const char *what, group_names[i], rd_kafka_error_string( rd_kafka_ConsumerGroupDescription_error(resgroups[i]))); + + rd_kafka_ConsumerGroupDescription_authorized_operations( + resgroups[i], &authorized_operation_cnt); + TEST_ASSERT(authorized_operation_cnt == 0, + "Got authorized operations" + "when not requested"); } rd_kafka_event_destroy(rkev); @@ -744,6 +763,219 @@ 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_kafka_queue_t *q; +#define TEST_DESCRIBE_TOPICS_CNT 4 + const char *topic_names[TEST_DESCRIBE_TOPICS_CNT]; + rd_kafka_TopicCollection_t *topics; + 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)); + } + + topics = rd_kafka_TopicCollection_of_topic_names( + topic_names, TEST_DESCRIBE_TOPICS_CNT); + + 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_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, topics, options, q); + TIMING_ASSERT_LATER(&timing, 0, 50); + + /* 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 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 DescribeTopics to return error %s, not %s (%s)", + rd_kafka_err2str(RD_KAFKA_RESP_ERR__TIMED_OUT), + rd_kafka_err2str(err), err ? errstr2 : "n/a"); + + /* Extract topics, should return 0 topics. */ + restopics = rd_kafka_DescribeTopics_result_topics(res, &restopic_cnt); + TEST_ASSERT(!restopics && restopic_cnt == 0, + "expected no result topics, got %p cnt %" PRIusz, restopics, + restopic_cnt); + + rd_kafka_event_destroy(rkev); + + for (i = 0; i < TEST_DESCRIBE_TOPICS_CNT; i++) { + rd_free((char *)topic_names[i]); + } + rd_kafka_TopicCollection_destroy(topics); + + 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_kafka_queue_t *q; + rd_kafka_AdminOptions_t *options = NULL; + int exp_timeout = MY_SOCKET_TIMEOUT_MS; + 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; + 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_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); + + /* 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); + + 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, @@ -2579,6 +2811,14 @@ 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); + do_test_DescribeTopics("temp queue, options", rk, NULL, 1); + do_test_DescribeTopics("main queue, options", rk, mainq, 1); + + do_test_DescribeCluster("temp queue, no options", rk, NULL, 0); + do_test_DescribeCluster("temp queue, options", rk, NULL, 1); + do_test_DescribeCluster("main queue, options", rk, mainq, 1); + 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 ed39cfc9bb..3107c048b0 100644 --- a/tests/0081-admin.c +++ b/tests/0081-admin.c @@ -2866,6 +2866,7 @@ static void do_test_DescribeConsumerGroups(const char *what, 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; + size_t authorized_operation_cnt; rd_bool_t has_group_instance_id = test_broker_version >= TEST_BRKVER(2, 4, 0, 0); @@ -2986,6 +2987,16 @@ 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); + const rd_kafka_AclOperation_t *authorized_operations = + rd_kafka_ConsumerGroupDescription_authorized_operations( + act, &authorized_operation_cnt); + TEST_ASSERT( + authorized_operation_cnt == 0, + "Authorized operation count should be 0, is %" PRIusz, + authorized_operation_cnt); + TEST_ASSERT( + authorized_operations == NULL, + "Authorized operations should be NULL when not requested"); TEST_ASSERT( strcmp(exp->group_id, rd_kafka_ConsumerGroupDescription_group_id(act)) == @@ -3092,6 +3103,8 @@ static void do_test_DescribeConsumerGroups(const char *what, rd_free(expected[i].group_id); } + test_DeleteTopics_simple(rk, NULL, &topic, 1, NULL); + rd_free(topic); if (options) @@ -3106,6 +3119,709 @@ static void do_test_DescribeConsumerGroups(const char *what, SUB_TEST_PASS(); } +/** @brief Helper function to check whether \p expected and \p actual contain + * the same values. */ +static void +test_match_authorized_operations(const rd_kafka_AclOperation_t *expected, + size_t expected_cnt, + const rd_kafka_AclOperation_t *actual, + size_t actual_cnt) { + size_t i, j; + TEST_ASSERT(expected_cnt == actual_cnt, + "Expected %" PRIusz " authorized operations, got %" PRIusz, + expected_cnt, actual_cnt); + + for (i = 0; i < expected_cnt; i++) { + for (j = 0; j < actual_cnt; j++) + if (expected[i] == actual[j]) + break; + + if (j == actual_cnt) + TEST_FAIL( + "Did not find expected authorized operation in " + "result %s\n", + rd_kafka_AclOperation_name(expected[i])); + } +} + +/** + * @brief Test DescribeTopics: create a topic, describe it, and then + * delete it. + * + * @param include_authorized_operations if true, check authorized + * operations included in topic descriptions, and if they're changed if + * ACLs are defined. + */ +static void do_test_DescribeTopics(const char *what, + rd_kafka_t *rk, + rd_kafka_queue_t *rkqu, + int request_timeout, + rd_bool_t include_authorized_operations) { + rd_kafka_queue_t *q; +#define TEST_DESCRIBE_TOPICS_CNT 3 + char *topic_names[TEST_DESCRIBE_TOPICS_CNT]; + rd_kafka_TopicCollection_t *topics; + rd_kafka_AdminOptions_t *options; + rd_kafka_event_t *rkev; + const 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; + const rd_kafka_TopicPartitionInfo_t **partitions; + size_t partitions_cnt; + size_t result_topics_cnt; + char errstr[128]; + const char *errstr2; + const char *sasl_username; + const char *sasl_mechanism; + const char *principal; + rd_kafka_AclBinding_t *acl_bindings[1]; + int i; + const rd_kafka_AclOperation_t *authorized_operations; + size_t authorized_operations_cnt; + + SUB_TEST_QUICK( + "%s DescribeTopics with %s, request_timeout %d, " + "%s authorized operations", + rd_kafka_name(rk), what, request_timeout, + include_authorized_operations ? "with" : "without"); + + q = rkqu ? rkqu : rd_kafka_queue_new(rk); + + /* Only create one topic, the others will be non-existent. */ + for (i = 0; i < TEST_DESCRIBE_TOPICS_CNT; i++) { + rd_strdupa(&topic_names[i], + test_mk_topic_name(__FUNCTION__, 1)); + } + topics = rd_kafka_TopicCollection_of_topic_names( + (const char **)topic_names, TEST_DESCRIBE_TOPICS_CNT); + + test_CreateTopics_simple(rk, NULL, topic_names, 1, 1, NULL); + test_wait_topic_exists(rk, topic_names[0], 10000); + + /* Call DescribeTopics. */ + options = + rd_kafka_AdminOptions_new(rk, RD_KAFKA_ADMIN_OP_DESCRIBETOPICS); + TEST_CALL_ERR__(rd_kafka_AdminOptions_set_request_timeout( + options, request_timeout, errstr, sizeof(errstr))); + TEST_CALL_ERROR__( + rd_kafka_AdminOptions_set_include_authorized_operations( + options, include_authorized_operations)); + + TIMING_START(&timing, "DescribeTopics"); + rd_kafka_DescribeTopics(rk, topics, options, q); + TIMING_ASSERT_LATER(&timing, 0, 50); + rd_kafka_AdminOptions_destroy(options); + + /* Check DescribeTopics results. */ + rkev = test_wait_admin_result(q, RD_KAFKA_EVENT_DESCRIBETOPICS_RESULT, + tmout_multip(20 * 1000)); + TEST_ASSERT(rkev, "Expected DescribeTopicsResult on queue"); + + /* 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((int)result_topics_cnt == TEST_DESCRIBE_TOPICS_CNT, + "Expected %d topics in result, got %d", + TEST_DESCRIBE_TOPICS_CNT, (int)result_topics_cnt); + + /* Check if topics[0] succeeded. */ + error = rd_kafka_TopicDescription_error(result_topics[0]); + TEST_ASSERT(rd_kafka_error_code(error) == RD_KAFKA_RESP_ERR_NO_ERROR, + "Expected no error, not %s\n", + rd_kafka_error_string(error)); + + /* + * Check whether the topics which are non-existent have + * RD_KAFKA_RESP_ERR_UNKNOWN_TOPIC_OR_PART error. + */ + for (i = 1; i < TEST_DESCRIBE_TOPICS_CNT; i++) { + error = rd_kafka_TopicDescription_error(result_topics[i]); + TEST_ASSERT(rd_kafka_error_code(error) == + RD_KAFKA_RESP_ERR_UNKNOWN_TOPIC_OR_PART, + "Expected unknown Topic or partition, not %s\n", + rd_kafka_error_string(error)); + } + + /* Check fields inside the first (existent) topic. */ + TEST_ASSERT(strcmp(rd_kafka_TopicDescription_name(result_topics[0]), + topic_names[0]) == 0, + "Expected topic name %s, got %s", topic_names[0], + rd_kafka_TopicDescription_name(result_topics[0])); + + partitions = rd_kafka_TopicDescription_partitions(result_topics[0], + &partitions_cnt); + + TEST_ASSERT(partitions_cnt == 1, "Expected %d partitions, got %" PRIusz, + 1, partitions_cnt); + + TEST_ASSERT(rd_kafka_TopicPartitionInfo_partition(partitions[0]) == 0, + "Expected partion id to be %d, got %d", 0, + rd_kafka_TopicPartitionInfo_partition(partitions[0])); + + authorized_operations = rd_kafka_TopicDescription_authorized_operations( + result_topics[0], &authorized_operations_cnt); + if (include_authorized_operations) { + const rd_kafka_AclOperation_t expected[] = { + RD_KAFKA_ACL_OPERATION_ALTER, + RD_KAFKA_ACL_OPERATION_ALTER_CONFIGS, + RD_KAFKA_ACL_OPERATION_CREATE, + RD_KAFKA_ACL_OPERATION_DELETE, + RD_KAFKA_ACL_OPERATION_DESCRIBE, + RD_KAFKA_ACL_OPERATION_DESCRIBE_CONFIGS, + RD_KAFKA_ACL_OPERATION_READ, + RD_KAFKA_ACL_OPERATION_WRITE}; + + test_match_authorized_operations(expected, 8, + authorized_operations, + authorized_operations_cnt); + } else { + TEST_ASSERT( + authorized_operations_cnt == 0, + "Authorized operation count should be 0, is %" PRIusz, + authorized_operations_cnt); + TEST_ASSERT( + authorized_operations == NULL, + "Authorized operations should be NULL when not requested"); + } + + rd_kafka_event_destroy(rkev); + + /* If we don't have authentication/authorization set up in our + * broker, the following test doesn't make sense, since we're + * testing ACLs and authorized operations for our principal. The + * same goes for `include_authorized_operations`, if it's not + * true, it doesn't make sense to change the ACLs and check. We + * limit ourselves to SASL_PLAIN and SASL_SCRAM.*/ + if (!test_needs_auth() || !include_authorized_operations) + goto done; + + sasl_mechanism = test_conf_get(NULL, "sasl.mechanism"); + if (strcmp(sasl_mechanism, "PLAIN") != 0 && + strncmp(sasl_mechanism, "SCRAM", 5) != 0) + goto done; + + sasl_username = test_conf_get(NULL, "sasl.username"); + principal = tsprintf("User:%s", sasl_username); + + /* Change authorized operations for the principal which we're + * using to connect to the broker. */ + acl_bindings[0] = rd_kafka_AclBinding_new( + RD_KAFKA_RESOURCE_TOPIC, topic_names[0], + RD_KAFKA_RESOURCE_PATTERN_LITERAL, principal, "*", + RD_KAFKA_ACL_OPERATION_READ, RD_KAFKA_ACL_PERMISSION_TYPE_ALLOW, + NULL, 0); + TEST_CALL_ERR__( + test_CreateAcls_simple(rk, NULL, acl_bindings, 1, NULL)); + rd_kafka_AclBinding_destroy(acl_bindings[0]); + + /* Call DescribeTopics. */ + options = + rd_kafka_AdminOptions_new(rk, RD_KAFKA_ADMIN_OP_DESCRIBETOPICS); + TEST_CALL_ERR__(rd_kafka_AdminOptions_set_request_timeout( + options, request_timeout, errstr, sizeof(errstr))); + TEST_CALL_ERROR__( + rd_kafka_AdminOptions_set_include_authorized_operations(options, + 1)); + + TIMING_START(&timing, "DescribeTopics"); + rd_kafka_DescribeTopics(rk, topics, options, q); + TIMING_ASSERT_LATER(&timing, 0, 50); + rd_kafka_AdminOptions_destroy(options); + + /* Check DescribeTopics results. */ + rkev = test_wait_admin_result(q, RD_KAFKA_EVENT_DESCRIBETOPICS_RESULT, + tmout_multip(20 * 1000)); + TEST_ASSERT(rkev, "Expected DescribeTopicsResult on queue"); + + /* 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((int)result_topics_cnt == TEST_DESCRIBE_TOPICS_CNT, + "Expected %d topics in result, got %d", + TEST_DESCRIBE_TOPICS_CNT, (int)result_topics_cnt); + + /* Check if topics[0] succeeded. */ + error = rd_kafka_TopicDescription_error(result_topics[0]); + TEST_ASSERT(rd_kafka_error_code(error) == RD_KAFKA_RESP_ERR_NO_ERROR, + "Expected no error, not %s\n", + rd_kafka_error_string(error)); + + /* Check if ACLs changed. */ + { + const rd_kafka_AclOperation_t expected[] = { + RD_KAFKA_ACL_OPERATION_READ, + RD_KAFKA_ACL_OPERATION_DESCRIBE}; + authorized_operations = + rd_kafka_TopicDescription_authorized_operations( + result_topics[0], &authorized_operations_cnt); + + test_match_authorized_operations(expected, 2, + authorized_operations, + authorized_operations_cnt); + } + rd_kafka_event_destroy(rkev); + + /* + * Allow RD_KAFKA_ACL_OPERATION_DELETE to allow deletion + * of the created topic as currently our principal only has read + * and describe. + */ + acl_bindings[0] = rd_kafka_AclBinding_new( + RD_KAFKA_RESOURCE_TOPIC, topic_names[0], + RD_KAFKA_RESOURCE_PATTERN_LITERAL, principal, "*", + RD_KAFKA_ACL_OPERATION_DELETE, RD_KAFKA_ACL_PERMISSION_TYPE_ALLOW, + NULL, 0); + TEST_CALL_ERR__( + test_CreateAcls_simple(rk, NULL, acl_bindings, 1, NULL)); + rd_kafka_AclBinding_destroy(acl_bindings[0]); + +done: + test_DeleteTopics_simple(rk, NULL, topic_names, 1, NULL); + if (!rkqu) + rd_kafka_queue_destroy(q); + + rd_kafka_TopicCollection_destroy(topics); + + + TEST_LATER_CHECK(); +#undef TEST_DESCRIBE_TOPICS_CNT + + SUB_TEST_PASS(); +} + +/** + * @brief Test DescribeCluster for the test cluster. + * + * @param include_authorized_operations if true, check authorized operations + * included in cluster description, and if they're changed if ACLs are defined. + */ +static void do_test_DescribeCluster(const char *what, + rd_kafka_t *rk, + rd_kafka_queue_t *rkqu, + int request_timeout, + rd_bool_t include_authorized_operations) { + rd_kafka_queue_t *q; + rd_kafka_AdminOptions_t *options; + rd_kafka_event_t *rkev; + rd_kafka_resp_err_t err; + test_timing_t timing; + const rd_kafka_DescribeCluster_result_t *res; + const rd_kafka_Node_t **nodes; + size_t node_cnt; + char errstr[128]; + const char *errstr2; + rd_kafka_AclBinding_t *acl_bindings[1]; + rd_kafka_AclBindingFilter_t *acl_bindings_delete; + const rd_kafka_AclOperation_t *authorized_operations; + size_t authorized_operations_cnt; + const char *sasl_username; + const char *sasl_mechanism; + const char *principal; + + SUB_TEST_QUICK( + "%s DescribeCluster with %s, request_timeout %d, %s authorized " + "operations", + rd_kafka_name(rk), what, request_timeout, + include_authorized_operations ? "with" : "without"); + + q = rkqu ? rkqu : rd_kafka_queue_new(rk); + + /* Call DescribeCluster. */ + options = + rd_kafka_AdminOptions_new(rk, RD_KAFKA_ADMIN_OP_DESCRIBECLUSTER); + TEST_CALL_ERR__(rd_kafka_AdminOptions_set_request_timeout( + options, request_timeout, errstr, sizeof(errstr))); + TEST_CALL_ERROR__( + rd_kafka_AdminOptions_set_include_authorized_operations( + options, include_authorized_operations)); + + TIMING_START(&timing, "DescribeCluster"); + rd_kafka_DescribeCluster(rk, options, q); + TIMING_ASSERT_LATER(&timing, 0, 50); + rd_kafka_AdminOptions_destroy(options); + + /* Wait for DescribeCluster result.*/ + rkev = test_wait_admin_result(q, RD_KAFKA_EVENT_DESCRIBECLUSTER_RESULT, + tmout_multip(20 * 1000)); + TEST_ASSERT(rkev, "Should receive describe cluster event."); + + /* 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); + + /* Sanity checks on fields inside the result. There's not much we can + * say here deterministically, since it depends on the test environment. + */ + TEST_ASSERT(strlen(rd_kafka_DescribeCluster_result_cluster_id(res)), + "Length of cluster id should be non-null."); + + nodes = rd_kafka_DescribeCluster_result_nodes(res, &node_cnt); + TEST_ASSERT(node_cnt, "Expected non-zero node count for cluster."); + + TEST_ASSERT(rd_kafka_Node_host(nodes[0]), + "Expected first node of cluster to have a hostname"); + TEST_ASSERT(rd_kafka_Node_port(nodes[0]), + "Expected first node of cluster to have a port"); + + authorized_operations = + rd_kafka_DescribeCluster_result_authorized_operations( + res, &authorized_operations_cnt); + if (include_authorized_operations) { + const rd_kafka_AclOperation_t expected[] = { + RD_KAFKA_ACL_OPERATION_ALTER, + RD_KAFKA_ACL_OPERATION_ALTER_CONFIGS, + RD_KAFKA_ACL_OPERATION_CLUSTER_ACTION, + RD_KAFKA_ACL_OPERATION_CREATE, + RD_KAFKA_ACL_OPERATION_DESCRIBE, + RD_KAFKA_ACL_OPERATION_DESCRIBE_CONFIGS, + RD_KAFKA_ACL_OPERATION_IDEMPOTENT_WRITE}; + + test_match_authorized_operations(expected, 7, + authorized_operations, + authorized_operations_cnt); + } else { + TEST_ASSERT( + authorized_operations_cnt == 0, + "Authorized operation count should be 0, is %" PRIusz, + authorized_operations_cnt); + TEST_ASSERT( + authorized_operations == NULL, + "Authorized operations should be NULL when not requested"); + } + + rd_kafka_event_destroy(rkev); + + /* If we don't have authentication/authorization set up in our broker, + * the following test doesn't make sense, since we're testing ACLs and + * authorized operations for our principal. The same goes for + * `include_authorized_operations`, if it's not true, it doesn't make + * sense to change the ACLs and check. We limit ourselves to SASL_PLAIN + * and SASL_SCRAM.*/ + if (!test_needs_auth() || !include_authorized_operations) + goto done; + + sasl_mechanism = test_conf_get(NULL, "sasl.mechanism"); + if (strcmp(sasl_mechanism, "PLAIN") != 0 && + strncmp(sasl_mechanism, "SCRAM", 5) != 0) + goto done; + + sasl_username = test_conf_get(NULL, "sasl.username"); + principal = tsprintf("User:%s", sasl_username); + + /* Change authorized operations for the principal which we're using to + * connect to the broker. */ + acl_bindings[0] = rd_kafka_AclBinding_new( + RD_KAFKA_RESOURCE_BROKER, "kafka-cluster", + RD_KAFKA_RESOURCE_PATTERN_LITERAL, principal, "*", + RD_KAFKA_ACL_OPERATION_ALTER, RD_KAFKA_ACL_PERMISSION_TYPE_ALLOW, + NULL, 0); + test_CreateAcls_simple(rk, NULL, acl_bindings, 1, NULL); + rd_kafka_AclBinding_destroy(acl_bindings[0]); + + /* Call DescribeCluster. */ + options = + rd_kafka_AdminOptions_new(rk, RD_KAFKA_ADMIN_OP_DESCRIBECLUSTER); + + TEST_CALL_ERR__(rd_kafka_AdminOptions_set_request_timeout( + options, request_timeout, errstr, sizeof(errstr))); + TEST_CALL_ERROR__( + rd_kafka_AdminOptions_set_include_authorized_operations(options, + 1)); + + TIMING_START(&timing, "DescribeCluster"); + rd_kafka_DescribeCluster(rk, options, q); + TIMING_ASSERT_LATER(&timing, 0, 50); + rd_kafka_AdminOptions_destroy(options); + + rkev = test_wait_admin_result(q, RD_KAFKA_EVENT_DESCRIBECLUSTER_RESULT, + tmout_multip(20 * 1000)); + TEST_ASSERT(rkev, "Should receive describe cluster event."); + + /* 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); + + /* + * After CreateAcls call with + * only RD_KAFKA_ACL_OPERATION_ALTER allowed, the allowed operations + * should be 2 (DESCRIBE is implicitly derived from ALTER). + */ + { + const rd_kafka_AclOperation_t expected[] = { + RD_KAFKA_ACL_OPERATION_ALTER, + RD_KAFKA_ACL_OPERATION_DESCRIBE}; + authorized_operations = + rd_kafka_DescribeCluster_result_authorized_operations( + res, &authorized_operations_cnt); + + test_match_authorized_operations(expected, 2, + authorized_operations, + authorized_operations_cnt); + } + + rd_kafka_event_destroy(rkev); + + /* + * Remove the previously created ACL so that it doesn't affect other + * tests. + */ + acl_bindings_delete = rd_kafka_AclBindingFilter_new( + RD_KAFKA_RESOURCE_BROKER, "kafka-cluster", + RD_KAFKA_RESOURCE_PATTERN_MATCH, principal, "*", + RD_KAFKA_ACL_OPERATION_ALTER, RD_KAFKA_ACL_PERMISSION_TYPE_ALLOW, + NULL, 0); + test_DeleteAcls_simple(rk, NULL, &acl_bindings_delete, 1, NULL); + rd_kafka_AclBinding_destroy(acl_bindings_delete); + +done: + TEST_LATER_CHECK(); + + if (!rkqu) + rd_kafka_queue_destroy(q); + + SUB_TEST_PASS(); +} + +/** + * @brief Test DescribeConsumerGroups's authorized_operations, creating a + * consumer for a group, describing it, changing ACLs, and describing it again. + */ +static void +do_test_DescribeConsumerGroups_with_authorized_ops(const char *what, + rd_kafka_t *rk, + rd_kafka_queue_t *useq, + int request_timeout) { + rd_kafka_queue_t *q; + rd_kafka_AdminOptions_t *options = NULL; + rd_kafka_event_t *rkev = NULL; + rd_kafka_resp_err_t err; + const rd_kafka_error_t *error; + char errstr[512]; + const char *errstr2; +#define TEST_DESCRIBE_CONSUMER_GROUPS_CNT 4 + const int partitions_cnt = 1; + const int msgs_cnt = 100; + char *topic, *group_id; + rd_kafka_AclBinding_t *acl_bindings[TEST_DESCRIBE_CONSUMER_GROUPS_CNT]; + int64_t testid = test_id_generate(); + const rd_kafka_ConsumerGroupDescription_t **results = NULL; + size_t results_cnt; + const rd_kafka_DescribeConsumerGroups_result_t *res; + const char *principal, *sasl_mechanism, *sasl_username; + const rd_kafka_AclOperation_t *authorized_operations; + size_t authorized_operations_cnt; + + SUB_TEST_QUICK("%s DescribeConsumerGroups with %s, request_timeout %d", + rd_kafka_name(rk), what, request_timeout); + + if (!test_needs_auth()) + SUB_TEST_SKIP("Test requires authorization to be setup."); + + sasl_mechanism = test_conf_get(NULL, "sasl.mechanism"); + if (strcmp(sasl_mechanism, "PLAIN") != 0 && + strncmp(sasl_mechanism, "SCRAM", 5) != 0) + SUB_TEST_SKIP("Test requites SASL_PLAIN or SASL_SCRAM, got %s", + sasl_mechanism); + + sasl_username = test_conf_get(NULL, "sasl.username"); + principal = tsprintf("User:%s", sasl_username); + + topic = rd_strdup(test_mk_topic_name(__FUNCTION__, 1)); + + /* Create the topic. */ + test_CreateTopics_simple(rk, NULL, &topic, 1, partitions_cnt, NULL); + test_wait_topic_exists(rk, topic, 10000); + + /* Produce 100 msgs */ + test_produce_msgs_easy(topic, testid, 0, msgs_cnt); + + /* Create and consumer (and consumer group). */ + group_id = rd_strdup(test_mk_topic_name(__FUNCTION__, 1)); + test_consume_msgs_easy(group_id, topic, testid, -1, 100, NULL); + + q = useq ? useq : rd_kafka_queue_new(rk); + + options = rd_kafka_AdminOptions_new( + rk, RD_KAFKA_ADMIN_OP_DESCRIBECONSUMERGROUPS); + + TEST_CALL_ERR__(rd_kafka_AdminOptions_set_request_timeout( + options, request_timeout, errstr, sizeof(errstr))); + TEST_CALL_ERROR__( + rd_kafka_AdminOptions_set_include_authorized_operations(options, + 1)); + + rd_kafka_DescribeConsumerGroups(rk, (const char **)(&group_id), 1, + options, q); + rd_kafka_AdminOptions_destroy(options); + + rkev = test_wait_admin_result( + q, RD_KAFKA_EVENT_DESCRIBECONSUMERGROUPS_RESULT, + tmout_multip(20 * 1000)); + TEST_ASSERT(rkev, "Should receive describe consumer groups event."); + + /* Extract result. */ + res = rd_kafka_event_DescribeConsumerGroups_result(rkev); + TEST_ASSERT(res, "Expected DescribeConsumerGroup 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); + + results = + rd_kafka_DescribeConsumerGroups_result_groups(res, &results_cnt); + TEST_ASSERT((int)results_cnt == 1, "Expected 1 group, got %d", + (int)results_cnt); + + error = rd_kafka_ConsumerGroupDescription_error(results[0]); + TEST_ASSERT(!error, "Expected no error in describing group, got: %s", + rd_kafka_error_string(error)); + + { + const rd_kafka_AclOperation_t expected[] = { + RD_KAFKA_ACL_OPERATION_DELETE, + RD_KAFKA_ACL_OPERATION_DESCRIBE, + RD_KAFKA_ACL_OPERATION_READ}; + authorized_operations = + rd_kafka_ConsumerGroupDescription_authorized_operations( + results[0], &authorized_operations_cnt); + test_match_authorized_operations(expected, 3, + authorized_operations, + authorized_operations_cnt); + } + + rd_kafka_event_destroy(rkev); + + /* Change authorized operations for the principal which we're using to + * connect to the broker. */ + acl_bindings[0] = rd_kafka_AclBinding_new( + RD_KAFKA_RESOURCE_GROUP, group_id, + RD_KAFKA_RESOURCE_PATTERN_LITERAL, principal, "*", + RD_KAFKA_ACL_OPERATION_READ, RD_KAFKA_ACL_PERMISSION_TYPE_ALLOW, + NULL, 0); + test_CreateAcls_simple(rk, NULL, acl_bindings, 1, NULL); + rd_kafka_AclBinding_destroy(acl_bindings[0]); + + /* It seems to be taking some time on the cluster for the ACLs to + * propagate for a group.*/ + rd_sleep(tmout_multip(2)); + + options = rd_kafka_AdminOptions_new( + rk, RD_KAFKA_ADMIN_OP_DESCRIBECONSUMERGROUPS); + + TEST_CALL_ERR__(rd_kafka_AdminOptions_set_request_timeout( + options, request_timeout, errstr, sizeof(errstr))); + TEST_CALL_ERROR__( + rd_kafka_AdminOptions_set_include_authorized_operations(options, + 1)); + + rd_kafka_DescribeConsumerGroups(rk, (const char **)(&group_id), 1, + options, q); + rd_kafka_AdminOptions_destroy(options); + + rkev = test_wait_admin_result( + q, RD_KAFKA_EVENT_DESCRIBECONSUMERGROUPS_RESULT, + tmout_multip(20 * 1000)); + TEST_ASSERT(rkev, "Should receive describe consumer groups event."); + + /* Extract result. */ + res = rd_kafka_event_DescribeConsumerGroups_result(rkev); + TEST_ASSERT(res, "Expected DescribeConsumerGroup 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); + + results = + rd_kafka_DescribeConsumerGroups_result_groups(res, &results_cnt); + TEST_ASSERT((int)results_cnt == 1, "Expected 1 group, got %d", + (int)results_cnt); + + error = rd_kafka_ConsumerGroupDescription_error(results[0]); + TEST_ASSERT(!error, "Expected no error in describing group, got: %s", + rd_kafka_error_string(error)); + + + { + const rd_kafka_AclOperation_t expected[] = { + RD_KAFKA_ACL_OPERATION_DESCRIBE, + RD_KAFKA_ACL_OPERATION_READ}; + authorized_operations = + rd_kafka_ConsumerGroupDescription_authorized_operations( + results[0], &authorized_operations_cnt); + test_match_authorized_operations(expected, 2, + authorized_operations, + authorized_operations_cnt); + } + + rd_kafka_event_destroy(rkev); + + acl_bindings[0] = rd_kafka_AclBinding_new( + RD_KAFKA_RESOURCE_GROUP, group_id, + RD_KAFKA_RESOURCE_PATTERN_LITERAL, principal, "*", + RD_KAFKA_ACL_OPERATION_DELETE, RD_KAFKA_ACL_PERMISSION_TYPE_ALLOW, + NULL, 0); + test_CreateAcls_simple(rk, NULL, acl_bindings, 1, NULL); + rd_kafka_AclBinding_destroy(acl_bindings[0]); + + /* It seems to be taking some time on the cluster for the ACLs to + * propagate for a group.*/ + rd_sleep(tmout_multip(2)); + + test_DeleteGroups_simple(rk, NULL, &group_id, 1, NULL); + test_DeleteTopics_simple(rk, q, &topic, 1, NULL); + + rd_free(topic); + rd_free(group_id); + + if (!useq) + rd_kafka_queue_destroy(q); + + + TEST_LATER_CHECK(); +#undef TEST_DESCRIBE_CONSUMER_GROUPS_CNT + + SUB_TEST_PASS(); +} /** * @brief Test deletion of committed offsets. * @@ -4295,6 +5011,28 @@ 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, rd_false); + do_test_DescribeTopics("main queue", rk, mainq, 15000, rd_false); + + /* Describe cluster */ + do_test_DescribeCluster("temp queue", rk, NULL, 1500, rd_false); + do_test_DescribeCluster("main queue", rk, mainq, 1500, rd_false); + + if (test_broker_version >= TEST_BRKVER(2, 3, 0, 0)) { + /* Describe topics */ + do_test_DescribeTopics("temp queue", rk, NULL, 15000, rd_true); + do_test_DescribeTopics("main queue", rk, mainq, 15000, rd_true); + + do_test_DescribeCluster("temp queue", rk, NULL, 1500, rd_true); + do_test_DescribeCluster("main queue", rk, mainq, 1500, rd_true); + + do_test_DescribeConsumerGroups_with_authorized_ops( + "temp queue", rk, NULL, 1500); + do_test_DescribeConsumerGroups_with_authorized_ops( + "main queue", rk, mainq, 1500); + } + /* Delete groups */ do_test_DeleteGroups("temp queue", rk, NULL, -1); do_test_DeleteGroups("main queue", rk, mainq, 1500); diff --git a/tests/test.c b/tests/test.c index 000e3badab..5a9c4fce89 100644 --- a/tests/test.c +++ b/tests/test.c @@ -5872,13 +5872,15 @@ rd_kafka_resp_err_t test_wait_topic_admin_result(rd_kafka_queue_t *q, size_t aclres_cnt = 0; int errcnt = 0; rd_kafka_resp_err_t err; - const rd_kafka_group_result_t **gres = NULL; - size_t gres_cnt = 0; - const rd_kafka_ConsumerGroupDescription_t **gdescs = NULL; - size_t gdescs_cnt = 0; - const rd_kafka_error_t **glists_errors = NULL; - size_t glists_error_cnt = 0; - const rd_kafka_topic_partition_list_t *offsets = NULL; + const rd_kafka_group_result_t **gres = NULL; + size_t gres_cnt = 0; + const rd_kafka_ConsumerGroupDescription_t **gdescs = NULL; + size_t gdescs_cnt = 0; + const rd_kafka_error_t **glists_errors = NULL; + size_t glists_error_cnt = 0; + const rd_kafka_topic_partition_list_t *offsets = NULL; + const rd_kafka_DeleteAcls_result_response_t **delete_aclres = NULL; + size_t delete_aclres_cnt = 0; rkev = test_wait_admin_result(q, evtype, tmout); @@ -5951,6 +5953,15 @@ rd_kafka_resp_err_t test_wait_topic_admin_result(rd_kafka_queue_t *q, rd_kafka_event_name(rkev)); aclres = rd_kafka_CreateAcls_result_acls(res, &aclres_cnt); + } else if (evtype == RD_KAFKA_EVENT_DELETEACLS_RESULT) { + const rd_kafka_DeleteAcls_result_t *res; + + if (!(res = rd_kafka_event_DeleteAcls_result(rkev))) + TEST_FAIL("Expected a DeleteAcls result, not %s", + rd_kafka_event_name(rkev)); + + delete_aclres = rd_kafka_DeleteAcls_result_responses( + res, &delete_aclres_cnt); } else if (evtype == RD_KAFKA_EVENT_LISTCONSUMERGROUPS_RESULT) { const rd_kafka_ListConsumerGroups_result_t *res; if (!(res = rd_kafka_event_ListConsumerGroups_result(rkev))) @@ -6111,6 +6122,20 @@ rd_kafka_resp_err_t test_wait_topic_admin_result(rd_kafka_queue_t *q, } } + /* Check delete ACL errors. */ + for (i = 0; i < delete_aclres_cnt; i++) { + const rd_kafka_DeleteAcls_result_response_t *res_resp = + delete_aclres[i]; + const rd_kafka_error_t *error = + rd_kafka_DeleteAcls_result_response_error(res_resp); + if (error) { + TEST_WARN("DeleteAcls result error: %s\n", + rd_kafka_error_string(error)); + if ((errcnt++) == 0) + err = rd_kafka_error_code(error); + } + } + if (!err && retevent) *retevent = rkev; else @@ -6298,7 +6323,7 @@ rd_kafka_resp_err_t test_DeleteTopics_simple(rd_kafka_t *rk, TEST_SAY("Deleting %" PRIusz " topics\n", topic_cnt); - rd_kafka_DeleteTopics(rk, del_topics, topic_cnt, options, useq); + rd_kafka_DeleteTopics(rk, del_topics, topic_cnt, options, q); rd_kafka_AdminOptions_destroy(options); @@ -6724,6 +6749,56 @@ rd_kafka_resp_err_t test_CreateAcls_simple(rd_kafka_t *rk, return err; } +/** + * @brief Topic Admin API helpers + * + * @param useq Makes the call async and posts the response in this queue. + * If NULL this call will be synchronous and return the error + * result. + * + * @remark Fails the current test on failure. + */ + +rd_kafka_resp_err_t +test_DeleteAcls_simple(rd_kafka_t *rk, + rd_kafka_queue_t *useq, + rd_kafka_AclBindingFilter_t **acl_filters, + size_t acl_filters_cnt, + void *opaque) { + rd_kafka_AdminOptions_t *options; + rd_kafka_queue_t *q; + rd_kafka_resp_err_t err; + const int tmout = 30 * 1000; + + options = rd_kafka_AdminOptions_new(rk, RD_KAFKA_ADMIN_OP_DELETEACLS); + rd_kafka_AdminOptions_set_opaque(options, opaque); + + if (!useq) { + q = rd_kafka_queue_new(rk); + } else { + q = useq; + } + + TEST_SAY("Deleting acls using %" PRIusz " filters\n", acl_filters_cnt); + + rd_kafka_DeleteAcls(rk, acl_filters, acl_filters_cnt, options, q); + + rd_kafka_AdminOptions_destroy(options); + + if (useq) + return RD_KAFKA_RESP_ERR_NO_ERROR; + + err = test_wait_topic_admin_result(q, RD_KAFKA_EVENT_DELETEACLS_RESULT, + NULL, tmout + 5000); + + rd_kafka_queue_destroy(q); + + if (err) + TEST_FAIL("Failed to delete acl(s): %s", rd_kafka_err2str(err)); + + return err; +} + static void test_free_string_array(char **strs, size_t cnt) { size_t i; for (i = 0; i < cnt; i++) diff --git a/tests/test.h b/tests/test.h index a1f5cc2cb6..671472b43a 100644 --- a/tests/test.h +++ b/tests/test.h @@ -833,6 +833,13 @@ rd_kafka_resp_err_t test_CreateAcls_simple(rd_kafka_t *rk, size_t acl_cnt, void *opaque); +rd_kafka_resp_err_t +test_DeleteAcls_simple(rd_kafka_t *rk, + rd_kafka_queue_t *useq, + rd_kafka_AclBindingFilter_t **acl_filters, + size_t acl_filters_cnt, + void *opaque); + rd_kafka_resp_err_t test_delete_all_test_topics(int timeout_ms); void test_mock_cluster_destroy(rd_kafka_mock_cluster_t *mcluster);