From 069b8e8f8223f1fded0c46e445b4b10ace2af856 Mon Sep 17 00:00:00 2001 From: jainruchir Date: Fri, 24 Feb 2023 14:32:49 +0530 Subject: [PATCH 01/13] include authorized topic/cluster ops initial draft --- examples/describe_consumer_groups.c | 90 +++++++++++++++++++++++++++-- src/rdkafka_cgrp.c | 4 ++ src/rdkafka_metadata.c | 29 +++++++++- src/rdkafka_metadata.h | 15 ++++- src/rdkafka_op.h | 3 + src/rdkafka_request.c | 17 ++++-- src/rdkafka_request.h | 2 + 7 files changed, 149 insertions(+), 11 deletions(-) diff --git a/examples/describe_consumer_groups.c b/examples/describe_consumer_groups.c index 45b6b8d0b0..00608eac27 100644 --- a/examples/describe_consumer_groups.c +++ b/examples/describe_consumer_groups.c @@ -49,7 +49,7 @@ const char *argv0; - +typedef struct rd_kafka_topic_authorized_operations_pair rd_kafka_topic_authorized_operations_pair_t; static rd_kafka_queue_t *queue; /** Admin result queue. * This is a global so we can * yield in stop() */ @@ -236,6 +236,85 @@ print_groups_info(const rd_kafka_DescribeConsumerGroups_result_t *grpdesc, } return 0; } +/** + * @brief Parse an integer or fail. + */ +int64_t parse_int(const char *what, const char *str) { + char *end; + unsigned long n = strtoull(str, &end, 0); + + if (end != str + strlen(str)) { + fprintf(stderr, "%% Invalid input for %s: %s: not an integer\n", + what, str); + exit(1); + } + + return (int64_t)n; +} +/** + * @brief Call rd_kafka_DescribeConsumerGroups() with a list of + * groups. + */ +static void +cmd_metadata_request(rd_kafka_conf_t *conf, int argc, char **argv) { + rd_kafka_t *rk; + rd_kafka_topic_conf_t *topic_conf; + rd_kafka_topic_t *rkt; + rd_kafka_topic_authorized_operations_pair_t* res; + const struct rd_kafka_metadata *metadatap; + const char **topics = NULL; + char errstr[512]; + int retval = 0; + int topic_cnt = 0; + int i; + + int allow_auto_topic_creation, include_cluster_authorized_operations, + include_topic_authorized_operations; + + metadatap = malloc(sizeof(*metadatap)); + allow_auto_topic_creation = + parse_int("allow_auto_topic_creation", argv[0]); + include_cluster_authorized_operations = + parse_int("include_cluster_authorized_operations", argv[1]); + include_topic_authorized_operations = + parse_int("include_topic_authorized_operations", argv[2]); + + if (allow_auto_topic_creation < 0 || + allow_auto_topic_creation > 1) + usage("Require allow_auto_topic_creation not a 0-1 int"); + + if (include_cluster_authorized_operations < 0 || + include_cluster_authorized_operations > 1) + usage("Require include_cluster_authorized_operations not a 0-1 int"); + + if (include_topic_authorized_operations < 0 || + include_topic_authorized_operations > 1) + usage("Require include_topic_authorized_operations not a 0-1 int"); + + + if (argc >= 1) { + topics = (const char **)&argv[3]; + topic_cnt = argc - 3; + } + + // topic_conf = rd_kafka_topic_conf_new(); + // rkt = rd_kafka_topic_new(rk, topics[0], topic_conf); + + rk = rd_kafka_new(RD_KAFKA_CONSUMER, conf, errstr, sizeof(errstr)); + if (!rk) + fatal("Failed to create new consumer: %s", errstr); + + rd_kafka_metadata(rk, 1, NULL, &metadatap, 100000); + //rd_kafka_metadata(rk, 0, rkt, &metadatap, 100000); + if(metadatap->orig_broker_name) + printf("%s\n", metadatap->orig_broker_name); + printf("Topic count: %d\n", metadatap->topic_cnt); +exit: + /* Destroy the client instance */ + rd_kafka_destroy(rk); + + exit(retval); +} /** * @brief Call rd_kafka_DescribeConsumerGroups() with a list of @@ -334,7 +413,10 @@ int main(int argc, char **argv) { * Create Kafka client configuration place-holder */ conf = rd_kafka_conf_new(); - + conf_set(conf, "sasl.username", "broker"); + conf_set(conf, "sasl.password", "broker"); + conf_set(conf, "sasl.mechanism", "SCRAM-SHA-256"); + conf_set(conf, "security.protocol", "SASL_PLAINTEXT"); /* * Parse common options @@ -367,7 +449,7 @@ int main(int argc, char **argv) { } } - cmd_describe_consumer_groups(conf, argc - optind, &argv[optind]); - + //cmd_describe_consumer_groups(conf, argc - optind, &argv[optind]); + cmd_metadata_request(conf, argc - optind, &argv[optind]); return 0; } diff --git a/src/rdkafka_cgrp.c b/src/rdkafka_cgrp.c index 026e933210..a1004e5833 100644 --- a/src/rdkafka_cgrp.c +++ b/src/rdkafka_cgrp.c @@ -2024,6 +2024,8 @@ static void rd_kafka_cgrp_handle_JoinGroup(rd_kafka_t *rk, rd_kafka_MetadataRequest( rkb, &topics, "partition assignor", rd_false /*!allow_auto_create*/, + rd_true /*include cluster authorized operations */, + rd_true /*include topic authorized operations */, /* cgrp_update=false: * Since the subscription list may not be identical * across all members of the group and thus the @@ -2220,6 +2222,8 @@ static int rd_kafka_cgrp_metadata_refresh(rd_kafka_cgrp_t *rkcg, err = rd_kafka_metadata_request(rkcg->rkcg_rk, NULL, &topics, rd_false /*!allow auto create */, + rd_true /*include cluster authorized operations */, + rd_true /*include topic authorized operations */, rd_true /*cgrp_update*/, reason, rko); if (err) { rd_kafka_dbg(rk, CGRP | RD_KAFKA_DBG_METADATA, "CGRPMETADATA", diff --git a/src/rdkafka_metadata.c b/src/rdkafka_metadata.c index 4e32e5d584..aabbd92035 100644 --- a/src/rdkafka_metadata.c +++ b/src/rdkafka_metadata.c @@ -84,6 +84,8 @@ rd_kafka_metadata(rd_kafka_t *rk, * of outstanding metadata requests. */ rd_kafka_MetadataRequest(rkb, &topics, "application requested", allow_auto_create_topics, + rd_true /*include cluster authorized operations */, + rd_true /*include topic authorized operations */, /* cgrp_update: * Only update consumer group state * on response if this lists all @@ -285,7 +287,9 @@ 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, - struct rd_kafka_metadata **mdp) { + struct rd_kafka_metadata **mdp, + rd_kafka_topic_authorized_operations_pair_t** topic_authorized_operations, + int32_t* cluster_authorized_operations) { rd_kafka_t *rk = rkb->rkb_rk; int i, j, k; rd_tmpabuf_t tbuf; @@ -394,6 +398,10 @@ rd_kafka_resp_err_t rd_kafka_parse_Metadata(rd_kafka_broker_t *rkb, rd_kafka_buf_parse_fail( rkbuf, "%d topics: tmpabuf memory shortage", md->topic_cnt); + if ((ApiVersion >= 8 && ApiVersion <= 10) && topic_authorized_operations){ + *topic_authorized_operations = rd_malloc(sizeof(rd_kafka_topic_authorized_operations_pair_t) * md->topic_cnt); + } + for (i = 0; i < md->topic_cnt; i++) { rd_kafka_buf_read_i16a(rkbuf, md->topics[i].err); rd_kafka_buf_read_str_tmpabuf(rkbuf, &tbuf, @@ -516,6 +524,11 @@ rd_kafka_resp_err_t rd_kafka_parse_Metadata(rd_kafka_broker_t *rkb, /* TopicAuthorizedOperations */ rd_kafka_buf_read_i32(rkbuf, &TopicAuthorizedOperations); + (*topic_authorized_operations)[i].topic_name = md->topics[i].topic; + (*topic_authorized_operations)[i].authorized_operations = TopicAuthorizedOperations; + printf("%s topic has %d authorized operations\n", + (*topic_authorized_operations)[i].topic_name, + (*topic_authorized_operations)[i].authorized_operations); } rd_kafka_buf_skip_tags(rkbuf); @@ -588,6 +601,8 @@ rd_kafka_resp_err_t rd_kafka_parse_Metadata(rd_kafka_broker_t *rkb, int32_t ClusterAuthorizedOperations; /* ClusterAuthorizedOperations */ rd_kafka_buf_read_i32(rkbuf, &ClusterAuthorizedOperations); + *cluster_authorized_operations = ClusterAuthorizedOperations; + printf("Cluster authorized operations: %d\n", ClusterAuthorizedOperations); } rd_kafka_buf_skip_tags(rkbuf); @@ -1041,6 +1056,8 @@ rd_kafka_metadata_refresh_topics(rd_kafka_t *rk, rd_list_cnt(&q_topics), rd_list_cnt(topics), reason); rd_kafka_MetadataRequest(rkb, &q_topics, reason, allow_auto_create, + rd_true /*include cluster authorized operations */, + rd_true /*include topic authorized operations */, cgrp_update, NULL); rd_list_destroy(&q_topics); @@ -1183,6 +1200,8 @@ rd_kafka_resp_err_t rd_kafka_metadata_refresh_brokers(rd_kafka_t *rk, const char *reason) { return rd_kafka_metadata_request(rk, rkb, NULL /*brokers only*/, rd_false /*!allow auto create topics*/, + rd_true /*include cluster authorized operations */, + rd_true /*include topic authorized operations */, rd_false /*no cgrp update */, reason, NULL); } @@ -1218,6 +1237,8 @@ rd_kafka_resp_err_t rd_kafka_metadata_refresh_all(rd_kafka_t *rk, rd_list_init(&topics, 0, NULL); /* empty list = all topics */ rd_kafka_MetadataRequest(rkb, &topics, reason, rd_false /*no auto create*/, + rd_true /*include cluster authorized operations */, + rd_true /*include topic authorized operations */, rd_true /*cgrp update*/, NULL); rd_list_destroy(&topics); @@ -1243,6 +1264,8 @@ rd_kafka_metadata_request(rd_kafka_t *rk, rd_kafka_broker_t *rkb, const rd_list_t *topics, rd_bool_t allow_auto_create_topics, + rd_bool_t include_cluster_authorized_operations, + rd_bool_t include_topic_authorized_operations, rd_bool_t cgrp_update, const char *reason, rd_kafka_op_t *rko) { @@ -1255,7 +1278,9 @@ rd_kafka_metadata_request(rd_kafka_t *rk, destroy_rkb = 1; } - rd_kafka_MetadataRequest(rkb, topics, reason, allow_auto_create_topics, + rd_kafka_MetadataRequest(rkb, topics, reason, allow_auto_create_topics, + include_cluster_authorized_operations, + include_topic_authorized_operations, cgrp_update, rko); if (destroy_rkb) diff --git a/src/rdkafka_metadata.h b/src/rdkafka_metadata.h index 53a959b8ec..297aa24191 100644 --- a/src/rdkafka_metadata.h +++ b/src/rdkafka_metadata.h @@ -36,7 +36,9 @@ 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, - struct rd_kafka_metadata **mdp); + struct rd_kafka_metadata **mdp, + rd_kafka_topic_authorized_operations_pair_t** topic_authorized_operations, + int32_t* cluster_authorized_operations); struct rd_kafka_metadata * rd_kafka_metadata_copy(const struct rd_kafka_metadata *md, size_t size); @@ -87,6 +89,8 @@ rd_kafka_metadata_request(rd_kafka_t *rk, rd_kafka_broker_t *rkb, const rd_list_t *topics, rd_bool_t allow_auto_create_topics, + rd_bool_t include_cluster_authorized_operations, + rd_bool_t include_topic_authorized_operations, rd_bool_t cgrp_update, const char *reason, rd_kafka_op_t *rko); @@ -100,7 +104,16 @@ rd_kafka_metadata_new_topic_mock(const rd_kafka_metadata_topic_t *topics, size_t topic_cnt); rd_kafka_metadata_t *rd_kafka_metadata_new_topic_mockv(size_t topic_cnt, ...); +/** + * @{ + * + * @brief Metadata topic name and authorized operations pair + */ +typedef struct rd_kafka_topic_authorized_operations_pair { + char* topic_name; + int32_t authorized_operations; +}rd_kafka_topic_authorized_operations_pair; /** * @{ * diff --git a/src/rdkafka_op.h b/src/rdkafka_op.h index 57c07491a2..aa32750e78 100644 --- a/src/rdkafka_op.h +++ b/src/rdkafka_op.h @@ -38,6 +38,7 @@ typedef struct rd_kafka_q_s rd_kafka_q_t; typedef struct rd_kafka_toppar_s rd_kafka_toppar_t; typedef struct rd_kafka_op_s rd_kafka_op_t; +typedef struct rd_kafka_topic_authorized_operations_pair rd_kafka_topic_authorized_operations_pair_t; /* One-off reply queue + reply version. * All APIs that take a rd_kafka_replyq_t makes a copy of the @@ -370,6 +371,8 @@ struct rd_kafka_op_s { /* RD_KAFKA_OP_METADATA */ struct { rd_kafka_metadata_t *md; + rd_kafka_topic_authorized_operations_pair_t* topic_authorized_operations; + int32_t cluster_authorized_operations; int force; /* force request regardless of outstanding * metadata requests. */ } metadata; diff --git a/src/rdkafka_request.c b/src/rdkafka_request.c index 12d9eb30e0..cc03ce691f 100644 --- a/src/rdkafka_request.c +++ b/src/rdkafka_request.c @@ -2086,6 +2086,8 @@ static void rd_kafka_handle_Metadata(rd_kafka_t *rk, void *opaque) { rd_kafka_op_t *rko = opaque; /* Possibly NULL */ struct rd_kafka_metadata *md = NULL; + rd_kafka_topic_authorized_operations_pair_t* topic_authorized_operations = NULL; + int32_t cluster_authorized_operations; const rd_list_t *topics = request->rkbuf_u.Metadata.topics; int actions; @@ -2113,7 +2115,7 @@ static void rd_kafka_handle_Metadata(rd_kafka_t *rk, rd_list_cnt(topics), request->rkbuf_u.Metadata.reason); - err = rd_kafka_parse_Metadata(rkb, request, rkbuf, &md); + err = rd_kafka_parse_Metadata(rkb, request, rkbuf, &md, &topic_authorized_operations,&cluster_authorized_operations); if (err) goto err; @@ -2122,7 +2124,8 @@ static void rd_kafka_handle_Metadata(rd_kafka_t *rk, * Reuse requesting rko for the reply. */ rko->rko_err = err; rko->rko_u.metadata.md = md; - + rko->rko_u.metadata.cluster_authorized_operations = cluster_authorized_operations; + rko->rko_u.metadata.topic_authorized_operations = topic_authorized_operations; rd_kafka_replyq_enq(&rko->rko_replyq, rko, 0); rko = NULL; } else { @@ -2155,6 +2158,8 @@ static void rd_kafka_handle_Metadata(rd_kafka_t *rk, if (rko && rko->rko_replyq.q) { rko->rko_err = err; rko->rko_u.metadata.md = NULL; + rko->rko_u.metadata.topic_authorized_operations = NULL; + rko->rko_u.metadata.cluster_authorized_operations = -1; rd_kafka_replyq_enq(&rko->rko_replyq, rko, 0); rko = NULL; } @@ -2199,6 +2204,8 @@ rd_kafka_resp_err_t rd_kafka_MetadataRequest(rd_kafka_broker_t *rkb, const rd_list_t *topics, const char *reason, rd_bool_t allow_auto_create_topics, + rd_bool_t include_cluster_authorized_operations, + rd_bool_t include_topic_authorized_operations, rd_bool_t cgrp_update, rd_kafka_op_t *rko) { rd_kafka_buf_t *rkbuf; @@ -2332,13 +2339,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); + printf("include cluster authorized operations is: %d\n", include_cluster_authorized_operations); + 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); + printf("include topic authorized operations is: %d\n", include_topic_authorized_operations); + rd_kafka_buf_write_bool(rkbuf, include_topic_authorized_operations); } rd_kafka_buf_ApiVersion_set(rkbuf, ApiVersion, 0); diff --git a/src/rdkafka_request.h b/src/rdkafka_request.h index 3eda6be61c..b1a9a4df8c 100644 --- a/src/rdkafka_request.h +++ b/src/rdkafka_request.h @@ -257,6 +257,8 @@ rd_kafka_resp_err_t rd_kafka_MetadataRequest(rd_kafka_broker_t *rkb, const rd_list_t *topics, const char *reason, rd_bool_t allow_auto_create_topics, + rd_bool_t include_cluster_authorized_operations, + rd_bool_t include_topic_authorized_operations, rd_bool_t cgrp_update, rd_kafka_op_t *rko); From 8df6f625d7cfbb288c5a59e637347cdaab520cf4 Mon Sep 17 00:00:00 2001 From: jainruchir Date: Tue, 14 Mar 2023 14:04:44 +0530 Subject: [PATCH 02/13] describe topics and cluster --- examples/.gitignore | 2 + examples/Makefile | 10 + examples/README.md | 2 + examples/describe_cluster.c | 317 +++++++++++ examples/describe_consumer_groups.c | 69 +-- examples/describe_topics.c | 377 +++++++++++++ src/rdkafka.h | 436 +++++++++++++++ src/rdkafka_admin.c | 809 ++++++++++++++++++++++++++++ src/rdkafka_admin.h | 49 ++ src/rdkafka_cgrp.c | 2 +- src/rdkafka_event.c | 22 + src/rdkafka_event.h | 2 + src/rdkafka_metadata.c | 47 +- src/rdkafka_metadata.h | 5 +- src/rdkafka_op.c | 6 + src/rdkafka_op.h | 6 + src/rdkafka_request.c | 31 +- src/rdkafka_request.h | 5 +- tests/0080-admin_ut.c | 258 +++++++++ tests/0081-admin.c | 661 +++++++++++++++++++---- 20 files changed, 2920 insertions(+), 196 deletions(-) create mode 100644 examples/describe_cluster.c create mode 100644 examples/describe_topics.c diff --git a/examples/.gitignore b/examples/.gitignore index 4190608c42..a19747b5cd 100644 --- a/examples/.gitignore +++ b/examples/.gitignore @@ -15,5 +15,7 @@ rdkafka_performance transactions list_consumer_groups describe_consumer_groups +describe_topics +describe_cluster list_consumer_group_offsets alter_consumer_group_offsets diff --git a/examples/Makefile b/examples/Makefile index 15fba3c2af..b8d39643d2 100644 --- a/examples/Makefile +++ b/examples/Makefile @@ -6,6 +6,8 @@ EXAMPLES ?= rdkafka_example rdkafka_performance rdkafka_example_cpp \ openssl_engine_example_cpp \ list_consumer_groups \ describe_consumer_groups \ + describe_topics \ + describe_cluster \ list_consumer_group_offsets \ alter_consumer_group_offsets \ misc @@ -72,6 +74,14 @@ describe_consumer_groups: ../src/librdkafka.a describe_consumer_groups.c $(CC) $(CPPFLAGS) $(CFLAGS) $@.c -o $@ $(LDFLAGS) \ ../src/librdkafka.a $(LIBS) +describe_topics: ../src/librdkafka.a describe_topics.c + $(CC) $(CPPFLAGS) $(CFLAGS) $@.c -o $@ $(LDFLAGS) \ + ../src/librdkafka.a $(LIBS) + +describe_cluster: ../src/librdkafka.a describe_cluster.c + $(CC) $(CPPFLAGS) $(CFLAGS) $@.c -o $@ $(LDFLAGS) \ + ../src/librdkafka.a $(LIBS) + list_consumer_group_offsets: ../src/librdkafka.a list_consumer_group_offsets.c $(CC) $(CPPFLAGS) $(CFLAGS) $@.c -o $@ $(LDFLAGS) \ ../src/librdkafka.a $(LIBS) diff --git a/examples/README.md b/examples/README.md index 3caee3b861..ce05ab5be7 100644 --- a/examples/README.md +++ b/examples/README.md @@ -34,5 +34,7 @@ For more complex uses, see: * [delete_records.c](delete_records.c) - Delete records. * [list_consumer_groups.c](list_consumer_groups.c) - List consumer groups. * [describe_consumer_groups.c](describe_consumer_groups.c) - Describe consumer groups. + * [describe_topics.c](describe_topics.c) - Describe topics. + * [describe_cluster.c](describe_cluster.c) - Describe cluster. * [list_consumer_group_offsets.c](list_consumer_group_offsets.c) - List offsets of a consumer group. * [alter_consumer_group_offsets.c](alter_consumer_group_offsets.c) - Alter offsets of a consumer group. diff --git a/examples/describe_cluster.c b/examples/describe_cluster.c new file mode 100644 index 0000000000..968b8b591e --- /dev/null +++ b/examples/describe_cluster.c @@ -0,0 +1,317 @@ +/* + * librdkafka - Apache Kafka C library + * + * Copyright (c) 2022, Magnus Edenhill + * All rights reserved. + * + * Redistribution and use in source and binary forms, with or without + * modification, are permitted provided that the following conditions are met: + * + * 1. Redistributions of source code must retain the above copyright notice, + * this list of conditions and the following disclaimer. + * 2. Redistributions in binary form must reproduce the above copyright notice, + * this list of conditions and the following disclaimer in the documentation + * and/or other materials provided with the distribution. + * + * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" + * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE + * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE + * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE + * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR + * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF + * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS + * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN + * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) + * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE + * POSSIBILITY OF SUCH DAMAGE. + */ + +/** + * DescribeCluster usage example. + */ + +#include +#include +#include +#include +#include + +#ifdef _WIN32 +#include "../win32/wingetopt.h" +#else +#include +#endif + + +/* Typical include path would be , but this program + * is builtin from within the librdkafka source tree and thus differs. */ +#include "rdkafka.h" + + +const char *argv0; +static rd_kafka_queue_t *queue; /** Admin result queue. + * This is a global so we can + * yield in stop() */ +static volatile sig_atomic_t run = 1; + +/** + * @brief Signal termination of program + */ +static void stop(int sig) { + if (!run) { + fprintf(stderr, "%% Forced termination\n"); + exit(2); + } + run = 0; + rd_kafka_queue_yield(queue); +} + + +static void usage(const char *reason, ...) { + + fprintf(stderr, + "Describe cluster usage examples\n" + "\n" + "Usage: %s ...\n" + "\n" + "Options:\n" + " -b Bootstrap server list to connect to.\n" + " -X Set librdkafka configuration property.\n" + " See CONFIGURATION.md for full list.\n" + " -d Enable librdkafka debugging (%s).\n" + "\n", + argv0, rd_kafka_get_debug_contexts()); + + if (reason) { + va_list ap; + char reasonbuf[512]; + + va_start(ap, reason); + vsnprintf(reasonbuf, sizeof(reasonbuf), reason, ap); + va_end(ap); + + fprintf(stderr, "ERROR: %s\n", reasonbuf); + } + + exit(reason ? 1 : 0); +} + + +#define fatal(...) \ + do { \ + fprintf(stderr, "ERROR: "); \ + fprintf(stderr, __VA_ARGS__); \ + fprintf(stderr, "\n"); \ + exit(2); \ + } while (0) + + +/** + * @brief Set config property. Exit on failure. + */ +static void conf_set(rd_kafka_conf_t *conf, const char *name, const char *val) { + char errstr[512]; + + if (rd_kafka_conf_set(conf, name, val, errstr, sizeof(errstr)) != + RD_KAFKA_CONF_OK) + fatal("Failed to set %s=%s: %s", name, val, errstr); +} + +/** + * @brief Parse an integer or fail. + */ +int64_t parse_int(const char *what, const char *str) { + char *end; + unsigned long n = strtoull(str, &end, 0); + + if (end != str + strlen(str)) { + fprintf(stderr, "%% Invalid input for %s: %s: not an integer\n", + what, str); + exit(1); + } + + return (int64_t)n; +} + +/** + * @brief Print cluster information. + */ +static int +print_cluster_info(const rd_kafka_DescribeCluster_result_t *clusterdesc) { + size_t i; + int j, acl_operation; + const rd_kafka_ClusterDescription_t *desc; + int controller_id, node_cnt, cluster_authorized_operations_cnt; + const char* cluster_id; + + desc = rd_kafka_DescribeCluster_result_description(clusterdesc); + + controller_id = rd_kafka_ClusterDescription_controller_id(desc); + node_cnt = rd_kafka_ClusterDescription_node_cnt(desc); + cluster_authorized_operations_cnt = rd_kafka_ClusterDescription_cluster_acl_operations_cnt(desc); + cluster_id = rd_kafka_ClusterDescription_cluster_id(desc); + + printf("Cluster id: %s\t Controller id: %d\t ACL operations count allowed: %d\n", + cluster_id, controller_id, cluster_authorized_operations_cnt); + for(j=0;j 1) + usage("include_cluster_authorized_operations not a 0-1 int"); + + /* + * Create consumer instance + * NOTE: rd_kafka_new() takes ownership of the conf object + * and the application must not reference it again after + * this call. + */ + rk = rd_kafka_new(RD_KAFKA_CONSUMER, conf, errstr, sizeof(errstr)); + if (!rk) + fatal("Failed to create new consumer: %s", errstr); + + /* + * Describe cluster + */ + queue = rd_kafka_queue_new(rk); + + /* Signal handler for clean shutdown */ + signal(SIGINT, stop); + + options = rd_kafka_AdminOptions_new( + rk, RD_KAFKA_ADMIN_OP_DESCRIBECLUSTER); + + if (rd_kafka_AdminOptions_set_request_timeout( + options, 10 * 1000 /* 10s */, errstr, sizeof(errstr))) { + fprintf(stderr, "%% Failed to set timeout: %s\n", errstr); + goto exit; + } + if ((error = rd_kafka_AdminOptions_set_include_cluster_authorized_operations( + options, include_cluster_authorized_operations))) { + fprintf(stderr, + "%% Failed to set require cluster authorized operations: %s\n", + rd_kafka_error_string(error)); + rd_kafka_error_destroy(error); + exit(1); + } + + rd_kafka_DescribeCluster(rk, options, queue); + + /* Wait for results */ + event = rd_kafka_queue_poll(queue, -1 /* indefinitely but limited by + * the request timeout set + * above (10s) */); + + if (!event) { + /* User hit Ctrl-C, + * see yield call in stop() signal handler */ + fprintf(stderr, "%% Cancelled by user\n"); + + } else if (rd_kafka_event_error(event)) { + rd_kafka_resp_err_t err = rd_kafka_event_error(event); + /* DescribeCluster request failed */ + fprintf(stderr, + "%% DescribeCluster failed[%" PRId32 "]: %s\n", + err, rd_kafka_event_error_string(event)); + goto exit; + + } else { + /* DescribeCluster request succeeded */ + const rd_kafka_DescribeCluster_result_t *result; + + result = rd_kafka_event_DescribeCluster_result(event); + printf("DescribeCluster results:\n"); + retval = print_cluster_info(result); + } + + +exit: + if (event) + rd_kafka_event_destroy(event); + rd_kafka_AdminOptions_destroy(options); + rd_kafka_queue_destroy(queue); + /* Destroy the client instance */ + rd_kafka_destroy(rk); + + exit(retval); +} + +int main(int argc, char **argv) { + rd_kafka_conf_t *conf; /**< Client configuration object */ + int opt; + argv0 = argv[0]; + + /* + * Create Kafka client configuration place-holder + */ + conf = rd_kafka_conf_new(); + conf_set(conf, "sasl.username", "broker"); + conf_set(conf, "sasl.password", "broker"); + conf_set(conf, "sasl.mechanism", "SCRAM-SHA-256"); + conf_set(conf, "security.protocol", "SASL_PLAINTEXT"); + + /* + * 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 00608eac27..e93b059cf8 100644 --- a/examples/describe_consumer_groups.c +++ b/examples/describe_consumer_groups.c @@ -49,7 +49,6 @@ const char *argv0; -typedef struct rd_kafka_topic_authorized_operations_pair rd_kafka_topic_authorized_operations_pair_t; static rd_kafka_queue_t *queue; /** Admin result queue. * This is a global so we can * yield in stop() */ @@ -251,71 +250,6 @@ int64_t parse_int(const char *what, const char *str) { return (int64_t)n; } -/** - * @brief Call rd_kafka_DescribeConsumerGroups() with a list of - * groups. - */ -static void -cmd_metadata_request(rd_kafka_conf_t *conf, int argc, char **argv) { - rd_kafka_t *rk; - rd_kafka_topic_conf_t *topic_conf; - rd_kafka_topic_t *rkt; - rd_kafka_topic_authorized_operations_pair_t* res; - const struct rd_kafka_metadata *metadatap; - const char **topics = NULL; - char errstr[512]; - int retval = 0; - int topic_cnt = 0; - int i; - - int allow_auto_topic_creation, include_cluster_authorized_operations, - include_topic_authorized_operations; - - metadatap = malloc(sizeof(*metadatap)); - allow_auto_topic_creation = - parse_int("allow_auto_topic_creation", argv[0]); - include_cluster_authorized_operations = - parse_int("include_cluster_authorized_operations", argv[1]); - include_topic_authorized_operations = - parse_int("include_topic_authorized_operations", argv[2]); - - if (allow_auto_topic_creation < 0 || - allow_auto_topic_creation > 1) - usage("Require allow_auto_topic_creation not a 0-1 int"); - - if (include_cluster_authorized_operations < 0 || - include_cluster_authorized_operations > 1) - usage("Require include_cluster_authorized_operations not a 0-1 int"); - - if (include_topic_authorized_operations < 0 || - include_topic_authorized_operations > 1) - usage("Require include_topic_authorized_operations not a 0-1 int"); - - - if (argc >= 1) { - topics = (const char **)&argv[3]; - topic_cnt = argc - 3; - } - - // topic_conf = rd_kafka_topic_conf_new(); - // rkt = rd_kafka_topic_new(rk, topics[0], topic_conf); - - rk = rd_kafka_new(RD_KAFKA_CONSUMER, conf, errstr, sizeof(errstr)); - if (!rk) - fatal("Failed to create new consumer: %s", errstr); - - rd_kafka_metadata(rk, 1, NULL, &metadatap, 100000); - //rd_kafka_metadata(rk, 0, rkt, &metadatap, 100000); - if(metadatap->orig_broker_name) - printf("%s\n", metadatap->orig_broker_name); - printf("Topic count: %d\n", metadatap->topic_cnt); -exit: - /* Destroy the client instance */ - rd_kafka_destroy(rk); - - exit(retval); -} - /** * @brief Call rd_kafka_DescribeConsumerGroups() with a list of * groups. @@ -449,7 +383,6 @@ int main(int argc, char **argv) { } } - //cmd_describe_consumer_groups(conf, argc - optind, &argv[optind]); - cmd_metadata_request(conf, argc - optind, &argv[optind]); + cmd_describe_consumer_groups(conf, argc - optind, &argv[optind]); return 0; } diff --git a/examples/describe_topics.c b/examples/describe_topics.c new file mode 100644 index 0000000000..057a43a2ec --- /dev/null +++ b/examples/describe_topics.c @@ -0,0 +1,377 @@ +/* + * librdkafka - Apache Kafka C library + * + * Copyright (c) 2022, Magnus Edenhill + * All rights reserved. + * + * Redistribution and use in source and binary forms, with or without + * modification, are permitted provided that the following conditions are met: + * + * 1. Redistributions of source code must retain the above copyright notice, + * this list of conditions and the following disclaimer. + * 2. Redistributions in binary form must reproduce the above copyright notice, + * this list of conditions and the following disclaimer in the documentation + * and/or other materials provided with the distribution. + * + * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" + * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE + * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE + * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE + * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR + * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF + * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS + * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN + * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) + * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE + * POSSIBILITY OF SUCH DAMAGE. + */ + +/** + * DescribeTopics usage example. + */ + +#include +#include +#include +#include +#include + +#ifdef _WIN32 +#include "../win32/wingetopt.h" +#else +#include +#endif + + +/* Typical include path would be , but this program + * is builtin from within the librdkafka source tree and thus differs. */ +#include "rdkafka.h" + + +const char *argv0; +static rd_kafka_queue_t *queue; /** Admin result queue. + * This is a global so we can + * yield in stop() */ +static volatile sig_atomic_t run = 1; + +/** + * @brief Signal termination of program + */ +static void stop(int sig) { + if (!run) { + fprintf(stderr, "%% Forced termination\n"); + exit(2); + } + run = 0; + rd_kafka_queue_yield(queue); +} + + +static void usage(const char *reason, ...) { + + fprintf(stderr, + "Describe topics usage examples\n" + "\n" + "Usage: %s \n" + " ...\n" + "\n" + "Options:\n" + " -b Bootstrap server list to connect to.\n" + " -X Set librdkafka configuration property.\n" + " See CONFIGURATION.md for full list.\n" + " -d Enable librdkafka debugging (%s).\n" + "\n", + argv0, rd_kafka_get_debug_contexts()); + + if (reason) { + va_list ap; + char reasonbuf[512]; + + va_start(ap, reason); + vsnprintf(reasonbuf, sizeof(reasonbuf), reason, ap); + va_end(ap); + + fprintf(stderr, "ERROR: %s\n", reasonbuf); + } + + exit(reason ? 1 : 0); +} + + +#define fatal(...) \ + do { \ + fprintf(stderr, "ERROR: "); \ + fprintf(stderr, __VA_ARGS__); \ + fprintf(stderr, "\n"); \ + exit(2); \ + } while (0) + + +/** + * @brief Set config property. Exit on failure. + */ +static void conf_set(rd_kafka_conf_t *conf, const char *name, const char *val) { + char errstr[512]; + + if (rd_kafka_conf_set(conf, name, val, errstr, sizeof(errstr)) != + RD_KAFKA_CONF_OK) + fatal("Failed to set %s=%s: %s", name, val, errstr); +} + +/** + * @brief Parse an integer or fail. + */ +int64_t parse_int(const char *what, const char *str) { + char *end; + unsigned long n = strtoull(str, &end, 0); + + if (end != str + strlen(str)) { + fprintf(stderr, "%% Invalid input for %s: %s: not an integer\n", + what, str); + exit(1); + } + + return (int64_t)n; +} +/** + * @brief Print topics information. + */ +static int +print_topics_info(const rd_kafka_DescribeTopics_result_t *topicdesc, + int topic_cnt) { + size_t i, j; + const rd_kafka_TopicDescription_t **result_topics; + size_t result_topics_cnt; + result_topics = rd_kafka_DescribeTopics_result_topics( + topicdesc, &result_topics_cnt); + + if (result_topics_cnt == 0) { + if (topic_cnt > 0) { + fprintf(stderr, "No matching topics found\n"); + return 1; + } else { + fprintf(stderr, "No topics in cluster\n"); + } + } + + for (i = 0; i < result_topics_cnt; i++) { + int j, acl_operation; + const rd_kafka_error_t *error; + const rd_kafka_TopicDescription_t *topic = + result_topics[i]; + const char *topic_name = + rd_kafka_TopicDescription_topic_name(topic); + int topic_authorized_operations_cnt = rd_kafka_TopicDescription_topic_authorized_operations_cnt(topic); + int partition_cnt = rd_kafka_TopicDescription_topic_partition_cnt(topic); + error = rd_kafka_TopicDescription_error(topic); + + if (rd_kafka_error_code(error)){ + printf("Topic: %s has error[%" PRId32 "]: %s\n", + topic_name, + rd_kafka_error_code(error), + rd_kafka_error_string(error)); + continue; + } + + printf("Topic: %s succeeded, has %d topic authorized operations allowed, they are:\n", + topic_name, topic_authorized_operations_cnt); + for(j=0;j 1) + usage("include_topic_authorized_operations not a 0-1 int"); + + if (argc >= 1) { + topics = (const char **)&argv[1]; + topics_cnt = argc - 1; + } + /* + * Create consumer instance + * NOTE: rd_kafka_new() takes ownership of the conf object + * and the application must not reference it again after + * this call. + */ + rk = rd_kafka_new(RD_KAFKA_CONSUMER, conf, errstr, sizeof(errstr)); + if (!rk) + fatal("Failed to create new consumer: %s", errstr); + + /* + * Describe topics + */ + queue = rd_kafka_queue_new(rk); + + /* Signal handler for clean shutdown */ + signal(SIGINT, stop); + + options = rd_kafka_AdminOptions_new( + rk, RD_KAFKA_ADMIN_OP_DESCRIBETOPICS); + + if (rd_kafka_AdminOptions_set_request_timeout( + options, 10 * 1000 /* 10s */, errstr, sizeof(errstr))) { + fprintf(stderr, "%% Failed to set timeout: %s\n", errstr); + goto exit; + } + if ((error = rd_kafka_AdminOptions_set_include_topic_authorized_operations( + options, include_topic_authorized_operations))) { + fprintf(stderr, + "%% Failed to set require topic authorized operations: %s\n", + rd_kafka_error_string(error)); + rd_kafka_error_destroy(error); + exit(1); + } + + rd_kafka_DescribeTopics(rk, topics, topics_cnt, options, queue); + + /* Wait for results */ + event = rd_kafka_queue_poll(queue, -1 /* indefinitely but limited by + * the request timeout set + * above (10s) */); + + if (!event) { + /* User hit Ctrl-C, + * see yield call in stop() signal handler */ + fprintf(stderr, "%% Cancelled by user\n"); + + } else if (rd_kafka_event_error(event)) { + rd_kafka_resp_err_t err = rd_kafka_event_error(event); + /* DescribeTopics request failed */ + fprintf(stderr, + "%% DescribeTopics failed[%" PRId32 "]: %s\n", + err, rd_kafka_event_error_string(event)); + goto exit; + + } else { + /* DescribeTopics request succeeded, but individual + * groups may have errors. */ + const rd_kafka_DescribeTopics_result_t *result; + + result = rd_kafka_event_DescribeTopics_result(event); + printf("DescribeTopics results:\n"); + retval = print_topics_info(result, topics_cnt); + } + + +exit: + if (event) + rd_kafka_event_destroy(event); + rd_kafka_AdminOptions_destroy(options); + rd_kafka_queue_destroy(queue); + /* Destroy the client instance */ + rd_kafka_destroy(rk); + + exit(retval); +} + +int main(int argc, char **argv) { + rd_kafka_conf_t *conf; /**< Client configuration object */ + int opt; + argv0 = argv[0]; + + /* + * Create Kafka client configuration place-holder + */ + conf = rd_kafka_conf_new(); + conf_set(conf, "sasl.username", "broker"); + conf_set(conf, "sasl.password", "broker"); + conf_set(conf, "sasl.mechanism", "SCRAM-SHA-256"); + conf_set(conf, "security.protocol", "SASL_PLAINTEXT"); + + /* + * 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.h b/src/rdkafka.h index e3474e50ff..2e0a8269e6 100644 --- a/src/rdkafka.h +++ b/src/rdkafka.h @@ -5360,6 +5360,10 @@ typedef int rd_kafka_event_type_t; #define RD_KAFKA_EVENT_LISTCONSUMERGROUPOFFSETS_RESULT 0x8000 /** AlterConsumerGroupOffsets_result_t */ #define RD_KAFKA_EVENT_ALTERCONSUMERGROUPOFFSETS_RESULT 0x10000 +/** DescribeTopics_result_t */ +#define RD_KAFKA_EVENT_DESCRIBETOPICS_RESULT 0x20000 +/** DescribeCluster_result_t */ +#define RD_KAFKA_EVENT_DESCRIBECLUSTER_RESULT 0x40000 /** @@ -5515,6 +5519,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); @@ -5626,6 +5632,10 @@ typedef rd_kafka_event_t rd_kafka_DeleteConsumerGroupOffsets_result_t; typedef rd_kafka_event_t rd_kafka_AlterConsumerGroupOffsets_result_t; /*! ListConsumerGroupOffsets result type */ typedef rd_kafka_event_t rd_kafka_ListConsumerGroupOffsets_result_t; +/*! DescribeTopics result type */ +typedef rd_kafka_event_t rd_kafka_DescribeTopics_result_t; +/*! DescribeCluster result type */ +typedef rd_kafka_event_t rd_kafka_DescribeCluster_result_t; /** * @brief Get CreateTopics result. @@ -5727,6 +5737,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. * @@ -6714,6 +6753,10 @@ typedef enum rd_kafka_admin_op_t { RD_KAFKA_ADMIN_OP_LISTCONSUMERGROUPOFFSETS, /** AlterConsumerGroupOffsets */ RD_KAFKA_ADMIN_OP_ALTERCONSUMERGROUPOFFSETS, + /**< DescribeTopics */ + RD_KAFKA_ADMIN_OP_DESCRIBETOPICS, + /**< DescribeCluster */ + RD_KAFKA_ADMIN_OP_DESCRIBECLUSTER, RD_KAFKA_ADMIN_OP__CNT /**< Number of ops defined */ } rd_kafka_admin_op_t; @@ -7725,6 +7768,399 @@ rd_kafka_DeleteRecords_result_offsets( /**@}*/ +/** + * @name Admin API - DescribeTopic + * @{ + */ +/** + * @brief Describe a topic and the authorized acl operations + * + * @param rk Client instance. + * @param topics Topic names + * @param topics_cnt Count of topics sent in topic names. + * @param options Optional admin options, or NULL for defaults. + * @param rkqu Queue to emit result on. + * + * @remark The result event type emitted on the supplied queue is of type + * \c RD_KAFKA_EVENT_DESCRIBETOPICS_RESULT + */ + +/** + * @brief DescribeTopics result type. + * + */ +typedef struct rd_kafka_TopicDescription_s rd_kafka_TopicDescription_t; + +/** + * @brief Gets the partiton id for partition at index position for the \p topicdesc topic. + * + * @param topicdesc The topic description. + * @param idx Index for the partitions. + * + * @return The partiton id. + * + * @remark The lifetime of the returned memory is the same + * as the lifetime of the \p topicdesc object. + */ +RD_EXPORT +const int rd_kafka_TopicDescription_partiton_id( + const rd_kafka_TopicDescription_t *topicdesc, int idx); + +/** + * @brief Gets the partiton leader for partition at index position for the \p topicdesc topic. + * + * @param topicdesc The topic description. + * @param idx Index for the partitions. + * + * @return The partiton leader. + * + * @remark The lifetime of the returned memory is the same + * as the lifetime of the \p topicdesc object. + */ +RD_EXPORT +const int rd_kafka_TopicDescription_partiton_leader( + const rd_kafka_TopicDescription_t *topicdesc, int idx); + +/** + * @brief Gets the partiton in-sync replica count for partition at index position for the \p topicdesc topic. + * + * @param topicdesc The topic description. + * @param idx Index for the partitions. + * + * @return The partiton replica count. + * + * @remark The lifetime of the returned memory is the same + * as the lifetime of the \p topicdesc object. + */ +RD_EXPORT +const int rd_kafka_TopicDescription_partiton_isr_cnt( + const rd_kafka_TopicDescription_t *topicdesc, int idx); + +/** + * @brief Gets the partiton replica count for partition at index position for the \p topicdesc topic. + * + * @param topicdesc The topic description. + * @param idx Index for the partitions. + * + * @return The partiton replica count. + * + * @remark The lifetime of the returned memory is the same + * as the lifetime of the \p topicdesc object. + */ +RD_EXPORT +const int rd_kafka_TopicDescription_partiton_replica_cnt( + const rd_kafka_TopicDescription_t *topicdesc, int idx); + +/** + * @brief Gets the partiton in-sync replica at isr index + * for partition at partition index position for the \p topicdesc topic. + * + * @param topicdesc The topic description. + * @param partition_idx Index for the partitions. + * @param isr_idx Index for the in-sync replica. + * + * @return The partiton in-sync replica. + * + * @remark The lifetime of the returned memory is the same + * as the lifetime of the \p topicdesc object. + */ +RD_EXPORT +const int rd_kafka_TopicDescription_partiton_isrs_idx( + const rd_kafka_TopicDescription_t *topicdesc, int partition_idx, int isr_idx); + +/** + * @brief Gets the partiton replica at replica index + * for partition at partition index position for the \p topicdesc topic. + * + * @param topicdesc The topic description. + * @param partition_idx Index for the partitions. + * @param replica_idx Index for the in-sync replica. + * + * @return The partiton replica. + * + * @remark The lifetime of the returned memory is the same + * as the lifetime of the \p topicdesc object. + */ +RD_EXPORT +const int rd_kafka_TopicDescription_partiton_replica_idx( + const rd_kafka_TopicDescription_t *topicdesc, int partition_idx, int replica_idx); + +/** + * @brief Gets the topic partiton count for the \p topicdesc topic. + * + * @param topicdesc The topic description. + * + * @return The topic partiton count. + * + * @remark The lifetime of the returned memory is the same + * as the lifetime of the \p topicdesc object. + */ +RD_EXPORT +const int rd_kafka_TopicDescription_topic_partition_cnt( + const rd_kafka_TopicDescription_t *topicdesc); + +/** + * @brief Gets the partiton error for partition at index position for the \p topicdesc topic. + * + * @param topicdesc The topic description. + * @param idx Index for the partitions. + * + * @return The partiton error. + * + * @remark The lifetime of the returned memory is the same + * as the lifetime of the \p topicdesc object. + */ +RD_EXPORT +const rd_kafka_error_t *rd_kafka_TopicDescription_partition_error( + const rd_kafka_TopicDescription_t *topicdesc, int idx); + +/** + * @brief Gets operation at idx index of topic authorized operations for the \p topicdesc topic. + * + * @param topicdesc The topic description. + * @param idx The index for which element is needed. + * + * @return Authorized operation at given index. + */ +RD_EXPORT +const int rd_kafka_TopicDescription_authorized_operation_idx( + const rd_kafka_TopicDescription_t *topicdesc, + size_t idx); + +/** + * @brief Gets the topic authorized acl operations count for the \p topicdesc topic. + * + * @param topicdesc The topic description. + * + * @return The topic authorized operations count. + * + * @remark The lifetime of the returned memory is the same + * as the lifetime of the \p topicdesc object. + */ +RD_EXPORT +const int rd_kafka_TopicDescription_topic_authorized_operations_cnt( + const rd_kafka_TopicDescription_t *topicdesc); + +/** + * @brief Gets the topic name for the \p topicdesc topic. + * + * @param topicdesc The topic description. + * + * @return The topic name. + * + * @remark The lifetime of the returned memory is the same + * as the lifetime of the \p topicdesc object. + */ +RD_EXPORT +const char *rd_kafka_TopicDescription_topic_name( + const rd_kafka_TopicDescription_t *topicdesc) ; + +/** + * @brief Gets the error for the \p topicdesc topic. + * + * @param topicdesc The topic description. + * + * @return The topic description error. + * + * @remark The lifetime of the returned memory is the same + * as the lifetime of the \p topicdesc object. + */ +RD_EXPORT +const rd_kafka_error_t *rd_kafka_TopicDescription_error( + const rd_kafka_TopicDescription_t *topicdesc) ; +/** + * @brief Get an array of topic results from a DescribeTopics result. + * + * The returned topics life-time is the same as the \p result object. + * + * @param result Result to get topics results from. + * @param cntp is updated to the number of elements in the array. + * + * @remark The lifetime of the returned memory is the same + * as the lifetime of the \p result object. + */ +RD_EXPORT +const rd_kafka_TopicDescription_t ** +rd_kafka_DescribeTopics_result_topics( + const rd_kafka_DescribeTopics_result_t *result, + size_t *cntp); +/** + * @brief Describe topics as specified by the \p topics + * array of size \p topics_cnt elements. + * + * @param rk Client instance. + * @param topics Array of topics to describe. + * @param topics_cnt Number of elements in \p topics array. + * @param options Optional admin options, or NULL for defaults. + * @param rkqu Queue to emit result on. + * + * @remark The result event type emitted on the supplied queue is of type + * \c RD_KAFKA_EVENT_DESCRIBETOPICS_RESULT + */ +RD_EXPORT +void rd_kafka_DescribeTopics(rd_kafka_t *rk, + const char** topics, + size_t topics_cnt, + const rd_kafka_AdminOptions_t *options, + rd_kafka_queue_t *rkqu); + +/** + * @brief Whether broker should return topic authorized operations + * (DescribeTopic). + * + * @param options Admin options. + * @param true_or_false Defaults to false. + * + * @return NULL on success, a new error instance that must be + * released with rd_kafka_error_destroy() in case of error. + * + * @remark This option is valid for DescribeTopic. + */ +RD_EXPORT +rd_kafka_error_t *rd_kafka_AdminOptions_set_include_topic_authorized_operations( + rd_kafka_AdminOptions_t *options, + int true_or_false); + +/**@}*/ + +/** + * @name Admin API - DescribeCluster + * @{ + */ + +typedef struct rd_kafka_ClusterDescription_s rd_kafka_ClusterDescription_t; + +/** + * @brief Gets the node for the \p clusterdesc cluster at idx position. + * + * @param clusterdesc The cluster description. + * + * @return The node at idx position. + * + * @remark The lifetime of the returned memory is the same + * as the lifetime of the \p clusterdesc object. + */ +RD_EXPORT +const rd_kafka_Node_t *rd_kafka_ClusterDescription_node_idx( + const rd_kafka_ClusterDescription_t *clusterdesc, int idx); + +/** + * @brief Gets operation at idx index of cluster authorized operations for the \p clusterdesc cluster. + * + * @param clusterdesc The cluster description. + * @param idx The index for which element is needed. + * + * @return Authorized operation at given index. + */ +RD_EXPORT +const int rd_kafka_ClusterDescription_authorized_operation_idx( + const rd_kafka_ClusterDescription_t *clusterdesc, + size_t idx); + +/** + * @brief Gets the cluster authorized acl operations for the \p clusterdesc cluster. + * + * @param clusterdesc The cluster description. + * + * @return The cluster authorized operations. + * + * @remark The lifetime of the returned memory is the same + * as the lifetime of the \p clusterdesc object. + */ +RD_EXPORT +const int rd_kafka_ClusterDescription_cluster_acl_operations_cnt( + const rd_kafka_ClusterDescription_t *clusterdesc); + +/** + * @brief Gets the cluster current controller id for the \p clusterdesc cluster. + * + * @param clusterdesc The cluster description. + * + * @return The cluster current controller id. + * + * @remark The lifetime of the returned memory is the same + * as the lifetime of the \p clusterdesc object. + */ +RD_EXPORT +const int rd_kafka_ClusterDescription_controller_id( + const rd_kafka_ClusterDescription_t *clusterdesc); + +/** + * @brief Gets the cluster current cluster id for the \p clusterdesc cluster. + * + * @param clusterdesc The cluster description. + * + * @return The cluster current cluster id (char*). + * + * @remark The lifetime of the returned memory is the same + * as the lifetime of the \p clusterdesc object. + */ +RD_EXPORT +const char *rd_kafka_ClusterDescription_cluster_id( + const rd_kafka_ClusterDescription_t *clusterdesc); + +/** + * @brief Gets the node count for the \p clusterdesc cluster. + * + * @param clusterdesc The cluster description. + * + * @return The node count. + * + * @remark The lifetime of the returned memory is the same + * as the lifetime of the \p clusterdesc object. + */ +RD_EXPORT +const int rd_kafka_ClusterDescription_node_cnt( + const rd_kafka_ClusterDescription_t *clusterdesc) ; + +/** + * @brief Get the DescribeCluster result. + * + * The returned description life-time is the same as the \p result object. + * + * @param result Result to get group results from. + * + * @remark The lifetime of the returned memory is the same + * as the lifetime of the \p result object. + */ +RD_EXPORT +const rd_kafka_ClusterDescription_t * +rd_kafka_DescribeCluster_result_description( + const rd_kafka_DescribeCluster_result_t *result); + +/** + * @brief Describes the cluster. + * + * @param rk Client instance. + * @param options Optional admin options, or NULL for defaults. + * @param rkqu Queue to emit result on. + * + * @remark The result event type emitted on the supplied queue is of type + * \c RD_KAFKA_EVENT_DESCRIBECLUSTER_RESULT + */ +RD_EXPORT +void rd_kafka_DescribeCluster(rd_kafka_t *rk, + const rd_kafka_AdminOptions_t *options, + rd_kafka_queue_t *rkqu); + +/** + * @brief Whether broker should return cluster authorized operations + * (DescribeCluster). + * + * @param options Admin options. + * @param true_or_false Defaults to false. + * + * @return NULL on success, a new error instance that must be + * released with rd_kafka_error_destroy() in case of error. + * + * @remark This option is valid for DescribeCluster. + */ +RD_EXPORT +rd_kafka_error_t *rd_kafka_AdminOptions_set_include_cluster_authorized_operations( + rd_kafka_AdminOptions_t *options, + int true_or_false); +/**@}*/ + /** * @name Admin API - ListConsumerGroups * @{ diff --git a/src/rdkafka_admin.c b/src/rdkafka_admin.c index 6aaec636d5..0c349c8391 100644 --- a/src/rdkafka_admin.c +++ b/src/rdkafka_admin.c @@ -1557,6 +1557,26 @@ 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_topic_authorized_operations( + rd_kafka_AdminOptions_t *options, + int true_or_false) { + char errstr[512]; + rd_kafka_resp_err_t err = rd_kafka_confval_set_type( + &options->include_topic_authorized_operations, RD_KAFKA_CONFVAL_INT, + &true_or_false, errstr, sizeof(errstr)); + return !err ? NULL : rd_kafka_error_new(err, "%s", errstr); +} + +rd_kafka_error_t *rd_kafka_AdminOptions_set_include_cluster_authorized_operations( + rd_kafka_AdminOptions_t *options, + int true_or_false) { + char errstr[512]; + rd_kafka_resp_err_t err = rd_kafka_confval_set_type( + &options->include_cluster_authorized_operations, RD_KAFKA_CONFVAL_INT, + &true_or_false, errstr, sizeof(errstr)); + return !err ? NULL : rd_kafka_error_new(err, "%s", errstr); +} + rd_kafka_error_t *rd_kafka_AdminOptions_set_match_consumer_group_states( rd_kafka_AdminOptions_t *options, const rd_kafka_consumer_group_state_t *consumer_group_states, @@ -1656,6 +1676,21 @@ static void rd_kafka_AdminOptions_init(rd_kafka_t *rk, else rd_kafka_confval_disable(&options->require_stable_offsets, "require_stable_offsets"); + if (options->for_api == RD_KAFKA_ADMIN_OP_ANY || + options->for_api == RD_KAFKA_ADMIN_OP_DESCRIBETOPICS) + rd_kafka_confval_init_int(&options->include_topic_authorized_operations, + "include_topic_authorized_operations",0,1,0); + else + rd_kafka_confval_disable(&options->include_topic_authorized_operations, + "include_topic_authorized_operations"); + + if (options->for_api == RD_KAFKA_ADMIN_OP_ANY || + options->for_api == RD_KAFKA_ADMIN_OP_DESCRIBECLUSTER) + rd_kafka_confval_init_int(&options->include_cluster_authorized_operations, + "include_cluster_authorized_operations",0,1,0); + else + rd_kafka_confval_disable(&options->include_cluster_authorized_operations, + "include_cluster_authorized_operations"); if (options->for_api == RD_KAFKA_ADMIN_OP_ANY || options->for_api == RD_KAFKA_ADMIN_OP_LISTCONSUMERGROUPS) @@ -5981,6 +6016,32 @@ const rd_kafka_error_t **rd_kafka_ListConsumerGroups_result_errors( * */ +/** + * @brief Parse authorized_operations returned in + * - DescribeConsumerGroups + * - DescribeTopics + * - DescribeCluster + * @returns list of acl operations + */ +rd_list_t* rd_kafka_AuthorizedOperations_parse(int32_t authorized_operations){ + int i, bit; + rd_list_t* authorized_operations_list = NULL; + rd_kafka_AclOperation_t* entry; + /* in case of authorized_operations not requested, return NULL*/ + if(authorized_operations<0) + return NULL; + authorized_operations_list = rd_list_new(0, rd_free); + for(i=0; i> i) & 1; + if(bit){ + entry = malloc(sizeof(rd_kafka_AclOperation_t)); + *entry = i; + rd_list_add(authorized_operations_list, entry); + } + } + return authorized_operations_list; +} + /** * @brief Create a new MemberDescription object. This object is used for * creating a ConsumerGroupDescription. @@ -6666,3 +6727,751 @@ rd_kafka_DescribeConsumerGroups_result_groups( } /**@}*/ + +/** + * @name Describe Topic + * @{ + * + * + * + * + */ + +const int rd_kafka_TopicDescription_authorized_operation_idx( + const rd_kafka_TopicDescription_t *topicdesc, + size_t idx) { + rd_kafka_AclOperation_t* entry = rd_list_elem(topicdesc->topic_authorized_operations, idx); + return *entry; +} + +const int rd_kafka_TopicDescription_topic_authorized_operations_cnt( + const rd_kafka_TopicDescription_t *topicdesc) { + if(topicdesc->topic_authorized_operations) + return rd_list_cnt(topicdesc->topic_authorized_operations); + return 0; +} + +const int rd_kafka_TopicDescription_partiton_id( + const rd_kafka_TopicDescription_t *topicdesc, int idx) { + return topicdesc->partitions[idx].id; +} + +const int rd_kafka_TopicDescription_partiton_leader( + const rd_kafka_TopicDescription_t *topicdesc, int idx) { + return topicdesc->partitions[idx].leader; +} + +const int rd_kafka_TopicDescription_partiton_isr_cnt( + const rd_kafka_TopicDescription_t *topicdesc, int idx) { + return topicdesc->partitions[idx].isr_cnt; +} + +const int rd_kafka_TopicDescription_partiton_replica_cnt( + const rd_kafka_TopicDescription_t *topicdesc, int idx) { + return topicdesc->partitions[idx].replica_cnt; +} + +const int rd_kafka_TopicDescription_partiton_isrs_idx( + const rd_kafka_TopicDescription_t *topicdesc, int partition_idx, int isr_idx) { + return topicdesc->partitions[partition_idx].isrs[isr_idx]; +} + +const int rd_kafka_TopicDescription_partiton_replica_idx( + const rd_kafka_TopicDescription_t *topicdesc, int partition_idx, int replica_idx) { + return topicdesc->partitions[partition_idx].replicas[replica_idx]; +} + +const rd_kafka_error_t *rd_kafka_TopicDescription_partition_error( + const rd_kafka_TopicDescription_t *topicdesc, int idx) { + return rd_kafka_error_new(topicdesc->partitions[idx].err, NULL); +} + +const int rd_kafka_TopicDescription_topic_partition_cnt( + const rd_kafka_TopicDescription_t *topicdesc) { + return topicdesc->partition_cnt; +} + +const char *rd_kafka_TopicDescription_topic_name( + const rd_kafka_TopicDescription_t *topicdesc) { + return topicdesc->topic; +} + +const rd_kafka_error_t *rd_kafka_TopicDescription_error( + const rd_kafka_TopicDescription_t *topicdesc) { + return rd_kafka_error_new(topicdesc->err, NULL); +} +const rd_kafka_TopicDescription_t ** +rd_kafka_DescribeTopics_result_topics( + const rd_kafka_DescribeTopics_result_t *result, + size_t *cntp) { + const rd_kafka_op_t *rko = (const rd_kafka_op_t *)result; + rd_kafka_op_type_t reqtype = + rko->rko_u.admin_result.reqtype & ~RD_KAFKA_OP_FLAGMASK; + rd_assert(reqtype == RD_KAFKA_OP_DESCRIBETOPICS); + + *cntp = rd_list_cnt(&rko->rko_u.admin_result.results); + return (const rd_kafka_TopicDescription_t **) + rko->rko_u.admin_result.results.rl_elems; +} +static void rd_kafka_TopicDescription_destroy( + rd_kafka_TopicDescription_t *topicdesc) { + int i; + if (likely(topicdesc->topic != NULL)) + rd_free(topicdesc->topic); + // if (likely(topicdesc->err != NULL)) + // rd_kafka_error_destroy(topicdesc->err); + for(i=0;ipartition_cnt;i++){ + if(likely(topicdesc->partitions[i].isrs != NULL)) + rd_free(topicdesc->partitions[i].isrs); + if(likely(topicdesc->partitions[i].replicas != NULL)) + rd_free(topicdesc->partitions[i].replicas); + } + if(likely(topicdesc->partitions != NULL)) + rd_free(topicdesc->partitions); + if(likely(topicdesc->topic_authorized_operations != NULL)) + rd_list_destroy(topicdesc->topic_authorized_operations); + rd_free(topicdesc); +} +static void rd_kafka_TopicDescription_free(void *ptr) { + rd_kafka_TopicDescription_destroy(ptr); +} + +/** + * @brief Topics arguments comparator for DescribeTopics args + */ +static int rd_kafka_DescribeTopics_cmp(const void *a, const void *b) { + return strcmp(a, b); +} +/** + * @brief Create a new ConsumerGroupDescription object. + * + * @param topic topic name + * @param partitions Array of partition metadata (rd_kafka_metadata_partition). + * @param partition_cnt Number of partitons in partition metadata. + * @param topic_authorized_operations acl operations allowed for topic. + * @param err Topic error reported by broker. + * @return A new allocated TopicDescription object. + * Use rd_kafka_TopicDescription_destroy() to free when done. + */ +static rd_kafka_TopicDescription_t * +rd_kafka_TopicDescription_new(const char *topic, + struct rd_kafka_metadata_partition* partitions, + int partition_cnt, + rd_list_t* topic_authorized_operations, + rd_kafka_resp_err_t err) { + rd_kafka_TopicDescription_t *topicdesc; + int i,j; + topicdesc = rd_calloc(1, sizeof(*topicdesc)); + topicdesc->topic = rd_strdup(topic); + topicdesc->partition_cnt = partition_cnt; + topicdesc->err = err; + if(topic_authorized_operations){ + topicdesc->topic_authorized_operations = + rd_list_new(rd_list_cnt(topic_authorized_operations), rd_free); + for(i=0;itopic_authorized_operations, + oper); + } + } + else + topicdesc->topic_authorized_operations = NULL; + + if (partitions == NULL) { + topicdesc->partitions = NULL; + } else { + topicdesc->partitions = rd_malloc(sizeof(*partitions) * partition_cnt); + for(i=0; i < partition_cnt; i++){ + topicdesc->partitions[i].err = partitions[i].err; + topicdesc->partitions[i].id = partitions[i].id; + topicdesc->partitions[i].isr_cnt = partitions[i].isr_cnt; + topicdesc->partitions[i].isrs = + rd_malloc(sizeof(int32_t) * topicdesc->partitions[i].isr_cnt); + for(j=0; jpartitions[i].isr_cnt; j++){ + topicdesc->partitions[i].isrs[j] = + partitions[i].isrs[j]; + } + topicdesc->partitions[i].leader = partitions[i].leader; + topicdesc->partitions[i].replica_cnt = partitions[i].replica_cnt; + topicdesc->partitions[i].replicas = + rd_malloc(sizeof(int32_t) * topicdesc->partitions[i].replica_cnt); + for(j=0; jpartitions[i].replica_cnt; j++){ + topicdesc->partitions[i].replicas[j] = + partitions[i].replicas[j]; + } + + } + } + return topicdesc; +} +/** + * @brief Copy \p desc TopicDescription. + * + * @param desc The topic description to copy. + * @return A new allocated copy of the passed TopicDescription. + */ +static rd_kafka_TopicDescription_t * +rd_kafka_TopicDescription_copy( + const rd_kafka_TopicDescription_t *topicdesc) { + return rd_kafka_TopicDescription_new( + topicdesc->topic, topicdesc->partitions, + topicdesc->partition_cnt, topicdesc->topic_authorized_operations, + topicdesc->err); +} +/** + * @brief Same as rd_kafka_TopicDescription_copy() but suitable for + * rd_list_copy(). The \p opaque is ignored. + */ +static void *rd_kafka_TopicDescription_copy_opaque(const void *topicdesc, + void *opaque) { + return rd_kafka_TopicDescription_copy(topicdesc); +} +/** + * @brief New instance of TopicDescription from an error. + * + * @param group_id The topic. + * @param error The error. + * @return A new allocated TopicDescription with the passed error. + */ +static rd_kafka_TopicDescription_t * +rd_kafka_TopicDescription_new_error(const char *topic, + rd_kafka_resp_err_t err) { + return rd_kafka_TopicDescription_new( + topic, NULL, 0, NULL, err); +} +/** @brief Merge the DescribeTopics response from a single broker + * into the user response list. + */ +static void rd_kafka_DescribeTopics_response_merge( + rd_kafka_op_t *rko_fanout, + const rd_kafka_op_t *rko_partial) { + rd_kafka_TopicDescription_t *topicres = NULL; + rd_kafka_TopicDescription_t *newtopicres; + const char *topic = rko_partial->rko_u.admin_result.opaque; + int orig_pos; + + rd_assert(rko_partial->rko_evtype == + RD_KAFKA_EVENT_DESCRIBETOPICS_RESULT); + + if (!rko_partial->rko_err) { + /* Proper results. + * We only send one topic per request, make sure it matches */ + topicres = + rd_list_elem(&rko_partial->rko_u.admin_result.results, 0); + rd_assert(topicres); + rd_assert(!strcmp(topicres->topic, topic)); + newtopicres = rd_kafka_TopicDescription_copy(topicres); + } else { + /* Op errored, e.g. timeout */ + rd_kafka_error_t *error = + rd_kafka_error_new(rko_partial->rko_err, NULL); + newtopicres = + rd_kafka_TopicDescription_new_error(topic, error->code); + rd_kafka_error_destroy(error); + } + + /* As a convenience to the application we insert group result + * in the same order as they were requested. */ + orig_pos = rd_list_index(&rko_fanout->rko_u.admin_request.args, topic, + rd_kafka_DescribeTopics_cmp); + rd_assert(orig_pos != -1); + + /* Make sure result is not already set */ + rd_assert(rd_list_elem(&rko_fanout->rko_u.admin_request.fanout.results, + orig_pos) == NULL); + + rd_list_set(&rko_fanout->rko_u.admin_request.fanout.results, orig_pos, + newtopicres); +} +/** + * @brief Construct and send DescribeTopicsRequest to \p rkb + * with the topics (char *) in \p groups, using + * \p options. + * + * The response (unparsed) will be enqueued on \p replyq + * for handling by \p resp_cb (with \p opaque passed). + * + * @returns RD_KAFKA_RESP_ERR_NO_ERROR if the request was enqueued for + * transmission, otherwise an error code and errstr will be + * updated with a human readable error string. + */ +static rd_kafka_resp_err_t rd_kafka_admin_DescribeTopicsRequest( + rd_kafka_broker_t *rkb, + const rd_list_t *topics /*(char*)*/, + rd_kafka_AdminOptions_t *options, + char *errstr, + size_t errstr_size, + rd_kafka_replyq_t replyq, + rd_kafka_resp_cb_t *resp_cb, + void *opaque){ + int i; + rd_kafka_resp_err_t err; + int include_topic_authorized_operations; + + include_topic_authorized_operations = + rd_kafka_confval_get_int(&options->include_topic_authorized_operations); + /* resp_cb = rd_kafka_admin_handle_response; */ + + // error = Call metadata request + err = rd_kafka_MetadataRequest(rkb, topics, + "describe topics", + rd_false, + rd_false, + include_topic_authorized_operations, + rd_false, + NULL, + resp_cb, 0, opaque); + + if (err) { + rd_snprintf(errstr, errstr_size, "%s", + rd_kafka_err2str(err)); + return err; + } + + return RD_KAFKA_RESP_ERR_NO_ERROR; +} + +/** + * @brief Parse DescribeTopicsResponse and create ADMIN_RESULT op. + */ +static rd_kafka_resp_err_t +rd_kafka_DescribeTopicsResponse_parse(rd_kafka_op_t *rko_req, + rd_kafka_op_t **rko_resultp, + rd_kafka_buf_t *reply, + char *errstr, + size_t errstr_size){ + 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; + rd_kafka_topic_authorized_operations_pair_t* topic_authorized_operations = NULL; + int32_t cluster_authorized_operations; + char* cluster_id = NULL; + int controller_id; + int i, cnt; + rd_kafka_op_t *rko_result = NULL; + // rd_kafka_assert(NULL, err == RD_KAFKA_RESP_ERR__DESTROY || + // thrd_is_current(rk->rk_thread)); + // rd_kafka_assert(NULL, err == RD_KAFKA_RESP_ERR__DESTROY); + + err = rd_kafka_parse_Metadata(rkb, NULL, reply, &md, &topic_authorized_operations,&cluster_authorized_operations, &topics, + &cluster_id, &controller_id); + if (err) + goto err; + rko_result = rd_kafka_admin_result_new(rko_req); + rd_list_init(&rko_result->rko_u.admin_result.results, md->topic_cnt, + rd_kafka_TopicDescription_free); + cnt = md->topic_cnt; + i=0; + while(cnt--){ + rd_kafka_TopicDescription_t *topicdesc = NULL; + /* topics in md should be in the same order as in topic_authorized_operations*/ + rd_assert(strcmp(md->topics[i].topic, topic_authorized_operations[i].topic_name) == 0); + if(md->topics[i].err == RD_KAFKA_RESP_ERR_NO_ERROR){ + rd_list_t* authorized_operations; + authorized_operations = + rd_kafka_AuthorizedOperations_parse( + topic_authorized_operations[i].authorized_operations); + topicdesc = rd_kafka_TopicDescription_new(md->topics[i].topic, + md->topics[i].partitions, + md->topics[i].partition_cnt, + authorized_operations, + md->topics[i].err); + rd_list_destroy(authorized_operations); + } + else + topicdesc = rd_kafka_TopicDescription_new_error(md->topics[i].topic, + md->topics[i].err); + rd_list_add(&rko_result->rko_u.admin_result.results, topicdesc); + } + *rko_resultp = rko_result; + return RD_KAFKA_RESP_ERR_NO_ERROR; + +err: + if (rko_result) + rd_kafka_op_destroy(rko_result); + rd_snprintf( + errstr, errstr_size, + "DescribeTopics response protocol parse failure: %s", + rd_kafka_err2str(reply->rkbuf_err)); + return reply->rkbuf_err; +} +void rd_kafka_DescribeTopics(rd_kafka_t *rk, + const char** topics, + size_t topics_cnt, + const rd_kafka_AdminOptions_t *options, + rd_kafka_queue_t *rkqu){ + rd_kafka_op_t *rko_fanout; + rd_list_t dup_list; + size_t i; + + static const struct rd_kafka_admin_fanout_worker_cbs fanout_cbs = { + rd_kafka_DescribeTopics_response_merge, + rd_kafka_TopicDescription_copy_opaque}; + + rd_assert(rkqu); + + rko_fanout = rd_kafka_admin_fanout_op_new( + rk, RD_KAFKA_OP_DESCRIBETOPICS, + RD_KAFKA_EVENT_DESCRIBETOPICS_RESULT, &fanout_cbs, options, + rkqu->rkqu_q); + + if (topics_cnt == 0) { + rd_kafka_admin_result_fail(rko_fanout, + RD_KAFKA_RESP_ERR__INVALID_ARG, + "No topics to describe"); + rd_kafka_admin_common_worker_destroy(rk, rko_fanout, + rd_true /*destroy*/); + return; + } + + /* Copy topic list and store it on the request op. + * Maintain original ordering. */ + rd_list_init(&rko_fanout->rko_u.admin_request.args, (int)topics_cnt, + rd_free); + for (i = 0; i < topics_cnt; i++) + rd_list_add(&rko_fanout->rko_u.admin_request.args, + rd_strdup(topics[i])); + + /* Check for duplicates. + * Make a temporary copy of the topic list and sort it to check for + * duplicates, we don't want the original list sorted since we want + * to maintain ordering. */ + rd_list_init(&dup_list, + rd_list_cnt(&rko_fanout->rko_u.admin_request.args), NULL); + rd_list_copy_to(&dup_list, &rko_fanout->rko_u.admin_request.args, NULL, + NULL); + rd_list_sort(&dup_list, rd_kafka_DescribeTopics_cmp); + if (rd_list_find_duplicate(&dup_list, + rd_kafka_DescribeTopics_cmp)) { + rd_list_destroy(&dup_list); + rd_kafka_admin_result_fail(rko_fanout, + RD_KAFKA_RESP_ERR__INVALID_ARG, + "Duplicate topics not allowed"); + rd_kafka_admin_common_worker_destroy(rk, rko_fanout, + rd_true /*destroy*/); + return; + } + + rd_list_destroy(&dup_list); + + /* Prepare results list where fanned out op's results will be + * accumulated. */ + rd_list_init(&rko_fanout->rko_u.admin_request.fanout.results, + (int)topics_cnt, rd_kafka_TopicDescription_free); + rko_fanout->rko_u.admin_request.fanout.outstanding = (int)topics_cnt; + + /* Create individual request ops for each topic. + * FIXME: A future optimization is to coalesce all topic for a single + * coordinator into one op. */ + for (i = 0; i < topics_cnt; i++) { + static const struct rd_kafka_admin_worker_cbs cbs = { + rd_kafka_admin_DescribeTopicsRequest, + rd_kafka_DescribeTopicsResponse_parse, + }; + char *topic = + rd_list_elem(&rko_fanout->rko_u.admin_request.args, (int)i); + rd_kafka_op_t *rko = rd_kafka_admin_request_op_new( + rk, RD_KAFKA_OP_DESCRIBETOPICS, + RD_KAFKA_EVENT_DESCRIBETOPICS_RESULT, &cbs, options, + rk->rk_ops); + + rko->rko_u.admin_request.fanout_parent = rko_fanout; + // rko->rko_u.admin_request.broker_id = + // RD_KAFKA_ADMIN_TARGET_COORDINATOR; + rko->rko_u.admin_request.coordtype = RD_KAFKA_COORD_GROUP; + rko->rko_u.admin_request.coordkey = rd_strdup(topic); + + /* Set the topic name as the opaque so the fanout worker use it + * to fill in errors. + * References rko_fanout's memory, which will always outlive + * the fanned out op. */ + rd_kafka_AdminOptions_set_opaque( + &rko->rko_u.admin_request.options, topic); + + rd_list_init(&rko->rko_u.admin_request.args, 1, rd_free); + rd_list_add(&rko->rko_u.admin_request.args, + rd_strdup(topics[i])); + + rd_kafka_q_enq(rk->rk_ops, rko); + } + +} + +/**@}*/ + +/** + * @name Describe cluster + * @{ + * + * + * + * + */ + +const rd_kafka_Node_t *rd_kafka_ClusterDescription_node_idx( + const rd_kafka_ClusterDescription_t *clusterdesc, int idx) { + return &(clusterdesc->Nodes[idx]); +} + +const int rd_kafka_ClusterDescription_authorized_operation_idx( + const rd_kafka_ClusterDescription_t *clusterdesc, + size_t idx) { + rd_kafka_AclOperation_t* entry = rd_list_elem(clusterdesc->cluster_authorized_operations, idx); + return *entry; +} + +const char *rd_kafka_ClusterDescription_cluster_id( + const rd_kafka_ClusterDescription_t *clusterdesc) { + return clusterdesc->cluster_id; +} +const int rd_kafka_ClusterDescription_controller_id( + const rd_kafka_ClusterDescription_t *clusterdesc) { + return clusterdesc->controller_id; +} +const int rd_kafka_ClusterDescription_cluster_acl_operations_cnt( + const rd_kafka_ClusterDescription_t *clusterdesc) { + if(clusterdesc->cluster_authorized_operations) + return rd_list_cnt(clusterdesc->cluster_authorized_operations); + return 0; +} +const int rd_kafka_ClusterDescription_node_cnt( + const rd_kafka_ClusterDescription_t *clusterdesc) { + return clusterdesc->node_cnt; +} + +const rd_kafka_ClusterDescription_t * +rd_kafka_DescribeCluster_result_description( + const rd_kafka_DescribeTopics_result_t *result) { + int cluster_result_cnt; + const rd_kafka_ClusterDescription_t* clusterdesc; + const rd_kafka_op_t *rko = (const rd_kafka_op_t *)result; + rd_kafka_op_type_t reqtype = + rko->rko_u.admin_result.reqtype & ~RD_KAFKA_OP_FLAGMASK; + rd_assert(reqtype == RD_KAFKA_OP_DESCRIBECLUSTER); + + cluster_result_cnt = rd_list_cnt( + &rko->rko_u.admin_result.results); + printf("%d is the value\n", cluster_result_cnt); + rd_assert(cluster_result_cnt == 1); + clusterdesc = rd_list_elem( + &rko->rko_u.admin_result.results, 0); + + return clusterdesc; +} + +/** + * @brief Copy \p desc ClusterDescription. + * + * @param desc The cluster description to copy. + * @return A new allocated copy of the passed ClusterDescription. + */ +static rd_kafka_ClusterDescription_t * +rd_kafka_ClusterDescription_copy( + const rd_kafka_ClusterDescription_t *desc) { + rd_kafka_ClusterDescription_t *clusterdesc; + int i; + clusterdesc = rd_calloc(1, sizeof(*clusterdesc)); + + clusterdesc->cluster_id = rd_strdup(desc->cluster_id); + clusterdesc->controller_id = desc->controller_id; + + if(desc->cluster_authorized_operations){ + clusterdesc->cluster_authorized_operations = + rd_list_new(rd_list_cnt(desc->cluster_authorized_operations), rd_free); + for(i=0;icluster_authorized_operations); i++){ + int* entry = rd_list_elem(desc->cluster_authorized_operations, i); + int* oper = malloc(sizeof(int)); + *oper = *entry; + rd_list_add(clusterdesc->cluster_authorized_operations, + oper); + } + } + else + clusterdesc->cluster_authorized_operations = NULL; + + clusterdesc->node_cnt = desc->node_cnt; + clusterdesc->Nodes = malloc(sizeof(rd_kafka_Node_t) * desc->node_cnt); + for(i=0;inode_cnt;i++){ + clusterdesc->Nodes[i].host = rd_strdup(desc->Nodes[i].host); + clusterdesc->Nodes[i].port = desc->Nodes[i].port; + clusterdesc->Nodes[i].id = desc->Nodes[i].id; + } + return clusterdesc; +} + +/** + * @brief Create a new ClusterDescription object. + * + * @param cluster_id current cluster_id + * @param controller_id current controller_id. + * @param md metadata struct returned by parse_metadata(). + * + * @returns new ClusterDescription object. + */ +static rd_kafka_ClusterDescription_t * +rd_kafka_ClusterDescription_new(const char *cluster_id, + int controller_id, + rd_list_t* cluster_authorized_operations, + struct rd_kafka_metadata *md) { + rd_kafka_ClusterDescription_t *clusterdesc; + int i; + clusterdesc = rd_calloc(1, sizeof(*clusterdesc)); + + clusterdesc->cluster_id = rd_strdup(cluster_id); + clusterdesc->controller_id = controller_id; + + if(cluster_authorized_operations){ + clusterdesc->cluster_authorized_operations = + rd_list_new(rd_list_cnt(cluster_authorized_operations), rd_free); + for(i=0;icluster_authorized_operations, + oper); + } + } + else + clusterdesc->cluster_authorized_operations = NULL; + + clusterdesc->node_cnt = md->broker_cnt; + clusterdesc->Nodes = malloc(sizeof(rd_kafka_Node_t) * md->broker_cnt); + for(i=0;ibroker_cnt;i++){ + clusterdesc->Nodes[i].host = rd_strdup(md->brokers[i].host); + clusterdesc->Nodes[i].port = md->brokers[i].port; + clusterdesc->Nodes[i].id = md->brokers[i].id; + } + return clusterdesc; +} + +static void rd_kafka_ClusterDescription_destroy( + rd_kafka_ClusterDescription_t *clusterdesc) { + int i; + RD_IF_FREE(clusterdesc->cluster_id, rd_free); + if(clusterdesc->cluster_authorized_operations) + rd_list_destroy(clusterdesc->cluster_authorized_operations); + for(i=0;inode_cnt;i++){ + rd_kafka_Node_destroy(&clusterdesc->Nodes[i]); + } + rd_free(clusterdesc); +} +static void rd_kafka_ClusterDescription_free(void *ptr) { + rd_kafka_ClusterDescription_destroy(ptr); +} +/** + * @brief Send DescribeClusterRequest. Admin worker compatible callback. + */ +static rd_kafka_resp_err_t +rd_kafka_admin_DescribeClusterRequest(rd_kafka_broker_t *rkb, + const rd_list_t *topics /*(char*)*/, + rd_kafka_AdminOptions_t *options, + char *errstr, + size_t errstr_size, + rd_kafka_replyq_t replyq, + rd_kafka_resp_cb_t *resp_cb, + void *opaque) { + rd_kafka_resp_err_t err; + int include_topic_authorized_operations, cluster_authorized_operations; + + include_topic_authorized_operations = + rd_kafka_confval_get_int(&options->include_topic_authorized_operations); + cluster_authorized_operations = + rd_kafka_confval_get_int(&options->include_cluster_authorized_operations); + /* resp_cb = rd_kafka_admin_handle_response; */ + + // err = Call metadata request with NULL topics + err = rd_kafka_MetadataRequest(rkb, NULL, + "describe cluster", + rd_false, + cluster_authorized_operations, + include_topic_authorized_operations, + rd_false, + NULL, + resp_cb, 1, opaque); + + if (err) { + rd_snprintf(errstr, errstr_size, "%s", + rd_kafka_err2str(err)); + return err; + } + + return RD_KAFKA_RESP_ERR_NO_ERROR; +} +/** + * @brief Parse DescribeCluster and create ADMIN_RESULT op. + */ +static rd_kafka_resp_err_t +rd_kafka_DescribeClusterResponse_parse(rd_kafka_op_t *rko_req, + rd_kafka_op_t **rko_resultp, + rd_kafka_buf_t *reply, + char *errstr, + size_t errstr_size){ + struct rd_kafka_metadata *md = 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_topic_authorized_operations_pair_t* topic_authorized_operations = NULL; + int32_t cluster_authorized_operations; + char* cluster_id = NULL; + int controller_id; + int i; + rd_kafka_op_t *rko_result = NULL; + // rd_kafka_assert(NULL, err == RD_KAFKA_RESP_ERR__DESTROY || + // thrd_is_current(rk->rk_thread)); + // rd_kafka_assert(NULL, err == RD_KAFKA_RESP_ERR__DESTROY); + + err = rd_kafka_parse_Metadata(rkb, NULL, reply, &md, &topic_authorized_operations,&cluster_authorized_operations, &topics, + &cluster_id, &controller_id); + if (err) + goto err; + rko_result = rd_kafka_admin_result_new(rko_req); + rd_list_init(&rko_result->rko_u.admin_result.results, 1, + rd_kafka_ClusterDescription_free); + + rd_list_t* authorized_operations; + authorized_operations = + rd_kafka_AuthorizedOperations_parse(cluster_authorized_operations); + + clusterdesc = rd_kafka_ClusterDescription_new(cluster_id, + controller_id, authorized_operations, md); + rd_free(cluster_id); + + rd_list_add(&rko_result->rko_u.admin_result.results, clusterdesc); + *rko_resultp = rko_result; + return RD_KAFKA_RESP_ERR_NO_ERROR; + +err: + if (rko_result) + rd_kafka_op_destroy(rko_result); + rd_snprintf( + errstr, errstr_size, + "DescribeTopics response protocol parse failure: %s", + rd_kafka_err2str(reply->rkbuf_err)); + return reply->rkbuf_err; +} +void rd_kafka_DescribeCluster(rd_kafka_t *rk, + const rd_kafka_AdminOptions_t *options, + rd_kafka_queue_t *rkqu) { + rd_kafka_op_t *rko; + static const struct rd_kafka_admin_worker_cbs cbs = { + rd_kafka_admin_DescribeClusterRequest, + rd_kafka_DescribeClusterResponse_parse}; + // static const struct rd_kafka_admin_fanout_worker_cbs fanout_cbs = { + // rd_kafka_ClusterDescription_response_merge, + // rd_kafka_ClusterDescription_copy_opaque, + // }; + rko = rd_kafka_admin_request_op_new( + rk, RD_KAFKA_OP_DESCRIBECLUSTER, + RD_KAFKA_EVENT_DESCRIBECLUSTER_RESULT, &cbs, options, + rkqu->rkqu_q); + // rko = rd_kafka_admin_request_op_target_all_new( + // rk, RD_KAFKA_OP_DESCRIBECLUSTER, + // RD_KAFKA_EVENT_DESCRIBECLUSTER_RESULT, &cbs, &fanout_cbs, + // rd_kafka_ClusterDescriptionResult_free, options, rkqu->rkqu_q); + rd_kafka_q_enq(rk->rk_ops, rko); +} +/**@}*/ \ No newline at end of file diff --git a/src/rdkafka_admin.h b/src/rdkafka_admin.h index 62fe9e87a3..463379c569 100644 --- a/src/rdkafka_admin.h +++ b/src/rdkafka_admin.h @@ -91,6 +91,21 @@ struct rd_kafka_AdminOptions_s { * Valid for: * ListConsumerGroupOffsets */ + + rd_kafka_confval_t + include_topic_authorized_operations; /**< BOOL: Whether broker should return + * topic authorized operations. + * Valid for: + * DescribeTopic + * MetadataRequest + */ + rd_kafka_confval_t + include_cluster_authorized_operations; /**< BOOL: Whether broker should return + * cluster authorized operations. + * Valid for: + * DescribeCluster + * MetadataRequest + */ rd_kafka_confval_t match_consumer_group_states; /**< PTR: list of consumer group states @@ -479,4 +494,38 @@ struct rd_kafka_ConsumerGroupDescription_s { /**@}*/ +/** + * @name DescribeTopics + * @{ + */ +/** + * @struct DescribeTopics result + */ +struct rd_kafka_TopicDescription_s { + char *topic; /**< Topic name */ + int partition_cnt; /**< Number of partitions in \p partitions*/ + struct rd_kafka_metadata_partition *partitions; /**< Partitions */ + rd_kafka_resp_err_t err; /**< Topic error reported by broker */ + rd_list_t* topic_authorized_operations; /**< ACL operations allowed for topics */ +}; + +/**@}*/ + +/** + * @name DescribeCluster + * @{ + */ +/** + * @struct DescribeCluster result + */ +struct rd_kafka_ClusterDescription_s { + char *cluster_id; /**< current cluster id in \p cluster*/ + int controller_id; /**< current controller id in \p cluster*/ + int node_cnt; /**< Number of brokers in \p cluster*/ + rd_kafka_Node_t *Nodes; /**< Nodes */ + rd_list_t* cluster_authorized_operations; /**< ACL operations allowed for cluster */ +}; + +/**@}*/ + #endif /* _RDKAFKA_ADMIN_H_ */ diff --git a/src/rdkafka_cgrp.c b/src/rdkafka_cgrp.c index a1004e5833..17fb813526 100644 --- a/src/rdkafka_cgrp.c +++ b/src/rdkafka_cgrp.c @@ -2034,7 +2034,7 @@ static void rd_kafka_cgrp_handle_JoinGroup(rd_kafka_t *rk, * avoid triggering a rejoin or error propagation * on receiving the response since some topics * may be missing. */ - rd_false, rko); + rd_false, rko, NULL, 0, NULL); rd_list_destroy(&topics); } else { diff --git a/src/rdkafka_event.c b/src/rdkafka_event.c index ffd1a17805..9571ebedc4 100644 --- a/src/rdkafka_event.c +++ b/src/rdkafka_event.c @@ -68,6 +68,10 @@ const char *rd_kafka_event_name(const rd_kafka_event_t *rkev) { return "ListConsumerGroupsResult"; case RD_KAFKA_EVENT_DESCRIBECONSUMERGROUPS_RESULT: return "DescribeConsumerGroupsResult"; + case RD_KAFKA_EVENT_DESCRIBETOPICS_RESULT: + return "DescribeTopicsResult"; + case RD_KAFKA_EVENT_DESCRIBECLUSTER_RESULT: + return "DescribeClusterResult"; case RD_KAFKA_EVENT_DELETEGROUPS_RESULT: return "DeleteGroupsResult"; case RD_KAFKA_EVENT_DELETECONSUMERGROUPOFFSETS_RESULT: @@ -364,6 +368,24 @@ rd_kafka_event_DescribeConsumerGroups_result(rd_kafka_event_t *rkev) { return (const rd_kafka_DescribeConsumerGroups_result_t *)rkev; } +const rd_kafka_DescribeTopics_result_t * +rd_kafka_event_DescribeTopics_result(rd_kafka_event_t *rkev) { + if (!rkev || + rkev->rko_evtype != RD_KAFKA_EVENT_DESCRIBETOPICS_RESULT) + return NULL; + else + return (const rd_kafka_DescribeTopics_result_t *)rkev; +} + +const rd_kafka_DescribeCluster_result_t * +rd_kafka_event_DescribeCluster_result(rd_kafka_event_t *rkev) { + if (!rkev || + rkev->rko_evtype != RD_KAFKA_EVENT_DESCRIBECLUSTER_RESULT) + return NULL; + else + return (const rd_kafka_DescribeCluster_result_t *)rkev; +} + const rd_kafka_DeleteGroups_result_t * rd_kafka_event_DeleteGroups_result(rd_kafka_event_t *rkev) { if (!rkev || rkev->rko_evtype != RD_KAFKA_EVENT_DELETEGROUPS_RESULT) diff --git a/src/rdkafka_event.h b/src/rdkafka_event.h index 3f9c22e34b..5abf9ea2b3 100644 --- a/src/rdkafka_event.h +++ b/src/rdkafka_event.h @@ -102,6 +102,8 @@ static RD_UNUSED RD_INLINE int rd_kafka_event_setup(rd_kafka_t *rk, case RD_KAFKA_EVENT_DELETERECORDS_RESULT: case RD_KAFKA_EVENT_LISTCONSUMERGROUPS_RESULT: case RD_KAFKA_EVENT_DESCRIBECONSUMERGROUPS_RESULT: + case RD_KAFKA_EVENT_DESCRIBETOPICS_RESULT: + case RD_KAFKA_EVENT_DESCRIBECLUSTER_RESULT: case RD_KAFKA_EVENT_DELETEGROUPS_RESULT: case RD_KAFKA_EVENT_DELETECONSUMERGROUPOFFSETS_RESULT: case RD_KAFKA_EVENT_CREATEACLS_RESULT: diff --git a/src/rdkafka_metadata.c b/src/rdkafka_metadata.c index aabbd92035..e896b62ee8 100644 --- a/src/rdkafka_metadata.c +++ b/src/rdkafka_metadata.c @@ -92,7 +92,7 @@ rd_kafka_metadata(rd_kafka_t *rk, * topics in the cluster, since a * partial request may make it seem * like some subscribed topics are missing. */ - all_topics ? rd_true : rd_false, rko); + all_topics ? rd_true : rd_false, rko, NULL, 0, NULL); rd_list_destroy(&topics); rd_kafka_broker_destroy(rkb); @@ -275,7 +275,11 @@ rd_bool_t rd_kafka_has_reliable_leader_epochs(rd_kafka_broker_t *rkb) { * @brief Handle a Metadata response message. * * @param topics are the requested topics (may be NULL) - * + * @param topic_authorized_operations topics mapped to their acl operations allowed + * @param cluster_authorized_operations acl operations allowed for cluster + * @param rk_cluster_id returns the current cluster_id (char*) + * @param rk_controller_id returns the current controller_id (int) + * * The metadata will be marshalled into 'struct rd_kafka_metadata*' structs. * * The marshalled metadata is returned in \p *mdp, (NULL on error). @@ -289,7 +293,10 @@ rd_kafka_resp_err_t rd_kafka_parse_Metadata(rd_kafka_broker_t *rkb, rd_kafka_buf_t *rkbuf, struct rd_kafka_metadata **mdp, rd_kafka_topic_authorized_operations_pair_t** topic_authorized_operations, - int32_t* cluster_authorized_operations) { + int32_t* cluster_authorized_operations, + rd_list_t* request_topics, + char** rk_cluster_id, + int* rk_controller_id) { rd_kafka_t *rk = rkb->rkb_rk; int i, j, k; rd_tmpabuf_t tbuf; @@ -297,14 +304,16 @@ rd_kafka_resp_err_t 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 char *reason = request->rkbuf_u.Metadata.reason + const rd_list_t *requested_topics = request ? request->rkbuf_u.Metadata.topics + : request_topics; + rd_bool_t all_topics = request ? request->rkbuf_u.Metadata.all_topics : rd_false; + rd_bool_t cgrp_update = request ? + (request->rkbuf_u.Metadata.cgrp_update && rk->rk_cgrp) : rd_false; + const char *reason = request ? (request->rkbuf_u.Metadata.reason ? request->rkbuf_u.Metadata.reason - : "(no reason)"; - int ApiVersion = request->rkbuf_reqhdr.ApiVersion; + : "(no reason)") : "(admin request)"; + /* changed from request->rkbuf_reqhdr.ApiVersion as request buffer may be NULL*/ + int ApiVersion = rkbuf->rkbuf_reqhdr.ApiVersion; rd_kafkap_str_t cluster_id = RD_ZERO_INIT; int32_t controller_id = -1; rd_kafka_resp_err_t err = RD_KAFKA_RESP_ERR_NO_ERROR; @@ -376,11 +385,15 @@ rd_kafka_resp_err_t rd_kafka_parse_Metadata(rd_kafka_broker_t *rkb, rd_kafka_buf_skip_tags(rkbuf); } - if (ApiVersion >= 2) + if (ApiVersion >= 2){ rd_kafka_buf_read_str(rkbuf, &cluster_id); + *rk_cluster_id = RD_KAFKAP_STR_DUP(&cluster_id); + } + if (ApiVersion >= 1) { rd_kafka_buf_read_i32(rkbuf, &controller_id); + *rk_controller_id = controller_id; rd_rkb_dbg(rkb, METADATA, "METADATA", "ClusterId: %.*s, ControllerId: %" PRId32, RD_KAFKAP_STR_PR(&cluster_id), controller_id); @@ -400,7 +413,7 @@ rd_kafka_resp_err_t rd_kafka_parse_Metadata(rd_kafka_broker_t *rkb, if ((ApiVersion >= 8 && ApiVersion <= 10) && topic_authorized_operations){ *topic_authorized_operations = rd_malloc(sizeof(rd_kafka_topic_authorized_operations_pair_t) * md->topic_cnt); - } + } for (i = 0; i < md->topic_cnt; i++) { rd_kafka_buf_read_i16a(rkbuf, md->topics[i].err); @@ -526,9 +539,6 @@ rd_kafka_resp_err_t rd_kafka_parse_Metadata(rd_kafka_broker_t *rkb, &TopicAuthorizedOperations); (*topic_authorized_operations)[i].topic_name = md->topics[i].topic; (*topic_authorized_operations)[i].authorized_operations = TopicAuthorizedOperations; - printf("%s topic has %d authorized operations\n", - (*topic_authorized_operations)[i].topic_name, - (*topic_authorized_operations)[i].authorized_operations); } rd_kafka_buf_skip_tags(rkbuf); @@ -602,7 +612,6 @@ rd_kafka_resp_err_t rd_kafka_parse_Metadata(rd_kafka_broker_t *rkb, /* ClusterAuthorizedOperations */ rd_kafka_buf_read_i32(rkbuf, &ClusterAuthorizedOperations); *cluster_authorized_operations = ClusterAuthorizedOperations; - printf("Cluster authorized operations: %d\n", ClusterAuthorizedOperations); } rd_kafka_buf_skip_tags(rkbuf); @@ -1058,7 +1067,7 @@ rd_kafka_metadata_refresh_topics(rd_kafka_t *rk, rd_kafka_MetadataRequest(rkb, &q_topics, reason, allow_auto_create, rd_true /*include cluster authorized operations */, rd_true /*include topic authorized operations */, - cgrp_update, NULL); + cgrp_update, NULL, NULL, 0, NULL); rd_list_destroy(&q_topics); @@ -1239,7 +1248,7 @@ rd_kafka_resp_err_t rd_kafka_metadata_refresh_all(rd_kafka_t *rk, rd_false /*no auto create*/, rd_true /*include cluster authorized operations */, rd_true /*include topic authorized operations */, - rd_true /*cgrp update*/, NULL); + rd_true /*cgrp update*/, NULL, NULL, 0, NULL); rd_list_destroy(&topics); if (destroy_rkb) @@ -1281,7 +1290,7 @@ rd_kafka_metadata_request(rd_kafka_t *rk, rd_kafka_MetadataRequest(rkb, topics, reason, allow_auto_create_topics, include_cluster_authorized_operations, include_topic_authorized_operations, - cgrp_update, rko); + cgrp_update, rko, NULL, 0, NULL); if (destroy_rkb) rd_kafka_broker_destroy(rkb); diff --git a/src/rdkafka_metadata.h b/src/rdkafka_metadata.h index 297aa24191..d4cd72954c 100644 --- a/src/rdkafka_metadata.h +++ b/src/rdkafka_metadata.h @@ -38,7 +38,10 @@ rd_kafka_resp_err_t rd_kafka_parse_Metadata(rd_kafka_broker_t *rkb, rd_kafka_buf_t *rkbuf, struct rd_kafka_metadata **mdp, rd_kafka_topic_authorized_operations_pair_t** topic_authorized_operations, - int32_t* cluster_authorized_operations); + int32_t* cluster_authorized_operations, + rd_list_t* request_topics, + char** rk_cluster_id, + int* rk_controller_id); struct rd_kafka_metadata * rd_kafka_metadata_copy(const struct rd_kafka_metadata *md, size_t size); diff --git a/src/rdkafka_op.c b/src/rdkafka_op.c index 128b8bb404..f4ae046d3d 100644 --- a/src/rdkafka_op.c +++ b/src/rdkafka_op.c @@ -84,6 +84,8 @@ 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_DESCRIBETOPICS] = "REPLY:DESCRIBETOPICS", + [RD_KAFKA_OP_DESCRIBECLUSTER] = "REPLY:DESCRIBECLUSTER", [RD_KAFKA_OP_DELETEGROUPS] = "REPLY:DELETEGROUPS", [RD_KAFKA_OP_DELETECONSUMERGROUPOFFSETS] = "REPLY:DELETECONSUMERGROUPOFFSETS", @@ -233,6 +235,8 @@ 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_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), @@ -397,6 +401,8 @@ void rd_kafka_op_destroy(rd_kafka_op_t *rko) { case RD_KAFKA_OP_DESCRIBEACLS: case RD_KAFKA_OP_DELETEACLS: case RD_KAFKA_OP_ALTERCONSUMERGROUPOFFSETS: + case RD_KAFKA_OP_DESCRIBETOPICS: + case RD_KAFKA_OP_DESCRIBECLUSTER: case RD_KAFKA_OP_LISTCONSUMERGROUPOFFSETS: rd_kafka_replyq_destroy(&rko->rko_u.admin_request.replyq); rd_list_destroy(&rko->rko_u.admin_request.args); diff --git a/src/rdkafka_op.h b/src/rdkafka_op.h index aa32750e78..92fd5295fc 100644 --- a/src/rdkafka_op.h +++ b/src/rdkafka_op.h @@ -138,6 +138,12 @@ typedef enum { RD_KAFKA_OP_DESCRIBECONSUMERGROUPS, /**< Admin: * DescribeConsumerGroups * u.admin_request */ + RD_KAFKA_OP_DESCRIBETOPICS, /**< Admin: + * DescribeTopics + * u.admin_request */ + RD_KAFKA_OP_DESCRIBECLUSTER, /**< Admin: + * DescribeCluster + * u.admin_request */ RD_KAFKA_OP_DELETEGROUPS, /**< Admin: DeleteGroups: u.admin_request*/ RD_KAFKA_OP_DELETECONSUMERGROUPOFFSETS, /**< Admin: * DeleteConsumerGroupOffsets diff --git a/src/rdkafka_request.c b/src/rdkafka_request.c index cc03ce691f..56a5d6fb48 100644 --- a/src/rdkafka_request.c +++ b/src/rdkafka_request.c @@ -2088,6 +2088,8 @@ static void rd_kafka_handle_Metadata(rd_kafka_t *rk, struct rd_kafka_metadata *md = NULL; rd_kafka_topic_authorized_operations_pair_t* topic_authorized_operations = NULL; int32_t cluster_authorized_operations; + char* cluster_id = NULL; + int controller_id; const rd_list_t *topics = request->rkbuf_u.Metadata.topics; int actions; @@ -2115,7 +2117,8 @@ static void rd_kafka_handle_Metadata(rd_kafka_t *rk, rd_list_cnt(topics), request->rkbuf_u.Metadata.reason); - err = rd_kafka_parse_Metadata(rkb, request, rkbuf, &md, &topic_authorized_operations,&cluster_authorized_operations); + err = rd_kafka_parse_Metadata(rkb, request, rkbuf, &md, &topic_authorized_operations,&cluster_authorized_operations, NULL, + &cluster_id, &controller_id); if (err) goto err; @@ -2189,11 +2192,18 @@ static void rd_kafka_handle_Metadata(rd_kafka_t *rk, * @param allow_auto_create_topics - allow broker-side auto topic creation. * This is best-effort, depending on broker * config and version. + * @param include_cluster_authorized_operations - request for cluster + * authorized operations. + * @param include_topic_authorized_operations - request for topic authorized + * operations. * @param cgrp_update - Update cgrp in parse_Metadata (see comment there). * @param rko - (optional) rko with replyq for handling response. * Specifying an rko forces a metadata request even if * there is already a matching one in-transit. - * + * @param resp_cb - callback to be used for handling response. If NULL, then + * rd_kafka_handle_Metadata() is used. + * @param force - 1: force a full request. + * 0: check if there are multiple outstanding full requests. * If full metadata for all topics is requested (or all brokers, which * results in all-topics on older brokers) and there is already a full request * in transit then this function will return RD_KAFKA_RESP_ERR__PREV_IN_PROGRESS @@ -2207,17 +2217,20 @@ rd_kafka_resp_err_t rd_kafka_MetadataRequest(rd_kafka_broker_t *rkb, rd_bool_t include_cluster_authorized_operations, rd_bool_t include_topic_authorized_operations, rd_bool_t cgrp_update, - rd_kafka_op_t *rko) { + rd_kafka_op_t *rko, + rd_kafka_resp_cb_t *resp_cb, + int force, + void *opaque) { rd_kafka_buf_t *rkbuf; int16_t ApiVersion = 0; + int i; size_t of_TopicArrayCnt; int features; int topic_cnt = topics ? rd_list_cnt(topics) : 0; int *full_incr = NULL; ApiVersion = rd_kafka_broker_ApiVersion_supported( - rkb, RD_KAFKAP_Metadata, 0, 9, &features); - + rkb, RD_KAFKAP_Metadata, 0, 8, &features); rkbuf = rd_kafka_buf_new_flexver_request(rkb, RD_KAFKAP_Metadata, 1, 4 + (50 * topic_cnt) + 1, ApiVersion >= 9); @@ -2282,7 +2295,7 @@ rd_kafka_resp_err_t rd_kafka_MetadataRequest(rd_kafka_broker_t *rkb, * through regardless. */ mtx_lock(&rkb->rkb_rk->rk_metadata_cache.rkmc_full_lock); - if (*full_incr > 0 && (!rko || !rko->rko_u.metadata.force)) { + if(!force && (*full_incr > 0 && (!rko || !rko->rko_u.metadata.force)) ){ mtx_unlock( &rkb->rkb_rk->rk_metadata_cache.rkmc_full_lock); rd_rkb_dbg(rkb, METADATA, "METADATA", @@ -2339,14 +2352,13 @@ rd_kafka_resp_err_t rd_kafka_MetadataRequest(rd_kafka_broker_t *rkb, if (ApiVersion >= 8 && ApiVersion < 10) { /* TODO: implement KIP-430 */ /* IncludeClusterAuthorizedOperations */ - printf("include cluster authorized operations is: %d\n", include_cluster_authorized_operations); + rd_kafka_buf_write_bool(rkbuf, include_cluster_authorized_operations); } if (ApiVersion >= 8) { /* TODO: implement KIP-430 */ /* IncludeTopicAuthorizedOperations */ - printf("include topic authorized operations is: %d\n", include_topic_authorized_operations); rd_kafka_buf_write_bool(rkbuf, include_topic_authorized_operations); } @@ -2361,7 +2373,8 @@ rd_kafka_resp_err_t rd_kafka_MetadataRequest(rd_kafka_broker_t *rkb, * but forward parsed result to * rko's replyq when done. */ RD_KAFKA_REPLYQ(rkb->rkb_rk->rk_ops, 0), - rd_kafka_handle_Metadata, rko); + resp_cb ? resp_cb : rd_kafka_handle_Metadata, + rko ? rko : opaque); return RD_KAFKA_RESP_ERR_NO_ERROR; } diff --git a/src/rdkafka_request.h b/src/rdkafka_request.h index b1a9a4df8c..d28c9aad23 100644 --- a/src/rdkafka_request.h +++ b/src/rdkafka_request.h @@ -260,7 +260,10 @@ rd_kafka_resp_err_t rd_kafka_MetadataRequest(rd_kafka_broker_t *rkb, rd_bool_t include_cluster_authorized_operations, rd_bool_t include_topic_authorized_operations, rd_bool_t cgrp_update, - rd_kafka_op_t *rko); + rd_kafka_op_t *rko, + rd_kafka_resp_cb_t *resp_cb, + int force, + void *opaque); rd_kafka_resp_err_t rd_kafka_handle_ApiVersion(rd_kafka_t *rk, diff --git a/tests/0080-admin_ut.c b/tests/0080-admin_ut.c index 9d049e5b14..8408974fb1 100644 --- a/tests/0080-admin_ut.c +++ b/tests/0080-admin_ut.c @@ -743,6 +743,250 @@ static void do_test_DescribeConsumerGroups(const char *what, SUB_TEST_PASS(); } +/** + * @brief DescribeTopics tests + * + * + * + */ +static void do_test_DescribeTopics(const char *what, + rd_kafka_t *rk, + rd_kafka_queue_t *useq, + int with_options, + rd_bool_t destroy) { + rd_kafka_queue_t *q; +#define TEST_DESCRIBE_TOPICS_CNT 4 + const char *topic_names[TEST_DESCRIBE_TOPICS_CNT]; + rd_kafka_AdminOptions_t *options = NULL; + int exp_timeout = MY_SOCKET_TIMEOUT_MS; + int i; + char errstr[512]; + const char *errstr2; + rd_kafka_resp_err_t err; + rd_kafka_error_t *error; + test_timing_t timing; + rd_kafka_event_t *rkev; + const rd_kafka_DescribeTopics_result_t *res; + const rd_kafka_TopicDescription_t **restopics; + size_t restopic_cnt; + void *my_opaque = NULL, *opaque; + + SUB_TEST_QUICK("%s DescribeTopics with %s, timeout %dms", + rd_kafka_name(rk), what, exp_timeout); + + q = useq ? useq : rd_kafka_queue_new(rk); + + for (i = 0; i < TEST_DESCRIBE_TOPICS_CNT; i++) { + topic_names[i] = rd_strdup(test_mk_topic_name(__FUNCTION__, 1)); + } + + if (with_options) { + options = rd_kafka_AdminOptions_new( + rk, RD_KAFKA_ADMIN_OP_DESCRIBETOPICS); + + exp_timeout = MY_SOCKET_TIMEOUT_MS * 2; + err = rd_kafka_AdminOptions_set_request_timeout( + options, exp_timeout, errstr, sizeof(errstr)); + TEST_ASSERT(!err, "%s", rd_kafka_err2str(err)); + if ((error = + rd_kafka_AdminOptions_set_include_topic_authorized_operations( + options, 0))) { + fprintf(stderr, + "%% Failed to set topic authorized operations: %s\n", + rd_kafka_error_string(error)); + rd_kafka_error_destroy(error); + TEST_FAIL("Failed to set topic authorized operations\n"); + } + + if (useq) { + my_opaque = (void *)456; + rd_kafka_AdminOptions_set_opaque(options, my_opaque); + } + } + + TIMING_START(&timing, "DescribeTopics"); + TEST_SAY("Call DescribeTopics, timeout is %dms\n", exp_timeout); + rd_kafka_DescribeTopics( + rk, topic_names, TEST_DESCRIBE_TOPICS_CNT, options, q); + TIMING_ASSERT_LATER(&timing, 0, 50); + + if (destroy) + goto destroy; + + /* Poll result queue */ + TIMING_START(&timing, "DescribeTopics.queue_poll"); + rkev = rd_kafka_queue_poll(q, exp_timeout + 1000); + TIMING_ASSERT_LATER(&timing, exp_timeout - 100, exp_timeout + 100); + TEST_ASSERT(rkev != NULL, "expected result in %dms", exp_timeout); + TEST_SAY("DescribeTopics: got %s in %.3fs\n", + rd_kafka_event_name(rkev), TIMING_DURATION(&timing) / 1000.0f); + + /* Convert event to proper result */ + res = rd_kafka_event_DescribeTopics_result(rkev); + TEST_ASSERT(res, "expected DescribeTopics_result, not %s", + rd_kafka_event_name(rkev)); + + opaque = rd_kafka_event_opaque(rkev); + TEST_ASSERT(opaque == my_opaque, "expected opaque to be %p, not %p", + my_opaque, opaque); + + /* Expecting no error (errors will be per-topic) */ + err = rd_kafka_event_error(rkev); + errstr2 = rd_kafka_event_error_string(rkev); + TEST_ASSERT( + err == RD_KAFKA_RESP_ERR_NO_ERROR, + "expected DescribeTopics to return error %s, not %s (%s)", + rd_kafka_err2str(RD_KAFKA_RESP_ERR_NO_ERROR), rd_kafka_err2str(err), + err ? errstr2 : "n/a"); + + /* Extract topics, should return TEST_DESCRIBE_TOPICS_CNT topics. */ + restopics = + rd_kafka_DescribeTopics_result_topics(res, &restopic_cnt); + TEST_ASSERT(restopics && + restopic_cnt == TEST_DESCRIBE_TOPICS_CNT, + "expected %d result_topics, got %p cnt %" PRIusz, + TEST_DESCRIBE_TOPICS_CNT, restopics, restopic_cnt); + + /* The returned topics should be in the original order, and + * should all have timed out. */ + for (i = 0; i < TEST_DESCRIBE_TOPICS_CNT; i++) { + TEST_ASSERT( + !strcmp(topic_names[i], + rd_kafka_TopicDescription_topic_name( + restopics[i])), + "expected topic '%s' at position %d, not '%s'", + topic_names[i], i, + rd_kafka_TopicDescription_topic_name(restopics[i])); + TEST_ASSERT( + rd_kafka_error_code(rd_kafka_TopicDescription_error( + restopics[i])) == RD_KAFKA_RESP_ERR__TIMED_OUT, + "expected topic '%s' to have timed out, got %s", + topic_names[i], + rd_kafka_error_string( + rd_kafka_TopicDescription_error(restopics[i]))); + TEST_ASSERT( + rd_kafka_TopicDescription_topic_authorized_operations_cnt( + restopics[i]) == 0, "Got topic authorized operations" + "when not requested"); + } + + rd_kafka_event_destroy(rkev); + +destroy: + for (i = 0; i < TEST_DESCRIBE_TOPICS_CNT; i++) { + rd_free((char *)topic_names[i]); + } + + if (options) + rd_kafka_AdminOptions_destroy(options); + + if (!useq) + rd_kafka_queue_destroy(q); +#undef TEST_DESCRIBE_TOPICS_CNT + + SUB_TEST_PASS(); +} + +/** + * @brief DescribeCluster tests + * + * + * + */ +static void do_test_DescribeCluster(const char *what, + rd_kafka_t *rk, + rd_kafka_queue_t *useq, + int with_options, + rd_bool_t destroy) { + rd_kafka_queue_t *q; + rd_kafka_AdminOptions_t *options = NULL; + int exp_timeout = MY_SOCKET_TIMEOUT_MS; + int i; + char errstr[512]; + const char *errstr2; + rd_kafka_resp_err_t err; + rd_kafka_error_t *error; + test_timing_t timing; + rd_kafka_event_t *rkev; + const rd_kafka_DescribeCluster_result_t *res; + const rd_kafka_ClusterDescription_t *clusterdesc; + void *my_opaque = NULL, *opaque; + + SUB_TEST_QUICK("%s DescribeCluster with %s, timeout %dms", + rd_kafka_name(rk), what, exp_timeout); + + q = useq ? useq : rd_kafka_queue_new(rk); + + if (with_options) { + options = rd_kafka_AdminOptions_new( + rk, RD_KAFKA_ADMIN_OP_DESCRIBECLUSTER); + + exp_timeout = MY_SOCKET_TIMEOUT_MS * 2; + err = rd_kafka_AdminOptions_set_request_timeout( + options, exp_timeout, errstr, sizeof(errstr)); + TEST_ASSERT(!err, "%s", rd_kafka_err2str(err)); + if ((error = + rd_kafka_AdminOptions_set_include_cluster_authorized_operations( + options, 0))) { + fprintf(stderr, + "%% Failed to set cluster authorized operations: %s\n", + rd_kafka_error_string(error)); + rd_kafka_error_destroy(error); + TEST_FAIL("Failed to set cluster authorized operations\n"); + } + + if (useq) { + my_opaque = (void *)456; + rd_kafka_AdminOptions_set_opaque(options, my_opaque); + } + } + + TIMING_START(&timing, "DescribeCluster"); + TEST_SAY("Call DescribeCluster, timeout is %dms\n", exp_timeout); + rd_kafka_DescribeCluster(rk, options, q); + TIMING_ASSERT_LATER(&timing, 0, 50); + + if (destroy) + goto destroy; + + /* Poll result queue */ + TIMING_START(&timing, "DescribeCluster.queue_poll"); + rkev = rd_kafka_queue_poll(q, exp_timeout + 1000); + TIMING_ASSERT_LATER(&timing, exp_timeout - 100, exp_timeout + 100); + TEST_ASSERT(rkev != NULL, "expected result in %dms", exp_timeout); + TEST_SAY("DescribeCluster: got %s in %.3fs\n", + rd_kafka_event_name(rkev), TIMING_DURATION(&timing) / 1000.0f); + + /* Convert event to proper result */ + res = rd_kafka_event_DescribeCluster_result(rkev); + TEST_ASSERT(res, "expected DescribeCluster_result, not %s", + rd_kafka_event_name(rkev)); + + opaque = rd_kafka_event_opaque(rkev); + TEST_ASSERT(opaque == my_opaque, "expected opaque to be %p, not %p", + my_opaque, opaque); + + /* Expecting error (Fail while waiting for controller)*/ + err = rd_kafka_event_error(rkev); + errstr2 = rd_kafka_event_error_string(rkev); + TEST_ASSERT(err == RD_KAFKA_RESP_ERR__TIMED_OUT, + "expected DescribeCluster to return error %s, not %s (%s)", + rd_kafka_err2str(RD_KAFKA_RESP_ERR__TIMED_OUT), + rd_kafka_err2str(err), err ? errstr2 : "n/a"); + + rd_kafka_event_destroy(rkev); + +destroy: + + if (options) + rd_kafka_AdminOptions_destroy(options); + + if (!useq) + rd_kafka_queue_destroy(q); + + SUB_TEST_PASS(); +} + static void do_test_DeleteRecords(const char *what, rd_kafka_t *rk, rd_kafka_queue_t *useq, @@ -2444,6 +2688,20 @@ static void do_test_apis(rd_kafka_type_t cltype) { do_test_DescribeConsumerGroups("main queue, options", rk, mainq, 1, rd_false); + do_test_DescribeTopics("temp queue, no options", rk, NULL, 0, + rd_false); + do_test_DescribeTopics("temp queue, options", rk, NULL, 1, + rd_false); + do_test_DescribeTopics("main queue, options", rk, mainq, 1, + rd_false); + + do_test_DescribeCluster("temp queue, no options", rk, NULL, 0, + rd_false); + do_test_DescribeCluster("temp queue, options", rk, NULL, 1, + rd_false); + do_test_DescribeCluster("main queue, options", rk, mainq, 1, + rd_false); + do_test_DeleteGroups("temp queue, no options", rk, NULL, 0, rd_false); do_test_DeleteGroups("temp queue, options", rk, NULL, 1, rd_false); do_test_DeleteGroups("main queue, options", rk, mainq, 1, rd_false); diff --git a/tests/0081-admin.c b/tests/0081-admin.c index 7da2dff156..5f5680e4a4 100644 --- a/tests/0081-admin.c +++ b/tests/0081-admin.c @@ -2859,6 +2859,465 @@ static void do_test_DescribeConsumerGroups(const char *what, SUB_TEST_PASS(); } +/** + * @brief Test DescribeTopics + */ +static void do_test_DescribeTopics(const char *what, rd_kafka_t *rk, rd_kafka_queue_t *rkqu, + int request_timeout) { +#define MY_TOPICS_CNT 3 + rd_kafka_queue_t *q; + char *topics[MY_TOPICS_CNT]; + rd_kafka_AdminOptions_t *options; + rd_kafka_event_t *rkev; + rd_kafka_error_t *error; + rd_kafka_resp_err_t err; + test_timing_t timing; + const rd_kafka_DescribeTopics_result_t *res; + const rd_kafka_TopicDescription_t **result_topics; + size_t result_topics_cnt; + char errstr[128]; + const char *errstr2; + const char *user_test1 = "User:broker"; + rd_kafka_AclBinding_t *acl_bindings[1]; + rd_kafka_AdminOptions_t *admin_options; + const rd_kafka_CreateAcls_result_t *acl_res; + const rd_kafka_acl_result_t **acl_res_acls; + rd_kafka_event_t *rkev_acl_create; + size_t resacl_cnt; + int i; + int initial_acl_cnt, final_acl_cnt; + + SUB_TEST_QUICK("%s DescribeTopics with %s, request_timeout %d", + rd_kafka_name(rk), what, request_timeout); + + q = rkqu ? rkqu : rd_kafka_queue_new(rk); + /* + * Only create one topic, the others will be non-existent. + */ + for (i = 0; i < MY_TOPICS_CNT; i++){ + rd_strdupa(&topics[i], test_mk_topic_name(__FUNCTION__, 1)); + } + test_CreateTopics_simple(rk, NULL, topics, 1, 1, NULL); + + test_wait_topic_exists(rk, topics[0], 10000); + + /* + * Timeout options + */ + options = rd_kafka_AdminOptions_new(rk, RD_KAFKA_ADMIN_OP_DESCRIBETOPICS); + + err = rd_kafka_AdminOptions_set_request_timeout(options, request_timeout, errstr, + sizeof(errstr)); + TEST_ASSERT(!err, "%s", errstr); + if ((error = rd_kafka_AdminOptions_set_include_topic_authorized_operations( + options, 1))) { + fprintf(stderr, + "%% Failed to set require authorized operations: %s\n", + rd_kafka_error_string(error)); + rd_kafka_error_destroy(error); + TEST_FAIL("Failed to set include authorized operations\n"); + } + + TIMING_START(&timing, "DescribeTopics"); + TEST_SAY("Call DescribeTopics\n"); + rd_kafka_DescribeTopics(rk, topics, MY_TOPICS_CNT, options, q); + TIMING_ASSERT_LATER(&timing, 0, 50); + + TIMING_START(&timing, "DescribeTopics.queue_poll"); + + /* Poll result queue for DescribeTopics result. + * Print but otherwise ignore other event types + * (typically generic Error events). */ + while (1) { + rkev = rd_kafka_queue_poll(q, tmout_multip(20 * 1000)); + TEST_SAY("DescribeTopics: got %s in %.3fms\n", + rd_kafka_event_name(rkev), + TIMING_DURATION(&timing) / 1000.0f); + if (rkev == NULL) + continue; + if (rd_kafka_event_error(rkev)) + TEST_SAY("%s: %s\n", rd_kafka_event_name(rkev), + rd_kafka_event_error_string(rkev)); + if (rd_kafka_event_type(rkev) == + RD_KAFKA_EVENT_DESCRIBETOPICS_RESULT) { + break; + } + rd_kafka_event_destroy(rkev); + } + rd_kafka_AdminOptions_destroy(options); + + /* + * Extract result + */ + res = rd_kafka_event_DescribeTopics_result(rkev); + TEST_ASSERT(res, "Expected DescribeTopics result, not %s", + rd_kafka_event_name(rkev)); + + err = rd_kafka_event_error(rkev); + errstr2 = rd_kafka_event_error_string(rkev); + TEST_ASSERT(!err, "Expected success, not %s: %s", + rd_kafka_err2name(err), errstr2); + + result_topics = rd_kafka_DescribeTopics_result_topics( + res, &result_topics_cnt); + + /* + * Check if results have been received for all topics + */ + TEST_ASSERT(result_topics_cnt == MY_TOPICS_CNT, "Number of topics in result didnt match"); + + /* + * Check if topics[0] succeeded. + */ + TEST_ASSERT(rd_kafka_error_code( + rd_kafka_TopicDescription_error(result_topics[0])) == + RD_KAFKA_RESP_ERR_NO_ERROR, + "Expected expected unknown Topic or partition, not %s\n", + rd_kafka_error_string(rd_kafka_TopicDescription_error(result_topics[0]))); + + /* + * Check whether the topics which are non-existent have + * RD_KAFKA_RESP_ERR_UNKNOWN_TOPIC_OR_PART error. + */ + for(i=1; i 0, + "Expected 8 acl operations allowed"); + rd_kafka_event_destroy(rkev); + + acl_bindings[0] = rd_kafka_AclBinding_new( + RD_KAFKA_RESOURCE_TOPIC, topics[0], RD_KAFKA_RESOURCE_PATTERN_LITERAL, + user_test1, "*", RD_KAFKA_ACL_OPERATION_READ, + RD_KAFKA_ACL_PERMISSION_TYPE_ALLOW, NULL, 0); + admin_options = + rd_kafka_AdminOptions_new(rk, RD_KAFKA_ADMIN_OP_CREATEACLS); + err = rd_kafka_AdminOptions_set_request_timeout(admin_options, 10000, + errstr, sizeof(errstr)); + TEST_ASSERT(!err, "%s", errstr); + rd_kafka_CreateAcls(rk, acl_bindings, 1, admin_options, q); + rkev_acl_create = test_wait_admin_result( + q, RD_KAFKA_EVENT_CREATEACLS_RESULT, 10000 + 1000); + acl_res = rd_kafka_event_CreateAcls_result(rkev_acl_create); + acl_res_acls = rd_kafka_CreateAcls_result_acls(acl_res, &resacl_cnt); + rd_kafka_event_destroy(rkev_acl_create); + rd_kafka_AdminOptions_destroy(admin_options); + /* + * Timeout options + */ + options = rd_kafka_AdminOptions_new(rk, RD_KAFKA_ADMIN_OP_DESCRIBETOPICS); + + err = rd_kafka_AdminOptions_set_request_timeout(options, request_timeout, errstr, + sizeof(errstr)); + TEST_ASSERT(!err, "%s", errstr); + if ((error = rd_kafka_AdminOptions_set_include_topic_authorized_operations( + options, 1))) { + fprintf(stderr, + "%% Failed to set require authorized operations: %s\n", + rd_kafka_error_string(error)); + rd_kafka_error_destroy(error); + TEST_FAIL("Failed to set include authorized operations\n"); + } + + TIMING_START(&timing, "DescribeTopics"); + TEST_SAY("Call DescribeTopics\n"); + rd_kafka_DescribeTopics(rk, topics, MY_TOPICS_CNT, options, q); + TIMING_ASSERT_LATER(&timing, 0, 50); + + TIMING_START(&timing, "DescribeTopics.queue_poll"); + + /* Poll result queue for DescribeTopics result. + * Print but otherwise ignore other event types + * (typically generic Error events). */ + while (1) { + rkev = rd_kafka_queue_poll(q, tmout_multip(20 * 1000)); + TEST_SAY("DescribeTopics: got %s in %.3fms\n", + rd_kafka_event_name(rkev), + TIMING_DURATION(&timing) / 1000.0f); + if (rkev == NULL) + continue; + if (rd_kafka_event_error(rkev)) + TEST_SAY("%s: %s\n", rd_kafka_event_name(rkev), + rd_kafka_event_error_string(rkev)); + if (rd_kafka_event_type(rkev) == + RD_KAFKA_EVENT_DESCRIBETOPICS_RESULT) { + break; + } + rd_kafka_event_destroy(rkev); + } + rd_kafka_AdminOptions_destroy(options); + + /* + * Extract result + */ + res = rd_kafka_event_DescribeTopics_result(rkev); + TEST_ASSERT(res, "Expected DescribeTopics result, not %s", + rd_kafka_event_name(rkev)); + + err = rd_kafka_event_error(rkev); + errstr2 = rd_kafka_event_error_string(rkev); + TEST_ASSERT(!err, "Expected success, not %s: %s", + rd_kafka_err2name(err), errstr2); + + result_topics = rd_kafka_DescribeTopics_result_topics( + res, &result_topics_cnt); + + /* + * Check if results have been received for all topics + */ + TEST_ASSERT(result_topics_cnt == MY_TOPICS_CNT, "Number of topics in result didnt match"); + + /* + * Check if topics[0] succeeded. + */ + TEST_ASSERT(rd_kafka_error_code( + rd_kafka_TopicDescription_error(result_topics[0])) == + RD_KAFKA_RESP_ERR_NO_ERROR, + "Expected expected unknown Topic or partition, not %s\n", + rd_kafka_error_string(rd_kafka_TopicDescription_error(result_topics[0]))); + final_acl_cnt = rd_kafka_TopicDescription_topic_authorized_operations_cnt( + result_topics[0]); + + /* + * Initally count should be 8. After createAcls call with + * only RD_KAFKA_ACL_OPERATION_READ allowed, it should reduce + * to 2 (read and describe). 8>2 should hold. + */ + TEST_ASSERT(initial_acl_cnt > final_acl_cnt, + "Expected the acl operations allowed to have reduced after" + " call to CreateAcls"); + + /* + * Allow RD_KAFKA_ACL_OPERATION_DELETE to allow deletion + * of the created topic as currently it only has read and + * describe. + */ + acl_bindings[0] = rd_kafka_AclBinding_new( + RD_KAFKA_RESOURCE_TOPIC, topics[0], RD_KAFKA_RESOURCE_PATTERN_LITERAL, + user_test1, "*", RD_KAFKA_ACL_OPERATION_DELETE, + RD_KAFKA_ACL_PERMISSION_TYPE_ALLOW, NULL, 0); + admin_options = + rd_kafka_AdminOptions_new(rk, RD_KAFKA_ADMIN_OP_CREATEACLS); + err = rd_kafka_AdminOptions_set_request_timeout(admin_options, 10000, + errstr, sizeof(errstr)); + TEST_ASSERT(!err, "%s", errstr); + rd_kafka_CreateAcls(rk, acl_bindings, 1, admin_options, q); + rkev_acl_create = test_wait_admin_result( + q, RD_KAFKA_EVENT_CREATEACLS_RESULT, 10000 + 1000); + acl_res = rd_kafka_event_CreateAcls_result(rkev_acl_create); + acl_res_acls = rd_kafka_CreateAcls_result_acls(acl_res, &resacl_cnt); + rd_kafka_event_destroy(rkev_acl_create); + rd_kafka_AdminOptions_destroy(admin_options); + rd_kafka_event_destroy(rkev); + test_DeleteTopics_simple(rk, q, topics, 1, NULL); + + TEST_LATER_CHECK(); +#undef MY_TOPICS_CNT + + SUB_TEST_PASS(); +} + +/** + * @brief Test DescribeCluster + */ +static void do_test_DescribeCluster(const char *what, rd_kafka_t *rk, rd_kafka_queue_t *rkqu, + int request_timeout) { + rd_kafka_queue_t *q; + rd_kafka_AdminOptions_t *options; + rd_kafka_event_t *rkev; + rd_kafka_error_t *error; + rd_kafka_resp_err_t err; + test_timing_t timing; + const rd_kafka_DescribeCluster_result_t *res; + const rd_kafka_ClusterDescription_t *result_cluster; + char errstr[128]; + const char *errstr2; + const char *user_test1 = "User:broker"; + rd_kafka_AclBinding_t *acl_bindings[1]; + rd_kafka_AdminOptions_t *admin_options; + const rd_kafka_CreateAcls_result_t *acl_res; + const rd_kafka_acl_result_t **acl_res_acls; + rd_kafka_event_t *rkev_acl_create; + size_t resacl_cnt; + int i; + int initial_acl_cnt, final_acl_cnt; + + SUB_TEST_QUICK("%s DescribeCluster with %s, request_timeout %d", + rd_kafka_name(rk), what, request_timeout); + + q = rkqu ? rkqu : rd_kafka_queue_new(rk); + + /* + * Timeout options + */ + options = rd_kafka_AdminOptions_new(rk, RD_KAFKA_ADMIN_OP_DESCRIBECLUSTER); + + err = rd_kafka_AdminOptions_set_request_timeout(options, request_timeout, errstr, + sizeof(errstr)); + TEST_ASSERT(!err, "%s", errstr); + if ((error = rd_kafka_AdminOptions_set_include_cluster_authorized_operations( + options, 1))) { + fprintf(stderr, + "%% Failed to set require authorized operations: %s\n", + rd_kafka_error_string(error)); + rd_kafka_error_destroy(error); + TEST_FAIL("Failed to set include authorized operations\n"); + } + + TIMING_START(&timing, "DescribeCluster"); + TEST_SAY("Call DescribeCluster\n"); + rd_kafka_DescribeCluster(rk, options, q); + TIMING_ASSERT_LATER(&timing, 0, 50); + + TIMING_START(&timing, "DescribeCluster.queue_poll"); + + /* Poll result queue for DescribeCluster result. + * Print but otherwise ignore other event types + * (typically generic Error events). */ + while (1) { + rkev = rd_kafka_queue_poll(q, tmout_multip(20 * 1000)); + TEST_SAY("DescribeCluster: got %s in %.3fms\n", + rd_kafka_event_name(rkev), + TIMING_DURATION(&timing) / 1000.0f); + if (rkev == NULL) + continue; + if (rd_kafka_event_error(rkev)) + TEST_SAY("%s: %s\n", rd_kafka_event_name(rkev), + rd_kafka_event_error_string(rkev)); + if (rd_kafka_event_type(rkev) == + RD_KAFKA_EVENT_DESCRIBECLUSTER_RESULT) { + break; + } + rd_kafka_event_destroy(rkev); + } + rd_kafka_AdminOptions_destroy(options); + + /* + * Extract result + */ + res = rd_kafka_event_DescribeCluster_result(rkev); + TEST_ASSERT(res, "Expected DescribeCluster result, not %s", + rd_kafka_event_name(rkev)); + + err = rd_kafka_event_error(rkev); + errstr2 = rd_kafka_event_error_string(rkev); + TEST_ASSERT(!err, "Expected success, not %s: %s", + rd_kafka_err2name(err), errstr2); + + result_cluster = rd_kafka_DescribeCluster_result_description( + res); + + initial_acl_cnt = rd_kafka_ClusterDescription_cluster_acl_operations_cnt( + result_cluster); + + TEST_ASSERT(initial_acl_cnt> 0, + "Expected 7 acl operations allowed"); + TEST_SAY("initial count is: %d\n", initial_acl_cnt); + rd_kafka_event_destroy(rkev); + + acl_bindings[0] = rd_kafka_AclBinding_new( + RD_KAFKA_RESOURCE_BROKER, "kafka-cluster", RD_KAFKA_RESOURCE_PATTERN_LITERAL, + user_test1, "*", RD_KAFKA_ACL_OPERATION_DESCRIBE, + RD_KAFKA_ACL_PERMISSION_TYPE_ALLOW, NULL, 0); + admin_options = + rd_kafka_AdminOptions_new(rk, RD_KAFKA_ADMIN_OP_CREATEACLS); + err = rd_kafka_AdminOptions_set_request_timeout(admin_options, 10000, + errstr, sizeof(errstr)); + TEST_ASSERT(!err, "%s", errstr); + rd_kafka_CreateAcls(rk, acl_bindings, 1, admin_options, q); + rkev_acl_create = test_wait_admin_result( + q, RD_KAFKA_EVENT_CREATEACLS_RESULT, 10000 + 1000); + acl_res = rd_kafka_event_CreateAcls_result(rkev_acl_create); + acl_res_acls = rd_kafka_CreateAcls_result_acls(acl_res, &resacl_cnt); + rd_kafka_event_destroy(rkev_acl_create); + rd_kafka_AdminOptions_destroy(admin_options); + /* + * Timeout options + */ + options = rd_kafka_AdminOptions_new(rk, RD_KAFKA_ADMIN_OP_DESCRIBECLUSTER); + + err = rd_kafka_AdminOptions_set_request_timeout(options, request_timeout, errstr, + sizeof(errstr)); + TEST_ASSERT(!err, "%s", errstr); + if ((error = rd_kafka_AdminOptions_set_include_cluster_authorized_operations( + options, 1))) { + fprintf(stderr, + "%% Failed to set require authorized operations: %s\n", + rd_kafka_error_string(error)); + rd_kafka_error_destroy(error); + TEST_FAIL("Failed to set include authorized operations\n"); + } + + TIMING_START(&timing, "DescribeCluster"); + TEST_SAY("Call DescribeCluster\n"); + rd_kafka_DescribeCluster(rk, options, q); + TIMING_ASSERT_LATER(&timing, 0, 50); + + TIMING_START(&timing, "DescribeCluster.queue_poll"); + + /* Poll result queue for DescribeCluster result. + * Print but otherwise ignore other event types + * (typically generic Error events). */ + while (1) { + rkev = rd_kafka_queue_poll(q, tmout_multip(20 * 1000)); + TEST_SAY("DescribeCluster: got %s in %.3fms\n", + rd_kafka_event_name(rkev), + TIMING_DURATION(&timing) / 1000.0f); + if (rkev == NULL) + continue; + if (rd_kafka_event_error(rkev)) + TEST_SAY("%s: %s\n", rd_kafka_event_name(rkev), + rd_kafka_event_error_string(rkev)); + if (rd_kafka_event_type(rkev) == + RD_KAFKA_EVENT_DESCRIBECLUSTER_RESULT) { + break; + } + rd_kafka_event_destroy(rkev); + } + rd_kafka_AdminOptions_destroy(options); + + /* + * Extract result + */ + res = rd_kafka_event_DescribeCluster_result(rkev); + TEST_ASSERT(res, "Expected DescribeCluster result, not %s", + rd_kafka_event_name(rkev)); + + err = rd_kafka_event_error(rkev); + errstr2 = rd_kafka_event_error_string(rkev); + TEST_ASSERT(!err, "Expected success, not %s: %s", + rd_kafka_err2name(err), errstr2); + + result_cluster = rd_kafka_DescribeCluster_result_description( + res); + + final_acl_cnt = rd_kafka_ClusterDescription_cluster_acl_operations_cnt( + result_cluster); + + /* + * Initally count should be 7. After createAcls call with + * only RD_KAFKA_ACL_OPERATION_DESCRIBE allowed, it should reduce + * to 1 (describe). 7>1 should hold. + */ + TEST_SAY("final count is: %d\n", final_acl_cnt); + // TEST_ASSERT(initial_acl_cnt > final_acl_cnt, + // "Expected the acl operations allowed to have reduced after" + // " call to CreateAcls"); + + rd_kafka_event_destroy(rkev); + + TEST_LATER_CHECK(); + + SUB_TEST_PASS(); +} + /** * @brief Test deletion of committed offsets. * @@ -3674,105 +4133,113 @@ static void do_test_apis(rd_kafka_type_t cltype) { mainq = rd_kafka_queue_get_main(rk); - /* Create topics */ - do_test_CreateTopics("temp queue, op timeout 0", rk, NULL, 0, 0); - do_test_CreateTopics("temp queue, op timeout 15000", rk, NULL, 15000, - 0); - do_test_CreateTopics( - "temp queue, op timeout 300, " - "validate only", - rk, NULL, 300, rd_true); - do_test_CreateTopics("temp queue, op timeout 9000, validate_only", rk, - NULL, 9000, rd_true); - do_test_CreateTopics("main queue, options", rk, mainq, -1, 0); - - /* Delete topics */ - do_test_DeleteTopics("temp queue, op timeout 0", rk, NULL, 0); - do_test_DeleteTopics("main queue, op timeout 15000", rk, mainq, 1500); - - if (test_broker_version >= TEST_BRKVER(1, 0, 0, 0)) { - /* Create Partitions */ - do_test_CreatePartitions("temp queue, op timeout 6500", rk, - NULL, 6500); - do_test_CreatePartitions("main queue, op timeout 0", rk, mainq, - 0); - } - - /* CreateAcls */ - do_test_CreateAcls(rk, mainq, 0); - do_test_CreateAcls(rk, mainq, 1); - - /* DescribeAcls */ - do_test_DescribeAcls(rk, mainq, 0); - do_test_DescribeAcls(rk, mainq, 1); - - /* DeleteAcls */ - do_test_DeleteAcls(rk, mainq, 0); - do_test_DeleteAcls(rk, mainq, 1); - - /* AlterConfigs */ - do_test_AlterConfigs(rk, mainq); - - /* DescribeConfigs */ - do_test_DescribeConfigs(rk, mainq); - - /* Delete records */ - do_test_DeleteRecords("temp queue, op timeout 0", rk, NULL, 0); - do_test_DeleteRecords("main queue, op timeout 1500", rk, mainq, 1500); - - /* List groups */ - do_test_ListConsumerGroups("temp queue", rk, NULL, -1, rd_false); - do_test_ListConsumerGroups("main queue", rk, mainq, 1500, rd_true); + // /* Create topics */ + // do_test_CreateTopics("temp queue, op timeout 0", rk, NULL, 0, 0); + // do_test_CreateTopics("temp queue, op timeout 15000", rk, NULL, 15000, + // 0); + // do_test_CreateTopics( + // "temp queue, op timeout 300, " + // "validate only", + // rk, NULL, 300, rd_true); + // do_test_CreateTopics("temp queue, op timeout 9000, validate_only", rk, + // NULL, 9000, rd_true); + // do_test_CreateTopics("main queue, options", rk, mainq, -1, 0); + + // /* Delete topics */ + // do_test_DeleteTopics("temp queue, op timeout 0", rk, NULL, 0); + // do_test_DeleteTopics("main queue, op timeout 15000", rk, mainq, 1500); + + // if (test_broker_version >= TEST_BRKVER(1, 0, 0, 0)) { + // /* Create Partitions */ + // do_test_CreatePartitions("temp queue, op timeout 6500", rk, + // NULL, 6500); + // do_test_CreatePartitions("main queue, op timeout 0", rk, mainq, + // 0); + // } + + // /* CreateAcls */ + // do_test_CreateAcls(rk, mainq, 0); + // do_test_CreateAcls(rk, mainq, 1); + + // /* DescribeAcls */ + // do_test_DescribeAcls(rk, mainq, 0); + // do_test_DescribeAcls(rk, mainq, 1); + + // /* DeleteAcls */ + // do_test_DeleteAcls(rk, mainq, 0); + // do_test_DeleteAcls(rk, mainq, 1); + + // /* AlterConfigs */ + // do_test_AlterConfigs(rk, mainq); + + // /* DescribeConfigs */ + // do_test_DescribeConfigs(rk, mainq); + + // /* Delete records */ + // do_test_DeleteRecords("temp queue, op timeout 0", rk, NULL, 0); + // do_test_DeleteRecords("main queue, op timeout 1500", rk, mainq, 1500); + + // /* List groups */ + // do_test_ListConsumerGroups("temp queue", rk, NULL, -1, rd_false); + // do_test_ListConsumerGroups("main queue", rk, mainq, 1500, rd_true); /* Describe groups */ - do_test_DescribeConsumerGroups("temp queue", rk, NULL, -1); - do_test_DescribeConsumerGroups("main queue", rk, mainq, 1500); - - /* Delete groups */ - do_test_DeleteGroups("temp queue", rk, NULL, -1); - do_test_DeleteGroups("main queue", rk, mainq, 1500); - - if (test_broker_version >= TEST_BRKVER(2, 4, 0, 0)) { - /* Delete committed offsets */ - do_test_DeleteConsumerGroupOffsets("temp queue", rk, NULL, -1, - rd_false); - do_test_DeleteConsumerGroupOffsets("main queue", rk, mainq, - 1500, rd_false); - do_test_DeleteConsumerGroupOffsets( - "main queue", rk, mainq, 1500, - rd_true /*with subscribing consumer*/); - - /* Alter committed offsets */ - do_test_AlterConsumerGroupOffsets("temp queue", rk, NULL, -1, - rd_false, rd_true); - do_test_AlterConsumerGroupOffsets("main queue", rk, mainq, 1500, - rd_false, rd_true); - do_test_AlterConsumerGroupOffsets( - "main queue, nonexistent topics", rk, mainq, 1500, rd_false, - rd_false /* don't create topics */); - do_test_AlterConsumerGroupOffsets( - "main queue", rk, mainq, 1500, - rd_true, /*with subscribing consumer*/ - rd_true); - - /* List committed offsets */ - do_test_ListConsumerGroupOffsets("temp queue", rk, NULL, -1, - rd_false, rd_false); - do_test_ListConsumerGroupOffsets( - "main queue, op timeout " - "1500", - rk, mainq, 1500, rd_false, rd_false); - do_test_ListConsumerGroupOffsets( - "main queue", rk, mainq, 1500, - rd_true /*with subscribing consumer*/, rd_false); - do_test_ListConsumerGroupOffsets("temp queue", rk, NULL, -1, - rd_false, rd_true); - do_test_ListConsumerGroupOffsets("main queue", rk, mainq, 1500, - rd_false, rd_true); - do_test_ListConsumerGroupOffsets( - "main queue", rk, mainq, 1500, - rd_true /*with subscribing consumer*/, rd_true); - } + // do_test_DescribeConsumerGroups("temp queue", rk, NULL, -1); + // do_test_DescribeConsumerGroups("main queue", rk, mainq, 1500); + + // /* Describe topics */ + // do_test_DescribeTopics("temp queue", rk, NULL, 15000); + // do_test_DescribeTopics("main queue", rk, mainq, 15000); + + /* Describe cluster */ + do_test_DescribeCluster("temp queue", rk, NULL, 1500); + do_test_DescribeCluster("main queue", rk, mainq, 1500); + + // /* Delete groups */ + // do_test_DeleteGroups("temp queue", rk, NULL, -1); + // do_test_DeleteGroups("main queue", rk, mainq, 1500); + + // if (test_broker_version >= TEST_BRKVER(2, 4, 0, 0)) { + // /* Delete committed offsets */ + // do_test_DeleteConsumerGroupOffsets("temp queue", rk, NULL, -1, + // rd_false); + // do_test_DeleteConsumerGroupOffsets("main queue", rk, mainq, + // 1500, rd_false); + // do_test_DeleteConsumerGroupOffsets( + // "main queue", rk, mainq, 1500, + // rd_true /*with subscribing consumer*/); + + // /* Alter committed offsets */ + // do_test_AlterConsumerGroupOffsets("temp queue", rk, NULL, -1, + // rd_false, rd_true); + // do_test_AlterConsumerGroupOffsets("main queue", rk, mainq, 1500, + // rd_false, rd_true); + // do_test_AlterConsumerGroupOffsets( + // "main queue, nonexistent topics", rk, mainq, 1500, rd_false, + // rd_false /* don't create topics */); + // do_test_AlterConsumerGroupOffsets( + // "main queue", rk, mainq, 1500, + // rd_true, /*with subscribing consumer*/ + // rd_true); + + // /* List committed offsets */ + // do_test_ListConsumerGroupOffsets("temp queue", rk, NULL, -1, + // rd_false, rd_false); + // do_test_ListConsumerGroupOffsets( + // "main queue, op timeout " + // "1500", + // rk, mainq, 1500, rd_false, rd_false); + // do_test_ListConsumerGroupOffsets( + // "main queue", rk, mainq, 1500, + // rd_true /*with subscribing consumer*/, rd_false); + // do_test_ListConsumerGroupOffsets("temp queue", rk, NULL, -1, + // rd_false, rd_true); + // do_test_ListConsumerGroupOffsets("main queue", rk, mainq, 1500, + // rd_false, rd_true); + // do_test_ListConsumerGroupOffsets( + // "main queue", rk, mainq, 1500, + // rd_true /*with subscribing consumer*/, rd_true); + // } rd_kafka_queue_destroy(mainq); From a542f4dcc8a635def2635164a718ba495672f78b Mon Sep 17 00:00:00 2001 From: jainruchir Date: Wed, 15 Mar 2023 10:25:40 +0530 Subject: [PATCH 03/13] integration tests for describecluster --- examples/describe_consumer_groups.c | 7 +- src/rdkafka_admin.c | 1 - tests/0081-admin.c | 295 ++++++++++++++++++---------- 3 files changed, 192 insertions(+), 111 deletions(-) diff --git a/examples/describe_consumer_groups.c b/examples/describe_consumer_groups.c index e93b059cf8..3506c55b13 100644 --- a/examples/describe_consumer_groups.c +++ b/examples/describe_consumer_groups.c @@ -49,6 +49,7 @@ const char *argv0; + static rd_kafka_queue_t *queue; /** Admin result queue. * This is a global so we can * yield in stop() */ @@ -347,10 +348,7 @@ int main(int argc, char **argv) { * Create Kafka client configuration place-holder */ conf = rd_kafka_conf_new(); - conf_set(conf, "sasl.username", "broker"); - conf_set(conf, "sasl.password", "broker"); - conf_set(conf, "sasl.mechanism", "SCRAM-SHA-256"); - conf_set(conf, "security.protocol", "SASL_PLAINTEXT"); + /* * Parse common options @@ -384,5 +382,6 @@ int main(int argc, char **argv) { } cmd_describe_consumer_groups(conf, argc - optind, &argv[optind]); + return 0; } diff --git a/src/rdkafka_admin.c b/src/rdkafka_admin.c index 0c349c8391..3a1e6b2f09 100644 --- a/src/rdkafka_admin.c +++ b/src/rdkafka_admin.c @@ -7254,7 +7254,6 @@ rd_kafka_DescribeCluster_result_description( cluster_result_cnt = rd_list_cnt( &rko->rko_u.admin_result.results); - printf("%d is the value\n", cluster_result_cnt); rd_assert(cluster_result_cnt == 1); clusterdesc = rd_list_elem( &rko->rko_u.admin_result.results, 0); diff --git a/tests/0081-admin.c b/tests/0081-admin.c index 5f5680e4a4..afc0430bc4 100644 --- a/tests/0081-admin.c +++ b/tests/0081-admin.c @@ -3007,8 +3007,18 @@ static void do_test_DescribeTopics(const char *what, rd_kafka_t *rk, rd_kafka_qu q, RD_KAFKA_EVENT_CREATEACLS_RESULT, 10000 + 1000); acl_res = rd_kafka_event_CreateAcls_result(rkev_acl_create); acl_res_acls = rd_kafka_CreateAcls_result_acls(acl_res, &resacl_cnt); + for (i = 0; i < resacl_cnt; i++) { + const rd_kafka_acl_result_t *acl_res_acl = *(acl_res_acls + i); + const rd_kafka_error_t *error_acl = + rd_kafka_acl_result_error(acl_res_acl); + + TEST_ASSERT(!error_acl, + "Expected RD_KAFKA_RESP_ERR_NO_ERROR, not %s", + rd_kafka_error_string(error)); + } rd_kafka_event_destroy(rkev_acl_create); rd_kafka_AdminOptions_destroy(admin_options); + rd_kafka_AclBinding_destroy(acl_bindings[0]); /* * Timeout options */ @@ -3113,9 +3123,19 @@ static void do_test_DescribeTopics(const char *what, rd_kafka_t *rk, rd_kafka_qu q, RD_KAFKA_EVENT_CREATEACLS_RESULT, 10000 + 1000); acl_res = rd_kafka_event_CreateAcls_result(rkev_acl_create); acl_res_acls = rd_kafka_CreateAcls_result_acls(acl_res, &resacl_cnt); + for (i = 0; i < resacl_cnt; i++) { + const rd_kafka_acl_result_t *acl_res_acl = *(acl_res_acls + i); + const rd_kafka_error_t *error_acl = + rd_kafka_acl_result_error(acl_res_acl); + + TEST_ASSERT(!error_acl, + "Expected RD_KAFKA_RESP_ERR_NO_ERROR, not %s", + rd_kafka_error_string(error)); + } rd_kafka_event_destroy(rkev_acl_create); rd_kafka_AdminOptions_destroy(admin_options); rd_kafka_event_destroy(rkev); + rd_kafka_AclBinding_destroy(acl_bindings[0]); test_DeleteTopics_simple(rk, q, topics, 1, NULL); TEST_LATER_CHECK(); @@ -3146,6 +3166,14 @@ static void do_test_DescribeCluster(const char *what, rd_kafka_t *rk, rd_kafka_q const rd_kafka_acl_result_t **acl_res_acls; rd_kafka_event_t *rkev_acl_create; size_t resacl_cnt; + rd_kafka_AdminOptions_t *admin_options_delete; + rd_kafka_AclBindingFilter_t *acl_bindings_delete; + rd_kafka_event_t *rkev_acl_delete; + const rd_kafka_DeleteAcls_result_t *acl_delete_result; + const rd_kafka_DeleteAcls_result_response_t * + *DeleteAcls_result_responses; + const rd_kafka_DeleteAcls_result_response_t *DeleteAcls_result_response; + size_t DeleteAcls_result_responses_cntp; int i; int initial_acl_cnt, final_acl_cnt; @@ -3224,7 +3252,7 @@ static void do_test_DescribeCluster(const char *what, rd_kafka_t *rk, rd_kafka_q acl_bindings[0] = rd_kafka_AclBinding_new( RD_KAFKA_RESOURCE_BROKER, "kafka-cluster", RD_KAFKA_RESOURCE_PATTERN_LITERAL, - user_test1, "*", RD_KAFKA_ACL_OPERATION_DESCRIBE, + user_test1, "*", RD_KAFKA_ACL_OPERATION_ALTER, RD_KAFKA_ACL_PERMISSION_TYPE_ALLOW, NULL, 0); admin_options = rd_kafka_AdminOptions_new(rk, RD_KAFKA_ADMIN_OP_CREATEACLS); @@ -3236,8 +3264,18 @@ static void do_test_DescribeCluster(const char *what, rd_kafka_t *rk, rd_kafka_q q, RD_KAFKA_EVENT_CREATEACLS_RESULT, 10000 + 1000); acl_res = rd_kafka_event_CreateAcls_result(rkev_acl_create); acl_res_acls = rd_kafka_CreateAcls_result_acls(acl_res, &resacl_cnt); + for (i = 0; i < resacl_cnt; i++) { + const rd_kafka_acl_result_t *acl_res_acl = *(acl_res_acls + i); + const rd_kafka_error_t *error_acl = + rd_kafka_acl_result_error(acl_res_acl); + + TEST_ASSERT(!error_acl, + "Expected RD_KAFKA_RESP_ERR_NO_ERROR, not %s", + rd_kafka_error_string(error)); + } rd_kafka_event_destroy(rkev_acl_create); rd_kafka_AdminOptions_destroy(admin_options); + rd_kafka_AclBinding_destroy(acl_bindings[0]); /* * Timeout options */ @@ -3303,18 +3341,63 @@ static void do_test_DescribeCluster(const char *what, rd_kafka_t *rk, rd_kafka_q /* * Initally count should be 7. After createAcls call with - * only RD_KAFKA_ACL_OPERATION_DESCRIBE allowed, it should reduce - * to 1 (describe). 7>1 should hold. + * only RD_KAFKA_ACL_OPERATION_ALTER allowed, it should reduce + * to 2 (DESCRIBE is implicitly derived, and ALTER ). 7>2 should hold. */ TEST_SAY("final count is: %d\n", final_acl_cnt); - // TEST_ASSERT(initial_acl_cnt > final_acl_cnt, - // "Expected the acl operations allowed to have reduced after" - // " call to CreateAcls"); + TEST_ASSERT(initial_acl_cnt > final_acl_cnt, + "Expected the acl operations allowed to have reduced after" + " call to CreateAcls"); rd_kafka_event_destroy(rkev); - + + /* + * Remove the previously created Acl so that it doesn't affect other tests + */ + admin_options_delete = + rd_kafka_AdminOptions_new(rk, RD_KAFKA_ADMIN_OP_DELETEACLS); + rd_kafka_AdminOptions_set_request_timeout(admin_options_delete, 10000, + errstr, sizeof(errstr)); + + acl_bindings_delete = rd_kafka_AclBindingFilter_new( + RD_KAFKA_RESOURCE_BROKER, "kafka-cluster", RD_KAFKA_RESOURCE_PATTERN_MATCH, + user_test1 ,"*", RD_KAFKA_ACL_OPERATION_ALTER, RD_KAFKA_ACL_PERMISSION_TYPE_ALLOW, + NULL, 0); + + TIMING_START(&timing, "DeleteAcls"); + rd_kafka_DeleteAcls(rk, &acl_bindings_delete, 1, admin_options_delete, + q); + TIMING_ASSERT_LATER(&timing, 0, 50); + + /* + * Wait for result + */ + rkev_acl_delete = test_wait_admin_result( + q, RD_KAFKA_EVENT_DELETEACLS_RESULT, 10000 + 1000); + + acl_delete_result = rd_kafka_event_DeleteAcls_result(rkev_acl_delete); + + TEST_ASSERT(acl_delete_result, "acl_delete_result should not be NULL"); + + DeleteAcls_result_responses_cntp = 0; + DeleteAcls_result_responses = rd_kafka_DeleteAcls_result_responses( + acl_delete_result, &DeleteAcls_result_responses_cntp); + + TEST_ASSERT(DeleteAcls_result_responses_cntp == 1, + "DeleteAcls_result_responses_cntp should be 1, not %zu\n", + DeleteAcls_result_responses_cntp); + + DeleteAcls_result_response = DeleteAcls_result_responses[0]; + + TEST_CALL_ERROR__(rd_kafka_DeleteAcls_result_response_error( + DeleteAcls_result_response)); + rd_kafka_event_destroy(rkev_acl_delete); + rd_kafka_AclBinding_destroy(acl_bindings_delete); + rd_kafka_AdminOptions_destroy(admin_options_delete); TEST_LATER_CHECK(); + if (!rkqu) + rd_kafka_queue_destroy(q); SUB_TEST_PASS(); } @@ -4133,113 +4216,113 @@ static void do_test_apis(rd_kafka_type_t cltype) { mainq = rd_kafka_queue_get_main(rk); - // /* Create topics */ - // do_test_CreateTopics("temp queue, op timeout 0", rk, NULL, 0, 0); - // do_test_CreateTopics("temp queue, op timeout 15000", rk, NULL, 15000, - // 0); - // do_test_CreateTopics( - // "temp queue, op timeout 300, " - // "validate only", - // rk, NULL, 300, rd_true); - // do_test_CreateTopics("temp queue, op timeout 9000, validate_only", rk, - // NULL, 9000, rd_true); - // do_test_CreateTopics("main queue, options", rk, mainq, -1, 0); - - // /* Delete topics */ - // do_test_DeleteTopics("temp queue, op timeout 0", rk, NULL, 0); - // do_test_DeleteTopics("main queue, op timeout 15000", rk, mainq, 1500); - - // if (test_broker_version >= TEST_BRKVER(1, 0, 0, 0)) { - // /* Create Partitions */ - // do_test_CreatePartitions("temp queue, op timeout 6500", rk, - // NULL, 6500); - // do_test_CreatePartitions("main queue, op timeout 0", rk, mainq, - // 0); - // } - - // /* CreateAcls */ - // do_test_CreateAcls(rk, mainq, 0); - // do_test_CreateAcls(rk, mainq, 1); - - // /* DescribeAcls */ - // do_test_DescribeAcls(rk, mainq, 0); - // do_test_DescribeAcls(rk, mainq, 1); - - // /* DeleteAcls */ - // do_test_DeleteAcls(rk, mainq, 0); - // do_test_DeleteAcls(rk, mainq, 1); - - // /* AlterConfigs */ - // do_test_AlterConfigs(rk, mainq); - - // /* DescribeConfigs */ - // do_test_DescribeConfigs(rk, mainq); - - // /* Delete records */ - // do_test_DeleteRecords("temp queue, op timeout 0", rk, NULL, 0); - // do_test_DeleteRecords("main queue, op timeout 1500", rk, mainq, 1500); - - // /* List groups */ - // do_test_ListConsumerGroups("temp queue", rk, NULL, -1, rd_false); - // do_test_ListConsumerGroups("main queue", rk, mainq, 1500, rd_true); + /* Create topics */ + do_test_CreateTopics("temp queue, op timeout 0", rk, NULL, 0, 0); + do_test_CreateTopics("temp queue, op timeout 15000", rk, NULL, 15000, + 0); + do_test_CreateTopics( + "temp queue, op timeout 300, " + "validate only", + rk, NULL, 300, rd_true); + do_test_CreateTopics("temp queue, op timeout 9000, validate_only", rk, + NULL, 9000, rd_true); + do_test_CreateTopics("main queue, options", rk, mainq, -1, 0); + + /* Delete topics */ + do_test_DeleteTopics("temp queue, op timeout 0", rk, NULL, 0); + do_test_DeleteTopics("main queue, op timeout 15000", rk, mainq, 1500); + + if (test_broker_version >= TEST_BRKVER(1, 0, 0, 0)) { + /* Create Partitions */ + do_test_CreatePartitions("temp queue, op timeout 6500", rk, + NULL, 6500); + do_test_CreatePartitions("main queue, op timeout 0", rk, mainq, + 0); + } + + /* CreateAcls */ + do_test_CreateAcls(rk, mainq, 0); + do_test_CreateAcls(rk, mainq, 1); + + /* DescribeAcls */ + do_test_DescribeAcls(rk, mainq, 0); + do_test_DescribeAcls(rk, mainq, 1); + + /* DeleteAcls */ + do_test_DeleteAcls(rk, mainq, 0); + do_test_DeleteAcls(rk, mainq, 1); + + /* AlterConfigs */ + do_test_AlterConfigs(rk, mainq); + + /* DescribeConfigs */ + do_test_DescribeConfigs(rk, mainq); + + /* Delete records */ + do_test_DeleteRecords("temp queue, op timeout 0", rk, NULL, 0); + do_test_DeleteRecords("main queue, op timeout 1500", rk, mainq, 1500); + + /* List groups */ + do_test_ListConsumerGroups("temp queue", rk, NULL, -1, rd_false); + do_test_ListConsumerGroups("main queue", rk, mainq, 1500, rd_true); /* Describe groups */ - // do_test_DescribeConsumerGroups("temp queue", rk, NULL, -1); - // do_test_DescribeConsumerGroups("main queue", rk, mainq, 1500); + do_test_DescribeConsumerGroups("temp queue", rk, NULL, -1); + do_test_DescribeConsumerGroups("main queue", rk, mainq, 1500); - // /* Describe topics */ - // do_test_DescribeTopics("temp queue", rk, NULL, 15000); - // do_test_DescribeTopics("main queue", rk, mainq, 15000); + /* Describe topics */ + do_test_DescribeTopics("temp queue", rk, NULL, 15000); + do_test_DescribeTopics("main queue", rk, mainq, 15000); /* Describe cluster */ do_test_DescribeCluster("temp queue", rk, NULL, 1500); do_test_DescribeCluster("main queue", rk, mainq, 1500); - // /* Delete groups */ - // do_test_DeleteGroups("temp queue", rk, NULL, -1); - // do_test_DeleteGroups("main queue", rk, mainq, 1500); - - // if (test_broker_version >= TEST_BRKVER(2, 4, 0, 0)) { - // /* Delete committed offsets */ - // do_test_DeleteConsumerGroupOffsets("temp queue", rk, NULL, -1, - // rd_false); - // do_test_DeleteConsumerGroupOffsets("main queue", rk, mainq, - // 1500, rd_false); - // do_test_DeleteConsumerGroupOffsets( - // "main queue", rk, mainq, 1500, - // rd_true /*with subscribing consumer*/); - - // /* Alter committed offsets */ - // do_test_AlterConsumerGroupOffsets("temp queue", rk, NULL, -1, - // rd_false, rd_true); - // do_test_AlterConsumerGroupOffsets("main queue", rk, mainq, 1500, - // rd_false, rd_true); - // do_test_AlterConsumerGroupOffsets( - // "main queue, nonexistent topics", rk, mainq, 1500, rd_false, - // rd_false /* don't create topics */); - // do_test_AlterConsumerGroupOffsets( - // "main queue", rk, mainq, 1500, - // rd_true, /*with subscribing consumer*/ - // rd_true); - - // /* List committed offsets */ - // do_test_ListConsumerGroupOffsets("temp queue", rk, NULL, -1, - // rd_false, rd_false); - // do_test_ListConsumerGroupOffsets( - // "main queue, op timeout " - // "1500", - // rk, mainq, 1500, rd_false, rd_false); - // do_test_ListConsumerGroupOffsets( - // "main queue", rk, mainq, 1500, - // rd_true /*with subscribing consumer*/, rd_false); - // do_test_ListConsumerGroupOffsets("temp queue", rk, NULL, -1, - // rd_false, rd_true); - // do_test_ListConsumerGroupOffsets("main queue", rk, mainq, 1500, - // rd_false, rd_true); - // do_test_ListConsumerGroupOffsets( - // "main queue", rk, mainq, 1500, - // rd_true /*with subscribing consumer*/, rd_true); - // } + /* Delete groups */ + do_test_DeleteGroups("temp queue", rk, NULL, -1); + do_test_DeleteGroups("main queue", rk, mainq, 1500); + + if (test_broker_version >= TEST_BRKVER(2, 4, 0, 0)) { + /* Delete committed offsets */ + do_test_DeleteConsumerGroupOffsets("temp queue", rk, NULL, -1, + rd_false); + do_test_DeleteConsumerGroupOffsets("main queue", rk, mainq, + 1500, rd_false); + do_test_DeleteConsumerGroupOffsets( + "main queue", rk, mainq, 1500, + rd_true /*with subscribing consumer*/); + + /* Alter committed offsets */ + do_test_AlterConsumerGroupOffsets("temp queue", rk, NULL, -1, + rd_false, rd_true); + do_test_AlterConsumerGroupOffsets("main queue", rk, mainq, 1500, + rd_false, rd_true); + do_test_AlterConsumerGroupOffsets( + "main queue, nonexistent topics", rk, mainq, 1500, rd_false, + rd_false /* don't create topics */); + do_test_AlterConsumerGroupOffsets( + "main queue", rk, mainq, 1500, + rd_true, /*with subscribing consumer*/ + rd_true); + + /* List committed offsets */ + do_test_ListConsumerGroupOffsets("temp queue", rk, NULL, -1, + rd_false, rd_false); + do_test_ListConsumerGroupOffsets( + "main queue, op timeout " + "1500", + rk, mainq, 1500, rd_false, rd_false); + do_test_ListConsumerGroupOffsets( + "main queue", rk, mainq, 1500, + rd_true /*with subscribing consumer*/, rd_false); + do_test_ListConsumerGroupOffsets("temp queue", rk, NULL, -1, + rd_false, rd_true); + do_test_ListConsumerGroupOffsets("main queue", rk, mainq, 1500, + rd_false, rd_true); + do_test_ListConsumerGroupOffsets( + "main queue", rk, mainq, 1500, + rd_true /*with subscribing consumer*/, rd_true); + } rd_kafka_queue_destroy(mainq); From d0e21562b734633ee620fe7914ebcb6db989ad94 Mon Sep 17 00:00:00 2001 From: jainruchir Date: Fri, 17 Mar 2023 10:01:08 +0530 Subject: [PATCH 04/13] bug fix when include authorized ops is 0 desc Topics/Cluster --- src/rdkafka_admin.c | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/rdkafka_admin.c b/src/rdkafka_admin.c index 3a1e6b2f09..25f55ffc16 100644 --- a/src/rdkafka_admin.c +++ b/src/rdkafka_admin.c @@ -7079,7 +7079,8 @@ rd_kafka_DescribeTopicsResponse_parse(rd_kafka_op_t *rko_req, md->topics[i].partition_cnt, authorized_operations, md->topics[i].err); - rd_list_destroy(authorized_operations); + if(authorized_operations) + rd_list_destroy(authorized_operations); } else topicdesc = rd_kafka_TopicDescription_new_error(md->topics[i].topic, @@ -7437,6 +7438,8 @@ rd_kafka_DescribeClusterResponse_parse(rd_kafka_op_t *rko_req, clusterdesc = rd_kafka_ClusterDescription_new(cluster_id, controller_id, authorized_operations, md); + if(authorized_operations) + rd_list_destroy(authorized_operations); rd_free(cluster_id); rd_list_add(&rko_result->rko_u.admin_result.results, clusterdesc); From 3a79ed615d389e3950cb970e68bb5bccbc1e52a3 Mon Sep 17 00:00:00 2001 From: jainruchir Date: Fri, 17 Feb 2023 14:25:37 +0530 Subject: [PATCH 05/13] KIP-430 initial changes without checking tests --- examples/describe_consumer_groups.c | 47 +++++++++++--- src/rdkafka.c | 2 +- src/rdkafka.h | 40 ++++++++++++ src/rdkafka_admin.c | 95 +++++++++++++++++++++++++---- src/rdkafka_admin.h | 8 +++ src/rdkafka_request.c | 18 +++--- src/rdkafka_request.h | 16 ++--- 7 files changed, 189 insertions(+), 37 deletions(-) diff --git a/examples/describe_consumer_groups.c b/examples/describe_consumer_groups.c index 3506c55b13..5eb9b776cf 100644 --- a/examples/describe_consumer_groups.c +++ b/examples/describe_consumer_groups.c @@ -73,7 +73,8 @@ static void usage(const char *reason, ...) { fprintf(stderr, "Describe groups usage examples\n" "\n" - "Usage: %s ...\n" + "Usage: %s " + " ...\n" "\n" "Options:\n" " -b Bootstrap server list to connect to.\n" @@ -167,7 +168,8 @@ print_groups_info(const rd_kafka_DescribeConsumerGroups_result_t *grpdesc, } for (i = 0; i < result_groups_cnt; i++) { - int j, member_cnt; + int j, member_cnt, authorized_operation_count, + acl_operation; const rd_kafka_error_t *error; const rd_kafka_ConsumerGroupDescription_t *group = result_groups[i]; @@ -179,6 +181,8 @@ print_groups_info(const rd_kafka_DescribeConsumerGroups_result_t *grpdesc, rd_kafka_ConsumerGroupDescription_partition_assignor(group); rd_kafka_consumer_group_state_t state = rd_kafka_ConsumerGroupDescription_state(group); + authorized_operation_count = + rd_kafka_ConsumerGroupDescription_authorized_operations_count(group); member_cnt = rd_kafka_ConsumerGroupDescription_member_count(group); error = rd_kafka_ConsumerGroupDescription_error(group); @@ -197,10 +201,19 @@ print_groups_info(const rd_kafka_DescribeConsumerGroups_result_t *grpdesc, } printf( "Group \"%s\", partition assignor \"%s\", " - "state %s%s, with %" PRId32 " member(s)", + " state %s%s, with %" PRId32 " member(s)\n", group_id, partition_assignor, rd_kafka_consumer_group_state_name(state), coordinator_desc, member_cnt); + for(j=0;j 1) + usage("Require stable not a 0-1 int"); if (argc >= 1) { - groups = (const char **)&argv[0]; - groups_cnt = argc; + groups = (const char **)&argv[1]; + groups_cnt = argc - 1; } /* @@ -296,6 +316,14 @@ cmd_describe_consumer_groups(rd_kafka_conf_t *conf, int argc, char **argv) { fprintf(stderr, "%% Failed to set timeout: %s\n", errstr); goto exit; } + if ((error = rd_kafka_AdminOptions_set_include_authorized_operations( + options, include_authorized_operations))) { + fprintf(stderr, + "%% Failed to set require authorized operations: %s\n", + rd_kafka_error_string(error)); + rd_kafka_error_destroy(error); + exit(1); + } rd_kafka_DescribeConsumerGroups(rk, groups, groups_cnt, options, queue); @@ -348,7 +376,10 @@ int main(int argc, char **argv) { * Create Kafka client configuration place-holder */ conf = rd_kafka_conf_new(); - + conf_set(conf, "sasl.username", "broker"); + conf_set(conf, "sasl.password", "broker"); + conf_set(conf, "sasl.mechanism", "SCRAM-SHA-256"); + conf_set(conf, "security.protocol", "SASL_PLAINTEXT"); /* * Parse common options diff --git a/src/rdkafka.c b/src/rdkafka.c index b254748eb6..d0125cd1d2 100644 --- a/src/rdkafka.c +++ b/src/rdkafka.c @@ -4769,7 +4769,7 @@ static void rd_kafka_ListGroups_resp_cb(rd_kafka_t *rk, state->wait_cnt++; error = rd_kafka_DescribeGroupsRequest( - rkb, 0, grps, i, RD_KAFKA_REPLYQ(state->q, 0), + rkb, 0, grps, i, rd_false, RD_KAFKA_REPLYQ(state->q, 0), rd_kafka_DescribeGroups_resp_cb, state); if (error) { rd_kafka_DescribeGroups_resp_cb( diff --git a/src/rdkafka.h b/src/rdkafka.h index 2e0a8269e6..4d03d67f23 100644 --- a/src/rdkafka.h +++ b/src/rdkafka.h @@ -6935,6 +6935,23 @@ rd_kafka_error_t *rd_kafka_AdminOptions_set_require_stable_offsets( rd_kafka_AdminOptions_t *options, int true_or_false); +/** + * @brief Whether broker should return authorized operations + * (DescribeConsumerGroups). + * + * @param options Admin options. + * @param true_or_false Defaults to false. + * + * @return NULL on success, a new error instance that must be + * released with rd_kafka_error_destroy() in case of error. + * + * @remark This option is valid for DescribeConsumerGroups. + */ +RD_EXPORT +rd_kafka_error_t *rd_kafka_AdminOptions_set_include_authorized_operations( + rd_kafka_AdminOptions_t *options, + int true_or_false); + /** * @brief Set consumer groups states to query for. * @@ -8383,6 +8400,29 @@ RD_EXPORT const char *rd_kafka_ConsumerGroupDescription_partition_assignor( const rd_kafka_ConsumerGroupDescription_t *grpdesc); +/** + * @brief Gets count of authorized operations for the \p grpdesc group. + * + * @param grpdesc The group description. + * + * @return count of Authorized operations allowed, 0 if authorized operations list is NULL or empty. + */ +RD_EXPORT +size_t rd_kafka_ConsumerGroupDescription_authorized_operations_count( + const rd_kafka_ConsumerGroupDescription_t *grpdesc); + +/** + * @brief Gets operation at idx index of authorized operations for the \p grpdesc group. + * + * @param grpdesc The group description. + * @param idx The index for which element is needed. + * + * @return Authorized operation at given index. + */ +RD_EXPORT +int rd_kafka_ConsumerGroupDescription_authorized_operation( + const rd_kafka_ConsumerGroupDescription_t *grpdesc, + size_t idx); /** * @brief Gets state for the \p grpdesc group. diff --git a/src/rdkafka_admin.c b/src/rdkafka_admin.c index 25f55ffc16..119a493367 100644 --- a/src/rdkafka_admin.c +++ b/src/rdkafka_admin.c @@ -230,11 +230,12 @@ static const char *rd_kafka_admin_state_desc[] = { * @enum Admin request target broker. Must be negative values since the field * used is broker_id. */ -enum { RD_KAFKA_ADMIN_TARGET_CONTROLLER = -1, /**< Cluster controller */ - RD_KAFKA_ADMIN_TARGET_COORDINATOR = -2, /**< (Group) Coordinator */ - RD_KAFKA_ADMIN_TARGET_FANOUT = -3, /**< This rko is a fanout and - * and has no target broker */ - RD_KAFKA_ADMIN_TARGET_ALL = -4, /**< All available brokers */ +enum { + RD_KAFKA_ADMIN_TARGET_CONTROLLER = -1, /**< Cluster controller */ + RD_KAFKA_ADMIN_TARGET_COORDINATOR = -2, /**< (Group) Coordinator */ + RD_KAFKA_ADMIN_TARGET_FANOUT = -3, /**< This rko is a fanout and + * and has no target broker */ + RD_KAFKA_ADMIN_TARGET_ALL = -4, /**< All available brokers */ }; /** @@ -1556,6 +1557,15 @@ rd_kafka_error_t *rd_kafka_AdminOptions_set_require_stable_offsets( &true_or_false, errstr, sizeof(errstr)); return !err ? NULL : rd_kafka_error_new(err, "%s", errstr); } +rd_kafka_error_t *rd_kafka_AdminOptions_set_include_authorized_operations( + rd_kafka_AdminOptions_t *options, + int true_or_false) { + char errstr[512]; + rd_kafka_resp_err_t err = rd_kafka_confval_set_type( + &options->include_authorized_operations, RD_KAFKA_CONFVAL_INT, + &true_or_false, errstr, sizeof(errstr)); + return !err ? NULL : rd_kafka_error_new(err, "%s", errstr); +} rd_kafka_error_t *rd_kafka_AdminOptions_set_include_topic_authorized_operations( rd_kafka_AdminOptions_t *options, @@ -1692,6 +1702,15 @@ static void rd_kafka_AdminOptions_init(rd_kafka_t *rk, rd_kafka_confval_disable(&options->include_cluster_authorized_operations, "include_cluster_authorized_operations"); + if (options->for_api == RD_KAFKA_ADMIN_OP_ANY || + options->for_api == RD_KAFKA_ADMIN_OP_DESCRIBECONSUMERGROUPS) + rd_kafka_confval_init_int( + &options->include_authorized_operations, + "include_authorized_operations", 0, 1, 0); + else + rd_kafka_confval_disable(&options->require_stable_offsets, + "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, @@ -6160,6 +6179,7 @@ const rd_kafka_topic_partition_list_t *rd_kafka_MemberAssignment_partitions( * @param members List of members (rd_kafka_MemberDescription_t) of this * group. * @param partition_assignor (optional) Chosen assignor. + * @param authorized_operations authorized operations. * @param state Group state. * @param coordinator (optional) Group coordinator. * @param error (optional) Error received for this group. @@ -6171,6 +6191,7 @@ rd_kafka_ConsumerGroupDescription_new(const char *group_id, rd_bool_t is_simple_consumer_group, const rd_list_t *members, const char *partition_assignor, + rd_list_t* authorized_operations, rd_kafka_consumer_group_state_t state, const rd_kafka_Node_t *coordinator, rd_kafka_error_t *error) { @@ -6186,10 +6207,11 @@ rd_kafka_ConsumerGroupDescription_new(const char *group_id, rd_list_copy_to(&grpdesc->members, members, rd_kafka_MemberDescription_list_copy, NULL); } - grpdesc->partition_assignor = !partition_assignor - ? (char *)partition_assignor - : rd_strdup(partition_assignor); - grpdesc->state = state; + grpdesc->partition_assignor = !partition_assignor + ? (char *)partition_assignor + : rd_strdup(partition_assignor); + grpdesc->authorized_operations = authorized_operations; + grpdesc->state = state; if (coordinator != NULL) grpdesc->coordinator = rd_kafka_Node_copy(coordinator); grpdesc->error = @@ -6210,7 +6232,7 @@ static rd_kafka_ConsumerGroupDescription_t * rd_kafka_ConsumerGroupDescription_new_error(const char *group_id, rd_kafka_error_t *error) { return rd_kafka_ConsumerGroupDescription_new( - group_id, rd_false, NULL, NULL, + group_id, rd_false, NULL, NULL, NULL, RD_KAFKA_CONSUMER_GROUP_STATE_UNKNOWN, NULL, error); } @@ -6225,7 +6247,8 @@ rd_kafka_ConsumerGroupDescription_copy( const rd_kafka_ConsumerGroupDescription_t *grpdesc) { return rd_kafka_ConsumerGroupDescription_new( grpdesc->group_id, grpdesc->is_simple_consumer_group, - &grpdesc->members, grpdesc->partition_assignor, grpdesc->state, + &grpdesc->members, grpdesc->partition_assignor, + grpdesc->authorized_operations, grpdesc->state, grpdesc->coordinator, grpdesc->error); } @@ -6278,6 +6301,19 @@ const char *rd_kafka_ConsumerGroupDescription_partition_assignor( return grpdesc->partition_assignor; } +size_t rd_kafka_ConsumerGroupDescription_authorized_operations_count( + const rd_kafka_ConsumerGroupDescription_t *grpdesc) { + if(grpdesc->authorized_operations) + return grpdesc->authorized_operations->rl_cnt; + return 0; +} + +int rd_kafka_ConsumerGroupDescription_authorized_operation( + const rd_kafka_ConsumerGroupDescription_t *grpdesc, + size_t idx) { + rd_kafka_AclOperation_t* entry = rd_list_elem(grpdesc->authorized_operations, idx); + return *entry; +} rd_kafka_consumer_group_state_t rd_kafka_ConsumerGroupDescription_state( const rd_kafka_ConsumerGroupDescription_t *grpdesc) { @@ -6375,7 +6411,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); @@ -6385,7 +6421,10 @@ static rd_kafka_resp_err_t rd_kafka_admin_DescribeConsumerGroupsRequest( RD_LIST_FOREACH(group, groups, i) { groups_arr[i] = rd_list_elem(groups, i); } + include_authorized_operations = + rd_kafka_confval_get_int(&options->include_authorized_operations); error = rd_kafka_DescribeGroupsRequest(rkb, -1, groups_arr, groups_cnt, + include_authorized_operations, replyq, resp_cb, opaque); rd_free(groups_arr); @@ -6400,6 +6439,28 @@ static rd_kafka_resp_err_t rd_kafka_admin_DescribeConsumerGroupsRequest( return RD_KAFKA_RESP_ERR_NO_ERROR; } +/** + * @brief Parse authorized_operations returned in DescribeConsumerGroups + * @returns array of rd_bool_t of size RD_KAFKA_ACL_OPERATIONS__CNT + */ +rd_list_t* rd_kafka_AuthorizedOperations_parse(int32_t authorized_operations){ + int i, bit; + rd_list_t* authorized_operations_list = NULL; + rd_kafka_AclOperation_t* entry; + /* in case of authorized_operations not requested, return NULL*/ + if(authorized_operations<0) + return NULL; + authorized_operations_list = rd_list_new(0, NULL); + for(i=0; i> i) & 1; + if(bit){ + entry = malloc(sizeof(rd_kafka_AclOperation_t)); + *entry = i; + rd_list_add(authorized_operations_list, entry); + } + } + return authorized_operations_list; +} /** * @brief Parse DescribeConsumerGroupsResponse and create ADMIN_RESULT op. */ @@ -6441,6 +6502,8 @@ rd_kafka_DescribeConsumerGroupsResponse_parse(rd_kafka_op_t *rko_req, node = rd_kafka_Node_new(nodeid, host, port, NULL); while (cnt-- > 0) { int16_t error_code; + int32_t authorized_operations = 0; + rd_list_t* authorized_operations_list; rd_kafkap_str_t GroupId, GroupState, ProtocolType, ProtocolData; rd_bool_t is_simple_consumer_group, is_consumer_protocol_type; int32_t member_cnt; @@ -6551,13 +6614,19 @@ rd_kafka_DescribeConsumerGroupsResponse_parse(rd_kafka_op_t *rko_req, if (api_version >= 3) { /* TODO: implement KIP-430 */ - int32_t authorized_operations; rd_kafka_buf_read_i32(reply, &authorized_operations); + /* assert that the last 3 bits are never set*/ + rd_assert(!((authorized_operations >> 0) & 1)); + rd_assert(!((authorized_operations >> 1) & 1)); + rd_assert(!((authorized_operations >> 2) & 1)); + /* authorized_operations is -2147483648 in case of not requested, list has no elements in that case*/ + authorized_operations_list = rd_kafka_AuthorizedOperations_parse(authorized_operations); } if (error == NULL) { grpdesc = rd_kafka_ConsumerGroupDescription_new( group_id, is_simple_consumer_group, &members, proto, + authorized_operations_list, rd_kafka_consumer_group_state_code(group_state), node, error); } else { diff --git a/src/rdkafka_admin.h b/src/rdkafka_admin.h index 463379c569..8c6ba17748 100644 --- a/src/rdkafka_admin.h +++ b/src/rdkafka_admin.h @@ -91,6 +91,12 @@ struct rd_kafka_AdminOptions_s { * Valid for: * ListConsumerGroupOffsets */ + rd_kafka_confval_t + include_authorized_operations; /**< BOOL: Whether broker should + * return authorized operations. + * Valid for: + * DescribeConsumerGroups + */ rd_kafka_confval_t include_topic_authorized_operations; /**< BOOL: Whether broker should return @@ -488,6 +494,8 @@ struct rd_kafka_ConsumerGroupDescription_s { rd_kafka_consumer_group_state_t state; /** Consumer group coordinator. */ rd_kafka_Node_t *coordinator; + /** Authorized operations. */ + rd_list_t* authorized_operations; /** Group specific error. */ rd_kafka_error_t *error; }; diff --git a/src/rdkafka_request.c b/src/rdkafka_request.c index 56a5d6fb48..cffa201448 100644 --- a/src/rdkafka_request.c +++ b/src/rdkafka_request.c @@ -2021,13 +2021,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; @@ -2065,7 +2067,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); diff --git a/src/rdkafka_request.h b/src/rdkafka_request.h index d28c9aad23..7db5a7bc58 100644 --- a/src/rdkafka_request.h +++ b/src/rdkafka_request.h @@ -235,13 +235,15 @@ rd_kafka_error_t *rd_kafka_ListGroupsRequest(rd_kafka_broker_t *rkb, rd_kafka_resp_cb_t *resp_cb, void *opaque); -rd_kafka_error_t *rd_kafka_DescribeGroupsRequest(rd_kafka_broker_t *rkb, - int16_t max_ApiVersion, - char **groups, - size_t group_cnt, - rd_kafka_replyq_t replyq, - rd_kafka_resp_cb_t *resp_cb, - void *opaque); +rd_kafka_error_t * +rd_kafka_DescribeGroupsRequest(rd_kafka_broker_t *rkb, + int16_t max_ApiVersion, + char **groups, + size_t group_cnt, + rd_bool_t include_authorized_operations, + rd_kafka_replyq_t replyq, + rd_kafka_resp_cb_t *resp_cb, + void *opaque); void rd_kafka_HeartbeatRequest(rd_kafka_broker_t *rkb, From d2c902b0460a7fda78d55f0d0f425f341182b168 Mon Sep 17 00:00:00 2001 From: jainruchir Date: Tue, 21 Feb 2023 15:55:50 +0530 Subject: [PATCH 06/13] added test for include auth ops in 0081 initial commit --- examples/describe_consumer_groups.c | 7 +- tests/0081-admin.c | 265 ++++++++++++++++++++++++++++ 2 files changed, 267 insertions(+), 5 deletions(-) diff --git a/examples/describe_consumer_groups.c b/examples/describe_consumer_groups.c index 5eb9b776cf..edad573d30 100644 --- a/examples/describe_consumer_groups.c +++ b/examples/describe_consumer_groups.c @@ -208,11 +208,8 @@ print_groups_info(const rd_kafka_DescribeConsumerGroups_result_t *grpdesc, for(j=0;jgroup_id, rd_kafka_ConsumerGroupDescription_group_id(act)) == @@ -3401,6 +3404,260 @@ static void do_test_DescribeCluster(const char *what, rd_kafka_t *rk, rd_kafka_q SUB_TEST_PASS(); } +/** + * @brief Test describe groups, creating consumers for a set of groups, + * describing and deleting them at the end. + */ +static void do_test_DescribeConsumerGroups_with_authorized_ops(const char *what, + rd_kafka_t *rk, + rd_kafka_queue_t *useq, + int request_timeout, + rd_bool_t include_authorized_operations) { + rd_kafka_queue_t *q; + rd_kafka_AdminOptions_t *options = NULL; + rd_kafka_event_t *rkev = NULL; + rd_kafka_resp_err_t err; + rd_kafka_error_t *error; + rd_kafka_event_t *rkev_acl_create; + rd_kafka_AclOperation_t acl_operation; + rd_kafka_ResourcePatternType_t pattern_type_first_topic = + RD_KAFKA_RESOURCE_PATTERN_LITERAL; + char errstr[512]; + const char *errstr2; + const char *user_test1 = "User:broker"; +#define TEST_DESCRIBE_CONSUMER_GROUPS_CNT 4 + int known_groups = TEST_DESCRIBE_CONSUMER_GROUPS_CNT - 1; + int i,j; + const int partitions_cnt = 1; + const int msgs_cnt = 100; + char *topic; + size_t resacl_cnt; + rd_kafka_AclBinding_t *acl_bindings[TEST_DESCRIBE_CONSUMER_GROUPS_CNT]; + rd_kafka_metadata_topic_t exp_mdtopic = {0}; + int64_t testid = test_id_generate(); + test_timing_t timing; + rd_kafka_resp_err_t exp_err = RD_KAFKA_RESP_ERR_NO_ERROR; + const rd_kafka_ConsumerGroupDescription_t **results = NULL; + expected_DescribeConsumerGroups_result_t + expected[TEST_DESCRIBE_CONSUMER_GROUPS_CNT] = RD_ZERO_INIT; + const char *describe_groups[TEST_DESCRIBE_CONSUMER_GROUPS_CNT]; + char group_instance_ids[TEST_DESCRIBE_CONSUMER_GROUPS_CNT][512]; + char client_ids[TEST_DESCRIBE_CONSUMER_GROUPS_CNT][512]; + rd_kafka_t *rks[TEST_DESCRIBE_CONSUMER_GROUPS_CNT]; + const rd_kafka_DescribeConsumerGroups_result_t *res; + rd_bool_t has_group_instance_id = + test_broker_version >= TEST_BRKVER(2, 4, 0, 0); + rd_kafka_AdminOptions_t *admin_options; + const rd_kafka_CreateAcls_result_t *acl_res; + const rd_kafka_acl_result_t **acl_res_acls; + + SUB_TEST_QUICK("%s DescribeConsumerGroups with %s, request_timeout %d", + rd_kafka_name(rk), what, request_timeout); + + q = useq ? useq : rd_kafka_queue_new(rk); + + if (request_timeout != -1) { + options = rd_kafka_AdminOptions_new( + rk, RD_KAFKA_ADMIN_OP_DESCRIBECONSUMERGROUPS); + + err = rd_kafka_AdminOptions_set_request_timeout( + options, request_timeout, errstr, sizeof(errstr)); + TEST_ASSERT(!err, "%s", rd_kafka_err2str(err)); + if ((error = rd_kafka_AdminOptions_set_include_authorized_operations( + options, 1))) { + fprintf(stderr, + "%% Failed to set require authorized operations: %s\n", + rd_kafka_error_string(error)); + rd_kafka_error_destroy(error); + TEST_FAIL("Failed to set include authorized operations\n"); + } + } + + topic = rd_strdup(test_mk_topic_name(__FUNCTION__, 1)); + exp_mdtopic.topic = topic; + + /* Create the topics first. */ + test_CreateTopics_simple(rk, NULL, &topic, 1, partitions_cnt, NULL); + + /* Verify that topics are reported by metadata */ + test_wait_metadata_update(rk, &exp_mdtopic, 1, NULL, 0, 15 * 1000); + + /* Produce 100 msgs */ + test_produce_msgs_easy(topic, testid, 0, msgs_cnt); + + for (i = 0; i < TEST_DESCRIBE_CONSUMER_GROUPS_CNT; i++) { + rd_kafka_conf_t *conf; + char *group_id = rd_strdup(test_mk_topic_name(__FUNCTION__, 1)); + acl_bindings[i] = rd_kafka_AclBinding_new( + RD_KAFKA_RESOURCE_GROUP, group_id, RD_KAFKA_RESOURCE_PATTERN_LITERAL, + user_test1, "*", RD_KAFKA_ACL_OPERATION_READ, + RD_KAFKA_ACL_PERMISSION_TYPE_ALLOW, NULL, 0); + if (i < known_groups) { + snprintf(group_instance_ids[i], + sizeof(group_instance_ids[i]), + "group_instance_id_%" PRId32, i); + snprintf(client_ids[i], sizeof(client_ids[i]), + "client_id_%" PRId32, i); + + test_conf_init(&conf, NULL, 0); + test_conf_set(conf, "client.id", client_ids[i]); + test_conf_set(conf, "group.instance.id", + group_instance_ids[i]); + test_conf_set(conf, "session.timeout.ms", "5000"); + test_conf_set(conf, "auto.offset.reset", "earliest"); + rks[i] = + test_create_consumer(group_id, NULL, conf, NULL); + test_consumer_subscribe(rks[i], topic); + /* Consume messages */ + test_consumer_poll("consumer", rks[i], testid, -1, -1, + msgs_cnt, NULL); + } + expected[i].group_id = group_id; + expected[i].err = RD_KAFKA_RESP_ERR_NO_ERROR; + describe_groups[i] = group_id; + } + // acl_bindings[0] = rd_kafka_AclBinding_new( + // RD_KAFKA_RESOURCE_TOPIC, topic, pattern_type_first_topic, + // user_test1, "*", RD_KAFKA_ACL_OPERATION_WRITE, + // RD_KAFKA_ACL_PERMISSION_TYPE_ALLOW, NULL, 0); + // acl_bindings[1] = rd_kafka_AclBinding_new( + // RD_KAFKA_RESOURCE_TOPIC, topic, pattern_type_first_topic, + // user_test1, "*", RD_KAFKA_ACL_OPERATION_READ, + // RD_KAFKA_ACL_PERMISSION_TYPE_ALLOW, NULL, 0); + admin_options = + rd_kafka_AdminOptions_new(rk, RD_KAFKA_ADMIN_OP_CREATEACLS); + err = rd_kafka_AdminOptions_set_request_timeout(admin_options, 10000, + errstr, sizeof(errstr)); + TEST_ASSERT(!err, "%s", errstr); + rd_kafka_CreateAcls(rk, acl_bindings, TEST_DESCRIBE_CONSUMER_GROUPS_CNT, admin_options, q); + rkev_acl_create = test_wait_admin_result( + q, RD_KAFKA_EVENT_CREATEACLS_RESULT, 10000 + 1000); + acl_res = rd_kafka_event_CreateAcls_result(rkev_acl_create); + acl_res_acls = rd_kafka_CreateAcls_result_acls(acl_res, &resacl_cnt); + + TIMING_START(&timing, "DescribeConsumerGroups"); + TEST_SAY("Call DescribeConsumerGroups with authorized operations\n"); + rd_kafka_DescribeConsumerGroups( + rk, describe_groups, TEST_DESCRIBE_CONSUMER_GROUPS_CNT, options, q); + TIMING_ASSERT_LATER(&timing, 0, 50); + + TIMING_START(&timing, "DescribeConsumerGroups.queue_poll"); + + /* Poll result queue for DescribeConsumerGroups result. + * Print but otherwise ignore other event types + * (typically generic Error events). */ + while (1) { + rkev = rd_kafka_queue_poll(q, tmout_multip(20 * 1000)); + TEST_SAY("DescribeConsumerGroups: got %s in %.3fms\n", + rd_kafka_event_name(rkev), + TIMING_DURATION(&timing) / 1000.0f); + if (rkev == NULL) + continue; + if (rd_kafka_event_error(rkev)) + TEST_SAY("%s: %s\n", rd_kafka_event_name(rkev), + rd_kafka_event_error_string(rkev)); + + if (rd_kafka_event_type(rkev) == + RD_KAFKA_EVENT_DESCRIBECONSUMERGROUPS_RESULT) { + break; + } + + rd_kafka_event_destroy(rkev); + } + /* Convert event to proper result */ + res = rd_kafka_event_DescribeConsumerGroups_result(rkev); + TEST_ASSERT(res, "expected DescribeConsumerGroups_result, got %s", + rd_kafka_event_name(rkev)); + + /* Expecting error */ + err = rd_kafka_event_error(rkev); + errstr2 = rd_kafka_event_error_string(rkev); + TEST_ASSERT(err == exp_err, + "expected DescribeConsumerGroups to return %s, got %s (%s)", + rd_kafka_err2str(exp_err), rd_kafka_err2str(err), + err ? errstr2 : "n/a"); + + TEST_SAY("DescribeConsumerGroups: returned %s (%s)\n", + rd_kafka_err2str(err), err ? errstr2 : "n/a"); + + size_t cnt = 0; + results = rd_kafka_DescribeConsumerGroups_result_groups(res, &cnt); + + TEST_ASSERT( + TEST_DESCRIBE_CONSUMER_GROUPS_CNT == cnt, + "expected DescribeConsumerGroups_result_groups to return %d items, " + "got %" PRIusz, + TEST_DESCRIBE_CONSUMER_GROUPS_CNT, cnt); + for (i = 0; i < TEST_DESCRIBE_CONSUMER_GROUPS_CNT; i++) { + expected_DescribeConsumerGroups_result_t *exp = &expected[i]; + rd_kafka_resp_err_t exp_err = exp->err; + const rd_kafka_ConsumerGroupDescription_t *act = results[i]; + rd_kafka_resp_err_t act_err = rd_kafka_error_code( + rd_kafka_ConsumerGroupDescription_error(act)); + rd_kafka_consumer_group_state_t state = + rd_kafka_ConsumerGroupDescription_state(act); + if(include_authorized_operations){ + TEST_ASSERT( + rd_kafka_ConsumerGroupDescription_authorized_operations_count(act) != 0, + "Authorized operations not returned when requested\n"); + for(j=0;j< rd_kafka_ConsumerGroupDescription_authorized_operations_count(act);j++){ + acl_operation = + rd_kafka_ConsumerGroupDescription_authorized_operation(act,j); + TEST_SAY("%s operation is allowed\n", + rd_kafka_AclOperation_name(acl_operation)); + } + } + + } + + rd_kafka_event_destroy(rkev); + + for (i = 0; i < known_groups; i++) { + test_consumer_close(rks[i]); + rd_kafka_destroy(rks[i]); + } + + /* Wait session timeout + 1s. Because using static group membership */ + rd_sleep(6); + for(i=0;i Date: Fri, 17 Mar 2023 15:05:36 +0530 Subject: [PATCH 07/13] changes to describeCG tests --- examples/describe_consumer_groups.c | 2 +- src/rdkafka_admin.c | 46 ++++++++++++++++++++++------- src/rdkafka_request.c | 3 +- tests/0080-admin_ut.c | 16 +++++++++- tests/0081-admin.c | 29 ++++++++---------- 5 files changed, 66 insertions(+), 30 deletions(-) diff --git a/examples/describe_consumer_groups.c b/examples/describe_consumer_groups.c index edad573d30..020299a7d9 100644 --- a/examples/describe_consumer_groups.c +++ b/examples/describe_consumer_groups.c @@ -243,6 +243,7 @@ print_groups_info(const rd_kafka_DescribeConsumerGroups_result_t *grpdesc, 0, " "); } } + printf("\n"); } return 0; } @@ -410,6 +411,5 @@ int main(int argc, char **argv) { } cmd_describe_consumer_groups(conf, argc - optind, &argv[optind]); - return 0; } diff --git a/src/rdkafka_admin.c b/src/rdkafka_admin.c index 119a493367..25b3f0fba6 100644 --- a/src/rdkafka_admin.c +++ b/src/rdkafka_admin.c @@ -6196,6 +6196,7 @@ rd_kafka_ConsumerGroupDescription_new(const char *group_id, const rd_kafka_Node_t *coordinator, rd_kafka_error_t *error) { rd_kafka_ConsumerGroupDescription_t *grpdesc; + int i; grpdesc = rd_calloc(1, sizeof(*grpdesc)); grpdesc->group_id = rd_strdup(group_id); grpdesc->is_simple_consumer_group = is_simple_consumer_group; @@ -6210,7 +6211,19 @@ rd_kafka_ConsumerGroupDescription_new(const char *group_id, grpdesc->partition_assignor = !partition_assignor ? (char *)partition_assignor : rd_strdup(partition_assignor); - grpdesc->authorized_operations = authorized_operations; + if(authorized_operations == NULL) + grpdesc->authorized_operations = authorized_operations; + else{ + grpdesc->authorized_operations = + rd_list_new(rd_list_cnt(authorized_operations), rd_free); + for(i=0;iauthorized_operations, + oper); + } + } grpdesc->state = state; if (coordinator != NULL) grpdesc->coordinator = rd_kafka_Node_copy(coordinator); @@ -6272,6 +6285,8 @@ static void rd_kafka_ConsumerGroupDescription_destroy( rd_kafka_error_destroy(grpdesc->error); if (grpdesc->coordinator) rd_kafka_Node_destroy(grpdesc->coordinator); + if(likely(grpdesc->authorized_operations != NULL)) + rd_list_destroy(grpdesc->authorized_operations); rd_free(grpdesc); } @@ -6311,8 +6326,11 @@ size_t rd_kafka_ConsumerGroupDescription_authorized_operations_count( int rd_kafka_ConsumerGroupDescription_authorized_operation( const rd_kafka_ConsumerGroupDescription_t *grpdesc, size_t idx) { - rd_kafka_AclOperation_t* entry = rd_list_elem(grpdesc->authorized_operations, idx); - return *entry; + if(grpdesc->authorized_operations){ + rd_kafka_AclOperation_t* entry = rd_list_elem(grpdesc->authorized_operations, idx); + return *entry; + } + return 0; } rd_kafka_consumer_group_state_t rd_kafka_ConsumerGroupDescription_state( @@ -6440,8 +6458,11 @@ static rd_kafka_resp_err_t rd_kafka_admin_DescribeConsumerGroupsRequest( } /** - * @brief Parse authorized_operations returned in DescribeConsumerGroups - * @returns array of rd_bool_t of size RD_KAFKA_ACL_OPERATIONS__CNT + * @brief Parse authorized_operations returned in + * - DescribeConsumerGroups + * - DescribeTopics + * - DescribeCluster + * @returns list of acl operations */ rd_list_t* rd_kafka_AuthorizedOperations_parse(int32_t authorized_operations){ int i, bit; @@ -6450,11 +6471,11 @@ rd_list_t* rd_kafka_AuthorizedOperations_parse(int32_t authorized_operations){ /* in case of authorized_operations not requested, return NULL*/ if(authorized_operations<0) return NULL; - authorized_operations_list = rd_list_new(0, NULL); + authorized_operations_list = rd_list_new(0, rd_free); for(i=0; i> i) & 1; if(bit){ - entry = malloc(sizeof(rd_kafka_AclOperation_t)); + entry = rd_malloc(sizeof(rd_kafka_AclOperation_t)); *entry = i; rd_list_add(authorized_operations_list, entry); } @@ -6503,7 +6524,7 @@ rd_kafka_DescribeConsumerGroupsResponse_parse(rd_kafka_op_t *rko_req, while (cnt-- > 0) { int16_t error_code; int32_t authorized_operations = 0; - rd_list_t* authorized_operations_list; + rd_list_t* authorized_operations_list = NULL; rd_kafkap_str_t GroupId, GroupState, ProtocolType, ProtocolData; rd_bool_t is_simple_consumer_group, is_consumer_protocol_type; int32_t member_cnt; @@ -6613,14 +6634,15 @@ rd_kafka_DescribeConsumerGroupsResponse_parse(rd_kafka_op_t *rko_req, } if (api_version >= 3) { - /* TODO: implement KIP-430 */ rd_kafka_buf_read_i32(reply, &authorized_operations); /* assert that the last 3 bits are never set*/ rd_assert(!((authorized_operations >> 0) & 1)); rd_assert(!((authorized_operations >> 1) & 1)); rd_assert(!((authorized_operations >> 2) & 1)); - /* authorized_operations is -2147483648 in case of not requested, list has no elements in that case*/ - authorized_operations_list = rd_kafka_AuthorizedOperations_parse(authorized_operations); + /* authorized_operations is -2147483648 + * in case of not requested, list has no elements in that case*/ + authorized_operations_list = + rd_kafka_AuthorizedOperations_parse(authorized_operations); } if (error == NULL) { @@ -6629,6 +6651,8 @@ rd_kafka_DescribeConsumerGroupsResponse_parse(rd_kafka_op_t *rko_req, authorized_operations_list, rd_kafka_consumer_group_state_code(group_state), node, error); + if(authorized_operations_list) + rd_list_destroy(authorized_operations_list); } else { grpdesc = rd_kafka_ConsumerGroupDescription_new_error( group_id, error); diff --git a/src/rdkafka_request.c b/src/rdkafka_request.c index cffa201448..3b16004458 100644 --- a/src/rdkafka_request.c +++ b/src/rdkafka_request.c @@ -2014,6 +2014,8 @@ rd_kafka_error_t *rd_kafka_ListGroupsRequest(rd_kafka_broker_t *rkb, * with the groups (const char *) in \p groups. * Uses \p max_ApiVersion as maximum API version, * pass -1 to use the maximum available version. + * Uses \p include_authorized_operations to get + * group ACL authorized operations, 1 to request. * * The response (unparsed) will be enqueued on \p replyq * for handling by \p resp_cb (with \p opaque passed). @@ -2066,7 +2068,6 @@ rd_kafka_DescribeGroupsRequest(rd_kafka_broker_t *rkb, /* write IncludeAuthorizedOperations */ if (ApiVersion >= 3) { - /* TODO: implement KIP-430 */ rd_kafka_buf_write_bool(rkbuf, include_authorized_operations); } diff --git a/tests/0080-admin_ut.c b/tests/0080-admin_ut.c index 8408974fb1..fc59f269a1 100644 --- a/tests/0080-admin_ut.c +++ b/tests/0080-admin_ut.c @@ -633,9 +633,10 @@ static void do_test_DescribeConsumerGroups(const char *what, char errstr[512]; const char *errstr2; rd_kafka_resp_err_t err; + rd_kafka_error_t *error; test_timing_t timing; rd_kafka_event_t *rkev; - const rd_kafka_DeleteGroups_result_t *res; + const rd_kafka_DescribeConsumerGroups_result_t *res; const rd_kafka_ConsumerGroupDescription_t **resgroups; size_t resgroup_cnt; void *my_opaque = NULL, *opaque; @@ -657,6 +658,15 @@ static void do_test_DescribeConsumerGroups(const char *what, err = rd_kafka_AdminOptions_set_request_timeout( options, exp_timeout, errstr, sizeof(errstr)); TEST_ASSERT(!err, "%s", rd_kafka_err2str(err)); + if ((error = + rd_kafka_AdminOptions_set_include_authorized_operations( + options, 0))) { + fprintf(stderr, + "%% Failed to set require authorized operations: %s\n", + rd_kafka_error_string(error)); + rd_kafka_error_destroy(error); + TEST_FAIL("Failed to set include authorized operations\n"); + } if (useq) { my_opaque = (void *)456; @@ -724,6 +734,10 @@ static void do_test_DescribeConsumerGroups(const char *what, group_names[i], rd_kafka_error_string( rd_kafka_ConsumerGroupDescription_error(resgroups[i]))); + TEST_ASSERT( + rd_kafka_ConsumerGroupDescription_authorized_operations_count( + resgroups[i]) == 0, "Got authorized operations" + "when not requested"); } rd_kafka_event_destroy(rkev); diff --git a/tests/0081-admin.c b/tests/0081-admin.c index 686b1c40cc..1888133bc0 100644 --- a/tests/0081-admin.c +++ b/tests/0081-admin.c @@ -2848,6 +2848,8 @@ static void do_test_DescribeConsumerGroups(const char *what, rd_free(expected[i].group_id); } + test_DeleteTopics_simple(rk, q, &topic, 1, NULL); + rd_free(topic); if (options) @@ -3465,11 +3467,11 @@ static void do_test_DescribeConsumerGroups_with_authorized_ops(const char *what, TEST_ASSERT(!err, "%s", rd_kafka_err2str(err)); if ((error = rd_kafka_AdminOptions_set_include_authorized_operations( options, 1))) { - fprintf(stderr, - "%% Failed to set require authorized operations: %s\n", - rd_kafka_error_string(error)); - rd_kafka_error_destroy(error); - TEST_FAIL("Failed to set include authorized operations\n"); + 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"); } } @@ -3600,12 +3602,10 @@ static void do_test_DescribeConsumerGroups_with_authorized_ops(const char *what, TEST_ASSERT( rd_kafka_ConsumerGroupDescription_authorized_operations_count(act) != 0, "Authorized operations not returned when requested\n"); - for(j=0;j< rd_kafka_ConsumerGroupDescription_authorized_operations_count(act);j++){ - acl_operation = - rd_kafka_ConsumerGroupDescription_authorized_operation(act,j); - TEST_SAY("%s operation is allowed\n", - rd_kafka_AclOperation_name(acl_operation)); - } + TEST_ASSERT( + rd_kafka_ConsumerGroupDescription_authorized_operations_count(act) < 3, + "Expected only READ and DESCRIBE operations after createAcl(), got DELETE" + "as well\n"); } } @@ -3645,6 +3645,8 @@ static void do_test_DescribeConsumerGroups_with_authorized_ops(const char *what, rd_free(expected[i].group_id); } + test_DeleteTopics_simple(rk, q, &topic, 1, NULL); + rd_free(topic); if (options) @@ -4468,11 +4470,6 @@ static void do_test_apis(rd_kafka_type_t cltype) { do_test_unclean_destroy(cltype, 1 /*mainq*/); test_conf_init(&conf, NULL, 180); - // test_conf_set(conf, "sasl.username", "broker"); - // test_conf_set(conf, "sasl.password", "broker"); - // test_conf_set(conf, "sasl.mechanism", "SCRAM-SHA-256"); - // test_conf_set(conf, "security.protocol", "SASL_PLAINTEXT"); - // test_conf_set(conf, "bootstrap.servers", "localhost:9092"); test_conf_set(conf, "socket.timeout.ms", "10000"); rk = test_create_handle(cltype, conf); From 0bfbf67d9080b999b449a9440f70021efa1d6b4c Mon Sep 17 00:00:00 2001 From: jainruchir Date: Fri, 17 Mar 2023 16:05:22 +0530 Subject: [PATCH 08/13] dev_KIP430 branch merged --- src/rdkafka_admin.c | 27 +-------------------------- 1 file changed, 1 insertion(+), 26 deletions(-) diff --git a/src/rdkafka_admin.c b/src/rdkafka_admin.c index 25b3f0fba6..485c0d8821 100644 --- a/src/rdkafka_admin.c +++ b/src/rdkafka_admin.c @@ -6053,7 +6053,7 @@ rd_list_t* rd_kafka_AuthorizedOperations_parse(int32_t authorized_operations){ for(i=0; i> i) & 1; if(bit){ - entry = malloc(sizeof(rd_kafka_AclOperation_t)); + entry = rd_malloc(sizeof(rd_kafka_AclOperation_t)); *entry = i; rd_list_add(authorized_operations_list, entry); } @@ -6457,31 +6457,6 @@ static rd_kafka_resp_err_t rd_kafka_admin_DescribeConsumerGroupsRequest( return RD_KAFKA_RESP_ERR_NO_ERROR; } -/** - * @brief Parse authorized_operations returned in - * - DescribeConsumerGroups - * - DescribeTopics - * - DescribeCluster - * @returns list of acl operations - */ -rd_list_t* rd_kafka_AuthorizedOperations_parse(int32_t authorized_operations){ - int i, bit; - rd_list_t* authorized_operations_list = NULL; - rd_kafka_AclOperation_t* entry; - /* in case of authorized_operations not requested, return NULL*/ - if(authorized_operations<0) - return NULL; - authorized_operations_list = rd_list_new(0, rd_free); - for(i=0; i> i) & 1; - if(bit){ - entry = rd_malloc(sizeof(rd_kafka_AclOperation_t)); - *entry = i; - rd_list_add(authorized_operations_list, entry); - } - } - return authorized_operations_list; -} /** * @brief Parse DescribeConsumerGroupsResponse and create ADMIN_RESULT op. */ From 830d9b7635720e7d1f00ef4d28d0a0b4673f63b8 Mon Sep 17 00:00:00 2001 From: jainruchir Date: Thu, 30 Mar 2023 14:36:25 +0530 Subject: [PATCH 09/13] bug fix for adminoptions --- src/rdkafka_admin.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/rdkafka_admin.c b/src/rdkafka_admin.c index 485c0d8821..853f5ff18a 100644 --- a/src/rdkafka_admin.c +++ b/src/rdkafka_admin.c @@ -1708,7 +1708,7 @@ static void rd_kafka_AdminOptions_init(rd_kafka_t *rk, &options->include_authorized_operations, "include_authorized_operations", 0, 1, 0); else - rd_kafka_confval_disable(&options->require_stable_offsets, + rd_kafka_confval_disable(&options->include_authorized_operations, "include_authorized_operations"); if (options->for_api == RD_KAFKA_ADMIN_OP_ANY || From b9fea3cd0d22426aa668d3c28d0940f6608558c0 Mon Sep 17 00:00:00 2001 From: jainruchir Date: Tue, 11 Apr 2023 09:33:57 +0530 Subject: [PATCH 10/13] ClusterDescription destroy having double free bug removed --- src/rdkafka_admin.c | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/src/rdkafka_admin.c b/src/rdkafka_admin.c index 853f5ff18a..56dc1c3482 100644 --- a/src/rdkafka_admin.c +++ b/src/rdkafka_admin.c @@ -7406,7 +7406,7 @@ rd_kafka_ClusterDescription_new(const char *cluster_id, clusterdesc->cluster_authorized_operations = NULL; clusterdesc->node_cnt = md->broker_cnt; - clusterdesc->Nodes = malloc(sizeof(rd_kafka_Node_t) * md->broker_cnt); + clusterdesc->Nodes = rd_malloc(sizeof(rd_kafka_Node_t) * md->broker_cnt); for(i=0;ibroker_cnt;i++){ clusterdesc->Nodes[i].host = rd_strdup(md->brokers[i].host); clusterdesc->Nodes[i].port = md->brokers[i].port; @@ -7421,8 +7421,13 @@ static void rd_kafka_ClusterDescription_destroy( RD_IF_FREE(clusterdesc->cluster_id, rd_free); if(clusterdesc->cluster_authorized_operations) rd_list_destroy(clusterdesc->cluster_authorized_operations); - for(i=0;inode_cnt;i++){ - rd_kafka_Node_destroy(&clusterdesc->Nodes[i]); + if (clusterdesc->Nodes) { + for (i = 0; i < clusterdesc->node_cnt; i++) { + rd_kafka_Node_t *node = &(clusterdesc->Nodes[i]); + RD_IF_FREE(node->host, rd_free); + RD_IF_FREE(node->rack_id, rd_free); + } + RD_IF_FREE(clusterdesc->Nodes, rd_free); } rd_free(clusterdesc); } From 0e16f981138df2b95e5d092650f89cdd54b72805 Mon Sep 17 00:00:00 2001 From: jainruchir Date: Tue, 11 Apr 2023 10:04:40 +0530 Subject: [PATCH 11/13] style-fix --- examples/describe_cluster.c | 84 ++-- examples/describe_consumer_groups.c | 21 +- examples/describe_topics.c | 124 +++--- src/rdkafka.h | 97 +++-- src/rdkafka_admin.c | 580 ++++++++++++++-------------- src/rdkafka_admin.h | 39 +- src/rdkafka_cgrp.c | 10 +- src/rdkafka_event.c | 6 +- src/rdkafka_metadata.c | 124 +++--- src/rdkafka_metadata.h | 23 +- src/rdkafka_op.c | 8 +- src/rdkafka_op.h | 46 ++- src/rdkafka_partition.h | 15 +- src/rdkafka_request.c | 70 ++-- src/rdkafka_request.h | 23 +- src/rdkafka_topic.h | 11 +- tests/0080-admin_ut.c | 128 +++--- tests/0081-admin.c | 342 ++++++++-------- 18 files changed, 928 insertions(+), 823 deletions(-) diff --git a/examples/describe_cluster.c b/examples/describe_cluster.c index 968b8b591e..5e66230a86 100644 --- a/examples/describe_cluster.c +++ b/examples/describe_cluster.c @@ -69,18 +69,19 @@ static void stop(int sig) { static void usage(const char *reason, ...) { - fprintf(stderr, - "Describe cluster usage examples\n" - "\n" - "Usage: %s ...\n" - "\n" - "Options:\n" - " -b Bootstrap server list to connect to.\n" - " -X Set librdkafka configuration property.\n" - " See CONFIGURATION.md for full list.\n" - " -d Enable librdkafka debugging (%s).\n" - "\n", - argv0, rd_kafka_get_debug_contexts()); + fprintf( + stderr, + "Describe cluster usage examples\n" + "\n" + "Usage: %s ...\n" + "\n" + "Options:\n" + " -b Bootstrap server list to connect to.\n" + " -X Set librdkafka configuration property.\n" + " See CONFIGURATION.md for full list.\n" + " -d Enable librdkafka debugging (%s).\n" + "\n", + argv0, rd_kafka_get_debug_contexts()); if (reason) { va_list ap; @@ -142,33 +143,36 @@ print_cluster_info(const rd_kafka_DescribeCluster_result_t *clusterdesc) { int j, acl_operation; const rd_kafka_ClusterDescription_t *desc; int controller_id, node_cnt, cluster_authorized_operations_cnt; - const char* cluster_id; + const char *cluster_id; desc = rd_kafka_DescribeCluster_result_description(clusterdesc); controller_id = rd_kafka_ClusterDescription_controller_id(desc); - node_cnt = rd_kafka_ClusterDescription_node_cnt(desc); - cluster_authorized_operations_cnt = rd_kafka_ClusterDescription_cluster_acl_operations_cnt(desc); + node_cnt = rd_kafka_ClusterDescription_node_cnt(desc); + cluster_authorized_operations_cnt = + rd_kafka_ClusterDescription_cluster_acl_operations_cnt(desc); cluster_id = rd_kafka_ClusterDescription_cluster_id(desc); - printf("Cluster id: %s\t Controller id: %d\t ACL operations count allowed: %d\n", - cluster_id, controller_id, cluster_authorized_operations_cnt); - for(j=0;j \n" + "Usage: %s \n" " ...\n" "\n" "Options:\n" @@ -136,9 +136,8 @@ int64_t parse_int(const char *what, const char *str) { /** * @brief Print topics information. */ -static int -print_topics_info(const rd_kafka_DescribeTopics_result_t *topicdesc, - int topic_cnt) { +static int print_topics_info(const rd_kafka_DescribeTopics_result_t *topicdesc, + int topic_cnt) { size_t i, j; const rd_kafka_TopicDescription_t **result_topics; size_t result_topics_cnt; @@ -157,68 +156,87 @@ print_topics_info(const rd_kafka_DescribeTopics_result_t *topicdesc, for (i = 0; i < result_topics_cnt; i++) { int j, acl_operation; const rd_kafka_error_t *error; - const rd_kafka_TopicDescription_t *topic = - result_topics[i]; + const rd_kafka_TopicDescription_t *topic = result_topics[i]; const char *topic_name = rd_kafka_TopicDescription_topic_name(topic); - int topic_authorized_operations_cnt = rd_kafka_TopicDescription_topic_authorized_operations_cnt(topic); - int partition_cnt = rd_kafka_TopicDescription_topic_partition_cnt(topic); + int topic_authorized_operations_cnt = + rd_kafka_TopicDescription_topic_authorized_operations_cnt( + topic); + int partition_cnt = + rd_kafka_TopicDescription_topic_partition_cnt(topic); error = rd_kafka_TopicDescription_error(topic); - - if (rd_kafka_error_code(error)){ + + if (rd_kafka_error_code(error)) { printf("Topic: %s has error[%" PRId32 "]: %s\n", - topic_name, - rd_kafka_error_code(error), + topic_name, rd_kafka_error_code(error), rd_kafka_error_string(error)); continue; } - printf("Topic: %s succeeded, has %d topic authorized operations allowed, they are:\n", - topic_name, topic_authorized_operations_cnt); - for(j=0;jinclude_cluster_authorized_operations, RD_KAFKA_CONFVAL_INT, - &true_or_false, errstr, sizeof(errstr)); + &options->include_cluster_authorized_operations, + RD_KAFKA_CONFVAL_INT, &true_or_false, errstr, sizeof(errstr)); return !err ? NULL : rd_kafka_error_new(err, "%s", errstr); } @@ -1688,19 +1689,23 @@ static void rd_kafka_AdminOptions_init(rd_kafka_t *rk, "require_stable_offsets"); if (options->for_api == RD_KAFKA_ADMIN_OP_ANY || options->for_api == RD_KAFKA_ADMIN_OP_DESCRIBETOPICS) - rd_kafka_confval_init_int(&options->include_topic_authorized_operations, - "include_topic_authorized_operations",0,1,0); + rd_kafka_confval_init_int( + &options->include_topic_authorized_operations, + "include_topic_authorized_operations", 0, 1, 0); else - rd_kafka_confval_disable(&options->include_topic_authorized_operations, - "include_topic_authorized_operations"); - + rd_kafka_confval_disable( + &options->include_topic_authorized_operations, + "include_topic_authorized_operations"); + if (options->for_api == RD_KAFKA_ADMIN_OP_ANY || options->for_api == RD_KAFKA_ADMIN_OP_DESCRIBECLUSTER) - rd_kafka_confval_init_int(&options->include_cluster_authorized_operations, - "include_cluster_authorized_operations",0,1,0); + rd_kafka_confval_init_int( + &options->include_cluster_authorized_operations, + "include_cluster_authorized_operations", 0, 1, 0); else - rd_kafka_confval_disable(&options->include_cluster_authorized_operations, - "include_cluster_authorized_operations"); + rd_kafka_confval_disable( + &options->include_cluster_authorized_operations, + "include_cluster_authorized_operations"); if (options->for_api == RD_KAFKA_ADMIN_OP_ANY || options->for_api == RD_KAFKA_ADMIN_OP_DESCRIBECONSUMERGROUPS) @@ -1708,8 +1713,9 @@ static void rd_kafka_AdminOptions_init(rd_kafka_t *rk, &options->include_authorized_operations, "include_authorized_operations", 0, 1, 0); else - rd_kafka_confval_disable(&options->include_authorized_operations, - "include_authorized_operations"); + 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) @@ -6036,24 +6042,24 @@ const rd_kafka_error_t **rd_kafka_ListConsumerGroups_result_errors( */ /** - * @brief Parse authorized_operations returned in + * @brief Parse authorized_operations returned in * - DescribeConsumerGroups * - DescribeTopics * - DescribeCluster * @returns list of acl operations */ -rd_list_t* rd_kafka_AuthorizedOperations_parse(int32_t authorized_operations){ +rd_list_t *rd_kafka_AuthorizedOperations_parse(int32_t authorized_operations) { int i, bit; - rd_list_t* authorized_operations_list = NULL; - rd_kafka_AclOperation_t* entry; + rd_list_t *authorized_operations_list = NULL; + rd_kafka_AclOperation_t *entry; /* in case of authorized_operations not requested, return NULL*/ - if(authorized_operations<0) + if (authorized_operations < 0) return NULL; authorized_operations_list = rd_list_new(0, rd_free); - for(i=0; i> i) & 1; - if(bit){ - entry = rd_malloc(sizeof(rd_kafka_AclOperation_t)); + if (bit) { + entry = rd_malloc(sizeof(rd_kafka_AclOperation_t)); *entry = i; rd_list_add(authorized_operations_list, entry); } @@ -6191,7 +6197,7 @@ rd_kafka_ConsumerGroupDescription_new(const char *group_id, rd_bool_t is_simple_consumer_group, const rd_list_t *members, const char *partition_assignor, - rd_list_t* authorized_operations, + rd_list_t *authorized_operations, rd_kafka_consumer_group_state_t state, const rd_kafka_Node_t *coordinator, rd_kafka_error_t *error) { @@ -6208,23 +6214,22 @@ rd_kafka_ConsumerGroupDescription_new(const char *group_id, rd_list_copy_to(&grpdesc->members, members, rd_kafka_MemberDescription_list_copy, NULL); } - grpdesc->partition_assignor = !partition_assignor - ? (char *)partition_assignor - : rd_strdup(partition_assignor); - if(authorized_operations == NULL) + grpdesc->partition_assignor = !partition_assignor + ? (char *)partition_assignor + : rd_strdup(partition_assignor); + if (authorized_operations == NULL) grpdesc->authorized_operations = authorized_operations; - else{ - grpdesc->authorized_operations = - rd_list_new(rd_list_cnt(authorized_operations), rd_free); - for(i=0;iauthorized_operations, - oper); + else { + grpdesc->authorized_operations = + rd_list_new(rd_list_cnt(authorized_operations), rd_free); + for (i = 0; i < rd_list_cnt(authorized_operations); i++) { + int *entry = rd_list_elem(authorized_operations, i); + int *oper = rd_malloc(sizeof(int)); + *oper = *entry; + rd_list_add(grpdesc->authorized_operations, oper); } } - grpdesc->state = state; + grpdesc->state = state; if (coordinator != NULL) grpdesc->coordinator = rd_kafka_Node_copy(coordinator); grpdesc->error = @@ -6285,7 +6290,7 @@ static void rd_kafka_ConsumerGroupDescription_destroy( rd_kafka_error_destroy(grpdesc->error); if (grpdesc->coordinator) rd_kafka_Node_destroy(grpdesc->coordinator); - if(likely(grpdesc->authorized_operations != NULL)) + if (likely(grpdesc->authorized_operations != NULL)) rd_list_destroy(grpdesc->authorized_operations); rd_free(grpdesc); } @@ -6318,7 +6323,7 @@ const char *rd_kafka_ConsumerGroupDescription_partition_assignor( size_t rd_kafka_ConsumerGroupDescription_authorized_operations_count( const rd_kafka_ConsumerGroupDescription_t *grpdesc) { - if(grpdesc->authorized_operations) + if (grpdesc->authorized_operations) return grpdesc->authorized_operations->rl_cnt; return 0; } @@ -6326,8 +6331,9 @@ size_t rd_kafka_ConsumerGroupDescription_authorized_operations_count( int rd_kafka_ConsumerGroupDescription_authorized_operation( const rd_kafka_ConsumerGroupDescription_t *grpdesc, size_t idx) { - if(grpdesc->authorized_operations){ - rd_kafka_AclOperation_t* entry = rd_list_elem(grpdesc->authorized_operations, idx); + if (grpdesc->authorized_operations) { + rd_kafka_AclOperation_t *entry = + rd_list_elem(grpdesc->authorized_operations, idx); return *entry; } return 0; @@ -6498,8 +6504,8 @@ rd_kafka_DescribeConsumerGroupsResponse_parse(rd_kafka_op_t *rko_req, node = rd_kafka_Node_new(nodeid, host, port, NULL); while (cnt-- > 0) { int16_t error_code; - int32_t authorized_operations = 0; - rd_list_t* authorized_operations_list = NULL; + int32_t authorized_operations = 0; + rd_list_t *authorized_operations_list = NULL; rd_kafkap_str_t GroupId, GroupState, ProtocolType, ProtocolData; rd_bool_t is_simple_consumer_group, is_consumer_protocol_type; int32_t member_cnt; @@ -6614,10 +6620,12 @@ rd_kafka_DescribeConsumerGroupsResponse_parse(rd_kafka_op_t *rko_req, rd_assert(!((authorized_operations >> 0) & 1)); rd_assert(!((authorized_operations >> 1) & 1)); rd_assert(!((authorized_operations >> 2) & 1)); - /* authorized_operations is -2147483648 - * in case of not requested, list has no elements in that case*/ + /* authorized_operations is -2147483648 + * in case of not requested, list has no elements in + * that case*/ authorized_operations_list = - rd_kafka_AuthorizedOperations_parse(authorized_operations); + rd_kafka_AuthorizedOperations_parse( + authorized_operations); } if (error == NULL) { @@ -6626,7 +6634,7 @@ rd_kafka_DescribeConsumerGroupsResponse_parse(rd_kafka_op_t *rko_req, authorized_operations_list, rd_kafka_consumer_group_state_code(group_state), node, error); - if(authorized_operations_list) + if (authorized_operations_list) rd_list_destroy(authorized_operations_list); } else { grpdesc = rd_kafka_ConsumerGroupDescription_new_error( @@ -6808,49 +6816,59 @@ rd_kafka_DescribeConsumerGroups_result_groups( const int rd_kafka_TopicDescription_authorized_operation_idx( const rd_kafka_TopicDescription_t *topicdesc, size_t idx) { - rd_kafka_AclOperation_t* entry = rd_list_elem(topicdesc->topic_authorized_operations, idx); + rd_kafka_AclOperation_t *entry = + rd_list_elem(topicdesc->topic_authorized_operations, idx); return *entry; } const int rd_kafka_TopicDescription_topic_authorized_operations_cnt( const rd_kafka_TopicDescription_t *topicdesc) { - if(topicdesc->topic_authorized_operations) + if (topicdesc->topic_authorized_operations) return rd_list_cnt(topicdesc->topic_authorized_operations); return 0; } const int rd_kafka_TopicDescription_partiton_id( - const rd_kafka_TopicDescription_t *topicdesc, int idx) { + const rd_kafka_TopicDescription_t *topicdesc, + int idx) { return topicdesc->partitions[idx].id; } const int rd_kafka_TopicDescription_partiton_leader( - const rd_kafka_TopicDescription_t *topicdesc, int idx) { + const rd_kafka_TopicDescription_t *topicdesc, + int idx) { return topicdesc->partitions[idx].leader; } const int rd_kafka_TopicDescription_partiton_isr_cnt( - const rd_kafka_TopicDescription_t *topicdesc, int idx) { + const rd_kafka_TopicDescription_t *topicdesc, + int idx) { return topicdesc->partitions[idx].isr_cnt; } const int rd_kafka_TopicDescription_partiton_replica_cnt( - const rd_kafka_TopicDescription_t *topicdesc, int idx) { + const rd_kafka_TopicDescription_t *topicdesc, + int idx) { return topicdesc->partitions[idx].replica_cnt; } const int rd_kafka_TopicDescription_partiton_isrs_idx( - const rd_kafka_TopicDescription_t *topicdesc, int partition_idx, int isr_idx) { + const rd_kafka_TopicDescription_t *topicdesc, + int partition_idx, + int isr_idx) { return topicdesc->partitions[partition_idx].isrs[isr_idx]; } const int rd_kafka_TopicDescription_partiton_replica_idx( - const rd_kafka_TopicDescription_t *topicdesc, int partition_idx, int replica_idx) { + const rd_kafka_TopicDescription_t *topicdesc, + int partition_idx, + int replica_idx) { return topicdesc->partitions[partition_idx].replicas[replica_idx]; } const rd_kafka_error_t *rd_kafka_TopicDescription_partition_error( - const rd_kafka_TopicDescription_t *topicdesc, int idx) { + const rd_kafka_TopicDescription_t *topicdesc, + int idx) { return rd_kafka_error_new(topicdesc->partitions[idx].err, NULL); } @@ -6864,12 +6882,11 @@ const char *rd_kafka_TopicDescription_topic_name( return topicdesc->topic; } -const rd_kafka_error_t *rd_kafka_TopicDescription_error( - const rd_kafka_TopicDescription_t *topicdesc) { +const rd_kafka_error_t * +rd_kafka_TopicDescription_error(const rd_kafka_TopicDescription_t *topicdesc) { return rd_kafka_error_new(topicdesc->err, NULL); } -const rd_kafka_TopicDescription_t ** -rd_kafka_DescribeTopics_result_topics( +const rd_kafka_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; @@ -6881,22 +6898,22 @@ rd_kafka_DescribeTopics_result_topics( return (const rd_kafka_TopicDescription_t **) rko->rko_u.admin_result.results.rl_elems; } -static void rd_kafka_TopicDescription_destroy( - rd_kafka_TopicDescription_t *topicdesc) { +static void +rd_kafka_TopicDescription_destroy(rd_kafka_TopicDescription_t *topicdesc) { int i; if (likely(topicdesc->topic != NULL)) rd_free(topicdesc->topic); // if (likely(topicdesc->err != NULL)) // rd_kafka_error_destroy(topicdesc->err); - for(i=0;ipartition_cnt;i++){ - if(likely(topicdesc->partitions[i].isrs != NULL)) + for (i = 0; i < topicdesc->partition_cnt; i++) { + if (likely(topicdesc->partitions[i].isrs != NULL)) rd_free(topicdesc->partitions[i].isrs); - if(likely(topicdesc->partitions[i].replicas != NULL)) + if (likely(topicdesc->partitions[i].replicas != NULL)) rd_free(topicdesc->partitions[i].replicas); } - if(likely(topicdesc->partitions != NULL)) + if (likely(topicdesc->partitions != NULL)) rd_free(topicdesc->partitions); - if(likely(topicdesc->topic_authorized_operations != NULL)) + if (likely(topicdesc->topic_authorized_operations != NULL)) rd_list_destroy(topicdesc->topic_authorized_operations); rd_free(topicdesc); } @@ -6923,53 +6940,57 @@ static int rd_kafka_DescribeTopics_cmp(const void *a, const void *b) { */ static rd_kafka_TopicDescription_t * rd_kafka_TopicDescription_new(const char *topic, - struct rd_kafka_metadata_partition* partitions, - int partition_cnt, - rd_list_t* topic_authorized_operations, - rd_kafka_resp_err_t err) { + struct rd_kafka_metadata_partition *partitions, + int partition_cnt, + rd_list_t *topic_authorized_operations, + rd_kafka_resp_err_t err) { rd_kafka_TopicDescription_t *topicdesc; - int i,j; - topicdesc = rd_calloc(1, sizeof(*topicdesc)); - topicdesc->topic = rd_strdup(topic); + int i, j; + topicdesc = rd_calloc(1, sizeof(*topicdesc)); + topicdesc->topic = rd_strdup(topic); topicdesc->partition_cnt = partition_cnt; - topicdesc->err = err; - if(topic_authorized_operations){ - topicdesc->topic_authorized_operations = - rd_list_new(rd_list_cnt(topic_authorized_operations), rd_free); - for(i=0;itopic_authorized_operations, - oper); + topicdesc->err = err; + if (topic_authorized_operations) { + topicdesc->topic_authorized_operations = rd_list_new( + rd_list_cnt(topic_authorized_operations), rd_free); + for (i = 0; i < rd_list_cnt(topic_authorized_operations); i++) { + int *entry = + rd_list_elem(topic_authorized_operations, i); + int *oper = rd_malloc(sizeof(int)); + *oper = *entry; + rd_list_add(topicdesc->topic_authorized_operations, + oper); } - } - else + } else topicdesc->topic_authorized_operations = NULL; if (partitions == NULL) { topicdesc->partitions = NULL; } else { - topicdesc->partitions = rd_malloc(sizeof(*partitions) * partition_cnt); - for(i=0; i < partition_cnt; i++){ + topicdesc->partitions = + rd_malloc(sizeof(*partitions) * partition_cnt); + for (i = 0; i < partition_cnt; i++) { topicdesc->partitions[i].err = partitions[i].err; - topicdesc->partitions[i].id = partitions[i].id; - topicdesc->partitions[i].isr_cnt = partitions[i].isr_cnt; - topicdesc->partitions[i].isrs = - rd_malloc(sizeof(int32_t) * topicdesc->partitions[i].isr_cnt); - for(j=0; jpartitions[i].isr_cnt; j++){ - topicdesc->partitions[i].isrs[j] = - partitions[i].isrs[j]; + topicdesc->partitions[i].id = partitions[i].id; + topicdesc->partitions[i].isr_cnt = + partitions[i].isr_cnt; + topicdesc->partitions[i].isrs = rd_malloc( + sizeof(int32_t) * topicdesc->partitions[i].isr_cnt); + for (j = 0; j < topicdesc->partitions[i].isr_cnt; j++) { + topicdesc->partitions[i].isrs[j] = + partitions[i].isrs[j]; } topicdesc->partitions[i].leader = partitions[i].leader; - topicdesc->partitions[i].replica_cnt = partitions[i].replica_cnt; - topicdesc->partitions[i].replicas = - rd_malloc(sizeof(int32_t) * topicdesc->partitions[i].replica_cnt); - for(j=0; jpartitions[i].replica_cnt; j++){ - topicdesc->partitions[i].replicas[j] = - partitions[i].replicas[j]; + topicdesc->partitions[i].replica_cnt = + partitions[i].replica_cnt; + topicdesc->partitions[i].replicas = + rd_malloc(sizeof(int32_t) * + topicdesc->partitions[i].replica_cnt); + for (j = 0; j < topicdesc->partitions[i].replica_cnt; + j++) { + topicdesc->partitions[i].replicas[j] = + partitions[i].replicas[j]; } - } } return topicdesc; @@ -6981,19 +7002,17 @@ rd_kafka_TopicDescription_new(const char *topic, * @return A new allocated copy of the passed TopicDescription. */ static rd_kafka_TopicDescription_t * -rd_kafka_TopicDescription_copy( - const rd_kafka_TopicDescription_t *topicdesc) { +rd_kafka_TopicDescription_copy(const rd_kafka_TopicDescription_t *topicdesc) { return rd_kafka_TopicDescription_new( - topicdesc->topic, topicdesc->partitions, - topicdesc->partition_cnt, topicdesc->topic_authorized_operations, - topicdesc->err); + topicdesc->topic, topicdesc->partitions, topicdesc->partition_cnt, + topicdesc->topic_authorized_operations, topicdesc->err); } /** * @brief Same as rd_kafka_TopicDescription_copy() but suitable for * rd_list_copy(). The \p opaque is ignored. */ static void *rd_kafka_TopicDescription_copy_opaque(const void *topicdesc, - void *opaque) { + void *opaque) { return rd_kafka_TopicDescription_copy(topicdesc); } /** @@ -7005,16 +7024,15 @@ static void *rd_kafka_TopicDescription_copy_opaque(const void *topicdesc, */ static rd_kafka_TopicDescription_t * rd_kafka_TopicDescription_new_error(const char *topic, - rd_kafka_resp_err_t err) { - return rd_kafka_TopicDescription_new( - topic, NULL, 0, NULL, err); + rd_kafka_resp_err_t err) { + return rd_kafka_TopicDescription_new(topic, NULL, 0, NULL, err); } /** @brief Merge the DescribeTopics response from a single broker * into the user response list. */ -static void rd_kafka_DescribeTopics_response_merge( - rd_kafka_op_t *rko_fanout, - const rd_kafka_op_t *rko_partial) { +static void +rd_kafka_DescribeTopics_response_merge(rd_kafka_op_t *rko_fanout, + const rd_kafka_op_t *rko_partial) { rd_kafka_TopicDescription_t *topicres = NULL; rd_kafka_TopicDescription_t *newtopicres; const char *topic = rko_partial->rko_u.admin_result.opaque; @@ -7065,36 +7083,31 @@ static void rd_kafka_DescribeTopics_response_merge( * 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){ +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) { int i; rd_kafka_resp_err_t err; int include_topic_authorized_operations; - include_topic_authorized_operations = - rd_kafka_confval_get_int(&options->include_topic_authorized_operations); + include_topic_authorized_operations = rd_kafka_confval_get_int( + &options->include_topic_authorized_operations); /* resp_cb = rd_kafka_admin_handle_response; */ - + // error = Call metadata request - err = rd_kafka_MetadataRequest(rkb, topics, - "describe topics", - rd_false, - rd_false, - include_topic_authorized_operations, - rd_false, - NULL, - resp_cb, 0, opaque); + err = rd_kafka_MetadataRequest(rkb, topics, "describe topics", rd_false, + rd_false, + include_topic_authorized_operations, + rd_false, NULL, resp_cb, 0, opaque); if (err) { - rd_snprintf(errstr, errstr_size, "%s", - rd_kafka_err2str(err)); + rd_snprintf(errstr, errstr_size, "%s", rd_kafka_err2str(err)); return err; } @@ -7106,17 +7119,18 @@ static rd_kafka_resp_err_t rd_kafka_admin_DescribeTopicsRequest( */ 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_op_t **rko_resultp, + rd_kafka_buf_t *reply, + char *errstr, + size_t errstr_size) { 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; - rd_kafka_topic_authorized_operations_pair_t* topic_authorized_operations = NULL; + rd_list_t topics = rko_req->rko_u.admin_request.args; + rd_kafka_broker_t *rkb = reply->rkbuf_rkb; + rd_kafka_topic_authorized_operations_pair_t + *topic_authorized_operations = NULL; int32_t cluster_authorized_operations; - char* cluster_id = NULL; + char *cluster_id = NULL; int controller_id; int i, cnt; rd_kafka_op_t *rko_result = NULL; @@ -7124,35 +7138,39 @@ rd_kafka_DescribeTopicsResponse_parse(rd_kafka_op_t *rko_req, // thrd_is_current(rk->rk_thread)); // rd_kafka_assert(NULL, err == RD_KAFKA_RESP_ERR__DESTROY); - err = rd_kafka_parse_Metadata(rkb, NULL, reply, &md, &topic_authorized_operations,&cluster_authorized_operations, &topics, - &cluster_id, &controller_id); + err = rd_kafka_parse_Metadata(rkb, NULL, reply, &md, + &topic_authorized_operations, + &cluster_authorized_operations, &topics, + &cluster_id, &controller_id); if (err) goto err; rko_result = rd_kafka_admin_result_new(rko_req); rd_list_init(&rko_result->rko_u.admin_result.results, md->topic_cnt, - rd_kafka_TopicDescription_free); + rd_kafka_TopicDescription_free); cnt = md->topic_cnt; - i=0; - while(cnt--){ + i = 0; + while (cnt--) { rd_kafka_TopicDescription_t *topicdesc = NULL; - /* topics in md should be in the same order as in topic_authorized_operations*/ - rd_assert(strcmp(md->topics[i].topic, topic_authorized_operations[i].topic_name) == 0); - if(md->topics[i].err == RD_KAFKA_RESP_ERR_NO_ERROR){ - rd_list_t* authorized_operations; - authorized_operations = - rd_kafka_AuthorizedOperations_parse( - topic_authorized_operations[i].authorized_operations); - topicdesc = rd_kafka_TopicDescription_new(md->topics[i].topic, - md->topics[i].partitions, - md->topics[i].partition_cnt, - authorized_operations, - md->topics[i].err); - if(authorized_operations) + /* topics in md should be in the same order as in + * topic_authorized_operations*/ + rd_assert(strcmp(md->topics[i].topic, + topic_authorized_operations[i].topic_name) == + 0); + if (md->topics[i].err == RD_KAFKA_RESP_ERR_NO_ERROR) { + rd_list_t *authorized_operations; + authorized_operations = + rd_kafka_AuthorizedOperations_parse( + topic_authorized_operations[i] + .authorized_operations); + topicdesc = rd_kafka_TopicDescription_new( + md->topics[i].topic, md->topics[i].partitions, + md->topics[i].partition_cnt, authorized_operations, + md->topics[i].err); + if (authorized_operations) rd_list_destroy(authorized_operations); - } - else - topicdesc = rd_kafka_TopicDescription_new_error(md->topics[i].topic, - md->topics[i].err); + } else + topicdesc = rd_kafka_TopicDescription_new_error( + md->topics[i].topic, md->topics[i].err); rd_list_add(&rko_result->rko_u.admin_result.results, topicdesc); } *rko_resultp = rko_result; @@ -7161,31 +7179,30 @@ rd_kafka_DescribeTopicsResponse_parse(rd_kafka_op_t *rko_req, err: if (rko_result) rd_kafka_op_destroy(rko_result); - rd_snprintf( - errstr, errstr_size, - "DescribeTopics response protocol parse failure: %s", - rd_kafka_err2str(reply->rkbuf_err)); + rd_snprintf(errstr, errstr_size, + "DescribeTopics response protocol parse failure: %s", + rd_kafka_err2str(reply->rkbuf_err)); return reply->rkbuf_err; } void rd_kafka_DescribeTopics(rd_kafka_t *rk, - const char** topics, - size_t topics_cnt, - const rd_kafka_AdminOptions_t *options, - rd_kafka_queue_t *rkqu){ + const char **topics, + size_t topics_cnt, + const rd_kafka_AdminOptions_t *options, + rd_kafka_queue_t *rkqu) { rd_kafka_op_t *rko_fanout; rd_list_t dup_list; size_t i; - + static const struct rd_kafka_admin_fanout_worker_cbs fanout_cbs = { rd_kafka_DescribeTopics_response_merge, rd_kafka_TopicDescription_copy_opaque}; rd_assert(rkqu); - rko_fanout = rd_kafka_admin_fanout_op_new( - rk, RD_KAFKA_OP_DESCRIBETOPICS, - RD_KAFKA_EVENT_DESCRIBETOPICS_RESULT, &fanout_cbs, options, - rkqu->rkqu_q); + rko_fanout = + rd_kafka_admin_fanout_op_new(rk, RD_KAFKA_OP_DESCRIBETOPICS, + RD_KAFKA_EVENT_DESCRIBETOPICS_RESULT, + &fanout_cbs, options, rkqu->rkqu_q); if (topics_cnt == 0) { rd_kafka_admin_result_fail(rko_fanout, @@ -7213,8 +7230,7 @@ void rd_kafka_DescribeTopics(rd_kafka_t *rk, rd_list_copy_to(&dup_list, &rko_fanout->rko_u.admin_request.args, NULL, NULL); rd_list_sort(&dup_list, rd_kafka_DescribeTopics_cmp); - if (rd_list_find_duplicate(&dup_list, - rd_kafka_DescribeTopics_cmp)) { + if (rd_list_find_duplicate(&dup_list, rd_kafka_DescribeTopics_cmp)) { rd_list_destroy(&dup_list); rd_kafka_admin_result_fail(rko_fanout, RD_KAFKA_RESP_ERR__INVALID_ARG, @@ -7266,7 +7282,6 @@ void rd_kafka_DescribeTopics(rd_kafka_t *rk, rd_kafka_q_enq(rk->rk_ops, rko); } - } /**@}*/ @@ -7281,14 +7296,16 @@ void rd_kafka_DescribeTopics(rd_kafka_t *rk, */ const rd_kafka_Node_t *rd_kafka_ClusterDescription_node_idx( - const rd_kafka_ClusterDescription_t *clusterdesc, int idx) { + const rd_kafka_ClusterDescription_t *clusterdesc, + int idx) { return &(clusterdesc->Nodes[idx]); } const int rd_kafka_ClusterDescription_authorized_operation_idx( const rd_kafka_ClusterDescription_t *clusterdesc, size_t idx) { - rd_kafka_AclOperation_t* entry = rd_list_elem(clusterdesc->cluster_authorized_operations, idx); + rd_kafka_AclOperation_t *entry = + rd_list_elem(clusterdesc->cluster_authorized_operations, idx); return *entry; } @@ -7302,7 +7319,7 @@ const int rd_kafka_ClusterDescription_controller_id( } const int rd_kafka_ClusterDescription_cluster_acl_operations_cnt( const rd_kafka_ClusterDescription_t *clusterdesc) { - if(clusterdesc->cluster_authorized_operations) + if (clusterdesc->cluster_authorized_operations) return rd_list_cnt(clusterdesc->cluster_authorized_operations); return 0; } @@ -7315,18 +7332,16 @@ const rd_kafka_ClusterDescription_t * rd_kafka_DescribeCluster_result_description( const rd_kafka_DescribeTopics_result_t *result) { int cluster_result_cnt; - const rd_kafka_ClusterDescription_t* clusterdesc; + const rd_kafka_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); + 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); - + clusterdesc = rd_list_elem(&rko->rko_u.admin_result.results, 0); + return clusterdesc; } @@ -7337,35 +7352,36 @@ rd_kafka_DescribeCluster_result_description( * @return A new allocated copy of the passed ClusterDescription. */ static rd_kafka_ClusterDescription_t * -rd_kafka_ClusterDescription_copy( - const rd_kafka_ClusterDescription_t *desc) { +rd_kafka_ClusterDescription_copy(const rd_kafka_ClusterDescription_t *desc) { rd_kafka_ClusterDescription_t *clusterdesc; int i; clusterdesc = rd_calloc(1, sizeof(*clusterdesc)); - clusterdesc->cluster_id = rd_strdup(desc->cluster_id); + clusterdesc->cluster_id = rd_strdup(desc->cluster_id); clusterdesc->controller_id = desc->controller_id; - - if(desc->cluster_authorized_operations){ - clusterdesc->cluster_authorized_operations = - rd_list_new(rd_list_cnt(desc->cluster_authorized_operations), rd_free); - for(i=0;icluster_authorized_operations); i++){ - int* entry = rd_list_elem(desc->cluster_authorized_operations, i); - int* oper = malloc(sizeof(int)); - *oper = *entry; - rd_list_add(clusterdesc->cluster_authorized_operations, - oper); + + if (desc->cluster_authorized_operations) { + clusterdesc->cluster_authorized_operations = rd_list_new( + rd_list_cnt(desc->cluster_authorized_operations), rd_free); + for (i = 0; + i < rd_list_cnt(desc->cluster_authorized_operations); + i++) { + int *entry = rd_list_elem( + desc->cluster_authorized_operations, i); + int *oper = malloc(sizeof(int)); + *oper = *entry; + rd_list_add(clusterdesc->cluster_authorized_operations, + oper); } - } - else + } else clusterdesc->cluster_authorized_operations = NULL; clusterdesc->node_cnt = desc->node_cnt; clusterdesc->Nodes = malloc(sizeof(rd_kafka_Node_t) * desc->node_cnt); - for(i=0;inode_cnt;i++){ + for (i = 0; i < desc->node_cnt; i++) { clusterdesc->Nodes[i].host = rd_strdup(desc->Nodes[i].host); clusterdesc->Nodes[i].port = desc->Nodes[i].port; - clusterdesc->Nodes[i].id = desc->Nodes[i].id; + clusterdesc->Nodes[i].id = desc->Nodes[i].id; } return clusterdesc; } @@ -7376,41 +7392,43 @@ rd_kafka_ClusterDescription_copy( * @param cluster_id current cluster_id * @param controller_id current controller_id. * @param md metadata struct returned by parse_metadata(). - * + * * @returns new ClusterDescription object. */ static rd_kafka_ClusterDescription_t * rd_kafka_ClusterDescription_new(const char *cluster_id, - int controller_id, - rd_list_t* cluster_authorized_operations, - struct rd_kafka_metadata *md) { + int controller_id, + rd_list_t *cluster_authorized_operations, + struct rd_kafka_metadata *md) { rd_kafka_ClusterDescription_t *clusterdesc; int i; clusterdesc = rd_calloc(1, sizeof(*clusterdesc)); - clusterdesc->cluster_id = rd_strdup(cluster_id); + clusterdesc->cluster_id = rd_strdup(cluster_id); clusterdesc->controller_id = controller_id; - - if(cluster_authorized_operations){ - clusterdesc->cluster_authorized_operations = - rd_list_new(rd_list_cnt(cluster_authorized_operations), rd_free); - for(i=0;icluster_authorized_operations, - oper); + + if (cluster_authorized_operations) { + clusterdesc->cluster_authorized_operations = rd_list_new( + rd_list_cnt(cluster_authorized_operations), rd_free); + for (i = 0; i < rd_list_cnt(cluster_authorized_operations); + i++) { + int *entry = + rd_list_elem(cluster_authorized_operations, i); + int *oper = malloc(sizeof(int)); + *oper = *entry; + rd_list_add(clusterdesc->cluster_authorized_operations, + oper); } - } - else + } else clusterdesc->cluster_authorized_operations = NULL; clusterdesc->node_cnt = md->broker_cnt; - clusterdesc->Nodes = rd_malloc(sizeof(rd_kafka_Node_t) * md->broker_cnt); - for(i=0;ibroker_cnt;i++){ + clusterdesc->Nodes = + rd_malloc(sizeof(rd_kafka_Node_t) * md->broker_cnt); + for (i = 0; i < md->broker_cnt; i++) { clusterdesc->Nodes[i].host = rd_strdup(md->brokers[i].host); clusterdesc->Nodes[i].port = md->brokers[i].port; - clusterdesc->Nodes[i].id = md->brokers[i].id; + clusterdesc->Nodes[i].id = md->brokers[i].id; } return clusterdesc; } @@ -7419,15 +7437,15 @@ static void rd_kafka_ClusterDescription_destroy( rd_kafka_ClusterDescription_t *clusterdesc) { int i; RD_IF_FREE(clusterdesc->cluster_id, rd_free); - if(clusterdesc->cluster_authorized_operations) + if (clusterdesc->cluster_authorized_operations) rd_list_destroy(clusterdesc->cluster_authorized_operations); if (clusterdesc->Nodes) { - for (i = 0; i < clusterdesc->node_cnt; i++) { - rd_kafka_Node_t *node = &(clusterdesc->Nodes[i]); - RD_IF_FREE(node->host, rd_free); - RD_IF_FREE(node->rack_id, rd_free); - } - RD_IF_FREE(clusterdesc->Nodes, rd_free); + for (i = 0; i < clusterdesc->node_cnt; i++) { + rd_kafka_Node_t *node = &(clusterdesc->Nodes[i]); + RD_IF_FREE(node->host, rd_free); + RD_IF_FREE(node->rack_id, rd_free); + } + RD_IF_FREE(clusterdesc->Nodes, rd_free); } rd_free(clusterdesc); } @@ -7439,35 +7457,28 @@ static void rd_kafka_ClusterDescription_free(void *ptr) { */ static rd_kafka_resp_err_t rd_kafka_admin_DescribeClusterRequest(rd_kafka_broker_t *rkb, - const rd_list_t *topics /*(char*)*/, - rd_kafka_AdminOptions_t *options, - char *errstr, - size_t errstr_size, - rd_kafka_replyq_t replyq, - rd_kafka_resp_cb_t *resp_cb, - void *opaque) { + 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, cluster_authorized_operations; + int include_cluster_authorized_operations; - include_topic_authorized_operations = - rd_kafka_confval_get_int(&options->include_topic_authorized_operations); - cluster_authorized_operations = - rd_kafka_confval_get_int(&options->include_cluster_authorized_operations); + include_cluster_authorized_operations = rd_kafka_confval_get_int( + &options->include_cluster_authorized_operations); /* resp_cb = rd_kafka_admin_handle_response; */ - + // err = Call metadata request with NULL topics - err = rd_kafka_MetadataRequest(rkb, NULL, - "describe cluster", - rd_false, - cluster_authorized_operations, - include_topic_authorized_operations, - rd_false, - NULL, - resp_cb, 1, opaque); + err = rd_kafka_MetadataRequest(rkb, NULL, "describe cluster", rd_false, + include_cluster_authorized_operations, + rd_false, rd_false, NULL, resp_cb, 1, + opaque); if (err) { - rd_snprintf(errstr, errstr_size, "%s", - rd_kafka_err2str(err)); + rd_snprintf(errstr, errstr_size, "%s", rd_kafka_err2str(err)); return err; } @@ -7478,18 +7489,19 @@ rd_kafka_admin_DescribeClusterRequest(rd_kafka_broker_t *rkb, */ 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_op_t **rko_resultp, + rd_kafka_buf_t *reply, + char *errstr, + size_t errstr_size) { struct rd_kafka_metadata *md = 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_topic_authorized_operations_pair_t* topic_authorized_operations = NULL; + rd_list_t topics = rko_req->rko_u.admin_request.args; + rd_kafka_broker_t *rkb = reply->rkbuf_rkb; + rd_kafka_topic_authorized_operations_pair_t + *topic_authorized_operations = NULL; int32_t cluster_authorized_operations; - char* cluster_id = NULL; + char *cluster_id = NULL; int controller_id; int i; rd_kafka_op_t *rko_result = NULL; @@ -7497,24 +7509,26 @@ rd_kafka_DescribeClusterResponse_parse(rd_kafka_op_t *rko_req, // thrd_is_current(rk->rk_thread)); // rd_kafka_assert(NULL, err == RD_KAFKA_RESP_ERR__DESTROY); - err = rd_kafka_parse_Metadata(rkb, NULL, reply, &md, &topic_authorized_operations,&cluster_authorized_operations, &topics, - &cluster_id, &controller_id); + err = rd_kafka_parse_Metadata(rkb, NULL, reply, &md, + &topic_authorized_operations, + &cluster_authorized_operations, &topics, + &cluster_id, &controller_id); if (err) goto err; rko_result = rd_kafka_admin_result_new(rko_req); rd_list_init(&rko_result->rko_u.admin_result.results, 1, - rd_kafka_ClusterDescription_free); + rd_kafka_ClusterDescription_free); - rd_list_t* authorized_operations; - authorized_operations = - rd_kafka_AuthorizedOperations_parse(cluster_authorized_operations); + rd_list_t *authorized_operations; + authorized_operations = + rd_kafka_AuthorizedOperations_parse(cluster_authorized_operations); - clusterdesc = rd_kafka_ClusterDescription_new(cluster_id, - controller_id, authorized_operations, md); - if(authorized_operations) + clusterdesc = rd_kafka_ClusterDescription_new( + cluster_id, controller_id, authorized_operations, md); + if (authorized_operations) rd_list_destroy(authorized_operations); rd_free(cluster_id); - + rd_list_add(&rko_result->rko_u.admin_result.results, clusterdesc); *rko_resultp = rko_result; return RD_KAFKA_RESP_ERR_NO_ERROR; @@ -7522,15 +7536,14 @@ rd_kafka_DescribeClusterResponse_parse(rd_kafka_op_t *rko_req, err: if (rko_result) rd_kafka_op_destroy(rko_result); - rd_snprintf( - errstr, errstr_size, - "DescribeTopics response protocol parse failure: %s", - rd_kafka_err2str(reply->rkbuf_err)); + rd_snprintf(errstr, errstr_size, + "DescribeTopics response protocol parse failure: %s", + rd_kafka_err2str(reply->rkbuf_err)); return reply->rkbuf_err; } void rd_kafka_DescribeCluster(rd_kafka_t *rk, - const rd_kafka_AdminOptions_t *options, - rd_kafka_queue_t *rkqu) { + 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, @@ -7541,8 +7554,7 @@ void rd_kafka_DescribeCluster(rd_kafka_t *rk, // }; rko = rd_kafka_admin_request_op_new( rk, RD_KAFKA_OP_DESCRIBECLUSTER, - RD_KAFKA_EVENT_DESCRIBECLUSTER_RESULT, &cbs, options, - rkqu->rkqu_q); + RD_KAFKA_EVENT_DESCRIBECLUSTER_RESULT, &cbs, options, rkqu->rkqu_q); // rko = rd_kafka_admin_request_op_target_all_new( // rk, RD_KAFKA_OP_DESCRIBECLUSTER, // RD_KAFKA_EVENT_DESCRIBECLUSTER_RESULT, &cbs, &fanout_cbs, diff --git a/src/rdkafka_admin.h b/src/rdkafka_admin.h index 8c6ba17748..0175f3b3e6 100644 --- a/src/rdkafka_admin.h +++ b/src/rdkafka_admin.h @@ -97,21 +97,22 @@ struct rd_kafka_AdminOptions_s { * Valid for: * DescribeConsumerGroups */ - + rd_kafka_confval_t - include_topic_authorized_operations; /**< BOOL: Whether broker should return - * topic authorized operations. - * Valid for: - * DescribeTopic - * MetadataRequest - */ + include_topic_authorized_operations; /**< BOOL: Whether broker + * should return topic + * authorized operations. Valid + * for: DescribeTopic + * MetadataRequest + */ rd_kafka_confval_t - include_cluster_authorized_operations; /**< BOOL: Whether broker should return - * cluster authorized operations. - * Valid for: - * DescribeCluster - * MetadataRequest - */ + include_cluster_authorized_operations; /**< BOOL: Whether broker + * should return cluster + * authorized operations. + * Valid for: + * DescribeCluster + * MetadataRequest + */ rd_kafka_confval_t match_consumer_group_states; /**< PTR: list of consumer group states @@ -495,7 +496,7 @@ struct rd_kafka_ConsumerGroupDescription_s { /** Consumer group coordinator. */ rd_kafka_Node_t *coordinator; /** Authorized operations. */ - rd_list_t* authorized_operations; + rd_list_t *authorized_operations; /** Group specific error. */ rd_kafka_error_t *error; }; @@ -514,7 +515,8 @@ struct rd_kafka_TopicDescription_s { int partition_cnt; /**< Number of partitions in \p partitions*/ struct rd_kafka_metadata_partition *partitions; /**< Partitions */ rd_kafka_resp_err_t err; /**< Topic error reported by broker */ - rd_list_t* topic_authorized_operations; /**< ACL operations allowed for topics */ + rd_list_t *topic_authorized_operations; /**< ACL operations allowed for + topics */ }; /**@}*/ @@ -528,10 +530,11 @@ struct rd_kafka_TopicDescription_s { */ struct rd_kafka_ClusterDescription_s { char *cluster_id; /**< current cluster id in \p cluster*/ - int controller_id; /**< current controller id in \p cluster*/ - int node_cnt; /**< Number of brokers in \p cluster*/ + int controller_id; /**< current controller id in \p cluster*/ + int node_cnt; /**< Number of brokers in \p cluster*/ rd_kafka_Node_t *Nodes; /**< Nodes */ - rd_list_t* cluster_authorized_operations; /**< ACL operations allowed for cluster */ + rd_list_t *cluster_authorized_operations; /**< ACL operations allowed + for cluster */ }; /**@}*/ diff --git a/src/rdkafka_cgrp.c b/src/rdkafka_cgrp.c index 17fb813526..43d012106c 100644 --- a/src/rdkafka_cgrp.c +++ b/src/rdkafka_cgrp.c @@ -2220,11 +2220,11 @@ static int rd_kafka_cgrp_metadata_refresh(rd_kafka_cgrp_t *rkcg, rd_kafka_cgrp_handle_Metadata_op); rd_kafka_op_set_replyq(rko, rkcg->rkcg_ops, 0); - err = rd_kafka_metadata_request(rkcg->rkcg_rk, NULL, &topics, - rd_false /*!allow auto create */, - rd_true /*include cluster authorized operations */, - rd_true /*include topic authorized operations */, - rd_true /*cgrp_update*/, reason, rko); + err = rd_kafka_metadata_request( + rkcg->rkcg_rk, NULL, &topics, rd_false /*!allow auto create */, + rd_true /*include cluster authorized operations */, + rd_true /*include topic authorized operations */, + rd_true /*cgrp_update*/, reason, rko); if (err) { rd_kafka_dbg(rk, CGRP | RD_KAFKA_DBG_METADATA, "CGRPMETADATA", "%s: need to refresh metadata (%dms old) " diff --git a/src/rdkafka_event.c b/src/rdkafka_event.c index 9571ebedc4..7e2aa2e4e0 100644 --- a/src/rdkafka_event.c +++ b/src/rdkafka_event.c @@ -370,8 +370,7 @@ rd_kafka_event_DescribeConsumerGroups_result(rd_kafka_event_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) + if (!rkev || rkev->rko_evtype != RD_KAFKA_EVENT_DESCRIBETOPICS_RESULT) return NULL; else return (const rd_kafka_DescribeTopics_result_t *)rkev; @@ -379,8 +378,7 @@ rd_kafka_event_DescribeTopics_result(rd_kafka_event_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) + if (!rkev || rkev->rko_evtype != RD_KAFKA_EVENT_DESCRIBECLUSTER_RESULT) return NULL; else return (const rd_kafka_DescribeCluster_result_t *)rkev; diff --git a/src/rdkafka_metadata.c b/src/rdkafka_metadata.c index e896b62ee8..c204326d99 100644 --- a/src/rdkafka_metadata.c +++ b/src/rdkafka_metadata.c @@ -82,17 +82,17 @@ rd_kafka_metadata(rd_kafka_t *rk, rd_kafka_op_set_replyq(rko, rkq, 0); rko->rko_u.metadata.force = 1; /* Force metadata request regardless * of outstanding metadata requests. */ - rd_kafka_MetadataRequest(rkb, &topics, "application requested", - allow_auto_create_topics, - rd_true /*include cluster authorized operations */, - rd_true /*include topic authorized operations */, - /* cgrp_update: - * Only update consumer group state - * on response if this lists all - * topics in the cluster, since a - * partial request may make it seem - * like some subscribed topics are missing. */ - all_topics ? rd_true : rd_false, rko, NULL, 0, NULL); + rd_kafka_MetadataRequest( + rkb, &topics, "application requested", allow_auto_create_topics, + rd_true /*include cluster authorized operations */, + rd_true /*include topic authorized operations */, + /* cgrp_update: + * Only update consumer group state + * on response if this lists all + * topics in the cluster, since a + * partial request may make it seem + * like some subscribed topics are missing. */ + all_topics ? rd_true : rd_false, rko, NULL, 0, NULL); rd_list_destroy(&topics); rd_kafka_broker_destroy(rkb); @@ -275,11 +275,12 @@ rd_bool_t rd_kafka_has_reliable_leader_epochs(rd_kafka_broker_t *rkb) { * @brief Handle a Metadata response message. * * @param topics are the requested topics (may be NULL) - * @param topic_authorized_operations topics mapped to their acl operations allowed + * @param topic_authorized_operations topics mapped to their acl operations + allowed * @param cluster_authorized_operations acl operations allowed for cluster * @param rk_cluster_id returns the current cluster_id (char*) * @param rk_controller_id returns the current controller_id (int) - * + * * The metadata will be marshalled into 'struct rd_kafka_metadata*' structs. * * The marshalled metadata is returned in \p *mdp, (NULL on error). @@ -288,31 +289,36 @@ rd_bool_t rd_kafka_has_reliable_leader_epochs(rd_kafka_broker_t *rkb) { * * @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, - struct rd_kafka_metadata **mdp, - rd_kafka_topic_authorized_operations_pair_t** topic_authorized_operations, - int32_t* cluster_authorized_operations, - rd_list_t* request_topics, - char** rk_cluster_id, - int* rk_controller_id) { +rd_kafka_resp_err_t rd_kafka_parse_Metadata( + rd_kafka_broker_t *rkb, + rd_kafka_buf_t *request, + rd_kafka_buf_t *rkbuf, + struct rd_kafka_metadata **mdp, + rd_kafka_topic_authorized_operations_pair_t **topic_authorized_operations, + int32_t *cluster_authorized_operations, + rd_list_t *request_topics, + char **rk_cluster_id, + int *rk_controller_id) { rd_kafka_t *rk = rkb->rkb_rk; int i, j, k; rd_tmpabuf_t tbuf; struct rd_kafka_metadata *md = NULL; size_t rkb_namelen; - const int log_decode_errors = LOG_ERR; - rd_list_t *missing_topics = NULL; - const rd_list_t *requested_topics = request ? request->rkbuf_u.Metadata.topics - : request_topics; - rd_bool_t all_topics = request ? request->rkbuf_u.Metadata.all_topics : rd_false; - rd_bool_t cgrp_update = request ? - (request->rkbuf_u.Metadata.cgrp_update && rk->rk_cgrp) : rd_false; + const int log_decode_errors = LOG_ERR; + rd_list_t *missing_topics = NULL; + const rd_list_t *requested_topics = + request ? request->rkbuf_u.Metadata.topics : request_topics; + rd_bool_t all_topics = + request ? request->rkbuf_u.Metadata.all_topics : rd_false; + rd_bool_t cgrp_update = + request ? (request->rkbuf_u.Metadata.cgrp_update && rk->rk_cgrp) + : rd_false; const char *reason = request ? (request->rkbuf_u.Metadata.reason - ? request->rkbuf_u.Metadata.reason - : "(no reason)") : "(admin request)"; - /* changed from request->rkbuf_reqhdr.ApiVersion as request buffer may be NULL*/ + ? request->rkbuf_u.Metadata.reason + : "(no reason)") + : "(admin request)"; + /* changed from request->rkbuf_reqhdr.ApiVersion as request buffer may + * be NULL*/ int ApiVersion = rkbuf->rkbuf_reqhdr.ApiVersion; rd_kafkap_str_t cluster_id = RD_ZERO_INIT; int32_t controller_id = -1; @@ -385,11 +391,11 @@ rd_kafka_resp_err_t rd_kafka_parse_Metadata(rd_kafka_broker_t *rkb, rd_kafka_buf_skip_tags(rkbuf); } - if (ApiVersion >= 2){ + if (ApiVersion >= 2) { rd_kafka_buf_read_str(rkbuf, &cluster_id); *rk_cluster_id = RD_KAFKAP_STR_DUP(&cluster_id); } - + if (ApiVersion >= 1) { rd_kafka_buf_read_i32(rkbuf, &controller_id); @@ -411,8 +417,11 @@ rd_kafka_resp_err_t rd_kafka_parse_Metadata(rd_kafka_broker_t *rkb, rd_kafka_buf_parse_fail( rkbuf, "%d topics: tmpabuf memory shortage", md->topic_cnt); - if ((ApiVersion >= 8 && ApiVersion <= 10) && topic_authorized_operations){ - *topic_authorized_operations = rd_malloc(sizeof(rd_kafka_topic_authorized_operations_pair_t) * md->topic_cnt); + if ((ApiVersion >= 8 && ApiVersion <= 10) && + topic_authorized_operations) { + *topic_authorized_operations = rd_malloc( + sizeof(rd_kafka_topic_authorized_operations_pair_t) * + md->topic_cnt); } for (i = 0; i < md->topic_cnt; i++) { @@ -537,8 +546,10 @@ rd_kafka_resp_err_t rd_kafka_parse_Metadata(rd_kafka_broker_t *rkb, /* TopicAuthorizedOperations */ rd_kafka_buf_read_i32(rkbuf, &TopicAuthorizedOperations); - (*topic_authorized_operations)[i].topic_name = md->topics[i].topic; - (*topic_authorized_operations)[i].authorized_operations = TopicAuthorizedOperations; + (*topic_authorized_operations)[i].topic_name = + md->topics[i].topic; + (*topic_authorized_operations)[i] + .authorized_operations = TopicAuthorizedOperations; } rd_kafka_buf_skip_tags(rkbuf); @@ -1064,10 +1075,11 @@ rd_kafka_metadata_refresh_topics(rd_kafka_t *rk, "Requesting metadata for %d/%d topics: %s", rd_list_cnt(&q_topics), rd_list_cnt(topics), reason); - rd_kafka_MetadataRequest(rkb, &q_topics, reason, allow_auto_create, - rd_true /*include cluster authorized operations */, - rd_true /*include topic authorized operations */, - cgrp_update, NULL, NULL, 0, NULL); + rd_kafka_MetadataRequest( + rkb, &q_topics, reason, allow_auto_create, + rd_true /*include cluster authorized operations */, + rd_true /*include topic authorized operations */, cgrp_update, NULL, + NULL, 0, NULL); rd_list_destroy(&q_topics); @@ -1207,12 +1219,12 @@ rd_kafka_metadata_refresh_consumer_topics(rd_kafka_t *rk, rd_kafka_resp_err_t rd_kafka_metadata_refresh_brokers(rd_kafka_t *rk, rd_kafka_broker_t *rkb, const char *reason) { - return rd_kafka_metadata_request(rk, rkb, NULL /*brokers only*/, - rd_false /*!allow auto create topics*/, - rd_true /*include cluster authorized operations */, - rd_true /*include topic authorized operations */, - rd_false /*no cgrp update */, reason, - NULL); + return rd_kafka_metadata_request( + rk, rkb, NULL /*brokers only*/, + rd_false /*!allow auto create topics*/, + rd_true /*include cluster authorized operations */, + rd_true /*include topic authorized operations */, + rd_false /*no cgrp update */, reason, NULL); } @@ -1244,11 +1256,11 @@ rd_kafka_resp_err_t rd_kafka_metadata_refresh_all(rd_kafka_t *rk, } rd_list_init(&topics, 0, NULL); /* empty list = all topics */ - rd_kafka_MetadataRequest(rkb, &topics, reason, - rd_false /*no auto create*/, - rd_true /*include cluster authorized operations */, - rd_true /*include topic authorized operations */, - rd_true /*cgrp update*/, NULL, NULL, 0, NULL); + rd_kafka_MetadataRequest( + rkb, &topics, reason, rd_false /*no auto create*/, + rd_true /*include cluster authorized operations */, + rd_true /*include topic authorized operations */, + rd_true /*cgrp update*/, NULL, NULL, 0, NULL); rd_list_destroy(&topics); if (destroy_rkb) @@ -1287,9 +1299,9 @@ rd_kafka_metadata_request(rd_kafka_t *rk, destroy_rkb = 1; } - rd_kafka_MetadataRequest(rkb, topics, reason, allow_auto_create_topics, - include_cluster_authorized_operations, - include_topic_authorized_operations, + rd_kafka_MetadataRequest(rkb, topics, reason, allow_auto_create_topics, + include_cluster_authorized_operations, + include_topic_authorized_operations, cgrp_update, rko, NULL, 0, NULL); if (destroy_rkb) diff --git a/src/rdkafka_metadata.h b/src/rdkafka_metadata.h index d4cd72954c..9344e470b1 100644 --- a/src/rdkafka_metadata.h +++ b/src/rdkafka_metadata.h @@ -33,15 +33,16 @@ 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, - struct rd_kafka_metadata **mdp, - rd_kafka_topic_authorized_operations_pair_t** topic_authorized_operations, - int32_t* cluster_authorized_operations, - rd_list_t* request_topics, - char** rk_cluster_id, - int* rk_controller_id); +rd_kafka_resp_err_t rd_kafka_parse_Metadata( + rd_kafka_broker_t *rkb, + rd_kafka_buf_t *request, + rd_kafka_buf_t *rkbuf, + struct rd_kafka_metadata **mdp, + rd_kafka_topic_authorized_operations_pair_t **topic_authorized_operations, + int32_t *cluster_authorized_operations, + rd_list_t *request_topics, + char **rk_cluster_id, + int *rk_controller_id); struct rd_kafka_metadata * rd_kafka_metadata_copy(const struct rd_kafka_metadata *md, size_t size); @@ -114,9 +115,9 @@ rd_kafka_metadata_t *rd_kafka_metadata_new_topic_mockv(size_t topic_cnt, ...); */ typedef struct rd_kafka_topic_authorized_operations_pair { - char* topic_name; + char *topic_name; int32_t authorized_operations; -}rd_kafka_topic_authorized_operations_pair; +} rd_kafka_topic_authorized_operations_pair; /** * @{ * diff --git a/src/rdkafka_op.c b/src/rdkafka_op.c index f4ae046d3d..b1f8774089 100644 --- a/src/rdkafka_op.c +++ b/src/rdkafka_op.c @@ -84,9 +84,9 @@ const char *rd_kafka_op2str(rd_kafka_op_type_t type) { [RD_KAFKA_OP_LISTCONSUMERGROUPS] = "REPLY:LISTCONSUMERGROUPS", [RD_KAFKA_OP_DESCRIBECONSUMERGROUPS] = "REPLY:DESCRIBECONSUMERGROUPS", - [RD_KAFKA_OP_DESCRIBETOPICS] = "REPLY:DESCRIBETOPICS", + [RD_KAFKA_OP_DESCRIBETOPICS] = "REPLY:DESCRIBETOPICS", [RD_KAFKA_OP_DESCRIBECLUSTER] = "REPLY:DESCRIBECLUSTER", - [RD_KAFKA_OP_DELETEGROUPS] = "REPLY:DELETEGROUPS", + [RD_KAFKA_OP_DELETEGROUPS] = "REPLY:DELETEGROUPS", [RD_KAFKA_OP_DELETECONSUMERGROUPOFFSETS] = "REPLY:DELETECONSUMERGROUPOFFSETS", [RD_KAFKA_OP_CREATEACLS] = "REPLY:CREATEACLS", @@ -235,9 +235,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_DESCRIBETOPICS] = 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_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), diff --git a/src/rdkafka_op.h b/src/rdkafka_op.h index 92fd5295fc..52153ed7af 100644 --- a/src/rdkafka_op.h +++ b/src/rdkafka_op.h @@ -38,7 +38,8 @@ typedef struct rd_kafka_q_s rd_kafka_q_t; typedef struct rd_kafka_toppar_s rd_kafka_toppar_t; typedef struct rd_kafka_op_s rd_kafka_op_t; -typedef struct rd_kafka_topic_authorized_operations_pair rd_kafka_topic_authorized_operations_pair_t; +typedef struct rd_kafka_topic_authorized_operations_pair + rd_kafka_topic_authorized_operations_pair_t; /* One-off reply queue + reply version. * All APIs that take a rd_kafka_replyq_t makes a copy of the @@ -138,10 +139,10 @@ typedef enum { RD_KAFKA_OP_DESCRIBECONSUMERGROUPS, /**< Admin: * DescribeConsumerGroups * u.admin_request */ - RD_KAFKA_OP_DESCRIBETOPICS, /**< Admin: + RD_KAFKA_OP_DESCRIBETOPICS, /**< Admin: * DescribeTopics * u.admin_request */ - RD_KAFKA_OP_DESCRIBECLUSTER, /**< Admin: + RD_KAFKA_OP_DESCRIBECLUSTER, /**< Admin: * DescribeCluster * u.admin_request */ RD_KAFKA_OP_DELETEGROUPS, /**< Admin: DeleteGroups: u.admin_request*/ @@ -377,7 +378,8 @@ struct rd_kafka_op_s { /* RD_KAFKA_OP_METADATA */ struct { rd_kafka_metadata_t *md; - rd_kafka_topic_authorized_operations_pair_t* topic_authorized_operations; + rd_kafka_topic_authorized_operations_pair_t + *topic_authorized_operations; int32_t cluster_authorized_operations; int force; /* force request regardless of outstanding * metadata requests. */ @@ -448,13 +450,14 @@ struct rd_kafka_op_s { struct rd_kafka_admin_worker_cbs *cbs; /** Worker state */ - enum { RD_KAFKA_ADMIN_STATE_INIT, - RD_KAFKA_ADMIN_STATE_WAIT_BROKER, - RD_KAFKA_ADMIN_STATE_WAIT_CONTROLLER, - RD_KAFKA_ADMIN_STATE_WAIT_FANOUTS, - RD_KAFKA_ADMIN_STATE_CONSTRUCT_REQUEST, - RD_KAFKA_ADMIN_STATE_WAIT_RESPONSE, - RD_KAFKA_ADMIN_STATE_WAIT_BROKER_LIST, + enum { + RD_KAFKA_ADMIN_STATE_INIT, + RD_KAFKA_ADMIN_STATE_WAIT_BROKER, + RD_KAFKA_ADMIN_STATE_WAIT_CONTROLLER, + RD_KAFKA_ADMIN_STATE_WAIT_FANOUTS, + RD_KAFKA_ADMIN_STATE_CONSTRUCT_REQUEST, + RD_KAFKA_ADMIN_STATE_WAIT_RESPONSE, + RD_KAFKA_ADMIN_STATE_WAIT_BROKER_LIST, } state; int32_t broker_id; /**< Requested broker id to @@ -544,16 +547,17 @@ struct rd_kafka_op_s { /**< Mock cluster command */ struct { - enum { RD_KAFKA_MOCK_CMD_TOPIC_SET_ERROR, - RD_KAFKA_MOCK_CMD_TOPIC_CREATE, - RD_KAFKA_MOCK_CMD_PART_SET_LEADER, - RD_KAFKA_MOCK_CMD_PART_SET_FOLLOWER, - RD_KAFKA_MOCK_CMD_PART_SET_FOLLOWER_WMARKS, - RD_KAFKA_MOCK_CMD_BROKER_SET_UPDOWN, - RD_KAFKA_MOCK_CMD_BROKER_SET_RTT, - RD_KAFKA_MOCK_CMD_BROKER_SET_RACK, - RD_KAFKA_MOCK_CMD_COORD_SET, - RD_KAFKA_MOCK_CMD_APIVERSION_SET, + enum { + RD_KAFKA_MOCK_CMD_TOPIC_SET_ERROR, + RD_KAFKA_MOCK_CMD_TOPIC_CREATE, + RD_KAFKA_MOCK_CMD_PART_SET_LEADER, + RD_KAFKA_MOCK_CMD_PART_SET_FOLLOWER, + RD_KAFKA_MOCK_CMD_PART_SET_FOLLOWER_WMARKS, + RD_KAFKA_MOCK_CMD_BROKER_SET_UPDOWN, + RD_KAFKA_MOCK_CMD_BROKER_SET_RTT, + RD_KAFKA_MOCK_CMD_BROKER_SET_RACK, + RD_KAFKA_MOCK_CMD_COORD_SET, + RD_KAFKA_MOCK_CMD_APIVERSION_SET, } cmd; rd_kafka_resp_err_t err; /**< Error for: diff --git a/src/rdkafka_partition.h b/src/rdkafka_partition.h index a1f1f47cd9..77a56f8049 100644 --- a/src/rdkafka_partition.h +++ b/src/rdkafka_partition.h @@ -285,13 +285,14 @@ struct rd_kafka_toppar_s { /* rd_kafka_toppar_t */ int32_t rktp_fetch_version; /* Op version of curr fetch. (broker thread) */ - enum { RD_KAFKA_TOPPAR_FETCH_NONE = 0, - RD_KAFKA_TOPPAR_FETCH_STOPPING, - RD_KAFKA_TOPPAR_FETCH_STOPPED, - RD_KAFKA_TOPPAR_FETCH_OFFSET_QUERY, - RD_KAFKA_TOPPAR_FETCH_OFFSET_WAIT, - RD_KAFKA_TOPPAR_FETCH_VALIDATE_EPOCH_WAIT, - RD_KAFKA_TOPPAR_FETCH_ACTIVE, + enum { + RD_KAFKA_TOPPAR_FETCH_NONE = 0, + RD_KAFKA_TOPPAR_FETCH_STOPPING, + RD_KAFKA_TOPPAR_FETCH_STOPPED, + RD_KAFKA_TOPPAR_FETCH_OFFSET_QUERY, + RD_KAFKA_TOPPAR_FETCH_OFFSET_WAIT, + RD_KAFKA_TOPPAR_FETCH_VALIDATE_EPOCH_WAIT, + RD_KAFKA_TOPPAR_FETCH_ACTIVE, } rktp_fetch_state; /* Broker thread's state */ #define RD_KAFKA_TOPPAR_FETCH_IS_STARTED(fetch_state) \ diff --git a/src/rdkafka_request.c b/src/rdkafka_request.c index 3b16004458..64abded1ec 100644 --- a/src/rdkafka_request.c +++ b/src/rdkafka_request.c @@ -2089,11 +2089,12 @@ static void rd_kafka_handle_Metadata(rd_kafka_t *rk, void *opaque) { rd_kafka_op_t *rko = opaque; /* Possibly NULL */ struct rd_kafka_metadata *md = NULL; - rd_kafka_topic_authorized_operations_pair_t* topic_authorized_operations = NULL; + rd_kafka_topic_authorized_operations_pair_t + *topic_authorized_operations = NULL; int32_t cluster_authorized_operations; - char* cluster_id = NULL; + char *cluster_id = NULL; int controller_id; - const rd_list_t *topics = request->rkbuf_u.Metadata.topics; + const rd_list_t *topics = request->rkbuf_u.Metadata.topics; int actions; rd_kafka_assert(NULL, err == RD_KAFKA_RESP_ERR__DESTROY || @@ -2120,8 +2121,9 @@ static void rd_kafka_handle_Metadata(rd_kafka_t *rk, rd_list_cnt(topics), request->rkbuf_u.Metadata.reason); - err = rd_kafka_parse_Metadata(rkb, request, rkbuf, &md, &topic_authorized_operations,&cluster_authorized_operations, NULL, - &cluster_id, &controller_id); + err = rd_kafka_parse_Metadata( + rkb, request, rkbuf, &md, &topic_authorized_operations, + &cluster_authorized_operations, NULL, &cluster_id, &controller_id); if (err) goto err; @@ -2130,8 +2132,10 @@ static void rd_kafka_handle_Metadata(rd_kafka_t *rk, * Reuse requesting rko for the reply. */ rko->rko_err = err; rko->rko_u.metadata.md = md; - rko->rko_u.metadata.cluster_authorized_operations = cluster_authorized_operations; - rko->rko_u.metadata.topic_authorized_operations = topic_authorized_operations; + rko->rko_u.metadata.cluster_authorized_operations = + cluster_authorized_operations; + rko->rko_u.metadata.topic_authorized_operations = + topic_authorized_operations; rd_kafka_replyq_enq(&rko->rko_replyq, rko, 0); rko = NULL; } else { @@ -2162,8 +2166,8 @@ static void rd_kafka_handle_Metadata(rd_kafka_t *rk, rd_kafka_actions2str(actions)); /* Respond back to caller on non-retriable errors */ if (rko && rko->rko_replyq.q) { - rko->rko_err = err; - rko->rko_u.metadata.md = NULL; + rko->rko_err = err; + rko->rko_u.metadata.md = NULL; rko->rko_u.metadata.topic_authorized_operations = NULL; rko->rko_u.metadata.cluster_authorized_operations = -1; rd_kafka_replyq_enq(&rko->rko_replyq, rko, 0); @@ -2205,7 +2209,7 @@ static void rd_kafka_handle_Metadata(rd_kafka_t *rk, * there is already a matching one in-transit. * @param resp_cb - callback to be used for handling response. If NULL, then * rd_kafka_handle_Metadata() is used. - * @param force - 1: force a full request. + * @param force - 1: force a full request. * 0: check if there are multiple outstanding full requests. * If full metadata for all topics is requested (or all brokers, which * results in all-topics on older brokers) and there is already a full request @@ -2213,17 +2217,18 @@ static void rd_kafka_handle_Metadata(rd_kafka_t *rk, * 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 include_cluster_authorized_operations, - rd_bool_t include_topic_authorized_operations, - rd_bool_t cgrp_update, - rd_kafka_op_t *rko, - rd_kafka_resp_cb_t *resp_cb, - int force, - void *opaque) { +rd_kafka_resp_err_t +rd_kafka_MetadataRequest(rd_kafka_broker_t *rkb, + const rd_list_t *topics, + const char *reason, + rd_bool_t allow_auto_create_topics, + rd_bool_t include_cluster_authorized_operations, + rd_bool_t include_topic_authorized_operations, + rd_bool_t cgrp_update, + rd_kafka_op_t *rko, + rd_kafka_resp_cb_t *resp_cb, + int force, + void *opaque) { rd_kafka_buf_t *rkbuf; int16_t ApiVersion = 0; int i; @@ -2298,7 +2303,8 @@ rd_kafka_resp_err_t rd_kafka_MetadataRequest(rd_kafka_broker_t *rkb, * through regardless. */ mtx_lock(&rkb->rkb_rk->rk_metadata_cache.rkmc_full_lock); - if(!force && (*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", @@ -2356,13 +2362,15 @@ rd_kafka_resp_err_t rd_kafka_MetadataRequest(rd_kafka_broker_t *rkb, /* TODO: implement KIP-430 */ /* IncludeClusterAuthorizedOperations */ - rd_kafka_buf_write_bool(rkbuf, include_cluster_authorized_operations); + rd_kafka_buf_write_bool(rkbuf, + include_cluster_authorized_operations); } if (ApiVersion >= 8) { /* TODO: implement KIP-430 */ /* IncludeTopicAuthorizedOperations */ - rd_kafka_buf_write_bool(rkbuf, include_topic_authorized_operations); + rd_kafka_buf_write_bool(rkbuf, + include_topic_authorized_operations); } rd_kafka_buf_ApiVersion_set(rkbuf, ApiVersion, 0); @@ -2371,13 +2379,13 @@ rd_kafka_resp_err_t rd_kafka_MetadataRequest(rd_kafka_broker_t *rkb, * and should go before most other requests (Produce, Fetch, etc). */ rkbuf->rkbuf_prio = RD_KAFKA_PRIO_HIGH; - rd_kafka_broker_buf_enq_replyq(rkb, rkbuf, - /* Handle response thru rk_ops, - * but forward parsed result to - * rko's replyq when done. */ - RD_KAFKA_REPLYQ(rkb->rkb_rk->rk_ops, 0), - resp_cb ? resp_cb : rd_kafka_handle_Metadata, - rko ? rko : opaque); + rd_kafka_broker_buf_enq_replyq( + rkb, rkbuf, + /* Handle response thru rk_ops, + * but forward parsed result to + * rko's replyq when done. */ + RD_KAFKA_REPLYQ(rkb->rkb_rk->rk_ops, 0), + resp_cb ? resp_cb : rd_kafka_handle_Metadata, rko ? rko : opaque); return RD_KAFKA_RESP_ERR_NO_ERROR; } diff --git a/src/rdkafka_request.h b/src/rdkafka_request.h index 7db5a7bc58..5398a8f121 100644 --- a/src/rdkafka_request.h +++ b/src/rdkafka_request.h @@ -255,17 +255,18 @@ void rd_kafka_HeartbeatRequest(rd_kafka_broker_t *rkb, rd_kafka_resp_cb_t *resp_cb, void *opaque); -rd_kafka_resp_err_t rd_kafka_MetadataRequest(rd_kafka_broker_t *rkb, - const rd_list_t *topics, - const char *reason, - rd_bool_t allow_auto_create_topics, - rd_bool_t include_cluster_authorized_operations, - rd_bool_t include_topic_authorized_operations, - rd_bool_t cgrp_update, - rd_kafka_op_t *rko, - rd_kafka_resp_cb_t *resp_cb, - int force, - void *opaque); +rd_kafka_resp_err_t +rd_kafka_MetadataRequest(rd_kafka_broker_t *rkb, + const rd_list_t *topics, + const char *reason, + rd_bool_t allow_auto_create_topics, + rd_bool_t include_cluster_authorized_operations, + rd_bool_t include_topic_authorized_operations, + rd_bool_t cgrp_update, + rd_kafka_op_t *rko, + rd_kafka_resp_cb_t *resp_cb, + int force, + void *opaque); rd_kafka_resp_err_t rd_kafka_handle_ApiVersion(rd_kafka_t *rk, diff --git a/src/rdkafka_topic.h b/src/rdkafka_topic.h index cbed9308a7..0df289b287 100644 --- a/src/rdkafka_topic.h +++ b/src/rdkafka_topic.h @@ -147,11 +147,12 @@ struct rd_kafka_topic_s { rd_refcnt_t rkt_app_refcnt; /**< Number of active rkt's new()ed * by application. */ - enum { RD_KAFKA_TOPIC_S_UNKNOWN, /* No cluster information yet */ - RD_KAFKA_TOPIC_S_EXISTS, /* Topic exists in cluster */ - RD_KAFKA_TOPIC_S_NOTEXISTS, /* Topic is not known in cluster */ - RD_KAFKA_TOPIC_S_ERROR, /* Topic exists but is in an errored - * state, such as auth failure. */ + enum { + RD_KAFKA_TOPIC_S_UNKNOWN, /* No cluster information yet */ + RD_KAFKA_TOPIC_S_EXISTS, /* Topic exists in cluster */ + RD_KAFKA_TOPIC_S_NOTEXISTS, /* Topic is not known in cluster */ + RD_KAFKA_TOPIC_S_ERROR, /* Topic exists but is in an errored + * state, such as auth failure. */ } rkt_state; int rkt_flags; diff --git a/tests/0080-admin_ut.c b/tests/0080-admin_ut.c index fc59f269a1..5b6eb1936a 100644 --- a/tests/0080-admin_ut.c +++ b/tests/0080-admin_ut.c @@ -658,14 +658,16 @@ 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))) { + if ((error = + rd_kafka_AdminOptions_set_include_authorized_operations( + options, 0))) { fprintf(stderr, - "%% Failed to set require authorized operations: %s\n", + "%% 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"); + TEST_FAIL( + "Failed to set include authorized operations\n"); } if (useq) { @@ -735,9 +737,10 @@ static void do_test_DescribeConsumerGroups(const char *what, rd_kafka_error_string( rd_kafka_ConsumerGroupDescription_error(resgroups[i]))); TEST_ASSERT( - rd_kafka_ConsumerGroupDescription_authorized_operations_count( - resgroups[i]) == 0, "Got authorized operations" - "when not requested"); + rd_kafka_ConsumerGroupDescription_authorized_operations_count( + resgroups[i]) == 0, + "Got authorized operations" + "when not requested"); } rd_kafka_event_destroy(rkev); @@ -764,10 +767,10 @@ static void do_test_DescribeConsumerGroups(const char *what, * */ static void do_test_DescribeTopics(const char *what, - rd_kafka_t *rk, - rd_kafka_queue_t *useq, - int with_options, - rd_bool_t destroy) { + rd_kafka_t *rk, + rd_kafka_queue_t *useq, + int with_options, + rd_bool_t destroy) { rd_kafka_queue_t *q; #define TEST_DESCRIBE_TOPICS_CNT 4 const char *topic_names[TEST_DESCRIBE_TOPICS_CNT]; @@ -802,14 +805,16 @@ static void do_test_DescribeTopics(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_topic_authorized_operations( - options, 0))) { + if ((error = + rd_kafka_AdminOptions_set_include_topic_authorized_operations( + options, 0))) { fprintf(stderr, - "%% Failed to set topic authorized operations: %s\n", + "%% 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"); + TEST_FAIL( + "Failed to set topic authorized operations\n"); } if (useq) { @@ -820,8 +825,8 @@ static void do_test_DescribeTopics(const char *what, TIMING_START(&timing, "DescribeTopics"); TEST_SAY("Call DescribeTopics, timeout is %dms\n", exp_timeout); - rd_kafka_DescribeTopics( - rk, topic_names, TEST_DESCRIBE_TOPICS_CNT, options, q); + rd_kafka_DescribeTopics(rk, topic_names, TEST_DESCRIBE_TOPICS_CNT, + options, q); TIMING_ASSERT_LATER(&timing, 0, 50); if (destroy) @@ -832,8 +837,8 @@ static void do_test_DescribeTopics(const char *what, 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); + 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); @@ -847,17 +852,14 @@ static void do_test_DescribeTopics(const char *what, /* Expecting no error (errors will be per-topic) */ err = rd_kafka_event_error(rkev); errstr2 = rd_kafka_event_error_string(rkev); - TEST_ASSERT( - err == RD_KAFKA_RESP_ERR_NO_ERROR, - "expected DescribeTopics to return error %s, not %s (%s)", - rd_kafka_err2str(RD_KAFKA_RESP_ERR_NO_ERROR), rd_kafka_err2str(err), - err ? errstr2 : "n/a"); + TEST_ASSERT(err == RD_KAFKA_RESP_ERR_NO_ERROR, + "expected DescribeTopics to return error %s, not %s (%s)", + rd_kafka_err2str(RD_KAFKA_RESP_ERR_NO_ERROR), + rd_kafka_err2str(err), err ? errstr2 : "n/a"); /* Extract topics, should return TEST_DESCRIBE_TOPICS_CNT topics. */ - restopics = - rd_kafka_DescribeTopics_result_topics(res, &restopic_cnt); - TEST_ASSERT(restopics && - restopic_cnt == TEST_DESCRIBE_TOPICS_CNT, + restopics = rd_kafka_DescribeTopics_result_topics(res, &restopic_cnt); + TEST_ASSERT(restopics && restopic_cnt == TEST_DESCRIBE_TOPICS_CNT, "expected %d result_topics, got %p cnt %" PRIusz, TEST_DESCRIBE_TOPICS_CNT, restopics, restopic_cnt); @@ -866,22 +868,21 @@ static void do_test_DescribeTopics(const char *what, for (i = 0; i < TEST_DESCRIBE_TOPICS_CNT; i++) { TEST_ASSERT( !strcmp(topic_names[i], - rd_kafka_TopicDescription_topic_name( - restopics[i])), + rd_kafka_TopicDescription_topic_name(restopics[i])), "expected topic '%s' at position %d, not '%s'", topic_names[i], i, rd_kafka_TopicDescription_topic_name(restopics[i])); + TEST_ASSERT(rd_kafka_error_code(rd_kafka_TopicDescription_error( + restopics[i])) == RD_KAFKA_RESP_ERR__TIMED_OUT, + "expected topic '%s' to have timed out, got %s", + topic_names[i], + rd_kafka_error_string( + rd_kafka_TopicDescription_error(restopics[i]))); TEST_ASSERT( - rd_kafka_error_code(rd_kafka_TopicDescription_error( - restopics[i])) == RD_KAFKA_RESP_ERR__TIMED_OUT, - "expected topic '%s' to have timed out, got %s", - topic_names[i], - rd_kafka_error_string( - rd_kafka_TopicDescription_error(restopics[i]))); - TEST_ASSERT( - rd_kafka_TopicDescription_topic_authorized_operations_cnt( - restopics[i]) == 0, "Got topic authorized operations" - "when not requested"); + rd_kafka_TopicDescription_topic_authorized_operations_cnt( + restopics[i]) == 0, + "Got topic authorized operations" + "when not requested"); } rd_kafka_event_destroy(rkev); @@ -908,10 +909,10 @@ static void do_test_DescribeTopics(const char *what, * */ static void do_test_DescribeCluster(const char *what, - rd_kafka_t *rk, - rd_kafka_queue_t *useq, - int with_options, - rd_bool_t destroy) { + rd_kafka_t *rk, + rd_kafka_queue_t *useq, + int with_options, + rd_bool_t destroy) { rd_kafka_queue_t *q; rd_kafka_AdminOptions_t *options = NULL; int exp_timeout = MY_SOCKET_TIMEOUT_MS; @@ -939,14 +940,16 @@ static void do_test_DescribeCluster(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_cluster_authorized_operations( - options, 0))) { + if ((error = + rd_kafka_AdminOptions_set_include_cluster_authorized_operations( + options, 0))) { fprintf(stderr, - "%% Failed to set cluster authorized operations: %s\n", + "%% 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"); + TEST_FAIL( + "Failed to set cluster authorized operations\n"); } if (useq) { @@ -984,9 +987,9 @@ static void do_test_DescribeCluster(const char *what, 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"); + "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); @@ -2702,19 +2705,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, - rd_false); - do_test_DescribeTopics("temp queue, options", rk, NULL, 1, - rd_false); - do_test_DescribeTopics("main queue, options", rk, mainq, 1, - rd_false); + do_test_DescribeTopics("temp queue, no options", rk, NULL, 0, rd_false); + do_test_DescribeTopics("temp queue, options", rk, NULL, 1, rd_false); + do_test_DescribeTopics("main queue, options", rk, mainq, 1, rd_false); do_test_DescribeCluster("temp queue, no options", rk, NULL, 0, - rd_false); - do_test_DescribeCluster("temp queue, options", rk, NULL, 1, - rd_false); - do_test_DescribeCluster("main queue, options", rk, mainq, 1, - rd_false); + rd_false); + do_test_DescribeCluster("temp queue, options", rk, NULL, 1, rd_false); + do_test_DescribeCluster("main queue, options", rk, mainq, 1, rd_false); do_test_DeleteGroups("temp queue, no options", rk, NULL, 0, rd_false); do_test_DeleteGroups("temp queue, options", rk, NULL, 1, rd_false); diff --git a/tests/0081-admin.c b/tests/0081-admin.c index 1888133bc0..96ab22be54 100644 --- a/tests/0081-admin.c +++ b/tests/0081-admin.c @@ -2740,7 +2740,8 @@ static void do_test_DescribeConsumerGroups(const char *what, rd_kafka_consumer_group_state_t state = rd_kafka_ConsumerGroupDescription_state(act); TEST_ASSERT( - rd_kafka_ConsumerGroupDescription_authorized_operations_count(act) == 0, + rd_kafka_ConsumerGroupDescription_authorized_operations_count( + act) == 0, "Authorized operations returned when not requested\n"); TEST_ASSERT( strcmp(exp->group_id, @@ -2867,8 +2868,10 @@ static void do_test_DescribeConsumerGroups(const char *what, /** * @brief Test DescribeTopics */ -static void do_test_DescribeTopics(const char *what, rd_kafka_t *rk, rd_kafka_queue_t *rkqu, - int request_timeout) { +static void do_test_DescribeTopics(const char *what, + rd_kafka_t *rk, + rd_kafka_queue_t *rkqu, + int request_timeout) { #define MY_TOPICS_CNT 3 rd_kafka_queue_t *q; char *topics[MY_TOPICS_CNT]; @@ -2899,33 +2902,35 @@ static void do_test_DescribeTopics(const char *what, rd_kafka_t *rk, rd_kafka_qu /* * Only create one topic, the others will be non-existent. */ - for (i = 0; i < MY_TOPICS_CNT; i++){ - rd_strdupa(&topics[i], test_mk_topic_name(__FUNCTION__, 1)); + for (i = 0; i < MY_TOPICS_CNT; i++) { + rd_strdupa(&topics[i], test_mk_topic_name(__FUNCTION__, 1)); } test_CreateTopics_simple(rk, NULL, topics, 1, 1, NULL); test_wait_topic_exists(rk, topics[0], 10000); - + /* * Timeout options */ - options = rd_kafka_AdminOptions_new(rk, RD_KAFKA_ADMIN_OP_DESCRIBETOPICS); + options = + rd_kafka_AdminOptions_new(rk, RD_KAFKA_ADMIN_OP_DESCRIBETOPICS); - err = rd_kafka_AdminOptions_set_request_timeout(options, request_timeout, errstr, - sizeof(errstr)); + err = rd_kafka_AdminOptions_set_request_timeout( + options, request_timeout, errstr, sizeof(errstr)); TEST_ASSERT(!err, "%s", errstr); - if ((error = rd_kafka_AdminOptions_set_include_topic_authorized_operations( - options, 1))) { - fprintf(stderr, - "%% Failed to set require authorized operations: %s\n", - rd_kafka_error_string(error)); - rd_kafka_error_destroy(error); - TEST_FAIL("Failed to set include authorized operations\n"); + if ((error = + rd_kafka_AdminOptions_set_include_topic_authorized_operations( + options, 1))) { + fprintf(stderr, + "%% Failed to set require authorized operations: %s\n", + rd_kafka_error_string(error)); + rd_kafka_error_destroy(error); + TEST_FAIL("Failed to set include authorized operations\n"); } TIMING_START(&timing, "DescribeTopics"); TEST_SAY("Call DescribeTopics\n"); - rd_kafka_DescribeTopics(rk, topics, MY_TOPICS_CNT, options, q); + rd_kafka_DescribeTopics(rk, topics, MY_TOPICS_CNT, options, q); TIMING_ASSERT_LATER(&timing, 0, 50); TIMING_START(&timing, "DescribeTopics.queue_poll"); @@ -2963,45 +2968,48 @@ static void do_test_DescribeTopics(const char *what, rd_kafka_t *rk, rd_kafka_qu TEST_ASSERT(!err, "Expected success, not %s: %s", rd_kafka_err2name(err), errstr2); - result_topics = rd_kafka_DescribeTopics_result_topics( - res, &result_topics_cnt); + result_topics = + rd_kafka_DescribeTopics_result_topics(res, &result_topics_cnt); /* * Check if results have been received for all topics */ - TEST_ASSERT(result_topics_cnt == MY_TOPICS_CNT, "Number of topics in result didnt match"); + TEST_ASSERT(result_topics_cnt == MY_TOPICS_CNT, + "Number of topics in result didnt match"); /* * Check if topics[0] succeeded. */ - TEST_ASSERT(rd_kafka_error_code( - rd_kafka_TopicDescription_error(result_topics[0])) == - RD_KAFKA_RESP_ERR_NO_ERROR, - "Expected expected unknown Topic or partition, not %s\n", - rd_kafka_error_string(rd_kafka_TopicDescription_error(result_topics[0]))); + TEST_ASSERT(rd_kafka_error_code(rd_kafka_TopicDescription_error( + result_topics[0])) == RD_KAFKA_RESP_ERR_NO_ERROR, + "Expected expected unknown Topic or partition, not %s\n", + rd_kafka_error_string( + rd_kafka_TopicDescription_error(result_topics[0]))); /* - * Check whether the topics which are non-existent have + * Check whether the topics which are non-existent have * RD_KAFKA_RESP_ERR_UNKNOWN_TOPIC_OR_PART error. */ - for(i=1; i 0, - "Expected 8 acl operations allowed"); + TEST_ASSERT(initial_acl_cnt > 0, "Expected 8 acl operations allowed"); rd_kafka_event_destroy(rkev); acl_bindings[0] = rd_kafka_AclBinding_new( - RD_KAFKA_RESOURCE_TOPIC, topics[0], RD_KAFKA_RESOURCE_PATTERN_LITERAL, - user_test1, "*", RD_KAFKA_ACL_OPERATION_READ, - RD_KAFKA_ACL_PERMISSION_TYPE_ALLOW, NULL, 0); + RD_KAFKA_RESOURCE_TOPIC, topics[0], + RD_KAFKA_RESOURCE_PATTERN_LITERAL, user_test1, "*", + RD_KAFKA_ACL_OPERATION_READ, RD_KAFKA_ACL_PERMISSION_TYPE_ALLOW, + NULL, 0); admin_options = rd_kafka_AdminOptions_new(rk, RD_KAFKA_ADMIN_OP_CREATEACLS); err = rd_kafka_AdminOptions_set_request_timeout(admin_options, 10000, @@ -3010,7 +3018,7 @@ static void do_test_DescribeTopics(const char *what, rd_kafka_t *rk, rd_kafka_qu rd_kafka_CreateAcls(rk, acl_bindings, 1, admin_options, q); rkev_acl_create = test_wait_admin_result( q, RD_KAFKA_EVENT_CREATEACLS_RESULT, 10000 + 1000); - acl_res = rd_kafka_event_CreateAcls_result(rkev_acl_create); + acl_res = rd_kafka_event_CreateAcls_result(rkev_acl_create); acl_res_acls = rd_kafka_CreateAcls_result_acls(acl_res, &resacl_cnt); for (i = 0; i < resacl_cnt; i++) { const rd_kafka_acl_result_t *acl_res_acl = *(acl_res_acls + i); @@ -3027,23 +3035,25 @@ static void do_test_DescribeTopics(const char *what, rd_kafka_t *rk, rd_kafka_qu /* * Timeout options */ - options = rd_kafka_AdminOptions_new(rk, RD_KAFKA_ADMIN_OP_DESCRIBETOPICS); + options = + rd_kafka_AdminOptions_new(rk, RD_KAFKA_ADMIN_OP_DESCRIBETOPICS); - err = rd_kafka_AdminOptions_set_request_timeout(options, request_timeout, errstr, - sizeof(errstr)); + err = rd_kafka_AdminOptions_set_request_timeout( + options, request_timeout, errstr, sizeof(errstr)); TEST_ASSERT(!err, "%s", errstr); - if ((error = rd_kafka_AdminOptions_set_include_topic_authorized_operations( - options, 1))) { - fprintf(stderr, - "%% Failed to set require authorized operations: %s\n", - rd_kafka_error_string(error)); - rd_kafka_error_destroy(error); - TEST_FAIL("Failed to set include authorized operations\n"); + if ((error = + rd_kafka_AdminOptions_set_include_topic_authorized_operations( + options, 1))) { + fprintf(stderr, + "%% Failed to set require authorized operations: %s\n", + rd_kafka_error_string(error)); + rd_kafka_error_destroy(error); + TEST_FAIL("Failed to set include authorized operations\n"); } TIMING_START(&timing, "DescribeTopics"); TEST_SAY("Call DescribeTopics\n"); - rd_kafka_DescribeTopics(rk, topics, MY_TOPICS_CNT, options, q); + rd_kafka_DescribeTopics(rk, topics, MY_TOPICS_CNT, options, q); TIMING_ASSERT_LATER(&timing, 0, 50); TIMING_START(&timing, "DescribeTopics.queue_poll"); @@ -3081,33 +3091,35 @@ static void do_test_DescribeTopics(const char *what, rd_kafka_t *rk, rd_kafka_qu TEST_ASSERT(!err, "Expected success, not %s: %s", rd_kafka_err2name(err), errstr2); - result_topics = rd_kafka_DescribeTopics_result_topics( - res, &result_topics_cnt); + result_topics = + rd_kafka_DescribeTopics_result_topics(res, &result_topics_cnt); /* * Check if results have been received for all topics */ - TEST_ASSERT(result_topics_cnt == MY_TOPICS_CNT, "Number of topics in result didnt match"); + TEST_ASSERT(result_topics_cnt == MY_TOPICS_CNT, + "Number of topics in result didnt match"); /* * Check if topics[0] succeeded. */ - TEST_ASSERT(rd_kafka_error_code( - rd_kafka_TopicDescription_error(result_topics[0])) == - RD_KAFKA_RESP_ERR_NO_ERROR, - "Expected expected unknown Topic or partition, not %s\n", - rd_kafka_error_string(rd_kafka_TopicDescription_error(result_topics[0]))); - final_acl_cnt = rd_kafka_TopicDescription_topic_authorized_operations_cnt( + TEST_ASSERT(rd_kafka_error_code(rd_kafka_TopicDescription_error( + result_topics[0])) == RD_KAFKA_RESP_ERR_NO_ERROR, + "Expected expected unknown Topic or partition, not %s\n", + rd_kafka_error_string( + rd_kafka_TopicDescription_error(result_topics[0]))); + final_acl_cnt = + rd_kafka_TopicDescription_topic_authorized_operations_cnt( result_topics[0]); /* - * Initally count should be 8. After createAcls call with + * Initally count should be 8. After createAcls call with * only RD_KAFKA_ACL_OPERATION_READ allowed, it should reduce * to 2 (read and describe). 8>2 should hold. */ TEST_ASSERT(initial_acl_cnt > final_acl_cnt, - "Expected the acl operations allowed to have reduced after" - " call to CreateAcls"); + "Expected the acl operations allowed to have reduced after" + " call to CreateAcls"); /* * Allow RD_KAFKA_ACL_OPERATION_DELETE to allow deletion @@ -3115,9 +3127,10 @@ static void do_test_DescribeTopics(const char *what, rd_kafka_t *rk, rd_kafka_qu * describe. */ acl_bindings[0] = rd_kafka_AclBinding_new( - RD_KAFKA_RESOURCE_TOPIC, topics[0], RD_KAFKA_RESOURCE_PATTERN_LITERAL, - user_test1, "*", RD_KAFKA_ACL_OPERATION_DELETE, - RD_KAFKA_ACL_PERMISSION_TYPE_ALLOW, NULL, 0); + RD_KAFKA_RESOURCE_TOPIC, topics[0], + RD_KAFKA_RESOURCE_PATTERN_LITERAL, user_test1, "*", + RD_KAFKA_ACL_OPERATION_DELETE, RD_KAFKA_ACL_PERMISSION_TYPE_ALLOW, + NULL, 0); admin_options = rd_kafka_AdminOptions_new(rk, RD_KAFKA_ADMIN_OP_CREATEACLS); err = rd_kafka_AdminOptions_set_request_timeout(admin_options, 10000, @@ -3126,7 +3139,7 @@ static void do_test_DescribeTopics(const char *what, rd_kafka_t *rk, rd_kafka_qu rd_kafka_CreateAcls(rk, acl_bindings, 1, admin_options, q); rkev_acl_create = test_wait_admin_result( q, RD_KAFKA_EVENT_CREATEACLS_RESULT, 10000 + 1000); - acl_res = rd_kafka_event_CreateAcls_result(rkev_acl_create); + acl_res = rd_kafka_event_CreateAcls_result(rkev_acl_create); acl_res_acls = rd_kafka_CreateAcls_result_acls(acl_res, &resacl_cnt); for (i = 0; i < resacl_cnt; i++) { const rd_kafka_acl_result_t *acl_res_acl = *(acl_res_acls + i); @@ -3142,7 +3155,7 @@ static void do_test_DescribeTopics(const char *what, rd_kafka_t *rk, rd_kafka_qu rd_kafka_event_destroy(rkev); rd_kafka_AclBinding_destroy(acl_bindings[0]); test_DeleteTopics_simple(rk, q, topics, 1, NULL); - + TEST_LATER_CHECK(); #undef MY_TOPICS_CNT @@ -3152,8 +3165,10 @@ static void do_test_DescribeTopics(const char *what, rd_kafka_t *rk, rd_kafka_qu /** * @brief Test DescribeCluster */ -static void do_test_DescribeCluster(const char *what, rd_kafka_t *rk, rd_kafka_queue_t *rkqu, - int request_timeout) { +static void do_test_DescribeCluster(const char *what, + rd_kafka_t *rk, + rd_kafka_queue_t *rkqu, + int request_timeout) { rd_kafka_queue_t *q; rd_kafka_AdminOptions_t *options; rd_kafka_event_t *rkev; @@ -3186,27 +3201,29 @@ static void do_test_DescribeCluster(const char *what, rd_kafka_t *rk, rd_kafka_q rd_kafka_name(rk), what, request_timeout); q = rkqu ? rkqu : rd_kafka_queue_new(rk); - + /* * Timeout options */ - options = rd_kafka_AdminOptions_new(rk, RD_KAFKA_ADMIN_OP_DESCRIBECLUSTER); + options = + rd_kafka_AdminOptions_new(rk, RD_KAFKA_ADMIN_OP_DESCRIBECLUSTER); - err = rd_kafka_AdminOptions_set_request_timeout(options, request_timeout, errstr, - sizeof(errstr)); + err = rd_kafka_AdminOptions_set_request_timeout( + options, request_timeout, errstr, sizeof(errstr)); TEST_ASSERT(!err, "%s", errstr); - if ((error = rd_kafka_AdminOptions_set_include_cluster_authorized_operations( - options, 1))) { - fprintf(stderr, - "%% Failed to set require authorized operations: %s\n", - rd_kafka_error_string(error)); - rd_kafka_error_destroy(error); - TEST_FAIL("Failed to set include authorized operations\n"); + if ((error = + rd_kafka_AdminOptions_set_include_cluster_authorized_operations( + options, 1))) { + fprintf(stderr, + "%% Failed to set require authorized operations: %s\n", + rd_kafka_error_string(error)); + rd_kafka_error_destroy(error); + TEST_FAIL("Failed to set include authorized operations\n"); } TIMING_START(&timing, "DescribeCluster"); TEST_SAY("Call DescribeCluster\n"); - rd_kafka_DescribeCluster(rk, options, q); + rd_kafka_DescribeCluster(rk, options, q); TIMING_ASSERT_LATER(&timing, 0, 50); TIMING_START(&timing, "DescribeCluster.queue_poll"); @@ -3244,21 +3261,21 @@ static void do_test_DescribeCluster(const char *what, rd_kafka_t *rk, rd_kafka_q TEST_ASSERT(!err, "Expected success, not %s: %s", rd_kafka_err2name(err), errstr2); - result_cluster = rd_kafka_DescribeCluster_result_description( - res); - - initial_acl_cnt = rd_kafka_ClusterDescription_cluster_acl_operations_cnt( + result_cluster = rd_kafka_DescribeCluster_result_description(res); + + initial_acl_cnt = + rd_kafka_ClusterDescription_cluster_acl_operations_cnt( result_cluster); - TEST_ASSERT(initial_acl_cnt> 0, - "Expected 7 acl operations allowed"); + TEST_ASSERT(initial_acl_cnt > 0, "Expected 7 acl operations allowed"); TEST_SAY("initial count is: %d\n", initial_acl_cnt); rd_kafka_event_destroy(rkev); acl_bindings[0] = rd_kafka_AclBinding_new( - RD_KAFKA_RESOURCE_BROKER, "kafka-cluster", RD_KAFKA_RESOURCE_PATTERN_LITERAL, - user_test1, "*", RD_KAFKA_ACL_OPERATION_ALTER, - RD_KAFKA_ACL_PERMISSION_TYPE_ALLOW, NULL, 0); + RD_KAFKA_RESOURCE_BROKER, "kafka-cluster", + RD_KAFKA_RESOURCE_PATTERN_LITERAL, user_test1, "*", + RD_KAFKA_ACL_OPERATION_ALTER, RD_KAFKA_ACL_PERMISSION_TYPE_ALLOW, + NULL, 0); admin_options = rd_kafka_AdminOptions_new(rk, RD_KAFKA_ADMIN_OP_CREATEACLS); err = rd_kafka_AdminOptions_set_request_timeout(admin_options, 10000, @@ -3267,7 +3284,7 @@ static void do_test_DescribeCluster(const char *what, rd_kafka_t *rk, rd_kafka_q rd_kafka_CreateAcls(rk, acl_bindings, 1, admin_options, q); rkev_acl_create = test_wait_admin_result( q, RD_KAFKA_EVENT_CREATEACLS_RESULT, 10000 + 1000); - acl_res = rd_kafka_event_CreateAcls_result(rkev_acl_create); + acl_res = rd_kafka_event_CreateAcls_result(rkev_acl_create); acl_res_acls = rd_kafka_CreateAcls_result_acls(acl_res, &resacl_cnt); for (i = 0; i < resacl_cnt; i++) { const rd_kafka_acl_result_t *acl_res_acl = *(acl_res_acls + i); @@ -3284,23 +3301,25 @@ static void do_test_DescribeCluster(const char *what, rd_kafka_t *rk, rd_kafka_q /* * Timeout options */ - options = rd_kafka_AdminOptions_new(rk, RD_KAFKA_ADMIN_OP_DESCRIBECLUSTER); + options = + rd_kafka_AdminOptions_new(rk, RD_KAFKA_ADMIN_OP_DESCRIBECLUSTER); - err = rd_kafka_AdminOptions_set_request_timeout(options, request_timeout, errstr, - sizeof(errstr)); + err = rd_kafka_AdminOptions_set_request_timeout( + options, request_timeout, errstr, sizeof(errstr)); TEST_ASSERT(!err, "%s", errstr); - if ((error = rd_kafka_AdminOptions_set_include_cluster_authorized_operations( - options, 1))) { - fprintf(stderr, - "%% Failed to set require authorized operations: %s\n", - rd_kafka_error_string(error)); - rd_kafka_error_destroy(error); - TEST_FAIL("Failed to set include authorized operations\n"); + if ((error = + rd_kafka_AdminOptions_set_include_cluster_authorized_operations( + options, 1))) { + fprintf(stderr, + "%% Failed to set require authorized operations: %s\n", + rd_kafka_error_string(error)); + rd_kafka_error_destroy(error); + TEST_FAIL("Failed to set include authorized operations\n"); } TIMING_START(&timing, "DescribeCluster"); TEST_SAY("Call DescribeCluster\n"); - rd_kafka_DescribeCluster(rk, options, q); + rd_kafka_DescribeCluster(rk, options, q); TIMING_ASSERT_LATER(&timing, 0, 50); TIMING_START(&timing, "DescribeCluster.queue_poll"); @@ -3338,35 +3357,36 @@ static void do_test_DescribeCluster(const char *what, rd_kafka_t *rk, rd_kafka_q TEST_ASSERT(!err, "Expected success, not %s: %s", rd_kafka_err2name(err), errstr2); - result_cluster = rd_kafka_DescribeCluster_result_description( - res); - + result_cluster = rd_kafka_DescribeCluster_result_description(res); + final_acl_cnt = rd_kafka_ClusterDescription_cluster_acl_operations_cnt( - result_cluster); + result_cluster); /* - * Initally count should be 7. After createAcls call with + * Initally count should be 7. After createAcls call with * only RD_KAFKA_ACL_OPERATION_ALTER allowed, it should reduce * to 2 (DESCRIBE is implicitly derived, and ALTER ). 7>2 should hold. */ TEST_SAY("final count is: %d\n", final_acl_cnt); TEST_ASSERT(initial_acl_cnt > final_acl_cnt, - "Expected the acl operations allowed to have reduced after" - " call to CreateAcls"); - + "Expected the acl operations allowed to have reduced after" + " call to CreateAcls"); + rd_kafka_event_destroy(rkev); /* - * Remove the previously created Acl so that it doesn't affect other tests - */ + * Remove the previously created Acl so that it doesn't affect other + * tests + */ admin_options_delete = rd_kafka_AdminOptions_new(rk, RD_KAFKA_ADMIN_OP_DELETEACLS); rd_kafka_AdminOptions_set_request_timeout(admin_options_delete, 10000, errstr, sizeof(errstr)); acl_bindings_delete = rd_kafka_AclBindingFilter_new( - RD_KAFKA_RESOURCE_BROKER, "kafka-cluster", RD_KAFKA_RESOURCE_PATTERN_MATCH, - user_test1 ,"*", RD_KAFKA_ACL_OPERATION_ALTER, RD_KAFKA_ACL_PERMISSION_TYPE_ALLOW, + RD_KAFKA_RESOURCE_BROKER, "kafka-cluster", + RD_KAFKA_RESOURCE_PATTERN_MATCH, user_test1, "*", + RD_KAFKA_ACL_OPERATION_ALTER, RD_KAFKA_ACL_PERMISSION_TYPE_ALLOW, NULL, 0); TIMING_START(&timing, "DeleteAcls"); @@ -3410,11 +3430,12 @@ static void do_test_DescribeCluster(const char *what, rd_kafka_t *rk, rd_kafka_q * @brief Test describe groups, creating consumers for a set of groups, * describing and deleting them at the end. */ -static void do_test_DescribeConsumerGroups_with_authorized_ops(const char *what, - rd_kafka_t *rk, - rd_kafka_queue_t *useq, - int request_timeout, - rd_bool_t include_authorized_operations) { +static void do_test_DescribeConsumerGroups_with_authorized_ops( + const char *what, + rd_kafka_t *rk, + rd_kafka_queue_t *useq, + int request_timeout, + rd_bool_t include_authorized_operations) { rd_kafka_queue_t *q; rd_kafka_AdminOptions_t *options = NULL; rd_kafka_event_t *rkev = NULL; @@ -3429,7 +3450,7 @@ static void do_test_DescribeConsumerGroups_with_authorized_ops(const char *what, const char *user_test1 = "User:broker"; #define TEST_DESCRIBE_CONSUMER_GROUPS_CNT 4 int known_groups = TEST_DESCRIBE_CONSUMER_GROUPS_CNT - 1; - int i,j; + int i, j; const int partitions_cnt = 1; const int msgs_cnt = 100; char *topic; @@ -3465,16 +3486,19 @@ static void do_test_DescribeConsumerGroups_with_authorized_ops(const char *what, err = rd_kafka_AdminOptions_set_request_timeout( options, request_timeout, errstr, sizeof(errstr)); TEST_ASSERT(!err, "%s", rd_kafka_err2str(err)); - if ((error = rd_kafka_AdminOptions_set_include_authorized_operations( - options, 1))) { - fprintf(stderr, - "%% Failed to set require authorized operations: %s\n", - rd_kafka_error_string(error)); - rd_kafka_error_destroy(error); - TEST_FAIL("Failed to set include authorized operations\n"); + if ((error = + rd_kafka_AdminOptions_set_include_authorized_operations( + options, 1))) { + fprintf(stderr, + "%% Failed to set require authorized " + "operations: %s\n", + rd_kafka_error_string(error)); + rd_kafka_error_destroy(error); + TEST_FAIL( + "Failed to set include authorized operations\n"); } } - + topic = rd_strdup(test_mk_topic_name(__FUNCTION__, 1)); exp_mdtopic.topic = topic; @@ -3491,8 +3515,9 @@ static void do_test_DescribeConsumerGroups_with_authorized_ops(const char *what, rd_kafka_conf_t *conf; char *group_id = rd_strdup(test_mk_topic_name(__FUNCTION__, 1)); acl_bindings[i] = rd_kafka_AclBinding_new( - RD_KAFKA_RESOURCE_GROUP, group_id, RD_KAFKA_RESOURCE_PATTERN_LITERAL, - user_test1, "*", RD_KAFKA_ACL_OPERATION_READ, + RD_KAFKA_RESOURCE_GROUP, group_id, + RD_KAFKA_RESOURCE_PATTERN_LITERAL, user_test1, "*", + RD_KAFKA_ACL_OPERATION_READ, RD_KAFKA_ACL_PERMISSION_TYPE_ALLOW, NULL, 0); if (i < known_groups) { snprintf(group_instance_ids[i], @@ -3531,10 +3556,11 @@ static void do_test_DescribeConsumerGroups_with_authorized_ops(const char *what, err = rd_kafka_AdminOptions_set_request_timeout(admin_options, 10000, errstr, sizeof(errstr)); TEST_ASSERT(!err, "%s", errstr); - rd_kafka_CreateAcls(rk, acl_bindings, TEST_DESCRIBE_CONSUMER_GROUPS_CNT, admin_options, q); + rd_kafka_CreateAcls(rk, acl_bindings, TEST_DESCRIBE_CONSUMER_GROUPS_CNT, + admin_options, q); rkev_acl_create = test_wait_admin_result( q, RD_KAFKA_EVENT_CREATEACLS_RESULT, 10000 + 1000); - acl_res = rd_kafka_event_CreateAcls_result(rkev_acl_create); + acl_res = rd_kafka_event_CreateAcls_result(rkev_acl_create); acl_res_acls = rd_kafka_CreateAcls_result_acls(acl_res, &resacl_cnt); TIMING_START(&timing, "DescribeConsumerGroups"); @@ -3598,16 +3624,19 @@ static void do_test_DescribeConsumerGroups_with_authorized_ops(const char *what, rd_kafka_ConsumerGroupDescription_error(act)); rd_kafka_consumer_group_state_t state = rd_kafka_ConsumerGroupDescription_state(act); - if(include_authorized_operations){ - TEST_ASSERT( - rd_kafka_ConsumerGroupDescription_authorized_operations_count(act) != 0, - "Authorized operations not returned when requested\n"); - TEST_ASSERT( - rd_kafka_ConsumerGroupDescription_authorized_operations_count(act) < 3, - "Expected only READ and DESCRIBE operations after createAcl(), got DELETE" - "as well\n"); + if (include_authorized_operations) { + TEST_ASSERT( + rd_kafka_ConsumerGroupDescription_authorized_operations_count( + act) != 0, + "Authorized operations not returned when " + "requested\n"); + TEST_ASSERT( + rd_kafka_ConsumerGroupDescription_authorized_operations_count( + act) < 3, + "Expected only READ and DESCRIBE operations after " + "createAcl(), got DELETE" + "as well\n"); } - } rd_kafka_event_destroy(rkev); @@ -3619,25 +3648,28 @@ static void do_test_DescribeConsumerGroups_with_authorized_ops(const char *what, /* Wait session timeout + 1s. Because using static group membership */ rd_sleep(6); - for(i=0;i Date: Wed, 14 Jun 2023 18:11:47 +0530 Subject: [PATCH 12/13] bug fixes and include auth ops false default --- examples/describe_consumer_groups.c | 4 ++++ src/rdkafka_admin.c | 8 +------- src/rdkafka_cgrp.c | 4 ++-- src/rdkafka_metadata.c | 4 ++-- src/rdkafka_metadata.h | 4 ++-- src/rdkafka_op.h | 2 -- src/rdkafka_request.c | 2 -- 7 files changed, 11 insertions(+), 17 deletions(-) diff --git a/examples/describe_consumer_groups.c b/examples/describe_consumer_groups.c index b392f3fa42..707937a7da 100644 --- a/examples/describe_consumer_groups.c +++ b/examples/describe_consumer_groups.c @@ -375,6 +375,10 @@ int main(int argc, char **argv) { * Create Kafka client configuration place-holder */ conf = rd_kafka_conf_new(); + conf_set(conf, "sasl.username", "broker"); + conf_set(conf, "sasl.password", "broker"); + conf_set(conf, "sasl.mechanism", "SCRAM-SHA-256"); + conf_set(conf, "security.protocol", "SASL_PLAINTEXT"); /* * Parse common options diff --git a/src/rdkafka_admin.c b/src/rdkafka_admin.c index 305f5c016b..ed5ea1d30b 100644 --- a/src/rdkafka_admin.c +++ b/src/rdkafka_admin.c @@ -7092,7 +7092,6 @@ rd_kafka_admin_DescribeTopicsRequest(rd_kafka_broker_t *rkb, rd_kafka_replyq_t replyq, rd_kafka_resp_cb_t *resp_cb, void *opaque) { - int i; rd_kafka_resp_err_t err; int include_topic_authorized_operations; @@ -7128,8 +7127,6 @@ rd_kafka_DescribeTopicsResponse_parse(rd_kafka_op_t *rko_req, 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; - rd_kafka_topic_authorized_operations_pair_t - *topic_authorized_operations = NULL; int i, cnt; rd_kafka_op_t *rko_result = NULL; // rd_kafka_assert(NULL, err == RD_KAFKA_RESP_ERR__DESTROY || @@ -7148,7 +7145,7 @@ rd_kafka_DescribeTopicsResponse_parse(rd_kafka_op_t *rko_req, while (cnt--) { rd_kafka_TopicDescription_t *topicdesc = NULL; /* topics in md should be in the same order as in - * topic_authorized_operations*/ + * mdi->topics[i]*/ rd_assert(strcmp(md->topics[i].topic, mdi->topics[i].topic_name) == 0); @@ -7496,12 +7493,9 @@ rd_kafka_DescribeClusterResponse_parse(rd_kafka_op_t *rko_req, 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_topic_authorized_operations_pair_t - *topic_authorized_operations = NULL; int32_t cluster_authorized_operations; char *cluster_id = NULL; int controller_id; - int i; rd_kafka_op_t *rko_result = NULL; // rd_kafka_assert(NULL, err == RD_KAFKA_RESP_ERR__DESTROY || // thrd_is_current(rk->rk_thread)); diff --git a/src/rdkafka_cgrp.c b/src/rdkafka_cgrp.c index 61cbd1f16f..0a38f76980 100644 --- a/src/rdkafka_cgrp.c +++ b/src/rdkafka_cgrp.c @@ -2244,8 +2244,8 @@ static int rd_kafka_cgrp_metadata_refresh(rd_kafka_cgrp_t *rkcg, err = rd_kafka_metadata_request( rkcg->rkcg_rk, NULL, &topics, rd_false /*!allow auto create */, - rd_true /*include cluster authorized operations */, - rd_true /*include topic authorized operations */, + rd_false /*!include cluster authorized operations */, + rd_false /*!include topic authorized operations */, rd_true /*cgrp_update*/, reason, rko); if (err) { rd_kafka_dbg(rk, CGRP | RD_KAFKA_DBG_METADATA, "CGRPMETADATA", diff --git a/src/rdkafka_metadata.c b/src/rdkafka_metadata.c index 4550451997..19f4a2cad3 100644 --- a/src/rdkafka_metadata.c +++ b/src/rdkafka_metadata.c @@ -1410,8 +1410,8 @@ rd_kafka_resp_err_t rd_kafka_metadata_refresh_brokers(rd_kafka_t *rk, return rd_kafka_metadata_request( rk, rkb, NULL /*brokers only*/, rd_false /*!allow auto create topics*/, - rd_true /*include cluster authorized operations */, - rd_true /*include topic authorized operations */, + rd_false /*!include cluster authorized operations */, + rd_false /*!include topic authorized operations */, rd_false /*no cgrp update */, reason, NULL); } diff --git a/src/rdkafka_metadata.h b/src/rdkafka_metadata.h index 464fab5b8d..f7f97a7195 100644 --- a/src/rdkafka_metadata.h +++ b/src/rdkafka_metadata.h @@ -55,7 +55,7 @@ typedef struct rd_kafka_metadata_topic_internal_s { rd_kafka_metadata_partition_internal_t *partitions; /** Topic Name. */ char* topic_name; - int32_t *topic_authorized_operations; /**< ACL operations allowed + int32_t topic_authorized_operations; /**< ACL operations allowed for topic */ } rd_kafka_metadata_topic_internal_t; @@ -86,7 +86,7 @@ typedef struct rd_kafka_metadata_internal_s { rd_kafka_metadata_topic_internal_t *topics; char *cluster_id; /**< current cluster id in \p cluster*/ int controller_id; /**< current controller id in \p cluster*/ - rd_list_t *cluster_authorized_operations; /**< ACL operations allowed + int32_t cluster_authorized_operations; /**< ACL operations allowed for cluster */ } rd_kafka_metadata_internal_t; diff --git a/src/rdkafka_op.h b/src/rdkafka_op.h index 73dee13ca4..6531cc883e 100644 --- a/src/rdkafka_op.h +++ b/src/rdkafka_op.h @@ -38,8 +38,6 @@ typedef struct rd_kafka_q_s rd_kafka_q_t; typedef struct rd_kafka_toppar_s rd_kafka_toppar_t; typedef struct rd_kafka_op_s rd_kafka_op_t; -typedef struct rd_kafka_topic_authorized_operations_pair - rd_kafka_topic_authorized_operations_pair_t; /* One-off reply queue + reply version. * All APIs that take a rd_kafka_replyq_t makes a copy of the diff --git a/src/rdkafka_request.c b/src/rdkafka_request.c index d7fa17dca5..6aeb86c3a0 100644 --- a/src/rdkafka_request.c +++ b/src/rdkafka_request.c @@ -2090,8 +2090,6 @@ static void rd_kafka_handle_Metadata(rd_kafka_t *rk, void *opaque) { rd_kafka_op_t *rko = opaque; /* Possibly NULL */ rd_kafka_metadata_internal_t *mdi = NULL; - // rd_kafka_topic_authorized_operations_pair_t - // *topic_authorized_operations = NULL; const rd_list_t *topics = request->rkbuf_u.Metadata.topics; int actions; From 7c157463da420830230f874bf2c7d676121902ea Mon Sep 17 00:00:00 2001 From: jainruchir Date: Wed, 14 Jun 2023 18:49:32 +0530 Subject: [PATCH 13/13] bug fix: request NULL, force_racks in MetadataReq --- examples/describe_consumer_groups.c | 4 ---- src/rdkafka_metadata.c | 2 +- 2 files changed, 1 insertion(+), 5 deletions(-) diff --git a/examples/describe_consumer_groups.c b/examples/describe_consumer_groups.c index 707937a7da..b392f3fa42 100644 --- a/examples/describe_consumer_groups.c +++ b/examples/describe_consumer_groups.c @@ -375,10 +375,6 @@ int main(int argc, char **argv) { * Create Kafka client configuration place-holder */ conf = rd_kafka_conf_new(); - conf_set(conf, "sasl.username", "broker"); - conf_set(conf, "sasl.password", "broker"); - conf_set(conf, "sasl.mechanism", "SCRAM-SHA-256"); - conf_set(conf, "security.protocol", "SASL_PLAINTEXT"); /* * Parse common options diff --git a/src/rdkafka_metadata.c b/src/rdkafka_metadata.c index 19f4a2cad3..84ed53ac20 100644 --- a/src/rdkafka_metadata.c +++ b/src/rdkafka_metadata.c @@ -493,7 +493,7 @@ rd_kafka_parse_Metadata(rd_kafka_broker_t *rkb, /* If force_racks is true, the outptr mdip has to contain the partition * to rack map. */ rd_bool_t force_rack_computation = - request->rkbuf_u.Metadata.force_racks; + request ? request->rkbuf_u.Metadata.force_racks : rd_false; rd_bool_t compute_racks = has_client_rack || force_rack_computation; /* Ignore metadata updates when terminating */