Skip to content

Commit

Permalink
bug fixes and include auth ops false default
Browse files Browse the repository at this point in the history
  • Loading branch information
jainruchir committed Jun 14, 2023
1 parent 060d694 commit 88707a0
Show file tree
Hide file tree
Showing 7 changed files with 11 additions and 17 deletions.
4 changes: 4 additions & 0 deletions examples/describe_consumer_groups.c
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
8 changes: 1 addition & 7 deletions src/rdkafka_admin.c
Original file line number Diff line number Diff line change
Expand Up @@ -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;

Expand Down Expand Up @@ -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 ||
Expand All @@ -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);
Expand Down Expand Up @@ -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));
Expand Down
4 changes: 2 additions & 2 deletions src/rdkafka_cgrp.c
Original file line number Diff line number Diff line change
Expand Up @@ -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",
Expand Down
4 changes: 2 additions & 2 deletions src/rdkafka_metadata.c
Original file line number Diff line number Diff line change
Expand Up @@ -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);
}

Expand Down
4 changes: 2 additions & 2 deletions src/rdkafka_metadata.h
Original file line number Diff line number Diff line change
Expand Up @@ -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;

Expand Down Expand Up @@ -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;

Expand Down
2 changes: 0 additions & 2 deletions src/rdkafka_op.h
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
2 changes: 0 additions & 2 deletions src/rdkafka_request.c
Original file line number Diff line number Diff line change
Expand Up @@ -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;

Expand Down

0 comments on commit 88707a0

Please sign in to comment.